From 4f27a47b81314b2eb52a5bc5b1d938bb586ae2aa Mon Sep 17 00:00:00 2001 From: EdShangGG Date: Mon, 1 Aug 2016 20:25:54 +0300 Subject: [PATCH 001/487] IGNITE-3513 Cleanup worker is placed in the Thread's waiting queue using Thread.sleep method --- modules/benchmarks/pom.xml | 4 +- .../jmh/notify/JmhParkVsNotifyBenchmark.java | 105 +++++++ .../jmh/notify/JmhWaitStategyBenchmark.java | 259 ++++++++++++++++++ .../processors/cache/GridCacheTtlManager.java | 65 ++++- .../GridCacheTtlManagerNotificationTest.java | 202 ++++++++++++++ .../IgniteCacheExpiryPolicyTestSuite.java | 3 + 6 files changed, 626 insertions(+), 12 deletions(-) create mode 100644 modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/notify/JmhParkVsNotifyBenchmark.java create mode 100644 modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/notify/JmhWaitStategyBenchmark.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerNotificationTest.java diff --git a/modules/benchmarks/pom.xml b/modules/benchmarks/pom.xml index a7d823d32112f..00315a83b7ac2 100644 --- a/modules/benchmarks/pom.xml +++ b/modules/benchmarks/pom.xml @@ -36,8 +36,8 @@ UTF-8 - 1.11.3 - 1.6 + 1.13 + 1.7 benchmarks diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/notify/JmhParkVsNotifyBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/notify/JmhParkVsNotifyBenchmark.java new file mode 100644 index 0000000000000..b85f6d876e1b8 --- /dev/null +++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/notify/JmhParkVsNotifyBenchmark.java @@ -0,0 +1,105 @@ +/* + * 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.benchmarks.jmh.notify; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.LockSupport; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Group; +import org.openjdk.jmh.annotations.GroupThreads; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; + +/** */ +@OutputTimeUnit(TimeUnit.MICROSECONDS) +@BenchmarkMode({/*Mode.AverageTime,*/ Mode.Throughput}) +@Warmup(iterations = 5, time = 5, timeUnit = TimeUnit.SECONDS) +@Measurement(iterations = 5, time = 10, timeUnit = TimeUnit.SECONDS) +@Fork(1) +@State(Scope.Group) +public class JmhParkVsNotifyBenchmark { + /** level of concurrency */ + private static final int THREAD_COUNT = 16; + + /** Thread. */ + private volatile Thread thread; + + /** + * + */ + @Setup(Level.Iteration) + public void setup() { + thread = null; + } + + /** + * + */ + @Benchmark + @Group("park") + public void park() { + if (thread == null) + thread = Thread.currentThread(); + + LockSupport.park(thread); + } + + /** + * + */ + @Benchmark + @GroupThreads(THREAD_COUNT) + @Group("park") + public void unpark() { + LockSupport.unpark(thread); + } + + /** Mutex. */ + private final Object mux = new Object(); + + /** + * + */ + @Benchmark + @Group("condition") + @GroupThreads(THREAD_COUNT) + public void notifyAll0() { + synchronized (mux) { + mux.notify(); + } + } + + /** + * + */ + @Benchmark + @Group("condition") + public void wait0() throws InterruptedException { + synchronized (mux) { + mux.wait(); + } + } +} diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/notify/JmhWaitStategyBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/notify/JmhWaitStategyBenchmark.java new file mode 100644 index 0000000000000..4a7ee2306903b --- /dev/null +++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/notify/JmhWaitStategyBenchmark.java @@ -0,0 +1,259 @@ +/* + * 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.benchmarks.jmh.notify; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import javax.cache.configuration.Factory; +import javax.cache.expiry.Duration; +import javax.cache.expiry.ExpiryPolicy; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.benchmarks.jmh.cache.JmhCacheAbstractBenchmark; +import org.apache.ignite.internal.benchmarks.model.IntValue; +import org.jsr166.ThreadLocalRandom8; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.BenchmarkParams; +import org.openjdk.jmh.results.RunResult; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.ChainedOptionsBuilder; +import org.openjdk.jmh.runner.options.OptionsBuilder; +import org.openjdk.jmh.runner.options.TimeValue; + +/** + * + */ +@OutputTimeUnit(TimeUnit.MICROSECONDS) +@BenchmarkMode({/*Mode.AverageTime,*/ Mode.Throughput}) +@Warmup(iterations = 5, time = 5, timeUnit = TimeUnit.SECONDS) +@Measurement(iterations = 5, time = 10, timeUnit = TimeUnit.SECONDS) +@Fork(1) +@State(Scope.Benchmark) +public class JmhWaitStategyBenchmark extends JmhCacheAbstractBenchmark { + /** */ + private static class RandomExpiryPolicy implements ExpiryPolicy { + /** rate duration will decrease with */ + private final double rate; + + /** current duration. */ + private final AtomicLong duration = new AtomicLong(1_000_000_000); + + /** */ + RandomExpiryPolicy(double rate) { + this.rate = rate; + } + + /** {@inheritDoc} */ + @Override public Duration getExpiryForCreation() { + boolean generateEvt = ThreadLocalRandom8.current().nextDouble() < rate; + return generateEvt ? new Duration(TimeUnit.MILLISECONDS, duration.decrementAndGet()) : + new Duration(TimeUnit.MILLISECONDS, duration.get()); + } + + /** {@inheritDoc} */ + @Override public Duration getExpiryForAccess() { + boolean generateEvt = ThreadLocalRandom8.current().nextDouble() < rate; + return generateEvt ? new Duration(TimeUnit.MILLISECONDS, duration.decrementAndGet()) : + new Duration(TimeUnit.MILLISECONDS, duration.get()); + } + + /** {@inheritDoc} */ + @Override public Duration getExpiryForUpdate() { + boolean generateEvt = ThreadLocalRandom8.current().nextDouble() < rate; + return generateEvt ? new Duration(TimeUnit.MILLISECONDS, duration.decrementAndGet()) : + new Duration(TimeUnit.MILLISECONDS, duration.get()); + } + } + + /** @param rate duration will decrease with */ + private static Factory getExpiryPolicyFactoryWithDecreasingRate(final double rate) { + return new Factory() { + @Override public ExpiryPolicy create() { + return new RandomExpiryPolicy(rate); + } + }; + } + + /** Decreasing expiry policy. */ + private static final ExpiryPolicy DECREASING_EXPIRY_POLICY = new ExpiryPolicy() { + AtomicLong duration = new AtomicLong(1_000_000_000); + + @Override public Duration getExpiryForCreation() { + return new Duration(TimeUnit.MILLISECONDS, duration.decrementAndGet()); + } + + @Override public Duration getExpiryForAccess() { + return new Duration(TimeUnit.MILLISECONDS, duration.decrementAndGet()); + } + + @Override public Duration getExpiryForUpdate() { + return new Duration(TimeUnit.MILLISECONDS, duration.decrementAndGet()); + } + }; + + /** Increasing expiry policy. */ + private static final ExpiryPolicy INCREASING_EXPIRY_POLICY = new ExpiryPolicy() { + AtomicLong duration = new AtomicLong(1_000_000); + + @Override public Duration getExpiryForCreation() { + return new Duration(TimeUnit.MILLISECONDS, duration.incrementAndGet()); + } + + @Override public Duration getExpiryForAccess() { + return new Duration(TimeUnit.MILLISECONDS, duration.incrementAndGet()); + } + + @Override public Duration getExpiryForUpdate() { + return new Duration(TimeUnit.MILLISECONDS, duration.incrementAndGet()); + } + }; + + /** Decreasing policy factory. */ + private final static Factory DECREASING_POLICY_FACTORY = new Factory() { + @Override public ExpiryPolicy create() { + return DECREASING_EXPIRY_POLICY; + } + }; + + /** Increasing policy factory. */ + private final static Factory INCREASING_POLICY_FACTORY = new Factory() { + @Override public ExpiryPolicy create() { + return INCREASING_EXPIRY_POLICY; + } + }; + + /** {@inheritDoc} */ + @Setup (Level.Iteration) + @Override public void setup() throws Exception { + Ignition.stopAll(true); + + super.setup(); + + CacheConfiguration cfg = new CacheConfiguration<>(); + cfg.setName("cache"); + cfg.setEagerTtl(true); + cfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + + String prop = System.getProperty("bench.exp.policy"); + + switch (prop) { + case "inc": + cfg.setExpiryPolicyFactory(INCREASING_POLICY_FACTORY); + break; + case "dec": + cfg.setExpiryPolicyFactory(DECREASING_POLICY_FACTORY); + break; + default: + assert prop.charAt(0) == 'r'; + double rate = Double.parseDouble(prop.trim().substring(1)) / 100; + cfg.setExpiryPolicyFactory(getExpiryPolicyFactoryWithDecreasingRate(rate)); + break; + } + + node.createCache(cfg); + + cache = node.getOrCreateCache("cache"); + + IgniteDataStreamer dataLdr = node.dataStreamer(cache.getName()); + + for (int i = 0; i < CNT; i++) + dataLdr.addData(i, new IntValue(i)); + + dataLdr.close(); + + System.out.println("Cache populated."); + } + + /** {@inheritDoc} */ + @TearDown + public void tearDown() throws Exception { + Ignition.stopAll(true); + } + + /** + * Test PUT operation. + * + * @throws Exception If failed. + */ + @Benchmark + public void put() throws Exception { + int key = ThreadLocalRandom.current().nextInt(CNT); + + cache.put(key, new IntValue(key)); + } + + /** + * Benchmark runner + */ + public static void main(String[] args) throws RunnerException { + List policies = Arrays.asList("inc", "dec", "r25", "r50", "r75"); + int[] threads = {2, 4, 8, 16, 32}; + + List results = new ArrayList<>(); + + for (String policy : policies) { + for (int thread : threads) { + ChainedOptionsBuilder builder = new OptionsBuilder() + .jvmArgs() + .timeUnit(TimeUnit.MILLISECONDS) + .measurementIterations(10) + .measurementTime(TimeValue.seconds(20)) + .warmupIterations(5) + .warmupTime(TimeValue.seconds(10)) + .jvmArgs("-Dbench.exp.policy=" + policy) + .forks(1) + .threads(thread) + .mode(Mode.Throughput) + .include(JmhWaitStategyBenchmark.class.getSimpleName()); + + results.addAll(new Runner(builder.build()).run()); + } + } + + for (RunResult result : results) { + BenchmarkParams params = result.getParams(); + Collection args1 = params.getJvmArgs(); + for (String s : args1) { + System.out.print(s.substring(s.length() -3, s.length())); + System.out.print(" x "); + } + System.out.print(params.getThreads()); + System.out.print("\t\t"); + System.out.println(result.getPrimaryResult().toString()); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java index 3e4561bce3305..ae2895e19253b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.configuration.CacheConfiguration; @@ -41,9 +42,19 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter { /** Entries pending removal. */ private final GridConcurrentSkipListSetEx pendingEntries = new GridConcurrentSkipListSetEx(); - /** Cleanup worker thread. */ + /** Cleanup worker. */ private CleanupWorker cleanupWorker; + /** Mutex. */ + private final Object mux = new Object(); + + /** Next expire time. */ + private volatile long nextExpireTime; + + /** Next expire time updater. */ + private static final AtomicLongFieldUpdater nextExpireTimeUpdater = + AtomicLongFieldUpdater.newUpdater(GridCacheTtlManager.class, "nextExpireTime"); + /** {@inheritDoc} */ @Override protected void start0() throws IgniteCheckedException { boolean cleanupDisabled = cctx.kernalContext().isDaemon() || @@ -80,7 +91,24 @@ public void addTrackedEntry(GridCacheMapEntry entry) { assert Thread.holdsLock(entry); assert cleanupWorker != null; - pendingEntries.add(new EntryWrapper(entry)); + EntryWrapper e = new EntryWrapper(entry); + + pendingEntries.add(e); + + while (true) { + long nextExpireTime = this.nextExpireTime; + + if (e.expireTime < nextExpireTime) { + if (nextExpireTimeUpdater.compareAndSet(this, nextExpireTime, e.expireTime)) { + synchronized (mux) { + mux.notifyAll(); + } + + break; + } + } else + break; + } } /** @@ -159,7 +187,7 @@ private class CleanupWorker extends GridWorker { /** * Creates cleanup worker. */ - protected CleanupWorker() { + CleanupWorker() { super(cctx.gridName(), "ttl-cleanup-worker-" + cctx.name(), cctx.logger(GridCacheTtlManager.class)); } @@ -168,16 +196,33 @@ protected CleanupWorker() { while (!isCancelled()) { expire(); - EntryWrapper first = pendingEntries.firstx(); + long waitTime; - if (first != null) { - long waitTime = first.expireTime - U.currentTimeMillis(); + while (true) { + long curTime = U.currentTimeMillis(); + + GridCacheTtlManager.EntryWrapper first = pendingEntries.firstx(); - if (waitTime > 0) - U.sleep(waitTime); + if (first == null) { + waitTime = 500; + nextExpireTime = curTime + 500; + } + else { + long expireTime = first.expireTime; + + waitTime = expireTime - curTime; + nextExpireTime = expireTime; + } + + synchronized (mux) { + if (pendingEntries.firstx() == first) { + if (waitTime > 0) + mux.wait(waitTime); + + break; + } + } } - else - U.sleep(500); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerNotificationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerNotificationTest.java new file mode 100644 index 0000000000000..85a491e3af55d --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerNotificationTest.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.expiry.CreatedExpiryPolicy; +import javax.cache.expiry.Duration; +import javax.cache.expiry.ExpiryPolicy; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.Event; +import org.apache.ignite.events.EventType; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.eclipse.jetty.util.BlockingArrayQueue; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; + +/** + * + */ +public class GridCacheTtlManagerNotificationTest extends GridCommonAbstractTest { + /** IP finder. */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** Test cache mode. */ + protected CacheMode cacheMode; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.setIpFinder(IP_FINDER); + + cfg.setDiscoverySpi(discoSpi); + + CacheConfiguration ccfg = new CacheConfiguration(); + + ccfg.setCacheMode(cacheMode); + ccfg.setEagerTtl(true); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testThatNotificationWorkAsExpected() throws Exception { + try (final Ignite g = startGrid(0)) { + final BlockingArrayQueue queue = new BlockingArrayQueue<>(); + + g.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + queue.add(evt); + + return true; + } + }, EventType.EVT_CACHE_OBJECT_EXPIRED); + + final String key = "key"; + + IgniteCache cache = g.cache(null); + + ExpiryPolicy plc1 = new CreatedExpiryPolicy(new Duration(MILLISECONDS, 100_000)); + + cache.withExpiryPolicy(plc1).put(key + 1, 1); + + Thread.sleep(1_000); // Cleaner should see entry. + + ExpiryPolicy plc2 = new CreatedExpiryPolicy(new Duration(MILLISECONDS, 1000)); + + cache.withExpiryPolicy(plc2).put(key + 2, 1); + + assertNotNull(queue.poll(5, SECONDS)); // We should receive event about second entry expiration. + } + } + + /** + * Add in several threads value to cache with different expiration policy. + * Wait for expiration of keys with small expiration duration. + */ + public void testThatNotificationWorkAsExpectedInMultithreadedMode() throws Exception { + final CyclicBarrier barrier = new CyclicBarrier(21); + final AtomicInteger keysRangeGen = new AtomicInteger(); + final AtomicInteger evtCnt = new AtomicInteger(); + final int cnt = 1_000; + + try (final Ignite g = startGrid(0)) { + final IgniteCache cache = g.cache(null); + + g.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + evtCnt.incrementAndGet(); + + return true; + } + }, EventType.EVT_CACHE_OBJECT_EXPIRED); + + + int smallDuration = 2000; + + int threadCnt = 10; + + GridTestUtils.runMultiThreadedAsync( + new CacheFiller(cache, 100_000, barrier, keysRangeGen, cnt), + threadCnt, ""); + + GridTestUtils.runMultiThreadedAsync( + new CacheFiller(cache, smallDuration, barrier, keysRangeGen, cnt), + threadCnt, ""); + + barrier.await(); + + Thread.sleep(1_000); // Cleaner should see at least one entry. + + barrier.await(); + + assertEquals(2 * threadCnt * cnt, cache.size()); + + Thread.sleep(2 * smallDuration); + + assertEquals(threadCnt * cnt, cache.size()); + assertEquals(threadCnt * cnt, evtCnt.get()); + } + } + + /** */ + private static class CacheFiller implements Runnable { + /** Barrier. */ + private final CyclicBarrier barrier; + /** Keys range generator. */ + private final AtomicInteger keysRangeGenerator; + /** Count. */ + private final int cnt; + /** Cache. */ + private final IgniteCache cache; + /** Expiration duration. */ + private final int expirationDuration; + + /** + * @param cache Cache. + * @param expirationDuration Expiration duration. + * @param barrier Barrier. + * @param keysRangeGenerator Keys. + * @param cnt Count. + */ + CacheFiller(IgniteCache cache, int expirationDuration, CyclicBarrier barrier, + AtomicInteger keysRangeGenerator, int cnt) { + this.expirationDuration = expirationDuration; + this.barrier = barrier; + this.keysRangeGenerator = keysRangeGenerator; + this.cnt = cnt; + this.cache = cache; + } + + /** {@inheritDoc} */ + @Override public void run() { + try { + barrier.await(); + + ExpiryPolicy plc1 = new CreatedExpiryPolicy(new Duration(MILLISECONDS, expirationDuration)); + int keyStart = keysRangeGenerator.getAndIncrement() * cnt; + + for (int i = keyStart; i < keyStart + cnt; i++) + cache.withExpiryPolicy(plc1).put("key" + i, 1); + + barrier.await(); + } + catch (Exception e) { + e.printStackTrace(); + } + } + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java index 21935e521052e..28cb2da94e4f8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java @@ -19,6 +19,7 @@ import junit.framework.TestSuite; import org.apache.ignite.cache.store.IgniteCacheExpiryStoreLoadSelfTest; +import org.apache.ignite.internal.processors.cache.GridCacheTtlManagerNotificationTest; import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerExpiredEventsTest; import org.apache.ignite.internal.processors.cache.IgniteCacheExpireAndUpdateConsistencyTest; @@ -71,6 +72,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCacheExpireAndUpdateConsistencyTest.class); + suite.addTestSuite(GridCacheTtlManagerNotificationTest.class); + return suite; } } \ No newline at end of file From 4f8d1822520afcffd1c97c3e643f16ff1bd85513 Mon Sep 17 00:00:00 2001 From: Ivan Veselovskiy Date: Tue, 2 Aug 2016 11:11:24 +0300 Subject: [PATCH 002/487] IGNITE-3343: IGFS: Secondary file system is not queried for statuses during MKDIRS and CREATE operations. This closes #896. --- .../org/apache/ignite/IgniteFileSystem.java | 3 +- .../secondary/IgfsSecondaryFileSystem.java | 4 +- .../processors/igfs/IgfsCreateResult.java | 2 +- .../internal/processors/igfs/IgfsImpl.java | 63 ++++++---------- .../processors/igfs/IgfsMetaManager.java | 74 +++++-------------- .../processors/igfs/IgfsModeResolver.java | 31 ++++++-- .../internal/processors/igfs/IgfsPaths.java | 6 +- .../IgfsSecondaryFileSystemCreateContext.java | 3 + .../internal/processors/igfs/IgfsUtils.java | 12 ++- .../processors/igfs/IgfsAbstractSelfTest.java | 34 ++++++++- .../igfs/IgfsDualAbstractSelfTest.java | 13 ++++ .../igfs/IgfsModeResolverSelfTest.java | 51 +++++++++++-- .../hadoop/fs/v1/IgniteHadoopFileSystem.java | 7 +- .../hadoop/fs/v2/IgniteHadoopFileSystem.java | 7 +- ...gniteHadoopFileSystemAbstractSelfTest.java | 1 - .../hadoop/HadoopAbstractMapReduceTest.java | 13 +++- 16 files changed, 196 insertions(+), 128 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java index bbc5bead0a126..f9aeb8da09a8b 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java @@ -362,8 +362,7 @@ public R execute(Class> taskCls, * will not be affected. Other properties will be added or overwritten. Passed properties with {@code null} values * will be removed from the stored properties or ignored if they don't exist in the file info. *

- * When working in {@code DUAL_SYNC} or {@code DUAL_ASYNC} modes only the following properties will be propagated - * to the secondary file system: + * When working in {@code DUAL_SYNC} or {@code DUAL_ASYNC} modes with Hadoop secondary file system only the following properties will be updated: *

    *
  • {@code usrName} - file owner name;
  • *
  • {@code grpName} - file owner group;
  • diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java index 3f124eb081381..4d9d255e26432 100644 --- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java +++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java @@ -43,8 +43,8 @@ public interface IgfsSecondaryFileSystem { * will not be affected. Other properties will be added or overwritten. Passed properties with {@code null} values * will be removed from the stored properties or ignored if they don't exist in the file info. *

    - * When working in {@code DUAL_SYNC} or {@code DUAL_ASYNC} modes only the following properties will be propagated - * to the secondary file system: + * When working in {@code DUAL_SYNC} or {@code DUAL_ASYNC} modes with Hadoop secondary file system only the + * following properties will be updated on the secondary file system: *

      *
    • {@code usrName} - file owner name;
    • *
    • {@code grpName} - file owner group;
    • diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java index 0b09e02deb4aa..eb1bc9d0ca9df 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java @@ -23,7 +23,7 @@ import java.io.OutputStream; /** - * IGFS file create result. + * IGFS file create or append result. */ public class IgfsCreateResult { /** File info in the primary file system. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 0d1ffc123ceff..e1f8e611b714e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.igfs; +import java.util.Set; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -248,7 +249,7 @@ public final class IgfsImpl implements IgfsEx { } } - modeRslvr = new IgfsModeResolver(dfltMode, IgfsUtils.preparePathModes(dfltMode, modes)); + modeRslvr = new IgfsModeResolver(dfltMode, modes); Object secondaryFsPayload = null; @@ -816,12 +817,14 @@ else if (val) } } - IgniteUuid fileId = meta.fileId(path); + if (!IgfsUtils.isDualMode(mode) || modeRslvr.hasPrimaryChild(path)) { + IgniteUuid fileId = meta.fileId(path); - if (fileId != null) - files.addAll(meta.directoryListing(fileId).keySet()); - else if (mode == PRIMARY) - throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path); + if (fileId != null) + files.addAll(meta.directoryListing(fileId).keySet()); + else if (mode == PRIMARY) + throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path); + } return F.viewReadOnly(files, new C1() { @Override public IgfsPath apply(String e) { @@ -846,7 +849,7 @@ else if (mode == PRIMARY) IgfsMode mode = resolveMode(path); - Collection files = new HashSet<>(); + Set files = new HashSet<>(); if (IgfsUtils.isDualMode(mode)) { assert secondaryFs != null; @@ -859,6 +862,9 @@ else if (mode == PRIMARY) files.add(impl); } + + if (!modeRslvr.hasPrimaryChild(path)) + return files; } catch (Exception e) { U.error(log, "List files in DUAL mode failed [path=" + path + ']', e); @@ -1345,29 +1351,6 @@ IgniteInternalFuture formatAsync() { return fut; } - /** - * Get file descriptor for specified path. - * - * @param path Path to file. - * @return Detailed file descriptor or {@code null}, if file does not exist. - * @throws IgniteCheckedException If failed. - */ - @Nullable private FileDescriptor getFileDescriptor(IgfsPath path) throws IgniteCheckedException { - assert path != null; - - List ids = meta.idsForPath(path); - - IgfsEntryInfo fileInfo = meta.info(ids.get(ids.size() - 1)); - - if (fileInfo == null) - return null; // File does not exist. - - // Resolve parent ID for removed file. - IgniteUuid parentId = ids.size() >= 2 ? ids.get(ids.size() - 2) : null; - - return new FileDescriptor(parentId, path.name(), fileInfo.id(), fileInfo.isFile()); - } - /** {@inheritDoc} */ @Override public R execute(IgfsTask task, @Nullable IgfsRecordResolver rslvr, Collection paths, @Nullable T arg) { @@ -1519,20 +1502,16 @@ private IgfsFileImpl resolveFileInfo(IgfsPath path, IgfsMode mode) throws Except case DUAL_SYNC: case DUAL_ASYNC: - info = meta.infoForPath(path); - - if (info == null) { - try { - IgfsFile status = secondaryFs.info(path); + try { + IgfsFile status = secondaryFs.info(path); - if (status != null) - return new IgfsFileImpl(status, data.groupBlockSize()); - } - catch (Exception e) { - U.error(log, "File info operation in DUAL mode failed [path=" + path + ']', e); + if (status != null) + return new IgfsFileImpl(status, data.groupBlockSize()); + } + catch (Exception e) { + U.error(log, "File info operation in DUAL mode failed [path=" + path + ']', e); - throw e; - } + throw e; } break; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java index d891b3887762f..0d14a3a271b6e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java @@ -3051,29 +3051,19 @@ IgfsCreateResult create( if (secondaryCtx != null) { secondaryOut = secondaryCtx.create(); - IgfsFile secondaryFile = secondaryCtx.info(); - - if (secondaryFile == null) - throw fsException("Failed to open output stream to the file created in " + - "the secondary file system because it no longer exists: " + path); - else if (secondaryFile.isDirectory()) - throw fsException("Failed to open output stream to the file created in " + - "the secondary file system because the path points to a directory: " + path); - - newAccessTime = secondaryFile.accessTime(); - newModificationTime = secondaryFile.modificationTime(); - newProps = secondaryFile.properties(); - newLen = secondaryFile.length(); - newBlockSize = secondaryFile.blockSize(); + newAccessTime = 0L; + newModificationTime = 0L; + newProps = null; } else { newAccessTime = System.currentTimeMillis(); newModificationTime = newAccessTime; newProps = fileProps; - newLen = 0L; - newBlockSize = blockSize; } + newLen = 0L; + newBlockSize = blockSize; + IgfsEntryInfo newInfo = invokeAndGet(overwriteId, new IgfsMetaFileCreateProcessor(newAccessTime, newModificationTime, newProps, newBlockSize, affKey, newLockId, evictExclude, newLen)); @@ -3254,18 +3244,9 @@ private IgfsPathsCreateResult createFileOrDirectory(boolean dir, IgfsPathIds pat Map props; if (secondaryCtx != null) { - IgfsFile secondaryInfo = secondaryCtx.fileSystem().info(lastCreatedPath); - - if (secondaryInfo == null) - throw new IgfsException("Failed to perform operation because secondary file system path was " + - "modified concurrently: " + lastCreatedPath); - else if (secondaryInfo.isFile()) - throw new IgfsException("Failed to perform operation because secondary file system entity is " + - "not directory: " + lastCreatedPath); - - accessTime = secondaryInfo.accessTime(); - modificationTime = secondaryInfo.modificationTime(); - props = secondaryInfo.properties(); + accessTime = 0L; + modificationTime = 0L; + props = null; } else { accessTime = curTime; @@ -3293,18 +3274,9 @@ else if (secondaryInfo.isFile()) Map props; if (secondaryCtx != null) { - IgfsFile secondaryInfo = secondaryCtx.fileSystem().info(pathIds.path()); - - if (secondaryInfo == null) - throw new IgfsException("Failed to perform operation because secondary file system path was " + - "modified concurrnetly: " + pathIds.path()); - else if (secondaryInfo.isFile()) - throw new IgfsException("Failed to perform operation because secondary file system entity is " + - "not directory: " + lastCreatedPath); - - accessTime = secondaryInfo.accessTime(); - modificationTime = secondaryInfo.modificationTime(); - props = secondaryInfo.properties(); + accessTime = 0L; + modificationTime = 0L; + props = null; } else { accessTime = curTime; @@ -3322,29 +3294,19 @@ else if (secondaryInfo.isFile()) int newBlockSize; if (secondaryCtx != null) { - IgfsFile secondaryFile = secondaryCtx.info(); - - if (secondaryFile == null) - throw fsException("Failed to open output stream to the file created in " + - "the secondary file system because it no longer exists: " + pathIds.path()); - else if (secondaryFile.isDirectory()) - throw fsException("Failed to open output stream to the file created in " + - "the secondary file system because the path points to a directory: " + pathIds.path()); - - newAccessTime = secondaryFile.accessTime(); - newModificationTime = secondaryFile.modificationTime(); - newProps = secondaryFile.properties(); - newLen = secondaryFile.length(); - newBlockSize = secondaryFile.blockSize(); + newAccessTime = 0L; + newModificationTime = 0L; + newProps = null; } else { newAccessTime = curTime; newModificationTime = curTime; newProps = fileProps; - newLen = 0L; - newBlockSize = blockSize; } + newLen = 0L; + newBlockSize = blockSize; + procMap.put(curId, new IgfsMetaFileCreateProcessor(newAccessTime, newModificationTime, newProps, newBlockSize, affKey, createFileLockId(false), evictExclude, newLen)); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java index 907051f1d7a80..33b835f9f276b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java @@ -17,9 +17,12 @@ package org.apache.ignite.internal.processors.igfs; -import java.util.Collections; +import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap; @@ -42,18 +45,24 @@ public class IgfsModeResolver { /** Cached modes per path. */ private Map modesCache; + /** Set to store parent dual paths that have primary children. */ + private final Set dualParentsWithPrimaryChildren; + /** * Constructor * * @param dfltMode Default IGFS mode. * @param modes List of configured modes. The order is significant as modes are added in order of occurrence. */ - public IgfsModeResolver(IgfsMode dfltMode, @Nullable List> modes) { + public IgfsModeResolver(IgfsMode dfltMode, @Nullable ArrayList> modes) + throws IgniteCheckedException { assert dfltMode != null; this.dfltMode = dfltMode; - this.modes = modes; + this.dualParentsWithPrimaryChildren = new HashSet<>(); + + this.modes = IgfsUtils.preparePathModes(dfltMode, modes, dualParentsWithPrimaryChildren); if (modes != null) modesCache = new GridBoundedConcurrentLinkedHashMap<>(MAX_PATH_CACHE); @@ -94,10 +103,20 @@ public IgfsMode resolveMode(IgfsPath path) { } /** - * @return Unmodifiable copy of properly ordered modes prefixes + * @return Copy of properly ordered modes prefixes * or {@code null} if no modes set. */ - @Nullable public List> modesOrdered() { - return modes != null ? Collections.unmodifiableList(modes) : null; + @Nullable public ArrayList> modesOrdered() { + return modes != null ? new ArrayList<>(modes) : null; + } + + /** + * Answers if the given path has an immediate child of PRIMARY mode. + * + * @param path The path to query. + * @return If the given path has an immediate child of PRIMARY mode. + */ + public boolean hasPrimaryChild(IgfsPath path) { + return dualParentsWithPrimaryChildren.contains(path); } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java index 4a79259d1f77b..2bdb23b1c64b0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java @@ -48,7 +48,7 @@ public class IgfsPaths implements Externalizable { private IgfsMode dfltMode; /** Path modes. */ - private List> pathModes; + private ArrayList> pathModes; /** * Empty constructor required by {@link Externalizable}. @@ -65,7 +65,7 @@ public IgfsPaths() { * @param pathModes Path modes. * @throws IgniteCheckedException If failed. */ - public IgfsPaths(Object payload, IgfsMode dfltMode, @Nullable List> pathModes) + public IgfsPaths(Object payload, IgfsMode dfltMode, @Nullable ArrayList> pathModes) throws IgniteCheckedException { this.dfltMode = dfltMode; this.pathModes = pathModes; @@ -91,7 +91,7 @@ public IgfsMode defaultMode() { /** * @return Path modes. */ - @Nullable public List> pathModes() { + @Nullable public ArrayList> pathModes() { return pathModes; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java index 1c0efd6e4f8eb..788efe391ad5e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java @@ -28,6 +28,7 @@ /** * Context for secondary file system create request. + * Note that it is never used for dual mode append operation. */ public class IgfsSecondaryFileSystemCreateContext { /** File system. */ @@ -68,6 +69,8 @@ public class IgfsSecondaryFileSystemCreateContext { */ public IgfsSecondaryFileSystemCreateContext(IgfsSecondaryFileSystem fs, IgfsPath path, boolean overwrite, boolean simpleCreate, @Nullable Map props, short replication, long blockSize, int bufSize) { + assert fs != null; + this.fs = fs; this.path = path; this.overwrite = overwrite; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java index 03b761146ecb3..b9788f881675f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.igfs; +import java.util.Set; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteSystemProperties; @@ -66,6 +67,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_RETRIES_COUNT; import static org.apache.ignite.igfs.IgfsMode.DUAL_ASYNC; import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC; +import static org.apache.ignite.igfs.IgfsMode.PRIMARY; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGFS; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; @@ -819,11 +821,13 @@ public static boolean canContain(IgfsMode parent, IgfsMode child) { * * @param dfltMode The root mode. Must always be not null. * @param modes The subdirectory modes. + * @param dualParentsContainingPrimaryChildren The set to store parents into. * @return Descending list of filtered and checked modes. - * @throws IgniteCheckedException On error or + * @throws IgniteCheckedException On error. */ public static ArrayList> preparePathModes(final IgfsMode dfltMode, - @Nullable List> modes) throws IgniteCheckedException { + @Nullable List> modes, Set dualParentsContainingPrimaryChildren) + throws IgniteCheckedException { if (modes == null) return null; @@ -857,6 +861,10 @@ public static ArrayList> preparePathModes(final IgfsMode // Add to the 1st position (deep first). resModes.add(0, mode); + // Store primary paths inside dual paths in separate collection: + if (mode.getValue() == PRIMARY) + dualParentsContainingPrimaryChildren.add(mode.getKey().parent()); + break; } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java index 384da954bc557..fd3f9b6b3774b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java @@ -244,6 +244,13 @@ protected boolean relaxedConsistency() { return false; } + /** + * @return Relaxed consistency flag. + */ + protected boolean initializeDefaultPathModes() { + return false; + } + /** * @return Client flag. */ @@ -369,6 +376,8 @@ protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mo igfsCfg.setSequentialReadsBeforePrefetch(SEQ_READS_BEFORE_PREFETCH); igfsCfg.setRelaxedConsistency(relaxedConsistency()); + igfsCfg.setInitializeDefaultPathModes(initializeDefaultPathModes()); + CacheConfiguration dataCacheCfg = defaultCacheConfiguration(); dataCacheCfg.setName("dataCache"); @@ -1071,17 +1080,36 @@ public void testFormat() throws Exception { * @throws Exception If failed. */ public void testRootPropertiesPersistAfterFormat() throws Exception { - igfs.update(new IgfsPath("/"), Collections.singletonMap("foo", "moo")); + if (dual && !(igfsSecondaryFileSystem instanceof IgfsSecondaryFileSystemImpl)) { + // In case of Hadoop dual mode only user name, group name, and permission properties are updated, + // an arbitrary named property is just ignored: + checkRootPropertyUpdate("foo", "moo", null); + checkRootPropertyUpdate(IgfsUtils.PROP_PERMISSION, "0777", "0777"); + } + else { + checkRootPropertyUpdate("foo", "moo", "moo"); + checkRootPropertyUpdate(IgfsUtils.PROP_PERMISSION, "0777", "0777"); + } + } + + /** + * + * @throws Exception + */ + private void checkRootPropertyUpdate(String prop, String setVal, String expGetVal) throws Exception { + final IgfsPath rootPath = new IgfsPath("/"); + + igfs.update(rootPath, Collections.singletonMap(prop, setVal)); igfs.format(); - IgfsFile file = igfs.info(new IgfsPath("/")); + IgfsFile file = igfs.info(rootPath); assert file != null; Map props = file.properties(); - assertEquals("moo", props.get("foo")); + assertEquals(expGetVal, props.get(prop)); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java index 124bec692af25..92e1178d35839 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java @@ -55,6 +55,12 @@ protected IgfsDualAbstractSelfTest(IgfsMode mode) { assert mode == DUAL_SYNC || mode == DUAL_ASYNC; } + /** {@inheritDoc} */ + @Override protected boolean initializeDefaultPathModes() { + // Enable default modes in order to test various modes. + return true; + } + /** * @throws Exception If failed. */ @@ -68,6 +74,13 @@ public void testDefaultDirectories() throws Exception { for (IgfsPath p : paths) assert igfs.exists(p); + assert igfs.modeResolver().resolveMode(gg) == mode; + assert igfs.modeResolver().resolveMode(new IgfsPath(gg, "sync")) == IgfsMode.DUAL_SYNC; + assert igfs.modeResolver().resolveMode(new IgfsPath(gg, "async")) == IgfsMode.DUAL_ASYNC; + assert igfs.modeResolver().resolveMode(new IgfsPath(gg, "primary")) == IgfsMode.PRIMARY; + assert !igfsSecondary.exists("/ignite/primary"); // PRIMARY mode path must exist in upper level fs only. + + // All the child paths of "/ignite/" must be visible in listings: assert igfs.listFiles(gg).size() == 3; assert igfs.listPaths(gg).size() == 3; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java index f3bb516041619..bd7e413e1850e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java @@ -17,8 +17,11 @@ package org.apache.ignite.internal.processors.igfs; +import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.List; +import java.util.Set; import junit.framework.TestCase; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.igfs.IgfsMode; @@ -38,8 +41,9 @@ public class IgfsModeResolverSelfTest extends TestCase { /** {@inheritDoc} */ @Override protected void setUp() throws Exception { - reslvr = new IgfsModeResolver(DUAL_SYNC, Arrays.asList(new T2<>(new IgfsPath("/a/b/c/d"), PROXY), new T2<> - (new IgfsPath("/a/P/"), PRIMARY), new T2<>(new IgfsPath("/a/b/"), DUAL_ASYNC))); + reslvr = new IgfsModeResolver(DUAL_SYNC, new ArrayList<>(Arrays.asList(new T2<>( + new IgfsPath("/a/b/c/d"), PROXY), new T2<>(new IgfsPath("/a/P/"), PRIMARY), new T2<>(new IgfsPath("/a/b/"), + DUAL_ASYNC)))); } /** @@ -90,7 +94,7 @@ public void testModesValidation() throws Exception { try { IgfsUtils.preparePathModes(DUAL_SYNC, Arrays.asList( new T2<>(new IgfsPath("/a/"), PRIMARY), - new T2<>(new IgfsPath("/a/b/"), DUAL_ASYNC))); + new T2<>(new IgfsPath("/a/b/"), DUAL_ASYNC)), new HashSet()); fail("IgniteCheckedException expected"); } @@ -102,7 +106,8 @@ public void testModesValidation() throws Exception { for (IgfsMode m: IgfsMode.values()) { if (m != IgfsMode.PRIMARY) { try { - IgfsUtils.preparePathModes(PRIMARY, Arrays.asList(new T2<>(new IgfsPath("/a/"), DUAL_ASYNC))); + IgfsUtils.preparePathModes(PRIMARY, Arrays.asList(new T2<>(new IgfsPath("/a/"), DUAL_ASYNC)), + new HashSet()); fail("IgniteCheckedException expected"); } @@ -117,7 +122,7 @@ public void testModesValidation() throws Exception { new T2<>(new IgfsPath("/a"), PRIMARY), new T2<>(new IgfsPath("/c/d/"), PRIMARY), new T2<>(new IgfsPath("/c/d/e/f"), PRIMARY) - )); + ), new HashSet()); assertNotNull(modes); assertEquals(2, modes.size()); assertEquals(modes, Arrays.asList( @@ -130,7 +135,7 @@ public void testModesValidation() throws Exception { new T2<>(new IgfsPath("/a/b"), DUAL_ASYNC), new T2<>(new IgfsPath("/a/b/c"), DUAL_SYNC), new T2<>(new IgfsPath("/a/b/c/d"), DUAL_ASYNC) - )); + ), new HashSet()); assertNotNull(modes); assertEquals(modes.size(), 3); assertEquals(modes, Arrays.asList( @@ -139,4 +144,38 @@ public void testModesValidation() throws Exception { new T2<>(new IgfsPath("/a/b"), DUAL_ASYNC) )); } + + /** + * @throws Exception If failed. + */ + public void testDualParentsWithPrimaryChild() throws Exception { + Set set = new HashSet<>(); + + IgfsUtils.preparePathModes(DUAL_SYNC, Arrays.asList( + new T2<>(new IgfsPath("/a/b"), DUAL_ASYNC), + new T2<>(new IgfsPath("/a/b/c"), PRIMARY), + new T2<>(new IgfsPath("/a/b/x/y"), PRIMARY), + new T2<>(new IgfsPath("/a/b/x/z"), PRIMARY), + new T2<>(new IgfsPath("/m"), PRIMARY) + ), set); + assertEquals(set, new HashSet() {{ + add(new IgfsPath("/a/b")); + add(new IgfsPath("/a/b/x")); + add(new IgfsPath("/")); + }}); + + set = new HashSet<>(); + + IgfsUtils.preparePathModes(DUAL_ASYNC, Arrays.asList( + new T2<>(new IgfsPath("/a/b/x/y/z"), PRIMARY), + new T2<>(new IgfsPath("/a/b/c"), PRIMARY), + new T2<>(new IgfsPath("/a/k"), PRIMARY), + new T2<>(new IgfsPath("/a/z"), PRIMARY) + ), set); + assertEquals(set, new HashSet() {{ + add(new IgfsPath("/a/b")); + add(new IgfsPath("/a")); + add(new IgfsPath("/a/b/x/y")); + }}); + } } \ No newline at end of file diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java index 83991aa7466a1..a06129e43fc63 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java @@ -309,7 +309,12 @@ public void colocateFileWrites(boolean colocateFileWrites) { else clientLog = IgfsLogger.disabledLogger(); - modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes()); + try { + modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes()); + } + catch (IgniteCheckedException ice) { + throw new IOException(ice); + } boolean initSecondary = paths.defaultMode() == PROXY; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java index cc02d0567c25e..bd8ed2db35c93 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java @@ -318,7 +318,12 @@ private void initialize(URI name, Configuration cfg) throws IOException { else clientLog = IgfsLogger.disabledLogger(); - modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes()); + try { + modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes()); + } + catch (IgniteCheckedException ice) { + throw new IOException(ice); + } boolean initSecondary = paths.defaultMode() == PROXY; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java index 083ee545ee088..f793ec31f0b5a 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java @@ -48,7 +48,6 @@ import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; -import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java index ef886e47ce838..37312133ea5ac 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java @@ -41,6 +41,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.fs.IgniteHadoopFileSystemCounterWriter; import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem; +import org.apache.ignite.igfs.IgfsFile; import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; import org.apache.ignite.igfs.IgfsMode; @@ -123,8 +124,16 @@ public class HadoopAbstractMapReduceTest extends HadoopAbstractWordCountTest { * @param p The path. * @return The owner. */ - private static String getOwner(IgfsEx i, IgfsPath p) { - return i.info(p).property(IgfsUtils.PROP_USER_NAME); + private static String getOwner(final IgfsEx i, final IgfsPath p) { + return IgfsUserContext.doAs(USER, new IgniteOutClosure() { + @Override public String apply() { + IgfsFile f = i.info(p); + + assert f != null; + + return f.property(IgfsUtils.PROP_USER_NAME); + } + }); } /** From ae23dab0af8caaccd3db1dca8e435b57b8844a31 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Wed, 3 Aug 2016 17:10:03 +0700 Subject: [PATCH 003/487] Visor: scan cache with filter. --- .../internal/visor/query/VisorQueryJob.java | 38 ++++++++--- .../query/VisorQueryScanSubstringFilter.java | 63 +++++++++++++++++++ .../internal/visor/query/VisorQueryUtils.java | 6 ++ 3 files changed, 99 insertions(+), 8 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryScanSubstringFilter.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java index 0f2f82ecefd57..153c564439920 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java @@ -37,9 +37,12 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.visor.VisorJob; import org.apache.ignite.internal.visor.util.VisorExceptionWrapper; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteBiTuple; import static org.apache.ignite.internal.visor.query.VisorQueryUtils.RMV_DELAY; +import static org.apache.ignite.internal.visor.query.VisorQueryUtils.SCAN_CACHE_WITH_FILTER; +import static org.apache.ignite.internal.visor.query.VisorQueryUtils.SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE; import static org.apache.ignite.internal.visor.query.VisorQueryUtils.SCAN_COL_NAMES; import static org.apache.ignite.internal.visor.query.VisorQueryUtils.SCAN_QRY_NAME; import static org.apache.ignite.internal.visor.query.VisorQueryUtils.SCAN_NEAR_CACHE; @@ -81,8 +84,9 @@ protected IgniteCache cache(String cacheName) { * @param arg Job argument with query parameters. * @return Query cursor. */ - private QueryCursor> scan(IgniteCache c, VisorQueryArg arg) { - ScanQuery qry = new ScanQuery<>(null); + private QueryCursor> scan(IgniteCache c, VisorQueryArg arg, + IgniteBiPredicate filter) { + ScanQuery qry = new ScanQuery<>(filter); qry.setPageSize(arg.pageSize()); qry.setLocal(arg.local()); @@ -100,23 +104,41 @@ private QueryCursor> near(IgniteCache run(VisorQueryArg arg) { + @Override protected IgniteBiTuple run(final VisorQueryArg arg) { try { UUID nid = ignite.localNode().id(); - boolean near = SCAN_NEAR_CACHE.equalsIgnoreCase(arg.queryTxt()); + String qryTxt = arg.queryTxt(); - boolean scan = arg.queryTxt() == null; + boolean scan = qryTxt == null; + + boolean scanWithFilter = qryTxt != null && qryTxt.startsWith(SCAN_CACHE_WITH_FILTER); + + boolean near = qryTxt != null && qryTxt.startsWith(SCAN_NEAR_CACHE); + + boolean scanAny = scan || scanWithFilter || near; // Generate query ID to store query cursor in node local storage. - String qryId = ((scan || near) ? SCAN_QRY_NAME : SQL_QRY_NAME) + "-" + UUID.randomUUID(); + String qryId = (scanAny ? SCAN_QRY_NAME : SQL_QRY_NAME) + "-" + UUID.randomUUID(); IgniteCache c = cache(arg.cacheName()); - if (near || scan) { + if (scanAny) { long start = U.currentTimeMillis(); - VisorQueryCursor> cur = new VisorQueryCursor<>(near ? near(c) : scan(c, arg)); + IgniteBiPredicate filter = null; + + if (scanWithFilter) { + boolean caseSensitive = qryTxt.startsWith(SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE); + + String ptrn = caseSensitive + ? qryTxt.substring(SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE.length()) + : qryTxt.substring(SCAN_CACHE_WITH_FILTER.length()); + + filter = new VisorQueryScanSubstringFilter(caseSensitive, ptrn); + } + + VisorQueryCursor> cur = new VisorQueryCursor<>(near ? near(c) : scan(c, arg, filter)); List rows = fetchScanQueryRows(cur, arg.pageSize()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryScanSubstringFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryScanSubstringFilter.java new file mode 100644 index 0000000000000..43eb6dda64e7f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryScanSubstringFilter.java @@ -0,0 +1,63 @@ +/* + * 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.visor.query; + +import org.apache.ignite.lang.IgniteBiPredicate; + +/** + * Filter scan results by specified substring in string presentation of key or value. + */ +public class VisorQueryScanSubstringFilter implements IgniteBiPredicate { + /** */ + private static final long serialVersionUID = 0L; + + /** Case sensitive flag. */ + private final boolean caseSensitive; + + /** String to search in string presentation of key or value. */ + private final String ptrn; + + /** + * Create filter instance. + * + * @param caseSensitive Case sensitive flag. + * @param ptrn String to search in string presentation of key or value. + */ + public VisorQueryScanSubstringFilter(boolean caseSensitive, String ptrn) { + this.caseSensitive = caseSensitive; + + this.ptrn = caseSensitive ? ptrn : ptrn.toUpperCase(); + } + + /** + * Check that key or value contains specified string. + * + * @param key Key object. + * @param val Value object. + * @return {@code true} when string presentation of key or value contain specified string. + */ + @Override public boolean apply(Object key, Object val) { + String k = key.toString(); + String v = val.toString(); + + if (caseSensitive) + return k.contains(ptrn) || v.contains(ptrn); + + return k.toUpperCase().contains(ptrn) || v.toUpperCase().contains(ptrn); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryUtils.java index 64d2b958878ba..0b8cf837a339c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryUtils.java @@ -43,6 +43,12 @@ public class VisorQueryUtils { /** Prefix for node local key for SCAN near queries. */ public static final String SCAN_NEAR_CACHE = "VISOR_SCAN_NEAR_CACHE"; + /** Prefix for node local key for SCAN near queries. */ + public static final String SCAN_CACHE_WITH_FILTER = "VISOR_SCAN_CACHE_WITH_FILTER"; + + /** Prefix for node local key for SCAN near queries. */ + public static final String SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE = "VISOR_SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE"; + /** Columns for SCAN queries. */ public static final Collection SCAN_COL_NAMES = Arrays.asList( new VisorQueryField(null, null, "Key Class", ""), new VisorQueryField(null, null, "Key", ""), From a90d05b30f39da001156fdd4053790f8cd352406 Mon Sep 17 00:00:00 2001 From: Valentin Kulichenko Date: Wed, 3 Aug 2016 22:00:15 -0700 Subject: [PATCH 004/487] IGNITE-3408 - Provided basic implementation of AddressResolver --- .../configuration/BasicAddressResolver.java | 146 +++++++++++ .../spi/GridTcpSpiForwardingSelfTest.java | 237 ++++++++++++++++-- 2 files changed, 364 insertions(+), 19 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/configuration/BasicAddressResolver.java diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/BasicAddressResolver.java b/modules/core/src/main/java/org/apache/ignite/configuration/BasicAddressResolver.java new file mode 100644 index 0000000000000..d29689f2016b0 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/configuration/BasicAddressResolver.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.configuration; + +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.UnknownHostException; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * Basic implementation of {@link AddressResolver}. + *

      + * Allows to provide simple mapping between different address, which is useful + * different parts of the cluster are located in different subnetworks and port + * forwarding on the router is used for communication between them. Another + * common case is Docker environment which can create a pair of public and private + * address per container. + *

      + * There are two different types of mapping supported by this implementation of + * {@link AddressResolver}. + *

      + * First type maps specific socket addresses (host and port pairs) are mapped to + * each other. This is useful for port forwarding, where multiple nodes will + * typically map to the same external address (router), but with different port + * numbers. Here is the example: + *

      + * <property name="addressResolver">
      + *     <bean class="org.apache.ignite.configuration.BasicAddressResolver">
      + *         <constructor-arg>
      + *             <map>
      + *                 <entry key="10.0.0.1:47100" value="123.123.123.123:1111"/>
      + *                 <entry key="10.0.0.2:47100" value="123.123.123.123:2222"/>
      + *                 <entry key="10.0.0.3:47100" value="123.123.123.123:3333"/>
      + *             </map>
      + *         </constructor-arg>
      + *     </bean>
      + * </property>
      + * 
      + *

      + * Second type maps one host to another. In this case any internal address a node + * is bound to will be mapped to the corresponding external host with the same + * host number. Here is the example: + *

      + * <property name="addressResolver">
      + *     <bean class="org.apache.ignite.configuration.BasicAddressResolver">
      + *         <constructor-arg>
      + *             <map>
      + *                 <entry key="10.0.0.1" value="123.123.123.123"/>
      + *             </map>
      + *         </constructor-arg>
      + *     </bean>
      + * </property>
      + * 
      + * Here any port on {@code 10.0.0.1} will be mapped to the same port number on {@code 123.123.123.123}. + * E.g., {@code 10.0.0.1:47100} will be mapped to {@code 123.123.123.123:47100}, and {@code 10.0.0.1:47500} + * will be mapped to {@code 123.123.123.123:47500}. + *

      + * Two types of mappings described above can be mixed within one address resolver. + */ +public class BasicAddressResolver implements AddressResolver { + /** Address map. */ + private final Map inetAddrMap; + + /** Socket address map. */ + private final Map inetSockAddrMap; + + /** + * Created the address resolver. + * + * @param addrMap Address mappings. + * @throws UnknownHostException If any of the hosts can't be resolved. + */ + public BasicAddressResolver(Map addrMap) throws UnknownHostException { + if (addrMap == null || addrMap.isEmpty()) + throw new IllegalArgumentException("At least one address mapping is required."); + + inetAddrMap = U.newHashMap(addrMap.size()); + inetSockAddrMap = U.newHashMap(addrMap.size()); + + for (Map.Entry e : addrMap.entrySet()) { + String from = e.getKey(); + String to = e.getValue(); + + if (F.isEmpty(from) || F.isEmpty(to)) + throw new IllegalArgumentException("Invalid address mapping: " + e); + + String[] fromArr = from.split(":"); + String[] toArr = to.split(":"); + + assert fromArr.length > 0; + assert toArr.length > 0; + + if (fromArr.length == 1) { + if (toArr.length != 1) + throw new IllegalArgumentException("Invalid address mapping: " + e); + + inetAddrMap.put(InetAddress.getByName(fromArr[0]), InetAddress.getByName(toArr[0])); + } + else if (fromArr.length == 2) { + if (toArr.length != 2) + throw new IllegalArgumentException("Invalid address mapping: " + e); + + inetSockAddrMap.put(new InetSocketAddress(fromArr[0], Integer.parseInt(fromArr[1])), + new InetSocketAddress(toArr[0], Integer.parseInt(toArr[1]))); + } + else + throw new IllegalArgumentException("Invalid address mapping: " + e); + } + } + + /** {@inheritDoc} */ + @Override public Collection getExternalAddresses(InetSocketAddress addr) + throws IgniteCheckedException { + InetSocketAddress inetSockAddr = inetSockAddrMap.get(addr); + + if (inetSockAddr != null) + return Collections.singletonList(inetSockAddr); + + InetAddress inetAddr = inetAddrMap.get(addr.getAddress()); + + if (inetAddr != null) + return Collections.singletonList(new InetSocketAddress(inetAddr, addr.getPort())); + + return Collections.emptyList(); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java index 0c3ae121db9c6..652e47f0841db 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java @@ -23,10 +23,12 @@ import java.util.Collection; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.Callable; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.AddressResolver; +import org.apache.ignite.configuration.BasicAddressResolver; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.util.nio.GridCommunicationClient; import org.apache.ignite.internal.util.typedef.F; @@ -35,10 +37,11 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** - * Test for {@link org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi} and {@link org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi}. + * Test for {@link TcpDiscoverySpi} and {@link TcpCommunicationSpi}. */ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest { /** */ @@ -65,30 +68,34 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest { /** */ private static final int commExtPort2 = 20100; + /** */ + private AddressResolver resolver; + + /** */ + private boolean ipFinderUseLocPorts; + /** {@inheritDoc} */ @SuppressWarnings({"IfMayBeConditional", "deprecation"}) @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(); - ipFinder.setAddresses(Arrays.asList("127.0.0.1:" + extPort1, "127.0.0.1:" + extPort2)); + + if (ipFinderUseLocPorts) + ipFinder.setAddresses(Arrays.asList("127.0.0.1:" + locPort1, "127.0.0.1:" + locPort2)); + else + ipFinder.setAddresses(Arrays.asList("127.0.0.1:" + extPort1, "127.0.0.1:" + extPort2)); TcpDiscoverySpi spi = new TcpDiscoverySpi(); final int locPort; - final int extPort; - final int commExtPort; final int commLocPort; if (getTestGridName(0).equals(gridName)) { locPort = locPort1; - extPort = extPort1; commLocPort = commLocPort1; - commExtPort = commExtPort1; } else if (getTestGridName(1).equals(gridName)) { locPort = locPort2; - extPort = extPort2; commLocPort = commLocPort2; - commExtPort = commExtPort2; } else throw new IllegalArgumentException("Unknown grid name"); @@ -106,6 +113,7 @@ else if (getTestGridName(1).equals(gridName)) { TcpCommunicationSpi commSpi = new TcpCommunicationSpi() { @Override protected GridCommunicationClient createTcpClient(ClusterNode node) throws IgniteCheckedException { Map attrs = new HashMap<>(node.attributes()); + attrs.remove(createSpiAttributeName(ATTR_PORT)); ((TcpDiscoveryNode)node).setAttributes(attrs); @@ -121,16 +129,9 @@ else if (getTestGridName(1).equals(gridName)) { cfg.setCommunicationSpi(commSpi); - final Map> mp = F.asMap( - new InetSocketAddress("127.0.0.1", locPort), F.asList(new InetSocketAddress("127.0.0.1", extPort)), - new InetSocketAddress("127.0.0.1", commLocPort), F.asList(new InetSocketAddress("127.0.0.1", commExtPort)) - ); + assert resolver != null; - cfg.setAddressResolver(new AddressResolver() { - @Override public Collection getExternalAddresses(InetSocketAddress addr) { - return mp.get(addr); - } - }); + cfg.setAddressResolver(resolver); return cfg; } @@ -138,8 +139,206 @@ else if (getTestGridName(1).equals(gridName)) { /** * @throws Exception If any error occurs. */ + public void testCustomResolver() throws Exception { + final Map> map = new HashMap<>(); + + map.put(new InetSocketAddress("127.0.0.1", locPort1), F.asList(new InetSocketAddress("127.0.0.1", extPort1))); + map.put(new InetSocketAddress("127.0.0.1", commLocPort1), F.asList(new InetSocketAddress("127.0.0.1", commExtPort1))); + map.put(new InetSocketAddress("127.0.0.1", locPort2), F.asList(new InetSocketAddress("127.0.0.1", extPort2))); + map.put(new InetSocketAddress("127.0.0.1", commLocPort2), F.asList(new InetSocketAddress("127.0.0.1", commExtPort2))); + + resolver = new AddressResolver() { + @Override public Collection getExternalAddresses(InetSocketAddress addr) { + return map.get(addr); + } + }; + + doTestForward(); + } + + /** + * @throws Exception If failed. + */ + public void testBasicResolverMapPorts() throws Exception { + Map map = new HashMap<>(); + + map.put("127.0.0.1:" + locPort1, "127.0.0.1:" + extPort1); + map.put("127.0.0.1:" + commLocPort1, "127.0.0.1:" + commExtPort1); + map.put("127.0.0.1:" + locPort2, "127.0.0.1:" + extPort2); + map.put("127.0.0.1:" + commLocPort2, "127.0.0.1:" + commExtPort2); + + resolver = new BasicAddressResolver(map); + + doTestForward(); + } + + /** + * @throws Exception If failed. + */ + public void testBasicResolverMapAddress() throws Exception { + Map map = new HashMap<>(); + + map.put("127.0.0.1", "127.0.0.1"); + + resolver = new BasicAddressResolver(map); + + ipFinderUseLocPorts = true; + + doTestForward(); + } + + /** + * @throws Exception If failed. + */ + public void testBasicResolverErrors() throws Exception { + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + return new BasicAddressResolver(null); + } + }, + IllegalArgumentException.class, + "At least one address mapping is required." + ); + + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + return new BasicAddressResolver(new HashMap()); + } + }, + IllegalArgumentException.class, + "At least one address mapping is required." + ); + + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + Map map = new HashMap<>(); + + map.put("from", null); + + return new BasicAddressResolver(map); + } + }, + IllegalArgumentException.class, + "Invalid address mapping: from=null" + ); + + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + Map map = new HashMap<>(); + + map.put("from", ""); + + return new BasicAddressResolver(map); + } + }, + IllegalArgumentException.class, + "Invalid address mapping: from=" + ); + + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + Map map = new HashMap<>(); + + map.put(null, "to"); + + return new BasicAddressResolver(map); + } + }, + IllegalArgumentException.class, + "Invalid address mapping: null=to" + ); + + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + Map map = new HashMap<>(); + + map.put("", "to"); + + return new BasicAddressResolver(map); + } + }, + IllegalArgumentException.class, + "Invalid address mapping: =to" + ); + + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + Map map = new HashMap<>(); + + map.put("from", "to:1111"); + + return new BasicAddressResolver(map); + } + }, + IllegalArgumentException.class, + "Invalid address mapping: from=to:1111" + ); + + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + Map map = new HashMap<>(); + + map.put("from:1111", "to"); + + return new BasicAddressResolver(map); + } + }, + IllegalArgumentException.class, + "Invalid address mapping: from:1111=to" + ); + + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + Map map = new HashMap<>(); + + map.put("from:1111:2222", "to:1111"); + + return new BasicAddressResolver(map); + } + }, + IllegalArgumentException.class, + "Invalid address mapping: from:1111:2222=to:1111" + ); + + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + Map map = new HashMap<>(); + + map.put("from:1111", "to:1111:2222"); + + return new BasicAddressResolver(map); + } + }, + IllegalArgumentException.class, + "Invalid address mapping: from:1111=to:1111:2222" + ); + } + + /** + * @throws Exception If failed. + */ @SuppressWarnings("UnusedDeclaration") - public void testForward() throws Exception { + private void doTestForward() throws Exception { InetAddress locHost = InetAddress.getByName("127.0.0.1"); try ( @@ -163,4 +362,4 @@ public void testForward() throws Exception { assertEquals(F.asList(13, 13), t); } } -} \ No newline at end of file +} From a596e67a1cf329ef07ebeef250166829a05ce901 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Thu, 4 Aug 2016 17:04:41 +0300 Subject: [PATCH 005/487] IGNITE-3331 IGFS: Route client tasks to primary node when metadata co-location is enabled. This closes #921. --- .../processors/igfs/IgfsMetaManager.java | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java index 0d14a3a271b6e..d1151c5b49d55 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java @@ -248,7 +248,25 @@ boolean isClient() { */ T runClientTask(IgfsClientAbstractCallable task) { try { - return clientCompute().call(task); + return runClientTask(IgfsUtils.ROOT_ID, task); + } + catch (ClusterTopologyException e) { + throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes." , e); + } + } + + /** + * Run client task. + * + * @param affinityFileId Affinity fileId. + * @param task Task. + * @return Result. + */ + T runClientTask(IgniteUuid affinityFileId, IgfsClientAbstractCallable task) { + try { + return (cfg.isColocateMetadata()) ? + clientCompute().affinityCall(cfg.getMetaCacheName(), affinityFileId, task) : + clientCompute().call(task); } catch (ClusterTopologyException e) { throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes." , e); From d945a5dd89b01f05a59e2b02fb86c1007736aff9 Mon Sep 17 00:00:00 2001 From: Ignite Teamcity Date: Thu, 4 Aug 2016 18:37:37 +0300 Subject: [PATCH 006/487] version changed to: 1.6.4 --- RELEASE_NOTES.txt | 312 ++++++++++++++---- examples/pom-standalone-lgpl.xml | 7 + examples/pom-standalone.xml | 7 + examples/pom.xml | 2 +- examples/schema-import/pom-standalone.xml | 7 + examples/schema-import/pom.xml | 2 +- modules/aop/pom.xml | 2 +- modules/apache-license-gen/pom.xml | 2 +- modules/aws/pom.xml | 2 +- modules/benchmarks/pom.xml | 2 +- modules/camel/pom.xml | 2 +- modules/cassandra/pom.xml | 2 +- modules/clients/pom.xml | 2 +- modules/cloud/pom.xml | 2 +- modules/codegen/pom.xml | 2 +- modules/core/pom.xml | 4 +- modules/extdata/p2p/pom.xml | 2 +- .../uri/modules/uri-dependency/pom.xml | 2 +- modules/extdata/uri/pom.xml | 2 +- modules/flink/pom.xml | 2 +- modules/flume/pom.xml | 2 +- modules/gce/pom.xml | 2 +- modules/geospatial/pom.xml | 2 +- modules/hadoop/pom.xml | 2 +- modules/hibernate/pom.xml | 2 +- modules/indexing/pom.xml | 2 +- modules/jcl/pom.xml | 2 +- modules/jms11/pom.xml | 2 +- modules/jta/pom.xml | 2 +- modules/kafka/pom.xml | 2 +- modules/log4j/pom.xml | 2 +- modules/log4j2/pom.xml | 2 +- modules/mesos/pom.xml | 2 +- modules/mqtt/pom.xml | 2 +- modules/osgi-karaf/pom.xml | 2 +- modules/osgi-paxlogging/pom.xml | 2 +- modules/osgi/pom.xml | 2 +- modules/rest-http/pom.xml | 2 +- modules/scalar-2.10/pom.xml | 2 +- modules/scalar/pom.xml | 2 +- modules/schedule/pom.xml | 2 +- modules/schema-import-db/pom.xml | 2 +- modules/schema-import/pom.xml | 2 +- modules/slf4j/pom.xml | 2 +- modules/spark-2.10/pom.xml | 2 +- modules/spark/pom.xml | 2 +- modules/spring/pom.xml | 2 +- modules/ssh/pom.xml | 2 +- modules/storm/pom.xml | 2 +- modules/tools/pom.xml | 2 +- modules/twitter/pom.xml | 2 +- modules/urideploy/pom.xml | 2 +- modules/visor-console-2.10/pom.xml | 2 +- modules/visor-console/pom.xml | 2 +- modules/visor-plugins/pom.xml | 2 +- modules/web-agent/pom.xml | 2 +- modules/web-console/pom.xml | 2 +- modules/web/ignite-appserver-test/pom.xml | 2 +- modules/web/ignite-websphere-test/pom.xml | 2 +- modules/web/pom.xml | 2 +- modules/yardstick/pom.xml | 2 +- modules/yarn/pom.xml | 2 +- modules/zookeeper/pom.xml | 2 +- pom.xml | 4 +- 64 files changed, 328 insertions(+), 129 deletions(-) diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt index cef51bd6fe1af..67b18ad9f4504 100644 --- a/RELEASE_NOTES.txt +++ b/RELEASE_NOTES.txt @@ -1,8 +1,23 @@ -Apache Ignite Release Notes -=========================== +GridGain In-Memory Data Fabric Professional Edition Release Notes +======================================= -Apache Ignite In-Memory Data Fabric 1.6 ---------------------------------------- +GridGain In-Memory Data Fabric Professional Edition 1.6.4 +------------------------------------ +* Fixed a bug in GridCacheTtlManager. +* Added basic implementation of address resolver. + +GridGain In-Memory Data Fabric Professional Edition 1.6.3 +------------------------------------ +* .NET: Fixed a bug in IBinaryObjectBuilder causing type name to be lost. + +GridGain In-Memory Data Fabric Professional Edition 1.6.2 +------------------------------------ +* Fixed CountDownLatch recreation. +* CPP: Fixed string deserialization performance issue. +* .NET: Added NuGet distribution. + +GridGain In-Memory Data Fabric Professional Edition 1.6.1 +------------------------------------ Ignite .NET: * Added LINQ Provider for cache SQL queries * Added native configuration mechanism (C#, app.config, web.config - instead of Spring XML) @@ -20,77 +35,218 @@ Ignite C++: * Simplified Autotools build process for C++ client. Ignite: -* Added ability to get partition 'updateCntr' with continuous query public API. -* Added asynchronous execution of ContinuousQuery's remote filter and local listener. -* Added backup partitions storing to local store default behavior. -* Added cache deadlock detection. * Added Cache Store implementation backed by Cassandra DB. -* Added method to get versioned cache entry. * Added ODBC driver for Ignite. -* Added support for join timeout while registering local addresses with IP finder in TcpDiscoverySpi. -* Added support for JTA transactions via synchronization callback. * Added Web Console for Ignite. +* ODBC: Added decimal type support. + +Changes in GridGain In-Memory Data Fabric Professional Edition 1.5.30 +------------------------------------ +* Ignite Cache: new API that lets calculating size per particular partition. +* Visor CMD: Added skip confirm in batch mode. Improved kill command. +* Hadoop: Implemented new weight-based map-reduce planner (IgniteHadoopWeightedMapReducePlanner). +* .NET: Added ability to implement custom affintiy functions. +* Added support of EXPIRED events to continuous queries. +* Data Streamer: ability to set timeout for a number of methods. +* Fixed NPE in Ignite Services internals that happened when a service was executed before system cache was started. +* Fixed NPE in OptimizedMarshaller when pool of streams is used. +* Fixed the issue when it wasn't possible to deserialize an object on Java side if the object was placed in the cache on C++ or .Net side. +* Fixed SQL indexes processing under Java 8. + +GridGain In-Memory Data Fabric Professional Edition 1.5.29 +------------------------------------ +* Binary objects: performance improvements. +* IGFS: Performance and usability improvements. +* Fixed issue with unnecessary discovery messages resend that may cause slow discovery message delivery. +* Fixed possible IO message loss during reconnect. +* Additional system properties for troubleshooting logging. +* Fixed issue when job ID was generated in a non unique way. + +GridGain In-Memory Data Fabric Professional Edition 1.5.28 +------------------------------------ +* IGFS: Usability improvements. +* Fixed performance degradation when a cache is preloaded from many IgniteDataStreamers in parallel from the same node. +* Fixed issue with service redeployment after topology change. + +GridGain In-Memory Data Fabric Professional Edition 1.5.27 +------------------------------------ +* IGFS: Fixed a problem with native libraries loading. +* REST HTTP: Updated from json-lib 2.4 to Jackson 2.7.5. +* External addresses are now registered in IP finder. +* Client node's addresses are now not registered in IP finder. + +GridGain In-Memory Data Fabric Professional Edition 1.5.26 +------------------------------------ +* IGFS: Performance improvements. +* Spark: added IgniteRDD.withKeepBinary() method for full support of binary objects. +* Added Cassandra-based persistent store implementation. +* Continuous queries: warning will be printed on client nodes instead of exception trace if the remote filter is not in classpath. +* Fixed missing IgniteCountdownLatch updates that can lead to a deadlock or wrong behavior. +* Fixed local store conflict resolving with keepBinary flag set to false. +* Fixed addresses duplication at IP finder. +* ZookeeperIpFinder now properly handles reconnects. + +GridGain In-Memory Data Fabric Professional Edition 1.5.25 +------------------------------------ +* IGFS: Performance improvements for client mode. +* Hadoop: fixed inefficient memory usage during mapper output processing. +* Fixed marshalling of arrays of primitives types when their are being placed into a cache. +* Spark: added transformer closure to savePairs(..) and saveValues(..) methods of IgniteRDD. +* Spark: fixed support for java.sql.Date in queries executed from IgniteRDD. + +GridGain In-Memory Data Fabric Professional Edition 1.5.24 +------------------------------------ +* IGFS: Added user name mapper to properly deal with security in complex Hadoop environments. +* Validating that the same IgniteConfiguration instance is not used more than once. +* Fixed marshalling of Void type with Binary marshaller. +* Fixed metrics calculation for OFFHEAP_TIRED mode. +* ScanQuery and localEntries now honor keepBinary flag. +* Zookeeper module dependencies set fixed. + +GridGain In-Memory Data Fabric Professional Edition 1.5.23 +------------------------------------ +* IGFS: Improved error logging for operations in DUAL mode. +* Added Ignition.getOrStart method. +* Spark: support BigDecimal for queries executed from IgniteRDD. +* Mesos integration: added possibility to configure http server timeout. +* Fixed a bug with Ignite API usage during node stop. +* Fixed a bug with 'keepBinary' handling for CacheInterceptor. +* Fixed a bug with CacheEntryEvent cleanup for filtered events. +* Fixed a bug with broken synchronization between cache entry expiration and update. + +GridGain In-Memory Data Fabric Professional Edition 1.5.22 +------------------------------------ +* Fixed UTF-16 surrogate pairs marshalling. +* Fixed local store behavior at cross cache transactions. +* Fixed memory leak in IgniteH2Indexing. +* Added backup partitions storing to local store default behavior. +* ScanQueries over local partitions performance optimisation. +* Visor CMD: Added ability to attach custom scripts to alerts. +* WebSession: fixed a bug causing invalidated session to be returned from HttpServletRequest.getSession() call. +* Hadoop: fixed a bug causing exception during job resource unpacking. + +GridGain In-Memory Data Fabric Professional Edition 1.5.21 +------------------------------------ +* Web sessions: minor fix to prevent NullPointerException in some special case. + +GridGain In-Memory Data Fabric Professional Edition 1.5.20 +------------------------------------ +* Web sessions: user session classes are no longer needed on server nodes. + +GridGain In-Memory Data Fabric Professional Edition 1.5.19 +------------------------------------ +* Added cache deadlock detection. * Fixed a bug causing object deserialization when local store is configured for cache. * Fixed a problem with incorrect classloader picked in OSGI environment. +* Hadoop: fixed a bug causing exception during MR planning when input split file doesn't exist. + +GridGain In-Memory Data Fabric Professional Edition 1.5.18 +------------------------------------ +* Fixed potential thread starvation during cache rebalancing. +* Fixed deadlock in services deployment when Spring is used. +* Mesos integration: added possibility to load Ignite package from file system. + +GridGain In-Memory Data Fabric Professional Edition 1.5.17 +------------------------------------ +* Performance optimization for transactions that do not acquire locks. +* Optimized memory usage on server nodes to allow more client connections. +* Added support for join timeout while registering local addresses with IP finder in TcpDiscoverySpi. + +GridGain In-Memory Data Fabric Professional Edition 1.5.16 +------------------------------------ +* Added asynchronous execution of ContinuousQuery's remote filter and local listener. + +GridGain In-Memory Data Fabric Professional Edition 1.5.15 +------------------------------------ +* Fixed discovery and node start procedure issues that may cause slowdowns during larger topologies start (>100 nodes). + +GridGain In-Memory Data Fabric Professional Edition 1.5.14 +------------------------------------ * Fixed a race condition when evicted offheap data was not delivered to query engine leading to query timeout. -* Fixed an issue in ContinuousQueries that caused missing of notifications. + +GridGain In-Memory Data Fabric Professional Edition 1.5.12 +------------------------------------ +* Improved marshaller behavior during cluster topology change. +* .NET: Added ability to use Java-based remote filters in continuous queries. + +GridGain In-Memory Data Fabric Professional Edition 1.5.11 +------------------------------------ +* Fixed cluster stability with 500+ clients. * Fixed background cache partition map exchange not to flood network. -* Fixed BinaryContext to honor custom loader set through IgniteConfiguration. * Fixed BinaryObjectOffHeapImpl leak to public code. -* Fixed cluster stability with 500+ clients. -* Fixed continuous queries to send filter factory instead of filter. -* Fixed continuous query deployment in case originating node has left. -* Fixed issues with continuous query. -* Fixed deadlock in services deployment when Spring is used. -* Fixed discovery and node start procedure issues that may cause slowdowns during larger topologies start (>100 nodes). -* Fixed eviction policy notification if swap or off heap is enabled and an entry is not loaded during preloading. -* Fixed excessive thread stack usage in case of high contention on cache entries. -* Fixed execution of Splunk MapReduce jobs on top of Ignite MR. -* Fixed GridClosureProcessor internal closures to be deserialized by BinaryMarshaller. +* Fixed BinaryContext to honor custom loader set through IgniteConfiguration. +* Fixed continuous query not to call listener with empty set of events. +* IGFS: Massive performance and stability fixes. +* Many stability and fault-tolerance fixes. + +GridGain In-Memory Data Fabric Professional Edition 1.5.10 +------------------------------------ +* Fixed continuous query listener notification handling. + +GridGain In-Memory Data Fabric Community Edition 1.5.9 +--------------------------------------- * Fixed issue with AWS dependencies. -* Fixed java proxies deserialization with optimized marshaller. -* Fixed local store behavior at cross cache transactions. -* Fixed marshalling of Java collection and maps for BinaryMarshaller. -* Fixed memory leak in IgniteH2Indexing. +* Fixed excessive thread stack usage in case of high contention on cache entries. +* Fixed continuous queries to send filter factory instead of filter. * Fixed NPE during rebalancing. -* Fixed NPE in GridMergeIndex. -* Fixed OOME when OFFHEAP_TIERED mode is used. -* Fixed potential thread starvation during cache rebalancing. -* Fixed race condition on load cache on changing topology. -* Fixed race in marshalling logic that could arise when several nodes are running on the same host. -* Fixed redeployment issues happened with ScanQueries filters. +* Fixed system caches not to use user-defined TransactionConfiguration. +* Fixed GridClosureProcessor internal closures to be deserialized by BinaryMarshaller. +* Fixed web session clustering with WebLogic. +* Fixed value copying in entry processor with OptimizedMarshaller. +* Fixed continuous query listener notification during concurrent key put. * Fixed service proxy not to make remote call for methods declared in java.lang.Object. -* Fixed several ClassNotFoundException in OSGi environment. +* Added support for JTA transactions via synchronization callback. + +GridGain In-Memory Data Fabric Community Edition 1.5.8 +--------------------------------------- +* Fixed java proxies deserialization with optimized marshaller. +* Fixed excessive thread stack usage in case of high contention on cache entries. * Fixed SKIP_STORE flag behavior for transaction entry. -* Fixed support for classes with equal simple name for binary marshaller. -* Fixed system caches not to use user-defined TransactionConfiguration. -* Fixed the issue when CacheStore was updated even if EntryProcessor didn't update an entry. * Fixed thread safety for TcpDiscoveryMulticastIpFinder. -* Fixed unintentional deserialization of BinaryObjects in OFFHEAP mode with peer class loading enabled. -* Fixed UTF-16 surrogate pairs marshalling. -* Fixed value copying in entry processor with OptimizedMarshaller. -* Fixed web session clustering with WebLogic. -* Hadoop: fixed a bug causing exception during MR planning when input split file doesn't exist. -* IGFS: Added configuration flag to disable default path modes under "/ignite" folder. -* IGFS: Added pluggable factory interface for Hadoop FileSystem creation. * IGFS: Fixed file properties when running over secondary file system. * IGFS: Fixed issues with Kerberized HDFS. -* IGFS: Massive performance and stability fixes. -* Improved marshaller behavior during cluster topology change. -* Mesos integration: added possibility to load Ignite package from file system. -* Optimized cache 'get' operations on affinity node. -* Optimized memory usage on server nodes to allow more client connections. -* Passed update notifier flag to discovery data to be sure that all cluster nodes have the same notifier status as the first node. -* Performance optimization for transactions that do not acquire locks. -* ScanQueries over local partitions performance optimisation. + +GridGain In-Memory Data Fabric Community Edition 1.5.7 +--------------------------------------- +* Added method to get versioned cache entry. +* Added ability to get partition 'updateCntr' with continuous query public API. +* Fixed several ClassNotFoundException in OSGi environment. +* Fixed continuous query notification with single node topology. +* Fixed continuous query notification with cache listeners. +* Fixed continuous query deployment in case originating node has left. +* Fixed NPE in GridMergeIndex. +* Fixed race condition on load cache on changing topology. +* Fixed OOME when OFFHEAP_TIERED mode is used. +* Fixed support for classes with equal simple name for binary marshaller. + +GridGain In-Memory Data Fabric Community Edition 1.5.6 +--------------------------------------- * Support of package-private implementations for Ignite Services. +* Fixed unintentional deserialization of BinaryObjects in OFFHEAP mode with peer class loading enabled. + +GridGain In-Memory Data Fabric Community Edition 1.5.5 +--------------------------------------- +* Fixed the issue when CacheStore was updated even if EntryProcessor didn't update an entry. +* Fixed execution of Splunk MapReduce jobs on top of Ignite MR. + +GridGain In-Memory Data Fabric Community Edition 1.5.4 +--------------------------------------- +* Passed update notifier flag to discovery data to be sure that all cluster nodes have the same notifier status as the first node. +* Optimized cache 'get' operations on affinity node. * Supported Ignite Compute tasks cancellation for Ignite.NET. -* Visor CMD: Added ability to attach custom scripts to alerts. -* Web sessions: minor fix to prevent NullPointerException in some special case. -* Web sessions: user session classes are no longer needed on server nodes. -* A lot of stability and fault-tolerance fixes. +* Fixed race in marshalling logic that could arise when several nodes are running on the same host. +* Fixed marshalling of Java collection and maps for BinaryMarshaller. +* Fixed eviction policy notification if swap or off heap is enabled and an entry is not loaded during preloading. +* Fixed redeployment issues happened with ScanQueries filters. +* Fixed an issue in ContinuousQueries that caused missing of notifications. -Apache Ignite In-Memory Data Fabric 1.5 +GridGain In-Memory Data Fabric Community Edition 1.5.3 +--------------------------------------- +* IGFS: Added pluggable factory interface for Hadoop FileSystem creation. +* IGFS: Added configuration flag to disable default path modes under "/ignite" folder. +* IGFS: Various stability fixes. + +GridGain In-Memory Data Fabric Community Edition 1.5.1 --------------------------------------- * Ignite.NET: Initial Release. * Ignite C++: Initial Release. @@ -113,14 +269,12 @@ Apache Ignite In-Memory Data Fabric 1.5 Complete list of closed issues: https://issues.apache.org/jira/issues/?jql=project%20%3D%20IGNITE%20AND%20fixVersion%20%3D%201.5%20AND%20status%20%3D%20closed -Apache Ignite In-Memory Data Fabric 1.4 +GridGain In-Memory Data Fabric Community Edition 1.4.1 --------------------------------------- * Added SSL support to communication and discovery. * Added support for log4j2. * Added versioned entry to cache API. * Added client node based JDBC driver implementation. -* Fixed consistency issue for custom events processing within discovery. -* Fixed race in cache swap/unswap logic. * Fixed IGNITE_HOME resolution with JBoss. * Fixed configuration file resolution on classpath. * Fixed MBean registration for caches. @@ -131,21 +285,39 @@ Apache Ignite In-Memory Data Fabric 1.4 * Fixed affinity routing in compute grid. * Many stability and fault-tolerance fixes. -Apache Ignite In-Memory Data Fabric 1.3 +GridGain In-Memory Data Fabric Community Edition 1.3.3 +--------------------------------------- + +* Added SSL support to communication and discovery. +* Fixed IGNITE_HOME resolution with JBoss. +* Fixed configuration file resolution on classpath. +* Fixed MBean registration for caches. +* Many stability and fault-tolerance fixes. + +GridGain In-Memory Data Fabric Community Edition 1.3.1 --------------------------------------- * Added auto-retries for cache operations in recoverable cases. * Added integration with Apache YARN. -* Added auto detection and dropping of slow client nodes. * Fixed several issues with JTA integration. * Fixed several issues with Hibernate L2 cache. -* Fixed issue with GAR files in source release. * Stability fixes for TCP discovery SPI. * Stability fixes for onheap and offheap SQL queries. * Bug fixes in In-Memory Accelerator For Apache Hadoop. * Many stability and fault-tolerance fixes. -Apache Ignite In-Memory Data Fabric 1.2 +GridGain In-Memory Data Fabric Community Edition 1.1.4 +--------------------------------------- + +* Added dropping of slow clients. +* Fixed several connectivity issues. + +GridGain In-Memory Data Fabric Community Edition 1.1.3 +--------------------------------------- + +* Fixed cache store configuration validation. + +GridGain In-Memory Data Fabric Community Edition 1.1.2 --------------------------------------- * Added client mode to TCP discovery SPI. @@ -157,7 +329,13 @@ Apache Ignite In-Memory Data Fabric 1.2 * Bug fixes in In-Memory Accelerator For Apache Hadoop. * Many stability and fault-tolerance fixes. -Apache Ignite In-Memory Data Fabric 1.1 +GridGain In-Memory Data Fabric Community Edition 1.0.5 +--------------------------------------- + +* Fixed unnecessary value sending for near cache with near lock response +* Optimized getOrCreateCache methods for case when cache is already started locally + +GridGain In-Memory Data Fabric Community Edition 1.0.4 --------------------------------------- * Added Google Compute Engine TCP discovery IP finder. @@ -177,7 +355,7 @@ Apache Ignite In-Memory Data Fabric 1.1 * Made deployment scanners for URI-based deployment pluggable. * Many stability and fault-tolerance fixes. -Apache Ignite In-Memory Data Fabric 1.0 +GridGain In-Memory Data Fabric Community Edition 1.0 --------------------------------------- * Simplified query API. @@ -193,7 +371,7 @@ Apache Ignite In-Memory Data Fabric 1.0 * Added ability to automatically exclude LGPL optional dependencies during build. -Apache Ignite In-Memory Data Fabric 1.0 RC3 +GridGain In-Memory Data Fabric Community Edition 1.0 RC3 ------------------------------------------- This is the first release of Apache Ignite project. The source code in large part is based diff --git a/examples/pom-standalone-lgpl.xml b/examples/pom-standalone-lgpl.xml index 7e7ccc22f0d4a..73bbc691fb65d 100644 --- a/examples/pom-standalone-lgpl.xml +++ b/examples/pom-standalone-lgpl.xml @@ -26,6 +26,13 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 + + + GridGain External Repository + http://www.gridgainsystems.com/nexus/content/repositories/external + + + UTF-8 diff --git a/examples/pom-standalone.xml b/examples/pom-standalone.xml index 6247ced4ef758..b01d67a1e4e7b 100644 --- a/examples/pom-standalone.xml +++ b/examples/pom-standalone.xml @@ -26,6 +26,13 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 + + + GridGain External Repository + http://www.gridgainsystems.com/nexus/content/repositories/external + + + UTF-8 diff --git a/examples/pom.xml b/examples/pom.xml index 979a46f4a92dc..203e1d39644f9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ ignite-examples - 1.7.0-SNAPSHOT + 1.6.4 diff --git a/examples/schema-import/pom-standalone.xml b/examples/schema-import/pom-standalone.xml index e443728bb97d1..95c8a9d93c3a3 100644 --- a/examples/schema-import/pom-standalone.xml +++ b/examples/schema-import/pom-standalone.xml @@ -26,6 +26,13 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 + + + GridGain External Repository + http://www.gridgainsystems.com/nexus/content/repositories/external + + + UTF-8 diff --git a/examples/schema-import/pom.xml b/examples/schema-import/pom.xml index b6e36efed3064..6c5a957f7e949 100644 --- a/examples/schema-import/pom.xml +++ b/examples/schema-import/pom.xml @@ -35,7 +35,7 @@ ignite-schema-import-demo - 1.7.0-SNAPSHOT + 1.6.4 diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml index eaebaf25c137a..a3b2727949402 100644 --- a/modules/aop/pom.xml +++ b/modules/aop/pom.xml @@ -31,7 +31,7 @@ ignite-aop - 1.7.0-SNAPSHOT + 1.6.4 http://ignite.apache.org diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml index 53b1a8f9114de..cdf15fbf3ac55 100644 --- a/modules/apache-license-gen/pom.xml +++ b/modules/apache-license-gen/pom.xml @@ -31,7 +31,7 @@ org.apache.ignite ignite-apache-license-gen - 1.7.0-SNAPSHOT + 1.6.4 http://ignite.apache.org diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml index 2aee655a2ddbc..04e9585de13ae 100644 --- a/modules/aws/pom.xml +++ b/modules/aws/pom.xml @@ -31,7 +31,7 @@ ignite-aws - 1.7.0-SNAPSHOT + 1.6.4 http://ignite.apache.org diff --git a/modules/benchmarks/pom.xml b/modules/benchmarks/pom.xml index 00315a83b7ac2..28c2ed120e637 100644 --- a/modules/benchmarks/pom.xml +++ b/modules/benchmarks/pom.xml @@ -31,7 +31,7 @@ ignite-benchmarks - 1.7.0-SNAPSHOT + 1.6.4 http://ignite.apache.org diff --git a/modules/camel/pom.xml b/modules/camel/pom.xml index f40ff62e01146..52ba0916fc7aa 100644 --- a/modules/camel/pom.xml +++ b/modules/camel/pom.xml @@ -31,7 +31,7 @@ ignite-camel - 1.7.0-SNAPSHOT + 1.6.4 http://ignite.apache.org diff --git a/modules/cassandra/pom.xml b/modules/cassandra/pom.xml index dc8b08ca89892..77a4414a953b1 100644 --- a/modules/cassandra/pom.xml +++ b/modules/cassandra/pom.xml @@ -31,7 +31,7 @@ ignite-cassandra - 1.7.0-SNAPSHOT + 1.6.4 http://ignite.apache.org diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml index fa25d18902fff..bb53662d5bfad 100644 --- a/modules/clients/pom.xml +++ b/modules/clients/pom.xml @@ -31,7 +31,7 @@ ignite-clients - 1.7.0-SNAPSHOT + 1.6.4 http://ignite.apache.org diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml index 8232d0028d2d9..6ebc61c01fe37 100644 --- a/modules/cloud/pom.xml +++ b/modules/cloud/pom.xml @@ -29,7 +29,7 @@ ignite-cloud - 1.7.0-SNAPSHOT + 1.6.4 http://ignite.apache.org diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml index aff0d3ac9cf25..ef21c6bf33ad1 100644 --- a/modules/codegen/pom.xml +++ b/modules/codegen/pom.xml @@ -31,7 +31,7 @@ ignite-codegen - 1.7.0-SNAPSHOT + 1.6.4 http://ignite.apache.org diff --git a/modules/core/pom.xml b/modules/core/pom.xml index 5e32caf580a4b..d287967173e1a 100644 --- a/modules/core/pom.xml +++ b/modules/core/pom.xml @@ -31,7 +31,7 @@ ignite-core - 1.7.0-SNAPSHOT + 1.6.4 http://ignite.apache.org @@ -42,7 +42,7 @@ - apache-ignite + gridgain-community - ${project.artifactId}-${ignite.edition}-${project.version}-bin + gridgain-professional-${ignite.edition}-${project.version} From 311428eee8d52640a6efdac721335f63a4244d38 Mon Sep 17 00:00:00 2001 From: isapego Date: Fri, 5 Aug 2016 10:38:50 +0300 Subject: [PATCH 007/487] IGNITE-3587: ODBC: Added distributed joins support. This closes #908. --- .../processors/odbc/OdbcHandshakeRequest.java | 42 +++- .../processors/odbc/OdbcHandshakeResult.java | 17 +- .../processors/odbc/OdbcMessageParser.java | 34 ++- .../processors/odbc/OdbcProtocolVersion.java | 125 ++++++++++ .../processors/odbc/OdbcRequestHandler.java | 26 ++- modules/platforms/cpp/odbc-test/Makefile.am | 1 + .../odbc-test/config/queries-test-noodbc.xml | 103 +++++++++ .../cpp/odbc-test/config/queries-test.xml | 45 ++-- .../odbc-test/project/vs/odbc-test.vcxproj | 1 + .../project/vs/odbc-test.vcxproj.filters | 3 + .../cpp/odbc-test/src/configuration_test.cpp | 148 ++++++++++-- .../cpp/odbc-test/src/queries_test.cpp | 218 ++++++++++++++++-- modules/platforms/cpp/odbc/Makefile.am | 1 + .../platforms/cpp/odbc/include/Makefile.am | 1 + .../odbc/include/ignite/odbc/common_types.h | 3 + .../ignite/odbc/config/configuration.h | 67 +++++- .../cpp/odbc/include/ignite/odbc/connection.h | 16 +- .../cpp/odbc/include/ignite/odbc/message.h | 35 ++- .../cpp/odbc/include/ignite/odbc/parser.h | 3 - .../include/ignite/odbc/protocol_version.h | 172 ++++++++++++++ .../cpp/odbc/project/vs/odbc.vcxproj | 2 + .../cpp/odbc/project/vs/odbc.vcxproj.filters | 6 + .../cpp/odbc/src/config/configuration.cpp | 92 ++++++-- modules/platforms/cpp/odbc/src/connection.cpp | 23 +- .../odbc/src/diagnostic/diagnostic_record.cpp | 6 + .../cpp/odbc/src/protocol_version.cpp | 134 +++++++++++ 26 files changed, 1204 insertions(+), 120 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolVersion.java create mode 100644 modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml create mode 100644 modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h create mode 100644 modules/platforms/cpp/odbc/src/protocol_version.cpp diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java index 5e09041ed3044..55ff21f687350 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java @@ -24,24 +24,58 @@ */ public class OdbcHandshakeRequest extends OdbcRequest { /** Protocol version. */ - private final long ver; + private final OdbcProtocolVersion ver; + + /** Distributed joins flag. */ + private boolean distributedJoins = false; + + /** Enforce join order flag. */ + private boolean enforceJoinOrder = false; /** - * @param ver Protocol version. + * @param ver Long value for protocol version. */ public OdbcHandshakeRequest(long ver) { super(HANDSHAKE); - this.ver = ver; + this.ver = OdbcProtocolVersion.fromLong(ver); } /** * @return Protocol version. */ - public long version() { + public OdbcProtocolVersion version() { return ver; } + /** + * @return Distributed joins flag. + */ + public boolean distributedJoins() { + return distributedJoins; + } + + /** + * @param distributedJoins Distributed joins flag. + */ + public void distributedJoins(boolean distributedJoins) { + this.distributedJoins = distributedJoins; + } + + /** + * @return Enforce join order flag. + */ + public boolean enforceJoinOrder() { + return enforceJoinOrder; + } + + /** + * @param enforceJoinOrder Enforce join order flag. + */ + public void enforceJoinOrder(boolean enforceJoinOrder) { + this.enforceJoinOrder = enforceJoinOrder; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(OdbcHandshakeRequest.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java index bf1c61eee8794..74c5bd4ec08bb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java @@ -17,7 +17,7 @@ package org.apache.ignite.internal.processors.odbc; -import org.jetbrains.annotations.Nullable; +import org.apache.ignite.internal.util.typedef.internal.S; /** * ODBC handshake result. @@ -33,11 +33,13 @@ public class OdbcHandshakeResult { private final String curVer; /** - * @param accepted Handshake accepted. + * Constructor. + * + * @param accepted Indicates whether handshake accepted or not. * @param protoVerSince Apache Ignite version when protocol version has been introduced. * @param curVer Current Apache Ignite version. */ - public OdbcHandshakeResult(boolean accepted, @Nullable String protoVerSince, @Nullable String curVer) { + public OdbcHandshakeResult(boolean accepted, String protoVerSince, String curVer) { this.accepted = accepted; this.protoVerSince = protoVerSince; this.curVer = curVer; @@ -53,14 +55,19 @@ public boolean accepted() { /** * @return Apache Ignite version when protocol version has been introduced. */ - @Nullable public String protoVerSince() { + public String protocolVersionSince() { return protoVerSince; } /** * @return Current Apache Ignite version. */ - @Nullable public String currentVer() { + public String currentVersion() { return curVer; } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(OdbcHandshakeResult.class, this); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java index fce8b1badba11..e8b594e264b86 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java @@ -34,12 +34,6 @@ * ODBC message parser. */ public class OdbcMessageParser { - /** Current ODBC communication protocol version. */ - public static final long PROTO_VER = 1; - - /** Apache Ignite version when ODBC communication protocol version has been introduced. */ - public static final String PROTO_VER_SINCE = "1.6.0"; - /** Initial output stream capacity. */ private static final int INIT_CAP = 1024; @@ -82,10 +76,26 @@ public OdbcRequest decode(byte[] msg) { // we has not confirmed that the remote client uses the same protocol version. if (!verConfirmed) { if (cmd == OdbcRequest.HANDSHAKE) - return new OdbcHandshakeRequest(reader.readLong()); + { + long longVersion = reader.readLong(); + + OdbcHandshakeRequest res = new OdbcHandshakeRequest(longVersion); + + OdbcProtocolVersion version = res.version(); + + if (version.isUnknown()) + return res; + + if (version.isDistributedJoinsSupported()) { + res.distributedJoins(reader.readBoolean()); + res.enforceJoinOrder(reader.readBoolean()); + } + + return res; + } else - throw new IgniteException("Unexpected ODBC command (first message is not a handshake request): [cmd=" + - cmd + ']'); + throw new IgniteException("Unexpected ODBC command " + + "(first message is not a handshake request): [cmd=" + cmd + ']'); } OdbcRequest res; @@ -174,6 +184,8 @@ public byte[] encode(OdbcResponse msg) { Object res0 = msg.response(); + if (res0 == null) + return writer.array(); if (res0 instanceof OdbcHandshakeResult) { OdbcHandshakeResult res = (OdbcHandshakeResult) res0; @@ -189,8 +201,8 @@ public byte[] encode(OdbcResponse msg) { } else { writer.writeBoolean(false); - writer.writeString(res.protoVerSince()); - writer.writeString(res.currentVer()); + writer.writeString(res.protocolVersionSince()); + writer.writeString(res.currentVersion()); } } else if (res0 instanceof OdbcQueryExecuteResult) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolVersion.java new file mode 100644 index 0000000000000..57efa02a473b8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolVersion.java @@ -0,0 +1,125 @@ +/* + * 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.odbc; + +import java.util.HashMap; +import java.util.Map; + +/** + * ODBC protocol version. + */ +public enum OdbcProtocolVersion { + /** First version of the ODBC. Released with Ignite 1.6 */ + VERSION_1_6_0(1), + + /** Second version of the ODBC. Released with Ignite 1.8 */ + VERSION_1_8_0(makeVersion(1,8,0)), + + /** Unknown version. */ + VERSION_UNKNOWN(Long.MIN_VALUE); + + /** Mask to get 2 lowest bytes of the value and cast to long. */ + private static final long LONG_MASK = 0x000000000000FFFFL; + + /** Long value to enum map. */ + private static final Map versions = new HashMap<>(); + + /** Enum value to Ignite version map */ + private static final Map since = new HashMap<>(); + + /** + * Map long values to version. + */ + static { + for (OdbcProtocolVersion version : values()) + versions.put(version.longValue(), version); + + since.put(VERSION_1_6_0, "1.6.0"); + since.put(VERSION_1_8_0, "1.8.0"); + } + + /** Long value for version. */ + private final long longVal; + + /** + * @param longVal Long value. + */ + OdbcProtocolVersion(long longVal) { + this.longVal = longVal; + } + + /** + * Make long value for the version. + * + * @param major Major version. + * @param minor Minor version. + * @param maintenance Maintenance version. + * @return Long value for the version. + */ + private static long makeVersion(int major, int minor, int maintenance) { + return ((major & LONG_MASK) << 48) | ((minor & LONG_MASK) << 32) | ((maintenance & LONG_MASK) << 16); + } + + /** + * @param longVal Long value. + * @return Protocol version. + */ + public static OdbcProtocolVersion fromLong(long longVal) { + OdbcProtocolVersion res = versions.get(longVal); + + return res == null ? VERSION_UNKNOWN : res; + } + + /** + * @return Current version. + */ + public static OdbcProtocolVersion current() { + return VERSION_1_8_0; + } + + /** + * @return Long value. + */ + public long longValue() { + return longVal; + } + + /** + * @return {@code true} if this version is unknown. + */ + public boolean isUnknown() { + return longVal == VERSION_UNKNOWN.longVal; + } + + /** + * @return {@code true} if this version supports distributed joins. + */ + public boolean isDistributedJoinsSupported() { + assert !isUnknown(); + + return longVal >= VERSION_1_8_0.longVal; + } + + /** + * @return Ignite version when introduced. + */ + public String since() { + assert !isUnknown(); + + return since.get(this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java index 43a1fa4d2899a..8f2d092188192 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java @@ -54,6 +54,12 @@ public class OdbcRequestHandler { /** Current queries cursors. */ private final ConcurrentHashMap> qryCursors = new ConcurrentHashMap<>(); + /** Distributed joins flag. */ + private boolean distributedJoins = false; + + /** Enforce join order flag. */ + private boolean enforceJoinOrder = false; + /** * Constructor. * @@ -115,16 +121,23 @@ public OdbcResponse handle(OdbcRequest req) { * @return Response. */ private OdbcResponse performHandshake(OdbcHandshakeRequest req) { - OdbcHandshakeResult res; + OdbcProtocolVersion version = req.version(); - if (req.version() == OdbcMessageParser.PROTO_VER) - res = new OdbcHandshakeResult(true, null, null); - else { + if (version.isUnknown()) { IgniteProductVersion ver = ctx.grid().version(); String verStr = Byte.toString(ver.major()) + '.' + ver.minor() + '.' + ver.maintenance(); - res = new OdbcHandshakeResult(false, OdbcMessageParser.PROTO_VER_SINCE, verStr); + OdbcHandshakeResult res = new OdbcHandshakeResult(false, OdbcProtocolVersion.current().since(), verStr); + + return new OdbcResponse(res); + } + + OdbcHandshakeResult res = new OdbcHandshakeResult(true, null, null); + + if (version.isDistributedJoinsSupported()) { + distributedJoins = req.distributedJoins(); + enforceJoinOrder = req.enforceJoinOrder(); } return new OdbcResponse(res); @@ -151,6 +164,9 @@ private OdbcResponse executeQuery(OdbcQueryExecuteRequest req) { qry.setArgs(req.arguments()); + qry.setDistributedJoins(distributedJoins); + qry.setEnforceJoinOrder(enforceJoinOrder); + IgniteCache cache = ctx.grid().cache(req.cacheName()); if (cache == null) diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index 48b193abbcd7d..a878d0231acd5 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -66,6 +66,7 @@ ignite_odbc_tests_SOURCES = \ ../odbc/src/app/application_data_buffer.cpp \ ../odbc/src/config/configuration.cpp \ ../odbc/src/row.cpp \ + ../odbc/src/protocol_version.cpp \ ../odbc/src/column.cpp \ ../odbc/src/utility.cpp \ ../odbc/src/result_page.cpp diff --git a/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml new file mode 100644 index 0000000000000..18447c28446f3 --- /dev/null +++ b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml @@ -0,0 +1,103 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500 + + + + + + + + + diff --git a/modules/platforms/cpp/odbc-test/config/queries-test.xml b/modules/platforms/cpp/odbc-test/config/queries-test.xml index 054da4272a02b..54cb9beb694df 100644 --- a/modules/platforms/cpp/odbc-test/config/queries-test.xml +++ b/modules/platforms/cpp/odbc-test/config/queries-test.xml @@ -41,17 +41,18 @@ - - + + - - - + + @@ -81,12 +93,11 @@ instead os static IP based discovery. --> - - + - 127.0.0.1:47500..47501 + 127.0.0.1:47500 diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index fbc0929f4ba58..833904e318e9a 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -157,6 +157,7 @@ + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index 2e38c242b7742..aead2af8ca3d8 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -76,6 +76,9 @@ Code + + Externals + diff --git a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp index 10fd137282b25..c0bb439024954 100644 --- a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp @@ -20,6 +20,7 @@ #endif #include +#include #include @@ -36,6 +37,8 @@ namespace const uint16_t testServerPort = 4242; const std::string testCacheName = "TestCache"; const std::string testDsn = "Ignite DSN"; + const bool testDistributedJoins = true; + const bool testEnforceJoinOrder = true; const std::string testAddress = testServerHost + ':' + ignite::common::LexicalCast(testServerPort); } @@ -49,6 +52,42 @@ void CheckValidAddress(const char* connectStr, uint16_t port) BOOST_CHECK_EQUAL(cfg.GetPort(), port); } +void CheckValidProtocolVersion(const char* connectStr, ignite::odbc::ProtocolVersion version) +{ + Configuration cfg; + + BOOST_CHECK_NO_THROW(cfg.FillFromConnectString(connectStr)); + + BOOST_CHECK(cfg.GetProtocolVersion() == version); +} + +void CheckInvalidProtocolVersion(const char* connectStr) +{ + Configuration cfg; + + cfg.FillFromConnectString(connectStr); + + BOOST_CHECK_THROW(cfg.GetProtocolVersion(), ignite::IgniteError); +} + +void CheckValidBoolValue(const std::string& connectStr, const std::string& key, bool val) +{ + Configuration cfg; + + BOOST_CHECK_NO_THROW(cfg.FillFromConnectString(connectStr)); + + BOOST_CHECK_EQUAL(cfg.GetBoolValue(key, val), val); +} + +void CheckInvalidBoolValue(const std::string& connectStr, const std::string& key) +{ + Configuration cfg; + + cfg.FillFromConnectString(connectStr); + + BOOST_CHECK_THROW(cfg.GetBoolValue(key, false), ignite::IgniteError); +} + void CheckConnectionConfig(const Configuration& cfg) { BOOST_CHECK_EQUAL(cfg.GetDriver(), testDriverName); @@ -57,16 +96,20 @@ void CheckConnectionConfig(const Configuration& cfg) BOOST_CHECK_EQUAL(cfg.GetAddress(), testAddress); BOOST_CHECK_EQUAL(cfg.GetCache(), testCacheName); BOOST_CHECK_EQUAL(cfg.GetDsn(), std::string()); + BOOST_CHECK_EQUAL(cfg.IsDistributedJoins(), testDistributedJoins); + BOOST_CHECK_EQUAL(cfg.IsEnforceJoinOrder(), testEnforceJoinOrder); std::stringstream constructor; constructor << "address=" << testAddress << ';' << "cache=" << testCacheName << ';' - << "driver={" << testDriverName << "};"; + << "distributed_joins=" << (testDistributedJoins ? "true" : "false") << ';' + << "driver={" << testDriverName << "};" + << "enforce_join_order=" << (testEnforceJoinOrder ? "true" : "false") << ';'; const std::string& expectedStr = constructor.str(); - BOOST_CHECK_EQUAL(cfg.ToConnectString(), expectedStr); + BOOST_CHECK_EQUAL(ignite::common::ToLower(cfg.ToConnectString()), ignite::common::ToLower(expectedStr)); } void CheckDsnConfig(const Configuration& cfg) @@ -76,7 +119,9 @@ void CheckDsnConfig(const Configuration& cfg) BOOST_CHECK_EQUAL(cfg.GetCache(), Configuration::DefaultValue::cache); BOOST_CHECK_EQUAL(cfg.GetAddress(), Configuration::DefaultValue::address); BOOST_CHECK_EQUAL(cfg.GetHost(), std::string()); - BOOST_CHECK_EQUAL(cfg.GetPort(), Configuration::DefaultValue::uintPort); + BOOST_CHECK_EQUAL(cfg.GetPort(), Configuration::DefaultValue::port); + BOOST_CHECK_EQUAL(cfg.IsDistributedJoins(), false); + BOOST_CHECK_EQUAL(cfg.IsEnforceJoinOrder(), false); } BOOST_AUTO_TEST_SUITE(ConfigurationTestSuite) @@ -85,9 +130,11 @@ BOOST_AUTO_TEST_CASE(CheckTestValuesNotEquealDefault) { BOOST_CHECK_NE(testDriverName, Configuration::DefaultValue::driver); BOOST_CHECK_NE(testAddress, Configuration::DefaultValue::address); - BOOST_CHECK_NE(testServerPort, Configuration::DefaultValue::uintPort); + BOOST_CHECK_NE(testServerPort, Configuration::DefaultValue::port); BOOST_CHECK_NE(testCacheName, Configuration::DefaultValue::cache); BOOST_CHECK_NE(testDsn, Configuration::DefaultValue::dsn); + BOOST_CHECK_NE(testDistributedJoins, Configuration::DefaultValue::distributedJoins); + BOOST_CHECK_NE(testEnforceJoinOrder, Configuration::DefaultValue::enforceJoinOrder); } BOOST_AUTO_TEST_CASE(TestConnectStringUppercase) @@ -98,7 +145,9 @@ BOOST_AUTO_TEST_CASE(TestConnectStringUppercase) constructor << "DRIVER={" << testDriverName << "};" << "ADDRESS=" << testAddress << ';' - << "CACHE=" << testCacheName; + << "CACHE=" << testCacheName << ';' + << "DISTRIBUTED_JOINS=" << (testDistributedJoins ? "TRUE" : "FALSE") << ';' + << "ENFORCE_JOIN_ORDER=" << (testEnforceJoinOrder ? "TRUE" : "FALSE"); const std::string& connectStr = constructor.str(); @@ -115,7 +164,9 @@ BOOST_AUTO_TEST_CASE(TestConnectStringLowercase) constructor << "driver={" << testDriverName << "};" << "address=" << testAddress << ';' - << "cache=" << testCacheName; + << "cache=" << testCacheName << ';' + << "distributed_joins=" << (testDistributedJoins ? "true" : "false") << ';' + << "enforce_join_order=" << (testEnforceJoinOrder ? "true" : "false"); const std::string& connectStr = constructor.str(); @@ -132,7 +183,9 @@ BOOST_AUTO_TEST_CASE(TestConnectStringZeroTerminated) constructor << "driver={" << testDriverName << "};" << "address=" << testAddress << ';' - << "cache=" << testCacheName; + << "cache=" << testCacheName << ';' + << "distributed_joins=" << (testDistributedJoins ? "true" : "false") << ';' + << "enforce_join_order=" << (testEnforceJoinOrder ? "true" : "false"); const std::string& connectStr = constructor.str(); @@ -149,7 +202,9 @@ BOOST_AUTO_TEST_CASE(TestConnectStringMixed) constructor << "Driver={" << testDriverName << "};" << "Address=" << testAddress << ';' - << "Cache=" << testCacheName; + << "Cache=" << testCacheName << ';' + << "Distributed_Joins=" << (testDistributedJoins ? "True" : "False") << ';' + << "Enforce_Join_Order=" << (testEnforceJoinOrder ? "True" : "False"); const std::string& connectStr = constructor.str(); @@ -166,7 +221,9 @@ BOOST_AUTO_TEST_CASE(TestConnectStringWhitepaces) constructor << "DRIVER = {" << testDriverName << "} ;\n" << " ADDRESS =" << testAddress << "; " - << "CACHE = \n\r" << testCacheName; + << "CACHE = \n\r" << testCacheName << ';' + << " DISTRIBUTED_JOINS=" << (testDistributedJoins ? "TRUE" : "FALSE") << ';' + << "ENFORCE_JOIN_ORDER= " << (testEnforceJoinOrder ? "TRUE " : "FALSE "); const std::string& connectStr = constructor.str(); @@ -190,13 +247,74 @@ BOOST_AUTO_TEST_CASE(TestConnectStringInvalidAddress) BOOST_AUTO_TEST_CASE(TestConnectStringValidAddress) { - Configuration cfg; - CheckValidAddress("Address=example.com:1;", 1); CheckValidAddress("Address=example.com:31242;", 31242); CheckValidAddress("Address=example.com:55555;", 55555); CheckValidAddress("Address=example.com:110;", 110); - CheckValidAddress("Address=example.com;", Configuration::DefaultValue::uintPort); + CheckValidAddress("Address=example.com;", Configuration::DefaultValue::port); +} + +BOOST_AUTO_TEST_CASE(TestConnectStringInvalidVersion) +{ + CheckInvalidProtocolVersion("Protocol_Version=0;"); + CheckInvalidProtocolVersion("Protocol_Version=1;"); + CheckInvalidProtocolVersion("Protocol_Version=2;"); + CheckInvalidProtocolVersion("Protocol_Version=1.6.1;"); + CheckInvalidProtocolVersion("Protocol_Version=1.7.0;"); + CheckInvalidProtocolVersion("Protocol_Version=1.8.1;"); +} + +BOOST_AUTO_TEST_CASE(TestConnectStringValidVersion) +{ + CheckValidProtocolVersion("Protocol_Version=1.6.0;", ignite::odbc::ProtocolVersion::VERSION_1_6_0); + CheckValidProtocolVersion("Protocol_Version=1.8.0;", ignite::odbc::ProtocolVersion::VERSION_1_8_0); +} + +BOOST_AUTO_TEST_CASE(TestConnectStringInvalidBoolKeys) +{ + typedef std::set Set; + + Set keys; + + keys.insert("distributed_joins"); + keys.insert("enforce_join_order"); + + for (Set::const_iterator it = keys.begin(); it != keys.end(); ++it) + { + const std::string& key = *it; + + CheckInvalidBoolValue(key + "=1;", key); + CheckInvalidBoolValue(key + "=0;", key); + CheckInvalidBoolValue(key + "=42;", key); + CheckInvalidBoolValue(key + "=truee;", key); + CheckInvalidBoolValue(key + "=flase;", key); + CheckInvalidBoolValue(key + "=falsee;", key); + CheckInvalidBoolValue(key + "=yes;", key); + CheckInvalidBoolValue(key + "=no;", key); + } +} + +BOOST_AUTO_TEST_CASE(TestConnectStringValidBoolKeys) +{ + typedef std::set Set; + + Set keys; + + keys.insert("distributed_joins"); + keys.insert("enforce_join_order"); + + for (Set::const_iterator it = keys.begin(); it != keys.end(); ++it) + { + const std::string& key = *it; + + CheckValidBoolValue(key + "=true;", key, true); + CheckValidBoolValue(key + "=True;", key, true); + CheckValidBoolValue(key + "=TRUE;", key, true); + + CheckValidBoolValue(key + "=false;", key, false); + CheckValidBoolValue(key + "=False;", key, false); + CheckValidBoolValue(key + "=FALSE;", key, false); + } } BOOST_AUTO_TEST_CASE(TestDsnStringUppercase) @@ -215,7 +333,7 @@ BOOST_AUTO_TEST_CASE(TestDsnStringUppercase) CheckDsnConfig(cfg); } -BOOST_AUTO_TEST_CASE(TestDsnStrinLowercase) +BOOST_AUTO_TEST_CASE(TestDsnStringLowercase) { Configuration cfg; @@ -231,7 +349,7 @@ BOOST_AUTO_TEST_CASE(TestDsnStrinLowercase) CheckDsnConfig(cfg); } -BOOST_AUTO_TEST_CASE(TestDsnStrinMixed) +BOOST_AUTO_TEST_CASE(TestDsnStringMixed) { Configuration cfg; @@ -247,7 +365,7 @@ BOOST_AUTO_TEST_CASE(TestDsnStrinMixed) CheckDsnConfig(cfg); } -BOOST_AUTO_TEST_CASE(TestDsnStrinWhitespaces) +BOOST_AUTO_TEST_CASE(TestDsnStringWhitespaces) { Configuration cfg; diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index ccb3a4d6f5696..58b5f64f661a1 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -122,6 +122,44 @@ struct QueriesTestSuiteFixture BOOST_REQUIRE(stmt != NULL); } + void Disconnect() + { + // Releasing statement handle. + SQLFreeHandle(SQL_HANDLE_STMT, stmt); + + // Disconneting from the server. + SQLDisconnect(dbc); + + // Releasing allocated handles. + SQLFreeHandle(SQL_HANDLE_DBC, dbc); + SQLFreeHandle(SQL_HANDLE_ENV, env); + } + + static Ignite StartAdditionalNode(const char* name) + { + IgniteConfiguration cfg; + + cfg.jvmOpts.push_back("-Xdebug"); + cfg.jvmOpts.push_back("-Xnoagent"); + cfg.jvmOpts.push_back("-Djava.compiler=NONE"); + cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005"); + cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError"); + +#ifdef IGNITE_TESTS_32 + cfg.jvmInitMem = 256; + cfg.jvmMaxMem = 768; +#else + cfg.jvmInitMem = 1024; + cfg.jvmMaxMem = 4096; +#endif + + cfg.springCfgPath.assign(getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH")).append("/queries-test-noodbc.xml"); + + IgniteError err; + + return Ignition::Start(cfg, name); + } + /** * Constructor. */ @@ -143,16 +181,11 @@ struct QueriesTestSuiteFixture cfg.jvmMaxMem = 4096; #endif - char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH"); - - cfg.springCfgPath = std::string(cfgPath).append("/").append("queries-test.xml"); + cfg.springCfgPath.assign(getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH")).append("/queries-test.xml"); IgniteError err; - grid = Ignition::Start(cfg, &err); - - if (err.GetCode() != IgniteError::IGNITE_SUCCESS) - BOOST_FAIL(err.GetText()); + grid = Ignition::Start(cfg, "NodeMain"); testCache = grid.GetCache("cache"); } @@ -162,17 +195,9 @@ struct QueriesTestSuiteFixture */ ~QueriesTestSuiteFixture() { - // Releasing statement handle. - SQLFreeHandle(SQL_HANDLE_STMT, stmt); - - // Disconneting from the server. - SQLDisconnect(dbc); - - // Releasing allocated handles. - SQLFreeHandle(SQL_HANDLE_DBC, dbc); - SQLFreeHandle(SQL_HANDLE_ENV, env); + Disconnect(); - Ignition::Stop(grid.GetName(), true); + Ignition::StopAll(true); } template @@ -269,6 +294,28 @@ struct QueriesTestSuiteFixture BOOST_CHECK(ret == SQL_NO_DATA); } + int CountRows(SQLHSTMT stmt) + { + int res = 0; + + SQLRETURN ret = SQL_SUCCESS; + + while (ret == SQL_SUCCESS) + { + ret = SQLFetch(stmt); + + if (ret == SQL_NO_DATA) + break; + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ++res; + } + + return res; + } + /** Node started during the test. */ Ignite grid; @@ -292,6 +339,16 @@ BOOST_AUTO_TEST_CASE(TestLegacyConnection) Connect("DRIVER={Apache Ignite};SERVER=127.0.0.1;PORT=11110;CACHE=cache"); } +BOOST_AUTO_TEST_CASE(TestConnectionProtocolVersion_1_6_0) +{ + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache;PROTOCOL_VERSION=1.6.0"); +} + +BOOST_AUTO_TEST_CASE(TestConnectionProtocolVersion_1_8_0) +{ + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache;PROTOCOL_VERSION=1.8.0"); +} + BOOST_AUTO_TEST_CASE(TestTwoRowsInt8) { CheckTwoRowsInt(SQL_C_STINYINT); @@ -538,4 +595,131 @@ BOOST_AUTO_TEST_CASE(TestOneRowStringLen) BOOST_CHECK(ret == SQL_NO_DATA); } +BOOST_AUTO_TEST_CASE(TestDistributedJoins) +{ + // Starting additional node. + Ignite node1 = StartAdditionalNode("Node1"); + Ignite node2 = StartAdditionalNode("Node2"); + + const int entriesNum = 1000; + + // Filling cache with data. + for (int i = 0; i < entriesNum; ++i) + { + TestType entry; + + entry.i32Field = i; + entry.i64Field = entriesNum - i - 1; + + testCache.Put(i, entry); + } + + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); + + SQLRETURN ret; + + const size_t columnsCnt = 2; + + SQLBIGINT columns[columnsCnt] = { 0 }; + + // Binding colums. + for (SQLSMALLINT i = 0; i < columnsCnt; ++i) + { + ret = SQLBindCol(stmt, i + 1, SQL_C_SLONG, &columns[i], 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + } + + SQLCHAR request[] = + "SELECT T0.i32Field, T1.i64Field FROM TestType AS T0 " + "INNER JOIN TestType AS T1 " + "ON (T0.i32Field = T1.i64Field)"; + + ret = SQLExecDirect(stmt, request, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + int rowsNum = CountRows(stmt); + + BOOST_CHECK_GT(rowsNum, 0); + BOOST_CHECK_LT(rowsNum, entriesNum); + + Disconnect(); + + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache;DISTRIBUTED_JOINS=true;"); + + // Binding colums. + for (SQLSMALLINT i = 0; i < columnsCnt; ++i) + { + ret = SQLBindCol(stmt, i + 1, SQL_C_SLONG, &columns[i], 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + } + + ret = SQLExecDirect(stmt, request, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + rowsNum = CountRows(stmt); + + BOOST_CHECK_EQUAL(rowsNum, entriesNum); +} + +BOOST_AUTO_TEST_CASE(TestDistributedJoinsWithOldVersion) +{ + // Starting additional node. + Ignite node1 = StartAdditionalNode("Node1"); + Ignite node2 = StartAdditionalNode("Node2"); + + const int entriesNum = 1000; + + // Filling cache with data. + for (int i = 0; i < entriesNum; ++i) + { + TestType entry; + + entry.i32Field = i; + entry.i64Field = entriesNum - i - 1; + + testCache.Put(i, entry); + } + + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache;DISTRIBUTED_JOINS=true;PROTOCOL_VERSION=1.6.0"); + + SQLRETURN ret; + + const size_t columnsCnt = 2; + + SQLBIGINT columns[columnsCnt] = { 0 }; + + // Binding colums. + for (SQLSMALLINT i = 0; i < columnsCnt; ++i) + { + ret = SQLBindCol(stmt, i + 1, SQL_C_SLONG, &columns[i], 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + } + + SQLCHAR request[] = + "SELECT T0.i32Field, T1.i64Field FROM TestType AS T0 " + "INNER JOIN TestType AS T1 " + "ON (T0.i32Field = T1.i64Field)"; + + ret = SQLExecDirect(stmt, request, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + int rowsNum = CountRows(stmt); + + BOOST_CHECK_GT(rowsNum, 0); + BOOST_CHECK_LT(rowsNum, entriesNum); +} + + BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am index 29f0ef48e9c7d..9faa999bf4ed8 100644 --- a/modules/platforms/cpp/odbc/Makefile.am +++ b/modules/platforms/cpp/odbc/Makefile.am @@ -69,6 +69,7 @@ libignite_odbc_la_SOURCES = \ src/query/table_metadata_query.cpp \ src/query/type_info_query.cpp \ src/query/special_columns_query.cpp \ + src/protocol_version.cpp \ src/result_page.cpp \ src/row.cpp \ src/column.cpp \ diff --git a/modules/platforms/cpp/odbc/include/Makefile.am b/modules/platforms/cpp/odbc/include/Makefile.am index 192021dc7764f..0776548849ca6 100644 --- a/modules/platforms/cpp/odbc/include/Makefile.am +++ b/modules/platforms/cpp/odbc/include/Makefile.am @@ -27,6 +27,7 @@ noinst_HEADERS = \ ignite/odbc/query/column_metadata_query.h \ ignite/odbc/query/query.h \ ignite/odbc/query/primary_keys_query.h \ + ignite/odbc/protocol_version.h \ ignite/odbc/statement.h \ ignite/odbc/config/configuration.h \ ignite/odbc/config/connection_info.h \ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h index 6636ca4d28050..250eaf2c16c0c 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h @@ -60,6 +60,9 @@ namespace ignite /** Output data has been truncated. */ SQL_STATE_01004_DATA_TRUNCATED, + /** Invalid connection string attribute. */ + SQL_STATE_01S00_INVALID_CONNECTION_STRING_ATTRIBUTE, + /** Error in row. */ SQL_STATE_01S01_ERROR_IN_ROW, diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h index 05fe8bfdb9c95..30f9ad6afcb36 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h @@ -24,6 +24,7 @@ #include #include +#include "ignite/odbc/protocol_version.h" namespace ignite { @@ -60,6 +61,15 @@ namespace ignite /** Connection attribute keyword for port attribute. */ static const std::string port; + + /** Connection attribute keyword for distributed joins attribute. */ + static const std::string distributedJoins; + + /** Connection attribute keyword for enforce join order attribute. */ + static const std::string enforceJoinOrder; + + /** Connection attribute keyword for protocol version attribute. */ + static const std::string protocolVersion; }; /** Default values for configuration. */ @@ -80,11 +90,17 @@ namespace ignite /** Default value for server attribute. */ static const std::string server; + /** Default value for protocol version. */ + static const ProtocolVersion& protocolVersion; + /** Default value for port attribute. */ - static const std::string port; + static const uint16_t port; - /** Default value for port attribute. Uint16 value. */ - static const uint16_t uintPort; + /** Default value for distributed joins attribute. */ + static const bool distributedJoins; + + /** Default value for enforce join order attribute. */ + static const bool enforceJoinOrder; }; /** @@ -199,6 +215,33 @@ namespace ignite return GetStringValue(Key::address, DefaultValue::address); } + /** + * Check distributed joins flag. + * + * @return True if distributed joins are enabled. + */ + bool IsDistributedJoins() const + { + return GetBoolValue(Key::distributedJoins, DefaultValue::distributedJoins); + } + + /** + * Check enforce join order flag. + * + * @return True if enforcing of join order is enabled. + */ + bool IsEnforceJoinOrder() const + { + return GetBoolValue(Key::enforceJoinOrder, DefaultValue::enforceJoinOrder); + } + + /** + * Get protocol version. + * + * @return Protocol version. + */ + ProtocolVersion GetProtocolVersion() const; + /** * Get string value from the config. * @@ -208,6 +251,24 @@ namespace ignite */ const std::string& GetStringValue(const std::string& key, const std::string& dflt) const; + /** + * Get int value from the config. + * + * @param key Configuration key. + * @param dflt Default value to be returned if there is no value stored. + * @return Found or default value. + */ + int64_t GetIntValue(const std::string& key, int64_t dflt) const; + + /** + * Get bool value from the config. + * + * @param key Configuration key. + * @param dflt Default value to be returned if there is no value stored. + * @return Found or default value. + */ + bool GetBoolValue(const std::string& key, bool dflt) const; + private: /** * Parse connect string into key-value storage. diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h b/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h index 00bdfc81c3c0d..9fe46df6d779a 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h @@ -41,15 +41,6 @@ namespace ignite { friend class Environment; public: - /** ODBC communication protocol version. */ - enum { PROTOCOL_VERSION = 1 }; - - /** - * Apache Ignite version when the current ODBC communication - * protocol version has been introduced. - */ - static const std::string PROTOCOL_VERSION_SINCE; - /** * Destructor. */ @@ -262,6 +253,13 @@ namespace ignite */ SqlResult MakeRequestHandshake(); + /** + * Perform configure request. + * + * @return Operation result. + */ + SqlResult MakeRequestConfigure(); + /** * Constructor. */ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h index f0b40e284d5bd..03fa6279c6cf8 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/message.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/message.h @@ -66,8 +66,13 @@ namespace ignite * Constructor. * * @param version Protocol version. + * @param distributedJoins Distributed joins flag. + * @param enforceJoinOrder Enforce join order flag. */ - HandshakeRequest(int64_t version) : version(version) + HandshakeRequest(int64_t version, bool distributedJoins, bool enforceJoinOrder) : + version(version), + distributedJoins(distributedJoins), + enforceJoinOrder(enforceJoinOrder) { // No-op. } @@ -89,11 +94,20 @@ namespace ignite writer.WriteInt8(REQUEST_TYPE_HANDSHAKE); writer.WriteInt64(version); + + writer.WriteBool(distributedJoins); + writer.WriteBool(enforceJoinOrder); } private: /** Protocol version. */ int64_t version; + + /** Distributed joins flag. */ + bool distributedJoins; + + /** Enforce join order flag. */ + bool enforceJoinOrder; }; /** @@ -153,6 +167,7 @@ namespace ignite const app::ParameterBindingMap& params; }; + /** * Query close request. */ @@ -348,13 +363,13 @@ namespace ignite /** * Query close response. */ - class QueryResponse + class Response { public: /** * Constructor. */ - QueryResponse() : status(RESPONSE_STATUS_FAILED), error() + Response() : status(RESPONSE_STATUS_FAILED), error() { // No-op. } @@ -362,7 +377,7 @@ namespace ignite /** * Destructor. */ - ~QueryResponse() + virtual ~Response() { // No-op. } @@ -426,7 +441,7 @@ namespace ignite /** * Handshake response. */ - class HandshakeResponse : public QueryResponse + class HandshakeResponse : public Response { public: /** @@ -504,7 +519,7 @@ namespace ignite /** * Query close response. */ - class QueryCloseResponse : public QueryResponse + class QueryCloseResponse : public Response { public: /** @@ -549,7 +564,7 @@ namespace ignite /** * Query execute response. */ - class QueryExecuteResponse : public QueryResponse + class QueryExecuteResponse : public Response { public: /** @@ -608,7 +623,7 @@ namespace ignite /** * Query fetch response. */ - class QueryFetchResponse : public QueryResponse + class QueryFetchResponse : public Response { public: /** @@ -659,7 +674,7 @@ namespace ignite /** * Query get column metadata response. */ - class QueryGetColumnsMetaResponse : public QueryResponse + class QueryGetColumnsMetaResponse : public Response { public: /** @@ -704,7 +719,7 @@ namespace ignite /** * Query get table metadata response. */ - class QueryGetTablesMetaResponse : public QueryResponse + class QueryGetTablesMetaResponse : public Response { public: /** diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h b/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h index c19e08c10d8ae..a91af229bcb3d 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h @@ -42,9 +42,6 @@ namespace ignite /** Default initial size of operational memory. */ enum { DEFAULT_MEM_ALLOCATION = 4096 }; - /** ODBC communication protocol version. */ - enum { PROTOCOL_VERSION = 1 }; - /** * Constructor. */ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h b/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h new file mode 100644 index 0000000000000..747d78d31efeb --- /dev/null +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h @@ -0,0 +1,172 @@ +/* + * 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. + */ + +#ifndef _IGNITE_ODBC_PROTOCOL_VERSION +#define _IGNITE_ODBC_PROTOCOL_VERSION + +#include + +#include +#include + +namespace ignite +{ + namespace odbc + { + /** Protocol version. */ + class ProtocolVersion + { + public: + /** String to version map type alias. */ + typedef std::map StringToVersionMap; + + /** Version to string map type alias. */ + typedef std::map VersionToStringMap; + + /** First version of the protocol that was introduced in Ignite 1.6.0. */ + static const ProtocolVersion VERSION_1_6_0; + + /** First version of the protocol that was introduced in Ignite 1.8.0. */ + static const ProtocolVersion VERSION_1_8_0; + + /** Unknown version of the protocol. */ + static const ProtocolVersion VERSION_UNKNOWN; + + /** + * Get current version. + * + * @return Current version. + */ + static const ProtocolVersion& GetCurrent(); + + /** + * Parse string and extract protocol version. + * + * @throw IgniteException if version can not be parsed. + * @param version Version string to parse. + * @return Protocol version. + */ + static ProtocolVersion FromString(const std::string& version); + + /** + * Convert to string value. + * + * @throw IgniteException if version is unknow parsed. + * @param version Version string to parse. + * @return Protocol version. + */ + const std::string& ToString() const; + + /** + * Get int value. + * + * @return Integer value. + */ + int64_t GetIntValue() const; + + /** + * Check if the version is unknown. + * + * @return True if the version is unknown. + */ + bool IsUnknown() const; + + /** + * Comparison operator. + * + * @param val1 First value. + * @param val2 Second value. + * @return True if equal. + */ + friend bool operator==(const ProtocolVersion& val1, const ProtocolVersion& val2); + + /** + * Comparison operator. + * + * @param val1 First value. + * @param val2 Second value. + * @return True if not equal. + */ + friend bool operator!=(const ProtocolVersion& val1, const ProtocolVersion& val2); + + /** + * Comparison operator. + * + * @param val1 First value. + * @param val2 Second value. + * @return True if less. + */ + friend bool operator<(const ProtocolVersion& val1, const ProtocolVersion& val2); + + /** + * Comparison operator. + * + * @param val1 First value. + * @param val2 Second value. + * @return True if less or equal. + */ + friend bool operator<=(const ProtocolVersion& val1, const ProtocolVersion& val2); + + /** + * Comparison operator. + * + * @param val1 First value. + * @param val2 Second value. + * @return True if gretter. + */ + friend bool operator>(const ProtocolVersion& val1, const ProtocolVersion& val2); + + /** + * Comparison operator. + * + * @param val1 First value. + * @param val2 Second value. + * @return True if gretter or equal. + */ + friend bool operator>=(const ProtocolVersion& val1, const ProtocolVersion& val2); + + private: + /** + * Constructor. + * + * @param val Underlying value. + */ + explicit ProtocolVersion(int64_t val); + + /** + * Make int value for the version. + * + * @param major Major version. + * @param minor Minor version. + * @param maintenance Maintenance version. + * @return Int value for the version. + */ + static int64_t MakeVersion(uint16_t major, uint16_t minor, uint16_t maintenance); + + /** String to version map. */ + static const StringToVersionMap stringToVersionMap; + + /** Version to string map. */ + static const VersionToStringMap versionToStringMap; + + /** Underlying int value. */ + int64_t val; + }; + } +} + +#endif //_IGNITE_ODBC_PROTOCOL_VERSION \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj index 5820030d00cb1..0e0f0d3a65142 100644 --- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj +++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj @@ -170,6 +170,7 @@ + @@ -206,6 +207,7 @@ + diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters index 6ca58e22a3ad8..9caf4839f8e6e 100644 --- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters +++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters @@ -115,6 +115,9 @@ Code + + Code + @@ -224,5 +227,8 @@ Code + + Code + \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/src/config/configuration.cpp b/modules/platforms/cpp/odbc/src/config/configuration.cpp index 45b05079537a8..24c2bdf456245 100644 --- a/modules/platforms/cpp/odbc/src/config/configuration.cpp +++ b/modules/platforms/cpp/odbc/src/config/configuration.cpp @@ -20,6 +20,9 @@ #include #include +#include "ignite/common/common.h" +#include "ignite/common/utils.h" + #include "ignite/odbc/utility.h" #include "ignite/odbc/config/configuration.h" @@ -29,20 +32,29 @@ namespace ignite { namespace config { - const std::string Configuration::Key::dsn = "dsn"; - const std::string Configuration::Key::driver = "driver"; - const std::string Configuration::Key::cache = "cache"; - const std::string Configuration::Key::address = "address"; - const std::string Configuration::Key::server = "server"; - const std::string Configuration::Key::port = "port"; - - const std::string Configuration::DefaultValue::dsn = "Apache Ignite DSN"; - const std::string Configuration::DefaultValue::driver = "Apache Ignite"; - const std::string Configuration::DefaultValue::cache = ""; - const std::string Configuration::DefaultValue::address = ""; - const std::string Configuration::DefaultValue::server = ""; - const std::string Configuration::DefaultValue::port = "10800"; - const uint16_t Configuration::DefaultValue::uintPort = common::LexicalCast(port); + const std::string Configuration::Key::dsn = "dsn"; + const std::string Configuration::Key::driver = "driver"; + const std::string Configuration::Key::cache = "cache"; + const std::string Configuration::Key::address = "address"; + const std::string Configuration::Key::server = "server"; + const std::string Configuration::Key::port = "port"; + const std::string Configuration::Key::distributedJoins = "distributed_joins"; + const std::string Configuration::Key::enforceJoinOrder = "enforce_join_order"; + const std::string Configuration::Key::protocolVersion = "protocol_version"; + + const std::string Configuration::DefaultValue::dsn = "Apache Ignite DSN"; + const std::string Configuration::DefaultValue::driver = "Apache Ignite"; + const std::string Configuration::DefaultValue::cache = ""; + const std::string Configuration::DefaultValue::address = ""; + const std::string Configuration::DefaultValue::server = ""; + + const uint16_t Configuration::DefaultValue::port = 10800; + + const bool Configuration::DefaultValue::distributedJoins = false; + const bool Configuration::DefaultValue::enforceJoinOrder = false; + + const ProtocolVersion& Configuration::DefaultValue::protocolVersion = ProtocolVersion::GetCurrent(); + Configuration::Configuration() : arguments() @@ -80,7 +92,7 @@ namespace ignite else { endPoint.host = GetStringValue(Key::server, DefaultValue::server); - endPoint.port = common::LexicalCast(GetStringValue(Key::port, DefaultValue::port)); + endPoint.port = static_cast(GetIntValue(Key::port, DefaultValue::port)); } } @@ -134,10 +146,20 @@ namespace ignite else { endPoint.host = GetStringValue(Key::server, DefaultValue::server); - endPoint.port = common::LexicalCast(GetStringValue(Key::port, DefaultValue::port)); + endPoint.port = static_cast(GetIntValue(Key::port, DefaultValue::port)); } } + ProtocolVersion Configuration::GetProtocolVersion() const + { + ArgumentMap::const_iterator it = arguments.find(Key::protocolVersion); + + if (it != arguments.end()) + return ProtocolVersion::FromString(it->second); + + return DefaultValue::protocolVersion; + } + const std::string& Configuration::GetStringValue(const std::string& key, const std::string& dflt) const { ArgumentMap::const_iterator it = arguments.find(common::ToLower(key)); @@ -148,6 +170,42 @@ namespace ignite return dflt; } + int64_t Configuration::GetIntValue(const std::string& key, int64_t dflt) const + { + ArgumentMap::const_iterator it = arguments.find(common::ToLower(key)); + + if (it != arguments.end()) + { + const std::string& val = it->second; + + if (!common::AllOf(val.begin(), val.end(), isdigit)) + IGNITE_ERROR_FORMATTED_1(IgniteError::IGNITE_ERR_GENERIC, + "Invalid argument value: Integer value is expected.", "key", key); + + return common::LexicalCast(val); + } + + return dflt; + } + + bool Configuration::GetBoolValue(const std::string& key, bool dflt) const + { + ArgumentMap::const_iterator it = arguments.find(common::ToLower(key)); + + if (it != arguments.end()) + { + std::string lowercaseVal = common::ToLower(it->second); + + if (lowercaseVal != "true" && lowercaseVal != "false") + IGNITE_ERROR_FORMATTED_1(IgniteError::IGNITE_ERR_GENERIC, + "Invalid argument value: Boolean value is expected (true or false).", "key", key); + + return lowercaseVal == "true"; + } + + return dflt; + } + void Configuration::ParseAttributeList(const char * str, size_t len, char delimeter, ArgumentMap & args) { std::string connect_str(str, len); @@ -199,7 +257,7 @@ namespace ignite if (colonNum == 0) { res.host = address; - res.port = DefaultValue::uintPort; + res.port = DefaultValue::port; } else if (colonNum == 1) { diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp index 844ad706b9877..4315698756717 100644 --- a/modules/platforms/cpp/odbc/src/connection.cpp +++ b/modules/platforms/cpp/odbc/src/connection.cpp @@ -39,8 +39,6 @@ namespace ignite { namespace odbc { - const std::string Connection::PROTOCOL_VERSION_SINCE = "1.6.0"; - Connection::Connection() : socket(), connected(false), @@ -309,7 +307,24 @@ namespace ignite SqlResult Connection::MakeRequestHandshake() { - HandshakeRequest req(PROTOCOL_VERSION); + bool distributedJoins = false; + bool enforceJoinOrder = false; + int64_t protocolVersion = 0; + + try + { + distributedJoins = config.IsDistributedJoins(); + enforceJoinOrder = config.IsEnforceJoinOrder(); + protocolVersion = config.GetProtocolVersion().GetIntValue(); + } + catch (const IgniteError& err) + { + AddStatusRecord(SQL_STATE_01S00_INVALID_CONNECTION_STRING_ATTRIBUTE, err.GetText()); + + return SQL_RESULT_ERROR; + } + + HandshakeRequest req(protocolVersion, distributedJoins, enforceJoinOrder); HandshakeResponse rsp; try @@ -343,7 +358,7 @@ namespace ignite constructor << "Node rejected handshake message. " << "Current node Apache Ignite version: " << rsp.CurrentVer() << ", " << "node protocol version introduced in version: " << rsp.ProtoVerSince() << ", " - << "driver protocol version introduced in version: " << PROTOCOL_VERSION_SINCE << "."; + << "driver protocol version introduced in version: " << config.GetProtocolVersion().ToString() << "."; AddStatusRecord(SQL_STATE_08001_CANNOT_CONNECT, constructor.str()); diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp index 568c125046b42..0fdfbc87fb3e3 100644 --- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp +++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp @@ -34,6 +34,9 @@ namespace /** SQL state 01004 constant. */ const std::string STATE_01004 = "01004"; + /** SQL state 01S00 constant. */ + const std::string STATE_01S00 = "01S00"; + /** SQL state 01S01 constant. */ const std::string STATE_01S01 = "01S01"; @@ -190,6 +193,9 @@ namespace ignite case SQL_STATE_01004_DATA_TRUNCATED: return STATE_01004; + case SQL_STATE_01S00_INVALID_CONNECTION_STRING_ATTRIBUTE: + return STATE_01S00; + case SQL_STATE_01S01_ERROR_IN_ROW: return STATE_01S01; diff --git a/modules/platforms/cpp/odbc/src/protocol_version.cpp b/modules/platforms/cpp/odbc/src/protocol_version.cpp new file mode 100644 index 0000000000000..c65099d38ead8 --- /dev/null +++ b/modules/platforms/cpp/odbc/src/protocol_version.cpp @@ -0,0 +1,134 @@ +/* + * 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. + */ + +#include "ignite/odbc/protocol_version.h" +#include +#include +#include + +namespace ignite +{ + namespace odbc + { + const ProtocolVersion ProtocolVersion::VERSION_1_6_0(1); + const ProtocolVersion ProtocolVersion::VERSION_1_8_0(MakeVersion(1,8,0)); + const ProtocolVersion ProtocolVersion::VERSION_UNKNOWN(INT64_MIN); + + ProtocolVersion::StringToVersionMap::value_type s2vInitVals[] = { + std::make_pair("1.6.0", ProtocolVersion::VERSION_1_6_0), + std::make_pair("1.8.0", ProtocolVersion::VERSION_1_8_0) + }; + + const ProtocolVersion::StringToVersionMap ProtocolVersion::stringToVersionMap(s2vInitVals, + s2vInitVals + (sizeof(s2vInitVals) / sizeof(s2vInitVals[0]))); + + ProtocolVersion::VersionToStringMap::value_type v2sInitVals[] = { + std::make_pair(ProtocolVersion::VERSION_1_6_0, "1.6.0"), + std::make_pair(ProtocolVersion::VERSION_1_8_0, "1.8.0") + }; + + const ProtocolVersion::VersionToStringMap ProtocolVersion::versionToStringMap(v2sInitVals, + v2sInitVals + (sizeof(v2sInitVals) / sizeof(v2sInitVals[0]))); + + ProtocolVersion::ProtocolVersion(int64_t val) : + val(val) + { + // No-op. + } + + int64_t ProtocolVersion::MakeVersion(uint16_t major, uint16_t minor, uint16_t maintenance) + { + const static int64_t MASK = 0x000000000000FFFFLL; + return ((major & MASK) << 48) | ((minor & MASK) << 32) | ((maintenance & MASK) << 16); + } + + const ProtocolVersion& ProtocolVersion::GetCurrent() + { + return VERSION_1_8_0; + } + + ProtocolVersion ProtocolVersion::FromString(const std::string& version) + { + StringToVersionMap::const_iterator it = stringToVersionMap.find(common::ToLower(version)); + + if (it == stringToVersionMap.end()) + { + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, + "Invalid version format. Valid format is X.Y.Z, where X, Y and Z are major, " + "minor and maintenance versions of Ignite since which protocol is introduced."); + } + + return it->second; + } + + const std::string& ProtocolVersion::ToString() const + { + VersionToStringMap::const_iterator it = versionToStringMap.find(*this); + + if (it == versionToStringMap.end()) + { + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, + "Unknown protocol version can not be converted to string."); + } + + return it->second; + } + + int64_t ProtocolVersion::GetIntValue() const + { + assert(!IsUnknown()); + + return val; + } + + bool ProtocolVersion::IsUnknown() const + { + return *this == VERSION_UNKNOWN; + } + + bool operator==(const ProtocolVersion& val1, const ProtocolVersion& val2) + { + return val1.val == val2.val; + } + + bool operator!=(const ProtocolVersion& val1, const ProtocolVersion& val2) + { + return val1.val != val2.val; + } + + bool operator<(const ProtocolVersion& val1, const ProtocolVersion& val2) + { + return val1.val < val2.val; + } + + bool operator<=(const ProtocolVersion& val1, const ProtocolVersion& val2) + { + return val1.val <= val2.val; + } + + bool operator>(const ProtocolVersion& val1, const ProtocolVersion& val2) + { + return val1.val > val2.val; + } + + bool operator>=(const ProtocolVersion& val1, const ProtocolVersion& val2) + { + return val1.val >= val2.val; + } + } +} + From 78a21f100849dfbd2c3663a79c701e892a836618 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Fri, 5 Aug 2016 15:05:32 +0300 Subject: [PATCH 008/487] IGNITE-3631: IGFS: Now metadata co-location is used for PARTITIONED cache as well. --- .../org/apache/ignite/internal/processors/igfs/IgfsUtils.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java index b9788f881675f..3a313ad7e57a5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java @@ -376,8 +376,7 @@ public static void prepareCacheConfiguration(IgniteConfiguration cfg, CacheConfi ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); // Set co-located affinity mapper if needed. - if (igfsCfg.isColocateMetadata() && ccfg.getCacheMode() == CacheMode.REPLICATED && - ccfg.getAffinityMapper() == null) + if (igfsCfg.isColocateMetadata() && ccfg.getAffinityMapper() == null) ccfg.setAffinityMapper(new IgfsColocatedMetadataAffinityKeyMapper()); return; From bb0fb7293b7e430431e7699101320f59b3b93067 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Fri, 5 Aug 2016 15:05:43 +0300 Subject: [PATCH 009/487] IGNITE-3631: IGFS: Now metadata co-location is used for PARTITIONED cache as well. --- .../apache/ignite/internal/processors/igfs/IgfsUtils.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java index 3a313ad7e57a5..3d86fbb1f5376 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java @@ -17,13 +17,11 @@ package org.apache.ignite.internal.processors.igfs; -import java.util.Set; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.binary.BinaryRawReader; import org.apache.ignite.binary.BinaryRawWriter; -import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.cluster.ClusterTopologyException; @@ -32,8 +30,8 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.IgfsEvent; import org.apache.ignite.igfs.IgfsException; -import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.binary.BinaryUtils; @@ -61,6 +59,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; From 00f47d78216db5de75dda2a1b5fef2c1e65d9871 Mon Sep 17 00:00:00 2001 From: Valentin Kulichenko Date: Fri, 5 Aug 2016 18:04:01 -0700 Subject: [PATCH 010/487] IGNITE-3634 - SpringCacheManager should support null values --- .../ignite/cache/spring/SpringCache.java | 44 +++++++- .../cache/spring/SpringCacheManager.java | 7 +- .../GridSpringCacheManagerSelfTest.java | 102 +++++++++++++++++- .../spring/GridSpringCacheTestKey.java | 4 +- .../GridSpringCacheTestKeyGenerator.java | 4 +- .../spring/GridSpringCacheTestService.java | 60 ++++++++++- .../GridSpringDynamicCacheTestService.java | 4 +- ...pringCacheManagerContextInjectionTest.java | 3 +- .../{ => cache}/spring/spring-caching.xml | 6 +- .../testsuites/IgniteSpringTestSuite.java | 4 +- 10 files changed, 212 insertions(+), 26 deletions(-) rename modules/spring/src/test/java/org/apache/ignite/{ => cache}/spring/GridSpringCacheManagerSelfTest.java (78%) rename modules/spring/src/test/java/org/apache/ignite/{ => cache}/spring/GridSpringCacheTestKey.java (97%) rename modules/spring/src/test/java/org/apache/ignite/{ => cache}/spring/GridSpringCacheTestKeyGenerator.java (97%) rename modules/spring/src/test/java/org/apache/ignite/{ => cache}/spring/GridSpringCacheTestService.java (72%) rename modules/spring/src/test/java/org/apache/ignite/{ => cache}/spring/GridSpringDynamicCacheTestService.java (98%) rename modules/spring/src/test/java/org/apache/ignite/{ => cache}/spring/SpringCacheManagerContextInjectionTest.java (97%) rename modules/spring/src/test/java/org/apache/ignite/{ => cache}/spring/spring-caching.xml (86%) diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java index 6014205452b64..afd060ad8d174 100644 --- a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java +++ b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java @@ -17,6 +17,7 @@ package org.apache.ignite.cache.spring; +import java.io.Serializable; import org.apache.ignite.IgniteCache; import org.springframework.cache.Cache; import org.springframework.cache.support.SimpleValueWrapper; @@ -25,6 +26,9 @@ * Spring cache implementation. */ class SpringCache implements Cache { + /** */ + private static final Object NULL = new NullValue(); + /** */ private final IgniteCache cache; @@ -51,7 +55,7 @@ class SpringCache implements Cache { @Override public ValueWrapper get(Object key) { Object val = cache.get(key); - return val != null ? new SimpleValueWrapper(val) : null; + return val != null ? fromValue(val) : null; } /** {@inheritDoc} */ @@ -59,6 +63,9 @@ class SpringCache implements Cache { @Override public T get(Object key, Class type) { Object val = cache.get(key); + if (NULL.equals(val)) + val = null; + if (val != null && type != null && !type.isInstance(val)) throw new IllegalStateException("Cached value is not of required type [cacheName=" + cache.getName() + ", key=" + key + ", val=" + val + ", requiredType=" + type + ']'); @@ -68,14 +75,22 @@ class SpringCache implements Cache { /** {@inheritDoc} */ @Override public void put(Object key, Object val) { - cache.put(key, val); + if (val == null) + cache.withSkipStore().put(key, NULL); + else + cache.put(key, val); } /** {@inheritDoc} */ @Override public ValueWrapper putIfAbsent(Object key, Object val) { - Object old = cache.putIfAbsent(key, val); + Object old; + + if (val == null) + old = cache.withSkipStore().putIfAbsent(key, NULL); + else + old = cache.putIfAbsent(key, val); - return old != null ? new SimpleValueWrapper(old) : null; + return old != null ? fromValue(old) : null; } /** {@inheritDoc} */ @@ -87,4 +102,23 @@ class SpringCache implements Cache { @Override public void clear() { cache.removeAll(); } -} \ No newline at end of file + + /** + * @param val Cache value. + * @return Wrapped value. + */ + private static ValueWrapper fromValue(Object val) { + assert val != null; + + return new SimpleValueWrapper(NULL.equals(val) ? null : val); + } + + /** + */ + private static class NullValue implements Serializable { + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + return this == o || (o != null && getClass() == o.getClass()); + } + } +} diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java index 6d1a9b6ffc90d..fbdeb73e204be 100644 --- a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java +++ b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java @@ -28,6 +28,7 @@ import org.apache.ignite.configuration.NearCacheConfiguration; import org.jsr166.ConcurrentHashMap8; import org.springframework.beans.factory.InitializingBean; +import org.springframework.cache.Cache; import org.springframework.cache.CacheManager; import org.springframework.context.ApplicationContext; import org.springframework.context.ApplicationContextAware; @@ -158,7 +159,7 @@ public class SpringCacheManager implements CacheManager, InitializingBean, Appli /** Ignite instance. */ private Ignite ignite; - /** Spring context */ + /** Spring context. */ private ApplicationContext springCtx; /** {@inheritDoc} */ @@ -276,7 +277,7 @@ else if (cfg != null) } /** {@inheritDoc} */ - @Override public org.springframework.cache.Cache getCache(String name) { + @Override public Cache getCache(String name) { assert ignite != null; SpringCache cache = caches.get(name); @@ -308,4 +309,4 @@ else if (cfg != null) return new ArrayList<>(caches.keySet()); } -} \ No newline at end of file +} diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/GridSpringCacheManagerSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheManagerSelfTest.java similarity index 78% rename from modules/spring/src/test/java/org/apache/ignite/spring/GridSpringCacheManagerSelfTest.java rename to modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheManagerSelfTest.java index 0293e66ec6268..601401c8b1bae 100644 --- a/modules/spring/src/test/java/org/apache/ignite/spring/GridSpringCacheManagerSelfTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheManagerSelfTest.java @@ -15,11 +15,13 @@ * limitations under the License. */ -package org.apache.ignite.spring; +package org.apache.ignite.cache.spring; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; @@ -40,6 +42,20 @@ public class GridSpringCacheManagerSelfTest extends GridCommonAbstractTest { /** */ private static final String DYNAMIC_CACHE_NAME = "dynamicCache"; + /** */ + private static final Object NULL; + + /** + */ + static { + try { + NULL = U.field(SpringCache.class, "NULL"); + } + catch (IgniteCheckedException e) { + throw new RuntimeException(e); + } + } + /** */ private GridSpringCacheTestService svc; @@ -82,7 +98,7 @@ public class GridSpringCacheManagerSelfTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - BeanFactory factory = new ClassPathXmlApplicationContext("org/apache/ignite/spring/spring-caching.xml"); + BeanFactory factory = new ClassPathXmlApplicationContext("org/apache/ignite/cache/spring/spring-caching.xml"); svc = (GridSpringCacheTestService)factory.getBean("testService"); dynamicSvc = (GridSpringDynamicCacheTestService)factory.getBean("dynamicTestService"); @@ -117,6 +133,25 @@ public void testSimpleKey() throws Exception { assertEquals("value" + i, c.get(i)); } + /** + * @throws Exception If failed. + */ + public void testSimpleKeyNullValue() throws Exception { + for (int i = 0; i < 3; i++) { + assertNull(svc.simpleKeyNullValue(i)); + assertNull(svc.simpleKeyNullValue(i)); + } + + assertEquals(3, svc.called()); + + IgniteCache c = grid().cache(CACHE_NAME); + + assertEquals(3, c.size()); + + for (int i = 0; i < 3; i++) + assertEquals(NULL, c.get(i)); + } + /** * @throws Exception If failed. */ @@ -136,6 +171,25 @@ public void testComplexKey() throws Exception { assertEquals("value" + i + "suffix" + i, c.get(new GridSpringCacheTestKey(i, "suffix" + i))); } + /** + * @throws Exception If failed. + */ + public void testComplexKeyNullValue() throws Exception { + for (int i = 0; i < 3; i++) { + assertNull(svc.complexKeyNullValue(i, "suffix" + i)); + assertNull(svc.complexKeyNullValue(i, "suffix" + i)); + } + + assertEquals(3, svc.called()); + + IgniteCache c = grid().cache(CACHE_NAME); + + assertEquals(3, c.size()); + + for (int i = 0; i < 3; i++) + assertEquals(NULL, c.get(new GridSpringCacheTestKey(i, "suffix" + i))); + } + /** * @throws Exception If failed. */ @@ -157,6 +211,27 @@ public void testSimpleKeyPut() throws Exception { assertEquals(6, svc.called()); } + /** + * @throws Exception If failed. + */ + public void testSimpleKeyPutNullValue() throws Exception { + IgniteCache c = grid().cache(CACHE_NAME); + + for (int i = 0; i < 3; i++) { + assertNull(svc.simpleKeyPutNullValue(i)); + + assertEquals(i + 1, c.size()); + assertEquals(NULL, c.get(i)); + + assertNull(svc.simpleKeyPutNullValue(i)); + + assertEquals(i + 1, c.size()); + assertEquals(NULL, c.get(i)); + } + + assertEquals(6, svc.called()); + } + /** * @throws Exception If failed. */ @@ -178,6 +253,27 @@ public void testComplexKeyPut() throws Exception { assertEquals(6, svc.called()); } + /** + * @throws Exception If failed. + */ + public void testComplexKeyPutNullValue() throws Exception { + IgniteCache c = grid().cache(CACHE_NAME); + + for (int i = 0; i < 3; i++) { + assertNull(svc.complexKeyPutNullValue(i, "suffix" + i)); + + assertEquals(i + 1, c.size()); + assertEquals(NULL, c.get(new GridSpringCacheTestKey(i, "suffix" + i))); + + assertNull(svc.complexKeyPutNullValue(i, "suffix" + i)); + + assertEquals(i + 1, c.size()); + assertEquals(NULL, c.get(new GridSpringCacheTestKey(i, "suffix" + i))); + } + + assertEquals(6, svc.called()); + } + /** * @throws Exception If failed. */ @@ -339,4 +435,4 @@ public void testDynamicCacheEvictAll() throws Exception { assertEquals(0, c.size()); } -} \ No newline at end of file +} diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/GridSpringCacheTestKey.java b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheTestKey.java similarity index 97% rename from modules/spring/src/test/java/org/apache/ignite/spring/GridSpringCacheTestKey.java rename to modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheTestKey.java index c54fa82ce892b..3f551121177d4 100644 --- a/modules/spring/src/test/java/org/apache/ignite/spring/GridSpringCacheTestKey.java +++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheTestKey.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.spring; +package org.apache.ignite.cache.spring; import java.io.Serializable; @@ -58,4 +58,4 @@ public GridSpringCacheTestKey(Integer p1, String p2) { @Override public int hashCode() { return 31 * p1 + p2.hashCode(); } -} \ No newline at end of file +} diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/GridSpringCacheTestKeyGenerator.java b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheTestKeyGenerator.java similarity index 97% rename from modules/spring/src/test/java/org/apache/ignite/spring/GridSpringCacheTestKeyGenerator.java rename to modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheTestKeyGenerator.java index 4585ceb52dd8e..7bab6cbf32a56 100644 --- a/modules/spring/src/test/java/org/apache/ignite/spring/GridSpringCacheTestKeyGenerator.java +++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheTestKeyGenerator.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.spring; +package org.apache.ignite.cache.spring; import java.lang.reflect.Method; import org.springframework.cache.interceptor.KeyGenerator; @@ -37,4 +37,4 @@ public class GridSpringCacheTestKeyGenerator implements KeyGenerator { return new GridSpringCacheTestKey((Integer)params[0], (String)params[1]); } } -} \ No newline at end of file +} diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/GridSpringCacheTestService.java b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheTestService.java similarity index 72% rename from modules/spring/src/test/java/org/apache/ignite/spring/GridSpringCacheTestService.java rename to modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheTestService.java index c528650aa68f8..544997d3548f8 100644 --- a/modules/spring/src/test/java/org/apache/ignite/spring/GridSpringCacheTestService.java +++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheTestService.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.spring; +package org.apache.ignite.cache.spring; import java.util.concurrent.atomic.AtomicInteger; import org.springframework.cache.annotation.CacheEvict; @@ -56,6 +56,19 @@ public String simpleKey(Integer key) { return "value" + key; } + /** + * @param key Key. + * @return Value. + */ + @Cacheable("testCache") + public String simpleKeyNullValue(Integer key) { + assert key != null; + + cnt.incrementAndGet(); + + return null; + } + /** * @param p1 Parameter 1. * @param p2 Parameter 2. @@ -71,6 +84,21 @@ public String complexKey(Integer p1, String p2) { return "value" + p1 + p2; } + /** + * @param p1 Parameter 1. + * @param p2 Parameter 2. + * @return Value. + */ + @Cacheable("testCache") + public String complexKeyNullValue(Integer p1, String p2) { + assert p1 != null; + assert p2 != null; + + cnt.incrementAndGet(); + + return null; + } + /** * @param key Key. * @return Value. @@ -84,6 +112,19 @@ public String simpleKeyPut(Integer key) { return "value" + key + (cnt0 % 2 == 0 ? "even" : "odd"); } + /** + * @param key Key. + * @return Value. + */ + @CachePut("testCache") + public String simpleKeyPutNullValue(Integer key) { + assert key != null; + + cnt.incrementAndGet(); + + return null; + } + /** * @param p1 Parameter 1. * @param p2 Parameter 2. @@ -99,6 +140,21 @@ public String complexKeyPut(Integer p1, String p2) { return "value" + p1 + p2 + (cnt0 % 2 == 0 ? "even" : "odd"); } + /** + * @param p1 Parameter 1. + * @param p2 Parameter 2. + * @return Value. + */ + @CachePut("testCache") + public String complexKeyPutNullValue(Integer p1, String p2) { + assert p1 != null; + assert p2 != null; + + cnt.incrementAndGet(); + + return null; + } + /** * @param key Key. */ @@ -122,4 +178,4 @@ public void complexKeyEvict(Integer p1, String p2) { public void evictAll() { // No-op. } -} \ No newline at end of file +} diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/GridSpringDynamicCacheTestService.java b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringDynamicCacheTestService.java similarity index 98% rename from modules/spring/src/test/java/org/apache/ignite/spring/GridSpringDynamicCacheTestService.java rename to modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringDynamicCacheTestService.java index 510d24d283b33..6584277c51546 100644 --- a/modules/spring/src/test/java/org/apache/ignite/spring/GridSpringDynamicCacheTestService.java +++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringDynamicCacheTestService.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.spring; +package org.apache.ignite.cache.spring; import java.util.concurrent.atomic.AtomicInteger; import org.springframework.cache.annotation.CacheEvict; @@ -82,4 +82,4 @@ public int called() { public void reset() { cnt.set(0); } -} \ No newline at end of file +} diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/SpringCacheManagerContextInjectionTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/spring/SpringCacheManagerContextInjectionTest.java similarity index 97% rename from modules/spring/src/test/java/org/apache/ignite/spring/SpringCacheManagerContextInjectionTest.java rename to modules/spring/src/test/java/org/apache/ignite/cache/spring/SpringCacheManagerContextInjectionTest.java index 1554198d65991..0a898b6a3cc49 100644 --- a/modules/spring/src/test/java/org/apache/ignite/spring/SpringCacheManagerContextInjectionTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/SpringCacheManagerContextInjectionTest.java @@ -15,11 +15,10 @@ * limitations under the License. */ -package org.apache.ignite.spring; +package org.apache.ignite.cache.spring; import org.apache.ignite.Ignite; import org.apache.ignite.TestInjectionLifecycleBean; -import org.apache.ignite.cache.spring.SpringCacheManager; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgnitionEx; import org.apache.ignite.lifecycle.LifecycleBean; diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/spring-caching.xml b/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching.xml similarity index 86% rename from modules/spring/src/test/java/org/apache/ignite/spring/spring-caching.xml rename to modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching.xml index 784cf01caf55f..355f6e0a4f313 100644 --- a/modules/spring/src/test/java/org/apache/ignite/spring/spring-caching.xml +++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching.xml @@ -26,12 +26,12 @@ - + - + - + - gridgain-professional-${ignite.edition}-${project.version} + ${project.artifactId}-${ignite.edition}-${project.version}-bin From 8bf0a48cf21c5b75feb62481ddfca102b4a84fd9 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Wed, 10 Aug 2016 12:07:21 +0300 Subject: [PATCH 017/487] IGNITE-2310 Lock cache partition for affinityRun/affinityCall execution - fix reservable --- .../processors/cache/distributed/dht/GridReservable.java | 5 +---- .../ignite/internal/processors/job/GridJobProcessor.java | 4 ++-- .../ignite/internal/processors/query/GridQueryProcessor.java | 4 ++-- .../IgniteCacheLockPartitionOnAffinityRunAbstractTest.java | 2 +- 4 files changed, 6 insertions(+), 9 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridReservable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridReservable.java index 068c68dd31d03..51f22bc3dec62 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridReservable.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridReservable.java @@ -17,8 +17,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht; -import org.apache.ignite.IgniteCheckedException; - /** * Reservations support. */ @@ -27,9 +25,8 @@ public interface GridReservable { * Reserves. * * @return {@code true} If reserved successfully. - * @throws IgniteCheckedException If failed. */ - public boolean reserve() throws IgniteCheckedException; + public boolean reserve(); /** * Releases. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java index 6a162d377c3f0..8db09367530c6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java @@ -1518,7 +1518,7 @@ public PartitionsReservation(int[] cacheIds, int partId, } /** {@inheritDoc} */ - @Override public boolean reserve() throws IgniteCheckedException { + @Override public boolean reserve() { boolean reserved = false; try { @@ -1568,7 +1568,7 @@ public PartitionsReservation(int[] cacheIds, int partId, } finally { if (checkPartMapping && !cctx.affinity().primary(partId, topVer).id().equals(ctx.localNodeId())) - throw new IgniteCheckedException("Failed partition reservation. " + + throw new IgniteException("Failed partition reservation. " + "Partition is not primary on the node. [partition=" + partId + ", cacheName=" + cctx.name() + ", nodeId=" + ctx.localNodeId() + ", topology=" + topVer + ']'); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index d73b6ca1e8a72..7ed4a2f886741 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -808,7 +808,7 @@ public Iterator> queryLocal( sqlQry, F.asList(params), typeDesc, - idx.backupFilter(null, requestTopVer.get(), null)); + idx.backupFilter(requestTopVer.get(), null)); sendQueryExecutedEvent( sqlQry, @@ -894,7 +894,7 @@ public QueryCursor> queryLocalFields(final GridCacheContext cctx, f Object[] args = qry.getArgs(); final GridQueryFieldsResult res = idx.queryLocalSqlFields(space, sql, F.asList(args), - idx.backupFilter(null, requestTopVer.get(), null), qry.isEnforceJoinOrder()); + idx.backupFilter(requestTopVer.get(), null), qry.isEnforceJoinOrder()); sendQueryExecutedEvent(sql, args); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunAbstractTest.java index 11dc6c53ad8db..4c99421ac5c6f 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunAbstractTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunAbstractTest.java @@ -32,7 +32,7 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED; /** - * Base class of for the tests to validate https://issues.apache.org/jira/browse/IGNITE-2310 + * Base class of the tests to validate https://issues.apache.org/jira/browse/IGNITE-2310 */ public class IgniteCacheLockPartitionOnAffinityRunAbstractTest extends GridCacheAbstractSelfTest { /** Count of affinity run threads. */ From 317a675095d96ce158dd66c1739107a2e18cfc8c Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Wed, 10 Aug 2016 17:44:59 +0700 Subject: [PATCH 018/487] Fixed checks for client mode + daemon mode. --- .../internal/managers/discovery/GridDiscoveryManager.java | 3 ++- .../apache/ignite/internal/visor/util/VisorTaskUtils.java | 8 ++++++++ .../org/apache/ignite/spi/discovery/tcp/ClientImpl.java | 8 ++++++-- 3 files changed, 16 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java index bbf3ebd2a2fc3..2a47ca20d28ec 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java @@ -2225,7 +2225,8 @@ else if (log.isDebugEnabled()) if (log.isInfoEnabled()) log.info("Client node reconnected to topology: " + node); - ackTopology(topVer.topologyVersion(), true); + if (!isLocDaemon) + ackTopology(topVer.topologyVersion(), true); break; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java index 7eebbf1a34158..2721be4ee32ca 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java @@ -1024,4 +1024,12 @@ public static byte[] zipBytes(byte[] input, int initBufSize) throws IOException return bos.toByteArray(); } + + /** + * @param msg Exception message. + * @return {@code true} if node failed to join grid. + */ + public static boolean joinTimedOut(String msg) { + return msg != null && msg.startsWith("Join process timed out."); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 9821134abfbc8..4230a264748a1 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -2018,15 +2018,19 @@ private void updateMetrics(UUID nodeId, Map cacheMetrics, long tstamp) { + boolean isLocDaemon = spi.locNode.isDaemon(); + assert nodeId != null; assert metrics != null; - assert cacheMetrics != null; + assert isLocDaemon || cacheMetrics != null; TcpDiscoveryNode node = nodeId.equals(getLocalNodeId()) ? locNode : rmtNodes.get(nodeId); if (node != null && node.visible()) { node.setMetrics(metrics); - node.setCacheMetrics(cacheMetrics); + + if (!isLocDaemon) + node.setCacheMetrics(cacheMetrics); node.lastUpdateTime(tstamp); From ff3e00caa892a7399622711b620fcb4dcfbbfb56 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Wed, 10 Aug 2016 16:21:52 +0300 Subject: [PATCH 019/487] IGNITE-3272 Fixed "Memory consumption in ContinuousQueryHandler". This close #930. --- .../internal/GridEventConsumeHandler.java | 5 + .../internal/GridMessageListenHandler.java | 5 + .../continuous/CacheContinuousQueryEntry.java | 16 +- .../CacheContinuousQueryHandler.java | 85 +++++--- .../continuous/GridContinuousHandler.java | 5 + .../continuous/GridContinuousProcessor.java | 11 +- .../continuous/GridContinuousQueryBatch.java | 47 +++++ ...teCacheContinuousQueryBackupQueueTest.java | 184 +++++++++++++++++- 8 files changed, 318 insertions(+), 40 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousQueryBatch.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java index 19bf1a7f06b17..b4b1e58b93727 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java @@ -401,6 +401,11 @@ public GridEventConsumeHandler() { // No-op. } + /** {@inheritDoc} */ + @Override public void onNodeLeft() { + // No-op. + } + /** {@inheritDoc} */ @Nullable @Override public Object orderedTopic() { return null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java index 0ac6877aa72ec..2b8041d5a7ab1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java @@ -223,6 +223,11 @@ public GridMessageListenHandler(GridMessageListenHandler orig) { } } + /** {@inheritDoc} */ + @Override public void onNodeLeft() { + // No-op. + } + /** {@inheritDoc} */ @Override public void writeExternal(ObjectOutput out) throws IOException { out.writeBoolean(depEnabled); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java index 74f930a64ff58..366a1e05fa46c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java @@ -199,6 +199,20 @@ void markFiltered() { depInfo = null; } + /** + * @param topVer Topology version. + */ + void topologyVersion(AffinityTopologyVersion topVer) { + this.topVer = topVer; + } + + /** + * @return Size include this event and filtered. + */ + public int size() { + return filteredEvts != null ? filteredEvts.size() + 1 : 1; + } + /** * @return If entry filtered then will return light-weight new entry without values and key * (avoid to huge memory consumption), otherwise {@code this}. @@ -208,7 +222,7 @@ CacheContinuousQueryEntry forBackupQueue() { return this; CacheContinuousQueryEntry e = - new CacheContinuousQueryEntry(cacheId, evtType, null, null, null, keepBinary, part, updateCntr, topVer); + new CacheContinuousQueryEntry(cacheId, null, null, null, null, keepBinary, part, updateCntr, null); e.flags = flags; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 50125690b5d63..7b3b47b5a9dfb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -64,8 +64,8 @@ import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager.JCacheQueryLocalListener; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager.JCacheQueryRemoteFilter; import org.apache.ignite.internal.processors.continuous.GridContinuousBatch; -import org.apache.ignite.internal.processors.continuous.GridContinuousBatchAdapter; import org.apache.ignite.internal.processors.continuous.GridContinuousHandler; +import org.apache.ignite.internal.processors.continuous.GridContinuousQueryBatch; import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryFilter; import org.apache.ignite.internal.util.GridConcurrentSkipListSet; import org.apache.ignite.internal.util.GridLongList; @@ -132,7 +132,7 @@ public class CacheContinuousQueryHandler implements GridContinuousHandler private transient boolean skipPrimaryCheck; /** Backup queue. */ - private transient Collection backupQueue; + private transient volatile Collection backupQueue; /** */ private boolean locCache; @@ -430,33 +430,48 @@ public void keepBinary(boolean keepBinary) { } @Override public void cleanupBackupQueue(Map updateCntrs) { - Iterator it = backupQueue.iterator(); + Collection backupQueue0 = backupQueue; - while (it.hasNext()) { - CacheContinuousQueryEntry backupEntry = it.next(); + if (backupQueue0 != null) { + Iterator it = backupQueue0.iterator(); - Long updateCntr = updateCntrs.get(backupEntry.partition()); + while (it.hasNext()) { + CacheContinuousQueryEntry backupEntry = it.next(); - if (updateCntr != null && backupEntry.updateCounter() <= updateCntr) - it.remove(); + Long updateCntr = updateCntrs.get(backupEntry.partition()); + + if (updateCntr != null && backupEntry.updateCounter() <= updateCntr) + it.remove(); + } } } @Override public void flushBackupQueue(GridKernalContext ctx, AffinityTopologyVersion topVer) { - if (backupQueue.isEmpty()) + Collection backupQueue0 = backupQueue; + + if (backupQueue0 == null) return; try { - GridCacheContext cctx = cacheContext(ctx); + ClusterNode nodeId0 = ctx.discovery().node(nodeId); - for (CacheContinuousQueryEntry e : backupQueue) { - if (!e.isFiltered()) - prepareEntry(cctx, nodeId, e); - } + if (nodeId0 != null) { + GridCacheContext cctx = cacheContext(ctx); - ctx.continuous().addBackupNotification(nodeId, routineId, backupQueue, topic); + for (CacheContinuousQueryEntry e : backupQueue0) { + if (!e.isFiltered()) + prepareEntry(cctx, nodeId, e); + + e.topologyVersion(topVer); + } + + ctx.continuous().addBackupNotification(nodeId, routineId, backupQueue0, topic); + } + else + // Node which start CQ leave topology. Not needed to put data to backup queue. + backupQueue = null; - backupQueue.clear(); + backupQueue0.clear(); } catch (IgniteCheckedException e) { U.error(ctx.log(getClass()), "Failed to send backup event notification to node: " + nodeId, e); @@ -479,9 +494,13 @@ public void keepBinary(boolean keepBinary) { } @Override public void onPartitionEvicted(int part) { - for (Iterator it = backupQueue.iterator(); it.hasNext();) { - if (it.next().partition() == part) - it.remove(); + Collection backupQueue0 = backupQueue; + + if (backupQueue0 != null) { + for (Iterator it = backupQueue0.iterator(); it.hasNext(); ) { + if (it.next().partition() == part) + it.remove(); + } } } @@ -740,7 +759,10 @@ public boolean filter(CacheContinuousQueryEvent evt, boolean primary) { if (!primary && !internal && entry.updateCounter() != -1L /* Skip init query and expire entries */) { entry.markBackup(); - backupQueue.add(entry.forBackupQueue()); + Collection backupQueue0 = backupQueue; + + if (backupQueue0 != null) + backupQueue0.add(entry.forBackupQueue()); } return notify; @@ -765,12 +787,11 @@ private void onEntryUpdate(CacheContinuousQueryEvent evt, boolean notify, boolea if (!locCache) { Collection> evts = handleEvent(ctx, entry); - if (!evts.isEmpty()) { + if (!evts.isEmpty()) locLsnr.onUpdated(evts); - if (!internal && !skipPrimaryCheck) - sendBackupAcknowledge(ackBuf.onAcknowledged(entry), routineId, ctx); - } + if (!internal && !skipPrimaryCheck) + sendBackupAcknowledge(ackBuf.onAcknowledged(entry), routineId, ctx); } else { if (!entry.isFiltered()) @@ -931,7 +952,7 @@ private static class PartitionRecovery { * @param topVer Topology version. * @param initCntr Update counters. */ - public PartitionRecovery(IgniteLogger log, AffinityTopologyVersion topVer, @Nullable Long initCntr) { + PartitionRecovery(IgniteLogger log, AffinityTopologyVersion topVer, @Nullable Long initCntr) { this.log = log; if (initCntr != null) { @@ -1175,6 +1196,14 @@ public CacheContinuousQueryEntry handle(CacheContinuousQueryEntry e) { } } + /** {@inheritDoc} */ + @Override public void onNodeLeft() { + Collection backupQueue0 = backupQueue; + + if (backupQueue0 != null) + backupQueue = null; + } + /** {@inheritDoc} */ @Override public void p2pMarshal(GridKernalContext ctx) throws IgniteCheckedException { assert ctx != null; @@ -1196,7 +1225,7 @@ public CacheContinuousQueryEntry handle(CacheContinuousQueryEntry e) { /** {@inheritDoc} */ @Override public GridContinuousBatch createBatch() { - return new GridContinuousBatchAdapter(); + return new GridContinuousQueryBatch(); } /** {@inheritDoc} */ @@ -1345,7 +1374,9 @@ private static class AcknowledgeBuffer { @SuppressWarnings("unchecked") @Nullable synchronized IgniteBiTuple, Set> onAcknowledged(GridContinuousBatch batch) { - size += batch.size(); + assert batch instanceof GridContinuousQueryBatch; + + size += ((GridContinuousQueryBatch)batch).entriesCount(); Collection entries = (Collection)batch.collect(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java index 46e87af28ec8f..c90746d759e6a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java @@ -115,6 +115,11 @@ public enum RegisterStatus { */ public void onBatchAcknowledged(UUID routineId, GridContinuousBatch batch, GridKernalContext ctx); + /** + * Node which started routine leave topology. + */ + public void onNodeLeft(); + /** * @return Topic for ordered notifications. If {@code null}, notifications * will be sent in non-ordered messages. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java index fce48c4c12535..5f61051b984be 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java @@ -175,8 +175,13 @@ public GridContinuousProcessor(GridKernalContext ctx) { UUID routineId = e.getKey(); RemoteRoutineInfo info = e.getValue(); - if (info.autoUnsubscribe && nodeId.equals(info.nodeId)) - unregisterRemote(routineId); + if (nodeId.equals(info.nodeId)) { + if (info.autoUnsubscribe) + unregisterRemote(routineId); + + if (info.hnd.isQuery()) + info.hnd.onNodeLeft(); + } } for (Map.Entry e : syncMsgFuts.entrySet()) { @@ -865,6 +870,8 @@ public void addNotification(UUID nodeId, try { sendNotification(nodeId, routineId, futId, F.asList(obj), null, msg, null); + + info.hnd.onBatchAcknowledged(routineId, info.add(obj), ctx); } catch (IgniteCheckedException e) { syncMsgFuts.remove(futId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousQueryBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousQueryBatch.java new file mode 100644 index 0000000000000..c5d854b6402b5 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousQueryBatch.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.continuous; + +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryEntry; + +/** + * Continuous routine batch adapter. + */ +public class GridContinuousQueryBatch extends GridContinuousBatchAdapter { + /** Entries size included filtered entries. */ + private final AtomicInteger size = new AtomicInteger(); + + /** {@inheritDoc} */ + @Override public void add(Object obj) { + assert obj != null; + assert obj instanceof CacheContinuousQueryEntry; + + super.add(obj); + + size.addAndGet(((CacheContinuousQueryEntry)obj).size()); + } + + /** + * @return Entries count. + */ + public int entriesCount() { + return size.get(); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryBackupQueueTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryBackupQueueTest.java index aea1954112422..d823409e3f28f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryBackupQueueTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryBackupQueueTest.java @@ -18,18 +18,29 @@ package org.apache.ignite.internal.processors.cache.query.continuous; import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import javax.cache.configuration.Factory; import javax.cache.event.CacheEntryEvent; import javax.cache.event.CacheEntryEventFilter; import javax.cache.event.CacheEntryUpdatedListener; +import org.apache.ignite.Ignite; import org.apache.ignite.cache.query.ContinuousQuery; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.processors.continuous.GridContinuousHandler; +import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; @@ -46,14 +57,27 @@ public class IgniteCacheContinuousQueryBackupQueueTest extends GridCommonAbstrac /** Keys count. */ private static final int KEYS_COUNT = 1024; + /** CQ count. */ + private static final int QUERY_COUNT = 20; + /** Grid count. */ private static final int GRID_COUNT = 2; + /** */ + private static boolean client = false; + + /** */ + private static String CACHE_NAME = "test-cache"; + + /** */ + private static final int BACKUP_ACK_THRESHOLD = 100; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); - CacheConfiguration ccfg = new CacheConfiguration(); + CacheConfiguration ccfg = new CacheConfiguration(CACHE_NAME); + ccfg.setCacheMode(PARTITIONED); ccfg.setAtomicityMode(ATOMIC); ccfg.setWriteSynchronizationMode(FULL_SYNC); @@ -61,62 +85,202 @@ public class IgniteCacheContinuousQueryBackupQueueTest extends GridCommonAbstrac cfg.setCacheConfiguration(ccfg); + cfg.setClientMode(client); + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); return cfg; } + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + startGridsMultiThreaded(GRID_COUNT); + } + /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { super.afterTest(); stopAllGrids(); + + client = false; } /** {@inheritDoc} */ @Override protected long getTestTimeout() { - return TimeUnit.MINUTES.toMillis(2); + return TimeUnit.MINUTES.toMillis(10); } /** * @throws Exception If failed. */ public void testBackupQueue() throws Exception { - startGridsMultiThreaded(GRID_COUNT); - final CacheEventListener lsnr = new CacheEventListener(); ContinuousQuery qry = new ContinuousQuery<>(); qry.setLocalListener(lsnr); - qry.setRemoteFilterFactory(new FilterFactory()); + qry.setRemoteFilterFactory(new AlwaysFalseFilterFactory()); - try (QueryCursor ignore = grid(0).cache(null).query(qry)) { + try (QueryCursor ignore = grid(0).cache(CACHE_NAME).query(qry)) { for (int i = 0; i < KEYS_COUNT; i++) { log.info("Put key: " + i); for (int j = 0; j < 100; j++) - grid(i % GRID_COUNT).cache(null).put(i, new byte[1024 * 50]); + grid(j % GRID_COUNT).cache(CACHE_NAME).put(i, new byte[1024 * 50]); } log.info("Finish."); } } + /** + * @throws Exception If failed. + */ + public void testManyQueryBackupQueue() throws Exception { + List qryCursors = new ArrayList<>(); + + for (int i = 0; i < QUERY_COUNT; i++) { + ContinuousQuery qry = new ContinuousQuery<>(); + + qry.setLocalListener(new CacheEventListener()); + qry.setRemoteFilterFactory(new AlwaysFalseFilterFactory()); + + qryCursors.add(grid(0).cache(CACHE_NAME).query(qry)); + } + + for (int i = 0; i < KEYS_COUNT; i++) { + log.info("Put key: " + i); + + for (int j = 0; j < 150; j++) + grid(ThreadLocalRandom.current().nextInt(GRID_COUNT)).cache(CACHE_NAME).put(i, new byte[1024 * 50]); + } + + int size = backupQueueSize(); + + assertTrue(size > 0); + assertTrue(size <= BACKUP_ACK_THRESHOLD * QUERY_COUNT * /* partition count */1024); + + for (QueryCursor qry : qryCursors) + qry.close(); + } + + + /** + * @throws Exception If failed. + */ + public void testBackupQueueAutoUnsubscribeFalse() throws Exception { + try { + client = true; + + Ignite client = startGrid(GRID_COUNT); + + awaitPartitionMapExchange(); + + List qryCursors = new ArrayList<>(); + + for (int i = 0; i < QUERY_COUNT; i++) { + ContinuousQuery qry = new ContinuousQuery<>(); + + qry.setLocalListener(new CacheEventListener()); + qry.setRemoteFilterFactory(new AlwaysFalseFilterFactory()); + qry.setAutoUnsubscribe(false); + + qryCursors.add(client.cache(CACHE_NAME).query(qry)); + } + + for (int i = 0; i < KEYS_COUNT; i++) { + log.info("Put key: " + i); + + grid(i % GRID_COUNT).cache(CACHE_NAME).put(i, new byte[1024 * 50]); + } + + int size = backupQueueSize(); + + assertTrue(size > 0); + assertTrue(size <= BACKUP_ACK_THRESHOLD * QUERY_COUNT * /* partition count */1024); + + stopGrid(GRID_COUNT); + + awaitPartitionMapExchange(); + + for (int i = 0; i < KEYS_COUNT; i++) { + log.info("Put key: " + i); + + grid(i % GRID_COUNT).cache(CACHE_NAME).put(i, new byte[1024 * 50]); + } + + size = backupQueueSize(); + + assertEquals(-1, size); + } + finally { + stopGrid(GRID_COUNT); + } + } + + /** + * @return Backup queue size or {@code -1} if backup queue doesn't exist. + */ + private int backupQueueSize() { + int backupQueueSize = -1; + + for (int i = 0; i < GRID_COUNT; i++) { + for (Collection backQueue : backupQueues(grid(i))) + backupQueueSize += backQueue.size(); + } + + return backupQueueSize; + } + + /** + * @param ignite Ignite. + * @return Backup queue for test query. + */ + private List> backupQueues(Ignite ignite) { + GridContinuousProcessor proc = ((IgniteKernal)ignite).context().continuous(); + + Map infos = new HashMap<>(); + + Map rmtInfos = GridTestUtils.getFieldValue(proc, "rmtInfos"); + Map locInfos = GridTestUtils.getFieldValue(proc, "locInfos"); + + infos.putAll(rmtInfos); + infos.putAll(locInfos); + + List> backupQueues = new ArrayList<>(); + + for (Object info : infos.values()) { + GridContinuousHandler hnd = GridTestUtils.getFieldValue(info, "hnd"); + + if (hnd.isQuery() && hnd.cacheName().equals(CACHE_NAME)) { + Collection q = GridTestUtils.getFieldValue(hnd, + CacheContinuousQueryHandler.class, "backupQueue"); + + if (q != null) + backupQueues.add(q); + } + } + + return backupQueues; + } + /** * */ - private static class FilterFactory implements Factory> { + private static class AlwaysFalseFilterFactory implements Factory> { /** {@inheritDoc} */ @Override public CacheEntryEventFilter create() { - return new CacheEventFilter(); + return new AlwaysFalseFilter(); } } /** * */ - private static class CacheEventFilter implements CacheEntryEventFilter, Serializable { + private static class AlwaysFalseFilter implements CacheEntryEventFilter, Serializable { /** {@inheritDoc} */ @Override public boolean evaluate(CacheEntryEvent evt) { return false; From 1d0cbb45cd61c5c8e6ec926d7e629eb94111b32f Mon Sep 17 00:00:00 2001 From: vd-pyatkov Date: Thu, 11 Aug 2016 08:43:50 +0300 Subject: [PATCH 020/487] IGNITE-3618: Client can not load data after server restarts. This closes #941. --- .../ignite/internal/binary/BinaryContext.java | 7 + .../CacheObjectBinaryProcessorImpl.java | 19 ++ ...lientReconnectAfterClusterRestartTest.java | 225 ++++++++++++++++++ .../IgniteCacheWithIndexingTestSuite.java | 5 +- 4 files changed, 255 insertions(+), 1 deletion(-) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ClientReconnectAfterClusterRestartTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java index d78c126090c2a..a603894cb94c1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java @@ -1251,6 +1251,13 @@ public BinarySchemaRegistry schemaRegistry(int typeId) { return reg; } + /** + * Unregister all binary schemas. + */ + public void unregisterBinarySchemas() { + schemas = null; + } + /** * Returns instance of {@link OptimizedMarshaller}. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java index 6d980a866a751..03378743c2ee3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java @@ -50,6 +50,7 @@ import org.apache.ignite.cluster.ClusterTopologyException; import org.apache.ignite.configuration.BinaryConfiguration; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.events.Event; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.binary.BinaryContext; @@ -67,6 +68,7 @@ import org.apache.ignite.internal.binary.streams.BinaryInputStream; import org.apache.ignite.internal.binary.streams.BinaryOffheapInputStream; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; import org.apache.ignite.internal.processors.cache.CacheEntryPredicateAdapter; @@ -103,6 +105,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK; import static org.apache.ignite.IgniteSystemProperties.getBoolean; +import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED; /** * Binary processor implementation. @@ -146,6 +149,13 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm @GridToStringExclude private IgniteBinary binaries; + /** Listener removes all registred binary schemas after the local client reconnected. */ + private final GridLocalEventListener clientDisconLsnr = new GridLocalEventListener() { + @Override public void onEvent(Event evt) { + binaryContext().unregisterBinarySchemas(); + } + }; + /** Metadata updates collected before metadata cache is initialized. */ private final Map metaBuf = new ConcurrentHashMap<>(); @@ -165,6 +175,9 @@ public CacheObjectBinaryProcessorImpl(GridKernalContext ctx) { /** {@inheritDoc} */ @Override public void start() throws IgniteCheckedException { if (marsh instanceof BinaryMarshaller) { + if (ctx.clientNode()) + ctx.event().addLocalEventListener(clientDisconLsnr, EVT_CLIENT_NODE_DISCONNECTED); + BinaryMetadataHandler metaHnd = new BinaryMetadataHandler() { @Override public void addMeta(int typeId, BinaryType newMeta) throws BinaryObjectException { assert newMeta != null; @@ -251,6 +264,12 @@ public CacheObjectBinaryProcessorImpl(GridKernalContext ctx) { } } + /** {@inheritDoc} */ + @Override public void stop(boolean cancel) { + if (ctx.clientNode()) + ctx.event().removeLocalEventListener(clientDisconLsnr); + } + /** {@inheritDoc} */ @Override public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteCheckedException { if (clientNode && !ctx.isDaemon()) { diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ClientReconnectAfterClusterRestartTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ClientReconnectAfterClusterRestartTest.java new file mode 100644 index 0000000000000..b31447c9769d9 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ClientReconnectAfterClusterRestartTest.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.binary.BinaryObjectBuilder; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.QueryIndex; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.Event; +import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.binary.BinaryMarshaller; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.NotNull; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; + +/** + */ +public class ClientReconnectAfterClusterRestartTest extends GridCommonAbstractTest { + /** Client id. */ + public static final int CLIENT_ID = 1; + + /** Cache params. */ + public static final String CACHE_PARAMS = "PPRB_PARAMS"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setMarshaller(new BinaryMarshaller()); + cfg.setIncludeEventTypes(EventType.EVTS_CACHE); + + if (getTestGridName(CLIENT_ID).equals(gridName)) + cfg.setClientMode(true); + else { + CacheConfiguration ccfg = getCacheConfiguration(); + + cfg.setCacheConfiguration(ccfg); + } + + return cfg; + } + + /** + * @return CacheConfiguration Cache configuration. + */ + @NotNull private CacheConfiguration getCacheConfiguration() { + CacheConfiguration ccfg = defaultCacheConfiguration(); + + ccfg.setName(CACHE_PARAMS); + ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + ccfg.setCacheMode(CacheMode.PARTITIONED); + ccfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED); + + List queryEntities = new ArrayList<>(); + + QueryEntity entity = new QueryEntity(); + + entity.setValueType("Params"); + entity.setKeyType("java.lang.Long"); + + LinkedHashMap fields = new LinkedHashMap<>(); + + fields.put("ID", "java.lang.Long" ); + fields.put("PARTITIONID", "java.lang.Long"); + fields.put("CLIENTID", "java.lang.Long"); + fields.put("PARAMETRCODE", "java.lang.Long"); + fields.put("PARAMETRVALUE", "java.lang.Object"); + fields.put("PARENTID", "java.lang.Long"); + + entity.setFields(fields); + + List indexes = new ArrayList<>(); + + indexes.add(new QueryIndex("CLIENTID")); + indexes.add(new QueryIndex("ID")); + indexes.add(new QueryIndex("PARENTID")); + + entity.setIndexes(indexes); + + queryEntities.add(entity); + + ccfg.setQueryEntities(queryEntities); + return ccfg; + } + + /** */ + public void testReconnectClient() throws Exception { + try { + Ignite igniteSrv = startGrid(0); + + Ignite client = startGrid(1); + + checkTopology(2); + + client.events().localListen(new IgnitePredicate() { + + @Override public boolean apply(Event event) { + switch (event.type()) { + case EventType.EVT_CLIENT_NODE_DISCONNECTED: + info("Client disconnected"); + + break; + case EventType.EVT_CLIENT_NODE_RECONNECTED: + info("Client reconnected"); + } + + return true; + } + }, EventType.EVT_CLIENT_NODE_DISCONNECTED, EventType.EVT_CLIENT_NODE_RECONNECTED); + + IgniteDataStreamer streamer = client.dataStreamer(CACHE_PARAMS); + + streamer.allowOverwrite(true); + streamer.keepBinary(true); + streamer.perNodeBufferSize(10000); + streamer.perNodeParallelOperations(100); + + BinaryObjectBuilder builder = client.binary().builder("PARAMS"); + + builder.setField("ID", 1L); + builder.setField("PARTITIONID", 1L); + builder.setField("CLIENTID", 1L); + builder.setField("PARAMETRCODE", 1L); + builder.setField("PARAMETRVALUE", "Test value"); + builder.setField("PARENTID", 1L); + BinaryObject obj = builder.build(); + + streamer.addData(1L, obj); + streamer.flush(); + + stopAllServers(false); + + Thread.sleep(2_000); + + igniteSrv = startGrid(0); + + Thread.sleep(2_000); + + checkTopology(2); + + info("Pre-insert"); + + streamer = client.dataStreamer("PPRB_PARAMS"); + streamer.allowOverwrite(true); + streamer.keepBinary(true); + streamer.perNodeBufferSize(10000); + streamer.perNodeParallelOperations(100); + + IgniteCache cache = client.getOrCreateCache(CACHE_PARAMS).withKeepBinary(); + + builder = client.binary().builder("PARAMS"); + builder.setField("ID", 2L); + builder.setField("PARTITIONID", 1L); + builder.setField("CLIENTID", 1L); + builder.setField("PARAMETRCODE", 1L); + builder.setField("PARAMETRVALUE", "Test value"); + builder.setField("PARENTID", 1L); + obj = builder.build(); + + //streamer.addData(2L, obj); + cache.put(2L, obj); + + builder = client.binary().builder("PARAMS"); + builder.setField("ID", 3L); + builder.setField("PARTITIONID", 1L); + builder.setField("CLIENTID", 1L); + builder.setField("PARAMETRCODE", 1L); + builder.setField("PARAMETRVALUE", "Test value"); + builder.setField("PARENTID", 1L); + obj = builder.build(); + + //streamer.addData(3L, obj); + cache.put(3L, obj); + + builder = client.binary().builder("PARAMS"); + builder.setField("ID", 4L); + builder.setField("PARTITIONID", 1L); + builder.setField("CLIENTID", 1L); + builder.setField("PARAMETRCODE", 1L); + builder.setField("PARAMETRVALUE", "Test value"); + builder.setField("PARENTID", 1L); + obj = builder.build(); + + cache.put(4L, obj); + + info("Post-insert"); + + obj = cache.get(4L); + + assertNotNull(obj); + + info("End"); + } + finally { + stopAllGrids(); + } + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java index 96e8551c4170d..4528b304c3ef2 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java @@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.cache.CacheIndexStreamerTest; import org.apache.ignite.internal.processors.cache.CacheOperationsWithExpirationTest; import org.apache.ignite.internal.processors.cache.CacheRandomOperationsMultithreadedTest; +import org.apache.ignite.internal.processors.cache.ClientReconnectAfterClusterRestartTest; import org.apache.ignite.internal.processors.cache.GridCacheOffHeapAndSwapSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheOffHeapSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheOffheapIndexEntryEvictTest; @@ -79,6 +80,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheOperationsWithExpirationTest.class); suite.addTestSuite(CacheBinaryKeyConcurrentQueryTest.class); + suite.addTestSuite(ClientReconnectAfterClusterRestartTest.class); + return suite; } -} \ No newline at end of file +} From 1139a9f76b5d37073261d729a15b1fbec674d48d Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Thu, 11 Aug 2016 08:47:48 +0300 Subject: [PATCH 021/487] Added missing license. --- ...eLockPartitionOnAffinityRunAbstractTest.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunAbstractTest.java index 11dc6c53ad8db..f3cef96f20f94 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunAbstractTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunAbstractTest.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.ignite.internal.processors.cache; import java.io.Serializable; From 0b4ffdbcce63e5ce53572f71af967cff300d5670 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Sun, 14 Aug 2016 18:18:40 +0300 Subject: [PATCH 022/487] IGNITE-2852: Fixed TreeMap and TreeSet serialization. --- .../apache/ignite/IgniteSystemProperties.java | 10 + .../binary/BinaryClassDescriptor.java | 43 ++- .../ignite/internal/binary/BinaryContext.java | 14 +- .../binary/BinaryMethodWriteReplacer.java | 59 +++ .../ignite/internal/binary/BinaryTreeMap.java | 96 +++++ .../binary/BinaryTreeMapWriteReplacer.java | 34 ++ .../ignite/internal/binary/BinaryTreeSet.java | 93 +++++ .../binary/BinaryTreeSetWriteReplacer.java | 34 ++ .../ignite/internal/binary/BinaryUtils.java | 37 +- .../internal/binary/BinaryWriteReplacer.java | 33 ++ .../internal/binary/BinaryWriterExImpl.java | 35 +- .../internal/binary/BinaryTreeSelfTest.java | 341 ++++++++++++++++++ .../IgniteBinaryObjectsTestSuite.java | 2 + 13 files changed, 790 insertions(+), 41 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMethodWriteReplacer.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMap.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMapWriteReplacer.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSet.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSetWriteReplacer.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteReplacer.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryTreeSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 0c22c9da89d32..7c428a6c6e8ac 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -451,6 +451,16 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_SERVICES_COMPATIBILITY_MODE = "IGNITE_SERVICES_COMPATIBILITY_MODE"; + /** + * When set to {@code true} tree-based data structures - {@code TreeMap} and {@code TreeSet} - will not be + * wrapped into special holders introduced to overcome serialization issue caused by missing {@code Comparable} + * interface on {@code BinaryObject}. + *

      + * @deprecated Should be removed in Apache Ignite 2.0. + */ + @Deprecated + public static final String IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES = "IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java index d2d715bd3cd7b..083057ddd5eea 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java @@ -89,8 +89,8 @@ public class BinaryClassDescriptor { /** */ private final BinaryFieldAccessor[] fields; - /** */ - private final Method writeReplaceMtd; + /** Write replacer. */ + private final BinaryWriteReplacer writeReplacer; /** */ private final Method readResolveMtd; @@ -147,7 +147,7 @@ public class BinaryClassDescriptor { initialSerializer = serializer; - // If serializer is not defined at this point, then we have to user OptimizedMarshaller. + // If serializer is not defined at this point, then we have to use OptimizedMarshaller. useOptMarshaller = serializer == null; // Reset reflective serializer so that we rely on existing reflection-based serialization. @@ -298,11 +298,8 @@ else if (useOptMarshaller) schemaBuilder.addField(fieldId); - if (metaDataEnabled) { - assert stableFieldsMeta != null; - + if (metaDataEnabled) stableFieldsMeta.put(name, fieldInfo.mode().typeId()); - } } } } @@ -320,14 +317,24 @@ else if (useOptMarshaller) throw new BinaryObjectException("Invalid mode: " + mode); } + BinaryWriteReplacer writeReplacer0 = BinaryUtils.writeReplacer(cls); + + Method writeReplaceMthd; + if (mode == BinaryWriteMode.BINARY || mode == BinaryWriteMode.OBJECT) { readResolveMtd = U.findNonPublicMethod(cls, "readResolve"); - writeReplaceMtd = U.findNonPublicMethod(cls, "writeReplace"); + + writeReplaceMthd = U.findNonPublicMethod(cls, "writeReplace"); } else { readResolveMtd = null; - writeReplaceMtd = null; + writeReplaceMthd = null; } + + if (writeReplaceMthd != null && writeReplacer0 == null) + writeReplacer0 = new BinaryMethodWriteReplacer(writeReplaceMthd); + + writeReplacer = writeReplacer0; } /** @@ -469,10 +476,22 @@ public boolean excluded() { } /** - * @return binaryWriteReplace() method + * @return {@code True} if write-replace should be performed for class. */ - @Nullable Method getWriteReplaceMethod() { - return writeReplaceMtd; + public boolean isWriteReplace() { + return writeReplacer != null; + } + + /** + * Perform write replace. + * + * @param obj Original object. + * @return Replaced object. + */ + public Object writeReplace(Object obj) { + assert isWriteReplace(); + + return writeReplacer.replace(obj); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java index a603894cb94c1..8517acf972f00 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java @@ -105,6 +105,8 @@ import java.util.LinkedList; import java.util.Map; import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.ConcurrentMap; import java.util.jar.JarEntry; @@ -182,6 +184,11 @@ public class BinaryContext { sysClss.add(GridClosureProcessor.C4V2.class.getName()); sysClss.add(GridClosureProcessor.C4MLAV2.class.getName()); + if (BinaryUtils.wrapTrees()) { + sysClss.add(TreeMap.class.getName()); + sysClss.add(TreeSet.class.getName()); + } + BINARYLIZABLE_SYS_CLSS = Collections.unmodifiableSet(sysClss); } @@ -332,11 +339,16 @@ public BinaryMarshaller marshaller() { * @param cls Class. * @return {@code True} if must be deserialized. */ + @SuppressWarnings("SimplifiableIfStatement") public boolean mustDeserialize(Class cls) { BinaryClassDescriptor desc = descByCls.get(cls); - if (desc == null) + if (desc == null) { + if (BinaryUtils.wrapTrees() && (cls == TreeMap.class || cls == TreeSet.class)) + return false; + return marshCtx.isSystemType(cls.getName()) || serializerForClass(cls) == null; + } else return desc.useOptimizedMarshaller(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMethodWriteReplacer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMethodWriteReplacer.java new file mode 100644 index 0000000000000..783048ed2ed2f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMethodWriteReplacer.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.binary; + +import org.apache.ignite.binary.BinaryObjectException; +import org.jetbrains.annotations.Nullable; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; + +/** + * Write replacer based on method invocation. + */ +public class BinaryMethodWriteReplacer implements BinaryWriteReplacer { + /** Method. */ + private final Method mthd; + + /** + * Constructor. + * + * @param mthd Method. + */ + public BinaryMethodWriteReplacer(Method mthd) { + assert mthd != null; + + this.mthd = mthd; + } + + /** {@inheritDoc} */ + @Nullable @Override public Object replace(Object target) { + try { + return mthd.invoke(target); + } + catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + catch (InvocationTargetException e) { + if (e.getTargetException() instanceof BinaryObjectException) + throw (BinaryObjectException)e.getTargetException(); + + throw new BinaryObjectException("Failed to execute writeReplace() method on " + target, e); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMap.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMap.java new file mode 100644 index 0000000000000..6a7cf9b3db50d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMap.java @@ -0,0 +1,96 @@ +/* + * 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.binary; + +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryRawReader; +import org.apache.ignite.binary.BinaryRawWriter; +import org.apache.ignite.binary.BinaryReader; +import org.apache.ignite.binary.BinaryWriter; +import org.apache.ignite.binary.Binarylizable; + +import java.io.ObjectStreamException; +import java.util.Comparator; +import java.util.Map; +import java.util.TreeMap; + +/** + * Binary {@link TreeMap} wrapper. + */ +public class BinaryTreeMap implements Binarylizable { + /** Original map. */ + private TreeMap map; + + /** + * Default constructor. + */ + public BinaryTreeMap() { + // No-op. + } + + /** + * Constructor. + * + * @param map Original map. + */ + public BinaryTreeMap(TreeMap map) { + this.map = map; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException { + BinaryRawWriter rawWriter = writer.rawWriter(); + + rawWriter.writeObject(map.comparator()); + + int size = map.size(); + + rawWriter.writeInt(size); + + for (Map.Entry entry : ((TreeMap)map).entrySet()) { + rawWriter.writeObject(entry.getKey()); + rawWriter.writeObject(entry.getValue()); + } + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public void readBinary(BinaryReader reader) throws BinaryObjectException { + BinaryRawReader rawReader = reader.rawReader(); + + Comparator comp = rawReader.readObject(); + + map = comp == null ? new TreeMap() : new TreeMap(comp); + + int size = rawReader.readInt(); + + for (int i = 0; i < size; i++) + map.put(rawReader.readObject(), rawReader.readObject()); + } + + /** + * Reconstructs object on unmarshalling. + * + * @return Reconstructed object. + * @throws ObjectStreamException Thrown in case of unmarshalling error. + */ + protected Object readResolve() throws ObjectStreamException { + return map; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMapWriteReplacer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMapWriteReplacer.java new file mode 100644 index 0000000000000..049db8eb2695c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMapWriteReplacer.java @@ -0,0 +1,34 @@ +/* + * 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.binary; + +import org.jetbrains.annotations.Nullable; + +import java.util.TreeMap; + +/** + * Binary {@link TreeMap} write replacer. + */ +public class BinaryTreeMapWriteReplacer implements BinaryWriteReplacer { + /** {@inheritDoc} */ + @Nullable @Override public Object replace(Object target) { + assert target instanceof TreeMap; + + return new BinaryTreeMap((TreeMap)target); + } +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSet.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSet.java new file mode 100644 index 0000000000000..2b01528848efb --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSet.java @@ -0,0 +1,93 @@ +/* + * 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.binary; + +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryRawReader; +import org.apache.ignite.binary.BinaryRawWriter; +import org.apache.ignite.binary.BinaryReader; +import org.apache.ignite.binary.BinaryWriter; +import org.apache.ignite.binary.Binarylizable; + +import java.io.ObjectStreamException; +import java.util.Comparator; +import java.util.TreeSet; + +/** + * Binary {@link TreeSet} wrapper. + */ +public class BinaryTreeSet implements Binarylizable { + /** Original set. */ + private TreeSet set; + + /** + * Default constructor. + */ + public BinaryTreeSet() { + // No-op. + } + + /** + * Constructor. + * + * @param set Original set. + */ + public BinaryTreeSet(TreeSet set) { + this.set = set; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException { + BinaryRawWriter rawWriter = writer.rawWriter(); + + rawWriter.writeObject(set.comparator()); + + int size = set.size(); + + rawWriter.writeInt(size); + + for (Object val : set) + rawWriter.writeObject(val); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public void readBinary(BinaryReader reader) throws BinaryObjectException { + BinaryRawReader rawReader = reader.rawReader(); + + Comparator comp = rawReader.readObject(); + + set = comp == null ? new TreeSet() : new TreeSet(comp); + + int size = rawReader.readInt(); + + for (int i = 0; i < size; i++) + set.add(rawReader.readObject()); + } + + /** + * Reconstructs object on unmarshalling. + * + * @return Reconstructed object. + * @throws ObjectStreamException Thrown in case of unmarshalling error. + */ + protected Object readResolve() throws ObjectStreamException { + return set; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSetWriteReplacer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSetWriteReplacer.java new file mode 100644 index 0000000000000..435077727c613 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeSetWriteReplacer.java @@ -0,0 +1,34 @@ +/* + * 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.binary; + +import org.jetbrains.annotations.Nullable; + +import java.util.TreeSet; + +/** + * Binary {@link TreeSet} write replacer. + */ +public class BinaryTreeSetWriteReplacer implements BinaryWriteReplacer { + /** {@inheritDoc} */ + @Nullable @Override public Object replace(Object target) { + assert target instanceof TreeSet; + + return new BinaryTreeSet((TreeSet)target); + } +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java index 76e5b31ac8198..b5834a5254425 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java @@ -82,6 +82,9 @@ public class BinaryUtils { public static final boolean USE_STR_SERIALIZATION_VER_2 = IgniteSystemProperties.getBoolean( IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2, false); + /** Map from class to associated write replacer. */ + public static final Map CLS_TO_WRITE_REPLACER = new HashMap<>(); + /** {@code true} if serialized value of this type cannot contain references to objects. */ private static final boolean[] PLAIN_TYPE_FLAG = new boolean[102]; @@ -118,6 +121,10 @@ public class BinaryUtils { /** Field ID length. */ public static final int FIELD_ID_LEN = 4; + /** Whether to skip TreeMap/TreeSet wrapping. */ + public static final boolean WRAP_TREES = + !IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES); + /** Field type names. */ private static final String[] FIELD_TYPE_NAMES; @@ -244,6 +251,11 @@ public class BinaryUtils { FIELD_TYPE_NAMES[GridBinaryMarshaller.TIMESTAMP_ARR] = "Timestamp[]"; FIELD_TYPE_NAMES[GridBinaryMarshaller.OBJ_ARR] = "Object[]"; FIELD_TYPE_NAMES[GridBinaryMarshaller.ENUM_ARR] = "Enum[]"; + + if (wrapTrees()) { + CLS_TO_WRITE_REPLACER.put(TreeMap.class, new BinaryTreeMapWriteReplacer()); + CLS_TO_WRITE_REPLACER.put(TreeSet.class, new BinaryTreeSetWriteReplacer()); + } } /** @@ -583,6 +595,13 @@ public static boolean isBinaryType(Class cls) { BINARY_CLS.contains(cls); } + /** + * @return Whether tree structures should be wrapped. + */ + public static boolean wrapTrees() { + return WRAP_TREES; + } + /** * @param map Map to check. * @return {@code True} if this map type is supported. @@ -592,7 +611,7 @@ public static boolean knownMap(Object map) { return cls == HashMap.class || cls == LinkedHashMap.class || - cls == TreeMap.class || + (!wrapTrees() && cls == TreeMap.class) || cls == ConcurrentHashMap8.class || cls == ConcurrentHashMap.class; } @@ -611,7 +630,7 @@ public static Map newKnownMap(Object map) { return U.newHashMap(((Map)map).size()); else if (cls == LinkedHashMap.class) return U.newLinkedHashMap(((Map)map).size()); - else if (cls == TreeMap.class) + else if (!wrapTrees() && cls == TreeMap.class) return new TreeMap<>(((TreeMap)map).comparator()); else if (cls == ConcurrentHashMap8.class) return new ConcurrentHashMap8<>(U.capacity(((Map)map).size())); @@ -650,7 +669,7 @@ public static boolean knownCollection(Object col) { return cls == HashSet.class || cls == LinkedHashSet.class || - cls == TreeSet.class || + (!wrapTrees() && cls == TreeSet.class) || cls == ConcurrentSkipListSet.class || cls == ArrayList.class || cls == LinkedList.class; @@ -686,7 +705,7 @@ public static Collection newKnownCollection(Object col) { return U.newHashSet(((Collection)col).size()); else if (cls == LinkedHashSet.class) return U.newLinkedHashSet(((Collection)col).size()); - else if (cls == TreeSet.class) + else if (!wrapTrees() && cls == TreeSet.class) return new TreeSet<>(((TreeSet)col).comparator()); else if (cls == ConcurrentSkipListSet.class) return new ConcurrentSkipListSet<>(((ConcurrentSkipListSet)col).comparator()); @@ -2213,6 +2232,16 @@ public static BinaryType type(BinaryContext ctx, BinaryObjectEx obj) { return ctx.metadata(obj.typeId()); } + /** + * Get predefined write-replacer associated with class. + * + * @param cls Class. + * @return Write replacer. + */ + public static BinaryWriteReplacer writeReplacer(Class cls) { + return cls != null ? CLS_TO_WRITE_REPLACER.get(cls) : null; + } + /** * Enum type. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteReplacer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteReplacer.java new file mode 100644 index 0000000000000..9376c3703066b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteReplacer.java @@ -0,0 +1,33 @@ +/* + * 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.binary; + +import org.jetbrains.annotations.Nullable; + +/** + * Interface to perform write replace. + */ +public interface BinaryWriteReplacer { + /** + * Perform replace. + * + * @param target Original object. + * @return Replaced object. + */ + @Nullable public Object replace(Object target); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java index 30710f43d20a2..94504829a32cd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java @@ -17,10 +17,18 @@ package org.apache.ignite.internal.binary; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryRawWriter; +import org.apache.ignite.binary.BinaryWriter; +import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream; +import org.apache.ignite.internal.binary.streams.BinaryOutputStream; +import org.apache.ignite.internal.util.typedef.internal.A; +import org.jetbrains.annotations.Nullable; + import java.io.IOException; import java.io.ObjectOutput; import java.lang.reflect.InvocationHandler; -import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Proxy; import java.math.BigDecimal; import java.math.BigInteger; @@ -29,14 +37,6 @@ import java.util.Date; import java.util.Map; import java.util.UUID; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.BinaryRawWriter; -import org.apache.ignite.binary.BinaryWriter; -import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream; -import org.apache.ignite.internal.binary.streams.BinaryOutputStream; -import org.apache.ignite.internal.util.typedef.internal.A; -import org.jetbrains.annotations.Nullable; import static java.nio.charset.StandardCharsets.UTF_8; @@ -170,21 +170,8 @@ void marshal(Object obj, boolean enableReplace) throws BinaryObjectException { return; } - if (enableReplace && desc.getWriteReplaceMethod() != null) { - Object replacedObj; - - try { - replacedObj = desc.getWriteReplaceMethod().invoke(obj); - } - catch (IllegalAccessException e) { - throw new RuntimeException(e); - } - catch (InvocationTargetException e) { - if (e.getTargetException() instanceof BinaryObjectException) - throw (BinaryObjectException)e.getTargetException(); - - throw new BinaryObjectException("Failed to execute writeReplace() method on " + obj, e); - } + if (enableReplace && desc.isWriteReplace()) { + Object replacedObj = desc.writeReplace(obj); if (replacedObj == null) { out.writeByte(GridBinaryMarshaller.NULL); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryTreeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryTreeSelfTest.java new file mode 100644 index 0000000000000..d57b34de7eab8 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryTreeSelfTest.java @@ -0,0 +1,341 @@ +/* + * 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.binary; + +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.NotNull; + +import java.util.Collections; +import java.util.Comparator; +import java.util.TreeMap; +import java.util.TreeSet; + +/** + * Tests for TreeMap and TreeSet structures. + */ +public class BinaryTreeSelfTest extends GridCommonAbstractTest { + /** Data structure size. */ + private static final int SIZE = 100; + + /** Node name: server. */ + private static final String NODE_SRV = "srv"; + + /** Node name: client. */ + private static final String NODE_CLI = "cli"; + + /** Key to be used for cache operations. */ + private static final int KEY = 1; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + Ignition.start(configuration(NODE_SRV, false)); + Ignition.start(configuration(NODE_CLI, true)); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + G.stop(NODE_CLI, true); + G.stop(NODE_SRV, true); + } + + /** + * Test {@code TreeMap} data structure. + * + * @throws Exception If failed. + */ + public void testTreeMapRegularNoComparator() throws Exception { + checkTreeMap(false, false); + } + + /** + * Test {@code TreeMap} data structure with comparator. + * + * @throws Exception If failed. + */ + public void testTreeMapRegularComparator() throws Exception { + checkTreeMap(false, true); + } + + /** + * Test {@code TreeMap} data structure with binary mode. + * + * @throws Exception If failed. + */ + public void testTreeMapBinaryNoComparator() throws Exception { + checkTreeMap(true, false); + } + + /** + * Test {@code TreeMap} data structure with binary mode and comparator. + * + * @throws Exception If failed. + */ + public void testTreeMapBinaryComparator() throws Exception { + checkTreeMap(true, true); + } + + /** + * Check {@code TreeMap} data structure. + * + * @param useBinary Whether to go through binary mode. + * @param useComparator Whether comparator should be used. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + private void checkTreeMap(boolean useBinary, boolean useComparator) throws Exception { + // Populate map. + TreeMap map; + + if (useComparator) { + map = new TreeMap<>(new TestKeyComparator()); + + for (int i = 0; i < SIZE; i++) + map.put(key(false, i), i); + } + else { + map = new TreeMap<>(); + + for (int i = 0; i < SIZE; i++) + map.put(key(true, i), i); + } + + // Put and get value from cache. + cache().put(KEY, map); + + TreeMap resMap; + + if (useBinary) { + BinaryObject resMapBinary = (BinaryObject)cache().withKeepBinary().get(KEY); + + resMap = resMapBinary.deserialize(); + } + else + resMap = (TreeMap)cache().get(KEY); + + // Ensure content is correct. + if (useComparator) + assert resMap.comparator() instanceof TestKeyComparator; + else + assertNull(resMap.comparator()); + + assertEquals(map, resMap); + } + + /** + * Test {@code TreeSet} data structure. + * + * @throws Exception If failed. + */ + public void testTreeSetRegularNoComparator() throws Exception { + checkTreeSet(false, false); + } + + /** + * Test {@code TreeSet} data structure with comparator. + * + * @throws Exception If failed. + */ + public void testTreeSetRegularComparator() throws Exception { + checkTreeSet(false, true); + } + + /** + * Test {@code TreeSet} data structure with binary mode. + * + * @throws Exception If failed. + */ + public void testTreeSetBinaryNoComparator() throws Exception { + checkTreeSet(true, false); + } + + /** + * Test {@code TreeSet} data structure with binary mode and comparator. + * + * @throws Exception If failed. + */ + public void testTreeSetBinaryComparator() throws Exception { + checkTreeSet(true, true); + } + + /** + * Check {@code TreeSet} data structure. + * + * @param useBinary Whether to go through binary mode. + * @param useComparator Whether comparator should be used. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + private void checkTreeSet(boolean useBinary, boolean useComparator) throws Exception { + // Populate set. + TreeSet set; + + if (useComparator) { + set = new TreeSet<>(new TestKeyComparator()); + + for (int i = 0; i < SIZE; i++) + set.add(key(false, i)); + } + else { + set = new TreeSet<>(); + + for (int i = 0; i < SIZE; i++) + set.add(key(true, i)); + } + + // Put and get value from cache. + cache().put(KEY, set); + + TreeSet resSet; + + if (useBinary) { + BinaryObject resMapBinary = (BinaryObject)cache().withKeepBinary().get(KEY); + + resSet = resMapBinary.deserialize(); + } + else + resSet = (TreeSet)cache().get(KEY); + + // Ensure content is correct. + if (useComparator) + assert resSet.comparator() instanceof TestKeyComparator; + else + assertNull(resSet.comparator()); + + assertEquals(set, resSet); + } + + /** + * @return Cache. + */ + private IgniteCache cache() { + return G.ignite(NODE_CLI).cache(null); + } + + /** + * Get key. + * + * @param comparable Whether it should be comparable. + * @param id ID. + * @return Key. + */ + private static TestKey key(boolean comparable, int id) { + return comparable ? new TestKeyComparable(id) : new TestKey(id); + } + + /** + * Create configuration. + * + * @param name Node name. + * @param client Client mode flag. + * @return Configuration. + */ + private static IgniteConfiguration configuration(String name, boolean client) { + IgniteConfiguration cfg = new IgniteConfiguration(); + + cfg.setGridName(name); + + cfg.setClientMode(client); + + cfg.setLocalHost("127.0.0.1"); + cfg.setPeerClassLoadingEnabled(false); + + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(); + + ipFinder.setAddresses(Collections.singletonList("127.0.0.1:47500")); + + discoSpi.setIpFinder(ipFinder); + + cfg.setDiscoverySpi(discoSpi); + + CacheConfiguration ccfg = new CacheConfiguration(); + + ccfg.setCacheMode(CacheMode.PARTITIONED); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** + * Test key. + */ + private static class TestKey { + /** ID. */ + private int id; + + /** + * Constructor. + * + * @param id ID. + */ + public TestKey(int id) { + this.id = id; + } + + /** + * @return ID. + */ + public int id() { + return id; + } + } + + /** + * Test key implementing comparable interface. + */ + private static class TestKeyComparable extends TestKey implements Comparable { + /** + * Constructor. + * + * @param id ID. + */ + private TestKeyComparable(int id) { + super(id); + } + + /** {@inheritDoc} */ + @Override public int compareTo(@NotNull TestKey o) { + return id() - o.id(); + } + } + + /** + * Test key comparator. + */ + private static class TestKeyComparator implements Comparator { + /** {@inheritDoc} */ + @Override public int compare(TestKey o1, TestKey o2) { + return o1.id() - o2.id(); + } + } +} 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 cedf9a73df67e..dc0540df5962d 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 @@ -31,6 +31,7 @@ import org.apache.ignite.internal.binary.BinaryObjectBuilderDefaultMappersSelfTest; import org.apache.ignite.internal.binary.BinaryObjectBuilderSimpleNameLowerCaseMappersSelfTest; import org.apache.ignite.internal.binary.BinarySimpleNameTestPropertySelfTest; +import org.apache.ignite.internal.binary.BinaryTreeSelfTest; import org.apache.ignite.internal.binary.GridBinaryAffinityKeySelfTest; import org.apache.ignite.internal.binary.GridBinaryMarshallerCtxDisabledSelfTest; import org.apache.ignite.internal.binary.GridBinaryWildcardsSelfTest; @@ -85,6 +86,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(BinaryBasicIdMapperSelfTest.class); suite.addTestSuite(BinaryBasicNameMapperSelfTest.class); + suite.addTestSuite(BinaryTreeSelfTest.class); suite.addTestSuite(BinaryMarshallerSelfTest.class); suite.addTestSuite(BinaryConfigurationConsistencySelfTest.class); suite.addTestSuite(GridBinaryMarshallerCtxDisabledSelfTest.class); From 89bce0fc5cb3dba56626c7088e607d0b25c3353c Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Thu, 30 Jun 2016 13:14:16 +0300 Subject: [PATCH 023/487] IGNITE-3021: IGFS: Fixed failiing IgfsStreamsSelfTest.testCreateFileColocated() test. Failure as caused by misconfiguration. --- .../ignite/internal/processors/igfs/IgfsStreamsSelfTest.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java index 724e80af5e060..2af8c5c986af3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java @@ -19,6 +19,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteFileSystem; +import org.apache.ignite.cache.CacheRebalanceMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.FileSystemConfiguration; @@ -123,6 +124,7 @@ public class IgfsStreamsSelfTest extends IgfsCommonAbstractTest { @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); + cfg.setLateAffinityAssignment(false); cfg.setCacheConfiguration(cacheConfiguration(META_CACHE_NAME), cacheConfiguration(DATA_CACHE_NAME)); TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); @@ -155,13 +157,13 @@ protected CacheConfiguration cacheConfiguration(String cacheName) { else { cacheCfg.setCacheMode(PARTITIONED); cacheCfg.setNearConfiguration(null); - cacheCfg.setBackups(0); cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(CFG_GRP_SIZE)); } cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); cacheCfg.setAtomicityMode(TRANSACTIONAL); + cacheCfg.setRebalanceMode(CacheRebalanceMode.SYNC); return cacheCfg; } @@ -232,6 +234,7 @@ public void testCreateFileColocated() throws Exception { Collection affNodes = fs.affinity(path, 0, info.length()); assertEquals(1, affNodes.size()); + Collection nodeIds = F.first(affNodes).nodeIds(); assertEquals(1, nodeIds.size()); From 78aa065b4c7b05381b1fa31159b74969ec4a2bfe Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Thu, 21 Jul 2016 13:15:35 +0300 Subject: [PATCH 024/487] IGNITE-826: Removed HadoopHashMapSelfTest.testAllocation() as it tested nothing. --- .../collections/HadoopHashMapSelfTest.java | 56 ++----------------- 1 file changed, 5 insertions(+), 51 deletions(-) diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.java index 2e1f66e961249..c0f8325bea533 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.java @@ -36,60 +36,14 @@ * */ public class HadoopHashMapSelfTest extends HadoopAbstractMapTest { - - public void testAllocation() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-826"); - - final GridUnsafeMemory mem = new GridUnsafeMemory(0); - - long size = 3L * 1024 * 1024 * 1024; - - final long chunk = 16;// * 1024; - - final int page = 4 * 1024; - - final int writes = chunk < page ? 1 : (int)(chunk / page); - - final long cnt = size / chunk; - - assert cnt < Integer.MAX_VALUE; - - final int threads = 4; - - long start = System.currentTimeMillis(); - - multithreaded(new Callable() { - @Override public Object call() throws Exception { - int cnt0 = (int)(cnt / threads); - - for (int i = 0; i < cnt0; i++) { - long ptr = mem.allocate(chunk); - - for (int j = 0; j < writes; j++) - mem.writeInt(ptr + j * page, 100500); - } - - return null; - } - }, threads); - - X.println("End: " + (System.currentTimeMillis() - start) + " mem: " + mem.allocatedSize() + " cnt: " + cnt); - - Thread.sleep(30000); - } - - - /** */ + /** + * Test simple map. + * + * @throws Exception If failed. + */ public void testMapSimple() throws Exception { GridUnsafeMemory mem = new GridUnsafeMemory(0); -// mem.listen(new GridOffHeapEventListener() { -// @Override public void onEvent(GridOffHeapEvent evt) { -// if (evt == GridOffHeapEvent.ALLOCATE) -// U.dumpStack(); -// } -// }); - Random rnd = new Random(); int mapSize = 16 << rnd.nextInt(3); From f87ca482420fc1e6ffcb000a227717142d24e270 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Thu, 21 Jul 2016 13:15:41 +0300 Subject: [PATCH 025/487] IGNITE-826: Removed HadoopHashMapSelfTest.testAllocation() as it tested nothing. --- .../shuffle/collections/HadoopHashMapSelfTest.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.java index c0f8325bea533..04585eccff2f3 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.java @@ -19,12 +19,6 @@ import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Iterator; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.Callable; import org.apache.hadoop.io.IntWritable; import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput; @@ -32,6 +26,12 @@ import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory; import org.apache.ignite.internal.util.typedef.X; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.Random; + /** * */ From 9ddf9d846f52a4e8fc433643409993884c70ce37 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Thu, 21 Jul 2016 16:00:45 +0300 Subject: [PATCH 026/487] IGNITE-466: IGFS: Added "IgfsMode mode(IgfsPath)" method. --- .../main/java/org/apache/ignite/IgniteFileSystem.java | 9 +++++++++ .../ignite/internal/processors/igfs/IgfsAsyncImpl.java | 6 ++++++ .../apache/ignite/internal/processors/igfs/IgfsImpl.java | 7 +++++++ .../apache/ignite/internal/processors/igfs/IgfsMock.java | 8 ++++++++ 4 files changed, 30 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java index def296553d581..bbc5bead0a126 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java @@ -24,6 +24,7 @@ import org.apache.ignite.igfs.IgfsFile; import org.apache.ignite.igfs.IgfsInputStream; import org.apache.ignite.igfs.IgfsMetrics; +import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsOutputStream; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.IgfsPathSummary; @@ -455,6 +456,14 @@ public R execute(Class> taskCls, */ @Nullable public IgfsFile info(IgfsPath path) throws IgniteException; + /** + * Get mode for the given path. + * + * @param path Path. + * @return Mode used for this path. + */ + public IgfsMode mode(IgfsPath path); + /** * Gets used space in bytes. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java index 7530557174a90..bf3d22b53975a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java @@ -26,6 +26,7 @@ import org.apache.ignite.igfs.IgfsBlockLocation; import org.apache.ignite.igfs.IgfsFile; import org.apache.ignite.igfs.IgfsMetrics; +import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsOutputStream; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.IgfsPathSummary; @@ -307,6 +308,11 @@ public IgfsAsyncImpl(IgfsImpl igfs) { return igfs.info(path); } + /** {@inheritDoc} */ + @Override public IgfsMode mode(IgfsPath path) { + return igfs.mode(path); + } + /** {@inheritDoc} */ @Override public long usedSpaceSize() { return igfs.usedSpaceSize(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index df7dfb57cedb5..0d1ffc123ceff 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -576,6 +576,13 @@ else if (val) }); } + /** {@inheritDoc} */ + @Override public IgfsMode mode(IgfsPath path) { + A.notNull(path, "path"); + + return modeRslvr.resolveMode(path); + } + /** {@inheritDoc} */ @Override public IgfsPathSummary summary(final IgfsPath path) { A.notNull(path, "path"); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java index dccab4a44f442..a2bd9ca5a82f2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java @@ -24,6 +24,7 @@ import org.apache.ignite.igfs.IgfsBlockLocation; import org.apache.ignite.igfs.IgfsFile; import org.apache.ignite.igfs.IgfsMetrics; +import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsOutputStream; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.IgfsPathSummary; @@ -360,6 +361,13 @@ public IgfsMock(@Nullable String name) { return null; } + /** {@inheritDoc} */ + @Override public IgfsMode mode(IgfsPath path) { + throwUnsupported(); + + return null; + } + /** {@inheritDoc} */ @Override public long usedSpaceSize() throws IgniteException { throwUnsupported(); From dc81069ba9ebb88bc11cf6917e8733cc1f6de2fb Mon Sep 17 00:00:00 2001 From: Ivan Veselovskiy Date: Tue, 2 Aug 2016 11:11:24 +0300 Subject: [PATCH 027/487] IGNITE-3343: IGFS: Secondary file system is not queried for statuses during MKDIRS and CREATE operations. This closes #896. --- .../org/apache/ignite/IgniteFileSystem.java | 3 +- .../secondary/IgfsSecondaryFileSystem.java | 4 +- .../processors/igfs/IgfsCreateResult.java | 2 +- .../internal/processors/igfs/IgfsImpl.java | 63 ++++++---------- .../processors/igfs/IgfsMetaManager.java | 74 +++++-------------- .../processors/igfs/IgfsModeResolver.java | 31 ++++++-- .../internal/processors/igfs/IgfsPaths.java | 6 +- .../IgfsSecondaryFileSystemCreateContext.java | 3 + .../internal/processors/igfs/IgfsUtils.java | 12 ++- .../processors/igfs/IgfsAbstractSelfTest.java | 34 ++++++++- .../igfs/IgfsDualAbstractSelfTest.java | 13 ++++ .../igfs/IgfsModeResolverSelfTest.java | 51 +++++++++++-- .../hadoop/fs/v1/IgniteHadoopFileSystem.java | 7 +- .../hadoop/fs/v2/IgniteHadoopFileSystem.java | 7 +- ...gniteHadoopFileSystemAbstractSelfTest.java | 1 - .../hadoop/HadoopAbstractMapReduceTest.java | 13 +++- 16 files changed, 196 insertions(+), 128 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java index bbc5bead0a126..f9aeb8da09a8b 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java @@ -362,8 +362,7 @@ public R execute(Class> taskCls, * will not be affected. Other properties will be added or overwritten. Passed properties with {@code null} values * will be removed from the stored properties or ignored if they don't exist in the file info. *

      - * When working in {@code DUAL_SYNC} or {@code DUAL_ASYNC} modes only the following properties will be propagated - * to the secondary file system: + * When working in {@code DUAL_SYNC} or {@code DUAL_ASYNC} modes with Hadoop secondary file system only the following properties will be updated: *

        *
      • {@code usrName} - file owner name;
      • *
      • {@code grpName} - file owner group;
      • diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java index 3f124eb081381..4d9d255e26432 100644 --- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java +++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java @@ -43,8 +43,8 @@ public interface IgfsSecondaryFileSystem { * will not be affected. Other properties will be added or overwritten. Passed properties with {@code null} values * will be removed from the stored properties or ignored if they don't exist in the file info. *

        - * When working in {@code DUAL_SYNC} or {@code DUAL_ASYNC} modes only the following properties will be propagated - * to the secondary file system: + * When working in {@code DUAL_SYNC} or {@code DUAL_ASYNC} modes with Hadoop secondary file system only the + * following properties will be updated on the secondary file system: *

          *
        • {@code usrName} - file owner name;
        • *
        • {@code grpName} - file owner group;
        • diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java index 0b09e02deb4aa..eb1bc9d0ca9df 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java @@ -23,7 +23,7 @@ import java.io.OutputStream; /** - * IGFS file create result. + * IGFS file create or append result. */ public class IgfsCreateResult { /** File info in the primary file system. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 0d1ffc123ceff..e1f8e611b714e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.igfs; +import java.util.Set; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -248,7 +249,7 @@ public final class IgfsImpl implements IgfsEx { } } - modeRslvr = new IgfsModeResolver(dfltMode, IgfsUtils.preparePathModes(dfltMode, modes)); + modeRslvr = new IgfsModeResolver(dfltMode, modes); Object secondaryFsPayload = null; @@ -816,12 +817,14 @@ else if (val) } } - IgniteUuid fileId = meta.fileId(path); + if (!IgfsUtils.isDualMode(mode) || modeRslvr.hasPrimaryChild(path)) { + IgniteUuid fileId = meta.fileId(path); - if (fileId != null) - files.addAll(meta.directoryListing(fileId).keySet()); - else if (mode == PRIMARY) - throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path); + if (fileId != null) + files.addAll(meta.directoryListing(fileId).keySet()); + else if (mode == PRIMARY) + throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path); + } return F.viewReadOnly(files, new C1() { @Override public IgfsPath apply(String e) { @@ -846,7 +849,7 @@ else if (mode == PRIMARY) IgfsMode mode = resolveMode(path); - Collection files = new HashSet<>(); + Set files = new HashSet<>(); if (IgfsUtils.isDualMode(mode)) { assert secondaryFs != null; @@ -859,6 +862,9 @@ else if (mode == PRIMARY) files.add(impl); } + + if (!modeRslvr.hasPrimaryChild(path)) + return files; } catch (Exception e) { U.error(log, "List files in DUAL mode failed [path=" + path + ']', e); @@ -1345,29 +1351,6 @@ IgniteInternalFuture formatAsync() { return fut; } - /** - * Get file descriptor for specified path. - * - * @param path Path to file. - * @return Detailed file descriptor or {@code null}, if file does not exist. - * @throws IgniteCheckedException If failed. - */ - @Nullable private FileDescriptor getFileDescriptor(IgfsPath path) throws IgniteCheckedException { - assert path != null; - - List ids = meta.idsForPath(path); - - IgfsEntryInfo fileInfo = meta.info(ids.get(ids.size() - 1)); - - if (fileInfo == null) - return null; // File does not exist. - - // Resolve parent ID for removed file. - IgniteUuid parentId = ids.size() >= 2 ? ids.get(ids.size() - 2) : null; - - return new FileDescriptor(parentId, path.name(), fileInfo.id(), fileInfo.isFile()); - } - /** {@inheritDoc} */ @Override public R execute(IgfsTask task, @Nullable IgfsRecordResolver rslvr, Collection paths, @Nullable T arg) { @@ -1519,20 +1502,16 @@ private IgfsFileImpl resolveFileInfo(IgfsPath path, IgfsMode mode) throws Except case DUAL_SYNC: case DUAL_ASYNC: - info = meta.infoForPath(path); - - if (info == null) { - try { - IgfsFile status = secondaryFs.info(path); + try { + IgfsFile status = secondaryFs.info(path); - if (status != null) - return new IgfsFileImpl(status, data.groupBlockSize()); - } - catch (Exception e) { - U.error(log, "File info operation in DUAL mode failed [path=" + path + ']', e); + if (status != null) + return new IgfsFileImpl(status, data.groupBlockSize()); + } + catch (Exception e) { + U.error(log, "File info operation in DUAL mode failed [path=" + path + ']', e); - throw e; - } + throw e; } break; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java index d891b3887762f..0d14a3a271b6e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java @@ -3051,29 +3051,19 @@ IgfsCreateResult create( if (secondaryCtx != null) { secondaryOut = secondaryCtx.create(); - IgfsFile secondaryFile = secondaryCtx.info(); - - if (secondaryFile == null) - throw fsException("Failed to open output stream to the file created in " + - "the secondary file system because it no longer exists: " + path); - else if (secondaryFile.isDirectory()) - throw fsException("Failed to open output stream to the file created in " + - "the secondary file system because the path points to a directory: " + path); - - newAccessTime = secondaryFile.accessTime(); - newModificationTime = secondaryFile.modificationTime(); - newProps = secondaryFile.properties(); - newLen = secondaryFile.length(); - newBlockSize = secondaryFile.blockSize(); + newAccessTime = 0L; + newModificationTime = 0L; + newProps = null; } else { newAccessTime = System.currentTimeMillis(); newModificationTime = newAccessTime; newProps = fileProps; - newLen = 0L; - newBlockSize = blockSize; } + newLen = 0L; + newBlockSize = blockSize; + IgfsEntryInfo newInfo = invokeAndGet(overwriteId, new IgfsMetaFileCreateProcessor(newAccessTime, newModificationTime, newProps, newBlockSize, affKey, newLockId, evictExclude, newLen)); @@ -3254,18 +3244,9 @@ private IgfsPathsCreateResult createFileOrDirectory(boolean dir, IgfsPathIds pat Map props; if (secondaryCtx != null) { - IgfsFile secondaryInfo = secondaryCtx.fileSystem().info(lastCreatedPath); - - if (secondaryInfo == null) - throw new IgfsException("Failed to perform operation because secondary file system path was " + - "modified concurrently: " + lastCreatedPath); - else if (secondaryInfo.isFile()) - throw new IgfsException("Failed to perform operation because secondary file system entity is " + - "not directory: " + lastCreatedPath); - - accessTime = secondaryInfo.accessTime(); - modificationTime = secondaryInfo.modificationTime(); - props = secondaryInfo.properties(); + accessTime = 0L; + modificationTime = 0L; + props = null; } else { accessTime = curTime; @@ -3293,18 +3274,9 @@ else if (secondaryInfo.isFile()) Map props; if (secondaryCtx != null) { - IgfsFile secondaryInfo = secondaryCtx.fileSystem().info(pathIds.path()); - - if (secondaryInfo == null) - throw new IgfsException("Failed to perform operation because secondary file system path was " + - "modified concurrnetly: " + pathIds.path()); - else if (secondaryInfo.isFile()) - throw new IgfsException("Failed to perform operation because secondary file system entity is " + - "not directory: " + lastCreatedPath); - - accessTime = secondaryInfo.accessTime(); - modificationTime = secondaryInfo.modificationTime(); - props = secondaryInfo.properties(); + accessTime = 0L; + modificationTime = 0L; + props = null; } else { accessTime = curTime; @@ -3322,29 +3294,19 @@ else if (secondaryInfo.isFile()) int newBlockSize; if (secondaryCtx != null) { - IgfsFile secondaryFile = secondaryCtx.info(); - - if (secondaryFile == null) - throw fsException("Failed to open output stream to the file created in " + - "the secondary file system because it no longer exists: " + pathIds.path()); - else if (secondaryFile.isDirectory()) - throw fsException("Failed to open output stream to the file created in " + - "the secondary file system because the path points to a directory: " + pathIds.path()); - - newAccessTime = secondaryFile.accessTime(); - newModificationTime = secondaryFile.modificationTime(); - newProps = secondaryFile.properties(); - newLen = secondaryFile.length(); - newBlockSize = secondaryFile.blockSize(); + newAccessTime = 0L; + newModificationTime = 0L; + newProps = null; } else { newAccessTime = curTime; newModificationTime = curTime; newProps = fileProps; - newLen = 0L; - newBlockSize = blockSize; } + newLen = 0L; + newBlockSize = blockSize; + procMap.put(curId, new IgfsMetaFileCreateProcessor(newAccessTime, newModificationTime, newProps, newBlockSize, affKey, createFileLockId(false), evictExclude, newLen)); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java index 907051f1d7a80..33b835f9f276b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java @@ -17,9 +17,12 @@ package org.apache.ignite.internal.processors.igfs; -import java.util.Collections; +import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap; @@ -42,18 +45,24 @@ public class IgfsModeResolver { /** Cached modes per path. */ private Map modesCache; + /** Set to store parent dual paths that have primary children. */ + private final Set dualParentsWithPrimaryChildren; + /** * Constructor * * @param dfltMode Default IGFS mode. * @param modes List of configured modes. The order is significant as modes are added in order of occurrence. */ - public IgfsModeResolver(IgfsMode dfltMode, @Nullable List> modes) { + public IgfsModeResolver(IgfsMode dfltMode, @Nullable ArrayList> modes) + throws IgniteCheckedException { assert dfltMode != null; this.dfltMode = dfltMode; - this.modes = modes; + this.dualParentsWithPrimaryChildren = new HashSet<>(); + + this.modes = IgfsUtils.preparePathModes(dfltMode, modes, dualParentsWithPrimaryChildren); if (modes != null) modesCache = new GridBoundedConcurrentLinkedHashMap<>(MAX_PATH_CACHE); @@ -94,10 +103,20 @@ public IgfsMode resolveMode(IgfsPath path) { } /** - * @return Unmodifiable copy of properly ordered modes prefixes + * @return Copy of properly ordered modes prefixes * or {@code null} if no modes set. */ - @Nullable public List> modesOrdered() { - return modes != null ? Collections.unmodifiableList(modes) : null; + @Nullable public ArrayList> modesOrdered() { + return modes != null ? new ArrayList<>(modes) : null; + } + + /** + * Answers if the given path has an immediate child of PRIMARY mode. + * + * @param path The path to query. + * @return If the given path has an immediate child of PRIMARY mode. + */ + public boolean hasPrimaryChild(IgfsPath path) { + return dualParentsWithPrimaryChildren.contains(path); } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java index 4a79259d1f77b..2bdb23b1c64b0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java @@ -48,7 +48,7 @@ public class IgfsPaths implements Externalizable { private IgfsMode dfltMode; /** Path modes. */ - private List> pathModes; + private ArrayList> pathModes; /** * Empty constructor required by {@link Externalizable}. @@ -65,7 +65,7 @@ public IgfsPaths() { * @param pathModes Path modes. * @throws IgniteCheckedException If failed. */ - public IgfsPaths(Object payload, IgfsMode dfltMode, @Nullable List> pathModes) + public IgfsPaths(Object payload, IgfsMode dfltMode, @Nullable ArrayList> pathModes) throws IgniteCheckedException { this.dfltMode = dfltMode; this.pathModes = pathModes; @@ -91,7 +91,7 @@ public IgfsMode defaultMode() { /** * @return Path modes. */ - @Nullable public List> pathModes() { + @Nullable public ArrayList> pathModes() { return pathModes; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java index 1c0efd6e4f8eb..788efe391ad5e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java @@ -28,6 +28,7 @@ /** * Context for secondary file system create request. + * Note that it is never used for dual mode append operation. */ public class IgfsSecondaryFileSystemCreateContext { /** File system. */ @@ -68,6 +69,8 @@ public class IgfsSecondaryFileSystemCreateContext { */ public IgfsSecondaryFileSystemCreateContext(IgfsSecondaryFileSystem fs, IgfsPath path, boolean overwrite, boolean simpleCreate, @Nullable Map props, short replication, long blockSize, int bufSize) { + assert fs != null; + this.fs = fs; this.path = path; this.overwrite = overwrite; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java index 03b761146ecb3..b9788f881675f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.igfs; +import java.util.Set; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteSystemProperties; @@ -66,6 +67,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_RETRIES_COUNT; import static org.apache.ignite.igfs.IgfsMode.DUAL_ASYNC; import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC; +import static org.apache.ignite.igfs.IgfsMode.PRIMARY; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGFS; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; @@ -819,11 +821,13 @@ public static boolean canContain(IgfsMode parent, IgfsMode child) { * * @param dfltMode The root mode. Must always be not null. * @param modes The subdirectory modes. + * @param dualParentsContainingPrimaryChildren The set to store parents into. * @return Descending list of filtered and checked modes. - * @throws IgniteCheckedException On error or + * @throws IgniteCheckedException On error. */ public static ArrayList> preparePathModes(final IgfsMode dfltMode, - @Nullable List> modes) throws IgniteCheckedException { + @Nullable List> modes, Set dualParentsContainingPrimaryChildren) + throws IgniteCheckedException { if (modes == null) return null; @@ -857,6 +861,10 @@ public static ArrayList> preparePathModes(final IgfsMode // Add to the 1st position (deep first). resModes.add(0, mode); + // Store primary paths inside dual paths in separate collection: + if (mode.getValue() == PRIMARY) + dualParentsContainingPrimaryChildren.add(mode.getKey().parent()); + break; } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java index 384da954bc557..fd3f9b6b3774b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java @@ -244,6 +244,13 @@ protected boolean relaxedConsistency() { return false; } + /** + * @return Relaxed consistency flag. + */ + protected boolean initializeDefaultPathModes() { + return false; + } + /** * @return Client flag. */ @@ -369,6 +376,8 @@ protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mo igfsCfg.setSequentialReadsBeforePrefetch(SEQ_READS_BEFORE_PREFETCH); igfsCfg.setRelaxedConsistency(relaxedConsistency()); + igfsCfg.setInitializeDefaultPathModes(initializeDefaultPathModes()); + CacheConfiguration dataCacheCfg = defaultCacheConfiguration(); dataCacheCfg.setName("dataCache"); @@ -1071,17 +1080,36 @@ public void testFormat() throws Exception { * @throws Exception If failed. */ public void testRootPropertiesPersistAfterFormat() throws Exception { - igfs.update(new IgfsPath("/"), Collections.singletonMap("foo", "moo")); + if (dual && !(igfsSecondaryFileSystem instanceof IgfsSecondaryFileSystemImpl)) { + // In case of Hadoop dual mode only user name, group name, and permission properties are updated, + // an arbitrary named property is just ignored: + checkRootPropertyUpdate("foo", "moo", null); + checkRootPropertyUpdate(IgfsUtils.PROP_PERMISSION, "0777", "0777"); + } + else { + checkRootPropertyUpdate("foo", "moo", "moo"); + checkRootPropertyUpdate(IgfsUtils.PROP_PERMISSION, "0777", "0777"); + } + } + + /** + * + * @throws Exception + */ + private void checkRootPropertyUpdate(String prop, String setVal, String expGetVal) throws Exception { + final IgfsPath rootPath = new IgfsPath("/"); + + igfs.update(rootPath, Collections.singletonMap(prop, setVal)); igfs.format(); - IgfsFile file = igfs.info(new IgfsPath("/")); + IgfsFile file = igfs.info(rootPath); assert file != null; Map props = file.properties(); - assertEquals("moo", props.get("foo")); + assertEquals(expGetVal, props.get(prop)); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java index 124bec692af25..92e1178d35839 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java @@ -55,6 +55,12 @@ protected IgfsDualAbstractSelfTest(IgfsMode mode) { assert mode == DUAL_SYNC || mode == DUAL_ASYNC; } + /** {@inheritDoc} */ + @Override protected boolean initializeDefaultPathModes() { + // Enable default modes in order to test various modes. + return true; + } + /** * @throws Exception If failed. */ @@ -68,6 +74,13 @@ public void testDefaultDirectories() throws Exception { for (IgfsPath p : paths) assert igfs.exists(p); + assert igfs.modeResolver().resolveMode(gg) == mode; + assert igfs.modeResolver().resolveMode(new IgfsPath(gg, "sync")) == IgfsMode.DUAL_SYNC; + assert igfs.modeResolver().resolveMode(new IgfsPath(gg, "async")) == IgfsMode.DUAL_ASYNC; + assert igfs.modeResolver().resolveMode(new IgfsPath(gg, "primary")) == IgfsMode.PRIMARY; + assert !igfsSecondary.exists("/ignite/primary"); // PRIMARY mode path must exist in upper level fs only. + + // All the child paths of "/ignite/" must be visible in listings: assert igfs.listFiles(gg).size() == 3; assert igfs.listPaths(gg).size() == 3; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java index f3bb516041619..bd7e413e1850e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java @@ -17,8 +17,11 @@ package org.apache.ignite.internal.processors.igfs; +import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.List; +import java.util.Set; import junit.framework.TestCase; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.igfs.IgfsMode; @@ -38,8 +41,9 @@ public class IgfsModeResolverSelfTest extends TestCase { /** {@inheritDoc} */ @Override protected void setUp() throws Exception { - reslvr = new IgfsModeResolver(DUAL_SYNC, Arrays.asList(new T2<>(new IgfsPath("/a/b/c/d"), PROXY), new T2<> - (new IgfsPath("/a/P/"), PRIMARY), new T2<>(new IgfsPath("/a/b/"), DUAL_ASYNC))); + reslvr = new IgfsModeResolver(DUAL_SYNC, new ArrayList<>(Arrays.asList(new T2<>( + new IgfsPath("/a/b/c/d"), PROXY), new T2<>(new IgfsPath("/a/P/"), PRIMARY), new T2<>(new IgfsPath("/a/b/"), + DUAL_ASYNC)))); } /** @@ -90,7 +94,7 @@ public void testModesValidation() throws Exception { try { IgfsUtils.preparePathModes(DUAL_SYNC, Arrays.asList( new T2<>(new IgfsPath("/a/"), PRIMARY), - new T2<>(new IgfsPath("/a/b/"), DUAL_ASYNC))); + new T2<>(new IgfsPath("/a/b/"), DUAL_ASYNC)), new HashSet()); fail("IgniteCheckedException expected"); } @@ -102,7 +106,8 @@ public void testModesValidation() throws Exception { for (IgfsMode m: IgfsMode.values()) { if (m != IgfsMode.PRIMARY) { try { - IgfsUtils.preparePathModes(PRIMARY, Arrays.asList(new T2<>(new IgfsPath("/a/"), DUAL_ASYNC))); + IgfsUtils.preparePathModes(PRIMARY, Arrays.asList(new T2<>(new IgfsPath("/a/"), DUAL_ASYNC)), + new HashSet()); fail("IgniteCheckedException expected"); } @@ -117,7 +122,7 @@ public void testModesValidation() throws Exception { new T2<>(new IgfsPath("/a"), PRIMARY), new T2<>(new IgfsPath("/c/d/"), PRIMARY), new T2<>(new IgfsPath("/c/d/e/f"), PRIMARY) - )); + ), new HashSet()); assertNotNull(modes); assertEquals(2, modes.size()); assertEquals(modes, Arrays.asList( @@ -130,7 +135,7 @@ public void testModesValidation() throws Exception { new T2<>(new IgfsPath("/a/b"), DUAL_ASYNC), new T2<>(new IgfsPath("/a/b/c"), DUAL_SYNC), new T2<>(new IgfsPath("/a/b/c/d"), DUAL_ASYNC) - )); + ), new HashSet()); assertNotNull(modes); assertEquals(modes.size(), 3); assertEquals(modes, Arrays.asList( @@ -139,4 +144,38 @@ public void testModesValidation() throws Exception { new T2<>(new IgfsPath("/a/b"), DUAL_ASYNC) )); } + + /** + * @throws Exception If failed. + */ + public void testDualParentsWithPrimaryChild() throws Exception { + Set set = new HashSet<>(); + + IgfsUtils.preparePathModes(DUAL_SYNC, Arrays.asList( + new T2<>(new IgfsPath("/a/b"), DUAL_ASYNC), + new T2<>(new IgfsPath("/a/b/c"), PRIMARY), + new T2<>(new IgfsPath("/a/b/x/y"), PRIMARY), + new T2<>(new IgfsPath("/a/b/x/z"), PRIMARY), + new T2<>(new IgfsPath("/m"), PRIMARY) + ), set); + assertEquals(set, new HashSet() {{ + add(new IgfsPath("/a/b")); + add(new IgfsPath("/a/b/x")); + add(new IgfsPath("/")); + }}); + + set = new HashSet<>(); + + IgfsUtils.preparePathModes(DUAL_ASYNC, Arrays.asList( + new T2<>(new IgfsPath("/a/b/x/y/z"), PRIMARY), + new T2<>(new IgfsPath("/a/b/c"), PRIMARY), + new T2<>(new IgfsPath("/a/k"), PRIMARY), + new T2<>(new IgfsPath("/a/z"), PRIMARY) + ), set); + assertEquals(set, new HashSet() {{ + add(new IgfsPath("/a/b")); + add(new IgfsPath("/a")); + add(new IgfsPath("/a/b/x/y")); + }}); + } } \ No newline at end of file diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java index 83991aa7466a1..a06129e43fc63 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java @@ -309,7 +309,12 @@ public void colocateFileWrites(boolean colocateFileWrites) { else clientLog = IgfsLogger.disabledLogger(); - modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes()); + try { + modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes()); + } + catch (IgniteCheckedException ice) { + throw new IOException(ice); + } boolean initSecondary = paths.defaultMode() == PROXY; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java index cc02d0567c25e..bd8ed2db35c93 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java @@ -318,7 +318,12 @@ private void initialize(URI name, Configuration cfg) throws IOException { else clientLog = IgfsLogger.disabledLogger(); - modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes()); + try { + modeRslvr = new IgfsModeResolver(paths.defaultMode(), paths.pathModes()); + } + catch (IgniteCheckedException ice) { + throw new IOException(ice); + } boolean initSecondary = paths.defaultMode() == PROXY; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java index 083ee545ee088..f793ec31f0b5a 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java @@ -48,7 +48,6 @@ import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; -import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java index ef886e47ce838..37312133ea5ac 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java @@ -41,6 +41,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.fs.IgniteHadoopFileSystemCounterWriter; import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem; +import org.apache.ignite.igfs.IgfsFile; import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; import org.apache.ignite.igfs.IgfsMode; @@ -123,8 +124,16 @@ public class HadoopAbstractMapReduceTest extends HadoopAbstractWordCountTest { * @param p The path. * @return The owner. */ - private static String getOwner(IgfsEx i, IgfsPath p) { - return i.info(p).property(IgfsUtils.PROP_USER_NAME); + private static String getOwner(final IgfsEx i, final IgfsPath p) { + return IgfsUserContext.doAs(USER, new IgniteOutClosure() { + @Override public String apply() { + IgfsFile f = i.info(p); + + assert f != null; + + return f.property(IgfsUtils.PROP_USER_NAME); + } + }); } /** From ae54e36f27719f46d1d276f62a977c3f8d053b44 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Thu, 4 Aug 2016 17:04:41 +0300 Subject: [PATCH 028/487] IGNITE-3331 IGFS: Route client tasks to primary node when metadata co-location is enabled. This closes #921. --- .../processors/igfs/IgfsMetaManager.java | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java index 0d14a3a271b6e..d1151c5b49d55 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java @@ -248,7 +248,25 @@ boolean isClient() { */ T runClientTask(IgfsClientAbstractCallable task) { try { - return clientCompute().call(task); + return runClientTask(IgfsUtils.ROOT_ID, task); + } + catch (ClusterTopologyException e) { + throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes." , e); + } + } + + /** + * Run client task. + * + * @param affinityFileId Affinity fileId. + * @param task Task. + * @return Result. + */ + T runClientTask(IgniteUuid affinityFileId, IgfsClientAbstractCallable task) { + try { + return (cfg.isColocateMetadata()) ? + clientCompute().affinityCall(cfg.getMetaCacheName(), affinityFileId, task) : + clientCompute().call(task); } catch (ClusterTopologyException e) { throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes." , e); From 970137b1db7dc6c5e546581e22e428ae15c86513 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Fri, 5 Aug 2016 15:05:32 +0300 Subject: [PATCH 029/487] IGNITE-3631: IGFS: Now metadata co-location is used for PARTITIONED cache as well. --- .../org/apache/ignite/internal/processors/igfs/IgfsUtils.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java index b9788f881675f..3a313ad7e57a5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java @@ -376,8 +376,7 @@ public static void prepareCacheConfiguration(IgniteConfiguration cfg, CacheConfi ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); // Set co-located affinity mapper if needed. - if (igfsCfg.isColocateMetadata() && ccfg.getCacheMode() == CacheMode.REPLICATED && - ccfg.getAffinityMapper() == null) + if (igfsCfg.isColocateMetadata() && ccfg.getAffinityMapper() == null) ccfg.setAffinityMapper(new IgfsColocatedMetadataAffinityKeyMapper()); return; From 4d876a7560060c892908da447178e97bfe12ca9c Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Fri, 5 Aug 2016 15:05:43 +0300 Subject: [PATCH 030/487] IGNITE-3631: IGFS: Now metadata co-location is used for PARTITIONED cache as well. --- .../apache/ignite/internal/processors/igfs/IgfsUtils.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java index 3a313ad7e57a5..3d86fbb1f5376 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java @@ -17,13 +17,11 @@ package org.apache.ignite.internal.processors.igfs; -import java.util.Set; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.binary.BinaryRawReader; import org.apache.ignite.binary.BinaryRawWriter; -import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.cluster.ClusterTopologyException; @@ -32,8 +30,8 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.IgfsEvent; import org.apache.ignite.igfs.IgfsException; -import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.binary.BinaryUtils; @@ -61,6 +59,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; From f5a040a01280c654df1fc4789cc39ff1ac2d32a4 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Tue, 9 Aug 2016 10:01:56 +0300 Subject: [PATCH 031/487] IGNITE-3332: IGFS: Optimized file unlock routine with help of a client callable. This closes #916. --- .../ignite/internal/binary/BinaryContext.java | 2 + .../igfs/IgfsFileAffinityRange.java | 40 ++++-- .../processors/igfs/IgfsMetaManager.java | 9 ++ .../internal/processors/igfs/IgfsUtils.java | 34 +++++ .../meta/IgfsClientMetaUnlockCallable.java | 123 ++++++++++++++++++ 5 files changed, 197 insertions(+), 11 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaUnlockCallable.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java index 8517acf972f00..97afef1292e8b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java @@ -60,6 +60,7 @@ import org.apache.ignite.internal.processors.igfs.client.IgfsClientUpdateCallable; import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaIdsForPathCallable; import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaInfoForPathCallable; +import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaUnlockCallable; import org.apache.ignite.internal.processors.igfs.data.IgfsDataPutProcessor; import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor; import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingAddProcessor; @@ -162,6 +163,7 @@ public class BinaryContext { sysClss.add(IgfsClientMetaIdsForPathCallable.class.getName()); sysClss.add(IgfsClientMetaInfoForPathCallable.class.getName()); + sysClss.add(IgfsClientMetaUnlockCallable.class.getName()); sysClss.add(IgfsClientAffinityCallable.class.getName()); sysClss.add(IgfsClientDeleteCallable.class.getName()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileAffinityRange.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileAffinityRange.java index 953461ca3c51c..194869e2a0a76 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileAffinityRange.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileAffinityRange.java @@ -266,22 +266,40 @@ public boolean regionEqual(IgfsFileAffinityRange other) { /** {@inheritDoc} */ @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException { - BinaryRawWriter out = writer.rawWriter(); - - BinaryUtils.writeIgniteUuid(out, affKey); - out.writeInt(status); - out.writeLong(startOff); - out.writeLong(endOff); + writeRawBinary(writer.rawWriter()); } /** {@inheritDoc} */ @Override public void readBinary(BinaryReader reader) throws BinaryObjectException { - BinaryRawReader in = reader.rawReader(); + readRawBinary(reader.rawReader()); + } - affKey = BinaryUtils.readIgniteUuid(in); - status = in.readInt(); - startOff = in.readLong(); - endOff = in.readLong(); + /** + * Writes fields to provided writer. + * + * @param writer Writer. + * @throws BinaryObjectException If fails. + */ + public void writeRawBinary(BinaryRawWriter writer) throws BinaryObjectException { + BinaryUtils.writeIgniteUuid(writer, affKey); + + writer.writeInt(status); + writer.writeLong(startOff); + writer.writeLong(endOff); + } + + /** + * Reads fields from provided reader. + * + * @param reader Reader. + * @throws BinaryObjectException If fails. + */ + public void readRawBinary(BinaryRawReader reader) throws BinaryObjectException { + affKey = BinaryUtils.readIgniteUuid(reader); + + status = reader.readInt(); + startOff = reader.readLong(); + endOff = reader.readLong(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java index d1151c5b49d55..1364491ce5b24 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java @@ -50,6 +50,7 @@ import org.apache.ignite.internal.processors.igfs.client.IgfsClientAbstractCallable; import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaIdsForPathCallable; import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaInfoForPathCallable; +import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaUnlockCallable; import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor; import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingAddProcessor; import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingRemoveProcessor; @@ -664,6 +665,14 @@ public void unlock(final IgniteUuid fileId, final IgniteUuid lockId, final long public void unlock(final IgniteUuid fileId, final IgniteUuid lockId, final long modificationTime, final boolean updateSpace, final long space, @Nullable final IgfsFileAffinityRange affRange) throws IgniteCheckedException { + + if(client) { + runClientTask(new IgfsClientMetaUnlockCallable(cfg.getName(), fileId, lockId, modificationTime, + updateSpace, space, affRange)); + + return; + } + validTxState(false); if (busyLock.enterBusy()) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java index 3d86fbb1f5376..a79d9652ed478 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java @@ -743,6 +743,40 @@ public static void writePath(BinaryRawWriter writer, @Nullable IgfsPath path) { return null; } + /** + * Write IgfsFileAffinityRange. + * + * @param writer Writer + * @param affRange affinity range. + */ + public static void writeFileAffinityRange(BinaryRawWriter writer, @Nullable IgfsFileAffinityRange affRange) { + if (affRange != null) { + writer.writeBoolean(true); + + affRange.writeRawBinary(writer); + } + else + writer.writeBoolean(false); + } + + /** + * Read IgfsFileAffinityRange. + * + * @param reader Reader. + * @return File affinity range. + */ + public static IgfsFileAffinityRange readFileAffinityRange(BinaryRawReader reader) { + if (reader.readBoolean()) { + IgfsFileAffinityRange affRange = new IgfsFileAffinityRange(); + + affRange.readRawBinary(reader); + + return affRange; + } + else + return null; + } + /** * Parses the TRASH file name to extract the original path. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaUnlockCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaUnlockCallable.java new file mode 100644 index 0000000000000..a2f4ab7e84596 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaUnlockCallable.java @@ -0,0 +1,123 @@ +/* + * 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.igfs.client.meta; + +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryRawReader; +import org.apache.ignite.binary.BinaryRawWriter; +import org.apache.ignite.internal.binary.BinaryUtils; +import org.apache.ignite.internal.processors.igfs.IgfsContext; +import org.apache.ignite.internal.processors.igfs.IgfsFileAffinityRange; +import org.apache.ignite.internal.processors.igfs.IgfsUtils; +import org.apache.ignite.internal.processors.igfs.client.IgfsClientAbstractCallable; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +/** + * IGFS client unlock callable. + */ +public class IgfsClientMetaUnlockCallable extends IgfsClientAbstractCallable { + /** */ + private static final long serialVersionUID = 0L; + + /** File id. */ + private IgniteUuid fileId; + + /** Lock id. */ + private IgniteUuid lockId; + + /** Modification time. */ + private long modificationTime; + + /** Update space. */ + private boolean updateSpace; + + /** Space. */ + private long space; + + /** Aff range. */ + private IgfsFileAffinityRange affRange; + /** + * Default constructor. + */ + public IgfsClientMetaUnlockCallable() { + // NO-op. + } + + /** + * Constructor. + * + * @param igfsName IGFS name. + * @param fileId File ID. + * @param lockId Lock ID. + * @param modificationTime Modification time to write to file info. + * @param updateSpace Whether to update space. + * @param space Space. + * @param affRange Affinity range. + */ + public IgfsClientMetaUnlockCallable(@Nullable String igfsName, IgniteUuid fileId, IgniteUuid lockId, + long modificationTime, boolean updateSpace, long space, final IgfsFileAffinityRange affRange) { + super(igfsName, null); + + this.fileId = fileId; + this.lockId = lockId; + this.modificationTime = modificationTime; + this.updateSpace = updateSpace; + this.space = space; + this.affRange = affRange; + } + + /** {@inheritDoc} */ + @Override protected Void call0(IgfsContext ctx) throws Exception { + ctx.meta().unlock(fileId, lockId, modificationTime, updateSpace, space, affRange); + + return null; + } + + /** {@inheritDoc} */ + @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException { + BinaryUtils.writeIgniteUuid(writer, fileId); + BinaryUtils.writeIgniteUuid(writer, lockId); + writer.writeLong(modificationTime); + writer.writeBoolean(updateSpace); + + if (updateSpace) + writer.writeLong(space); + + IgfsUtils.writeFileAffinityRange(writer, affRange); + } + + /** {@inheritDoc} */ + @Override public void readBinary0(BinaryRawReader reader) throws BinaryObjectException { + fileId = BinaryUtils.readIgniteUuid(reader); + lockId = BinaryUtils.readIgniteUuid(reader); + modificationTime = reader.readLong(); + updateSpace = reader.readBoolean(); + + if (updateSpace) + space = reader.readLong(); + + affRange = IgfsUtils.readFileAffinityRange(reader); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(IgfsClientMetaUnlockCallable.class, this); + } +} From 5cf3bea32a25ccc78641f083aa7f1ac81b4187ba Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 15 Aug 2016 13:40:41 +0300 Subject: [PATCH 032/487] IGNITE-1926: IGFS: Implemented local secondary file system. --- .../org/apache/ignite/IgniteFileSystem.java | 27 +- .../secondary/IgfsSecondaryFileSystem.java | 2 +- .../local/LocalIgfsSecondaryFileSystem.java | 396 +++++++ .../igfs/secondary/local/package-info.java | 22 + .../ignite/igfs/secondary/package-info.java | 2 +- .../processors/igfs/IgfsFileImpl.java | 20 +- .../processors/igfs/IgfsMetaManager.java | 2 +- .../internal/processors/igfs/IgfsUtils.java | 53 + .../local/LocalFileSystemIgfsFile.java | 134 +++ ...SecondaryFileSystemPositionedReadable.java | 65 ++ ...ltIgfsSecondaryFileSystemTestAdapter.java} | 19 +- .../processors/igfs/IgfsAbstractSelfTest.java | 995 ++++++++++-------- .../igfs/IgfsDualAbstractSelfTest.java | 198 ++-- ...condaryFileSystemDualAbstractSelfTest.java | 76 ++ ...daryFileSystemDualAsyncClientSelfTest.java | 28 + ...lSecondaryFileSystemDualAsyncSelfTest.java | 32 + ...ndaryFileSystemDualSyncClientSelfTest.java | 28 + ...alSecondaryFileSystemDualSyncSelfTest.java | 32 + ...fsLocalSecondaryFileSystemTestAdapter.java | 141 +++ ...> IgfsSecondaryFileSystemTestAdapter.java} | 23 +- .../testsuites/IgniteIgfsTestSuite.java | 9 + .../IgniteHadoopIgfsSecondaryFileSystem.java | 2 +- .../ignite/igfs/Hadoop1DualAbstractTest.java | 3 +- ...opIgfsSecondaryFileSystemTestAdapter.java} | 38 +- 24 files changed, 1732 insertions(+), 615 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java create mode 100644 modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/package-info.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemIgfsFile.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalIgfsSecondaryFileSystemPositionedReadable.java rename modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/{IgfsExUniversalFileSystemAdapter.java => DefaultIgfsSecondaryFileSystemTestAdapter.java} (87%) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncClientSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncClientSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemTestAdapter.java rename modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/{UniversalFileSystemAdapter.java => IgfsSecondaryFileSystemTestAdapter.java} (87%) rename modules/hadoop/src/test/java/org/apache/ignite/igfs/{HadoopFileSystemUniversalFileSystemAdapter.java => HadoopIgfsSecondaryFileSystemTestAdapter.java} (79%) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java index f9aeb8da09a8b..8fb4fcd2a476a 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java @@ -34,6 +34,7 @@ import org.apache.ignite.lang.IgniteAsyncSupported; import org.apache.ignite.lang.IgniteUuid; import org.jetbrains.annotations.Nullable; +import org.apache.ignite.igfs.IgfsPathNotFoundException; /** * IGnite File System API. It provides a typical file system "view" on a particular cache: @@ -90,7 +91,7 @@ public interface IgniteFileSystem extends IgniteAsyncSupport { * * @param path Path to get information for. * @return Summary object. - * @throws org.apache.ignite.igfs.IgfsPathNotFoundException If path is not found. + * @throws IgfsPathNotFoundException If path is not found. * @throws IgniteException If failed. */ public IgfsPathSummary summary(IgfsPath path) throws IgniteException; @@ -101,7 +102,7 @@ public interface IgniteFileSystem extends IgniteAsyncSupport { * @param path File path to read. * @return File input stream to read data from. * @throws IgniteException In case of error. - * @throws org.apache.ignite.igfs.IgfsPathNotFoundException If path doesn't exist. + * @throws IgfsPathNotFoundException If path doesn't exist. */ public IgfsInputStream open(IgfsPath path) throws IgniteException; @@ -112,7 +113,7 @@ public interface IgniteFileSystem extends IgniteAsyncSupport { * @param bufSize Read buffer size (bytes) or {@code zero} to use default value. * @return File input stream to read data from. * @throws IgniteException In case of error. - * @throws org.apache.ignite.igfs.IgfsPathNotFoundException If path doesn't exist. + * @throws IgfsPathNotFoundException If path doesn't exist. */ public IgfsInputStream open(IgfsPath path, int bufSize) throws IgniteException; @@ -124,7 +125,7 @@ public interface IgniteFileSystem extends IgniteAsyncSupport { * @param seqReadsBeforePrefetch Amount of sequential reads before prefetch is started. * @return File input stream to read data from. * @throws IgniteException In case of error. - * @throws org.apache.ignite.igfs.IgfsPathNotFoundException If path doesn't exist. + * @throws IgfsPathNotFoundException If path doesn't exist. */ public IgfsInputStream open(IgfsPath path, int bufSize, int seqReadsBeforePrefetch) throws IgniteException; @@ -178,7 +179,7 @@ public IgfsOutputStream create(IgfsPath path, int bufSize, boolean overwrite, * @param create Create file if it doesn't exist yet. * @return File output stream to append data to. * @throws IgniteException In case of error. - * @throws org.apache.ignite.igfs.IgfsPathNotFoundException If path doesn't exist and create flag is {@code false}. + * @throws IgfsPathNotFoundException If path doesn't exist and create flag is {@code false}. */ public IgfsOutputStream append(IgfsPath path, boolean create) throws IgniteException; @@ -191,7 +192,7 @@ public IgfsOutputStream create(IgfsPath path, int bufSize, boolean overwrite, * @param props File properties to set only in case it file was just created. * @return File output stream to append data to. * @throws IgniteException In case of error. - * @throws org.apache.ignite.igfs.IgfsPathNotFoundException If path doesn't exist and create flag is {@code false}. + * @throws IgfsPathNotFoundException If path doesn't exist and create flag is {@code false}. */ public IgfsOutputStream append(IgfsPath path, int bufSize, boolean create, @Nullable Map props) throws IgniteException; @@ -204,7 +205,7 @@ public IgfsOutputStream append(IgfsPath path, int bufSize, boolean create, @Null * @param accessTime Optional last access time to set. Value {@code -1} does not update access time. * @param modificationTime Optional last modification time to set. Value {@code -1} does not update * modification time. - * @throws org.apache.ignite.igfs.IgfsPathNotFoundException If target was not found. + * @throws IgfsPathNotFoundException If target was not found. * @throws IgniteException If error occurred. */ public void setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteException; @@ -218,7 +219,7 @@ public IgfsOutputStream append(IgfsPath path, int bufSize, boolean create, @Null * @param len Size of data in the file to resolve affinity for. * @return Affinity block locations. * @throws IgniteException In case of error. - * @throws org.apache.ignite.igfs.IgfsPathNotFoundException If path doesn't exist. + * @throws IgfsPathNotFoundException If path doesn't exist. */ public Collection affinity(IgfsPath path, long start, long len) throws IgniteException; @@ -233,7 +234,7 @@ public IgfsOutputStream append(IgfsPath path, int bufSize, boolean create, @Null * @param maxLen Maximum length of a single returned block location length. * @return Affinity block locations. * @throws IgniteException In case of error. - * @throws org.apache.ignite.igfs.IgfsPathNotFoundException If path doesn't exist. + * @throws IgfsPathNotFoundException If path doesn't exist. */ public Collection affinity(IgfsPath path, long start, long len, long maxLen) throws IgniteException; @@ -393,7 +394,7 @@ public R execute(Class> taskCls, * @param dest Destination file path. If destination path is a directory, then source file will be placed * into destination directory with original name. * @throws IgniteException In case of error. - * @throws org.apache.ignite.igfs.IgfsPathNotFoundException If source file doesn't exist. + * @throws IgfsPathNotFoundException If source file doesn't exist. */ public void rename(IgfsPath src, IgfsPath dest) throws IgniteException; @@ -430,9 +431,9 @@ public R execute(Class> taskCls, * Lists file paths under the specified path. * * @param path Path to list files under. - * @return List of files under the specified path. + * @return List of paths under the specified path. * @throws IgniteException In case of error. - * @throws org.apache.ignite.igfs.IgfsPathNotFoundException If path doesn't exist. + * @throws IgfsPathNotFoundException If path doesn't exist. */ public Collection listPaths(IgfsPath path) throws IgniteException; @@ -442,7 +443,7 @@ public R execute(Class> taskCls, * @param path Path to list files under. * @return List of files under the specified path. * @throws IgniteException In case of error. - * @throws org.apache.ignite.igfs.IgfsPathNotFoundException If path doesn't exist. + * @throws IgfsPathNotFoundException If path doesn't exist. */ public Collection listFiles(IgfsPath path) throws IgniteException; diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java index 4d9d255e26432..47a0dbd69c55f 100644 --- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java +++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java @@ -110,7 +110,7 @@ public interface IgfsSecondaryFileSystem { * Lists file paths under the specified path. * * @param path Path to list files under. - * @return List of files under the specified path. + * @return List of paths under the specified path. * @throws IgniteException In case of error. * @throws org.apache.ignite.igfs.IgfsPathNotFoundException If path doesn't exist. */ diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java new file mode 100644 index 0000000000000..3d3a35065dd56 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java @@ -0,0 +1,396 @@ +/* + * 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.igfs.secondary.local; + +import org.apache.ignite.IgniteException; +import org.apache.ignite.igfs.IgfsException; +import org.apache.ignite.igfs.IgfsFile; +import org.apache.ignite.igfs.IgfsPath; +import org.apache.ignite.igfs.IgfsPathAlreadyExistsException; +import org.apache.ignite.igfs.IgfsPathIsNotDirectoryException; +import org.apache.ignite.igfs.IgfsPathNotFoundException; +import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; +import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable; +import org.apache.ignite.internal.processors.igfs.secondary.local.LocalFileSystemIgfsFile; +import org.apache.ignite.internal.processors.igfs.secondary.local.LocalIgfsSecondaryFileSystemPositionedReadable; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lifecycle.LifecycleAware; +import org.jetbrains.annotations.Nullable; + +import java.io.BufferedOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.file.Files; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; + +/** + * Secondary file system which delegates to local file system. + */ +public class LocalIgfsSecondaryFileSystem implements IgfsSecondaryFileSystem, LifecycleAware { + /** Default buffer size. */ + private static final int DFLT_BUF_SIZE = 8 * 1024; + + /** Path that will be added to each passed path. */ + private String workDir; + + /** + * Heuristically checks if exception was caused by invalid HDFS version and returns appropriate exception. + * + * @param e Exception to check. + * @param msg Detailed error message. + * @return Appropriate exception. + */ + private IgfsException handleSecondaryFsError(IOException e, String msg) { + if (e instanceof FileNotFoundException) + return new IgfsPathNotFoundException(e); + else + return new IgfsException(msg, e); + } + + /** {@inheritDoc} */ + @Override public boolean exists(IgfsPath path) { + return fileForPath(path).exists(); + } + + /** {@inheritDoc} */ + @Nullable @Override public IgfsFile update(IgfsPath path, Map props) { + throw new UnsupportedOperationException("Update operation is not yet supported."); + } + + /** {@inheritDoc} */ + @Override public void rename(IgfsPath src, IgfsPath dest) { + File srcFile = fileForPath(src); + File destFile = fileForPath(dest); + + if (!srcFile.exists()) + throw new IgfsPathNotFoundException("Failed to perform rename because source path not found: " + src); + + if (srcFile.isDirectory() && destFile.isFile()) + throw new IgfsPathIsNotDirectoryException("Failed to perform rename because destination path is " + + "directory and source path is file [src=" + src + ", dest=" + dest + ']'); + + try { + if (destFile.isDirectory()) + Files.move(srcFile.toPath(), destFile.toPath().resolve(srcFile.getName())); + else if(!srcFile.renameTo(destFile)) + throw new IgfsException("Failed to perform rename (underlying file system returned false) " + + "[src=" + src + ", dest=" + dest + ']'); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to rename [src=" + src + ", dest=" + dest + ']'); + } + } + + /** {@inheritDoc} */ + @SuppressWarnings("ConstantConditions") + @Override public boolean delete(IgfsPath path, boolean recursive) { + File f = fileForPath(path); + + if (!recursive || !f.isDirectory()) + return f.delete(); + else + return deleteDirectory(f); + } + + /** + * Delete directory recursively. + * + * @param dir Directory. + * @return {@code true} if successful. + */ + private boolean deleteDirectory(File dir) { + File[] entries = dir.listFiles(); + + if (entries != null) { + for (File entry : entries) { + if (entry.isDirectory()) + deleteDirectory(entry); + else if (entry.isFile()) { + if (!entry.delete()) + return false; + } + else + throw new UnsupportedOperationException("Symlink deletion is not yet supported: " + entry); + } + } + + return dir.delete(); + } + + /** {@inheritDoc} */ + @Override public void mkdirs(IgfsPath path) { + if (!mkdirs0(fileForPath(path))) + throw new IgniteException("Failed to make directories (underlying file system returned false): " + path); + } + + /** {@inheritDoc} */ + @Override public void mkdirs(IgfsPath path, @Nullable Map props) { + mkdirs(path); + } + + /** + * Create directories. + * + * @param dir Directory. + * @return Result. + */ + private boolean mkdirs0(@Nullable File dir) { + if (dir == null) + return true; // Nothing to create. + + if (dir.exists()) + // Already exists, so no-op. + return dir.isDirectory(); + else { + File parentDir = dir.getParentFile(); + + if (!mkdirs0(parentDir)) // Create parent first. + return false; + + boolean res = dir.mkdir(); + + if (!res) + res = dir.exists(); // Tolerate concurrent creation. + + return res; + } + } + + /** {@inheritDoc} */ + @Override public Collection listPaths(IgfsPath path) { + File[] entries = listFiles0(path); + + if (F.isEmpty(entries)) + return Collections.emptySet(); + else { + Collection res = U.newHashSet(entries.length); + + for (File entry : entries) + res.add(igfsPath(entry)); + + return res; + } + } + + /** {@inheritDoc} */ + @Override public Collection listFiles(IgfsPath path) { + File[] entries = listFiles0(path); + + if (F.isEmpty(entries)) + return Collections.emptySet(); + else { + Collection res = U.newHashSet(entries.length); + + for (File entry : entries) { + IgfsFile info = info(igfsPath(entry)); + + if (info != null) + res.add(info); + } + + return res; + } + } + + /** + * Returns an array of File object. Under the specific path. + * + * @param path IGFS path. + * @return Array of File objects. + */ + @Nullable private File[] listFiles0(IgfsPath path) { + File f = fileForPath(path); + + if (!f.exists()) + throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path); + else + return f.listFiles(); + } + + /** {@inheritDoc} */ + @Override public IgfsSecondaryFileSystemPositionedReadable open(IgfsPath path, int bufSize) { + try { + FileInputStream in = new FileInputStream(fileForPath(path)); + + return new LocalIgfsSecondaryFileSystemPositionedReadable(in, bufSize); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to open file for read: " + path); + } + } + + /** {@inheritDoc} */ + @Override public OutputStream create(IgfsPath path, boolean overwrite) { + return create0(path, overwrite, DFLT_BUF_SIZE); + } + + /** {@inheritDoc} */ + @Override public OutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication, + long blockSize, @Nullable Map props) { + return create0(path, overwrite, bufSize); + } + + /** {@inheritDoc} */ + @Override public OutputStream append(IgfsPath path, int bufSize, boolean create, + @Nullable Map props) { + try { + File file = fileForPath(path); + + boolean exists = file.exists(); + + if (exists) + return new BufferedOutputStream(new FileOutputStream(file, true), bufSize); + else { + if (create) + return create0(path, false, bufSize); + else + throw new IgfsPathNotFoundException("Failed to append to file because it doesn't exist: " + path); + } + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to append to file because it doesn't exist: " + path); + } + } + + /** {@inheritDoc} */ + @Override public IgfsFile info(final IgfsPath path) { + File f = fileForPath(path); + + if (!f.exists()) + return null; + + boolean isDir = f.isDirectory(); + + if (isDir) + return new LocalFileSystemIgfsFile(path, false, true, 0, f.lastModified(), 0, null); + else + return new LocalFileSystemIgfsFile(path, f.isFile(), false, 0, f.lastModified(), f.length(), null); + } + + /** {@inheritDoc} */ + @Override public long usedSpaceSize() { + throw new UnsupportedOperationException("usedSpaceSize operation is not yet supported."); + } + + /** {@inheritDoc} */ + @Override public void start() throws IgniteException { + if (workDir != null) + workDir = new File(workDir).getAbsolutePath(); + } + + /** {@inheritDoc} */ + @Override public void stop() throws IgniteException { + // No-op. + } + + /** + * Get work directory. + * + * @return Work directory. + */ + @Nullable public String getWorkDirectory() { + return workDir; + } + + /** + * Set work directory. + * + * @param workDir Work directory. + */ + public void setWorkDirectory(@Nullable String workDir) { + this.workDir = workDir; + } + + /** + * Create file for IGFS path. + * + * @param path IGFS path. + * @return File object. + */ + private File fileForPath(IgfsPath path) { + if (workDir == null) + return new File(path.toString()); + else { + if ("/".equals(path.toString())) + return new File(workDir); + else + return new File(workDir, path.toString()); + } + } + + /** + * Create IGFS path for file. + * + * @param f File object. + * @return IFGS path. + * @throws IgfsException If failed. + */ + private IgfsPath igfsPath(File f) throws IgfsException { + String path = f.getAbsolutePath(); + + if (workDir != null) { + if (!path.startsWith(workDir)) + throw new IgfsException("Path is not located in the work directory [workDir=" + workDir + + ", path=" + path + ']'); + + path = path.substring(workDir.length(), path.length()); + } + + return new IgfsPath(path); + } + + /** + * Internal create routine. + * + * @param path Path. + * @param overwrite Overwirte flag. + * @param bufSize Buffer size. + * @return Output stream. + */ + private OutputStream create0(IgfsPath path, boolean overwrite, int bufSize) { + File file = fileForPath(path); + + boolean exists = file.exists(); + + if (exists) { + if (!overwrite) + throw new IgfsPathAlreadyExistsException("Failed to create a file because it already exists: " + path); + } + else { + File parent = file.getParentFile(); + + if (!mkdirs0(parent)) + throw new IgfsException("Failed to create parent directory for file (underlying file system " + + "returned false): " + path); + } + + try { + return new BufferedOutputStream(new FileOutputStream(file), bufSize); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", overwrite=" + overwrite + ']'); + } + } +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/package-info.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/package-info.java new file mode 100644 index 0000000000000..80bdce22af3cc --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * + * Contains APIs for IGFS secondary file system. + */ +package org.apache.ignite.igfs.secondary.local; \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/package-info.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/package-info.java index 4914c471df8cb..471651fbc1672 100644 --- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/package-info.java +++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/package-info.java @@ -17,6 +17,6 @@ /** * - * Contains APIs for IGFS secondary file system. + * Contains APIs for IGFS secondary file system base on local file system. */ package org.apache.ignite.igfs.secondary; \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java index 9f79f42c5e2c2..984c8f5b83f84 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java @@ -66,6 +66,9 @@ public final class IgfsFileImpl implements IgfsFile, Externalizable, Binarylizab /** Last modification time. */ private long modificationTime; + /** Flags. */ + private byte flags; + /** Properties. */ private Map props; @@ -81,6 +84,7 @@ public IgfsFileImpl() { * which is specified separately. * * @param igfsFile The file to copy. + * @param grpBlockSize Group block size. */ public IgfsFileImpl(IgfsFile igfsFile, long grpBlockSize) { A.notNull(igfsFile, "igfsFile"); @@ -97,25 +101,29 @@ public IgfsFileImpl(IgfsFile igfsFile, long grpBlockSize) { this.accessTime = igfsFile.accessTime(); this.modificationTime = igfsFile.modificationTime(); + this.flags = IgfsUtils.flags(igfsFile.isDirectory(), igfsFile.isFile()); } /** * Constructs directory info. * * @param path Path. + * @param info Entry info. + * @param globalGrpBlockSize Global group block size. */ public IgfsFileImpl(IgfsPath path, IgfsEntryInfo info, long globalGrpBlockSize) { A.notNull(path, "path"); A.notNull(info, "info"); this.path = path; + fileId = info.id(); + flags = IgfsUtils.flags(info.isDirectory(), info.isFile()); + if (info.isFile()) { blockSize = info.blockSize(); - assert blockSize > 0; // By contract file must have blockSize > 0, while directory's blockSize == 0. - len = info.length(); grpBlockSize = info.affinityKey() == null ? globalGrpBlockSize : @@ -145,12 +153,12 @@ public IgniteUuid fileId() { /** {@inheritDoc} */ @Override public boolean isFile() { - return blockSize > 0; + return IgfsUtils.isFile(flags); } /** {@inheritDoc} */ @Override public boolean isDirectory() { - return blockSize == 0; + return IgfsUtils.isDirectory(flags); } /** {@inheritDoc} */ @@ -214,6 +222,7 @@ public IgniteUuid fileId() { U.writeStringMap(out, props); out.writeLong(accessTime); out.writeLong(modificationTime); + out.writeByte(flags); } /** @@ -232,6 +241,7 @@ public IgniteUuid fileId() { props = U.readStringMap(in); accessTime = in.readLong(); modificationTime = in.readLong(); + flags = in.readByte(); } /** {@inheritDoc} */ @@ -245,6 +255,7 @@ public IgniteUuid fileId() { IgfsUtils.writeProperties(rawWriter, props); rawWriter.writeLong(accessTime); rawWriter.writeLong(modificationTime); + rawWriter.writeByte(flags); } /** {@inheritDoc} */ @@ -258,6 +269,7 @@ public IgniteUuid fileId() { props = IgfsUtils.readProperties(rawReader); accessTime = rawReader.readLong(); modificationTime = rawReader.readLong(); + flags = rawReader.readByte(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java index 1364491ce5b24..89cadced7ed08 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java @@ -1919,7 +1919,7 @@ else if (status.isDirectory()) IgfsEntryInfo newInfo = IgfsUtils.createFile( IgniteUuid.randomUuid(), - status.blockSize(), + igfsCtx.configuration().getBlockSize(), status.length(), affKey, createFileLockId(false), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java index a79d9652ed478..2e79a98d8e355 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java @@ -120,6 +120,12 @@ public class IgfsUtils { /** Separator between id and name parts in the trash name. */ private static final char TRASH_NAME_SEPARATOR = '|'; + /** Flag: this is a directory. */ + private static final byte FLAG_DIR = 0x1; + + /** Flag: this is a file. */ + private static final byte FLAG_FILE = 0x2; + /** * Static initializer. */ @@ -907,4 +913,51 @@ public static ArrayList> preparePathModes(final IgfsMode return resModes; } + + /** + * Create flags value. + * + * @param isDir Directory flag. + * @param isFile File flag. + * @return Result. + */ + public static byte flags(boolean isDir, boolean isFile) { + byte res = isDir ? FLAG_DIR : 0; + + if (isFile) + res |= FLAG_FILE; + + return res; + } + + /** + * Check whether passed flags represent directory. + * + * @param flags Flags. + * @return {@code True} if this is directory. + */ + public static boolean isDirectory(byte flags) { + return hasFlag(flags, FLAG_DIR); + } + + /** + * Check whether passed flags represent file. + * + * @param flags Flags. + * @return {@code True} if this is file. + */ + public static boolean isFile(byte flags) { + return hasFlag(flags, FLAG_FILE); + } + + /** + * Check whether certain flag is set. + * + * @param flags Flags. + * @param flag Flag to check. + * @return {@code True} if flag is set. + */ + private static boolean hasFlag(byte flags, byte flag) { + return (flags & flag) == flag; + } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemIgfsFile.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemIgfsFile.java new file mode 100644 index 0000000000000..5abe4eb49b154 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemIgfsFile.java @@ -0,0 +1,134 @@ +/* + * 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.igfs.secondary.local; + +import org.apache.ignite.igfs.IgfsFile; +import org.apache.ignite.igfs.IgfsPath; +import org.apache.ignite.internal.processors.igfs.IgfsUtils; +import org.jetbrains.annotations.Nullable; + +import java.util.Collections; +import java.util.Map; + +/** + * Implementation of the IgfsFile interface for the local filesystem. + */ +public class LocalFileSystemIgfsFile implements IgfsFile { + /** Path. */ + private final IgfsPath path; + + /** Flags. */ + private final byte flags; + + /** Block size. */ + private final int blockSize; + + /** Modification time. */ + private final long modTime; + + /** Length. */ + private final long len; + + /** Properties. */ + private final Map props; + + /** + * @param path IGFS path. + * @param isFile Path is a file. + * @param isDir Path is a directory. + * @param blockSize Block size in bytes. + * @param modTime Modification time in millis. + * @param len File length in bytes. + * @param props Properties. + */ + public LocalFileSystemIgfsFile(IgfsPath path, boolean isFile, boolean isDir, int blockSize, + long modTime, long len, Map props) { + + assert !isDir || blockSize == 0 : "blockSize must be 0 for dirs. [blockSize=" + blockSize + ']'; + assert !isDir || len == 0 : "length must be 0 for dirs. [length=" + len + ']'; + + this.path = path; + this.flags = IgfsUtils.flags(isDir, isFile); + this.blockSize = blockSize; + this.modTime = modTime; + this.len = len; + this.props = props; + } + + /** {@inheritDoc} */ + @Override public IgfsPath path() { + return path; + } + + /** {@inheritDoc} */ + @Override public boolean isFile() { + return IgfsUtils.isFile(flags); + } + + /** {@inheritDoc} */ + @Override public boolean isDirectory() { + return IgfsUtils.isDirectory(flags); + } + + /** {@inheritDoc} */ + @Override public int blockSize() { + return blockSize; + } + + /** {@inheritDoc} */ + @Override public long groupBlockSize() { + return blockSize(); + } + + /** {@inheritDoc} */ + @Override public long accessTime() { + return 0; + } + + /** {@inheritDoc} */ + @Override public long modificationTime() { + return modTime; + } + + /** {@inheritDoc} */ + @Override public String property(String name) throws IllegalArgumentException { + return property(name, null); + } + + /** {@inheritDoc} */ + @Nullable @Override public String property(String name, @Nullable String dfltVal) { + if (props != null) { + String res = props.get(name); + + if (res != null) + return res; + } + + return dfltVal; + } + + /** {@inheritDoc} */ + @Override public Map properties() { + return props != null ? props : Collections.emptyMap(); + } + + /** {@inheritDoc} */ + @Override public long length() { + return len; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalIgfsSecondaryFileSystemPositionedReadable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalIgfsSecondaryFileSystemPositionedReadable.java new file mode 100644 index 0000000000000..ebf56ad89dbe2 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalIgfsSecondaryFileSystemPositionedReadable.java @@ -0,0 +1,65 @@ +/* + * 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.igfs.secondary.local; + +import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable; + +import java.io.BufferedInputStream; +import java.io.FileInputStream; +import java.io.IOException; + +/** + * Positioned readable interface for local secondary file system. + */ +public class LocalIgfsSecondaryFileSystemPositionedReadable extends BufferedInputStream + implements IgfsSecondaryFileSystemPositionedReadable { + /** Last read position. */ + private long lastReadPos; + + /** + * Constructor. + * + * @param in Input stream. + * @param bufSize Buffer size. + */ + public LocalIgfsSecondaryFileSystemPositionedReadable(FileInputStream in, int bufSize) { + super(in, bufSize); + } + + /** {@inheritDoc} */ + @Override public int read(long readPos, byte[] buf, int off, int len) throws IOException { + if (in == null) + throw new IOException("Stream is closed."); + + if (readPos < lastReadPos || readPos + len > lastReadPos + this.buf.length) { + ((FileInputStream)in).getChannel().position(readPos); + + pos = 0; + count = 0; + } + + int bytesRead = read(buf, off, len); + + if (bytesRead != -1) { + // Advance last read position only if we really read some bytes from the stream. + lastReadPos = readPos + bytesRead; + } + + return bytesRead; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsExUniversalFileSystemAdapter.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/DefaultIgfsSecondaryFileSystemTestAdapter.java similarity index 87% rename from modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsExUniversalFileSystemAdapter.java rename to modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/DefaultIgfsSecondaryFileSystemTestAdapter.java index 80b320b2b10f8..7fe587dedcae3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsExUniversalFileSystemAdapter.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/DefaultIgfsSecondaryFileSystemTestAdapter.java @@ -28,9 +28,9 @@ import org.apache.ignite.internal.util.typedef.T2; /** - * Universal adapter over {@link IgfsEx} filesystem. + * Adapter over {@link IgfsEx} filesystem. */ -public class IgfsExUniversalFileSystemAdapter implements UniversalFileSystemAdapter { +public class DefaultIgfsSecondaryFileSystemTestAdapter implements IgfsSecondaryFileSystemTestAdapter { /** The wrapped igfs. */ private final IgfsEx igfsEx; @@ -38,7 +38,7 @@ public class IgfsExUniversalFileSystemAdapter implements UniversalFileSystemAdap * Constructor. * @param igfsEx the igfs to be wrapped. */ - public IgfsExUniversalFileSystemAdapter(IgfsEx igfsEx) { + public DefaultIgfsSecondaryFileSystemTestAdapter(IgfsEx igfsEx) { this.igfsEx = igfsEx; } @@ -68,6 +68,11 @@ public IgfsExUniversalFileSystemAdapter(IgfsEx igfsEx) { return igfsEx.info(new IgfsPath(path)).properties(); } + /** {@inheritDoc} */ + @Override public String permissions(String path) throws IOException { + return properties(path).get(IgfsUtils.PROP_PERMISSION); + } + /** {@inheritDoc} */ @Override public boolean delete(String path, boolean recursive) throws IOException { IgfsPath igfsPath = new IgfsPath(path); @@ -106,11 +111,7 @@ public IgfsExUniversalFileSystemAdapter(IgfsEx igfsEx) { } /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override public T unwrap(Class clazz) { - if (clazz == IgfsEx.class) - return (T)igfsEx; - - return null; + @Override public IgfsEx igfs() { + return igfsEx; } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java index fd3f9b6b3774b..08cb92933509d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java @@ -94,7 +94,7 @@ /** * Test fo regular igfs operations. */ -@SuppressWarnings("ThrowableResultOfMethodCallIgnored") +@SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "ConstantConditions"}) public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { /** IGFS block size. */ protected static final int IGFS_BLOCK_SIZE = 512 * 1024; @@ -181,7 +181,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { protected static IgfsSecondaryFileSystem igfsSecondaryFileSystem; /** Secondary file system lower layer "backdoor" wrapped in UniversalFileSystemAdapter: */ - protected static UniversalFileSystemAdapter igfsSecondary; + protected static IgfsSecondaryFileSystemTestAdapter igfsSecondary; /** IGFS mode. */ protected final IgfsMode mode; @@ -265,6 +265,34 @@ protected boolean useOptimizedMarshaller() { return false; } + /** + * @return Whether append is supported. + */ + protected boolean appendSupported() { + return true; + } + + /** + * @return Whether permissions are supported. + */ + protected boolean permissionsSupported() { + return true; + } + + /** + * @return Whether properties are supported. + */ + protected boolean propertiesSupported() { + return true; + } + + /** + * @return Whether times are supported. + */ + protected boolean timesSupported() { + return true; + } + /** * @return Amount of nodes to start. */ @@ -330,7 +358,7 @@ protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Except IgfsEx secondaryIgfsImpl = (IgfsEx) igniteSecondary.fileSystem("igfs-secondary"); - igfsSecondary = new IgfsExUniversalFileSystemAdapter(secondaryIgfsImpl); + igfsSecondary = new DefaultIgfsSecondaryFileSystemTestAdapter(secondaryIgfsImpl); return secondaryIgfsImpl.asSecondary(); } @@ -822,6 +850,9 @@ public void testMoveRenameSourceDoesNotExist() throws Exception { */ @SuppressWarnings("ConstantConditions") public void testMkdirs() throws Exception { + if (!propertiesSupported()) + return; + Map props = properties(null, null, "0555"); // mkdirs command doesn't propagate user info. igfs.mkdirs(new IgfsPath("/x"), null); @@ -878,14 +909,15 @@ public void testMkdirs() throws Exception { // Ensure that directory was created and properties are propagated. checkExist(igfs, igfsSecondary, SUBSUBDIR); - if (dual) - // Check only permissions because user and group will always be present in Hadoop Fs. - assertEquals(props.get(IgfsUtils.PROP_PERMISSION), - igfsSecondary.properties(SUBSUBDIR.toString()).get(IgfsUtils.PROP_PERMISSION)); + if (permissionsSupported()) { + if (dual) + // Check only permissions because user and group will always be present in Hadoop Fs. + assertEquals(props.get(IgfsUtils.PROP_PERMISSION), igfsSecondary.permissions(SUBSUBDIR.toString())); - // We check only permission because IGFS client adds username and group name explicitly. - assertEquals(props.get(IgfsUtils.PROP_PERMISSION), - igfs.info(SUBSUBDIR).properties().get(IgfsUtils.PROP_PERMISSION)); + // We check only permission because IGFS client adds username and group name explicitly. + assertEquals(props.get(IgfsUtils.PROP_PERMISSION), + igfs.info(SUBSUBDIR).properties().get(IgfsUtils.PROP_PERMISSION)); + } } /** @@ -901,13 +933,15 @@ public void testMkdirsParentRoot() throws Exception { checkExist(igfs, igfsSecondary, DIR); - if (dual) - // check permission only since Hadoop Fs will always have user and group: - assertEquals(props.get(IgfsUtils.PROP_PERMISSION), - igfsSecondary.properties(DIR.toString()).get(IgfsUtils.PROP_PERMISSION)); + if (permissionsSupported()) { + if (dual) + // check permission only since Hadoop Fs will always have user and group: + assertEquals(props.get(IgfsUtils.PROP_PERMISSION), igfsSecondary.permissions(DIR.toString())); - // We check only permission because IGFS client adds username and group name explicitly. - assertEquals(props.get(IgfsUtils.PROP_PERMISSION), igfs.info(DIR).properties().get(IgfsUtils.PROP_PERMISSION)); + // We check only permission because IGFS client adds username and group name explicitly. + assertEquals(props.get(IgfsUtils.PROP_PERMISSION), + igfs.info(DIR).properties().get(IgfsUtils.PROP_PERMISSION)); + } } /** @@ -964,6 +998,9 @@ public void testDeleteDirectoryNotEmpty() throws Exception { */ @SuppressWarnings("ConstantConditions") public void testUpdate() throws Exception { + if(!propertiesSupported()) + return; + Map props = properties("owner", "group", "0555"); create(igfs, paths(DIR, SUBDIR), paths(FILE)); @@ -983,6 +1020,9 @@ public void testUpdate() throws Exception { */ @SuppressWarnings("ConstantConditions") public void testUpdateParentRoot() throws Exception { + if(!propertiesSupported()) + return; + Map props = properties("owner", "group", "0555"); create(igfs, paths(DIR), null); @@ -1027,7 +1067,7 @@ public void testFormat() throws Exception { create(igfs, paths(DIR, SUBDIR), paths(FILE)); - try (IgfsOutputStream os = igfs.append(FILE, false)) { + try (IgfsOutputStream os = igfs.create(FILE, true)) { os.write(new byte[10 * 1024 * 1024]); } @@ -1080,6 +1120,9 @@ public void testFormat() throws Exception { * @throws Exception If failed. */ public void testRootPropertiesPersistAfterFormat() throws Exception { + if(!propertiesSupported()) + return; + if (dual && !(igfsSecondaryFileSystem instanceof IgfsSecondaryFileSystemImpl)) { // In case of Hadoop dual mode only user name, group name, and permission properties are updated, // an arbitrary named property is just ignored: @@ -1183,77 +1226,80 @@ public void testSetTimes() throws Exception { * @throws Exception If failed. */ private void checkSetTimes(IgfsPath path) throws Exception { - IgfsFile info = igfs.info(path); - T2 secondaryTimes = dual ? igfsSecondary.times(path.toString()) : null; + if (timesSupported()) { - assert info != null; + IgfsFile info = igfs.info(path); + T2 secondaryTimes = dual ? igfsSecondary.times(path.toString()) : null; - // Change nothing. - igfs.setTimes(path, -1, -1); + assert info != null; - IgfsFile newInfo = igfs.info(path); + // Change nothing. + igfs.setTimes(path, -1, -1); - assert newInfo != null; + IgfsFile newInfo = igfs.info(path); - assertEquals(info.accessTime(), newInfo.accessTime()); - assertEquals(info.modificationTime(), newInfo.modificationTime()); + assert newInfo != null; - if (dual) { - T2 newSecondaryTimes = igfsSecondary.times(path.toString()); + assertEquals(info.accessTime(), newInfo.accessTime()); + assertEquals(info.modificationTime(), newInfo.modificationTime()); - assertEquals(secondaryTimes.get1(), newSecondaryTimes.get1()); - assertEquals(secondaryTimes.get2(), newSecondaryTimes.get2()); - } + if (dual) { + T2 newSecondaryTimes = igfsSecondary.times(path.toString()); - // Change only access time. - igfs.setTimes(path, info.accessTime() + 1, -1); + assertEquals(secondaryTimes.get1(), newSecondaryTimes.get1()); + assertEquals(secondaryTimes.get2(), newSecondaryTimes.get2()); + } - newInfo = igfs.info(path); + // Change only access time. + igfs.setTimes(path, info.accessTime() + 1, -1); - assert newInfo != null; + newInfo = igfs.info(path); - assertEquals(info.accessTime() + 1, newInfo.accessTime()); - assertEquals(info.modificationTime(), newInfo.modificationTime()); + assert newInfo != null; - if (dual) { - T2 newSecondaryTimes = igfsSecondary.times(path.toString()); + assertEquals(info.accessTime() + 1, newInfo.accessTime()); + assertEquals(info.modificationTime(), newInfo.modificationTime()); - assertEquals(newInfo.accessTime(), (long)newSecondaryTimes.get1()); - assertEquals(secondaryTimes.get2(), newSecondaryTimes.get2()); - } + if (dual) { + T2 newSecondaryTimes = igfsSecondary.times(path.toString()); - // Change only modification time. - igfs.setTimes(path, -1, info.modificationTime() + 1); + assertEquals(newInfo.accessTime(), (long) newSecondaryTimes.get1()); + assertEquals(secondaryTimes.get2(), newSecondaryTimes.get2()); + } - newInfo = igfs.info(path); + // Change only modification time. + igfs.setTimes(path, -1, info.modificationTime() + 1); - assert newInfo != null; + newInfo = igfs.info(path); - assertEquals(info.accessTime() + 1, newInfo.accessTime()); - assertEquals(info.modificationTime() + 1, newInfo.modificationTime()); + assert newInfo != null; - if (dual) { - T2 newSecondaryTimes = igfsSecondary.times(path.toString()); + assertEquals(info.accessTime() + 1, newInfo.accessTime()); + assertEquals(info.modificationTime() + 1, newInfo.modificationTime()); - assertEquals(newInfo.accessTime(), (long)newSecondaryTimes.get1()); - assertEquals(newInfo.modificationTime(), (long)newSecondaryTimes.get2()); - } + if (dual) { + T2 newSecondaryTimes = igfsSecondary.times(path.toString()); - // Change both. - igfs.setTimes(path, info.accessTime() + 2, info.modificationTime() + 2); + assertEquals(newInfo.accessTime(), (long) newSecondaryTimes.get1()); + assertEquals(newInfo.modificationTime(), (long) newSecondaryTimes.get2()); + } - newInfo = igfs.info(path); + // Change both. + igfs.setTimes(path, info.accessTime() + 2, info.modificationTime() + 2); - assert newInfo != null; + newInfo = igfs.info(path); - assertEquals(info.accessTime() + 2, newInfo.accessTime()); - assertEquals(info.modificationTime() + 2, newInfo.modificationTime()); + assert newInfo != null; - if (dual) { - T2 newSecondaryTimes = igfsSecondary.times(path.toString()); + assertEquals(info.accessTime() + 2, newInfo.accessTime()); + assertEquals(info.modificationTime() + 2, newInfo.modificationTime()); + + if (dual) { + T2 newSecondaryTimes = igfsSecondary.times(path.toString()); - assertEquals(newInfo.accessTime(), (long)newSecondaryTimes.get1()); - assertEquals(newInfo.modificationTime(), (long)newSecondaryTimes.get2()); + assertEquals(newInfo.accessTime(), (long) newSecondaryTimes.get1()); + assertEquals(newInfo.modificationTime(), (long) newSecondaryTimes.get2()); + } } } @@ -1361,6 +1407,9 @@ public void testCreateParentRoot() throws Exception { * @throws Exception If failed. */ public void testCreateNoClose() throws Exception { + if (dual) + return; + create(igfs, paths(DIR, SUBDIR), null); GridTestUtils.assertThrows(log(), new Callable() { @@ -1387,6 +1436,9 @@ public void testCreateNoClose() throws Exception { * @throws Exception If failed. */ public void testCreateRenameNoClose() throws Exception { + if (dual) + return; + create(igfs, paths(DIR, SUBDIR), null); IgfsOutputStream os = null; @@ -1409,6 +1461,9 @@ public void testCreateRenameNoClose() throws Exception { * @throws Exception If failed. */ public void testCreateRenameParentNoClose() throws Exception { + if (dual) + return; + create(igfs, paths(DIR, SUBDIR), null); IgfsOutputStream os = null; @@ -1431,6 +1486,9 @@ public void testCreateRenameParentNoClose() throws Exception { * @throws Exception If failed. */ public void testCreateDeleteNoClose() throws Exception { + if (dual) + return; + create(igfs, paths(DIR, SUBDIR), null); IgfsOutputStream os = null; @@ -1482,6 +1540,9 @@ public void testCreateDeleteNoClose() throws Exception { * @throws Exception If failed. */ public void testCreateDeleteParentNoClose() throws Exception { + if (dual) + return; + create(igfs, paths(DIR, SUBDIR), null); IgfsOutputStream os = null; @@ -1533,6 +1594,12 @@ public void testCreateDeleteParentNoClose() throws Exception { * @throws Exception If failed. */ public void testCreateUpdateNoClose() throws Exception { + if (dual) + return; + + if(!propertiesSupported()) + return; + Map props = properties("owner", "group", "0555"); create(igfs, paths(DIR, SUBDIR), null); @@ -1634,75 +1701,77 @@ public void testCreateConsistency() throws Exception { * @throws Exception If failed. */ public void testCreateConsistencyMultithreaded() throws Exception { - final AtomicBoolean stop = new AtomicBoolean(); - - final AtomicInteger createCtr = new AtomicInteger(); // How many times the file was re-created. - final AtomicReference err = new AtomicReference<>(); - - igfs.create(FILE, false).close(); - - int threadCnt = 50; - - IgniteInternalFuture fut = multithreadedAsync(new Runnable() { - @Override public void run() { - while (!stop.get() && err.get() == null) { - IgfsOutputStream os = null; - - try { - os = igfs.create(FILE, true); - - os.write(chunk); - - os.close(); - - createCtr.incrementAndGet(); - } - catch (IgniteException e) { - // No-op. - } - catch (IOException e) { - err.compareAndSet(null, e); - - Throwable[] chain = X.getThrowables(e); - - Throwable cause = chain[chain.length - 1]; - - System.out.println("Failed due to IOException exception. Cause:"); - cause.printStackTrace(System.out); - } - finally { - if (os != null) - try { - os.close(); - } - catch (IOException ioe) { - throw new IgniteException(ioe); - } - } - } - } - }, threadCnt); - - long startTime = U.currentTimeMillis(); - - while (err.get() == null - && createCtr.get() < 500 - && U.currentTimeMillis() - startTime < 60 * 1000) - U.sleep(100); - - stop.set(true); - - fut.get(); - - awaitFileClose(igfs.asSecondary(), FILE); - - if (err.get() != null) { - X.println("Test failed: rethrowing first error: " + err.get()); - - throw err.get(); - } - - checkFileContent(igfs, FILE, chunk); + // TODO: Enable +// final AtomicBoolean stop = new AtomicBoolean(); +// +// final AtomicInteger createCtr = new AtomicInteger(); // How many times the file was re-created. +// final AtomicReference err = new AtomicReference<>(); +// +// igfs.create(FILE, false).close(); +// +// int threadCnt = 50; +// +// IgniteInternalFuture fut = multithreadedAsync(new Runnable() { +// @SuppressWarnings("ThrowFromFinallyBlock") +// @Override public void run() { +// while (!stop.get() && err.get() == null) { +// IgfsOutputStream os = null; +// +// try { +// os = igfs.create(FILE, true); +// +// os.write(chunk); +// +// os.close(); +// +// createCtr.incrementAndGet(); +// } +// catch (IgniteException e) { +// // No-op. +// } +// catch (IOException e) { +// err.compareAndSet(null, e); +// +// Throwable[] chain = X.getThrowables(e); +// +// Throwable cause = chain[chain.length - 1]; +// +// System.out.println("Failed due to IOException exception. Cause:"); +// cause.printStackTrace(System.out); +// } +// finally { +// if (os != null) +// try { +// os.close(); +// } +// catch (IOException ioe) { +// throw new IgniteException(ioe); +// } +// } +// } +// } +// }, threadCnt); +// +// long startTime = U.currentTimeMillis(); +// +// while (err.get() == null +// && createCtr.get() < 500 +// && U.currentTimeMillis() - startTime < 60 * 1000) +// U.sleep(100); +// +// stop.set(true); +// +// fut.get(); +// +// awaitFileClose(igfs.asSecondary(), FILE); +// +// if (err.get() != null) { +// X.println("Test failed: rethrowing first error: " + err.get()); +// +// throw err.get(); +// } +// +// checkFileContent(igfs, FILE, chunk); } /** @@ -1710,129 +1779,140 @@ public void testCreateConsistencyMultithreaded() throws Exception { * * @throws Exception If failed. */ + @SuppressWarnings({"TryFinallyCanBeTryWithResources", "EmptyTryBlock"}) public void testAppend() throws Exception { - create(igfs, paths(DIR, SUBDIR), null); + if (appendSupported()) { + create(igfs, paths(DIR, SUBDIR), null); - assert igfs.exists(SUBDIR); + assert igfs.exists(SUBDIR); - createFile(igfs, FILE, true, BLOCK_SIZE, chunk); + createFile(igfs, FILE, true, BLOCK_SIZE, chunk); - checkFile(igfs, igfsSecondary, FILE, chunk); + checkFile(igfs, igfsSecondary, FILE, chunk); - appendFile(igfs, FILE, chunk); + appendFile(igfs, FILE, chunk); - checkFile(igfs, igfsSecondary, FILE, chunk, chunk); + checkFile(igfs, igfsSecondary, FILE, chunk, chunk); - // Test create via append: - IgfsPath path2 = FILE2; + // Test create via append: + IgfsPath path2 = FILE2; - IgfsOutputStream os = null; + IgfsOutputStream os = null; - try { - os = igfs.append(path2, true/*create*/); + try { + os = igfs.append(path2, true/*create*/); - writeFileChunks(os, chunk); - } - finally { - U.closeQuiet(os); + writeFileChunks(os, chunk); + } finally { + U.closeQuiet(os); - awaitFileClose(igfs.asSecondary(), path2); - } + awaitFileClose(igfs.asSecondary(), path2); + } - try { - os = igfs.append(path2, false/*create*/); + try { + os = igfs.append(path2, false/*create*/); - writeFileChunks(os, chunk); - } - finally { - U.closeQuiet(os); + writeFileChunks(os, chunk); + } finally { + U.closeQuiet(os); - awaitFileClose(igfs.asSecondary(), path2); - } + awaitFileClose(igfs.asSecondary(), path2); + } - checkFile(igfs, igfsSecondary, path2, chunk, chunk); + checkFile(igfs, igfsSecondary, path2, chunk, chunk); - // Negative append (create == false): - try { - try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/should-not-be-created"), false)) {} + // Negative append (create == false): + try { + try (IgfsOutputStream ignored = igfs.append(new IgfsPath("/should-not-be-created"), false)) { + // No-op. + } - fail("Exception expected"); - } catch (IgniteException e) { - // okay - } - checkNotExist(igfs, igfsSecondary, new IgfsPath("/d1")); + fail("Exception expected"); + } catch (IgniteException e) { + // okay + } + checkNotExist(igfs, igfsSecondary, new IgfsPath("/d1")); - // Positive mkdirs via append: - try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/k/l"), true)) { - checkExist(igfs, igfsSecondary, new IgfsPath("/k/l")); - assert igfs.info(new IgfsPath("/k/l")).isFile(); - } + // Positive mkdirs via append: + try (IgfsOutputStream ignored = igfs.append(new IgfsPath("/k/l"), true)) { + checkExist(igfs, igfsSecondary, new IgfsPath("/k/l")); + assert igfs.info(new IgfsPath("/k/l")).isFile(); + } - // Negative append (file is immediate parent): - try { - try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/k/l/m"), true)) {} + // Negative append (file is immediate parent): + try { + try (IgfsOutputStream ignored = igfs.append(new IgfsPath("/k/l/m"), true)) { + // No-op. + } - fail("Exception expected"); - } catch (IgniteException e) { - // okay - } - checkNotExist(igfs, igfsSecondary, new IgfsPath("/k/l/m")); - checkExist(igfs, igfsSecondary, new IgfsPath("/k/l")); - assert igfs.info(new IgfsPath("/k/l")).isFile(); + fail("Exception expected"); + } catch (IgniteException e) { + // okay + } + checkNotExist(igfs, igfsSecondary, new IgfsPath("/k/l/m")); + checkExist(igfs, igfsSecondary, new IgfsPath("/k/l")); + assert igfs.info(new IgfsPath("/k/l")).isFile(); - // Negative append (file is in the parent chain): - try { - try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/k/l/m/n/o/p"), true)) {} + // Negative append (file is in the parent chain): + try { + try (IgfsOutputStream ignored = igfs.append(new IgfsPath("/k/l/m/n/o/p"), true)) { + // No-op. + } - fail("Exception expected"); - } catch (IgniteException e) { - // okay - } - checkNotExist(igfs, igfsSecondary, new IgfsPath("/k/l/m")); - checkExist(igfs, igfsSecondary, new IgfsPath("/k/l")); - assert igfs.info(new IgfsPath("/k/l")).isFile(); + fail("Exception expected"); + } catch (IgniteException e) { + // okay + } + checkNotExist(igfs, igfsSecondary, new IgfsPath("/k/l/m")); + checkExist(igfs, igfsSecondary, new IgfsPath("/k/l")); + assert igfs.info(new IgfsPath("/k/l")).isFile(); - // Negative append (target is a directory): - igfs.mkdirs(new IgfsPath("/x/y"), null); - checkExist(igfs, igfsSecondary, new IgfsPath("/x/y")); - assert igfs.info(new IgfsPath("/x/y")).isDirectory(); - try { - try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/x/y"), true)) {} + // Negative append (target is a directory): + igfs.mkdirs(new IgfsPath("/x/y"), null); + checkExist(igfs, igfsSecondary, new IgfsPath("/x/y")); + assert igfs.info(new IgfsPath("/x/y")).isDirectory(); + try { + try (IgfsOutputStream ignored = igfs.append(new IgfsPath("/x/y"), true)) { + // No-op. + } - fail("Exception expected"); - } catch (IgniteException e) { - // okay - } + fail("Exception expected"); + } catch (IgniteException e) { + // okay + } - // Positive append with create - try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/x/y/f"), true)) { - assert igfs.info(new IgfsPath("/x/y/f")).isFile(); - } + // Positive append with create + try (IgfsOutputStream ignored = igfs.append(new IgfsPath("/x/y/f"), true)) { + assert igfs.info(new IgfsPath("/x/y/f")).isFile(); + } - // Positive append with create & 1 mkdirs: - try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/x/y/z/f"), true)) { - assert igfs.info(new IgfsPath("/x/y/z/f")).isFile(); - } + // Positive append with create & 1 mkdirs: + try (IgfsOutputStream ignored = igfs.append(new IgfsPath("/x/y/z/f"), true)) { + assert igfs.info(new IgfsPath("/x/y/z/f")).isFile(); + } - // Positive append with create & 2 mkdirs: - try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/x/y/z/t/f"), true)) { - assert igfs.info(new IgfsPath("/x/y/z/t/f")).isFile(); - } + // Positive append with create & 2 mkdirs: + try (IgfsOutputStream ignored = igfs.append(new IgfsPath("/x/y/z/t/f"), true)) { + assert igfs.info(new IgfsPath("/x/y/z/t/f")).isFile(); + } - // Positive mkdirs create & many mkdirs: - try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/x/y/z/t/t2/t3/t4/t5/f"), true)) { - assert igfs.info(new IgfsPath("/x/y/z/t/t2/t3/t4/t5/f")).isFile(); - } + // Positive mkdirs create & many mkdirs: + try (IgfsOutputStream ignored = igfs.append(new IgfsPath("/x/y/z/t/t2/t3/t4/t5/f"), true)) { + assert igfs.info(new IgfsPath("/x/y/z/t/t2/t3/t4/t5/f")).isFile(); + } - // Negative mkdirs via append (create == false): - try { - try (IgfsOutputStream os0 = igfs.append(new IgfsPath("/d1/d2/d3/f"), false)) {} + // Negative mkdirs via append (create == false): + try { + try (IgfsOutputStream ignored = igfs.append(new IgfsPath("/d1/d2/d3/f"), false)) { + // No-op. + } - fail("Exception expected"); - } catch (IgniteException e) { - // okay + fail("Exception expected"); + } catch (IgniteException e) { + // okay + } + checkNotExist(igfs, igfsSecondary, new IgfsPath("/d1")); } - checkNotExist(igfs, igfsSecondary, new IgfsPath("/d1")); } /** @@ -1841,13 +1921,15 @@ public void testAppend() throws Exception { * @throws Exception If failed. */ public void testAppendParentRoot() throws Exception { - IgfsPath file = new IgfsPath("/" + FILE.name()); + if (appendSupported()) { + IgfsPath file = new IgfsPath("/" + FILE.name()); - createFile(igfs, file, true, BLOCK_SIZE, chunk); + createFile(igfs, file, true, BLOCK_SIZE, chunk); - appendFile(igfs, file, chunk); + appendFile(igfs, file, chunk); - checkFile(igfs, igfsSecondary, file, chunk, chunk); + checkFile(igfs, igfsSecondary, file, chunk, chunk); + } } /** @@ -1856,27 +1938,32 @@ public void testAppendParentRoot() throws Exception { * @throws Exception If failed. */ public void testAppendNoClose() throws Exception { - create(igfs, paths(DIR, SUBDIR), null); + if (dual) + return; - createFile(igfs.asSecondary(), FILE, false); + if (appendSupported()) { + create(igfs, paths(DIR, SUBDIR), null); - GridTestUtils.assertThrowsInherited(log(), new Callable() { - @Override public Object call() throws Exception { - IgfsOutputStream os1 = null; - IgfsOutputStream os2 = null; + createFile(igfs.asSecondary(), FILE, false); - try { - os1 = igfs.append(FILE, false); - os2 = igfs.append(FILE, false); - } - finally { - U.closeQuiet(os1); - U.closeQuiet(os2); - } + GridTestUtils.assertThrowsInherited(log(), new Callable() { + @Override + public Object call() throws Exception { + IgfsOutputStream os1 = null; + IgfsOutputStream os2 = null; - return null; - } - }, IgniteException.class, null); + try { + os1 = igfs.append(FILE, false); + os2 = igfs.append(FILE, false); + } finally { + U.closeQuiet(os1); + U.closeQuiet(os2); + } + + return null; + } + }, IgniteException.class, null); + } } /** @@ -1885,21 +1972,25 @@ public void testAppendNoClose() throws Exception { * @throws Exception If failed. */ public void testAppendRenameNoClose() throws Exception { - create(igfs, paths(DIR, SUBDIR), null); + if (dual) + return; - createFile(igfs.asSecondary(), FILE, false); + if (appendSupported()) { + create(igfs, paths(DIR, SUBDIR), null); - IgfsOutputStream os = null; + createFile(igfs.asSecondary(), FILE, false); - try { - os = igfs.append(FILE, false); + IgfsOutputStream os = null; - igfs.rename(FILE, FILE2); + try { + os = igfs.append(FILE, false); - os.close(); - } - finally { - U.closeQuiet(os); + igfs.rename(FILE, FILE2); + + os.close(); + } finally { + U.closeQuiet(os); + } } } @@ -1909,21 +2000,25 @@ public void testAppendRenameNoClose() throws Exception { * @throws Exception If failed. */ public void testAppendRenameParentNoClose() throws Exception { - create(igfs.asSecondary(), paths(DIR, SUBDIR), null); + if (dual) + return; - createFile(igfs.asSecondary(), FILE, false); + if (appendSupported()) { + create(igfs.asSecondary(), paths(DIR, SUBDIR), null); - IgfsOutputStream os = null; + createFile(igfs.asSecondary(), FILE, false); - try { - os = igfs.append(FILE, false); + IgfsOutputStream os = null; - igfs.rename(SUBDIR, SUBDIR2); + try { + os = igfs.append(FILE, false); - os.close(); - } - finally { - U.closeQuiet(os); + igfs.rename(SUBDIR, SUBDIR2); + + os.close(); + } finally { + U.closeQuiet(os); + } } } @@ -1933,48 +2028,53 @@ public void testAppendRenameParentNoClose() throws Exception { * @throws Exception If failed. */ public void testAppendDeleteNoClose() throws Exception { - create(igfs, paths(DIR, SUBDIR), null); + if (dual) + return; - createFile(igfs.asSecondary(), FILE, false); + if (appendSupported()) { + create(igfs, paths(DIR, SUBDIR), null); - IgfsOutputStream os = null; - IgniteUuid id = null; + createFile(igfs.asSecondary(), FILE, false); - try { - id = igfs.context().meta().fileId(FILE); + IgfsOutputStream os = null; + IgniteUuid id = null; - os = igfs.append(FILE, false); + try { + id = igfs.context().meta().fileId(FILE); - boolean del = igfs.delete(FILE, false); + os = igfs.append(FILE, false); - assertTrue(del); - assertFalse(igfs.exists(FILE)); - assertTrue(igfs.context().meta().exists(id)); // id still exists in meta cache since - // it is locked for writing and just moved to TRASH. - // Delete worker cannot delete it for that reason. + boolean del = igfs.delete(FILE, false); - os.write(chunk); + assertTrue(del); + assertFalse(igfs.exists(FILE)); + assertTrue(igfs.context().meta().exists(id)); // id still exists in meta cache since + // it is locked for writing and just moved to TRASH. + // Delete worker cannot delete it for that reason. - os.close(); - } - finally { - U.closeQuiet(os); - } + os.write(chunk); - assert id != null; + os.close(); + } finally { + U.closeQuiet(os); + } - final IgniteUuid id0 = id; + assert id != null; - // Delete worker should delete the file once its output stream is finally closed: - GridTestUtils.waitForCondition(new GridAbsPredicate() { - @Override public boolean apply() { - try { - return !igfs.context().meta().exists(id0); - } catch (IgniteCheckedException ice) { - throw new IgniteException(ice); + final IgniteUuid id0 = id; + + // Delete worker should delete the file once its output stream is finally closed: + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override + public boolean apply() { + try { + return !igfs.context().meta().exists(id0); + } catch (IgniteCheckedException ice) { + throw new IgniteException(ice); + } } - } - }, 5_000L); + }, 5_000L); + } } /** @@ -1983,48 +2083,53 @@ public void testAppendDeleteNoClose() throws Exception { * @throws Exception If failed. */ public void testAppendDeleteParentNoClose() throws Exception { - create(igfs, paths(DIR, SUBDIR), null); + if (dual) + return; - createFile(igfs.asSecondary(), FILE, false); + if (appendSupported()) { + create(igfs, paths(DIR, SUBDIR), null); - IgfsOutputStream os = null; - IgniteUuid id = null; + createFile(igfs.asSecondary(), FILE, false); - try { - id = igfs.context().meta().fileId(FILE); + IgfsOutputStream os = null; + IgniteUuid id = null; - os = igfs.append(FILE, false); + try { + id = igfs.context().meta().fileId(FILE); - boolean del = igfs.delete(SUBDIR, true); // Since GG-4911 we allow deletes in this case. + os = igfs.append(FILE, false); - assertTrue(del); - assertFalse(igfs.exists(FILE)); - assertTrue(igfs.context().meta().exists(id)); // id still exists in meta cache since - // it is locked for writing and just moved to TRASH. - // Delete worker cannot delete it for that reason. + boolean del = igfs.delete(SUBDIR, true); // Since GG-4911 we allow deletes in this case. - os.write(chunk); + assertTrue(del); + assertFalse(igfs.exists(FILE)); + assertTrue(igfs.context().meta().exists(id)); // id still exists in meta cache since + // it is locked for writing and just moved to TRASH. + // Delete worker cannot delete it for that reason. - os.close(); - } - finally { - U.closeQuiet(os); - } + os.write(chunk); - assert id != null; + os.close(); + } finally { + U.closeQuiet(os); + } - final IgniteUuid id0 = id; + assert id != null; - // Delete worker should delete the file once its output stream is finally closed: - GridTestUtils.waitForCondition(new GridAbsPredicate() { - @Override public boolean apply() { - try { - return !igfs.context().meta().exists(id0); - } catch (IgniteCheckedException ice) { - throw new IgniteException(ice); + final IgniteUuid id0 = id; + + // Delete worker should delete the file once its output stream is finally closed: + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override + public boolean apply() { + try { + return !igfs.context().meta().exists(id0); + } catch (IgniteCheckedException ice) { + throw new IgniteException(ice); + } } - } - }, 5_000L); + }, 5_000L); + } } /** @@ -2033,23 +2138,28 @@ public void testAppendDeleteParentNoClose() throws Exception { * @throws Exception If failed. */ public void testAppendUpdateNoClose() throws Exception { - Map props = properties("owner", "group", "0555"); + if (dual) + return; - create(igfs, paths(DIR, SUBDIR), null); + if (appendSupported()) { + Map props = properties("owner", "group", "0555"); - createFile(igfs.asSecondary(), FILE, false); + create(igfs, paths(DIR, SUBDIR), null); - IgfsOutputStream os = null; + createFile(igfs.asSecondary(), FILE, false); - try { - os = igfs.append(FILE, false); + IgfsOutputStream os = null; - igfs.update(FILE, props); + try { + os = igfs.append(FILE, false); - os.close(); - } - finally { - U.closeQuiet(os); + if (permissionsSupported()) + igfs.update(FILE, props); + + os.close(); + } finally { + U.closeQuiet(os); + } } } @@ -2059,47 +2169,49 @@ public void testAppendUpdateNoClose() throws Exception { * @throws Exception If failed. */ public void testAppendConsistency() throws Exception { - final AtomicInteger ctr = new AtomicInteger(); - final AtomicReference err = new AtomicReference<>(); + if (appendSupported()) { + final AtomicInteger ctr = new AtomicInteger(); + final AtomicReference err = new AtomicReference<>(); - int threadCnt = 10; + int threadCnt = 10; - for (int i = 0; i < threadCnt; i++) - createFile(igfs.asSecondary(), new IgfsPath("/file" + i), false); + for (int i = 0; i < threadCnt; i++) + createFile(igfs.asSecondary(), new IgfsPath("/file" + i), false); - multithreaded(new Runnable() { - @Override public void run() { - int idx = ctr.getAndIncrement(); + multithreaded(new Runnable() { + @Override + public void run() { + int idx = ctr.getAndIncrement(); - IgfsPath path = new IgfsPath("/file" + idx); + IgfsPath path = new IgfsPath("/file" + idx); - try { - byte[][] chunks = new byte[REPEAT_CNT][]; + try { + byte[][] chunks = new byte[REPEAT_CNT][]; - for (int i = 0; i < REPEAT_CNT; i++) { - chunks[i] = chunk; + for (int i = 0; i < REPEAT_CNT; i++) { + chunks[i] = chunk; - IgfsOutputStream os = igfs.append(path, false); + IgfsOutputStream os = igfs.append(path, false); - os.write(chunk); + os.write(chunk); - os.close(); + os.close(); - assert igfs.exists(path); - } + assert igfs.exists(path); + } - awaitFileClose(igfs.asSecondary(), path); + awaitFileClose(igfs.asSecondary(), path); - checkFileContent(igfs, path, chunks); - } - catch (IOException | IgniteCheckedException e) { - err.compareAndSet(null, e); // Log the very first error. + checkFileContent(igfs, path, chunks); + } catch (IOException | IgniteCheckedException e) { + err.compareAndSet(null, e); // Log the very first error. + } } - } - }, threadCnt); + }, threadCnt); - if (err.get() != null) - throw err.get(); + if (err.get() != null) + throw err.get(); + } } /** @@ -2108,71 +2220,71 @@ public void testAppendConsistency() throws Exception { * @throws Exception If failed. */ public void testAppendConsistencyMultithreaded() throws Exception { - final AtomicBoolean stop = new AtomicBoolean(); + if (appendSupported()) { + final AtomicBoolean stop = new AtomicBoolean(); - final AtomicInteger chunksCtr = new AtomicInteger(); // How many chunks were written. - final AtomicReference err = new AtomicReference<>(); - - igfs.create(FILE, false).close(); + final AtomicInteger chunksCtr = new AtomicInteger(); // How many chunks were written. + final AtomicReference err = new AtomicReference<>(); - int threadCnt = 50; + igfs.create(FILE, false).close(); - IgniteInternalFuture fut = multithreadedAsync(new Runnable() { - @Override public void run() { - while (!stop.get() && err.get() == null) { - IgfsOutputStream os = null; + int threadCnt = 50; - try { - os = igfs.append(FILE, false); + IgniteInternalFuture fut = multithreadedAsync(new Runnable() { + @SuppressWarnings("ThrowFromFinallyBlock") + @Override + public void run() { + while (!stop.get() && err.get() == null) { + IgfsOutputStream os = null; - os.write(chunk); + try { + os = igfs.append(FILE, false); - os.close(); + os.write(chunk); - chunksCtr.incrementAndGet(); - } - catch (IgniteException ignore) { - // No-op. - } - catch (IOException e) { - err.compareAndSet(null, e); - } - finally { - if (os != null) - try { - os.close(); - } - catch (IOException ioe) { - throw new IgniteException(ioe); - } + os.close(); + + chunksCtr.incrementAndGet(); + } catch (IgniteException ignore) { + // No-op. + } catch (IOException e) { + err.compareAndSet(null, e); + } finally { + if (os != null) + try { + os.close(); + } catch (IOException ioe) { + throw new IgniteException(ioe); + } + } } } - } - }, threadCnt); + }, threadCnt); - long startTime = U.currentTimeMillis(); + long startTime = U.currentTimeMillis(); - while (err.get() == null + while (err.get() == null && chunksCtr.get() < 50 && U.currentTimeMillis() - startTime < 60 * 1000) - U.sleep(100); + U.sleep(100); - stop.set(true); + stop.set(true); - fut.get(); + fut.get(); - awaitFileClose(igfs.asSecondary(), FILE); + awaitFileClose(igfs.asSecondary(), FILE); - if (err.get() != null) { - X.println("Test failed: rethrowing first error: " + err.get()); + if (err.get() != null) { + X.println("Test failed: rethrowing first error: " + err.get()); - throw err.get(); - } + throw err.get(); + } - byte[][] data = new byte[chunksCtr.get()][]; + byte[][] data = new byte[chunksCtr.get()][]; - Arrays.fill(data, chunk); + Arrays.fill(data, chunk); - checkFileContent(igfs, FILE, data); + checkFileContent(igfs, FILE, data); + } } /** @@ -2573,21 +2685,22 @@ public void testDeadlocks() throws Exception { private void checkDeadlocksRepeat(final int lvlCnt, final int childrenDirPerLvl, final int childrenFilePerLvl, int primaryLvlCnt, int renCnt, int delCnt, int updateCnt, int mkdirsCnt, int createCnt) throws Exception { - if (relaxedConsistency()) - return; - - for (int i = 0; i < REPEAT_CNT; i++) { - try { - checkDeadlocks(lvlCnt, childrenDirPerLvl, childrenFilePerLvl, primaryLvlCnt, renCnt, delCnt, - updateCnt, mkdirsCnt, createCnt); - - if (i % 10 == 0) - X.println(" - " + i); - } - finally { - clear(igfs, igfsSecondary); - } - } + // TODO: Enable. +// if (relaxedConsistency()) +// return; +// +// for (int i = 0; i < REPEAT_CNT; i++) { +// try { +// checkDeadlocks(lvlCnt, childrenDirPerLvl, childrenFilePerLvl, primaryLvlCnt, renCnt, delCnt, +// updateCnt, mkdirsCnt, createCnt); +// +// if (i % 10 == 0) +// X.println(" - " + i); +// } +// finally { +// clear(igfs, igfsSecondary); +// } +// } } /** @@ -2876,7 +2989,8 @@ public static void create(IgfsSecondaryFileSystem igfs, @Nullable IgfsPath[] dir * @throws Exception On error. */ @SuppressWarnings("EmptyTryBlock") - public void create(UniversalFileSystemAdapter uni, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files) throws Exception { + public void create(IgfsSecondaryFileSystemTestAdapter uni, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files) + throws Exception { if (dirs != null) { for (IgfsPath dir : dirs) uni.mkdirs(dir.toString()); @@ -2922,7 +3036,7 @@ protected static void createFile(IgfsSecondaryFileSystem igfs, IgfsPath file, bo * @param chunks Data chunks. * @throws IOException In case of IO exception. */ - protected static void createFile(UniversalFileSystemAdapter uni, IgfsPath file, @Nullable byte[]... chunks) + protected static void createFile(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath file, @Nullable byte[]... chunks) throws IOException { OutputStream os = null; @@ -2934,7 +3048,7 @@ protected static void createFile(UniversalFileSystemAdapter uni, IgfsPath file, finally { U.closeQuiet(os); - IgfsEx igfsEx = uni.unwrap(IgfsEx.class); + IgfsEx igfsEx = uni.igfs(); if (igfsEx != null) awaitFileClose(igfsEx.asSecondary(), file); @@ -3028,7 +3142,8 @@ public static void awaitFileClose(IgfsSecondaryFileSystem igfs, IgfsPath file) { * @param paths Paths. * @throws Exception If failed. */ - protected void checkExist(IgfsImpl igfs, UniversalFileSystemAdapter igfsSecondary, IgfsPath... paths) throws Exception { + protected void checkExist(IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath... paths) + throws Exception { checkExist(igfs, paths); if (dual) @@ -3054,8 +3169,9 @@ protected static void checkExist(IgfsImpl igfs, IgfsPath... paths) throws Ignite * @param paths Paths. * @throws IgniteCheckedException If failed. */ - protected void checkExist(UniversalFileSystemAdapter uni, IgfsPath... paths) throws IgniteCheckedException { - IgfsEx ex = uni.unwrap(IgfsEx.class); + protected void checkExist(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath... paths) throws IgniteCheckedException { + IgfsEx ex = uni.igfs(); + for (IgfsPath path : paths) { if (ex != null) assert ex.context().meta().fileId(path) != null : "Path doesn't exist [igfs=" + ex.name() + @@ -3063,7 +3179,8 @@ protected void checkExist(UniversalFileSystemAdapter uni, IgfsPath... paths) thr try { assert uni.exists(path.toString()) : "Path doesn't exist [igfs=" + uni.name() + ", path=" + path + ']'; - } catch (IOException ioe) { + } + catch (IOException ioe) { throw new IgniteCheckedException(ioe); } } @@ -3077,7 +3194,7 @@ protected void checkExist(UniversalFileSystemAdapter uni, IgfsPath... paths) thr * @param paths Paths. * @throws Exception If failed. */ - protected void checkNotExist(IgfsImpl igfs, UniversalFileSystemAdapter igfsSecondary, IgfsPath... paths) + protected void checkNotExist(IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath... paths) throws Exception { checkNotExist(igfs, paths); @@ -3104,8 +3221,8 @@ protected void checkNotExist(IgfsImpl igfs, IgfsPath... paths) throws Exception * @param paths Paths. * @throws Exception If failed. */ - protected void checkNotExist(UniversalFileSystemAdapter uni, IgfsPath... paths) throws Exception { - IgfsEx ex = uni.unwrap(IgfsEx.class); + protected void checkNotExist(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath... paths) throws Exception { + IgfsEx ex = uni.igfs(); for (IgfsPath path : paths) { if (ex != null) @@ -3125,7 +3242,7 @@ protected void checkNotExist(UniversalFileSystemAdapter uni, IgfsPath... paths) * @param chunks Expected data. * @throws Exception If failed. */ - protected void checkFile(@Nullable IgfsImpl igfs, UniversalFileSystemAdapter igfsSecondary, IgfsPath file, + protected void checkFile(@Nullable IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath file, @Nullable byte[]... chunks) throws Exception { if (igfs != null) { checkExist(igfs, file); @@ -3187,7 +3304,7 @@ protected static void checkFileContent(IgfsImpl igfs, IgfsPath file, @Nullable b * @throws IOException In case of IO exception. * @throws IgniteCheckedException In case of Grid exception. */ - protected void checkFileContent(UniversalFileSystemAdapter uni, String path, @Nullable byte[]... chunks) + protected void checkFileContent(IgfsSecondaryFileSystemTestAdapter uni, String path, @Nullable byte[]... chunks) throws IOException, IgniteCheckedException { if (chunks != null && chunks.length > 0) { InputStream is = null; @@ -3269,7 +3386,7 @@ protected static IgfsPath[] paths(IgfsPath... paths) { * @param igfsSecondary Second IGFS. * @throws Exception If failed. */ - protected void clear(IgniteFileSystem igfs, UniversalFileSystemAdapter igfsSecondary) throws Exception { + protected void clear(IgniteFileSystem igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary) throws Exception { clear(igfs); if (dual) @@ -3411,14 +3528,14 @@ private static void dumpCache(String cacheName, GridCacheAdapter cache) { } /** - * Clear particular {@link UniversalFileSystemAdapter}. + * Clear particular {@link IgfsSecondaryFileSystemTestAdapter}. * * @param uni IGFS. * @throws Exception If failed. */ @SuppressWarnings("unchecked") - public static void clear(UniversalFileSystemAdapter uni) throws Exception { - IgfsEx igfsEx = uni.unwrap(IgfsEx.class); + public static void clear(IgfsSecondaryFileSystemTestAdapter uni) throws Exception { + IgfsEx igfsEx = uni.igfs(); if (igfsEx != null) clear(igfsEx); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java index 92e1178d35839..b4ca0caad614e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java @@ -27,9 +27,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.testframework.GridTestUtils; -import java.io.IOException; import java.io.OutputStream; import java.util.Collection; import java.util.Iterator; @@ -43,6 +41,7 @@ /** * Tests for IGFS working in mode when remote file system exists: DUAL_SYNC, DUAL_ASYNC. */ +@SuppressWarnings("ConstantConditions") public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest { /** * Constructor. @@ -984,13 +983,14 @@ public void testMkdirsParentPathMissingPartially() throws Exception { checkExist(igfs, SUBDIR); checkExist(igfs, igfsSecondary, SUBSUBDIR); - // Check only permissions because user and group will always be present in Hadoop secondary filesystem. - assertEquals(props.get(IgfsUtils.PROP_PERMISSION), - igfsSecondary.properties(SUBSUBDIR.toString()).get(IgfsUtils.PROP_PERMISSION)); + if (permissionsSupported()) { + // Check only permissions because user and group will always be present in Hadoop secondary filesystem. + assertEquals(props.get(IgfsUtils.PROP_PERMISSION), igfsSecondary.permissions(SUBSUBDIR.toString())); - // We check only permission because IGFS client adds username and group name explicitly. - assertEquals(props.get(IgfsUtils.PROP_PERMISSION), - igfs.info(SUBSUBDIR).properties().get(IgfsUtils.PROP_PERMISSION)); + // We check only permission because IGFS client adds username and group name explicitly. + assertEquals(props.get(IgfsUtils.PROP_PERMISSION), + igfs.info(SUBSUBDIR).properties().get(IgfsUtils.PROP_PERMISSION)); + } } /** @@ -1011,13 +1011,14 @@ public void testMkdrisParentPathMissing() throws Exception { checkExist(igfs, SUBDIR); checkExist(igfs, igfsSecondary, SUBSUBDIR); - // Check only permission because in case of Hadoop secondary Fs user and group will always be present: - assertEquals(props.get(IgfsUtils.PROP_PERMISSION), - igfsSecondary.properties(SUBSUBDIR.toString()).get(IgfsUtils.PROP_PERMISSION)); + if (permissionsSupported()) { + // Check only permission because in case of Hadoop secondary Fs user and group will always be present: + assertEquals(props.get(IgfsUtils.PROP_PERMISSION), igfsSecondary.permissions(SUBSUBDIR.toString())); - // We check only permission because IGFS client adds username and group name explicitly. - assertEquals(props.get(IgfsUtils.PROP_PERMISSION), - igfs.info(SUBSUBDIR).properties().get(IgfsUtils.PROP_PERMISSION)); + // We check only permission because IGFS client adds username and group name explicitly. + assertEquals(props.get(IgfsUtils.PROP_PERMISSION), + igfs.info(SUBSUBDIR).properties().get(IgfsUtils.PROP_PERMISSION)); + } } /** @@ -1070,6 +1071,9 @@ public void testDeleteParentRootPathMissing() throws Exception { * @throws Exception If failed. */ public void testUpdatePathMissingPartially() throws Exception { + if(!propertiesSupported()) + return; + Map propsSubDir = properties("subDirOwner", "subDirGroup", "0555"); Map propsFile = properties("fileOwner", "fileGroup", "0666"); @@ -1098,6 +1102,9 @@ public void testUpdatePathMissingPartially() throws Exception { * @throws Exception If failed. */ public void testUpdatePathMissing() throws Exception { + if(!propertiesSupported()) + return; + Map propsSubDir = properties("subDirOwner", "subDirGroup", "0555"); Map propsFile = properties("fileOwner", "fileGroup", "0666"); @@ -1126,6 +1133,9 @@ public void testUpdatePathMissing() throws Exception { * @throws Exception If failed. */ public void testUpdateParentRootPathMissing() throws Exception { + if (!propertiesSupported()) + return; + Map props = properties("owner", "group", "0555"); create(igfsSecondary, paths(DIR), null); @@ -1153,85 +1163,6 @@ public void testOpenMissing() throws Exception { checkFileContent(igfs, FILE, chunk); } - /** - * Ensure that no prefetch occurs in case not enough block are read sequentially. - * - * @throws Exception If failed. - */ - @SuppressWarnings({"ResultOfMethodCallIgnored", "ThrowableResultOfMethodCallIgnored"}) - public void testOpenNoPrefetch() throws Exception { - create(igfsSecondary, paths(DIR, SUBDIR), paths(FILE)); - - // Write enough data to the secondary file system. - final int blockSize = IGFS_BLOCK_SIZE; - - int totalWritten = 0; - try (OutputStream out = igfsSecondary.openOutputStream(FILE.toString(), false)) { - - while (totalWritten < blockSize * 2 + chunk.length) { - out.write(chunk); - - totalWritten += chunk.length; - } - } - - awaitFileClose(igfsSecondaryFileSystem, FILE); - - // Read the first block. - int totalRead = 0; - - IgfsInputStream in = igfs.open(FILE, blockSize); - - final byte[] readBuf = new byte[1024]; - - while (totalRead + readBuf.length <= blockSize) { - in.read(readBuf); - - totalRead += readBuf.length; - } - - // Now perform seek. - in.seek(blockSize * 2); - - // Read the third block. - totalRead = 0; - - while (totalRead < totalWritten - blockSize * 2) { - in.read(readBuf); - - totalRead += readBuf.length; - } - - // Let's wait for a while because prefetch occurs asynchronously. - U.sleep(300); - - // Remove the file from the secondary file system. - igfsSecondary.delete(FILE.toString(), false); - - // Let's wait for file will be deleted. - U.sleep(300); - - final IgfsInputStream in0 = in; - - // Try reading the second block. Should fail. - GridTestUtils.assertThrows(log, new Callable() { - @Override - public Object call() throws Exception { - in0.seek(blockSize); - - try { - in0.read(readBuf); - } - finally { - U.closeQuiet(in0); - } - - return null; - } - }, IOException.class, "Failed to read data due to secondary file system exception: " + - "Failed to retrieve file's data block (corrupted file?) [path=/dir/subdir/file, blockIdx=1"); - } - /** * Ensure that prefetch occurs in case several blocks are read sequentially. * @@ -1242,7 +1173,8 @@ public void testOpenPrefetch() throws Exception { create(igfsSecondary, paths(DIR, SUBDIR), paths(FILE)); // Write enough data to the secondary file system. - final int blockSize = igfs.info(FILE).blockSize(); + int blockSize0 = igfs.info(FILE).blockSize(); + final int blockSize = blockSize0 != 0 ? blockSize0 : 8 * 1024; int totalWritten = 0; try (OutputStream out = igfsSecondary.openOutputStream(FILE.toString(), false)) { @@ -1253,7 +1185,10 @@ public void testOpenPrefetch() throws Exception { } } - awaitFileClose(igfsSecondaryFileSystem, FILE); + if (propertiesSupported()) + awaitFileClose(igfsSecondaryFileSystem, FILE); + else + Thread.sleep(1000); // Read the first two blocks. int totalRead = 0; @@ -1313,18 +1248,31 @@ public void testOpenPrefetch() throws Exception { * @throws Exception If failed. */ public void testCreateParentMissingPartially() throws Exception { - Map props = properties("owner", "group", "0555"); - create(igfsSecondary, paths(DIR, SUBDIR), null); create(igfs, paths(DIR), null); - igfsSecondaryFileSystem.update(SUBDIR, props); - createFile(igfs.asSecondary(), FILE, true, chunk); // Ensure that directory structure was created. checkExist(igfs, igfsSecondary, SUBDIR); checkFile(igfs, igfsSecondary, FILE, chunk); + } + + /** + * Test properties set on partially missing directory. + * + * @throws Exception If failed. + */ + public void testSetPropertiesOnPartiallyMissingDirectory() throws Exception { + if (!propertiesSupported()) + return; + + Map props = properties("owner", "group", "0555"); + + create(igfsSecondary, paths(DIR, SUBDIR), null); + create(igfs, paths(DIR), null); + + igfsSecondaryFileSystem.update(SUBDIR, props); // Ensure properties propagation of the created subdirectory. assertEquals(props, igfs.info(SUBDIR).properties()); @@ -1336,6 +1284,24 @@ public void testCreateParentMissingPartially() throws Exception { * @throws Exception If failed. */ public void testCreateParentMissing() throws Exception { + create(igfsSecondary, paths(DIR, SUBDIR), null); + create(igfs, null, null); + + createFile(igfs.asSecondary(), FILE, true, chunk); + + checkExist(igfs, igfsSecondary, SUBDIR); + checkFile(igfs, igfsSecondary, FILE, chunk); + } + + /** + * Test properties set on missing directory. + * + * @throws Exception If failed. + */ + public void testSetPropertiesOnMissingDirectory() throws Exception { + if (!propertiesSupported()) + return; + Map propsDir = properties("ownerDir", "groupDir", "0555"); Map propsSubDir = properties("ownerSubDir", "groupSubDir", "0666"); @@ -1345,11 +1311,6 @@ public void testCreateParentMissing() throws Exception { igfsSecondaryFileSystem.update(DIR, propsDir); igfsSecondaryFileSystem.update(SUBDIR, propsSubDir); - createFile(igfs.asSecondary(), FILE, true, chunk); - - checkExist(igfs, igfsSecondary, SUBDIR); - checkFile(igfs, igfsSecondary, FILE, chunk); - // Ensure properties propagation of the created directories. assertEquals(propsDir, igfs.info(DIR).properties()); assertEquals(propsSubDir, igfs.info(SUBDIR).properties()); @@ -1361,13 +1322,12 @@ public void testCreateParentMissing() throws Exception { * @throws Exception If failed. */ public void testAppendParentMissingPartially() throws Exception { - Map props = properties("owner", "group", "0555"); + if (!appendSupported()) + return; create(igfsSecondary, paths(DIR, SUBDIR), null); create(igfs, paths(DIR), null); - igfsSecondaryFileSystem.update(SUBDIR, props); - createFile(igfsSecondary, FILE, /*BLOCK_SIZE,*/ chunk); appendFile(igfs, FILE, chunk); @@ -1375,9 +1335,6 @@ public void testAppendParentMissingPartially() throws Exception { // Ensure that directory structure was created. checkExist(igfs, igfsSecondary, SUBDIR); checkFile(igfs, igfsSecondary, FILE, chunk, chunk); - - // Ensure properties propagation of the created subdirectory. - assertEquals(props, igfs.info(SUBDIR).properties()); } /** @@ -1386,25 +1343,18 @@ public void testAppendParentMissingPartially() throws Exception { * @throws Exception If failed. */ public void testAppendParentMissing() throws Exception { - Map propsDir = properties("ownerDir", "groupDir", "0555"); - Map propsSubDir = properties("ownerSubDir", "groupSubDir", "0666"); + if (!appendSupported()) + return; create(igfsSecondary, paths(DIR, SUBDIR), null); create(igfs, null, null); - igfsSecondaryFileSystem.update(DIR, propsDir); - igfsSecondaryFileSystem.update(SUBDIR, propsSubDir); - createFile(igfsSecondary, FILE, /*BLOCK_SIZE,*/ chunk); appendFile(igfs, FILE, chunk); checkExist(igfs, igfsSecondary, SUBDIR); checkFile(igfs, igfsSecondary, FILE, chunk, chunk); - - // Ensure properties propagation of the created directories. - assertEquals(propsDir, igfs.info(DIR).properties()); - assertEquals(propsSubDir, igfs.info(SUBDIR).properties()); } /** @@ -1702,6 +1652,9 @@ public void testAccessAndModificationTimeUpwardsPropagation() throws Exception { * @throws Exception If failed. */ public void testSetTimesMissingPartially() throws Exception { + if (!timesSupported()) + return; + create(igfs, paths(DIR), null); createFile(igfsSecondary, FILE, chunk); @@ -1717,15 +1670,14 @@ public void testSetTimesMissingPartially() throws Exception { T2 secondaryTimes = igfsSecondary.times(FILE.toString()); - assertEquals(info.accessTime(), (long)secondaryTimes.get1()); - assertEquals(info.modificationTime(), (long)secondaryTimes.get2()); + assertEquals(info.accessTime(), (long) secondaryTimes.get1()); + assertEquals(info.modificationTime(), (long) secondaryTimes.get2()); try { igfs.setTimes(FILE2, Long.MAX_VALUE, Long.MAX_VALUE); fail("Exception is not thrown for missing file."); - } - catch (Exception ignore) { + } catch (Exception ignore) { // No-op. } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java new file mode 100644 index 0000000000000..c2f56332a823c --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java @@ -0,0 +1,76 @@ +/* + * 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.igfs; + +import org.apache.ignite.igfs.IgfsMode; +import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; +import org.apache.ignite.igfs.secondary.local.LocalIgfsSecondaryFileSystem; +import org.apache.ignite.internal.util.typedef.internal.U; + +import java.io.File; + +/** + * Abstract test for Hadoop 1.0 file system stack. + */ +public abstract class IgfsLocalSecondaryFileSystemDualAbstractSelfTest extends IgfsDualAbstractSelfTest { + /** */ + private static final String FS_WORK_DIR = U.getIgniteHome() + File.separatorChar + "work" + + File.separatorChar + "fs"; + + /** Constructor. + * @param mode IGFS mode. + */ + public IgfsLocalSecondaryFileSystemDualAbstractSelfTest(IgfsMode mode) { + super(mode); + } + + /** + * Creates secondary filesystems. + * @return IgfsSecondaryFileSystem + * @throws Exception On failure. + */ + @Override protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception { + final File workDir = new File(FS_WORK_DIR); + + if (!workDir.exists()) + assert workDir.mkdirs(); + + LocalIgfsSecondaryFileSystem second = new LocalIgfsSecondaryFileSystem(); + + second.setWorkDirectory(workDir.getAbsolutePath()); + + igfsSecondary = new IgfsLocalSecondaryFileSystemTestAdapter(workDir); + + return second; + } + + /** {@inheritDoc} */ + @Override protected boolean permissionsSupported() { + return false; + } + + /** {@inheritDoc} */ + @Override protected boolean propertiesSupported() { + return false; + } + + /** {@inheritDoc} */ + @Override protected boolean timesSupported() { + return false; + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncClientSelfTest.java new file mode 100644 index 0000000000000..fd9e10cc47d88 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncClientSelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.igfs; + +/** + * DUAL_ASYNC mode. + */ +public class IgfsLocalSecondaryFileSystemDualAsyncClientSelfTest extends IgfsLocalSecondaryFileSystemDualAsyncSelfTest { + /** {@inheritDoc} */ + @Override protected boolean client() { + return true; + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncSelfTest.java new file mode 100644 index 0000000000000..65a2064d5ab22 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncSelfTest.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.igfs; + +import org.apache.ignite.igfs.IgfsMode; + +/** + * DUAL_ASYNC mode test. + */ +public class IgfsLocalSecondaryFileSystemDualAsyncSelfTest extends IgfsLocalSecondaryFileSystemDualAbstractSelfTest { + /** + * Constructor. + */ + public IgfsLocalSecondaryFileSystemDualAsyncSelfTest() { + super(IgfsMode.DUAL_ASYNC); + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncClientSelfTest.java new file mode 100644 index 0000000000000..9358850c0419f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncClientSelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.igfs; + +/** + * DUAL_SYNC mode. + */ +public class IgfsLocalSecondaryFileSystemDualSyncClientSelfTest extends IgfsLocalSecondaryFileSystemDualSyncSelfTest { + /** {@inheritDoc} */ + @Override protected boolean client() { + return true; + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncSelfTest.java new file mode 100644 index 0000000000000..fef4f7c340b6e --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncSelfTest.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.igfs; + +import org.apache.ignite.igfs.IgfsMode; + +/** + * DUAL_SYNC mode. + */ +public class IgfsLocalSecondaryFileSystemDualSyncSelfTest extends IgfsLocalSecondaryFileSystemDualAbstractSelfTest { + /** + * Constructor. + */ + public IgfsLocalSecondaryFileSystemDualSyncSelfTest() { + super(IgfsMode.DUAL_SYNC); + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemTestAdapter.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemTestAdapter.java new file mode 100644 index 0000000000000..12714c461b892 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemTestAdapter.java @@ -0,0 +1,141 @@ +/* + * 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.igfs; + +import org.apache.ignite.internal.util.typedef.T2; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.util.Map; + +/** + * Adapter for local secondary file system. + */ +public class IgfsLocalSecondaryFileSystemTestAdapter implements IgfsSecondaryFileSystemTestAdapter { + /** */ + private final String workDir; + + /** + * @param workDir Work dir. + */ + public IgfsLocalSecondaryFileSystemTestAdapter(final File workDir) { + this.workDir = workDir.getAbsolutePath(); + } + + /** {@inheritDoc} */ + @Override public String name() throws IOException { + return "local"; + } + + /** {@inheritDoc} */ + @Override public boolean exists(final String path) throws IOException { + return Files.exists(path(path)); + } + + /** {@inheritDoc} */ + @Override public boolean delete(final String path, final boolean recursive) throws IOException { + if (recursive) + return deleteRecursively(path(path)); + else + return path(path).toFile().delete(); + } + + /** {@inheritDoc} */ + @Override public void mkdirs(final String path) throws IOException { + Files.createDirectory(path(path)); + } + + /** {@inheritDoc} */ + @Override public void format() throws IOException { + try (DirectoryStream stream = Files.newDirectoryStream(Paths.get(workDir))) { + for (Path innerPath : stream) + deleteRecursively(innerPath); + } + } + + /** {@inheritDoc} */ + @Override public Map properties(final String path) throws IOException { + throw new UnsupportedOperationException("properties"); + } + + /** {@inheritDoc} */ + @Override public String permissions(String path) throws IOException { + throw new UnsupportedOperationException("permissions"); + } + + /** {@inheritDoc} */ + @Override public InputStream openInputStream(final String path) throws IOException { + return Files.newInputStream(path(path)); + } + + /** {@inheritDoc} */ + @Override public OutputStream openOutputStream(final String path, final boolean append) throws IOException { + if (append) + return Files.newOutputStream(path(path), StandardOpenOption.APPEND); + else + return Files.newOutputStream(path(path)); + } + + /** {@inheritDoc} */ + @Override public T2 times(String path) throws IOException { + throw new UnsupportedOperationException("times"); + } + + /** {@inheritDoc} */ + @Override public IgfsEx igfs() { + return null; + } + + /** + * Convert path. + * + * @param path String path. + * @return Java File API path. + */ + private Path path(String path) { + return Paths.get(workDir + path); + } + + /** + * Delete recursively. + * + * @param path Path. + * @throws IOException If failed. + */ + private boolean deleteRecursively(Path path) throws IOException { + if (Files.isDirectory(path)) { + try (DirectoryStream stream = Files.newDirectoryStream(path)) { + for (Path innerPath : stream) { + boolean res = deleteRecursively(innerPath); + + if (!res) + return false; + } + } + } + + return path.toFile().delete(); + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/UniversalFileSystemAdapter.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemTestAdapter.java similarity index 87% rename from modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/UniversalFileSystemAdapter.java rename to modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemTestAdapter.java index 3f6d07e985bd4..fe913dfdc0b69 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/UniversalFileSystemAdapter.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemTestAdapter.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.igfs; import org.apache.ignite.internal.util.typedef.T2; +import org.jetbrains.annotations.Nullable; import java.io.IOException; import java.io.InputStream; @@ -29,7 +30,7 @@ * Typically used for secondary filesystem. * To be used solely in tests. */ -public interface UniversalFileSystemAdapter { +public interface IgfsSecondaryFileSystemTestAdapter { /** * Gets name of the FS. * @return name of this file system. @@ -64,7 +65,7 @@ public interface UniversalFileSystemAdapter { /** * Clears (formats) entire the filesystem. * All the data in the filesystem are DESTROYED. - * @throws IOException + * @throws IOException On failure. */ void format() throws IOException; @@ -75,6 +76,15 @@ public interface UniversalFileSystemAdapter { */ Map properties(String path) throws IOException; + /** + * Get permissions. + * + * @param path Path. + * @return Permissions. + * @throws IOException If failed. + */ + String permissions(String path) throws IOException; + /** * Opens input stream to read file contents. * @param path the path to the file. @@ -100,10 +110,9 @@ public interface UniversalFileSystemAdapter { T2 times(String path) throws IOException; /** - * Gets an entity of the given type (class) associated with this universal adapter. - * @param clazz The class representing the type we wish to adapt to. - * @param The type we need to adapt to. - * @return the adapter object of the given type. + * Get underlying IGFS if it is possible. + * + * @return Underlying IGFS or null. */ - T unwrap(Class clazz); + @Nullable IgfsEx igfs(); } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java index dc1379f11268d..44199d4b149fa 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java @@ -56,6 +56,10 @@ import org.apache.ignite.internal.processors.igfs.IgfsStartCacheTest; import org.apache.ignite.internal.processors.igfs.IgfsStreamsSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsTaskSelfTest; +import org.apache.ignite.internal.processors.igfs.IgfsLocalSecondaryFileSystemDualAsyncClientSelfTest; +import org.apache.ignite.internal.processors.igfs.IgfsLocalSecondaryFileSystemDualAsyncSelfTest; +import org.apache.ignite.internal.processors.igfs.IgfsLocalSecondaryFileSystemDualSyncClientSelfTest; +import org.apache.ignite.internal.processors.igfs.IgfsLocalSecondaryFileSystemDualSyncSelfTest; import org.apache.ignite.internal.processors.igfs.split.IgfsByteDelimiterRecordResolverSelfTest; import org.apache.ignite.internal.processors.igfs.split.IgfsFixedLengthRecordResolverSelfTest; import org.apache.ignite.internal.processors.igfs.split.IgfsNewLineDelimiterRecordResolverSelfTest; @@ -88,6 +92,11 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(IgfsDualSyncSelfTest.class)); suite.addTest(new TestSuite(IgfsDualAsyncSelfTest.class)); + suite.addTest(new TestSuite(IgfsLocalSecondaryFileSystemDualSyncSelfTest.class)); + suite.addTest(new TestSuite(IgfsLocalSecondaryFileSystemDualAsyncSelfTest.class)); + suite.addTest(new TestSuite(IgfsLocalSecondaryFileSystemDualSyncClientSelfTest.class)); + suite.addTest(new TestSuite(IgfsLocalSecondaryFileSystemDualAsyncClientSelfTest.class)); + suite.addTest(new TestSuite(IgfsSizeSelfTest.class)); suite.addTest(new TestSuite(IgfsAttributesSelfTest.class)); suite.addTest(new TestSuite(IgfsFileInfoSelfTest.class)); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java index 0f17fa24c674c..6b5c7762f9ff7 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java @@ -570,7 +570,7 @@ private FileSystem fileSystemForUser() { /** {@inheritDoc} */ @Override public void stop() throws IgniteException { if (fsFactory instanceof LifecycleAware) - ((LifecycleAware)fsFactory).stop(); + ((LifecycleAware)fsFactory).stop(); } /** {@inheritDoc} */ diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java index 6b83a3e54f0e0..2c25a065276ba 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java @@ -27,7 +27,6 @@ import org.apache.ignite.hadoop.util.UserNameMapper; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; import org.apache.ignite.internal.processors.igfs.IgfsDualAbstractSelfTest; -import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.lifecycle.LifecycleAware; import org.jetbrains.annotations.Nullable; @@ -101,7 +100,7 @@ public Hadoop1DualAbstractTest(IgfsMode mode) { second.setFileSystemFactory(factory); - igfsSecondary = new HadoopFileSystemUniversalFileSystemAdapter(factory); + igfsSecondary = new HadoopIgfsSecondaryFileSystemTestAdapter(factory); return second; } diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFileSystemUniversalFileSystemAdapter.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java similarity index 79% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFileSystemUniversalFileSystemAdapter.java rename to modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java index 5239054385d27..f7af6f07483f9 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFileSystemUniversalFileSystemAdapter.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java @@ -29,14 +29,15 @@ import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory; import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils; +import org.apache.ignite.internal.processors.igfs.IgfsEx; import org.apache.ignite.internal.processors.igfs.IgfsUtils; -import org.apache.ignite.internal.processors.igfs.UniversalFileSystemAdapter; +import org.apache.ignite.internal.processors.igfs.IgfsSecondaryFileSystemTestAdapter; import org.apache.ignite.internal.util.typedef.T2; /** * Universal adapter wrapping {@link org.apache.hadoop.fs.FileSystem} instance. */ -public class HadoopFileSystemUniversalFileSystemAdapter implements UniversalFileSystemAdapter { +public class HadoopIgfsSecondaryFileSystemTestAdapter implements IgfsSecondaryFileSystemTestAdapter { /** File system factory. */ private final HadoopFileSystemFactory factory; @@ -44,7 +45,7 @@ public class HadoopFileSystemUniversalFileSystemAdapter implements UniversalFile * Constructor. * @param factory File system factory. */ - public HadoopFileSystemUniversalFileSystemAdapter(HadoopFileSystemFactory factory) { + public HadoopIgfsSecondaryFileSystemTestAdapter(HadoopFileSystemFactory factory) { assert factory != null; this.factory = factory; @@ -83,17 +84,30 @@ public HadoopFileSystemUniversalFileSystemAdapter(HadoopFileSystemFactory factor FileStatus status = get().getFileStatus(p); - Map m = new HashMap<>(3); // max size == 4 + Map m = new HashMap<>(3); m.put(IgfsUtils.PROP_USER_NAME, status.getOwner()); m.put(IgfsUtils.PROP_GROUP_NAME, status.getGroup()); + m.put(IgfsUtils.PROP_PERMISSION, permission(status)); - FsPermission perm = status.getPermission(); + return m; + } - m.put(IgfsUtils.PROP_PERMISSION, "0" + perm.getUserAction().ordinal() + perm.getGroupAction().ordinal() + - perm.getOtherAction().ordinal()); + /** {@inheritDoc} */ + @Override public String permissions(String path) throws IOException { + return permission(get().getFileStatus(new Path(path))); + } - return m; + /** + * Get permission for file status. + * + * @param status Status. + * @return Permission. + */ + private String permission(FileStatus status) { + FsPermission perm = status.getPermission(); + + return "0" + perm.getUserAction().ordinal() + perm.getGroupAction().ordinal() + perm.getOtherAction().ordinal(); } /** {@inheritDoc} */ @@ -119,11 +133,7 @@ public HadoopFileSystemUniversalFileSystemAdapter(HadoopFileSystemFactory factor } /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override public T unwrap(Class cls) { - if (HadoopFileSystemFactory.class.isAssignableFrom(cls)) - return (T)factory; - + @Override public IgfsEx igfs() { return null; } @@ -133,7 +143,7 @@ public HadoopFileSystemUniversalFileSystemAdapter(HadoopFileSystemFactory factor * @return File system. * @throws IOException If failed. */ - private FileSystem get() throws IOException { + protected FileSystem get() throws IOException { return factory.get(FileSystemConfiguration.DFLT_USER_NAME); } } \ No newline at end of file From 278633eced6d8039b5be4a18eefe6c65650aba4f Mon Sep 17 00:00:00 2001 From: Yakov Zhdanov Date: Mon, 15 Aug 2016 14:27:22 +0300 Subject: [PATCH 033/487] IGNITE-3688: Fixed visiblity issue in GridCacheIoManager.idxClsHandlers. --- .../processors/cache/GridCacheIoManager.java | 24 ++- .../processors/cache/GridCacheMessage.java | 2 +- .../dht/atomic/GridDhtAtomicCache.java | 176 ++++++++++++++---- 3 files changed, 155 insertions(+), 47 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java index 488a22ca00e54..78dddd3340270 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache; +import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.Iterator; @@ -97,7 +98,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter { private int retryCnt; /** Indexed class handlers. */ - private Map idxClsHandlers = new HashMap<>(); + private volatile Map idxClsHandlers = new HashMap<>(); /** Handler registry. */ private ConcurrentMap> @@ -241,7 +242,9 @@ private void handleMessage(UUID nodeId, GridCacheMessage cacheMsg) { IgniteBiInClosure c = null; if (msgIdx >= 0) { - IgniteBiInClosure[] cacheClsHandlers = idxClsHandlers.get(cacheMsg.cacheId()); + Map idxClsHandlers0 = idxClsHandlers; + + IgniteBiInClosure[] cacheClsHandlers = idxClsHandlers0.get(cacheMsg.cacheId()); if (cacheClsHandlers != null) c = cacheClsHandlers[msgIdx]; @@ -262,12 +265,19 @@ private void handleMessage(UUID nodeId, GridCacheMessage cacheMsg) { append(", cacheDesc=").append(cctx.cache().cacheDescriptor(cacheMsg.cacheId())). append(']'); + msg0.append(U.nl()).append("Registered listeners:"); + + Map idxClsHandlers0 = idxClsHandlers; + + for (Map.Entry e : idxClsHandlers0.entrySet()) + msg0.append(U.nl()).append(e.getKey()).append("=").append(Arrays.toString(e.getValue())); + if (cctx.kernalContext().isStopping()) { if (log.isDebugEnabled()) log.debug(msg0.toString()); } else - U.warn(log, msg0.toString()); + U.error(log, msg0.toString()); return; } @@ -1062,12 +1072,14 @@ public void addHandler( int msgIdx = messageIndex(type); if (msgIdx != -1) { - IgniteBiInClosure[] cacheClsHandlers = idxClsHandlers.get(cacheId); + Map idxClsHandlers0 = idxClsHandlers; + + IgniteBiInClosure[] cacheClsHandlers = idxClsHandlers0.get(cacheId); if (cacheClsHandlers == null) { cacheClsHandlers = new IgniteBiInClosure[GridCacheMessage.MAX_CACHE_MSG_LOOKUP_INDEX]; - idxClsHandlers.put(cacheId, cacheClsHandlers); + idxClsHandlers0.put(cacheId, cacheClsHandlers); } if (cacheClsHandlers[msgIdx] != null) @@ -1076,6 +1088,8 @@ public void addHandler( cacheClsHandlers[msgIdx] = c; + idxClsHandlers = idxClsHandlers0; + return; } else { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java index f99d2cdd07389..c5407b8ba30ce 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java @@ -50,7 +50,7 @@ public abstract class GridCacheMessage implements Message { private static final long serialVersionUID = 0L; /** Maximum number of cache lookup indexes. */ - public static final int MAX_CACHE_MSG_LOOKUP_INDEX = 256; + public static final int MAX_CACHE_MSG_LOOKUP_INDEX = 5; /** Cache message index field name. */ public static final String CACHE_MSG_INDEX_FIELD_NAME = "CACHE_MSG_IDX"; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java index 3616082c24879..1e45fa7a2bdc9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java @@ -251,61 +251,155 @@ else if (res.error() != null) { preldr.start(); - ctx.io().addHandler(ctx.cacheId(), GridNearGetRequest.class, new CI2() { - @Override public void apply(UUID nodeId, GridNearGetRequest req) { - processNearGetRequest(nodeId, req); - } - }); + ctx.io().addHandler( + ctx.cacheId(), + GridNearGetRequest.class, + new CI2() { + @Override public void apply( + UUID nodeId, + GridNearGetRequest req + ) { + processNearGetRequest( + nodeId, + req); + } + }); - ctx.io().addHandler(ctx.cacheId(), GridNearSingleGetRequest.class, new CI2() { - @Override public void apply(UUID nodeId, GridNearSingleGetRequest req) { - processNearSingleGetRequest(nodeId, req); - } - }); + ctx.io().addHandler( + ctx.cacheId(), + GridNearSingleGetRequest.class, + new CI2() { + @Override public void apply( + UUID nodeId, + GridNearSingleGetRequest req + ) { + processNearSingleGetRequest( + nodeId, + req); + } + }); - ctx.io().addHandler(ctx.cacheId(), GridNearAtomicUpdateRequest.class, new CI2() { - @Override public void apply(UUID nodeId, GridNearAtomicUpdateRequest req) { - processNearAtomicUpdateRequest(nodeId, req); - } - }); + ctx.io().addHandler( + ctx.cacheId(), + GridNearAtomicUpdateRequest.class, + new CI2() { + @Override public void apply( + UUID nodeId, + GridNearAtomicUpdateRequest req + ) { + processNearAtomicUpdateRequest( + nodeId, + req); + } - ctx.io().addHandler(ctx.cacheId(), GridNearAtomicUpdateResponse.class, new CI2() { - @Override public void apply(UUID nodeId, GridNearAtomicUpdateResponse res) { - processNearAtomicUpdateResponse(nodeId, res); - } - }); + @Override public String toString() { + return "GridNearAtomicUpdateRequest handler " + + "[msgIdx=" + GridNearAtomicUpdateRequest.CACHE_MSG_IDX + ']'; + } + }); - ctx.io().addHandler(ctx.cacheId(), GridDhtAtomicUpdateRequest.class, new CI2() { - @Override public void apply(UUID nodeId, GridDhtAtomicUpdateRequest req) { - processDhtAtomicUpdateRequest(nodeId, req); - } - }); + ctx.io().addHandler(ctx.cacheId(), + GridNearAtomicUpdateResponse.class, + new CI2() { + @Override public void apply( + UUID nodeId, + GridNearAtomicUpdateResponse res + ) { + processNearAtomicUpdateResponse( + nodeId, + res); + } - ctx.io().addHandler(ctx.cacheId(), GridDhtAtomicUpdateResponse.class, new CI2() { - @Override public void apply(UUID nodeId, GridDhtAtomicUpdateResponse res) { - processDhtAtomicUpdateResponse(nodeId, res); - } - }); + @Override public String toString() { + return "GridNearAtomicUpdateResponse handler " + + "[msgIdx=" + GridNearAtomicUpdateResponse.CACHE_MSG_IDX + ']'; + } + }); - ctx.io().addHandler(ctx.cacheId(), GridDhtAtomicDeferredUpdateResponse.class, - new CI2() { - @Override public void apply(UUID nodeId, GridDhtAtomicDeferredUpdateResponse res) { - processDhtAtomicDeferredUpdateResponse(nodeId, res); + ctx.io().addHandler( + ctx.cacheId(), + GridDhtAtomicUpdateRequest.class, + new CI2() { + @Override public void apply( + UUID nodeId, + GridDhtAtomicUpdateRequest req + ) { + processDhtAtomicUpdateRequest( + nodeId, + req); + } + + @Override public String toString() { + return "GridDhtAtomicUpdateRequest handler " + + "[msgIdx=" + GridDhtAtomicUpdateRequest.CACHE_MSG_IDX + ']'; } }); - if (near == null) { - ctx.io().addHandler(ctx.cacheId(), GridNearGetResponse.class, new CI2() { - @Override public void apply(UUID nodeId, GridNearGetResponse res) { - processNearGetResponse(nodeId, res); + ctx.io().addHandler( + ctx.cacheId(), + GridDhtAtomicUpdateResponse.class, + new CI2() { + @Override public void apply( + UUID nodeId, + GridDhtAtomicUpdateResponse res + ) { + processDhtAtomicUpdateResponse( + nodeId, + res); + } + + @Override public String toString() { + return "GridDhtAtomicUpdateResponse handler " + + "[msgIdx=" + GridDhtAtomicUpdateResponse.CACHE_MSG_IDX + ']'; } }); - ctx.io().addHandler(ctx.cacheId(), GridNearSingleGetResponse.class, new CI2() { - @Override public void apply(UUID nodeId, GridNearSingleGetResponse res) { - processNearSingleGetResponse(nodeId, res); + ctx.io().addHandler(ctx.cacheId(), + GridDhtAtomicDeferredUpdateResponse.class, + new CI2() { + @Override public void apply( + UUID nodeId, + GridDhtAtomicDeferredUpdateResponse res + ) { + processDhtAtomicDeferredUpdateResponse( + nodeId, + res); + } + + @Override public String toString() { + return "GridDhtAtomicDeferredUpdateResponse handler " + + "[msgIdx=" + GridDhtAtomicDeferredUpdateResponse.CACHE_MSG_IDX + ']'; } }); + + if (near == null) { + ctx.io().addHandler( + ctx.cacheId(), + GridNearGetResponse.class, + new CI2() { + @Override public void apply( + UUID nodeId, + GridNearGetResponse res + ) { + processNearGetResponse( + nodeId, + res); + } + }); + + ctx.io().addHandler( + ctx.cacheId(), + GridNearSingleGetResponse.class, + new CI2() { + @Override public void apply( + UUID nodeId, + GridNearSingleGetResponse res + ) { + processNearSingleGetResponse( + nodeId, + res); + } + }); } } From 09a3922d57f9a4c8fbe6c1056f3ea128869c250e Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Tue, 16 Aug 2016 12:52:09 +0300 Subject: [PATCH 034/487] IGNITE-3692: IGFS: Test fixes. --- .../processors/igfs/IgfsAsyncImpl.java | 5 ++ .../internal/processors/igfs/IgfsEx.java | 7 ++ .../internal/processors/igfs/IgfsImpl.java | 8 +- .../processors/igfs/IgfsAbstractSelfTest.java | 86 +++++++++---------- .../igfs/IgfsBackupFailoverSelfTest.java | 2 +- .../igfs/IgfsDualAbstractSelfTest.java | 86 +------------------ .../internal/processors/igfs/IgfsMock.java | 5 ++ .../igfs/HadoopIgfsDualAbstractSelfTest.java | 2 +- 8 files changed, 66 insertions(+), 135 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java index bf3d22b53975a..07b070eae8de2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java @@ -322,4 +322,9 @@ public IgfsAsyncImpl(IgfsImpl igfs) { @Override public IgfsSecondaryFileSystem asSecondary() { return igfs.asSecondary(); } + + /** {@inheritDoc} */ + @Override public void await(IgfsPath... paths) { + igfs.await(paths); + } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java index 4c64bc925e0f7..9760f435f22d7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java @@ -142,4 +142,11 @@ public interface IgfsEx extends IgniteFileSystem { * @return Secondary file system wrapper. */ public IgfsSecondaryFileSystem asSecondary(); + + /** + * Await for any pending finished writes on the children paths. + * + * @param paths Paths to check. + */ + public void await(IgfsPath... paths); } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index e1f8e611b714e..6707acc5e0381 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -374,12 +374,8 @@ private boolean enterBusy() { return busyLock.enterBusy(); } - /** - * Await for any pending finished writes on the children paths. - * - * @param paths Paths to check. - */ - void await(IgfsPath... paths) { + /** {@inheritDoc} */ + @Override public void await(IgfsPath... paths) { assert paths != null; for (Map.Entry workerEntry : workerMap.entrySet()) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java index 08cb92933509d..236a5893f275d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java @@ -489,7 +489,7 @@ protected static IgniteInternalFuture execute(final Callable task) { * @throws Exception If failed. */ public void testExists() throws Exception { - create(igfs.asSecondary(), paths(DIR), null); + create(igfs, paths(DIR), null); checkExist(igfs, igfsSecondary, DIR); } @@ -600,7 +600,7 @@ public void testRenameFileParentRoot() throws Exception { IgfsPath file1 = new IgfsPath("/file1"); IgfsPath file2 = new IgfsPath("/file2"); - create(igfs.asSecondary(), null, paths(file1)); + create(igfs, null, paths(file1)); igfs.rename(file1, file2); @@ -632,7 +632,7 @@ public void testRenameDirectoryParentRoot() throws Exception { IgfsPath dir1 = new IgfsPath("/dir1"); IgfsPath dir2 = new IgfsPath("/dir2"); - create(igfs.asSecondary(), paths(dir1), null); + create(igfs, paths(dir1), null); igfs.rename(dir1, dir2); @@ -661,7 +661,7 @@ public void testMoveFile() throws Exception { * @throws Exception If failed. */ public void testMoveFileDestinationRoot() throws Exception { - create(igfs.asSecondary(), paths(DIR, SUBDIR), paths(FILE)); + create(igfs, paths(DIR, SUBDIR), paths(FILE)); igfs.rename(FILE, new IgfsPath()); @@ -753,7 +753,7 @@ public void testMoveDirectory() throws Exception { * @throws Exception If failed. */ public void testMoveDirectoryDestinationRoot() throws Exception { - create(igfs.asSecondary(), paths(DIR, SUBDIR, SUBSUBDIR), null); + create(igfs, paths(DIR, SUBDIR, SUBSUBDIR), null); igfs.rename(SUBSUBDIR, new IgfsPath()); @@ -769,7 +769,7 @@ public void testMoveDirectoryDestinationRoot() throws Exception { public void testMoveDirectorySourceParentRoot() throws Exception { IgfsPath dir = new IgfsPath("/" + SUBSUBDIR.name()); - create(igfs.asSecondary(), paths(DIR_NEW, SUBDIR_NEW, dir), null); + create(igfs, paths(DIR_NEW, SUBDIR_NEW, dir), null); igfs.rename(dir, SUBDIR_NEW); @@ -1163,7 +1163,7 @@ private void checkRootPropertyUpdate(String prop, String setVal, String expGetVa public void testOpen() throws Exception { create(igfs, paths(DIR, SUBDIR), null); - createFile(igfs.asSecondary(), FILE, true, chunk); + createFile(igfs, FILE, true, chunk); checkFileContent(igfs, FILE, chunk); @@ -1200,7 +1200,7 @@ public void testOpenDoesNotExist() throws Exception { * @throws Exception If failed. */ public void testSetTimes() throws Exception { - createFile(igfs.asSecondary(), FILE, true, chunk); + createFile(igfs, FILE, true, chunk); checkExist(igfs, igfsSecondary, DIR); checkExist(igfs, igfsSecondary, SUBDIR); @@ -1312,7 +1312,7 @@ private void checkSetTimes(IgfsPath path) throws Exception { public void testCreate() throws Exception { create(igfs, paths(DIR, SUBDIR), null); - createFile(igfs.asSecondary(), FILE, true, chunk); + createFile(igfs, FILE, true, chunk); checkFile(igfs, igfsSecondary, FILE, chunk); @@ -1396,7 +1396,7 @@ public void testCreate() throws Exception { public void testCreateParentRoot() throws Exception { IgfsPath file = new IgfsPath("/" + FILE.name()); - createFile(igfs.asSecondary(), file, true, chunk); + createFile(igfs, file, true, chunk); checkFile(igfs, igfsSecondary, file, chunk); } @@ -1681,7 +1681,7 @@ public void testCreateConsistency() throws Exception { assert igfs.exists(path); } - awaitFileClose(igfs.asSecondary(), path); + awaitFileClose(igfs, path); checkFileContent(igfs, path, chunk); } @@ -1806,7 +1806,7 @@ public void testAppend() throws Exception { } finally { U.closeQuiet(os); - awaitFileClose(igfs.asSecondary(), path2); + awaitFileClose(igfs, path2); } try { @@ -1816,7 +1816,7 @@ public void testAppend() throws Exception { } finally { U.closeQuiet(os); - awaitFileClose(igfs.asSecondary(), path2); + awaitFileClose(igfs, path2); } checkFile(igfs, igfsSecondary, path2, chunk, chunk); @@ -1944,7 +1944,7 @@ public void testAppendNoClose() throws Exception { if (appendSupported()) { create(igfs, paths(DIR, SUBDIR), null); - createFile(igfs.asSecondary(), FILE, false); + createFile(igfs, FILE, false); GridTestUtils.assertThrowsInherited(log(), new Callable() { @Override @@ -1978,7 +1978,7 @@ public void testAppendRenameNoClose() throws Exception { if (appendSupported()) { create(igfs, paths(DIR, SUBDIR), null); - createFile(igfs.asSecondary(), FILE, false); + createFile(igfs, FILE, false); IgfsOutputStream os = null; @@ -2004,9 +2004,9 @@ public void testAppendRenameParentNoClose() throws Exception { return; if (appendSupported()) { - create(igfs.asSecondary(), paths(DIR, SUBDIR), null); + create(igfs, paths(DIR, SUBDIR), null); - createFile(igfs.asSecondary(), FILE, false); + createFile(igfs, FILE, false); IgfsOutputStream os = null; @@ -2034,7 +2034,7 @@ public void testAppendDeleteNoClose() throws Exception { if (appendSupported()) { create(igfs, paths(DIR, SUBDIR), null); - createFile(igfs.asSecondary(), FILE, false); + createFile(igfs, FILE, false); IgfsOutputStream os = null; IgniteUuid id = null; @@ -2089,7 +2089,7 @@ public void testAppendDeleteParentNoClose() throws Exception { if (appendSupported()) { create(igfs, paths(DIR, SUBDIR), null); - createFile(igfs.asSecondary(), FILE, false); + createFile(igfs, FILE, false); IgfsOutputStream os = null; IgniteUuid id = null; @@ -2146,7 +2146,7 @@ public void testAppendUpdateNoClose() throws Exception { create(igfs, paths(DIR, SUBDIR), null); - createFile(igfs.asSecondary(), FILE, false); + createFile(igfs, FILE, false); IgfsOutputStream os = null; @@ -2176,7 +2176,7 @@ public void testAppendConsistency() throws Exception { int threadCnt = 10; for (int i = 0; i < threadCnt; i++) - createFile(igfs.asSecondary(), new IgfsPath("/file" + i), false); + createFile(igfs, new IgfsPath("/file" + i), false); multithreaded(new Runnable() { @Override @@ -2200,7 +2200,7 @@ public void run() { assert igfs.exists(path); } - awaitFileClose(igfs.asSecondary(), path); + awaitFileClose(igfs, path); checkFileContent(igfs, path, chunks); } catch (IOException | IgniteCheckedException e) { @@ -2271,7 +2271,7 @@ public void run() { fut.get(); - awaitFileClose(igfs.asSecondary(), FILE); + awaitFileClose(igfs, FILE); if (err.get() != null) { X.println("Test failed: rethrowing first error: " + err.get()); @@ -2952,20 +2952,8 @@ private void checkDeadlocks(final int lvlCnt, final int childrenDirPerLvl, final * @param files Files. * @throws Exception If failed. */ + @SuppressWarnings("EmptyTryBlock") public static void create(IgfsImpl igfs, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files) throws Exception { - create(igfs.asSecondary(), dirs, files); - } - - /** - * Create the given directories and files in the given IGFS. - * - * @param igfs IGFS. - * @param dirs Directories. - * @param files Files. - * @throws Exception If failed. - */ - public static void create(IgfsSecondaryFileSystem igfs, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files) - throws Exception { if (dirs != null) { for (IgfsPath dir : dirs) igfs.mkdirs(dir); @@ -2973,9 +2961,11 @@ public static void create(IgfsSecondaryFileSystem igfs, @Nullable IgfsPath[] dir if (files != null) { for (IgfsPath file : files) { - OutputStream os = igfs.create(file, true); + try (OutputStream os = igfs.create(file, true)) { + // No-op. + } - os.close(); + igfs.await(file); } } } @@ -3013,8 +3003,8 @@ public void create(IgfsSecondaryFileSystemTestAdapter uni, @Nullable IgfsPath[] * @param chunks Data chunks. * @throws IOException In case of IO exception. */ - protected static void createFile(IgfsSecondaryFileSystem igfs, IgfsPath file, boolean overwrite, - @Nullable byte[]... chunks) throws IOException { + protected static void createFile(IgfsEx igfs, IgfsPath file, boolean overwrite, @Nullable byte[]... chunks) + throws IOException { OutputStream os = null; try { @@ -3051,7 +3041,7 @@ protected static void createFile(IgfsSecondaryFileSystemTestAdapter uni, IgfsPat IgfsEx igfsEx = uni.igfs(); if (igfsEx != null) - awaitFileClose(igfsEx.asSecondary(), file); + awaitFileClose(igfsEx, file); } } @@ -3077,7 +3067,7 @@ protected static void createFile(IgfsImpl igfs, IgfsPath file, boolean overwrite finally { U.closeQuiet(os); - awaitFileClose(igfs.asSecondary(), file); + awaitFileClose(igfs, file); } } @@ -3101,7 +3091,7 @@ protected static void appendFile(IgfsImpl igfs, IgfsPath file, @Nullable byte[]. finally { U.closeQuiet(os); - awaitFileClose(igfs.asSecondary(), file); + awaitFileClose(igfs, file); } } @@ -3134,6 +3124,16 @@ public static void awaitFileClose(IgfsSecondaryFileSystem igfs, IgfsPath file) { } } + /** + * Await for previously opened output stream to close. + * + * @param igfs IGFS. + * @param file File. + */ + public static void awaitFileClose(@Nullable IgfsEx igfs, IgfsPath file) { + igfs.await(file); + } + /** * Ensure that the given paths exist in the given IGFSs. * diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java index 5be9c09e8ab86..187aeeb2d4ca7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java @@ -563,7 +563,7 @@ void writeChunks0(IgfsEx igfs0, IgfsOutputStream ios, int fileIdx) throws IOExce U.closeQuiet(ios); - awaitFileClose(igfs0.asSecondary(), filePath(fileIdx)); + awaitFileClose(igfs0, filePath(fileIdx)); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java index b4ca0caad614e..02027d6b609bc 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java @@ -17,10 +17,8 @@ package org.apache.ignite.internal.processors.igfs; -import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; import org.apache.ignite.igfs.IgfsFile; -import org.apache.ignite.igfs.IgfsInputStream; import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; @@ -28,7 +26,6 @@ import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.U; -import java.io.OutputStream; import java.util.Collection; import java.util.Iterator; import java.util.Map; @@ -1163,85 +1160,6 @@ public void testOpenMissing() throws Exception { checkFileContent(igfs, FILE, chunk); } - /** - * Ensure that prefetch occurs in case several blocks are read sequentially. - * - * @throws Exception If failed. - */ - @SuppressWarnings("ResultOfMethodCallIgnored") - public void testOpenPrefetch() throws Exception { - create(igfsSecondary, paths(DIR, SUBDIR), paths(FILE)); - - // Write enough data to the secondary file system. - int blockSize0 = igfs.info(FILE).blockSize(); - final int blockSize = blockSize0 != 0 ? blockSize0 : 8 * 1024; - - int totalWritten = 0; - try (OutputStream out = igfsSecondary.openOutputStream(FILE.toString(), false)) { - while (totalWritten < blockSize * 2 + chunk.length) { - out.write(chunk); - - totalWritten += chunk.length; - } - } - - if (propertiesSupported()) - awaitFileClose(igfsSecondaryFileSystem, FILE); - else - Thread.sleep(1000); - - // Read the first two blocks. - int totalRead = 0; - - IgfsInputStream in = igfs.open(FILE, blockSize); - - final byte[] readBuf = new byte[1024]; - - while (totalRead + readBuf.length <= blockSize * 2) { - in.read(readBuf); - - totalRead += readBuf.length; - } - - // Wait for a while for prefetch to finish. - IgfsMetaManager meta = igfs.context().meta(); - - IgfsEntryInfo info = meta.info(meta.fileId(FILE)); - - assert info != null; - - IgfsBlockKey key = new IgfsBlockKey(info.id(), info.affinityKey(), info.evictExclude(), 2); - - IgniteCache dataCache = igfs.context().kernalContext().cache().jcache( - igfs.configuration().getDataCacheName()); - - for (int i = 0; i < 10; i++) { - if (dataCache.containsKey(key)) - break; - else - U.sleep(100); - } - - // Remove the file from the secondary file system. - igfsSecondary.delete(FILE.toString(), false); - - // Let's wait for file will be deleted. - U.sleep(300); - - // Read the third block. - totalRead = 0; - - in.seek(blockSize * 2); - - while (totalRead + readBuf.length <= blockSize) { - in.read(readBuf); - - totalRead += readBuf.length; - } - - in.close(); - } - /** * Test create when parent directory is partially missing locally. * @@ -1251,7 +1169,7 @@ public void testCreateParentMissingPartially() throws Exception { create(igfsSecondary, paths(DIR, SUBDIR), null); create(igfs, paths(DIR), null); - createFile(igfs.asSecondary(), FILE, true, chunk); + createFile(igfs, FILE, true, chunk); // Ensure that directory structure was created. checkExist(igfs, igfsSecondary, SUBDIR); @@ -1287,7 +1205,7 @@ public void testCreateParentMissing() throws Exception { create(igfsSecondary, paths(DIR, SUBDIR), null); create(igfs, null, null); - createFile(igfs.asSecondary(), FILE, true, chunk); + createFile(igfs, FILE, true, chunk); checkExist(igfs, igfsSecondary, SUBDIR); checkFile(igfs, igfsSecondary, FILE, chunk); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java index a2bd9ca5a82f2..01389079649f9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java @@ -375,6 +375,11 @@ public IgfsMock(@Nullable String name) { return 0; } + /** {@inheritDoc} */ + @Override public void await(IgfsPath... paths) { + throwUnsupported(); + } + /** {@inheritDoc} */ @Override public IgniteFileSystem withAsync() { throwUnsupported(); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java index 40cf4938da703..bb155b40528c5 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java @@ -252,7 +252,7 @@ public void testOpenPrefetchOverride() throws Exception { out.close(); - awaitFileClose(igfsSecondary.asSecondary(), FILE); + awaitFileClose(igfsSecondary, FILE); // Instantiate file system with overridden "seq reads before prefetch" property. Configuration cfg = new Configuration(); From 476081b9171b1b8e5ec0a1ffd7e87092da3601d6 Mon Sep 17 00:00:00 2001 From: Tikhonov Nikolay Date: Wed, 17 Aug 2016 14:31:20 +0300 Subject: [PATCH 035/487] Fixed isReadFromBackup behaviour for transaction. This closes #955. --- .../near/GridNearTransactionalCache.java | 2 +- .../distributed/near/GridNearTxLocal.java | 4 +- .../CacheTxNotAllowReadFromBackupTest.java | 297 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite4.java | 2 + 4 files changed, 302 insertions(+), 3 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheTxNotAllowReadFromBackupTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java index e954a7f6ef7bf..cf5d2e2ea36fa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java @@ -192,7 +192,7 @@ IgniteInternalFuture> txLoadAsync(GridNearTxLocal tx, GridNearGetFuture fut = new GridNearGetFuture<>(ctx, keys, readThrough, - /*force primary*/needVer, + /*force primary*/needVer || !ctx.config().isReadFromBackup(), tx, CU.subjectId(tx, ctx.shared()), tx.resolveTaskName(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index db736a529184e..62cf74beed27d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -363,7 +363,7 @@ else if (cacheCtx.isColocated()) { return cacheCtx.colocated().loadAsync( key, readThrough, - /*force primary*/needVer, + /*force primary*/needVer || !cacheCtx.config().isReadFromBackup(), topVer, CU.subjectId(this, cctx), resolveTaskName(), @@ -394,7 +394,7 @@ else if (cacheCtx.isColocated()) { return cacheCtx.colocated().loadAsync( keys, readThrough, - /*force primary*/needVer, + /*force primary*/needVer || !cacheCtx.config().isReadFromBackup(), topVer, CU.subjectId(this, cctx), resolveTaskName(), diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheTxNotAllowReadFromBackupTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheTxNotAllowReadFromBackupTest.java new file mode 100644 index 0000000000000..30de2f93f1a75 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheTxNotAllowReadFromBackupTest.java @@ -0,0 +1,297 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.jetbrains.annotations.NotNull; + +/** + * Test for query with BinaryMarshaller and different serialization modes and with reflective serializer. + */ +public class CacheTxNotAllowReadFromBackupTest extends GridCommonAbstractTest { + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int NODES = 2; + + /** */ + private static final int KEYS = 1000; + + /** */ + private static final int BATCH_SIZE = 10; + + /** */ + private static final int ITERATION_CNT = 5; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGridsMultiThreaded(NODES); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + + /** + * @throws Exception If failed. + */ + public void testBackupConsistencyReplicated() throws Exception { + CacheConfiguration cfg = new CacheConfiguration<>("test-cache"); + + cfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC); + cfg.setCacheMode(CacheMode.REPLICATED); + cfg.setReadFromBackup(false); + + checkBackupConsistency(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistency(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistency(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.SERIALIZABLE); + + checkBackupConsistency(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistency(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistency(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.SERIALIZABLE); + + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.SERIALIZABLE); + + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testBackupConsistencyReplicatedFullSync() throws Exception { + CacheConfiguration cfg = new CacheConfiguration<>("test-cache"); + + cfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + cfg.setCacheMode(CacheMode.REPLICATED); + cfg.setReadFromBackup(false); + + checkBackupConsistency(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistency(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistency(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.SERIALIZABLE); + + checkBackupConsistency(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistency(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistency(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.SERIALIZABLE); + + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.SERIALIZABLE); + + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testBackupConsistencyPartitioned() throws Exception { + CacheConfiguration cfg = new CacheConfiguration<>("test-cache"); + + cfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC); + cfg.setCacheMode(CacheMode.PARTITIONED); + cfg.setBackups(NODES - 1); + cfg.setReadFromBackup(false); + + checkBackupConsistency(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistency(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistency(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.SERIALIZABLE); + + checkBackupConsistency(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistency(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistency(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.SERIALIZABLE); + + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.SERIALIZABLE); + + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testBackupConsistencyPartitionedFullSync() throws Exception { + CacheConfiguration cfg = new CacheConfiguration<>("test-cache"); + + cfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + cfg.setCacheMode(CacheMode.PARTITIONED); + cfg.setBackups(NODES - 1); + cfg.setReadFromBackup(false); + + checkBackupConsistency(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistency(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistency(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.SERIALIZABLE); + + checkBackupConsistency(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistency(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistency(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.SERIALIZABLE); + + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.SERIALIZABLE); + + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.READ_COMMITTED); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ); + checkBackupConsistencyGetAll(cfg, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.SERIALIZABLE); + } + + /** + * @param ccfg Cache configuration. + * @throws Exception If failed. + */ + private void checkBackupConsistency(CacheConfiguration ccfg, TransactionConcurrency txConcurrency, + TransactionIsolation txIsolation) throws Exception { + IgniteCache cache = grid(0).getOrCreateCache(ccfg); + + int nodeIdx = ThreadLocalRandom.current().nextInt(NODES); + + try { + for (int i = 0; i < ITERATION_CNT; i++) { + log.info("Iteration: " + i); + + // Put data in one transaction. + try (Transaction tx = grid(nodeIdx).transactions().txStart(txConcurrency, txIsolation)) { + for (int key = 0; key < KEYS; key++) + cache.put(key, key); + + tx.commit(); + } + + int missCnt = 0; + + // Try to load data from another transaction. + try (Transaction tx = grid(nodeIdx).transactions().txStart(txConcurrency, txIsolation)) { + for (int key = 0; key < KEYS; key++) + if (cache.get(key) == null) + ++missCnt; + + tx.commit(); + } + + assertEquals("Failed. Found missing get()", 0, missCnt); + } + } + finally { + grid(0).destroyCache(ccfg.getName()); + } + } + + /** + * @param ccfg Cache configuration. + * @throws Exception If failed. + */ + private void checkBackupConsistencyGetAll(CacheConfiguration ccfg, + TransactionConcurrency txConcurrency, TransactionIsolation txIsolation) throws Exception { + IgniteCache cache = grid(0).getOrCreateCache(ccfg); + + int nodeIdx = ThreadLocalRandom.current().nextInt(NODES); + + try { + for (int i = 0; i < ITERATION_CNT; i++) { + log.info("Iteration: " + i); + + List> batches = createBatches(); + + // Put data in one transaction. + try (Transaction tx = grid(nodeIdx).transactions().txStart(txConcurrency, txIsolation)) { + for (int key = 0; key < KEYS; key++) + cache.put(key, key); + + tx.commit(); + } + + // Try to load data from another transaction. + try (Transaction tx = grid(nodeIdx).transactions().txStart(txConcurrency, txIsolation)) { + for (Set batch : batches) + assertEquals("Failed. Found missing entries.", batch.size(), cache.getAll(batch).size()); + + tx.commit(); + } + } + } + finally { + grid(0).destroyCache(ccfg.getName()); + } + } + + /** + * @return Batches. + */ + @NotNull private List> createBatches() { + List> batches = new ArrayList<>(KEYS / BATCH_SIZE + 1); + + int size = BATCH_SIZE; + Set batch = new HashSet<>(); + + for (int key = 0; key < KEYS; key++) { + batch.add(key); + + if (--size == 0) { + size = BATCH_SIZE; + batch = new HashSet<>(); + batches.add(batch); + } + } + return batches; + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java index 018fa17f382ab..60d59d70424b9 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java @@ -41,6 +41,7 @@ import org.apache.ignite.internal.processors.cache.CacheStoreUsageMultinodeStaticStartAtomicTest; import org.apache.ignite.internal.processors.cache.CacheStoreUsageMultinodeStaticStartTxTest; import org.apache.ignite.internal.processors.cache.CacheSwapUnswapGetTest; +import org.apache.ignite.internal.processors.cache.CacheTxNotAllowReadFromBackupTest; import org.apache.ignite.internal.processors.cache.CrossCacheLockTest; import org.apache.ignite.internal.processors.cache.GridCacheMarshallingNodeJoinSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheMultinodeUpdateAtomicNearEnabledSelfTest; @@ -263,6 +264,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheGetEntryPessimisticReadCommittedSeltTest.class); suite.addTestSuite(CacheGetEntryPessimisticRepeatableReadSeltTest.class); suite.addTestSuite(CacheGetEntryPessimisticSerializableSeltTest.class); + suite.addTestSuite(CacheTxNotAllowReadFromBackupTest.class); suite.addTestSuite(CacheStopAndDestroySelfTest.class); From 97d1a6f6f1d30507ad93122eb9ea63285cdffde7 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Thu, 18 Aug 2016 15:06:04 +0300 Subject: [PATCH 036/487] IGNITE-3694 IgfsLocalSecondaryFileSystemDualAsyncSelfTest.testAppendConsistencyMultithreaded hangs --- .../internal/processors/igfs/IgfsOutputStreamImpl.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java index f6b1104fff925..bbff93b4fc0fa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java @@ -315,6 +315,11 @@ private void flushRemainder() throws IOException { ", fileInfo=" + fileInfo + ']', e); } + // Finish batch before file unlocking to support the assertion that unlocked file batch, + // if any, must be in finishing state (e.g. append see more IgfsImpl.newBatch) + if (batch != null) + batch.finish(); + // Unlock the file after data is flushed. try { if (flushSuccess && space > 0) @@ -332,8 +337,6 @@ private void flushRemainder() throws IOException { // Finally, await secondary file system flush. if (batch != null) { - batch.finish(); - if (mode == DUAL_SYNC) { try { batch.await(); From 8aa534a6124c066801e6627f36179934653fe59f Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Thu, 18 Aug 2016 18:21:22 +0300 Subject: [PATCH 037/487] IGNITE-2559 Fixed Transaction hangs if entry processor is not serializable. This closes #951. --- .../GridNearPessimisticTxPrepareFuture.java | 2 + .../near/GridNearTxFinishFuture.java | 28 +- .../distributed/near/GridNearTxLocal.java | 10 +- .../cache/transactions/IgniteTxHandler.java | 1 - ...acheEntryProcessorNonSerializableTest.java | 410 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite.java | 2 + 6 files changed, 437 insertions(+), 16 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorNonSerializableTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java index 5d347d7c5f23d..ef2edc992cda6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java @@ -291,6 +291,8 @@ private void preparePessimistic() { } fut.onError(e); + + break; } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java index 39f3ff3e06712..adde63cc13dc5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java @@ -97,7 +97,7 @@ public final class GridNearTxFinishFuture extends GridCompoundIdentityFutu @GridToStringInclude private GridNearTxLocal tx; - /** Commit flag. */ + /** Commit flag. This flag used only for one-phase commit transaction. */ private boolean commit; /** Node mappings. */ @@ -313,6 +313,9 @@ else if (tx.implicit() && tx.isSystemInvalidate()) { // Finish implicit transact } } + if (commit && tx.commitError() != null) + err = tx.commitError(); + if (initialized() || err != null) { if (tx.needCheckBackup()) { assert tx.onePhaseCommit(); @@ -386,9 +389,11 @@ private boolean isMini(IgniteInternalFuture fut) { /** * Initializes future. + * + * @param commit Commit flag. */ @SuppressWarnings("ForLoopReplaceableByForEach") - void finish() { + void finish(boolean commit) { if (tx.onNeedCheckBackup()) { assert tx.onePhaseCommit(); @@ -403,15 +408,15 @@ void finish() { try { if (tx.finish(commit) || (!commit && tx.state() == UNKNOWN)) { - if ((tx.onePhaseCommit() && needFinishOnePhase()) || (!tx.onePhaseCommit() && mappings != null)) { + if ((tx.onePhaseCommit() && needFinishOnePhase(commit)) || (!tx.onePhaseCommit() && mappings != null)) { if (mappings.single()) { GridDistributedTxMapping mapping = mappings.singleMapping(); if (mapping != null) - finish(mapping); + finish(mapping, commit); } else - finish(mappings.mappings()); + finish(mappings.mappings(), commit); } markInitialized(); @@ -543,13 +548,14 @@ private void checkBackup() { } /** + * @param commit Commit flag. * @return {@code True} if need to send finish request for one phase commit transaction. */ - private boolean needFinishOnePhase() { + private boolean needFinishOnePhase(boolean commit) { if (tx.mappings().empty()) return false; - boolean finish = tx.txState().hasNearCache(cctx); + boolean finish = tx.txState().hasNearCache(cctx) || !commit; if (finish) { GridDistributedTxMapping mapping = tx.mappings().singleMapping(); @@ -605,17 +611,19 @@ private void readyNearMappingFromBackup(GridDistributedTxMapping mapping) { /** * @param mappings Mappings. + * @param commit Commit flag. */ - private void finish(Iterable mappings) { + private void finish(Iterable mappings, boolean commit) { // Create mini futures. for (GridDistributedTxMapping m : mappings) - finish(m); + finish(m, commit); } /** * @param m Mapping. + * @param commit Commit flag. */ - private void finish(GridDistributedTxMapping m) { + private void finish(GridDistributedTxMapping m, boolean commit) { ClusterNode n = m.node(); assert !m.empty(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index 62cf74beed27d..28c60d4ca4fd9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -857,19 +857,19 @@ void readyNearLock(IgniteTxEntry txEntry, // Make sure that here are no exceptions. prepareFut.get(); - fut0.finish(); + fut0.finish(true); } catch (Error | RuntimeException e) { COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); - fut0.onDone(e); + fut0.finish(false); throw e; } catch (IgniteCheckedException e) { COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); - fut0.onDone(e); + fut0.finish(false); } } }); @@ -917,7 +917,7 @@ void readyNearLock(IgniteTxEntry txEntry, log.debug("Got optimistic tx failure [tx=" + this + ", err=" + e + ']'); } - fut.finish(); + fut.finish(false); } else { prepFut.listen(new CI1>() { @@ -933,7 +933,7 @@ void readyNearLock(IgniteTxEntry txEntry, GridNearTxFinishFuture fut0 = rollbackFut; - fut0.finish(); + fut0.finish(false); } }); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index ba30e1056b16c..7c3c206c1c137 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -639,7 +639,6 @@ else if (txFinishMsgLog.isDebugEnabled()) { ", node=" + nodeId + ']'); } - fut.onResult(nodeId, res); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorNonSerializableTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorNonSerializableTest.java new file mode 100644 index 0000000000000..79aa34fcd337b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorNonSerializableTest.java @@ -0,0 +1,410 @@ +/* + * 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.query.continuous; + +import java.io.NotSerializableException; +import java.util.concurrent.Callable; +import java.util.concurrent.ThreadLocalRandom; +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.marshaller.optimized.OptimizedMarshaller; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC; +import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; +import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE; + +/** + * + */ +public class CacheEntryProcessorNonSerializableTest extends GridCommonAbstractTest { + /** */ + private static final int EXPECTED_VALUE = 42; + + /** */ + private static final int WRONG_VALUE = -1; + + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int NODES = 3; + + /** */ + public static final int ITERATION_CNT = 1; + + /** */ + public static final int KEYS = 10; + + /** */ + private boolean client; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1); + + cfg.setClientMode(client); + cfg.setMarshaller(new OptimizedMarshaller()); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGridsMultiThreaded(getServerNodeCount()); + + client = true; + + startGrid(getServerNodeCount()); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + + /** + * @return Server nodes. + */ + private int getServerNodeCount() { + return NODES; + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticOnePhaseCommit() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PRIMARY_SYNC, 1); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticOnePhaseCommitWithNearCache() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PRIMARY_SYNC, 1) + .setNearConfiguration(new NearCacheConfiguration()); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticOnePhaseCommitFullSync() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(FULL_SYNC, 1); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticOnePhaseCommitFullSyncWithNearCache() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(FULL_SYNC, 1) + .setNearConfiguration(new NearCacheConfiguration()); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testPessimistic() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PRIMARY_SYNC, 2); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticWithNearCache() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PRIMARY_SYNC, 2) + .setNearConfiguration(new NearCacheConfiguration()); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticFullSync() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(FULL_SYNC, 2); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticOnePhaseCommit() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PRIMARY_SYNC, 1); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticOnePhaseCommitFullSync() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(FULL_SYNC, 1); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticOnePhaseCommitFullSyncWithNearCache() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(FULL_SYNC, 1) + .setNearConfiguration(new NearCacheConfiguration()); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testOptimistic() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PRIMARY_SYNC, 2); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticFullSync() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(FULL_SYNC, 2); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticFullSyncWithNearCache() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(FULL_SYNC, 2); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + } + + /** + * @param ccfg Cache configuration. + * @throws Exception If failed. + */ + private void doTestInvokeTest(CacheConfiguration ccfg, TransactionConcurrency txConcurrency, + TransactionIsolation txIsolation) throws Exception { + IgniteEx cln = grid(getServerNodeCount()); + + grid(0).createCache(ccfg); + + IgniteCache clnCache; + + if (ccfg.getNearConfiguration() != null) + clnCache = cln.createNearCache(ccfg.getName(), ccfg.getNearConfiguration()); + else + clnCache = cln.cache(ccfg.getName()); + + putKeys(clnCache, EXPECTED_VALUE); + + try { + // Explicit tx. + for (int i = 0; i < ITERATION_CNT; i++) { + try (final Transaction tx = cln.transactions().txStart(txConcurrency, txIsolation)) { + putKeys(clnCache, WRONG_VALUE); + + clnCache.invoke(KEYS, new NonSerialazibleEntryProcessor()); + + GridTestUtils.assertThrowsWithCause(new Callable() { + @Override public Object call() throws Exception { + tx.commit(); + + return null; + } + }, NotSerializableException.class); + } + + checkKeys(clnCache, EXPECTED_VALUE); + } + + // From affinity node. + Ignite grid = grid(ThreadLocalRandom.current().nextInt(NODES)); + + final IgniteCache cache = grid.cache(ccfg.getName()); + + // Explicit tx. + for (int i = 0; i < ITERATION_CNT; i++) { + try (final Transaction tx = grid.transactions().txStart(txConcurrency, txIsolation)) { + putKeys(cache, WRONG_VALUE); + + cache.invoke(KEYS, new NonSerialazibleEntryProcessor()); + + GridTestUtils.assertThrowsWithCause(new Callable() { + @Override public Object call() throws Exception { + tx.commit(); + + return null; + } + }, NotSerializableException.class); + } + + checkKeys(cache, EXPECTED_VALUE); + } + + final IgniteCache clnCache0 = clnCache; + + // Implicit tx. + for (int i = 0; i < ITERATION_CNT; i++) { + GridTestUtils.assertThrowsWithCause(new Callable() { + @Override public Object call() throws Exception { + clnCache0.invoke(KEYS, new NonSerialazibleEntryProcessor()); + + return null; + } + }, NotSerializableException.class); + } + + checkKeys(clnCache, EXPECTED_VALUE); + } + finally { + grid(0).destroyCache(ccfg.getName()); + } + } + + /** + * @param cache Cache. + * @param val Value. + */ + private void putKeys(IgniteCache cache, int val) { + cache.put(KEYS, val); + } + + /** + * @param cache Cache. + * @param expVal Expected value. + */ + private void checkKeys(IgniteCache cache, int expVal) { + assertEquals(expVal, cache.get(KEYS)); + } + + /** + * @return Cache configuration. + */ + private CacheConfiguration cacheConfiguration(CacheWriteSynchronizationMode wrMode, int backup) { + return new CacheConfiguration("test-cache-" + wrMode + "-" + backup) + .setAtomicityMode(TRANSACTIONAL) + .setWriteSynchronizationMode(FULL_SYNC) + .setBackups(backup); + } + + /** + * + */ + private static class NonSerialazibleEntryProcessor implements EntryProcessor { + /** {@inheritDoc} */ + @Override public Integer process(MutableEntry entry, Object... arguments) + throws EntryProcessorException { + entry.setValue(42); + + return null; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java index 8c3f4def8663a..84e1502d63e4a 100755 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java @@ -137,6 +137,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearTxExceptionSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedTxExceptionSelfTest; import org.apache.ignite.internal.processors.cache.local.GridCacheLocalTxExceptionSelfTest; +import org.apache.ignite.internal.processors.cache.query.continuous.CacheEntryProcessorNonSerializableTest; import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorSelfTest; import org.apache.ignite.internal.processors.datastreamer.DataStreamerImplSelfTest; import org.apache.ignite.internal.processors.datastreamer.DataStreamerMultiThreadedSelfTest; @@ -186,6 +187,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(IgniteCacheAtomicLocalInvokeTest.class); suite.addTestSuite(IgniteCacheAtomicLocalWithStoreInvokeTest.class); suite.addTestSuite(IgniteCacheTxInvokeTest.class); + suite.addTestSuite(CacheEntryProcessorNonSerializableTest.class); suite.addTestSuite(IgniteCacheEntryProcessorCallTest.class); GridTestUtils.addTestIfNeeded(suite, CacheEntryProcessorCopySelfTest.class, ignoredTests); suite.addTestSuite(IgniteCacheTxNearEnabledInvokeTest.class); From a20ca351b33efb07b83c6f5967fa7a3cef154c83 Mon Sep 17 00:00:00 2001 From: sboikov Date: Fri, 19 Aug 2016 10:37:59 +0300 Subject: [PATCH 038/487] ignite-3547 Do not try to re-send message using the same client. Remove disconnected client from 'onDisconnected' callback. --- .../nio/GridTcpNioCommunicationClient.java | 5 +- .../tcp/TcpCommunicationSpi.java | 49 ++--- .../CacheSerializableTransactionsTest.java | 5 + .../IgniteCacheConnectionRecoveryTest.java | 205 ++++++++++++++++++ ...gniteCacheMessageRecoveryAbstractTest.java | 14 +- ...cheMessageRecoveryIdleConnectionTest.java} | 7 +- .../testsuites/IgniteCacheTestSuite.java | 6 +- 7 files changed, 255 insertions(+), 36 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheConnectionRecoveryTest.java rename modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/{IgniteCacheMessageRecoveryIdleConnection.java => IgniteCacheMessageRecoveryIdleConnectionTest.java} (94%) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java index 4022bc5e01146..5fe521d7e9c8f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java @@ -125,8 +125,11 @@ public GridNioSession session() { if (log.isDebugEnabled()) log.debug("Failed to send message [client=" + this + ", err=" + e + ']'); - if (e.getCause() instanceof IOException) + if (e.getCause() instanceof IOException) { + ses.close(); + return true; + } else throw new IgniteCheckedException("Failed to send message [client=" + this + ']', e); } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 2c03b2db7bfe0..d81b9f3397aa2 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -353,6 +353,15 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter UUID id = ses.meta(NODE_ID_META); if (id != null) { + GridCommunicationClient client = clients.get(id); + + if (client instanceof GridTcpNioCommunicationClient && + ((GridTcpNioCommunicationClient) client).session() == ses) { + client.close(); + + clients.remove(id, client); + } + if (!stopping) { boolean reconnect = false; @@ -372,9 +381,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter recoveryData.onNodeLeft(); } - DisconnectedSessionInfo disconnectData = new DisconnectedSessionInfo(id, - ses, - recoveryData, + DisconnectedSessionInfo disconnectData = new DisconnectedSessionInfo(recoveryData, reconnect); commWorker.addProcessDisconnectRequest(disconnectData); @@ -1400,6 +1407,14 @@ public CommunicationListener getListener() { .append(']').append(U.nl()); } + sb.append("Communication SPI clients: ").append(U.nl()); + + for (Map.Entry entry : clients.entrySet()) { + sb.append(" [node=").append(entry.getKey()) + .append(", client=").append(entry.getValue()) + .append(']').append(U.nl()); + } + U.warn(log, sb.toString()); } @@ -1978,17 +1993,19 @@ private void sendMessage0(ClusterNode node, Message msg, IgniteInClosure data = new TreeMap<>(); + + for (int i = 0; i < 500; i++) + data.put(i, i); + + final AtomicInteger idx = new AtomicInteger(); + + final long stopTime = U.currentTimeMillis() + 30_000; + + final AtomicReference barrierRef = new AtomicReference<>(); + + final int TEST_THREADS = (CLIENTS + SRVS) * 2; + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Object call() throws Exception { + int idx0 = idx.getAndIncrement(); + Ignite node = ignite(idx0 % (SRVS + CLIENTS)); + + Thread.currentThread().setName("test-thread-" + idx0 + "-" + node.name()); + + IgniteCache cache1 = node.cache("cache1").withAsync(); + IgniteCache cache2 = node.cache("cache2").withAsync(); + + int iter = 0; + + while (U.currentTimeMillis() < stopTime) { + try { + cache1.putAll(data); + cache1.future().get(15, SECONDS); + + cache2.putAll(data); + cache2.future().get(15, SECONDS); + + CyclicBarrier b = barrierRef.get(); + + if (b != null) + b.await(15, SECONDS); + } + catch (Exception e) { + synchronized (IgniteCacheConnectionRecoveryTest.class) { + log.error("Failed to execute update, will dump debug information" + + " [err=" + e+ ", iter=" + iter + ']', e); + + List nodes = IgnitionEx.allGridsx(); + + for (Ignite node0 : nodes) + ((IgniteKernal)node0).dumpDebugInfo(); + + U.dumpThreads(log); + } + + throw e; + } + } + + return null; + } + }, TEST_THREADS, "test-thread"); + + while (System.currentTimeMillis() < stopTime) { + boolean closed = false; + + for (Ignite node : G.allGrids()) { + if (IgniteCacheMessageRecoveryAbstractTest.closeSessions(node)) + closed = true; + } + + if (closed) { + CyclicBarrier b = new CyclicBarrier(TEST_THREADS + 1, new Runnable() { + @Override public void run() { + barrierRef.set(null); + } + }); + + barrierRef.set(b); + + b.await(); + } + + U.sleep(50); + } + + fut.get(); + } + + /** + * @param name Cache name. + * @param atomicityMode Cache atomicity mode. + * @return Configuration. + */ + private CacheConfiguration cacheConfiguration(String name, CacheAtomicityMode atomicityMode) { + CacheConfiguration ccfg = new CacheConfiguration(); + + ccfg.setName(name); + ccfg.setAtomicityMode(atomicityMode); + ccfg.setCacheMode(REPLICATED); + ccfg.setWriteSynchronizationMode(FULL_SYNC); + + return ccfg; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java index 16d7e5d9cecc1..0460a8ff6b9f0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java @@ -150,7 +150,11 @@ public void testMessageRecovery() throws Exception { for (int i = 0; i < 30; i++) { Thread.sleep(1000); - closed |= closeSessions(); + Ignite node0 = ignite(ThreadLocalRandom.current().nextInt(0, GRID_CNT)); + + log.info("Close sessions for: " + ignite.name()); + + closed |= closeSessions(node0); } assertTrue(closed); @@ -163,13 +167,11 @@ public void testMessageRecovery() throws Exception { } /** + * @param ignite Node. * @throws Exception If failed. + * @return {@code True} if closed at least one session. */ - private boolean closeSessions() throws Exception { - Ignite ignite = ignite(ThreadLocalRandom.current().nextInt(0, GRID_CNT)); - - log.info("Close sessions for: " + ignite.name()); - + static boolean closeSessions(Ignite ignite) throws Exception { TcpCommunicationSpi commSpi = (TcpCommunicationSpi)ignite.configuration().getCommunicationSpi(); Map clients = U.field(commSpi, "clients"); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryIdleConnection.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryIdleConnectionTest.java similarity index 94% rename from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryIdleConnection.java rename to modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryIdleConnectionTest.java index 618fe2a776344..b9003cdb840d9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryIdleConnection.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryIdleConnectionTest.java @@ -43,7 +43,7 @@ /** * */ -public class IgniteCacheMessageRecoveryIdleConnection extends GridCommonAbstractTest { +public class IgniteCacheMessageRecoveryIdleConnectionTest extends GridCommonAbstractTest { /** */ private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); @@ -123,7 +123,7 @@ private void cacheOperationsIdleConnectionClose(CacheAtomicityMode atomicityMode long stopTime = System.currentTimeMillis() + 90_000; while (System.currentTimeMillis() < stopTime) { - if (iter++ % 10 == 0) + if (iter++ % 50 == 0) log.info("Iteration: " + iter); cache.put(iter, 1); @@ -134,6 +134,9 @@ private void cacheOperationsIdleConnectionClose(CacheAtomicityMode atomicityMode fut.get(10_000); } catch (IgniteException e) { + log.error("Failed to execute update, will dump debug information" + + " [err=" + e+ ", iter=" + iter + ']', e); + List nodes = IgnitionEx.allGridsx(); for (Ignite node : nodes) diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java index 84e1502d63e4a..9240ef5959f59 100755 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java @@ -125,7 +125,8 @@ import org.apache.ignite.internal.processors.cache.distributed.CacheTxNearUpdateTopologyChangeTest; import org.apache.ignite.internal.processors.cache.distributed.GridCacheEntrySetIterationPreloadingSelfTest; import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicMessageRecoveryTest; -import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheMessageRecoveryIdleConnection; +import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheConnectionRecoveryTest; +import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheMessageRecoveryIdleConnectionTest; import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheMessageWriteTimeoutTest; import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheSystemTransactionsSelfTest; import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheTxMessageRecoveryTest; @@ -283,7 +284,8 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(IgniteCacheAtomicMessageRecoveryTest.class); suite.addTestSuite(IgniteCacheTxMessageRecoveryTest.class); suite.addTestSuite(IgniteCacheMessageWriteTimeoutTest.class); - suite.addTestSuite(IgniteCacheMessageRecoveryIdleConnection.class); + suite.addTestSuite(IgniteCacheMessageRecoveryIdleConnectionTest.class); + suite.addTestSuite(IgniteCacheConnectionRecoveryTest.class); GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredEvictionAtomicSelfTest.class, ignoredTests); GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredEvictionSelfTest.class, ignoredTests); GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredAtomicSelfTest.class, ignoredTests); From 9db16d1b2a23e703546c472676620139e82e747f Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 19 Aug 2016 17:40:30 +0700 Subject: [PATCH 039/487] Fixed NPE in special case. --- .../visor/cache/VisorCacheTypeMetadata.java | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java index 4e38d8102f83e..870e0333eb0c5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java @@ -134,21 +134,25 @@ public static Collection list(Collection qr JdbcTypeField[] keyFields = jdbcType.getKeyFields(); - meta.keyFields = new ArrayList<>(keyFields.length); + if (keyFields != null) { + meta.keyFields = new ArrayList<>(keyFields.length); - for (JdbcTypeField fld : keyFields) - meta.keyFields.add(new VisorCacheTypeFieldMetadata( - fld.getDatabaseFieldName(), fld.getDatabaseFieldType(), - fld.getDatabaseFieldName(), U.compact(fld.getJavaFieldType().getName()))); + for (JdbcTypeField fld : keyFields) + meta.keyFields.add(new VisorCacheTypeFieldMetadata( + fld.getDatabaseFieldName(), fld.getDatabaseFieldType(), + fld.getDatabaseFieldName(), U.compact(fld.getJavaFieldType().getName()))); + } JdbcTypeField[] valFields = jdbcType.getValueFields(); - meta.valFields = new ArrayList<>(valFields.length); + if (valFields != null) { + meta.valFields = new ArrayList<>(valFields.length); - for (JdbcTypeField fld : valFields) - meta.valFields.add(new VisorCacheTypeFieldMetadata( + for (JdbcTypeField fld : valFields) + meta.valFields.add(new VisorCacheTypeFieldMetadata( fld.getDatabaseFieldName(), fld.getDatabaseFieldType(), fld.getDatabaseFieldName(), U.compact(fld.getJavaFieldType().getName()))); + } if (notFound) metas.add(meta); From 7422195f17df2c69a98eeed202f14eea80ad70b4 Mon Sep 17 00:00:00 2001 From: Valentin Kulichenko Date: Fri, 19 Aug 2016 16:53:40 -0700 Subject: [PATCH 040/487] IGNITE-473 - Fixed HibernateL2CacheExample output --- .../examples/datagrid/hibernate/HibernateL2CacheExample.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/hibernate/HibernateL2CacheExample.java b/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/hibernate/HibernateL2CacheExample.java index 4f399fb4dbd30..a31d394b75538 100644 --- a/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/hibernate/HibernateL2CacheExample.java +++ b/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/hibernate/HibernateL2CacheExample.java @@ -251,7 +251,7 @@ private static void printStats(SessionFactory sesFactory) { SecondLevelCacheStatistics stats = sesFactory.getStatistics().getSecondLevelCacheStatistics(entityName); - System.out.println("\t\tL2 cache entries: " + stats.getEntries()); + System.out.println("\t\tPuts: " + stats.getPutCount()); System.out.println("\t\tHits: " + stats.getHitCount()); System.out.println("\t\tMisses: " + stats.getMissCount()); } From 974467aa1e8a1cdfcd25bd2218d71e70c0dceddc Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Mon, 22 Aug 2016 10:14:27 +0700 Subject: [PATCH 041/487] IGNITE-3702 Fixed missing query index type setter generation. Added tests. Minor code improvements. --- .../dialect/DatabaseMetadataDialect.java | 18 + .../parser/dialect/JdbcMetadataDialect.java | 5 +- .../parser/dialect/OracleMetadataDialect.java | 5 +- .../schema/generator/CodeGenerator.java | 7 +- .../schema/parser/DatabaseMetadataParser.java | 4 +- .../ignite/schema/ui/SchemaImportApp.java | 3 +- .../schema/test/AbstractSchemaImportTest.java | 27 +- .../test/generator/CodeGeneratorTest.java | 28 +- .../test/generator/XmlGeneratorTest.java | 22 +- .../ignite/schema/test/model/CacheConfig.txt | 409 ++++++++++++++++++ .../test/model/ignite-type-metadata.xml | 50 ++- .../test/parser/DbMetadataParserTest.java | 6 +- 12 files changed, 538 insertions(+), 46 deletions(-) create mode 100644 modules/schema-import/src/test/java/org/apache/ignite/schema/test/model/CacheConfig.txt diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java index f34eee4440fee..5ef088ad1799c 100644 --- a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java +++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java @@ -21,10 +21,12 @@ import java.sql.SQLException; import java.util.Collection; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; import java.util.Set; import org.apache.ignite.cache.QueryIndex; +import org.apache.ignite.cache.QueryIndexType; import org.apache.ignite.schema.parser.DbColumn; import org.apache.ignite.schema.parser.DbTable; @@ -72,4 +74,20 @@ public Set systemSchemas() { protected DbTable table(String schema, String tbl, Collection cols, Collectionidxs) { return new DbTable(schema, tbl, cols, idxs); } + + /** + * Create index descriptor. + * + * @param idxName Index name. + * @return New initialized {@code QueryIndex} instance. + */ + protected QueryIndex index(String idxName) { + QueryIndex idx = new QueryIndex(); + + idx.setName(idxName); + idx.setIndexType(QueryIndexType.SORTED); + idx.setFields(new LinkedHashMap()); + + return idx; + } } diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java index 0b54b2554e1da..235c84c3edfd0 100644 --- a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java +++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java @@ -171,10 +171,7 @@ protected boolean useSchema() { QueryIndex idx = idxs.get(idxName); if (idx == null) { - idx = new QueryIndex(); - idx.setName(idxName); - idx.setIndexType(QueryIndexType.SORTED); - idx.setFields(new LinkedHashMap()); + idx = index(idxName); idxs.put(idxName, idx); } diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java index ed5cdc53aa9b9..35356558e91d2 100644 --- a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java +++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java @@ -272,10 +272,7 @@ private Collection indexes(PreparedStatement stmt, String owner, Str QueryIndex idx = idxs.get(idxName); if (idx == null) { - idx = new QueryIndex(); - idx.setName(idxName); - idx.setIndexType(QueryIndexType.SORTED); - idx.setFields(new LinkedHashMap()); + idx = index(idxName); idxs.put(idxName, idx); } diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/generator/CodeGenerator.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/generator/CodeGenerator.java index 01fcfa6b5fdc1..59dfdc45737d4 100644 --- a/modules/schema-import/src/main/java/org/apache/ignite/schema/generator/CodeGenerator.java +++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/generator/CodeGenerator.java @@ -728,6 +728,9 @@ public static void snippet(Collection pojos, String pkg, boolean add2(src, "idx.setName(\"" + idx.getName() + "\");"); add0(src, ""); + add2(src, "idx.setIndexType(QueryIndexType." + idx.getIndexType() + ");"); + add0(src, ""); + add2(src, (firstIdx ? "LinkedHashMap " : "") + "idxFlds = new LinkedHashMap<>();"); add0(src, ""); @@ -781,7 +784,7 @@ public static void snippet(Collection pojos, String pkg, boolean add2(src, "ccfg.setWriteThrough(true);"); add0(src, ""); - add2(src, "// Configure JDBC types. "); + add2(src, "// Configure JDBC types."); add2(src, "Collection jdbcTypes = new ArrayList<>();"); add0(src, ""); @@ -794,7 +797,7 @@ public static void snippet(Collection pojos, String pkg, boolean add0(src, ""); - add2(src, "// Configure query entities. "); + add2(src, "// Configure query entities."); add2(src, "Collection qryEntities = new ArrayList<>();"); add0(src, ""); diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DatabaseMetadataParser.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DatabaseMetadataParser.java index b86346f6dc74f..6d87ed512e892 100644 --- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DatabaseMetadataParser.java +++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DatabaseMetadataParser.java @@ -99,8 +99,8 @@ public static ObservableList parse(Connection conn, List if (!children.isEmpty()) { parent.children(children); - res.add(parent); - res.addAll(children); + res.add(parent); // Add schema description. + res.addAll(children); // Add tables in schema. } } diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java index a02d31e2d0581..6305341f2f5de 100644 --- a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java +++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java @@ -798,8 +798,9 @@ private int compareVersionNumbers(String v1, String v2) { /** * Check that system has Nautilus. + * * @return {@code True} when Nautilus is installed or {@code false} otherwise. - * @throws IOException + * @throws IOException If failed to detect Nautilus. */ private boolean canUseNautilus() throws IOException { if (U.isUnix() || new File("/usr/bin/xdg-mime").canExecute() || new File("/usr/bin/nautilus").canExecute()) { diff --git a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/AbstractSchemaImportTest.java b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/AbstractSchemaImportTest.java index d4534be5d19f5..8cb1196f4e49c 100644 --- a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/AbstractSchemaImportTest.java +++ b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/AbstractSchemaImportTest.java @@ -35,8 +35,6 @@ import org.apache.ignite.schema.ui.ConfirmCallable; import org.apache.ignite.schema.ui.MessageBox; -import static org.apache.ignite.schema.ui.MessageBox.Result.YES_TO_ALL; - /** * Base functional for Ignite Schema Import utility tests. */ @@ -48,13 +46,16 @@ public abstract class AbstractSchemaImportTest extends TestCase { protected static final String OUT_DIR_PATH = System.getProperty("java.io.tmpdir") + "/ignite-schema-import/out"; /** Auto confirmation of file conflicts. */ - protected final ConfirmCallable askOverwrite = new ConfirmCallable(null, "") { + protected static final ConfirmCallable YES_TO_ALL = new ConfirmCallable(null, "") { @Override public MessageBox.Result confirm(String msg) { - return YES_TO_ALL; + return MessageBox.Result.YES_TO_ALL; } }; - /** List of generated for test database POJO objects. */ + /** List of ALL object parsed from test database. */ + protected List all; + + /** List of ONLY POJO descriptors. */ protected List pojos; /** {@inheritDoc} */ @@ -99,6 +100,10 @@ public abstract class AbstractSchemaImportTest extends TestCase { " arrCol BINARY(10)," + " FIELD_WITH_ALIAS VARCHAR(10))"); + stmt.executeUpdate("CREATE INDEX IF NOT EXISTS IDX_1 ON OBJECTS (INTCOL ASC, LONGCOL ASC)"); + + stmt.executeUpdate("CREATE INDEX IF NOT EXISTS IDX_2 ON OBJECTS (INTCOL ASC, LONGCOL DESC)"); + stmt.executeUpdate("CREATE SCHEMA IF NOT EXISTS TESTSCHEMA"); stmt.executeUpdate("CREATE TABLE IF NOT EXISTS TESTSCHEMA.TST(pk INTEGER PRIMARY KEY, " + @@ -118,13 +123,23 @@ public abstract class AbstractSchemaImportTest extends TestCase { " arrCol BINARY(10)," + " FIELD_WITH_ALIAS VARCHAR(10))"); + stmt.executeUpdate("CREATE INDEX IF NOT EXISTS IDX_3 ON TESTSCHEMA.TST (INTCOL ASC, LONGCOL ASC)"); + + stmt.executeUpdate("CREATE INDEX IF NOT EXISTS IDX_4 ON TESTSCHEMA.TST (INTCOL ASC, LONGCOL DESC)"); + conn.commit(); U.closeQuiet(stmt); List schemas = new ArrayList<>(); - pojos = DatabaseMetadataParser.parse(conn, schemas, false); + all = DatabaseMetadataParser.parse(conn, schemas, false); + + pojos = new ArrayList<>(); + + for (PojoDescriptor pojo : all) + if (pojo.parent() != null) + pojos.add(pojo); U.closeQuiet(conn); } diff --git a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/generator/CodeGeneratorTest.java b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/generator/CodeGeneratorTest.java index 2fef9e1eb9f1b..0917139026b64 100644 --- a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/generator/CodeGeneratorTest.java +++ b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/generator/CodeGeneratorTest.java @@ -29,32 +29,44 @@ public class CodeGeneratorTest extends AbstractSchemaImportTest { /** Marker string to skip date generation while comparing.*/ private static final String GEN_PTRN = "Code generated by Apache Ignite Schema Import utility"; + /** Test package. */ + private static final String TEST_PACKAGE = "org.apache.ignite.schema.test.model"; + + /** Path to generated model. */ + private static final String TEST_PATH = "org/apache/ignite/schema/test/model"; + /** * Test that POJOs generated correctly. */ public void testPojoGeneration() throws Exception { - String pkg = "org.apache.ignite.schema.test.model"; - String intPath = "org/apache/ignite/schema/test/model"; - Boolean containsSchema = false; - for (PojoDescriptor pojo : pojos) { + for (PojoDescriptor pojo : all) { if (pojo.valueClassName().isEmpty()) containsSchema = true; else { - CodeGenerator.pojos(pojo, OUT_DIR_PATH, pkg, true, true, askOverwrite); + CodeGenerator.pojos(pojo, OUT_DIR_PATH, TEST_PACKAGE, true, true, YES_TO_ALL); assertTrue("Generated key class POJO content is differ from expected for type " + pojo.keyClassName(), - compareFiles(pojo.keyClassName(), intPath, GEN_PTRN)); + compareFiles(pojo.keyClassName(), TEST_PATH, GEN_PTRN)); assertTrue("Generated value class POJO content is differ from expected for type " + pojo.valueClassName(), - compareFiles(pojo.valueClassName(), intPath, GEN_PTRN)); + compareFiles(pojo.valueClassName(), TEST_PATH, GEN_PTRN)); } } assertTrue("Generated POJOs does not contains schema.", containsSchema); } + /** + * Test that configuration generated correctly. + */ + public void testConfigGeneration() throws Exception { + CodeGenerator.snippet(pojos, TEST_PACKAGE, true, true, OUT_DIR_PATH, YES_TO_ALL); + + assertTrue("Generated configuration is differ from expected", compareFiles("CacheConfig", TEST_PATH, GEN_PTRN)); + } + /** * @param typeName Type name. * @param intPath Internal path. @@ -66,4 +78,4 @@ private boolean compareFiles(String typeName, String intPath, String excludePtrn return compareFilesInt(getClass().getResourceAsStream("/" + relPath + ".txt"), new File(OUT_DIR_PATH + "/" + relPath + ".java"), excludePtrn); } -} \ No newline at end of file +} diff --git a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/generator/XmlGeneratorTest.java b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/generator/XmlGeneratorTest.java index 2ed36e9de2796..8b123926b42ca 100644 --- a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/generator/XmlGeneratorTest.java +++ b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/generator/XmlGeneratorTest.java @@ -18,33 +18,25 @@ package org.apache.ignite.schema.test.generator; import java.io.File; -import java.util.ArrayList; -import java.util.Collection; import org.apache.ignite.schema.generator.XmlGenerator; -import org.apache.ignite.schema.model.PojoDescriptor; import org.apache.ignite.schema.test.AbstractSchemaImportTest; /** * Tests for XML generator. */ public class XmlGeneratorTest extends AbstractSchemaImportTest { + /** */ + private static final String TEST_XML_FILE_NAME = "ignite-type-metadata.xml"; + /** * Test that XML generated correctly. */ public void testXmlGeneration() throws Exception { - Collection all = new ArrayList<>(); - - for (PojoDescriptor pojo : pojos) - if (pojo.parent() != null) - all.add(pojo); - - String fileName = "ignite-type-metadata.xml"; - - XmlGenerator.generate("org.apache.ignite.schema.test.model", all, true, true, new File(OUT_DIR_PATH, fileName), - askOverwrite); + XmlGenerator.generate("org.apache.ignite.schema.test.model", pojos, true, true, + new File(OUT_DIR_PATH, TEST_XML_FILE_NAME), YES_TO_ALL); assertTrue("Generated XML file content is differ from expected one", - compareFilesInt(getClass().getResourceAsStream("/org/apache/ignite/schema/test/model/" + fileName), - new File(OUT_DIR_PATH + "/" + fileName), "XML generated by Apache Ignite Schema Import utility")); + compareFilesInt(getClass().getResourceAsStream("/org/apache/ignite/schema/test/model/" + TEST_XML_FILE_NAME), + new File(OUT_DIR_PATH, TEST_XML_FILE_NAME), "XML generated by Apache Ignite Schema Import utility")); } } diff --git a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/model/CacheConfig.txt b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/model/CacheConfig.txt new file mode 100644 index 0000000000000..9573eb719963c --- /dev/null +++ b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/model/CacheConfig.txt @@ -0,0 +1,409 @@ +/* + * 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.schema.test.model; + +import java.sql.*; +import java.util.*; + +import org.apache.ignite.cache.*; +import org.apache.ignite.cache.store.jdbc.*; +import org.apache.ignite.configuration.*; + +/** + * CacheConfig definition. + * + * Code generated by Apache Ignite Schema Import utility: 08/18/2016. + */ +public class CacheConfig { + /** + * Create JDBC type for OBJECTS. + * + * @param cacheName Cache name. + * @return Configured JDBC type. + */ + private static JdbcType jdbcTypeObjects(String cacheName) { + JdbcType jdbcType = new JdbcType(); + + jdbcType.setCacheName(cacheName); + jdbcType.setDatabaseSchema("PUBLIC"); + jdbcType.setDatabaseTable("OBJECTS"); + jdbcType.setKeyType("org.apache.ignite.schema.test.model.ObjectsKey"); + jdbcType.setValueType("org.apache.ignite.schema.test.model.Objects"); + + // Key fields for OBJECTS. + Collection keys = new ArrayList<>(); + keys.add(new JdbcTypeField(Types.INTEGER, "PK", int.class, "pk")); + jdbcType.setKeyFields(keys.toArray(new JdbcTypeField[keys.size()])); + + // Value fields for OBJECTS. + Collection vals = new ArrayList<>(); + vals.add(new JdbcTypeField(Types.INTEGER, "PK", int.class, "pk")); + vals.add(new JdbcTypeField(Types.BOOLEAN, "BOOLCOL", Boolean.class, "boolcol")); + vals.add(new JdbcTypeField(Types.TINYINT, "BYTECOL", Byte.class, "bytecol")); + vals.add(new JdbcTypeField(Types.SMALLINT, "SHORTCOL", Short.class, "shortcol")); + vals.add(new JdbcTypeField(Types.INTEGER, "INTCOL", Integer.class, "intcol")); + vals.add(new JdbcTypeField(Types.BIGINT, "LONGCOL", Long.class, "longcol")); + vals.add(new JdbcTypeField(Types.REAL, "FLOATCOL", Float.class, "floatcol")); + vals.add(new JdbcTypeField(Types.DOUBLE, "DOUBLECOL", Double.class, "doublecol")); + vals.add(new JdbcTypeField(Types.DOUBLE, "DOUBLECOL2", Double.class, "doublecol2")); + vals.add(new JdbcTypeField(Types.DECIMAL, "BIGDECIMALCOL", java.math.BigDecimal.class, "bigdecimalcol")); + vals.add(new JdbcTypeField(Types.VARCHAR, "STRCOL", String.class, "strcol")); + vals.add(new JdbcTypeField(Types.DATE, "DATECOL", java.sql.Date.class, "datecol")); + vals.add(new JdbcTypeField(Types.TIME, "TIMECOL", java.sql.Time.class, "timecol")); + vals.add(new JdbcTypeField(Types.TIMESTAMP, "TSCOL", java.sql.Timestamp.class, "tscol")); + vals.add(new JdbcTypeField(Types.VARBINARY, "ARRCOL", Object.class, "arrcol")); + vals.add(new JdbcTypeField(Types.VARCHAR, "FIELD_WITH_ALIAS", String.class, "fieldWithAlias")); + jdbcType.setValueFields(vals.toArray(new JdbcTypeField[vals.size()])); + + return jdbcType; + } + + /** + * Create SQL Query descriptor for OBJECTS. + * + * @return Configured query entity. + */ + private static QueryEntity queryEntityObjects() { + QueryEntity qryEntity = new QueryEntity(); + + qryEntity.setKeyType("org.apache.ignite.schema.test.model.ObjectsKey"); + qryEntity.setValueType("org.apache.ignite.schema.test.model.Objects"); + + // Query fields for OBJECTS. + LinkedHashMap fields = new LinkedHashMap<>(); + + fields.put("pk", "java.lang.Integer"); + fields.put("boolcol", "java.lang.Boolean"); + fields.put("bytecol", "java.lang.Byte"); + fields.put("shortcol", "java.lang.Short"); + fields.put("intcol", "java.lang.Integer"); + fields.put("longcol", "java.lang.Long"); + fields.put("floatcol", "java.lang.Float"); + fields.put("doublecol", "java.lang.Double"); + fields.put("doublecol2", "java.lang.Double"); + fields.put("bigdecimalcol", "java.math.BigDecimal"); + fields.put("strcol", "java.lang.String"); + fields.put("datecol", "java.sql.Date"); + fields.put("timecol", "java.sql.Time"); + fields.put("tscol", "java.sql.Timestamp"); + fields.put("arrcol", "java.lang.Object"); + fields.put("fieldWithAlias", "java.lang.String"); + + qryEntity.setFields(fields); + + // Aliases for fields. + Map aliases = new HashMap<>(); + + aliases.put("fieldWithAlias", "FIELD_WITH_ALIAS"); + + qryEntity.setAliases(aliases); + + // Indexes for OBJECTS. + Collection idxs = new ArrayList<>(); + + idxs.add(new QueryIndex("pk", true, "PRIMARY_KEY_C")); + + QueryIndex idx = new QueryIndex(); + + idx.setName("IDX_1"); + + idx.setIndexType(QueryIndexType.SORTED); + + LinkedHashMap idxFlds = new LinkedHashMap<>(); + + idxFlds.put("intcol", true); + idxFlds.put("longcol", true); + + idx.setFields(idxFlds); + + idxs.add(idx); + + idx = new QueryIndex(); + + idx.setName("IDX_2"); + + idx.setIndexType(QueryIndexType.SORTED); + + idxFlds = new LinkedHashMap<>(); + + idxFlds.put("intcol", true); + idxFlds.put("longcol", false); + + idx.setFields(idxFlds); + + idxs.add(idx); + + qryEntity.setIndexes(idxs); + + return qryEntity; + } + + /** + * Create JDBC type for PRIMITIVES. + * + * @param cacheName Cache name. + * @return Configured JDBC type. + */ + private static JdbcType jdbcTypePrimitives(String cacheName) { + JdbcType jdbcType = new JdbcType(); + + jdbcType.setCacheName(cacheName); + jdbcType.setDatabaseSchema("PUBLIC"); + jdbcType.setDatabaseTable("PRIMITIVES"); + jdbcType.setKeyType("org.apache.ignite.schema.test.model.PrimitivesKey"); + jdbcType.setValueType("org.apache.ignite.schema.test.model.Primitives"); + + // Key fields for PRIMITIVES. + Collection keys = new ArrayList<>(); + keys.add(new JdbcTypeField(Types.INTEGER, "PK", int.class, "pk")); + jdbcType.setKeyFields(keys.toArray(new JdbcTypeField[keys.size()])); + + // Value fields for PRIMITIVES. + Collection vals = new ArrayList<>(); + vals.add(new JdbcTypeField(Types.INTEGER, "PK", int.class, "pk")); + vals.add(new JdbcTypeField(Types.BOOLEAN, "BOOLCOL", boolean.class, "boolcol")); + vals.add(new JdbcTypeField(Types.TINYINT, "BYTECOL", byte.class, "bytecol")); + vals.add(new JdbcTypeField(Types.SMALLINT, "SHORTCOL", short.class, "shortcol")); + vals.add(new JdbcTypeField(Types.INTEGER, "INTCOL", int.class, "intcol")); + vals.add(new JdbcTypeField(Types.BIGINT, "LONGCOL", long.class, "longcol")); + vals.add(new JdbcTypeField(Types.REAL, "FLOATCOL", float.class, "floatcol")); + vals.add(new JdbcTypeField(Types.DOUBLE, "DOUBLECOL", double.class, "doublecol")); + vals.add(new JdbcTypeField(Types.DOUBLE, "DOUBLECOL2", double.class, "doublecol2")); + vals.add(new JdbcTypeField(Types.DECIMAL, "BIGDECIMALCOL", java.math.BigDecimal.class, "bigdecimalcol")); + vals.add(new JdbcTypeField(Types.VARCHAR, "STRCOL", String.class, "strcol")); + vals.add(new JdbcTypeField(Types.DATE, "DATECOL", java.sql.Date.class, "datecol")); + vals.add(new JdbcTypeField(Types.TIME, "TIMECOL", java.sql.Time.class, "timecol")); + vals.add(new JdbcTypeField(Types.TIMESTAMP, "TSCOL", java.sql.Timestamp.class, "tscol")); + vals.add(new JdbcTypeField(Types.VARBINARY, "ARRCOL", Object.class, "arrcol")); + vals.add(new JdbcTypeField(Types.VARCHAR, "FIELD_WITH_ALIAS", String.class, "fieldWithAlias")); + jdbcType.setValueFields(vals.toArray(new JdbcTypeField[vals.size()])); + + return jdbcType; + } + + /** + * Create SQL Query descriptor for PRIMITIVES. + * + * @return Configured query entity. + */ + private static QueryEntity queryEntityPrimitives() { + QueryEntity qryEntity = new QueryEntity(); + + qryEntity.setKeyType("org.apache.ignite.schema.test.model.PrimitivesKey"); + qryEntity.setValueType("org.apache.ignite.schema.test.model.Primitives"); + + // Query fields for PRIMITIVES. + LinkedHashMap fields = new LinkedHashMap<>(); + + fields.put("pk", "java.lang.Integer"); + fields.put("boolcol", "java.lang.Boolean"); + fields.put("bytecol", "java.lang.Byte"); + fields.put("shortcol", "java.lang.Short"); + fields.put("intcol", "java.lang.Integer"); + fields.put("longcol", "java.lang.Long"); + fields.put("floatcol", "java.lang.Float"); + fields.put("doublecol", "java.lang.Double"); + fields.put("doublecol2", "java.lang.Double"); + fields.put("bigdecimalcol", "java.math.BigDecimal"); + fields.put("strcol", "java.lang.String"); + fields.put("datecol", "java.sql.Date"); + fields.put("timecol", "java.sql.Time"); + fields.put("tscol", "java.sql.Timestamp"); + fields.put("arrcol", "java.lang.Object"); + fields.put("fieldWithAlias", "java.lang.String"); + + qryEntity.setFields(fields); + + // Aliases for fields. + Map aliases = new HashMap<>(); + + aliases.put("fieldWithAlias", "FIELD_WITH_ALIAS"); + + qryEntity.setAliases(aliases); + + // Indexes for PRIMITIVES. + Collection idxs = new ArrayList<>(); + + idxs.add(new QueryIndex("pk", true, "PRIMARY_KEY_D")); + + qryEntity.setIndexes(idxs); + + return qryEntity; + } + + /** + * Create JDBC type for TST. + * + * @param cacheName Cache name. + * @return Configured JDBC type. + */ + private static JdbcType jdbcTypeTst(String cacheName) { + JdbcType jdbcType = new JdbcType(); + + jdbcType.setCacheName(cacheName); + jdbcType.setDatabaseSchema("TESTSCHEMA"); + jdbcType.setDatabaseTable("TST"); + jdbcType.setKeyType("org.apache.ignite.schema.test.model.TstKey"); + jdbcType.setValueType("org.apache.ignite.schema.test.model.Tst"); + + // Key fields for TST. + Collection keys = new ArrayList<>(); + keys.add(new JdbcTypeField(Types.INTEGER, "PK", int.class, "pk")); + jdbcType.setKeyFields(keys.toArray(new JdbcTypeField[keys.size()])); + + // Value fields for TST. + Collection vals = new ArrayList<>(); + vals.add(new JdbcTypeField(Types.INTEGER, "PK", int.class, "pk")); + vals.add(new JdbcTypeField(Types.BOOLEAN, "BOOLCOL", boolean.class, "boolcol")); + vals.add(new JdbcTypeField(Types.TINYINT, "BYTECOL", byte.class, "bytecol")); + vals.add(new JdbcTypeField(Types.SMALLINT, "SHORTCOL", short.class, "shortcol")); + vals.add(new JdbcTypeField(Types.INTEGER, "INTCOL", int.class, "intcol")); + vals.add(new JdbcTypeField(Types.BIGINT, "LONGCOL", long.class, "longcol")); + vals.add(new JdbcTypeField(Types.REAL, "FLOATCOL", float.class, "floatcol")); + vals.add(new JdbcTypeField(Types.DOUBLE, "DOUBLECOL", double.class, "doublecol")); + vals.add(new JdbcTypeField(Types.DOUBLE, "DOUBLECOL2", double.class, "doublecol2")); + vals.add(new JdbcTypeField(Types.DECIMAL, "BIGDECIMALCOL", java.math.BigDecimal.class, "bigdecimalcol")); + vals.add(new JdbcTypeField(Types.VARCHAR, "STRCOL", String.class, "strcol")); + vals.add(new JdbcTypeField(Types.DATE, "DATECOL", java.sql.Date.class, "datecol")); + vals.add(new JdbcTypeField(Types.TIME, "TIMECOL", java.sql.Time.class, "timecol")); + vals.add(new JdbcTypeField(Types.TIMESTAMP, "TSCOL", java.sql.Timestamp.class, "tscol")); + vals.add(new JdbcTypeField(Types.VARBINARY, "ARRCOL", Object.class, "arrcol")); + vals.add(new JdbcTypeField(Types.VARCHAR, "FIELD_WITH_ALIAS", String.class, "fieldWithAlias")); + jdbcType.setValueFields(vals.toArray(new JdbcTypeField[vals.size()])); + + return jdbcType; + } + + /** + * Create SQL Query descriptor for TST. + * + * @return Configured query entity. + */ + private static QueryEntity queryEntityTst() { + QueryEntity qryEntity = new QueryEntity(); + + qryEntity.setKeyType("org.apache.ignite.schema.test.model.TstKey"); + qryEntity.setValueType("org.apache.ignite.schema.test.model.Tst"); + + // Query fields for TST. + LinkedHashMap fields = new LinkedHashMap<>(); + + fields.put("pk", "java.lang.Integer"); + fields.put("boolcol", "java.lang.Boolean"); + fields.put("bytecol", "java.lang.Byte"); + fields.put("shortcol", "java.lang.Short"); + fields.put("intcol", "java.lang.Integer"); + fields.put("longcol", "java.lang.Long"); + fields.put("floatcol", "java.lang.Float"); + fields.put("doublecol", "java.lang.Double"); + fields.put("doublecol2", "java.lang.Double"); + fields.put("bigdecimalcol", "java.math.BigDecimal"); + fields.put("strcol", "java.lang.String"); + fields.put("datecol", "java.sql.Date"); + fields.put("timecol", "java.sql.Time"); + fields.put("tscol", "java.sql.Timestamp"); + fields.put("arrcol", "java.lang.Object"); + fields.put("fieldWithAlias", "java.lang.String"); + + qryEntity.setFields(fields); + + // Aliases for fields. + Map aliases = new HashMap<>(); + + aliases.put("fieldWithAlias", "FIELD_WITH_ALIAS"); + + qryEntity.setAliases(aliases); + + // Indexes for TST. + Collection idxs = new ArrayList<>(); + + idxs.add(new QueryIndex("pk", true, "PRIMARY_KEY_1")); + + QueryIndex idx = new QueryIndex(); + + idx.setName("IDX_3"); + + idx.setIndexType(QueryIndexType.SORTED); + + LinkedHashMap idxFlds = new LinkedHashMap<>(); + + idxFlds.put("intcol", true); + idxFlds.put("longcol", true); + + idx.setFields(idxFlds); + + idxs.add(idx); + + idx = new QueryIndex(); + + idx.setName("IDX_4"); + + idx.setIndexType(QueryIndexType.SORTED); + + idxFlds = new LinkedHashMap<>(); + + idxFlds.put("intcol", true); + idxFlds.put("longcol", false); + + idx.setFields(idxFlds); + + idxs.add(idx); + + qryEntity.setIndexes(idxs); + + return qryEntity; + } + + /** + * Configure cache. + * + * @param cacheName Cache name. + * @param storeFactory Cache store factory. + * @return Cache configuration. + */ + public static CacheConfiguration cache(String cacheName, CacheJdbcPojoStoreFactory storeFactory) { + if (storeFactory == null) + throw new IllegalArgumentException("Cache store factory cannot be null."); + + CacheConfiguration ccfg = new CacheConfiguration<>(cacheName); + + ccfg.setCacheStoreFactory(storeFactory); + ccfg.setReadThrough(true); + ccfg.setWriteThrough(true); + + // Configure JDBC types. + Collection jdbcTypes = new ArrayList<>(); + + jdbcTypes.add(jdbcTypeObjects(cacheName)); + jdbcTypes.add(jdbcTypePrimitives(cacheName)); + jdbcTypes.add(jdbcTypeTst(cacheName)); + + storeFactory.setTypes(jdbcTypes.toArray(new JdbcType[jdbcTypes.size()])); + + // Configure query entities. + Collection qryEntities = new ArrayList<>(); + + qryEntities.add(queryEntityObjects()); + qryEntities.add(queryEntityPrimitives()); + qryEntities.add(queryEntityTst()); + + ccfg.setQueryEntities(qryEntities); + + return ccfg; + } +} diff --git a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/model/ignite-type-metadata.xml b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/model/ignite-type-metadata.xml index 514445f6fa26b..fc481f03826c6 100644 --- a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/model/ignite-type-metadata.xml +++ b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/model/ignite-type-metadata.xml @@ -18,7 +18,7 @@ --> + + + + + + + + + + + + + + + + + + + + + + + + @@ -612,6 +636,30 @@ + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/parser/DbMetadataParserTest.java b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/parser/DbMetadataParserTest.java index f918012036552..a550bfcfef933 100644 --- a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/parser/DbMetadataParserTest.java +++ b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/parser/DbMetadataParserTest.java @@ -106,7 +106,7 @@ private void checkType(PojoDescriptor type) { * * @param type Type to check. */ - public void checkSchemaHasFullDbName(PojoDescriptor type) { + private void checkSchemaHasFullDbName(PojoDescriptor type) { assertNotNull("The DB schema should have a non-null fullDbName", type.fullDbName()); } @@ -114,9 +114,9 @@ public void checkSchemaHasFullDbName(PojoDescriptor type) { * Test that metadata generated correctly. */ public void testCheckMetadata() { - assertEquals("Metadata should contain 5 elements", 5, pojos.size()); + assertEquals("Metadata should contain 5 elements", 5, all.size()); - Iterator it = pojos.iterator(); + Iterator it = all.iterator(); PojoDescriptor schema = it.next(); From d399db92ab4e147a3933a42dd5635b225665ac63 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Mon, 22 Aug 2016 15:00:06 +0700 Subject: [PATCH 042/487] IGNITE-3708 Fixed multithreaded loading entries for MySql. --- .../store/jdbc/CacheAbstractJdbcStore.java | 43 +++++++++++-------- .../store/jdbc/dialect/BasicJdbcDialect.java | 7 ++- .../cache/store/jdbc/dialect/JdbcDialect.java | 11 ++++- .../store/jdbc/dialect/MySQLDialect.java | 18 +++++++- .../CacheJdbcPojoStoreFactorySelfTest.java | 11 +++-- 5 files changed, 65 insertions(+), 25 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java index c16f2c6a8516d..aad05e085535d 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java @@ -495,10 +495,11 @@ else if (type == Float.class) * @param clo Closure that will be applied to loaded values. * @param lowerBound Lower bound for range. * @param upperBound Upper bound for range. + * @param fetchSize Number of rows to fetch from DB. * @return Callable for pool submit. */ private Callable loadCacheRange(final EntryMapping em, final IgniteBiInClosure clo, - @Nullable final Object[] lowerBound, @Nullable final Object[] upperBound) { + @Nullable final Object[] lowerBound, @Nullable final Object[] upperBound, final int fetchSize) { return new Callable() { @Override public Void call() throws Exception { Connection conn = null; @@ -512,6 +513,8 @@ private Callable loadCacheRange(final EntryMapping em, final IgniteBiInClo ? em.loadCacheQry : em.loadCacheRangeQuery(lowerBound != null, upperBound != null)); + stmt.setFetchSize(fetchSize); + int idx = 1; if (lowerBound != null) @@ -555,7 +558,7 @@ private Callable loadCacheRange(final EntryMapping em, final IgniteBiInClo * @return Callable for pool submit. */ private Callable loadCacheFull(EntryMapping m, IgniteBiInClosure clo) { - return loadCacheRange(m, clo, null, null); + return loadCacheRange(m, clo, null, null, dialect.getFetchSize()); } /** @@ -811,10 +814,6 @@ private EntryMapping entryMapping(String cacheName, Object typeId) throws CacheE for (EntryMapping em : entryMappings) { if (parallelLoadCacheMinThreshold > 0) { - if (log.isDebugEnabled()) - log.debug("Multithread loading entries from db [cache=" + U.maskName(cacheName) + - ", keyType=" + em.keyType() + " ]"); - Connection conn = null; try { @@ -827,6 +826,10 @@ private EntryMapping entryMapping(String cacheName, Object typeId) throws CacheE ResultSet rs = stmt.executeQuery(); if (rs.next()) { + if (log.isDebugEnabled()) + log.debug("Multithread loading entries from db [cache=" + U.maskName(cacheName) + + ", keyType=" + em.keyType() + " ]"); + int keyCnt = em.keyCols.size(); Object[] upperBound = new Object[keyCnt]; @@ -834,7 +837,7 @@ private EntryMapping entryMapping(String cacheName, Object typeId) throws CacheE for (int i = 0; i < keyCnt; i++) upperBound[i] = rs.getObject(i + 1); - futs.add(pool.submit(loadCacheRange(em, clo, null, upperBound))); + futs.add(pool.submit(loadCacheRange(em, clo, null, upperBound, 0))); while (rs.next()) { Object[] lowerBound = upperBound; @@ -844,28 +847,28 @@ private EntryMapping entryMapping(String cacheName, Object typeId) throws CacheE for (int i = 0; i < keyCnt; i++) upperBound[i] = rs.getObject(i + 1); - futs.add(pool.submit(loadCacheRange(em, clo, lowerBound, upperBound))); + futs.add(pool.submit(loadCacheRange(em, clo, lowerBound, upperBound, 0))); } - futs.add(pool.submit(loadCacheRange(em, clo, upperBound, null))); + futs.add(pool.submit(loadCacheRange(em, clo, upperBound, null, 0))); + + continue; } - else - futs.add(pool.submit(loadCacheFull(em, clo))); } - catch (SQLException ignored) { - futs.add(pool.submit(loadCacheFull(em, clo))); + catch (SQLException e) { + log.warning("Failed to load entries from db in multithreaded mode [cache=" + U.maskName(cacheName) + + ", keyType=" + em.keyType() + " ]", e); } finally { U.closeQuiet(conn); } } - else { - if (log.isDebugEnabled()) - log.debug("Single thread loading entries from db [cache=" + U.maskName(cacheName) + - ", keyType=" + em.keyType() + " ]"); - futs.add(pool.submit(loadCacheFull(em, clo))); - } + if (log.isDebugEnabled()) + log.debug("Single thread loading entries from db [cache=" + U.maskName(cacheName) + + ", keyType=" + em.keyType() + " ]"); + + futs.add(pool.submit(loadCacheFull(em, clo))); } } @@ -1926,6 +1929,8 @@ private LoadCacheCustomQueryWorker(EntryMapping em, String qry, IgniteBiInClosur stmt = conn.prepareStatement(qry); + stmt.setFetchSize(dialect.getFetchSize()); + ResultSet rs = stmt.executeQuery(); ResultSetMetaData meta = rs.getMetaData(); diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java index abb59d38f4260..cd9c9868a9495 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java @@ -274,4 +274,9 @@ private static String where(Collection keyCols, int keyCnt) { public void setMaxParameterCount(int maxParamsCnt) { this.maxParamsCnt = maxParamsCnt; } -} \ No newline at end of file + + /** {@inheritDoc} */ + @Override public int getFetchSize() { + return 0; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java index 38e981f8b8209..9daa00b36bec5 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java @@ -115,4 +115,13 @@ public String loadQuery(String fullTblName, Collection keyCols, Iterable * @return Max query parameters count. */ public int getMaxParameterCount(); -} \ No newline at end of file + + /** + * Gives the JDBC driver a hint how many rows should be fetched from the database when more rows are needed. + * If the value specified is zero, then the hint is ignored. + * The default value is zero. + * + * @return The fetch size for result sets. + */ + public int getFetchSize(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java index f7512a779238c..84e6d05f56f46 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java @@ -28,6 +28,15 @@ public class MySQLDialect extends BasicJdbcDialect { /** */ private static final long serialVersionUID = 0L; + /** {@inheritDoc} */ + @Override public String loadCacheSelectRangeQuery(String fullTblName, Collection keyCols) { + String cols = mkString(keyCols, ","); + + return String.format("SELECT %s " + + "FROM (SELECT %s, @rownum := @rownum + 1 AS rn FROM %s, (SELECT @rownum := 0) r ORDER BY %s) as r " + + "WHERE mod(rn, ?) = 0", cols, cols, fullTblName, cols); + } + /** {@inheritDoc} */ @Override public boolean hasMerge() { return true; @@ -48,4 +57,11 @@ public class MySQLDialect extends BasicJdbcDialect { return String.format("INSERT INTO %s (%s) VALUES (%s) ON DUPLICATE KEY UPDATE %s", fullTblName, mkString(cols, ", "), repeat("?", cols.size(), "", ",", ""), updPart); } -} \ No newline at end of file + + /** {@inheritDoc} */ + @Override public int getFetchSize() { + // Workaround for known issue with MySQL large result set. + // See: http://dev.mysql.com/doc/connector-j/5.1/en/connector-j-reference-implementation-notes.html + return Integer.MIN_VALUE; + } +} diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java index 2f36017beb5cf..dfa14529c4f50 100644 --- a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java @@ -129,9 +129,9 @@ private void checkStore(IgniteCache cache, Class dataSrcClas } /** - * + * Dummy JDBC dialect that does nothing. */ - public static class DummyDialect implements JdbcDialect, Serializable { + public static class DummyDialect implements JdbcDialect { /** {@inheritDoc} */ @Override public String loadCacheSelectRangeQuery(String fullTblName, Collection keyCols) { return null; @@ -185,5 +185,10 @@ public static class DummyDialect implements JdbcDialect, Serializable { @Override public int getMaxParameterCount() { return 0; } + + /** {@inheritDoc} */ + @Override public int getFetchSize() { + return 0; + } } -} \ No newline at end of file +} From 5b57fc04866677fbf1221a253fe28905dbfc2d7d Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 22 Aug 2016 17:16:45 +0300 Subject: [PATCH 043/487] IGNITE-3694: IGFS: Remove invalid assertion in IgfsImpl.newBatch() method. --- .../apache/ignite/internal/processors/igfs/IgfsImpl.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 6707acc5e0381..c704e00555e5d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -344,12 +344,8 @@ private IgfsFileWorkerBatch newBatch(final IgfsPath path, OutputStream out) thro if (prevBatch == null) break; - else { - assert prevBatch.finishing() : - "File lock should prevent stream creation on a not-closed-yet file."; - + else prevBatch.await(); - } } return batch; From fa13832132ad71cd3b3ad259784e2e7ec5366c28 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Mon, 22 Aug 2016 17:37:14 +0300 Subject: [PATCH 044/487] IGNITE-3693: IGFS: Fixed sporadical failures in IgfsLocalSecondaryFileSystemDualAsyncSelfTest.testFormat. This closes #960. --- .../ignite/internal/processors/igfs/IgfsAbstractSelfTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java index 236a5893f275d..86c2449145b71 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java @@ -1071,12 +1071,14 @@ public void testFormat() throws Exception { os.write(new byte[10 * 1024 * 1024]); } + awaitFileClose(igfs, FILE); + if (dual) checkExist(igfsSecondary, DIR, SUBDIR, FILE, DIR_NEW, SUBDIR_NEW, FILE_NEW); checkExist(igfs, DIR, SUBDIR, FILE); - assert igfs.info(FILE).length() == 10 * 1024 * 1024; + assertEquals(10 * 1024 * 1024, igfs.info(FILE).length()); assert dataCache.size(new CachePeekMode[] {CachePeekMode.ALL}) > 0; From a0c7c44fc864aa965fd4d14e5362a4f6356c01ab Mon Sep 17 00:00:00 2001 From: AndreVel Date: Mon, 22 Aug 2016 20:50:57 -0700 Subject: [PATCH 045/487] IGNITE-3197 - OverlappingFileLockException in marshaller context --- .../internal/MarshallerContextImpl.java | 9 +- .../CacheContinuousQueryManager.java | 2 +- .../MarshallerContextLockingSelfTest.java | 139 ++++++++++++++++++ .../testsuites/IgniteBasicTestSuite.java | 3 + 4 files changed, 150 insertions(+), 3 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/MarshallerContextLockingSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java index 504ea6e3bf4fe..0420e184f6b25 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java @@ -27,6 +27,7 @@ import java.io.Writer; import java.nio.channels.FileChannel; import java.nio.channels.FileLock; +import java.nio.channels.OverlappingFileLockException; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ThreadLocalRandom; @@ -258,7 +259,7 @@ private static FileLock fileLock( /** */ - private static class ContinuousQueryListener implements CacheEntryUpdatedListener { + public static class ContinuousQueryListener implements CacheEntryUpdatedListener { /** */ private final IgniteLogger log; @@ -269,7 +270,7 @@ private static class ContinuousQueryListener implements CacheEntryUpdatedListene * @param log Logger. * @param workDir Work directory. */ - private ContinuousQueryListener(IgniteLogger log, File workDir) { + public ContinuousQueryListener(IgniteLogger log, File workDir) { this.log = log; this.workDir = workDir; } @@ -306,6 +307,10 @@ private ContinuousQueryListener(IgniteLogger log, File workDir) { U.error(log, "Failed to write class name to file [id=" + evt.getKey() + ", clsName=" + evt.getValue() + ", file=" + file.getAbsolutePath() + ']', e); } + catch(OverlappingFileLockException ignored) { + if (log.isDebugEnabled()) + log.debug("File already locked (will ignore): " + file.getAbsolutePath()); + } catch (IgniteInterruptedCheckedException e) { U.error(log, "Interrupted while waiting for acquiring file lock: " + file, e); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java index a8e5a6b99db6a..2863f3d12b2ba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java @@ -1204,7 +1204,7 @@ public BackupCleaner(Map lsnrs, GridKernalCo /** * */ - private static class CacheEntryEventImpl extends CacheQueryEntryEvent { + public static class CacheEntryEventImpl extends CacheQueryEntryEvent { /** */ private static final long serialVersionUID = 0L; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/MarshallerContextLockingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/MarshallerContextLockingSelfTest.java new file mode 100644 index 0000000000000..e549336d49b08 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/MarshallerContextLockingSelfTest.java @@ -0,0 +1,139 @@ +/* + * 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 java.io.File; +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.event.CacheEntryEvent; +import javax.cache.event.EventType; +import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; +import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridTestClassLoader; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.testframework.junits.logger.GridTestLog4jLogger; + +/** + * Test marshaller context. + */ +public class MarshallerContextLockingSelfTest extends GridCommonAbstractTest { + /** Inner logger. */ + private InnerLogger innerLog = null; + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + innerLog = new InnerLogger(); + + log = innerLog; + } + + /** + * Mumtithread test, used custom class loader + */ + public void testMultithreadedUpdate() throws Exception { + multithreaded(new Callable() { + @Override public Object call() throws Exception { + GridTestClassLoader classLoader = new GridTestClassLoader( + InternalExecutor.class.getName(), + MarshallerContextImpl.class.getName(), + MarshallerContextImpl.ContinuousQueryListener.class.getName() + ); + + Thread.currentThread().setContextClassLoader(classLoader); + + Class clazz = classLoader.loadClass(InternalExecutor.class.getName()); + + Object internelExecutor = clazz.newInstance(); + + clazz.getMethod("executeTest", GridTestLog4jLogger.class).invoke(internelExecutor, log); + + return null; + } + }, 4); + + assertTrue(InternalExecutor.counter.get() == 0); + + assertTrue(innerLog.contains("File already locked")); + + assertTrue(!innerLog.contains("Exception")); + } + + /** + * Internal test executor + */ + public static class InternalExecutor { + /** Counter. */ + public static AtomicInteger counter = new AtomicInteger(); + + /** + * Executes onUpdated + */ + public void executeTest(GridTestLog4jLogger log) throws Exception { + counter.incrementAndGet(); + + File workDir = U.resolveWorkDirectory("marshaller", false); + + final MarshallerContextImpl.ContinuousQueryListener queryListener = new MarshallerContextImpl.ContinuousQueryListener(log, workDir); + + final ArrayList evts = new ArrayList>(); + + IgniteCacheProxy cache = new IgniteCacheProxy(); + + evts.add(new CacheContinuousQueryManager.CacheEntryEventImpl(cache, EventType.CREATED, 1, String.class.getName())); + + for (int i = 0; i < 100; i++) + queryListener.onUpdated(evts); + } + } + + /** + * Internal logger + */ + public static class InnerLogger extends GridTestLog4jLogger { + /** */ + private Collection logs = new ConcurrentLinkedDeque<>(); + + /** + * Returns true if and only if this string contains the specified + * sequence of char values. + * + * @param str String. + */ + public boolean contains(String str) { + for (String text : logs) + if (text != null && text.contains(str)) + return true; + + return false; + } + + /** {@inheritDoc} */ + @Override public void debug(String msg) { + logs.add(msg); + } + + /** {@inheritDoc} */ + @Override public boolean isDebugEnabled() { + return true; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java index 6611500bc21f9..d56c29dbbeb76 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java @@ -33,6 +33,7 @@ import org.apache.ignite.internal.GridStartStopSelfTest; import org.apache.ignite.internal.GridStopWithCancelSelfTest; import org.apache.ignite.internal.IgniteSlowClientDetectionSelfTest; +import org.apache.ignite.internal.MarshallerContextLockingSelfTest; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessorRendezvousSelfTest; import org.apache.ignite.internal.processors.cache.GridProjectionForCachesOnDaemonNodeSelfTest; import org.apache.ignite.internal.processors.cache.IgniteDaemonNodeMarshallerCacheTest; @@ -136,6 +137,8 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(ConfigVariationsTestSuiteBuilderTest.class); suite.addTestSuite(NotStringSystemPropertyTest.class); + suite.addTestSuite(MarshallerContextLockingSelfTest.class); + return suite; } } From eea42827738301168d66562a05d3360150e6fb89 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 23 Aug 2016 10:47:15 +0300 Subject: [PATCH 046/487] Fix for IgniteDynamicCacheStartNoExchangeTimeoutTest. --- .../cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java index ac80d69908a2f..263e9cad6b105 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java @@ -84,6 +84,8 @@ public class IgniteDynamicCacheStartNoExchangeTimeoutTest extends GridCommonAbst super.beforeTest(); startGrids(NODES); + + awaitPartitionMapExchange(); } /** From f9258736c5cfa157e138d879aa0dbacc5a7cb4b2 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Tue, 23 Aug 2016 14:59:45 +0700 Subject: [PATCH 047/487] Fixed tests. --- .../ignite/schema/generator/CodeGenerator.java | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/generator/CodeGenerator.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/generator/CodeGenerator.java index 59dfdc45737d4..5cdc6c6cb37cd 100644 --- a/modules/schema-import/src/main/java/org/apache/ignite/schema/generator/CodeGenerator.java +++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/generator/CodeGenerator.java @@ -255,6 +255,17 @@ private static void write(Collection src, File out) throws IOException { } } + /** + * Ensure that all folders for packages exist. + * + * @param pkg Packages. + * @throws IOException If failed to ensure. + */ + private static void ensurePackages(File pkg) throws IOException { + if (!pkg.exists() && !pkg.mkdirs()) + throw new IOException("Failed to create folders for package: " + pkg); + } + /** * Generate java class code. * @@ -275,8 +286,7 @@ private static void generateCode(PojoDescriptor pojo, boolean key, String pkg, F checkValidJavaIdentifier(type, false, "Type", type); - if (!pkgFolder.exists() && !pkgFolder.mkdirs()) - throw new IOException("Failed to create folders for package: " + pkg); + ensurePackages(pkgFolder); File out = new File(pkgFolder, type + ".java"); @@ -567,6 +577,8 @@ public static void snippet(Collection pojos, String pkg, boolean boolean generateAliases, String outFolder, ConfirmCallable askOverwrite) throws IOException { File pkgFolder = new File(outFolder, pkg.replace('.', File.separatorChar)); + ensurePackages(pkgFolder); + File cacheCfg = new File(pkgFolder, "CacheConfig.java"); if (cacheCfg.exists()) { From d6033712425cd0157fa640a7d46ec6579fd7e74b Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 23 Aug 2016 14:10:55 +0300 Subject: [PATCH 048/487] IGNITE-3325 .NET: Improve CompiledQuery in LINQ provider --- .../Cache/Query/CacheLinqTest.cs | 196 ++++++++++++- .../Apache.Ignite.Linq.csproj | 2 +- .../Apache.Ignite.Linq/CompiledQuery.cs | 1 + .../Apache.Ignite.Linq/CompiledQuery2.cs | 257 ++++++++++++++++++ .../Impl/CacheFieldsQueryExecutor.cs | 116 +++++++- .../Impl/CacheQueryExpressionVisitor.cs | 15 + .../Impl/CacheQueryModelVisitor.cs | 15 +- .../Impl/CacheQueryableBase.cs | 22 ++ .../Impl/ExpressionWalker.cs | 9 + .../Impl/ICacheQueryProxy.cs | 40 --- .../Impl/ICacheQueryableInternal.cs | 14 +- 11 files changed, 622 insertions(+), 65 deletions(-) create mode 100644 modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery2.cs delete mode 100644 modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ICacheQueryProxy.cs diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs index a47f3f0bbcb2d..a8cc8fd823591 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs @@ -28,6 +28,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query { using System; using System.Collections; + using System.Diagnostics.CodeAnalysis; using System.Linq; using System.Linq.Expressions; using System.Text.RegularExpressions; @@ -506,9 +507,11 @@ public void TestSubqueryJoin() public void TestInvalidJoin() { // Join on non-IQueryable - Assert.Throws(() => + var ex = Assert.Throws(() => // ReSharper disable once ReturnValueOfPureMethodIsNotUsed GetPersonCache().AsCacheQueryable().Join(GetOrgCache(), p => p.Key, o => o.Key, (p, o) => p).ToList()); + + Assert.IsTrue(ex.Message.StartsWith("Unexpected query source")); } /// @@ -836,10 +839,186 @@ public void TestAliases() } /// - /// Tests the compiled query. + /// Tests the compiled query with various constructs. /// [Test] public void TestCompiledQuery() + { + var persons = GetPersonCache().AsCacheQueryable(); + var roles = GetRoleCache().AsCacheQueryable(); + + // Embedded args + var qry0 = CompiledQuery2.Compile(() => persons.Where(x => x.Key < 3 && x.Value.Name.Contains("son"))); + Assert.AreEqual(3, qry0().Count()); + + // Lambda args + var qry1 = CompiledQuery2.Compile((int minKey, int take, int skip) => persons.Where(x => x.Key > minKey) + .Take(take).Skip(skip)); + Assert.AreEqual(3, qry1(-1, 3, 1).GetAll().Count); + + qry1 = CompiledQuery2.Compile((int skip, int take, int minKey) => persons.Where(x => x.Key > minKey) + .Take(take).Skip(skip)); + + Assert.AreEqual(5, qry1(2, 5, 20).GetAll().Count); + + // Mixed args + var qry2 = CompiledQuery2.Compile((int maxKey, int minKey) => + persons.Where(x => x.Key < maxKey + && x.Value.Name.Contains("er") + && x.Value.Age < maxKey + && x.Key > minKey).Skip(2)); + + Assert.AreEqual(6, qry2(10, 1).Count()); + + // Join + var qry3 = CompiledQuery2.Compile(() => + roles.Join(persons, r => r.Key.Foo, p => p.Key, (r, p) => r.Value.Name)); + + Assert.AreEqual(RoleCount, qry3().Count()); + + // Join with subquery + var qry4 = CompiledQuery2.Compile( + (int a, int b, string sep) => + roles + .Where(x => x.Key.Bar > a) + .Join(persons.Where(x => x.Key < b && x.Key > 0), + r => r.Key.Foo, + p => p.Value.Address.Zip, + (r, p) => p.Value.Name + sep + r.Value.Name + "|") + .Skip(a).Take(1000) + ); + + Assert.AreEqual(new[] { " Person_2 =Role_2|", " Person_3 =|"}, qry4(1, PersonCount, "=").ToArray()); + + // Union + var qry5 = CompiledQuery2.Compile(() => roles.Select(x => -x.Key.Foo).Union(persons.Select(x => x.Key))); + + Assert.AreEqual(RoleCount + PersonCount, qry5().Count()); + + // Projection + var qry6 = CompiledQuery2.Compile((int minAge) => persons + .Select(x => x.Value) + .Where(x => x.Age >= minAge) + .Select(x => new {x.Name, x.Age}) + .OrderBy(x => x.Name)); + + var res = qry6(PersonCount - 3).GetAll(); + + Assert.AreEqual(3, res.Count); + Assert.AreEqual(PersonCount - 3, res[0].Age); + } + + /// + /// Tests the compiled query overloads. + /// + [Test] + public void TestCompiledQueryOverloads() + { + var cache = GetPersonCache().AsCacheQueryable(); + + // const args are allowed + Assert.AreEqual(5, CompiledQuery2.Compile(() => cache.Where(x => x.Key < 5))().GetAll().Count); + + // 0 arg + var qry0 = CompiledQuery2.Compile(() => cache.Select(x => x.Value.Name)); + Assert.AreEqual(PersonCount, qry0().ToArray().Length); + + // 1 arg + var qry1 = CompiledQuery2.Compile((int k) => cache.Where(x => x.Key < k)); + Assert.AreEqual(3, qry1(3).ToArray().Length); + + // 1 arg twice + var qry1T = CompiledQuery2.Compile((int k) => cache.Where(x => x.Key < k && x.Value.Age < k)); + Assert.AreEqual(3, qry1T(3).ToArray().Length); + + // 2 arg + var qry2 = + CompiledQuery2.Compile((int i, string s) => cache.Where(x => x.Key < i && x.Value.Name.StartsWith(s))); + Assert.AreEqual(5, qry2(5, " Pe").ToArray().Length); + + // Changed param order + var qry2R = + CompiledQuery2.Compile((string s, int i) => cache.Where(x => x.Key < i && x.Value.Name.StartsWith(s))); + Assert.AreEqual(5, qry2R(" Pe", 5).ToArray().Length); + + // 3 arg + var qry3 = CompiledQuery2.Compile((int i, string s, double d) => + cache.Where(x => x.Value.Address.Zip > d && x.Key < i && x.Value.Name.Contains(s))); + Assert.AreEqual(5, qry3(5, "son", -10).ToArray().Length); + + // 4 arg + var qry4 = CompiledQuery2.Compile((int a, int b, int c, int d) => + cache.Select(x => x.Key).Where(k => k > a && k > b && k > c && k < d)); + Assert.AreEqual(new[] {3, 4}, qry4(0, 1, 2, 5).ToArray()); + + // 5 arg + var qry5 = CompiledQuery2.Compile((int a, int b, int c, int d, int e) => + cache.Select(x => x.Key).Where(k => k > a && k > b && k > c && k < d && k < e)); + Assert.AreEqual(new[] {3, 4}, qry5(0, 1, 2, 5, 6).ToArray()); + + // 6 arg + var qry6 = CompiledQuery2.Compile((int a, int b, int c, int d, int e, int f) => + cache.Select(x => x.Key).Where(k => k > a && k > b && k > c && k < d && k < e && k < f)); + Assert.AreEqual(new[] {3, 4}, qry6(0, 1, 2, 5, 6, 7).ToArray()); + + // 7 arg + var qry7 = CompiledQuery2.Compile((int a, int b, int c, int d, int e, int f, int g) => + cache.Select(x => x.Key).Where(k => k > a && k > b && k > c && k < d && k < e && k < f && k < g)); + Assert.AreEqual(new[] {3, 4}, qry7(0, 1, 2, 5, 6, 7, 8).ToArray()); + + // 8 arg + var qry8 = CompiledQuery2.Compile((int a, int b, int c, int d, int e, int f, int g, int h) => + cache.Select(x => x.Key).Where(k => k > a && k > b && k > c && k < d && k < e && k < f && k < g && k < h)); + Assert.AreEqual(new[] {3, 4}, qry8(0, 1, 2, 5, 6, 7, 8, 9).ToArray()); + } + + /// + /// Tests the free-form compiled query, where user provides an array of arguments. + /// + [Test] + public void TestCompiledQueryFreeform() + { + var cache = GetPersonCache().AsCacheQueryable(); + + var qry = cache.Where(x => x.Key < 5); + + // Simple + var compiled = CompiledQuery2.Compile(qry); + + Assert.AreEqual(5, compiled(5).Count()); + Assert.AreEqual(6, compiled(6).Count()); + + // Select + var compiledSelect = CompiledQuery2.Compile(qry.Select(x => x.Value.Name).OrderBy(x => x)); + + Assert.AreEqual(3, compiledSelect(3).Count()); + Assert.AreEqual(" Person_0 ", compiledSelect(1).Single()); + + // Join + var compiledJoin = CompiledQuery2.Compile(qry.Join( + GetOrgCache().AsCacheQueryable().Where(x => x.Value.Name.StartsWith("Org")), + p => p.Value.OrganizationId, o => o.Value.Id, (p, o) => o.Key)); + + Assert.AreEqual(1000, compiledJoin("Org", 1).Single()); + + // Many parameters + var qry2 = cache.Where(x => x.Key < 3) + .Where(x => x.Key > 0) + .Where(x => x.Value.Name.Contains("")) + .Where(x => x.Value.Address.Zip > 0) + .Where(x => x.Value.Age == 7); + + var compiled2 = CompiledQuery2.Compile(qry2); + + Assert.AreEqual(17, compiled2(18, 16, "ers", 13, 17).Single().Key); + } + +#pragma warning disable 618 // obsolete class + /// + /// Tests the old, deprecated compiled query. + /// + [Test] + public void TestCompiledQueryOld() { var cache = GetPersonCache().AsCacheQueryable(); @@ -865,7 +1044,7 @@ public void TestCompiledQuery() Assert.AreEqual(5, qry2R(" Pe", 5).ToArray().Length); // 3 arg - var qry3 = CompiledQuery.Compile((int i, string s, double d) => + var qry3 = CompiledQuery.Compile((int i, string s, double d) => cache.Where(x => x.Value.Address.Zip > d && x.Key < i && x.Value.Name.Contains(s))); Assert.AreEqual(5, qry3(5, "son", -10).ToArray().Length); @@ -873,28 +1052,29 @@ public void TestCompiledQuery() var keys = cache.Select(x => x.Key); var qry4 = CompiledQuery.Compile((int a, int b, int c, int d) => keys.Where(k => k > a && k > b && k > c && k < d)); - Assert.AreEqual(new[] {3, 4}, qry4(0, 1, 2, 5).ToArray()); + Assert.AreEqual(new[] { 3, 4 }, qry4(0, 1, 2, 5).ToArray()); // 5 arg var qry5 = CompiledQuery.Compile((int a, int b, int c, int d, int e) => keys.Where(k => k > a && k > b && k > c && k < d && k < e)); - Assert.AreEqual(new[] {3, 4}, qry5(0, 1, 2, 5, 6).ToArray()); + Assert.AreEqual(new[] { 3, 4 }, qry5(0, 1, 2, 5, 6).ToArray()); // 6 arg var qry6 = CompiledQuery.Compile((int a, int b, int c, int d, int e, int f) => keys.Where(k => k > a && k > b && k > c && k < d && k < e && k < f)); - Assert.AreEqual(new[] {3, 4}, qry6(0, 1, 2, 5, 6, 7).ToArray()); + Assert.AreEqual(new[] { 3, 4 }, qry6(0, 1, 2, 5, 6, 7).ToArray()); // 7 arg var qry7 = CompiledQuery.Compile((int a, int b, int c, int d, int e, int f, int g) => keys.Where(k => k > a && k > b && k > c && k < d && k < e && k < f && k < g)); - Assert.AreEqual(new[] {3, 4}, qry7(0, 1, 2, 5, 6, 7, 8).ToArray()); + Assert.AreEqual(new[] { 3, 4 }, qry7(0, 1, 2, 5, 6, 7, 8).ToArray()); // 8 arg var qry8 = CompiledQuery.Compile((int a, int b, int c, int d, int e, int f, int g, int h) => keys.Where(k => k > a && k > b && k > c && k < d && k < e && k < f && k < g && k < h)); - Assert.AreEqual(new[] {3, 4}, qry8(0, 1, 2, 5, 6, 7, 8, 9).ToArray()); + Assert.AreEqual(new[] { 3, 4 }, qry8(0, 1, 2, 5, 6, 7, 8, 9).ToArray()); } +#pragma warning restore 618 /// /// Tests the cache of primitive types. diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj b/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj index d1dad1669ce32..d00ebea0c457f 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj @@ -52,6 +52,7 @@ + @@ -64,7 +65,6 @@ - diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery.cs index 9e6c6c7ad3627..817c4232f6aa8 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery.cs @@ -28,6 +28,7 @@ namespace Apache.Ignite.Linq /// /// Represents a compiled cache query. /// + [Obsolete("Use CompiledQuery2 class.")] public static class CompiledQuery { /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery2.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery2.cs new file mode 100644 index 0000000000000..85aafd950903f --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery2.cs @@ -0,0 +1,257 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Linq +{ + using System; + using System.Diagnostics; + using System.Diagnostics.CodeAnalysis; + using System.Linq; + using System.Linq.Expressions; + using Apache.Ignite.Core.Cache.Query; + using Apache.Ignite.Core.Impl.Common; + using Apache.Ignite.Linq.Impl; + + /// + /// Represents a compiled cache query. + /// + public static class CompiledQuery2 + { + /// + /// Delegate for compiled query with arbitrary number of arguments. + /// + /// Result type. + /// The arguments. + /// Query cursor. + public delegate IQueryCursor CompiledQueryDelegate(params object[] args); + + /// + /// Creates a new delegate that represents the compiled cache query. + /// + /// The query to compile. + /// Delegate that represents the compiled cache query. + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "0", + Justification = "Invalid warning, validation is present.")] + public static Func> Compile(Expression>> query) + { + IgniteArgumentCheck.NotNull(query, "query"); + + var compiledQuery = GetCompiledQuery(query, query.Compile()); + + return () => compiledQuery(new object[0]); + } + + /// + /// Creates a new delegate that represents the compiled cache query with any number of arguments. + /// + /// This method differs from other Compile methods in that it takes in directly, + /// and returns a delegate that takes an array of parameters. + /// It is up to the user to provide query arguments in correct order. + /// + /// This method also imposes no restrictions on where the query comes from (in contrary to other methods). + /// + /// The query to compile. + /// Delegate that represents the compiled cache query. + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "0", + Justification = "Invalid warning, validation is present.")] + public static CompiledQueryDelegate Compile(IQueryable query) + { + IgniteArgumentCheck.NotNull(query, "query"); + + var cacheQueryable = query as ICacheQueryableInternal; + + if (cacheQueryable == null) + throw GetInvalidQueryException(query); + + var compileQuery = cacheQueryable.CompileQuery(); + + // Special delegate is required to allow params[]. + return args => compileQuery(args); + } + + /// + /// Creates a new delegate that represents the compiled cache query. + /// + /// The query to compile. + /// Delegate that represents the compiled cache query. + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "0", + Justification = "Invalid warning, validation is present.")] + public static Func> Compile(Expression>> query) + { + IgniteArgumentCheck.NotNull(query, "query"); + + var compiledQuery = GetCompiledQuery(query, query.Compile()); + + return x => compiledQuery(new object[] {x}); + } + + /// + /// Creates a new delegate that represents the compiled cache query. + /// + /// The query to compile. + /// Delegate that represents the compiled cache query. + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "0", + Justification = "Invalid warning, validation is present.")] + public static Func> Compile(Expression>> query) + { + IgniteArgumentCheck.NotNull(query, "query"); + + var compiledQuery = GetCompiledQuery(query, query.Compile()); + + return (x, y) => compiledQuery(new object[] {x, y}); + } + + /// + /// Creates a new delegate that represents the compiled cache query. + /// + /// The query to compile. + /// Delegate that represents the compiled cache query. + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "0", + Justification = "Invalid warning, validation is present.")] + public static Func> Compile(Expression>> query) + { + IgniteArgumentCheck.NotNull(query, "query"); + + var compiledQuery = GetCompiledQuery(query, query.Compile()); + + return (x, y, z) => compiledQuery(new object[] {x, y, z}); + } + + /// + /// Creates a new delegate that represents the compiled cache query. + /// + /// The query to compile. + /// Delegate that represents the compiled cache query. + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "0", + Justification = "Invalid warning, validation is present.")] + public static Func> Compile(Expression>> query) + { + IgniteArgumentCheck.NotNull(query, "query"); + + var compiledQuery = GetCompiledQuery(query, query.Compile()); + + return (x, y, z, a) => compiledQuery(new object[] {x, y, z, a}); + } + + /// + /// Creates a new delegate that represents the compiled cache query. + /// + /// The query to compile. + /// Delegate that represents the compiled cache query. + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "0", + Justification = "Invalid warning, validation is present.")] + public static Func> Compile( + Expression>> query) + { + IgniteArgumentCheck.NotNull(query, "query"); + + var compiledQuery = GetCompiledQuery(query, query.Compile()); + + return (x, y, z, a, b) => compiledQuery(new object[] {x, y, z, a, b}); + } + + /// + /// Creates a new delegate that represents the compiled cache query. + /// + /// The query to compile. + /// Delegate that represents the compiled cache query. + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "0", + Justification = "Invalid warning, validation is present.")] + public static Func> Compile( + Expression>> query) + { + IgniteArgumentCheck.NotNull(query, "query"); + + var compiledQuery = GetCompiledQuery(query, query.Compile()); + + return (x, y, z, a, b, c) => compiledQuery(new object[] {x, y, z, a, b, c}); + } + + /// + /// Creates a new delegate that represents the compiled cache query. + /// + /// The query to compile. + /// Delegate that represents the compiled cache query. + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "0", + Justification = "Invalid warning, validation is present.")] + public static Func> Compile( + Expression>> query) + { + IgniteArgumentCheck.NotNull(query, "query"); + + var compiledQuery = GetCompiledQuery(query, query.Compile()); + + return (x, y, z, a, b, c, d) => compiledQuery(new object[] {x, y, z, a, b, c, d}); + } + + /// + /// Creates a new delegate that represents the compiled cache query. + /// + /// The query to compile. + /// Delegate that represents the compiled cache query. + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "0", + Justification = "Invalid warning, validation is present.")] + public static Func> Compile( + Expression>> query) + { + IgniteArgumentCheck.NotNull(query, "query"); + + var compiledQuery = GetCompiledQuery(query, query.Compile()); + + return (x, y, z, a, b, c, d, e) => compiledQuery(new object[] {x, y, z, a, b, c, d, e}); + } + + /// + /// Gets the compiled query. + /// + private static Func> GetCompiledQuery(LambdaExpression expression, + Delegate queryCaller) + { + Debug.Assert(expression != null); + Debug.Assert(queryCaller != null); + + // Get default parameter values. + var paramValues = expression.Parameters + .Select(x => x.Type) + .Select(x => x.IsValueType ? Activator.CreateInstance(x) : null) + .ToArray(); + + // Invoke the delegate to obtain the cacheQueryable. + var queryable = queryCaller.DynamicInvoke(paramValues); + + var cacheQueryable = queryable as ICacheQueryableInternal; + + if (cacheQueryable == null) + throw GetInvalidQueryException(queryable); + + return cacheQueryable.CompileQuery(expression); + } + + /// + /// Gets the invalid query exception. + /// + private static ArgumentException GetInvalidQueryException(object queryable) + { + return new ArgumentException( + string.Format("{0} can only compile cache queries produced by AsCacheQueryable method. " + + "Provided query is not valid: '{1}'", typeof(CompiledQuery2).FullName, queryable)); + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheFieldsQueryExecutor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheFieldsQueryExecutor.cs index c715e4cd4688e..b4c32925bfa58 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheFieldsQueryExecutor.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheFieldsQueryExecutor.cs @@ -92,7 +92,7 @@ public IEnumerable ExecuteCollection(QueryModel queryModel) Debug.WriteLine("\nFields Query: {0} | {1}", qryData.QueryText, string.Join(", ", qryData.Parameters.Select(x => x == null ? "null" : x.ToString()))); - var qry = new SqlFieldsQuery(qryData.QueryText, _local, qryData.Parameters.ToArray()); + var qry = GetFieldsQuery(qryData.QueryText, qryData.Parameters.ToArray()); var selector = GetResultSelector(queryModel.SelectClause.Selector); @@ -100,7 +100,7 @@ public IEnumerable ExecuteCollection(QueryModel queryModel) } /// - /// Compiles the query. + /// Compiles the query (old method, does not support some scenarios). /// public Func> CompileQuery(QueryModel queryModel, Delegate queryCaller) { @@ -132,12 +132,120 @@ public IEnumerable ExecuteCollection(QueryModel queryModel) // Check if user param order is already correct if (indices.SequenceEqual(Enumerable.Range(0, indices.Length))) - return args => _cache.QueryFields(new SqlFieldsQuery(qryText, _local, args), selector); + return args => _cache.QueryFields(GetFieldsQuery(qryText, args), selector); // Return delegate with reorder - return args => _cache.QueryFields(new SqlFieldsQuery(qryText, _local, + return args => _cache.QueryFields(GetFieldsQuery(qryText, args.Select((x, i) => args[indices[i]]).ToArray()), selector); } + /// + /// Compiles the query without regard to number or order of arguments. + /// + public Func> CompileQuery(QueryModel queryModel) + { + Debug.Assert(queryModel != null); + + var qryText = GetQueryData(queryModel).QueryText; + var selector = GetResultSelector(queryModel.SelectClause.Selector); + + return args => _cache.QueryFields(GetFieldsQuery(qryText, args), selector); + } + + /// + /// Compiles the query. + /// + /// Result type. + /// The query model. + /// The query model generated from lambda body. + /// The query lambda. + /// Compiled query func. + public Func> CompileQuery(QueryModel queryModel, QueryModel queryLambdaModel, + LambdaExpression queryLambda) + { + Debug.Assert(queryModel != null); + + // Get model from lambda to map arguments properly. + var qryData = GetQueryData(queryLambdaModel); + + var qryText = GetQueryData(queryModel).QueryText; + var qryTextLambda = qryData.QueryText; + + if (qryText != qryTextLambda) + { + Debug.WriteLine(qryText); + Debug.WriteLine(qryTextLambda); + + throw new InvalidOperationException("Error compiling query: entire LINQ expression should be " + + "specified within lambda passed to Compile method. " + + "Part of the query can't be outside the Compile method call."); + } + + var selector = GetResultSelector(queryModel.SelectClause.Selector); + + var qryParams = qryData.Parameters.ToArray(); + + // Compiled query is a delegate with query parameters + // Delegate parameters order and query parameters order may differ + + // Simple case: lambda with no parameters. Only embedded parameters are used. + if (!queryLambda.Parameters.Any()) + { + return argsUnused => _cache.QueryFields(GetFieldsQuery(qryText, qryParams), selector); + } + + // These are in order of usage in query + var qryOrderArgs = qryParams.OfType().Select(x => x.Name).ToArray(); + + // These are in order they come from user + var userOrderArgs = queryLambda.Parameters.Select(x => x.Name).ToList(); + + // Simple case: all query args directly map to the lambda args in the same order + if (qryOrderArgs.Length == qryParams.Length + && qryOrderArgs.SequenceEqual(userOrderArgs)) + { + return args => _cache.QueryFields(GetFieldsQuery(qryText, args), selector); + } + + // General case: embedded args and lambda args are mixed; same args can be used multiple times. + // Produce a mapping that defines where query arguments come from. + var mapping = qryParams.Select(x => + { + var pe = x as ParameterExpression; + + if (pe != null) + return userOrderArgs.IndexOf(pe.Name); + + return -1; + }).ToArray(); + + return args => _cache.QueryFields( + GetFieldsQuery(qryText, MapQueryArgs(args, qryParams, mapping)), selector); + } + + /// + /// Maps the query arguments. + /// + private static object[] MapQueryArgs(object[] userArgs, object[] embeddedArgs, int[] mapping) + { + var mappedArgs = new object[embeddedArgs.Length]; + + for (var i = 0; i < mappedArgs.Length; i++) + { + var map = mapping[i]; + + mappedArgs[i] = map < 0 ? embeddedArgs[i] : userArgs[map]; + } + + return mappedArgs; + } + + /// + /// Gets the fields query. + /// + private SqlFieldsQuery GetFieldsQuery(string text, object[] args) + { + return new SqlFieldsQuery(text, _local, args); + } /** */ public static QueryData GetQueryData(QueryModel queryModel) diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs index eaca07a36a993..2c485547be977 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs @@ -241,6 +241,21 @@ protected override Expression VisitBinary(BinaryExpression expression) return expression; } + /** */ + public override Expression Visit(Expression expression) + { + var paramExpr = expression as ParameterExpression; + + if (paramExpr != null) + { + // This happens only with compiled queries, where parameters come from enclosing lambda. + AppendParameter(paramExpr); + return expression; + } + + return base.Visit(expression); + } + /** */ protected override Expression VisitQuerySourceReference(QuerySourceReferenceExpression expression) { diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryModelVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryModelVisitor.cs index 1888414668d6c..51297eebee52c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryModelVisitor.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryModelVisitor.cs @@ -273,6 +273,10 @@ private void ProcessSkipTake(QueryModel queryModel) // Workaround for unlimited offset (IGNITE-2602) // H2 allows NULL & -1 for unlimited, but Ignite indexing does not // Maximum limit that works is (int.MaxValue - offset) + + if (offset.Count is ParameterExpression) + throw new NotSupportedException("Skip() without Take() is not supported in compiled queries."); + var offsetInt = (int) ((ConstantExpression) offset.Count).Value; _builder.Append((int.MaxValue - offsetInt).ToString(CultureInfo.InvariantCulture)); } @@ -424,17 +428,6 @@ public override void VisitJoinClause(JoinClause joinClause, QueryModel queryMode } else { - var innerExpr = joinClause.InnerSequence as ConstantExpression; - - if (innerExpr == null) - throw new NotSupportedException("Unexpected JOIN inner sequence (subqueries are not supported): " + - joinClause.InnerSequence); - - if (!(innerExpr.Value is ICacheQueryable)) - throw new NotSupportedException("Unexpected JOIN inner sequence " + - "(only results of cache.ToQueryable() are supported): " + - innerExpr.Value); - var queryable = ExpressionWalker.GetCacheQueryable(joinClause); var tableName = ExpressionWalker.GetTableNameWithSchema(queryable); _builder.AppendFormat("inner join {0} as {1} on (", tableName, _aliases.GetTableAlias(tableName)); diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryableBase.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryableBase.cs index d3115be0cc2f7..ee0239c22c78e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryableBase.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryableBase.cs @@ -89,6 +89,28 @@ public string TableName return executor.CompileQuery(GetQueryModel(), queryCaller); } + /** */ + public Func> CompileQuery(LambdaExpression queryExpression) + { + var executor = CacheQueryProvider.Executor; + + // Generate two models: from current expression, and from provided lambda. + // Lambda expression provides a way to identify argument mapping. + // Comparing two models allows to check whether whole query is within lambda. + var model = GetQueryModel(); + var lambdaModel = CacheQueryProvider.GenerateQueryModel(queryExpression.Body); + + return executor.CompileQuery(model, lambdaModel, queryExpression); + } + + /** */ + public Func> CompileQuery() + { + var executor = CacheQueryProvider.Executor; + + return executor.CompileQuery(GetQueryModel()); + } + /// /// Gets the cache query provider. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs index 96371ccbf8fdd..f529da7517ab5 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs @@ -118,6 +118,11 @@ public static ICacheQueryableInternal GetCacheQueryable(Expression expression, b /// public static T EvaluateExpression(Expression expr) { + var constExpr = expr as ConstantExpression; + + if (constExpr != null) + return (T)constExpr.Value; + var memberExpr = expr as MemberExpression; if (memberExpr != null) @@ -137,6 +142,10 @@ public static T EvaluateExpression(Expression expr) } } + // Case for compiled queries: return unchanged. + if (expr is ParameterExpression) + return (T) (object) expr; + throw new NotSupportedException("Expression not supported: " + expr); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ICacheQueryProxy.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ICacheQueryProxy.cs deleted file mode 100644 index b00937cd9459e..0000000000000 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ICacheQueryProxy.cs +++ /dev/null @@ -1,40 +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. - */ - -namespace Apache.Ignite.Linq.Impl -{ - using System; - using Apache.Ignite.Core.Binary; - using Apache.Ignite.Core.Cache.Query; - - /// - /// Cache proxy interface that allows query execution without key/value generic arguments. - /// - internal interface ICacheQueryProxy - { - /// - /// Queries separate entry fields. - /// - /// Type of the result. - /// SQL fields query. - /// Reader function, takes raw reader and field count, returns typed result. - /// - /// Cursor. - /// - IQueryCursor QueryFields(SqlFieldsQuery qry, Func readerFunc); - } -} \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ICacheQueryableInternal.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ICacheQueryableInternal.cs index 3e252c56a22ac..ffc81b46ba12d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ICacheQueryableInternal.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ICacheQueryableInternal.cs @@ -18,6 +18,7 @@ namespace Apache.Ignite.Linq.Impl { using System; + using System.Linq.Expressions; using Apache.Ignite.Core.Cache.Configuration; using Apache.Ignite.Core.Cache.Query; using Remotion.Linq; @@ -46,9 +47,20 @@ internal interface ICacheQueryableInternal : ICacheQueryable QueryModel GetQueryModel(); /// - /// Compiles the query. + /// Compiles the query (the old way). /// /// Caller expression to examine argument order. Func> CompileQuery(Delegate queryCaller); + + /// + /// Compiles the query. + /// + /// The query expression. + Func> CompileQuery(LambdaExpression queryExpression); + + /// + /// Compiles the query without regard to the order and number of arguments. + /// + Func> CompileQuery(); } } \ No newline at end of file From 207e8cb3b939670655da14082ac9e1cf1d822fd0 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 23 Aug 2016 20:01:48 +0300 Subject: [PATCH 049/487] IGNITE-3325 .NET: Improve CompiledQuery in LINQ provider - minor fixes --- .../dotnet/Apache.Ignite.Linq/CompiledQuery2.cs | 16 ++++++++-------- .../Apache.Ignite.Linq/Impl/ExpressionWalker.cs | 1 + 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery2.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery2.cs index 85aafd950903f..1ecc6e959d125 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery2.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery2.cs @@ -26,19 +26,19 @@ namespace Apache.Ignite.Linq using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Linq.Impl; + /// + /// Delegate for compiled query with arbitrary number of arguments. + /// + /// Result type. + /// The arguments. + /// Query cursor. + public delegate IQueryCursor CompiledQueryDelegate(params object[] args); + /// /// Represents a compiled cache query. /// public static class CompiledQuery2 { - /// - /// Delegate for compiled query with arbitrary number of arguments. - /// - /// Result type. - /// The arguments. - /// Query cursor. - public delegate IQueryCursor CompiledQueryDelegate(params object[] args); - /// /// Creates a new delegate that represents the compiled cache query. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs index f529da7517ab5..5d94ccc5340d0 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs @@ -143,6 +143,7 @@ public static T EvaluateExpression(Expression expr) } // Case for compiled queries: return unchanged. + // ReSharper disable once CanBeReplacedWithTryCastAndCheckForNull if (expr is ParameterExpression) return (T) (object) expr; From 417b269fe844c2a0b2c18aff8f58a2b38bc27531 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Wed, 24 Aug 2016 09:44:04 +0700 Subject: [PATCH 050/487] IGNITE-3005 Implemented showing of node ID8 , IP and checked values on alert. --- .../commands/alert/VisorAlertCommand.scala | 43 +++++++++++-------- 1 file changed, 26 insertions(+), 17 deletions(-) diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala index 401a3f24a5213..fb60a1e5bbb47 100644 --- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala +++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala @@ -156,7 +156,7 @@ class VisorAlertCommand extends VisorConsoleCommand { private val guard = new AtomicBoolean(false) /** Node metric update listener. */ - private var lsnr: IgnitePredicate[Event] = null + private var lsnr: IgnitePredicate[Event] = _ /** * ===Command=== @@ -274,8 +274,8 @@ class VisorAlertCommand extends VisorConsoleCommand { val (n, v) = arg n match { - case c if alertDescr.contains(c) && v != null => - val meta = alertDescr(c) + case c if ALERT_DESCRIPTORS.contains(c) && v != null => + val meta = ALERT_DESCRIPTORS(c) if (meta.byGrid) conditions += VisorAlertCondition(arg, gridFunc = makeGridFilter(v, meta.gridFunc)) @@ -310,21 +310,18 @@ class VisorAlertCommand extends VisorConsoleCommand { break() } - val alert = new VisorAlert( + val alert = VisorAlert( id = id8, name = name, - conditions = conditions.toSeq, + conditions = conditions, perGrid = conditions.exists(_.gridFunc.isDefined), perNode = conditions.exists(_.nodeFunc.isDefined), - spec = makeArgs(args), + spec = makeArgs(args.filter((arg) => arg._1 != "r")), conditionSpec = makeArgs(conditions.map(_.arg)), freq = freq, createdOn = System.currentTimeMillis(), varName = setVar(id8, "a"), - notification = new VisorAlertNotification( - script = script, - throttleInterval = interval * 1000 - ) + notification = VisorAlertNotification(script, interval * 1000) ) // Subscribe for node metric updates - if needed. @@ -359,27 +356,33 @@ class VisorAlertCommand extends VisorConsoleCommand { val (id, alert) = t var check = (true, true) + var printNid = false val values = mutable.ArrayBuffer.empty[String] + var valuesView = mutable.ArrayBuffer.empty[String] try check = alert.conditions.foldLeft(check) { (res, cond) => { - val gridRes = cond.gridFunc.map(f => { + val gridRes = cond.gridFunc.forall(f => { val (value, check) = f() values += value.toString + valuesView += cond.arg._1 + "=" + value.toString check - }).getOrElse(true) + }) - val nodeRes = cond.nodeFunc.map(f => { + val nodeRes = cond.nodeFunc.forall(f => { val (value, check) = f(node) values += value.toString + valuesView += cond.arg._1 + "=" + value.toString + + printNid = true check - }).getOrElse(true) + }) (res._1 && gridRes) -> (res._2 && nodeRes) } @@ -412,16 +415,22 @@ class VisorAlertCommand extends VisorConsoleCommand { stats = stats + (id -> stat) + val nodeIdIp = nodeId8Addr(node.id()).split(", ") + // Write to Visor log if it is started (see 'log' command). logText( "Alert [" + "id=" + alert.id + "(@" + alert.varName + "), " + - "spec=" + alert.spec + ", " + + (if (printNid) + "nid8=" + nodeIdIp(0) + ", ip=" + nodeIdIp.lift(1).getOrElse(NA) + ", " + else "") + + "spec=[" + alert.spec + "], " + + "values=[" + valuesView.mkString("; ") + "], " + "created on=" + formatDateTime(alert.createdOn) + "]" ) - executeAlertScript(alert, node, values.toSeq) + executeAlertScript(alert, node, values) last10 = VisorSentAlert( id = alert.id, @@ -794,7 +803,7 @@ object VisorAlertCommand { private[this] val BY_GRID = true private[this] val BY_NODE = false - private val alertDescr = Map( + private val ALERT_DESCRIPTORS = Map( "cc" -> VisorAlertMeta(BY_GRID, () => cl().metrics().getTotalCpus, dfltNodeValF), "nc" -> VisorAlertMeta(BY_GRID, () => cl().nodes().size, dfltNodeValF), "hc" -> VisorAlertMeta(BY_GRID, () => U.neighborhood(cl().nodes()).size, dfltNodeValF), From 8c370c6927848b66782ecc18df499460c0bfdd11 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Wed, 24 Aug 2016 10:03:33 +0700 Subject: [PATCH 051/487] IGNITE-2726 Added per-nodes cache information about off-heap. --- .../visor/commands/cache/VisorCacheCommand.scala | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) 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 a9f9137ed1cc7..68465de062ac3 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 @@ -323,7 +323,7 @@ class VisorCacheCommand { val sumT = VisorTextTable() - sumT #= ("Name(@)", "Mode", "Nodes", "Entries (Heap / Off heap)", "Hits", "Misses", "Reads", "Writes") + sumT #= ("Name(@)", "Mode", "Nodes", "Entries (Heap / Off-heap)", "Hits", "Misses", "Reads", "Writes") sortAggregatedData(aggrData, sortType.getOrElse("cn"), reversed).foreach( ad => { @@ -397,7 +397,7 @@ class VisorCacheCommand { (ad.maximumHeapSize() + ad.maximumOffHeapSize())) csT += (" Heap size Min/Avg/Max", ad.minimumHeapSize() + " / " + formatDouble(ad.averageHeapSize()) + " / " + ad.maximumHeapSize()) - csT += (" Off heap size Min/Avg/Max", ad.minimumOffHeapSize() + " / " + + csT += (" Off-heap size Min/Avg/Max", ad.minimumOffHeapSize() + " / " + formatDouble(ad.averageOffHeapSize()) + " / " + ad.maximumOffHeapSize()) val ciT = VisorTextTable() @@ -414,7 +414,15 @@ class VisorCacheCommand { formatDouble(nm.getCurrentCpuLoad * 100d) + " %", X.timeSpan2HMSM(nm.getUpTime), - cm.keySize(), + 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() + }, ( "Hi: " + cm.hits(), "Mi: " + cm.misses(), @@ -633,7 +641,7 @@ class VisorCacheCommand { val sumT = VisorTextTable() - sumT #= ("#", "Name(@)", "Mode", "Size (Heap / Off heap)") + sumT #= ("#", "Name(@)", "Mode", "Size (Heap / Off-heap)") sortedAggrData.indices.foreach(i => { val ad = sortedAggrData(i) From 25b59d7c6ea7bd23d5a8366fb5618e11990327c8 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Wed, 24 Aug 2016 11:07:17 +0300 Subject: [PATCH 052/487] IGNITE-3325 .NET: Rename CompiledQueryDelegate -> CompiledQueryFunc to conform to design guidelines --- modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery2.cs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery2.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery2.cs index 1ecc6e959d125..4131992a76d2f 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery2.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/CompiledQuery2.cs @@ -32,7 +32,7 @@ namespace Apache.Ignite.Linq /// Result type. /// The arguments. /// Query cursor. - public delegate IQueryCursor CompiledQueryDelegate(params object[] args); + public delegate IQueryCursor CompiledQueryFunc(params object[] args); /// /// Represents a compiled cache query. @@ -68,7 +68,7 @@ public static Func> Compile(Expression>> q /// Delegate that represents the compiled cache query. [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "0", Justification = "Invalid warning, validation is present.")] - public static CompiledQueryDelegate Compile(IQueryable query) + public static CompiledQueryFunc Compile(IQueryable query) { IgniteArgumentCheck.NotNull(query, "query"); From 4e9e7b8ee1c990bacdc2d081b706ca315927fdce Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Wed, 24 Aug 2016 12:12:00 +0300 Subject: [PATCH 053/487] IGNITE-3716: ODBC: Added SQL escape sequence parsing. --- .../processors/odbc/OdbcNioListener.java | 2 +- .../processors/odbc/OdbcRequestHandler.java | 36 ++- .../odbc/escape/OdbcEscapeParseResult.java | 73 +++++ .../odbc/escape/OdbcEscapeType.java | 26 ++ .../odbc/escape/OdbcEscapeUtils.java | 263 ++++++++++++++++++ .../odbc/OdbcEscapeSequenceSelfTest.java | 184 ++++++++++++ .../testsuites/IgniteBasicTestSuite.java | 2 + 7 files changed, 575 insertions(+), 11 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeParseResult.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java index f720096070e5e..e7baaff328687 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java @@ -121,7 +121,7 @@ public OdbcNioListener(GridKernalContext ctx, GridSpinBusyLock busyLock, int max OdbcRequestHandler handler = connData.getHandler(); - OdbcResponse resp = handler.handle(req); + OdbcResponse resp = handler.handle(reqId, req); if (log.isDebugEnabled()) { long dur = (System.nanoTime() - startTime) / 1000; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java index 43a1fa4d2899a..ce98720e80f8d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java @@ -18,10 +18,12 @@ package org.apache.ignite.internal.processors.odbc; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.cache.QueryCursorImpl; +import org.apache.ignite.internal.processors.odbc.escape.OdbcEscapeUtils; import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; import org.apache.ignite.internal.util.GridSpinBusyLock; @@ -45,6 +47,9 @@ public class OdbcRequestHandler { /** Kernel context. */ private final GridKernalContext ctx; + /** Logger. */ + private final IgniteLogger log; + /** Busy lock. */ private final GridSpinBusyLock busyLock; @@ -65,15 +70,18 @@ public OdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int this.ctx = ctx; this.busyLock = busyLock; this.maxCursors = maxCursors; + + log = ctx.log(OdbcRequestHandler.class); } /** * Handle request. * + * @param reqId Request ID. * @param req Request. * @return Response. */ - public OdbcResponse handle(OdbcRequest req) { + public OdbcResponse handle(long reqId, OdbcRequest req) { assert req != null; if (!busyLock.enterBusy()) @@ -83,22 +91,22 @@ public OdbcResponse handle(OdbcRequest req) { try { switch (req.command()) { case HANDSHAKE: - return performHandshake((OdbcHandshakeRequest) req); + return performHandshake(reqId, (OdbcHandshakeRequest)req); case EXECUTE_SQL_QUERY: - return executeQuery((OdbcQueryExecuteRequest) req); + return executeQuery(reqId, (OdbcQueryExecuteRequest)req); case FETCH_SQL_QUERY: - return fetchQuery((OdbcQueryFetchRequest) req); + return fetchQuery((OdbcQueryFetchRequest)req); case CLOSE_SQL_QUERY: - return closeQuery((OdbcQueryCloseRequest) req); + return closeQuery((OdbcQueryCloseRequest)req); case GET_COLUMNS_META: - return getColumnsMeta((OdbcQueryGetColumnsMetaRequest) req); + return getColumnsMeta((OdbcQueryGetColumnsMetaRequest)req); case GET_TABLES_META: - return getTablesMeta((OdbcQueryGetTablesMetaRequest) req); + return getTablesMeta((OdbcQueryGetTablesMetaRequest)req); } return new OdbcResponse(OdbcResponse.STATUS_FAILED, "Unsupported ODBC request: " + req); @@ -111,10 +119,11 @@ public OdbcResponse handle(OdbcRequest req) { /** * {@link OdbcHandshakeRequest} command handler. * + * @param reqId Request ID. * @param req Handshake request. * @return Response. */ - private OdbcResponse performHandshake(OdbcHandshakeRequest req) { + private OdbcResponse performHandshake(long reqId, OdbcHandshakeRequest req) { OdbcHandshakeResult res; if (req.version() == OdbcMessageParser.PROTO_VER) @@ -133,10 +142,11 @@ private OdbcResponse performHandshake(OdbcHandshakeRequest req) { /** * {@link OdbcQueryExecuteRequest} command handler. * + * @param reqId Request ID. * @param req Execute query request. * @return Response. */ - private OdbcResponse executeQuery(OdbcQueryExecuteRequest req) { + private OdbcResponse executeQuery(long reqId, OdbcQueryExecuteRequest req) { int cursorCnt = qryCursors.size(); if (maxCursors > 0 && cursorCnt >= maxCursors) @@ -147,7 +157,13 @@ private OdbcResponse executeQuery(OdbcQueryExecuteRequest req) { long qryId = QRY_ID_GEN.getAndIncrement(); try { - SqlFieldsQuery qry = new SqlFieldsQuery(req.sqlQuery()); + String sql = OdbcEscapeUtils.parse(req.sqlQuery()); + + if (log.isDebugEnabled()) + log.debug("ODBC query parsed [reqId=" + reqId + ", original=" + req.sqlQuery() + + ", parsed=" + sql + ']'); + + SqlFieldsQuery qry = new SqlFieldsQuery(sql); qry.setArgs(req.arguments()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeParseResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeParseResult.java new file mode 100644 index 0000000000000..cf05651c4d5e7 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeParseResult.java @@ -0,0 +1,73 @@ +/* + * 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.odbc.escape; + +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * ODBC escape sequence parse result. + */ +public class OdbcEscapeParseResult { + /** Original start position. */ + private final int originalStart; + + /** Original length. */ + private final int originalLen; + + /** Resulting text. */ + private final String res; + + /** + * Constructor. + * + * @param originalStart Original start position. + * @param originalLen Original length. + * @param res Resulting text. + */ + public OdbcEscapeParseResult(int originalStart, int originalLen, String res) { + this.originalStart = originalStart; + this.originalLen = originalLen; + this.res = res; + } + + /** + * @return Original start position. + */ + public int originalStart() { + return originalStart; + } + + /** + * @return Original length. + */ + public int originalLength() { + return originalLen; + } + + /** + * @return Resulting text. + */ + public String result() { + return res; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(OdbcEscapeParseResult.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java new file mode 100644 index 0000000000000..2df413f03761b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java @@ -0,0 +1,26 @@ +/* + * 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.odbc.escape; + +/** + * ODBC escape sequence type. + */ +public enum OdbcEscapeType { + /** Scalar function. */ + FN +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java new file mode 100644 index 0000000000000..4d8ca6983bceb --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java @@ -0,0 +1,263 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.odbc.escape; + +import org.apache.ignite.IgniteException; + +import java.util.LinkedList; + +/** + * ODBC escape sequence parse. + */ +public class OdbcEscapeUtils { + /** + * Parse escape sequence. + * + * @param text Original text. + * @return Result. + */ + public static String parse(String text) { + if (text == null) + throw new IgniteException("Text cannot be null."); + + return parse0(text.trim(), 0, false).result(); + } + + /** + * Internal parse routine. + * + * @param text Text. + * @param startPos Start position. + * @param earlyExit When set to {@code true} we must return as soon as single expression is parsed. + * @return Parse result. + */ + private static OdbcEscapeParseResult parse0(String text, int startPos, boolean earlyExit) { + StringBuilder res = new StringBuilder(); + + int curPos = startPos; + + int plainPos = startPos; + int openPos = -1; + + LinkedList nested = null; + + while (curPos < text.length()) { + char curChar = text.charAt(curPos); + + if (curChar == '{') { + if (openPos == -1) { + // Top-level opening brace. Append previous portion and remember current position. + res.append(text, plainPos, curPos); + + openPos = curPos; + } + else { + // Nested opening brace -> perform recursion. + OdbcEscapeParseResult nestedRes = parse0(text, curPos, true); + + if (nested == null) + nested = new LinkedList<>(); + + nested.add(nestedRes); + + curPos += nestedRes.originalLength() - 1; + + plainPos = curPos + 1; + } + } + else if (curChar == '}') { + if (openPos == -1) + // Close without open -> exception. + throw new IgniteException("Malformed escape sequence " + + "(closing curly brace without opening curly brace): " + text); + else { + String parseRes; + + if (nested == null) + // Found sequence without nesting, process it. + parseRes = parseExpression(text, openPos, curPos - openPos); + else { + // Special case to process nesting. + String res0 = appendNested(text, openPos, curPos + 1, nested); + + nested = null; + + parseRes = parseExpression(res0, 0, res0.length()-1); + } + + if (earlyExit) + return new OdbcEscapeParseResult(startPos, curPos - startPos + 1, parseRes); + else + res.append(parseRes); + + openPos = -1; + + plainPos = curPos + 1; + } + } + + curPos++; + } + + if (openPos != -1) + throw new IgniteException("Malformed escape sequence (closing curly brace missing): " + text); + + if (curPos > plainPos) + res.append(text, plainPos, curPos); + + return new OdbcEscapeParseResult(startPos, curPos - startPos + 1, res.toString()); + } + + /** + * Parse concrete expression. + * + * @param text Text. + * @param startPos Start position within text. + * @param len Length. + * @return Result. + */ + private static String parseExpression(String text, int startPos, int len) { + assert validSubstring(text, startPos, len); + + char firstChar = text.charAt(startPos); + + if (firstChar == '{') { + char lastChar = text.charAt(startPos + len); + + if (lastChar != '}') + throw new IgniteException("Failed to parse escape sequence because it is not enclosed: " + + substring(text, startPos, len)); + + OdbcEscapeType typ = sequenceType(text, startPos, len); + + switch (typ) { + case FN: + return parseScalarExpression(text, startPos, len); + + default: { + assert false : "Unknown expression type: " + typ; + + return null; + } + } + } + else { + // Nothing to escape, return original string. + if (startPos == 0 || text.length() == len) + return text; + else + return text.substring(startPos, startPos + len); + } + } + + /** + * Parse concrete expression. + * + * @param text Text. + * @param startPos Start position. + * @param len Length. + * @return Parsed expression. + */ + private static String parseScalarExpression(String text, int startPos, int len) { + assert validSubstring(text, startPos, len); + + return substring(text, startPos + 3, len - 3).trim(); + } + + /** + * Append nested results. + * + * @param text Original text. + * @param startPos Start position. + * @param endPos End position. + * @param nestedRess Nested results. + * @return Result. + */ + private static String appendNested(String text, int startPos, int endPos, + LinkedList nestedRess) { + StringBuilder res = new StringBuilder(); + + int curPos = startPos; + + for (OdbcEscapeParseResult nestedRes : nestedRess) { + // Append text between current position and replace. + res.append(text, curPos, nestedRes.originalStart()); + + // Append replaced text. + res.append(nestedRes.result()); + + // Advance position. + curPos = nestedRes.originalStart() + nestedRes.originalLength(); + } + + // Append remainder. + res.append(text, curPos, endPos); + + return res.toString(); + } + + /** + * Get escape sequence type. + * + * @param text Text. + * @param startPos Start position. + * @return Escape sequence type. + */ + private static OdbcEscapeType sequenceType(String text, int startPos, int len) { + assert validSubstring(text, startPos, len); + assert text.charAt(startPos) == '{'; + + if (text.startsWith("fn", startPos + 1)) + return OdbcEscapeType.FN; + + throw new IgniteException("Unsupported escape sequence: " + text.substring(startPos, startPos + len)); + } + + /** + * Perform "substring" using start position and length. + * + * @param text Text. + * @param startPos Start position. + * @param len Length. + * @return Substring. + */ + private static String substring(String text, int startPos, int len) { + assert validSubstring(text, startPos, len); + + return text.substring(startPos, startPos + len); + } + + /** + * Check whether substring is valid. + * + * @param text Substring. + * @param startPos Start position. + * @param len Length. + * @return {@code True} if valid. + */ + private static boolean validSubstring(String text, int startPos, int len) { + return text != null && startPos + len <= text.length(); + } + + /** + * Private constructor. + */ + private OdbcEscapeUtils() { + // No-op. + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java new file mode 100644 index 0000000000000..73fa0f4c57bb9 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java @@ -0,0 +1,184 @@ +/* + * 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.odbc; + +import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.processors.odbc.escape.OdbcEscapeUtils; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import java.util.concurrent.Callable; + +/** + * Scalar function escape sequence parser tests. + */ +public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest { + /** + * Test simple cases. + */ + public void testSimple() { + check( + "select * from table;", + "select * from table;" + ); + + check( + "test()", + "{fn test()}" + ); + + check( + "select test()", + "select {fn test()}" + ); + + check( + "select test() from table;", + "select {fn test()} from table;" + ); + } + + /** + * Test escape sequence series. + */ + public void testSimpleFunction() throws Exception { + check( + "func(field1) func(field2)", + "{fn func(field1)} {fn func(field2)}" + ); + + check( + "select func(field1), func(field2)", + "select {fn func(field1)}, {fn func(field2)}" + ); + + check( + "select func(field1), func(field2) from table;", + "select {fn func(field1)}, {fn func(field2)} from table;" + ); + } + + /** + * Test simple nested escape sequences. Depth = 2. + */ + public void testNestedFunction() throws Exception { + check( + "func1(field1, func2(field2))", + "{fn func1(field1, {fn func2(field2)})}" + ); + + check( + "select func1(field1, func2(field2))", + "select {fn func1(field1, {fn func2(field2)})}" + ); + + check( + "select func1(field1, func2(field2), field3) from SomeTable;", + "select {fn func1(field1, {fn func2(field2)}, field3)} from SomeTable;" + ); + } + + /** + * Test nested escape sequences. Depth > 2. + */ + public void testDeepNestedFunction() { + check( + "func1(func2(func3(field1)))", + "{fn func1({fn func2({fn func3(field1)})})}" + ); + + check( + "func1(func2(func3(func4(field1))))", + "{fn func1({fn func2({fn func3({fn func4(field1)})})})}" + ); + + check( + "select func1(field1, func2(func3(field2), field3))", + "select {fn func1(field1, {fn func2({fn func3(field2)}, field3)})}" + ); + + check( + "select func1(field1, func2(func3(field2), field3)) from SomeTable;", + "select {fn func1(field1, {fn func2({fn func3(field2)}, field3)})} from SomeTable;" + ); + } + + /** + * Test series of nested escape sequences. + */ + public void testNestedFunctionMixed() { + check( + "func1(func2(field1), func3(field2))", + "{fn func1({fn func2(field1)}, {fn func3(field2)})}" + ); + + check( + "select func1(func2(field1), func3(field2)) from table;", + "select {fn func1({fn func2(field1)}, {fn func3(field2)})} from table;" + ); + + check( + "func1(func2(func3(field1))) func1(func2(field2))", + "{fn func1({fn func2({fn func3(field1)})})} {fn func1({fn func2(field2)})}" + ); + } + + /** + * Test non-closed escape sequence. + */ + public void testFailedOnInvalidSequence1() { + checkFail("select {fn func1(field1, {fn func2(field2), field3)} from SomeTable;"); + } + + /** + * Test closing undeclared escape sequence. + */ + public void testFailedOnClosingNotOpenedSequence() { + checkFail("select {fn func1(field1, func2(field2)}, field3)} from SomeTable;"); + } + + /** + * Check parsing logic. + * + * @param exp Expected result. + * @param qry SQL query text. + */ + private void check(String exp, String qry) { + String actualRes = OdbcEscapeUtils.parse(qry); + + assertEquals(exp, actualRes); + } + + /** + * Check that query parsing fails. + * + * @param qry Query. + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + private void checkFail(final String qry) { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Void call() throws Exception { + OdbcEscapeUtils.parse(qry); + + fail("Parsing should fail: " + qry); + + return null; + } + }, IgniteException.class, null); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java index d56c29dbbeb76..6bb2c11bc00b6 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java @@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.continuous.GridEventConsumeSelfTest; import org.apache.ignite.internal.processors.continuous.GridMessageListenSelfTest; import org.apache.ignite.internal.processors.odbc.OdbcProcessorValidationSelfTest; +import org.apache.ignite.internal.processors.odbc.OdbcEscapeSequenceSelfTest; import org.apache.ignite.internal.processors.service.ClosureServiceClientsNodesTest; import org.apache.ignite.internal.product.GridProductVersionSelfTest; import org.apache.ignite.internal.util.nio.IgniteExceptionInNioWorkerSelfTest; @@ -128,6 +129,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(IgniteExceptionInNioWorkerSelfTest.class); suite.addTestSuite(OdbcProcessorValidationSelfTest.class); + suite.addTestSuite(OdbcEscapeSequenceSelfTest.class); GridTestUtils.addTestIfNeeded(suite, DynamicProxySerializationMultiJvmSelfTest.class, ignoredTests); From c02ad87f863bd730e60fb7052547fa0848e7eb77 Mon Sep 17 00:00:00 2001 From: isapego Date: Wed, 24 Aug 2016 17:21:48 +0300 Subject: [PATCH 054/487] IGNITE-3736: ODBC: Added support for string scalar functions. This closes #979. --- modules/platforms/cpp/odbc-test/Makefile.am | 2 + .../cpp/odbc-test/config/queries-test.xml | 7 +- .../cpp/odbc-test/include/Makefile.am | 3 +- .../cpp/odbc-test/include/test_utils.h | 45 ++ .../odbc-test/project/vs/odbc-test.vcxproj | 3 + .../project/vs/odbc-test.vcxproj.filters | 9 + .../cpp/odbc-test/src/queries_test.cpp | 41 +- .../src/sql_string_functions_test.cpp | 515 ++++++++++++++++++ .../cpp/odbc-test/src/test_utils.cpp | 36 ++ .../cpp/odbc/src/config/connection_info.cpp | 64 +-- .../cpp/odbc/src/query/data_query.cpp | 6 +- 11 files changed, 653 insertions(+), 78 deletions(-) create mode 100644 modules/platforms/cpp/odbc-test/include/test_utils.h create mode 100644 modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp create mode 100644 modules/platforms/cpp/odbc-test/src/test_utils.cpp diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index 48b193abbcd7d..cc84a7f84b450 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -61,6 +61,8 @@ ignite_odbc_tests_SOURCES = \ src/row_test.cpp \ src/utility_test.cpp \ src/queries_test.cpp \ + src/test_utils.cpp \ + src/sql_string_functions_test.cpp \ ../odbc/src/cursor.cpp \ ../odbc/src/config/connection_info.cpp \ ../odbc/src/app/application_data_buffer.cpp \ diff --git a/modules/platforms/cpp/odbc-test/config/queries-test.xml b/modules/platforms/cpp/odbc-test/config/queries-test.xml index f08f86d5d2468..67415fb6ff266 100644 --- a/modules/platforms/cpp/odbc-test/config/queries-test.xml +++ b/modules/platforms/cpp/odbc-test/config/queries-test.xml @@ -26,8 +26,8 @@ http://www.springframework.org/schema/util http://www.springframework.org/schema/util/spring-util.xsd"> - - + + @@ -84,11 +84,12 @@ - 127.0.0.1:47500..47501 + 127.0.0.1:47500 + diff --git a/modules/platforms/cpp/odbc-test/include/Makefile.am b/modules/platforms/cpp/odbc-test/include/Makefile.am index 6b2d8b6bc12ac..7b171027e9515 100644 --- a/modules/platforms/cpp/odbc-test/include/Makefile.am +++ b/modules/platforms/cpp/odbc-test/include/Makefile.am @@ -19,5 +19,6 @@ ACLOCAL_AMFLAGS =-I m4 noinst_HEADERS = \ teamcity/teamcity_messages.h \ - test_type.h + test_type.h \ + test_utils.h diff --git a/modules/platforms/cpp/odbc-test/include/test_utils.h b/modules/platforms/cpp/odbc-test/include/test_utils.h new file mode 100644 index 0000000000000..e8cd089287af1 --- /dev/null +++ b/modules/platforms/cpp/odbc-test/include/test_utils.h @@ -0,0 +1,45 @@ +/* + * 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. + */ + +#ifndef _IGNITE_ODBC_TEST_TEST_UTILS +#define _IGNITE_ODBC_TEST_TEST_UTILS + +#ifdef _WIN32 +# include +#endif + +#include +#include + +#include + +namespace ignite +{ + /** Read buffer size. */ + enum { ODBC_BUFFER_SIZE = 1024 }; + + /** + * Extract error message. + * + * @param handleType Type of the handle. + * @param handle Handle. + * @return Error message. + */ + std::string GetOdbcErrorMessage(SQLSMALLINT handleType, SQLHANDLE handle); +} + +#endif // _IGNITE_ODBC_TEST_TEST_UTILS \ No newline at end of file diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index fbc0929f4ba58..619b4b24cb9d8 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -168,13 +168,16 @@ + + + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index 2e38c242b7742..18711492ed9c0 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -76,6 +76,12 @@ Code + + Code + + + Code + @@ -84,6 +90,9 @@ Code\TeamCity + + Code + diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index cc3fa8eacf2ad..2d9bd582d3e1b 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -36,6 +36,7 @@ #include "ignite/impl/binary/binary_utils.h" #include "test_type.h" +#include "test_utils.h" using namespace ignite; using namespace ignite::cache; @@ -46,30 +47,6 @@ using namespace boost::unit_test; using ignite::impl::binary::BinaryUtils; -/** Read buffer size. */ -enum { ODBC_BUFFER_SIZE = 1024 }; - -/** - * Extract error message. - * - * @param handleType Type of the handle. - * @param handle Handle. - * @return Error message. - */ -std::string GetOdbcErrorMessage(SQLSMALLINT handleType, SQLHANDLE handle) -{ - SQLCHAR sqlstate[7] = {}; - SQLINTEGER nativeCode; - - SQLCHAR message[ODBC_BUFFER_SIZE]; - SQLSMALLINT reallen = 0; - - SQLGetDiagRec(handleType, handle, 1, sqlstate, &nativeCode, message, ODBC_BUFFER_SIZE, &reallen); - - return std::string(reinterpret_cast(sqlstate)) + ": " + - std::string(reinterpret_cast(message), reallen); -} - /** * Test setup fixture. */ @@ -98,7 +75,9 @@ struct QueriesTestSuiteFixture char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH"); - cfg.springCfgPath = std::string(cfgPath).append("/").append("queries-test.xml"); + BOOST_REQUIRE(cfgPath != 0); + + cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml"); IgniteError err; @@ -178,7 +157,7 @@ struct QueriesTestSuiteFixture T columns[columnsCnt] = { 0 }; - // Binding colums. + // Binding columns. for (SQLSMALLINT i = 0; i < columnsCnt; ++i) { ret = SQLBindCol(stmt, i + 1, type, &columns[i], sizeof(columns[i]), 0); @@ -214,7 +193,7 @@ struct QueriesTestSuiteFixture SQLLEN columnLens[columnsCnt] = { 0 }; - // Binding colums. + // Binding columns. for (SQLSMALLINT i = 0; i < columnsCnt; ++i) { ret = SQLBindCol(stmt, i + 1, type, &columns[i], sizeof(columns[i]), &columnLens[i]); @@ -327,7 +306,7 @@ BOOST_AUTO_TEST_CASE(TestTwoRowsString) SQLCHAR columns[columnsCnt][ODBC_BUFFER_SIZE] = { 0 }; - // Binding colums. + // Binding columns. for (SQLSMALLINT i = 0; i < columnsCnt; ++i) { ret = SQLBindCol(stmt, i + 1, SQL_C_CHAR, &columns[i], ODBC_BUFFER_SIZE, 0); @@ -364,7 +343,7 @@ BOOST_AUTO_TEST_CASE(TestTwoRowsString) SQLLEN columnLens[columnsCnt] = { 0 }; - // Binding colums. + // Binding columns. for (SQLSMALLINT i = 0; i < columnsCnt; ++i) { ret = SQLBindCol(stmt, i + 1, SQL_C_CHAR, &columns[i], ODBC_BUFFER_SIZE, &columnLens[i]); @@ -420,7 +399,7 @@ BOOST_AUTO_TEST_CASE(TestOneRowString) SQLLEN columnLens[columnsCnt] = { 0 }; - // Binding colums. + // Binding columns. for (SQLSMALLINT i = 0; i < columnsCnt; ++i) { ret = SQLBindCol(stmt, i + 1, SQL_C_CHAR, &columns[i], ODBC_BUFFER_SIZE, &columnLens[i]); @@ -479,7 +458,7 @@ BOOST_AUTO_TEST_CASE(TestOneRowStringLen) SQLLEN columnLens[columnsCnt] = { 0 }; - // Binding colums. + // Binding columns. for (SQLSMALLINT i = 0; i < columnsCnt; ++i) { ret = SQLBindCol(stmt, i + 1, SQL_C_CHAR, 0, 0, &columnLens[i]); diff --git a/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp new file mode 100644 index 0000000000000..e84aba64e173a --- /dev/null +++ b/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp @@ -0,0 +1,515 @@ +/* + * 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. + */ + +#define _USE_MATH_DEFINES + +#ifdef _WIN32 +# include +#endif + +#include +#include + +#include + +#include +#include + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include "ignite/ignite.h" +#include "ignite/ignition.h" +#include "ignite/impl/binary/binary_utils.h" + +#include "test_type.h" +#include "test_utils.h" + +using namespace ignite; +using namespace ignite::cache; +using namespace ignite::cache::query; +using namespace ignite::common; + +using namespace boost::unit_test; + +using ignite::impl::binary::BinaryUtils; + +/** + * Test setup fixture. + */ +struct SqlStringFunctionTestSuiteFixture +{ + /** + * Constructor. + */ + SqlStringFunctionTestSuiteFixture() : + testCache(0), + env(NULL), + dbc(NULL), + stmt(NULL) + { + IgniteConfiguration cfg; + + cfg.jvmOpts.push_back("-Xdebug"); + cfg.jvmOpts.push_back("-Xnoagent"); + cfg.jvmOpts.push_back("-Djava.compiler=NONE"); + cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005"); + cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError"); + +#ifdef IGNITE_TESTS_32 + cfg.jvmInitMem = 256; + cfg.jvmMaxMem = 768; +#else + cfg.jvmInitMem = 1024; + cfg.jvmMaxMem = 4096; +#endif + + char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH"); + + BOOST_REQUIRE(cfgPath != 0); + + cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml"); + + IgniteError err; + + grid = Ignition::Start(cfg, &err); + + if (err.GetCode() != IgniteError::IGNITE_SUCCESS) + BOOST_FAIL(err.GetText()); + + testCache = grid.GetCache("cache"); + + // Allocate an environment handle + SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env); + + BOOST_REQUIRE(env != NULL); + + // We want ODBC 3 support + SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast(SQL_OV_ODBC3), 0); + + // Allocate a connection handle + SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc); + + BOOST_REQUIRE(dbc != NULL); + + // Connect string + SQLCHAR connectStr[] = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=cache"; + + SQLCHAR outstr[ODBC_BUFFER_SIZE]; + SQLSMALLINT outstrlen; + + // Connecting to ODBC server. + SQLRETURN ret = SQLDriverConnect(dbc, NULL, connectStr, static_cast(sizeof(connectStr)), + outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE); + + if (!SQL_SUCCEEDED(ret)) + { + Ignition::Stop(grid.GetName(), true); + + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc)); + } + + // Allocate a statement handle + SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt); + + BOOST_REQUIRE(stmt != NULL); + } + + /** + * Destructor. + */ + ~SqlStringFunctionTestSuiteFixture() + { + // Releasing statement handle. + SQLFreeHandle(SQL_HANDLE_STMT, stmt); + + // Disconneting from the server. + SQLDisconnect(dbc); + + // Releasing allocated handles. + SQLFreeHandle(SQL_HANDLE_DBC, dbc); + SQLFreeHandle(SQL_HANDLE_ENV, env); + + Ignition::Stop(grid.GetName(), true); + } + + void CheckSingleResult0(const char* request, SQLSMALLINT type, void* column, SQLLEN bufSize, SQLLEN* resSize) + { + SQLRETURN ret; + + ret = SQLBindCol(stmt, 1, type, column, bufSize, resSize); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLExecDirect(stmt, reinterpret_cast(const_cast(request)), SQL_NTS); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFetch(stmt); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFetch(stmt); + BOOST_CHECK(ret == SQL_NO_DATA); + } + + /** + * Run query returning single result and check it to be equal to expected. + * + * @param request SQL request. + * @param expected Expected result. + */ + template + void CheckSingleResult(const char* request, const T& expected) + { + BOOST_FAIL("Function is not defined for the type."); + } + + /** + * Run query returning single result and check it to be equal to expected. + * + * @param request SQL request. + * @param expected Expected result. + * @param type Result type. + */ + template + void CheckSingleResultNum0(const char* request, const T& expected, SQLSMALLINT type) + { + T res = 0; + + CheckSingleResult0(request, type, &res, 0, 0); + + BOOST_CHECK_EQUAL(res, expected); + } + + + /** Node started during the test. */ + Ignite grid; + + /** Test cache instance. */ + Cache testCache; + + /** ODBC Environment. */ + SQLHENV env; + + /** ODBC Connect. */ + SQLHDBC dbc; + + /** ODBC Statement. */ + SQLHSTMT stmt; +}; + +template<> +void SqlStringFunctionTestSuiteFixture::CheckSingleResult(const char* request, const std::string& expected) +{ + SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 }; + SQLLEN resLen = 0; + + CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen); + + BOOST_CHECK_EQUAL(std::string(reinterpret_cast(res), static_cast(resLen)), expected); +} + +template<> +void SqlStringFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int64_t& expected) +{ + CheckSingleResultNum0(request, expected, SQL_C_SBIGINT); +} + +template<> +void SqlStringFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int32_t& expected) +{ + CheckSingleResultNum0(request, expected, SQL_C_SLONG); +} + +template<> +void SqlStringFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int16_t& expected) +{ + CheckSingleResultNum0(request, expected, SQL_C_SSHORT); +} + +template<> +void SqlStringFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int8_t& expected) +{ + CheckSingleResultNum0(request, expected, SQL_C_STINYINT); +} + +template<> +void SqlStringFunctionTestSuiteFixture::CheckSingleResult(const char* request, const float& expected) +{ + SQLFLOAT res = 0; + + CheckSingleResult0(request, SQL_C_FLOAT, &res, 0, 0); + + BOOST_CHECK_CLOSE(static_cast(res), expected, 1E-6f); +} + +template<> +void SqlStringFunctionTestSuiteFixture::CheckSingleResult(const char* request, const double& expected) +{ + SQLDOUBLE res = 0; + + CheckSingleResult0(request, SQL_C_DOUBLE, &res, 0, 0); + + BOOST_CHECK_CLOSE(static_cast(res), expected, 1E-6); +} + +BOOST_FIXTURE_TEST_SUITE(SqlStringFunctionTestSuite, SqlStringFunctionTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(TestStringFunctionAscii) +{ + TestType in; + + in.strField = "Hi"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn ASCII(strField)} FROM TestType", static_cast('H')); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionBitLength) +{ + TestType in; + in.strField = "Lorem ipsum dolor"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn BIT_LENGTH(strField)} FROM TestType", in.strField.size() * 16); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionChar) +{ + TestType in; + + in.i32Field = static_cast('H'); + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn CHAR(i32Field)} FROM TestType", "H"); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionCharLength) +{ + TestType in; + in.strField = "Lorem ipsum dolor"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn CHAR_LENGTH(strField)} FROM TestType", in.strField.size()); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionCharacterLength) +{ + TestType in; + in.strField = "Lorem ipsum dolor"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn CHARACTER_LENGTH(strField)} FROM TestType", in.strField.size()); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionConcat) +{ + TestType in; + in.strField = "Lorem ipsum dolor sit amet,"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn CONCAT(strField, \' consectetur adipiscing elit\')} FROM TestType", + in.strField + " consectetur adipiscing elit"); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionDifference) +{ + TestType in; + in.strField = "Hello"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn DIFFERENCE(strField, \'Hola!\')} FROM TestType", 4); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionInsert) +{ + TestType in; + in.strField = "Hello World!"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn INSERT(strField, 7, 5, \'Ignite\')} FROM TestType", "Hello Ignite!"); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionLcase) +{ + TestType in; + in.strField = "Hello World!"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn LCASE(strField)} FROM TestType", "hello world!"); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionLeft) +{ + TestType in; + in.strField = "Hello World!"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn LEFT(strField, 5)} FROM TestType", "Hello"); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionLength) +{ + TestType in; + in.strField = "Lorem ipsum dolor sit amet, consectetur adipiscing elit"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn LENGTH(strField)} FROM TestType", in.strField.size()); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionLocate) +{ + TestType in; + in.strField = "Lorem ipsum dolor sit amet, consectetur adipiscing elit"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn LOCATE(\'ip\', strField)} FROM TestType", 7); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionLocate2) +{ + TestType in; + in.strField = "Lorem ipsum dolor sit amet, consectetur adipiscing elit"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn LOCATE(\'ip\', strField, 10)} FROM TestType", 43); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionLtrim) +{ + TestType in; + in.strField = " Lorem ipsum "; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn LTRIM(strField)} FROM TestType", "Lorem ipsum "); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionOctetLength) +{ + TestType in; + in.strField = "Lorem ipsum dolor sit amet, consectetur adipiscing elit"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn OCTET_LENGTH(strField)} FROM TestType", in.strField.size() * 2); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionPosition) +{ + TestType in; + in.strField = "Lorem ipsum dolor sit amet, consectetur adipiscing elit"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn POSITION(\'sit\', strField)} FROM TestType", 19); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionRepeat) +{ + TestType in; + in.strField = "Test"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn REPEAT(strField,4)} FROM TestType", "TestTestTestTest"); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionReplace) +{ + TestType in; + in.strField = "Hello Ignite!"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn REPLACE(strField, \'Ignite\', \'World\')} FROM TestType", "Hello World!"); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionRight) +{ + TestType in; + in.strField = "Hello World!"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn RIGHT(strField, 6)} FROM TestType", "World!"); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionRtrim) +{ + TestType in; + in.strField = " Lorem ipsum "; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn RTRIM(strField)} FROM TestType", " Lorem ipsum"); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionSoundex) +{ + TestType in; + in.strField = "Hello Ignite!"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn SOUNDEX(strField)} FROM TestType", "H425"); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionSpace) +{ + CheckSingleResult("SELECT {fn SPACE(10)}", " "); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionSubstring) +{ + TestType in; + in.strField = "Hello Ignite!"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn SUBSTRING(strField, 7, 6)} FROM TestType", "Ignite"); +} + +BOOST_AUTO_TEST_CASE(TestStringFunctionUcase) +{ + TestType in; + in.strField = "Hello World!"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn UCASE(strField)} FROM TestType", "HELLO WORLD!"); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc-test/src/test_utils.cpp b/modules/platforms/cpp/odbc-test/src/test_utils.cpp new file mode 100644 index 0000000000000..48f409b770976 --- /dev/null +++ b/modules/platforms/cpp/odbc-test/src/test_utils.cpp @@ -0,0 +1,36 @@ +/* + * 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. + */ + +#include "test_utils.h" + +namespace ignite +{ + + std::string GetOdbcErrorMessage(SQLSMALLINT handleType, SQLHANDLE handle) + { + SQLCHAR sqlstate[7] = {}; + SQLINTEGER nativeCode; + + SQLCHAR message[ODBC_BUFFER_SIZE]; + SQLSMALLINT reallen = 0; + + SQLGetDiagRec(handleType, handle, 1, sqlstate, &nativeCode, message, ODBC_BUFFER_SIZE, &reallen); + + return std::string(reinterpret_cast(sqlstate)) + ": " + + std::string(reinterpret_cast(message), reallen); + } +} diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp index 7f0e3bdfd24fb..f35b790077681 100644 --- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp +++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp @@ -124,7 +124,7 @@ namespace ignite #ifdef SQL_COLUMN_ALIAS // A character string: "Y" if the data source supports column // aliases; otherwise, "N". - strParams[SQL_COLUMN_ALIAS] = "N"; + strParams[SQL_COLUMN_ALIAS] = "Y"; #endif // SQL_COLUMN_ALIAS #ifdef SQL_IDENTIFIER_QUOTE_CHAR @@ -192,7 +192,7 @@ namespace ignite #ifdef SQL_GETDATA_EXTENSIONS // Bitmask enumerating extensions to SQLGetData. - intParams[SQL_GETDATA_EXTENSIONS] = SQL_GD_ANY_COLUMN; + intParams[SQL_GETDATA_EXTENSIONS] = SQL_GD_ANY_COLUMN | SQL_GD_ANY_ORDER | SQL_GD_BOUND; #endif // SQL_GETDATA_EXTENSIONS #ifdef SQL_ODBC_INTERFACE_CONFORMANCE @@ -203,7 +203,7 @@ namespace ignite #ifdef SQL_SQL_CONFORMANCE // Indicates the level of SQL-92 supported by the driver. - intParams[SQL_SQL_CONFORMANCE] = 0; // SQL_SC_SQL92_ENTRY; + intParams[SQL_SQL_CONFORMANCE] = SQL_SC_SQL92_ENTRY; #endif // SQL_SQL_CONFORMANCE #ifdef SQL_CATALOG_USAGE @@ -213,7 +213,8 @@ namespace ignite #ifdef SQL_SCHEMA_USAGE // Bitmask enumerating the statements in which schemas can be used. - intParams[SQL_SCHEMA_USAGE] = 0; + intParams[SQL_SCHEMA_USAGE] = SQL_SU_DML_STATEMENTS | + SQL_SU_TABLE_DEFINITION | SQL_SU_PRIVILEGE_DEFINITION; #endif // SQL_SCHEMA_USAGE #ifdef SQL_MAX_IDENTIFIER_LEN @@ -224,21 +225,24 @@ namespace ignite #ifdef SQL_AGGREGATE_FUNCTIONS // Bitmask enumerating support for aggregation functions. - intParams[SQL_AGGREGATE_FUNCTIONS] = SQL_AF_ALL | SQL_AF_AVG | - SQL_AF_COUNT | SQL_AF_DISTINCT | SQL_AF_MAX | SQL_AF_MIN | - SQL_AF_SUM; + intParams[SQL_AGGREGATE_FUNCTIONS] = 0; #endif // SQL_AGGREGATE_FUNCTIONS #ifdef SQL_NUMERIC_FUNCTIONS // Bitmask enumerating the scalar numeric functions supported by // the driver and associated data source. - intParams[SQL_NUMERIC_FUNCTIONS] = SQL_FN_NUM_ABS; + intParams[SQL_NUMERIC_FUNCTIONS] = 0; #endif // SQL_NUMERIC_FUNCTIONS #ifdef SQL_STRING_FUNCTIONS // Bitmask enumerating the scalar string functions supported by the // driver and associated data source. - intParams[SQL_STRING_FUNCTIONS] = 0; + intParams[SQL_STRING_FUNCTIONS] = SQL_FN_STR_ASCII | SQL_FN_STR_BIT_LENGTH | SQL_FN_STR_CHAR | + SQL_FN_STR_CONCAT | SQL_FN_STR_DIFFERENCE | SQL_FN_STR_INSERT | SQL_FN_STR_LEFT | + SQL_FN_STR_LENGTH | SQL_FN_STR_LOCATE | SQL_FN_STR_LTRIM | SQL_FN_STR_OCTET_LENGTH | + SQL_FN_STR_POSITION | SQL_FN_STR_REPEAT | SQL_FN_STR_REPLACE | SQL_FN_STR_RIGHT | SQL_FN_STR_RTRIM | + SQL_FN_STR_SOUNDEX | SQL_FN_STR_SPACE | SQL_FN_STR_SUBSTRING | SQL_FN_STR_LCASE | SQL_FN_STR_UCASE | + SQL_FN_STR_LOCATE_2 | SQL_FN_STR_CHAR_LENGTH | SQL_FN_STR_CHARACTER_LENGTH; #endif // SQL_STRING_FUNCTIONS #ifdef SQL_TIMEDATE_FUNCTIONS @@ -262,15 +266,7 @@ namespace ignite #ifdef SQL_DATETIME_LITERALS // Bitmask enumerating the SQL-92 datetime literals supported by // the data source. - intParams[SQL_DATETIME_LITERALS] = SQL_DL_SQL92_INTERVAL_HOUR | - SQL_DL_SQL92_DATE | SQL_DL_SQL92_INTERVAL_MINUTE_TO_SECOND | - SQL_DL_SQL92_TIME | SQL_DL_SQL92_INTERVAL_HOUR_TO_SECOND | - SQL_DL_SQL92_TIMESTAMP | SQL_DL_SQL92_INTERVAL_HOUR_TO_MINUTE | - SQL_DL_SQL92_INTERVAL_YEAR | SQL_DL_SQL92_INTERVAL_DAY_TO_SECOND | - SQL_DL_SQL92_INTERVAL_MONTH | SQL_DL_SQL92_INTERVAL_DAY_TO_HOUR | - SQL_DL_SQL92_INTERVAL_DAY | SQL_DL_SQL92_INTERVAL_DAY_TO_MINUTE | - SQL_DL_SQL92_INTERVAL_MINUTE | SQL_DL_SQL92_INTERVAL_SECOND | - SQL_DL_SQL92_INTERVAL_YEAR_TO_MONTH; + intParams[SQL_DATETIME_LITERALS] = 0; #endif // SQL_DATETIME_LITERALS #ifdef SQL_SYSTEM_FUNCTIONS @@ -288,9 +284,7 @@ namespace ignite #ifdef SQL_OJ_CAPABILITIES // Bitmask enumerating the types of outer joins supported by the // driver and data source. - intParams[SQL_OJ_CAPABILITIES] = SQL_OJ_LEFT | SQL_OJ_RIGHT | - SQL_OJ_FULL | SQL_OJ_NESTED | SQL_OJ_INNER | - SQL_OJ_ALL_COMPARISON_OPS; + intParams[SQL_OJ_CAPABILITIES] = 0; #endif // SQL_OJ_CAPABILITIES #ifdef SQL_POS_OPERATIONS @@ -305,42 +299,36 @@ namespace ignite #ifdef SQL_SQL92_STRING_FUNCTIONS // Bitmask enumerating the string scalar functions. - intParams[SQL_SQL92_STRING_FUNCTIONS] = 0; + intParams[SQL_SQL92_STRING_FUNCTIONS] = SQL_SSF_CONVERT | SQL_SSF_LOWER | SQL_SSF_UPPER | + SQL_SSF_SUBSTRING | SQL_SSF_TRANSLATE; #endif // SQL_SQL92_STRING_FUNCTIONS #ifdef SQL_SQL92_DATETIME_FUNCTIONS // Bitmask enumerating the datetime scalar functions. - intParams[SQL_SQL92_DATETIME_FUNCTIONS] = SQL_SDF_CURRENT_DATE | - SQL_SDF_CURRENT_TIMESTAMP; + intParams[SQL_SQL92_DATETIME_FUNCTIONS] = 0; #endif // SQL_SQL92_DATETIME_FUNCTIONS #ifdef SQL_SQL92_VALUE_EXPRESSIONS // Bitmask enumerating the value expressions supported, // as defined in SQL-92. intParams[SQL_SQL92_VALUE_EXPRESSIONS] = SQL_SVE_CASE | - SQL_SVE_COALESCE | SQL_SVE_NULLIF; + SQL_SVE_CAST | SQL_SVE_COALESCE | SQL_SVE_NULLIF; #endif // SQL_SQL92_VALUE_EXPRESSIONS #ifdef SQL_SQL92_PREDICATES // Bitmask enumerating the datetime scalar functions. - intParams[SQL_SQL92_PREDICATES] = SQL_SP_BETWEEN | - SQL_SP_COMPARISON | SQL_SP_EXISTS | SQL_SP_IN | - SQL_SP_ISNOTNULL | SQL_SP_ISNULL | SQL_SP_LIKE | - SQL_SP_MATCH_FULL | SQL_SP_MATCH_PARTIAL | - SQL_SP_MATCH_UNIQUE_FULL | SQL_SP_MATCH_UNIQUE_PARTIAL | - SQL_SP_OVERLAPS | SQL_SP_QUANTIFIED_COMPARISON | - SQL_SP_UNIQUE; + intParams[SQL_SQL92_PREDICATES] = SQL_SP_BETWEEN | SQL_SP_COMPARISON | SQL_SP_EXISTS | SQL_SP_IN | + SQL_SP_ISNOTNULL | SQL_SP_ISNULL | SQL_SP_LIKE | SQL_SP_MATCH_FULL | SQL_SP_MATCH_PARTIAL | + SQL_SP_MATCH_UNIQUE_FULL | SQL_SP_MATCH_UNIQUE_PARTIAL | SQL_SP_OVERLAPS | SQL_SP_UNIQUE | + SQL_SP_QUANTIFIED_COMPARISON; #endif // SQL_SQL92_PREDICATES #ifdef SQL_SQL92_RELATIONAL_JOIN_OPERATORS // Bitmask enumerating the relational join operators supported // in a SELECT statement, as defined in SQL-92. - intParams[SQL_SQL92_RELATIONAL_JOIN_OPERATORS] = - SQL_SRJO_CORRESPONDING_CLAUSE | SQL_SRJO_CROSS_JOIN | - SQL_SRJO_EXCEPT_JOIN | SQL_SRJO_EXCEPT_JOIN | - SQL_SRJO_INNER_JOIN | SQL_SRJO_INTERSECT_JOIN | - SQL_SRJO_LEFT_OUTER_JOIN | SQL_SRJO_NATURAL_JOIN | - SQL_SRJO_RIGHT_OUTER_JOIN | SQL_SRJO_UNION_JOIN; + intParams[SQL_SQL92_RELATIONAL_JOIN_OPERATORS] = SQL_SRJO_CORRESPONDING_CLAUSE | SQL_SRJO_CROSS_JOIN | + SQL_SRJO_EXCEPT_JOIN | SQL_SRJO_INNER_JOIN | SQL_SRJO_LEFT_OUTER_JOIN| SQL_SRJO_RIGHT_OUTER_JOIN | + SQL_SRJO_NATURAL_JOIN | SQL_SRJO_INTERSECT_JOIN | SQL_SRJO_UNION_JOIN; #endif // SQL_SQL92_RELATIONAL_JOIN_OPERATORS //========================= Short Params ========================== diff --git a/modules/platforms/cpp/odbc/src/query/data_query.cpp b/modules/platforms/cpp/odbc/src/query/data_query.cpp index e96f1da387a1f..183bbb564046f 100644 --- a/modules/platforms/cpp/odbc/src/query/data_query.cpp +++ b/modules/platforms/cpp/odbc/src/query/data_query.cpp @@ -141,11 +141,7 @@ namespace ignite SqlResult DataQuery::Close() { if (!cursor.get()) - { - diag.AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Query cursor is not in open state."); - - return SQL_RESULT_ERROR; - } + return SQL_RESULT_SUCCESS; SqlResult result = MakeRequestClose(); From 13dfcbe03aca137ee70698f6083df27c10ecdaf9 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Wed, 24 Aug 2016 17:31:38 +0300 Subject: [PATCH 055/487] IGNITE-3736: ODBC: Reverted back removed "supported" futures. --- .../cpp/odbc/src/config/connection_info.cpp | 23 +++++++++++++++---- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp index f35b790077681..f9a3f96a6fb19 100644 --- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp +++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp @@ -225,13 +225,15 @@ namespace ignite #ifdef SQL_AGGREGATE_FUNCTIONS // Bitmask enumerating support for aggregation functions. - intParams[SQL_AGGREGATE_FUNCTIONS] = 0; + intParams[SQL_AGGREGATE_FUNCTIONS] = SQL_AF_ALL | SQL_AF_AVG | + SQL_AF_COUNT | SQL_AF_DISTINCT | SQL_AF_MAX | SQL_AF_MIN | + SQL_AF_SUM; #endif // SQL_AGGREGATE_FUNCTIONS #ifdef SQL_NUMERIC_FUNCTIONS // Bitmask enumerating the scalar numeric functions supported by // the driver and associated data source. - intParams[SQL_NUMERIC_FUNCTIONS] = 0; + intParams[SQL_NUMERIC_FUNCTIONS] = SQL_FN_NUM_ABS; #endif // SQL_NUMERIC_FUNCTIONS #ifdef SQL_STRING_FUNCTIONS @@ -266,7 +268,15 @@ namespace ignite #ifdef SQL_DATETIME_LITERALS // Bitmask enumerating the SQL-92 datetime literals supported by // the data source. - intParams[SQL_DATETIME_LITERALS] = 0; + intParams[SQL_DATETIME_LITERALS] = SQL_DL_SQL92_INTERVAL_HOUR | + SQL_DL_SQL92_DATE | SQL_DL_SQL92_INTERVAL_MINUTE_TO_SECOND | + SQL_DL_SQL92_TIME | SQL_DL_SQL92_INTERVAL_HOUR_TO_SECOND | + SQL_DL_SQL92_TIMESTAMP | SQL_DL_SQL92_INTERVAL_HOUR_TO_MINUTE | + SQL_DL_SQL92_INTERVAL_YEAR | SQL_DL_SQL92_INTERVAL_DAY_TO_SECOND | + SQL_DL_SQL92_INTERVAL_MONTH | SQL_DL_SQL92_INTERVAL_DAY_TO_HOUR | + SQL_DL_SQL92_INTERVAL_DAY | SQL_DL_SQL92_INTERVAL_DAY_TO_MINUTE | + SQL_DL_SQL92_INTERVAL_MINUTE | SQL_DL_SQL92_INTERVAL_SECOND | + SQL_DL_SQL92_INTERVAL_YEAR_TO_MONTH; #endif // SQL_DATETIME_LITERALS #ifdef SQL_SYSTEM_FUNCTIONS @@ -284,7 +294,9 @@ namespace ignite #ifdef SQL_OJ_CAPABILITIES // Bitmask enumerating the types of outer joins supported by the // driver and data source. - intParams[SQL_OJ_CAPABILITIES] = 0; + intParams[SQL_OJ_CAPABILITIES] = SQL_OJ_LEFT | SQL_OJ_RIGHT | + SQL_OJ_FULL | SQL_OJ_NESTED | SQL_OJ_INNER | + SQL_OJ_ALL_COMPARISON_OPS; #endif // SQL_OJ_CAPABILITIES #ifdef SQL_POS_OPERATIONS @@ -305,7 +317,8 @@ namespace ignite #ifdef SQL_SQL92_DATETIME_FUNCTIONS // Bitmask enumerating the datetime scalar functions. - intParams[SQL_SQL92_DATETIME_FUNCTIONS] = 0; + intParams[SQL_SQL92_DATETIME_FUNCTIONS] = SQL_SDF_CURRENT_DATE | + SQL_SDF_CURRENT_TIMESTAMP; #endif // SQL_SQL92_DATETIME_FUNCTIONS #ifdef SQL_SQL92_VALUE_EXPRESSIONS From d6449ffbc65acda6a2cf4484608188367837dd17 Mon Sep 17 00:00:00 2001 From: sboikov Date: Wed, 24 Aug 2016 18:34:02 +0300 Subject: [PATCH 056/487] Fixed issues on node stop: - in service processor need guard depExe access with busyLock - do not error log IO errors in ClientImpl on stop --- .../service/GridServiceProcessor.java | 18 +++++++++++++----- .../ignite/spi/discovery/tcp/ClientImpl.java | 7 ++++++- 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 01b73026198c6..7b76c4839be34 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -1355,11 +1355,19 @@ public void onUtilityCacheStarted() { private class ServiceEntriesListener implements CacheEntryUpdatedListener { /** {@inheritDoc} */ @Override public void onUpdated(final Iterable> deps) { - depExe.submit(new BusyRunnable() { - @Override public void run0() { - onSystemCacheUpdated(deps); - } - }); + if (!busyLock.enterBusy()) + return; + + try { + depExe.submit(new BusyRunnable() { + @Override public void run0() { + onSystemCacheUpdated(deps); + } + }); + } + finally { + busyLock.leaveBusy(); + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 9821134abfbc8..1e71888f7fd62 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -1115,7 +1115,12 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { } } catch (IgniteCheckedException e) { - U.error(log, "Failed to send message: " + msg, e); + if (spi.getSpiContext().isStopping()) { + if (log.isDebugEnabled()) + log.debug("Failed to send message, node is stopping [msg=" + msg + ", err=" + e + ']'); + } + else + U.error(log, "Failed to send message: " + msg, e); msg = null; } From 12fd4976f482ebc43831754645e34042c9073b2d Mon Sep 17 00:00:00 2001 From: sboikov Date: Thu, 25 Aug 2016 12:29:04 +0300 Subject: [PATCH 057/487] Fixed GridQueryParsingTest. --- .../internal/processors/query/h2/sql/GridQueryParsingTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java index ce1f241d5680e..1d54bbfcb5014 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java @@ -301,7 +301,7 @@ private void assertSqlEquals(String sql1, String sql2) { private static String normalizeSql(String sql) { return sql.toLowerCase() .replaceAll("/\\*(?:.|\r|\n)*?\\*/", " ") - .replaceAll("\\s*on\\s+1\\s*=\\s*1\\s*", " ") + .replaceAll("\\s*on\\s+1\\s*=\\s*1\\s*", " on true ") .replaceAll("\\s+", " ") .replaceAll("\\( +", "(") .replaceAll(" +\\)", ")") From 5a3b3e2c6ecb5d6c96513b79f21828526b4a98a0 Mon Sep 17 00:00:00 2001 From: isapego Date: Thu, 25 Aug 2016 12:35:07 +0300 Subject: [PATCH 058/487] IGNITE-3749: ODBC: Added support for numeric scalar functions. This closes #981. --- modules/platforms/cpp/odbc-test/Makefile.am | 2 + .../cpp/odbc-test/include/Makefile.am | 3 +- .../include/sql_function_test_suite_fixture.h | 138 ++++++++ .../odbc-test/project/vs/odbc-test.vcxproj | 3 + .../project/vs/odbc-test.vcxproj.filters | 9 + .../src/sql_function_test_suite_fixture.cpp | 188 +++++++++++ .../src/sql_numeric_functions_test.cpp | 309 ++++++++++++++++++ .../src/sql_string_functions_test.cpp | 228 +------------ .../cpp/odbc/src/config/connection_info.cpp | 7 +- 9 files changed, 659 insertions(+), 228 deletions(-) create mode 100644 modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h create mode 100644 modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp create mode 100644 modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index cc84a7f84b450..62bdec3c7e557 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -62,7 +62,9 @@ ignite_odbc_tests_SOURCES = \ src/utility_test.cpp \ src/queries_test.cpp \ src/test_utils.cpp \ + src/sql_function_test_suite_fixture.cpp \ src/sql_string_functions_test.cpp \ + src/sql_numeric_functions_test.cpp \ ../odbc/src/cursor.cpp \ ../odbc/src/config/connection_info.cpp \ ../odbc/src/app/application_data_buffer.cpp \ diff --git a/modules/platforms/cpp/odbc-test/include/Makefile.am b/modules/platforms/cpp/odbc-test/include/Makefile.am index 7b171027e9515..baca1b7d6e42b 100644 --- a/modules/platforms/cpp/odbc-test/include/Makefile.am +++ b/modules/platforms/cpp/odbc-test/include/Makefile.am @@ -20,5 +20,6 @@ ACLOCAL_AMFLAGS =-I m4 noinst_HEADERS = \ teamcity/teamcity_messages.h \ test_type.h \ - test_utils.h + test_utils.h \ + sql_function_test_suite_fixture.h diff --git a/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h b/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h new file mode 100644 index 0000000000000..c09c1ae9ff97a --- /dev/null +++ b/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h @@ -0,0 +1,138 @@ +/* + * 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. + */ + +#ifndef _IGNITE_ODBC_TEST_SQL_FUNCTION_TEST_FIXTURE +#define _IGNITE_ODBC_TEST_SQL_FUNCTION_TEST_FIXTURE + +#ifdef _WIN32 +# include +#endif + +#include +#include + +#include + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include "ignite/ignite.h" +#include "ignite/ignition.h" + +#include "test_type.h" + +namespace ignite +{ + /** + * Test setup fixture. + */ + struct SqlFunctionTestSuiteFixture + { + /** + * Constructor. + */ + SqlFunctionTestSuiteFixture(); + + /** + * Destructor. + */ + ~SqlFunctionTestSuiteFixture(); + + /** + * Run query returning single result and stores it to buffer. + * + * @param request SQL request. + * @param type Result type. + * @param column Result buffer. + * @param bufSize Result buffer size. + * @param resSize Size of received value. + */ + void CheckSingleResult0(const char* request, SQLSMALLINT type, + void* column, SQLLEN bufSize, SQLLEN* resSize) const; + + /** + * Run query returning single result and check it to be equal to expected. + * + * @param request SQL request. + * @param expected Expected result. + */ + template + void CheckSingleResult(const char* request, const T& expected) + { + BOOST_FAIL("Function is not defined for the type."); + } + + /** + * Run query returning single result and check it to be equal to expected. + * + * @param request SQL request. + * @param expected Expected result. + * @param type Result type. + */ + template + void CheckSingleResultNum0(const char* request, const T& expected, SQLSMALLINT type) + { + T res = 0; + + CheckSingleResult0(request, type, &res, 0, 0); + + BOOST_CHECK_EQUAL(res, expected); + } + + + /** Node started during the test. */ + Ignite grid; + + /** Test cache instance. */ + cache::Cache testCache; + + /** ODBC Environment. */ + SQLHENV env; + + /** ODBC Connect. */ + SQLHDBC dbc; + + /** ODBC Statement. */ + SQLHSTMT stmt; + }; + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const std::string& expected); + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int64_t& expected); + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int32_t& expected); + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int16_t& expected); + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int8_t& expected); + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const float& expected); + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const double& expected); +} + +#endif //_IGNITE_ODBC_TEST_SQL_FUNCTION_TEST_FIXTURE diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index 619b4b24cb9d8..a83ed6b0e3404 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -168,6 +168,8 @@ + + @@ -175,6 +177,7 @@ + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index 18711492ed9c0..f993ed9eb33ca 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -82,6 +82,12 @@ Code + + Code + + + Code + @@ -93,6 +99,9 @@ Code + + Code + diff --git a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp new file mode 100644 index 0000000000000..23983931503e2 --- /dev/null +++ b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp @@ -0,0 +1,188 @@ +/* + * 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. + */ + +#include "sql_function_test_suite_fixture.h" + +#include "test_utils.h" + +namespace ignite +{ + SqlFunctionTestSuiteFixture::SqlFunctionTestSuiteFixture(): + testCache(0), + env(NULL), + dbc(NULL), + stmt(NULL) + { + IgniteConfiguration cfg; + + cfg.jvmOpts.push_back("-Xdebug"); + cfg.jvmOpts.push_back("-Xnoagent"); + cfg.jvmOpts.push_back("-Djava.compiler=NONE"); + cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005"); + cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError"); + +#ifdef IGNITE_TESTS_32 + cfg.jvmInitMem = 256; + cfg.jvmMaxMem = 768; +#else + cfg.jvmInitMem = 1024; + cfg.jvmMaxMem = 4096; +#endif + + char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH"); + + BOOST_REQUIRE(cfgPath != 0) ; + + cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml"); + + IgniteError err; + + grid = Ignition::Start(cfg, &err); + + if (err.GetCode() != IgniteError::IGNITE_SUCCESS) + BOOST_FAIL(err.GetText()) ; + + testCache = grid.GetCache("cache"); + + // Allocate an environment handle + SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env); + + BOOST_REQUIRE(env != NULL) ; + + // We want ODBC 3 support + SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast(SQL_OV_ODBC3), 0); + + // Allocate a connection handle + SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc); + + BOOST_REQUIRE(dbc != NULL) ; + + // Connect string + SQLCHAR connectStr[] = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=cache"; + + SQLCHAR outstr[ODBC_BUFFER_SIZE]; + SQLSMALLINT outstrlen; + + // Connecting to ODBC server. + SQLRETURN ret = SQLDriverConnect(dbc, NULL, connectStr, static_cast(sizeof(connectStr)), + outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE); + + if (!SQL_SUCCEEDED(ret)) + { + Ignition::Stop(grid.GetName(), true); + + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc)) ; + } + + // Allocate a statement handle + SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt); + + BOOST_REQUIRE(stmt != NULL) ; + } + + SqlFunctionTestSuiteFixture::~SqlFunctionTestSuiteFixture() + { + // Releasing statement handle. + SQLFreeHandle(SQL_HANDLE_STMT, stmt); + + // Disconneting from the server. + SQLDisconnect(dbc); + + // Releasing allocated handles. + SQLFreeHandle(SQL_HANDLE_DBC, dbc); + SQLFreeHandle(SQL_HANDLE_ENV, env); + + ignite::Ignition::Stop(grid.GetName(), true); + } + + void SqlFunctionTestSuiteFixture::CheckSingleResult0(const char* request, + SQLSMALLINT type, void* column, SQLLEN bufSize, SQLLEN* resSize) const + { + SQLRETURN ret; + + ret = SQLBindCol(stmt, 1, type, column, bufSize, resSize); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ; + + ret = SQLExecDirect(stmt, reinterpret_cast(const_cast(request)), SQL_NTS); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ; + + ret = SQLFetch(stmt); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ; + + ret = SQLFetch(stmt); + BOOST_CHECK(ret == SQL_NO_DATA) ; + } + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const std::string& expected) + { + SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 }; + SQLLEN resLen = 0; + + CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen); + + BOOST_CHECK_EQUAL(std::string(reinterpret_cast(res), static_cast(resLen)), expected); + } + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int64_t& expected) + { + CheckSingleResultNum0(request, expected, SQL_C_SBIGINT); + } + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int32_t& expected) + { + CheckSingleResultNum0(request, expected, SQL_C_SLONG); + } + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int16_t& expected) + { + CheckSingleResultNum0(request, expected, SQL_C_SSHORT); + } + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int8_t& expected) + { + CheckSingleResultNum0(request, expected, SQL_C_STINYINT); + } + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const float& expected) + { + SQLFLOAT res = 0; + + CheckSingleResult0(request, SQL_C_FLOAT, &res, 0, 0); + + BOOST_CHECK_CLOSE(static_cast(res), expected, 1E-6f); + } + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const double& expected) + { + SQLDOUBLE res = 0; + + CheckSingleResult0(request, SQL_C_DOUBLE, &res, 0, 0); + + BOOST_CHECK_CLOSE(static_cast(res), expected, 1E-6); + } +} diff --git a/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp new file mode 100644 index 0000000000000..80ffd83164320 --- /dev/null +++ b/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp @@ -0,0 +1,309 @@ +/* + * 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. + */ + +#define _USE_MATH_DEFINES + +#ifdef _WIN32 +# include +#endif + +#include +#include + +#include + +#include +#include + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include "ignite/ignite.h" +#include "ignite/ignition.h" +#include "ignite/impl/binary/binary_utils.h" + +#include "test_type.h" +#include "test_utils.h" +#include "sql_function_test_suite_fixture.h" + +using namespace ignite; +using namespace ignite::cache; +using namespace ignite::cache::query; +using namespace ignite::common; + +using namespace boost::unit_test; + +using ignite::impl::binary::BinaryUtils; + +BOOST_FIXTURE_TEST_SUITE(SqlNumericFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(TestNumericFunctionAbs) +{ + TestType in; + + in.i32Field = -42; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn ABS(i32Field)} FROM TestType", std::abs(in.i32Field)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionAcos) +{ + TestType in; + + in.doubleField = 0.32; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn ACOS(doubleField)} FROM TestType", std::acos(in.doubleField)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionAsin) +{ + TestType in; + + in.doubleField = 0.12; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn ASIN(doubleField)} FROM TestType", std::asin(in.doubleField)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionAtan) +{ + TestType in; + + in.doubleField = 0.14; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn ATAN(doubleField)} FROM TestType", std::atan(in.doubleField)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionAtan2) +{ + TestType in; + + in.doubleField = 0.24; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn ATAN2(doubleField, 0.2)} FROM TestType", std::atan2(in.doubleField, 0.2)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionCeiling) +{ + TestType in; + + in.doubleField = 7.31; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn CEILING(doubleField)} FROM TestType", std::ceil(in.doubleField)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionCos) +{ + TestType in; + + in.doubleField = 2.31; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn COS(doubleField)} FROM TestType", std::cos(in.doubleField)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionCot) +{ + TestType in; + + in.doubleField = 2.31; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn COT(doubleField)} FROM TestType", 1 / std::tan(in.doubleField)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionDegrees) +{ + TestType in; + + in.doubleField = 2.31; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn DEGREES(doubleField)} FROM TestType", in.doubleField * M_1_PI * 180); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionExp) +{ + TestType in; + + in.doubleField = 1.23; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn EXP(doubleField)} FROM TestType", std::exp(in.doubleField)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionFloor) +{ + TestType in; + + in.doubleField = 5.29; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn FLOOR(doubleField)} FROM TestType", std::floor(in.doubleField)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionLog) +{ + TestType in; + + in.doubleField = 15.3; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn LOG(doubleField)} FROM TestType", std::log(in.doubleField)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionLog10) +{ + TestType in; + + in.doubleField = 15.3; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn LOG10(doubleField)} FROM TestType", std::log10(in.doubleField)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionMod) +{ + TestType in; + + in.i64Field = 26; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn MOD(i64Field, 3)} FROM TestType", in.i64Field % 3); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionPi) +{ + CheckSingleResult("SELECT {fn PI()}", M_PI); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionPower) +{ + TestType in; + + in.doubleField = 1.81; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn POWER(doubleField, 2.5)} FROM TestType", std::pow(in.doubleField, 2.5)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionRadians) +{ + TestType in; + + in.doubleField = 161; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn RADIANS(doubleField)} FROM TestType", in.doubleField * M_PI / 180.0); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionRand) +{ + CheckSingleResult("SELECT {fn RAND()} * 0", 0); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionRound) +{ + TestType in; + + in.doubleField = 5.29; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn ROUND(doubleField)} FROM TestType", std::floor(in.doubleField)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionSign) +{ + TestType in; + + in.doubleField = -1.39; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn SIGN(doubleField)} FROM TestType", in.doubleField < 0 ? -1 : in.doubleField == 0 ? 0 : 1); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionSin) +{ + TestType in; + + in.doubleField = 1.01; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn SIN(doubleField)} FROM TestType", std::sin(in.doubleField)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionSqrt) +{ + TestType in; + + in.doubleField = 2.56; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn SQRT(doubleField)} FROM TestType", std::sqrt(in.doubleField)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionTan) +{ + TestType in; + + in.doubleField = 0.56; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn TAN(doubleField)} FROM TestType", std::tan(in.doubleField)); +} + +BOOST_AUTO_TEST_CASE(TestNumericFunctionTruncate) +{ + TestType in; + + in.doubleField = 4.17133; + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn TRUNCATE(doubleField, 3)} FROM TestType", 4.171); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp index e84aba64e173a..d22bb834ca2d4 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp @@ -15,8 +15,6 @@ * limitations under the License. */ -#define _USE_MATH_DEFINES - #ifdef _WIN32 # include #endif @@ -24,8 +22,6 @@ #include #include -#include - #include #include @@ -41,6 +37,7 @@ #include "test_type.h" #include "test_utils.h" +#include "sql_function_test_suite_fixture.h" using namespace ignite; using namespace ignite::cache; @@ -51,228 +48,7 @@ using namespace boost::unit_test; using ignite::impl::binary::BinaryUtils; -/** - * Test setup fixture. - */ -struct SqlStringFunctionTestSuiteFixture -{ - /** - * Constructor. - */ - SqlStringFunctionTestSuiteFixture() : - testCache(0), - env(NULL), - dbc(NULL), - stmt(NULL) - { - IgniteConfiguration cfg; - - cfg.jvmOpts.push_back("-Xdebug"); - cfg.jvmOpts.push_back("-Xnoagent"); - cfg.jvmOpts.push_back("-Djava.compiler=NONE"); - cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005"); - cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError"); - -#ifdef IGNITE_TESTS_32 - cfg.jvmInitMem = 256; - cfg.jvmMaxMem = 768; -#else - cfg.jvmInitMem = 1024; - cfg.jvmMaxMem = 4096; -#endif - - char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH"); - - BOOST_REQUIRE(cfgPath != 0); - - cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml"); - - IgniteError err; - - grid = Ignition::Start(cfg, &err); - - if (err.GetCode() != IgniteError::IGNITE_SUCCESS) - BOOST_FAIL(err.GetText()); - - testCache = grid.GetCache("cache"); - - // Allocate an environment handle - SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env); - - BOOST_REQUIRE(env != NULL); - - // We want ODBC 3 support - SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast(SQL_OV_ODBC3), 0); - - // Allocate a connection handle - SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc); - - BOOST_REQUIRE(dbc != NULL); - - // Connect string - SQLCHAR connectStr[] = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=cache"; - - SQLCHAR outstr[ODBC_BUFFER_SIZE]; - SQLSMALLINT outstrlen; - - // Connecting to ODBC server. - SQLRETURN ret = SQLDriverConnect(dbc, NULL, connectStr, static_cast(sizeof(connectStr)), - outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE); - - if (!SQL_SUCCEEDED(ret)) - { - Ignition::Stop(grid.GetName(), true); - - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc)); - } - - // Allocate a statement handle - SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt); - - BOOST_REQUIRE(stmt != NULL); - } - - /** - * Destructor. - */ - ~SqlStringFunctionTestSuiteFixture() - { - // Releasing statement handle. - SQLFreeHandle(SQL_HANDLE_STMT, stmt); - - // Disconneting from the server. - SQLDisconnect(dbc); - - // Releasing allocated handles. - SQLFreeHandle(SQL_HANDLE_DBC, dbc); - SQLFreeHandle(SQL_HANDLE_ENV, env); - - Ignition::Stop(grid.GetName(), true); - } - - void CheckSingleResult0(const char* request, SQLSMALLINT type, void* column, SQLLEN bufSize, SQLLEN* resSize) - { - SQLRETURN ret; - - ret = SQLBindCol(stmt, 1, type, column, bufSize, resSize); - - if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); - - ret = SQLExecDirect(stmt, reinterpret_cast(const_cast(request)), SQL_NTS); - if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); - - ret = SQLFetch(stmt); - if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); - - ret = SQLFetch(stmt); - BOOST_CHECK(ret == SQL_NO_DATA); - } - - /** - * Run query returning single result and check it to be equal to expected. - * - * @param request SQL request. - * @param expected Expected result. - */ - template - void CheckSingleResult(const char* request, const T& expected) - { - BOOST_FAIL("Function is not defined for the type."); - } - - /** - * Run query returning single result and check it to be equal to expected. - * - * @param request SQL request. - * @param expected Expected result. - * @param type Result type. - */ - template - void CheckSingleResultNum0(const char* request, const T& expected, SQLSMALLINT type) - { - T res = 0; - - CheckSingleResult0(request, type, &res, 0, 0); - - BOOST_CHECK_EQUAL(res, expected); - } - - - /** Node started during the test. */ - Ignite grid; - - /** Test cache instance. */ - Cache testCache; - - /** ODBC Environment. */ - SQLHENV env; - - /** ODBC Connect. */ - SQLHDBC dbc; - - /** ODBC Statement. */ - SQLHSTMT stmt; -}; - -template<> -void SqlStringFunctionTestSuiteFixture::CheckSingleResult(const char* request, const std::string& expected) -{ - SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 }; - SQLLEN resLen = 0; - - CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen); - - BOOST_CHECK_EQUAL(std::string(reinterpret_cast(res), static_cast(resLen)), expected); -} - -template<> -void SqlStringFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int64_t& expected) -{ - CheckSingleResultNum0(request, expected, SQL_C_SBIGINT); -} - -template<> -void SqlStringFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int32_t& expected) -{ - CheckSingleResultNum0(request, expected, SQL_C_SLONG); -} - -template<> -void SqlStringFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int16_t& expected) -{ - CheckSingleResultNum0(request, expected, SQL_C_SSHORT); -} - -template<> -void SqlStringFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int8_t& expected) -{ - CheckSingleResultNum0(request, expected, SQL_C_STINYINT); -} - -template<> -void SqlStringFunctionTestSuiteFixture::CheckSingleResult(const char* request, const float& expected) -{ - SQLFLOAT res = 0; - - CheckSingleResult0(request, SQL_C_FLOAT, &res, 0, 0); - - BOOST_CHECK_CLOSE(static_cast(res), expected, 1E-6f); -} - -template<> -void SqlStringFunctionTestSuiteFixture::CheckSingleResult(const char* request, const double& expected) -{ - SQLDOUBLE res = 0; - - CheckSingleResult0(request, SQL_C_DOUBLE, &res, 0, 0); - - BOOST_CHECK_CLOSE(static_cast(res), expected, 1E-6); -} - -BOOST_FIXTURE_TEST_SUITE(SqlStringFunctionTestSuite, SqlStringFunctionTestSuiteFixture) +BOOST_FIXTURE_TEST_SUITE(SqlStringFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture) BOOST_AUTO_TEST_CASE(TestStringFunctionAscii) { diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp index f9a3f96a6fb19..1fed7c556973e 100644 --- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp +++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp @@ -233,7 +233,12 @@ namespace ignite #ifdef SQL_NUMERIC_FUNCTIONS // Bitmask enumerating the scalar numeric functions supported by // the driver and associated data source. - intParams[SQL_NUMERIC_FUNCTIONS] = SQL_FN_NUM_ABS; + intParams[SQL_NUMERIC_FUNCTIONS] = SQL_FN_NUM_ABS | SQL_FN_NUM_ACOS | SQL_FN_NUM_ASIN | + SQL_FN_NUM_ATAN | SQL_FN_NUM_ATAN2 | SQL_FN_NUM_CEILING | SQL_FN_NUM_COS | SQL_FN_NUM_COT | + SQL_FN_NUM_EXP | SQL_FN_NUM_FLOOR | SQL_FN_NUM_LOG | SQL_FN_NUM_MOD | SQL_FN_NUM_SIGN | + SQL_FN_NUM_SIN | SQL_FN_NUM_SQRT | SQL_FN_NUM_TAN | SQL_FN_NUM_PI | SQL_FN_NUM_RAND | + SQL_FN_NUM_DEGREES | SQL_FN_NUM_LOG10 | SQL_FN_NUM_POWER | SQL_FN_NUM_RADIANS | SQL_FN_NUM_ROUND | + SQL_FN_NUM_TRUNCATE; #endif // SQL_NUMERIC_FUNCTIONS #ifdef SQL_STRING_FUNCTIONS From 0e3a6e2df8b42f255a5a4688d5827dccaabfd3a4 Mon Sep 17 00:00:00 2001 From: isapego Date: Thu, 25 Aug 2016 14:34:31 +0300 Subject: [PATCH 059/487] IGNITE-3757: ODBC: Added aggregate functions support. This closes #983. --- modules/platforms/cpp/odbc-test/Makefile.am | 1 + .../odbc-test/project/vs/odbc-test.vcxproj | 1 + .../project/vs/odbc-test.vcxproj.filters | 3 + .../src/sql_aggregate_functions_test.cpp | 249 ++++++++++++++++++ .../cpp/odbc/src/config/connection_info.cpp | 5 +- 5 files changed, 256 insertions(+), 3 deletions(-) create mode 100644 modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index 62bdec3c7e557..6288637e9c868 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -65,6 +65,7 @@ ignite_odbc_tests_SOURCES = \ src/sql_function_test_suite_fixture.cpp \ src/sql_string_functions_test.cpp \ src/sql_numeric_functions_test.cpp \ + src/sql_aggregate_functions_test.cpp \ ../odbc/src/cursor.cpp \ ../odbc/src/config/connection_info.cpp \ ../odbc/src/app/application_data_buffer.cpp \ diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index a83ed6b0e3404..e0b786a86ed36 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -168,6 +168,7 @@ + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index f993ed9eb33ca..763a36483cd77 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -88,6 +88,9 @@ Code + + Code + diff --git a/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp new file mode 100644 index 0000000000000..6254913976d99 --- /dev/null +++ b/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp @@ -0,0 +1,249 @@ +/* + * 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. + */ + +#define _USE_MATH_DEFINES + +#ifdef _WIN32 +# include +#endif + +#include +#include + +#include + +#include +#include + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include "ignite/ignite.h" +#include "ignite/ignition.h" +#include "ignite/impl/binary/binary_utils.h" + +#include "test_type.h" +#include "test_utils.h" +#include "sql_function_test_suite_fixture.h" + +using namespace ignite; +using namespace ignite::cache; +using namespace ignite::cache::query; +using namespace ignite::common; + +using namespace boost::unit_test; + +using ignite::impl::binary::BinaryUtils; + +BOOST_FIXTURE_TEST_SUITE(SqlAggregateFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(TestAggregateFunctionAvgInt) +{ + std::vector in(3); + + in[0].i32Field = 43; + in[1].i32Field = 311; + in[2].i32Field = 7; + + int32_t avg = 0; + + for (int32_t i = 0; i < static_cast(in.size()); ++i) + { + testCache.Put(i, in[i]); + + avg += in[i].i32Field; + } + + avg /= static_cast(in.size()); + + CheckSingleResult("SELECT {fn AVG(i32Field)} FROM TestType", avg); +} + +BOOST_AUTO_TEST_CASE(TestAggregateFunctionAvgIntDistinct) +{ + std::vector in(3); + + in[0].i32Field = 43; + in[1].i32Field = 311; + in[2].i32Field = 7; + + int32_t avg = 0; + + for (int32_t i = 0; i < static_cast(in.size()); ++i) + { + testCache.Put(i, in[i]); + + avg += in[i].i32Field; + } + + avg /= static_cast(in.size()); + + testCache.Put(in.size() + 10, in[0]); + + CheckSingleResult("SELECT {fn AVG(DISTINCT i32Field)} FROM TestType", avg); +} + +BOOST_AUTO_TEST_CASE(TestAggregateFunctionAvgFloat) +{ + std::vector in(3); + + in[0].floatField = 43.0; + in[1].floatField = 311.0; + in[2].floatField = 7.0; + + float avg = 0; + + for (int32_t i = 0; i < static_cast(in.size()); ++i) + { + testCache.Put(i, in[i]); + + avg += in[i].i32Field; + } + + avg /= in.size(); + + CheckSingleResult("SELECT {fn AVG(floatField)} FROM TestType", avg); +} + +BOOST_AUTO_TEST_CASE(TestAggregateFunctionAvgFloatDistinct) +{ + std::vector in(3); + + in[0].floatField = 43.0; + in[1].floatField = 311.0; + in[2].floatField = 7.0; + + float avg = 0; + + for (int32_t i = 0; i < static_cast(in.size()); ++i) + { + testCache.Put(i, in[i]); + + avg += in[i].i32Field; + } + + avg /= in.size(); + + testCache.Put(in.size() + 10, in[0]); + + CheckSingleResult("SELECT {fn AVG(DISTINCT floatField)} FROM TestType", avg); +} + +BOOST_AUTO_TEST_CASE(TestAggregateFunctionCount) +{ + std::vector in(8); + + for (int32_t i = 0; i < static_cast(in.size()); ++i) + testCache.Put(i, in[i]); + + CheckSingleResult("SELECT {fn COUNT(*)} FROM TestType", in.size()); +} + +BOOST_AUTO_TEST_CASE(TestAggregateFunctionCountDistinct) +{ + std::vector in(8); + + for (int32_t i = 0; i < static_cast(in.size()); ++i) + { + in[i].i32Field = i; + + testCache.Put(i, in[i]); + } + + testCache.Put(in.size() + 10, in[0]); + + CheckSingleResult("SELECT {fn COUNT(DISTINCT i32Field)} FROM TestType", in.size()); +} + +BOOST_AUTO_TEST_CASE(TestAggregateFunctionMax) +{ + std::vector in(4); + + in[0].i32Field = 121; + in[1].i32Field = 17; + in[2].i32Field = 314041; + in[3].i32Field = 9410; + + for (int32_t i = 0; i < static_cast(in.size()); ++i) + testCache.Put(i, in[i]); + + CheckSingleResult("SELECT {fn MAX(i32Field)} FROM TestType", in[2].i32Field); +} + +BOOST_AUTO_TEST_CASE(TestAggregateFunctionMin) +{ + std::vector in(4); + + in[0].i32Field = 121; + in[1].i32Field = 17; + in[2].i32Field = 314041; + in[3].i32Field = 9410; + + for (int32_t i = 0; i < static_cast(in.size()); ++i) + testCache.Put(i, in[i]); + + CheckSingleResult("SELECT {fn MIN(i32Field)} FROM TestType", in[1].i32Field); +} + +BOOST_AUTO_TEST_CASE(TestAggregateFunctionSum) +{ + std::vector in(4); + + in[0].i32Field = 121; + in[1].i32Field = 17; + in[2].i32Field = 314041; + in[3].i32Field = 9410; + + int64_t sum = 0; + + for (int32_t i = 0; i < static_cast(in.size()); ++i) + { + testCache.Put(i, in[i]); + + sum += in[i].i32Field; + } + + CheckSingleResult("SELECT {fn SUM(i32Field)} FROM TestType", sum); +} + +BOOST_AUTO_TEST_CASE(TestAggregateFunctionSumDistinct) +{ + std::vector in(4); + + in[0].i32Field = 121; + in[1].i32Field = 17; + in[2].i32Field = 314041; + in[3].i32Field = 9410; + + int64_t sum = 0; + + for (int32_t i = 0; i < static_cast(in.size()); ++i) + { + testCache.Put(i, in[i]); + + sum += in[i].i32Field; + } + + testCache.Put(in.size() + 10, in[0]); + + CheckSingleResult("SELECT {fn SUM(DISTINCT i32Field)} FROM TestType", sum); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp index 1fed7c556973e..b2480c2fdcef9 100644 --- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp +++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp @@ -225,9 +225,8 @@ namespace ignite #ifdef SQL_AGGREGATE_FUNCTIONS // Bitmask enumerating support for aggregation functions. - intParams[SQL_AGGREGATE_FUNCTIONS] = SQL_AF_ALL | SQL_AF_AVG | - SQL_AF_COUNT | SQL_AF_DISTINCT | SQL_AF_MAX | SQL_AF_MIN | - SQL_AF_SUM; + intParams[SQL_AGGREGATE_FUNCTIONS] = SQL_AF_AVG | SQL_AF_COUNT | + SQL_AF_DISTINCT | SQL_AF_MAX | SQL_AF_MIN | SQL_AF_SUM; #endif // SQL_AGGREGATE_FUNCTIONS #ifdef SQL_NUMERIC_FUNCTIONS From e21111f287039011bc9437c94fb574e61e2ac226 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 25 Aug 2016 16:26:02 +0300 Subject: [PATCH 060/487] IGNITE-3738: ODBC: Fixed escape sequence whitespaces handling. This closes #982. --- .../odbc/escape/OdbcEscapeToken.java | 61 +++++++++++ .../odbc/escape/OdbcEscapeType.java | 81 +++++++++++++- .../odbc/escape/OdbcEscapeUtils.java | 103 ++++++++++++------ .../odbc/OdbcEscapeSequenceSelfTest.java | 15 ++- 4 files changed, 225 insertions(+), 35 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeToken.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeToken.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeToken.java new file mode 100644 index 0000000000000..6bb4f814872a9 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeToken.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.odbc.escape; + +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * ODBC escape sequence token. + */ +public class OdbcEscapeToken { + /** Escape sequence type. */ + private final OdbcEscapeType type; + + /** Token length. */ + private final int len; + + /** + * Constructor. + * + * @param type Escape sequence type. + * @param len Token length. + */ + public OdbcEscapeToken(OdbcEscapeType type, int len) { + this.type = type; + this.len = len; + } + + /** + * @return Escape sequence type. + */ + public OdbcEscapeType type() { + return type; + } + + /** + * @return Token length. + */ + public int length() { + return len; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(OdbcEscapeToken.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java index 2df413f03761b..96a2127b1c1c7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java @@ -22,5 +22,84 @@ */ public enum OdbcEscapeType { /** Scalar function. */ - FN + SCALAR_FUNCTION("fn", true, false), + + /** Outer join. */ + OUTER_JOIN("oj", true, false), + + /** Date. */ + DATE("d", true, false), + + /** Timestamp. */ + TIMESTAMP("ts", true, false), + + /** Time. */ + TIME("t", true, false), + + /** GUID. */ + GUID("guid", true, false), + + /** LIKE clause. */ + LIKE("\'", false, true); + + /** Values in convenient order. */ + private static final OdbcEscapeType[] VALS = new OdbcEscapeType[] { + SCALAR_FUNCTION, // Assume that scalar functions are very frequent. + DATE, TIMESTAMP, // Date and timestamp are relatively frequent as well; also TS must go before T. + OUTER_JOIN, // Joins are less frequent, + LIKE, TIME, GUID // LIKE, TIME and GUID are even less frequent. + }; + + /** + * Get values in convenient order, where the most frequent values goes first, and "startsWith" invocation is + * enough to get type (i.e. "ts" goes before "t"). + * + * @return Values. + */ + public static OdbcEscapeType[] sortedValues() { + return VALS; + } + + /** Escape sequence body. */ + private final String body; + + /** Whether token must be delimited from the rest of escape sequence. */ + private final boolean delimited; + + /** Whether empty escape sequence is allowed. */ + private final boolean allowEmpty; + + /** + * Constructor. + * + * @param body Escape sequence body. + * @param delimited Whether token must be delimited from the rest of escape sequence. + * @param allowEmpty Whether empty escape sequence is allowed. + */ + OdbcEscapeType(String body, boolean delimited, boolean allowEmpty) { + this.body = body; + this.delimited = delimited; + this.allowEmpty = allowEmpty; + } + + /** + * @return Escape sequence body. + */ + public String body() { + return body; + } + + /** + * @return Whether token must be delimited from the rest of escape sequence. + */ + public boolean delimited() { + return delimited; + } + + /** + * @return Whether empty escape sequence is allowed. + */ + public boolean allowEmpty() { + return allowEmpty; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java index 4d8ca6983bceb..6299c7efede2e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java @@ -90,18 +90,18 @@ else if (curChar == '}') { if (nested == null) // Found sequence without nesting, process it. - parseRes = parseExpression(text, openPos, curPos - openPos); + parseRes = parseExpression(text, openPos, curPos + 1 - openPos); else { // Special case to process nesting. String res0 = appendNested(text, openPos, curPos + 1, nested); nested = null; - parseRes = parseExpression(res0, 0, res0.length()-1); + parseRes = parseExpression(res0, 0, res0.length()); } if (earlyExit) - return new OdbcEscapeParseResult(startPos, curPos - startPos + 1, parseRes); + return new OdbcEscapeParseResult(startPos, curPos + 1 - startPos, parseRes); else res.append(parseRes); @@ -137,23 +137,21 @@ private static String parseExpression(String text, int startPos, int len) { char firstChar = text.charAt(startPos); if (firstChar == '{') { - char lastChar = text.charAt(startPos + len); + char lastChar = text.charAt(startPos + len - 1); if (lastChar != '}') throw new IgniteException("Failed to parse escape sequence because it is not enclosed: " + substring(text, startPos, len)); - OdbcEscapeType typ = sequenceType(text, startPos, len); + OdbcEscapeToken token = parseToken(text, startPos, len); - switch (typ) { - case FN: - return parseScalarExpression(text, startPos, len); + switch (token.type()) { + case SCALAR_FUNCTION: + return parseScalarExpression(text, startPos, len, token); - default: { - assert false : "Unknown expression type: " + typ; - - return null; - } + default: + throw new IgniteException("Unsupported escape sequence token [text=" + + substring(text, startPos, len) + ", token=" + token.type().body() + ']'); } } else { @@ -161,8 +159,60 @@ private static String parseExpression(String text, int startPos, int len) { if (startPos == 0 || text.length() == len) return text; else - return text.substring(startPos, startPos + len); + return substring(text, startPos, len); + } + } + + /** + * Get escape sequence info. + * + * @param text Text. + * @param startPos Start position. + * @return Escape sequence info. + */ + private static OdbcEscapeToken parseToken(String text, int startPos, int len) { + assert validSubstring(text, startPos, len); + assert text.charAt(startPos) == '{'; + + int pos = startPos + 1; + + while (Character.isWhitespace(text.charAt(pos))) + pos++; + + OdbcEscapeType curTyp = null; + boolean empty = false; + + for (OdbcEscapeType typ : OdbcEscapeType.sortedValues()) { + if (text.startsWith(typ.body(), pos)) { + pos += typ.body().length(); + + if (typ == OdbcEscapeType.LIKE) + throw new IgniteException("LIKE escape sequence is not supported yet."); + else { + empty = (startPos + len == pos + 1); + + if (!empty && typ.delimited()) { + char charAfter = text.charAt(pos); + + if (!Character.isWhitespace(charAfter)) + throw new IgniteException("Unexpected escape sequence token: " + + substring(text, startPos, len)); + } + } + + curTyp = typ; + + break; + } } + + if (curTyp == null) + throw new IgniteException("Unsupported escape sequence: " + substring(text, startPos, len)); + + if (empty && !curTyp.allowEmpty()) + throw new IgniteException("Escape sequence cannot be empty: " + substring(text, startPos, len)); + + return new OdbcEscapeToken(curTyp, pos - (startPos + 1)); } /** @@ -171,12 +221,16 @@ private static String parseExpression(String text, int startPos, int len) { * @param text Text. * @param startPos Start position. * @param len Length. + * @param token Token. * @return Parsed expression. */ - private static String parseScalarExpression(String text, int startPos, int len) { + private static String parseScalarExpression(String text, int startPos, int len, OdbcEscapeToken token) { assert validSubstring(text, startPos, len); - return substring(text, startPos + 3, len - 3).trim(); + int startPos0 = startPos + 1 /* open brace */ + token.length() /* token. */; + int len0 = len - 1 /* open brace */ - token.length() /* token */ - 1 /* close brace */; + + return substring(text, startPos0, len0).trim(); } /** @@ -211,23 +265,6 @@ private static String appendNested(String text, int startPos, int endPos, return res.toString(); } - /** - * Get escape sequence type. - * - * @param text Text. - * @param startPos Start position. - * @return Escape sequence type. - */ - private static OdbcEscapeType sequenceType(String text, int startPos, int len) { - assert validSubstring(text, startPos, len); - assert text.charAt(startPos) == '{'; - - if (text.startsWith("fn", startPos + 1)) - return OdbcEscapeType.FN; - - throw new IgniteException("Unsupported escape sequence: " + text.substring(startPos, startPos + len)); - } - /** * Perform "substring" using start position and length. * diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java index 73fa0f4c57bb9..d9be6cc703c19 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java @@ -141,7 +141,7 @@ public void testNestedFunctionMixed() { /** * Test non-closed escape sequence. */ - public void testFailedOnInvalidSequence1() { + public void testFailedOnNonClosedEscapeSequence() { checkFail("select {fn func1(field1, {fn func2(field2), field3)} from SomeTable;"); } @@ -152,6 +152,19 @@ public void testFailedOnClosingNotOpenedSequence() { checkFail("select {fn func1(field1, func2(field2)}, field3)} from SomeTable;"); } + /** + * Test escape sequences with additional whitespace characters + */ + public void testFunctionEscapeSequenceWithWhitespaces() throws Exception { + check("func1()", "{ fn func1()}"); + + check("func1()", "{ fn func1()}"); + + check("func1()", "{ \n fn func1()}"); + + checkFail("{ \n func1()}"); + } + /** * Check parsing logic. * From 8aabd6ea65d883d3bbcf37c05c146105dff8a6e2 Mon Sep 17 00:00:00 2001 From: isapego Date: Thu, 25 Aug 2016 16:30:20 +0300 Subject: [PATCH 061/487] IGNITE-3751: ODBC: Added system functions support. This closes #985. --- modules/platforms/cpp/odbc-test/Makefile.am | 1 + .../include/sql_function_test_suite_fixture.h | 49 ++++++++++++- .../odbc-test/project/vs/odbc-test.vcxproj | 1 + .../project/vs/odbc-test.vcxproj.filters | 3 + .../src/sql_function_test_suite_fixture.cpp | 49 +++++++++++++ .../src/sql_system_functions_test.cpp | 68 +++++++++++++++++++ .../cpp/odbc/src/config/connection_info.cpp | 2 +- 7 files changed, 171 insertions(+), 2 deletions(-) create mode 100644 modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index 6288637e9c868..1e00b6f3547f3 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -66,6 +66,7 @@ ignite_odbc_tests_SOURCES = \ src/sql_string_functions_test.cpp \ src/sql_numeric_functions_test.cpp \ src/sql_aggregate_functions_test.cpp \ + src/sql_system_functions_test.cpp \ ../odbc/src/cursor.cpp \ ../odbc/src/config/connection_info.cpp \ ../odbc/src/app/application_data_buffer.cpp \ diff --git a/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h b/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h index c09c1ae9ff97a..eef0a9b3b0b9d 100644 --- a/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h +++ b/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h @@ -78,7 +78,18 @@ namespace ignite { BOOST_FAIL("Function is not defined for the type."); } - + + /** + * Run query returning single result and check it to be successful. + * + * @param request SQL request. + */ + template + void CheckSingleResult(const char* request) + { + BOOST_FAIL("Function is not defined for the type."); + } + /** * Run query returning single result and check it to be equal to expected. * @@ -96,6 +107,21 @@ namespace ignite BOOST_CHECK_EQUAL(res, expected); } + /** + * Run query returning single result and check it to be equal to expected. + * + * @param request SQL request. + * @param expected Expected result. + * @param type Result type. + */ + template + void CheckSingleResultNum0(const char* request, SQLSMALLINT type) + { + T res = 0; + + CheckSingleResult0(request, type, &res, 0, 0); + } + /** Node started during the test. */ Ignite grid; @@ -133,6 +159,27 @@ namespace ignite template<> void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const double& expected); + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); } #endif //_IGNITE_ODBC_TEST_SQL_FUNCTION_TEST_FIXTURE diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index e0b786a86ed36..2a94a2a541738 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -172,6 +172,7 @@ + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index 763a36483cd77..9ca51af62b450 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -91,6 +91,9 @@ Code + + Code + diff --git a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp index 23983931503e2..f848dc5572efe 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp @@ -185,4 +185,53 @@ namespace ignite BOOST_CHECK_CLOSE(static_cast(res), expected, 1E-6); } + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) + { + SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 }; + SQLLEN resLen = 0; + + CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen); + } + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) + { + CheckSingleResultNum0(request, SQL_C_SBIGINT); + } + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) + { + CheckSingleResultNum0(request, SQL_C_SLONG); + } + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) + { + CheckSingleResultNum0(request, SQL_C_SSHORT); + } + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) + { + CheckSingleResultNum0(request, SQL_C_STINYINT); + } + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) + { + SQLFLOAT res = 0; + + CheckSingleResult0(request, SQL_C_FLOAT, &res, 0, 0); + } + + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) + { + SQLDOUBLE res = 0; + + CheckSingleResult0(request, SQL_C_DOUBLE, &res, 0, 0); + } } diff --git a/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp new file mode 100644 index 0000000000000..d369b0f0c78e8 --- /dev/null +++ b/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp @@ -0,0 +1,68 @@ +/* + * 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. + */ + +#ifdef _WIN32 +# include +#endif + +#include +#include + +#include +#include + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include "ignite/ignite.h" +#include "ignite/ignition.h" +#include "ignite/impl/binary/binary_utils.h" + +#include "test_type.h" +#include "test_utils.h" +#include "sql_function_test_suite_fixture.h" + +using namespace ignite; +using namespace ignite::cache; +using namespace ignite::cache::query; +using namespace ignite::common; + +using namespace boost::unit_test; + +using ignite::impl::binary::BinaryUtils; + +BOOST_FIXTURE_TEST_SUITE(SqlSystemFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(TestSystemFunctionDatabase) +{ + CheckSingleResult("SELECT {fn DATABASE()}"); +} + +BOOST_AUTO_TEST_CASE(TestSystemFunctionUser) +{ + CheckSingleResult("SELECT {fn USER()}"); +} + +BOOST_AUTO_TEST_CASE(TestSystemFunctionIfnull) +{ + CheckSingleResult("SELECT {fn IFNULL(NULL, 42)}", 42); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp index b2480c2fdcef9..cff48cf5275e1 100644 --- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp +++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp @@ -286,7 +286,7 @@ namespace ignite #ifdef SQL_SYSTEM_FUNCTIONS // Bitmask enumerating the scalar system functions supported by the // driver and associated data source. - intParams[SQL_SYSTEM_FUNCTIONS] = 0; + intParams[SQL_SYSTEM_FUNCTIONS] = SQL_FN_SYS_USERNAME | SQL_FN_SYS_DBNAME | SQL_FN_SYS_IFNULL; #endif // SQL_SYSTEM_FUNCTIONS #ifdef SQL_CONVERT_FUNCTIONS From 60afa372906246d43456518e5f76a454ca2bcadf Mon Sep 17 00:00:00 2001 From: isapego Date: Thu, 25 Aug 2016 16:37:43 +0300 Subject: [PATCH 062/487] IGNITE-3390: ODBC: Added DSN configuration dialog for Windows. This closes #881. --- .../cpp/common/include/ignite/common/utils.h | 7 + .../cpp/common/os/win/src/common/utils.cpp | 20 ++ .../cpp/core/include/ignite/ignition.h | 2 +- modules/platforms/cpp/core/src/ignition.cpp | 8 +- .../cpp/odbc-test/src/configuration_test.cpp | 6 +- modules/platforms/cpp/odbc/Makefile.am | 1 + .../platforms/cpp/odbc/include/Makefile.am | 1 + .../platforms/cpp/odbc/include/ignite/odbc.h | 1 - .../ignite/odbc/config/configuration.h | 93 +++++- .../odbc/diagnostic/diagnostic_record.h | 2 +- .../cpp/odbc/include/ignite/odbc/dsn_config.h | 61 ++++ .../include/ignite/odbc/protocol_version.h | 20 +- .../ignite/odbc/system/odbc_constants.h | 4 - .../odbc/system/ui/dsn_configuration_window.h | 152 +++++++++ .../ignite/odbc/system/ui/custom_window.h | 189 +++++++++++ .../include/ignite/odbc/system/ui/window.h | 201 ++++++++++++ .../os/win/src/system/ui/custom_window.cpp | 184 +++++++++++ .../system/ui/dsn_configuration_window.cpp | 305 ++++++++++++++++++ .../cpp/odbc/os/win/src/system/ui/window.cpp | 192 +++++++++++ .../cpp/odbc/os/win/src/system_dsn.cpp | 218 +++++++++++++ .../cpp/odbc/project/vs/odbc.vcxproj | 25 +- .../cpp/odbc/project/vs/odbc.vcxproj.filters | 30 ++ .../cpp/odbc/src/config/configuration.cpp | 17 +- modules/platforms/cpp/odbc/src/connection.cpp | 2 +- .../odbc/src/diagnostic/diagnostic_record.cpp | 2 +- .../diagnostic/diagnostic_record_storage.cpp | 2 +- modules/platforms/cpp/odbc/src/dsn_config.cpp | 115 +++++++ .../platforms/cpp/odbc/src/entry_points.cpp | 8 - modules/platforms/cpp/odbc/src/odbc.cpp | 88 ++--- .../cpp/odbc/src/protocol_version.cpp | 23 +- 30 files changed, 1868 insertions(+), 111 deletions(-) create mode 100644 modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h create mode 100644 modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h create mode 100644 modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/custom_window.h create mode 100644 modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/window.h create mode 100644 modules/platforms/cpp/odbc/os/win/src/system/ui/custom_window.cpp create mode 100644 modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp create mode 100644 modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp create mode 100644 modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp create mode 100644 modules/platforms/cpp/odbc/src/dsn_config.cpp diff --git a/modules/platforms/cpp/common/include/ignite/common/utils.h b/modules/platforms/cpp/common/include/ignite/common/utils.h index 331fcb21d1c47..f4d2a9fcaae08 100644 --- a/modules/platforms/cpp/common/include/ignite/common/utils.h +++ b/modules/platforms/cpp/common/include/ignite/common/utils.h @@ -59,6 +59,13 @@ namespace ignite return res; } + /** + * Strips leading and trailing whitespaces from string. + * + * @param str String to be transformed. + */ + IGNITE_IMPORT_EXPORT void StripSurroundingWhitespaces(std::string& str); + /** * Get string representation of long in decimal form. * diff --git a/modules/platforms/cpp/common/os/win/src/common/utils.cpp b/modules/platforms/cpp/common/os/win/src/common/utils.cpp index 47d7f4304169a..77c90b88efb38 100644 --- a/modules/platforms/cpp/common/os/win/src/common/utils.cpp +++ b/modules/platforms/cpp/common/os/win/src/common/utils.cpp @@ -40,6 +40,26 @@ namespace ignite return false; } + void StripSurroundingWhitespaces(std::string& str) + { + std::string::size_type newBegin = 0; + while (newBegin < str.size() && ::isspace(str[newBegin])) + ++newBegin; + + if (newBegin == str.size()) + { + str.clear(); + + return; + } + + std::string::size_type newEnd = str.size() - 1; + while (::isspace(str[newEnd])) + --newEnd; + + str.assign(str, newBegin, (newEnd - newBegin) + 1); + } + time_t IgniteTimeGm(const tm& time) { tm tmc = time; diff --git a/modules/platforms/cpp/core/include/ignite/ignition.h b/modules/platforms/cpp/core/include/ignite/ignition.h index f88efe5358064..57fd51ca8e884 100644 --- a/modules/platforms/cpp/core/include/ignite/ignition.h +++ b/modules/platforms/cpp/core/include/ignite/ignition.h @@ -154,7 +154,7 @@ namespace ignite * @param err Error. */ static void StopAll(bool cancel, IgniteError* err); - }; + }; } #endif //_IGNITE_IGNITION \ No newline at end of file diff --git a/modules/platforms/cpp/core/src/ignition.cpp b/modules/platforms/cpp/core/src/ignition.cpp index 4a7d9413d264a..83adb4c38aaeb 100644 --- a/modules/platforms/cpp/core/src/ignition.cpp +++ b/modules/platforms/cpp/core/src/ignition.cpp @@ -216,9 +216,9 @@ namespace ignite int optsLen; char** opts = CreateJvmOptions(cfg, homeFound ? &home : NULL, cp, &optsLen); - + envTarget = new SharedPointer(env); - + SharedPointer ctx( JniContext::Create(opts, optsLen, env.Get()->GetJniHandlers(envTarget), &jniErr)); @@ -258,7 +258,7 @@ namespace ignite if (!javaRef) { IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); - + failed = true; } else { @@ -458,7 +458,7 @@ namespace ignite JniErrorInfo jniErr; SharedPointer ctx(JniContext::Create(NULL, 0, JniHandlers(), &jniErr)); - + IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); if (err->GetCode() == IgniteError::IGNITE_SUCCESS) diff --git a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp index c0bb439024954..1851eaeb9a4f0 100644 --- a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp @@ -49,7 +49,7 @@ void CheckValidAddress(const char* connectStr, uint16_t port) BOOST_CHECK_NO_THROW(cfg.FillFromConnectString(connectStr)); - BOOST_CHECK_EQUAL(cfg.GetPort(), port); + BOOST_CHECK_EQUAL(cfg.GetTcpPort(), port); } void CheckValidProtocolVersion(const char* connectStr, ignite::odbc::ProtocolVersion version) @@ -92,7 +92,7 @@ void CheckConnectionConfig(const Configuration& cfg) { BOOST_CHECK_EQUAL(cfg.GetDriver(), testDriverName); BOOST_CHECK_EQUAL(cfg.GetHost(), testServerHost); - BOOST_CHECK_EQUAL(cfg.GetPort(), testServerPort); + BOOST_CHECK_EQUAL(cfg.GetTcpPort(), testServerPort); BOOST_CHECK_EQUAL(cfg.GetAddress(), testAddress); BOOST_CHECK_EQUAL(cfg.GetCache(), testCacheName); BOOST_CHECK_EQUAL(cfg.GetDsn(), std::string()); @@ -119,7 +119,7 @@ void CheckDsnConfig(const Configuration& cfg) BOOST_CHECK_EQUAL(cfg.GetCache(), Configuration::DefaultValue::cache); BOOST_CHECK_EQUAL(cfg.GetAddress(), Configuration::DefaultValue::address); BOOST_CHECK_EQUAL(cfg.GetHost(), std::string()); - BOOST_CHECK_EQUAL(cfg.GetPort(), Configuration::DefaultValue::port); + BOOST_CHECK_EQUAL(cfg.GetTcpPort(), Configuration::DefaultValue::port); BOOST_CHECK_EQUAL(cfg.IsDistributedJoins(), false); BOOST_CHECK_EQUAL(cfg.IsEnforceJoinOrder(), false); } diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am index 9faa999bf4ed8..1781bc09ac4aa 100644 --- a/modules/platforms/cpp/odbc/Makefile.am +++ b/modules/platforms/cpp/odbc/Makefile.am @@ -62,6 +62,7 @@ libignite_odbc_la_SOURCES = \ src/meta/table_meta.cpp \ src/odbc.cpp \ src/entry_points.cpp \ + src/dsn_config.cpp \ src/query/column_metadata_query.cpp \ src/query/data_query.cpp \ src/query/foreign_keys_query.cpp \ diff --git a/modules/platforms/cpp/odbc/include/Makefile.am b/modules/platforms/cpp/odbc/include/Makefile.am index 0776548849ca6..073dcaa1898ba 100644 --- a/modules/platforms/cpp/odbc/include/Makefile.am +++ b/modules/platforms/cpp/odbc/include/Makefile.am @@ -38,6 +38,7 @@ noinst_HEADERS = \ ignite/odbc/row.h \ ignite/odbc/utility.h \ ignite/odbc/environment.h \ + ignite/odbc/dsn_config.h \ ignite/odbc/system/odbc_constants.h \ ignite/odbc/system/socket_client.h \ ignite/odbc/meta/primary_key_meta.h \ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc.h b/modules/platforms/cpp/odbc/include/ignite/odbc.h index 40158ddf51f08..810d186029d4d 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc.h @@ -251,7 +251,6 @@ namespace ignite SQLSMALLINT tableNameLen, SQLSMALLINT scope, SQLSMALLINT nullable); - } // namespace ignite #endif //_IGNITE_ODBC \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h index 30f9ad6afcb36..8e19a6ec6e085 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h @@ -160,11 +160,18 @@ namespace ignite * * @return Server port. */ - uint16_t GetPort() const + uint16_t GetTcpPort() const { return endPoint.port; } + /** + * Set server port. + * + * @param port Server port. + */ + void SetTcpPort(uint16_t port); + /** * Get DSN. * @@ -175,6 +182,16 @@ namespace ignite return GetStringValue(Key::dsn, DefaultValue::dsn); } + /** + * Set DSN. + * + * @param dsn Data Source Name. + */ + void SetDsn(const std::string& dsn) + { + arguments[Key::dsn] = dsn; + } + /** * Get Driver. * @@ -195,6 +212,16 @@ namespace ignite return endPoint.host; } + /** + * Set server host. + * + * @param server Server host. + */ + void SetHost(const std::string& server) + { + arguments[Key::server] = server; + } + /** * Get cache. * @@ -205,6 +232,16 @@ namespace ignite return GetStringValue(Key::cache, DefaultValue::cache); } + /** + * Set cache. + * + * @param cache Cache name. + */ + void SetCache(const std::string& cache) + { + arguments[Key::cache] = cache; + } + /** * Get address. * @@ -215,6 +252,16 @@ namespace ignite return GetStringValue(Key::address, DefaultValue::address); } + /** + * Set address. + * + * @param address Address. + */ + void SetAddress(const std::string& address) + { + arguments[Key::address] = address; + } + /** * Check distributed joins flag. * @@ -225,6 +272,16 @@ namespace ignite return GetBoolValue(Key::distributedJoins, DefaultValue::distributedJoins); } + /** + * Set distributed joins. + * + * @param val Value to set. + */ + void SetDistributedJoins(bool val) + { + SetBoolValue(Key::distributedJoins, val); + } + /** * Check enforce join order flag. * @@ -235,6 +292,26 @@ namespace ignite return GetBoolValue(Key::enforceJoinOrder, DefaultValue::enforceJoinOrder); } + /** + * Set enforce joins. + * + * @param val Value to set. + */ + void SetEnforceJoinOrder(bool val) + { + SetBoolValue(Key::enforceJoinOrder, val); + } + + /** + * Get argument map. + * + * @return Argument map. + */ + const ArgumentMap& GetMap() const + { + return arguments; + } + /** * Get protocol version. * @@ -242,6 +319,13 @@ namespace ignite */ ProtocolVersion GetProtocolVersion() const; + /** + * Set protocol version. + * + * @param version Version to set. + */ + void SetProtocolVersion(const std::string& version); + /** * Get string value from the config. * @@ -269,6 +353,13 @@ namespace ignite */ bool GetBoolValue(const std::string& key, bool dflt) const; + /** + * Set bool value to the config. + * + * @param key Configuration key. + * @param val Value to set. + */ + void SetBoolValue(const std::string& key, bool val); private: /** * Parse connect string into key-value storage. diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h index bfb4f4c01f28f..670e0aa535332 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h @@ -84,7 +84,7 @@ namespace ignite * * @return An informational message on the error or warning. */ - const std::string& GetMessage() const; + const std::string& GetMessageText() const; /** * Get connection name. diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h b/modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h new file mode 100644 index 0000000000000..dbad9b5b91e5f --- /dev/null +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#ifndef _IGNITE_ODBC_DSN_CONFIG +#define _IGNITE_ODBC_DSN_CONFIG + +#include "ignite/odbc/config/configuration.h" + +namespace ignite +{ + namespace odbc + { + /** + * Extract last setup error and throw it like IgniteError. + */ + void ThrowLastSetupError(); + + /** + * Add new string to the DSN file. + * + * @param dsn DSN name. + * @param key Key. + * @param value Value. + */ + void WriteDsnString(const char* dsn, const char* key, const char* value); + + /** + * Get string from the DSN file. + * + * @param dsn DSN name. + * @param key Key. + * @param dflt Default value. + * @return Value. + */ + std::string ReadDsnString(const char* dsn, const char* key, const char* dflt); + + /** + * Read DSN to fill the configuration. + * + * @param dsn DSN name. + * @param config Configuration. + */ + void ReadDsnConfiguration(const char* dsn, config::Configuration& config); + } +} + +#endif //_IGNITE_ODBC_DSN_CONFIG \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h b/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h index 747d78d31efeb..d3d288efe35c8 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h @@ -46,6 +46,13 @@ namespace ignite /** Unknown version of the protocol. */ static const ProtocolVersion VERSION_UNKNOWN; + /** + * Get string to version map. + * + * @return String to version map. + */ + static const StringToVersionMap& GetMap(); + /** * Get current version. * @@ -85,6 +92,13 @@ namespace ignite */ bool IsUnknown() const; + /** + * Check if the distributed joins supported. + * + * @retuen True if the distributed joins supported. + */ + bool IsDistributedJoinsSupported() const; + /** * Comparison operator. * @@ -152,10 +166,12 @@ namespace ignite * * @param major Major version. * @param minor Minor version. - * @param maintenance Maintenance version. + * @param revision Revision. * @return Int value for the version. */ - static int64_t MakeVersion(uint16_t major, uint16_t minor, uint16_t maintenance); + static int64_t MakeVersion(uint16_t major, uint16_t minor, uint16_t revision); + + ProtocolVersion(); /** String to version map. */ static const StringToVersionMap stringToVersionMap; diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h b/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h index 60a65524df8e4..ecd1a5551e2d9 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h @@ -27,10 +27,6 @@ # undef min #endif // min -#ifdef GetMessage -# undef GetMessage -#endif // GetMessage - #endif //_WIN32 #define ODBCVER 0x0380 diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h b/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h new file mode 100644 index 0000000000000..d2cb569286d30 --- /dev/null +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h @@ -0,0 +1,152 @@ +/* + * 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. + */ + +#ifndef _IGNITE_ODBC_SYSTEM_UI_DSN_CONFIGURATION_WINDOW +#define _IGNITE_ODBC_SYSTEM_UI_DSN_CONFIGURATION_WINDOW + +#include "ignite/odbc/config/configuration.h" +#include "ignite/odbc/system/ui/custom_window.h" + +namespace ignite +{ + namespace odbc + { + namespace system + { + namespace ui + { + /** + * DSN configuration window class. + */ + class DsnConfigurationWindow : public CustomWindow + { + /** + * Children windows ids. + */ + enum ChildId + { + ID_CONNECTION_SETTINGS_GROUP_BOX, + ID_NAME_EDIT, + ID_NAME_LABEL, + ID_ADDRESS_EDIT, + ID_ADDRESS_LABEL, + ID_CACHE_EDIT, + ID_CACHE_LABEL, + ID_DISTRIBUTED_JOINS_CHECK_BOX, + ID_ENFORCE_JOIN_ORDER_CHECK_BOX, + ID_PROTOCOL_VERSION_LABEL, + ID_PROTOCOL_VERSION_COMBO_BOX, + ID_OK_BUTTON, + ID_CANCEL_BUTTON + }; + + public: + /** + * Constructor. + * + * @param parent Parent window handle. + */ + explicit DsnConfigurationWindow(Window* parent, config::Configuration& config); + + /** + * Destructor. + */ + virtual ~DsnConfigurationWindow(); + + /** + * Create window in the center of the parent window. + */ + void Create(); + + /** + * @copedoc ignite::odbc::system::ui::CustomWindow::OnCreate + */ + virtual void OnCreate(); + + /** + * @copedoc ignite::odbc::system::ui::CustomWindow::OnMessage + */ + virtual bool OnMessage(UINT msg, WPARAM wParam, LPARAM lParam); + + private: + IGNITE_NO_COPY_ASSIGNMENT(DsnConfigurationWindow) + + /** + * Retrieves current values from the children and stores + * them to the specified configuration. + * + * @param cfg Configuration. + */ + void RetrieveParameters(config::Configuration& cfg) const; + + /** Window width. */ + int width; + + /** Window height. */ + int height; + + /** Connection settings group box. */ + std::auto_ptr connectionSettingsGroupBox; + + /** DSN name edit field label. */ + std::auto_ptr nameLabel; + + /** DSN name edit field. */ + std::auto_ptr nameEdit; + + /** DSN address edit field label. */ + std::auto_ptr addressLabel; + + /** DSN address edit field. */ + std::auto_ptr addressEdit; + + /** DSN cache edit field label. */ + std::auto_ptr cacheLabel; + + /** DSN cache edit field. */ + std::auto_ptr cacheEdit; + + /** Distributed joins CheckBox. */ + std::auto_ptr distributedJoinsCheckBox; + + /** Enforce join order CheckBox. */ + std::auto_ptr enforceJoinOrderCheckBox; + + /** Protocol version edit field. */ + std::auto_ptr protocolVersionLabel; + + /** Protocol verion ComboBox. */ + std::auto_ptr protocolVersionComboBox; + + /** Ok button. */ + std::auto_ptr okButton; + + /** Cancel button. */ + std::auto_ptr cancelButton; + + /** Configuration. */ + config::Configuration& config; + + /** Flag indicating whether OK option was selected. */ + bool accepted; + }; + } + } + } +} + +#endif //_IGNITE_ODBC_SYSTEM_UI_DSN_CONFIGURATION_WINDOW \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/custom_window.h b/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/custom_window.h new file mode 100644 index 0000000000000..1502c07026c3f --- /dev/null +++ b/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/custom_window.h @@ -0,0 +1,189 @@ +/* + * 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. + */ + +#ifndef _IGNITE_ODBC_SYSTEM_UI_CUSTOM_WINDOW +#define _IGNITE_ODBC_SYSTEM_UI_CUSTOM_WINDOW + +#include "ignite/odbc/system/ui/window.h" + +namespace ignite +{ + namespace odbc + { + namespace system + { + namespace ui + { + /** + * Application execution result. + */ + enum Result + { + RESULT_OK, + RESULT_CANCEL + }; + + /** + * Process UI messages in current thread. + * Blocks until quit message has been received. + * + * @param window Main window. + * @return Application execution result. + */ + Result ProcessMessages(Window& window); + + /** + * Window class. + */ + class CustomWindow : public Window + { + public: + /** + * Constructor. + * + * @param parent Parent window. + * @param className Window class name. + * @param title Window title. + * @param callback Event processing function. + */ + CustomWindow(Window* parent, const char* className, const char* title); + + /** + * Destructor. + */ + virtual ~CustomWindow(); + + /** + * Callback which is called upon receiving new message. + * Pure virtual. Should be defined by user. + * + * @param msg Message. + * @param wParam Word-sized parameter. + * @param lParam Long parameter. + * @return Should return true if the message has been + * processed by the handler and false otherwise. + */ + virtual bool OnMessage(UINT msg, WPARAM wParam, LPARAM lParam) = 0; + + /** + * Callback that is called upon window creation. + */ + virtual void OnCreate() = 0; + + /** + * Create child group box window. + * + * @param posX Position by X coordinate. + * @param posY Position by Y coordinate. + * @param sizeX Size by X coordinate. + * @param sizeY Size by Y coordinate. + * @param title Title. + * @param id ID to be assigned to the created window. + * @return Auto pointer containing new window. + */ + std::auto_ptr CreateGroupBox(int posX, int posY, + int sizeX, int sizeY, const char* title, int id); + + /** + * Create child label window. + * + * @param posX Position by X coordinate. + * @param posY Position by Y coordinate. + * @param sizeX Size by X coordinate. + * @param sizeY Size by Y coordinate. + * @param title Title. + * @param id ID to be assigned to the created window. + * @return Auto pointer containing new window. + */ + std::auto_ptr CreateLabel(int posX, int posY, + int sizeX, int sizeY, const char* title, int id); + + /** + * Create child Edit window. + * + * @param posX Position by X coordinate. + * @param posY Position by Y coordinate. + * @param sizeX Size by X coordinate. + * @param sizeY Size by Y coordinate. + * @param title Title. + * @param id ID to be assigned to the created window. + * @return Auto pointer containing new window. + */ + std::auto_ptr CreateEdit(int posX, int posY, + int sizeX, int sizeY, const char* title, int id, int style = 0); + + /** + * Create child button window. + * + * @param posX Position by X coordinate. + * @param posY Position by Y coordinate. + * @param sizeX Size by X coordinate. + * @param sizeY Size by Y coordinate. + * @param title Title. + * @param id ID to be assigned to the created window. + * @return Auto pointer containing new window. + */ + std::auto_ptr CreateButton(int posX, int posY, + int sizeX, int sizeY, const char* title, int id); + + /** + * Create child CheckBox window. + * + * @param posX Position by X coordinate. + * @param posY Position by Y coordinate. + * @param sizeX Size by X coordinate. + * @param sizeY Size by Y coordinate. + * @param title Title. + * @param id ID to be assigned to the created window. + * @return Auto pointer containing new window. + */ + std::auto_ptr CreateCheckBox(int posX, int posY, + int sizeX, int sizeY, const char* title, int id, bool state); + + /** + * Create child ComboBox window. + * + * @param posX Position by X coordinate. + * @param posY Position by Y coordinate. + * @param sizeX Size by X coordinate. + * @param sizeY Size by Y coordinate. + * @param title Title. + * @param id ID to be assigned to the created window. + * @return Auto pointer containing new window. + */ + std::auto_ptr CreateComboBox(int posX, int posY, + int sizeX, int sizeY, const char* title, int id); + private: + IGNITE_NO_COPY_ASSIGNMENT(CustomWindow) + + /** + * Static callback. + * + * @param hwnd Window handle. + * @param msg Message. + * @param wParam Word-sized parameter. + * @param lParam Long parameter. + * @return Operation result. + */ + static LRESULT CALLBACK WndProc(HWND hwnd, UINT msg, WPARAM wParam, LPARAM lParam); + }; + } + } + } +} + +#endif //_IGNITE_ODBC_SYSTEM_UI_CUSTOM_WINDOW \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/window.h b/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/window.h new file mode 100644 index 0000000000000..32a54b200eccd --- /dev/null +++ b/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/window.h @@ -0,0 +1,201 @@ +/* + * 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. + */ + +#ifndef _IGNITE_ODBC_SYSTEM_UI_WINDOW +#define _IGNITE_ODBC_SYSTEM_UI_WINDOW + +#include "ignite/odbc/utility.h" +#include "ignite/odbc/system/odbc_constants.h" + +namespace ignite +{ + namespace odbc + { + namespace system + { + namespace ui + { + /** + * Get handle for the current module. + * + * @return Handle for the current module. + */ + HINSTANCE GetHInstance(); + + /** + * Window class. + */ + class Window + { + public: + /** + * Constructor for a new window that is going to be created. + * + * @param parent Parent window handle. + * @param className Window class name. + * @param title Window title. + * @param callback Event processing function. + */ + Window(Window* parent, const char* className, const char* title); + + /** + * Constructor for the existing window. + * + * @param handle Window handle. + */ + Window(HWND handle); + + /** + * Destructor. + */ + virtual ~Window(); + + /** + * Create window. + * + * @param style Window style. + * @param posX Window x position. + * @param posY Window y position. + * @param width Window width. + * @param height Window height. + * @param id ID for child window. + */ + void Create(DWORD style, int posX, int posY, int width, int height, int id); + + /** + * Show window. + */ + void Show(); + + /** + * Update window. + */ + void Update(); + + /** + * Destroy window. + */ + void Destroy(); + + /** + * Get window handle. + * + * @return Window handle. + */ + HWND GetHandle() const + { + return handle; + } + + /** + * Get window text. + * + * @param text Text. + */ + void GetText(std::string& text) const; + + /** + * Set window text. + * + * @param text Text. + */ + void SetText(const std::string& text) const; + + /** + * Get CheckBox state. + * + * @param True if checked. + */ + bool IsChecked() const; + + /** + * Set CheckBox state. + * + * @param state True if checked. + */ + void SetChecked(bool state); + + /** + * Add string. + * + * @param str String. + */ + void AddString(const std::string& str); + + /** + * Set current ComboBox selection. + * + * @param idx List index. + */ + void SetSelection(int idx); + + /** + * Get current ComboBox selection. + * + * @return idx List index. + */ + int GetSelection() const; + + /** + * Set enabled. + * + * @param enabled Enable flag. + */ + void SetEnabled(bool enabled); + + /** + * Check if the window is enabled. + * + * @return True if enabled. + */ + bool IsEnabled() const; + + protected: + /** + * Set window handle. + * + * @param value Window handle. + */ + void SetHandle(HWND value) + { + handle = value; + } + + /** Window class name. */ + std::string className; + + /** Window title. */ + std::string title; + + /** Window handle. */ + HWND handle; + + /** Specifies whether window has been created by the thread and needs destruction. */ + bool created; + + /** Window parent. */ + Window* parent; + + private: + IGNITE_NO_COPY_ASSIGNMENT(Window) + }; + } + } + } +} + +#endif //_IGNITE_ODBC_SYSTEM_UI_WINDOW \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/os/win/src/system/ui/custom_window.cpp b/modules/platforms/cpp/odbc/os/win/src/system/ui/custom_window.cpp new file mode 100644 index 0000000000000..1e855a18701a4 --- /dev/null +++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/custom_window.cpp @@ -0,0 +1,184 @@ +/* + * 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. + */ + +#include + +#include "ignite/odbc/system/ui/custom_window.h" + +namespace ignite +{ + namespace odbc + { + namespace system + { + namespace ui + { + Result ProcessMessages(Window& window) + { + MSG msg; + + while (GetMessage(&msg, NULL, 0, 0) > 0) + { + if (!IsDialogMessage(window.GetHandle(), &msg)) + { + TranslateMessage(&msg); + + DispatchMessage(&msg); + } + } + + return static_cast(msg.wParam); + } + + LRESULT CALLBACK CustomWindow::WndProc(HWND hwnd, UINT msg, WPARAM wParam, LPARAM lParam) + { + CustomWindow* window = reinterpret_cast(GetWindowLongPtr(hwnd, GWLP_USERDATA)); + + switch (msg) + { + case WM_NCCREATE: + { + _ASSERT(lParam != NULL); + + CREATESTRUCT* createStruct = reinterpret_cast(lParam); + + LONG_PTR longSelfPtr = reinterpret_cast(createStruct->lpCreateParams); + + SetWindowLongPtr(hwnd, GWLP_USERDATA, longSelfPtr); + + return DefWindowProc(hwnd, msg, wParam, lParam); + } + + case WM_CREATE: + { + _ASSERT(window != NULL); + + window->SetHandle(hwnd); + + window->OnCreate(); + + return 0; + } + + default: + break; + } + + if (window && window->OnMessage(msg, wParam, lParam)) + return 0; + + return DefWindowProc(hwnd, msg, wParam, lParam); + } + + CustomWindow::CustomWindow(Window* parent, const char* className, const char* title) : + Window(parent, className, title) + { + WNDCLASS wcx; + + wcx.style = CS_HREDRAW | CS_VREDRAW; + wcx.lpfnWndProc = WndProc; + wcx.cbClsExtra = 0; + wcx.cbWndExtra = 0; + wcx.hInstance = GetHInstance(); + wcx.hIcon = NULL; + wcx.hCursor = LoadCursor(NULL, IDC_ARROW); + wcx.hbrBackground = (HBRUSH)COLOR_WINDOW; + wcx.lpszMenuName = NULL; + wcx.lpszClassName = className; + + + if (!RegisterClass(&wcx)) + { + std::stringstream buf; + + buf << "Can not register window class, error code: " << GetLastError(); + + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str()); + } + } + + CustomWindow::~CustomWindow() + { + UnregisterClass(className.c_str(), GetHInstance()); + } + + std::auto_ptr CustomWindow::CreateGroupBox(int posX, int posY, + int sizeX, int sizeY, const char* title, int id) + { + std::auto_ptr child(new Window(this, "Button", title)); + + child->Create(WS_CHILD | WS_VISIBLE | BS_GROUPBOX, posX, posY, sizeX, sizeY, id); + + return child; + } + + std::auto_ptr CustomWindow::CreateLabel(int posX, int posY, + int sizeX, int sizeY, const char* title, int id) + { + std::auto_ptr child(new Window(this, "Static", title)); + + child->Create(WS_CHILD | WS_VISIBLE, posX, posY, sizeX, sizeY, id); + + return child; + } + + std::auto_ptr CustomWindow::CreateEdit(int posX, int posY, + int sizeX, int sizeY, const char* title, int id, int style) + { + std::auto_ptr child(new Window(this, "Edit", title)); + + child->Create(WS_CHILD | WS_VISIBLE | WS_BORDER | ES_AUTOHSCROLL | WS_TABSTOP | style, + posX, posY, sizeX, sizeY, id); + + return child; + } + + std::auto_ptr CustomWindow::CreateButton(int posX, int posY, + int sizeX, int sizeY, const char* title, int id) + { + std::auto_ptr child(new Window(this, "Button", title)); + + child->Create(WS_CHILD | WS_VISIBLE | WS_TABSTOP, posX, posY, sizeX, sizeY, id); + + return child; + } + + std::auto_ptr CustomWindow::CreateCheckBox(int posX, int posY, + int sizeX, int sizeY, const char* title, int id, bool state) + { + std::auto_ptr child(new Window(this, "Button", title)); + + child->Create(WS_CHILD | WS_VISIBLE | BS_CHECKBOX, posX, posY, sizeX, sizeY, id); + + child->SetChecked(state); + + return child; + } + + std::auto_ptr CustomWindow::CreateComboBox(int posX, int posY, + int sizeX, int sizeY, const char * title, int id) + { + std::auto_ptr child(new Window(this, "Combobox", title)); + + child->Create(WS_CHILD | WS_VISIBLE | CBS_DROPDOWNLIST, posX, posY, sizeX, sizeY, id); + + return child; + } + } + } + } +} diff --git a/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp new file mode 100644 index 0000000000000..663333a8f485a --- /dev/null +++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp @@ -0,0 +1,305 @@ +/* + * 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. + */ + +#include + +#include "ignite/odbc/system/ui/dsn_configuration_window.h" + +namespace ignite +{ + namespace odbc + { + namespace system + { + namespace ui + { + DsnConfigurationWindow::DsnConfigurationWindow(Window* parent, config::Configuration& config): + CustomWindow(parent, "IgniteConfigureDsn", "Configure Apache Ignite DSN"), + width(360), + height(230), + connectionSettingsGroupBox(), + nameLabel(), + nameEdit(), + addressLabel(), + addressEdit(), + cacheLabel(), + cacheEdit(), + distributedJoinsCheckBox(), + enforceJoinOrderCheckBox(), + protocolVersionLabel(), + protocolVersionComboBox(), + okButton(), + cancelButton(), + config(config), + accepted(false) + { + // No-op. + } + + DsnConfigurationWindow::~DsnConfigurationWindow() + { + // No-op. + } + + void DsnConfigurationWindow::Create() + { + // Finding out parent position. + RECT parentRect; + GetWindowRect(parent->GetHandle(), &parentRect); + + // Positioning window to the center of parent window. + const int posX = parentRect.left + (parentRect.right - parentRect.left - width) / 2; + const int posY = parentRect.top + (parentRect.bottom - parentRect.top - height) / 2; + + RECT desiredRect = {posX, posY, posX + width, posY + height}; + AdjustWindowRect(&desiredRect, WS_BORDER | WS_CAPTION | WS_SYSMENU | WS_THICKFRAME, FALSE); + + Window::Create(WS_OVERLAPPED | WS_SYSMENU, desiredRect.left, desiredRect.top, + desiredRect.right - desiredRect.left, desiredRect.bottom - desiredRect.top, 0); + + if (!handle) + { + std::stringstream buf; + + buf << "Can not create window, error code: " << GetLastError(); + + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str()); + } + } + + void DsnConfigurationWindow::OnCreate() + { + int margin = 10; + int interval = 10; + + int labelSizeX = 80; + int labelPosX = margin + interval; + + int editSizeX = width - labelSizeX - 2 * margin - 3 * interval; + int editPosX = margin + labelSizeX + 2 * interval; + + int rowSize = 20; + int rowPos = margin + 2 * interval; + + int checkBoxSize = (editSizeX - interval) / 2; + + int sectionBegin = margin; + + const char* val = config.GetDsn().c_str(); + nameLabel = CreateLabel(labelPosX, rowPos, labelSizeX, rowSize, "DSN name:", ID_NAME_LABEL); + nameEdit = CreateEdit(editPosX, rowPos, editSizeX, rowSize, val, ID_NAME_EDIT); + + rowPos += interval + rowSize; + + val = config.GetAddress().c_str(); + addressLabel = CreateLabel(labelPosX, rowPos, labelSizeX, rowSize, "Address:", ID_ADDRESS_LABEL); + addressEdit = CreateEdit(editPosX, rowPos, editSizeX, rowSize, val, ID_ADDRESS_EDIT); + + rowPos += interval + rowSize; + + val = config.GetCache().c_str(); + cacheLabel = CreateLabel(labelPosX, rowPos, labelSizeX, rowSize, "Cache name:", ID_CACHE_LABEL); + cacheEdit = CreateEdit(editPosX, rowPos, editSizeX, rowSize, val, ID_CACHE_EDIT); + + rowPos += interval + rowSize; + + protocolVersionLabel = CreateLabel(labelPosX, rowPos, labelSizeX, rowSize, + "Protocol version:", ID_PROTOCOL_VERSION_LABEL); + protocolVersionComboBox = CreateComboBox(editPosX, rowPos, editSizeX, rowSize, + "Protocol version", ID_PROTOCOL_VERSION_COMBO_BOX); + + int id = 0; + + const ProtocolVersion::StringToVersionMap& versionMap = ProtocolVersion::GetMap(); + + ProtocolVersion::StringToVersionMap::const_iterator it; + for (it = versionMap.begin(); it != versionMap.end(); ++it) + { + protocolVersionComboBox->AddString(it->first); + + if (it->second == config.GetProtocolVersion()) + protocolVersionComboBox->SetSelection(id); + + ++id; + } + + rowPos += interval + rowSize; + + distributedJoinsCheckBox = CreateCheckBox(editPosX, rowPos, checkBoxSize, rowSize, + "Distributed Joins", ID_DISTRIBUTED_JOINS_CHECK_BOX, config.IsDistributedJoins()); + + enforceJoinOrderCheckBox = CreateCheckBox(editPosX + checkBoxSize + interval, rowPos, checkBoxSize, + rowSize, "Enforce Join Order", ID_ENFORCE_JOIN_ORDER_CHECK_BOX, config.IsEnforceJoinOrder()); + + if (!config.GetProtocolVersion().IsDistributedJoinsSupported()) + { + distributedJoinsCheckBox->SetEnabled(false); + enforceJoinOrderCheckBox->SetEnabled(false); + } + + rowPos += interval * 2 + rowSize; + + connectionSettingsGroupBox = CreateGroupBox(margin, sectionBegin, width - 2 * margin, + rowPos - interval - sectionBegin, "Connection settings", ID_CONNECTION_SETTINGS_GROUP_BOX); + + int buttonSizeX = 80; + int cancelPosX = width - margin - buttonSizeX; + int okPosX = cancelPosX - interval - buttonSizeX; + + rowSize = 25; + + okButton = CreateButton(okPosX, rowPos, buttonSizeX, rowSize, "Ok", ID_OK_BUTTON); + cancelButton = CreateButton(cancelPosX, rowPos, buttonSizeX, rowSize, "Cancel", ID_CANCEL_BUTTON); + } + + bool DsnConfigurationWindow::OnMessage(UINT msg, WPARAM wParam, LPARAM lParam) + { + switch (msg) + { + case WM_COMMAND: + { + switch (LOWORD(wParam)) + { + case ID_OK_BUTTON: + { + try + { + RetrieveParameters(config); + + accepted = true; + + PostMessage(GetHandle(), WM_CLOSE, 0, 0); + } + catch (IgniteError& err) + { + MessageBox(NULL, err.GetText(), "Error!", MB_ICONEXCLAMATION | MB_OK); + } + + break; + } + + case ID_PROTOCOL_VERSION_COMBO_BOX: + { + if (HIWORD(wParam) == CBN_SELCHANGE) + { + std::string text; + + protocolVersionComboBox->GetText(text); + + ProtocolVersion version = ProtocolVersion::FromString(text); + + if (!version.IsUnknown() && !version.IsDistributedJoinsSupported()) + { + distributedJoinsCheckBox->SetEnabled(false); + enforceJoinOrderCheckBox->SetEnabled(false); + } + else + { + distributedJoinsCheckBox->SetEnabled(true); + enforceJoinOrderCheckBox->SetEnabled(true); + } + } + + break; + } + + case IDCANCEL: + case ID_CANCEL_BUTTON: + { + PostMessage(GetHandle(), WM_CLOSE, 0, 0); + + break; + } + + case ID_DISTRIBUTED_JOINS_CHECK_BOX: + { + distributedJoinsCheckBox->SetChecked(!distributedJoinsCheckBox->IsChecked()); + + break; + } + + case ID_ENFORCE_JOIN_ORDER_CHECK_BOX: + { + enforceJoinOrderCheckBox->SetChecked(!enforceJoinOrderCheckBox->IsChecked()); + + break; + } + + default: + return false; + } + + break; + } + + case WM_DESTROY: + { + PostQuitMessage(accepted ? RESULT_OK : RESULT_CANCEL); + + break; + } + + default: + return false; + } + + return true; + } + + void DsnConfigurationWindow::RetrieveParameters(config::Configuration& cfg) const + { + std::string dsn; + std::string address; + std::string cache; + std::string version; + + bool distributedJoins; + bool enforceJoinOrder; + + nameEdit->GetText(dsn); + addressEdit->GetText(address); + cacheEdit->GetText(cache); + protocolVersionComboBox->GetText(version); + + common::StripSurroundingWhitespaces(address); + common::StripSurroundingWhitespaces(dsn); + + distributedJoins = distributedJoinsCheckBox->IsEnabled() && distributedJoinsCheckBox->IsChecked(); + enforceJoinOrder = enforceJoinOrderCheckBox->IsEnabled() && enforceJoinOrderCheckBox->IsChecked(); + + LOG_MSG("Retriving arguments:\n"); + LOG_MSG("DSN: %s\n", dsn.c_str()); + LOG_MSG("Address: %s\n", address.c_str()); + LOG_MSG("Cache: %s\n", cache.c_str()); + LOG_MSG("Protocol version: %s\n", version.c_str()); + LOG_MSG("Distributed Joins: %s\n", distributedJoins ? "true" : "false"); + LOG_MSG("Enforce Join Order: %s\n", enforceJoinOrder ? "true" : "false"); + + if (dsn.empty()) + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, "DSN name can not be empty."); + + cfg.SetDsn(dsn); + cfg.SetAddress(address); + cfg.SetCache(cache); + cfg.SetProtocolVersion(version); + cfg.SetDistributedJoins(distributedJoins); + cfg.SetEnforceJoinOrder(enforceJoinOrder); + } + } + } + } +} diff --git a/modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp b/modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp new file mode 100644 index 0000000000000..1143f01321600 --- /dev/null +++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp @@ -0,0 +1,192 @@ +/* + * 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. + */ + +#include + +#include "ignite/odbc/system/ui/window.h" + +namespace ignite +{ + namespace odbc + { + namespace system + { + namespace ui + { + HINSTANCE GetHInstance() + { + HINSTANCE hInstance = GetModuleHandle(TARGET_MODULE_FULL_NAME); + + if (hInstance == NULL) + { + std::stringstream buf; + + buf << "Can not get hInstance for the module, error code: " << GetLastError(); + + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str()); + } + + return hInstance; + } + + Window::Window(Window* parent, const char* className, const char* title) : + className(className), + title(title), + handle(NULL), + created(false), + parent(parent) + { + // No-op. + } + + Window::Window(HWND handle) : + className(), + title(), + handle(handle), + created(false), + parent(0) + { + // No-op. + } + + Window::~Window() + { + if (created) + Destroy(); + } + + void Window::Create(DWORD style, int posX, int posY, int width, int height, int id) + { + if (handle) + { + std::stringstream buf; + + buf << "Window already created, error code: " << GetLastError(); + + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str()); + } + + handle = CreateWindow( + className.c_str(), + title.c_str(), + style, + posX, + posY, + width, + height, + parent ? parent->GetHandle() : NULL, + reinterpret_cast(id), + GetHInstance(), + this + ); + + if (!handle) + { + std::stringstream buf; + + buf << "Can not create window, error code: " << GetLastError(); + + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str()); + } + + created = true; + + HGDIOBJ hfDefault = GetStockObject(DEFAULT_GUI_FONT); + + SendMessage(GetHandle(), WM_SETFONT, (WPARAM)hfDefault, MAKELPARAM(FALSE, 0)); + } + + void Window::Show() + { + ShowWindow(handle, SW_SHOW); + } + + void Window::Update() + { + UpdateWindow(handle); + } + + void Window::Destroy() + { + if (handle) + DestroyWindow(handle); + + handle = NULL; + } + + void Window::GetText(std::string& text) const + { + int len = GetWindowTextLength(handle); + + if (len <= 0) + { + text.clear(); + + return; + } + + text.resize(len + 1); + + if (!GetWindowText(handle, &text[0], len + 1)) + text.clear(); + + text.resize(len); + } + + void Window::SetText(const std::string& text) const + { + SNDMSG(handle, WM_SETTEXT, 0, reinterpret_cast(text.c_str())); + } + + bool Window::IsChecked() const + { + return Button_GetCheck(handle) == BST_CHECKED; + } + + void Window::SetChecked(bool state) + { + Button_SetCheck(handle, state ? BST_CHECKED : BST_UNCHECKED); + } + + void Window::AddString(const std::string & str) + { + SNDMSG(handle, CB_ADDSTRING, 0, reinterpret_cast(str.c_str())); + } + + void Window::SetSelection(int idx) + { + SNDMSG(handle, CB_SETCURSEL, static_cast(idx), 0); + } + + int Window::GetSelection() const + { + return static_cast(SNDMSG(handle, CB_GETCURSEL, 0, 0)); + } + + void Window::SetEnabled(bool enabled) + { + EnableWindow(GetHandle(), enabled); + } + + bool Window::IsEnabled() const + { + return IsWindowEnabled(GetHandle()) != 0; + } + } + } + } +} diff --git a/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp b/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp new file mode 100644 index 0000000000000..f432a406d60c9 --- /dev/null +++ b/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp @@ -0,0 +1,218 @@ +/* + * 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. + */ + +#include "ignite/odbc/utility.h" +#include "ignite/odbc/system/odbc_constants.h" + +#include "ignite/odbc/dsn_config.h" +#include "ignite/odbc/system/ui/window.h" +#include "ignite/odbc/system/ui/dsn_configuration_window.h" + +using ignite::odbc::config::Configuration; + +/** + * Display configuration window for user to configure DSN. + * + * @param hwndParent Parent window handle. + * @param config Output configuration. + * @return True on success and false on fail. + */ +bool DisplayConfigureDsnWindow(HWND hwndParent, Configuration& config) +{ + using namespace ignite::odbc::system::ui; + + if (!hwndParent) + return false; + + try + { + Window parent(hwndParent); + + DsnConfigurationWindow window(&parent, config); + + window.Create(); + + window.Show(); + window.Update(); + + return ProcessMessages(window) == RESULT_OK; + } + catch (const ignite::IgniteError& err) + { + std::stringstream buf; + + buf << "Message: " << err.GetText() << ", Code: " << err.GetCode(); + + std::string message = buf.str(); + + MessageBox(NULL, message.c_str(), "Error!", MB_ICONEXCLAMATION | MB_OK); + + SQLPostInstallerError(err.GetCode(), err.GetText()); + } + + return false; +} + +/** + * Register DSN with specified configuration. + * + * @param config Configuration. + * @param driver Driver. + * @return True on success and false on fail. + */ +bool RegisterDsn(const Configuration& config, LPCSTR driver) +{ + using namespace ignite::odbc::config; + using ignite::common::LexicalCast; + + typedef Configuration::ArgumentMap ArgMap; + + const char* dsn = config.GetDsn().c_str(); + + try + { + if (!SQLWriteDSNToIni(dsn, driver)) + ignite::odbc::ThrowLastSetupError(); + + const ArgMap& map = config.GetMap(); + + std::set ignore; + + ignore.insert(Configuration::Key::dsn); + ignore.insert(Configuration::Key::driver); + + for (ArgMap::const_iterator it = map.begin(); it != map.end(); ++it) + { + const std::string& key = it->first; + const std::string& value = it->second; + + if (ignore.find(key) != ignore.end()) + continue; + + ignite::odbc::WriteDsnString(dsn, key.c_str(), value.c_str()); + } + + return true; + } + catch (ignite::IgniteError& err) + { + MessageBox(NULL, err.GetText(), "Error!", MB_ICONEXCLAMATION | MB_OK); + + SQLPostInstallerError(err.GetCode(), err.GetText()); + } + + return false; +} + +/** + * Unregister specified DSN. + * + * @param dsn DSN name. + * @return True on success and false on fail. + */ +bool UnregisterDsn(const char* dsn) +{ + try + { + if (!SQLRemoveDSNFromIni(dsn)) + ignite::odbc::ThrowLastSetupError(); + + return true; + } + catch (ignite::IgniteError& err) + { + MessageBox(NULL, err.GetText(), "Error!", MB_ICONEXCLAMATION | MB_OK); + + SQLPostInstallerError(err.GetCode(), err.GetText()); + } + + return false; +} + +BOOL INSTAPI ConfigDSN(HWND hwndParent, WORD req, LPCSTR driver, LPCSTR attributes) +{ + using namespace ignite::odbc; + + LOG_MSG("ConfigDSN called\n"); + + Configuration config; + + LOG_MSG("Attributes: %s\n", attributes); + + config.FillFromConfigAttributes(attributes); + + if (!SQLValidDSN(config.GetDsn().c_str())) + return FALSE; + + LOG_MSG("Driver: %s\n", driver); + LOG_MSG("Attributes: %s\n", attributes); + + LOG_MSG("DSN: %s\n", config.GetDsn().c_str()); + + switch (req) + { + case ODBC_ADD_DSN: + { + LOG_MSG("ODBC_ADD_DSN\n"); + + if (!DisplayConfigureDsnWindow(hwndParent, config)) + return FALSE; + + if (!RegisterDsn(config, driver)) + return FALSE; + + break; + } + + case ODBC_CONFIG_DSN: + { + LOG_MSG("ODBC_CONFIG_DSN\n"); + + std::string dsn = config.GetDsn(); + + Configuration loaded(config); + + ReadDsnConfiguration(dsn.c_str(), loaded); + + if (!DisplayConfigureDsnWindow(hwndParent, loaded)) + return FALSE; + + if (!RegisterDsn(loaded, driver)) + return FALSE; + + if (loaded.GetDsn() != dsn && !UnregisterDsn(dsn.c_str())) + return FALSE; + + break; + } + + case ODBC_REMOVE_DSN: + { + LOG_MSG("ODBC_REMOVE_DSN\n"); + + if (!UnregisterDsn(config.GetDsn().c_str())) + return FALSE; + + break; + } + + default: + return FALSE; + } + + return TRUE; +} \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj index 0e0f0d3a65142..d48db21bd17b4 100644 --- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj +++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj @@ -93,8 +93,8 @@ Level3 Disabled false - $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win;$(ProjectDir)\..\..\src - _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;_DEBUG;%(PreprocessorDefinitions) + $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src + _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";_DEBUG;ODBC_DEBUG;ODBC_LOG_PATH="D:\\odbc.log";%(PreprocessorDefinitions) true @@ -107,8 +107,8 @@ Level3 Disabled false - $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win;$(ProjectDir)\..\..\src - _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;_DEBUG;%(PreprocessorDefinitions) + $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src + _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";_DEBUG;ODBC_DEBUG;ODBC_LOG_PATH="D:\\odbc.log";%(PreprocessorDefinitions) true @@ -123,8 +123,8 @@ true true false - $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win;$(ProjectDir)\..\..\src - _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;%(PreprocessorDefinitions) + $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src + _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";%(PreprocessorDefinitions) true @@ -141,8 +141,8 @@ true true false - $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win;$(ProjectDir)\..\..\src - _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;%(PreprocessorDefinitions) + $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src + _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";%(PreprocessorDefinitions) true @@ -153,7 +153,11 @@ + + + + @@ -165,6 +169,7 @@ + @@ -201,6 +206,7 @@ + @@ -221,8 +227,11 @@ + + + diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters index 9caf4839f8e6e..58764e4043871 100644 --- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters +++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters @@ -26,6 +26,9 @@ {df33e506-b5d8-423f-bcc5-1825242a3e28} + + {ff144e89-0a10-42c3-97dd-d22bfdbc7abb} + @@ -115,6 +118,21 @@ Code + + Code\system\ui + + + Code\system\ui + + + Code\system\ui + + + Code\system + + + Code + Code @@ -227,6 +245,18 @@ Code + + Code\system\ui + + + Code\system\ui + + + Code\system\ui + + + Code + Code diff --git a/modules/platforms/cpp/odbc/src/config/configuration.cpp b/modules/platforms/cpp/odbc/src/config/configuration.cpp index 24c2bdf456245..dbe40bde717e3 100644 --- a/modules/platforms/cpp/odbc/src/config/configuration.cpp +++ b/modules/platforms/cpp/odbc/src/config/configuration.cpp @@ -122,7 +122,7 @@ namespace ignite return connect_string_buffer.str(); } - void Configuration::FillFromConfigAttributes(const char * attributes) + void Configuration::FillFromConfigAttributes(const char* attributes) { // Initializing map. arguments.clear(); @@ -150,6 +150,11 @@ namespace ignite } } + void Configuration::SetTcpPort(uint16_t port) + { + arguments[Key::port] = common::LexicalCast(port); + } + ProtocolVersion Configuration::GetProtocolVersion() const { ArgumentMap::const_iterator it = arguments.find(Key::protocolVersion); @@ -160,6 +165,11 @@ namespace ignite return DefaultValue::protocolVersion; } + void Configuration::SetProtocolVersion(const std::string& version) + { + arguments[Key::protocolVersion] = version; + } + const std::string& Configuration::GetStringValue(const std::string& key, const std::string& dflt) const { ArgumentMap::const_iterator it = arguments.find(common::ToLower(key)); @@ -206,6 +216,11 @@ namespace ignite return dflt; } + void Configuration::SetBoolValue(const std::string& key, bool val) + { + arguments[key] = val ? "true" : "false"; + } + void Configuration::ParseAttributeList(const char * str, size_t len, char delimeter, ArgumentMap & args) { std::string connect_str(str, len); diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp index 4315698756717..0fd951383b764 100644 --- a/modules/platforms/cpp/odbc/src/connection.cpp +++ b/modules/platforms/cpp/odbc/src/connection.cpp @@ -117,7 +117,7 @@ namespace ignite return SQL_RESULT_ERROR; } - connected = socket.Connect(cfg.GetHost().c_str(), cfg.GetPort()); + connected = socket.Connect(cfg.GetHost().c_str(), cfg.GetTcpPort()); if (!connected) { diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp index 0fdfbc87fb3e3..8553ee47d5188 100644 --- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp +++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp @@ -171,7 +171,7 @@ namespace ignite return ORIGIN_ISO_9075; } - const std::string& DiagnosticRecord::GetMessage() const + const std::string& DiagnosticRecord::GetMessageText() const { return message; } diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp index 90c0a4fdefba4..99ef292534447 100644 --- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp +++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp @@ -190,7 +190,7 @@ namespace ignite case IGNITE_SQL_DIAG_STATUS_MESSAGE_TEXT: { - buffer.PutString(record.GetMessage()); + buffer.PutString(record.GetMessageText()); return SQL_RESULT_SUCCESS; } diff --git a/modules/platforms/cpp/odbc/src/dsn_config.cpp b/modules/platforms/cpp/odbc/src/dsn_config.cpp new file mode 100644 index 0000000000000..a304567565c0a --- /dev/null +++ b/modules/platforms/cpp/odbc/src/dsn_config.cpp @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include + +#include "ignite/odbc/utility.h" +#include "ignite/odbc/system/odbc_constants.h" + +#include "ignite/odbc/dsn_config.h" + +using ignite::odbc::config::Configuration; + +#define BUFFER_SIZE 1024 +#define CONFIG_FILE "ODBC.INI" + +namespace ignite +{ + namespace odbc + { + void ThrowLastSetupError() + { + DWORD code; + char msg[BUFFER_SIZE]; + + SQLInstallerError(1, &code, msg, sizeof(msg), NULL); + + std::stringstream buf; + + buf << "Message: \"" << msg << "\", Code: " << code; + + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str()); + } + + void WriteDsnString(const char* dsn, const char* key, const char* value) + { + if (!SQLWritePrivateProfileString(dsn, key, value, CONFIG_FILE)) + ThrowLastSetupError(); + } + + std::string ReadDsnString(const char* dsn, const std::string& key, const char* dflt) + { + char buf[BUFFER_SIZE]; + + memset(buf, 0, sizeof(buf)); + + SQLGetPrivateProfileString(dsn, key.c_str(), dflt, buf, sizeof(buf), CONFIG_FILE); + + return std::string(buf); + } + + int ReadDsnInt(const char* dsn, const std::string& key, int dflt) + { + char buf[BUFFER_SIZE]; + + memset(buf, 0, sizeof(buf)); + + std::string dflt0 = common::LexicalCast(dflt); + + SQLGetPrivateProfileString(dsn, key.c_str(), dflt0.c_str(), buf, sizeof(buf), CONFIG_FILE); + + return common::LexicalCast(buf); + } + + bool ReadDsnBool(const char* dsn, const std::string& key, bool dflt) + { + char buf[BUFFER_SIZE]; + + memset(buf, 0, sizeof(buf)); + + std::string dflt0 = dflt ? "true" : "false"; + + SQLGetPrivateProfileString(dsn, key.c_str(), dflt0.c_str(), buf, sizeof(buf), CONFIG_FILE); + + return std::string(buf) == "true"; + } + + void ReadDsnConfiguration(const char* dsn, Configuration& config) + { + std::string address = ReadDsnString(dsn, Configuration::Key::address, config.GetAddress().c_str()); + std::string server = ReadDsnString(dsn, Configuration::Key::server, config.GetHost().c_str()); + uint16_t port = ReadDsnInt(dsn, Configuration::Key::port, config.GetTcpPort()); + std::string cache = ReadDsnString(dsn, Configuration::Key::cache, config.GetCache().c_str()); + bool distributedJoins = ReadDsnBool(dsn, Configuration::Key::distributedJoins, config.IsDistributedJoins()); + bool enforceJoinOrder = ReadDsnBool(dsn, Configuration::Key::enforceJoinOrder, config.IsEnforceJoinOrder()); + std::string version = ReadDsnString(dsn, Configuration::Key::protocolVersion, + config.GetProtocolVersion().ToString().c_str()); + + LOG_MSG("%d\n", __LINE__); + + config.SetAddress(address); + config.SetHost(server); + config.SetTcpPort(port); + config.SetCache(cache); + config.SetDistributedJoins(distributedJoins); + config.SetEnforceJoinOrder(enforceJoinOrder); + config.SetProtocolVersion(version); + + LOG_MSG("%d\n", __LINE__); + } + } +} \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/src/entry_points.cpp b/modules/platforms/cpp/odbc/src/entry_points.cpp index c8e78a5c80007..f6195e1168b44 100644 --- a/modules/platforms/cpp/odbc/src/entry_points.cpp +++ b/modules/platforms/cpp/odbc/src/entry_points.cpp @@ -19,14 +19,6 @@ #include "ignite/odbc/utility.h" -BOOL INSTAPI ConfigDSN(HWND hwndParent, - WORD req, - LPCSTR driver, - LPCSTR attributes) -{ - return ignite::ConfigDSN(hwndParent, req, driver, attributes); -} - SQLRETURN SQL_API SQLGetInfo(SQLHDBC conn, SQLUSMALLINT infoType, SQLPOINTER infoValue, diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp index 9b4179ed75c6a..fd35cbab6875e 100644 --- a/modules/platforms/cpp/odbc/src/odbc.cpp +++ b/modules/platforms/cpp/odbc/src/odbc.cpp @@ -28,70 +28,11 @@ #include "ignite/odbc/environment.h" #include "ignite/odbc/connection.h" #include "ignite/odbc/statement.h" +#include "ignite/odbc/dsn_config.h" #include "ignite/odbc.h" namespace ignite { - - BOOL ConfigDSN(HWND hwndParent, - WORD req, - LPCSTR driver, - LPCSTR attributes) - { - LOG_MSG("ConfigDSN called\n"); - - ignite::odbc::config::Configuration config; - - try - { - config.FillFromConfigAttributes(attributes); - } - catch (IgniteError& e) - { - SQLPostInstallerError(e.GetCode(), e.GetText()); - - return SQL_FALSE; - } - - if (!SQLValidDSN(config.GetDsn().c_str())) - return SQL_FALSE; - - LOG_MSG("Driver: %s\n", driver); - LOG_MSG("Attributes: %s\n", attributes); - - LOG_MSG("DSN: %s\n", config.GetDsn().c_str()); - - switch (req) - { - case ODBC_ADD_DSN: - { - LOG_MSG("ODBC_ADD_DSN\n"); - - return SQLWriteDSNToIni(config.GetDsn().c_str(), driver); - } - - case ODBC_CONFIG_DSN: - { - LOG_MSG("ODBC_CONFIG_DSN\n"); - break; - } - - case ODBC_REMOVE_DSN: - { - LOG_MSG("ODBC_REMOVE_DSN\n"); - - return SQLRemoveDSNFromIni(config.GetDsn().c_str()); - } - - default: - { - return SQL_FALSE; - } - } - - return SQL_TRUE; - } - SQLRETURN SQLGetInfo(SQLHDBC conn, SQLUSMALLINT infoType, SQLPOINTER infoValue, @@ -315,10 +256,10 @@ namespace ignite SQLSMALLINT* outConnectionStringLen, SQLUSMALLINT driverCompletion) { - using ignite::odbc::Connection; - using ignite::odbc::diagnostic::DiagnosticRecordStorage; - using ignite::utility::SqlStringToString; - using ignite::utility::CopyStringToBuffer; + using odbc::Connection; + using odbc::diagnostic::DiagnosticRecordStorage; + using utility::SqlStringToString; + using utility::CopyStringToBuffer; UNREFERENCED_PARAMETER(windowHandle); @@ -332,7 +273,16 @@ namespace ignite std::string connectStr = SqlStringToString(inConnectionString, inConnectionStringLen); - connection->Establish(connectStr); + odbc::config::Configuration config; + + config.FillFromConnectString(connectStr); + + std::string dsn = config.GetDsn(); + + if (!dsn.empty()) + odbc::ReadDsnConfiguration(dsn.c_str(), config); + + connection->Establish(config); const DiagnosticRecordStorage& diag = connection->GetDiagnosticRecords(); @@ -372,9 +322,11 @@ namespace ignite if (!connection) return SQL_INVALID_HANDLE; - //std::string server = SqlStringToString(serverName, serverNameLen); + odbc::config::Configuration config; + + std::string dsn = SqlStringToString(serverName, serverNameLen); - Configuration config; + odbc::ReadDsnConfiguration(dsn.c_str(), config); connection->Establish(config); @@ -1175,7 +1127,7 @@ namespace ignite SqlLen outResLen; ApplicationDataBuffer outBuffer(IGNITE_ODBC_C_TYPE_CHAR, msgBuffer, msgBufferLen, &outResLen); - outBuffer.PutString(record.GetMessage()); + outBuffer.PutString(record.GetMessageText()); *msgLen = static_cast(outResLen); diff --git a/modules/platforms/cpp/odbc/src/protocol_version.cpp b/modules/platforms/cpp/odbc/src/protocol_version.cpp index c65099d38ead8..ebd3b6adbc1cb 100644 --- a/modules/platforms/cpp/odbc/src/protocol_version.cpp +++ b/modules/platforms/cpp/odbc/src/protocol_version.cpp @@ -14,12 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - -#include "ignite/odbc/protocol_version.h" #include #include #include +#include "ignite/odbc/protocol_version.h" +#include "ignite/odbc/utility.h" + namespace ignite { namespace odbc @@ -50,10 +51,15 @@ namespace ignite // No-op. } - int64_t ProtocolVersion::MakeVersion(uint16_t major, uint16_t minor, uint16_t maintenance) + int64_t ProtocolVersion::MakeVersion(uint16_t major, uint16_t minor, uint16_t revision) { const static int64_t MASK = 0x000000000000FFFFLL; - return ((major & MASK) << 48) | ((minor & MASK) << 32) | ((maintenance & MASK) << 16); + return ((major & MASK) << 48) | ((minor & MASK) << 32) | ((revision & MASK) << 16); + } + + const ProtocolVersion::StringToVersionMap& ProtocolVersion::GetMap() + { + return stringToVersionMap; } const ProtocolVersion& ProtocolVersion::GetCurrent() @@ -68,8 +74,8 @@ namespace ignite if (it == stringToVersionMap.end()) { throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, - "Invalid version format. Valid format is X.Y.Z, where X, Y and Z are major, " - "minor and maintenance versions of Ignite since which protocol is introduced."); + "Invalid version format. Valid format is X.Y.Z, where X, Y and Z are major " + "and minor versions and revision of Ignite since which protocol is introduced."); } return it->second; @@ -100,6 +106,11 @@ namespace ignite return *this == VERSION_UNKNOWN; } + bool ProtocolVersion::IsDistributedJoinsSupported() const + { + return *this >= VERSION_1_8_0; + } + bool operator==(const ProtocolVersion& val1, const ProtocolVersion& val2) { return val1.val == val2.val; From ae0b5ebf02f3eb70d24dd3b0eb63dde9843c82b0 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 26 Aug 2016 11:12:31 +0300 Subject: [PATCH 063/487] IGNITE-3739: ODBC: Added GUID escape sequence support. This closes #988. --- .../odbc/escape/OdbcEscapeType.java | 16 ++-- .../odbc/escape/OdbcEscapeUtils.java | 71 ++++++++++++--- .../odbc/OdbcEscapeSequenceSelfTest.java | 86 +++++++++++++++---- 3 files changed, 136 insertions(+), 37 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java index 96a2127b1c1c7..3bf0324b7b2cb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java @@ -63,8 +63,8 @@ public static OdbcEscapeType[] sortedValues() { /** Escape sequence body. */ private final String body; - /** Whether token must be delimited from the rest of escape sequence. */ - private final boolean delimited; + /** Whether this is a standard token with no special handling. */ + private final boolean standard; /** Whether empty escape sequence is allowed. */ private final boolean allowEmpty; @@ -73,12 +73,12 @@ public static OdbcEscapeType[] sortedValues() { * Constructor. * * @param body Escape sequence body. - * @param delimited Whether token must be delimited from the rest of escape sequence. + * @param standard Whether this is a standard token with no special handling. * @param allowEmpty Whether empty escape sequence is allowed. */ - OdbcEscapeType(String body, boolean delimited, boolean allowEmpty) { + OdbcEscapeType(String body, boolean standard, boolean allowEmpty) { this.body = body; - this.delimited = delimited; + this.standard = standard; this.allowEmpty = allowEmpty; } @@ -90,10 +90,10 @@ public String body() { } /** - * @return Whether token must be delimited from the rest of escape sequence. + * @return Whether this is a standard token with no special handling. */ - public boolean delimited() { - return delimited; + public boolean standard() { + return standard; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java index 6299c7efede2e..83ec9d8eee847 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java @@ -20,11 +20,19 @@ import org.apache.ignite.IgniteException; import java.util.LinkedList; +import java.util.regex.Pattern; /** * ODBC escape sequence parse. */ public class OdbcEscapeUtils { + + /** + * GUID regexp pattern: '12345678-9abc-def0-1234-123456789abc' + */ + private static final Pattern GUID_PATTERN = + Pattern.compile("^'\\p{XDigit}{8}-\\p{XDigit}{4}-\\p{XDigit}{4}-\\p{XDigit}{4}-\\p{XDigit}{12}'$"); + /** * Parse escape sequence. * @@ -145,14 +153,11 @@ private static String parseExpression(String text, int startPos, int len) { OdbcEscapeToken token = parseToken(text, startPos, len); - switch (token.type()) { - case SCALAR_FUNCTION: - return parseScalarExpression(text, startPos, len, token); - - default: - throw new IgniteException("Unsupported escape sequence token [text=" + - substring(text, startPos, len) + ", token=" + token.type().body() + ']'); - } + if (token.type().standard()) + return parseStandardExpression(text, startPos, len, token); + else + throw new IgniteException("Unsupported escape sequence token [text=" + + substring(text, startPos, len) + ", token=" + token.type().body() + ']'); } else { // Nothing to escape, return original string. @@ -191,7 +196,7 @@ private static OdbcEscapeToken parseToken(String text, int startPos, int len) { else { empty = (startPos + len == pos + 1); - if (!empty && typ.delimited()) { + if (!empty && typ.standard()) { char charAfter = text.charAt(pos); if (!Character.isWhitespace(charAfter)) @@ -216,21 +221,61 @@ private static OdbcEscapeToken parseToken(String text, int startPos, int len) { } /** - * Parse concrete expression. + * Parse standard token. * * @param text Text. * @param startPos Start position. * @param len Length. * @param token Token. - * @return Parsed expression. + * @return Result. */ - private static String parseScalarExpression(String text, int startPos, int len, OdbcEscapeToken token) { + private static String parseStandardExpression(String text, int startPos, int len, OdbcEscapeToken token) { assert validSubstring(text, startPos, len); + // Get expression borders. int startPos0 = startPos + 1 /* open brace */ + token.length() /* token. */; int len0 = len - 1 /* open brace */ - token.length() /* token */ - 1 /* close brace */; - return substring(text, startPos0, len0).trim(); + switch (token.type()) { + case SCALAR_FUNCTION: + return parseScalarExpression(text, startPos0, len0); + + case GUID: + return parseGuidExpression(text, startPos0, len0); + + default: + throw new IgniteException("Unsupported escape sequence token [text=" + + substring(text, startPos, len) + ", token=" + token.type().body() + ']'); + } + } + + /** + * Parse scalar function expression. + * + * @param text Text. + * @param startPos Start position. + * @param len Length. + * @return Parsed expression. + */ + private static String parseScalarExpression(String text, int startPos, int len) { + return substring(text, startPos, len).trim(); + } + + /** + * Parse concrete expression. + * + * @param text Text. + * @param startPos Start position. + * @param len Length. + * @return Parsed expression. + */ + private static String parseGuidExpression(String text, int startPos, int len) { + String val = substring(text, startPos, len).trim(); + + if (!GUID_PATTERN.matcher(val).matches()) + throw new IgniteException("Invalid GUID escape sequence: " + substring(text, startPos, len)); + + return val; } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java index d9be6cc703c19..7225c1a75a5d8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java @@ -25,18 +25,23 @@ import java.util.concurrent.Callable; /** - * Scalar function escape sequence parser tests. + * Escape sequence parser tests. */ public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest { /** * Test simple cases. */ - public void testSimple() { + public void testTrivial() { check( "select * from table;", "select * from table;" ); + } + /** + * Test escape sequence series. + */ + public void testSimpleFunction() throws Exception { check( "test()", "{fn test()}" @@ -51,12 +56,7 @@ public void testSimple() { "select test() from table;", "select {fn test()} from table;" ); - } - /** - * Test escape sequence series. - */ - public void testSimpleFunction() throws Exception { check( "func(field1) func(field2)", "{fn func(field1)} {fn func(field2)}" @@ -139,20 +139,15 @@ public void testNestedFunctionMixed() { } /** - * Test non-closed escape sequence. + * Test invalid escape sequence. */ - public void testFailedOnNonClosedEscapeSequence() { + public void testFailedOnInvalidFunctionSequence() { checkFail("select {fn func1(field1, {fn func2(field2), field3)} from SomeTable;"); - } - /** - * Test closing undeclared escape sequence. - */ - public void testFailedOnClosingNotOpenedSequence() { - checkFail("select {fn func1(field1, func2(field2)}, field3)} from SomeTable;"); + checkFail("select {fn func1(field1, fn func2(field2)}, field3)} from SomeTable;"); } - /** + /** * Test escape sequences with additional whitespace characters */ public void testFunctionEscapeSequenceWithWhitespaces() throws Exception { @@ -165,6 +160,65 @@ public void testFunctionEscapeSequenceWithWhitespaces() throws Exception { checkFail("{ \n func1()}"); } + /** + * Test guid escape sequences + */ + public void testGuidEscapeSequence() { + check( + "'12345678-9abc-def0-1234-123456789abc'", + "{guid '12345678-9abc-def0-1234-123456789abc'}" + ); + + check( + "select '12345678-9abc-def0-1234-123456789abc' from SomeTable;", + "select {guid '12345678-9abc-def0-1234-123456789abc'} from SomeTable;" + ); + + check( + "select '12345678-9abc-def0-1234-123456789abc'", + "select {guid '12345678-9abc-def0-1234-123456789abc'}" + ); + + checkFail("select {guid '1234567-1234-1234-1234-123456789abc'}"); + + checkFail("select {guid '1234567-8123-4123-4123-4123456789abc'}"); + + checkFail("select {guid '12345678-9abc-defg-1234-123456789abc'}"); + + checkFail("select {guid '12345678-12345678-1234-1234-1234-123456789abc'}"); + + checkFail("select {guid '12345678-1234-1234-1234-123456789abcdef'}"); + } + + /** + * Test invalid escape sequence. + */ + public void testFailedOnInvalidGuidSequence() { + checkFail("select {guid '12345678-9abc-def0-1234-123456789abc' from SomeTable;"); + + checkFail("select guid '12345678-9abc-def0-1234-123456789abc'} from SomeTable;"); + } + + /** + * Test escape sequences with additional whitespace characters + */ + public void testGuidEscapeSequenceWithWhitespaces() throws Exception { + check( + "'12345678-9abc-def0-1234-123456789abc'", + "{ guid '12345678-9abc-def0-1234-123456789abc'}" + ); + + check( + "'12345678-9abc-def0-1234-123456789abc'", + "{ guid '12345678-9abc-def0-1234-123456789abc'}" + ); + + check( + "'12345678-9abc-def0-1234-123456789abc'", + "{ \n guid\n'12345678-9abc-def0-1234-123456789abc'}" + ); + } + /** * Check parsing logic. * From 6fd53ea5b50148e5a1156d83ea28acb8faf84035 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Fri, 26 Aug 2016 11:19:39 +0300 Subject: [PATCH 064/487] IGNITE-3761: ODBC: Added tests for SQL_SQL92_VALUE_EXPRESSIONS. This closes #989. --- modules/platforms/cpp/odbc-test/Makefile.am | 1 + .../odbc-test/project/vs/odbc-test.vcxproj | 1 + .../project/vs/odbc-test.vcxproj.filters | 3 + .../src/sql_function_test_suite_fixture.cpp | 7 +- .../src/sql_system_functions_test.cpp | 21 ----- .../odbc-test/src/sql_value_expressions.cpp | 94 +++++++++++++++++++ modules/platforms/cpp/odbc/src/utility.cpp | 14 +-- 7 files changed, 112 insertions(+), 29 deletions(-) create mode 100644 modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index 1e00b6f3547f3..1f641bea43d3c 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -67,6 +67,7 @@ ignite_odbc_tests_SOURCES = \ src/sql_numeric_functions_test.cpp \ src/sql_aggregate_functions_test.cpp \ src/sql_system_functions_test.cpp \ + src/sql_value_expressions.cpp \ ../odbc/src/cursor.cpp \ ../odbc/src/config/connection_info.cpp \ ../odbc/src/app/application_data_buffer.cpp \ diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index 2a94a2a541738..4d3a9ccc8bd7a 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -173,6 +173,7 @@ + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index 9ca51af62b450..c4d70b7e67aa7 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -94,6 +94,9 @@ Code + + Code + diff --git a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp index f848dc5572efe..4d8b919a2ba54 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp @@ -139,7 +139,12 @@ namespace ignite CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen); - BOOST_CHECK_EQUAL(std::string(reinterpret_cast(res), static_cast(resLen)), expected); + std::string actual; + + if (resLen > 0) + actual.assign(reinterpret_cast(res), static_cast(resLen)); + + BOOST_CHECK_EQUAL(actual, expected); } template<> diff --git a/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp index d369b0f0c78e8..752925d974ba5 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp @@ -15,39 +15,18 @@ * limitations under the License. */ -#ifdef _WIN32 -# include -#endif - -#include -#include - -#include -#include - #ifndef _MSC_VER # define BOOST_TEST_DYN_LINK #endif #include -#include "ignite/ignite.h" -#include "ignite/ignition.h" -#include "ignite/impl/binary/binary_utils.h" - -#include "test_type.h" -#include "test_utils.h" #include "sql_function_test_suite_fixture.h" using namespace ignite; -using namespace ignite::cache; -using namespace ignite::cache::query; -using namespace ignite::common; using namespace boost::unit_test; -using ignite::impl::binary::BinaryUtils; - BOOST_FIXTURE_TEST_SUITE(SqlSystemFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture) BOOST_AUTO_TEST_CASE(TestSystemFunctionDatabase) diff --git a/modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp b/modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp new file mode 100644 index 0000000000000..e7d6598253a2f --- /dev/null +++ b/modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp @@ -0,0 +1,94 @@ +/* + * 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. + */ + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include "sql_function_test_suite_fixture.h" + +using namespace ignite; + +using namespace boost::unit_test; + +BOOST_FIXTURE_TEST_SUITE(SqlValueExpressionTestSuite, ignite::SqlFunctionTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(TestCase) +{ + TestType in; + + in.i32Field = 82; + + testCache.Put(1, in); + + CheckSingleResult( + "SELECT " + "CASE i32Field WHEN 82 " + "THEN (i32Field / 2) " + "ELSE (i32Field / 3) " + "END " + "FROM TestType", in.i32Field / 2); + + + CheckSingleResult( + "SELECT " + "CASE i32Field WHEN 22 " + "THEN (i32Field / 2) " + "ELSE (i32Field / 3) " + "END " + "FROM TestType", in.i32Field / 3);; +} + +BOOST_AUTO_TEST_CASE(TestCast) +{ + TestType in; + + in.i32Field = 12345; + in.strField = "54321"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT CAST(strField AS INT) + i32Field FROM TestType", + common::LexicalCast(in.strField) + in.i32Field); + + CheckSingleResult("SELECT CAST(i32Field AS VARCHAR) || strField FROM TestType", + common::LexicalCast(in.i32Field) + in.strField); +} + +BOOST_AUTO_TEST_CASE(TestCoalesce) +{ + CheckSingleResult("SELECT COALESCE('One', 'Two', 'Three')", "One"); + CheckSingleResult("SELECT COALESCE(NULL, 'Two', 'Three')", "Two"); + CheckSingleResult("SELECT COALESCE(NULL, 'Two', NULL)", "Two"); + CheckSingleResult("SELECT COALESCE(NULL, NULL, 'Three')", "Three"); +} + +BOOST_AUTO_TEST_CASE(TestNullif) +{ + TestType in; + + in.strField = "SomeValue"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT NULLIF(strField, 'blablabla') FROM TestType", in.strField); + CheckSingleResult("SELECT NULLIF(strField, 'SomeValue') FROM TestType", ""); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/src/utility.cpp b/modules/platforms/cpp/odbc/src/utility.cpp index 133f059348220..19be799a10aae 100644 --- a/modules/platforms/cpp/odbc/src/utility.cpp +++ b/modules/platforms/cpp/odbc/src/utility.cpp @@ -56,19 +56,19 @@ namespace ignite void ReadString(ignite::impl::binary::BinaryReaderImpl& reader, std::string& str) { int32_t strLen = reader.ReadString(0, 0); - if (!strLen) + if (strLen > 0) { - str.clear(); - - char dummy; + str.resize(strLen); - reader.ReadString(&dummy, sizeof(dummy)); + reader.ReadString(&str[0], static_cast(str.size())); } else { - str.resize(strLen); + str.clear(); - reader.ReadString(&str[0], static_cast(str.size())); + char dummy; + + reader.ReadString(&dummy, sizeof(dummy)); } } From 99e3e8a2d997aa681264460c2845984712ded90e Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Fri, 26 Aug 2016 11:23:49 +0300 Subject: [PATCH 065/487] IGNITE-3764: ODBC Added tests for SQL operators. This closes #986. --- .../cpp/common/src/common/big_integer.cpp | 1 - modules/platforms/cpp/odbc-test/Makefile.am | 1 + .../include/sql_function_test_suite_fixture.h | 3 + .../odbc-test/project/vs/odbc-test.vcxproj | 1 + .../project/vs/odbc-test.vcxproj.filters | 3 + .../src/sql_function_test_suite_fixture.cpp | 10 + .../cpp/odbc-test/src/sql_operators_test.cpp | 214 ++++++++++++++++++ 7 files changed, 232 insertions(+), 1 deletion(-) create mode 100644 modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp diff --git a/modules/platforms/cpp/common/src/common/big_integer.cpp b/modules/platforms/cpp/common/src/common/big_integer.cpp index 475ddc6bc3661..6aa6614fe5d72 100644 --- a/modules/platforms/cpp/common/src/common/big_integer.cpp +++ b/modules/platforms/cpp/common/src/common/big_integer.cpp @@ -58,7 +58,6 @@ namespace ignite assert(val != 0); assert(len > 0); assert(sign == 1 || sign == 0 || sign == -1); - assert(val[0] != 0); if (bigEndian) { diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index 1f641bea43d3c..a3f954efc05b6 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -67,6 +67,7 @@ ignite_odbc_tests_SOURCES = \ src/sql_numeric_functions_test.cpp \ src/sql_aggregate_functions_test.cpp \ src/sql_system_functions_test.cpp \ + src/sql_operators_test.cpp \ src/sql_value_expressions.cpp \ ../odbc/src/cursor.cpp \ ../odbc/src/config/connection_info.cpp \ diff --git a/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h b/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h index eef0a9b3b0b9d..57c3f0f367355 100644 --- a/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h +++ b/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h @@ -160,6 +160,9 @@ namespace ignite template<> void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const double& expected); + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const bool& expected); + template<> void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index 4d3a9ccc8bd7a..8b4bee66d197b 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -171,6 +171,7 @@ + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index c4d70b7e67aa7..4756f8328a957 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -94,6 +94,9 @@ Code + + Code + Code diff --git a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp index 4d8b919a2ba54..d6b175fff04b2 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp @@ -191,6 +191,16 @@ namespace ignite BOOST_CHECK_CLOSE(static_cast(res), expected, 1E-6); } + template<> + void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const bool& expected) + { + SQLCHAR res = 0; + + CheckSingleResult0(request, SQL_C_BIT, &res, 0, 0); + + BOOST_CHECK_EQUAL((res != 0), expected); + } + template<> void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) { diff --git a/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp new file mode 100644 index 0000000000000..db3d3f7bafc1e --- /dev/null +++ b/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp @@ -0,0 +1,214 @@ +/* + * 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. + */ + +#ifdef _WIN32 +# include +#endif + +#include +#include + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include "test_type.h" +#include "sql_function_test_suite_fixture.h" +#include + +using namespace ignite; + +BOOST_FIXTURE_TEST_SUITE(SqlOperatorTestSuite, ignite::SqlFunctionTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(TestOperatorAddInt) +{ + CheckSingleResult("SELECT 123 + 51", 123 + 51); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorSubInt) +{ + CheckSingleResult("SELECT 123 - 51", 123 - 51); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorDivInt) +{ + CheckSingleResult("SELECT 123 / 51", 123 / 51); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorModInt) +{ + CheckSingleResult("SELECT 123 % 51", 123 % 51); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorMultInt) +{ + CheckSingleResult("SELECT 123 * 51", 123 * 51); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorAddDouble) +{ + CheckSingleResult("SELECT 123.0 + 51.0", 123.0 + 51.0); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorSubDouble) +{ + CheckSingleResult("SELECT 123.0 - 51.0", 123.0 - 51.0); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorDivDouble) +{ + CheckSingleResult("SELECT 123.0 / 51.0", 123.0 / 51.0); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorModDouble) +{ + CheckSingleResult("SELECT 123.0 % 51.0", 123 % 51); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorMultDouble) +{ + CheckSingleResult("SELECT 123.0 * 51.0", 123.0 * 51.0); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorConcatString) +{ + CheckSingleResult("SELECT 'Hello' || ' ' || 'World' || '!'", "Hello World!"); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorGreaterInt) +{ + CheckSingleResult("SELECT 2 > 3", false); + CheckSingleResult("SELECT 3 > 3", false); + CheckSingleResult("SELECT 34 > 3", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorLessInt) +{ + CheckSingleResult("SELECT 4 < 4", false); + CheckSingleResult("SELECT 4 < 4", false); + CheckSingleResult("SELECT 8 < 42", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorGreaterEquealInt) +{ + CheckSingleResult("SELECT 2 >= 3", false); + CheckSingleResult("SELECT 3 >= 3", true); + CheckSingleResult("SELECT 34 >= 3", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorLessEquealInt) +{ + CheckSingleResult("SELECT 4 <= 3", false); + CheckSingleResult("SELECT 4 <= 4", true); + CheckSingleResult("SELECT 8 <= 42", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorEquealInt) +{ + CheckSingleResult("SELECT 4 = 3", false); + CheckSingleResult("SELECT 4 = 4", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorNotEquealInt) +{ + CheckSingleResult("SELECT 4 != 3", true); + CheckSingleResult("SELECT 4 != 4", false); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorGreaterDouble) +{ + CheckSingleResult("SELECT 2 > 3", false); + CheckSingleResult("SELECT 3 > 3", false); + CheckSingleResult("SELECT 34 > 3", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorLessDouble) +{ + CheckSingleResult("SELECT 4.0 < 4.0", false); + CheckSingleResult("SELECT 4.0 < 4.0", false); + CheckSingleResult("SELECT 8.0 < 42.0", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorGreaterEquealDouble) +{ + CheckSingleResult("SELECT 2.0 >= 3.0", false); + CheckSingleResult("SELECT 3.0 >= 3.0", true); + CheckSingleResult("SELECT 34.0 >= 3.0", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorLessEquealDouble) +{ + CheckSingleResult("SELECT 4.0 <= 3.0", false); + CheckSingleResult("SELECT 4.0 <= 4.0", true); + CheckSingleResult("SELECT 8.0 <= 42.0", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorEquealDouble) +{ + CheckSingleResult("SELECT 4.0 = 3.0", false); + CheckSingleResult("SELECT 4.0 = 4.0", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorNotEquealDouble) +{ + CheckSingleResult("SELECT 4.0 != 3.0", true); + CheckSingleResult("SELECT 4.0 != 4.0", false); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorGreaterString) +{ + CheckSingleResult("SELECT 'abc' > 'bcd'", false); + CheckSingleResult("SELECT 'abc' > 'abc'", false); + CheckSingleResult("SELECT 'bcd' > 'abc'", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorLessString) +{ + CheckSingleResult("SELECT 'bcd' < 'abc'", false); + CheckSingleResult("SELECT 'abc' < 'abc'", false); + CheckSingleResult("SELECT 'abc' < 'bcd'", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorGreaterEquealString) +{ + CheckSingleResult("SELECT 'abc' >= 'bcd'", false); + CheckSingleResult("SELECT 'abc' >= 'abc'", true); + CheckSingleResult("SELECT 'bcd' >= 'abc'", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorLessEquealString) +{ + CheckSingleResult("SELECT 'bcd' <= 'abc'", false); + CheckSingleResult("SELECT 'abc' <= 'bcd'", true); + CheckSingleResult("SELECT 'abc' <= 'abc'", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorEquealString) +{ + CheckSingleResult("SELECT 'bcd' = 'abc'", false); + CheckSingleResult("SELECT 'abc' = 'abc'", true); +}; + +BOOST_AUTO_TEST_CASE(TestOperatorNotEquealString) +{ + CheckSingleResult("SELECT 'abc' != 'abc'", false); + CheckSingleResult("SELECT 'bcd' != 'abc'", true); +}; + +BOOST_AUTO_TEST_SUITE_END() From 87a1928a4f90b4f8a221041cfff9d22e3dd801cc Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Fri, 26 Aug 2016 15:22:15 +0300 Subject: [PATCH 066/487] IGNITE-3776: Removed code duplication in GridNearAtomicAbstractUpdateFuture. --- .../GridNearAtomicAbstractUpdateFuture.java | 69 ++++++++++++++++++ .../GridNearAtomicSingleUpdateFuture.java | 70 +------------------ .../atomic/GridNearAtomicUpdateFuture.java | 68 +----------------- 3 files changed, 73 insertions(+), 134 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java index 3e69c0298d139..85751bb1ba66f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -29,6 +30,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheReturn; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.CI2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; @@ -246,4 +248,71 @@ public void map() { protected boolean storeFuture() { return cctx.config().getAtomicWriteOrderMode() == CLOCK || syncMode != FULL_ASYNC; } + + /** + * Maps future to single node. + * + * @param nodeId Node ID. + * @param req Request. + */ + protected void mapSingle(UUID nodeId, GridNearAtomicUpdateRequest req) { + if (cctx.localNodeId().equals(nodeId)) { + cache.updateAllAsyncInternal(nodeId, req, + new CI2() { + @Override public void apply(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateResponse res) { + onResult(res.nodeId(), res, false); + } + }); + } + else { + try { + cctx.io().send(req.nodeId(), req, cctx.ioPolicy()); + + if (msgLog.isDebugEnabled()) { + msgLog.debug("Near update fut, sent request [futId=" + req.futureVersion() + + ", writeVer=" + req.updateVersion() + + ", node=" + req.nodeId() + ']'); + } + + if (syncMode == FULL_ASYNC) + onDone(new GridCacheReturn(cctx, true, true, null, true)); + } + catch (IgniteCheckedException e) { + if (msgLog.isDebugEnabled()) { + msgLog.debug("Near update fut, failed to send request [futId=" + req.futureVersion() + + ", writeVer=" + req.updateVersion() + + ", node=" + req.nodeId() + + ", err=" + e + ']'); + } + + onSendError(req, e); + } + } + } + + /** + * Response callback. + * + * @param nodeId Node ID. + * @param res Update response. + * @param nodeErr {@code True} if response was created on node failure. + */ + public abstract void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr); + + /** + * @param req Request. + * @param e Error. + */ + protected void onSendError(GridNearAtomicUpdateRequest req, IgniteCheckedException e) { + synchronized (mux) { + GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(), + req.nodeId(), + req.futureVersion(), + cctx.deploymentEnabled()); + + res.addFailedKeys(req.keys(), e); + + onResult(req.nodeId(), res, true); + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java index aad4186f9f7b7..661a178435e1d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java @@ -37,7 +37,6 @@ import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.CI1; -import org.apache.ignite.internal.util.typedef.CI2; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; @@ -52,7 +51,6 @@ import java.util.UUID; import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK; -import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM; /** @@ -191,15 +189,9 @@ public GridNearAtomicSingleUpdateFuture( return false; } - /** - * Response callback. - * - * @param nodeId Node ID. - * @param res Update response. - * @param nodeErr {@code True} if response was created on node failure. - */ + /** {@inheritDoc} */ @SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"}) - public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) { + @Override public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) { GridNearAtomicUpdateRequest req; AffinityTopologyVersion remapTopVer = null; @@ -441,64 +433,6 @@ private void updateNear(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateRes map(topVer); } - /** - * Maps future to single node. - * - * @param nodeId Node ID. - * @param req Request. - */ - private void mapSingle(UUID nodeId, GridNearAtomicUpdateRequest req) { - if (cctx.localNodeId().equals(nodeId)) { - cache.updateAllAsyncInternal(nodeId, req, - new CI2() { - @Override public void apply(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateResponse res) { - onResult(res.nodeId(), res, false); - } - }); - } - else { - try { - cctx.io().send(req.nodeId(), req, cctx.ioPolicy()); - - if (msgLog.isDebugEnabled()) { - msgLog.debug("Near update single fut, sent request [futId=" + req.futureVersion() + - ", writeVer=" + req.updateVersion() + - ", node=" + req.nodeId() + ']'); - } - - if (syncMode == FULL_ASYNC) - onDone(new GridCacheReturn(cctx, true, true, null, true)); - } - catch (IgniteCheckedException e) { - if (msgLog.isDebugEnabled()) { - msgLog.debug("Near update single fut, failed to send request [futId=" + req.futureVersion() + - ", writeVer=" + req.updateVersion() + - ", node=" + req.nodeId() + - ", err=" + e + ']'); - } - - onSendError(req, e); - } - } - } - - /** - * @param req Request. - * @param e Error. - */ - void onSendError(GridNearAtomicUpdateRequest req, IgniteCheckedException e) { - synchronized (mux) { - GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(), - req.nodeId(), - req.futureVersion(), - cctx.deploymentEnabled()); - - res.addFailedKeys(req.keys(), e); - - onResult(req.nodeId(), res, true); - } - } - /** {@inheritDoc} */ protected void map(AffinityTopologyVersion topVer) { Collection topNodes = CU.affinityNodes(cctx, topVer); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java index 0d88ef8f4a6cf..2432f6355e1cf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java @@ -253,15 +253,9 @@ public GridNearAtomicUpdateFuture( return false; } - /** - * Response callback. - * - * @param nodeId Node ID. - * @param res Update response. - * @param nodeErr {@code True} if response was created on node failure. - */ + /** {@inheritDoc} */ @SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"}) - public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) { + @Override public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) { GridNearAtomicUpdateRequest req; AffinityTopologyVersion remapTopVer = null; @@ -551,47 +545,6 @@ private void updateNear(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateRes map(topVer, null); } - /** - * Maps future to single node. - * - * @param nodeId Node ID. - * @param req Request. - */ - private void mapSingle(UUID nodeId, GridNearAtomicUpdateRequest req) { - if (cctx.localNodeId().equals(nodeId)) { - cache.updateAllAsyncInternal(nodeId, req, - new CI2() { - @Override public void apply(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateResponse res) { - onResult(res.nodeId(), res, false); - } - }); - } - else { - try { - cctx.io().send(req.nodeId(), req, cctx.ioPolicy()); - - if (msgLog.isDebugEnabled()) { - msgLog.debug("Near update fut, sent request [futId=" + req.futureVersion() + - ", writeVer=" + req.updateVersion() + - ", node=" + req.nodeId() + ']'); - } - - if (syncMode == FULL_ASYNC) - onDone(new GridCacheReturn(cctx, true, true, null, true)); - } - catch (IgniteCheckedException e) { - if (msgLog.isDebugEnabled()) { - msgLog.debug("Near update fut, failed to send request [futId=" + req.futureVersion() + - ", writeVer=" + req.updateVersion() + - ", node=" + req.nodeId() + - ", err=" + e + ']'); - } - - onSendError(req, e); - } - } - } - /** * Sends messages to remote nodes and updates local cache. * @@ -646,23 +599,6 @@ private void doUpdate(Map mappings) { onDone(new GridCacheReturn(cctx, true, true, null, true)); } - /** - * @param req Request. - * @param e Error. - */ - void onSendError(GridNearAtomicUpdateRequest req, IgniteCheckedException e) { - synchronized (mux) { - GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(), - req.nodeId(), - req.futureVersion(), - cctx.deploymentEnabled()); - - res.addFailedKeys(req.keys(), e); - - onResult(req.nodeId(), res, true); - } - } - /** {@inheritDoc} */ protected void map(AffinityTopologyVersion topVer) { map(topVer, null); From 92f18bf353cc8c3821c6500ce9f1cd397a7cf17c Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 26 Aug 2016 15:31:30 +0300 Subject: [PATCH 067/487] IGNITE-3745: ODBC: Implemented date/time/timestamp escape sequence parsing. This closes #991. --- .../odbc/escape/OdbcEscapeUtils.java | 30 ++- .../odbc/OdbcEscapeSequenceSelfTest.java | 191 +++++++++++++++++- 2 files changed, 203 insertions(+), 18 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java index 83ec9d8eee847..4c1deb6110772 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java @@ -26,10 +26,17 @@ * ODBC escape sequence parse. */ public class OdbcEscapeUtils { + /** Odbc date regexp pattern: '2016-08-23' */ + private static final Pattern DATE_PATTERN = Pattern.compile("^'\\d{4}-\\d{2}-\\d{2}'$"); - /** - * GUID regexp pattern: '12345678-9abc-def0-1234-123456789abc' - */ + /** Odbc time regexp pattern: '14:33:44' */ + private static final Pattern TIME_PATTERN = Pattern.compile("^'\\d{2}:\\d{2}:\\d{2}'$"); + + /** Odbc timestamp regexp pattern: '2016-08-23 14:33:44.12345' */ + private static final Pattern TIMESTAMP_PATTERN = + Pattern.compile("^'\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}(\\.\\d+)?'$"); + + /** GUID regexp pattern: '12345678-9abc-def0-1234-123456789abc' */ private static final Pattern GUID_PATTERN = Pattern.compile("^'\\p{XDigit}{8}-\\p{XDigit}{4}-\\p{XDigit}{4}-\\p{XDigit}{4}-\\p{XDigit}{12}'$"); @@ -241,7 +248,16 @@ private static String parseStandardExpression(String text, int startPos, int len return parseScalarExpression(text, startPos0, len0); case GUID: - return parseGuidExpression(text, startPos0, len0); + return parseExpression(text, startPos0, len0, token.type(), GUID_PATTERN); + + case DATE: + return parseExpression(text, startPos0, len0, token.type(), DATE_PATTERN); + + case TIME: + return parseExpression(text, startPos0, len0, token.type(), TIME_PATTERN); + + case TIMESTAMP: + return parseExpression(text, startPos0, len0, token.type(), TIMESTAMP_PATTERN); default: throw new IgniteException("Unsupported escape sequence token [text=" + @@ -269,11 +285,11 @@ private static String parseScalarExpression(String text, int startPos, int len) * @param len Length. * @return Parsed expression. */ - private static String parseGuidExpression(String text, int startPos, int len) { + private static String parseExpression(String text, int startPos, int len, OdbcEscapeType type, Pattern pattern) { String val = substring(text, startPos, len).trim(); - if (!GUID_PATTERN.matcher(val).matches()) - throw new IgniteException("Invalid GUID escape sequence: " + substring(text, startPos, len)); + if (!pattern.matcher(val).matches()) + throw new IgniteException("Invalid " + type + " escape sequence: " + substring(text, startPos, len)); return val; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java index 7225c1a75a5d8..1aa90fd07e958 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java @@ -142,12 +142,14 @@ public void testNestedFunctionMixed() { * Test invalid escape sequence. */ public void testFailedOnInvalidFunctionSequence() { + checkFail("{fnfunc1()}"); + checkFail("select {fn func1(field1, {fn func2(field2), field3)} from SomeTable;"); checkFail("select {fn func1(field1, fn func2(field2)}, field3)} from SomeTable;"); } - /** + /** * Test escape sequences with additional whitespace characters */ public void testFunctionEscapeSequenceWithWhitespaces() throws Exception { @@ -155,7 +157,7 @@ public void testFunctionEscapeSequenceWithWhitespaces() throws Exception { check("func1()", "{ fn func1()}"); - check("func1()", "{ \n fn func1()}"); + check("func1()", "{ \n fn\nfunc1()}"); checkFail("{ \n func1()}"); } @@ -178,6 +180,21 @@ public void testGuidEscapeSequence() { "select '12345678-9abc-def0-1234-123456789abc'", "select {guid '12345678-9abc-def0-1234-123456789abc'}" ); + } + + /** + * Test invalid escape sequence. + */ + public void testFailedOnInvalidGuidSequence() { + checkFail("select {guid'12345678-9abc-def0-1234-123456789abc'}"); + + checkFail("select {guid 12345678-9abc-def0-1234-123456789abc'}"); + + checkFail("select {guid '12345678-9abc-def0-1234-123456789abc}"); + + checkFail("select {guid '12345678-9abc-def0-1234-123456789abc' from SomeTable;"); + + checkFail("select guid '12345678-9abc-def0-1234-123456789abc'} from SomeTable;"); checkFail("select {guid '1234567-1234-1234-1234-123456789abc'}"); @@ -190,15 +207,6 @@ public void testGuidEscapeSequence() { checkFail("select {guid '12345678-1234-1234-1234-123456789abcdef'}"); } - /** - * Test invalid escape sequence. - */ - public void testFailedOnInvalidGuidSequence() { - checkFail("select {guid '12345678-9abc-def0-1234-123456789abc' from SomeTable;"); - - checkFail("select guid '12345678-9abc-def0-1234-123456789abc'} from SomeTable;"); - } - /** * Test escape sequences with additional whitespace characters */ @@ -219,6 +227,167 @@ public void testGuidEscapeSequenceWithWhitespaces() throws Exception { ); } + /** + * Test date escape sequences + */ + public void testDateEscapeSequence() throws Exception { + check( + "'2016-08-26'", + "{d '2016-08-26'}" + ); + + check( + "select '2016-08-26'", + "select {d '2016-08-26'}" + ); + + check( + "select '2016-08-26' from table;", + "select {d '2016-08-26'} from table;" + ); + } + + /** + * Test date escape sequences with additional whitespace characters + */ + public void testDateEscapeSequenceWithWhitespaces() throws Exception { + check("'2016-08-26'", "{ d '2016-08-26'}"); + + check("'2016-08-26'", "{ d '2016-08-26'}"); + + check("'2016-08-26'", "{ \n d\n'2016-08-26'}"); + } + + /** + * Test invalid escape sequence. + */ + public void testFailedOnInvalidDateSequence() { + checkFail("{d'2016-08-26'}"); + + checkFail("{d 2016-08-26'}"); + + checkFail("{d '2016-08-26}"); + + checkFail("{d '16-08-26'}"); + + checkFail("{d '2016/08/02'}"); + + checkFail("select {d '2016-08-26' from table;"); + + checkFail("select {}d '2016-08-26'} from table;"); + } + + /** + * Test date escape sequences + */ + public void testTimeEscapeSequence() throws Exception { + check("'13:15:08'", "{t '13:15:08'}"); + + check("select '13:15:08'", "select {t '13:15:08'}"); + + check("select '13:15:08' from table;", "select {t '13:15:08'} from table;" + ); + } + + /** + * Test date escape sequences with additional whitespace characters + */ + public void testTimeEscapeSequenceWithWhitespaces() throws Exception { + check("'13:15:08'", "{ t '13:15:08'}"); + + check("'13:15:08'", "{ t '13:15:08'}"); + + check("'13:15:08'", "{ \n t\n'13:15:08'}"); + } + + /** + * Test invalid escape sequence. + */ + public void testFailedOnInvalidTimeSequence() { + checkFail("{t'13:15:08'}"); + + checkFail("{t 13:15:08'}"); + + checkFail("{t '13:15:08}"); + + checkFail("{t '13 15:08'}"); + + checkFail("{t '3:15:08'}"); + + checkFail("select {t '13:15:08' from table;"); + + checkFail("select {}t '13:15:08'} from table;"); + } + + /** + * Test timestamp escape sequences + */ + public void testTimestampEscapeSequence() throws Exception { + check( + "'2016-08-26 13:15:08'", + "{ts '2016-08-26 13:15:08'}" + ); + + check( + "'2016-08-26 13:15:08.123456'", + "{ts '2016-08-26 13:15:08.123456'}" + ); + + check( + "select '2016-08-26 13:15:08'", + "select {ts '2016-08-26 13:15:08'}" + ); + + check( + "select '2016-08-26 13:15:08' from table;", + "select {ts '2016-08-26 13:15:08'} from table;" + ); + } + + /** + * Test timestamp escape sequences with additional whitespace characters + */ + public void testTimestampEscapeSequenceWithWhitespaces() throws Exception { + check("'2016-08-26 13:15:08'", + "{ ts '2016-08-26 13:15:08'}" + ); + + check("'2016-08-26 13:15:08'", + "{ ts '2016-08-26 13:15:08'}" + ); + + check("'2016-08-26 13:15:08'", + "{ \n ts\n'2016-08-26 13:15:08'}" + ); + } + + /** + * Test invalid escape sequence. + */ + public void testFailedOnInvalidTimestampSequence() { + checkFail("{ts '2016-08-26 13:15:08,12345'}"); + + checkFail("{ts'2016-08-26 13:15:08'}"); + + checkFail("{ts 2016-08-26 13:15:08'}"); + + checkFail("{ts '2016-08-26 13:15:08}"); + + checkFail("{ts '16-08-26 13:15:08'}"); + + checkFail("{ts '2016-08-26 3:25:08'}"); + + checkFail("{ts '2016-08 26 03:25:08'}"); + + checkFail("{ts '2016-08-26 03 25:08'}"); + + checkFail("{t s '2016-08-26 13:15:08''}"); + + checkFail("select {ts '2016-08-26 13:15:08' from table;"); + + checkFail("select {}ts '2016-08-26 13:15:08'} from table;"); + } + /** * Check parsing logic. * From b5757642e135908d9baa027a605035dd0d4acfc9 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Fri, 26 Aug 2016 15:47:02 +0300 Subject: [PATCH 068/487] IGNITE-3670 IGFS: Improved symlink handling for delete operation and added more tests. This closes #975. --- .../local/LocalIgfsSecondaryFileSystem.java | 38 +- .../igfs/IgfsAbstractBaseSelfTest.java | 1067 +++++++++++++++++ .../processors/igfs/IgfsAbstractSelfTest.java | 1012 +--------------- ...condaryFileSystemDualAbstractSelfTest.java | 143 +++ 4 files changed, 1239 insertions(+), 1021 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java index 3d3a35065dd56..519f4728c5464 100644 --- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java +++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java @@ -41,6 +41,8 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.file.Files; +import java.nio.file.LinkOption; +import java.nio.file.attribute.BasicFileAttributes; import java.util.Collection; import java.util.Collections; import java.util.Map; @@ -108,35 +110,43 @@ else if(!srcFile.renameTo(destFile)) @Override public boolean delete(IgfsPath path, boolean recursive) { File f = fileForPath(path); - if (!recursive || !f.isDirectory()) + if (!recursive) return f.delete(); else - return deleteDirectory(f); + return deleteRecursive(f); } /** * Delete directory recursively. * - * @param dir Directory. + * @param f Directory. * @return {@code true} if successful. */ - private boolean deleteDirectory(File dir) { - File[] entries = dir.listFiles(); + private boolean deleteRecursive(File f) { + BasicFileAttributes attrs; + + try { + attrs = Files.readAttributes(f.toPath(), BasicFileAttributes.class, LinkOption.NOFOLLOW_LINKS); + } + catch (IOException ignore) { + return false; + } + + if (!attrs.isDirectory() || attrs.isSymbolicLink()) + return f.delete(); + + File[] entries = f.listFiles(); if (entries != null) { for (File entry : entries) { - if (entry.isDirectory()) - deleteDirectory(entry); - else if (entry.isFile()) { - if (!entry.delete()) - return false; - } - else - throw new UnsupportedOperationException("Symlink deletion is not yet supported: " + entry); + boolean res = deleteRecursive(entry); + + if (!res) + return false; } } - return dir.delete(); + return f.delete(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java new file mode 100644 index 0000000000000..9575bd04abd88 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java @@ -0,0 +1,1067 @@ +/* + * 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.igfs; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.lang.reflect.Field; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Callable; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteFileSystem; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.FileSystemConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.igfs.IgfsInputStream; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; +import org.apache.ignite.igfs.IgfsMode; +import org.apache.ignite.igfs.IgfsOutputStream; +import org.apache.ignite.igfs.IgfsPath; +import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.processors.cache.GridCacheAdapter; +import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.marshaller.optimized.OptimizedMarshaller; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheMode.REPLICATED; +import static org.apache.ignite.igfs.IgfsMode.PRIMARY; +import static org.apache.ignite.igfs.IgfsMode.PROXY; + +/** + * Test fo regular igfs operations. + */ +@SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "ConstantConditions"}) +public abstract class IgfsAbstractBaseSelfTest extends IgfsCommonAbstractTest { + /** IGFS block size. */ + protected static final int IGFS_BLOCK_SIZE = 512 * 1024; + + /** Default block size (32Mb). */ + protected static final long BLOCK_SIZE = 32 * 1024 * 1024; + + /** Default repeat count. */ + protected static final int REPEAT_CNT = 5; // Diagnostic: up to 500; Regression: 5 + + /** Concurrent operations count. */ + protected static final int OPS_CNT = 16; + + /** Renames count. */ + protected static final int RENAME_CNT = OPS_CNT; + + /** Deletes count. */ + protected static final int DELETE_CNT = OPS_CNT; + + /** Updates count. */ + protected static final int UPDATE_CNT = OPS_CNT; + + /** Mkdirs count. */ + protected static final int MKDIRS_CNT = OPS_CNT; + + /** Create count. */ + protected static final int CREATE_CNT = OPS_CNT; + + /** Time to wait until the caches get empty after format. */ + private static final long CACHE_EMPTY_TIMEOUT = 30_000L; + + /** Seed to generate random numbers. */ + protected static final long SEED = System.currentTimeMillis(); + + /** Amount of blocks to prefetch. */ + protected static final int PREFETCH_BLOCKS = 1; + + /** Amount of sequential block reads before prefetch is triggered. */ + protected static final int SEQ_READS_BEFORE_PREFETCH = 2; + + /** Primary file system REST endpoint configuration map. */ + protected static final IgfsIpcEndpointConfiguration PRIMARY_REST_CFG; + + /** Secondary file system REST endpoint configuration map. */ + protected static final IgfsIpcEndpointConfiguration SECONDARY_REST_CFG; + + /** Directory. */ + protected static final IgfsPath DIR = new IgfsPath("/dir"); + + /** Sub-directory. */ + protected static final IgfsPath SUBDIR = new IgfsPath(DIR, "subdir"); + + /** Another sub-directory in the same directory. */ + protected static final IgfsPath SUBDIR2 = new IgfsPath(DIR, "subdir2"); + + /** Sub-directory of the sub-directory. */ + protected static final IgfsPath SUBSUBDIR = new IgfsPath(SUBDIR, "subsubdir"); + + /** File. */ + protected static final IgfsPath FILE = new IgfsPath(SUBDIR, "file"); + + /** Another file in the same directory. */ + protected static final IgfsPath FILE2 = new IgfsPath(SUBDIR, "file2"); + + /** Other directory. */ + protected static final IgfsPath DIR_NEW = new IgfsPath("/dirNew"); + + /** Other subdirectory. */ + protected static final IgfsPath SUBDIR_NEW = new IgfsPath(DIR_NEW, "subdirNew"); + + /** Other sub-directory of the sub-directory. */ + protected static final IgfsPath SUBSUBDIR_NEW = new IgfsPath(SUBDIR_NEW, "subsubdirNew"); + + /** Other file. */ + protected static final IgfsPath FILE_NEW = new IgfsPath(SUBDIR_NEW, "fileNew"); + + /** Default data chunk (128 bytes). */ + protected static final byte[] chunk = createChunk(128); + + /** Primary IGFS. */ + protected static IgfsImpl igfs; + + /** Secondary IGFS */ + protected static IgfsSecondaryFileSystem igfsSecondaryFileSystem; + + /** Secondary file system lower layer "backdoor" wrapped in UniversalFileSystemAdapter: */ + protected static IgfsSecondaryFileSystemTestAdapter igfsSecondary; + + /** IGFS mode. */ + protected final IgfsMode mode; + + /** Dual mode flag. */ + protected final boolean dual; + + /** Memory mode. */ + protected final CacheMemoryMode memoryMode; + + /** IP finder for primary topology. */ + protected final TcpDiscoveryVmIpFinder primaryIpFinder = new TcpDiscoveryVmIpFinder(true); + + /** IP finder for secondary topology. */ + protected final TcpDiscoveryVmIpFinder secondaryIpFinder = new TcpDiscoveryVmIpFinder(true); + + /** Ignite nodes of cluster, excluding the secondary file system node, if any. */ + protected Ignite[] nodes; + + static { + PRIMARY_REST_CFG = new IgfsIpcEndpointConfiguration(); + + PRIMARY_REST_CFG.setType(IgfsIpcEndpointType.TCP); + PRIMARY_REST_CFG.setPort(10500); + + SECONDARY_REST_CFG = new IgfsIpcEndpointConfiguration(); + + SECONDARY_REST_CFG.setType(IgfsIpcEndpointType.TCP); + SECONDARY_REST_CFG.setPort(11500); + } + + /** + * Constructor. + * + * @param mode IGFS mode. + */ + protected IgfsAbstractBaseSelfTest(IgfsMode mode) { + this(mode, ONHEAP_TIERED); + } + + /** + * Constructor. + * + * @param mode IGFS mode. + * @param memoryMode Memory mode. + */ + protected IgfsAbstractBaseSelfTest(IgfsMode mode, CacheMemoryMode memoryMode) { + assert mode != null && mode != PROXY; + + this.mode = mode; + this.memoryMode = memoryMode; + + dual = mode != PRIMARY; + } + + /** + * @return Relaxed consistency flag. + */ + protected boolean relaxedConsistency() { + return false; + } + + /** + * @return Relaxed consistency flag. + */ + protected boolean initializeDefaultPathModes() { + return false; + } + + /** + * @return Client flag. + */ + protected boolean client() { + return false; + } + + /** + * @return Use optimized marshaller flag. + */ + protected boolean useOptimizedMarshaller() { + return false; + } + + /** + * @return Whether append is supported. + */ + protected boolean appendSupported() { + return true; + } + + /** + * @return Whether permissions are supported. + */ + protected boolean permissionsSupported() { + return true; + } + + /** + * @return Whether properties are supported. + */ + protected boolean propertiesSupported() { + return true; + } + + /** + * @return Whether times are supported. + */ + protected boolean timesSupported() { + return true; + } + + /** + * @return Amount of nodes to start. + */ + protected int nodeCount() { + return 1; + } + + /** + * Data chunk. + * + * @param len Length. + * @return Data chunk. + */ + static byte[] createChunk(int len) { + byte[] chunk = new byte[len]; + + for (int i = 0; i < chunk.length; i++) + chunk[i] = (byte)i; + + return chunk; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + igfsSecondaryFileSystem = createSecondaryFileSystemStack(); + + nodes = new Ignite[nodeCount()]; + + for (int i = 0; i < nodes.length; i++) { + String nodeName = i == 0 ? "ignite" : "ignite" + i; + + nodes[i] = startGridWithIgfs(nodeName, "igfs", mode, igfsSecondaryFileSystem, PRIMARY_REST_CFG, + primaryIpFinder); + } + + igfs = (IgfsImpl) nodes[0].fileSystem("igfs"); + + if (client()) { + // Start client. + Ignition.setClientMode(true); + + try { + Ignite ignite = startGridWithIgfs("ignite-client", "igfs", mode, igfsSecondaryFileSystem, + PRIMARY_REST_CFG, primaryIpFinder); + + igfs = (IgfsImpl) ignite.fileSystem("igfs"); + } + finally { + Ignition.setClientMode(false); + } + } + } + + /** + * Creates secondary file system stack. + * + * @return The secondary file system. + * @throws Exception On error. + */ + protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception { + Ignite igniteSecondary = startGridWithIgfs("ignite-secondary", "igfs-secondary", PRIMARY, null, + SECONDARY_REST_CFG, secondaryIpFinder); + + IgfsEx secondaryIgfsImpl = (IgfsEx) igniteSecondary.fileSystem("igfs-secondary"); + + igfsSecondary = new DefaultIgfsSecondaryFileSystemTestAdapter(secondaryIgfsImpl); + + return secondaryIgfsImpl.asSecondary(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + clear(igfs, igfsSecondary); + + assert igfs.listFiles(new IgfsPath("/")).isEmpty(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + G.stopAll(true); + } + + /** + * Start grid with IGFS. + * + * @param gridName Grid name. + * @param igfsName IGFS name + * @param mode IGFS mode. + * @param secondaryFs Secondary file system (optional). + * @param restCfg Rest configuration string (optional). + * @param ipFinder IP finder. + * @return Started grid instance. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mode, + @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg, + TcpDiscoveryIpFinder ipFinder) throws Exception { + FileSystemConfiguration igfsCfg = new FileSystemConfiguration(); + + igfsCfg.setDataCacheName("dataCache"); + igfsCfg.setMetaCacheName("metaCache"); + igfsCfg.setName(igfsName); + igfsCfg.setBlockSize(IGFS_BLOCK_SIZE); + igfsCfg.setDefaultMode(mode); + igfsCfg.setIpcEndpointConfiguration(restCfg); + igfsCfg.setSecondaryFileSystem(secondaryFs); + igfsCfg.setPrefetchBlocks(PREFETCH_BLOCKS); + igfsCfg.setSequentialReadsBeforePrefetch(SEQ_READS_BEFORE_PREFETCH); + igfsCfg.setRelaxedConsistency(relaxedConsistency()); + + igfsCfg.setInitializeDefaultPathModes(initializeDefaultPathModes()); + + CacheConfiguration dataCacheCfg = defaultCacheConfiguration(); + + dataCacheCfg.setName("dataCache"); + dataCacheCfg.setNearConfiguration(null); + dataCacheCfg.setCacheMode(PARTITIONED); + dataCacheCfg.setNearConfiguration(null); + dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(2)); + dataCacheCfg.setBackups(0); + dataCacheCfg.setAtomicityMode(TRANSACTIONAL); + dataCacheCfg.setMemoryMode(memoryMode); + dataCacheCfg.setOffHeapMaxMemory(0); + + CacheConfiguration metaCacheCfg = defaultCacheConfiguration(); + + metaCacheCfg.setName("metaCache"); + metaCacheCfg.setNearConfiguration(null); + metaCacheCfg.setCacheMode(REPLICATED); + metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + metaCacheCfg.setAtomicityMode(TRANSACTIONAL); + + IgniteConfiguration cfg = new IgniteConfiguration(); + + if (useOptimizedMarshaller()) + cfg.setMarshaller(new OptimizedMarshaller()); + + cfg.setGridName(gridName); + + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.setIpFinder(ipFinder); + + prepareCacheConfigurations(dataCacheCfg, metaCacheCfg); + + cfg.setDiscoverySpi(discoSpi); + cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg); + cfg.setFileSystemConfiguration(igfsCfg); + + cfg.setLocalHost("127.0.0.1"); + cfg.setConnectorConfiguration(null); + + return G.start(cfg); + } + + /** + * Prepare cache configuration. + * + * @param dataCacheCfg Data cache configuration. + * @param metaCacheCfg Meta cache configuration. + */ + protected void prepareCacheConfigurations(CacheConfiguration dataCacheCfg, CacheConfiguration metaCacheCfg) { + // Noop + } + + /** + * Execute provided task in a separate thread. + * + * @param task Task to execute. + * @return Result. + */ + protected static IgniteInternalFuture execute(final Callable task) { + final GridFutureAdapter fut = new GridFutureAdapter<>(); + + new Thread(new Runnable() { + @Override public void run() { + try { + fut.onDone(task.call()); + } + catch (Throwable e) { + fut.onDone(e); + } + } + }).start(); + + return fut; + } + + + /** + * Create the given directories and files in the given IGFS. + * + * @param igfs IGFS. + * @param dirs Directories. + * @param files Files. + * @throws Exception If failed. + */ + @SuppressWarnings("EmptyTryBlock") + public static void create(IgfsImpl igfs, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files) throws Exception { + if (dirs != null) { + for (IgfsPath dir : dirs) + igfs.mkdirs(dir); + } + + if (files != null) { + for (IgfsPath file : files) { + try (OutputStream ignored = igfs.create(file, true)) { + // No-op. + } + + igfs.await(file); + } + } + } + + /** + * Creates specified files/directories + * + * @param uni The file system to operate on. + * @param dirs The directories to create. + * @param files The files to create. + * @throws Exception On error. + */ + @SuppressWarnings("EmptyTryBlock") + public void create(IgfsSecondaryFileSystemTestAdapter uni, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files) + throws Exception { + if (dirs != null) { + for (IgfsPath dir : dirs) + uni.mkdirs(dir.toString()); + } + + if (files != null) { + for (IgfsPath file : files) + try (OutputStream ignore = uni.openOutputStream(file.toString(), false)) { + // No-op + } + } + } + + /** + * Create the file in the given IGFS and write provided data chunks to it. + * + * @param igfs IGFS. + * @param file File. + * @param overwrite Overwrite flag. + * @param chunks Data chunks. + * @throws IOException In case of IO exception. + */ + protected static void createFile(IgfsEx igfs, IgfsPath file, boolean overwrite, @Nullable byte[]... chunks) + throws IOException { + OutputStream os = null; + + try { + os = igfs.create(file, overwrite); + + writeFileChunks(os, chunks); + } + finally { + U.closeQuiet(os); + + awaitFileClose(igfs, file); + } + } + + /** + * Create the file in the given IGFS and write provided data chunks to it. + * + * @param file File. + * @param chunks Data chunks. + * @throws IOException In case of IO exception. + */ + protected static void createFile(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath file, @Nullable byte[]... chunks) + throws IOException { + OutputStream os = null; + + try { + os = uni.openOutputStream(file.toString(), false); + + writeFileChunks(os, chunks); + } + finally { + U.closeQuiet(os); + + IgfsEx igfsEx = uni.igfs(); + + if (igfsEx != null) + awaitFileClose(igfsEx, file); + } + } + + /** + * Create the file in the given IGFS and write provided data chunks to it. + * + * @param igfs IGFS. + * @param file File. + * @param overwrite Overwrite flag. + * @param blockSize Block size. + * @param chunks Data chunks. + * @throws Exception If failed. + */ + protected static void createFile(IgfsImpl igfs, IgfsPath file, boolean overwrite, long blockSize, + @Nullable byte[]... chunks) throws Exception { + IgfsOutputStream os = null; + + try { + os = igfs.create(file, 256, overwrite, null, 0, blockSize, null); + + writeFileChunks(os, chunks); + } + finally { + U.closeQuiet(os); + + awaitFileClose(igfs, file); + } + } + + /** + * Append to the file in the given IGFS provided data chunks. + * + * @param igfs IGFS. + * @param file File. + * @param chunks Data chunks. + * @throws Exception If failed. + */ + protected static void appendFile(IgfsImpl igfs, IgfsPath file, @Nullable byte[]... chunks) + throws Exception { + IgfsOutputStream os = null; + + try { + os = igfs.append(file, false); + + writeFileChunks(os, chunks); + } + finally { + U.closeQuiet(os); + + awaitFileClose(igfs, file); + } + } + + /** + * Write provided data chunks to the file output stream. + * + * @param os Output stream. + * @param chunks Data chunks. + * @throws IOException If failed. + */ + protected static void writeFileChunks(OutputStream os, @Nullable byte[]... chunks) throws IOException { + if (chunks != null && chunks.length > 0) { + for (byte[] chunk : chunks) + os.write(chunk); + } + } + + /** + * Await for previously opened output stream to close. This is achieved by requesting dummy update on the file. + * + * @param igfs IGFS. + * @param file File. + */ + public static void awaitFileClose(IgfsSecondaryFileSystem igfs, IgfsPath file) { + try { + igfs.update(file, Collections.singletonMap("prop", "val")); + } + catch (IgniteException ignore) { + // No-op. + } + } + + /** + * Await for previously opened output stream to close. + * + * @param igfs IGFS. + * @param file File. + */ + public static void awaitFileClose(@Nullable IgfsEx igfs, IgfsPath file) { + igfs.await(file); + } + + /** + * Ensure that the given paths exist in the given IGFSs. + * + * @param igfs First IGFS. + * @param igfsSecondary Second IGFS. + * @param paths Paths. + * @throws Exception If failed. + */ + protected void checkExist(IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath... paths) + throws Exception { + checkExist(igfs, paths); + + if (dual) + checkExist(igfsSecondary, paths); + } + + /** + * Ensure that the given paths exist in the given IGFS. + * + * @param igfs IGFS. + * @param paths Paths. + * @throws IgniteCheckedException If failed. + */ + protected static void checkExist(IgfsImpl igfs, IgfsPath... paths) throws IgniteCheckedException { + for (IgfsPath path : paths) + assert igfs.exists(path) : "Path doesn't exist [igfs=" + igfs.name() + ", path=" + path + ']'; + } + + /** + * Ensure that the given paths exist in the given IGFS. + * + * @param uni filesystem. + * @param paths Paths. + * @throws IgniteCheckedException If failed. + */ + protected void checkExist(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath... paths) throws IgniteCheckedException { + IgfsEx ex = uni.igfs(); + + for (IgfsPath path : paths) { + if (ex != null) + assert ex.context().meta().fileId(path) != null : "Path doesn't exist [igfs=" + ex.name() + + ", path=" + path + ']'; + + try { + assert uni.exists(path.toString()) : "Path doesn't exist [igfs=" + uni.name() + ", path=" + path + ']'; + } + catch (IOException ioe) { + throw new IgniteCheckedException(ioe); + } + } + } + + /** + * Ensure that the given paths don't exist in the given IGFSs. + * + * @param igfs First IGFS. + * @param igfsSecondary Second IGFS. + * @param paths Paths. + * @throws Exception If failed. + */ + protected void checkNotExist(IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath... paths) + throws Exception { + checkNotExist(igfs, paths); + + if (dual) + checkNotExist(igfsSecondary, paths); + } + + /** + * Ensure that the given paths don't exist in the given IGFS. + * + * @param igfs IGFS. + * @param paths Paths. + * @throws Exception If failed. + */ + protected void checkNotExist(IgfsImpl igfs, IgfsPath... paths) throws Exception { + for (IgfsPath path : paths) + assert !igfs.exists(path) : "Path exists [igfs=" + igfs.name() + ", path=" + path + ']'; + } + + /** + * Ensure that the given paths don't exist in the given IGFS. + * + * @param uni secondary FS. + * @param paths Paths. + * @throws Exception If failed. + */ + protected void checkNotExist(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath... paths) throws Exception { + IgfsEx ex = uni.igfs(); + + for (IgfsPath path : paths) { + if (ex != null) + assert !ex.exists(path) : "Path exists [igfs=" + ex.name() + ", path=" + path + ']'; + + assert !uni.exists(path.toString()) : "Path exists [igfs=" + uni.name() + ", path=" + path + ']'; + } + } + + /** + * Ensure that the given file exists in the given IGFSs and that it has exactly the same content as provided in the + * "data" parameter. + * + * @param igfs First IGFS. + * @param igfsSecondary Second IGFS. + * @param file File. + * @param chunks Expected data. + * @throws Exception If failed. + */ + protected void checkFile(@Nullable IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath file, + @Nullable byte[]... chunks) throws Exception { + if (igfs != null) { + checkExist(igfs, file); + checkFileContent(igfs, file, chunks); + } + + if (dual) { + checkExist(igfsSecondary, file); + checkFileContent(igfsSecondary, file.toString(), chunks); + } + } + + /** + * Ensure that the given file has exactly the same content as provided in the "data" parameter. + * + * @param igfs IGFS. + * @param file File. + * @param chunks Expected data. + * @throws IOException In case of IO exception. + * @throws IgniteCheckedException In case of Grid exception. + */ + protected static void checkFileContent(IgfsImpl igfs, IgfsPath file, @Nullable byte[]... chunks) + throws IOException, IgniteCheckedException { + if (chunks != null && chunks.length > 0) { + IgfsInputStream is = null; + + try { + is = igfs.open(file); + + int chunkIdx = 0; + int pos = 0; + + for (byte[] chunk : chunks) { + byte[] buf = new byte[chunk.length]; + + is.readFully(pos, buf); + + assert Arrays.equals(chunk, buf) : "Bad chunk [igfs=" + igfs.name() + ", chunkIdx=" + chunkIdx + + ", expected=" + Arrays.toString(chunk) + ", actual=" + Arrays.toString(buf) + ']'; + + chunkIdx++; + pos += chunk.length; + } + + is.close(); + } + finally { + U.closeQuiet(is); + } + } + } + + /** + * Ensure that the given file has exactly the same content as provided in the "data" parameter. + * + * @param uni FS. + * @param path File. + * @param chunks Expected data. + * @throws IOException In case of IO exception. + * @throws IgniteCheckedException In case of Grid exception. + */ + protected void checkFileContent(IgfsSecondaryFileSystemTestAdapter uni, String path, @Nullable byte[]... chunks) + throws IOException, IgniteCheckedException { + if (chunks != null && chunks.length > 0) { + InputStream is = null; + + try { + is = uni.openInputStream(path); + + int chunkIdx = 0; + + int read; + for (byte[] chunk: chunks) { + byte[] buf = new byte[chunk.length]; + + read = 0; + + while (true) { + int r = is.read(buf, read, buf.length - read); + + read += r; + + if (read == buf.length || r <= 0) + break; + } + + assert read == chunk.length : "Chunk #" + chunkIdx + " was not read fully:" + + " read=" + read + ", expected=" + chunk.length; + assert Arrays.equals(chunk, buf) : "Bad chunk [igfs=" + uni.name() + ", chunkIdx=" + chunkIdx + + ", expected=" + Arrays.toString(chunk) + ", actual=" + Arrays.toString(buf) + ']'; + + chunkIdx++; + } + + is.close(); + } + finally { + U.closeQuiet(is); + } + } + } + + /** + * Create map with properties. + * + * @param username User name. + * @param grpName Group name. + * @param perm Permission. + * @return Map with properties. + */ + protected Map properties(@Nullable String username, @Nullable String grpName, + @Nullable String perm) { + Map props = new HashMap<>(); + + if (username != null) + props.put(IgfsUtils.PROP_USER_NAME, username); + + if (grpName != null) + props.put(IgfsUtils.PROP_GROUP_NAME, grpName); + + if (perm != null) + props.put(IgfsUtils.PROP_PERMISSION, perm); + + return props; + } + + /** + * Convenient method to group paths. + * + * @param paths Paths to group. + * @return Paths as array. + */ + protected static IgfsPath[] paths(IgfsPath... paths) { + return paths; + } + + /** + * Safely clear IGFSs. + * + * @param igfs First IGFS. + * @param igfsSecondary Second IGFS. + * @throws Exception If failed. + */ + protected void clear(IgniteFileSystem igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary) throws Exception { + clear(igfs); + + if (dual) + clear(igfsSecondary); + } + + /** + * Gets the data cache instance for this IGFS instance. + * + * @param igfs The IGFS unstance. + * @return The data cache. + */ + protected static GridCacheAdapter getDataCache(IgniteFileSystem igfs) { + String dataCacheName = igfs.configuration().getDataCacheName(); + + IgniteEx igniteEx = ((IgfsEx)igfs).context().kernalContext().grid(); + + return ((IgniteKernal)igniteEx).internalCache(dataCacheName); + } + + /** + * Gets meta cache. + * + * @param igfs The IGFS instance. + * @return The data cache. + */ + protected static GridCacheAdapter getMetaCache(IgniteFileSystem igfs) { + String dataCacheName = igfs.configuration().getMetaCacheName(); + + IgniteEx igniteEx = ((IgfsEx)igfs).context().kernalContext().grid(); + + return ((IgniteKernal)igniteEx).internalCache(dataCacheName); + } + + /** + * Clear particular IGFS. + * + * @param igfs IGFS. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + public static void clear(IgniteFileSystem igfs) throws Exception { + Field workerMapFld = IgfsImpl.class.getDeclaredField("workerMap"); + + workerMapFld.setAccessible(true); + + // Wait for all workers to finish. + Map workerMap = (Map)workerMapFld.get(igfs); + + for (Map.Entry entry : workerMap.entrySet()) { + entry.getValue().cancel(); + + try { + entry.getValue().await(); + } + catch (IgniteCheckedException e) { + if (!(e instanceof IgfsFileWorkerBatchCancelledException)) + throw e; + } + } + + // Clear igfs. + igfs.format(); + + int prevDifferentSize = Integer.MAX_VALUE; // Previous different size. + int constCnt = 0, totalCnt = 0; + final int constThreshold = 20; + final long sleepPeriod = 500L; + final long totalThreshold = CACHE_EMPTY_TIMEOUT / sleepPeriod; + + while (true) { + int metaSize = 0; + + for (IgniteUuid metaId : getMetaCache(igfs).keySet()) { + if (!IgfsUtils.isRootOrTrashId(metaId)) + metaSize++; + } + + int dataSize = getDataCache(igfs).size(); + + int size = metaSize + dataSize; + + if (size <= 2) + return; // Caches are cleared, we're done. (2 because ROOT & TRASH always exist). + + X.println("Sum size: " + size); + + if (size > prevDifferentSize) { + X.println("Summary cache size has grown unexpectedly: size=" + size + ", prevSize=" + prevDifferentSize); + + break; + } + + if (totalCnt > totalThreshold) { + X.println("Timeout exceeded."); + + break; + } + + if (size == prevDifferentSize) { + constCnt++; + + if (constCnt == constThreshold) { + X.println("Summary cache size stays unchanged for too long: size=" + size); + + break; + } + } else { + constCnt = 0; + + prevDifferentSize = size; // renew; + } + + Thread.sleep(sleepPeriod); + + totalCnt++; + } + + dumpCache("MetaCache" , getMetaCache(igfs)); + + dumpCache("DataCache" , getDataCache(igfs)); + + fail("Caches are not empty."); + } + + /** + * Dumps given cache for diagnostic purposes. + * + * @param cacheName Name. + * @param cache The cache. + */ + private static void dumpCache(String cacheName, GridCacheAdapter cache) { + X.println("=============================== " + cacheName + " cache dump: "); + + Iterable entries = cache.entries(); + + for (GridCacheEntryEx e: entries) + X.println("Lost " + cacheName + " entry = " + e); + } + + /** + * Clear particular {@link IgfsSecondaryFileSystemTestAdapter}. + * + * @param uni IGFS. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + public static void clear(IgfsSecondaryFileSystemTestAdapter uni) throws Exception { + IgfsEx igfsEx = uni.igfs(); + + if (igfsEx != null) + clear(igfsEx); + + // Clear the filesystem. + uni.format(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + clear(igfs, igfsSecondary); + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java index 86c2449145b71..c9b08d9c83bab 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java @@ -17,55 +17,32 @@ package org.apache.ignite.internal.processors.igfs; -import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; -import org.apache.ignite.IgniteFileSystem; -import org.apache.ignite.Ignition; import org.apache.ignite.cache.CacheMemoryMode; import org.apache.ignite.cache.CachePeekMode; -import org.apache.ignite.cache.CacheWriteSynchronizationMode; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.configuration.FileSystemConfiguration; -import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.igfs.IgfsDirectoryNotEmptyException; import org.apache.ignite.igfs.IgfsException; import org.apache.ignite.igfs.IgfsFile; -import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; import org.apache.ignite.igfs.IgfsInputStream; -import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; -import org.apache.ignite.igfs.IgfsIpcEndpointType; import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsOutputStream; import org.apache.ignite.igfs.IgfsParentNotDirectoryException; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.IgfsPathNotFoundException; -import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; -import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; -import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteUuid; -import org.apache.ignite.marshaller.optimized.OptimizedMarshaller; -import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; -import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; -import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.GridTestUtils; -import org.jetbrains.annotations.Nullable; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.lang.reflect.Field; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; @@ -84,142 +61,18 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; -import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED; -import static org.apache.ignite.cache.CacheMode.PARTITIONED; -import static org.apache.ignite.cache.CacheMode.REPLICATED; -import static org.apache.ignite.igfs.IgfsMode.PRIMARY; -import static org.apache.ignite.igfs.IgfsMode.PROXY; - /** * Test fo regular igfs operations. */ @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "ConstantConditions"}) -public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { - /** IGFS block size. */ - protected static final int IGFS_BLOCK_SIZE = 512 * 1024; - - /** Default block size (32Mb). */ - protected static final long BLOCK_SIZE = 32 * 1024 * 1024; - - /** Default repeat count. */ - protected static final int REPEAT_CNT = 5; // Diagnostic: up to 500; Regression: 5 - - /** Concurrent operations count. */ - protected static final int OPS_CNT = 16; - - /** Renames count. */ - protected static final int RENAME_CNT = OPS_CNT; - - /** Deletes count. */ - protected static final int DELETE_CNT = OPS_CNT; - - /** Updates count. */ - protected static final int UPDATE_CNT = OPS_CNT; - - /** Mkdirs count. */ - protected static final int MKDIRS_CNT = OPS_CNT; - - /** Create count. */ - protected static final int CREATE_CNT = OPS_CNT; - - /** Time to wait until the caches get empty after format. */ - private static final long CACHE_EMPTY_TIMEOUT = 30_000L; - - /** Seed to generate random numbers. */ - protected static final long SEED = System.currentTimeMillis(); - - /** Amount of blocks to prefetch. */ - protected static final int PREFETCH_BLOCKS = 1; - - /** Amount of sequential block reads before prefetch is triggered. */ - protected static final int SEQ_READS_BEFORE_PREFETCH = 2; - - /** Primary file system REST endpoint configuration map. */ - protected static final IgfsIpcEndpointConfiguration PRIMARY_REST_CFG; - - /** Secondary file system REST endpoint configuration map. */ - protected static final IgfsIpcEndpointConfiguration SECONDARY_REST_CFG; - - /** Directory. */ - protected static final IgfsPath DIR = new IgfsPath("/dir"); - - /** Sub-directory. */ - protected static final IgfsPath SUBDIR = new IgfsPath(DIR, "subdir"); - - /** Another sub-directory in the same directory. */ - protected static final IgfsPath SUBDIR2 = new IgfsPath(DIR, "subdir2"); - - /** Sub-directory of the sub-directory. */ - protected static final IgfsPath SUBSUBDIR = new IgfsPath(SUBDIR, "subsubdir"); - - /** File. */ - protected static final IgfsPath FILE = new IgfsPath(SUBDIR, "file"); - - /** Another file in the same directory. */ - protected static final IgfsPath FILE2 = new IgfsPath(SUBDIR, "file2"); - - /** Other directory. */ - protected static final IgfsPath DIR_NEW = new IgfsPath("/dirNew"); - - /** Other subdirectory. */ - protected static final IgfsPath SUBDIR_NEW = new IgfsPath(DIR_NEW, "subdirNew"); - - /** Other sub-directory of the sub-directory. */ - protected static final IgfsPath SUBSUBDIR_NEW = new IgfsPath(SUBDIR_NEW, "subsubdirNew"); - - /** Other file. */ - protected static final IgfsPath FILE_NEW = new IgfsPath(SUBDIR_NEW, "fileNew"); - - /** Default data chunk (128 bytes). */ - protected static final byte[] chunk = createChunk(128); - - /** Primary IGFS. */ - protected static IgfsImpl igfs; - - /** Secondary IGFS */ - protected static IgfsSecondaryFileSystem igfsSecondaryFileSystem; - - /** Secondary file system lower layer "backdoor" wrapped in UniversalFileSystemAdapter: */ - protected static IgfsSecondaryFileSystemTestAdapter igfsSecondary; - - /** IGFS mode. */ - protected final IgfsMode mode; - - /** Dual mode flag. */ - protected final boolean dual; - - /** Memory mode. */ - protected final CacheMemoryMode memoryMode; - - /** IP finder for primary topology. */ - protected final TcpDiscoveryVmIpFinder primaryIpFinder = new TcpDiscoveryVmIpFinder(true); - - /** IP finder for secondary topology. */ - protected final TcpDiscoveryVmIpFinder secondaryIpFinder = new TcpDiscoveryVmIpFinder(true); - - /** Ignite nodes of cluster, excluding the secondary file system node, if any. */ - protected Ignite[] nodes; - - static { - PRIMARY_REST_CFG = new IgfsIpcEndpointConfiguration(); - - PRIMARY_REST_CFG.setType(IgfsIpcEndpointType.TCP); - PRIMARY_REST_CFG.setPort(10500); - - SECONDARY_REST_CFG = new IgfsIpcEndpointConfiguration(); - - SECONDARY_REST_CFG.setType(IgfsIpcEndpointType.TCP); - SECONDARY_REST_CFG.setPort(11500); - } - +public abstract class IgfsAbstractSelfTest extends IgfsAbstractBaseSelfTest { /** * Constructor. * * @param mode IGFS mode. */ protected IgfsAbstractSelfTest(IgfsMode mode) { - this(mode, ONHEAP_TIERED); + super(mode); } /** @@ -229,258 +82,7 @@ protected IgfsAbstractSelfTest(IgfsMode mode) { * @param memoryMode Memory mode. */ protected IgfsAbstractSelfTest(IgfsMode mode, CacheMemoryMode memoryMode) { - assert mode != null && mode != PROXY; - - this.mode = mode; - this.memoryMode = memoryMode; - - dual = mode != PRIMARY; - } - - /** - * @return Relaxed consistency flag. - */ - protected boolean relaxedConsistency() { - return false; - } - - /** - * @return Relaxed consistency flag. - */ - protected boolean initializeDefaultPathModes() { - return false; - } - - /** - * @return Client flag. - */ - protected boolean client() { - return false; - } - - /** - * @return Use optimized marshaller flag. - */ - protected boolean useOptimizedMarshaller() { - return false; - } - - /** - * @return Whether append is supported. - */ - protected boolean appendSupported() { - return true; - } - - /** - * @return Whether permissions are supported. - */ - protected boolean permissionsSupported() { - return true; - } - - /** - * @return Whether properties are supported. - */ - protected boolean propertiesSupported() { - return true; - } - - /** - * @return Whether times are supported. - */ - protected boolean timesSupported() { - return true; - } - - /** - * @return Amount of nodes to start. - */ - protected int nodeCount() { - return 1; - } - - /** - * Data chunk. - * - * @param len Length. - * @return Data chunk. - */ - static byte[] createChunk(int len) { - byte[] chunk = new byte[len]; - - for (int i = 0; i < chunk.length; i++) - chunk[i] = (byte)i; - - return chunk; - } - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - igfsSecondaryFileSystem = createSecondaryFileSystemStack(); - - nodes = new Ignite[nodeCount()]; - - for (int i = 0; i < nodes.length; i++) { - String nodeName = i == 0 ? "ignite" : "ignite" + i; - - nodes[i] = startGridWithIgfs(nodeName, "igfs", mode, igfsSecondaryFileSystem, PRIMARY_REST_CFG, - primaryIpFinder); - } - - igfs = (IgfsImpl) nodes[0].fileSystem("igfs"); - - if (client()) { - // Start client. - Ignition.setClientMode(true); - - try { - Ignite ignite = startGridWithIgfs("ignite-client", "igfs", mode, igfsSecondaryFileSystem, - PRIMARY_REST_CFG, primaryIpFinder); - - igfs = (IgfsImpl) ignite.fileSystem("igfs"); - } - finally { - Ignition.setClientMode(false); - } - } - } - - /** - * Creates secondary file system stack. - * - * @return The secondary file system. - * @throws Exception On error. - */ - protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception { - Ignite igniteSecondary = startGridWithIgfs("ignite-secondary", "igfs-secondary", PRIMARY, null, - SECONDARY_REST_CFG, secondaryIpFinder); - - IgfsEx secondaryIgfsImpl = (IgfsEx) igniteSecondary.fileSystem("igfs-secondary"); - - igfsSecondary = new DefaultIgfsSecondaryFileSystemTestAdapter(secondaryIgfsImpl); - - return secondaryIgfsImpl.asSecondary(); - } - - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - clear(igfs, igfsSecondary); - - assert igfs.listFiles(new IgfsPath("/")).isEmpty(); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - G.stopAll(true); - } - - /** - * Start grid with IGFS. - * - * @param gridName Grid name. - * @param igfsName IGFS name - * @param mode IGFS mode. - * @param secondaryFs Secondary file system (optional). - * @param restCfg Rest configuration string (optional). - * @param ipFinder IP finder. - * @return Started grid instance. - * @throws Exception If failed. - */ - @SuppressWarnings("unchecked") - protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mode, - @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg, - TcpDiscoveryIpFinder ipFinder) throws Exception { - FileSystemConfiguration igfsCfg = new FileSystemConfiguration(); - - igfsCfg.setDataCacheName("dataCache"); - igfsCfg.setMetaCacheName("metaCache"); - igfsCfg.setName(igfsName); - igfsCfg.setBlockSize(IGFS_BLOCK_SIZE); - igfsCfg.setDefaultMode(mode); - igfsCfg.setIpcEndpointConfiguration(restCfg); - igfsCfg.setSecondaryFileSystem(secondaryFs); - igfsCfg.setPrefetchBlocks(PREFETCH_BLOCKS); - igfsCfg.setSequentialReadsBeforePrefetch(SEQ_READS_BEFORE_PREFETCH); - igfsCfg.setRelaxedConsistency(relaxedConsistency()); - - igfsCfg.setInitializeDefaultPathModes(initializeDefaultPathModes()); - - CacheConfiguration dataCacheCfg = defaultCacheConfiguration(); - - dataCacheCfg.setName("dataCache"); - dataCacheCfg.setNearConfiguration(null); - dataCacheCfg.setCacheMode(PARTITIONED); - dataCacheCfg.setNearConfiguration(null); - dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); - dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(2)); - dataCacheCfg.setBackups(0); - dataCacheCfg.setAtomicityMode(TRANSACTIONAL); - dataCacheCfg.setMemoryMode(memoryMode); - dataCacheCfg.setOffHeapMaxMemory(0); - - CacheConfiguration metaCacheCfg = defaultCacheConfiguration(); - - metaCacheCfg.setName("metaCache"); - metaCacheCfg.setNearConfiguration(null); - metaCacheCfg.setCacheMode(REPLICATED); - metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); - metaCacheCfg.setAtomicityMode(TRANSACTIONAL); - - IgniteConfiguration cfg = new IgniteConfiguration(); - - if (useOptimizedMarshaller()) - cfg.setMarshaller(new OptimizedMarshaller()); - - cfg.setGridName(gridName); - - TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); - - discoSpi.setIpFinder(ipFinder); - - prepareCacheConfigurations(dataCacheCfg, metaCacheCfg); - - cfg.setDiscoverySpi(discoSpi); - cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg); - cfg.setFileSystemConfiguration(igfsCfg); - - cfg.setLocalHost("127.0.0.1"); - cfg.setConnectorConfiguration(null); - - return G.start(cfg); - } - - /** - * Prepare cache configuration. - * - * @param dataCacheCfg Data cache configuration. - * @param metaCacheCfg Meta cache configuration. - */ - protected void prepareCacheConfigurations(CacheConfiguration dataCacheCfg, CacheConfiguration metaCacheCfg) { - // Noop - } - - /** - * Execute provided task in a separate thread. - * - * @param task Task to execute. - * @return Result. - */ - protected static IgniteInternalFuture execute(final Callable task) { - final GridFutureAdapter fut = new GridFutureAdapter<>(); - - new Thread(new Runnable() { - @Override public void run() { - try { - fut.onDone(task.call()); - } - catch (Throwable e) { - fut.onDone(e); - } - } - }).start(); - - return fut; + super(mode, memoryMode); } /** @@ -1138,8 +740,9 @@ public void testRootPropertiesPersistAfterFormat() throws Exception { } /** + * Check root property update. * - * @throws Exception + * @throws Exception If failed. */ private void checkRootPropertyUpdate(String prop, String setVal, String expGetVal) throws Exception { final IgfsPath rootPath = new IgfsPath("/"); @@ -2945,609 +2548,4 @@ private void checkDeadlocks(final int lvlCnt, final int childrenDirPerLvl, final U.joinThreads(threads, null); } - - /** - * Create the given directories and files in the given IGFS. - * - * @param igfs IGFS. - * @param dirs Directories. - * @param files Files. - * @throws Exception If failed. - */ - @SuppressWarnings("EmptyTryBlock") - public static void create(IgfsImpl igfs, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files) throws Exception { - if (dirs != null) { - for (IgfsPath dir : dirs) - igfs.mkdirs(dir); - } - - if (files != null) { - for (IgfsPath file : files) { - try (OutputStream os = igfs.create(file, true)) { - // No-op. - } - - igfs.await(file); - } - } - } - - /** - * Creates specified files/directories - * - * @param uni The file system to operate on. - * @param dirs The directories to create. - * @param files The files to create. - * @throws Exception On error. - */ - @SuppressWarnings("EmptyTryBlock") - public void create(IgfsSecondaryFileSystemTestAdapter uni, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files) - throws Exception { - if (dirs != null) { - for (IgfsPath dir : dirs) - uni.mkdirs(dir.toString()); - } - - if (files != null) { - for (IgfsPath file : files) - try (OutputStream ignore = uni.openOutputStream(file.toString(), false)) { - // No-op - } - } - } - - /** - * Create the file in the given IGFS and write provided data chunks to it. - * - * @param igfs IGFS. - * @param file File. - * @param overwrite Overwrite flag. - * @param chunks Data chunks. - * @throws IOException In case of IO exception. - */ - protected static void createFile(IgfsEx igfs, IgfsPath file, boolean overwrite, @Nullable byte[]... chunks) - throws IOException { - OutputStream os = null; - - try { - os = igfs.create(file, overwrite); - - writeFileChunks(os, chunks); - } - finally { - U.closeQuiet(os); - - awaitFileClose(igfs, file); - } - } - - /** - * Create the file in the given IGFS and write provided data chunks to it. - * - * @param file File. - * @param chunks Data chunks. - * @throws IOException In case of IO exception. - */ - protected static void createFile(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath file, @Nullable byte[]... chunks) - throws IOException { - OutputStream os = null; - - try { - os = uni.openOutputStream(file.toString(), false); - - writeFileChunks(os, chunks); - } - finally { - U.closeQuiet(os); - - IgfsEx igfsEx = uni.igfs(); - - if (igfsEx != null) - awaitFileClose(igfsEx, file); - } - } - - /** - * Create the file in the given IGFS and write provided data chunks to it. - * - * @param igfs IGFS. - * @param file File. - * @param overwrite Overwrite flag. - * @param blockSize Block size. - * @param chunks Data chunks. - * @throws Exception If failed. - */ - protected static void createFile(IgfsImpl igfs, IgfsPath file, boolean overwrite, long blockSize, - @Nullable byte[]... chunks) throws Exception { - IgfsOutputStream os = null; - - try { - os = igfs.create(file, 256, overwrite, null, 0, blockSize, null); - - writeFileChunks(os, chunks); - } - finally { - U.closeQuiet(os); - - awaitFileClose(igfs, file); - } - } - - /** - * Append to the file in the given IGFS provided data chunks. - * - * @param igfs IGFS. - * @param file File. - * @param chunks Data chunks. - * @throws Exception If failed. - */ - protected static void appendFile(IgfsImpl igfs, IgfsPath file, @Nullable byte[]... chunks) - throws Exception { - IgfsOutputStream os = null; - - try { - os = igfs.append(file, false); - - writeFileChunks(os, chunks); - } - finally { - U.closeQuiet(os); - - awaitFileClose(igfs, file); - } - } - - /** - * Write provided data chunks to the file output stream. - * - * @param os Output stream. - * @param chunks Data chunks. - * @throws IOException If failed. - */ - protected static void writeFileChunks(OutputStream os, @Nullable byte[]... chunks) throws IOException { - if (chunks != null && chunks.length > 0) { - for (byte[] chunk : chunks) - os.write(chunk); - } - } - - /** - * Await for previously opened output stream to close. This is achieved by requesting dummy update on the file. - * - * @param igfs IGFS. - * @param file File. - */ - public static void awaitFileClose(IgfsSecondaryFileSystem igfs, IgfsPath file) { - try { - igfs.update(file, Collections.singletonMap("prop", "val")); - } - catch (IgniteException ignore) { - // No-op. - } - } - - /** - * Await for previously opened output stream to close. - * - * @param igfs IGFS. - * @param file File. - */ - public static void awaitFileClose(@Nullable IgfsEx igfs, IgfsPath file) { - igfs.await(file); - } - - /** - * Ensure that the given paths exist in the given IGFSs. - * - * @param igfs First IGFS. - * @param igfsSecondary Second IGFS. - * @param paths Paths. - * @throws Exception If failed. - */ - protected void checkExist(IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath... paths) - throws Exception { - checkExist(igfs, paths); - - if (dual) - checkExist(igfsSecondary, paths); - } - - /** - * Ensure that the given paths exist in the given IGFS. - * - * @param igfs IGFS. - * @param paths Paths. - * @throws IgniteCheckedException If failed. - */ - protected static void checkExist(IgfsImpl igfs, IgfsPath... paths) throws IgniteCheckedException { - for (IgfsPath path : paths) - assert igfs.exists(path) : "Path doesn't exist [igfs=" + igfs.name() + ", path=" + path + ']'; - } - - /** - * Ensure that the given paths exist in the given IGFS. - * - * @param uni filesystem. - * @param paths Paths. - * @throws IgniteCheckedException If failed. - */ - protected void checkExist(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath... paths) throws IgniteCheckedException { - IgfsEx ex = uni.igfs(); - - for (IgfsPath path : paths) { - if (ex != null) - assert ex.context().meta().fileId(path) != null : "Path doesn't exist [igfs=" + ex.name() + - ", path=" + path + ']'; - - try { - assert uni.exists(path.toString()) : "Path doesn't exist [igfs=" + uni.name() + ", path=" + path + ']'; - } - catch (IOException ioe) { - throw new IgniteCheckedException(ioe); - } - } - } - - /** - * Ensure that the given paths don't exist in the given IGFSs. - * - * @param igfs First IGFS. - * @param igfsSecondary Second IGFS. - * @param paths Paths. - * @throws Exception If failed. - */ - protected void checkNotExist(IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath... paths) - throws Exception { - checkNotExist(igfs, paths); - - if (dual) - checkNotExist(igfsSecondary, paths); - } - - /** - * Ensure that the given paths don't exist in the given IGFS. - * - * @param igfs IGFS. - * @param paths Paths. - * @throws Exception If failed. - */ - protected void checkNotExist(IgfsImpl igfs, IgfsPath... paths) throws Exception { - for (IgfsPath path : paths) - assert !igfs.exists(path) : "Path exists [igfs=" + igfs.name() + ", path=" + path + ']'; - } - - /** - * Ensure that the given paths don't exist in the given IGFS. - * - * @param uni secondary FS. - * @param paths Paths. - * @throws Exception If failed. - */ - protected void checkNotExist(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath... paths) throws Exception { - IgfsEx ex = uni.igfs(); - - for (IgfsPath path : paths) { - if (ex != null) - assert !ex.exists(path) : "Path exists [igfs=" + ex.name() + ", path=" + path + ']'; - - assert !uni.exists(path.toString()) : "Path exists [igfs=" + uni.name() + ", path=" + path + ']'; - } - } - - /** - * Ensure that the given file exists in the given IGFSs and that it has exactly the same content as provided in the - * "data" parameter. - * - * @param igfs First IGFS. - * @param igfsSecondary Second IGFS. - * @param file File. - * @param chunks Expected data. - * @throws Exception If failed. - */ - protected void checkFile(@Nullable IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath file, - @Nullable byte[]... chunks) throws Exception { - if (igfs != null) { - checkExist(igfs, file); - checkFileContent(igfs, file, chunks); - } - - if (dual) { - checkExist(igfsSecondary, file); - checkFileContent(igfsSecondary, file.toString(), chunks); - } - } - - /** - * Ensure that the given file has exactly the same content as provided in the "data" parameter. - * - * @param igfs IGFS. - * @param file File. - * @param chunks Expected data. - * @throws IOException In case of IO exception. - * @throws IgniteCheckedException In case of Grid exception. - */ - protected static void checkFileContent(IgfsImpl igfs, IgfsPath file, @Nullable byte[]... chunks) - throws IOException, IgniteCheckedException { - if (chunks != null && chunks.length > 0) { - IgfsInputStream is = null; - - try { - is = igfs.open(file); - - int chunkIdx = 0; - int pos = 0; - - for (byte[] chunk : chunks) { - byte[] buf = new byte[chunk.length]; - - is.readFully(pos, buf); - - assert Arrays.equals(chunk, buf) : "Bad chunk [igfs=" + igfs.name() + ", chunkIdx=" + chunkIdx + - ", expected=" + Arrays.toString(chunk) + ", actual=" + Arrays.toString(buf) + ']'; - - chunkIdx++; - pos += chunk.length; - } - - is.close(); - } - finally { - U.closeQuiet(is); - } - } - } - - /** - * Ensure that the given file has exactly the same content as provided in the "data" parameter. - * - * @param uni FS. - * @param path File. - * @param chunks Expected data. - * @throws IOException In case of IO exception. - * @throws IgniteCheckedException In case of Grid exception. - */ - protected void checkFileContent(IgfsSecondaryFileSystemTestAdapter uni, String path, @Nullable byte[]... chunks) - throws IOException, IgniteCheckedException { - if (chunks != null && chunks.length > 0) { - InputStream is = null; - - try { - is = uni.openInputStream(path); - - int chunkIdx = 0; - - int read; - for (byte[] chunk: chunks) { - byte[] buf = new byte[chunk.length]; - - read = 0; - - while (true) { - int r = is.read(buf, read, buf.length - read); - - read += r; - - if (read == buf.length || r <= 0) - break; - } - - assert read == chunk.length : "Chunk #" + chunkIdx + " was not read fully:" + - " read=" + read + ", expected=" + chunk.length; - assert Arrays.equals(chunk, buf) : "Bad chunk [igfs=" + uni.name() + ", chunkIdx=" + chunkIdx + - ", expected=" + Arrays.toString(chunk) + ", actual=" + Arrays.toString(buf) + ']'; - - chunkIdx++; - } - - is.close(); - } - finally { - U.closeQuiet(is); - } - } - } - - /** - * Create map with properties. - * - * @param username User name. - * @param grpName Group name. - * @param perm Permission. - * @return Map with properties. - */ - protected Map properties(@Nullable String username, @Nullable String grpName, - @Nullable String perm) { - Map props = new HashMap<>(); - - if (username != null) - props.put(IgfsUtils.PROP_USER_NAME, username); - - if (grpName != null) - props.put(IgfsUtils.PROP_GROUP_NAME, grpName); - - if (perm != null) - props.put(IgfsUtils.PROP_PERMISSION, perm); - - return props; - } - - /** - * Convenient method to group paths. - * - * @param paths Paths to group. - * @return Paths as array. - */ - protected static IgfsPath[] paths(IgfsPath... paths) { - return paths; - } - - /** - * Safely clear IGFSs. - * - * @param igfs First IGFS. - * @param igfsSecondary Second IGFS. - * @throws Exception If failed. - */ - protected void clear(IgniteFileSystem igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary) throws Exception { - clear(igfs); - - if (dual) - clear(igfsSecondary); - } - - /** - * Gets the data cache instance for this IGFS instance. - * - * @param igfs The IGFS unstance. - * @return The data cache. - */ - protected static GridCacheAdapter getDataCache(IgniteFileSystem igfs) { - String dataCacheName = igfs.configuration().getDataCacheName(); - - IgniteEx igniteEx = ((IgfsEx)igfs).context().kernalContext().grid(); - - return ((IgniteKernal)igniteEx).internalCache(dataCacheName); - } - - /** - * Gets meta cache. - * - * @param igfs The IGFS instance. - * @return The data cache. - */ - protected static GridCacheAdapter getMetaCache(IgniteFileSystem igfs) { - String dataCacheName = igfs.configuration().getMetaCacheName(); - - IgniteEx igniteEx = ((IgfsEx)igfs).context().kernalContext().grid(); - - return ((IgniteKernal)igniteEx).internalCache(dataCacheName); - } - - /** - * Clear particular IGFS. - * - * @param igfs IGFS. - * @throws Exception If failed. - */ - @SuppressWarnings("unchecked") - public static void clear(IgniteFileSystem igfs) throws Exception { - Field workerMapFld = IgfsImpl.class.getDeclaredField("workerMap"); - - workerMapFld.setAccessible(true); - - // Wait for all workers to finish. - Map workerMap = (Map)workerMapFld.get(igfs); - - for (Map.Entry entry : workerMap.entrySet()) { - entry.getValue().cancel(); - - try { - entry.getValue().await(); - } - catch (IgniteCheckedException e) { - if (!(e instanceof IgfsFileWorkerBatchCancelledException)) - throw e; - } - } - - // Clear igfs. - igfs.format(); - - int prevDifferentSize = Integer.MAX_VALUE; // Previous different size. - int constCnt = 0, totalCnt = 0; - final int constThreshold = 20; - final long sleepPeriod = 500L; - final long totalThreshold = CACHE_EMPTY_TIMEOUT / sleepPeriod; - - while (true) { - int metaSize = 0; - - for (IgniteUuid metaId : getMetaCache(igfs).keySet()) { - if (!IgfsUtils.isRootOrTrashId(metaId)) - metaSize++; - } - - int dataSize = getDataCache(igfs).size(); - - int size = metaSize + dataSize; - - if (size <= 2) - return; // Caches are cleared, we're done. (2 because ROOT & TRASH always exist). - - X.println("Sum size: " + size); - - if (size > prevDifferentSize) { - X.println("Summary cache size has grown unexpectedly: size=" + size + ", prevSize=" + prevDifferentSize); - - break; - } - - if (totalCnt > totalThreshold) { - X.println("Timeout exceeded."); - - break; - } - - if (size == prevDifferentSize) { - constCnt++; - - if (constCnt == constThreshold) { - X.println("Summary cache size stays unchanged for too long: size=" + size); - - break; - } - } else { - constCnt = 0; - - prevDifferentSize = size; // renew; - } - - Thread.sleep(sleepPeriod); - - totalCnt++; - } - - dumpCache("MetaCache" , getMetaCache(igfs)); - - dumpCache("DataCache" , getDataCache(igfs)); - - fail("Caches are not empty."); - } - - /** - * Dumps given cache for diagnostic purposes. - * - * @param cacheName Name. - * @param cache The cache. - */ - private static void dumpCache(String cacheName, GridCacheAdapter cache) { - X.println("=============================== " + cacheName + " cache dump: "); - - Iterable entries = cache.entries(); - - for (GridCacheEntryEx e: entries) - X.println("Lost " + cacheName + " entry = " + e); - } - - /** - * Clear particular {@link IgfsSecondaryFileSystemTestAdapter}. - * - * @param uni IGFS. - * @throws Exception If failed. - */ - @SuppressWarnings("unchecked") - public static void clear(IgfsSecondaryFileSystemTestAdapter uni) throws Exception { - IgfsEx igfsEx = uni.igfs(); - - if (igfsEx != null) - clear(igfsEx); - - // Clear the filesystem. - uni.format(); - } - - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - clear(igfs, igfsSecondary); - } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java index c2f56332a823c..1d1ce8d76cf7a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java @@ -17,12 +17,21 @@ package org.apache.ignite.internal.processors.igfs; +import org.apache.ignite.igfs.IgfsFile; import org.apache.ignite.igfs.IgfsMode; +import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; import org.apache.ignite.igfs.secondary.local.LocalIgfsSecondaryFileSystem; +import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.file.Files; +import java.util.Collection; /** * Abstract test for Hadoop 1.0 file system stack. @@ -32,6 +41,24 @@ public abstract class IgfsLocalSecondaryFileSystemDualAbstractSelfTest extends I private static final String FS_WORK_DIR = U.getIgniteHome() + File.separatorChar + "work" + File.separatorChar + "fs"; + /** */ + private static final String FS_EXT_DIR = U.getIgniteHome() + File.separatorChar + "work" + + File.separatorChar + "ext"; + + /** */ + private final File dirLinkDest = new File(FS_EXT_DIR + File.separatorChar + "extdir"); + + /** */ + private final File fileLinkDest = + new File(FS_EXT_DIR + File.separatorChar + "extdir" + File.separatorChar + "filedest"); + + /** */ + private final File dirLinkSrc = new File(FS_WORK_DIR + File.separatorChar + "dir"); + + /** */ + private final File fileLinkSrc = new File(FS_WORK_DIR + File.separatorChar + "file"); + + /** Constructor. * @param mode IGFS mode. */ @@ -39,6 +66,19 @@ public IgfsLocalSecondaryFileSystemDualAbstractSelfTest(IgfsMode mode) { super(mode); } + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + final File extDir = new File(FS_EXT_DIR); + + if (!extDir.exists()) + assert extDir.mkdirs(); + else + cleanDirectory(extDir); + } + + /** * Creates secondary filesystems. * @return IgfsSecondaryFileSystem @@ -73,4 +113,107 @@ public IgfsLocalSecondaryFileSystemDualAbstractSelfTest(IgfsMode mode) { @Override protected boolean timesSupported() { return false; } + + /** + * + * @throws Exception If failed. + */ + @SuppressWarnings("ConstantConditions") + public void testListPathForSymlink() throws Exception { + if (U.isWindows()) + return; + + createSymlinks(); + + assertTrue(igfs.info(DIR).isDirectory()); + + Collection pathes = igfs.listPaths(DIR); + Collection files = igfs.listFiles(DIR); + + assertEquals(1, pathes.size()); + assertEquals(1, files.size()); + + assertEquals("filedest", F.first(pathes).name()); + assertEquals("filedest", F.first(files).path().name()); + } + + /** + * + * @throws Exception If failed. + */ + public void testDeleteSymlinkDir() throws Exception { + if (U.isWindows()) + return; + + createSymlinks(); + + // Only symlink must be deleted. Destination content must be exist. + igfs.delete(DIR, true); + + assertTrue(fileLinkDest.exists()); + } + + /** + * + * @throws Exception If failed. + */ + public void testSymlinkToFile() throws Exception { + if (U.isWindows()) + return; + + createSymlinks(); + + checkFileContent(igfs, new IgfsPath("/file"), chunk); + } + + /** + * + * @throws Exception If failed. + */ + private void createSymlinks() throws Exception { + assert dirLinkDest.mkdir(); + + createFile(fileLinkDest, true, chunk); + + Files.createSymbolicLink(dirLinkSrc.toPath(), dirLinkDest.toPath()); + Files.createSymbolicLink(fileLinkSrc.toPath(), fileLinkDest.toPath()); + } + + /** + * @param dir Directory to clean. + */ + private static void cleanDirectory(File dir){ + File[] entries = dir.listFiles(); + + if (entries != null) { + for (File entry : entries) { + if (entry.isDirectory()) { + cleanDirectory(entry); + + assert entry.delete(); + } + else + assert entry.delete(); + } + } + } + + /** + * @param f File object. + * @param overwrite Overwrite flag. + * @param chunks File content. + * @throws IOException If failed. + */ + private static void createFile(File f, boolean overwrite, @Nullable byte[]... chunks) throws IOException { + OutputStream os = null; + + try { + os = new FileOutputStream(f, overwrite); + + writeFileChunks(os, chunks); + } + finally { + U.closeQuiet(os); + } + } } \ No newline at end of file From 289170346e40a89243b80d3743c1b66543a117ef Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Mon, 29 Aug 2016 15:00:08 +0300 Subject: [PATCH 069/487] IGNITE-3773: ODBC: Added tests for UUID data type. This closes #993. --- .../odbc/escape/OdbcEscapeUtils.java | 2 +- modules/platforms/cpp/odbc-test/Makefile.am | 5 +- .../cpp/odbc-test/include/Makefile.am | 2 +- ...ite_fixture.h => sql_test_suite_fixture.h} | 45 ++++++------- .../odbc-test/project/vs/odbc-test.vcxproj | 7 ++- .../project/vs/odbc-test.vcxproj.filters | 13 ++-- .../src/sql_aggregate_functions_test.cpp | 4 +- .../src/sql_numeric_functions_test.cpp | 4 +- .../cpp/odbc-test/src/sql_operators_test.cpp | 4 +- .../src/sql_string_functions_test.cpp | 4 +- .../src/sql_system_functions_test.cpp | 4 +- ...fixture.cpp => sql_test_suite_fixture.cpp} | 63 ++++++++++++------- .../cpp/odbc-test/src/sql_types_test.cpp | 60 ++++++++++++++++++ ...ons.cpp => sql_value_expressions_test.cpp} | 4 +- 14 files changed, 154 insertions(+), 67 deletions(-) rename modules/platforms/cpp/odbc-test/include/{sql_function_test_suite_fixture.h => sql_test_suite_fixture.h} (68%) rename modules/platforms/cpp/odbc-test/src/{sql_function_test_suite_fixture.cpp => sql_test_suite_fixture.cpp} (70%) create mode 100644 modules/platforms/cpp/odbc-test/src/sql_types_test.cpp rename modules/platforms/cpp/odbc-test/src/{sql_value_expressions.cpp => sql_value_expressions_test.cpp} (97%) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java index 4c1deb6110772..a4b89c399c72b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java @@ -291,7 +291,7 @@ private static String parseExpression(String text, int startPos, int len, OdbcEs if (!pattern.matcher(val).matches()) throw new IgniteException("Invalid " + type + " escape sequence: " + substring(text, startPos, len)); - return val; + return "CAST(" + val + " AS UUID)"; } /** diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index a3f954efc05b6..de8fb5d08aae6 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -62,13 +62,14 @@ ignite_odbc_tests_SOURCES = \ src/utility_test.cpp \ src/queries_test.cpp \ src/test_utils.cpp \ - src/sql_function_test_suite_fixture.cpp \ + src/sql_test_suite_fixture.cpp \ src/sql_string_functions_test.cpp \ src/sql_numeric_functions_test.cpp \ src/sql_aggregate_functions_test.cpp \ src/sql_system_functions_test.cpp \ src/sql_operators_test.cpp \ - src/sql_value_expressions.cpp \ + src/sql_value_expressions_test.cpp \ + src/sql_types_test.cpp \ ../odbc/src/cursor.cpp \ ../odbc/src/config/connection_info.cpp \ ../odbc/src/app/application_data_buffer.cpp \ diff --git a/modules/platforms/cpp/odbc-test/include/Makefile.am b/modules/platforms/cpp/odbc-test/include/Makefile.am index baca1b7d6e42b..832103ce7ab3a 100644 --- a/modules/platforms/cpp/odbc-test/include/Makefile.am +++ b/modules/platforms/cpp/odbc-test/include/Makefile.am @@ -21,5 +21,5 @@ noinst_HEADERS = \ teamcity/teamcity_messages.h \ test_type.h \ test_utils.h \ - sql_function_test_suite_fixture.h + sql_test_suite_fixture.h diff --git a/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h b/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h similarity index 68% rename from modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h rename to modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h index 57c3f0f367355..9e482da4cd449 100644 --- a/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h +++ b/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h @@ -15,8 +15,8 @@ * limitations under the License. */ -#ifndef _IGNITE_ODBC_TEST_SQL_FUNCTION_TEST_FIXTURE -#define _IGNITE_ODBC_TEST_SQL_FUNCTION_TEST_FIXTURE +#ifndef _IGNITE_ODBC_TEST_SQL_TEST_SUIT_FIXTURE +#define _IGNITE_ODBC_TEST_SQL_TEST_SUIT_FIXTURE #ifdef _WIN32 # include @@ -43,17 +43,17 @@ namespace ignite /** * Test setup fixture. */ - struct SqlFunctionTestSuiteFixture + struct SqlTestSuiteFixture { /** * Constructor. */ - SqlFunctionTestSuiteFixture(); + SqlTestSuiteFixture(); /** * Destructor. */ - ~SqlFunctionTestSuiteFixture(); + ~SqlTestSuiteFixture(); /** * Run query returning single result and stores it to buffer. @@ -140,49 +140,52 @@ namespace ignite }; template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const std::string& expected); + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const std::string& expected); template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int64_t& expected); + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const int64_t& expected); template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int32_t& expected); + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const int32_t& expected); template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int16_t& expected); + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const int16_t& expected); template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int8_t& expected); + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const int8_t& expected); template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const float& expected); + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const float& expected); template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const double& expected); + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const double& expected); template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const bool& expected); + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const bool& expected); template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const ignite::Guid& expected); template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); + void SqlTestSuiteFixture::CheckSingleResult(const char* request); template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); + void SqlTestSuiteFixture::CheckSingleResult(const char* request); template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); + void SqlTestSuiteFixture::CheckSingleResult(const char* request); template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); + void SqlTestSuiteFixture::CheckSingleResult(const char* request); template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); + void SqlTestSuiteFixture::CheckSingleResult(const char* request); template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request); + void SqlTestSuiteFixture::CheckSingleResult(const char* request); + + template<> + void SqlTestSuiteFixture::CheckSingleResult(const char* request); } -#endif //_IGNITE_ODBC_TEST_SQL_FUNCTION_TEST_FIXTURE +#endif //_IGNITE_ODBC_TEST_SQL_TEST_SUIT_FIXTURE diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index 8b4bee66d197b..a9423ca9bd0ea 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -169,19 +169,20 @@ - + - + + - + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index 4756f8328a957..84f5a29a38114 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -85,9 +85,6 @@ Code - - Code - Code @@ -97,7 +94,13 @@ Code - + + Code + + + Code + + Code @@ -111,7 +114,7 @@ Code - + Code diff --git a/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp index 6254913976d99..3fa4d970f06bc 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp @@ -41,7 +41,7 @@ #include "test_type.h" #include "test_utils.h" -#include "sql_function_test_suite_fixture.h" +#include "sql_test_suite_fixture.h" using namespace ignite; using namespace ignite::cache; @@ -52,7 +52,7 @@ using namespace boost::unit_test; using ignite::impl::binary::BinaryUtils; -BOOST_FIXTURE_TEST_SUITE(SqlAggregateFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture) +BOOST_FIXTURE_TEST_SUITE(SqlAggregateFunctionTestSuite, ignite::SqlTestSuiteFixture) BOOST_AUTO_TEST_CASE(TestAggregateFunctionAvgInt) { diff --git a/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp index 80ffd83164320..e16d8c1124291 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp @@ -41,7 +41,7 @@ #include "test_type.h" #include "test_utils.h" -#include "sql_function_test_suite_fixture.h" +#include "sql_test_suite_fixture.h" using namespace ignite; using namespace ignite::cache; @@ -52,7 +52,7 @@ using namespace boost::unit_test; using ignite::impl::binary::BinaryUtils; -BOOST_FIXTURE_TEST_SUITE(SqlNumericFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture) +BOOST_FIXTURE_TEST_SUITE(SqlNumericFunctionTestSuite, ignite::SqlTestSuiteFixture) BOOST_AUTO_TEST_CASE(TestNumericFunctionAbs) { diff --git a/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp index db3d3f7bafc1e..de884cabd80b2 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp @@ -29,12 +29,12 @@ #include #include "test_type.h" -#include "sql_function_test_suite_fixture.h" +#include "sql_test_suite_fixture.h" #include using namespace ignite; -BOOST_FIXTURE_TEST_SUITE(SqlOperatorTestSuite, ignite::SqlFunctionTestSuiteFixture) +BOOST_FIXTURE_TEST_SUITE(SqlOperatorTestSuite, ignite::SqlTestSuiteFixture) BOOST_AUTO_TEST_CASE(TestOperatorAddInt) { diff --git a/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp index d22bb834ca2d4..d1ce194c1abe4 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp @@ -37,7 +37,7 @@ #include "test_type.h" #include "test_utils.h" -#include "sql_function_test_suite_fixture.h" +#include "sql_test_suite_fixture.h" using namespace ignite; using namespace ignite::cache; @@ -48,7 +48,7 @@ using namespace boost::unit_test; using ignite::impl::binary::BinaryUtils; -BOOST_FIXTURE_TEST_SUITE(SqlStringFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture) +BOOST_FIXTURE_TEST_SUITE(SqlStringFunctionTestSuite, ignite::SqlTestSuiteFixture) BOOST_AUTO_TEST_CASE(TestStringFunctionAscii) { diff --git a/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp index 752925d974ba5..30fa0364e105f 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp @@ -21,13 +21,13 @@ #include -#include "sql_function_test_suite_fixture.h" +#include "sql_test_suite_fixture.h" using namespace ignite; using namespace boost::unit_test; -BOOST_FIXTURE_TEST_SUITE(SqlSystemFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture) +BOOST_FIXTURE_TEST_SUITE(SqlSystemFunctionTestSuite, ignite::SqlTestSuiteFixture) BOOST_AUTO_TEST_CASE(TestSystemFunctionDatabase) { diff --git a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp similarity index 70% rename from modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp rename to modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp index d6b175fff04b2..16e5ea03ab091 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp @@ -15,13 +15,13 @@ * limitations under the License. */ -#include "sql_function_test_suite_fixture.h" +#include "sql_test_suite_fixture.h" #include "test_utils.h" namespace ignite { - SqlFunctionTestSuiteFixture::SqlFunctionTestSuiteFixture(): + SqlTestSuiteFixture::SqlTestSuiteFixture(): testCache(0), env(NULL), dbc(NULL), @@ -94,7 +94,7 @@ namespace ignite BOOST_REQUIRE(stmt != NULL) ; } - SqlFunctionTestSuiteFixture::~SqlFunctionTestSuiteFixture() + SqlTestSuiteFixture::~SqlTestSuiteFixture() { // Releasing statement handle. SQLFreeHandle(SQL_HANDLE_STMT, stmt); @@ -109,7 +109,7 @@ namespace ignite ignite::Ignition::Stop(grid.GetName(), true); } - void SqlFunctionTestSuiteFixture::CheckSingleResult0(const char* request, + void SqlTestSuiteFixture::CheckSingleResult0(const char* request, SQLSMALLINT type, void* column, SQLLEN bufSize, SQLLEN* resSize) const { SQLRETURN ret; @@ -117,22 +117,24 @@ namespace ignite ret = SQLBindCol(stmt, 1, type, column, bufSize, resSize); if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ; + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ; ret = SQLExecDirect(stmt, reinterpret_cast(const_cast(request)), SQL_NTS); + if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ; + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ; ret = SQLFetch(stmt); + if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ; + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ; ret = SQLFetch(stmt); BOOST_CHECK(ret == SQL_NO_DATA) ; } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const std::string& expected) + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const std::string& expected) { SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 }; SQLLEN resLen = 0; @@ -148,31 +150,31 @@ namespace ignite } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int64_t& expected) + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const int64_t& expected) { CheckSingleResultNum0(request, expected, SQL_C_SBIGINT); } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int32_t& expected) + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const int32_t& expected) { CheckSingleResultNum0(request, expected, SQL_C_SLONG); } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int16_t& expected) + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const int16_t& expected) { CheckSingleResultNum0(request, expected, SQL_C_SSHORT); } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const int8_t& expected) + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const int8_t& expected) { CheckSingleResultNum0(request, expected, SQL_C_STINYINT); } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const float& expected) + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const float& expected) { SQLFLOAT res = 0; @@ -182,7 +184,7 @@ namespace ignite } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const double& expected) + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const double& expected) { SQLDOUBLE res = 0; @@ -192,7 +194,7 @@ namespace ignite } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request, const bool& expected) + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const bool& expected) { SQLCHAR res = 0; @@ -202,7 +204,24 @@ namespace ignite } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) + void SqlTestSuiteFixture::CheckSingleResult(const char* request, const ignite::Guid& expected) + { + SQLGUID res; + + memset(&res, 0, sizeof(res)); + + CheckSingleResult0(request, SQL_C_GUID, &res, 0, 0); + + BOOST_CHECK_EQUAL(res.Data1, expected.GetMostSignificantBits() & 0xFFFFFFFF00000000ULL >> 32); + BOOST_CHECK_EQUAL(res.Data2, expected.GetMostSignificantBits() & 0x00000000FFFF0000ULL >> 16); + BOOST_CHECK_EQUAL(res.Data3, expected.GetMostSignificantBits() & 0x000000000000FFFFULL); + + for (int i = 0; i < sizeof(res.Data4); ++i) + BOOST_CHECK_EQUAL(res.Data4[i], (expected.GetLeastSignificantBits() & (0xFFULL << (8 * i))) >> (8 * i)); + } + + template<> + void SqlTestSuiteFixture::CheckSingleResult(const char* request) { SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 }; SQLLEN resLen = 0; @@ -211,31 +230,31 @@ namespace ignite } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) + void SqlTestSuiteFixture::CheckSingleResult(const char* request) { CheckSingleResultNum0(request, SQL_C_SBIGINT); } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) + void SqlTestSuiteFixture::CheckSingleResult(const char* request) { CheckSingleResultNum0(request, SQL_C_SLONG); } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) + void SqlTestSuiteFixture::CheckSingleResult(const char* request) { CheckSingleResultNum0(request, SQL_C_SSHORT); } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) + void SqlTestSuiteFixture::CheckSingleResult(const char* request) { CheckSingleResultNum0(request, SQL_C_STINYINT); } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) + void SqlTestSuiteFixture::CheckSingleResult(const char* request) { SQLFLOAT res = 0; @@ -243,7 +262,7 @@ namespace ignite } template<> - void SqlFunctionTestSuiteFixture::CheckSingleResult(const char* request) + void SqlTestSuiteFixture::CheckSingleResult(const char* request) { SQLDOUBLE res = 0; diff --git a/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp new file mode 100644 index 0000000000000..ccbb9176efa6e --- /dev/null +++ b/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp @@ -0,0 +1,60 @@ +/* + * 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. + */ + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include "sql_test_suite_fixture.h" + +using namespace ignite; + +using namespace boost::unit_test; + +BOOST_FIXTURE_TEST_SUITE(SqlTypesTestSuite, ignite::SqlTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(TestGuidTrivial) +{ + CheckSingleResult("SELECT {guid '04CC382A-0B82-F520-08D0-07A0620C0004'}", + "04cc382a-0b82-f520-08d0-07a0620c0004"); + + CheckSingleResult("SELECT {guid '63802467-9f4a-4f71-8fc8-cf2d99a28ddf'}", + "63802467-9f4a-4f71-8fc8-cf2d99a28ddf"); +} + +BOOST_AUTO_TEST_CASE(TestGuidEqualsToColumn) +{ + TestType in1; + TestType in2; + + in1.guidField = Guid(0x638024679f4a4f71, 0x8fc8cf2d99a28ddf); + in2.guidField = Guid(0x04cc382a0b82f520, 0x08d007a0620c0004); + + in1.i32Field = 1; + in2.i32Field = 2; + + testCache.Put(1, in1); + testCache.Put(2, in2); + + CheckSingleResult( + "SELECT i32Field FROM TestType WHERE guidField = {guid '04cc382a-0b82-f520-08d0-07a0620c0004'}", in2.i32Field); +} + + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp b/modules/platforms/cpp/odbc-test/src/sql_value_expressions_test.cpp similarity index 97% rename from modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp rename to modules/platforms/cpp/odbc-test/src/sql_value_expressions_test.cpp index e7d6598253a2f..eb9b6ea185134 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_value_expressions_test.cpp @@ -21,13 +21,13 @@ #include -#include "sql_function_test_suite_fixture.h" +#include "sql_test_suite_fixture.h" using namespace ignite; using namespace boost::unit_test; -BOOST_FIXTURE_TEST_SUITE(SqlValueExpressionTestSuite, ignite::SqlFunctionTestSuiteFixture) +BOOST_FIXTURE_TEST_SUITE(SqlValueExpressionTestSuite, ignite::SqlTestSuiteFixture) BOOST_AUTO_TEST_CASE(TestCase) { From 0465874d9dddcf962a82a2ef38589121201f0b75 Mon Sep 17 00:00:00 2001 From: agura Date: Wed, 24 Aug 2016 21:13:29 +0300 Subject: [PATCH 070/487] ignite-2968 Deadlock detection for optimistic tx and near caches --- .../processors/cache/GridCacheMapEntry.java | 19 +- .../GridCachePartitionExchangeManager.java | 7 + .../GridDistributedTxPrepareRequest.java | 4 +- .../distributed/dht/GridDhtLockFuture.java | 53 +- .../dht/GridDhtTxFinishFuture.java | 4 +- .../cache/distributed/dht/GridDhtTxLocal.java | 26 +- .../dht/GridDhtTxPrepareFuture.java | 109 +++- .../dht/GridDhtTxPrepareRequest.java | 4 +- .../colocated/GridDhtColocatedLockFuture.java | 37 +- .../distributed/near/GridNearLockFuture.java | 90 ++- ...OptimisticSerializableTxPrepareFuture.java | 13 +- .../GridNearOptimisticTxPrepareFuture.java | 263 +++++--- ...dNearOptimisticTxPrepareFutureAdapter.java | 5 +- .../GridNearPessimisticTxPrepareFuture.java | 8 +- .../near/GridNearTxFinishFuture.java | 5 +- .../distributed/near/GridNearTxLocal.java | 16 +- .../near/GridNearTxPrepareRequest.java | 4 +- .../cache/transactions/IgniteInternalTx.java | 3 +- .../cache/transactions/IgniteTxAdapter.java | 37 +- .../cache/transactions/IgniteTxHandler.java | 9 +- .../transactions/IgniteTxLocalAdapter.java | 19 +- .../cache/transactions/IgniteTxManager.java | 86 ++- .../cache/transactions/IgniteTxStateImpl.java | 11 +- .../transactions/TxDeadlockDetection.java | 51 +- .../cache/IgniteTxConfigCacheSelfTest.java | 91 ++- .../IgniteTxTimeoutAbstractTest.java | 8 +- ...onedMultiNodeLongTxTimeoutFullApiTest.java | 34 ++ ...bledMultiNodeLongTxTimeoutFullApiTest.java | 41 ++ .../GridCacheLocalTxTimeoutSelfTest.java | 5 +- .../transactions/DepthFirstSearchTest.java | 100 ++- .../TxDeadlockDetectionNoHangsTest.java | 246 ++++++++ .../transactions/TxDeadlockDetectionTest.java | 13 +- ...misticDeadlockDetectionCrossCacheTest.java | 257 ++++++++ .../TxOptimisticDeadlockDetectionTest.java | 574 ++++++++++++++++++ ...misticDeadlockDetectionCrossCacheTest.java | 165 +++-- .../TxPessimisticDeadlockDetectionTest.java | 50 +- .../IgniteCacheFullApiSelfTestSuite.java | 4 + .../TxDeadlockDetectionTestSuite.java | 6 + .../cache/VisorCacheStopCommand.scala | 5 +- 39 files changed, 2127 insertions(+), 355 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedMultiNodeLongTxTimeoutFullApiTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionCrossCacheTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 57fa68e758c51..f692bf40592a5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -4493,17 +4493,30 @@ protected void decrementMapPublicSize() { } /** - * @return All MVCC local candidates. + * @return All MVCC local and non near candidates. */ + @SuppressWarnings("ForLoopReplaceableByForEach") @Nullable public synchronized List mvccAllLocal() { GridCacheMvcc mvcc = extras != null ? extras.mvcc() : null; if (mvcc == null) return null; - List locs = mvcc.allLocal(); + List allLocs = mvcc.allLocal(); - return (locs == null || locs.isEmpty()) ? null : new ArrayList<>(locs); + if (allLocs == null || allLocs.isEmpty()) + return null; + + List locs = new ArrayList<>(allLocs.size()); + + for (int i = 0; i < allLocs.size(); i++) { + GridCacheMvccCandidate loc = allLocs.get(i); + + if (!loc.nearLocal()) + locs.add(loc); + } + + return locs.isEmpty() ? null : locs; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index e6ab046d52ec7..4eb61e3922047 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -1302,6 +1302,13 @@ private void dumpPendingObjects(@Nullable AffinityTopologyVersion exchTopVer) { for (GridCacheFuture fut : mvcc.atomicFutures()) U.warn(log, ">>> " + fut); + + if (tm != null) { + U.warn(log, "Pending transaction deadlock detection futures:"); + + for (IgniteInternalFuture fut : tm.deadlockDetectionFutures()) + U.warn(log, ">>> " + fut); + } } for (GridCacheContext ctx : cctx.cacheContexts()) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java index 72e68db5743e5..c6913742399cb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java @@ -154,6 +154,7 @@ public GridDistributedTxPrepareRequest() { /** * @param tx Cache transaction. + * @param timeout Transactions timeout. * @param reads Read entries. * @param writes Write entries. * @param txNodes Transaction nodes mapping. @@ -162,6 +163,7 @@ public GridDistributedTxPrepareRequest() { */ public GridDistributedTxPrepareRequest( IgniteInternalTx tx, + long timeout, @Nullable Collection reads, Collection writes, Map> txNodes, @@ -174,12 +176,12 @@ public GridDistributedTxPrepareRequest( threadId = tx.threadId(); concurrency = tx.concurrency(); isolation = tx.isolation(); - timeout = tx.timeout(); invalidate = tx.isInvalidate(); txSize = tx.size(); sys = tx.system(); plc = tx.ioPolicy(); + this.timeout = timeout; this.reads = reads; this.writes = writes; this.txNodes = txNodes; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java index 64b874577fb56..b005b2951fdad 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java @@ -242,12 +242,6 @@ public GridDhtLockFuture( msgLog = cctx.shared().txLockMessageLogger(); log = U.logger(cctx.kernalContext(), logRef, GridDhtLockFuture.class); } - - if (timeout > 0) { - timeoutObj = new LockTimeoutObject(); - - cctx.time().addTimeoutObject(timeoutObj); - } } /** {@inheritDoc} */ @@ -298,8 +292,10 @@ public Collection entries() { /** * @return Entries. */ - public synchronized Collection entriesCopy() { - return new ArrayList<>(entries()); + public Collection entriesCopy() { + synchronized (futs) { + return new ArrayList<>(entries()); + } } /** @@ -412,7 +408,7 @@ private boolean locked(GridCacheEntryEx cached, GridCacheMvccCandidate owner) { return null; } - synchronized (this) { + synchronized (futs) { entries.add(c == null || c.reentry() ? null : entry); if (c != null && !c.reentry()) @@ -614,7 +610,7 @@ private void readyLocks() { * @param t Error. */ public void onError(Throwable t) { - synchronized (this) { + synchronized (futs) { if (err != null) return; @@ -654,15 +650,16 @@ private boolean filter(GridCacheEntryEx cached) { * @param entry Entry whose lock ownership changed. */ @Override public boolean onOwnerChanged(GridCacheEntryEx entry, GridCacheMvccCandidate owner) { - if (isDone()) + if (isDone() || (inTx() && tx.remainingTime() == -1)) return false; // Check other futures. if (log.isDebugEnabled()) log.debug("Received onOwnerChanged() callback [entry=" + entry + ", owner=" + owner + "]"); if (owner != null && owner.version().equals(lockVer)) { - synchronized (this) { - pendingLocks.remove(entry.key()); + synchronized (futs) { + if (!pendingLocks.remove(entry.key())) + return false; } if (checkLocks()) @@ -677,8 +674,10 @@ private boolean filter(GridCacheEntryEx cached) { /** * @return {@code True} if locks have been acquired. */ - private synchronized boolean checkLocks() { - return pendingLocks.isEmpty(); + private boolean checkLocks() { + synchronized (futs) { + return pendingLocks.isEmpty(); + } } /** {@inheritDoc} */ @@ -709,7 +708,7 @@ private synchronized boolean checkLocks() { if (isDone() || (err == null && success && !checkLocks())) return false; - synchronized (this) { + synchronized (futs) { if (this.err == null) this.err = err; } @@ -776,13 +775,19 @@ public void map() { } readyLocks(); + + if (timeout > 0) { + timeoutObj = new LockTimeoutObject(); + + cctx.time().addTimeoutObject(timeoutObj); + } } /** * @param entries Entries. */ private void map(Iterable entries) { - synchronized (this) { + synchronized (futs) { if (mapped) return; @@ -842,6 +847,8 @@ private void map(Iterable entries) { if (log.isDebugEnabled()) log.debug("Mapped DHT lock future [dhtMap=" + F.nodeIds(dhtMap.keySet()) + ", dhtLockFut=" + this + ']'); + long timeout = inTx() ? tx.remainingTime() : this.timeout; + // Create mini futures. for (Map.Entry> mapped : dhtMap.entrySet()) { ClusterNode n = mapped.getKey(); @@ -853,6 +860,9 @@ private void map(Iterable entries) { if (cnt > 0) { assert !n.id().equals(cctx.localNodeId()); + if (inTx() && tx.remainingTime() == -1) + return; + MiniFuture fut = new MiniFuture(n, dhtMapping); GridDhtLockRequest req = new GridDhtLockRequest( @@ -1109,7 +1119,14 @@ private class LockTimeoutObject extends GridTimeoutObjectAdapter { if (log.isDebugEnabled()) log.debug("Timed out waiting for lock response: " + this); - timedOut = true; + synchronized (futs) { + timedOut = true; + + // Stop locks and responses processing. + pendingLocks.clear(); + + futs.clear(); + } boolean releaseLocks = !(inTx() && cctx.tm().deadlockDetectionEnabled()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java index 4ece7758459cb..d2e26b4415f10 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java @@ -133,6 +133,7 @@ public GridDhtTxLocalAdapter tx() { } /** {@inheritDoc} */ + @SuppressWarnings("unchecked") @Override public boolean onNodeLeft(UUID nodeId) { for (IgniteInternalFuture fut : futures()) if (isMini(fut)) { @@ -391,8 +392,7 @@ private boolean rollbackLockTransactions(Collection nodes) { * @param nearMap Near map. * @return {@code True} in case there is at least one synchronous {@code MiniFuture} to wait for. */ - private boolean finish(Map dhtMap, - Map nearMap) { + private boolean finish(Map dhtMap, Map nearMap) { if (tx.onePhaseCommit()) return false; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java index c9d4345d33542..b659abb8ef679 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java @@ -314,6 +314,8 @@ public void nearFinishMiniId(IgniteUuid nearFinMiniId) { true); } + long timeout = remainingTime(); + // For pessimistic mode we don't distribute prepare request. GridDhtTxPrepareFuture fut = prepFut; @@ -322,11 +324,16 @@ public void nearFinishMiniId(IgniteUuid nearFinMiniId) { if (!PREP_FUT_UPD.compareAndSet(this, null, fut = new GridDhtTxPrepareFuture( cctx, this, + timeout, nearMiniId, Collections.emptyMap(), true, - needReturnValue()))) + needReturnValue()))) { + if (timeout == -1) + prepFut.onError(timeoutException()); + return prepFut; + } } else // Prepare was called explicitly. @@ -334,15 +341,16 @@ public void nearFinishMiniId(IgniteUuid nearFinMiniId) { if (!state(PREPARING)) { if (setRollbackOnly()) { - if (timedOut()) - fut.onError(new IgniteTxTimeoutCheckedException("Transaction timed out and was rolled back: " + this)); + if (timeout == -1) + fut.onError(new IgniteTxTimeoutCheckedException("Transaction timed out and was rolled back: " + + this)); else fut.onError(new IgniteCheckedException("Invalid transaction state for prepare [state=" + state() + ", tx=" + this + ']')); } else - fut.onError(new IgniteTxRollbackCheckedException("Invalid transaction state for prepare [state=" + state() - + ", tx=" + this + ']')); + fut.onError(new IgniteTxRollbackCheckedException("Invalid transaction state for prepare [state=" + + state() + ", tx=" + this + ']')); return fut; } @@ -394,6 +402,8 @@ public IgniteInternalFuture prepareAsync( // In optimistic mode prepare still can be called explicitly from salvageTx. GridDhtTxPrepareFuture fut = prepFut; + long timeout = remainingTime(); + if (fut == null) { init(); @@ -401,6 +411,7 @@ public IgniteInternalFuture prepareAsync( if (!PREP_FUT_UPD.compareAndSet(this, null, fut = new GridDhtTxPrepareFuture( cctx, this, + timeout, nearMiniId, verMap, last, @@ -410,6 +421,9 @@ public IgniteInternalFuture prepareAsync( assert f.nearMiniId().equals(nearMiniId) : "Wrong near mini id on existing future " + "[futMiniId=" + f.nearMiniId() + ", miniId=" + nearMiniId + ", fut=" + f + ']'; + if (timeout == -1) + f.onError(timeoutException()); + return chainOnePhasePrepare(f); } } @@ -427,7 +441,7 @@ public IgniteInternalFuture prepareAsync( fut.complete(); if (setRollbackOnly()) { - if (timedOut()) + if (timeout == -1) fut.onError(new IgniteTxTimeoutCheckedException("Transaction timed out and was rolled back: " + this)); else diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java index e9805aa033abf..1bdd9b8ab7b3e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java @@ -59,8 +59,10 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.dr.GridDrType; +import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException; +import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridLeanSet; import org.apache.ignite.internal.util.future.GridCompoundFuture; @@ -204,9 +206,13 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture dhtVerMap, boolean last, @@ -243,6 +250,8 @@ public GridDhtTxPrepareFuture( assert dhtMap != null; assert nearMap != null; + + timeoutObj = timeout > 0 ? new PrepareTimeoutObject(timeout) : null; } /** {@inheritDoc} */ @@ -269,7 +278,7 @@ public IgniteUuid nearMiniId() { boolean rmv; - synchronized (lockKeys) { + synchronized (futs) { rmv = lockKeys.remove(entry.txKey()); } @@ -300,7 +309,7 @@ private boolean checkLocks() { if (!locksReady) return false; - synchronized (lockKeys) { + synchronized (futs) { return lockKeys.isEmpty(); } } @@ -483,32 +492,28 @@ public void onError(Throwable t) { * @param res Result. */ public void onResult(UUID nodeId, GridDhtTxPrepareResponse res) { - if (!isDone()) { - boolean found = false; - - MiniFuture mini = miniFuture(res.miniId()); + if (isDone()) { + if (msgLog.isDebugEnabled()) { + msgLog.debug("DHT prepare fut, response for finished future [txId=" + tx.nearXidVersion() + + ", dhtTxId=" + tx.xidVersion() + + ", node=" + nodeId + + ", res=" + res + + ", fut=" + this + ']'); + } - if (mini != null) { - found = true; + return; + } - assert mini.node().id().equals(nodeId); + MiniFuture mini = miniFuture(res.miniId()); - mini.onResult(res); - } + if (mini != null) { + assert mini.node().id().equals(nodeId); - if (!found) { - if (msgLog.isDebugEnabled()) { - msgLog.debug("DHT prepare fut, failed to find mini future [txId=" + tx.nearXidVersion() + - ", dhtTxId=" + tx.xidVersion() + - ", node=" + nodeId + - ", res=" + res + - ", fut=" + this + ']'); - } - } + mini.onResult(res); } else { if (msgLog.isDebugEnabled()) { - msgLog.debug("DHT prepare fut, response for finished future [txId=" + tx.nearXidVersion() + + msgLog.debug("DHT prepare fut, failed to find mini future [txId=" + tx.nearXidVersion() + ", dhtTxId=" + tx.xidVersion() + ", node=" + nodeId + ", res=" + res + @@ -525,8 +530,8 @@ public void onResult(UUID nodeId, GridDhtTxPrepareResponse res) { */ @SuppressWarnings("ForLoopReplaceableByForEach") private MiniFuture miniFuture(IgniteUuid miniId) { - // We iterate directly over the futs collection here to avoid copy. synchronized (futs) { + // We iterate directly over the futs collection here to avoid copy. // Avoid iterator creation. for (int i = 0; i < futs.size(); i++) { IgniteInternalFuture fut = futs.get(i); @@ -543,9 +548,9 @@ private MiniFuture miniFuture(IgniteUuid miniId) { return null; } } - } - return null; + return null; + } } /** @@ -583,7 +588,7 @@ private void readyLocks(Iterable checkEntries) { } if (tx.optimistic() && txEntry.explicitVersion() == null) { - synchronized (lockKeys) { + synchronized (futs) { lockKeys.add(txEntry.txKey()); } } @@ -934,6 +939,9 @@ private boolean onComplete(@Nullable GridNearTxPrepareResponse res) { // Don't forget to clean up. cctx.mvcc().removeMvccFuture(this); + if (timeoutObj != null) + cctx.time().removeTimeoutObject(timeoutObj); + return true; } @@ -989,6 +997,11 @@ public void prepare(Collection reads, Collection w readyLocks(); + if (timeoutObj != null) { + // Start timeout tracking after 'readyLocks' to avoid race with timeout processing. + cctx.time().addTimeoutObject(timeoutObj); + } + mapIfLocked(); } @@ -1158,6 +1171,8 @@ private void prepare0() { if (last) { assert tx.transactionNodes() != null; + final long timeout = timeoutObj != null ? timeoutObj.timeout : 0; + // Create mini futures. for (GridDistributedTxMapping dhtMapping : tx.dhtMap().values()) { assert !dhtMapping.empty(); @@ -1175,6 +1190,9 @@ private void prepare0() { if (F.isEmpty(dhtWrites) && F.isEmpty(nearWrites)) continue; + if (tx.remainingTime() == -1) + return; + MiniFuture fut = new MiniFuture(n.id(), dhtMapping, nearMapping); add(fut); // Append new future. @@ -1186,6 +1204,7 @@ private void prepare0() { fut.futureId(), tx.topologyVersion(), tx, + timeout, dhtWrites, nearWrites, txNodes, @@ -1284,15 +1303,19 @@ private void prepare0() { for (GridDistributedTxMapping nearMapping : tx.nearMap().values()) { if (!tx.dhtMap().containsKey(nearMapping.node().id())) { + if (tx.remainingTime() == -1) + return; + MiniFuture fut = new MiniFuture(nearMapping.node().id(), null, nearMapping); - add(fut); // Append new future. + add(fut); GridDhtTxPrepareRequest req = new GridDhtTxPrepareRequest( futId, fut.futureId(), tx.topologyVersion(), tx, + timeout, null, nearMapping.writes(), tx.transactionNodes(), @@ -1719,4 +1742,38 @@ void onResult(GridDhtTxPrepareResponse res) { return S.toString(MiniFuture.class, this, "done", isDone(), "cancelled", isCancelled(), "err", error()); } } + + /** + * + */ + private class PrepareTimeoutObject extends GridTimeoutObjectAdapter { + /** */ + private final long timeout; + + /** + * @param timeout Timeout. + */ + PrepareTimeoutObject(long timeout) { + super(timeout); + + this.timeout = timeout; + } + + /** {@inheritDoc} */ + @Override public void onTimeout() { + synchronized (futs) { + futs.clear(); + + lockKeys.clear(); + } + + onError(new IgniteTxTimeoutCheckedException("Failed to acquire lock within " + + "provided timeout for transaction [timeout=" + tx.timeout() + ", tx=" + tx + ']')); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(PrepareTimeoutObject.class, this); + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java index d31ecbabeea6a..1cdc96fd74c05 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java @@ -112,6 +112,7 @@ public GridDhtTxPrepareRequest() { * @param miniId Mini future ID. * @param topVer Topology version. * @param tx Transaction. + * @param timeout Transaction timeout. * @param dhtWrites DHT writes. * @param nearWrites Near writes. * @param txNodes Transaction nodes mapping. @@ -124,6 +125,7 @@ public GridDhtTxPrepareRequest( IgniteUuid miniId, AffinityTopologyVersion topVer, GridDhtTxLocalAdapter tx, + long timeout, Collection dhtWrites, Collection nearWrites, Map> txNodes, @@ -133,7 +135,7 @@ public GridDhtTxPrepareRequest( UUID subjId, int taskNameHash, boolean addDepInfo) { - super(tx, null, dhtWrites, txNodes, onePhaseCommit, addDepInfo); + super(tx, timeout, null, dhtWrites, txNodes, onePhaseCommit, addDepInfo); assert futId != null; assert miniId != null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java index f77efeef9bac0..b0eea0191e808 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java @@ -443,23 +443,33 @@ void onResult(UUID nodeId, GridNearLockResponse res) { /** * @return Keys for which locks requested from remote nodes but response isn't received. */ - public Set requestedKeys() { - Set requestedKeys = null; + public Set requestedKeys() { + synchronized (futs) { + if (timeoutObj != null && timeoutObj.requestedKeys != null) + return timeoutObj.requestedKeys; + + return requestedKeys0(); + } + } + /** + * @return Keys for which locks requested from remote nodes but response isn't received. + */ + private Set requestedKeys0() { for (IgniteInternalFuture miniFut : futures()) { if (isMini(miniFut) && !miniFut.isDone()) { - if (requestedKeys == null) - requestedKeys = new HashSet<>(); - MiniFuture mini = (MiniFuture)miniFut; - requestedKeys.addAll(mini.keys); + Set requestedKeys = U.newHashSet(mini.keys.size()); + + for (KeyCacheObject key : mini.keys) + requestedKeys.add(new IgniteTxKey(key, cctx.cacheId())); return requestedKeys; } } - return requestedKeys; + return null; } /** @@ -1312,12 +1322,21 @@ private class LockTimeoutObject extends GridTimeoutObjectAdapter { super(timeout); } + /** Requested keys. */ + private Set requestedKeys; + /** {@inheritDoc} */ @Override public void onTimeout() { if (log.isDebugEnabled()) log.debug("Timed out waiting for lock response: " + this); if (inTx() && cctx.tm().deadlockDetectionEnabled()) { + synchronized (futs) { + requestedKeys = requestedKeys0(); + + futs.clear(); // Stop response processing. + } + Set keys = new HashSet<>(); for (IgniteTxEntry txEntry : tx.allEntries()) { @@ -1434,7 +1453,7 @@ void onResult(ClusterTopologyCheckedException e) { tx.removeMapping(node.id()); // Primary node left the grid, so fail the future. - GridDhtColocatedLockFuture.this.onDone(newTopologyException(e, node.id())); + GridDhtColocatedLockFuture.this.onDone(false, newTopologyException(e, node.id())); onDone(true); } @@ -1494,7 +1513,7 @@ void onResult(GridNearLockResponse res) { else remap(); } - else { + else { int i = 0; for (KeyCacheObject k : keys) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java index 4b6448bd2c9ad..3d9b6ab322aff 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java @@ -20,10 +20,12 @@ import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collection; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Queue; +import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; import org.apache.ignite.IgniteCheckedException; @@ -48,8 +50,10 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; +import org.apache.ignite.internal.processors.cache.transactions.TxDeadlock; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; +import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; import org.apache.ignite.internal.util.future.GridCompoundIdentityFuture; import org.apache.ignite.internal.util.future.GridEmbeddedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -63,7 +67,9 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.transactions.TransactionDeadlockException; import org.apache.ignite.transactions.TransactionIsolation; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -480,6 +486,38 @@ else if (log.isDebugEnabled()) ", fut=" + this + ']'); } + /** + * @return Keys for which locks requested from remote nodes but response isn't received. + */ + public Set requestedKeys() { + synchronized (futs) { + if (timeoutObj != null && timeoutObj.requestedKeys != null) + return timeoutObj.requestedKeys; + + return requestedKeys0(); + } + } + + /** + * @return Keys for which locks requested from remote nodes but response isn't received. + */ + private Set requestedKeys0() { + for (IgniteInternalFuture miniFut : futures()) { + if (isMini(miniFut) && !miniFut.isDone()) { + MiniFuture mini = (MiniFuture)miniFut; + + Set requestedKeys = U.newHashSet(mini.keys.size()); + + for (KeyCacheObject key : mini.keys) + requestedKeys.add(new IgniteTxKey(key, cctx.cacheId())); + + return requestedKeys; + } + } + + return null; + } + /** * Finds pending mini future by the given mini ID. * @@ -621,6 +659,10 @@ private boolean checkLocks() { if (log.isDebugEnabled()) log.debug("Received onDone(..) callback [success=" + success + ", err=" + err + ", fut=" + this + ']'); + if (inTx() && cctx.tm().deadlockDetectionEnabled() && + (this.err instanceof IgniteTxTimeoutCheckedException || timedOut)) + return false; + // If locks were not acquired yet, delay completion. if (isDone() || (err == null && success && !checkLocks())) return false; @@ -727,7 +769,7 @@ void map() { topVer = tx.topologyVersionSnapshot(); if (topVer != null) { - for (GridDhtTopologyFuture fut : cctx.shared().exchange().exchangeFutures()){ + for (GridDhtTopologyFuture fut : cctx.shared().exchange().exchangeFutures()) { if (fut.topologyVersion().equals(topVer)){ Throwable err = fut.validateCache(cctx); @@ -1373,6 +1415,9 @@ private class LockTimeoutObject extends GridTimeoutObjectAdapter { super(timeout); } + /** Requested keys. */ + private Set requestedKeys; + /** {@inheritDoc} */ @SuppressWarnings({"ThrowableInstanceNeverThrown"}) @Override public void onTimeout() { @@ -1381,7 +1426,42 @@ private class LockTimeoutObject extends GridTimeoutObjectAdapter { timedOut = true; - onComplete(false, true); + if (inTx() && cctx.tm().deadlockDetectionEnabled()) { + synchronized (futs) { + requestedKeys = requestedKeys0(); + + futs.clear(); // Stop response processing. + } + + Set keys = new HashSet<>(); + + for (IgniteTxEntry txEntry : tx.allEntries()) { + if (!txEntry.locked()) + keys.add(txEntry.txKey()); + } + + IgniteInternalFuture fut = cctx.tm().detectDeadlock(tx, keys); + + fut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture fut) { + try { + TxDeadlock deadlock = fut.get(); + + if (deadlock != null) + err = new TransactionDeadlockException(deadlock.toString(cctx.shared())); + } + catch (IgniteCheckedException e) { + err = e; + + U.warn(log, "Failed to detect deadlock.", e); + } + + onComplete(false, true); + } + }); + } + else + onComplete(false, true); } /** {@inheritDoc} */ @@ -1466,7 +1546,7 @@ void onResult(ClusterTopologyCheckedException e) { tx.removeMapping(node.id()); // Primary node left the grid, so fail the future. - GridNearLockFuture.this.onDone(newTopologyException(e, node.id())); + GridNearLockFuture.this.onDone(false, newTopologyException(e, node.id())); onDone(true); } @@ -1483,6 +1563,10 @@ void onResult(GridNearLockResponse res) { } if (res.error() != null) { + if (inTx() && cctx.tm().deadlockDetectionEnabled() && + (res.error() instanceof IgniteTxTimeoutCheckedException || tx.remainingTime() == -1)) + return; + if (log.isDebugEnabled()) log.debug("Finishing mini future with an error due to error in response [miniFut=" + this + ", res=" + res + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java index 6515140a1a958..d251528590278 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java @@ -184,7 +184,7 @@ private void onError(@Nullable GridDistributedTxMapping m, Throwable e) { } } - if (e instanceof IgniteTxOptimisticCheckedException) { + if (e instanceof IgniteTxOptimisticCheckedException || e instanceof IgniteTxTimeoutCheckedException) { if (m != null) tx.removeMapping(m.node().id()); } @@ -424,10 +424,21 @@ private boolean skipFuture(boolean remap, IgniteInternalFuture fut) { final ClusterNode n = m.node(); + long timeout = tx.remainingTime(); + + if (timeout == -1) { + IgniteCheckedException err = tx.timeoutException(); + + fut.onResult(err); + + return err; + } + GridNearTxPrepareRequest req = new GridNearTxPrepareRequest( futId, tx.topologyVersion(), tx, + timeout, m.reads(), m.writes(), m.near(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java index 1ea99c466c85e..5a300ff51ec12 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java @@ -20,9 +20,11 @@ import java.util.ArrayDeque; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Queue; +import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; @@ -41,8 +43,11 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxMapping; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; +import org.apache.ignite.internal.processors.cache.transactions.TxDeadlock; import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException; import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; +import org.apache.ignite.internal.util.future.GridEmbeddedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -53,7 +58,9 @@ import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiClosure; import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.transactions.TransactionDeadlockException; import org.apache.ignite.transactions.TransactionTimeoutException; import org.jetbrains.annotations.Nullable; @@ -73,8 +80,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa * @param cctx Context. * @param tx Transaction. */ - public GridNearOptimisticTxPrepareFuture(GridCacheSharedContext cctx, - GridNearTxLocal tx) { + public GridNearOptimisticTxPrepareFuture(GridCacheSharedContext cctx, GridNearTxLocal tx) { super(cctx, tx); assert tx.optimistic() && !tx.serializable() : tx; @@ -85,7 +91,11 @@ public GridNearOptimisticTxPrepareFuture(GridCacheSharedContext cctx, if (log.isDebugEnabled()) log.debug("Transaction future received owner changed callback: " + entry); - if ((entry.context().isNear() || entry.context().isLocal()) && owner != null && tx.hasWriteKey(entry.txKey())) { + if (tx.remainingTime() == -1) + return false; + + if ((entry.context().isNear() || entry.context().isLocal()) && + owner != null && tx.hasWriteKey(entry.txKey())) { if (keyLockFut != null) keyLockFut.onKeyLocked(entry.txKey()); @@ -124,6 +134,12 @@ public GridNearOptimisticTxPrepareFuture(GridCacheSharedContext cctx, * @param discoThread {@code True} if executed from discovery thread. */ void onError(Throwable e, boolean discoThread) { + if (e instanceof IgniteTxTimeoutCheckedException) { + onTimeout(); + + return; + } + if (X.hasCause(e, ClusterTopologyCheckedException.class) || X.hasCause(e, ClusterTopologyException.class)) { if (tx.onePhaseCommit()) { tx.markForBackupCheck(); @@ -160,7 +176,7 @@ void onError(Throwable e, boolean discoThread) { if (mini != null) { assert mini.node().id().equals(nodeId); - mini.onResult(nodeId, res); + mini.onResult(res); } else { if (msgLog.isDebugEnabled()) { @@ -181,6 +197,33 @@ void onError(Throwable e, boolean discoThread) { } } + /** + * @return Keys for which {@link MiniFuture} isn't completed. + */ + @SuppressWarnings("ForLoopReplaceableByForEach") + public Set requestedKeys() { + synchronized (futs) { + for (int i = 0; i < futs.size(); i++) { + IgniteInternalFuture fut = futs.get(i); + + if (isMini(fut) && !fut.isDone()) { + MiniFuture miniFut = (MiniFuture)fut; + + Collection entries = miniFut.mapping().entries(); + + Set keys = U.newHashSet(entries.size()); + + for (IgniteTxEntry entry : entries) + keys.add(entry.txKey()); + + return keys; + } + } + } + + return null; + } + /** * Finds pending mini future by the given mini ID. * @@ -264,7 +307,7 @@ private boolean onComplete(boolean discoThread) { if (!txStateCheck) { if (tx.setRollbackOnly()) { - if (tx.timedOut()) + if (tx.remainingTime() == -1) onError(new IgniteTxTimeoutCheckedException("Transaction timed out and " + "was rolled back: " + this), false); else @@ -437,89 +480,97 @@ private void proceedPrepare(GridDistributedTxMapping m, @Nullable final Queue prepFut = cctx.tm().txHandler().prepareTx(n.id(), tx, req); + // If this is the primary node for the keys. + if (n.isLocal()) { + // At this point, if any new node joined, then it is + // waiting for this transaction to complete, so + // partition reassignments are not possible here. + IgniteInternalFuture prepFut = + cctx.tm().txHandler().prepareTx(n.id(), tx, req); - prepFut.listen(new CI1>() { - @Override public void apply(IgniteInternalFuture prepFut) { - try { - fut.onResult(n.id(), prepFut.get()); + prepFut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture prepFut) { + try { + fut.onResult(prepFut.get()); + } + catch (IgniteCheckedException e) { + fut.onResult(e); + } } - catch (IgniteCheckedException e) { - fut.onResult(e); + }); + } + else { + try { + cctx.io().send(n, req, tx.ioPolicy()); + + if (msgLog.isDebugEnabled()) { + msgLog.debug("Near optimistic prepare fut, sent request [txId=" + tx.nearXidVersion() + + ", node=" + n.id() + ']'); } } - }); - } - else { - try { - cctx.io().send(n, req, tx.ioPolicy()); + catch (ClusterTopologyCheckedException e) { + e.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion())); - if (msgLog.isDebugEnabled()) { - msgLog.debug("Near optimistic prepare fut, sent request [txId=" + tx.nearXidVersion() + - ", node=" + n.id() + ']'); + fut.onNodeLeft(e, false); } - } - catch (ClusterTopologyCheckedException e) { - e.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion())); + catch (IgniteCheckedException e) { + if (msgLog.isDebugEnabled()) { + msgLog.debug("Near optimistic prepare fut, failed to sent request [txId=" + tx.nearXidVersion() + + ", node=" + n.id() + + ", err=" + e + ']'); + } - fut.onNodeLeft(e, false); - } - catch (IgniteCheckedException e) { - if (msgLog.isDebugEnabled()) { - msgLog.debug("Near optimistic prepare fut, failed to sent request [txId=" + tx.nearXidVersion() + - ", node=" + n.id() + - ", err=" + e + ']'); + fut.onResult(e); } - - fut.onResult(e); } } + else + onTimeout(); } finally { if (set) @@ -623,6 +674,61 @@ else if (!cacheCtx.isLocal()) return cur; } + /** + * + */ + @SuppressWarnings("ForLoopReplaceableByForEach") + private void onTimeout() { + if (cctx.tm().deadlockDetectionEnabled()) { + Set keys = null; + + if (keyLockFut != null) + keys = new HashSet<>(keyLockFut.lockKeys); + else { + if (futs != null && !futs.isEmpty()) { + for (int i = 0; i < futs.size(); i++) { + IgniteInternalFuture fut = futs.get(i); + + if (isMini(fut) && !fut.isDone()) { + MiniFuture miniFut = (MiniFuture)fut; + + Collection entries = miniFut.mapping().entries(); + + keys = U.newHashSet(entries.size()); + + for (IgniteTxEntry entry : entries) + keys.add(entry.txKey()); + + break; + } + } + } + } + + add(new GridEmbeddedFuture<>(new IgniteBiClosure() { + @Override public GridNearTxPrepareResponse apply(TxDeadlock deadlock, Exception e) { + if (e != null) + U.warn(log, "Failed to detect deadlock.", e); + else { + e = new IgniteTxTimeoutCheckedException("Failed to acquire lock within provided timeout for " + + "transaction [timeout=" + tx.timeout() + ", tx=" + tx + ']', + deadlock != null ? new TransactionDeadlockException(deadlock.toString(cctx)) : null); + } + + onDone(null, e); + + return null; + } + }, cctx.tm().detectDeadlock(tx, keys))); + } + else { + ERR_UPD.compareAndSet(this, null, new IgniteTxTimeoutCheckedException("Failed to acquire lock " + + "within provided timeout for transaction [timeout=" + tx.timeout() + ", tx=" + tx + ']')); + + onComplete(false); + } + } + /** {@inheritDoc} */ @Override public String toString() { Collection futs = F.viewReadOnly(futures(), new C1, String>() { @@ -652,7 +758,7 @@ private static class MiniFuture extends GridFutureAdapter RCV_RES_UPD = + private static final AtomicIntegerFieldUpdater RCV_RES_UPD = AtomicIntegerFieldUpdater.newUpdater(MiniFuture.class, "rcvRes"); /** Parent future. */ @@ -745,15 +851,21 @@ void onNodeLeft(ClusterTopologyCheckedException e, boolean discoThread) { } /** - * @param nodeId Failed node ID. * @param res Result callback. */ @SuppressWarnings("ThrowableResultOfMethodCallIgnored") - void onResult(UUID nodeId, final GridNearTxPrepareResponse res) { + void onResult(final GridNearTxPrepareResponse res) { if (isDone()) return; if (RCV_RES_UPD.compareAndSet(this, 0, 1)) { + if (parent.cctx.tm().deadlockDetectionEnabled() && + (parent.tx.remainingTime() == -1 || res.error() instanceof IgniteTxTimeoutCheckedException)) { + parent.onTimeout(); + + return; + } + if (res.error() != null) { // Fail the whole compound future. parent.onError(res.error(), false); @@ -801,8 +913,7 @@ void onResult(UUID nodeId, final GridNearTxPrepareResponse res) { */ private void remap() { parent.prepareOnTopology(true, new Runnable() { - @Override - public void run() { + @Override public void run() { onDone((GridNearTxPrepareResponse) null); } }); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java index 4d77a3ccbe9b2..a00cf3e95fd53 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java @@ -40,8 +40,7 @@ public abstract class GridNearOptimisticTxPrepareFutureAdapter extends GridNearT * @param cctx Context. * @param tx Transaction. */ - public GridNearOptimisticTxPrepareFutureAdapter(GridCacheSharedContext cctx, - GridNearTxLocal tx) { + public GridNearOptimisticTxPrepareFutureAdapter(GridCacheSharedContext cctx, GridNearTxLocal tx) { super(cctx, tx); assert tx.optimistic() : tx; @@ -172,7 +171,7 @@ protected final void prepareOnTopology(final boolean remap, @Nullable final Runn protected static class KeyLockFuture extends GridFutureAdapter { /** */ @GridToStringInclude - private Collection lockKeys = new GridConcurrentHashSet<>(); + protected Collection lockKeys = new GridConcurrentHashSet<>(); /** */ private volatile boolean allKeysAdded; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java index ef2edc992cda6..34b8281a2ee23 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java @@ -153,7 +153,7 @@ private MiniFuture miniFuture(IgniteUuid miniId) { @Override public void prepare() { if (!tx.state(PREPARING)) { if (tx.setRollbackOnly()) { - if (tx.timedOut()) + if (tx.remainingTime() == -1) onDone(new IgniteTxTimeoutCheckedException("Transaction timed out and was rolled back: " + tx)); else onDone(new IgniteCheckedException("Invalid transaction state for prepare " + @@ -222,6 +222,11 @@ private void preparePessimistic() { checkOnePhase(); + long timeout = tx.remainingTime(); + + if (timeout == -1) + onDone(new IgniteTxTimeoutCheckedException("Transaction timed out and was rolled back: " + tx)); + for (final GridDistributedTxMapping m : mappings.values()) { final ClusterNode node = m.node(); @@ -229,6 +234,7 @@ private void preparePessimistic() { futId, tx.topologyVersion(), tx, + timeout, m.reads(), m.writes(), m.near(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java index adde63cc13dc5..bb5d482ab211a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java @@ -633,6 +633,9 @@ private void finish(GridDistributedTxMapping m, boolean commit) { if (m.explicitLock()) syncMode = FULL_SYNC; + // Version to be added in completed versions on primary node. + GridCacheVersion completedVer = !commit && tx.timeout() > 0 ? tx.xidVersion() : null; + GridNearTxFinishRequest req = new GridNearTxFinishRequest( futId, tx.xidVersion(), @@ -645,7 +648,7 @@ private void finish(GridDistributedTxMapping m, boolean commit) { m.explicitLock(), tx.storeEnabled(), tx.topologyVersion(), - null, + completedVer, // Reuse 'baseVersion' to do not add new fields in message. null, null, tx.size(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index 28c60d4ca4fd9..410baf837f34c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -796,6 +796,8 @@ void readyNearLock(IgniteTxEntry txEntry, GridNearTxPrepareFutureAdapter fut = (GridNearTxPrepareFutureAdapter)prepFut; if (fut == null) { + long timeout = remainingTime(); + // Future must be created before any exception can be thrown. if (optimistic()) { fut = serializable() ? @@ -807,6 +809,12 @@ void readyNearLock(IgniteTxEntry txEntry, if (!PREP_FUT_UPD.compareAndSet(this, null, fut)) return prepFut; + + if (timeout == -1) { + fut.onDone(this, timeoutException()); + + return fut; + } } else // Prepare was called explicitly. @@ -964,8 +972,10 @@ public IgniteInternalFuture prepareAsyncLocal( Map> txNodes, boolean last ) { + long timeout = remainingTime(); + if (state() != PREPARING) { - if (timedOut()) + if (timeout == -1) return new GridFinishedFuture<>( new IgniteTxTimeoutCheckedException("Transaction timed out: " + this)); @@ -975,11 +985,15 @@ public IgniteInternalFuture prepareAsyncLocal( new IgniteCheckedException("Invalid transaction state for prepare [state=" + state() + ", tx=" + this + ']')); } + if (timeout == -1) + return new GridFinishedFuture<>(timeoutException()); + init(); GridDhtTxPrepareFuture fut = new GridDhtTxPrepareFuture( cctx, this, + timeout, IgniteUuid.randomUuid(), Collections.emptyMap(), last, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java index 9dfdb43e3c0f5..e55566b980e49 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java @@ -94,6 +94,7 @@ public GridNearTxPrepareRequest() { * @param futId Future ID. * @param topVer Topology version. * @param tx Transaction. + * @param timeout Transaction timeout. * @param reads Read entries. * @param writes Write entries. * @param near {@code True} if mapping is for near caches. @@ -112,6 +113,7 @@ public GridNearTxPrepareRequest( IgniteUuid futId, AffinityTopologyVersion topVer, IgniteInternalTx tx, + long timeout, Collection reads, Collection writes, boolean near, @@ -126,7 +128,7 @@ public GridNearTxPrepareRequest( boolean firstClientReq, boolean addDepInfo ) { - super(tx, reads, writes, txNodes, onePhaseCommit, addDepInfo); + super(tx, timeout, reads, writes, txNodes, onePhaseCommit, addDepInfo); assert futId != null; assert !firstClientReq || tx.optimistic() : tx; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java index 8c0425da31374..dd900fe37518b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java @@ -33,7 +33,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; -import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; import org.apache.ignite.internal.util.lang.GridTuple; import org.apache.ignite.lang.IgniteAsyncSupported; @@ -46,7 +45,7 @@ /** * Transaction managed by cache ({@code 'Ex'} stands for external). */ -public interface IgniteInternalTx extends AutoCloseable, GridTimeoutObject { +public interface IgniteInternalTx extends AutoCloseable { /** * */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java index f76f4bf898c44..eb2989e55a400 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java @@ -712,7 +712,7 @@ public void syncMode(CacheWriteSynchronizationMode syncMode) { /** * @return Transaction timeout exception. */ - protected final IgniteCheckedException timeoutException() { + public final IgniteCheckedException timeoutException() { return new IgniteTxTimeoutCheckedException("Failed to acquire lock within provided timeout " + "for transaction [timeout=" + timeout() + ", tx=" + this + ']'); } @@ -1032,7 +1032,7 @@ public void onePhaseCommit(boolean onePhaseCommit) { * @return {@code True} if state changed. */ @SuppressWarnings({"TooBroadScope"}) - private boolean state(TransactionState state, boolean timedOut) { + protected boolean state(TransactionState state, boolean timedOut) { boolean valid = false; TransactionState prev; @@ -1153,24 +1153,6 @@ private boolean state(TransactionState state, boolean timedOut) { this.writeVer = writeVer; } - /** {@inheritDoc} */ - @Override public IgniteUuid timeoutId() { - return xidVer.asGridUuid(); - } - - /** {@inheritDoc} */ - @Override public long endTime() { - long endTime = timeout == 0 ? Long.MAX_VALUE : startTime + timeout; - - return endTime > 0 ? endTime : endTime < 0 ? Long.MAX_VALUE : endTime; - } - - /** {@inheritDoc} */ - @Override public void onTimeout() { - if (local() && !dht()) - state(MARKED_ROLLBACK, true); - } - /** {@inheritDoc} */ @Override public boolean timedOut() { return timedOut; @@ -2386,21 +2368,6 @@ private static class TxShadow implements IgniteInternalTx { return null; } - /** {@inheritDoc} */ - @Override public IgniteUuid timeoutId() { - return null; - } - - /** {@inheritDoc} */ - @Override public long endTime() { - return 0; - } - - /** {@inheritDoc} */ - @Override public void onTimeout() { - // No-op. - } - /** {@inheritDoc} */ @Override public boolean equals(Object o) { return this == o || o instanceof IgniteInternalTx && xid.equals(((IgniteInternalTx)o).xid()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index 7c3c206c1c137..e67e60ff0543c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -112,8 +112,7 @@ public class IgniteTxHandler { * @param req Request. * @return Prepare future. */ - public IgniteInternalFuture processNearTxPrepareRequest(final UUID nearNodeId, - final GridNearTxPrepareRequest req) { + public IgniteInternalFuture processNearTxPrepareRequest(final UUID nearNodeId, GridNearTxPrepareRequest req) { if (txPrepareMsgLog.isDebugEnabled()) { txPrepareMsgLog.debug("Received near prepare request [txId=" + req.version() + ", node=" + nearNodeId + ']'); @@ -361,7 +360,7 @@ private IgniteInternalFuture prepareNearTx( req.deployInfo() != null); try { - ctx.io().send(nearNode, res, req.policy()); + ctx.io().send(nearNodeId, res, req.policy()); if (txPrepareMsgLog.isDebugEnabled()) { txPrepareMsgLog.debug("Sent remap response for near prepare [txId=" + req.version() + @@ -667,6 +666,10 @@ else if (txFinishMsgLog.isDebugEnabled()) { assert nodeId != null; assert req != null; + // 'baseVersion' message field is re-used for version to be added in completed versions. + if (!req.commit() && req.baseVersion() != null) + ctx.tm().addRolledbackTx(null, req.baseVersion()); + // Transaction on local cache only. if (locTx != null && !locTx.nearLocallyMapped() && !locTx.colocatedLocallyMapped()) return new GridFinishedFuture(locTx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index d9aca4a139a66..9ad7fb0a4c7f8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -107,6 +107,7 @@ import static org.apache.ignite.internal.processors.dr.GridDrType.DR_PRIMARY; import static org.apache.ignite.transactions.TransactionState.COMMITTED; import static org.apache.ignite.transactions.TransactionState.COMMITTING; +import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK; import static org.apache.ignite.transactions.TransactionState.PREPARING; import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK; import static org.apache.ignite.transactions.TransactionState.ROLLING_BACK; @@ -547,14 +548,15 @@ protected boolean commitAfterLock() { @SuppressWarnings({"CatchGenericClass"}) public void userPrepare() throws IgniteCheckedException { if (state() != PREPARING) { - if (timedOut()) + if (remainingTime() == -1) throw new IgniteTxTimeoutCheckedException("Transaction timed out: " + this); TransactionState state = state(); setRollbackOnly(); - throw new IgniteCheckedException("Invalid transaction state for prepare [state=" + state + ", tx=" + this + ']'); + throw new IgniteCheckedException("Invalid transaction state for prepare [state=" + + state + ", tx=" + this + ']'); } checkValid(); @@ -629,7 +631,7 @@ protected GridCacheEntryEx entryEx(GridCacheContext cacheCtx, IgniteTxKey key, A TransactionState state = state(); if (state != COMMITTING) { - if (timedOut()) + if (remainingTime() == -1) throw new IgniteTxTimeoutCheckedException("Transaction timed out: " + this); setRollbackOnly(); @@ -3540,8 +3542,11 @@ protected final void addActiveCache(GridCacheContext cacheCtx) throws IgniteChec * @throws IgniteCheckedException If transaction check failed. */ protected void checkValid() throws IgniteCheckedException { + if (local() && !dht() && remainingTime() == -1) + state(MARKED_ROLLBACK, true); + if (isRollbackOnly()) { - if (timedOut()) + if (remainingTime() == -1) throw new IgniteTxTimeoutCheckedException("Cache transaction timed out: " + this); TransactionState state = state(); @@ -3556,10 +3561,6 @@ protected void checkValid() throws IgniteCheckedException { throw new IgniteCheckedException("Cache transaction marked as rollback-only: " + this); } - - if (remainingTime() == -1 && setRollbackOnly()) - throw new IgniteTxTimeoutCheckedException("Cache transaction timed out " + - "(was rolled back automatically): " + this); } /** {@inheritDoc} */ @@ -3604,7 +3605,7 @@ protected final IgniteTxEntry addEntry(GridCacheOperation op, TransactionState state = state(); - assert state == TransactionState.ACTIVE || timedOut() : + assert state == TransactionState.ACTIVE || remainingTime() == -1 : "Invalid tx state for adding entry [op=" + op + ", val=" + val + ", entry=" + entry + ", filter=" + Arrays.toString(filter) + ", txCtx=" + cctx.tm().txContextVersion() + ", tx=" + this + ']'; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java index 63c991922e9ad..f9357f9aa755e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java @@ -50,7 +50,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.GridCacheMvccFuture; import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; -import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.GridCacheMappedVersion; import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxFinishSync; import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryFuture; @@ -62,6 +61,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedLockFuture; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetection.TxDeadlockFuture; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -126,7 +127,7 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { public static final IgniteProductVersion TX_DEADLOCK_DETECTION_SINCE = IgniteProductVersion.fromString("1.5.19"); /** Deadlock detection maximum iterations. */ - static final int DEADLOCK_MAX_ITERS = + static int DEADLOCK_MAX_ITERS = IgniteSystemProperties.getInteger(IGNITE_TX_DEADLOCK_DETECTION_MAX_ITERS, 1000); /** Committing transactions. */ @@ -389,7 +390,13 @@ public int completedVersionsSize() { * {@code false} otherwise. */ public boolean isCompleted(IgniteInternalTx tx) { - return completedVersHashMap.containsKey(tx.xidVersion()); + boolean completed = completedVersHashMap.containsKey(tx.xidVersion()); + + // Need check that for tx with timeout rollback message was not received before lock. + if (!completed && tx.local() && tx.dht() && tx.timeout() > 0) + return completedVersHashMap.containsKey(tx.nearXidVersion()); + + return completed; } /** @@ -495,13 +502,6 @@ public IgniteTxLocalAdapter newTx( return null; } - if (tx.timeout() > 0) { - cctx.time().addTimeoutObject(tx); - - if (log.isDebugEnabled()) - log.debug("Registered transaction with timeout processor: " + tx); - } - if (log.isDebugEnabled()) log.debug("Transaction created: " + tx); @@ -786,7 +786,7 @@ public boolean inUserTx() { */ public void prepareTx(IgniteInternalTx tx) throws IgniteCheckedException { if (tx.state() == MARKED_ROLLBACK) { - if (tx.timedOut()) + if (tx.remainingTime() == -1) throw new IgniteTxTimeoutCheckedException("Transaction timed out: " + this); throw new IgniteCheckedException("Transaction is marked for rollback: " + tx); @@ -1081,13 +1081,6 @@ public void commitTx(IgniteInternalTx tx) { if (log.isDebugEnabled()) log.debug("Committing from TM [locNodeId=" + cctx.localNodeId() + ", tx=" + tx + ']'); - if (tx.timeout() > 0) { - cctx.time().removeTimeoutObject(tx); - - if (log.isDebugEnabled()) - log.debug("Unregistered transaction with timeout processor: " + tx); - } - /* * Note that write phase is handled by transaction adapter itself, * so we don't do it here. @@ -2006,17 +1999,45 @@ private TxLocksResponse txLocksInfo(Collection txKeys) { if (!(nearTxLoc || tx instanceof GridDhtTxLocal) || !hasKeys(tx, txKeys)) continue; - Collection txEntries = tx.allEntries(); + IgniteTxState state = tx.txState(); + + assert state instanceof IgniteTxStateImpl || state instanceof IgniteTxImplicitSingleStateImpl; + + Collection txEntries = + state instanceof IgniteTxStateImpl ? ((IgniteTxStateImpl)state).allEntriesCopy() : state.allEntries(); - Set requestedKeys = null; + Set requestedKeys = null; // Try to get info about requested keys for detached entries in case of GridNearTxLocal transaction // in order to reduce amount of requests to remote nodes. if (nearTxLoc) { - GridDhtColocatedLockFuture fut = colocatedLockFuture(tx); + if (tx.pessimistic()) { + GridDhtColocatedLockFuture fut = + (GridDhtColocatedLockFuture)mvccFuture(tx, GridDhtColocatedLockFuture.class); + + if (fut != null) + requestedKeys = fut.requestedKeys(); + + GridNearLockFuture nearFut = (GridNearLockFuture)mvccFuture(tx, GridNearLockFuture.class); - if (fut != null) - requestedKeys = fut.requestedKeys(); + if (nearFut != null) { + Set nearRequestedKeys = nearFut.requestedKeys(); + + if (nearRequestedKeys != null) { + if (requestedKeys == null) + requestedKeys = nearRequestedKeys; + else + requestedKeys = nearRequestedKeys; + } + } + } + else { + GridNearOptimisticTxPrepareFuture fut = + (GridNearOptimisticTxPrepareFuture)mvccFuture(tx, GridNearOptimisticTxPrepareFuture.class); + + if (fut != null) + requestedKeys = fut.requestedKeys(); + } } for (IgniteTxEntry txEntry : txEntries) { @@ -2073,17 +2094,18 @@ else if (nearTxLoc && requestedKeys != null && requestedKeys.contains(txKey.key( /** * @param tx Tx. Must be instance of {@link GridNearTxLocal}. - * @return Colocated future. + * @param cls Future class. + * @return Cache future. */ - private GridDhtColocatedLockFuture colocatedLockFuture(IgniteInternalTx tx) { + private IgniteInternalFuture mvccFuture(IgniteInternalTx tx, Class cls) { assert tx instanceof GridNearTxLocal : tx; Collection> futs = cctx.mvcc().mvccFutures(tx.nearXidVersion()); if (futs != null) { for (GridCacheMvccFuture fut : futs) { - if (fut instanceof GridDhtColocatedLockFuture) - return (GridDhtColocatedLockFuture)fut; + if (fut.getClass().equals(cls)) + return fut; } } @@ -2114,6 +2136,16 @@ public void removeFuture(long futId) { deadlockDetectFuts.remove(futId); } + /** + * @return Collection of active transaction deadlock detection futures. + */ + @SuppressWarnings("unchecked") + public Collection> deadlockDetectionFutures() { + Collection> values = deadlockDetectFuts.values(); + + return (Collection>)values; + } + /** * Timeout object for node failure handler. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java index c116d0dd08992..f23cca980e064 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java @@ -20,6 +20,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.Map; import java.util.Set; import org.apache.ignite.IgniteCheckedException; @@ -39,6 +40,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; +import org.jsr166.ConcurrentLinkedHashMap; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; @@ -362,6 +364,13 @@ public class IgniteTxStateImpl extends IgniteTxLocalStateAdapter { return txMap == null ? Collections.emptySet() : txMap.values(); } + /** + * @return All entries. Returned collection is copy of internal collection. + */ + public synchronized Collection allEntriesCopy() { + return txMap == null ? Collections.emptySet() : new HashSet<>(txMap.values()); + } + /** {@inheritDoc} */ @Override public IgniteTxEntry entry(IgniteTxKey key) { return txMap == null ? null : txMap.get(key); @@ -408,7 +417,7 @@ public class IgniteTxStateImpl extends IgniteTxLocalStateAdapter { } /** {@inheritDoc} */ - @Override public void addEntry(IgniteTxEntry entry) { + @Override public synchronized void addEntry(IgniteTxEntry entry) { txMap.put(entry.txKey(), entry); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetection.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetection.java index 36843dd85a5a9..70d938e144ead 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetection.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetection.java @@ -36,8 +36,11 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_DEADLOCK_DETECTION_TIMEOUT; @@ -106,22 +109,30 @@ static List findCycle(Map children = wfg.get(v); - if (children == null || children.isEmpty()) + if (children == null || children.isEmpty()) { + stack.pop(); + inPath.remove(v); + continue; + } inPath.add(v); for (GridCacheVersion w : children) { - if (inPath.contains(w)) { + if (inPath.contains(w) && visited.contains(w)) { List cycle = new ArrayList<>(); for (GridCacheVersion x = v; !x.equals(w); x = edgeTo.get(x)) @@ -158,15 +169,18 @@ static class TxDeadlockFuture extends GridFutureAdapter { private final Set keys; /** Processed keys. */ + @GridToStringInclude private final Set processedKeys = new HashSet<>(); /** Processed nodes. */ private final Set processedNodes = new HashSet<>(); /** Pending keys. */ + @GridToStringInclude private Map> pendingKeys = new HashMap<>(); /** Nodes queue. */ + @GridToStringInclude private final UniqueDeque nodesQueue = new UniqueDeque<>(); /** Preferred nodes. */ @@ -194,6 +208,7 @@ static class TxDeadlockFuture extends GridFutureAdapter { private int itersCnt; /** Timeout object. */ + @GridToStringExclude private DeadlockTimeoutObject timeoutObj; /** Timed out flag. */ @@ -252,8 +267,8 @@ private void init() { if (topVer == null) // Tx manager already stopped onDone(); - - map(keys, Collections.emptyMap()); + else + map(keys, Collections.emptyMap()); } /** @@ -441,14 +456,17 @@ else if (lock.candiate()) { * @param txLocks Tx locks. */ private void updateWaitForGraph(Map txLocks) { + if (txLocks == null || txLocks.isEmpty()) + return; + for (Map.Entry e : txLocks.entrySet()) { GridCacheVersion txOwner = null; for (TxLock lock : e.getValue().txLocks()) { - if (lock.owner()) { - assert txOwner == null; - + if (lock.owner() && txOwner == null) { + // Actually we can get lock list with more than one owner. In this case ignore all owners + // except first because likely the first owner was cause of deadlock. txOwner = lock.txId(); if (keys.contains(e.getKey()) && !txId.equals(lock.txId())) { @@ -463,7 +481,7 @@ private void updateWaitForGraph(Map txLocks) { continue; } - if (lock.candiate()) { + if (lock.candiate() || lock.owner()) { GridCacheVersion txId0 = lock.txId(); Set waitForTxs = wfg.get(txId0); @@ -485,9 +503,9 @@ public void onResult(UUID nodeId, TxLocksResponse res) { if (res != null && set) { if (res.classError() != null) { - IgniteLogger log = cctx.logger(TxDeadlockDetection.class); + IgniteLogger log = cctx.kernalContext().log(this.getClass()); - log.warning("Failed to finish deadlock detection due to an error: " + nodeId); + U.warn(log, "Failed to finish deadlock detection due to an error: " + nodeId); onDone(); } @@ -528,6 +546,11 @@ private boolean compareAndSet(UUID exp, UUID val) { return false; } + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(TxDeadlockFuture.class, this); + } + /** * Lock request timeout object. */ @@ -543,6 +566,10 @@ private class DeadlockTimeoutObject extends GridTimeoutObjectAdapter { @Override public void onTimeout() { timedOut = true; + IgniteLogger log = cctx.kernalContext().log(this.getClass()); + + U.warn(log, "Deadlock detection was timed out [timeout=" + DEADLOCK_TIMEOUT + ", fut=" + this + ']'); + onDone(); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxConfigCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxConfigCacheSelfTest.java index 94b5620b1e64c..2b524e890bc57 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxConfigCacheSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxConfigCacheSelfTest.java @@ -18,15 +18,29 @@ package org.apache.ignite.internal.processors.cache; +import javax.cache.CacheException; +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.TransactionConfiguration; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; @@ -34,11 +48,6 @@ import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionTimeoutException; -import javax.cache.CacheException; -import javax.cache.processor.EntryProcessor; -import javax.cache.processor.EntryProcessorException; -import javax.cache.processor.MutableEntry; - import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; @@ -62,6 +71,10 @@ public class IgniteTxConfigCacheSelfTest extends GridCommonAbstractTest { ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + TcpCommunicationSpi commSpi = new TestCommunicationSpi(); + + cfg.setCommunicationSpi(commSpi); + CacheConfiguration ccfg = new CacheConfiguration(); ccfg.setAtomicityMode(atomicityMode()); @@ -87,7 +100,7 @@ public CacheAtomicityMode atomicityMode() { /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - startGrids(1); + startGrids(2); } /** {@inheritDoc} */ @@ -98,7 +111,7 @@ public CacheAtomicityMode atomicityMode() { /** * Success if user tx was timed out. * - * @throws Exception + * @throws Exception If failed. */ public void testUserTxTimeout() throws Exception { final Ignite ignite = grid(0); @@ -112,7 +125,7 @@ public void testUserTxTimeout() throws Exception { /** * Success if system caches weren't timed out. * - * @throws Exception + * @throws Exception If failed. */ public void testSystemCacheTx() throws Exception { final Ignite ignite = grid(0); @@ -143,27 +156,23 @@ protected IgniteInternalCache getSystemCache(final Ignite ignite * Success if implicit tx fails. * * @param cache Cache name. - * @throws Exception + * @throws Exception If failed. */ protected void checkImplicitTxTimeout(final IgniteCache cache) throws Exception { - try { - cache.invoke("key", new EntryProcessor() { - @Override public Object process(final MutableEntry entry, final Object... args) - throws EntryProcessorException { - try { - sleepForTxFailure(); - } catch (InterruptedException e) { - throw new EntryProcessorException(e); - } + TestCommunicationSpi.delay = true; - return null; - } - }); + Integer key = primaryKey(ignite(1).cache(CACHE_NAME)); + + try { + cache.put(key, 0); fail("Timeout exception must be thrown"); } catch (CacheException e) { - // OK + // No-op. + } + finally { + TestCommunicationSpi.delay = false; } cache.clear(); @@ -174,7 +183,7 @@ protected void checkImplicitTxTimeout(final IgniteCache cache) t * * @param cache Cache name. * @param ignite Ignite instance. - * @throws Exception + * @throws Exception If failed. */ protected void checkExplicitTxTimeout(final IgniteCache cache, final Ignite ignite) throws Exception { @@ -198,7 +207,7 @@ protected void checkExplicitTxTimeout(final IgniteCache cache, f * Success if explicit tx doesn't fail. * * @param cache Cache instance. - * @throws Exception + * @throws Exception If failed. */ protected void checkStartTxSuccess(final IgniteInternalCache cache) throws Exception { try (final IgniteInternalTx tx = CU.txStartInternal(cache.context(), cache, PESSIMISTIC, READ_COMMITTED)) { @@ -220,7 +229,7 @@ protected void checkStartTxSuccess(final IgniteInternalCache cac * Success if implicit tx fails. * * @param cache Cache instance. - * @throws Exception + * @throws Exception If failed. */ protected void checkImplicitTxSuccess(final IgniteInternalCache cache) throws Exception { cache.invoke("key", new EntryProcessor() { @@ -241,9 +250,39 @@ protected void checkImplicitTxSuccess(final IgniteInternalCache /** * Sleep multiple {@link #TX_TIMEOUT} times. * - * @throws InterruptedException + * @throws InterruptedException If interrupted. */ private void sleepForTxFailure() throws InterruptedException { Thread.sleep(TX_TIMEOUT * 3); } + + /** + * + */ + private static class TestCommunicationSpi extends TcpCommunicationSpi { + /** Delay. */ + private static volatile boolean delay; + + /** {@inheritDoc} */ + @Override public void sendMessage( + final ClusterNode node, + final Message msg, + final IgniteInClosure ackC + ) throws IgniteSpiException { + if (msg instanceof GridIoMessage) { + Message msg0 = ((GridIoMessage)msg).message(); + + if (msg0 instanceof GridNearTxPrepareRequest && delay) { + try { + U.sleep(TX_TIMEOUT * 2); + } + catch (IgniteInterruptedCheckedException e) { + e.printStackTrace(); + } + } + } + + super.sendMessage(node, msg, ackC); + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java index c4178219c8924..8475175573f76 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java @@ -20,10 +20,10 @@ import java.util.ArrayList; import java.util.List; import java.util.Random; -import javax.cache.CacheException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; @@ -154,8 +154,8 @@ private void checkTransactionTimeout(TransactionConcurrency concurrency, assert false : "Timeout never happened for transaction: " + tx; } - catch (CacheException e) { - if (!(e.getCause() instanceof TransactionTimeoutException)) + catch (Exception e) { + if (!(X.hasCause(e, TransactionTimeoutException.class))) throw e; info("Received expected timeout exception [msg=" + e.getMessage() + ", tx=" + tx + ']'); @@ -164,4 +164,4 @@ private void checkTransactionTimeout(TransactionConcurrency concurrency, tx.close(); } } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedMultiNodeLongTxTimeoutFullApiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedMultiNodeLongTxTimeoutFullApiTest.java new file mode 100644 index 0000000000000..89fe8e0dfb339 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedMultiNodeLongTxTimeoutFullApiTest.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.near; + +import org.apache.ignite.configuration.IgniteConfiguration; + +/** + * + */ +public class CachePartitionedMultiNodeLongTxTimeoutFullApiTest extends GridCachePartitionedMultiNodeFullApiSelfTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.getTransactionConfiguration().setDefaultTxTimeout(Long.MAX_VALUE); + + return cfg; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java new file mode 100644 index 0000000000000..3e3b84e4403bf --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.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.internal.processors.cache.distributed.near; + +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; + +/** + * + */ +public class CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest extends + GridCachePartitionedMultiNodeFullApiSelfTest { + /** {@inheritDoc} */ + @Override protected NearCacheConfiguration nearConfiguration() { + return new NearCacheConfiguration(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.getTransactionConfiguration().setDefaultTxTimeout(Long.MAX_VALUE); + + return cfg; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxTimeoutSelfTest.java index cfa93ac457178..e27207df0bd2f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxTimeoutSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxTimeoutSelfTest.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal.processors.cache.local; -import javax.cache.CacheException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; @@ -154,7 +153,7 @@ private void checkTransactionTimeout(TransactionConcurrency concurrency, tx.commit(); } - catch (CacheException e) { + catch (Exception e) { assertTrue(X.hasCause(e, TransactionTimeoutException.class)); info("Received expected optimistic exception: " + e.getMessage()); @@ -166,4 +165,4 @@ private void checkTransactionTimeout(TransactionConcurrency concurrency, assert wasEx; } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/DepthFirstSearchTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/DepthFirstSearchTest.java index 20467c211ba1c..b0a407c7f4115 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/DepthFirstSearchTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/DepthFirstSearchTest.java @@ -24,10 +24,12 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.Set; import junit.framework.TestCase; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; import static org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetection.findCycle; @@ -93,6 +95,14 @@ public void testNoCycle() throws Exception { assertAllNull(wfg); + wfg = new HashMap>() {{ + put(T1, new HashSet(){{add(T2);}}); + put(T2, new HashSet(){{add(T3);}}); + put(T4, new HashSet(){{add(T1); add(T2); add(T3);}}); + }}; + + assertAllNull(wfg); + wfg = new HashMap>() {{ put(T1, new HashSet(){{add(T2);}}); put(T3, new HashSet(){{add(T4);}}); @@ -227,6 +237,94 @@ public void testFindCycle3() throws Exception { } + /** + * @throws Exception If failed. + */ + public void testFindCycle4() throws Exception { + Map> wfg = new HashMap>() {{ + put(T1, Collections.singleton(T2)); + put(T2, asLinkedHashSet(T3, T4)); + put(T3, Collections.singleton(T4)); + put(T4, Collections.singleton(T5)); + put(T6, Collections.singleton(T3)); + }}; + + assertNull(findCycle(wfg, T1)); + } + + /** + * @throws Exception If failed. + */ + public void testRandomNoExceptions() throws Exception { + int maxNodesCnt = 100; + int minNodesCnt = 10; + int maxWaitForNodesCnt = 20; + + int cyclesFound = 0; + int cyclesNotFound = 0; + + Random seedRnd = new Random(); + + Random rnd = new Random(); + + for (int i = 0; i < 50000; i++) { + long seed = seedRnd.nextLong(); + + rnd.setSeed(seed); + + System.out.println(">>> Iteration " + i + " with seed " + seed); + + int nodesCnt = rnd.nextInt(maxNodesCnt - minNodesCnt) + minNodesCnt; + + Map> wfg = new HashMap<>(); + + for (int j = 0; j < nodesCnt; j++) { + if (rnd.nextInt(100) > 30) { + int waitForNodesCnt = rnd.nextInt(maxWaitForNodesCnt); + + Set waitForNodes = null; + + if (waitForNodesCnt > 0) { + waitForNodes = new LinkedHashSet<>(); + + for (int k = 0; k < waitForNodesCnt;) { + int n = rnd.nextInt(nodesCnt); + + if (n != j) { + waitForNodes.add(new GridCacheVersion(n, 0, 0, 0)); + k++; + } + } + } + + wfg.put(new GridCacheVersion(j, 0, 0, 0), waitForNodes); + } + } + + for (int j = 0; j < nodesCnt; j++) { + try { + List cycle = findCycle(wfg, new GridCacheVersion(j, 0, 0, 0)); + + if (cycle == null) + cyclesNotFound++; + else + cyclesFound++; + } + catch (Throwable e) { + U.error(null, "Error during finding cycle in graph: ", e); + + U.warn(null, "Seed: " + seed); + + U.warn(null, "Wait-for-graph: " + wfg); + + fail(); + } + } + } + + System.out.println(">>> Test finished. Cycles found: " + cyclesFound + ", cycles not found: " + cyclesNotFound); + } + /** * @param wfg Wait-for-graph. */ @@ -249,4 +347,4 @@ private static Set asLinkedHashSet(GridCacheVersion... txs) { return set; } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java new file mode 100644 index 0000000000000..c9d18ebdf95df --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java @@ -0,0 +1,246 @@ +/* + * 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.transactions; + +import java.util.Collection; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_DEADLOCK_DETECTION_MAX_ITERS; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_DEADLOCK_DETECTION_TIMEOUT; +import static org.apache.ignite.IgniteSystemProperties.getInteger; +import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * + */ +public class TxDeadlockDetectionNoHangsTest extends GridCommonAbstractTest { + /** Nodes count. */ + private static final int NODES_CNT = 3; + + /** Cache. */ + private static final String CACHE = "cache"; + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + CacheConfiguration ccfg = defaultCacheConfiguration(); + + ccfg.setName(CACHE); + ccfg.setCacheMode(CacheMode.PARTITIONED); + ccfg.setBackups(1); + ccfg.setNearConfiguration(null); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + startGridsMultiThreaded(NODES_CNT); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + GridTestUtils.setFieldValue(null, TxDeadlockDetection.class, "DEADLOCK_TIMEOUT", (int)(getTestTimeout() * 2)); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + GridTestUtils.setFieldValue(null, TxDeadlockDetection.class, "DEADLOCK_TIMEOUT", + getInteger(IGNITE_TX_DEADLOCK_DETECTION_TIMEOUT, 60000)); + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 10 * 60 * 1000; + } + + /** + * @throws Exception If failed. + */ + public void testNoHangsPessimistic() throws Exception { + assertTrue(grid(0).context().cache().context().tm().deadlockDetectionEnabled()); + + doTest(PESSIMISTIC); + + try { + GridTestUtils.setFieldValue(null, IgniteTxManager.class, "DEADLOCK_MAX_ITERS", 0); + + assertFalse(grid(0).context().cache().context().tm().deadlockDetectionEnabled()); + + doTest(PESSIMISTIC); + } + finally { + GridTestUtils.setFieldValue(null, IgniteTxManager.class, "DEADLOCK_MAX_ITERS", + IgniteSystemProperties.getInteger(IGNITE_TX_DEADLOCK_DETECTION_MAX_ITERS, 1000)); + } + } + + /** + * @throws Exception If failed. + */ + public void testNoHangsOptimistic() throws Exception { + assertTrue(grid(0).context().cache().context().tm().deadlockDetectionEnabled()); + + doTest(OPTIMISTIC); + + try { + GridTestUtils.setFieldValue(null, IgniteTxManager.class, "DEADLOCK_MAX_ITERS", 0); + + assertFalse(grid(0).context().cache().context().tm().deadlockDetectionEnabled()); + + doTest(OPTIMISTIC); + } + finally { + GridTestUtils.setFieldValue(null, IgniteTxManager.class, "DEADLOCK_MAX_ITERS", + IgniteSystemProperties.getInteger(IGNITE_TX_DEADLOCK_DETECTION_MAX_ITERS, 1000)); + } + } + + /** + * @param concurrency Concurrency. + * @throws IgniteCheckedException If failed. + */ + private void doTest(final TransactionConcurrency concurrency) throws IgniteCheckedException { + final AtomicBoolean stop = new AtomicBoolean(); + + IgniteInternalFuture restartFut = null; + + try { + restartFut = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + while (!stop.get()) { + try { + U.sleep(500); + + startGrid(NODES_CNT); + + awaitPartitionMapExchange(); + + U.sleep(500); + + stopGrid(NODES_CNT); + } + catch (Exception e) { + // No-op. + } + } + } + }, 1, "restart-thread"); + + long stopTime = System.currentTimeMillis() + 2 * 60_000L; + + for (int i = 0; System.currentTimeMillis() < stopTime; i++) { + boolean detectionEnabled = grid(0).context().cache().context().tm().deadlockDetectionEnabled(); + + log.info(">>> Iteration " + i + " (detection is " + (detectionEnabled ? "enabled" : "disabled") + ')'); + + final AtomicInteger threadCnt = new AtomicInteger(); + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + int threadNum = threadCnt.getAndIncrement(); + + Ignite ignite = ignite(threadNum % NODES_CNT); + + IgniteCache cache = ignite.cache(CACHE); + + try (Transaction tx = ignite.transactions().txStart(concurrency, REPEATABLE_READ, 500, 0)) { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + for (int i = 0; i < 50; i++) { + int key = rnd.nextInt(50); + + if (log.isDebugEnabled()) { + log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + + ", tx=" + tx + ", key=" + key + ']'); + } + + cache.put(key, 0); + } + + tx.commit(); + } + catch (Exception e) { + e.printStackTrace(); + } + } + }, NODES_CNT * 3, "tx-thread"); + + fut.get(); + } + } + finally { + stop.set(true); + + if (restartFut != null) + restartFut.get(); + + checkDetectionFutures(); + } + } + + /** + * + */ + private void checkDetectionFutures() { + for (int i = 0; i < NODES_CNT ; i++) { + Ignite ignite = ignite(i); + + IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm(); + + Collection> futs = txMgr.deadlockDetectionFutures(); + + assertTrue(futs.isEmpty()); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionTest.java index 3d0beac9ddf58..87bc70fdb04da 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionTest.java @@ -21,8 +21,9 @@ import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; -import java.util.concurrent.ConcurrentMap; +import java.util.Collection; import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -47,7 +48,6 @@ import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionDeadlockException; import org.apache.ignite.transactions.TransactionTimeoutException; -import org.jsr166.ThreadLocalRandom8; import static org.apache.ignite.internal.util.typedef.X.hasCause; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; @@ -152,7 +152,7 @@ public void testNoHangs() throws Exception { IgniteCache cache = ignite.cache(CACHE); try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ, 700, 0)) { - ThreadLocalRandom8 rnd = ThreadLocalRandom8.current(); + ThreadLocalRandom rnd = ThreadLocalRandom.current(); for (int i = 0; i < 50; i++) { int key = rnd.nextInt(50); @@ -217,7 +217,7 @@ public void testNoDeadlockSimple() throws Exception { cache.put(key, 0); - barrier.await(timeout + 100, TimeUnit.MILLISECONDS); + barrier.await(timeout + 1000, TimeUnit.MILLISECONDS); tx.commit(); } @@ -281,7 +281,7 @@ public void testNoDeadlock() throws Exception { log.info(">>> Performs sleep. [node=" + ((IgniteKernal)ignite).localNode() + ", tx=" + tx + ']'); - U.sleep(timeout * 2); + U.sleep(timeout * 3); } else { int key2 = threadNum + 1; @@ -406,8 +406,7 @@ private void checkDetectionFuts() { IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm(); - ConcurrentMap futs = - GridTestUtils.getFieldValue(txMgr, IgniteTxManager.class, "deadlockDetectFuts"); + Collection> futs = txMgr.deadlockDetectionFutures(); assertTrue(futs.isEmpty()); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionCrossCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionCrossCacheTest.java new file mode 100644 index 0000000000000..7b40da2a034e0 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionCrossCacheTest.java @@ -0,0 +1,257 @@ +/* + * 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.transactions; + +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.GridConcurrentHashSet; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionDeadlockException; +import org.apache.ignite.transactions.TransactionTimeoutException; + +import static org.apache.ignite.internal.util.typedef.X.hasCause; +import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * + */ +public class TxOptimisticDeadlockDetectionCrossCacheTest extends GridCommonAbstractTest { + /** Nodes count. */ + private static final int NODES_CNT = 2; + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + if (isDebug()) { + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.failureDetectionTimeoutEnabled(false); + + cfg.setDiscoverySpi(discoSpi); + } + + TcpCommunicationSpi commSpi = new TestCommunicationSpi(); + + cfg.setCommunicationSpi(commSpi); + + CacheConfiguration ccfg0 = defaultCacheConfiguration(); + + ccfg0.setName("cache0"); + ccfg0.setCacheMode(CacheMode.PARTITIONED); + ccfg0.setBackups(1); + ccfg0.setNearConfiguration(null); + + CacheConfiguration ccfg1 = defaultCacheConfiguration(); + + ccfg1.setName("cache1"); + ccfg1.setCacheMode(CacheMode.PARTITIONED); + ccfg1.setBackups(1); + ccfg1.setNearConfiguration(null); + + cfg.setCacheConfiguration(ccfg0, ccfg1); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrids(NODES_CNT); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testDeadlock() throws Exception { + // Sometimes boh transactions perform commit, so we repeat attempt. + while (!doTestDeadlock()) {} + } + + /** + * @throws Exception If failed. + */ + private boolean doTestDeadlock() throws Exception { + TestCommunicationSpi.init(2); + + final CyclicBarrier barrier = new CyclicBarrier(2); + + final AtomicInteger threadCnt = new AtomicInteger(); + + final AtomicBoolean deadlock = new AtomicBoolean(); + + final AtomicInteger commitCnt = new AtomicInteger(); + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + int threadNum = threadCnt.getAndIncrement(); + + Ignite ignite = ignite(0); + + IgniteCache cache1 = ignite.cache("cache" + (threadNum == 0 ? 0 : 1)); + + IgniteCache cache2 = ignite.cache("cache" + (threadNum == 0 ? 1 : 0)); + + try (Transaction tx = + ignite.transactions().txStart(OPTIMISTIC, REPEATABLE_READ, 500, 0) + ) { + int key1 = primaryKey(cache1); + + log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + + ", tx=" + tx + ", key=" + key1 + ", cache=" + cache1.getName() + ']'); + + cache1.put(key1, 0); + + barrier.await(); + + int key2 = primaryKey(cache2); + + log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + + ", tx=" + tx + ", key=" + key2 + ", cache=" + cache2.getName() + ']'); + + cache2.put(key2, 1); + + tx.commit(); + + commitCnt.incrementAndGet(); + } + catch (Throwable e) { + // At least one stack trace should contain TransactionDeadlockException. + if (hasCause(e, TransactionTimeoutException.class) && + hasCause(e, TransactionDeadlockException.class) + ) { + if (deadlock.compareAndSet(false, true)) + U.error(log, "At least one stack trace should contain " + + TransactionDeadlockException.class.getSimpleName(), e); + } + } + } + }, 2, "tx-thread"); + + fut.get(); + + if (commitCnt.get() == 2) + return false; + + assertTrue(deadlock.get()); + + for (int i = 0; i < NODES_CNT ; i++) { + Ignite ignite = ignite(i); + + IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm(); + + Collection> futs = txMgr.deadlockDetectionFutures(); + + assertTrue(futs.isEmpty()); + } + + return true; + } + + /** + * + */ + private static class TestCommunicationSpi extends TcpCommunicationSpi { + /** Tx count. */ + private static volatile int TX_CNT; + + /** Tx ids. */ + private static final Set TX_IDS = new GridConcurrentHashSet<>(); + + /** + * @param txCnt Tx count. + */ + private static void init(int txCnt) { + TX_CNT = txCnt; + TX_IDS.clear(); + } + + /** {@inheritDoc} */ + @Override public void sendMessage( + final ClusterNode node, + final Message msg, + final IgniteInClosure ackC + ) throws IgniteSpiException { + if (msg instanceof GridIoMessage) { + Message msg0 = ((GridIoMessage)msg).message(); + + if (msg0 instanceof GridNearTxPrepareRequest) { + final GridNearTxPrepareRequest req = (GridNearTxPrepareRequest)msg0; + + GridCacheVersion txId = req.version(); + + if (TX_IDS.contains(txId)) { + while (TX_IDS.size() < TX_CNT) { + try { + U.sleep(50); + } + catch (IgniteInterruptedCheckedException e) { + e.printStackTrace(); + } + } + } + } + else if (msg0 instanceof GridNearTxPrepareResponse) { + GridNearTxPrepareResponse res = (GridNearTxPrepareResponse)msg0; + + GridCacheVersion txId = res.version(); + + TX_IDS.add(txId); + } + } + + super.sendMessage(node, msg, ackC); + } + } + +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java new file mode 100644 index 0000000000000..aa240aae439a9 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java @@ -0,0 +1,574 @@ +/* + * 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.transactions; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.GridCacheAdapter; +import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap; +import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.GridConcurrentHashSet; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionDeadlockException; +import org.apache.ignite.transactions.TransactionTimeoutException; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheMode.REPLICATED; +import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE; +import static org.apache.ignite.internal.util.typedef.X.cause; +import static org.apache.ignite.internal.util.typedef.X.hasCause; +import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * + */ +public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest { + /** Cache name. */ + private static final String CACHE_NAME = "cache"; + + /** Nodes count (actually two times more nodes will started: server + client). */ + private static final int NODES_CNT = 4; + + /** No op transformer. */ + private static final NoOpTransformer NO_OP_TRANSFORMER = new NoOpTransformer(); + + /** Wrapping transformer. */ + private static final WrappingTransformer WRAPPING_TRANSFORMER = new WrappingTransformer(); + + /** Client mode flag. */ + private static boolean client; + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + if (isDebug()) { + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.failureDetectionTimeoutEnabled(false); + + cfg.setDiscoverySpi(discoSpi); + } + + TcpCommunicationSpi commSpi = new TestCommunicationSpi(); + + cfg.setCommunicationSpi(commSpi); + + cfg.setClientMode(client); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + client = false; + + startGrids(NODES_CNT); + + client = true; + + for (int i = 0; i < NODES_CNT; i++) + startGrid(i + NODES_CNT); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testDeadlocksPartitioned() throws Exception { + for (CacheWriteSynchronizationMode syncMode : CacheWriteSynchronizationMode.values()) { + doTestDeadlocks(createCache(PARTITIONED, syncMode, false), NO_OP_TRANSFORMER); + doTestDeadlocks(createCache(PARTITIONED, syncMode, false), WRAPPING_TRANSFORMER); + } + } + + /** + * @throws Exception If failed. + */ + public void testDeadlocksPartitionedNear() throws Exception { + for (CacheWriteSynchronizationMode syncMode : CacheWriteSynchronizationMode.values()) { + doTestDeadlocks(createCache(PARTITIONED, syncMode, true), NO_OP_TRANSFORMER); + doTestDeadlocks(createCache(PARTITIONED, syncMode, true), WRAPPING_TRANSFORMER); + } + } + + /** + * @throws Exception If failed. + */ + public void testDeadlocksReplicated() throws Exception { + for (CacheWriteSynchronizationMode syncMode : CacheWriteSynchronizationMode.values()) { + doTestDeadlocks(createCache(REPLICATED, syncMode, false), NO_OP_TRANSFORMER); + doTestDeadlocks(createCache(REPLICATED, syncMode, false), WRAPPING_TRANSFORMER); + } + } + + /** + * @param cacheMode Cache mode. + * @param syncMode Write sync mode. + * @param near Near. + * @return Created cache. + */ + @SuppressWarnings("unchecked") + private IgniteCache createCache(CacheMode cacheMode, CacheWriteSynchronizationMode syncMode, boolean near) { + CacheConfiguration ccfg = defaultCacheConfiguration(); + + ccfg.setName(CACHE_NAME); + ccfg.setCacheMode(cacheMode); + ccfg.setBackups(1); + ccfg.setNearConfiguration(near ? new NearCacheConfiguration() : null); + ccfg.setWriteSynchronizationMode(syncMode); + + IgniteCache cache = ignite(0).createCache(ccfg); + + if (near) { + for (int i = 0; i < NODES_CNT; i++) { + Ignite client = ignite(i + NODES_CNT); + + assertTrue(client.configuration().isClientMode()); + + client.createNearCache(ccfg.getName(), new NearCacheConfiguration<>()); + } + } + + return cache; + } + + /** + * @param cache Cache. + * @param transformer Transformer closure. + * @throws Exception If failed. + */ + private void doTestDeadlocks(IgniteCache cache, IgniteClosure transformer) throws Exception { + try { + awaitPartitionMapExchange(); + + doTestDeadlock(3, false, true, true, transformer); + doTestDeadlock(3, false, false, false, transformer); + doTestDeadlock(3, false, false, true, transformer); + + doTestDeadlock(4, false, true, true, transformer); + doTestDeadlock(4, false, false, false, transformer); + doTestDeadlock(4, false, false, true, transformer); + } + catch (Throwable e) { + U.error(log, "Unexpected exception: ", e); + + fail(); + } + finally { + if (cache != null) + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + private void doTestDeadlock( + final int txCnt, + final boolean loc, + boolean lockPrimaryFirst, + final boolean clientTx, + final IgniteClosure transformer + ) throws Exception { + log.info(">>> Test deadlock [txCnt=" + txCnt + ", loc=" + loc + ", lockPrimaryFirst=" + lockPrimaryFirst + + ", clientTx=" + clientTx + ", transformer=" + transformer.getClass().getName() + ']'); + + TestCommunicationSpi.init(txCnt); + + final AtomicInteger threadCnt = new AtomicInteger(); + + final CyclicBarrier barrier = new CyclicBarrier(txCnt); + + final AtomicReference deadlockErr = new AtomicReference<>(); + + final List> keySets = generateKeys(txCnt, loc, !lockPrimaryFirst); + + final Set involvedKeys = new GridConcurrentHashSet<>(); + final Set involvedLockedKeys = new GridConcurrentHashSet<>(); + final Set involvedTxs = new GridConcurrentHashSet<>(); + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + int threadNum = threadCnt.incrementAndGet(); + + Ignite ignite = loc ? ignite(0) : ignite(clientTx ? threadNum - 1 + txCnt : threadNum - 1); + + IgniteCache cache = ignite.cache(CACHE_NAME); + + List keys = keySets.get(threadNum - 1); + + int txTimeout = 500 + txCnt * 100; + + try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, REPEATABLE_READ, txTimeout, 0)) { + IgniteInternalTx tx0 = ((TransactionProxyImpl)tx).tx(); + + involvedTxs.add(tx0); + + Integer key = keys.get(0); + + involvedKeys.add(key); + + Object k; + + log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + + ", tx=" + tx + ", key=" + transformer.apply(key) + ']'); + + cache.put(transformer.apply(key), 0); + + involvedLockedKeys.add(key); + + barrier.await(); + + key = keys.get(1); + + ClusterNode primaryNode = + ((IgniteCacheProxy)cache).context().affinity().primary(key, NONE); + + List primaryKeys = + primaryKeys(grid(primaryNode).cache(CACHE_NAME), 5, key + (100 * threadNum)); + + Map entries = new HashMap<>(); + + involvedKeys.add(key); + + entries.put(transformer.apply(key), 0); + + for (Integer i : primaryKeys) { + involvedKeys.add(i); + + entries.put(transformer.apply(i), 1); + + k = transformer.apply(i + 13); + + involvedKeys.add(i + 13); + + entries.put(k, 2); + } + + log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + + ", tx=" + tx + ", entries=" + entries + ']'); + + cache.putAll(entries); + + tx.commit(); + } + catch (Throwable e) { + U.error(log, "Expected exception: ", e); + + // At least one stack trace should contain TransactionDeadlockException. + if (hasCause(e, TransactionTimeoutException.class) && + hasCause(e, TransactionDeadlockException.class) + ) { + if (deadlockErr.compareAndSet(null, cause(e, TransactionDeadlockException.class))) + U.error(log, "At least one stack trace should contain " + + TransactionDeadlockException.class.getSimpleName(), e); + } + } + } + }, loc ? 2 : txCnt, "tx-thread"); + + try { + fut.get(); + } + catch (IgniteCheckedException e) { + U.error(null, "Unexpected exception", e); + + fail(); + } + + U.sleep(1000); + + TransactionDeadlockException deadlockE = deadlockErr.get(); + + assertNotNull(deadlockE); + + boolean fail = false; + + // Check transactions, futures and entry locks state. + for (int i = 0; i < NODES_CNT * 2; i++) { + Ignite ignite = ignite(i); + + int cacheId = ((IgniteCacheProxy)ignite.cache(CACHE_NAME)).context().cacheId(); + + GridCacheSharedContext cctx = ((IgniteKernal)ignite).context().cache().context(); + + IgniteTxManager txMgr = cctx.tm(); + + Collection activeTxs = txMgr.activeTransactions(); + + for (IgniteInternalTx tx : activeTxs) { + Collection entries = tx.allEntries(); + + for (IgniteTxEntry entry : entries) { + if (entry.cacheId() == cacheId) { + fail = true; + + U.error(log, "Transaction still exists: " + "\n" + tx.xidVersion() + + "\n" + tx.nearXidVersion() + "\n nodeId=" + cctx.localNodeId() + "\n tx=" + tx); + } + } + } + + Collection> futs = txMgr.deadlockDetectionFutures(); + + assertTrue(futs.isEmpty()); + + GridCacheAdapter intCache = internalCache(i, CACHE_NAME); + + GridCacheConcurrentMap map = intCache.map(); + + for (Integer key : involvedKeys) { + Object key0 = transformer.apply(key); + + KeyCacheObject keyCacheObj = intCache.context().toCacheKeyObject(key0); + + GridCacheMapEntry entry = map.getEntry(keyCacheObj); + + if (entry != null) + assertNull("Entry still has locks " + entry, entry.mvccAllLocal()); + } + } + + if (fail) + fail("Some transactions still exist"); + + // Check deadlock report + String msg = deadlockE.getMessage(); + + for (IgniteInternalTx tx : involvedTxs) + assertTrue(msg.contains( + "[txId=" + tx.xidVersion() + ", nodeId=" + tx.nodeId() + ", threadId=" + tx.threadId() + ']')); + + for (Integer key : involvedKeys) { + if (involvedLockedKeys.contains(key)) + assertTrue(msg.contains("[key=" + transformer.apply(key) + ", cache=" + CACHE_NAME + ']')); + else + assertFalse(msg.contains("[key=" + transformer.apply(key))); + } + } + + /** + * @param nodesCnt Nodes count. + * @param loc Local cache. + */ + private List> generateKeys(int nodesCnt, boolean loc, boolean reverse) throws IgniteCheckedException { + List> keySets = new ArrayList<>(); + + if (loc) { + List keys = primaryKeys(ignite(0).cache(CACHE_NAME), 2); + + keySets.add(new ArrayList<>(keys)); + + Collections.reverse(keys); + + keySets.add(keys); + } + else { + for (int i = 0; i < nodesCnt; i++) { + List keys = new ArrayList<>(2); + + int n1 = i + 1; + int n2 = n1 + 1; + + int i1 = n1 < nodesCnt ? n1 : n1 - nodesCnt; + int i2 = n2 < nodesCnt ? n2 : n2 - nodesCnt; + + keys.add(primaryKey(ignite(i1).cache(CACHE_NAME))); + keys.add(primaryKey(ignite(i2).cache(CACHE_NAME))); + + if (reverse) + Collections.reverse(keys); + + keySets.add(keys); + } + } + + return keySets; + } + + /** + * + */ + private static class NoOpTransformer implements IgniteClosure { + /** {@inheritDoc} */ + @Override public Object apply(Integer val) { + return val; + } + } + + /** + * + */ + private static class WrappingTransformer implements IgniteClosure { + /** {@inheritDoc} */ + @Override public Object apply(Integer val) { + return new KeyObject(val); + } + } + + /** + * + */ + private static class KeyObject implements Serializable { + /** Id. */ + private int id; + + /** Name. */ + private String name; + + /** + * @param id Id. + */ + public KeyObject(int id) { + this.id = id; + this.name = "KeyObject" + id; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "KeyObject{" + + "id=" + id + + ", name='" + name + '\'' + + '}'; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + KeyObject obj = (KeyObject)o; + + return id == obj.id && name.equals(obj.name); + + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return id; + } + } + + /** + * + */ + private static class TestCommunicationSpi extends TcpCommunicationSpi { + /** Tx count. */ + private static volatile int TX_CNT; + + /** Tx ids. */ + private static final Set TX_IDS = new GridConcurrentHashSet<>(); + + /** + * @param txCnt Tx count. + */ + private static void init(int txCnt) { + TX_CNT = txCnt; + TX_IDS.clear(); + } + + /** {@inheritDoc} */ + @Override public void sendMessage( + final ClusterNode node, + final Message msg, + final IgniteInClosure ackC + ) throws IgniteSpiException { + if (msg instanceof GridIoMessage) { + Message msg0 = ((GridIoMessage)msg).message(); + + if (msg0 instanceof GridNearTxPrepareRequest) { + final GridNearTxPrepareRequest req = (GridNearTxPrepareRequest)msg0; + + GridCacheVersion txId = req.version(); + + if (TX_IDS.contains(txId)) { + while (TX_IDS.size() < TX_CNT) { + try { + U.sleep(50); + } + catch (IgniteInterruptedCheckedException e) { + e.printStackTrace(); + } + } + } + } + else if (msg0 instanceof GridNearTxPrepareResponse) { + GridNearTxPrepareResponse res = (GridNearTxPrepareResponse)msg0; + + GridCacheVersion txId = res.version(); + + TX_IDS.add(txId); + } + } + + super.sendMessage(node, msg, ackC); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionCrossCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionCrossCacheTest.java index abbefd0740a8d..4bf5a4166f89c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionCrossCacheTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionCrossCacheTest.java @@ -17,7 +17,7 @@ package org.apache.ignite.internal.processors.cache.transactions; -import java.util.concurrent.ConcurrentMap; +import java.util.Collection; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -26,6 +26,7 @@ import org.apache.ignite.cache.CacheMode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.util.typedef.internal.U; @@ -60,22 +61,6 @@ public class TxPessimisticDeadlockDetectionCrossCacheTest extends GridCommonAbst cfg.setDiscoverySpi(discoSpi); } - CacheConfiguration ccfg0 = defaultCacheConfiguration(); - - ccfg0.setName("cache0"); - ccfg0.setCacheMode(CacheMode.PARTITIONED); - ccfg0.setBackups(1); - ccfg0.setNearConfiguration(null); - - CacheConfiguration ccfg1 = defaultCacheConfiguration(); - - ccfg1.setName("cache1"); - ccfg1.setCacheMode(CacheMode.PARTITIONED); - ccfg1.setBackups(1); - ccfg1.setNearConfiguration(null); - - cfg.setCacheConfiguration(ccfg0, ccfg1); - return cfg; } @@ -96,70 +81,132 @@ public class TxPessimisticDeadlockDetectionCrossCacheTest extends GridCommonAbst /** * @throws Exception If failed. */ - public void testDeadlock() throws Exception { - final CyclicBarrier barrier = new CyclicBarrier(2); + public void testDeadlockNoNear() throws Exception { + doTestDeadlock(false, false); + } - final AtomicInteger threadCnt = new AtomicInteger(); + /** + * @throws Exception If failed. + */ + public void testDeadlockOneNear() throws Exception { + doTestDeadlock(false, true); + } - final AtomicBoolean deadlock = new AtomicBoolean(); + /** + * @throws Exception If failed. + */ + public void testDeadlockAnotherNear() throws Exception { + doTestDeadlock(true, false); + doTestDeadlock(false, true); + } - IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { - @Override public void run() { - int threadNum = threadCnt.getAndIncrement(); + /** + * @throws Exception If failed. + */ + public void testDeadlockBothNear() throws Exception { + doTestDeadlock(true, true); + } - Ignite ignite = ignite(0); + /** + * @param near0 Near flag for cache0. + * @param near1 Near flag for cache1. + */ + private void doTestDeadlock(boolean near0, boolean near1) throws Exception { + IgniteCache cache0 = null; + IgniteCache cache1 = null; - IgniteCache cache1 = ignite.cache("cache" + (threadNum == 0 ? 0 : 1)); + try { + cache0 = getCache(ignite(0), "cache0", near0); + cache1 = getCache(ignite(0), "cache1", near1); - IgniteCache cache2 = ignite.cache("cache" + (threadNum == 0 ? 1 : 0)); + awaitPartitionMapExchange(); - try (Transaction tx = - ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ, 500, 0) - ) { - int key1 = primaryKey(cache1); + final CyclicBarrier barrier = new CyclicBarrier(2); - log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + - ", tx=" + tx + ", key=" + key1 + ", cache=" + cache1.getName() + ']'); + final AtomicInteger threadCnt = new AtomicInteger(); - cache1.put(key1, 0); + final AtomicBoolean deadlock = new AtomicBoolean(); - barrier.await(); + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + int threadNum = threadCnt.getAndIncrement(); - int key2 = primaryKey(cache2); + Ignite ignite = ignite(0); - log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + - ", tx=" + tx + ", key=" + key2 + ", cache=" + cache2.getName() + ']'); + IgniteCache cache1 = ignite.cache("cache" + (threadNum == 0 ? 0 : 1)); - cache2.put(key2, 1); + IgniteCache cache2 = ignite.cache("cache" + (threadNum == 0 ? 1 : 0)); - tx.commit(); - } - catch (Throwable e) { - // At least one stack trace should contain TransactionDeadlockException. - if (hasCause(e, TransactionTimeoutException.class) && - hasCause(e, TransactionDeadlockException.class) - ) { - if (deadlock.compareAndSet(false, true)) - U.error(log, "At least one stack trace should contain " + - TransactionDeadlockException.class.getSimpleName(), e); + try (Transaction tx = + ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ, 500, 0) + ) { + int key1 = primaryKey(cache1); + + log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + + ", tx=" + tx + ", key=" + key1 + ", cache=" + cache1.getName() + ']'); + + cache1.put(key1, 0); + + barrier.await(); + + int key2 = primaryKey(cache2); + + log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + + ", tx=" + tx + ", key=" + key2 + ", cache=" + cache2.getName() + ']'); + + cache2.put(key2, 1); + + tx.commit(); + } + catch (Throwable e) { + // At least one stack trace should contain TransactionDeadlockException. + if (hasCause(e, TransactionTimeoutException.class) && + hasCause(e, TransactionDeadlockException.class) + ) { + if (deadlock.compareAndSet(false, true)) + U.error(log, "At least one stack trace should contain " + + TransactionDeadlockException.class.getSimpleName(), e); + } } } - } - }, 2, "tx-thread"); + }, 2, "tx-thread"); - fut.get(); + fut.get(); - assertTrue(deadlock.get()); + assertTrue(deadlock.get()); - for (int i = 0; i < NODES_CNT ; i++) { - Ignite ignite = ignite(i); + for (int i = 0; i < NODES_CNT ; i++) { + Ignite ignite = ignite(i); - IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm(); + IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm(); - ConcurrentMap futs = - GridTestUtils.getFieldValue(txMgr, IgniteTxManager.class, "deadlockDetectFuts"); + Collection> futs = txMgr.deadlockDetectionFutures(); - assertTrue(futs.isEmpty()); + assertTrue(futs.isEmpty()); + } + } + finally { + if (cache0 != null) + cache0.destroy(); + + if (cache1 != null) + cache1.destroy(); } } + + /** + * @param ignite Ignite. + * @param name Name. + * @param near Near. + */ + private IgniteCache getCache(Ignite ignite, String name, boolean near) { + CacheConfiguration ccfg = defaultCacheConfiguration(); + + ccfg.setName(name); + ccfg.setCacheMode(CacheMode.PARTITIONED); + ccfg.setBackups(1); + ccfg.setNearConfiguration(near ? new NearCacheConfiguration() : null); + + return ignite.getOrCreateCache(ccfg); + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java index ee1a98987b2e8..83eb908d21978 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -43,6 +42,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap; import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.util.GridConcurrentHashSet; @@ -135,7 +135,7 @@ public void testDeadlocksPartitioned() throws Exception { /** * @throws Exception If failed. */ - public void _testDeadlocksPartitionedNear() throws Exception { + public void testDeadlocksPartitionedNear() throws Exception { for (CacheWriteSynchronizationMode syncMode : CacheWriteSynchronizationMode.values()) { doTestDeadlocks(createCache(PARTITIONED, syncMode, true), NO_OP_TRANSFORMER); doTestDeadlocks(createCache(PARTITIONED, syncMode, true), WRAPPING_TRANSFORMER); @@ -178,6 +178,7 @@ public void testDeadlocksLocal() throws Exception { * @param cacheMode Cache mode. * @param syncMode Write sync mode. * @param near Near. + * @return Created cache. */ @SuppressWarnings("unchecked") private IgniteCache createCache(CacheMode cacheMode, CacheWriteSynchronizationMode syncMode, boolean near) { @@ -189,7 +190,19 @@ private IgniteCache createCache(CacheMode cacheMode, CacheWriteSynchronizationMo ccfg.setNearConfiguration(near ? new NearCacheConfiguration() : null); ccfg.setWriteSynchronizationMode(syncMode); - return ignite(0).getOrCreateCache(ccfg); + IgniteCache cache = ignite(0).createCache(ccfg); + + if (near) { + for (int i = 0; i < NODES_CNT; i++) { + Ignite client = ignite(i + NODES_CNT); + + assertTrue(client.configuration().isClientMode()); + + client.createNearCache(ccfg.getName(), new NearCacheConfiguration<>()); + } + } + + return cache; } /** @@ -323,7 +336,14 @@ private void doTestDeadlock( } }, loc ? 2 : txCnt, "tx-thread"); - fut.get(); + try { + fut.get(); + } + catch (IgniteCheckedException e) { + U.error(null, "Unexpected exception", e); + + fail(); + } U.sleep(1000); @@ -331,13 +351,17 @@ private void doTestDeadlock( assertNotNull(deadlockE); + boolean fail = false; + // Check transactions, futures and entry locks state. for (int i = 0; i < NODES_CNT * 2; i++) { Ignite ignite = ignite(i); int cacheId = ((IgniteCacheProxy)ignite.cache(CACHE_NAME)).context().cacheId(); - IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm(); + GridCacheSharedContext cctx = ((IgniteKernal)ignite).context().cache().context(); + + IgniteTxManager txMgr = cctx.tm(); Collection activeTxs = txMgr.activeTransactions(); @@ -345,13 +369,16 @@ private void doTestDeadlock( Collection entries = tx.allEntries(); for (IgniteTxEntry entry : entries) { - if (entry.cacheId() == cacheId) - fail("Transaction still exists: " + tx); + if (entry.cacheId() == cacheId) { + fail = true; + + U.error(log, "Transaction still exists: " + "\n" + tx.xidVersion() + + "\n" + tx.nearXidVersion() + "\n nodeId=" + cctx.localNodeId() + "\n tx=" + tx); + } } } - ConcurrentMap futs = - GridTestUtils.getFieldValue(txMgr, IgniteTxManager.class, "deadlockDetectFuts"); + Collection> futs = txMgr.deadlockDetectionFutures(); assertTrue(futs.isEmpty()); @@ -371,6 +398,9 @@ private void doTestDeadlock( } } + if (fail) + fail("Some transactions still exist"); + // Check deadlock report String msg = deadlockE.getMessage(); @@ -484,4 +514,4 @@ public KeyObject(int id) { return id; } } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java index 8414461bba663..14e58337b7c07 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java @@ -36,6 +36,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOffHeapTieredAtomicFullApiSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOffHeapTieredFullApiSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOffHeapTieredMultiNodeFullApiSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.near.CachePartitionedMultiNodeLongTxTimeoutFullApiTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyFairAffinityMultiNodeFullApiSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyMultiNodeP2PDisabledFullApiSelfTest; @@ -74,6 +75,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeCounterSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeFullApiSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeP2PDisabledFullApiSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.near.CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNearOnlyNoPrimaryFullApiSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapFullApiSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapMultiNodeFullApiSelfTest; @@ -167,6 +169,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheAtomicPrimaryWriteOrderMultiNodeP2PDisabledFullApiSelfTest.class); suite.addTestSuite(GridCacheAtomicNearEnabledMultiNodeFullApiSelfTest.class); suite.addTestSuite(GridCacheAtomicNearEnabledPrimaryWriteOrderMultiNodeFullApiSelfTest.class); + suite.addTestSuite(CachePartitionedMultiNodeLongTxTimeoutFullApiTest.class); + suite.addTestSuite(CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.class); suite.addTestSuite(GridCachePartitionedNearDisabledMultiNodeFullApiSelfTest.class); suite.addTestSuite(GridCachePartitionedNearDisabledMultiNodeP2PDisabledFullApiSelfTest.class); diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java index c057e55888cf9..5a1b1ad37168a 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java @@ -19,7 +19,10 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.processors.cache.transactions.DepthFirstSearchTest; +import org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetectionNoHangsTest; import org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetectionTest; +import org.apache.ignite.internal.processors.cache.transactions.TxOptimisticDeadlockDetectionCrossCacheTest; +import org.apache.ignite.internal.processors.cache.transactions.TxOptimisticDeadlockDetectionTest; import org.apache.ignite.internal.processors.cache.transactions.TxPessimisticDeadlockDetectionCrossCacheTest; import org.apache.ignite.internal.processors.cache.transactions.TxPessimisticDeadlockDetectionTest; @@ -35,9 +38,12 @@ public static TestSuite suite() throws Exception { TestSuite suite = new TestSuite("Ignite Deadlock Detection Test Suite"); suite.addTestSuite(DepthFirstSearchTest.class); + suite.addTestSuite(TxOptimisticDeadlockDetectionTest.class); + suite.addTestSuite(TxOptimisticDeadlockDetectionCrossCacheTest.class); suite.addTestSuite(TxPessimisticDeadlockDetectionTest.class); suite.addTestSuite(TxPessimisticDeadlockDetectionCrossCacheTest.class); suite.addTestSuite(TxDeadlockDetectionTest.class); + suite.addTestSuite(TxDeadlockDetectionNoHangsTest.class); return suite; } diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala index 22fb89d2047f3..1b555059c874b 100644 --- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala +++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala @@ -19,6 +19,7 @@ package org.apache.ignite.visor.commands.cache import org.apache.ignite.cluster.{ClusterGroupEmptyException, ClusterNode} import org.apache.ignite.visor.visor._ + import org.apache.ignite.internal.visor.cache.VisorCacheStopTask import org.apache.ignite.internal.visor.util.VisorTaskUtils._ @@ -101,9 +102,7 @@ class VisorCacheStopCommand { return } - val dflt = if (batchMode) "y" else "n" - - ask(s"Are you sure you want to stop cache: ${escapeName(cacheName)}? (y/n) [$dflt]: ", dflt) match { + ask(s"Are you sure you want to stop cache: ${escapeName(cacheName)}? (y/n) [n]: ", "n") match { case "y" | "Y" => try { executeRandom(grp, classOf[VisorCacheStopTask], cacheName) From 407071e466d1a4fcec86571d4791ace8bb206f53 Mon Sep 17 00:00:00 2001 From: Eduard Shangareev Date: Mon, 29 Aug 2016 17:32:31 -0700 Subject: [PATCH 071/487] IGNITE-2546 - Transformers for SCAN queries. Fixes #949. --- .../java/org/apache/ignite/IgniteCache.java | 15 + .../processors/cache/IgniteCacheProxy.java | 112 +++- .../processors/cache/query/CacheQuery.java | 11 +- .../GridCacheDistributedQueryManager.java | 22 +- .../query/GridCacheLocalQueryManager.java | 3 +- .../cache/query/GridCacheQueryAdapter.java | 69 ++- .../cache/query/GridCacheQueryBean.java | 8 +- .../cache/query/GridCacheQueryInfo.java | 8 +- .../cache/query/GridCacheQueryManager.java | 125 ++-- .../cache/query/GridCacheQueryRequest.java | 6 +- .../GridCacheQueryTransformerSelfTest.java | 570 ++++++++++++++++++ .../multijvm/IgniteCacheProcessProxy.java | 6 + .../IgniteCacheQuerySelfTestSuite.java | 2 + 13 files changed, 832 insertions(+), 125 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java index 40eedafcb28e7..2290fc561ed90 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java @@ -56,6 +56,7 @@ import org.apache.ignite.lang.IgniteAsyncSupported; import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.mxbean.CacheMetricsMXBean; import org.apache.ignite.transactions.TransactionHeuristicException; @@ -294,6 +295,20 @@ public interface IgniteCache extends javax.cache.Cache, IgniteAsyncS */ public QueryCursor query(Query qry); + /** + * Queries the cache transforming the entries on the server nodes. Can be used, for example, + * to avoid network overhead in case only one field out of the large is required by client. + *

          + * Currently transformers are supported ONLY for {@link ScanQuery}. Passing any other + * subclass of {@link Query} interface to this method will end up with + * {@link UnsupportedOperationException}. + * + * @param qry Query. + * @param transformer Transformer. + * @return Cursor. + */ + public QueryCursor query(Query qry, IgniteClosure transformer); + /** * Allows for iteration over local cache entries. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java index 0d7bc6a89c00e..9b26c1dec94c3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java @@ -82,6 +82,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.mxbean.CacheMetricsMXBean; import org.apache.ignite.plugin.security.SecurityPermission; @@ -466,50 +467,74 @@ public GridCacheGateway gate() { } /** - * @param filter Filter. + * @param scanQry ScanQry. + * @param transformer Transformer * @param grp Optional cluster group. * @return Cursor. */ @SuppressWarnings("unchecked") - private QueryCursor> query(final Query filter, @Nullable ClusterGroup grp) + private QueryCursor query( + final ScanQuery scanQry, + @Nullable final IgniteClosure transformer, + @Nullable ClusterGroup grp) throws IgniteCheckedException { - final CacheQuery> qry; + + final CacheQuery qry; boolean isKeepBinary = opCtx != null && opCtx.isKeepBinary(); - if (filter instanceof ScanQuery) { - IgniteBiPredicate p = ((ScanQuery)filter).getFilter(); + IgniteBiPredicate p = scanQry.getFilter(); - qry = ctx.queries().createScanQuery(p, ((ScanQuery)filter).getPartition(), isKeepBinary); + qry = ctx.queries().createScanQuery(p, transformer, scanQry.getPartition(), isKeepBinary); - if (grp != null) - qry.projection(grp); + if (grp != null) + qry.projection(grp); - final GridCloseableIterator> iter = ctx.kernalContext().query().executeQuery(ctx, - new IgniteOutClosureX>>() { - @Override public GridCloseableIterator> applyx() throws IgniteCheckedException { - final GridCloseableIterator iter0 = qry.executeScanQuery(); + final GridCloseableIterator iter = ctx.kernalContext().query().executeQuery(ctx, + new IgniteOutClosureX>() { + @Override public GridCloseableIterator applyx() throws IgniteCheckedException { + final GridCloseableIterator iter0 = qry.executeScanQuery(); - return new GridCloseableIteratorAdapter>() { - @Override protected Cache.Entry onNext() throws IgniteCheckedException { - Map.Entry next = iter0.nextX(); + final boolean needToConvert = transformer == null; - return new CacheEntryImpl<>(next.getKey(), next.getValue()); - } + return new GridCloseableIteratorAdapter() { + @Override protected R onNext() throws IgniteCheckedException { + Object next = iter0.nextX(); - @Override protected boolean onHasNext() throws IgniteCheckedException { - return iter0.hasNextX(); - } + if (needToConvert) { + Map.Entry entry = (Map.Entry)next; - @Override protected void onClose() throws IgniteCheckedException { - iter0.close(); + return (R) new CacheEntryImpl<>(entry.getKey(), entry.getValue()); } - }; - } - }, false); - return new QueryCursorImpl<>(iter); - } + return (R)next; + } + + @Override protected boolean onHasNext() throws IgniteCheckedException { + return iter0.hasNextX(); + } + + @Override protected void onClose() throws IgniteCheckedException { + iter0.close(); + } + }; + } + }, false); + + return new QueryCursorImpl<>(iter); + } + + /** + * @param filter Filter. + * @param grp Optional cluster group. + * @return Cursor. + */ + @SuppressWarnings("unchecked") + private QueryCursor> query(final Query filter, @Nullable ClusterGroup grp) + throws IgniteCheckedException { + final CacheQuery> qry; + + boolean isKeepBinary = opCtx != null && opCtx.isKeepBinary(); final CacheQueryFuture> fut; @@ -692,6 +717,9 @@ private QueryCursor> queryContinuous(ContinuousQuery qry, bool return (QueryCursor)ctx.kernalContext().query().queryTwoStep(ctx, p); } + if (qry instanceof ScanQuery) + return query((ScanQuery)qry, null, projection(qry.isLocal())); + return (QueryCursor)query(qry, projection(qry.isLocal())); } catch (Exception e) { @@ -705,6 +733,36 @@ private QueryCursor> queryContinuous(ContinuousQuery qry, bool } } + /** {@inheritDoc} */ + @Override public QueryCursor query(Query qry, IgniteClosure transformer) { + A.notNull(qry, "qry"); + A.notNull(transformer, "transformer"); + + if (!(qry instanceof ScanQuery)) + throw new UnsupportedOperationException("Transformers are supported only for SCAN queries."); + + GridCacheGateway gate = this.gate; + + CacheOperationContext prev = onEnter(gate, opCtx); + + try { + ctx.checkSecurity(SecurityPermission.CACHE_READ); + + validate(qry); + + return query((ScanQuery)qry, transformer, projection(qry.isLocal())); + } + catch (Exception e) { + if (e instanceof CacheException) + throw (CacheException)e; + + throw new CacheException(e); + } + finally { + onLeave(gate, prev); + } + } + /** * @return {@code true} If this is a replicated cache and we are on a data node. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQuery.java index 47c6e89ebb36d..3fa041b205e26 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQuery.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQuery.java @@ -272,15 +272,6 @@ public interface CacheQuery { */ public CacheQueryFuture execute(IgniteReducer rmtReducer, @Nullable Object... args); - /** - * Executes the query the same way as {@link #execute(Object...)} method but transforms result remotely. - * - * @param rmtTransform Remote transformer. - * @param args Optional arguments. - * @return Future for the query result. - */ - public CacheQueryFuture execute(IgniteClosure rmtTransform, @Nullable Object... args); - /** * Gets metrics for this query. * @@ -296,5 +287,5 @@ public interface CacheQuery { /** * @return Scan query iterator. */ - public GridCloseableIterator executeScanQuery() throws IgniteCheckedException; + public GridCloseableIterator executeScanQuery() throws IgniteCheckedException; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java index 5f6cb8f1c9267..d34047e4c430a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java @@ -253,7 +253,7 @@ protected void removeQueryFuture(long reqId) { */ @Nullable private GridCacheQueryInfo distributedQueryInfo(UUID sndId, GridCacheQueryRequest req) { IgniteReducer rdc = req.reducer(); - IgniteClosure trans = req.transformer(); + IgniteClosure trans = (IgniteClosure)req.transformer(); ClusterNode sndNode = cctx.node(sndId); @@ -578,16 +578,16 @@ else if (!cancelled.contains(res.requestId())) /** {@inheritDoc} */ @SuppressWarnings({"unchecked", "serial"}) - @Override public GridCloseableIterator> scanQueryDistributed(final GridCacheQueryAdapter qry, + @Override public GridCloseableIterator scanQueryDistributed(final GridCacheQueryAdapter qry, Collection nodes) throws IgniteCheckedException { assert !cctx.isLocal() : cctx.name(); assert qry.type() == GridCacheQueryType.SCAN: qry; - GridCloseableIterator> locIter0 = null; + GridCloseableIterator locIter0 = null; for (ClusterNode node : nodes) { if (node.isLocal()) { - locIter0 = (GridCloseableIterator)scanQueryLocal(qry, false); + locIter0 = scanQueryLocal(qry, false); Collection rmtNodes = new ArrayList<>(nodes.size() - 1); @@ -603,21 +603,21 @@ else if (!cancelled.contains(res.requestId())) } } - final GridCloseableIterator> locIter = locIter0; + final GridCloseableIterator locIter = locIter0; - final GridCacheQueryBean bean = new GridCacheQueryBean(qry, null, null, null); + final GridCacheQueryBean bean = new GridCacheQueryBean(qry, null, qry.transform(), null); - final CacheQueryFuture> fut = (CacheQueryFuture>)queryDistributed(bean, nodes); + final CacheQueryFuture fut = (CacheQueryFuture)queryDistributed(bean, nodes); - return new GridCloseableIteratorAdapter>() { + return new GridCloseableIteratorAdapter() { /** */ - private Map.Entry cur; + private Object cur; - @Override protected Map.Entry onNext() throws IgniteCheckedException { + @Override protected Object onNext() throws IgniteCheckedException { if (!onHasNext()) throw new NoSuchElementException(); - Map.Entry e = cur; + Object e = cur; cur = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java index 183abde8625ab..147725bd6353f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java @@ -19,7 +19,6 @@ import java.util.Collection; import java.util.List; -import java.util.Map; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; @@ -90,7 +89,7 @@ public class GridCacheLocalQueryManager extends GridCacheQueryManager> scanQueryDistributed(GridCacheQueryAdapter qry, + @Override public GridCloseableIterator scanQueryDistributed(GridCacheQueryAdapter qry, Collection nodes) throws IgniteCheckedException { assert cctx.isLocal() : cctx.name(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java index 90e14f4b30ba0..f65b733bfe969 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java @@ -87,6 +87,9 @@ public class GridCacheQueryAdapter implements CacheQuery { /** */ private final IgniteBiPredicate filter; + /** Transformer. */ + private IgniteClosure transform; + /** Partition. */ private Integer part; @@ -123,6 +126,39 @@ public class GridCacheQueryAdapter implements CacheQuery { /** */ private int taskHash; + /** + * @param cctx Context. + * @param type Query type. + * @param filter Scan filter. + * @param part Partition. + * @param keepBinary Keep binary flag. + */ + public GridCacheQueryAdapter(GridCacheContext cctx, + GridCacheQueryType type, + @Nullable IgniteBiPredicate filter, + @Nullable IgniteClosure transform, + @Nullable Integer part, + boolean keepBinary) { + assert cctx != null; + assert type != null; + assert part == null || part >= 0; + + this.cctx = cctx; + this.type = type; + this.filter = filter; + this.transform = transform; + this.part = part; + this.keepBinary = keepBinary; + + log = cctx.logger(getClass()); + + metrics = new GridCacheQueryMetricsAdapter(); + + this.incMeta = false; + this.clsName = null; + this.clause = null; + } + /** * @param cctx Context. * @param type Query type. @@ -375,6 +411,14 @@ public ClusterGroup projection() { return (IgniteBiPredicate)filter; } + /** + * @return Transformer. + */ + @SuppressWarnings("unchecked") + @Nullable public IgniteClosure, Object> transform() { + return (IgniteClosure, Object>) transform; + } + /** * @return Partition. */ @@ -402,17 +446,12 @@ public void onCompleted(Object res, Throwable err, long startTime, long duration /** {@inheritDoc} */ @Override public CacheQueryFuture execute(@Nullable Object... args) { - return execute(null, null, args); + return execute0(null, args); } /** {@inheritDoc} */ @Override public CacheQueryFuture execute(IgniteReducer rmtReducer, @Nullable Object... args) { - return execute(rmtReducer, null, args); - } - - /** {@inheritDoc} */ - @Override public CacheQueryFuture execute(IgniteClosure rmtTransform, @Nullable Object... args) { - return execute(null, rmtTransform, args); + return execute0(rmtReducer, args); } /** {@inheritDoc} */ @@ -427,13 +466,11 @@ public void onCompleted(Object res, Throwable err, long startTime, long duration /** * @param rmtReducer Optional reducer. - * @param rmtTransform Optional transformer. * @param args Arguments. * @return Future. */ @SuppressWarnings({"IfMayBeConditional", "unchecked"}) - private CacheQueryFuture execute(@Nullable IgniteReducer rmtReducer, - @Nullable IgniteClosure rmtTransform, @Nullable Object... args) { + private CacheQueryFuture execute0(@Nullable IgniteReducer rmtReducer, @Nullable Object... args) { assert type != SCAN : this; Collection nodes; @@ -455,7 +492,7 @@ private CacheQueryFuture execute(@Nullable IgniteReducer rmtReducer if (cctx.deploymentEnabled()) { try { - cctx.deploy().registerClasses(filter, rmtReducer, rmtTransform); + cctx.deploy().registerClasses(filter, rmtReducer); cctx.deploy().registerClasses(args); } catch (IgniteCheckedException e) { @@ -469,7 +506,7 @@ private CacheQueryFuture execute(@Nullable IgniteReducer rmtReducer taskHash = cctx.kernalContext().job().currentTaskNameHash(); final GridCacheQueryBean bean = new GridCacheQueryBean(this, (IgniteReducer)rmtReducer, - (IgniteClosure)rmtTransform, args); + null, args); final GridCacheQueryManager qryMgr = cctx.queries(); @@ -484,8 +521,8 @@ private CacheQueryFuture execute(@Nullable IgniteReducer rmtReducer /** {@inheritDoc} */ @SuppressWarnings({"IfMayBeConditional", "unchecked"}) - @Override public GridCloseableIterator executeScanQuery() throws IgniteCheckedException { - assert type == SCAN: "Wrong processing of qyery: " + type; + @Override public GridCloseableIterator executeScanQuery() throws IgniteCheckedException { + assert type == SCAN : "Wrong processing of qyery: " + type; Collection nodes = nodes(); @@ -508,7 +545,7 @@ private CacheQueryFuture execute(@Nullable IgniteReducer rmtReducer final GridCacheQueryManager qryMgr = cctx.queries(); if (part != null && !cctx.isLocal()) - return (GridCloseableIterator)new ScanQueryFallbackClosableIterator(part, this, qryMgr, cctx); + return new ScanQueryFallbackClosableIterator(part, this, qryMgr, cctx); else { boolean loc = nodes.size() == 1 && F.first(nodes).id().equals(cctx.localNodeId()); @@ -676,7 +713,7 @@ private void init() { try { GridCloseableIterator it = qryMgr.scanQueryLocal(qry, true); - tuple= new T2(it, null); + tuple = new T2(it, null); } catch (IgniteClientDisconnectedCheckedException e) { throw CU.convertToCacheException(e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryBean.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryBean.java index 5a4d6939538f1..286ddc2aff164 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryBean.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryBean.java @@ -33,7 +33,7 @@ public class GridCacheQueryBean { private final IgniteReducer rdc; /** */ - private final IgniteClosure trans; + private final IgniteClosure trans; /** */ private final Object[] args; @@ -45,7 +45,7 @@ public class GridCacheQueryBean { * @param args Optional arguments. */ public GridCacheQueryBean(GridCacheQueryAdapter qry, @Nullable IgniteReducer rdc, - @Nullable IgniteClosure trans, @Nullable Object[] args) { + @Nullable IgniteClosure trans, @Nullable Object[] args) { assert qry != null; this.qry = qry; @@ -71,7 +71,7 @@ public GridCacheQueryAdapter query() { /** * @return Transformer. */ - @Nullable public IgniteClosure transform() { + @Nullable public IgniteClosure transform() { return trans; @@ -88,4 +88,4 @@ public GridCacheQueryAdapter query() { @Override public String toString() { return S.toString(GridCacheQueryBean.class, this); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryInfo.java index 8d2e67d985d54..0a108d580aedb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryInfo.java @@ -31,7 +31,7 @@ class GridCacheQueryInfo { private boolean loc; /** */ - private IgniteClosure trans; + private IgniteClosure trans; /** */ private IgniteReducer rdc; @@ -71,7 +71,7 @@ class GridCacheQueryInfo { */ GridCacheQueryInfo( boolean loc, - IgniteClosure trans, + IgniteClosure trans, IgniteReducer rdc, GridCacheQueryAdapter qry, GridCacheLocalQueryFuture locFut, @@ -117,7 +117,7 @@ GridCacheQueryAdapter query() { /** * @return Transformer. */ - IgniteClosure transformer() { + IgniteClosure transformer() { return trans; } @@ -167,4 +167,4 @@ Object[] arguments() { @Override public String toString() { return S.toString(GridCacheQueryInfo.class, this); } -} \ No newline at end of file +} 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 163bac5b820e3..454ce047130dd 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 @@ -506,7 +506,7 @@ public CacheQueryFuture queryLocal(GridCacheQueryBean qry) { * @return Iterator. * @throws IgniteCheckedException If failed. */ - public abstract GridCloseableIterator> scanQueryDistributed(GridCacheQueryAdapter qry, + public abstract GridCloseableIterator scanQueryDistributed(GridCacheQueryAdapter qry, Collection nodes) throws IgniteCheckedException; /** @@ -1067,13 +1067,13 @@ else if (part < 0 || part >= cctx.affinity().partitions()) final GridDhtLocalPartition locPart0 = locPart; return new PeekValueExpiryAwareIterator(keyIter, plc, topVer, keyValFilter, qry.keepBinary(), locNode, true) { - @Override protected void onClose() { - super.onClose(); + @Override protected void onClose() { + super.onClose(); - if (locPart0 != null) - locPart0.release(); - } - }; + if (locPart0 != null) + locPart0.release(); + } + }; } /** @@ -1163,9 +1163,8 @@ private void advance() { key = (K)cctx.unwrapBinaryIfNeeded(key, keepBinary); - if (filter != null || locNode) { + if (filter != null || locNode) val = (V)cctx.unwrapBinaryIfNeeded(val, keepBinary); - } if (filter != null && !filter.apply(key, val)) continue; @@ -1187,14 +1186,14 @@ private void advance() { * @param locNode Local node. * @return Final key-value iterator. */ - private GridIterator> scanExpiryIterator( + private GridIterator> scanExpiryIterator( final Iterator> it, AffinityTopologyVersion topVer, @Nullable final IgniteBiPredicate filter, ExpiryPolicy expPlc, final boolean keepBinary, boolean locNode) { - Iterator keyIter = new Iterator() { + Iterator keyIter = new Iterator() { /** {@inheritDoc} */ @Override public boolean hasNext() { return it.hasNext(); @@ -1267,10 +1266,8 @@ protected void runFieldsQuery(GridCacheQueryInfo qryInfo) { try { // Preparing query closures. - IgniteClosure trans = (IgniteClosure)qryInfo.transformer(); IgniteReducer rdc = (IgniteReducer)qryInfo.reducer(); - injectResources(trans); injectResources(rdc); GridCacheQueryAdapter qry = qryInfo.query(); @@ -1289,7 +1286,7 @@ protected void runFieldsQuery(GridCacheQueryInfo qryInfo) { res = qryInfo.local() ? executeFieldsQuery(qry, qryInfo.arguments(), qryInfo.local(), qry.subjectId(), taskName, - recipient(qryInfo.senderId(), qryInfo.requestId())) : + recipient(qryInfo.senderId(), qryInfo.requestId())) : fieldsQueryResult(qryInfo, taskName); // If metadata needs to be returned to user and cleaned from internal fields - copy it. @@ -1460,10 +1457,10 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { try { // Preparing query closures. - IgniteClosure, Object> trans = - (IgniteClosure, Object>)qryInfo.transformer(); + IgniteClosure, Object> trans = + (IgniteClosure, Object>)qryInfo.transformer(); - IgniteReducer, Object> rdc = (IgniteReducer, Object>)qryInfo.reducer(); + IgniteReducer, Object> rdc = (IgniteReducer, Object>)qryInfo.reducer(); injectResources(trans); injectResources(rdc); @@ -1481,13 +1478,13 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { res = loc ? executeQuery(qry, qryInfo.arguments(), loc, qry.subjectId(), taskName, - recipient(qryInfo.senderId(), qryInfo.requestId())) : + recipient(qryInfo.senderId(), qryInfo.requestId())) : queryResult(qryInfo, taskName); iter = res.iterator(recipient(qryInfo.senderId(), qryInfo.requestId())); type = res.type(); - GridCacheAdapter cache = cctx.cache(); + final GridCacheAdapter cache = cctx.cache(); if (log.isDebugEnabled()) log.debug("Received index iterator [iterHasNext=" + iter.hasNext() + @@ -1518,7 +1515,7 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { break; } - K key = row.getKey(); + final K key = row.getKey(); // Filter backups for SCAN queries, if it isn't partition scan. // Other types are filtered in indexing manager. @@ -1561,8 +1558,8 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { } if (readEvt) { - K key0 = (K)cctx.unwrapBinaryIfNeeded(key, qry.keepBinary()); - V val0 = (V)cctx.unwrapBinaryIfNeeded(val, qry.keepBinary()); + K key0 = (K)cctx.unwrapBinaryIfNeeded(key, qry.keepBinary()); + V val0 = (V)cctx.unwrapBinaryIfNeeded(val, qry.keepBinary()); switch (type) { case SQL: @@ -1630,27 +1627,33 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { } } - Map.Entry entry = F.t(key, val); + if (rdc != null || trans != null) { + Cache.Entry entry; - // Unwrap entry for reducer or transformer only. - if (rdc != null || trans != null) - entry = (Map.Entry)cctx.unwrapBinaryIfNeeded(entry, qry.keepBinary()); + if (qry.keepBinary()) + entry = cache.keepBinary().getEntry(key); + else + entry = cache.getEntry(key); - // Reduce. - if (rdc != null) { - if (!rdc.collect(entry) || !iter.hasNext()) { - onPageReady(loc, qryInfo, Collections.singletonList(rdc.reduce()), true, null); + // Reduce. + if (rdc != null) { + if (!rdc.collect(entry) || !iter.hasNext()) { + onPageReady(loc, qryInfo, Collections.singletonList(rdc.reduce()), true, null); - pageSent = true; + pageSent = true; - break; + break; + } + else + continue; } - else - continue; + + data.add(trans != null ? trans.apply(entry) : + !loc ? new GridCacheQueryResponseEntry<>(key, val) : F.t(key, val)); } + else + data.add(!loc ? new GridCacheQueryResponseEntry<>(key, val) : F.t(key, val)); - data.add(trans != null ? trans.apply(entry) : - !loc ? new GridCacheQueryResponseEntry<>(key, val) : F.t(key, val)); if (!loc) { if (++cnt == pageSize || !iter.hasNext()) { @@ -1720,7 +1723,7 @@ else if (rmvIter) * @param updStatisticsIfNeeded Update statistics flag. */ @SuppressWarnings({"unchecked", "serial"}) - protected GridCloseableIterator> scanQueryLocal(final GridCacheQueryAdapter qry, + protected GridCloseableIterator scanQueryLocal(final GridCacheQueryAdapter qry, final boolean updStatisticsIfNeeded) throws IgniteCheckedException { if (!enterBusy()) throw new IllegalStateException("Failed to process query request (grid is stopping)."); @@ -1769,8 +1772,8 @@ protected GridCloseableIterator> scanQueryLocal(final GridCa final boolean readEvt = cctx.gridEvents().isRecordable(EVT_CACHE_QUERY_OBJECT_READ); - return new GridCloseableIteratorAdapter>() { - @Override protected IgniteBiTuple onNext() throws IgniteCheckedException { + return new GridCloseableIteratorAdapter() { + @Override protected Object onNext() throws IgniteCheckedException { long start = statsEnabled ? System.nanoTime() : 0L; IgniteBiTuple next = iter.nextX(); @@ -1803,7 +1806,20 @@ protected GridCloseableIterator> scanQueryLocal(final GridCa null)); } - return next; + IgniteClosure transform = qry.transform(); + + if (transform == null) + return next; + + Cache.Entry entry; + + if (qry.keepBinary()) + entry = cctx.cache().keepBinary().getEntry(next.getKey()); + else + entry = cctx.cache().getEntry(next.getKey()); + + + return transform.apply(entry); } @Override protected boolean onHasNext() throws IgniteCheckedException { @@ -1832,7 +1848,8 @@ protected GridCloseableIterator> scanQueryLocal(final GridCa * @return Iterator. * @throws IgniteCheckedException In case of error. */ - private QueryResult queryResult(final GridCacheQueryInfo qryInfo, String taskName) throws IgniteCheckedException { + private QueryResult queryResult(final GridCacheQueryInfo qryInfo, + String taskName) throws IgniteCheckedException { assert qryInfo != null; final UUID sndId = qryInfo.senderId(); @@ -2845,7 +2862,7 @@ private OffheapIteratorClosure( if (locNode) return new IgniteBiTuple<>(key, val); - else{ + else { if (key instanceof CacheObject) ((CacheObject)key).prepareMarshal(cctx.cacheObjectContext()); @@ -3256,16 +3273,28 @@ public CacheQuery createSpiQuery(boolean keepBinary) { * @param keepBinary Keep binary flag. * @return Created query. */ - public CacheQuery> createScanQuery(@Nullable IgniteBiPredicate filter, + public CacheQuery createScanQuery(@Nullable IgniteBiPredicate filter, @Nullable Integer part, boolean keepBinary) { + return createScanQuery(filter, null, part, keepBinary); + } - return new GridCacheQueryAdapter<>(cctx, + /** + * Creates user's predicate based scan query. + * + * @param filter Scan filter. + * @param part Partition. + * @param keepBinary Keep binary flag. + * @return Created query. + */ + public CacheQuery createScanQuery(@Nullable IgniteBiPredicate filter, + @Nullable IgniteClosure trans, + @Nullable Integer part, boolean keepBinary) { + + return new GridCacheQueryAdapter(cctx, SCAN, - null, - null, - (IgniteBiPredicate)filter, + filter, + trans, part, - false, keepBinary); } 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 f50fba03e1869..5610befe1d0f4 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 @@ -84,7 +84,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache /** */ @GridDirectTransient - private IgniteClosure trans; + private IgniteClosure trans; /** */ private byte[] transBytes; @@ -233,7 +233,7 @@ public GridCacheQueryRequest( IgniteBiPredicate keyValFilter, @Nullable Integer part, IgniteReducer rdc, - IgniteClosure trans, + IgniteClosure trans, int pageSize, boolean incBackups, Object[] args, @@ -422,7 +422,7 @@ public IgniteReducer reducer() { /** * @return Transformer. */ - public IgniteClosure transformer() { + public IgniteClosure transformer() { return trans; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java new file mode 100644 index 0000000000000..6b13e05988b4c --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java @@ -0,0 +1,570 @@ +/* + * 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.query; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.Callable; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.cache.query.ContinuousQuery; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.cache.query.SpiQuery; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.cache.query.TextQuery; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Test for scan query with transformer. + */ +public class GridCacheQueryTransformerSelfTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); + cfg.setMarshaller(null); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + startGridsMultiThreaded(3); + + Ignition.setClientMode(true); + + startGrid(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testGetKeys() throws Exception { + IgniteCache cache = grid().createCache("test-cache"); + + try { + for (int i = 0; i < 50; i++) + cache.put(i, "val" + i); + + IgniteClosure, Integer> transformer = + new IgniteClosure, Integer>() { + @Override public Integer apply(Cache.Entry e) { + return e.getKey(); + } + }; + + List keys = cache.query(new ScanQuery(), transformer).getAll(); + + assertEquals(50, keys.size()); + + Collections.sort(keys); + + for (int i = 0; i < 50; i++) + assertEquals(i, keys.get(i).intValue()); + } + finally { + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testGetKeysFiltered() throws Exception { + IgniteCache cache = grid().createCache("test-cache"); + + try { + for (int i = 0; i < 50; i++) + cache.put(i, "val" + i); + + IgniteBiPredicate filter = new IgniteBiPredicate() { + @Override public boolean apply(Integer k, String v) { + return k % 10 == 0; + } + }; + + IgniteClosure, Integer> transformer = + new IgniteClosure, Integer>() { + @Override public Integer apply(Cache.Entry e) { + return e.getKey(); + } + }; + + List keys = cache.query(new ScanQuery<>(filter), transformer).getAll(); + + assertEquals(5, keys.size()); + + Collections.sort(keys); + + for (int i = 0; i < 5; i++) + assertEquals(i * 10, keys.get(i).intValue()); + } + finally { + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testGetObjectField() throws Exception { + IgniteCache cache = grid().createCache("test-cache"); + + try { + for (int i = 0; i < 50; i++) + cache.put(i, new Value("str" + i, i * 100)); + + IgniteClosure, Integer> transformer = + new IgniteClosure, Integer>() { + @Override public Integer apply(Cache.Entry e) { + return e.getValue().idx; + } + }; + + List res = cache.query(new ScanQuery(), transformer).getAll(); + + assertEquals(50, res.size()); + + Collections.sort(res); + + for (int i = 0; i < 50; i++) + assertEquals(i * 100, res.get(i).intValue()); + } + finally { + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testGetObjectFieldFiltered() throws Exception { + IgniteCache cache = grid().createCache("test-cache"); + + try { + for (int i = 0; i < 50; i++) + cache.put(i, new Value("str" + i, i * 100)); + + IgniteBiPredicate filter = new IgniteBiPredicate() { + @Override public boolean apply(Integer k, Value v) { + return v.idx % 1000 == 0; + } + }; + + IgniteClosure, Integer> transformer = + new IgniteClosure, Integer>() { + @Override public Integer apply(Cache.Entry e) { + return e.getValue().idx; + } + }; + + List res = cache.query(new ScanQuery<>(filter), transformer).getAll(); + + assertEquals(5, res.size()); + + Collections.sort(res); + + for (int i = 0; i < 5; i++) + assertEquals(i * 1000, res.get(i).intValue()); + } + finally { + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testKeepBinary() throws Exception { + IgniteCache cache = grid().createCache("test-cache"); + + try { + for (int i = 0; i < 50; i++) + cache.put(i, new Value("str" + i, i * 100)); + + IgniteCache binaryCache = cache.withKeepBinary(); + + IgniteClosure, Integer> transformer = + new IgniteClosure, Integer>() { + @Override public Integer apply(Cache.Entry e) { + return e.getValue().field("idx"); + } + }; + + List res = binaryCache.query(new ScanQuery(), transformer).getAll(); + + assertEquals(50, res.size()); + + Collections.sort(res); + + for (int i = 0; i < 50; i++) + assertEquals(i * 100, res.get(i).intValue()); + } + finally { + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testKeepBinaryFiltered() throws Exception { + IgniteCache cache = grid().createCache("test-cache"); + + try { + for (int i = 0; i < 50; i++) + cache.put(i, new Value("str" + i, i * 100)); + + IgniteCache binaryCache = cache.withKeepBinary(); + + IgniteBiPredicate filter = new IgniteBiPredicate() { + @Override public boolean apply(Integer k, BinaryObject v) { + return v.field("idx") % 1000 == 0; + } + }; + + IgniteClosure, Integer> transformer = + new IgniteClosure, Integer>() { + @Override public Integer apply(Cache.Entry e) { + return e.getValue().field("idx"); + } + }; + + List res = binaryCache.query(new ScanQuery<>(filter), transformer).getAll(); + + assertEquals(5, res.size()); + + Collections.sort(res); + + for (int i = 0; i < 5; i++) + assertEquals(i * 1000, res.get(i).intValue()); + } + finally { + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testLocal() throws Exception { + IgniteCache cache = grid().createCache("test-cache"); + + try { + for (int i = 0; i < 50; i++) + cache.put(i, new Value("str" + i, i * 100)); + + Collection> lists = grid().compute().broadcast(new IgniteCallable>() { + @IgniteInstanceResource + private Ignite ignite; + + @Override public List call() throws Exception { + IgniteClosure, Integer> transformer = + new IgniteClosure, Integer>() { + @Override public Integer apply(Cache.Entry e) { + return e.getValue().idx; + } + }; + + return ignite.cache("test-cache").query(new ScanQuery().setLocal(true), + transformer).getAll(); + } + }); + + List res = new ArrayList<>(F.flatCollections(lists)); + + assertEquals(50, res.size()); + + Collections.sort(res); + + for (int i = 0; i < 50; i++) + assertEquals(i * 100, res.get(i).intValue()); + } + finally { + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testLocalFiltered() throws Exception { + IgniteCache cache = grid().createCache("test-cache"); + + try { + for (int i = 0; i < 50; i++) + cache.put(i, new Value("str" + i, i * 100)); + + Collection> lists = grid().compute().broadcast(new IgniteCallable>() { + @IgniteInstanceResource + private Ignite ignite; + + @Override public List call() throws Exception { + IgniteBiPredicate filter = new IgniteBiPredicate() { + @Override public boolean apply(Integer k, Value v) { + return v.idx % 1000 == 0; + } + }; + + IgniteClosure, Integer> transformer = + new IgniteClosure, Integer>() { + @Override public Integer apply(Cache.Entry e) { + return e.getValue().idx; + } + }; + + return ignite.cache("test-cache").query(new ScanQuery<>(filter).setLocal(true), + transformer).getAll(); + } + }); + + List res = new ArrayList<>(F.flatCollections(lists)); + + assertEquals(5, res.size()); + + Collections.sort(res); + + for (int i = 0; i < 5; i++) + assertEquals(i * 1000, res.get(i).intValue()); + } + finally { + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testLocalKeepBinary() throws Exception { + IgniteCache cache = grid().createCache("test-cache"); + + try { + for (int i = 0; i < 50; i++) + cache.put(i, new Value("str" + i, i * 100)); + + Collection> lists = grid().compute().broadcast(new IgniteCallable>() { + @IgniteInstanceResource + private Ignite ignite; + + @Override public List call() throws Exception { + IgniteClosure, Integer> transformer = + new IgniteClosure, Integer>() { + @Override public Integer apply(Cache.Entry e) { + return e.getValue().field("idx"); + } + }; + + return ignite.cache("test-cache").withKeepBinary().query( + new ScanQuery().setLocal(true), transformer).getAll(); + } + }); + + List res = new ArrayList<>(F.flatCollections(lists)); + + assertEquals(50, res.size()); + + Collections.sort(res); + + for (int i = 0; i < 50; i++) + assertEquals(i * 100, res.get(i).intValue()); + } + finally { + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testLocalKeepBinaryFiltered() throws Exception { + IgniteCache cache = grid().createCache("test-cache"); + + try { + for (int i = 0; i < 50; i++) + cache.put(i, new Value("str" + i, i * 100)); + + Collection> lists = grid().compute().broadcast(new IgniteCallable>() { + @IgniteInstanceResource + private Ignite ignite; + + @Override public List call() throws Exception { + IgniteBiPredicate filter = new IgniteBiPredicate() { + @Override public boolean apply(Integer k, BinaryObject v) { + return v.field("idx") % 1000 == 0; + } + }; + + IgniteClosure, Integer> transformer = + new IgniteClosure, Integer>() { + @Override public Integer apply(Cache.Entry e) { + return e.getValue().field("idx"); + } + }; + + return ignite.cache("test-cache").withKeepBinary().query(new ScanQuery<>(filter).setLocal(true), + transformer).getAll(); + } + }); + + List res = new ArrayList<>(F.flatCollections(lists)); + + assertEquals(5, res.size()); + + Collections.sort(res); + + for (int i = 0; i < 5; i++) + assertEquals(i * 1000, res.get(i).intValue()); + } + finally { + cache.destroy(); + } + } + + /** + * @throws Exception If failed. + */ + public void testUnsupported() throws Exception { + final IgniteCache cache = grid().createCache("test-cache"); + + final IgniteClosure, Integer> transformer = + new IgniteClosure, Integer>() { + @Override public Integer apply(Cache.Entry e) { + return null; + } + }; + + try { + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + cache.query(new SqlQuery(Integer.class, "clause"), transformer); + + return null; + } + }, + UnsupportedOperationException.class, + "Transformers are supported only for SCAN queries." + ); + + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + cache.query(new SqlFieldsQuery("clause"), new IgniteClosure, Object>() { + @Override public Object apply(List objects) { + return null; + } + }); + + return null; + } + }, + UnsupportedOperationException.class, + "Transformers are supported only for SCAN queries." + ); + + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + cache.query(new TextQuery(Integer.class, "clause"), transformer); + + return null; + } + }, + UnsupportedOperationException.class, + "Transformers are supported only for SCAN queries." + ); + + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + cache.query(new SpiQuery(), transformer); + + return null; + } + }, + UnsupportedOperationException.class, + "Transformers are supported only for SCAN queries." + ); + + GridTestUtils.assertThrows( + log, + new Callable() { + @Override public Object call() throws Exception { + cache.query(new ContinuousQuery(), transformer); + + return null; + } + }, + UnsupportedOperationException.class, + "Transformers are supported only for SCAN queries." + ); + } + finally { + cache.destroy(); + } + } + + /** + */ + private static class Value { + /** */ + @SuppressWarnings("unused") + private String str; + + /** */ + private int idx; + + /** + * @param str String. + * @param idx Integer. + */ + public Value(String str, int idx) { + this.str = str; + this.idx = idx; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java index 740b20152b0c7..71dc96453226e 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java @@ -46,6 +46,7 @@ import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl; import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.mxbean.CacheMetricsMXBean; import org.apache.ignite.resources.IgniteInstanceResource; @@ -171,6 +172,11 @@ private IgniteCacheProcessProxy(String name, boolean async, ExpiryPolicy plc, Ig throw new UnsupportedOperationException("Method should be supported."); } + /** {@inheritDoc} */ + @Override public QueryCursor query(Query qry, IgniteClosure transformer) { + throw new UnsupportedOperationException("Method should be supported."); + } + /** {@inheritDoc} */ @Override public Iterable> localEntries(CachePeekMode... peekModes) throws CacheException { return compute.call(new LocalEntriesTask(cacheName, isAsync, peekModes)); diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java index 1b1908d4add6e..3652acda85944 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java @@ -55,6 +55,7 @@ import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQuerySelfTest; import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest; import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQuerySelfTest; +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryTransformerSelfTest; import org.apache.ignite.internal.processors.query.IgniteSqlSchemaIndexingTest; import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest; import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest; @@ -115,6 +116,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteBinaryWrappedObjectFieldsQuerySelfTest.class); suite.addTestSuite(IgniteCacheQueryH2IndexingLeakTest.class); suite.addTestSuite(IgniteCacheQueryNoRebalanceSelfTest.class); + suite.addTestSuite(GridCacheQueryTransformerSelfTest.class); return suite; } From f9ff97c91374dcd9cd8ad08d46d1d2de44193060 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 30 Aug 2016 09:31:20 +0300 Subject: [PATCH 072/487] ignite-2560 Support resource injection for entry processor, optimizations for resource injection. --- .../processors/cache/CacheLazyEntry.java | 2 + .../EntryProcessorResourceInjectorProxy.java | 105 +++++ .../processors/cache/GridCacheMapEntry.java | 13 +- .../GridNearAtomicSingleUpdateFuture.java | 17 +- .../atomic/GridNearAtomicUpdateFuture.java | 8 +- .../local/atomic/GridLocalAtomicCache.java | 18 +- .../transactions/IgniteTxLocalAdapter.java | 5 +- .../processors/resource/GridResourceIoc.java | 438 ++++++++++++++---- .../resource/GridResourceProcessor.java | 396 +++++++--------- .../GridCacheAbstractFullApiSelfTest.java | 404 ++++++++++++++-- .../cache/GridCacheAbstractSelfTest.java | 140 +++++- .../GridCacheTransformEventSelfTest.java | 66 ++- ...artitionedBasicStoreMultiNodeSelfTest.java | 2 + .../GridTransformSpringInjectionSelfTest.java | 186 ++++++++ .../testsuites/IgniteSpringTestSuite.java | 7 +- .../IgniteInvokeWithInjectionBenchmark.java | 74 +++ .../IgniteInvokeWithInjectionTxBenchmark.java | 30 ++ 17 files changed, 1515 insertions(+), 396 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryProcessorResourceInjectorProxy.java create mode 100644 modules/spring/src/test/java/org/apache/ignite/internal/processors/resource/GridTransformSpringInjectionSelfTest.java create mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteInvokeWithInjectionBenchmark.java create mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteInvokeWithInjectionTxBenchmark.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java index c8cfc992034d3..02cccc742cc99 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java @@ -191,6 +191,8 @@ public void updateCounter(Long updateCntr) { @Override public T unwrap(Class cls) { if (cls.isAssignableFrom(Ignite.class)) return (T)cctx.kernalContext().grid(); + else if (cls.isAssignableFrom(GridCacheContext.class)) + return (T)cctx; else if (cls.isAssignableFrom(getClass())) return cls.cast(this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryProcessorResourceInjectorProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryProcessorResourceInjectorProxy.java new file mode 100644 index 0000000000000..76b2511138d1f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryProcessorResourceInjectorProxy.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.io.Serializable; +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.processors.resource.GridResourceIoc; +import org.apache.ignite.internal.processors.resource.GridResourceProcessor; +import org.jetbrains.annotations.Nullable; + +/** + * Entry processor wrapper injecting Ignite resources into target processor before execution. + */ +public class EntryProcessorResourceInjectorProxy implements EntryProcessor, Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** Delegate. */ + private EntryProcessor delegate; + + /** Injected flag. */ + private transient boolean injected; + + /** + * @param delegate Delegate. + */ + private EntryProcessorResourceInjectorProxy(EntryProcessor delegate) { + this.delegate = delegate; + } + + /** {@inheritDoc} */ + @Override public T process(MutableEntry entry, Object... arguments) throws EntryProcessorException { + if (!injected) { + GridCacheContext cctx = entry.unwrap(GridCacheContext.class); + + GridResourceProcessor rsrc = cctx.kernalContext().resource(); + + try { + rsrc.inject(delegate, GridResourceIoc.AnnotationSet.ENTRY_PROCESSOR, cctx.name()); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + + injected = true; + } + + return delegate.process(entry, arguments); + } + + /** + * @return Delegate entry processor. + */ + public EntryProcessor delegate() { + return delegate; + } + + /** + * Wraps EntryProcessor if needed. + * + * @param ctx Context. + * @param proc Entry proc. + * @return Wrapped entry proc if wrapping is needed. + */ + public static EntryProcessor wrap(GridKernalContext ctx, + @Nullable EntryProcessor proc) { + if (proc == null || proc instanceof EntryProcessorResourceInjectorProxy) + return proc; + + GridResourceProcessor rsrcProcessor = ctx.resource(); + + return rsrcProcessor.isAnnotationsPresent(null, proc, GridResourceIoc.AnnotationSet.ENTRY_PROCESSOR) ? + new EntryProcessorResourceInjectorProxy<>(proc) : proc; + } + + /** + * Unwraps EntryProcessor as Object if needed. + * + * @param obj Entry processor. + * @return Unwrapped entry processor. + */ + static Object unwrap(Object obj) { + return (obj instanceof EntryProcessorResourceInjectorProxy) ? ((EntryProcessorResourceInjectorProxy)obj).delegate() : obj; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index f692bf40592a5..c760ac17f5639 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -896,6 +896,8 @@ private Object innerGet0( } if (evt && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) { + transformClo = EntryProcessorResourceInjectorProxy.unwrap(transformClo); + cctx.events().addEvent( partition(), key, @@ -1004,7 +1006,9 @@ else if (tx.dht()) { deletedUnlocked(false); } - if (evt && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) + if (evt && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) { + transformClo = EntryProcessorResourceInjectorProxy.unwrap(transformClo); + cctx.events().addEvent( partition(), key, @@ -1019,6 +1023,7 @@ else if (tx.dht()) { transformClo != null ? transformClo.getClass().getName() : null, taskName, keepBinary); + } } } @@ -1685,7 +1690,7 @@ else if (ttl == CU.TTL_NOT_CHANGED) // Calculate new value. if (op == GridCacheOperation.TRANSFORM) { - transformCloClsName = writeObj.getClass().getName(); + transformCloClsName = EntryProcessorResourceInjectorProxy.unwrap(writeObj).getClass().getName(); EntryProcessor entryProcessor = (EntryProcessor)writeObj; @@ -2463,6 +2468,8 @@ assert deletedUnlocked() || new0 || isInternal(): "Invalid entry [entry=" + this if (transformClo != null && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) { evtOld = cctx.unwrapTemporary(oldVal); + transformClo = EntryProcessorResourceInjectorProxy.unwrap(transformClo); + cctx.events().addEvent(partition(), key, evtNodeId, null, newVer, EVT_CACHE_OBJECT_READ, evtOld, evtOld != null || hadVal, evtOld, evtOld != null || hadVal, subjId, transformClo.getClass().getName(), taskName, @@ -2553,6 +2560,8 @@ assert deletedUnlocked() || new0 || isInternal() : "Invalid entry [entry=" + thi if (transformClo != null && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) { evtOld = cctx.unwrapTemporary(oldVal); + transformClo = EntryProcessorResourceInjectorProxy.unwrap(transformClo); + cctx.events().addEvent(partition(), key, evtNodeId, null, newVer, EVT_CACHE_OBJECT_READ, evtOld, evtOld != null || hadVal, evtOld, evtOld != null || hadVal, subjId, transformClo.getClass().getName(), taskName, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java index 661a178435e1d..256c7ac01417d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java @@ -17,6 +17,13 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.UUID; +import javax.cache.expiry.ExpiryPolicy; +import javax.cache.processor.EntryProcessor; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cluster.ClusterNode; @@ -26,6 +33,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException; +import org.apache.ignite.internal.processors.cache.EntryProcessorResourceInjectorProxy; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheOperation; import org.apache.ignite.internal.processors.cache.GridCacheReturn; @@ -43,13 +51,6 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; -import javax.cache.expiry.ExpiryPolicy; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Map; -import java.util.UUID; - import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM; @@ -549,6 +550,8 @@ private GridNearAtomicUpdateRequest mapSingleUpdate(AffinityTopologyVersion topV if (op != TRANSFORM) val = cctx.toCacheObject(val); + else + val = EntryProcessorResourceInjectorProxy.wrap(cctx.kernalContext(), (EntryProcessor)val); ClusterNode primary = cctx.affinity().primary(cacheKey, topVer); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java index 2432f6355e1cf..30a0c3d9156a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.UUID; import javax.cache.expiry.ExpiryPolicy; +import javax.cache.processor.EntryProcessor; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cluster.ClusterNode; @@ -34,6 +35,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException; +import org.apache.ignite.internal.processors.cache.EntryProcessorResourceInjectorProxy; import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheOperation; @@ -800,7 +802,7 @@ else if (conflictPutVals != null) { val = conflictPutVal.valueEx(); conflictVer = conflictPutVal.version(); - conflictTtl = conflictPutVal.ttl(); + conflictTtl = conflictPutVal.ttl(); conflictExpireTime = conflictPutVal.expireTime(); } else if (conflictRmvVals != null) { @@ -826,6 +828,8 @@ else if (conflictRmvVals != null) { if (op != TRANSFORM) val = cctx.toCacheObject(val); + else + val = EntryProcessorResourceInjectorProxy.wrap(cctx.kernalContext(), (EntryProcessor)val); List affNodes = mapKey(cacheKey, topVer); @@ -940,6 +944,8 @@ else if (conflictRmvVals != null) { if (op != TRANSFORM) val = cctx.toCacheObject(val); + else + val = EntryProcessorResourceInjectorProxy.wrap(cctx.kernalContext(), (EntryProcessor)val); ClusterNode primary = cctx.affinity().primary(cacheKey.partition(), topVer); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java index ac08f8f695725..a419887698326 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java @@ -57,6 +57,8 @@ import org.apache.ignite.internal.processors.cache.local.GridLocalCache; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.resource.GridResourceIoc; +import org.apache.ignite.internal.processors.resource.GridResourceProcessor; import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.future.GridEmbeddedFuture; @@ -432,7 +434,6 @@ public GridLocalAtomicCache(GridCacheContext ctx) { needVer); } - /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public IgniteInternalFuture> getAllAsync( @@ -511,7 +512,7 @@ private Map getAllInternal(@Nullable Collection keys, entry = swapOrOffheap ? entryEx(cacheKey) : peekEx(cacheKey); if (entry != null) { - CacheObject v ; + CacheObject v; GridCacheVersion ver; if (needVer) { @@ -541,7 +542,8 @@ private Map getAllInternal(@Nullable Collection keys, deserializeBinary, true, ver); - }else + } + else success = false; } else { @@ -944,6 +946,8 @@ private Object updateAllInternal(GridCacheOperation op, if (op == UPDATE) val = ctx.toCacheObject(val); + else if (op == TRANSFORM) + ctx.kernalContext().resource().inject(val, GridResourceIoc.AnnotationSet.ENTRY_PROCESSOR, ctx.name()); while (true) { GridCacheEntryEx entry = null; @@ -1014,7 +1018,7 @@ else if (res == null) if (err != null) throw err; - Object ret = res == null ? null : rawRetval ? new GridCacheReturn(ctx, true, keepBinary, res.get2(), res.get1()) : + Object ret = res == null ? null : rawRetval ? new GridCacheReturn(ctx, true, keepBinary, res.get2(), res.get1()) : (retval || op == TRANSFORM) ? res.get2() : res.get1(); if (op == TRANSFORM && ret == null) @@ -1035,8 +1039,8 @@ else if (res == null) * @param filter Optional filter. * @param subjId Subject ID. * @param taskName Task name. - * @throws CachePartialUpdateCheckedException If update failed. * @return Results map for invoke operation. + * @throws CachePartialUpdateCheckedException If update failed. */ @SuppressWarnings({"ForLoopReplaceableByForEach", "unchecked"}) private Map updateWithBatch( @@ -1101,6 +1105,10 @@ private Map updateWithBatch( } if (op == TRANSFORM) { + ctx.kernalContext().resource().inject(val, + GridResourceIoc.AnnotationSet.ENTRY_PROCESSOR, + ctx.name()); + EntryProcessor entryProcessor = (EntryProcessor)val; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index 9ad7fb0a4c7f8..ee992cca2e2ba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.cache.CacheInvokeEntry; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.CacheOperationContext; +import org.apache.ignite.internal.processors.cache.EntryProcessorResourceInjectorProxy; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; @@ -65,7 +66,6 @@ import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException; import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; -import org.apache.ignite.transactions.TransactionDeadlockException; import org.apache.ignite.internal.util.GridLeanMap; import org.apache.ignite.internal.util.future.GridEmbeddedFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -89,6 +89,7 @@ import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.plugin.security.SecurityPermission; import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionDeadlockException; import org.apache.ignite.transactions.TransactionIsolation; import org.apache.ignite.transactions.TransactionState; import org.jetbrains.annotations.Nullable; @@ -3664,7 +3665,7 @@ protected final IgniteTxEntry addEntry(GridCacheOperation op, this, op, val, - entryProcessor, + EntryProcessorResourceInjectorProxy.wrap(cctx.kernalContext(), entryProcessor), invokeArgs, hasDrTtl ? drTtl : -1L, entry, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java index 35824fab4ac6f..0158973e8f186 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java @@ -21,12 +21,12 @@ import java.lang.reflect.Field; import java.lang.reflect.Method; import java.util.ArrayList; -import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicReference; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.managers.deployment.GridDeployment; import org.apache.ignite.internal.util.GridLeanIdentitySet; @@ -35,6 +35,17 @@ import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.resources.CacheNameResource; +import org.apache.ignite.resources.CacheStoreSessionResource; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.resources.JobContextResource; +import org.apache.ignite.resources.LoadBalancerResource; +import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.resources.ServiceResource; +import org.apache.ignite.resources.SpringApplicationContextResource; +import org.apache.ignite.resources.SpringResource; +import org.apache.ignite.resources.TaskContinuousMapperResource; +import org.apache.ignite.resources.TaskSessionResource; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -42,17 +53,12 @@ * Resource container contains caches for classes used for injection. * Caches used to improve the efficiency of standard Java reflection mechanism. */ -class GridResourceIoc { +public class GridResourceIoc { /** Task class resource mapping. Used to efficiently cleanup resources related to class loader. */ - private final ConcurrentMap>> taskMap = - new ConcurrentHashMap8<>(); + private final ConcurrentMap>> taskMap = new ConcurrentHashMap8<>(); /** Class descriptors cache. */ - private final ConcurrentMap, ClassDescriptor> clsDescs = new ConcurrentHashMap8<>(); - - /** */ - private final ConcurrentMap, Class[]> annCache = - new ConcurrentHashMap8<>(); + private AtomicReference, ClassDescriptor>> clsDescs = new AtomicReference<>(); /** * @param ldr Class loader. @@ -61,8 +67,22 @@ void onUndeployed(ClassLoader ldr) { Set> clss = taskMap.remove(ldr); if (clss != null) { - clsDescs.keySet().removeAll(clss); - annCache.keySet().removeAll(clss); + Map, ClassDescriptor> newMap, oldMap; + + do { + oldMap = clsDescs.get(); + + if (oldMap == null) + break; + + newMap = new HashMap<>(oldMap.size() - clss.size()); + + for (Map.Entry, ClassDescriptor> entry : oldMap.entrySet()) { + if (!clss.contains(entry.getKey())) + newMap.put(entry.getKey(), entry.getValue()); + } + } + while (!clsDescs.compareAndSet(oldMap, newMap)); } } @@ -71,8 +91,8 @@ void onUndeployed(ClassLoader ldr) { */ void undeployAll() { taskMap.clear(); - clsDescs.clear(); - annCache.clear(); + + clsDescs.set(null); } /** @@ -83,8 +103,8 @@ void undeployAll() { * @param injector Resource to inject. * @param dep Deployment. * @param depCls Deployment class. - * @throws IgniteCheckedException Thrown in case of any errors during injection. * @return {@code True} if resource was injected. + * @throws IgniteCheckedException Thrown in case of any errors during injection. */ @SuppressWarnings("SimplifiableIfStatement") boolean inject(Object target, @@ -92,26 +112,41 @@ boolean inject(Object target, GridResourceInjector injector, @Nullable GridDeployment dep, @Nullable Class depCls) - throws IgniteCheckedException - { + throws IgniteCheckedException { return injectInternal(target, annCls, injector, dep, depCls, null); } /** + * @param dep Deployment. * @param cls Class. + * @return Descriptor. */ - private ClassDescriptor descriptor(@Nullable GridDeployment dep, Class cls) { - ClassDescriptor res = clsDescs.get(cls); + ClassDescriptor descriptor(@Nullable GridDeployment dep, Class cls) { + Map, ClassDescriptor> newMap, oldMap; + ClassDescriptor res, newDesc = null; + + do { + oldMap = clsDescs.get(); + + if (oldMap != null && (res = oldMap.get(cls)) != null) + break; - if (res == null) { if (dep != null) { Set> classes = F.addIfAbsent(taskMap, dep.classLoader(), F.>newCSet()); classes.add(cls); + + dep = null; } - res = F.addIfAbsent(clsDescs, cls, new ClassDescriptor(cls)); + if (oldMap == null) + newMap = new HashMap<>(); + else + (newMap = new HashMap<>(oldMap.size() + 1)).putAll(oldMap); + + newMap.put(cls, res = newDesc == null ? (newDesc = new ClassDescriptor(cls)) : newDesc); } + while (!clsDescs.compareAndSet(oldMap, newMap)); return res; } @@ -123,8 +158,8 @@ private ClassDescriptor descriptor(@Nullable GridDeployment dep, Class cls) { * @param dep Deployment. * @param depCls Deployment class. * @param checkedObjs Set of already inspected objects to avoid indefinite recursion. - * @throws IgniteCheckedException Thrown in case of any errors during injection. * @return {@code True} if resource was injected. + * @throws IgniteCheckedException Thrown in case of any errors during injection. */ private boolean injectInternal(Object target, Class annCls, @@ -132,56 +167,14 @@ private boolean injectInternal(Object target, @Nullable GridDeployment dep, @Nullable Class depCls, @Nullable Set checkedObjs) - throws IgniteCheckedException - { + throws IgniteCheckedException { Class targetCls = target.getClass(); ClassDescriptor descr = descriptor(dep, targetCls); T2 annotatedMembers = descr.annotatedMembers(annCls); - if (descr.recursiveFields().length == 0 && annotatedMembers == null) - return false; - - if (checkedObjs == null && descr.recursiveFields().length > 0) - checkedObjs = new GridLeanIdentitySet<>(); - - if (checkedObjs != null && !checkedObjs.add(target)) - return false; - - boolean injected = false; - - for (Field field : descr.recursiveFields()) { - try { - Object obj = field.get(target); - - if (obj != null) { - assert checkedObjs != null; - - injected |= injectInternal(obj, annCls, injector, dep, depCls, checkedObjs); - } - } - catch (IllegalAccessException e) { - throw new IgniteCheckedException("Failed to inject resource [field=" + field.getName() + - ", target=" + target + ']', e); - } - } - - if (annotatedMembers != null) { - for (GridResourceField field : annotatedMembers.get1()) { - injector.inject(field, target, depCls, dep); - - injected = true; - } - - for (GridResourceMethod mtd : annotatedMembers.get2()) { - injector.inject(mtd, target, depCls, dep); - - injected = true; - } - } - - return injected; + return descr.injectInternal(target, annCls, annotatedMembers, injector, dep, depCls, checkedObjs); } /** @@ -202,36 +195,18 @@ boolean isAnnotationPresent(Object target, Class annCls, @ } /** + * Checks if annotation is presented on a field or method of the specified object. + * + * @param target Target object. + * @param annSet Annotation classes to find on fields or methods of target object. * @param dep Deployment. - * @param target Target. - * @param annClss Annotations. - * @return Filtered set of annotations that present in target. + * @return {@code true} if any annotation is presented, {@code false} if it's not. */ - @SuppressWarnings({"SuspiciousToArrayCall", "unchecked"}) - Class[] filter( - @Nullable GridDeployment dep, Object target, - Collection> annClss) { + boolean isAnnotationsPresent(@Nullable GridDeployment dep, Object target, AnnotationSet annSet) { assert target != null; - assert annClss != null && !annClss.isEmpty(); + assert annSet != null; - Class cls = target.getClass(); - - Class[] res = annCache.get(cls); - - if (res == null) { - Collection> res0 = new ArrayList<>(); - - for (Class annCls : annClss) { - if (isAnnotationPresent(target, annCls, dep)) - res0.add(annCls); - } - - res = res0.toArray(new Class[res0.size()]); - - annCache.putIfAbsent(cls, res); - } - - return res; + return descriptor(dep, target.getClass()).isAnnotated(annSet) != 0; } /** @@ -251,25 +226,37 @@ GridResourceMethod[] getMethodsWithAnnotation(@Nullable GridDeployment dep, Clas return t2 == null ? GridResourceMethod.EMPTY_ARRAY : t2.get2(); } - /** {@inheritDoc} */ + /** Print memory statistics */ public void printMemoryStats() { X.println(">>> taskMapSize: " + taskMap.size()); - X.println(">>> classDescriptorsCacheSize: " + clsDescs.size()); + + Map, ClassDescriptor> map = clsDescs.get(); + X.println(">>> classDescriptorsCacheSize: " + (map == null ? 0 : map.size())); } /** * */ - private static class ClassDescriptor { + class ClassDescriptor { /** */ private final Field[] recursiveFields; /** */ private final Map, T2> annMap; + /** + * Uses as enum-map with enum {@link AnnotationSet} member as key, + * and bitmap as a result of matching found annotations with enum set {@link ResourceAnnotation} as value. + */ + private final int[] containsAnnSets; + + /** Uses as enum-map with enum {@link ResourceAnnotation} member as a keys. */ + private final T2[] annArr; + /** * @param cls Class. */ + @SuppressWarnings("unchecked") ClassDescriptor(Class cls) { Map, T2, List>> annMap = new HashMap<>(); @@ -335,20 +322,277 @@ private static class ClassDescriptor { this.annMap.put(entry.getKey(), new T2<>(fields, mtds)); } + + T2[] annArr = null; + + if (annMap.isEmpty()) + containsAnnSets = null; + else { + int annotationsBits = 0; + + for (ResourceAnnotation ann : ResourceAnnotation.values()) { + T2 member = annotatedMembers(ann.clazz); + + if (member != null) { + if (annArr == null) + annArr = new T2[ResourceAnnotation.values().length]; + + annArr[ann.ordinal()] = member; + + annotationsBits |= 1 << ann.ordinal(); + } + } + + AnnotationSet[] annotationSets = AnnotationSet.values(); + + containsAnnSets = new int[annotationSets.length]; + + for (int i = 0; i < annotationSets.length; i++) + containsAnnSets[i] = annotationsBits & annotationSets[i].annotationsBitSet; + } + + this.annArr = annArr; } /** * @return Recursive fields. */ - public Field[] recursiveFields() { + Field[] recursiveFields() { return recursiveFields; } /** + * @param annCls Annotation class. * @return Fields. */ - @Nullable public T2 annotatedMembers(Class annCls) { + @Nullable T2 annotatedMembers(Class annCls) { return annMap.get(annCls); } + + /** + * @param set annotation set. + * @return {@code Bitmask} > 0 if any annotation is presented, otherwise return 0; + */ + int isAnnotated(AnnotationSet set) { + return recursiveFields.length > 0 ? set.annotationsBitSet : + (containsAnnSets == null ? 0 : containsAnnSets[set.ordinal()]); + } + + /** + * @param ann Annotation. + * @return {@code True} if annotation is presented. + */ + boolean isAnnotated(ResourceAnnotation ann) { + return recursiveFields.length > 0 || (annArr != null && annArr[ann.ordinal()] != null); + } + + /** + * @param target Target object. + * @param annCls Annotation class. + * @param annotatedMembers Setter annotation. + * @param injector Resource to inject. + * @param dep Deployment. + * @param depCls Deployment class. + * @param checkedObjs Set of already inspected objects to avoid indefinite recursion. + * @return {@code True} if resource was injected. + * @throws IgniteCheckedException Thrown in case of any errors during injection. + */ + boolean injectInternal(Object target, + Class annCls, + T2 annotatedMembers, + GridResourceInjector injector, + @Nullable GridDeployment dep, + @Nullable Class depCls, + @Nullable Set checkedObjs) + throws IgniteCheckedException { + if (recursiveFields.length == 0 && annotatedMembers == null) + return false; + + if (checkedObjs == null && recursiveFields.length > 0) + checkedObjs = new GridLeanIdentitySet<>(); + + if (checkedObjs != null && !checkedObjs.add(target)) + return false; + + boolean injected = false; + + for (Field field : recursiveFields) { + try { + Object obj = field.get(target); + + if (obj != null) { + assert checkedObjs != null; + + ClassDescriptor desc = descriptor(dep, obj.getClass()); + injected |= desc.injectInternal(obj, annCls, desc.annotatedMembers(annCls), + injector, dep, depCls, checkedObjs); + } + } + catch (IllegalAccessException e) { + throw new IgniteCheckedException("Failed to inject resource [field=" + field.getName() + + ", target=" + target + ']', e); + } + } + + if (annotatedMembers != null) { + for (GridResourceField field : annotatedMembers.get1()) { + injector.inject(field, target, depCls, dep); + + injected = true; + } + + for (GridResourceMethod mtd : annotatedMembers.get2()) { + injector.inject(mtd, target, depCls, dep); + + injected = true; + } + } + + return injected; + } + + /** + * @param target Target object. + * @param ann Setter annotation. + * @param injector Resource to inject. + * @param dep Deployment. + * @param depCls Deployment class. + * @return {@code True} if resource was injected. + * @throws IgniteCheckedException Thrown in case of any errors during injection. + */ + public boolean inject(Object target, + ResourceAnnotation ann, + GridResourceInjector injector, + @Nullable GridDeployment dep, + @Nullable Class depCls) + throws IgniteCheckedException { + return injectInternal(target, + ann.clazz, + annArr == null ? null : annArr[ann.ordinal()], + injector, + dep, + depCls, + null); + } + } + + /** + * + */ + enum ResourceAnnotation { + /** */ + CACHE_NAME(CacheNameResource.class), + + /** */ + SPRING_APPLICATION_CONTEXT(SpringApplicationContextResource.class), + + /** */ + SPRING(SpringResource.class), + + /** */ + IGNITE_INSTANCE(IgniteInstanceResource.class), + + /** */ + LOGGER(LoggerResource.class), + + /** */ + SERVICE(ServiceResource.class), + + /** */ + TASK_SESSION(TaskSessionResource.class), + + /** */ + LOAD_BALANCER(LoadBalancerResource.class), + + /** */ + TASK_CONTINUOUS_MAPPER(TaskContinuousMapperResource.class), + + /** */ + JOB_CONTEXT(JobContextResource.class), + + /** */ + CACHE_STORE_SESSION(CacheStoreSessionResource.class); + + /** */ + public final Class clazz; + + /** + * @param clazz annotation class. + */ + ResourceAnnotation(Class clazz) { + this.clazz = clazz; + } + } + + /** + * + */ + public enum AnnotationSet { + /** */ + GENERIC( + ResourceAnnotation.SPRING_APPLICATION_CONTEXT, + ResourceAnnotation.SPRING, + ResourceAnnotation.IGNITE_INSTANCE, + ResourceAnnotation.LOGGER, + ResourceAnnotation.SERVICE + ), + + /** */ + ENTRY_PROCESSOR( + ResourceAnnotation.CACHE_NAME, + + ResourceAnnotation.SPRING_APPLICATION_CONTEXT, + ResourceAnnotation.SPRING, + ResourceAnnotation.IGNITE_INSTANCE, + ResourceAnnotation.LOGGER, + ResourceAnnotation.SERVICE + ), + + /** */ + TASK( + ResourceAnnotation.TASK_SESSION, + ResourceAnnotation.LOAD_BALANCER, + ResourceAnnotation.TASK_CONTINUOUS_MAPPER, + + ResourceAnnotation.SPRING_APPLICATION_CONTEXT, + ResourceAnnotation.SPRING, + ResourceAnnotation.IGNITE_INSTANCE, + ResourceAnnotation.LOGGER, + ResourceAnnotation.SERVICE + ), + + /** */ + JOB( + ResourceAnnotation.TASK_SESSION, + ResourceAnnotation.JOB_CONTEXT, + + ResourceAnnotation.SPRING_APPLICATION_CONTEXT, + ResourceAnnotation.SPRING, + ResourceAnnotation.IGNITE_INSTANCE, + ResourceAnnotation.LOGGER, + ResourceAnnotation.SERVICE + ); + + /** Resource annotations bits for fast checks. */ + public final int annotationsBitSet; + + /** Holds annotations in order */ + public final ResourceAnnotation[] annotations; + + /** + * @param annotations ResourceAnnotations. + */ + AnnotationSet(ResourceAnnotation... annotations) { + assert annotations.length < 32 : annotations.length; + + this.annotations = annotations; + + int mask = 0; + + for (ResourceAnnotation ann : annotations) + mask |= 1 << ann.ordinal(); + + annotationsBitSet = mask; + } } -} \ No newline at end of file +} 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 afe0ef1f5c389..84d07b64e50e5 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 @@ -20,12 +20,11 @@ import java.lang.annotation.Annotation; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.util.Arrays; import java.util.Collection; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.store.CacheStoreSession; import org.apache.ignite.compute.ComputeJob; +import org.apache.ignite.compute.ComputeJobContext; import org.apache.ignite.compute.ComputeLoadBalancer; import org.apache.ignite.compute.ComputeTask; import org.apache.ignite.compute.ComputeTaskContinuousMapper; @@ -34,22 +33,10 @@ import org.apache.ignite.internal.GridJobContextImpl; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.GridTaskSessionImpl; -import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.managers.deployment.GridDeployment; import org.apache.ignite.internal.processors.GridProcessorAdapter; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.lifecycle.LifecycleBean; -import org.apache.ignite.resources.CacheNameResource; -import org.apache.ignite.resources.CacheStoreSessionResource; -import org.apache.ignite.resources.IgniteInstanceResource; -import org.apache.ignite.resources.JobContextResource; -import org.apache.ignite.resources.LoadBalancerResource; -import org.apache.ignite.resources.LoggerResource; -import org.apache.ignite.resources.ServiceResource; -import org.apache.ignite.resources.SpringApplicationContextResource; -import org.apache.ignite.resources.SpringResource; -import org.apache.ignite.resources.TaskContinuousMapperResource; -import org.apache.ignite.resources.TaskSessionResource; import org.apache.ignite.services.Service; import org.apache.ignite.spi.IgniteSpi; import org.jetbrains.annotations.Nullable; @@ -58,42 +45,6 @@ * Processor for all Ignite and task/job resources. */ public class GridResourceProcessor extends GridProcessorAdapter { - /** */ - private static final Collection> JOB_INJECTIONS = Arrays.asList( - TaskSessionResource.class, - JobContextResource.class, - IgniteInstanceResource.class, - SpringApplicationContextResource.class, - SpringResource.class, - LoggerResource.class, - ServiceResource.class); - - /** */ - private static final Collection> TASK_INJECTIONS = Arrays.asList( - TaskSessionResource.class, - LoadBalancerResource.class, - TaskContinuousMapperResource.class, - IgniteInstanceResource.class, - SpringApplicationContextResource.class, - SpringResource.class, - LoggerResource.class, - ServiceResource.class); - - /** Grid instance injector. */ - private GridResourceBasicInjector gridInjector; - - /** Spring application context injector. */ - private GridResourceInjector springCtxInjector; - - /** Logger injector. */ - private GridResourceBasicInjector logInjector; - - /** Services injector. */ - private GridResourceBasicInjector> srvcInjector; - - /** Spring bean resources injector. */ - private GridResourceInjector springBeanInjector; - /** Cleaning injector. */ private final GridResourceInjector nullInjector = new GridResourceBasicInjector<>(null); @@ -103,6 +54,9 @@ public class GridResourceProcessor extends GridProcessorAdapter { /** */ private final GridResourceIoc ioc = new GridResourceIoc(); + /** */ + private final GridResourceInjector[] injectorByAnnotation; + /** * Creates resources processor. * @@ -111,9 +65,14 @@ public class GridResourceProcessor extends GridProcessorAdapter { public GridResourceProcessor(GridKernalContext ctx) { super(ctx); - gridInjector = new GridResourceBasicInjector<>(ctx.grid()); - logInjector = new GridResourceLoggerInjector(ctx.config().getGridLogger()); - srvcInjector = new GridResourceServiceInjector(ctx.grid()); + injectorByAnnotation = new GridResourceInjector[GridResourceIoc.ResourceAnnotation.values().length]; + + injectorByAnnotation[GridResourceIoc.ResourceAnnotation.SERVICE.ordinal()] = + new GridResourceServiceInjector(ctx.grid()); + injectorByAnnotation[GridResourceIoc.ResourceAnnotation.LOGGER.ordinal()] = + new GridResourceLoggerInjector(ctx.config().getGridLogger()); + injectorByAnnotation[GridResourceIoc.ResourceAnnotation.IGNITE_INSTANCE.ordinal()] = + new GridResourceBasicInjector<>(ctx.grid()); } /** {@inheritDoc} */ @@ -138,8 +97,12 @@ public GridResourceProcessor(GridKernalContext ctx) { public void setSpringContext(@Nullable GridSpringResourceContext rsrcCtx) { this.rsrcCtx = rsrcCtx; - springCtxInjector = rsrcCtx != null ? rsrcCtx.springContextInjector() : nullInjector; - springBeanInjector = rsrcCtx != null ? rsrcCtx.springBeanInjector() : nullInjector; + GridResourceInjector springCtxInjector = rsrcCtx != null ? rsrcCtx.springContextInjector() : nullInjector; + GridResourceInjector springBeanInjector = rsrcCtx != null ? rsrcCtx.springBeanInjector() : nullInjector; + + injectorByAnnotation[GridResourceIoc.ResourceAnnotation.SPRING.ordinal()] = springBeanInjector; + injectorByAnnotation[GridResourceIoc.ResourceAnnotation.SPRING_APPLICATION_CONTEXT.ordinal()] = + springCtxInjector; } /** @@ -187,17 +150,15 @@ public void invokeAnnotated(GridDeployment dep, Object target, Class depCls, Object target) throws IgniteCheckedException { + assert target != null; + if (log.isDebugEnabled()) log.debug("Injecting resources: " + target); // Unwrap Proxy object. target = unwrapTarget(target); - ioc.inject(target, IgniteInstanceResource.class, gridInjector, dep, depCls); - ioc.inject(target, SpringApplicationContextResource.class, springCtxInjector, dep, depCls); - ioc.inject(target, SpringResource.class, springBeanInjector, dep, depCls); - ioc.inject(target, LoggerResource.class, logInjector, dep, depCls); - ioc.inject(target, ServiceResource.class, srvcInjector, dep, depCls); + inject(target, GridResourceIoc.AnnotationSet.GENERIC, dep, depCls); } /** @@ -216,7 +177,7 @@ public void injectCacheName(Object obj, String cacheName) throws IgniteCheckedEx // Unwrap Proxy object. obj = unwrapTarget(obj); - ioc.inject(obj, CacheNameResource.class, new GridResourceBasicInjector<>(cacheName), null, null); + inject(obj, GridResourceIoc.ResourceAnnotation.CACHE_NAME, null, null, cacheName); } /** @@ -236,7 +197,7 @@ public boolean injectStoreSession(Object obj, CacheStoreSession ses) throws Igni // Unwrap Proxy object. obj = unwrapTarget(obj); - return ioc.inject(obj, CacheStoreSessionResource.class, new GridResourceBasicInjector<>(ses), null, null); + return inject(obj, GridResourceIoc.ResourceAnnotation.CACHE_STORE_SESSION, null, null, ses); } /** @@ -244,6 +205,17 @@ public boolean injectStoreSession(Object obj, CacheStoreSession ses) throws Igni * @throws IgniteCheckedException If failed to inject. */ public void injectGeneric(Object obj) throws IgniteCheckedException { + inject(obj, GridResourceIoc.AnnotationSet.GENERIC); + } + + /** + * @param obj Object to inject. + * @param annSet Supported annotations. + * @param params Parameters. + * @throws IgniteCheckedException If failed to inject. + */ + public void inject(Object obj, GridResourceIoc.AnnotationSet annSet, Object... params) + throws IgniteCheckedException { assert obj != null; if (log.isDebugEnabled()) @@ -252,33 +224,126 @@ public void injectGeneric(Object obj) throws IgniteCheckedException { // Unwrap Proxy object. obj = unwrapTarget(obj); - // No deployment for lifecycle beans. - ioc.inject(obj, SpringApplicationContextResource.class, springCtxInjector, null, null); - ioc.inject(obj, SpringResource.class, springBeanInjector, null, null); - ioc.inject(obj, IgniteInstanceResource.class, gridInjector, null, null); - ioc.inject(obj, LoggerResource.class, logInjector, null, null); - ioc.inject(obj, ServiceResource.class, srvcInjector, null, null); + inject(obj, annSet, null, null, params); + } + + /** + * @param obj Object to inject. + * @param annSet Supported annotations. + * @param dep Deployment. + * @param depCls Deployment class. + * @param params Parameters. + * @throws IgniteCheckedException If failed to inject. + */ + private void inject(Object obj, + GridResourceIoc.AnnotationSet annSet, + @Nullable GridDeployment dep, + @Nullable Class depCls, + Object... params) + throws IgniteCheckedException { + GridResourceIoc.ClassDescriptor clsDesc = ioc.descriptor(null, obj.getClass()); + + assert clsDesc != null; + + if (clsDesc.isAnnotated(annSet) == 0) + return; + + int i = 0; + for (GridResourceIoc.ResourceAnnotation ann : annSet.annotations) { + if (clsDesc.isAnnotated(ann)) { + final GridResourceInjector injector = injectorByAnnotation(ann, i < params.length ? params[i] : null); + + if (injector != null) + clsDesc.inject(obj, ann, injector, dep, depCls); + } + + i++; + } } /** * @param obj Object. + * @param annSet Supported annotations. * @throws IgniteCheckedException If failed. */ - public void cleanupGeneric(Object obj) throws IgniteCheckedException { - if (obj != null) { - if (log.isDebugEnabled()) - log.debug("Cleaning up resources: " + obj); - - // Unwrap Proxy object. - obj = unwrapTarget(obj); - - // Caching key is null for the life-cycle beans. - ioc.inject(obj, LoggerResource.class, nullInjector, null, null); - ioc.inject(obj, ServiceResource.class, nullInjector, null, null); - ioc.inject(obj, SpringApplicationContextResource.class, nullInjector, null, null); - ioc.inject(obj, SpringResource.class, nullInjector, null, null); - ioc.inject(obj, IgniteInstanceResource.class, nullInjector, null, null); + private void cleanup(Object obj, GridResourceIoc.AnnotationSet annSet) + throws IgniteCheckedException { + assert obj != null; + + if (log.isDebugEnabled()) + log.debug("Cleaning up resources: " + obj); + + // Unwrap Proxy object. + obj = unwrapTarget(obj); + + GridResourceIoc.ClassDescriptor clsDesc = ioc.descriptor(null, obj.getClass()); + + assert clsDesc != null; + + if (clsDesc.isAnnotated(annSet) == 0) + return; + + for (GridResourceIoc.ResourceAnnotation ann : annSet.annotations) + clsDesc.inject(obj, ann, nullInjector, null, null); + } + + /** + * @param ann Annotation. + * @param param Injector parameter. + * @return Injector. + */ + private GridResourceInjector injectorByAnnotation(GridResourceIoc.ResourceAnnotation ann, Object param) { + final GridResourceInjector res; + + switch (ann) { + case CACHE_NAME: + case TASK_SESSION: + case LOAD_BALANCER: + case TASK_CONTINUOUS_MAPPER: + case CACHE_STORE_SESSION: + res = new GridResourceBasicInjector<>(param); + break; + + case JOB_CONTEXT: + res = new GridResourceJobContextInjector((ComputeJobContext)param); + break; + + default: + res = injectorByAnnotation[ann.ordinal()]; + break; } + + return res; + } + + /** + * @param obj Object to inject. + * @throws IgniteCheckedException If failed to inject. + */ + private boolean inject(Object obj, GridResourceIoc.ResourceAnnotation ann, @Nullable GridDeployment dep, + @Nullable Class depCls, Object param) + throws IgniteCheckedException { + GridResourceIoc.ClassDescriptor clsDesc = ioc.descriptor(null, obj.getClass()); + + assert clsDesc != null; + + if (clsDesc.isAnnotated(ann)) { + GridResourceInjector injector = injectorByAnnotation(ann, param); + + if (injector != null) + return clsDesc.inject(obj, ann, injector, dep, depCls); + } + + return false; + } + + /** + * @param obj Object. + * @throws IgniteCheckedException If failed. + */ + public void cleanupGeneric(Object obj) throws IgniteCheckedException { + if (obj != null) + cleanup(obj, GridResourceIoc.AnnotationSet.GENERIC); } /** @@ -321,30 +386,8 @@ public void inject(GridDeployment dep, Class taskCls, ComputeJob job, Compute */ private void injectToJob(GridDeployment dep, Class taskCls, Object job, ComputeTaskSession ses, GridJobContextImpl jobCtx) throws IgniteCheckedException { - Class[] filtered = ioc.filter(dep, job, JOB_INJECTIONS); - - if (filtered.length > 0) { - for (Class annCls : filtered) { - if (annCls == TaskSessionResource.class) - injectBasicResource(job, TaskSessionResource.class, ses, dep, taskCls); - else if (annCls == JobContextResource.class) - ioc.inject(job, JobContextResource.class, new GridResourceJobContextInjector(jobCtx), - dep, taskCls); - else if (annCls == IgniteInstanceResource.class) - ioc.inject(job, IgniteInstanceResource.class, gridInjector, dep, taskCls); - else if (annCls == SpringApplicationContextResource.class) - ioc.inject(job, SpringApplicationContextResource.class, springCtxInjector, dep, taskCls); - else if (annCls == SpringResource.class) - ioc.inject(job, SpringResource.class, springBeanInjector, dep, taskCls); - else if (annCls == LoggerResource.class) - ioc.inject(job, LoggerResource.class, logInjector, dep, taskCls); - else { - assert annCls == ServiceResource.class; - - ioc.inject(job, ServiceResource.class, srvcInjector, dep, taskCls); - } - } - } + + inject(job, GridResourceIoc.AnnotationSet.JOB, dep, taskCls, ses, jobCtx); } /** @@ -365,34 +408,7 @@ public void inject(GridDeployment dep, ComputeTask task, GridTaskSessionIm // Unwrap Proxy object. Object obj = unwrapTarget(task); - Class[] filtered = ioc.filter(dep, obj, TASK_INJECTIONS); - - if (filtered.length == 0) - return; - - Class taskCls = obj.getClass(); - - for (Class annCls : filtered) { - if (annCls == TaskSessionResource.class) - injectBasicResource(obj, TaskSessionResource.class, ses, dep, taskCls); - else if (annCls == LoadBalancerResource.class) - injectBasicResource(obj, LoadBalancerResource.class, balancer, dep, taskCls); - else if (annCls == TaskContinuousMapperResource.class) - injectBasicResource(obj, TaskContinuousMapperResource.class, mapper, dep, taskCls); - else if (annCls == IgniteInstanceResource.class) - ioc.inject(obj, IgniteInstanceResource.class, gridInjector, dep, taskCls); - else if (annCls == SpringApplicationContextResource.class) - ioc.inject(obj, SpringApplicationContextResource.class, springCtxInjector, dep, taskCls); - else if (annCls == SpringResource.class) - ioc.inject(obj, SpringResource.class, springBeanInjector, dep, taskCls); - else if (annCls == LoggerResource.class) - ioc.inject(obj, LoggerResource.class, logInjector, dep, taskCls); - else { - assert annCls == ServiceResource.class; - - ioc.inject(obj, ServiceResource.class, srvcInjector, dep, taskCls); - } - } + inject(obj, GridResourceIoc.AnnotationSet.TASK, dep, null, ses, balancer, mapper); } /** @@ -407,6 +423,18 @@ public boolean isAnnotationPresent(GridDeployment dep, Object target, Class annCls, Object rsrc, - GridDeployment dep, Class depCls) throws IgniteCheckedException { - // Safety. - assert !(rsrc instanceof GridResourceInjector) : "Invalid injection."; - - // Basic injection don't cache anything. Use null as a key. - ioc.inject(target, annCls, new GridResourceBasicInjector<>(rsrc), dep, depCls); + cleanupGeneric(svc); } /** @@ -602,4 +544,4 @@ private Object unwrapTarget(Object target) throws IgniteCheckedException { ioc.printMemoryStats(); } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java index 32d46e22b0cec..3f4d812efaaed 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java @@ -17,8 +17,6 @@ package org.apache.ignite.internal.processors.cache; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Sets; import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; @@ -34,7 +32,10 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Lock; import javax.cache.Cache; @@ -46,10 +47,13 @@ import javax.cache.processor.EntryProcessorException; import javax.cache.processor.EntryProcessorResult; import javax.cache.processor.MutableEntry; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; import junit.framework.AssertionFailedError; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteEvents; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteTransactions; @@ -61,12 +65,15 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.CacheEvent; import org.apache.ignite.events.Event; +import org.apache.ignite.events.EventType; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.IgnitionEx; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; import org.apache.ignite.internal.processors.resource.GridSpringResourceContext; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.lang.GridAbsPredicateX; import org.apache.ignite.internal.util.lang.IgnitePair; @@ -76,10 +83,16 @@ import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.resources.CacheNameResource; +import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.resources.ServiceResource; +import org.apache.ignite.services.Service; +import org.apache.ignite.services.ServiceContext; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.swapspace.inmemory.GridTestSwapSpaceSpi; @@ -124,6 +137,9 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract /** Test timeout */ private static final long TEST_TIMEOUT = 60 * 1000; + /** Service name. */ + private static final String SERVICE_NAME1 = "testService1"; + /** */ public static final CacheEntryProcessor ERR_PROCESSOR = new CacheEntryProcessor() { @@ -202,6 +218,21 @@ protected CacheMemoryMode memoryMode() { if (memoryMode() == OFFHEAP_TIERED || memoryMode() == OFFHEAP_VALUES) cfg.setSwapSpaceSpi(new GridTestSwapSpaceSpi()); + int[] evtTypes = cfg.getIncludeEventTypes(); + + if (evtTypes == null || evtTypes.length == 0) + cfg.setIncludeEventTypes(EventType.EVT_CACHE_OBJECT_READ); + else { + for (int evtType : evtTypes) { + if (evtType == EventType.EVT_CACHE_OBJECT_READ) + return cfg; + } + + int[] updatedEvtTypes = Arrays.copyOf(evtTypes, evtTypes.length + 1); + + updatedEvtTypes[updatedEvtTypes.length - 1] = EventType.EVT_CACHE_OBJECT_READ; + } + return cfg; } @@ -261,6 +292,18 @@ protected CacheMemoryMode memoryMode() { cacheCfgMap = null; } + // We won't deploy service unless non-client node is configured. + for (int i = 0; i < gridCount(); i++) { + Boolean clientMode = grid(i).configuration().isClientMode(); + + if (clientMode) + continue; + + grid(0).services(grid(0).cluster()).deployNodeSingleton(SERVICE_NAME1, new DummyServiceImpl()); + + break; + } + for (int i = 0; i < gridCount(); i++) info("Grid " + i + ": " + grid(i).localNode().id()); } @@ -619,9 +662,9 @@ public void testGetEntry() throws Exception { cache.put("key1", 1); cache.put("key2", 2); - CacheEntry key1e = cache.getEntry("key1"); - CacheEntry key2e = cache.getEntry("key2"); - CacheEntry wrongKeye = cache.getEntry("wrongKey"); + CacheEntry key1e = cache.getEntry("key1"); + CacheEntry key2e = cache.getEntry("key2"); + CacheEntry wrongKeye = cache.getEntry("wrongKey"); assert key1e.getValue() == 1; assert key1e.getKey().equals("key1"); @@ -781,7 +824,7 @@ public void testGetEntries() throws Exception { boolean b1 = false; boolean b2 = false; - for (CacheEntry e: c1){ + for (CacheEntry e : c1) { if (e.getKey().equals("key1") && e.getValue().equals(1)) b1 = true; @@ -800,7 +843,7 @@ public void testGetEntries() throws Exception { b1 = false; b2 = false; - for (CacheEntry e: c2){ + for (CacheEntry e : c2) { if (e.getKey().equals("key1") && e.getValue().equals(1)) b1 = true; @@ -1481,8 +1524,7 @@ public void testTransformReturnValuePutInTx() throws Exception { private void checkTransformReturnValue(boolean put, TransactionConcurrency concurrency, TransactionIsolation isolation) - throws Exception - { + throws Exception { IgniteCache cache = jcache(); if (!put) @@ -1790,7 +1832,7 @@ public void testNullInTx() throws Exception { cache.put(key, 1); - assertEquals(1, (int) cache.get(key)); + assertEquals(1, (int)cache.get(key)); GridTestUtils.assertThrows(log, new Callable() { @Override public Void call() throws Exception { @@ -1808,7 +1850,7 @@ public void testNullInTx() throws Exception { } }, NullPointerException.class, null); - assertEquals(1, (int) cache.get(key)); + assertEquals(1, (int)cache.get(key)); cache.put(key, 2); @@ -1839,7 +1881,7 @@ public void testNullInTx() throws Exception { assertNull(cache.get("k1")); assertNull(cache.get("k2")); - assertEquals(2, (int) cache.get(key)); + assertEquals(2, (int)cache.get(key)); cache.put(key, 3); @@ -1890,7 +1932,7 @@ public void testPutAllWithNulls() throws Exception { cache.putAll(m); - assertEquals(3, (int) cache.get("key3")); + assertEquals(3, (int)cache.get("key3")); assertEquals(4, (int)cache.get("key4")); } @@ -2215,7 +2257,7 @@ public void testPutxIfAbsentAsyncNoTx() throws Exception { } /** - * @param inTx In tx flag. + * @param inTx In tx flag. * @throws Exception If failed. */ private void checkPutxIfAbsentAsync(boolean inTx) throws Exception { @@ -2857,7 +2899,7 @@ private void globalRemoveAll(boolean async) throws Exception { /** * @return Count of entries to be removed in removeAll() test. */ - protected long hugeRemoveAllEntryCount(){ + protected long hugeRemoveAllEntryCount() { return 1000L; } @@ -3627,7 +3669,7 @@ private void checkTtl(boolean inTx, boolean oldEntry) throws Exception { assertNotNull(curEntryTtl.get1()); assertNotNull(curEntryTtl.get2()); - assertEquals(ttl, (long) curEntryTtl.get1()); + assertEquals(ttl, (long)curEntryTtl.get1()); assertTrue(curEntryTtl.get2() > startTime); expireTimes[i] = curEntryTtl.get2(); @@ -3656,7 +3698,7 @@ private void checkTtl(boolean inTx, boolean oldEntry) throws Exception { assertNotNull(curEntryTtl.get1()); assertNotNull(curEntryTtl.get2()); - assertEquals(ttl, (long) curEntryTtl.get1()); + assertEquals(ttl, (long)curEntryTtl.get1()); assertTrue(curEntryTtl.get2() > startTime); expireTimes[i] = curEntryTtl.get2(); @@ -3685,7 +3727,7 @@ private void checkTtl(boolean inTx, boolean oldEntry) throws Exception { assertNotNull(curEntryTtl.get1()); assertNotNull(curEntryTtl.get2()); - assertEquals(ttl, (long) curEntryTtl.get1()); + assertEquals(ttl, (long)curEntryTtl.get1()); assertTrue(curEntryTtl.get2() > startTime); expireTimes[i] = curEntryTtl.get2(); @@ -3897,7 +3939,7 @@ public void testUnswap() throws Exception { cache.localPromote(Collections.singleton(k2)); - assertEquals((Integer) 2, cache.localPeek(k2, ONHEAP_PEEK_MODES)); + assertEquals((Integer)2, cache.localPeek(k2, ONHEAP_PEEK_MODES)); cnt++; } @@ -5021,7 +5063,6 @@ public void testWithSkipStoreTx() throws Exception { * @param keys Keys list. * @param txConcurrency Concurrency mode. * @param txIsolation Isolation mode. - * * @throws Exception If failed. */ private void checkSkipStoreWithTransaction(IgniteCache cache, @@ -5030,8 +5071,7 @@ private void checkSkipStoreWithTransaction(IgniteCache cache, List keys, TransactionConcurrency txConcurrency, TransactionIsolation txIsolation) - throws Exception - { + throws Exception { info("Test tx skip store [concurrency=" + txConcurrency + ", isolation=" + txIsolation + ']'); cache.removeAll(data.keySet()); @@ -5043,10 +5083,10 @@ private void checkSkipStoreWithTransaction(IgniteCache cache, // Several put check. try (Transaction tx = txs.txStart(txConcurrency, txIsolation)) { - for (String key: keys) + for (String key : keys) cacheSkipStore.put(key, val); - for (String key: keys) { + for (String key : keys) { assertEquals(val, cacheSkipStore.get(key)); assertEquals(val, cache.get(key)); assertFalse(storeStgy.isInStore(key)); @@ -5055,7 +5095,7 @@ private void checkSkipStoreWithTransaction(IgniteCache cache, tx.commit(); } - for (String key: keys) { + for (String key : keys) { assertEquals(val, cacheSkipStore.get(key)); assertEquals(val, cache.get(key)); assertFalse(storeStgy.isInStore(key)); @@ -5070,7 +5110,7 @@ private void checkSkipStoreWithTransaction(IgniteCache cache, tx.commit(); } - for (String key: keys) { + for (String key : keys) { val = data.get(key); assertEquals(val, cacheSkipStore.get(key)); @@ -5086,7 +5126,7 @@ private void checkSkipStoreWithTransaction(IgniteCache cache, tx.commit(); } - for (String key: keys) { + for (String key : keys) { assertNull(cacheSkipStore.get(key)); assertNotNull(cache.get(key)); assertTrue(storeStgy.isInStore(key)); @@ -5100,7 +5140,7 @@ private void checkSkipStoreWithTransaction(IgniteCache cache, try (Transaction tx = txs.txStart(txConcurrency, txIsolation)) { cache.putAll(data); - for (String key: keys) { + for (String key : keys) { assertNotNull(cacheSkipStore.get(key)); assertNotNull(cache.get(key)); assertFalse(storeStgy.isInStore(key)); @@ -5108,7 +5148,7 @@ private void checkSkipStoreWithTransaction(IgniteCache cache, cache.removeAll(data.keySet()); - for (String key: keys) { + for (String key : keys) { assertNull(cacheSkipStore.get(key)); assertNull(cache.get(key)); assertFalse(storeStgy.isInStore(key)); @@ -5135,7 +5175,7 @@ private void checkSkipStoreWithTransaction(IgniteCache cache, cache.putAll(subMap); - for (String key: keys) { + for (String key : keys) { assertNotNull(cacheSkipStore.get(key)); assertNotNull(cache.get(key)); assertFalse(storeStgy.isInStore(key)); @@ -5162,7 +5202,7 @@ private void checkSkipStoreWithTransaction(IgniteCache cache, cache.removeAll(data.keySet()); - for (String key: keys) { + for (String key : keys) { assertNull(cacheSkipStore.get(key)); assertNull(cache.get(key)); assertFalse(storeStgy.isInStore(key)); @@ -5257,7 +5297,6 @@ private void checkSkipStoreWithTransaction(IgniteCache cache, /** * @param cache Cache instance. * @param cacheSkipStore Cache skip store projection. - * * @throws Exception If failed. */ private void checkEmpty(IgniteCache cache, IgniteCache cacheSkipStore) @@ -5425,6 +5464,155 @@ public void testLockInsideTransaction() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testTransformResourceInjection() throws Exception { + IgniteCache cache = jcache(); + Ignite ignite = ignite(0); + + doTransformResourceInjection(ignite, cache); + doTransformResourceInjection(ignite, cache.withAsync()); + + if (txEnabled()) { + doTransformResourceInjectionInTx(ignite, cache); + doTransformResourceInjectionInTx(ignite, cache.withAsync()); + } + } + + /** + * @param ignite Node. + * @param cache Cache. + * @throws Exception If failed. + */ + private void doTransformResourceInjectionInTx(Ignite ignite, IgniteCache cache) throws Exception { + for (TransactionConcurrency concurrency : TransactionConcurrency.values()) { + for (TransactionIsolation isolation : TransactionIsolation.values()) { + IgniteTransactions txs = ignite.transactions(); + + try (Transaction tx = txs.txStart(concurrency, isolation)) { + doTransformResourceInjection(ignite, cache); + + tx.commit(); + } + } + } + } + + /** + * @param ignite Node. + * @param cache Cache. + * @throws Exception If failed. + */ + private void doTransformResourceInjection(Ignite ignite, IgniteCache cache) throws Exception { + final Collection required = Arrays.asList(ResourceType.IGNITE_INSTANCE, + ResourceType.CACHE_NAME, + ResourceType.LOGGER, + ResourceType.SERVICE); + + final CacheEventListener lsnr = new CacheEventListener(); + + IgniteEvents evts = ignite.events(ignite.cluster()); + + UUID opId = evts.remoteListen(lsnr, null, EventType.EVT_CACHE_OBJECT_READ); + + try { + checkResourceInjectionOnInvoke(cache, required); + + checkResourceInjectionOnInvokeAll(cache, required); + + checkResourceInjectionOnInvokeAllMap(cache, required); + } + finally { + evts.stopRemoteListen(opId); + } + } + + /** + * Tests invokeAll method for map of pairs (key, entryProcessor). + * + * @param cache Cache. + * @param required Expected injected resources. + */ + private void checkResourceInjectionOnInvokeAllMap(IgniteCache cache, + Collection required) { + Map> results; + + Map> map = new HashMap<>(); + + map.put(UUID.randomUUID().toString(), new ResourceInjectionEntryProcessor()); + map.put(UUID.randomUUID().toString(), new ResourceInjectionEntryProcessor()); + map.put(UUID.randomUUID().toString(), new ResourceInjectionEntryProcessor()); + map.put(UUID.randomUUID().toString(), new ResourceInjectionEntryProcessor()); + + results = cache.invokeAll(map); + + if (cache.isAsync()) + results = cache.>>future().get(); + + assertEquals(map.size(), results.size()); + + for (EntryProcessorResult res : results.values()) { + Collection notInjected = ResourceInfoSet.valueOf(res.get()).notInjected(required); + + if (!notInjected.isEmpty()) + fail("Can't inject resource(s): " + Arrays.toString(notInjected.toArray())); + } + } + + /** + * Tests invokeAll method for set of keys. + * + * @param cache Cache. + * @param required Expected injected resources. + */ + private void checkResourceInjectionOnInvokeAll(IgniteCache cache, + Collection required) { + Set keys = new HashSet<>(Arrays.asList(UUID.randomUUID().toString(), + UUID.randomUUID().toString(), + UUID.randomUUID().toString(), + UUID.randomUUID().toString())); + + Map> results = cache.invokeAll(keys, + new ResourceInjectionEntryProcessor()); + + if (cache.isAsync()) + results = cache.>>future().get(); + + assertEquals(keys.size(), results.size()); + + for (EntryProcessorResult res : results.values()) { + Collection notInjected1 = ResourceInfoSet.valueOf(res.get()).notInjected(required); + + if (!notInjected1.isEmpty()) + fail("Can't inject resource(s): " + Arrays.toString(notInjected1.toArray())); + } + } + + /** + * Tests invoke for single key. + * + * @param cache Cache. + * @param required Expected injected resources. + */ + private void checkResourceInjectionOnInvoke(IgniteCache cache, + Collection required) { + + String key = UUID.randomUUID().toString(); + + Integer flags = cache.invoke(key, new GridCacheAbstractFullApiSelfTest.ResourceInjectionEntryProcessor()); + + if (cache.isAsync()) + flags = cache.future().get(); + + assertTrue("Processor result is null", flags != null); + + Collection notInjected = ResourceInfoSet.valueOf(flags).notInjected(required); + + if (!notInjected.isEmpty()) + fail("Can't inject resource(s): " + Arrays.toString(notInjected.toArray())); + } + /** * Sets given value, returns old value. */ @@ -5440,7 +5628,8 @@ public static final class SetValueProcessor implements EntryProcessor entry, Object... arguments) throws EntryProcessorException { + @Override public Integer process(MutableEntry entry, + Object... arguments) throws EntryProcessorException { Integer val = entry.getValue(); entry.setValue(newVal); @@ -5495,6 +5684,86 @@ private static class IncrementEntryProcessor implements EntryProcessor { + /** */ + protected transient Ignite ignite; + + /** */ + protected transient String cacheName; + + /** */ + protected transient IgniteLogger log; + + /** */ + protected transient DummyService svc; + + /** + * @param ignite Ignite. + */ + @IgniteInstanceResource + public void setIgnite(Ignite ignite) { + assert ignite != null; + + checkSet(); + + infoSet.set(ResourceType.IGNITE_INSTANCE, true); + + this.ignite = ignite; + } + + /** + * @param cacheName Cache name. + */ + @CacheNameResource + public void setCacheName(String cacheName) { + checkSet(); + + infoSet.set(ResourceType.CACHE_NAME, true); + + this.cacheName = cacheName; + } + + /** + * @param log Logger. + */ + @LoggerResource + public void setLoggerResource(IgniteLogger log) { + assert log != null; + + checkSet(); + + infoSet.set(ResourceType.LOGGER, true); + + this.log = log; + } + + /** + * @param svc Service. + */ + @ServiceResource(serviceName = SERVICE_NAME1) + public void setDummyService(DummyService svc) { + assert svc != null; + + checkSet(); + + infoSet.set(ResourceType.SERVICE, true); + + this.svc = svc; + } + + /** {@inheritDoc} */ + @Override public Integer process(MutableEntry e, Object... args) { + Integer oldVal = e.getValue(); + + e.setValue(ThreadLocalRandom.current().nextInt() + (oldVal == null ? 0 : oldVal)); + + return super.process(e, args); + } + } + /** * */ @@ -5674,6 +5943,7 @@ private static class RemoveAndReturnNullEntryProcessor implements * */ private static class SwapEvtsLocalListener implements IgnitePredicate { + /** */ @LoggerResource private IgniteLogger log; @@ -5711,13 +5981,21 @@ public SwapEvtsLocalListener(AtomicInteger swapEvts, AtomicInteger unswapEvts) { } } + /** + * + */ private static class CheckEntriesDeletedTask extends TestIgniteIdxRunnable { + /** */ private final int cnt; + /** + * @param cnt Keys count. + */ public CheckEntriesDeletedTask(int cnt) { this.cnt = cnt; } + /** {@inheritDoc} */ @Override public void run(int idx) throws Exception { for (int i = 0; i < cnt; i++) { String key = String.valueOf(i); @@ -5816,4 +6094,64 @@ public int value() { return val; } } + + /** + * Dummy Service. + */ + public interface DummyService { + /** + * + */ + public void noop(); + } + + /** + * No-op test service. + */ + public static class DummyServiceImpl implements DummyService, Service { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public void noop() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void cancel(ServiceContext ctx) { + System.out.println("Cancelling service: " + ctx.name()); + } + + /** {@inheritDoc} */ + @Override public void init(ServiceContext ctx) throws Exception { + System.out.println("Initializing service: " + ctx.name()); + } + + /** {@inheritDoc} */ + @Override public void execute(ServiceContext ctx) { + System.out.println("Executing service: " + ctx.name()); + } + } + + /** + * + */ + public static class CacheEventListener implements IgniteBiPredicate, IgnitePredicate { + /** */ + public final LinkedBlockingQueue evts = new LinkedBlockingQueue<>(); + + /** {@inheritDoc} */ + @Override public boolean apply(UUID uuid, CacheEvent evt) { + evts.add(evt); + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean apply(CacheEvent evt) { + evts.add(evt); + + return true; + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java index d58e560fed724..af316359d9abb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java @@ -17,9 +17,15 @@ package org.apache.ignite.internal.processors.cache; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.concurrent.atomic.AtomicInteger; import javax.cache.Cache; import javax.cache.configuration.Factory; +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.MutableEntry; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; @@ -80,6 +86,8 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + int cnt = gridCount(); assert cnt >= 1 : "At least one grid must be started"; @@ -188,7 +196,8 @@ void initStoreStrategy() throws IgniteCheckedException { assert jcache().unwrap(Ignite.class).transactions().tx() == null; assertEquals("Cache is not empty", 0, jcache().localSize(CachePeekMode.ALL)); - storeStgy.resetStore(); + if (storeStgy != null) + storeStgy.resetStore(); } /** {@inheritDoc} */ @@ -410,20 +419,20 @@ protected boolean containsKey(IgniteCache cache, Object key) throws Exception { */ protected static IgnitePredicate> entryKeyFilter = new P1>() { - @Override public boolean apply(Cache.Entry entry) { - return entry.getKey().contains("key"); - } - }; + @Override public boolean apply(Cache.Entry entry) { + return entry.getKey().contains("key"); + } + }; /** * Filters cache entry projections leaving only ones with keys not containing 'key'. */ protected static IgnitePredicate> entryKeyFilterInv = new P1>() { - @Override public boolean apply(Cache.Entry entry) { - return !entry.getKey().contains("key"); - } - }; + @Override public boolean apply(Cache.Entry entry) { + return !entry.getKey().contains("key"); + } + }; /** * Filters cache entry projections leaving only ones with values less than 50. @@ -528,4 +537,117 @@ public SumReducer() { } } + /** */ + protected enum ResourceType { + /** */ + IGNITE_INSTANCE, + + /** */ + CACHE_NAME, + + /** */ + SPRING_APPLICATION_CONTEXT, + + /** */ + LOGGER, + + /** */ + SERVICE, + + /** */ + SPRING_BEAN, + + } + + /** + * + */ + protected static class ResourceInfoSet { + /** */ + int val; + + /** */ + public ResourceInfoSet() { + this(0); + } + + /** */ + public ResourceInfoSet(int val) { + this.val = val; + } + + /** + * @param val Value. + */ + public static ResourceInfoSet valueOf(int val) { + return new ResourceInfoSet(val); + } + + /** */ + public int getValue() { + return val; + } + + /** + * @param type Type. + * @param injected Injected. + */ + public ResourceInfoSet set(ResourceType type, boolean injected) { + int mask = 1 << type.ordinal(); + + if (injected) + val |= mask; + else + val &= ~mask; + + return this; + } + + /** + * @see {@link #set(ResourceType, boolean)} + */ + public ResourceInfoSet set(ResourceType type, Object toCheck) { + return set(type, toCheck != null); + } + + /** + * @return collection of not injected resources + */ + public Collection notInjected(Collection exp) { + ArrayList res = null; + + for (ResourceType type : exp) { + int mask = 1 << type.ordinal(); + + if ((this.val & mask) == 0) { + if (res == null) + res = new ArrayList<>(); + + res.add(type); + } + } + + return res == null ? Collections.emptyList() : res; + } + } + + /** + * + */ + protected static abstract class ResourceInjectionEntryProcessorBase + implements EntryProcessor, Serializable { + /** */ + protected transient ResourceInfoSet infoSet; + + /** {@inheritDoc} */ + @Override public Integer process(MutableEntry e, Object... args) { + return infoSet == null ? null : infoSet.getValue(); + } + + /** */ + protected void checkSet() { + if (infoSet == null) + infoSet = new ResourceInfoSet(); + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java index a3caba605da1a..f36b060afd9dd 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTransformEventSelfTest.java @@ -38,6 +38,7 @@ import org.apache.ignite.events.Event; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; @@ -74,9 +75,6 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest { /** Cache name. */ private static final String CACHE_NAME = "cache"; - /** Closure name. */ - private static final String CLO_NAME = Transformer.class.getName(); - /** Key 1. */ private Integer key1; @@ -98,7 +96,7 @@ public class GridCacheTransformEventSelfTest extends GridCommonAbstractTest { /** Caches. */ private IgniteCache[] caches; - /** Recorded events.*/ + /** Recorded events. */ private ConcurrentHashSet evts; /** Cache mode. */ @@ -477,13 +475,25 @@ private void checkAtomic(CacheMode cacheMode) throws Exception { caches[0].invoke(key1, new Transformer()); - checkEventNodeIdsStrict(primaryIdsForKeys(key1)); + checkEventNodeIdsStrict(Transformer.class.getName(), primaryIdsForKeys(key1)); assert evts.isEmpty(); caches[0].invokeAll(keys, new Transformer()); - checkEventNodeIdsStrict(primaryIdsForKeys(key1, key2)); + checkEventNodeIdsStrict(Transformer.class.getName(), primaryIdsForKeys(key1, key2)); + + assert evts.isEmpty(); + + caches[0].invoke(key1, new TransformerWithInjection()); + + checkEventNodeIdsStrict(TransformerWithInjection.class.getName(), primaryIdsForKeys(key1)); + + assert evts.isEmpty(); + + caches[0].invokeAll(keys, new TransformerWithInjection()); + + checkEventNodeIdsStrict(TransformerWithInjection.class.getName(), primaryIdsForKeys(key1, key2)); } /** @@ -492,7 +502,6 @@ private void checkAtomic(CacheMode cacheMode) throws Exception { * @param cacheMode Cache mode. * @param txConcurrency TX concurrency. * @param txIsolation TX isolation. - * * @throws Exception If failed. */ private void checkTx(CacheMode cacheMode, TransactionConcurrency txConcurrency, @@ -505,13 +514,29 @@ private void checkTx(CacheMode cacheMode, TransactionConcurrency txConcurrency, System.out.println("AFTER: " + evts.size()); - checkEventNodeIdsStrict(idsForKeys(key1)); + checkEventNodeIdsStrict(Transformer.class.getName(), idsForKeys(key1)); assert evts.isEmpty(); caches[0].invokeAll(keys, new Transformer()); - checkEventNodeIdsStrict(idsForKeys(key1, key2)); + checkEventNodeIdsStrict(Transformer.class.getName(), idsForKeys(key1, key2)); + + assert evts.isEmpty(); + + System.out.println("BEFORE: " + evts.size()); + + caches[0].invoke(key1, new TransformerWithInjection()); + + System.out.println("AFTER: " + evts.size()); + + checkEventNodeIdsStrict(TransformerWithInjection.class.getName(), idsForKeys(key1)); + + assert evts.isEmpty(); + + caches[0].invokeAll(keys, new TransformerWithInjection()); + + checkEventNodeIdsStrict(TransformerWithInjection.class.getName(), idsForKeys(key1, key2)); } /** @@ -572,9 +597,10 @@ else if (cacheMode == REPLICATED) { /** * Ensure that events were recorded on the given nodes. * + * @param cClsName Entry processor class name. * @param ids Event IDs. */ - private void checkEventNodeIdsStrict(UUID... ids) { + private void checkEventNodeIdsStrict(String cClsName, UUID... ids) { if (ids == null) assertTrue(evts.isEmpty()); else { @@ -585,7 +611,7 @@ private void checkEventNodeIdsStrict(UUID... ids) { for (CacheEvent evt : evts) { if (F.eq(id, evt.node().id())) { - assertEquals(CLO_NAME, evt.closureClassName()); + assertEquals(cClsName, evt.closureClassName()); foundEvt = evt; @@ -625,4 +651,22 @@ private static class Transformer implements EntryProcessor, Serializable { + /** */ + @IgniteInstanceResource + private transient Ignite ignite; + + /** {@inheritDoc} */ + @Override public Void process(MutableEntry e, Object... args) { + assert ignite != null; + + e.setValue(e.getValue() + 1); + + return null; + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java index e78f3296b5d9f..bcf4ccdddf43d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedBasicStoreMultiNodeSelfTest.java @@ -70,6 +70,8 @@ public class GridCachePartitionedBasicStoreMultiNodeSelfTest extends GridCommonA /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + stores = Collections.synchronizedList(new ArrayList()); startGridsMultiThreaded(GRID_CNT); diff --git a/modules/spring/src/test/java/org/apache/ignite/internal/processors/resource/GridTransformSpringInjectionSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/internal/processors/resource/GridTransformSpringInjectionSelfTest.java new file mode 100644 index 0000000000000..cc61514d3b0b3 --- /dev/null +++ b/modules/spring/src/test/java/org/apache/ignite/internal/processors/resource/GridTransformSpringInjectionSelfTest.java @@ -0,0 +1,186 @@ +/* + * 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.resource; + +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import javax.cache.processor.EntryProcessorResult; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteSpring; +import org.apache.ignite.IgniteTransactions; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest; +import org.apache.ignite.resources.SpringApplicationContextResource; +import org.apache.ignite.resources.SpringResource; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.springframework.context.ApplicationContext; +import org.springframework.context.support.ClassPathXmlApplicationContext; + +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; + +/** + * + */ +public class GridTransformSpringInjectionSelfTest extends GridCacheAbstractSelfTest { + /** {@inheritDoc} */ + @Override protected int gridCount() { + return 1; + } + + /** {@inheritDoc} */ + @Override public void beforeTestsStarted() throws Exception { + IgniteSpring.start(getConfiguration(getTestGridName(0)), + new ClassPathXmlApplicationContext("/org/apache/ignite/internal/processors/resource/spring-resource.xml")); + } + + /** + * @throws Exception If failed. + */ + public void testTransformResourceInjection() throws Exception { + Ignite grid = grid(0); + + IgniteCache cache = grid.createCache(cacheConfiguration(ATOMIC)); + + try { + doTransformResourceInjection(cache); + } + finally { + cache.destroy(); + } + + cache = grid.createCache(cacheConfiguration(TRANSACTIONAL)); + + try { + doTransformResourceInjection(cache); + + for (TransactionConcurrency concurrency : TransactionConcurrency.values()) { + for (TransactionIsolation isolation : TransactionIsolation.values()) { + IgniteTransactions txs = grid.transactions(); + + try (Transaction tx = txs.txStart(concurrency, isolation)) { + doTransformResourceInjection(cache); + + tx.commit(); + } + } + } + } + finally { + cache.destroy(); + } + } + + /** + * @param atomicityMode Cache atomicity mode. + * @return Cache configuration. + */ + private CacheConfiguration cacheConfiguration(CacheAtomicityMode atomicityMode) { + CacheConfiguration ccfg = new CacheConfiguration<>(); + + ccfg.setName(getClass().getSimpleName()); + ccfg.setAtomicityMode(atomicityMode); + + return ccfg; + } + + /** + * @param cache Cache. + * @throws Exception If failed. + */ + private void doTransformResourceInjection(IgniteCache cache) throws Exception { + final Collection required = Arrays.asList( + ResourceType.SPRING_APPLICATION_CONTEXT, + ResourceType.SPRING_BEAN); + + Integer flags = cache.invoke(UUID.randomUUID().toString(), new SpringResourceInjectionEntryProcessor()); + + assertTrue("Processor result is null", flags != null); + + log.info("Injection flag: " + Integer.toBinaryString(flags)); + + Collection notInjected = ResourceInfoSet.valueOf(flags).notInjected(required); + + if (!notInjected.isEmpty()) + fail("Can't inject resource(s): " + Arrays.toString(notInjected.toArray())); + + Set keys = new HashSet<>(Arrays.asList(UUID.randomUUID().toString(), + UUID.randomUUID().toString(), + UUID.randomUUID().toString(), + UUID.randomUUID().toString())); + + Map> results = cache.invokeAll(keys, + new SpringResourceInjectionEntryProcessor()); + + assertEquals(keys.size(), results.size()); + + for (EntryProcessorResult res : results.values()) { + Collection notInjected1 = ResourceInfoSet.valueOf(res.get()).notInjected(required); + + if (!notInjected1.isEmpty()) + fail("Can't inject resource(s): " + Arrays.toString(notInjected1.toArray())); + } + } + + /** + * + */ + static class SpringResourceInjectionEntryProcessor extends ResourceInjectionEntryProcessorBase { + /** */ + private transient ApplicationContext appCtx; + + /** */ + private transient GridSpringResourceInjectionSelfTest.DummyResourceBean dummyBean; + + /** + * @param appCtx Context. + */ + @SpringApplicationContextResource + public void setApplicationContext(ApplicationContext appCtx) { + assert appCtx != null; + + checkSet(); + + infoSet.set(ResourceType.SPRING_APPLICATION_CONTEXT, true); + + this.appCtx = appCtx; + } + + /** + * @param dummyBean Resource bean. + */ + @SpringResource(resourceName = "dummyResourceBean") + public void setDummyBean(GridSpringResourceInjectionSelfTest.DummyResourceBean dummyBean) { + assert dummyBean != null; + + checkSet(); + + infoSet.set(ResourceType.SPRING_BEAN, true); + + this.dummyBean = dummyBean; + } + } +} diff --git a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java index cd5645dac637c..67b117dbee686 100644 --- a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java +++ b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java @@ -18,17 +18,18 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.cache.spring.GridSpringCacheManagerSelfTest; +import org.apache.ignite.cache.spring.SpringCacheManagerContextInjectionTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactorySelfTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactorySelfTest; import org.apache.ignite.cache.store.spring.CacheSpringStoreSessionListenerSelfTest; import org.apache.ignite.internal.GridFactorySelfTest; import org.apache.ignite.internal.GridSpringBeanSerializationSelfTest; import org.apache.ignite.internal.IgniteDynamicCacheConfigTest; +import org.apache.ignite.internal.processors.resource.GridTransformSpringInjectionSelfTest; import org.apache.ignite.p2p.GridP2PUserVersionChangeSelfTest; -import org.apache.ignite.cache.spring.GridSpringCacheManagerSelfTest; import org.apache.ignite.spring.IgniteExcludeInConfigurationTest; import org.apache.ignite.spring.IgniteStartFromStreamConfigurationTest; -import org.apache.ignite.cache.spring.SpringCacheManagerContextInjectionTest; import org.apache.ignite.spring.injection.GridServiceInjectionSpringResourceTest; import org.apache.ignite.transactions.spring.GridSpringTransactionManagerSelfTest; import org.apache.ignite.transactions.spring.SpringTransactionManagerContextInjectionTest; @@ -70,6 +71,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridServiceInjectionSpringResourceTest.class); + suite.addTestSuite(GridTransformSpringInjectionSelfTest.class); + suite.addTestSuite(SpringCacheManagerContextInjectionTest.class); suite.addTestSuite(SpringTransactionManagerContextInjectionTest.class); diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteInvokeWithInjectionBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteInvokeWithInjectionBenchmark.java new file mode 100644 index 0000000000000..ef9d17bd5bcbd --- /dev/null +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteInvokeWithInjectionBenchmark.java @@ -0,0 +1,74 @@ +/* + * 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.yardstick.cache; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheEntryProcessor; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.yardstick.cache.model.SampleValue; + +import javax.cache.processor.MutableEntry; +import java.util.Map; + +/** + * Ignite benchmark that performs invoke operations. + */ +public class IgniteInvokeWithInjectionBenchmark extends IgniteCacheAbstractBenchmark { + /** {@inheritDoc} */ + @Override public boolean test(Map ctx) throws Exception { + int key = nextRandom(args.range()); + + cache.invoke(key, new SetValueEntryProcessor(new SampleValue(key))); + + return true; + } + + /** {@inheritDoc} */ + @Override protected IgniteCache cache() { + return ignite().cache("atomic"); + } + + /** + * + */ + public static class SetValueEntryProcessor implements CacheEntryProcessor { + /** */ + @IgniteInstanceResource + private transient Ignite ignite; + + /** */ + private Object val; + + /** + * @param val Value. + */ + public SetValueEntryProcessor(Object val) { + this.val = val; + } + + /** {@inheritDoc} */ + @Override public Object process(MutableEntry entry, Object... args) { + assert ignite != null; + + entry.setValue(val); + + return null; + } + } +} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteInvokeWithInjectionTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteInvokeWithInjectionTxBenchmark.java new file mode 100644 index 0000000000000..2df93eeab7146 --- /dev/null +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteInvokeWithInjectionTxBenchmark.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.yardstick.cache; + +import org.apache.ignite.IgniteCache; + +/** + * Ignite benchmark that performs invoke operations. + */ +public class IgniteInvokeWithInjectionTxBenchmark extends IgniteInvokeWithInjectionBenchmark { + /** {@inheritDoc} */ + @Override protected IgniteCache cache() { + return ignite().cache("tx"); + } +} From 3244a5c9dabf6d4fcaa32a661cc0adc6f8ea30de Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 30 Aug 2016 11:49:11 +0300 Subject: [PATCH 073/487] IGNITE-3742: ODBC: Added support for OUTER JOIN escape sequence. This closes #1000. --- .../odbc/escape/OdbcEscapeUtils.java | 32 ++--- .../odbc/OdbcEscapeSequenceSelfTest.java | 109 +++++++++++++++++- 2 files changed, 122 insertions(+), 19 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java index a4b89c399c72b..27120d4a58c26 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java @@ -105,14 +105,14 @@ else if (curChar == '}') { if (nested == null) // Found sequence without nesting, process it. - parseRes = parseExpression(text, openPos, curPos + 1 - openPos); + parseRes = parseEscapeSequence(text, openPos, curPos + 1 - openPos); else { // Special case to process nesting. String res0 = appendNested(text, openPos, curPos + 1, nested); nested = null; - parseRes = parseExpression(res0, 0, res0.length()); + parseRes = parseEscapeSequence(res0, 0, res0.length()); } if (earlyExit) @@ -139,14 +139,14 @@ else if (curChar == '}') { } /** - * Parse concrete expression. + * Parse escape sequence: {escape_sequence}. * * @param text Text. * @param startPos Start position within text. * @param len Length. * @return Result. */ - private static String parseExpression(String text, int startPos, int len) { + private static String parseEscapeSequence(String text, int startPos, int len) { assert validSubstring(text, startPos, len); char firstChar = text.charAt(startPos); @@ -228,7 +228,7 @@ private static OdbcEscapeToken parseToken(String text, int startPos, int len) { } /** - * Parse standard token. + * Parse standard expression: {TOKEN expression} * * @param text Text. * @param startPos Start position. @@ -245,10 +245,13 @@ private static String parseStandardExpression(String text, int startPos, int len switch (token.type()) { case SCALAR_FUNCTION: - return parseScalarExpression(text, startPos0, len0); + return parseExpression(text, startPos0, len0); + + case GUID: { + String res = parseExpression(text, startPos0, len0, token.type(), GUID_PATTERN); - case GUID: - return parseExpression(text, startPos0, len0, token.type(), GUID_PATTERN); + return "CAST(" + res + " AS UUID)"; + } case DATE: return parseExpression(text, startPos0, len0, token.type(), DATE_PATTERN); @@ -259,6 +262,9 @@ private static String parseStandardExpression(String text, int startPos, int len case TIMESTAMP: return parseExpression(text, startPos0, len0, token.type(), TIMESTAMP_PATTERN); + case OUTER_JOIN: + return parseExpression(text, startPos0, len0); + default: throw new IgniteException("Unsupported escape sequence token [text=" + substring(text, startPos, len) + ", token=" + token.type().body() + ']'); @@ -266,19 +272,19 @@ private static String parseStandardExpression(String text, int startPos, int len } /** - * Parse scalar function expression. + * Parse simple expression. * * @param text Text. * @param startPos Start position. * @param len Length. * @return Parsed expression. */ - private static String parseScalarExpression(String text, int startPos, int len) { + private static String parseExpression(String text, int startPos, int len) { return substring(text, startPos, len).trim(); } /** - * Parse concrete expression. + * Parse expression and validate against ODBC specification with regex pattern. * * @param text Text. * @param startPos Start position. @@ -286,12 +292,12 @@ private static String parseScalarExpression(String text, int startPos, int len) * @return Parsed expression. */ private static String parseExpression(String text, int startPos, int len, OdbcEscapeType type, Pattern pattern) { - String val = substring(text, startPos, len).trim(); + String val = parseExpression(text, startPos, len); if (!pattern.matcher(val).matches()) throw new IgniteException("Invalid " + type + " escape sequence: " + substring(text, startPos, len)); - return "CAST(" + val + " AS UUID)"; + return val; } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java index 1aa90fd07e958..4887a67812eba 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java @@ -167,17 +167,17 @@ public void testFunctionEscapeSequenceWithWhitespaces() throws Exception { */ public void testGuidEscapeSequence() { check( - "'12345678-9abc-def0-1234-123456789abc'", + "CAST('12345678-9abc-def0-1234-123456789abc' AS UUID)", "{guid '12345678-9abc-def0-1234-123456789abc'}" ); check( - "select '12345678-9abc-def0-1234-123456789abc' from SomeTable;", + "select CAST('12345678-9abc-def0-1234-123456789abc' AS UUID) from SomeTable;", "select {guid '12345678-9abc-def0-1234-123456789abc'} from SomeTable;" ); check( - "select '12345678-9abc-def0-1234-123456789abc'", + "select CAST('12345678-9abc-def0-1234-123456789abc' AS UUID)", "select {guid '12345678-9abc-def0-1234-123456789abc'}" ); } @@ -212,17 +212,17 @@ public void testFailedOnInvalidGuidSequence() { */ public void testGuidEscapeSequenceWithWhitespaces() throws Exception { check( - "'12345678-9abc-def0-1234-123456789abc'", + "CAST('12345678-9abc-def0-1234-123456789abc' AS UUID)", "{ guid '12345678-9abc-def0-1234-123456789abc'}" ); check( - "'12345678-9abc-def0-1234-123456789abc'", + "CAST('12345678-9abc-def0-1234-123456789abc' AS UUID)", "{ guid '12345678-9abc-def0-1234-123456789abc'}" ); check( - "'12345678-9abc-def0-1234-123456789abc'", + "CAST('12345678-9abc-def0-1234-123456789abc' AS UUID)", "{ \n guid\n'12345678-9abc-def0-1234-123456789abc'}" ); } @@ -388,6 +388,103 @@ public void testFailedOnInvalidTimestampSequence() { checkFail("select {}ts '2016-08-26 13:15:08'} from table;"); } + + /** + * Test escape sequence series. + */ + public void testOuterJoinFunction() throws Exception { + check( + "t OUTER JOIN t2 ON t.id=t2.id", + "{oj t OUTER JOIN t2 ON t.id=t2.id}" + ); + + check( + "select * from t OUTER JOIN t2 ON t.id=t2.id", + "select * from {oj t OUTER JOIN t2 ON t.id=t2.id}" + ); + + check( + "select * from t OUTER JOIN t2 ON t.id=t2.id ORDER BY t2.id", + "select * from {oj t OUTER JOIN t2 ON t.id=t2.id} ORDER BY t2.id" + ); + } + + /** + * Test simple nested escape sequences. Depth = 2. + */ + public void testNestedOuterJoin() throws Exception { + check( + "t OUTER JOIN (t2 OUTER JOIN t3 ON t2.id=t3.id) ON t.id=t2.id", + "{oj t OUTER JOIN ({oj t2 OUTER JOIN t3 ON t2.id=t3.id}) ON t.id=t2.id}" + ); + + check( + "select * from t OUTER JOIN (t2 OUTER JOIN t3 ON t2.id=t3.id) ON t.id=t2.id", + "select * from {oj t OUTER JOIN ({oj t2 OUTER JOIN t3 ON t2.id=t3.id}) ON t.id=t2.id}" + ); + + check( + "select * from t OUTER JOIN (t2 OUTER JOIN t3 ON t2.id=t3.id) ON t.id=t2.id ORDER BY t2.id", + "select * from {oj t OUTER JOIN ({oj t2 OUTER JOIN t3 ON t2.id=t3.id}) ON t.id=t2.id} ORDER BY t2.id" + ); + } + + /** + * Test nested escape sequences. Depth > 2. + */ + public void testDeepNestedOuterJoin() { + check( + "t OUTER JOIN (t2 OUTER JOIN (t3 OUTER JOIN t4 ON t3.id=t4.id) ON t2.id=t3.id) ON t.id=t2.id", + "{oj t OUTER JOIN ({oj t2 OUTER JOIN ({oj t3 OUTER JOIN t4 ON t3.id=t4.id}) ON t2.id=t3.id}) ON t.id=t2.id}" + ); + + check( + "select * from " + + "t OUTER JOIN (t2 OUTER JOIN (t3 OUTER JOIN t4 ON t3.id=t4.id) ON t2.id=t3.id) ON t.id=t2.id", + "select * from " + + "{oj t OUTER JOIN ({oj t2 OUTER JOIN ({oj t3 OUTER JOIN t4 ON t3.id=t4.id}) ON t2.id=t3.id})" + + " ON t.id=t2.id}" + ); + + check( + "select * from t OUTER JOIN (t2 OUTER JOIN (t3 OUTER JOIN t4 ON t3.id=t4.id) " + + "ON t2.id=t3.id) ON t.id=t2.id ORDER BY t4.id", + "select * from {oj t OUTER JOIN ({oj t2 OUTER JOIN ({oj t3 OUTER JOIN t4 ON t3.id=t4.id}) " + + "ON t2.id=t3.id}) ON t.id=t2.id} ORDER BY t4.id" + ); + } + + /** + * Test invalid escape sequence. + */ + public void testFailedOnInvalidOuterJoinSequence() { + checkFail("{ojt OUTER JOIN t2 ON t.id=t2.id}"); + + checkFail("select {oj t OUTER JOIN ({oj t2 OUTER JOIN t3 ON t2.id=t3.id) ON t.id=t2.id} from SomeTable;"); + + checkFail("select oj t OUTER JOIN t2 ON t.id=t2.id} from SomeTable;"); + } + + /** + * Test escape sequences with additional whitespace characters + */ + public void testOuterJoinSequenceWithWhitespaces() throws Exception { + check( + "t OUTER JOIN t2 ON t.id=t2.id", + "{ oj t OUTER JOIN t2 ON t.id=t2.id}" + ); + + check( + "t OUTER JOIN t2 ON t.id=t2.id", + "{ oj t OUTER JOIN t2 ON t.id=t2.id}" + ); + + check( + "t OUTER JOIN t2 ON t.id=t2.id", + " \n { oj\nt OUTER JOIN t2 ON t.id=t2.id}" + ); + } + /** * Check parsing logic. * From fbbcaf4322548f61d2f63bf5d4e8f6d5284e73d3 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 30 Aug 2016 13:22:29 +0300 Subject: [PATCH 074/487] IGNITE-3798: ODBC: Added literals support. This closes #1005. --- .../odbc/escape/OdbcEscapeUtils.java | 87 +++++++++++-------- .../odbc/OdbcEscapeSequenceSelfTest.java | 55 ++++++++++++ 2 files changed, 105 insertions(+), 37 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java index 27120d4a58c26..48d4296e3d1e2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java @@ -69,60 +69,70 @@ private static OdbcEscapeParseResult parse0(String text, int startPos, boolean e int plainPos = startPos; int openPos = -1; + boolean insideLiteral = false; + LinkedList nested = null; while (curPos < text.length()) { char curChar = text.charAt(curPos); - if (curChar == '{') { - if (openPos == -1) { - // Top-level opening brace. Append previous portion and remember current position. - res.append(text, plainPos, curPos); + if (curChar == '\'') { + if (!insideLiteral) + insideLiteral = true; + else if (text.charAt(curPos - 1) != '\\') + insideLiteral = false; + } + else if (!insideLiteral) { + if (curChar == '{') { + if (openPos == -1) { + // Top-level opening brace. Append previous portion and remember current position. + res.append(text, plainPos, curPos); - openPos = curPos; - } - else { - // Nested opening brace -> perform recursion. - OdbcEscapeParseResult nestedRes = parse0(text, curPos, true); + openPos = curPos; + } + else { + // Nested opening brace -> perform recursion. + OdbcEscapeParseResult nestedRes = parse0(text, curPos, true); - if (nested == null) - nested = new LinkedList<>(); + if (nested == null) + nested = new LinkedList<>(); - nested.add(nestedRes); + nested.add(nestedRes); - curPos += nestedRes.originalLength() - 1; + curPos += nestedRes.originalLength() - 1; - plainPos = curPos + 1; + plainPos = curPos + 1; + } } - } - else if (curChar == '}') { - if (openPos == -1) - // Close without open -> exception. - throw new IgniteException("Malformed escape sequence " + - "(closing curly brace without opening curly brace): " + text); - else { - String parseRes; - - if (nested == null) - // Found sequence without nesting, process it. - parseRes = parseEscapeSequence(text, openPos, curPos + 1 - openPos); + else if (curChar == '}') { + if (openPos == -1) + // Close without open -> exception. + throw new IgniteException("Malformed escape sequence " + + "(closing curly brace without opening curly brace): " + text); else { - // Special case to process nesting. - String res0 = appendNested(text, openPos, curPos + 1, nested); + String parseRes; - nested = null; + if (nested == null) + // Found sequence without nesting, process it. + parseRes = parseEscapeSequence(text, openPos, curPos + 1 - openPos); + else { + // Special case to process nesting. + String res0 = appendNested(text, openPos, curPos + 1, nested); - parseRes = parseEscapeSequence(res0, 0, res0.length()); - } + nested = null; - if (earlyExit) - return new OdbcEscapeParseResult(startPos, curPos + 1 - startPos, parseRes); - else - res.append(parseRes); + parseRes = parseEscapeSequence(res0, 0, res0.length()); + } - openPos = -1; + if (earlyExit) + return new OdbcEscapeParseResult(startPos, curPos + 1 - startPos, parseRes); + else + res.append(parseRes); - plainPos = curPos + 1; + openPos = -1; + + plainPos = curPos + 1; + } } } @@ -132,6 +142,9 @@ else if (curChar == '}') { if (openPos != -1) throw new IgniteException("Malformed escape sequence (closing curly brace missing): " + text); + if (insideLiteral) + throw new IgniteException("Malformed literal expression (closing quote missing): " + text); + if (curPos > plainPos) res.append(text, plainPos, curPos); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java index 4887a67812eba..3fec7d3a2e368 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java @@ -485,6 +485,61 @@ public void testOuterJoinSequenceWithWhitespaces() throws Exception { ); } + /** + * Test non-escape sequences. + */ + public void testNonEscapeSequence() throws Exception { + check("'{fn test()}'", "'{fn test()}'"); + + check("select '{fn test()}'", "select '{fn test()}'"); + + check( + "select '{fn test()}' from table;", + "select '{fn test()}' from table;" + ); + + check( + "select test('arg') from table;", + "select {fn test('arg')} from table;" + ); + + check( + "select test('{fn func()}') from table;", + "select {fn test('{fn func()}')} from table;" + ); + + check( + "'{\\'some literal\\'}'", + "'{\\'some literal\\'}'" + ); + + check( + "select '{\\'some literal\\'}'", + "select '{\\'some literal\\'}'" + ); + + check( + "select '{\\'some literal\\'}' from table;", + "select '{\\'some literal\\'}' from table;" + ); + + check( + "select '{' + func() + '}' from table;", + "select '{' + {fn func()} + '}' from table;" + ); + + check( + "select '{\\'{fn test()}\\'}' from table;", + "select '{\\'{fn test()}\\'}' from table;" + ); + + checkFail("'{fn test()}"); + + checkFail("{fn func('arg)}"); + + checkFail("{fn func(arg')}"); + } + /** * Check parsing logic. * From 1ef150eba52eb63c2bfc3fafa0d036cf26be1c5b Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Tue, 30 Aug 2016 18:18:20 +0700 Subject: [PATCH 075/487] Revert wrong merge. --- .../ignite/visor/commands/cache/VisorCacheStopCommand.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala index 1b555059c874b..22fb89d2047f3 100644 --- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala +++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala @@ -19,7 +19,6 @@ package org.apache.ignite.visor.commands.cache import org.apache.ignite.cluster.{ClusterGroupEmptyException, ClusterNode} import org.apache.ignite.visor.visor._ - import org.apache.ignite.internal.visor.cache.VisorCacheStopTask import org.apache.ignite.internal.visor.util.VisorTaskUtils._ @@ -102,7 +101,9 @@ class VisorCacheStopCommand { return } - ask(s"Are you sure you want to stop cache: ${escapeName(cacheName)}? (y/n) [n]: ", "n") match { + val dflt = if (batchMode) "y" else "n" + + ask(s"Are you sure you want to stop cache: ${escapeName(cacheName)}? (y/n) [$dflt]: ", dflt) match { case "y" | "Y" => try { executeRandom(grp, classOf[VisorCacheStopTask], cacheName) From 31dbc5d65f8ea51010d2129e7c6e9a27acbf8528 Mon Sep 17 00:00:00 2001 From: isapego Date: Tue, 30 Aug 2016 17:28:34 +0300 Subject: [PATCH 076/487] Fix for C++ tests. --- .../odbc-test/config/queries-test-noodbc.xml | 48 +++++++------- .../cpp/odbc-test/config/queries-test.xml | 50 +++++++-------- .../odbc-test/project/vs/odbc-test.vcxproj | 1 + .../project/vs/odbc-test.vcxproj.filters | 3 + .../cpp/odbc-test/src/queries_test.cpp | 64 +++---------------- .../odbc-test/src/sql_test_suite_fixture.cpp | 14 ++-- 6 files changed, 68 insertions(+), 112 deletions(-) diff --git a/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml index 18447c28446f3..db19669218329 100644 --- a/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml +++ b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml @@ -28,6 +28,7 @@ + @@ -38,14 +39,14 @@ - - - - - + + + + + - - + + @@ -57,22 +58,22 @@ - - + + - - - - - - - - - - - - - + + + + + + + + + + + + + @@ -86,8 +87,7 @@ instead os static IP based discovery. --> - - + diff --git a/modules/platforms/cpp/odbc-test/config/queries-test.xml b/modules/platforms/cpp/odbc-test/config/queries-test.xml index 54cb9beb694df..26e63417e0334 100644 --- a/modules/platforms/cpp/odbc-test/config/queries-test.xml +++ b/modules/platforms/cpp/odbc-test/config/queries-test.xml @@ -32,8 +32,8 @@ - - + + @@ -43,16 +43,16 @@ - + - - - - - + + + + + - - + + @@ -64,22 +64,22 @@ - - + + - - - - - - - - - - - - - + + + + + + + + + + + + + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index 0702047cc466f..cb5735f707097 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -203,6 +203,7 @@ + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index 0a72640a6a691..270bdd624727c 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -125,5 +125,8 @@ Configs + + Configs + \ No newline at end of file diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index ab5995200aa59..c9077723b2149 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -60,35 +60,6 @@ struct QueriesTestSuiteFixture */ void Connect(const std::string& connectStr) { - IgniteConfiguration cfg; - - cfg.jvmOpts.push_back("-Xdebug"); - cfg.jvmOpts.push_back("-Xnoagent"); - cfg.jvmOpts.push_back("-Djava.compiler=NONE"); - cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005"); - cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError"); - -#ifdef IGNITE_TESTS_32 - cfg.jvmInitMem = 256; - cfg.jvmMaxMem = 768; -#else - cfg.jvmInitMem = 1024; - cfg.jvmMaxMem = 4096; -#endif - - char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH"); - - cfg.springCfgPath = std::string(cfgPath).append("/").append("queries-test.xml"); - - IgniteError err; - - grid = Ignition::Start(cfg, &err); - - if (err.GetCode() != IgniteError::IGNITE_SUCCESS) - BOOST_FAIL(err.GetText()); - - testCache = grid.GetCache("cache"); - // Allocate an environment handle SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env); @@ -141,7 +112,7 @@ struct QueriesTestSuiteFixture SQLFreeHandle(SQL_HANDLE_ENV, env); } - static Ignite StartAdditionalNode(const char* name) + static Ignite StartNode(const char* name, const char* config) { IgniteConfiguration cfg; @@ -163,43 +134,24 @@ struct QueriesTestSuiteFixture BOOST_REQUIRE(cfgPath != 0); - cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml"); + cfg.springCfgPath.assign(cfgPath).append("/").append(config); IgniteError err; return Ignition::Start(cfg, name); } + static Ignite StartAdditionalNode(const char* name) + { + return StartNode(name, "queries-test-noodbc.xml"); + } + /** * Constructor. */ QueriesTestSuiteFixture() : testCache(0), env(NULL), dbc(NULL), stmt(NULL) { - IgniteConfiguration cfg; - - cfg.jvmOpts.push_back("-Xdebug"); - cfg.jvmOpts.push_back("-Xnoagent"); - cfg.jvmOpts.push_back("-Djava.compiler=NONE"); - cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005"); - cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError"); - -#ifdef IGNITE_TESTS_32 - cfg.jvmInitMem = 256; - cfg.jvmMaxMem = 768; -#else - cfg.jvmInitMem = 1024; - cfg.jvmMaxMem = 4096; -#endif - - char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH"); - - BOOST_REQUIRE(cfgPath != 0); - - cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml"); - - IgniteError err; - - grid = Ignition::Start(cfg, "NodeMain"); + grid = StartNode("NodeMain", "queries-test.xml"); testCache = grid.GetCache("cache"); } diff --git a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp index 16e5ea03ab091..69b4bfa49662b 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp @@ -45,7 +45,7 @@ namespace ignite char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH"); - BOOST_REQUIRE(cfgPath != 0) ; + BOOST_REQUIRE(cfgPath != 0); cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml"); @@ -54,14 +54,14 @@ namespace ignite grid = Ignition::Start(cfg, &err); if (err.GetCode() != IgniteError::IGNITE_SUCCESS) - BOOST_FAIL(err.GetText()) ; + BOOST_FAIL(err.GetText()) ; testCache = grid.GetCache("cache"); // Allocate an environment handle SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env); - BOOST_REQUIRE(env != NULL) ; + BOOST_REQUIRE(env != NULL); // We want ODBC 3 support SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast(SQL_OV_ODBC3), 0); @@ -69,10 +69,10 @@ namespace ignite // Allocate a connection handle SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc); - BOOST_REQUIRE(dbc != NULL) ; + BOOST_REQUIRE(dbc != NULL); // Connect string - SQLCHAR connectStr[] = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=cache"; + SQLCHAR connectStr[] = "DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"; SQLCHAR outstr[ODBC_BUFFER_SIZE]; SQLSMALLINT outstrlen; @@ -85,13 +85,13 @@ namespace ignite { Ignition::Stop(grid.GetName(), true); - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc)) ; + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc)); } // Allocate a statement handle SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt); - BOOST_REQUIRE(stmt != NULL) ; + BOOST_REQUIRE(stmt != NULL); } SqlTestSuiteFixture::~SqlTestSuiteFixture() From dabd86c62e39eb983ef3d198c8b8ee96d2623c84 Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Wed, 31 Aug 2016 16:00:19 +0700 Subject: [PATCH 077/487] IGNITE-3811 Fixed query for merge with complex key on SQL Server. --- .../ignite/cache/store/jdbc/dialect/SQLServerDialect.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java index 0082617e0d10a..9831aa88b1dba 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java @@ -44,7 +44,7 @@ public class SQLServerDialect extends BasicJdbcDialect { @Override public String apply(String col) { return String.format("t.%s=v.%s", col, col); } - }, "", ", ", ""); + }, "", " AND ", ""); String setCols = mkString(uniqCols, new C1() { @Override public String apply(String col) { From 70e69cb7aa08c268b07920838add4a40e28fe25d Mon Sep 17 00:00:00 2001 From: isapego Date: Wed, 31 Aug 2016 16:47:11 +0300 Subject: [PATCH 078/487] IGNITE-3390: Added DSN configuration window. --- .../processors/odbc/OdbcHandshakeRequest.java | 8 +- .../processors/odbc/OdbcHandshakeResult.java | 17 +- .../processors/odbc/OdbcMessageParser.java | 22 +- .../processors/odbc/OdbcProtocolVersion.java | 106 +++++++ .../processors/odbc/OdbcRequestHandler.java | 17 +- .../cpp/common/include/ignite/common/utils.h | 25 ++ .../cpp/common/os/win/src/common/utils.cpp | 20 ++ modules/platforms/cpp/odbc-test/Makefile.am | 1 + .../odbc-test/config/queries-test-noodbc.xml | 103 +++++++ .../cpp/odbc-test/config/queries-test.xml | 31 +- .../odbc-test/project/vs/odbc-test.vcxproj | 2 + .../project/vs/odbc-test.vcxproj.filters | 6 + .../cpp/odbc-test/src/configuration_test.cpp | 156 +++++++--- .../cpp/odbc-test/src/queries_test.cpp | 122 +++++--- .../odbc-test/src/sql_test_suite_fixture.cpp | 14 +- modules/platforms/cpp/odbc/Makefile.am | 2 + .../platforms/cpp/odbc/include/Makefile.am | 2 + .../odbc/include/ignite/odbc/common_types.h | 3 + .../ignite/odbc/config/configuration.h | 207 ++++++++++++-- .../cpp/odbc/include/ignite/odbc/connection.h | 47 ++-- .../odbc/diagnostic/diagnostic_record.h | 2 +- .../cpp/odbc/include/ignite/odbc/dsn_config.h | 61 ++++ .../cpp/odbc/include/ignite/odbc/parser.h | 3 - .../include/ignite/odbc/protocol_version.h | 168 +++++++++++ .../ignite/odbc/system/odbc_constants.h | 4 - .../odbc/system/ui/dsn_configuration_window.h | 136 +++++++++ .../ignite/odbc/system/ui/custom_window.h | 189 +++++++++++++ .../include/ignite/odbc/system/ui/window.h | 201 +++++++++++++ .../os/win/src/system/ui/custom_window.cpp | 184 ++++++++++++ .../system/ui/dsn_configuration_window.cpp | 212 ++++++++++++++ .../cpp/odbc/os/win/src/system/ui/window.cpp | 192 +++++++++++++ .../cpp/odbc/os/win/src/system_dsn.cpp | 218 ++++++++++++++ .../cpp/odbc/project/vs/odbc.vcxproj | 27 +- .../cpp/odbc/project/vs/odbc.vcxproj.filters | 36 +++ .../cpp/odbc/src/config/configuration.cpp | 266 ++++++++++-------- modules/platforms/cpp/odbc/src/connection.cpp | 66 +++-- .../odbc/src/diagnostic/diagnostic_record.cpp | 8 +- .../diagnostic/diagnostic_record_storage.cpp | 2 +- modules/platforms/cpp/odbc/src/dsn_config.cpp | 111 ++++++++ .../platforms/cpp/odbc/src/entry_points.cpp | 8 - modules/platforms/cpp/odbc/src/odbc.cpp | 83 ++---- .../cpp/odbc/src/protocol_version.cpp | 131 +++++++++ 42 files changed, 2822 insertions(+), 397 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolVersion.java create mode 100644 modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml create mode 100644 modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h create mode 100644 modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h create mode 100644 modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h create mode 100644 modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/custom_window.h create mode 100644 modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/window.h create mode 100644 modules/platforms/cpp/odbc/os/win/src/system/ui/custom_window.cpp create mode 100644 modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp create mode 100644 modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp create mode 100644 modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp create mode 100644 modules/platforms/cpp/odbc/src/dsn_config.cpp create mode 100644 modules/platforms/cpp/odbc/src/protocol_version.cpp diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java index 5e09041ed3044..2ffd8cdc95bb0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeRequest.java @@ -24,21 +24,21 @@ */ public class OdbcHandshakeRequest extends OdbcRequest { /** Protocol version. */ - private final long ver; + private final OdbcProtocolVersion ver; /** - * @param ver Protocol version. + * @param ver Long value for protocol version. */ public OdbcHandshakeRequest(long ver) { super(HANDSHAKE); - this.ver = ver; + this.ver = OdbcProtocolVersion.fromLong(ver); } /** * @return Protocol version. */ - public long version() { + public OdbcProtocolVersion version() { return ver; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java index bf1c61eee8794..74c5bd4ec08bb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcHandshakeResult.java @@ -17,7 +17,7 @@ package org.apache.ignite.internal.processors.odbc; -import org.jetbrains.annotations.Nullable; +import org.apache.ignite.internal.util.typedef.internal.S; /** * ODBC handshake result. @@ -33,11 +33,13 @@ public class OdbcHandshakeResult { private final String curVer; /** - * @param accepted Handshake accepted. + * Constructor. + * + * @param accepted Indicates whether handshake accepted or not. * @param protoVerSince Apache Ignite version when protocol version has been introduced. * @param curVer Current Apache Ignite version. */ - public OdbcHandshakeResult(boolean accepted, @Nullable String protoVerSince, @Nullable String curVer) { + public OdbcHandshakeResult(boolean accepted, String protoVerSince, String curVer) { this.accepted = accepted; this.protoVerSince = protoVerSince; this.curVer = curVer; @@ -53,14 +55,19 @@ public boolean accepted() { /** * @return Apache Ignite version when protocol version has been introduced. */ - @Nullable public String protoVerSince() { + public String protocolVersionSince() { return protoVerSince; } /** * @return Current Apache Ignite version. */ - @Nullable public String currentVer() { + public String currentVersion() { return curVer; } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(OdbcHandshakeResult.class, this); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java index fce8b1badba11..a751eb211a98f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java @@ -34,12 +34,6 @@ * ODBC message parser. */ public class OdbcMessageParser { - /** Current ODBC communication protocol version. */ - public static final long PROTO_VER = 1; - - /** Apache Ignite version when ODBC communication protocol version has been introduced. */ - public static final String PROTO_VER_SINCE = "1.6.0"; - /** Initial output stream capacity. */ private static final int INIT_CAP = 1024; @@ -82,10 +76,14 @@ public OdbcRequest decode(byte[] msg) { // we has not confirmed that the remote client uses the same protocol version. if (!verConfirmed) { if (cmd == OdbcRequest.HANDSHAKE) - return new OdbcHandshakeRequest(reader.readLong()); + { + long longVersion = reader.readLong(); + + return new OdbcHandshakeRequest(longVersion); + } else - throw new IgniteException("Unexpected ODBC command (first message is not a handshake request): [cmd=" + - cmd + ']'); + throw new IgniteException("Unexpected ODBC command " + + "(first message is not a handshake request): [cmd=" + cmd + ']'); } OdbcRequest res; @@ -174,6 +172,8 @@ public byte[] encode(OdbcResponse msg) { Object res0 = msg.response(); + if (res0 == null) + return writer.array(); if (res0 instanceof OdbcHandshakeResult) { OdbcHandshakeResult res = (OdbcHandshakeResult) res0; @@ -189,8 +189,8 @@ public byte[] encode(OdbcResponse msg) { } else { writer.writeBoolean(false); - writer.writeString(res.protoVerSince()); - writer.writeString(res.currentVer()); + writer.writeString(res.protocolVersionSince()); + writer.writeString(res.currentVersion()); } } else if (res0 instanceof OdbcQueryExecuteResult) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolVersion.java new file mode 100644 index 0000000000000..97a13064e2ac3 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProtocolVersion.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.internal.processors.odbc; + +import java.util.HashMap; +import java.util.Map; + +/** + * ODBC protocol version. + */ +public enum OdbcProtocolVersion { + /** First version of the ODBC. Released with Ignite 1.6 */ + VERSION_1_6_0(1), + + /** Unknown version. */ + VERSION_UNKNOWN(Long.MIN_VALUE); + + /** Long value to enum map. */ + private static final Map versions = new HashMap<>(); + + /** Enum value to Ignite version map */ + private static final Map since = new HashMap<>(); + + /** + * Map long values to version. + */ + static { + for (OdbcProtocolVersion version : values()) + versions.put(version.longValue(), version); + + since.put(VERSION_1_6_0, "1.6.0"); + } + + /** Long value for version. */ + private final long longVal; + + /** + * @param longVal Long value. + */ + OdbcProtocolVersion(long longVal) { + this.longVal = longVal; + } + + /** + * @param longVal Long value. + * @return Protocol version. + */ + public static OdbcProtocolVersion fromLong(long longVal) { + OdbcProtocolVersion res = versions.get(longVal); + + return res == null ? VERSION_UNKNOWN : res; + } + + /** + * @return Current version. + */ + public static OdbcProtocolVersion current() { + return VERSION_1_6_0; + } + + /** + * @return Long value. + */ + public long longValue() { + return longVal; + } + + /** + * @return {@code true} if this version is unknown. + */ + public boolean isUnknown() { + return longVal == VERSION_UNKNOWN.longVal; + } + + /** + * @return {@code true} if this version supports distributed joins. + */ + public boolean isDistributedJoinsSupported() { + assert !isUnknown(); + + return longVal >= VERSION_1_6_0.longVal; + } + + /** + * @return Ignite version when introduced. + */ + public String since() { + assert !isUnknown(); + + return since.get(this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java index ce98720e80f8d..3f7d505403668 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java @@ -91,7 +91,7 @@ public OdbcResponse handle(long reqId, OdbcRequest req) { try { switch (req.command()) { case HANDSHAKE: - return performHandshake(reqId, (OdbcHandshakeRequest)req); + return performHandshake((OdbcHandshakeRequest)req); case EXECUTE_SQL_QUERY: return executeQuery(reqId, (OdbcQueryExecuteRequest)req); @@ -119,23 +119,24 @@ public OdbcResponse handle(long reqId, OdbcRequest req) { /** * {@link OdbcHandshakeRequest} command handler. * - * @param reqId Request ID. * @param req Handshake request. * @return Response. */ - private OdbcResponse performHandshake(long reqId, OdbcHandshakeRequest req) { - OdbcHandshakeResult res; + private OdbcResponse performHandshake(OdbcHandshakeRequest req) { + OdbcProtocolVersion version = req.version(); - if (req.version() == OdbcMessageParser.PROTO_VER) - res = new OdbcHandshakeResult(true, null, null); - else { + if (version.isUnknown()) { IgniteProductVersion ver = ctx.grid().version(); String verStr = Byte.toString(ver.major()) + '.' + ver.minor() + '.' + ver.maintenance(); - res = new OdbcHandshakeResult(false, OdbcMessageParser.PROTO_VER_SINCE, verStr); + OdbcHandshakeResult res = new OdbcHandshakeResult(false, OdbcProtocolVersion.current().since(), verStr); + + return new OdbcResponse(res); } + OdbcHandshakeResult res = new OdbcHandshakeResult(true, null, null); + return new OdbcResponse(res); } diff --git a/modules/platforms/cpp/common/include/ignite/common/utils.h b/modules/platforms/cpp/common/include/ignite/common/utils.h index c1046e2788ecc..f4d2a9fcaae08 100644 --- a/modules/platforms/cpp/common/include/ignite/common/utils.h +++ b/modules/platforms/cpp/common/include/ignite/common/utils.h @@ -59,6 +59,13 @@ namespace ignite return res; } + /** + * Strips leading and trailing whitespaces from string. + * + * @param str String to be transformed. + */ + IGNITE_IMPORT_EXPORT void StripSurroundingWhitespaces(std::string& str); + /** * Get string representation of long in decimal form. * @@ -179,6 +186,24 @@ namespace ignite return res; } + + /** + * Check if the predicate returns true for all the elements of the + * sequence. + * + * @return True if the predicate returns true for all the elements + * of the sequence and false otherwise. + */ + template + bool AllOf(Iter begin, Iter end, Pred pred) + { + Iter i = begin; + + while (i != end && pred(*i)) + ++i; + + return i == end; + } } } diff --git a/modules/platforms/cpp/common/os/win/src/common/utils.cpp b/modules/platforms/cpp/common/os/win/src/common/utils.cpp index 47d7f4304169a..77c90b88efb38 100644 --- a/modules/platforms/cpp/common/os/win/src/common/utils.cpp +++ b/modules/platforms/cpp/common/os/win/src/common/utils.cpp @@ -40,6 +40,26 @@ namespace ignite return false; } + void StripSurroundingWhitespaces(std::string& str) + { + std::string::size_type newBegin = 0; + while (newBegin < str.size() && ::isspace(str[newBegin])) + ++newBegin; + + if (newBegin == str.size()) + { + str.clear(); + + return; + } + + std::string::size_type newEnd = str.size() - 1; + while (::isspace(str[newEnd])) + --newEnd; + + str.assign(str, newBegin, (newEnd - newBegin) + 1); + } + time_t IgniteTimeGm(const tm& time) { tm tmc = time; diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index de8fb5d08aae6..1ca85a74cdae8 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -75,6 +75,7 @@ ignite_odbc_tests_SOURCES = \ ../odbc/src/app/application_data_buffer.cpp \ ../odbc/src/config/configuration.cpp \ ../odbc/src/row.cpp \ + ../odbc/src/protocol_version.cpp \ ../odbc/src/column.cpp \ ../odbc/src/utility.cpp \ ../odbc/src/result_page.cpp diff --git a/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml new file mode 100644 index 0000000000000..db19669218329 --- /dev/null +++ b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml @@ -0,0 +1,103 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500 + + + + + + + + + diff --git a/modules/platforms/cpp/odbc-test/config/queries-test.xml b/modules/platforms/cpp/odbc-test/config/queries-test.xml index 67415fb6ff266..26e63417e0334 100644 --- a/modules/platforms/cpp/odbc-test/config/queries-test.xml +++ b/modules/platforms/cpp/odbc-test/config/queries-test.xml @@ -31,7 +31,9 @@ - + + + @@ -39,16 +41,17 @@ - - - + + + - + - @@ -79,8 +93,7 @@ instead os static IP based discovery. --> - - + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index a9423ca9bd0ea..cb5735f707097 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -157,6 +157,7 @@ + @@ -202,6 +203,7 @@ + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index 84f5a29a38114..270bdd624727c 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -76,6 +76,9 @@ Code + + Externals + Code @@ -122,5 +125,8 @@ Configs + + Configs + \ No newline at end of file diff --git a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp index 85aa3ff8fca49..0fd3277ed6613 100644 --- a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp @@ -20,51 +20,93 @@ #endif #include +#include #include #include +#include +#include using namespace ignite::odbc::config; namespace { - const char* testDriverName = "Ignite"; - const char* testServerHost = "testhost.com"; + const std::string testDriverName = "Ignite Driver"; + const std::string testServerHost = "testhost.com"; const uint16_t testServerPort = 4242; - const char* testCacheName = "TestCache"; - const char* testDsn = "Ignite DSN"; + const std::string testCacheName = "TestCache"; + const std::string testDsn = "Ignite DSN"; + + const std::string testAddress = testServerHost + ':' + ignite::common::LexicalCast(testServerPort); } -BOOST_AUTO_TEST_SUITE(ConfigurationTestSuite) +void CheckValidAddress(const char* connectStr, uint16_t port) +{ + Configuration cfg; + + BOOST_CHECK_NO_THROW(cfg.FillFromConnectString(connectStr)); + + BOOST_CHECK_EQUAL(cfg.GetTcpPort(), port); +} + +void CheckValidProtocolVersion(const char* connectStr, ignite::odbc::ProtocolVersion version) +{ + Configuration cfg; + + BOOST_CHECK_NO_THROW(cfg.FillFromConnectString(connectStr)); + + BOOST_CHECK(cfg.GetProtocolVersion() == version); +} + +void CheckInvalidProtocolVersion(const char* connectStr) +{ + Configuration cfg; + + cfg.FillFromConnectString(connectStr); + + BOOST_CHECK_THROW(cfg.GetProtocolVersion(), ignite::IgniteError); +} void CheckConnectionConfig(const Configuration& cfg) { - BOOST_REQUIRE(cfg.GetDriver() == testDriverName); - BOOST_REQUIRE(cfg.GetHost() == testServerHost); - BOOST_REQUIRE(cfg.GetPort() == testServerPort); - BOOST_REQUIRE(cfg.GetCache() == testCacheName); - BOOST_REQUIRE(cfg.GetDsn().empty()); + BOOST_CHECK_EQUAL(cfg.GetDriver(), testDriverName); + BOOST_CHECK_EQUAL(cfg.GetHost(), testServerHost); + BOOST_CHECK_EQUAL(cfg.GetTcpPort(), testServerPort); + BOOST_CHECK_EQUAL(cfg.GetAddress(), testAddress); + BOOST_CHECK_EQUAL(cfg.GetCache(), testCacheName); + BOOST_CHECK_EQUAL(cfg.GetDsn(), std::string()); std::stringstream constructor; - constructor << "driver={" << testDriverName << "};" - << "server=" << testServerHost << ";" - << "port=" << testServerPort << ";" - << "cache=" << testCacheName << ";"; + constructor << "address=" << testAddress << ';' + << "cache=" << testCacheName << ';' + << "driver={" << testDriverName << "};"; const std::string& expectedStr = constructor.str(); - BOOST_REQUIRE(cfg.ToConnectString() == expectedStr); + BOOST_CHECK_EQUAL(ignite::common::ToLower(cfg.ToConnectString()), ignite::common::ToLower(expectedStr)); } void CheckDsnConfig(const Configuration& cfg) { - BOOST_REQUIRE(cfg.GetDriver() == testDriverName); - BOOST_REQUIRE(cfg.GetDsn() == testDsn); - BOOST_REQUIRE(cfg.GetHost().empty()); - BOOST_REQUIRE(cfg.GetCache().empty()); - BOOST_REQUIRE(cfg.GetPort() == 0); + BOOST_CHECK_EQUAL(cfg.GetDriver(), testDriverName); + BOOST_CHECK_EQUAL(cfg.GetDsn(), testDsn); + BOOST_CHECK_EQUAL(cfg.GetCache(), Configuration::DefaultValue::cache); + BOOST_CHECK_EQUAL(cfg.GetAddress(), Configuration::DefaultValue::address); + BOOST_CHECK_EQUAL(cfg.GetHost(), std::string()); + BOOST_CHECK_EQUAL(cfg.GetTcpPort(), Configuration::DefaultValue::port); +} + +BOOST_AUTO_TEST_SUITE(ConfigurationTestSuite) + +BOOST_AUTO_TEST_CASE(CheckTestValuesNotEquealDefault) +{ + BOOST_CHECK_NE(testDriverName, Configuration::DefaultValue::driver); + BOOST_CHECK_NE(testAddress, Configuration::DefaultValue::address); + BOOST_CHECK_NE(testServerPort, Configuration::DefaultValue::port); + BOOST_CHECK_NE(testCacheName, Configuration::DefaultValue::cache); + BOOST_CHECK_NE(testDsn, Configuration::DefaultValue::dsn); } BOOST_AUTO_TEST_CASE(TestConnectStringUppercase) @@ -74,13 +116,12 @@ BOOST_AUTO_TEST_CASE(TestConnectStringUppercase) std::stringstream constructor; constructor << "DRIVER={" << testDriverName << "};" - << "SERVER=" << testServerHost <<";" - << "PORT=" << testServerPort << ";" + << "ADDRESS=" << testAddress << ';' << "CACHE=" << testCacheName; const std::string& connectStr = constructor.str(); - cfg.FillFromConnectString(connectStr.c_str(), connectStr.size()); + cfg.FillFromConnectString(connectStr); CheckConnectionConfig(cfg); } @@ -92,13 +133,12 @@ BOOST_AUTO_TEST_CASE(TestConnectStringLowercase) std::stringstream constructor; constructor << "driver={" << testDriverName << "};" - << "server=" << testServerHost << ";" - << "port=" << testServerPort << ";" - << "cache=" << testCacheName; + << "address=" << testAddress << ';' + << "cache=" << testCacheName << ';'; const std::string& connectStr = constructor.str(); - cfg.FillFromConnectString(connectStr.c_str(), connectStr.size()); + cfg.FillFromConnectString(connectStr); CheckConnectionConfig(cfg); } @@ -110,9 +150,8 @@ BOOST_AUTO_TEST_CASE(TestConnectStringZeroTerminated) std::stringstream constructor; constructor << "driver={" << testDriverName << "};" - << "server=" << testServerHost << ";" - << "port=" << testServerPort << ";" - << "cache=" << testCacheName; + << "address=" << testAddress << ';' + << "cache=" << testCacheName << ';'; const std::string& connectStr = constructor.str(); @@ -128,13 +167,12 @@ BOOST_AUTO_TEST_CASE(TestConnectStringMixed) std::stringstream constructor; constructor << "Driver={" << testDriverName << "};" - << "Server=" << testServerHost << ";" - << "Port=" << testServerPort << ";" - << "Cache=" << testCacheName; + << "Address=" << testAddress << ';' + << "Cache=" << testCacheName << ';'; const std::string& connectStr = constructor.str(); - cfg.FillFromConnectString(connectStr.c_str(), connectStr.size()); + cfg.FillFromConnectString(connectStr); CheckConnectionConfig(cfg); } @@ -146,17 +184,53 @@ BOOST_AUTO_TEST_CASE(TestConnectStringWhitepaces) std::stringstream constructor; constructor << "DRIVER = {" << testDriverName << "} ;\n" - << " SERVER =" << testServerHost << " ; \n" - << "PORT= " << testServerPort << "; " - << "CACHE = \n\r" << testCacheName; + << " ADDRESS =" << testAddress << "; " + << "CACHE = \n\r" << testCacheName << ';'; const std::string& connectStr = constructor.str(); - cfg.FillFromConnectString(connectStr.c_str(), connectStr.size()); + cfg.FillFromConnectString(connectStr); CheckConnectionConfig(cfg); } +BOOST_AUTO_TEST_CASE(TestConnectStringInvalidAddress) +{ + Configuration cfg; + + BOOST_CHECK_THROW(cfg.FillFromConnectString("Address=example.com:0;"), ignite::IgniteError); + BOOST_CHECK_THROW(cfg.FillFromConnectString("Address=example.com:00000;"), ignite::IgniteError); + BOOST_CHECK_THROW(cfg.FillFromConnectString("Address=example.com:fdsf;"), ignite::IgniteError); + BOOST_CHECK_THROW(cfg.FillFromConnectString("Address=example.com:123:1;"), ignite::IgniteError); + BOOST_CHECK_THROW(cfg.FillFromConnectString("Address=example.com:12322221;"), ignite::IgniteError); + BOOST_CHECK_THROW(cfg.FillFromConnectString("Address=example.com:12322a;"), ignite::IgniteError); + BOOST_CHECK_THROW(cfg.FillFromConnectString("Address=example.com:;"), ignite::IgniteError); +} + +BOOST_AUTO_TEST_CASE(TestConnectStringValidAddress) +{ + CheckValidAddress("Address=example.com:1;", 1); + CheckValidAddress("Address=example.com:31242;", 31242); + CheckValidAddress("Address=example.com:55555;", 55555); + CheckValidAddress("Address=example.com:110;", 110); + CheckValidAddress("Address=example.com;", Configuration::DefaultValue::port); +} + +BOOST_AUTO_TEST_CASE(TestConnectStringInvalidVersion) +{ + CheckInvalidProtocolVersion("Protocol_Version=0;"); + CheckInvalidProtocolVersion("Protocol_Version=1;"); + CheckInvalidProtocolVersion("Protocol_Version=2;"); + CheckInvalidProtocolVersion("Protocol_Version=1.6.1;"); + CheckInvalidProtocolVersion("Protocol_Version=1.7.0;"); + CheckInvalidProtocolVersion("Protocol_Version=1.8.1;"); +} + +BOOST_AUTO_TEST_CASE(TestConnectStringValidVersion) +{ + CheckValidProtocolVersion("Protocol_Version=1.6.0;", ignite::odbc::ProtocolVersion::VERSION_1_6_0); +} + BOOST_AUTO_TEST_CASE(TestDsnStringUppercase) { Configuration cfg; @@ -173,7 +247,7 @@ BOOST_AUTO_TEST_CASE(TestDsnStringUppercase) CheckDsnConfig(cfg); } -BOOST_AUTO_TEST_CASE(TestDsnStrinLowercase) +BOOST_AUTO_TEST_CASE(TestDsnStringLowercase) { Configuration cfg; @@ -189,7 +263,7 @@ BOOST_AUTO_TEST_CASE(TestDsnStrinLowercase) CheckDsnConfig(cfg); } -BOOST_AUTO_TEST_CASE(TestDsnStrinMixed) +BOOST_AUTO_TEST_CASE(TestDsnStringMixed) { Configuration cfg; @@ -205,7 +279,7 @@ BOOST_AUTO_TEST_CASE(TestDsnStrinMixed) CheckDsnConfig(cfg); } -BOOST_AUTO_TEST_CASE(TestDsnStrinWhitespaces) +BOOST_AUTO_TEST_CASE(TestDsnStringWhitespaces) { Configuration cfg; diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 2d9bd582d3e1b..4ba3a634daee0 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -24,6 +24,7 @@ #include #include +#include #ifndef _MSC_VER # define BOOST_TEST_DYN_LINK @@ -53,41 +54,12 @@ using ignite::impl::binary::BinaryUtils; struct QueriesTestSuiteFixture { /** - * Constructor. + * Establish connection to node. + * + * @param connectStr Connection string. */ - QueriesTestSuiteFixture() : testCache(0), env(NULL), dbc(NULL), stmt(NULL) + void Connect(const std::string& connectStr) { - IgniteConfiguration cfg; - - cfg.jvmOpts.push_back("-Xdebug"); - cfg.jvmOpts.push_back("-Xnoagent"); - cfg.jvmOpts.push_back("-Djava.compiler=NONE"); - cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005"); - cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError"); - -#ifdef IGNITE_TESTS_32 - cfg.jvmInitMem = 256; - cfg.jvmMaxMem = 768; -#else - cfg.jvmInitMem = 1024; - cfg.jvmMaxMem = 4096; -#endif - - char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH"); - - BOOST_REQUIRE(cfgPath != 0); - - cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml"); - - IgniteError err; - - grid = Ignition::Start(cfg, &err); - - if (err.GetCode() != IgniteError::IGNITE_SUCCESS) - BOOST_FAIL(err.GetText()); - - testCache = grid.GetCache("cache"); - // Allocate an environment handle SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env); @@ -102,13 +74,16 @@ struct QueriesTestSuiteFixture BOOST_REQUIRE(dbc != NULL); // Connect string - SQLCHAR connectStr[] = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=cache"; + std::vector connectStr0; + + connectStr0.reserve(connectStr.size() + 1); + std::copy(connectStr.begin(), connectStr.end(), std::back_inserter(connectStr0)); SQLCHAR outstr[ODBC_BUFFER_SIZE]; SQLSMALLINT outstrlen; // Connecting to ODBC server. - SQLRETURN ret = SQLDriverConnect(dbc, NULL, connectStr, static_cast(sizeof(connectStr)), + SQLRETURN ret = SQLDriverConnect(dbc, NULL, &connectStr0[0], static_cast(connectStr0.size()), outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE); if (!SQL_SUCCEEDED(ret)) @@ -124,10 +99,7 @@ struct QueriesTestSuiteFixture BOOST_REQUIRE(stmt != NULL); } - /** - * Destructor. - */ - ~QueriesTestSuiteFixture() + void Disconnect() { // Releasing statement handle. SQLFreeHandle(SQL_HANDLE_STMT, stmt); @@ -138,13 +110,67 @@ struct QueriesTestSuiteFixture // Releasing allocated handles. SQLFreeHandle(SQL_HANDLE_DBC, dbc); SQLFreeHandle(SQL_HANDLE_ENV, env); + } + + static Ignite StartNode(const char* name, const char* config) + { + IgniteConfiguration cfg; - Ignition::Stop(grid.GetName(), true); + cfg.jvmOpts.push_back("-Xdebug"); + cfg.jvmOpts.push_back("-Xnoagent"); + cfg.jvmOpts.push_back("-Djava.compiler=NONE"); + cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005"); + cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError"); + +#ifdef IGNITE_TESTS_32 + cfg.jvmInitMem = 256; + cfg.jvmMaxMem = 768; +#else + cfg.jvmInitMem = 1024; + cfg.jvmMaxMem = 4096; +#endif + + char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH"); + + BOOST_REQUIRE(cfgPath != 0); + + cfg.springCfgPath.assign(cfgPath).append("/").append(config); + + IgniteError err; + + return Ignition::Start(cfg, name); + } + + static Ignite StartAdditionalNode(const char* name) + { + return StartNode(name, "queries-test-noodbc.xml"); + } + + /** + * Constructor. + */ + QueriesTestSuiteFixture() : testCache(0), env(NULL), dbc(NULL), stmt(NULL) + { + grid = StartNode("NodeMain", "queries-test.xml"); + + testCache = grid.GetCache("cache"); + } + + /** + * Destructor. + */ + ~QueriesTestSuiteFixture() + { + Disconnect(); + + Ignition::StopAll(true); } template void CheckTwoRowsInt(SQLSMALLINT type) { + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); + SQLRETURN ret; TestType in1(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); @@ -252,6 +278,16 @@ struct QueriesTestSuiteFixture BOOST_FIXTURE_TEST_SUITE(QueriesTestSuite, QueriesTestSuiteFixture) +BOOST_AUTO_TEST_CASE(TestLegacyConnection) +{ + Connect("DRIVER={Apache Ignite};SERVER=127.0.0.1;PORT=11110;CACHE=cache"); +} + +BOOST_AUTO_TEST_CASE(TestConnectionProtocolVersion_1_6_0) +{ + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache;PROTOCOL_VERSION=1.6.0"); +} + BOOST_AUTO_TEST_CASE(TestTwoRowsInt8) { CheckTwoRowsInt(SQL_C_STINYINT); @@ -294,6 +330,8 @@ BOOST_AUTO_TEST_CASE(TestTwoRowsUint64) BOOST_AUTO_TEST_CASE(TestTwoRowsString) { + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); + SQLRETURN ret; TestType in1(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); @@ -387,6 +425,8 @@ BOOST_AUTO_TEST_CASE(TestTwoRowsString) BOOST_AUTO_TEST_CASE(TestOneRowString) { + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); + SQLRETURN ret; TestType in(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); @@ -448,6 +488,8 @@ BOOST_AUTO_TEST_CASE(TestOneRowString) BOOST_AUTO_TEST_CASE(TestOneRowStringLen) { + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); + SQLRETURN ret; TestType in(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); diff --git a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp index 16e5ea03ab091..69b4bfa49662b 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp @@ -45,7 +45,7 @@ namespace ignite char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH"); - BOOST_REQUIRE(cfgPath != 0) ; + BOOST_REQUIRE(cfgPath != 0); cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml"); @@ -54,14 +54,14 @@ namespace ignite grid = Ignition::Start(cfg, &err); if (err.GetCode() != IgniteError::IGNITE_SUCCESS) - BOOST_FAIL(err.GetText()) ; + BOOST_FAIL(err.GetText()) ; testCache = grid.GetCache("cache"); // Allocate an environment handle SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env); - BOOST_REQUIRE(env != NULL) ; + BOOST_REQUIRE(env != NULL); // We want ODBC 3 support SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast(SQL_OV_ODBC3), 0); @@ -69,10 +69,10 @@ namespace ignite // Allocate a connection handle SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc); - BOOST_REQUIRE(dbc != NULL) ; + BOOST_REQUIRE(dbc != NULL); // Connect string - SQLCHAR connectStr[] = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=cache"; + SQLCHAR connectStr[] = "DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"; SQLCHAR outstr[ODBC_BUFFER_SIZE]; SQLSMALLINT outstrlen; @@ -85,13 +85,13 @@ namespace ignite { Ignition::Stop(grid.GetName(), true); - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc)) ; + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc)); } // Allocate a statement handle SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt); - BOOST_REQUIRE(stmt != NULL) ; + BOOST_REQUIRE(stmt != NULL); } SqlTestSuiteFixture::~SqlTestSuiteFixture() diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am index 29f0ef48e9c7d..1781bc09ac4aa 100644 --- a/modules/platforms/cpp/odbc/Makefile.am +++ b/modules/platforms/cpp/odbc/Makefile.am @@ -62,6 +62,7 @@ libignite_odbc_la_SOURCES = \ src/meta/table_meta.cpp \ src/odbc.cpp \ src/entry_points.cpp \ + src/dsn_config.cpp \ src/query/column_metadata_query.cpp \ src/query/data_query.cpp \ src/query/foreign_keys_query.cpp \ @@ -69,6 +70,7 @@ libignite_odbc_la_SOURCES = \ src/query/table_metadata_query.cpp \ src/query/type_info_query.cpp \ src/query/special_columns_query.cpp \ + src/protocol_version.cpp \ src/result_page.cpp \ src/row.cpp \ src/column.cpp \ diff --git a/modules/platforms/cpp/odbc/include/Makefile.am b/modules/platforms/cpp/odbc/include/Makefile.am index 192021dc7764f..073dcaa1898ba 100644 --- a/modules/platforms/cpp/odbc/include/Makefile.am +++ b/modules/platforms/cpp/odbc/include/Makefile.am @@ -27,6 +27,7 @@ noinst_HEADERS = \ ignite/odbc/query/column_metadata_query.h \ ignite/odbc/query/query.h \ ignite/odbc/query/primary_keys_query.h \ + ignite/odbc/protocol_version.h \ ignite/odbc/statement.h \ ignite/odbc/config/configuration.h \ ignite/odbc/config/connection_info.h \ @@ -37,6 +38,7 @@ noinst_HEADERS = \ ignite/odbc/row.h \ ignite/odbc/utility.h \ ignite/odbc/environment.h \ + ignite/odbc/dsn_config.h \ ignite/odbc/system/odbc_constants.h \ ignite/odbc/system/socket_client.h \ ignite/odbc/meta/primary_key_meta.h \ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h index 6636ca4d28050..250eaf2c16c0c 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h @@ -60,6 +60,9 @@ namespace ignite /** Output data has been truncated. */ SQL_STATE_01004_DATA_TRUNCATED, + /** Invalid connection string attribute. */ + SQL_STATE_01S00_INVALID_CONNECTION_STRING_ATTRIBUTE, + /** Error in row. */ SQL_STATE_01S01_ERROR_IN_ROW, diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h index d6d794469ac5d..f90fa2d6a642f 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h @@ -23,6 +23,8 @@ #include #include +#include +#include "ignite/odbc/protocol_version.h" namespace ignite { @@ -36,6 +38,71 @@ namespace ignite class Configuration { public: + /** Map containing connect arguments. */ + typedef std::map ArgumentMap; + + /** Connection attribute keywords. */ + struct Key + { + /** Connection attribute keyword for DSN attribute. */ + static const std::string dsn; + + /** Connection attribute keyword for Driver attribute. */ + static const std::string driver; + + /** Connection attribute keyword for cache attribute. */ + static const std::string cache; + + /** Connection attribute keyword for address attribute. */ + static const std::string address; + + /** Connection attribute keyword for server attribute. */ + static const std::string server; + + /** Connection attribute keyword for port attribute. */ + static const std::string port; + + /** Connection attribute keyword for protocol version attribute. */ + static const std::string protocolVersion; + }; + + /** Default values for configuration. */ + struct DefaultValue + { + /** Default value for DSN attribute. */ + static const std::string dsn; + + /** Default value for Driver attribute. */ + static const std::string driver; + + /** Default value for cache attribute. */ + static const std::string cache; + + /** Default value for address attribute. */ + static const std::string address; + + /** Default value for server attribute. */ + static const std::string server; + + /** Default value for protocol version. */ + static const ProtocolVersion& protocolVersion; + + /** Default value for port attribute. */ + static const uint16_t port; + }; + + /** + * Connection end point structure. + */ + struct EndPoint + { + /** Remote host. */ + std::string host; + + /** TCP port. */ + uint16_t port; + }; + /** * Default constructor. */ @@ -81,11 +148,18 @@ namespace ignite * * @return Server port. */ - uint16_t GetPort() const + uint16_t GetTcpPort() const { - return port; + return endPoint.port; } + /** + * Set server port. + * + * @param port Server port. + */ + void SetTcpPort(uint16_t port); + /** * Get DSN. * @@ -93,7 +167,17 @@ namespace ignite */ const std::string& GetDsn() const { - return dsn; + return GetStringValue(Key::dsn, DefaultValue::dsn); + } + + /** + * Set DSN. + * + * @param dsn Data Source Name. + */ + void SetDsn(const std::string& dsn) + { + arguments[Key::dsn] = dsn; } /** @@ -103,7 +187,7 @@ namespace ignite */ const std::string& GetDriver() const { - return driver; + return GetStringValue(Key::driver, DefaultValue::driver); } /** @@ -113,7 +197,17 @@ namespace ignite */ const std::string& GetHost() const { - return host; + return endPoint.host; + } + + /** + * Set server host. + * + * @param server Server host. + */ + void SetHost(const std::string& server) + { + arguments[Key::server] = server; } /** @@ -123,15 +217,82 @@ namespace ignite */ const std::string& GetCache() const { - return cache; + return GetStringValue(Key::cache, DefaultValue::cache); } - private: - IGNITE_NO_COPY_ASSIGNMENT(Configuration); + /** + * Set cache. + * + * @param cache Cache name. + */ + void SetCache(const std::string& cache) + { + arguments[Key::cache] = cache; + } - /** Map containing connect arguments. */ - typedef std::map ArgumentMap; + /** + * Get address. + * + * @return Address. + */ + const std::string& GetAddress() const + { + return GetStringValue(Key::address, DefaultValue::address); + } + + /** + * Set address. + * + * @param address Address. + */ + void SetAddress(const std::string& address) + { + arguments[Key::address] = address; + } + /** + * Get argument map. + * + * @return Argument map. + */ + const ArgumentMap& GetMap() const + { + return arguments; + } + + /** + * Get protocol version. + * + * @return Protocol version. + */ + ProtocolVersion GetProtocolVersion() const; + + /** + * Set protocol version. + * + * @param version Version to set. + */ + void SetProtocolVersion(const std::string& version); + + /** + * Get string value from the config. + * + * @param key Configuration key. + * @param dflt Default value to be returned if there is no value stored. + * @return Found or default value. + */ + const std::string& GetStringValue(const std::string& key, const std::string& dflt) const; + + /** + * Get int value from the config. + * + * @param key Configuration key. + * @param dflt Default value to be returned if there is no value stored. + * @return Found or default value. + */ + int64_t GetIntValue(const std::string& key, int64_t dflt) const; + + private: /** * Parse connect string into key-value storage. * @@ -139,22 +300,22 @@ namespace ignite * @param len String length. * @param params Parsing result. */ - void ParseAttributeList(const char* str, size_t len, char delimeter, ArgumentMap& args) const; + static void ParseAttributeList(const char* str, size_t len, char delimeter, ArgumentMap& args); - /** Data Source Name. */ - std::string dsn; - - /** Driver name. */ - std::string driver; - - /** Server hostname. */ - std::string host; + /** + * Parse address and extract connection end-point. + * + * @throw IgniteException if address can not be parsed. + * @param address Address string to parse. + * @param res Result is placed here. + */ + static void ParseAddress(const std::string& address, EndPoint& res); - /** Port of the server. */ - uint16_t port; + /** Arguments. */ + ArgumentMap arguments; - /** Cache name. */ - std::string cache; + /** Connection end-point. */ + EndPoint endPoint; }; } diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h b/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h index 10ceb19ca949a..acf82ba95c44e 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/connection.h @@ -25,6 +25,7 @@ #include "ignite/odbc/parser.h" #include "ignite/odbc/system/socket_client.h" #include "ignite/odbc/config/connection_info.h" +#include "ignite/odbc/config/configuration.h" #include "ignite/odbc/diagnostic/diagnosable_adapter.h" namespace ignite @@ -40,15 +41,6 @@ namespace ignite { friend class Environment; public: - /** ODBC communication protocol version. */ - enum { PROTOCOL_VERSION = 1 }; - - /** - * Apache Ignite version when the current ODBC communication - * protocol version has been introduced. - */ - static const std::string PROTOCOL_VERSION_SINCE; - /** * Destructor. */ @@ -74,18 +66,16 @@ namespace ignite /** * Establish connection to ODBC server. * - * @param server Server (DSN). + * @param connectStr Connection string. */ - void Establish(const std::string& server); + void Establish(const std::string& connectStr); /** * Establish connection to ODBC server. * - * @param host Host. - * @param port Port. - * @param cache Cache name to connect to. + * @param cfg Configuration. */ - void Establish(const std::string& host, uint16_t port, const std::string& cache); + void Establish(const config::Configuration cfg); /** * Release established connection. @@ -123,6 +113,13 @@ namespace ignite */ const std::string& GetCache() const; + /** + * Get configuration. + * + * @return Connection configuration. + */ + const config::Configuration& GetConfiguration() const; + /** * Create diagnostic record associated with the Connection instance. * @@ -132,8 +129,8 @@ namespace ignite * @param columnNum Associated column number. * @return DiagnosticRecord associated with the instance. */ - diagnostic::DiagnosticRecord CreateStatusRecord(SqlState sqlState, - const std::string& message, int32_t rowNum = 0, int32_t columnNum = 0) const; + static diagnostic::DiagnosticRecord CreateStatusRecord(SqlState sqlState, + const std::string& message, int32_t rowNum = 0, int32_t columnNum = 0); /** * Synchronously send request message and receive response. @@ -172,21 +169,19 @@ namespace ignite * Establish connection to ODBC server. * Internal call. * - * @param server Server (DNS). + * @param connectStr Connection string. * @return Operation result. */ - SqlResult InternalEstablish(const std::string& server); + SqlResult InternalEstablish(const std::string& connectStr); /** * Establish connection to ODBC server. * Internal call. * - * @param host Host. - * @param port Port. - * @param cache Cache name to connect to. + * @param cfg Configuration. * @return Operation result. */ - SqlResult InternalEstablish(const std::string& host, uint16_t port, const std::string& cache); + SqlResult InternalEstablish(const config::Configuration cfg); /** * Release established connection. @@ -269,11 +264,11 @@ namespace ignite /** State flag. */ bool connected; - /** Cache name. */ - std::string cache; - /** Message parser. */ Parser parser; + + /** Configuration. */ + config::Configuration config; }; } } diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h index bfb4f4c01f28f..670e0aa535332 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h @@ -84,7 +84,7 @@ namespace ignite * * @return An informational message on the error or warning. */ - const std::string& GetMessage() const; + const std::string& GetMessageText() const; /** * Get connection name. diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h b/modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h new file mode 100644 index 0000000000000..dbad9b5b91e5f --- /dev/null +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#ifndef _IGNITE_ODBC_DSN_CONFIG +#define _IGNITE_ODBC_DSN_CONFIG + +#include "ignite/odbc/config/configuration.h" + +namespace ignite +{ + namespace odbc + { + /** + * Extract last setup error and throw it like IgniteError. + */ + void ThrowLastSetupError(); + + /** + * Add new string to the DSN file. + * + * @param dsn DSN name. + * @param key Key. + * @param value Value. + */ + void WriteDsnString(const char* dsn, const char* key, const char* value); + + /** + * Get string from the DSN file. + * + * @param dsn DSN name. + * @param key Key. + * @param dflt Default value. + * @return Value. + */ + std::string ReadDsnString(const char* dsn, const char* key, const char* dflt); + + /** + * Read DSN to fill the configuration. + * + * @param dsn DSN name. + * @param config Configuration. + */ + void ReadDsnConfiguration(const char* dsn, config::Configuration& config); + } +} + +#endif //_IGNITE_ODBC_DSN_CONFIG \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h b/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h index c19e08c10d8ae..a91af229bcb3d 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/parser.h @@ -42,9 +42,6 @@ namespace ignite /** Default initial size of operational memory. */ enum { DEFAULT_MEM_ALLOCATION = 4096 }; - /** ODBC communication protocol version. */ - enum { PROTOCOL_VERSION = 1 }; - /** * Constructor. */ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h b/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h new file mode 100644 index 0000000000000..8682119459e1c --- /dev/null +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h @@ -0,0 +1,168 @@ +/* + * 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. + */ + +#ifndef _IGNITE_ODBC_PROTOCOL_VERSION +#define _IGNITE_ODBC_PROTOCOL_VERSION + +#include + +#include +#include + +namespace ignite +{ + namespace odbc + { + /** Protocol version. */ + class ProtocolVersion + { + public: + /** String to version map type alias. */ + typedef std::map StringToVersionMap; + + /** Version to string map type alias. */ + typedef std::map VersionToStringMap; + + /** First version of the protocol that was introduced in Ignite 1.6.0. */ + static const ProtocolVersion VERSION_1_6_0; + + /** Unknown version of the protocol. */ + static const ProtocolVersion VERSION_UNKNOWN; + + /** + * Get string to version map. + * + * @return String to version map. + */ + static const StringToVersionMap& GetMap(); + + /** + * Get current version. + * + * @return Current version. + */ + static const ProtocolVersion& GetCurrent(); + + /** + * Parse string and extract protocol version. + * + * @throw IgniteException if version can not be parsed. + * @param version Version string to parse. + * @return Protocol version. + */ + static ProtocolVersion FromString(const std::string& version); + + /** + * Convert to string value. + * + * @throw IgniteException if version is unknow parsed. + * @param version Version string to parse. + * @return Protocol version. + */ + const std::string& ToString() const; + + /** + * Get int value. + * + * @return Integer value. + */ + int64_t GetIntValue() const; + + /** + * Check if the version is unknown. + * + * @return True if the version is unknown. + */ + bool IsUnknown() const; + + /** + * Comparison operator. + * + * @param val1 First value. + * @param val2 Second value. + * @return True if equal. + */ + friend bool operator==(const ProtocolVersion& val1, const ProtocolVersion& val2); + + /** + * Comparison operator. + * + * @param val1 First value. + * @param val2 Second value. + * @return True if not equal. + */ + friend bool operator!=(const ProtocolVersion& val1, const ProtocolVersion& val2); + + /** + * Comparison operator. + * + * @param val1 First value. + * @param val2 Second value. + * @return True if less. + */ + friend bool operator<(const ProtocolVersion& val1, const ProtocolVersion& val2); + + /** + * Comparison operator. + * + * @param val1 First value. + * @param val2 Second value. + * @return True if less or equal. + */ + friend bool operator<=(const ProtocolVersion& val1, const ProtocolVersion& val2); + + /** + * Comparison operator. + * + * @param val1 First value. + * @param val2 Second value. + * @return True if gretter. + */ + friend bool operator>(const ProtocolVersion& val1, const ProtocolVersion& val2); + + /** + * Comparison operator. + * + * @param val1 First value. + * @param val2 Second value. + * @return True if gretter or equal. + */ + friend bool operator>=(const ProtocolVersion& val1, const ProtocolVersion& val2); + + private: + /** + * Constructor. + * + * @param val Underlying value. + */ + explicit ProtocolVersion(int64_t val); + + ProtocolVersion(); + + /** String to version map. */ + static const StringToVersionMap stringToVersionMap; + + /** Version to string map. */ + static const VersionToStringMap versionToStringMap; + + /** Underlying int value. */ + int64_t val; + }; + } +} + +#endif //_IGNITE_ODBC_PROTOCOL_VERSION \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h b/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h index 60a65524df8e4..ecd1a5551e2d9 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h @@ -27,10 +27,6 @@ # undef min #endif // min -#ifdef GetMessage -# undef GetMessage -#endif // GetMessage - #endif //_WIN32 #define ODBCVER 0x0380 diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h b/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h new file mode 100644 index 0000000000000..034de82c5d2f6 --- /dev/null +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h @@ -0,0 +1,136 @@ +/* + * 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. + */ + +#ifndef _IGNITE_ODBC_SYSTEM_UI_DSN_CONFIGURATION_WINDOW +#define _IGNITE_ODBC_SYSTEM_UI_DSN_CONFIGURATION_WINDOW + +#include "ignite/odbc/config/configuration.h" +#include "ignite/odbc/system/ui/custom_window.h" + +namespace ignite +{ + namespace odbc + { + namespace system + { + namespace ui + { + /** + * DSN configuration window class. + */ + class DsnConfigurationWindow : public CustomWindow + { + /** + * Children windows ids. + */ + enum ChildId + { + ID_CONNECTION_SETTINGS_GROUP_BOX, + ID_NAME_EDIT, + ID_NAME_LABEL, + ID_ADDRESS_EDIT, + ID_ADDRESS_LABEL, + ID_CACHE_EDIT, + ID_CACHE_LABEL, + ID_OK_BUTTON, + ID_CANCEL_BUTTON + }; + + public: + /** + * Constructor. + * + * @param parent Parent window handle. + */ + explicit DsnConfigurationWindow(Window* parent, config::Configuration& config); + + /** + * Destructor. + */ + virtual ~DsnConfigurationWindow(); + + /** + * Create window in the center of the parent window. + */ + void Create(); + + /** + * @copedoc ignite::odbc::system::ui::CustomWindow::OnCreate + */ + virtual void OnCreate(); + + /** + * @copedoc ignite::odbc::system::ui::CustomWindow::OnMessage + */ + virtual bool OnMessage(UINT msg, WPARAM wParam, LPARAM lParam); + + private: + IGNITE_NO_COPY_ASSIGNMENT(DsnConfigurationWindow) + + /** + * Retrieves current values from the children and stores + * them to the specified configuration. + * + * @param cfg Configuration. + */ + void RetrieveParameters(config::Configuration& cfg) const; + + /** Window width. */ + int width; + + /** Window height. */ + int height; + + /** Connection settings group box. */ + std::auto_ptr connectionSettingsGroupBox; + + /** DSN name edit field label. */ + std::auto_ptr nameLabel; + + /** DSN name edit field. */ + std::auto_ptr nameEdit; + + /** DSN address edit field label. */ + std::auto_ptr addressLabel; + + /** DSN address edit field. */ + std::auto_ptr addressEdit; + + /** DSN cache edit field label. */ + std::auto_ptr cacheLabel; + + /** DSN cache edit field. */ + std::auto_ptr cacheEdit; + + /** Ok button. */ + std::auto_ptr okButton; + + /** Cancel button. */ + std::auto_ptr cancelButton; + + /** Configuration. */ + config::Configuration& config; + + /** Flag indicating whether OK option was selected. */ + bool accepted; + }; + } + } + } +} + +#endif //_IGNITE_ODBC_SYSTEM_UI_DSN_CONFIGURATION_WINDOW \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/custom_window.h b/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/custom_window.h new file mode 100644 index 0000000000000..1502c07026c3f --- /dev/null +++ b/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/custom_window.h @@ -0,0 +1,189 @@ +/* + * 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. + */ + +#ifndef _IGNITE_ODBC_SYSTEM_UI_CUSTOM_WINDOW +#define _IGNITE_ODBC_SYSTEM_UI_CUSTOM_WINDOW + +#include "ignite/odbc/system/ui/window.h" + +namespace ignite +{ + namespace odbc + { + namespace system + { + namespace ui + { + /** + * Application execution result. + */ + enum Result + { + RESULT_OK, + RESULT_CANCEL + }; + + /** + * Process UI messages in current thread. + * Blocks until quit message has been received. + * + * @param window Main window. + * @return Application execution result. + */ + Result ProcessMessages(Window& window); + + /** + * Window class. + */ + class CustomWindow : public Window + { + public: + /** + * Constructor. + * + * @param parent Parent window. + * @param className Window class name. + * @param title Window title. + * @param callback Event processing function. + */ + CustomWindow(Window* parent, const char* className, const char* title); + + /** + * Destructor. + */ + virtual ~CustomWindow(); + + /** + * Callback which is called upon receiving new message. + * Pure virtual. Should be defined by user. + * + * @param msg Message. + * @param wParam Word-sized parameter. + * @param lParam Long parameter. + * @return Should return true if the message has been + * processed by the handler and false otherwise. + */ + virtual bool OnMessage(UINT msg, WPARAM wParam, LPARAM lParam) = 0; + + /** + * Callback that is called upon window creation. + */ + virtual void OnCreate() = 0; + + /** + * Create child group box window. + * + * @param posX Position by X coordinate. + * @param posY Position by Y coordinate. + * @param sizeX Size by X coordinate. + * @param sizeY Size by Y coordinate. + * @param title Title. + * @param id ID to be assigned to the created window. + * @return Auto pointer containing new window. + */ + std::auto_ptr CreateGroupBox(int posX, int posY, + int sizeX, int sizeY, const char* title, int id); + + /** + * Create child label window. + * + * @param posX Position by X coordinate. + * @param posY Position by Y coordinate. + * @param sizeX Size by X coordinate. + * @param sizeY Size by Y coordinate. + * @param title Title. + * @param id ID to be assigned to the created window. + * @return Auto pointer containing new window. + */ + std::auto_ptr CreateLabel(int posX, int posY, + int sizeX, int sizeY, const char* title, int id); + + /** + * Create child Edit window. + * + * @param posX Position by X coordinate. + * @param posY Position by Y coordinate. + * @param sizeX Size by X coordinate. + * @param sizeY Size by Y coordinate. + * @param title Title. + * @param id ID to be assigned to the created window. + * @return Auto pointer containing new window. + */ + std::auto_ptr CreateEdit(int posX, int posY, + int sizeX, int sizeY, const char* title, int id, int style = 0); + + /** + * Create child button window. + * + * @param posX Position by X coordinate. + * @param posY Position by Y coordinate. + * @param sizeX Size by X coordinate. + * @param sizeY Size by Y coordinate. + * @param title Title. + * @param id ID to be assigned to the created window. + * @return Auto pointer containing new window. + */ + std::auto_ptr CreateButton(int posX, int posY, + int sizeX, int sizeY, const char* title, int id); + + /** + * Create child CheckBox window. + * + * @param posX Position by X coordinate. + * @param posY Position by Y coordinate. + * @param sizeX Size by X coordinate. + * @param sizeY Size by Y coordinate. + * @param title Title. + * @param id ID to be assigned to the created window. + * @return Auto pointer containing new window. + */ + std::auto_ptr CreateCheckBox(int posX, int posY, + int sizeX, int sizeY, const char* title, int id, bool state); + + /** + * Create child ComboBox window. + * + * @param posX Position by X coordinate. + * @param posY Position by Y coordinate. + * @param sizeX Size by X coordinate. + * @param sizeY Size by Y coordinate. + * @param title Title. + * @param id ID to be assigned to the created window. + * @return Auto pointer containing new window. + */ + std::auto_ptr CreateComboBox(int posX, int posY, + int sizeX, int sizeY, const char* title, int id); + private: + IGNITE_NO_COPY_ASSIGNMENT(CustomWindow) + + /** + * Static callback. + * + * @param hwnd Window handle. + * @param msg Message. + * @param wParam Word-sized parameter. + * @param lParam Long parameter. + * @return Operation result. + */ + static LRESULT CALLBACK WndProc(HWND hwnd, UINT msg, WPARAM wParam, LPARAM lParam); + }; + } + } + } +} + +#endif //_IGNITE_ODBC_SYSTEM_UI_CUSTOM_WINDOW \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/window.h b/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/window.h new file mode 100644 index 0000000000000..32a54b200eccd --- /dev/null +++ b/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/window.h @@ -0,0 +1,201 @@ +/* + * 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. + */ + +#ifndef _IGNITE_ODBC_SYSTEM_UI_WINDOW +#define _IGNITE_ODBC_SYSTEM_UI_WINDOW + +#include "ignite/odbc/utility.h" +#include "ignite/odbc/system/odbc_constants.h" + +namespace ignite +{ + namespace odbc + { + namespace system + { + namespace ui + { + /** + * Get handle for the current module. + * + * @return Handle for the current module. + */ + HINSTANCE GetHInstance(); + + /** + * Window class. + */ + class Window + { + public: + /** + * Constructor for a new window that is going to be created. + * + * @param parent Parent window handle. + * @param className Window class name. + * @param title Window title. + * @param callback Event processing function. + */ + Window(Window* parent, const char* className, const char* title); + + /** + * Constructor for the existing window. + * + * @param handle Window handle. + */ + Window(HWND handle); + + /** + * Destructor. + */ + virtual ~Window(); + + /** + * Create window. + * + * @param style Window style. + * @param posX Window x position. + * @param posY Window y position. + * @param width Window width. + * @param height Window height. + * @param id ID for child window. + */ + void Create(DWORD style, int posX, int posY, int width, int height, int id); + + /** + * Show window. + */ + void Show(); + + /** + * Update window. + */ + void Update(); + + /** + * Destroy window. + */ + void Destroy(); + + /** + * Get window handle. + * + * @return Window handle. + */ + HWND GetHandle() const + { + return handle; + } + + /** + * Get window text. + * + * @param text Text. + */ + void GetText(std::string& text) const; + + /** + * Set window text. + * + * @param text Text. + */ + void SetText(const std::string& text) const; + + /** + * Get CheckBox state. + * + * @param True if checked. + */ + bool IsChecked() const; + + /** + * Set CheckBox state. + * + * @param state True if checked. + */ + void SetChecked(bool state); + + /** + * Add string. + * + * @param str String. + */ + void AddString(const std::string& str); + + /** + * Set current ComboBox selection. + * + * @param idx List index. + */ + void SetSelection(int idx); + + /** + * Get current ComboBox selection. + * + * @return idx List index. + */ + int GetSelection() const; + + /** + * Set enabled. + * + * @param enabled Enable flag. + */ + void SetEnabled(bool enabled); + + /** + * Check if the window is enabled. + * + * @return True if enabled. + */ + bool IsEnabled() const; + + protected: + /** + * Set window handle. + * + * @param value Window handle. + */ + void SetHandle(HWND value) + { + handle = value; + } + + /** Window class name. */ + std::string className; + + /** Window title. */ + std::string title; + + /** Window handle. */ + HWND handle; + + /** Specifies whether window has been created by the thread and needs destruction. */ + bool created; + + /** Window parent. */ + Window* parent; + + private: + IGNITE_NO_COPY_ASSIGNMENT(Window) + }; + } + } + } +} + +#endif //_IGNITE_ODBC_SYSTEM_UI_WINDOW \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/os/win/src/system/ui/custom_window.cpp b/modules/platforms/cpp/odbc/os/win/src/system/ui/custom_window.cpp new file mode 100644 index 0000000000000..1e855a18701a4 --- /dev/null +++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/custom_window.cpp @@ -0,0 +1,184 @@ +/* + * 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. + */ + +#include + +#include "ignite/odbc/system/ui/custom_window.h" + +namespace ignite +{ + namespace odbc + { + namespace system + { + namespace ui + { + Result ProcessMessages(Window& window) + { + MSG msg; + + while (GetMessage(&msg, NULL, 0, 0) > 0) + { + if (!IsDialogMessage(window.GetHandle(), &msg)) + { + TranslateMessage(&msg); + + DispatchMessage(&msg); + } + } + + return static_cast(msg.wParam); + } + + LRESULT CALLBACK CustomWindow::WndProc(HWND hwnd, UINT msg, WPARAM wParam, LPARAM lParam) + { + CustomWindow* window = reinterpret_cast(GetWindowLongPtr(hwnd, GWLP_USERDATA)); + + switch (msg) + { + case WM_NCCREATE: + { + _ASSERT(lParam != NULL); + + CREATESTRUCT* createStruct = reinterpret_cast(lParam); + + LONG_PTR longSelfPtr = reinterpret_cast(createStruct->lpCreateParams); + + SetWindowLongPtr(hwnd, GWLP_USERDATA, longSelfPtr); + + return DefWindowProc(hwnd, msg, wParam, lParam); + } + + case WM_CREATE: + { + _ASSERT(window != NULL); + + window->SetHandle(hwnd); + + window->OnCreate(); + + return 0; + } + + default: + break; + } + + if (window && window->OnMessage(msg, wParam, lParam)) + return 0; + + return DefWindowProc(hwnd, msg, wParam, lParam); + } + + CustomWindow::CustomWindow(Window* parent, const char* className, const char* title) : + Window(parent, className, title) + { + WNDCLASS wcx; + + wcx.style = CS_HREDRAW | CS_VREDRAW; + wcx.lpfnWndProc = WndProc; + wcx.cbClsExtra = 0; + wcx.cbWndExtra = 0; + wcx.hInstance = GetHInstance(); + wcx.hIcon = NULL; + wcx.hCursor = LoadCursor(NULL, IDC_ARROW); + wcx.hbrBackground = (HBRUSH)COLOR_WINDOW; + wcx.lpszMenuName = NULL; + wcx.lpszClassName = className; + + + if (!RegisterClass(&wcx)) + { + std::stringstream buf; + + buf << "Can not register window class, error code: " << GetLastError(); + + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str()); + } + } + + CustomWindow::~CustomWindow() + { + UnregisterClass(className.c_str(), GetHInstance()); + } + + std::auto_ptr CustomWindow::CreateGroupBox(int posX, int posY, + int sizeX, int sizeY, const char* title, int id) + { + std::auto_ptr child(new Window(this, "Button", title)); + + child->Create(WS_CHILD | WS_VISIBLE | BS_GROUPBOX, posX, posY, sizeX, sizeY, id); + + return child; + } + + std::auto_ptr CustomWindow::CreateLabel(int posX, int posY, + int sizeX, int sizeY, const char* title, int id) + { + std::auto_ptr child(new Window(this, "Static", title)); + + child->Create(WS_CHILD | WS_VISIBLE, posX, posY, sizeX, sizeY, id); + + return child; + } + + std::auto_ptr CustomWindow::CreateEdit(int posX, int posY, + int sizeX, int sizeY, const char* title, int id, int style) + { + std::auto_ptr child(new Window(this, "Edit", title)); + + child->Create(WS_CHILD | WS_VISIBLE | WS_BORDER | ES_AUTOHSCROLL | WS_TABSTOP | style, + posX, posY, sizeX, sizeY, id); + + return child; + } + + std::auto_ptr CustomWindow::CreateButton(int posX, int posY, + int sizeX, int sizeY, const char* title, int id) + { + std::auto_ptr child(new Window(this, "Button", title)); + + child->Create(WS_CHILD | WS_VISIBLE | WS_TABSTOP, posX, posY, sizeX, sizeY, id); + + return child; + } + + std::auto_ptr CustomWindow::CreateCheckBox(int posX, int posY, + int sizeX, int sizeY, const char* title, int id, bool state) + { + std::auto_ptr child(new Window(this, "Button", title)); + + child->Create(WS_CHILD | WS_VISIBLE | BS_CHECKBOX, posX, posY, sizeX, sizeY, id); + + child->SetChecked(state); + + return child; + } + + std::auto_ptr CustomWindow::CreateComboBox(int posX, int posY, + int sizeX, int sizeY, const char * title, int id) + { + std::auto_ptr child(new Window(this, "Combobox", title)); + + child->Create(WS_CHILD | WS_VISIBLE | CBS_DROPDOWNLIST, posX, posY, sizeX, sizeY, id); + + return child; + } + } + } + } +} diff --git a/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp new file mode 100644 index 0000000000000..76132bd1a4b0b --- /dev/null +++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp @@ -0,0 +1,212 @@ +/* + * 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. + */ + +#include + +#include "ignite/odbc/system/ui/dsn_configuration_window.h" + +namespace ignite +{ + namespace odbc + { + namespace system + { + namespace ui + { + DsnConfigurationWindow::DsnConfigurationWindow(Window* parent, config::Configuration& config): + CustomWindow(parent, "IgniteConfigureDsn", "Configure Apache Ignite DSN"), + width(360), + height(160), + connectionSettingsGroupBox(), + nameLabel(), + nameEdit(), + addressLabel(), + addressEdit(), + cacheLabel(), + cacheEdit(), + okButton(), + cancelButton(), + config(config), + accepted(false) + { + // No-op. + } + + DsnConfigurationWindow::~DsnConfigurationWindow() + { + // No-op. + } + + void DsnConfigurationWindow::Create() + { + // Finding out parent position. + RECT parentRect; + GetWindowRect(parent->GetHandle(), &parentRect); + + // Positioning window to the center of parent window. + const int posX = parentRect.left + (parentRect.right - parentRect.left - width) / 2; + const int posY = parentRect.top + (parentRect.bottom - parentRect.top - height) / 2; + + RECT desiredRect = {posX, posY, posX + width, posY + height}; + AdjustWindowRect(&desiredRect, WS_BORDER | WS_CAPTION | WS_SYSMENU | WS_THICKFRAME, FALSE); + + Window::Create(WS_OVERLAPPED | WS_SYSMENU, desiredRect.left, desiredRect.top, + desiredRect.right - desiredRect.left, desiredRect.bottom - desiredRect.top, 0); + + if (!handle) + { + std::stringstream buf; + + buf << "Can not create window, error code: " << GetLastError(); + + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str()); + } + } + + void DsnConfigurationWindow::OnCreate() + { + int margin = 10; + int interval = 10; + + int labelSizeX = 80; + int labelPosX = margin + interval; + + int editSizeX = width - labelSizeX - 2 * margin - 3 * interval; + int editPosX = margin + labelSizeX + 2 * interval; + + int rowSize = 20; + int rowPos = margin + 2 * interval; + + int checkBoxSize = (editSizeX - interval) / 2; + + int sectionBegin = margin; + + const char* val = config.GetDsn().c_str(); + nameLabel = CreateLabel(labelPosX, rowPos, labelSizeX, rowSize, "DSN name:", ID_NAME_LABEL); + nameEdit = CreateEdit(editPosX, rowPos, editSizeX, rowSize, val, ID_NAME_EDIT); + + rowPos += interval + rowSize; + + val = config.GetAddress().c_str(); + addressLabel = CreateLabel(labelPosX, rowPos, labelSizeX, rowSize, "Address:", ID_ADDRESS_LABEL); + addressEdit = CreateEdit(editPosX, rowPos, editSizeX, rowSize, val, ID_ADDRESS_EDIT); + + rowPos += interval + rowSize; + + val = config.GetCache().c_str(); + cacheLabel = CreateLabel(labelPosX, rowPos, labelSizeX, rowSize, "Cache name:", ID_CACHE_LABEL); + cacheEdit = CreateEdit(editPosX, rowPos, editSizeX, rowSize, val, ID_CACHE_EDIT); + + rowPos += interval * 2 + rowSize; + + connectionSettingsGroupBox = CreateGroupBox(margin, sectionBegin, width - 2 * margin, + rowPos - interval - sectionBegin, "Connection settings", ID_CONNECTION_SETTINGS_GROUP_BOX); + + int buttonSizeX = 80; + int cancelPosX = width - margin - buttonSizeX; + int okPosX = cancelPosX - interval - buttonSizeX; + + rowSize = 25; + + okButton = CreateButton(okPosX, rowPos, buttonSizeX, rowSize, "Ok", ID_OK_BUTTON); + cancelButton = CreateButton(cancelPosX, rowPos, buttonSizeX, rowSize, "Cancel", ID_CANCEL_BUTTON); + } + + bool DsnConfigurationWindow::OnMessage(UINT msg, WPARAM wParam, LPARAM lParam) + { + switch (msg) + { + case WM_COMMAND: + { + switch (LOWORD(wParam)) + { + case ID_OK_BUTTON: + { + try + { + RetrieveParameters(config); + + accepted = true; + + PostMessage(GetHandle(), WM_CLOSE, 0, 0); + } + catch (IgniteError& err) + { + MessageBox(NULL, err.GetText(), "Error!", MB_ICONEXCLAMATION | MB_OK); + } + + break; + } + + case IDCANCEL: + case ID_CANCEL_BUTTON: + { + PostMessage(GetHandle(), WM_CLOSE, 0, 0); + + break; + } + + default: + return false; + } + + break; + } + + case WM_DESTROY: + { + PostQuitMessage(accepted ? RESULT_OK : RESULT_CANCEL); + + break; + } + + default: + return false; + } + + return true; + } + + void DsnConfigurationWindow::RetrieveParameters(config::Configuration& cfg) const + { + std::string dsn; + std::string address; + std::string cache; + + nameEdit->GetText(dsn); + addressEdit->GetText(address); + cacheEdit->GetText(cache); + + common::StripSurroundingWhitespaces(address); + common::StripSurroundingWhitespaces(dsn); + + LOG_MSG("Retriving arguments:\n"); + LOG_MSG("DSN: %s\n", dsn.c_str()); + LOG_MSG("Address: %s\n", address.c_str()); + LOG_MSG("Cache: %s\n", cache.c_str()); + + if (dsn.empty()) + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, "DSN name can not be empty."); + + cfg.SetDsn(dsn); + cfg.SetAddress(address); + cfg.SetCache(cache); + } + } + } + } +} diff --git a/modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp b/modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp new file mode 100644 index 0000000000000..1143f01321600 --- /dev/null +++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp @@ -0,0 +1,192 @@ +/* + * 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. + */ + +#include + +#include "ignite/odbc/system/ui/window.h" + +namespace ignite +{ + namespace odbc + { + namespace system + { + namespace ui + { + HINSTANCE GetHInstance() + { + HINSTANCE hInstance = GetModuleHandle(TARGET_MODULE_FULL_NAME); + + if (hInstance == NULL) + { + std::stringstream buf; + + buf << "Can not get hInstance for the module, error code: " << GetLastError(); + + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str()); + } + + return hInstance; + } + + Window::Window(Window* parent, const char* className, const char* title) : + className(className), + title(title), + handle(NULL), + created(false), + parent(parent) + { + // No-op. + } + + Window::Window(HWND handle) : + className(), + title(), + handle(handle), + created(false), + parent(0) + { + // No-op. + } + + Window::~Window() + { + if (created) + Destroy(); + } + + void Window::Create(DWORD style, int posX, int posY, int width, int height, int id) + { + if (handle) + { + std::stringstream buf; + + buf << "Window already created, error code: " << GetLastError(); + + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str()); + } + + handle = CreateWindow( + className.c_str(), + title.c_str(), + style, + posX, + posY, + width, + height, + parent ? parent->GetHandle() : NULL, + reinterpret_cast(id), + GetHInstance(), + this + ); + + if (!handle) + { + std::stringstream buf; + + buf << "Can not create window, error code: " << GetLastError(); + + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str()); + } + + created = true; + + HGDIOBJ hfDefault = GetStockObject(DEFAULT_GUI_FONT); + + SendMessage(GetHandle(), WM_SETFONT, (WPARAM)hfDefault, MAKELPARAM(FALSE, 0)); + } + + void Window::Show() + { + ShowWindow(handle, SW_SHOW); + } + + void Window::Update() + { + UpdateWindow(handle); + } + + void Window::Destroy() + { + if (handle) + DestroyWindow(handle); + + handle = NULL; + } + + void Window::GetText(std::string& text) const + { + int len = GetWindowTextLength(handle); + + if (len <= 0) + { + text.clear(); + + return; + } + + text.resize(len + 1); + + if (!GetWindowText(handle, &text[0], len + 1)) + text.clear(); + + text.resize(len); + } + + void Window::SetText(const std::string& text) const + { + SNDMSG(handle, WM_SETTEXT, 0, reinterpret_cast(text.c_str())); + } + + bool Window::IsChecked() const + { + return Button_GetCheck(handle) == BST_CHECKED; + } + + void Window::SetChecked(bool state) + { + Button_SetCheck(handle, state ? BST_CHECKED : BST_UNCHECKED); + } + + void Window::AddString(const std::string & str) + { + SNDMSG(handle, CB_ADDSTRING, 0, reinterpret_cast(str.c_str())); + } + + void Window::SetSelection(int idx) + { + SNDMSG(handle, CB_SETCURSEL, static_cast(idx), 0); + } + + int Window::GetSelection() const + { + return static_cast(SNDMSG(handle, CB_GETCURSEL, 0, 0)); + } + + void Window::SetEnabled(bool enabled) + { + EnableWindow(GetHandle(), enabled); + } + + bool Window::IsEnabled() const + { + return IsWindowEnabled(GetHandle()) != 0; + } + } + } + } +} diff --git a/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp b/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp new file mode 100644 index 0000000000000..f432a406d60c9 --- /dev/null +++ b/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp @@ -0,0 +1,218 @@ +/* + * 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. + */ + +#include "ignite/odbc/utility.h" +#include "ignite/odbc/system/odbc_constants.h" + +#include "ignite/odbc/dsn_config.h" +#include "ignite/odbc/system/ui/window.h" +#include "ignite/odbc/system/ui/dsn_configuration_window.h" + +using ignite::odbc::config::Configuration; + +/** + * Display configuration window for user to configure DSN. + * + * @param hwndParent Parent window handle. + * @param config Output configuration. + * @return True on success and false on fail. + */ +bool DisplayConfigureDsnWindow(HWND hwndParent, Configuration& config) +{ + using namespace ignite::odbc::system::ui; + + if (!hwndParent) + return false; + + try + { + Window parent(hwndParent); + + DsnConfigurationWindow window(&parent, config); + + window.Create(); + + window.Show(); + window.Update(); + + return ProcessMessages(window) == RESULT_OK; + } + catch (const ignite::IgniteError& err) + { + std::stringstream buf; + + buf << "Message: " << err.GetText() << ", Code: " << err.GetCode(); + + std::string message = buf.str(); + + MessageBox(NULL, message.c_str(), "Error!", MB_ICONEXCLAMATION | MB_OK); + + SQLPostInstallerError(err.GetCode(), err.GetText()); + } + + return false; +} + +/** + * Register DSN with specified configuration. + * + * @param config Configuration. + * @param driver Driver. + * @return True on success and false on fail. + */ +bool RegisterDsn(const Configuration& config, LPCSTR driver) +{ + using namespace ignite::odbc::config; + using ignite::common::LexicalCast; + + typedef Configuration::ArgumentMap ArgMap; + + const char* dsn = config.GetDsn().c_str(); + + try + { + if (!SQLWriteDSNToIni(dsn, driver)) + ignite::odbc::ThrowLastSetupError(); + + const ArgMap& map = config.GetMap(); + + std::set ignore; + + ignore.insert(Configuration::Key::dsn); + ignore.insert(Configuration::Key::driver); + + for (ArgMap::const_iterator it = map.begin(); it != map.end(); ++it) + { + const std::string& key = it->first; + const std::string& value = it->second; + + if (ignore.find(key) != ignore.end()) + continue; + + ignite::odbc::WriteDsnString(dsn, key.c_str(), value.c_str()); + } + + return true; + } + catch (ignite::IgniteError& err) + { + MessageBox(NULL, err.GetText(), "Error!", MB_ICONEXCLAMATION | MB_OK); + + SQLPostInstallerError(err.GetCode(), err.GetText()); + } + + return false; +} + +/** + * Unregister specified DSN. + * + * @param dsn DSN name. + * @return True on success and false on fail. + */ +bool UnregisterDsn(const char* dsn) +{ + try + { + if (!SQLRemoveDSNFromIni(dsn)) + ignite::odbc::ThrowLastSetupError(); + + return true; + } + catch (ignite::IgniteError& err) + { + MessageBox(NULL, err.GetText(), "Error!", MB_ICONEXCLAMATION | MB_OK); + + SQLPostInstallerError(err.GetCode(), err.GetText()); + } + + return false; +} + +BOOL INSTAPI ConfigDSN(HWND hwndParent, WORD req, LPCSTR driver, LPCSTR attributes) +{ + using namespace ignite::odbc; + + LOG_MSG("ConfigDSN called\n"); + + Configuration config; + + LOG_MSG("Attributes: %s\n", attributes); + + config.FillFromConfigAttributes(attributes); + + if (!SQLValidDSN(config.GetDsn().c_str())) + return FALSE; + + LOG_MSG("Driver: %s\n", driver); + LOG_MSG("Attributes: %s\n", attributes); + + LOG_MSG("DSN: %s\n", config.GetDsn().c_str()); + + switch (req) + { + case ODBC_ADD_DSN: + { + LOG_MSG("ODBC_ADD_DSN\n"); + + if (!DisplayConfigureDsnWindow(hwndParent, config)) + return FALSE; + + if (!RegisterDsn(config, driver)) + return FALSE; + + break; + } + + case ODBC_CONFIG_DSN: + { + LOG_MSG("ODBC_CONFIG_DSN\n"); + + std::string dsn = config.GetDsn(); + + Configuration loaded(config); + + ReadDsnConfiguration(dsn.c_str(), loaded); + + if (!DisplayConfigureDsnWindow(hwndParent, loaded)) + return FALSE; + + if (!RegisterDsn(loaded, driver)) + return FALSE; + + if (loaded.GetDsn() != dsn && !UnregisterDsn(dsn.c_str())) + return FALSE; + + break; + } + + case ODBC_REMOVE_DSN: + { + LOG_MSG("ODBC_REMOVE_DSN\n"); + + if (!UnregisterDsn(config.GetDsn().c_str())) + return FALSE; + + break; + } + + default: + return FALSE; + } + + return TRUE; +} \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj index 5820030d00cb1..348a11aa06dec 100644 --- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj +++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj @@ -93,8 +93,8 @@ Level3 Disabled false - $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win;$(ProjectDir)\..\..\src - _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;_DEBUG;%(PreprocessorDefinitions) + $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src + _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";_DEBUG;%(PreprocessorDefinitions) true @@ -107,8 +107,8 @@ Level3 Disabled false - $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win;$(ProjectDir)\..\..\src - _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;_DEBUG;%(PreprocessorDefinitions) + $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src + _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";_DEBUG;%(PreprocessorDefinitions) true @@ -123,8 +123,8 @@ true true false - $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win;$(ProjectDir)\..\..\src - _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;%(PreprocessorDefinitions) + $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src + _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";%(PreprocessorDefinitions) true @@ -141,8 +141,8 @@ true true false - $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win;$(ProjectDir)\..\..\src - _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;%(PreprocessorDefinitions) + $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src + _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";%(PreprocessorDefinitions) true @@ -153,7 +153,11 @@ + + + + @@ -165,11 +169,13 @@ + + @@ -200,12 +206,14 @@ + + @@ -219,8 +227,11 @@ + + + diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters index 6ca58e22a3ad8..58764e4043871 100644 --- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters +++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters @@ -26,6 +26,9 @@ {df33e506-b5d8-423f-bcc5-1825242a3e28} + + {ff144e89-0a10-42c3-97dd-d22bfdbc7abb} + @@ -115,6 +118,24 @@ Code + + Code\system\ui + + + Code\system\ui + + + Code\system\ui + + + Code\system + + + Code + + + Code + @@ -224,5 +245,20 @@ Code + + Code\system\ui + + + Code\system\ui + + + Code\system\ui + + + Code + + + Code + \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/src/config/configuration.cpp b/modules/platforms/cpp/odbc/src/config/configuration.cpp index 8d57dee938bd8..f40c74f104eb2 100644 --- a/modules/platforms/cpp/odbc/src/config/configuration.cpp +++ b/modules/platforms/cpp/odbc/src/config/configuration.cpp @@ -15,13 +15,14 @@ * limitations under the License. */ -#include - #include #include #include #include +#include "ignite/common/common.h" +#include "ignite/common/utils.h" + #include "ignite/odbc/utility.h" #include "ignite/odbc/config/configuration.h" @@ -31,50 +32,29 @@ namespace ignite { namespace config { - /** Default values for configuration. */ - namespace dflt - { - /** Default value for DSN attribute. */ - const std::string dsn = "Default Apache Ignite DSN"; + const std::string Configuration::Key::dsn = "dsn"; + const std::string Configuration::Key::driver = "driver"; + const std::string Configuration::Key::cache = "cache"; + const std::string Configuration::Key::address = "address"; + const std::string Configuration::Key::server = "server"; + const std::string Configuration::Key::port = "port"; + const std::string Configuration::Key::protocolVersion = "protocol_version"; - /** Default value for Driver attribute. */ - const std::string driver = "Apache Ignite"; + const std::string Configuration::DefaultValue::dsn = "Apache Ignite DSN"; + const std::string Configuration::DefaultValue::driver = "Apache Ignite"; + const std::string Configuration::DefaultValue::cache = ""; + const std::string Configuration::DefaultValue::address = ""; + const std::string Configuration::DefaultValue::server = ""; - /** Default value for host attribute. */ - const std::string host = "localhost"; + const uint16_t Configuration::DefaultValue::port = 10800; - /** Default value for port attribute. */ - const uint16_t port = 10800; + const ProtocolVersion& Configuration::DefaultValue::protocolVersion = ProtocolVersion::GetCurrent(); - /** Default value for cache attribute. */ - const std::string cache = ""; - } - - /** Connection attribute keywords. */ - namespace attrkey - { - /** Connection attribute keyword for DSN attribute. */ - const std::string dsn = "dsn"; - - /** Connection attribute keyword for Driver attribute. */ - const std::string driver = "driver"; - - /** Connection attribute keyword for server host attribute. */ - const std::string host = "server"; - - /** Connection attribute keyword for server port attribute. */ - const std::string port = "port"; - - /** Connection attribute keyword for cache attribute. */ - const std::string cache = "cache"; - } Configuration::Configuration() : - dsn(dflt::dsn), driver(dflt::driver), - host(dflt::host), port(dflt::port), - cache(dflt::cache) + arguments() { - // No-op. + ParseAddress(DefaultValue::address, endPoint); } Configuration::~Configuration() @@ -84,7 +64,11 @@ namespace ignite void Configuration::FillFromConnectString(const char* str, size_t len) { - ArgumentMap connect_attributes; + // Initializing map. + arguments.clear(); + + // Initializing DSN to empty string. + arguments[Key::dsn].clear(); // Ignoring terminating zero byte if present. // Some Driver Managers pass zero-terminated connection string @@ -92,39 +76,19 @@ namespace ignite if (len && !str[len - 1]) --len; - ParseAttributeList(str, len, ';', connect_attributes); - - ArgumentMap::const_iterator it; - - it = connect_attributes.find(attrkey::dsn); - if (it != connect_attributes.end()) - dsn = it->second; - else - dsn.clear(); - - it = connect_attributes.find(attrkey::driver); - if (it != connect_attributes.end()) - driver = it->second; - else - driver = dflt::driver; - - it = connect_attributes.find(attrkey::host); - if (it != connect_attributes.end()) - host = it->second; - else - host = dflt::host; - - it = connect_attributes.find(attrkey::port); - if (it != connect_attributes.end()) - port = atoi(it->second.c_str()); - else - port = dflt::port; + ParseAttributeList(str, len, ';', arguments); - it = connect_attributes.find(attrkey::cache); - if (it != connect_attributes.end()) - cache = it->second; + ArgumentMap::const_iterator it = arguments.find(Key::address); + if (it != arguments.end()) + { + // Parsing address. + ParseAddress(it->second, endPoint); + } else - cache = dflt::cache; + { + endPoint.host = GetStringValue(Key::server, DefaultValue::server); + endPoint.port = static_cast(GetIntValue(Key::port, DefaultValue::port)); + } } void Configuration::FillFromConnectString(const std::string& str) @@ -136,27 +100,27 @@ namespace ignite { std::stringstream connect_string_buffer; - if (!driver.empty()) - connect_string_buffer << attrkey::driver << "={" << driver << "};"; - - if (!host.empty()) - connect_string_buffer << attrkey::host << '=' << host << ';'; - - if (port) - connect_string_buffer << attrkey::port << '=' << port << ';'; + for (ArgumentMap::const_iterator it = arguments.begin(); it != arguments.end(); ++it) + { + const std::string& key = it->first; + const std::string& value = it->second; - if (!dsn.empty()) - connect_string_buffer << attrkey::dsn << '=' << dsn << ';'; + if (value.empty()) + continue; - if (!cache.empty()) - connect_string_buffer << attrkey::cache << '=' << cache << ';'; + if (value.find(' ') == std::string::npos) + connect_string_buffer << key << '=' << value << ';'; + else + connect_string_buffer << key << "={" << value << "};"; + } return connect_string_buffer.str(); } - void Configuration::FillFromConfigAttributes(const char * attributes) + void Configuration::FillFromConfigAttributes(const char* attributes) { - ArgumentMap config_attributes; + // Initializing map. + arguments.clear(); size_t len = 0; @@ -166,45 +130,74 @@ namespace ignite ++len; - ParseAttributeList(attributes, len, '\0', config_attributes); - - ArgumentMap::const_iterator it; + ParseAttributeList(attributes, len, '\0', arguments); - it = config_attributes.find(attrkey::dsn); - if (it != config_attributes.end()) - dsn = it->second; + ArgumentMap::const_iterator it = arguments.find(Key::address); + if (it != arguments.end()) + { + // Parsing address. + ParseAddress(it->second, endPoint); + } else - dsn = dflt::dsn; + { + endPoint.host = GetStringValue(Key::server, DefaultValue::server); + endPoint.port = static_cast(GetIntValue(Key::port, DefaultValue::port)); + } + } - it = config_attributes.find(attrkey::driver); - if (it != config_attributes.end()) - driver = it->second; - else - driver.clear(); + void Configuration::SetTcpPort(uint16_t port) + { + arguments[Key::port] = common::LexicalCast(port); + } - it = config_attributes.find(attrkey::host); - if (it != config_attributes.end()) - host = it->second; - else - host.clear(); + ProtocolVersion Configuration::GetProtocolVersion() const + { + ArgumentMap::const_iterator it = arguments.find(Key::protocolVersion); - it = config_attributes.find(attrkey::port); - if (it != config_attributes.end()) - port = atoi(it->second.c_str()); - else - port = 0; + if (it != arguments.end()) + return ProtocolVersion::FromString(it->second); - it = config_attributes.find(attrkey::cache); - if (it != config_attributes.end()) - cache = it->second; - else - cache.clear(); + return DefaultValue::protocolVersion; + } + + void Configuration::SetProtocolVersion(const std::string& version) + { + arguments[Key::protocolVersion] = version; + } + + const std::string& Configuration::GetStringValue(const std::string& key, const std::string& dflt) const + { + ArgumentMap::const_iterator it = arguments.find(common::ToLower(key)); + + if (it != arguments.end()) + return it->second; + + return dflt; } - void Configuration::ParseAttributeList(const char * str, size_t len, char delimeter, ArgumentMap & args) const + int64_t Configuration::GetIntValue(const std::string& key, int64_t dflt) const + { + ArgumentMap::const_iterator it = arguments.find(common::ToLower(key)); + + if (it != arguments.end()) + { + const std::string& val = it->second; + + if (!common::AllOf(val.begin(), val.end(), isdigit)) + IGNITE_ERROR_FORMATTED_1(IgniteError::IGNITE_ERR_GENERIC, + "Invalid argument value: Integer value is expected.", "key", key); + + return common::LexicalCast(val); + } + + return dflt; + } + + + + void Configuration::ParseAttributeList(const char * str, size_t len, char delimeter, ArgumentMap & args) { std::string connect_str(str, len); - args.clear(); while (!connect_str.empty()) { @@ -245,6 +238,51 @@ namespace ignite connect_str.erase(attr_begin - 1); } } + + void Configuration::ParseAddress(const std::string& address, EndPoint& res) + { + int64_t colonNum = std::count(address.begin(), address.end(), ':'); + + if (colonNum == 0) + { + res.host = address; + res.port = DefaultValue::port; + } + else if (colonNum == 1) + { + size_t pos = address.find(':'); + + if (pos == address.size() - 1) + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, + "Invalid address format: no port after colon"); + + res.host = address.substr(0, pos); + + std::string port = address.substr(pos + 1); + + if (!common::AllOf(port.begin(), port.end(), isdigit)) + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, + "Invalid address format: port can only contain digits"); + + int32_t intPort = common::LexicalCast(port); + + if (port.size() > sizeof("65535") - 1 || intPort > UINT16_MAX) + { + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, + "Invalid address format: Port value is too large," + " valid value should be in range from 1 to 65535"); + } + + if (intPort == 0) + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, + "Invalid address format: Port value can not be zero"); + + res.port = static_cast(intPort); + } + else + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, + "Invalid address format: too many colons"); + } } } } diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp index 24417596ae894..cffecdf5798b3 100644 --- a/modules/platforms/cpp/odbc/src/connection.cpp +++ b/modules/platforms/cpp/odbc/src/connection.cpp @@ -39,9 +39,11 @@ namespace ignite { namespace odbc { - const std::string Connection::PROTOCOL_VERSION_SINCE = "1.6.0"; - - Connection::Connection() : socket(), connected(false), cache(), parser() + Connection::Connection() : + socket(), + connected(false), + parser(), + config() { // No-op. } @@ -53,8 +55,8 @@ namespace ignite const config::ConnectionInfo& Connection::GetInfo() const { - // Connection info is the same for all connections now. - static config::ConnectionInfo info; + // Connection info is constant and the same for all connections now. + const static config::ConnectionInfo info; return info; } @@ -76,32 +78,38 @@ namespace ignite return res; } - void Connection::Establish(const std::string& server) + void Connection::Establish(const std::string& connectStr) { - IGNITE_ODBC_API_CALL(InternalEstablish(server)); + IGNITE_ODBC_API_CALL(InternalEstablish(connectStr)); } - SqlResult Connection::InternalEstablish(const std::string& server) + SqlResult Connection::InternalEstablish(const std::string& connectStr) { config::Configuration config; - if (server != config.GetDsn()) + try { - AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, "Unknown server."); + config.FillFromConnectString(connectStr); + } + catch (IgniteError& e) + { + AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, e.GetText()); return SQL_RESULT_ERROR; } - return InternalEstablish(config.GetHost(), config.GetPort(), config.GetCache()); + return InternalEstablish(config); } - void Connection::Establish(const std::string& host, uint16_t port, const std::string& cache) + void Connection::Establish(const config::Configuration cfg) { - IGNITE_ODBC_API_CALL(InternalEstablish(host, port, cache)); + IGNITE_ODBC_API_CALL(InternalEstablish(cfg)); } - SqlResult Connection::InternalEstablish(const std::string & host, uint16_t port, const std::string & cache) + SqlResult Connection::InternalEstablish(const config::Configuration cfg) { + config = cfg; + if (connected) { AddStatusRecord(SQL_STATE_08002_ALREADY_CONNECTED, "Already connected."); @@ -109,9 +117,7 @@ namespace ignite return SQL_RESULT_ERROR; } - this->cache = cache; - - connected = socket.Connect(host.c_str(), port); + connected = socket.Connect(cfg.GetHost().c_str(), cfg.GetTcpPort()); if (!connected) { @@ -262,11 +268,16 @@ namespace ignite const std::string& Connection::GetCache() const { - return cache; + return config.GetCache(); + } + + const config::Configuration& Connection::GetConfiguration() const + { + return config; } diagnostic::DiagnosticRecord Connection::CreateStatusRecord(SqlState sqlState, - const std::string& message, int32_t rowNum, int32_t columnNum) const + const std::string& message, int32_t rowNum, int32_t columnNum) { return diagnostic::DiagnosticRecord(sqlState, message, "", "", rowNum, columnNum); } @@ -296,7 +307,20 @@ namespace ignite SqlResult Connection::MakeRequestHandshake() { - HandshakeRequest req(PROTOCOL_VERSION); + int64_t protocolVersion = 0; + + try + { + protocolVersion = config.GetProtocolVersion().GetIntValue(); + } + catch (const IgniteError& err) + { + AddStatusRecord(SQL_STATE_01S00_INVALID_CONNECTION_STRING_ATTRIBUTE, err.GetText()); + + return SQL_RESULT_ERROR; + } + + HandshakeRequest req(protocolVersion); HandshakeResponse rsp; try @@ -330,7 +354,7 @@ namespace ignite constructor << "Node rejected handshake message. " << "Current node Apache Ignite version: " << rsp.CurrentVer() << ", " << "node protocol version introduced in version: " << rsp.ProtoVerSince() << ", " - << "driver protocol version introduced in version: " << PROTOCOL_VERSION_SINCE << "."; + << "driver protocol version introduced in version: " << config.GetProtocolVersion().ToString() << "."; AddStatusRecord(SQL_STATE_08001_CANNOT_CONNECT, constructor.str()); diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp index 568c125046b42..8553ee47d5188 100644 --- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp +++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp @@ -34,6 +34,9 @@ namespace /** SQL state 01004 constant. */ const std::string STATE_01004 = "01004"; + /** SQL state 01S00 constant. */ + const std::string STATE_01S00 = "01S00"; + /** SQL state 01S01 constant. */ const std::string STATE_01S01 = "01S01"; @@ -168,7 +171,7 @@ namespace ignite return ORIGIN_ISO_9075; } - const std::string& DiagnosticRecord::GetMessage() const + const std::string& DiagnosticRecord::GetMessageText() const { return message; } @@ -190,6 +193,9 @@ namespace ignite case SQL_STATE_01004_DATA_TRUNCATED: return STATE_01004; + case SQL_STATE_01S00_INVALID_CONNECTION_STRING_ATTRIBUTE: + return STATE_01S00; + case SQL_STATE_01S01_ERROR_IN_ROW: return STATE_01S01; diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp index 90c0a4fdefba4..99ef292534447 100644 --- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp +++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp @@ -190,7 +190,7 @@ namespace ignite case IGNITE_SQL_DIAG_STATUS_MESSAGE_TEXT: { - buffer.PutString(record.GetMessage()); + buffer.PutString(record.GetMessageText()); return SQL_RESULT_SUCCESS; } diff --git a/modules/platforms/cpp/odbc/src/dsn_config.cpp b/modules/platforms/cpp/odbc/src/dsn_config.cpp new file mode 100644 index 0000000000000..99635dcd4d928 --- /dev/null +++ b/modules/platforms/cpp/odbc/src/dsn_config.cpp @@ -0,0 +1,111 @@ +/* + * 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. + */ + +#include + +#include "ignite/odbc/utility.h" +#include "ignite/odbc/system/odbc_constants.h" + +#include "ignite/odbc/dsn_config.h" + +using ignite::odbc::config::Configuration; + +#define BUFFER_SIZE 1024 +#define CONFIG_FILE "ODBC.INI" + +namespace ignite +{ + namespace odbc + { + void ThrowLastSetupError() + { + DWORD code; + char msg[BUFFER_SIZE]; + + SQLInstallerError(1, &code, msg, sizeof(msg), NULL); + + std::stringstream buf; + + buf << "Message: \"" << msg << "\", Code: " << code; + + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str()); + } + + void WriteDsnString(const char* dsn, const char* key, const char* value) + { + if (!SQLWritePrivateProfileString(dsn, key, value, CONFIG_FILE)) + ThrowLastSetupError(); + } + + std::string ReadDsnString(const char* dsn, const std::string& key, const char* dflt) + { + char buf[BUFFER_SIZE]; + + memset(buf, 0, sizeof(buf)); + + SQLGetPrivateProfileString(dsn, key.c_str(), dflt, buf, sizeof(buf), CONFIG_FILE); + + return std::string(buf); + } + + int ReadDsnInt(const char* dsn, const std::string& key, int dflt) + { + char buf[BUFFER_SIZE]; + + memset(buf, 0, sizeof(buf)); + + std::string dflt0 = common::LexicalCast(dflt); + + SQLGetPrivateProfileString(dsn, key.c_str(), dflt0.c_str(), buf, sizeof(buf), CONFIG_FILE); + + return common::LexicalCast(buf); + } + + bool ReadDsnBool(const char* dsn, const std::string& key, bool dflt) + { + char buf[BUFFER_SIZE]; + + memset(buf, 0, sizeof(buf)); + + std::string dflt0 = dflt ? "true" : "false"; + + SQLGetPrivateProfileString(dsn, key.c_str(), dflt0.c_str(), buf, sizeof(buf), CONFIG_FILE); + + return std::string(buf) == "true"; + } + + void ReadDsnConfiguration(const char* dsn, Configuration& config) + { + std::string address = ReadDsnString(dsn, Configuration::Key::address, config.GetAddress().c_str()); + std::string server = ReadDsnString(dsn, Configuration::Key::server, config.GetHost().c_str()); + uint16_t port = ReadDsnInt(dsn, Configuration::Key::port, config.GetTcpPort()); + std::string cache = ReadDsnString(dsn, Configuration::Key::cache, config.GetCache().c_str()); + std::string version = ReadDsnString(dsn, Configuration::Key::protocolVersion, + config.GetProtocolVersion().ToString().c_str()); + + LOG_MSG("%d\n", __LINE__); + + config.SetAddress(address); + config.SetHost(server); + config.SetTcpPort(port); + config.SetCache(cache); + config.SetProtocolVersion(version); + + LOG_MSG("%d\n", __LINE__); + } + } +} \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/src/entry_points.cpp b/modules/platforms/cpp/odbc/src/entry_points.cpp index c8e78a5c80007..f6195e1168b44 100644 --- a/modules/platforms/cpp/odbc/src/entry_points.cpp +++ b/modules/platforms/cpp/odbc/src/entry_points.cpp @@ -19,14 +19,6 @@ #include "ignite/odbc/utility.h" -BOOL INSTAPI ConfigDSN(HWND hwndParent, - WORD req, - LPCSTR driver, - LPCSTR attributes) -{ - return ignite::ConfigDSN(hwndParent, req, driver, attributes); -} - SQLRETURN SQL_API SQLGetInfo(SQLHDBC conn, SQLUSMALLINT infoType, SQLPOINTER infoValue, diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp index 3b31f1dd8a2dd..fd35cbab6875e 100644 --- a/modules/platforms/cpp/odbc/src/odbc.cpp +++ b/modules/platforms/cpp/odbc/src/odbc.cpp @@ -28,61 +28,11 @@ #include "ignite/odbc/environment.h" #include "ignite/odbc/connection.h" #include "ignite/odbc/statement.h" +#include "ignite/odbc/dsn_config.h" #include "ignite/odbc.h" namespace ignite { - - BOOL ConfigDSN(HWND hwndParent, - WORD req, - LPCSTR driver, - LPCSTR attributes) - { - LOG_MSG("ConfigDSN called\n"); - - ignite::odbc::config::Configuration config; - - config.FillFromConfigAttributes(attributes); - - if (!SQLValidDSN(config.GetDsn().c_str())) - return SQL_FALSE; - - LOG_MSG("Driver: %s\n", driver); - LOG_MSG("Attributes: %s\n", attributes); - - LOG_MSG("DSN: %s\n", config.GetDsn().c_str()); - - switch (req) - { - case ODBC_ADD_DSN: - { - LOG_MSG("ODBC_ADD_DSN\n"); - - return SQLWriteDSNToIni(config.GetDsn().c_str(), driver); - } - - case ODBC_CONFIG_DSN: - { - LOG_MSG("ODBC_CONFIG_DSN\n"); - break; - } - - case ODBC_REMOVE_DSN: - { - LOG_MSG("ODBC_REMOVE_DSN\n"); - - return SQLRemoveDSNFromIni(config.GetDsn().c_str()); - } - - default: - { - return SQL_FALSE; - } - } - - return SQL_TRUE; - } - SQLRETURN SQLGetInfo(SQLHDBC conn, SQLUSMALLINT infoType, SQLPOINTER infoValue, @@ -306,10 +256,10 @@ namespace ignite SQLSMALLINT* outConnectionStringLen, SQLUSMALLINT driverCompletion) { - using ignite::odbc::Connection; - using ignite::odbc::diagnostic::DiagnosticRecordStorage; - using ignite::utility::SqlStringToString; - using ignite::utility::CopyStringToBuffer; + using odbc::Connection; + using odbc::diagnostic::DiagnosticRecordStorage; + using utility::SqlStringToString; + using utility::CopyStringToBuffer; UNREFERENCED_PARAMETER(windowHandle); @@ -323,18 +273,23 @@ namespace ignite std::string connectStr = SqlStringToString(inConnectionString, inConnectionStringLen); - ignite::odbc::config::Configuration config; + odbc::config::Configuration config; config.FillFromConnectString(connectStr); - connection->Establish(config.GetHost(), config.GetPort(), config.GetCache()); + std::string dsn = config.GetDsn(); + + if (!dsn.empty()) + odbc::ReadDsnConfiguration(dsn.c_str(), config); + + connection->Establish(config); const DiagnosticRecordStorage& diag = connection->GetDiagnosticRecords(); if (!diag.IsSuccessful()) return diag.GetReturnCode(); - std::string outConnectStr = config.ToConnectString(); + std::string outConnectStr = connection->GetConfiguration().ToConnectString(); size_t reslen = CopyStringToBuffer(outConnectStr, reinterpret_cast(outConnectionString), @@ -357,7 +312,7 @@ namespace ignite SQLSMALLINT authLen) { using ignite::odbc::Connection; - using ignite::odbc::diagnostic::DiagnosticRecordStorage; + using ignite::odbc::config::Configuration; using ignite::utility::SqlStringToString; LOG_MSG("SQLConnect called\n"); @@ -367,9 +322,13 @@ namespace ignite if (!connection) return SQL_INVALID_HANDLE; - std::string server = SqlStringToString(serverName, serverNameLen); + odbc::config::Configuration config; + + std::string dsn = SqlStringToString(serverName, serverNameLen); + + odbc::ReadDsnConfiguration(dsn.c_str(), config); - connection->Establish(server); + connection->Establish(config); return connection->GetDiagnosticRecords().GetReturnCode(); } @@ -1168,7 +1127,7 @@ namespace ignite SqlLen outResLen; ApplicationDataBuffer outBuffer(IGNITE_ODBC_C_TYPE_CHAR, msgBuffer, msgBufferLen, &outResLen); - outBuffer.PutString(record.GetMessage()); + outBuffer.PutString(record.GetMessageText()); *msgLen = static_cast(outResLen); diff --git a/modules/platforms/cpp/odbc/src/protocol_version.cpp b/modules/platforms/cpp/odbc/src/protocol_version.cpp new file mode 100644 index 0000000000000..818df888386e3 --- /dev/null +++ b/modules/platforms/cpp/odbc/src/protocol_version.cpp @@ -0,0 +1,131 @@ +/* + * 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. + */ +#include +#include +#include + +#include "ignite/odbc/protocol_version.h" +#include "ignite/odbc/utility.h" + +namespace ignite +{ + namespace odbc + { + const ProtocolVersion ProtocolVersion::VERSION_1_6_0(1); + const ProtocolVersion ProtocolVersion::VERSION_UNKNOWN(INT64_MIN); + + ProtocolVersion::StringToVersionMap::value_type s2vInitVals[] = { + std::make_pair("1.6.0", ProtocolVersion::VERSION_1_6_0) + }; + + const ProtocolVersion::StringToVersionMap ProtocolVersion::stringToVersionMap(s2vInitVals, + s2vInitVals + (sizeof(s2vInitVals) / sizeof(s2vInitVals[0]))); + + ProtocolVersion::VersionToStringMap::value_type v2sInitVals[] = { + std::make_pair(ProtocolVersion::VERSION_1_6_0, "1.6.0") + }; + + const ProtocolVersion::VersionToStringMap ProtocolVersion::versionToStringMap(v2sInitVals, + v2sInitVals + (sizeof(v2sInitVals) / sizeof(v2sInitVals[0]))); + + ProtocolVersion::ProtocolVersion(int64_t val) : + val(val) + { + // No-op. + } + + const ProtocolVersion::StringToVersionMap& ProtocolVersion::GetMap() + { + return stringToVersionMap; + } + + const ProtocolVersion& ProtocolVersion::GetCurrent() + { + return VERSION_1_6_0; + } + + ProtocolVersion ProtocolVersion::FromString(const std::string& version) + { + StringToVersionMap::const_iterator it = stringToVersionMap.find(common::ToLower(version)); + + if (it == stringToVersionMap.end()) + { + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, + "Invalid version format. Valid format is X.Y.Z, where X, Y and Z are major " + "and minor versions and revision of Ignite since which protocol is introduced."); + } + + return it->second; + } + + const std::string& ProtocolVersion::ToString() const + { + VersionToStringMap::const_iterator it = versionToStringMap.find(*this); + + if (it == versionToStringMap.end()) + { + throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, + "Unknown protocol version can not be converted to string."); + } + + return it->second; + } + + int64_t ProtocolVersion::GetIntValue() const + { + assert(!IsUnknown()); + + return val; + } + + bool ProtocolVersion::IsUnknown() const + { + return *this == VERSION_UNKNOWN; + } + + bool operator==(const ProtocolVersion& val1, const ProtocolVersion& val2) + { + return val1.val == val2.val; + } + + bool operator!=(const ProtocolVersion& val1, const ProtocolVersion& val2) + { + return val1.val != val2.val; + } + + bool operator<(const ProtocolVersion& val1, const ProtocolVersion& val2) + { + return val1.val < val2.val; + } + + bool operator<=(const ProtocolVersion& val1, const ProtocolVersion& val2) + { + return val1.val <= val2.val; + } + + bool operator>(const ProtocolVersion& val1, const ProtocolVersion& val2) + { + return val1.val > val2.val; + } + + bool operator>=(const ProtocolVersion& val1, const ProtocolVersion& val2) + { + return val1.val >= val2.val; + } + } +} + From ae7765329fd6f7d50d13183d13626f39c5682334 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 2 Sep 2016 18:01:12 +0300 Subject: [PATCH 079/487] IGNITE-2208 - Queries with object arguments doesn't work wth BinaryMarshaller. --- .../internal/binary/BinaryMarshaller.java | 7 + .../processors/cache/IgniteCacheProxy.java | 41 ++ .../cache/query/GridCacheSqlQuery.java | 11 +- .../GridCacheQueryTransformerSelfTest.java | 9 +- .../query/h2/opt/GridH2ValueCacheObject.java | 9 - ...teBinaryObjectLocalQueryArgumentsTest.java | 28 ++ ...yObjectQueryArgumentsOffheapLocalTest.java | 28 ++ ...BinaryObjectQueryArgumentsOffheapTest.java | 30 ++ .../IgniteBinaryObjectQueryArgumentsTest.java | 469 +++++++++++++++++- .../query/h2/sql/GridQueryParsingTest.java | 4 +- .../IgniteCacheQuerySelfTestSuite.java | 9 + 11 files changed, 610 insertions(+), 35 deletions(-) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectLocalQueryArgumentsTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapLocalTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java index 29a1fcacd1856..39015e5312e69 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java @@ -136,4 +136,11 @@ private void setBinaryContext(BinaryContext ctx, IgniteConfiguration cfg) { @Override public void onUndeploy(ClassLoader ldr) { impl.context().onUndeploy(ldr); } + + /** + * @return GridBinaryMarshaller instance. + */ + public GridBinaryMarshaller binaryMarshaller() { + return impl; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java index 9b26c1dec94c3..8b2e6058c7f8a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java @@ -63,6 +63,7 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.binary.BinaryUtils; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.query.CacheQuery; import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture; @@ -688,6 +689,8 @@ private QueryCursor> queryContinuous(ContinuousQuery qry, bool validate(qry); + convertToBinary(qry); + final CacheOperationContext opCtxCall = ctx.operationContextPerCall(); if (qry instanceof ContinuousQuery) @@ -763,6 +766,44 @@ private QueryCursor> queryContinuous(ContinuousQuery qry, bool } } + /** + * Convert query arguments to BinaryObjects if binary marshaller used. + * + * @param qry Query. + */ + private void convertToBinary(final Query qry) { + if (ctx.binaryMarshaller()) { + if (qry instanceof SqlQuery) { + final SqlQuery sqlQry = (SqlQuery) qry; + + convertToBinary(sqlQry.getArgs()); + } else if (qry instanceof SpiQuery) { + final SpiQuery spiQry = (SpiQuery) qry; + + convertToBinary(spiQry.getArgs()); + } else if (qry instanceof SqlFieldsQuery) { + final SqlFieldsQuery fieldsQry = (SqlFieldsQuery) qry; + + convertToBinary(fieldsQry.getArgs()); + } + } + } + + /** + * Converts query arguments to BinaryObjects if binary marshaller used. + * + * @param args Arguments. + */ + private void convertToBinary(final Object[] args) { + if (args == null) + return; + + for (int i = 0; i < args.length; i++) { + if (args[i] != null && !BinaryUtils.isBinaryType(args[i].getClass())) + args[i] = ctx.toCacheObject(args[i]); + } + } + /** * @return {@code true} If this is a replicated cache and we are on a data node. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java index 07338278b627f..bcb37c5ec8cc7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java @@ -22,6 +22,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.GridDirectTransient; import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; @@ -153,7 +154,13 @@ public void unmarshallParams(Marshaller m, GridKernalContext ctx) throws IgniteC assert paramsBytes != null; - params = m.unmarshal(paramsBytes, U.resolveClassLoader(ctx.config())); + final ClassLoader ldr = U.resolveClassLoader(ctx.config()); + + if (m instanceof BinaryMarshaller) + // To avoid deserializing of enum types. + params = ((BinaryMarshaller)m).binaryMarshaller().unmarshal(paramsBytes, ldr); + else + params = m.unmarshal(paramsBytes, ldr); } /** {@inheritDoc} */ @@ -271,4 +278,4 @@ public GridCacheSqlQuery copy(Object[] args) { return cp; } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java index 6b13e05988b4c..e7e173bf5c4fc 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryTransformerSelfTest.java @@ -66,9 +66,14 @@ public class GridCacheQueryTransformerSelfTest extends GridCommonAbstractTest { @Override protected void beforeTestsStarted() throws Exception { startGridsMultiThreaded(3); - Ignition.setClientMode(true); + try { + Ignition.setClientMode(true); - startGrid(); + startGrid(); + } + finally { + Ignition.setClientMode(false); + } } /** {@inheritDoc} */ diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java index 29f967578606a..fd0e6ed334152 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java @@ -21,8 +21,6 @@ import java.sql.SQLException; import java.sql.Types; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.binary.BinaryObject; -import org.apache.ignite.internal.binary.BinaryEnumObjectImpl; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.CacheObjectContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; @@ -146,13 +144,6 @@ private CacheObjectContext objectContext() { return c1.compareTo(o2); } - if (o1 instanceof BinaryEnumObjectImpl && o2 instanceof Enum) { - final BinaryEnumObjectImpl bo1 = (BinaryEnumObjectImpl)o1; - - if (bo1.isTypeEquals(o2.getClass())) - return Integer.compare(bo1.enumOrdinal(), ((Enum)o2).ordinal()); - } - // Group by types. if (o1.getClass() != o2.getClass()) { if (o1Comparable != o2Comparable) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectLocalQueryArgumentsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectLocalQueryArgumentsTest.java new file mode 100644 index 0000000000000..7e35e51388c68 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectLocalQueryArgumentsTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +/** + * + */ +public class IgniteBinaryObjectLocalQueryArgumentsTest extends IgniteBinaryObjectQueryArgumentsTest { + /** {@inheritDoc} */ + @Override protected boolean isLocal() { + return true; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapLocalTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapLocalTest.java new file mode 100644 index 0000000000000..560d2580b49df --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapLocalTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +/** + * + */ +public class IgniteBinaryObjectQueryArgumentsOffheapLocalTest extends IgniteBinaryObjectQueryArgumentsOffheapTest { + /** {@inheritDoc} */ + @Override protected boolean isLocal() { + return true; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapTest.java new file mode 100644 index 0000000000000..d1428ae18a14f --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsOffheapTest.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; + +import org.apache.ignite.cache.CacheMemoryMode; + +/** + * + */ +public class IgniteBinaryObjectQueryArgumentsOffheapTest extends IgniteBinaryObjectQueryArgumentsTest { + /** {@inheritDoc} */ + @Override protected CacheMemoryMode memoryMode() { + return CacheMemoryMode.OFFHEAP_TIERED; + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java index 5676ddd0ec26d..8a0c5c89adb22 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java @@ -17,15 +17,27 @@ package org.apache.ignite.internal.processors.cache; -import java.util.Arrays; +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; import javax.cache.Cache; -import org.apache.ignite.IgniteBinary; + import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMemoryMode; import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; @@ -43,12 +55,64 @@ public class IgniteBinaryObjectQueryArgumentsTest extends GridCommonAbstractTest /** */ private static final int NODES = 3; + /** */ + public static final String PRIM_CACHE = "prim-cache"; + + /** */ + public static final String STR_CACHE = "str-cache"; + + /** */ + public static final String ENUM_CACHE = "enum-cache"; + + /** */ + public static final String UUID_CACHE = "uuid-cache"; + + /** */ + public static final String DATE_CACHE = "date-cache"; + + /** */ + public static final String TIMESTAMP_CACHE = "timestamp-cache"; + + /** */ + public static final String BIG_DECIMAL_CACHE = "decimal-cache"; + + /** */ + public static final String OBJECT_CACHE = "obj-cache"; + + /** */ + public static final String FIELD_CACHE = "field-cache"; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + cfg.setCacheConfiguration(getCacheConfigurations()); + + cfg.setMarshaller(null); + + return cfg; + } + + /** + * @return {@code True} If query is local. + */ + protected boolean isLocal() { + return false; + } + + /** + * @return Memory mode. + */ + protected CacheMemoryMode memoryMode() { + return CacheMemoryMode.ONHEAP_TIERED; + } + + /** + * @return Cache config. + */ + protected CacheConfiguration getCacheConfiguration(final String cacheName) { CacheConfiguration ccfg = new CacheConfiguration(); ccfg.setWriteSynchronizationMode(FULL_SYNC); @@ -57,11 +121,64 @@ public class IgniteBinaryObjectQueryArgumentsTest extends GridCommonAbstractTest person.setValueType(Person.class.getName()); person.addQueryField("name", String.class.getName(), null); - ccfg.setQueryEntities(Arrays.asList(person)); + ccfg.setQueryEntities(Collections.singletonList(person)); - cfg.setCacheConfiguration(ccfg); + ccfg.setMemoryMode(memoryMode()); - cfg.setMarshaller(null); + ccfg.setName(cacheName); + + return ccfg; + } + + /** + * @return Cache configurations. + */ + private CacheConfiguration[] getCacheConfigurations() { + final ArrayList ccfgs = new ArrayList<>(); + + ccfgs.add(getCacheConfiguration(OBJECT_CACHE)); + ccfgs.addAll(getCacheConfigurations(STR_CACHE, String.class, Person.class)); + ccfgs.addAll(getCacheConfigurations(PRIM_CACHE, Integer.class, Person.class)); + ccfgs.addAll(getCacheConfigurations(ENUM_CACHE, EnumKey.class, Person.class)); + ccfgs.addAll(getCacheConfigurations(UUID_CACHE, UUID.class, Person.class)); + ccfgs.addAll(getCacheConfigurations(DATE_CACHE, Date.class, Person.class)); + ccfgs.addAll(getCacheConfigurations(TIMESTAMP_CACHE, Timestamp.class, Person.class)); + ccfgs.addAll(getCacheConfigurations(BIG_DECIMAL_CACHE, BigDecimal.class, Person.class)); + ccfgs.add(getCacheConfiguration(FIELD_CACHE, Integer.class, SearchValue.class)); + + return ccfgs.toArray(new CacheConfiguration[ccfgs.size()]); + } + + /** + * + * @param cacheName Cache name. + * @param key Key type. + * @param val Value type. + * @return Configurations. + */ + private List getCacheConfigurations(final String cacheName, final Class key, final Class val) { + final List res = new ArrayList<>(); + + res.add(getCacheConfiguration(cacheName, key, val)); + res.add(getCacheConfiguration(cacheName + "-val", val, key)); + + return res; + } + + /** + * @param cacheName Cache name. + * @param key Key type. + * @param val Value type + * @return Configuration. + */ + @SuppressWarnings("unchecked") + private CacheConfiguration getCacheConfiguration(final String cacheName, final Class key, final Class val) { + CacheConfiguration cfg = new CacheConfiguration(); + + cfg.setName(cacheName); + + cfg.setMemoryMode(memoryMode()); + cfg.setIndexedTypes(key, val); return cfg; } @@ -70,7 +187,9 @@ public class IgniteBinaryObjectQueryArgumentsTest extends GridCommonAbstractTest @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); - startGridsMultiThreaded(NODES); + final int nodes = isLocal() ? 1 : NODES; + + startGridsMultiThreaded(nodes); } /** {@inheritDoc} */ @@ -84,31 +203,210 @@ public class IgniteBinaryObjectQueryArgumentsTest extends GridCommonAbstractTest * @throws Exception If failed. */ public void testObjectArgument() throws Exception { - IgniteCache cache = ignite(0).cache(null); + testKeyQuery(OBJECT_CACHE, new TestKey(1), new TestKey(2)); + } - for (int i = 0; i < 100; i++) - cache.put(new TestKey(i), new Person("name-" + i)); + /** + * @throws Exception If failed. + */ + public void testPrimitiveObjectArgument() throws Exception { + testKeyValQuery(PRIM_CACHE, 1, 2); + } - SqlQuery qry = new SqlQuery<>(Person.class, "where _key=?"); + /** + * @throws Exception If failed. + */ + public void testStringObjectArgument() throws Exception { + testKeyValQuery(STR_CACHE, "str1", "str2"); + } - IgniteBinary binary = ignite(0).binary(); + /** + * @throws Exception If failed. + */ + public void testEnumObjectArgument() throws Exception { + testKeyValQuery(ENUM_CACHE, EnumKey.KEY1, EnumKey.KEY2); + } - for (int i = 0; i < 100; i++) { - Object key = new TestKey(i); + /** + * @throws Exception If failed. + */ + public void testUuidObjectArgument() throws Exception { + final UUID uuid1 = UUID.randomUUID(); + UUID uuid2 = UUID.randomUUID(); - if (i % 2 == 0) - key = binary.toBinary(key); + while (uuid1.equals(uuid2)) + uuid2 = UUID.randomUUID(); - qry.setArgs(key); + testKeyValQuery(UUID_CACHE, uuid1, uuid2); + } - List> res = cache.query(qry).getAll(); + /** + * @throws Exception If failed. + */ + public void testDateObjectArgument() throws Exception { + testKeyValQuery(DATE_CACHE, new Date(0), new Date(1)); + } + + /** + * @throws Exception If failed. + */ + public void testTimestampArgument() throws Exception { + testKeyValQuery(TIMESTAMP_CACHE, new Timestamp(0), new Timestamp(1)); + } - assertEquals(1, res.size()); - Person p = res.get(0).getValue(); + /** + * @throws Exception If failed. + */ + public void testBigDecimalArgument() throws Exception { + final ThreadLocalRandom rnd = ThreadLocalRandom.current(); - assertEquals("name-" + i, p.name); + final BigDecimal bd1 = new BigDecimal(rnd.nextDouble()); + BigDecimal bd2 = new BigDecimal(rnd.nextDouble()); + + while (bd1.equals(bd2)) + bd2 = new BigDecimal(rnd.nextDouble()); + + testKeyValQuery(BIG_DECIMAL_CACHE, bd1, bd2); + } + + /** + * Test simple queries. + * + * @param cacheName Cache name. + * @param key1 Key 1. + * @param key2 Key 2. + * @param Key type. + */ + private void testKeyValQuery(final String cacheName, final T key1, final T key2) { + testKeyQuery(cacheName, key1, key2); + testValQuery(cacheName + "-val", key1, key2); + } + + /** + * Test simple query by key. + * + * @param cacheName Cache name. + * @param key1 Key 1. + * @param key2 Key 2. + * @param Key type. + */ + private void testKeyQuery(final String cacheName, final T key1, final T key2) { + final IgniteCache cache = ignite(0).cache(cacheName); + + final Person p1 = new Person("p1"); + final Person p2 = new Person("p2"); + + cache.put(key1, p1); + cache.put(key2, p2); + + final SqlQuery qry = new SqlQuery<>(Person.class, "where _key=?"); + + final SqlFieldsQuery fieldsQry = new SqlFieldsQuery("select * from Person where _key=?"); + + qry.setLocal(isLocal()); + fieldsQry.setLocal(isLocal()); + + qry.setArgs(key1); + fieldsQry.setArgs(key1); + + final List> res = cache.query(qry).getAll(); + final List> fieldsRes = cache.query(fieldsQry).getAll(); + + assertEquals(1, res.size()); + assertEquals(1, fieldsRes.size()); + + assertEquals(p1, res.get(0).getValue()); + assertEquals(key1, res.get(0).getKey()); + + assertTrue(fieldsRes.get(0).size() >= 2); + assertEquals(key1, fieldsRes.get(0).get(0)); + assertEquals(p1, fieldsRes.get(0).get(1)); + } + + /** + * Test simple query by value. + * + * @param cacheName Cache name. + * @param val1 Value 1. + * @param val2 Value 2. + * @param Value type. + */ + private void testValQuery(final String cacheName, final T val1, final T val2) { + final IgniteCache cache = ignite(0).cache(cacheName); + + final Class valType = val1.getClass(); + + final Person p1 = new Person("p1"); + final Person p2 = new Person("p2"); + + cache.put(p1, val1); + cache.put(p2, val2); + + final SqlQuery qry = new SqlQuery<>(valType, "where _val=?"); + + final SqlFieldsQuery fieldsQry = new SqlFieldsQuery("select * from " + valType.getSimpleName() + " where _val=?"); + + qry.setLocal(isLocal()); + fieldsQry.setLocal(isLocal()); + + qry.setArgs(val1); + fieldsQry.setArgs(val1); + + final List> res = cache.query(qry).getAll(); + final List> fieldsRes = cache.query(fieldsQry).getAll(); + + assertEquals(1, res.size()); + assertEquals(1, fieldsRes.size()); + + assertEquals(p1, res.get(0).getKey()); + assertEquals(val1, res.get(0).getValue()); + + assertTrue(fieldsRes.get(0).size() >= 2); + assertEquals(p1, fieldsRes.get(0).get(0)); + assertEquals(val1, fieldsRes.get(0).get(1)); + } + + /** + * @throws Exception + */ + public void testFieldSearch() throws Exception { + final IgniteCache cache = ignite(0).cache(FIELD_CACHE); + + final Map map = new HashMap<>(); + + for (int i = 0; i < 10; i++) { + map.put(i, + new SearchValue( + UUID.randomUUID(), + String.valueOf(i), + new BigDecimal(i * 0.1), + i, + new Date(i), + new Timestamp(i), + new Person(String.valueOf("name-" + i)), + i % 2 == 0 ? EnumKey.KEY1 : EnumKey.KEY2) + ); } + + cache.putAll(map); + + SqlQuery qry = new SqlQuery<>(SearchValue.class, + "where uuid=? and str=? and decimal=? and integer=? and date=? and ts=? and person=? and enumKey=?"); + + final int k = ThreadLocalRandom.current().nextInt(10); + + final SearchValue val = map.get(k); + + qry.setLocal(isLocal()); + qry.setArgs(val.uuid, val.str, val.decimal, val.integer, val.date, val.ts, val.person, val.enumKey); + + final List> res = cache.query(qry).getAll(); + + assertEquals(1, res.size()); + + assertEquals(val.integer, res.get(0).getKey()); + assertEquals(val, res.get(0).getValue()); } /** @@ -124,6 +422,27 @@ private static class Person { public Person(String name) { this.name = name; } + + /** {@inheritDoc} */ + @Override public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + final Person person = (Person) o; + + return name != null ? name.equals(person.name) : person.name == null; + + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return name != null ? name.hashCode() : 0; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(Person.class, this); + } } /** @@ -158,4 +477,114 @@ public TestKey(int id) { return id; } } + + /** + * + */ + private enum EnumKey { + /** */ + KEY1, + + /** */ + KEY2 + } + + /** + * + */ + private static class SearchValue { + /** */ + @QuerySqlField + private UUID uuid; + + /** */ + @QuerySqlField + private String str; + + /** */ + @QuerySqlField + private BigDecimal decimal; + + /** */ + @QuerySqlField + private Integer integer; + + /** */ + @QuerySqlField + private Date date; + + /** */ + @QuerySqlField + private Timestamp ts; + + /** */ + @QuerySqlField + private Person person; + + /** */ + @QuerySqlField + private EnumKey enumKey; + + /** + * + * @param uuid UUID. + * @param str String. + * @param decimal Decimal. + * @param integer Integer. + * @param date Date. + * @param ts Timestamp. + * @param person Person. + * @param enumKey Enum. + */ + public SearchValue( + final UUID uuid, + final String str, + final BigDecimal decimal, + final Integer integer, + final Date date, + final Timestamp ts, + final Person person, + final EnumKey enumKey + ) { + this.uuid = uuid; + this.str = str; + this.decimal = decimal; + this.integer = integer; + this.date = date; + this.ts = ts; + this.person = person; + this.enumKey = enumKey; + } + + /** {@inheritDoc} */ + @Override public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + final SearchValue that = (SearchValue) o; + + if (uuid != null ? !uuid.equals(that.uuid) : that.uuid != null) return false; + if (str != null ? !str.equals(that.str) : that.str != null) return false; + if (decimal != null ? !decimal.equals(that.decimal) : that.decimal != null) return false; + if (integer != null ? !integer.equals(that.integer) : that.integer != null) return false; + if (date != null ? !date.equals(that.date) : that.date != null) return false; + if (ts != null ? !ts.equals(that.ts) : that.ts != null) return false; + if (person != null ? !person.equals(that.person) : that.person != null) return false; + return enumKey == that.enumKey; + + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int res = uuid != null ? uuid.hashCode() : 0; + res = 31 * res + (str != null ? str.hashCode() : 0); + res = 31 * res + (decimal != null ? decimal.hashCode() : 0); + res = 31 * res + (integer != null ? integer.hashCode() : 0); + res = 31 * res + (date != null ? date.hashCode() : 0); + res = 31 * res + (ts != null ? ts.hashCode() : 0); + res = 31 * res + (person != null ? person.hashCode() : 0); + res = 31 * res + (enumKey != null ? enumKey.hashCode() : 0); + return res; + } + } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java index 1d54bbfcb5014..cf000e93c7466 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java @@ -301,7 +301,7 @@ private void assertSqlEquals(String sql1, String sql2) { private static String normalizeSql(String sql) { return sql.toLowerCase() .replaceAll("/\\*(?:.|\r|\n)*?\\*/", " ") - .replaceAll("\\s*on\\s+1\\s*=\\s*1\\s*", " on true ") + .replaceAll("\\s*on\\s+1\\s*=\\s*1\\s*", " ") .replaceAll("\\s+", " ") .replaceAll("\\( +", "(") .replaceAll(" +\\)", ")") @@ -366,4 +366,4 @@ public static class Address implements Serializable { @QuerySqlField(index = true) public String street = "Nevskiy"; } -} \ No newline at end of file +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java index 3652acda85944..e7f55a1faae2c 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java @@ -24,6 +24,10 @@ import org.apache.ignite.internal.processors.cache.GridCacheQuerySerializationSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheReduceQueryMultithreadedSelfTest; import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectFieldsQuerySelfTest; +import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectLocalQueryArgumentsTest; +import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgumentsOffheapLocalTest; +import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgumentsOffheapTest; +import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgumentsTest; import org.apache.ignite.internal.processors.cache.IgniteBinaryWrappedObjectFieldsQuerySelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest; @@ -118,6 +122,11 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCacheQueryNoRebalanceSelfTest.class); suite.addTestSuite(GridCacheQueryTransformerSelfTest.class); + suite.addTestSuite(IgniteBinaryObjectQueryArgumentsTest.class); + suite.addTestSuite(IgniteBinaryObjectQueryArgumentsOffheapTest.class); + suite.addTestSuite(IgniteBinaryObjectQueryArgumentsOffheapLocalTest.class); + suite.addTestSuite(IgniteBinaryObjectLocalQueryArgumentsTest.class); + return suite; } } From 7a84ab6a9163ca31fbcfcc6d7ff27e06bf9babef Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Fri, 2 Sep 2016 18:05:16 +0300 Subject: [PATCH 080/487] IGNITE-3827: Removed double marshalling of keys in DataStreamerImpl.addData(Map) method. --- .../datastreamer/DataStreamerImpl.java | 23 ++++++++----------- 1 file changed, 9 insertions(+), 14 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java index e565cbaa43676..a3bae249d13ef 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java @@ -87,7 +87,6 @@ import org.apache.ignite.internal.util.lang.GridPeerDeployAware; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; -import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; @@ -513,23 +512,19 @@ else if (autoFlushFreq == 0) activeFuts.add(resFut); - Collection keys = null; + Collection keys = + new GridConcurrentHashSet<>(entries.size(), U.capacity(entries.size()), 1); - if (entries.size() > 1) { - keys = new GridConcurrentHashSet<>(entries.size(), U.capacity(entries.size()), 1); + Collection entries0 = new ArrayList<>(entries.size()); - for (Map.Entry entry : entries) - keys.add(cacheObjProc.toCacheKeyObject(cacheObjCtx, null, entry.getKey(), true)); - } + for (Map.Entry entry : entries) { + KeyCacheObject key = cacheObjProc.toCacheKeyObject(cacheObjCtx, null, entry.getKey(), true); + CacheObject val = cacheObjProc.toCacheObject(cacheObjCtx, entry.getValue(), true); - Collection entries0 = F.viewReadOnly(entries, new C1, DataStreamerEntry>() { - @Override public DataStreamerEntry apply(Entry e) { - KeyCacheObject key = cacheObjProc.toCacheKeyObject(cacheObjCtx, null, e.getKey(), true); - CacheObject val = cacheObjProc.toCacheObject(cacheObjCtx, e.getValue(), true); + keys.add(key); - return new DataStreamerEntry(key, val); - } - }); + entries0.add(new DataStreamerEntry(key, val)); + } load0(entries0, resFut, keys, 0); From e3c4868d6737e5a0f0b90f99666242865add750c Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Fri, 2 Sep 2016 18:23:09 +0300 Subject: [PATCH 081/487] IGNITE-3829: Optimized affinity key field name handling. --- .../CacheObjectBinaryProcessorImpl.java | 35 +++++++++++++++---- 1 file changed, 28 insertions(+), 7 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java index 03378743c2ee3..ecd27f7a19af7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java @@ -40,6 +40,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.binary.BinaryBasicNameMapper; +import org.apache.ignite.binary.BinaryField; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.binary.BinaryObjectException; @@ -89,6 +90,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.T1; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; @@ -159,6 +161,9 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm /** Metadata updates collected before metadata cache is initialized. */ private final Map metaBuf = new ConcurrentHashMap<>(); + /** Cached affinity key field names. */ + private final ConcurrentHashMap> affKeyFields = new ConcurrentHashMap<>(); + /** * @param ctx Kernal context. */ @@ -684,22 +689,38 @@ public GridBinaryMarshaller marshaller() { * @return Affinity key. */ public Object affinityKey(BinaryObject po) { + // Fast path for already cached field. + if (po instanceof BinaryObjectEx) { + int typeId = ((BinaryObjectEx)po).typeId(); + + T1 fieldHolder = affKeyFields.get(typeId); + + if (fieldHolder != null) { + BinaryField field = fieldHolder.get(); + + return field != null ? field.value(po) : po; + } + } + + // Slow path if affinity field is not cached yet. try { BinaryType meta = po instanceof BinaryObjectEx ? ((BinaryObjectEx)po).rawType() : po.type(); if (meta != null) { - String affKeyFieldName = meta.affinityKeyFieldName(); + String name = meta.affinityKeyFieldName(); + + affKeyFields.putIfAbsent(meta.typeId(), new T1<>(meta.field(name))); - if (affKeyFieldName != null) - return po.field(affKeyFieldName); + if (name != null) + return po.field(name); } else if (po instanceof BinaryObjectEx) { - int id = ((BinaryObjectEx)po).typeId(); + int typeId = ((BinaryObjectEx)po).typeId(); - String affKeyFieldName = binaryCtx.affinityKeyFieldName(id); + String name = binaryCtx.affinityKeyFieldName(typeId); - if (affKeyFieldName != null) - return po.field(affKeyFieldName); + if (name != null) + return po.field(name); } } catch (BinaryObjectException e) { From e9c797fd964727882ad6f40f2a452b17ae7c857e Mon Sep 17 00:00:00 2001 From: isapego Date: Sun, 4 Sep 2016 16:47:40 +0300 Subject: [PATCH 082/487] IGNITE-2946: CPP: Optimized GetNext() method for cursors. This closes #992. --- .../query/PlatformAbstractQueryCursor.java | 11 +- .../query/PlatformFieldsQueryCursor.java | 6 + .../ignite/impl/binary/binary_reader_impl.h | 2 +- .../common/include/ignite/common/concurrent.h | 5 +- .../cpp/core-test/src/cache_query_test.cpp | 243 +++++++++++++----- modules/platforms/cpp/core/Makefile.am | 1 + .../platforms/cpp/core/include/Makefile.am | 1 + .../include/ignite/cache/query/query_cursor.h | 6 +- .../ignite/cache/query/query_fields_cursor.h | 4 +- .../ignite/impl/cache/query/query_batch.h | 148 +++++++++++ .../impl/cache/query/query_fields_row_impl.h | 30 +-- .../ignite/impl/cache/query/query_impl.h | 30 ++- .../cpp/core/project/vs/core.vcxproj | 2 + .../cpp/core/project/vs/core.vcxproj.filters | 6 + .../core/src/impl/cache/query/query_batch.cpp | 52 ++++ .../core/src/impl/cache/query/query_impl.cpp | 180 +++++++------ .../Impl/Cache/Query/FieldsQueryCursor.cs | 3 + 17 files changed, 537 insertions(+), 193 deletions(-) create mode 100644 modules/platforms/cpp/core/include/ignite/impl/cache/query/query_batch.h create mode 100644 modules/platforms/cpp/core/src/impl/cache/query/query_batch.cpp diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java index 742275779c0eb..ab52b52d1de4b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java @@ -70,13 +70,12 @@ public PlatformAbstractQueryCursor(PlatformContext platformCtx, QueryCursorEx try { int cntPos = writer.reserveInt(); - int cnt; + int cnt = 0; - for (cnt = 0; cnt < batchSize; cnt++) { - if (iter.hasNext()) - write(writer, iter.next()); - else - break; + while (cnt < batchSize && iter.hasNext()) { + write(writer, iter.next()); + + cnt++; } writer.writeInt(cntPos, cnt); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java index a4cdae6d87def..25f86f2da904a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java @@ -41,9 +41,15 @@ public PlatformFieldsQueryCursor(PlatformContext platformCtx, QueryCursorExIncrement(); } diff --git a/modules/platforms/cpp/core-test/src/cache_query_test.cpp b/modules/platforms/cpp/core-test/src/cache_query_test.cpp index 168f3f99a53cb..b8cd6120a1c05 100644 --- a/modules/platforms/cpp/core-test/src/cache_query_test.cpp +++ b/modules/platforms/cpp/core-test/src/cache_query_test.cpp @@ -212,63 +212,6 @@ namespace ignite } } -/** Node started during the test. */ -Ignite grid = Ignite(); - -/** Cache accessor. */ -Cache GetCache() -{ - return grid.GetCache("cache"); -} - -/** - * Test setup fixture. - */ -struct CacheQueryTestSuiteFixture { - /** - * Constructor. - */ - CacheQueryTestSuiteFixture() - { - IgniteConfiguration cfg; - - cfg.jvmOpts.push_back("-Xdebug"); - cfg.jvmOpts.push_back("-Xnoagent"); - cfg.jvmOpts.push_back("-Djava.compiler=NONE"); - cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005"); - cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError"); - -#ifdef IGNITE_TESTS_32 - cfg.jvmInitMem = 256; - cfg.jvmMaxMem = 768; -#else - cfg.jvmInitMem = 1024; - cfg.jvmMaxMem = 4096; -#endif - - char* cfgPath = getenv("IGNITE_NATIVE_TEST_CPP_CONFIG_PATH"); - - cfg.springCfgPath = std::string(cfgPath).append("/").append("cache-query.xml"); - - IgniteError err; - - Ignite grid0 = Ignition::Start(cfg, &err); - - if (err.GetCode() != IgniteError::IGNITE_SUCCESS) - BOOST_ERROR(err.GetText()); - - grid = grid0; - } - - /** - * Destructor. - */ - ~CacheQueryTestSuiteFixture() - { - Ignition::Stop(grid.GetName(), true); - } -}; - /** * Ensure that HasNext() fails. * @@ -522,6 +465,131 @@ void CheckMultipleGetAll(Cursor& cur, int key1, const std::string& name1, } } +/** + * Test setup fixture. + */ +struct CacheQueryTestSuiteFixture +{ + Ignite StartNode(const char* name) + { + IgniteConfiguration cfg; + + cfg.jvmOpts.push_back("-Xdebug"); + cfg.jvmOpts.push_back("-Xnoagent"); + cfg.jvmOpts.push_back("-Djava.compiler=NONE"); + cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005"); + cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError"); + +#ifdef IGNITE_TESTS_32 + cfg.jvmInitMem = 256; + cfg.jvmMaxMem = 768; +#else + cfg.jvmInitMem = 1024; + cfg.jvmMaxMem = 4096; +#endif + + cfg.springCfgPath.assign(getenv("IGNITE_NATIVE_TEST_CPP_CONFIG_PATH")).append("/cache-query.xml"); + + IgniteError err; + + Ignite grid0 = Ignition::Start(cfg, name, &err); + + if (err.GetCode() != IgniteError::IGNITE_SUCCESS) + BOOST_ERROR(err.GetText()); + + return grid0; + } + + void CheckFieldsQueryPages(int32_t pageSize, int32_t pagesNum, int32_t additionalNum) + { + // Test simple query. + Cache cache = GetPersonCache(); + + // Test query with two fields of different type. + SqlFieldsQuery qry("select name, age from QueryPerson"); + + QueryFieldsCursor cursor = cache.Query(qry); + CheckEmpty(cursor); + + const int32_t entryCnt = pageSize * pagesNum + additionalNum; // Number of entries. + + qry.SetPageSize(pageSize); + + for (int i = 0; i < entryCnt; i++) + { + std::stringstream stream; + + stream << "A" << i; + + cache.Put(i, QueryPerson(stream.str(), i * 10, BinaryUtils::MakeDateLocal(1970 + i), + BinaryUtils::MakeTimestampLocal(2016, 1, 1, i / 60, i % 60))); + } + + cursor = cache.Query(qry); + + IgniteError error; + + for (int i = 0; i < entryCnt; i++) + { + std::stringstream stream; + + stream << "A" << i; + + std::string expected_name = stream.str(); + int expected_age = i * 10; + + BOOST_REQUIRE(cursor.HasNext(error)); + BOOST_REQUIRE(error.GetCode() == IgniteError::IGNITE_SUCCESS); + + QueryFieldsRow row = cursor.GetNext(error); + BOOST_REQUIRE(error.GetCode() == IgniteError::IGNITE_SUCCESS); + + BOOST_REQUIRE(row.HasNext(error)); + BOOST_REQUIRE(error.GetCode() == IgniteError::IGNITE_SUCCESS); + + std::string name = row.GetNext(error); + BOOST_REQUIRE(error.GetCode() == IgniteError::IGNITE_SUCCESS); + + BOOST_REQUIRE(name == expected_name); + + int age = row.GetNext(error); + BOOST_REQUIRE(error.GetCode() == IgniteError::IGNITE_SUCCESS); + + BOOST_REQUIRE(age == expected_age); + + BOOST_REQUIRE(!row.HasNext()); + } + + CheckEmpty(cursor); + } + + /** + * Constructor. + */ + CacheQueryTestSuiteFixture() : + grid(StartNode("Node1")) + { + // No-op. + } + + /** + * Destructor. + */ + ~CacheQueryTestSuiteFixture() + { + Ignition::StopAll(true); + } + + /** Person cache accessor. */ + Cache GetPersonCache() + { + return grid.GetCache("cache"); + } + + /** Node started during the test. */ + Ignite grid; +}; + BOOST_FIXTURE_TEST_SUITE(CacheQueryTestSuite, CacheQueryTestSuiteFixture) /** @@ -529,7 +597,7 @@ BOOST_FIXTURE_TEST_SUITE(CacheQueryTestSuite, CacheQueryTestSuiteFixture) */ BOOST_AUTO_TEST_CASE(TestSqlQuery) { - Cache cache = GetCache(); + Cache cache = GetPersonCache(); // Test query with no results. SqlQuery qry("QueryPerson", "age < 20"); @@ -585,7 +653,7 @@ BOOST_AUTO_TEST_CASE(TestSqlQuery) */ BOOST_AUTO_TEST_CASE(TestTextQuery) { - Cache cache = GetCache(); + Cache cache = GetPersonCache(); // Test query with no results. TextQuery qry("QueryPerson", "A1"); @@ -631,7 +699,7 @@ BOOST_AUTO_TEST_CASE(TestTextQuery) BOOST_AUTO_TEST_CASE(TestScanQuery) { // Test simple query. - Cache cache = GetCache(); + Cache cache = GetPersonCache(); // Test query with no results. ScanQuery qry; @@ -667,7 +735,7 @@ BOOST_AUTO_TEST_CASE(TestScanQuery) BOOST_AUTO_TEST_CASE(TestScanQueryPartitioned) { // Populate cache with data. - Cache cache = GetCache(); + Cache cache = GetPersonCache(); int32_t partCnt = 256; // Defined in configuration explicitly. int32_t entryCnt = 1000; // Should be greater than partCnt. @@ -716,7 +784,7 @@ BOOST_AUTO_TEST_CASE(TestScanQueryPartitioned) BOOST_AUTO_TEST_CASE(TestFieldsQuerySingle) { // Test simple query. - Cache cache = GetCache(); + Cache cache = GetPersonCache(); // Test query with two fields of different type. SqlFieldsQuery qry("select age, name from QueryPerson"); @@ -761,7 +829,7 @@ BOOST_AUTO_TEST_CASE(TestFieldsQuerySingle) BOOST_AUTO_TEST_CASE(TestFieldsQueryExceptions) { // Test simple query. - Cache cache = GetCache(); + Cache cache = GetPersonCache(); // Test query with two fields of different type. SqlFieldsQuery qry("select age, name from QueryPerson"); @@ -806,7 +874,7 @@ BOOST_AUTO_TEST_CASE(TestFieldsQueryExceptions) BOOST_AUTO_TEST_CASE(TestFieldsQueryTwo) { // Test simple query. - Cache cache = GetCache(); + Cache cache = GetPersonCache(); // Test query with two fields of different type. SqlFieldsQuery qry("select age, name from QueryPerson"); @@ -869,7 +937,7 @@ BOOST_AUTO_TEST_CASE(TestFieldsQueryTwo) BOOST_AUTO_TEST_CASE(TestFieldsQuerySeveral) { // Test simple query. - Cache cache = GetCache(); + Cache cache = GetPersonCache(); // Test query with two fields of different type. SqlFieldsQuery qry("select name, age from QueryPerson"); @@ -935,7 +1003,7 @@ BOOST_AUTO_TEST_CASE(TestFieldsQuerySeveral) BOOST_AUTO_TEST_CASE(TestFieldsQueryDateLess) { // Test simple query. - Cache cache = GetCache(); + Cache cache = GetPersonCache(); // Test query with field of type 'Date'. SqlFieldsQuery qry("select birthday from QueryPerson where birthday<'1990-01-01'"); @@ -996,7 +1064,7 @@ BOOST_AUTO_TEST_CASE(TestFieldsQueryDateLess) BOOST_AUTO_TEST_CASE(TestFieldsQueryDateMore) { // Test simple query. - Cache cache = GetCache(); + Cache cache = GetPersonCache(); // Test query with field of type 'Date'. SqlFieldsQuery qry("select birthday from QueryPerson where birthday>'2070-01-01'"); @@ -1057,7 +1125,7 @@ BOOST_AUTO_TEST_CASE(TestFieldsQueryDateMore) BOOST_AUTO_TEST_CASE(TestFieldsQueryDateEqual) { // Test simple query. - Cache cache = GetCache(); + Cache cache = GetPersonCache(); // Test query with field of type 'Date'. SqlFieldsQuery qry("select birthday from QueryPerson where birthday='2032-01-01'"); @@ -1109,7 +1177,7 @@ BOOST_AUTO_TEST_CASE(TestFieldsQueryDateEqual) BOOST_AUTO_TEST_CASE(TestFieldsQueryTimestampLess) { // Test simple query. - Cache cache = GetCache(); + Cache cache = GetPersonCache(); // Test query with field of type 'Timestamp'. SqlFieldsQuery qry("select recordCreated from QueryPerson where recordCreated<'2016-01-01 01:00:00'"); @@ -1170,7 +1238,7 @@ BOOST_AUTO_TEST_CASE(TestFieldsQueryTimestampLess) BOOST_AUTO_TEST_CASE(TestFieldsQueryTimestampMore) { // Test simple query. - Cache cache = GetCache(); + Cache cache = GetPersonCache(); // Test query with field of type 'Timestamp'. SqlFieldsQuery qry("select recordCreated from QueryPerson where recordCreated>'2016-01-01 15:30:00'"); @@ -1233,7 +1301,7 @@ BOOST_AUTO_TEST_CASE(TestFieldsQueryTimestampMore) BOOST_AUTO_TEST_CASE(TestFieldsQueryTimestampEqual) { // Test simple query. - Cache cache = GetCache(); + Cache cache = GetPersonCache(); // Test query with field of type 'Timestamp'. SqlFieldsQuery qry("select recordCreated from QueryPerson where recordCreated='2016-01-01 09:18:00'"); @@ -1279,4 +1347,37 @@ BOOST_AUTO_TEST_CASE(TestFieldsQueryTimestampEqual) CheckEmpty(cursor); } +/** + * Test fields query with several pages. + */ +BOOST_AUTO_TEST_CASE(TestFieldsQueryPagesSeveral) +{ + CheckFieldsQueryPages(32, 8, 1); +} + +/** + * Test fields query with page size 1. + */ +BOOST_AUTO_TEST_CASE(TestFieldsQueryPageSingle) +{ + CheckFieldsQueryPages(1, 100, 0); +} + +/** + * Test fields query with page size 0. + */ +BOOST_AUTO_TEST_CASE(TestFieldsQueryPageZero) +{ + try + { + CheckFieldsQueryPages(0, 100, 0); + + BOOST_FAIL("Exception expected."); + } + catch (IgniteError&) + { + // Expected. + } +} + BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/core/Makefile.am b/modules/platforms/cpp/core/Makefile.am index 2b734764e4bb3..bbb77203da624 100644 --- a/modules/platforms/cpp/core/Makefile.am +++ b/modules/platforms/cpp/core/Makefile.am @@ -59,6 +59,7 @@ libignite_la_SOURCES = \ src/impl/handle_registry.cpp \ src/impl/cache/query/query_impl.cpp \ src/impl/cache/cache_impl.cpp \ + src/impl/cache/query/query_batch.cpp \ src/impl/interop/interop_external_memory.cpp \ src/impl/interop/interop_target.cpp \ src/impl/transactions/transaction_impl.cpp \ diff --git a/modules/platforms/cpp/core/include/Makefile.am b/modules/platforms/cpp/core/include/Makefile.am index f7159ae2200d5..fb84bc5d0bea4 100644 --- a/modules/platforms/cpp/core/include/Makefile.am +++ b/modules/platforms/cpp/core/include/Makefile.am @@ -27,6 +27,7 @@ nobase_include_HEADERS = \ ignite/impl/cache/query/query_fields_row_impl.h \ ignite/impl/cache/query/query_impl.h \ ignite/impl/cache/cache_impl.h \ + ignite/impl/cache/query/query_batch.h \ ignite/impl/interop/interop_target.h \ ignite/impl/interop/interop_external_memory.h \ ignite/impl/handle_registry.h \ diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h b/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h index 4c46662802dbb..61c6813a9b2a7 100644 --- a/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h +++ b/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h @@ -112,7 +112,7 @@ namespace ignite impl::cache::query::QueryCursorImpl* impl0 = impl.Get(); if (impl0) - return impl0->HasNext(&err); + return impl0->HasNext(err); else { err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, @@ -160,7 +160,7 @@ namespace ignite if (impl0) { impl::Out2Operation outOp; - impl0->GetNext(outOp, &err); + impl0->GetNext(outOp, err); if (err.GetCode() == IgniteError::IGNITE_SUCCESS) { @@ -215,7 +215,7 @@ namespace ignite if (impl0) { impl::OutQueryGetAllOperation outOp(&res); - impl0->GetAll(outOp, &err); + impl0->GetAll(outOp, err); } else err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_cursor.h b/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_cursor.h index 3946e1c8ed2da..36e5f5c53c6f8 100644 --- a/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_cursor.h +++ b/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_cursor.h @@ -108,7 +108,7 @@ namespace ignite impl::cache::query::QueryCursorImpl* impl0 = impl.Get(); if (impl0) - return impl0->HasNext(&err); + return impl0->HasNext(err); else { err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, @@ -153,7 +153,7 @@ namespace ignite impl::cache::query::QueryCursorImpl* impl0 = impl.Get(); if (impl0) - return impl0->GetNextRow(&err); + return impl0->GetNextRow(err); else { err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, diff --git a/modules/platforms/cpp/core/include/ignite/impl/cache/query/query_batch.h b/modules/platforms/cpp/core/include/ignite/impl/cache/query/query_batch.h new file mode 100644 index 0000000000000..15d6edbc39246 --- /dev/null +++ b/modules/platforms/cpp/core/include/ignite/impl/cache/query/query_batch.h @@ -0,0 +1,148 @@ +/* + * 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. + */ + +#ifndef _IGNITE_CACHE_QUERY_BATCH +#define _IGNITE_CACHE_QUERY_BATCH + +#include + +#include "ignite/ignite_error.h" +#include "ignite/impl/ignite_environment.h" +#include "ignite/impl/operations.h" + +namespace ignite +{ + namespace impl + { + namespace cache + { + namespace query + { + class QueryFieldsRowImpl; + + /** + * Query batch. + */ + class IGNITE_IMPORT_EXPORT QueryBatch + { + typedef common::concurrent::SharedPointer MemorySharedPtr; + + public: + /** + * Constructor. + * + * @param env Environment. + * @param mem Batch memory. + */ + QueryBatch(IgniteEnvironment& env, MemorySharedPtr mem) : + env(env), + mem(mem), + stream(mem.Get()), + reader(&stream), + size(reader.ReadInt32()), + pos(0) + { + // No-op. + } + + /** + * Destructor. + */ + ~QueryBatch() + { + // No-op. + } + + /** + * Check whether batch is empty. + * + * @return True if empty. + */ + bool IsEmpty() const + { + return size == 0; + } + + /** + * Get the number of the unread rows in the batch. + * + * @return Number of the unread rows in the batch. + */ + int32_t Left() const + { + return size - pos; + } + + /** + * Check whether next result exists. + * + * @param err Error. + * @return True if exists. + */ + int32_t Size() + { + return size; + } + + /** + * Get next object. + * + * @param op Operation. + */ + void GetNext(OutputOperation& op) + { + assert(Left() > 0); + + op.ProcessOutput(reader); + + ++pos; + } + + /** + * Get next row. + * + * @return Output row. + */ + QueryFieldsRowImpl* GetNextRow(); + + private: + /** Environment. */ + IgniteEnvironment& env; + + /** Memomy containing the batch. */ + MemorySharedPtr mem; + + /** Stream. */ + interop::InteropInputStream stream; + + /** Reader. */ + binary::BinaryReaderImpl reader; + + /** Result batch size. */ + int32_t size; + + /** Position in memory. */ + int32_t pos; + + IGNITE_NO_COPY_ASSIGNMENT(QueryBatch); + }; + } + } + } +} + +#endif // _IGNITE_CACHE_QUERY_BATCH diff --git a/modules/platforms/cpp/core/include/ignite/impl/cache/query/query_fields_row_impl.h b/modules/platforms/cpp/core/include/ignite/impl/cache/query/query_fields_row_impl.h index 233c2d4bcdb06..82cebd5f517fa 100644 --- a/modules/platforms/cpp/core/include/ignite/impl/cache/query/query_fields_row_impl.h +++ b/modules/platforms/cpp/core/include/ignite/impl/cache/query/query_fields_row_impl.h @@ -18,16 +18,9 @@ #ifndef _IGNITE_IMPL_CACHE_QUERY_CACHE_QUERY_FIELDS_ROW_IMPL #define _IGNITE_IMPL_CACHE_QUERY_CACHE_QUERY_FIELDS_ROW_IMPL -#include -#include - #include #include -#include "ignite/cache/cache_entry.h" -#include "ignite/impl/cache/query/query_impl.h" -#include "ignite/impl/operations.h" - namespace ignite { namespace impl @@ -44,24 +37,19 @@ namespace ignite public: typedef common::concurrent::SharedPointer SP_InteropMemory; - /** - * Default constructor. - */ - QueryFieldsRowImpl() : mem(0), stream(0), reader(0), size(0), - processed(0) - { - // No-op. - } - /** * Constructor. * * @param mem Memory containig row data. */ - QueryFieldsRowImpl(SP_InteropMemory mem) : mem(mem), stream(mem.Get()), - reader(&stream), size(reader.ReadInt32()), processed(0) + QueryFieldsRowImpl(SP_InteropMemory mem, int32_t rowBegin, int32_t columnNum) : + mem(mem), + stream(mem.Get()), + reader(&stream), + columnNum(columnNum), + processed(0) { - // No-op. + stream.Position(rowBegin); } /** @@ -89,7 +77,7 @@ namespace ignite bool HasNext(IgniteError& err) { if (IsValid()) - return processed < size; + return processed < columnNum; else { err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, @@ -165,7 +153,7 @@ namespace ignite binary::BinaryReaderImpl reader; /** Number of elements in a row. */ - int32_t size; + int32_t columnNum; /** Number of elements that have been read by now. */ int32_t processed; diff --git a/modules/platforms/cpp/core/include/ignite/impl/cache/query/query_impl.h b/modules/platforms/cpp/core/include/ignite/impl/cache/query/query_impl.h index 0f17c327212f4..4083c7c408011 100644 --- a/modules/platforms/cpp/core/include/ignite/impl/cache/query/query_impl.h +++ b/modules/platforms/cpp/core/include/ignite/impl/cache/query/query_impl.h @@ -22,6 +22,7 @@ #include "ignite/impl/ignite_environment.h" #include "ignite/impl/operations.h" +#include "ignite/impl/cache/query/query_batch.h" namespace ignite { @@ -58,7 +59,7 @@ namespace ignite * @param err Error. * @return True if exists. */ - bool HasNext(IgniteError* err); + bool HasNext(IgniteError& err); /** * Get next object. @@ -66,7 +67,7 @@ namespace ignite * @param op Operation. * @param err Error. */ - void GetNext(OutputOperation& op, IgniteError* err); + void GetNext(OutputOperation& op, IgniteError& err); /** * Get next row. @@ -74,7 +75,7 @@ namespace ignite * @param err Error. * @return Output row. */ - QueryFieldsRowImpl* GetNextRow(IgniteError* err); + QueryFieldsRowImpl* GetNextRow(IgniteError& err); /** * Get all cursor entries. @@ -82,7 +83,7 @@ namespace ignite * @param op Operation. * @param err Error. */ - void GetAll(OutputOperation& op, IgniteError* err); + void GetAll(OutputOperation& op, IgniteError& err); private: /** Environment. */ @@ -91,15 +92,18 @@ namespace ignite /** Handle to Java object. */ jobject javaRef; + /** Current result batch. */ + QueryBatch* batch; + + /** Whether cursor has no more elements available. */ + bool endReached; + /** Whether iteration methods were called. */ bool iterCalled; /** Whether GetAll() method was called. */ bool getAllCalled; - /** Whether next entry is available. */ - bool hasNext; - IGNITE_NO_COPY_ASSIGNMENT(QueryCursorImpl); /** @@ -108,7 +112,15 @@ namespace ignite * @param err Error. * @return True in case of success, false if an error is thrown. */ - bool CreateIteratorIfNeeded(IgniteError* err); + bool CreateIteratorIfNeeded(IgniteError& err); + + /** + * Get next result batch if update is needed. + * + * @param err Error. + * @return True if operation has been successful. + */ + bool GetNextBatchIfNeeded(IgniteError& err); /** * Check whether Java-side iterator has next element. @@ -116,7 +128,7 @@ namespace ignite * @param err Error. * @return True if the next element is available. */ - bool IteratorHasNext(IgniteError* err); + bool IteratorHasNext(IgniteError& err); }; } } diff --git a/modules/platforms/cpp/core/project/vs/core.vcxproj b/modules/platforms/cpp/core/project/vs/core.vcxproj index 0797c311788a2..ca14a1d1ad7aa 100644 --- a/modules/platforms/cpp/core/project/vs/core.vcxproj +++ b/modules/platforms/cpp/core/project/vs/core.vcxproj @@ -209,6 +209,7 @@ + @@ -229,6 +230,7 @@ + diff --git a/modules/platforms/cpp/core/project/vs/core.vcxproj.filters b/modules/platforms/cpp/core/project/vs/core.vcxproj.filters index c90b6971febf6..c5fb5323ecee9 100644 --- a/modules/platforms/cpp/core/project/vs/core.vcxproj.filters +++ b/modules/platforms/cpp/core/project/vs/core.vcxproj.filters @@ -43,6 +43,9 @@ Code\impl\interop + + Code\impl\cache\query + @@ -138,6 +141,9 @@ Code\impl\interop + + Code\impl\cache\query + diff --git a/modules/platforms/cpp/core/src/impl/cache/query/query_batch.cpp b/modules/platforms/cpp/core/src/impl/cache/query/query_batch.cpp new file mode 100644 index 0000000000000..44086af7da5e1 --- /dev/null +++ b/modules/platforms/cpp/core/src/impl/cache/query/query_batch.cpp @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "ignite/impl/cache/query/query_batch.h" +#include "ignite/impl/cache/query/query_fields_row_impl.h" + +namespace ignite +{ + namespace impl + { + namespace cache + { + namespace query + { + QueryFieldsRowImpl* QueryBatch::GetNextRow() + { + assert(Left() > 0); + + int32_t rowBegin = stream.Position(); + + int32_t rowLen = reader.ReadInt32(); + int32_t columnNum = reader.ReadInt32(); + + int32_t dataPos = stream.Position(); + + assert(rowLen >= 4); + + ++pos; + + stream.Position(rowBegin + rowLen); + + return new QueryFieldsRowImpl(mem, dataPos, columnNum); + } + + } + } + } +} diff --git a/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp b/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp index 880e8b1227029..73d99248f0d54 100644 --- a/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp +++ b/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp @@ -35,30 +35,41 @@ namespace ignite /** Operation: get all entries. */ const int32_t OP_GET_ALL = 1; + /** Operation: get multiple entries. */ + const int32_t OP_GET_BATCH = 2; + /** Operation: get single entry. */ const int32_t OP_GET_SINGLE = 3; QueryCursorImpl::QueryCursorImpl(SharedPointer env, jobject javaRef) : - env(env), javaRef(javaRef), iterCalled(false), getAllCalled(false), hasNext(false) + env(env), + javaRef(javaRef), + batch(0), + endReached(false), + iterCalled(false), + getAllCalled(false) { // No-op. } QueryCursorImpl::~QueryCursorImpl() { - // 1. Close the cursor. + // 1. Releasing memory. + delete batch; + + // 2. Close the cursor. env.Get()->Context()->QueryCursorClose(javaRef); - // 2. Release Java reference. + // 3. Release Java reference. JniContext::Release(javaRef); } - bool QueryCursorImpl::HasNext(IgniteError* err) + bool QueryCursorImpl::HasNext(IgniteError& err) { // Check whether GetAll() was called earlier. if (getAllCalled) { - *err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, + err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, "Cannot use HasNext() method because GetAll() was called."); return false; @@ -67,16 +78,21 @@ namespace ignite // Create iterator in Java if needed. if (!CreateIteratorIfNeeded(err)) return false; - - return hasNext; + + // Get next results batch if the end in the current batch + // has been reached. + if (!GetNextBatchIfNeeded(err)) + return false; + + return !endReached; } - void QueryCursorImpl::GetNext(OutputOperation& op, IgniteError* err) + void QueryCursorImpl::GetNext(OutputOperation& op, IgniteError& err) { // Check whether GetAll() was called earlier. if (getAllCalled) { - *err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, + err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, "Cannot use GetNext() method because GetAll() was called."); return; @@ -86,75 +102,52 @@ namespace ignite if (!CreateIteratorIfNeeded(err)) return; - if (hasNext) - { - JniErrorInfo jniErr; - - SharedPointer inMem = env.Get()->AllocateMemory(); - - env.Get()->Context()->TargetOutStream( - javaRef, OP_GET_SINGLE, inMem.Get()->PointerLong(), &jniErr); - - IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); - - if (jniErr.code == IGNITE_JNI_ERR_SUCCESS) - { - InteropInputStream in(inMem.Get()); - - binary::BinaryReaderImpl reader(&in); - - op.ProcessOutput(reader); + // Get next results batch if the end in the current batch + // has been reached. + if (!GetNextBatchIfNeeded(err)) + return; - hasNext = IteratorHasNext(err); - } - } - else + if (endReached) { // Ensure we do not overwrite possible previous error. - if (err->GetCode() == IgniteError::IGNITE_SUCCESS) - *err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, "No more elements available."); + if (err.GetCode() == IgniteError::IGNITE_SUCCESS) + err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, "No more elements available."); + + return; } + + batch->GetNext(op); } - QueryFieldsRowImpl* QueryCursorImpl::GetNextRow(IgniteError* err) + QueryFieldsRowImpl* QueryCursorImpl::GetNextRow(IgniteError& err) { // Create iterator in Java if needed. if (!CreateIteratorIfNeeded(err)) - return NULL; - - if (hasNext) - { - JniErrorInfo jniErr; - - SharedPointer inMem = env.Get()->AllocateMemory(); + return 0; - env.Get()->Context()->TargetOutStream(javaRef, OP_GET_SINGLE, inMem.Get()->PointerLong(), &jniErr); + // Get next results batch if the end in the current batch + // has been reached. + if (!GetNextBatchIfNeeded(err)) + return 0; - IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); - - if (jniErr.code == IGNITE_JNI_ERR_SUCCESS) - { - hasNext = IteratorHasNext(err); - - return new QueryFieldsRowImpl(inMem); - } - } - else + if (endReached) { // Ensure we do not overwrite possible previous error. - if (err->GetCode() == IgniteError::IGNITE_SUCCESS) - *err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, "No more elements available."); + if (err.GetCode() == IgniteError::IGNITE_SUCCESS) + err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, "No more elements available."); + + return 0; } - return NULL; + return batch->GetNextRow(); } - void QueryCursorImpl::GetAll(OutputOperation& op, IgniteError* err) + void QueryCursorImpl::GetAll(OutputOperation& op, IgniteError& err) { // Check whether any of iterator methods were called. if (iterCalled) { - *err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, + err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, "Cannot use GetAll() method because an iteration method was called."); return; @@ -163,7 +156,7 @@ namespace ignite // Check whether GetAll was called before. if (getAllCalled) { - *err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, + err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, "Cannot use GetNext() method because GetAll() was called."); return; @@ -176,7 +169,7 @@ namespace ignite env.Get()->Context()->TargetOutStream(javaRef, OP_GET_ALL, inMem.Get()->PointerLong(), &jniErr); - IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); + IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); if (jniErr.code == IGNITE_JNI_ERR_SUCCESS) { @@ -190,38 +183,71 @@ namespace ignite } } - bool QueryCursorImpl::CreateIteratorIfNeeded(IgniteError* err) + bool QueryCursorImpl::CreateIteratorIfNeeded(IgniteError& err) { - if (!iterCalled) - { - JniErrorInfo jniErr; + if (iterCalled) + return true; - env.Get()->Context()->QueryCursorIterator(javaRef, &jniErr); + JniErrorInfo jniErr; - IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); + env.Get()->Context()->QueryCursorIterator(javaRef, &jniErr); - if (jniErr.code == IGNITE_JNI_ERR_SUCCESS) - { - iterCalled = true; + IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); + + if (jniErr.code == IGNITE_JNI_ERR_SUCCESS) + iterCalled = true; + + return iterCalled; + } + + bool QueryCursorImpl::GetNextBatchIfNeeded(IgniteError& err) + { + assert(iterCalled); + + if (endReached || (batch && batch->Left() > 0)) + return true; + + endReached = !IteratorHasNext(err); + + if (endReached) + return true; + + JniErrorInfo jniErr; + + SharedPointer inMem = env.Get()->AllocateMemory(); + + env.Get()->Context()->TargetOutStream( + javaRef, OP_GET_BATCH, inMem.Get()->PointerLong(), &jniErr); + + IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); + + if (jniErr.code != IGNITE_JNI_ERR_SUCCESS) + return false; + + delete batch; + + // Needed for exception safety. + batch = 0; + + batch = new QueryBatch(*env.Get(), inMem); + + endReached = batch->IsEmpty(); - hasNext = IteratorHasNext(err); - } - else - return false; - } - return true; } - bool QueryCursorImpl::IteratorHasNext(IgniteError* err) + bool QueryCursorImpl::IteratorHasNext(IgniteError& err) { JniErrorInfo jniErr; bool res = env.Get()->Context()->QueryCursorIteratorHasNext(javaRef, &jniErr); - IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); + IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); + + if (jniErr.code == IGNITE_JNI_ERR_SUCCESS) + return res; - return jniErr.code == IGNITE_JNI_ERR_SUCCESS && res; + return false; } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/FieldsQueryCursor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/FieldsQueryCursor.cs index d33fdce9561e0..d928418a185a9 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/FieldsQueryCursor.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/FieldsQueryCursor.cs @@ -52,6 +52,9 @@ internal class FieldsQueryCursor : AbstractQueryCursor [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")] protected override T Read(BinaryReader reader) { + // Reading and skipping row size in bytes. + reader.ReadInt(); + int cnt = reader.ReadInt(); return _readerFunc(reader, cnt); From c992213274ec5872ef7ce359efa51e26003424ad Mon Sep 17 00:00:00 2001 From: isapego Date: Sun, 4 Sep 2016 16:49:42 +0300 Subject: [PATCH 083/487] IGNITE-3760: ODBC: Added tests for supported SQL92 string functions. This closes #1006. --- .../src/sql_string_functions_test.cpp | 63 +++++++++++++++++++ .../cpp/odbc/src/config/connection_info.cpp | 4 +- 2 files changed, 65 insertions(+), 2 deletions(-) diff --git a/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp index d1ce194c1abe4..c85f80ca0f9d7 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp @@ -288,4 +288,67 @@ BOOST_AUTO_TEST_CASE(TestStringFunctionUcase) CheckSingleResult("SELECT {fn UCASE(strField)} FROM TestType", "HELLO WORLD!"); } +BOOST_AUTO_TEST_CASE(Test92StringFunctionLower) +{ + TestType in; + in.strField = "Hello World!"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT LOWER(strField) FROM TestType", "hello world!"); +} + +BOOST_AUTO_TEST_CASE(Test92StringFunctionUpper) +{ + TestType in; + in.strField = "Hello World!"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT UPPER(strField) FROM TestType", "HELLO WORLD!"); +} + +BOOST_AUTO_TEST_CASE(Test92StringFunctionSubstring) +{ + TestType in; + in.strField = "Hello Ignite!"; + + testCache.Put(1, in); + + CheckSingleResult("SELECT SUBSTRING(strField, 7) FROM TestType", "Ignite!"); + CheckSingleResult("SELECT SUBSTRING(strField, 7, 6) FROM TestType", "Ignite"); + CheckSingleResult("SELECT SUBSTRING(strField FROM 7) FROM TestType", "Ignite!"); + CheckSingleResult("SELECT SUBSTRING(strField FROM 7 FOR 6) FROM TestType", "Ignite"); +} + +BOOST_AUTO_TEST_CASE(Test92StringFunctionTrimBoth) +{ + TestType in; + in.strField = " Lorem ipsum "; + + testCache.Put(1, in); + + CheckSingleResult("SELECT TRIM(BOTH FROM strField) FROM TestType", "Lorem ipsum"); +} + +BOOST_AUTO_TEST_CASE(Test92StringFunctionTrimLeading) +{ + TestType in; + in.strField = " Lorem ipsum "; + + testCache.Put(1, in); + + CheckSingleResult("SELECT TRIM(LEADING FROM strField) FROM TestType", "Lorem ipsum "); +} + +BOOST_AUTO_TEST_CASE(Test92StringFunctionTrimTrailing) +{ + TestType in; + in.strField = " Lorem ipsum "; + + testCache.Put(1, in); + + CheckSingleResult("SELECT TRIM(TRAILING FROM strField) FROM TestType", " Lorem ipsum"); +} + BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp index cff48cf5275e1..744a88e741f96 100644 --- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp +++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp @@ -315,8 +315,8 @@ namespace ignite #ifdef SQL_SQL92_STRING_FUNCTIONS // Bitmask enumerating the string scalar functions. - intParams[SQL_SQL92_STRING_FUNCTIONS] = SQL_SSF_CONVERT | SQL_SSF_LOWER | SQL_SSF_UPPER | - SQL_SSF_SUBSTRING | SQL_SSF_TRANSLATE; + intParams[SQL_SQL92_STRING_FUNCTIONS] = SQL_SSF_LOWER | SQL_SSF_UPPER | SQL_SSF_SUBSTRING | + SQL_SSF_TRIM_BOTH | SQL_SSF_TRIM_LEADING | SQL_SSF_TRIM_TRAILING; #endif // SQL_SQL92_STRING_FUNCTIONS #ifdef SQL_SQL92_DATETIME_FUNCTIONS From d06eaa2344a753e08d1e3cb00e6b4ab83c6a9a01 Mon Sep 17 00:00:00 2001 From: isapego Date: Sun, 4 Sep 2016 16:52:04 +0300 Subject: [PATCH 084/487] IGNITE-3801: ODBC: Added tests for OUTER JOIN. This closes #1027. --- modules/platforms/cpp/odbc-test/Makefile.am | 1 + .../odbc-test/project/vs/odbc-test.vcxproj | 1 + .../project/vs/odbc-test.vcxproj.filters | 3 + .../cpp/odbc-test/src/sql_outer_join_test.cpp | 498 ++++++++++++++++++ .../odbc-test/src/sql_test_suite_fixture.cpp | 6 +- .../cpp/odbc/src/config/connection_info.cpp | 4 +- 6 files changed, 507 insertions(+), 6 deletions(-) create mode 100644 modules/platforms/cpp/odbc-test/src/sql_outer_join_test.cpp diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index 1ca85a74cdae8..c3dd86abf70f9 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -70,6 +70,7 @@ ignite_odbc_tests_SOURCES = \ src/sql_operators_test.cpp \ src/sql_value_expressions_test.cpp \ src/sql_types_test.cpp \ + src/sql_outer_join_test.cpp \ ../odbc/src/cursor.cpp \ ../odbc/src/config/connection_info.cpp \ ../odbc/src/app/application_data_buffer.cpp \ diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index cb5735f707097..b85f1e6d6ef87 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -170,6 +170,7 @@ + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index 270bdd624727c..ee5df76b1e25f 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -106,6 +106,9 @@ Code + + Code + diff --git a/modules/platforms/cpp/odbc-test/src/sql_outer_join_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_outer_join_test.cpp new file mode 100644 index 0000000000000..426041b8602e5 --- /dev/null +++ b/modules/platforms/cpp/odbc-test/src/sql_outer_join_test.cpp @@ -0,0 +1,498 @@ +/* + * 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. + */ + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include "sql_test_suite_fixture.h" +#include "test_utils.h" + +using namespace ignite; + +using namespace boost::unit_test; + +BOOST_FIXTURE_TEST_SUITE(SqlOuterJoinTestSuite, ignite::SqlTestSuiteFixture) + +// Checking that left outer joins are supported. +// Corresponds to SQL_OJ_LEFT flag. +BOOST_AUTO_TEST_CASE(TestOuterJoinLeft) +{ + TestType in1; + TestType in2; + + in1.i32Field = 20; + in2.i32Field = 30; + + in1.i16Field = 40; + in2.i16Field = 20; + + testCache.Put(1, in1); + testCache.Put(2, in2); + + SQLINTEGER columns[2]; + SQLLEN columnsLen[2]; + + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &columns[0], 0, &columnsLen[0]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 2, SQL_C_SLONG, &columns[1], 0, &columnsLen[1]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + SQLCHAR request[] = "SELECT T1.i32Field, T2.i16Field FROM " + "{oj TestType T1 LEFT OUTER JOIN TestType T2 ON T1.i32Field = T2.i16Field}"; + + ret = SQLExecDirect(stmt, request, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 20); + + BOOST_CHECK_NE(columnsLen[1], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[1], 20); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 30); + + BOOST_CHECK_EQUAL(columnsLen[1], SQL_NULL_DATA); + + ret = SQLFetch(stmt); + BOOST_CHECK(ret == SQL_NO_DATA); +} + +// Checking that the column names in the ON clause of the outer join do not +// have to be in the same order as their respective table names in the OUTER +// JOIN clause. Corresponds to SQL_OJ_NOT_ORDERED flag. +BOOST_AUTO_TEST_CASE(TestOuterJoinOrdering) +{ + TestType in1; + TestType in2; + + in1.i32Field = 20; + in2.i32Field = 30; + + in1.i16Field = 40; + in2.i16Field = 20; + + testCache.Put(1, in1); + testCache.Put(2, in2); + + SQLINTEGER columns[2]; + SQLLEN columnsLen[2]; + + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &columns[0], 0, &columnsLen[0]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 2, SQL_C_SLONG, &columns[1], 0, &columnsLen[1]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + SQLCHAR request[] = "SELECT T1.i32Field, T2.i16Field FROM " + "{oj TestType T1 LEFT OUTER JOIN TestType T2 ON T2.i16Field = T1.i32Field}"; + + ret = SQLExecDirect(stmt, request, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 20); + + BOOST_CHECK_NE(columnsLen[1], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[1], 20); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 30); + + BOOST_CHECK_EQUAL(columnsLen[1], SQL_NULL_DATA); + + ret = SQLFetch(stmt); + BOOST_CHECK(ret == SQL_NO_DATA); +} + +// Checking that the comparison operator in the ON clause can be any of the ODBC +// comparison operators. Corresponds to SQL_OJ_ALL_COMPARISON_OPS flag. +// Operator '<'. +BOOST_AUTO_TEST_CASE(TestOuterJoinOpsLess) +{ + TestType in1; + TestType in2; + + in1.i32Field = 20; + in2.i32Field = 30; + + in1.i16Field = 40; + in2.i16Field = 20; + + testCache.Put(1, in1); + testCache.Put(2, in2); + + SQLINTEGER columns[2]; + SQLLEN columnsLen[2]; + + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &columns[0], 0, &columnsLen[0]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 2, SQL_C_SLONG, &columns[1], 0, &columnsLen[1]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + SQLCHAR request[] = "SELECT T1.i32Field, T2.i16Field FROM " + "{oj TestType T1 LEFT OUTER JOIN TestType T2 ON T2.i16Field < T1.i32Field}"; + + ret = SQLExecDirect(stmt, request, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 20); + + BOOST_CHECK_EQUAL(columnsLen[1], SQL_NULL_DATA); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 30); + + BOOST_CHECK_EQUAL(columnsLen[1], SQL_NULL_DATA); + + ret = SQLFetch(stmt); + BOOST_CHECK(ret == SQL_NO_DATA); +} + +// Checking that the comparison operator in the ON clause can be any of the ODBC +// comparison operators. Corresponds to SQL_OJ_ALL_COMPARISON_OPS flag. +// Operator '>'. +BOOST_AUTO_TEST_CASE(TestOuterJoinOpsGreater) +{ + TestType in1; + TestType in2; + + in1.i32Field = 20; + in2.i32Field = 30; + + in1.i16Field = 40; + in2.i16Field = 20; + + testCache.Put(1, in1); + testCache.Put(2, in2); + + SQLINTEGER columns[2]; + SQLLEN columnsLen[2]; + + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &columns[0], 0, &columnsLen[0]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 2, SQL_C_SLONG, &columns[1], 0, &columnsLen[1]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + SQLCHAR request[] = "SELECT T1.i32Field, T2.i16Field FROM " + "{oj TestType T1 LEFT OUTER JOIN TestType T2 ON T2.i16Field > T1.i32Field}"; + + ret = SQLExecDirect(stmt, request, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 20); + + BOOST_CHECK_NE(columnsLen[1], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[1], 40); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 30); + + BOOST_CHECK_NE(columnsLen[1], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[1], 40); + + ret = SQLFetch(stmt); + BOOST_CHECK(ret == SQL_NO_DATA); +} + +// Checking that the comparison operator in the ON clause can be any of the ODBC +// comparison operators. Corresponds to SQL_OJ_ALL_COMPARISON_OPS flag. +// Operator '<='. +BOOST_AUTO_TEST_CASE(TestOuterJoinOpsLessOrEqual) +{ + TestType in1; + TestType in2; + + in1.i32Field = 20; + in2.i32Field = 30; + + in1.i16Field = 40; + in2.i16Field = 20; + + testCache.Put(1, in1); + testCache.Put(2, in2); + + SQLINTEGER columns[2]; + SQLLEN columnsLen[2]; + + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &columns[0], 0, &columnsLen[0]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 2, SQL_C_SLONG, &columns[1], 0, &columnsLen[1]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + SQLCHAR request[] = "SELECT T1.i32Field, T2.i16Field FROM " + "{oj TestType T1 LEFT OUTER JOIN TestType T2 ON T2.i16Field <= T1.i32Field}"; + + ret = SQLExecDirect(stmt, request, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 20); + + BOOST_CHECK_NE(columnsLen[1], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[1], 20); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 30); + + BOOST_CHECK_NE(columnsLen[1], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[1], 20); + + ret = SQLFetch(stmt); + BOOST_CHECK(ret == SQL_NO_DATA); +} + +// Checking that the comparison operator in the ON clause can be any of the ODBC +// comparison operators. Corresponds to SQL_OJ_ALL_COMPARISON_OPS flag. +// Operator '>='. +BOOST_AUTO_TEST_CASE(TestOuterJoinOpsGreaterOrEqual) +{ + TestType in1; + TestType in2; + + in1.i32Field = 20; + in2.i32Field = 30; + + in1.i16Field = 40; + in2.i16Field = 20; + + testCache.Put(1, in1); + testCache.Put(2, in2); + + SQLINTEGER columns[2]; + SQLLEN columnsLen[2]; + + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &columns[0], 0, &columnsLen[0]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 2, SQL_C_SLONG, &columns[1], 0, &columnsLen[1]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + SQLCHAR request[] = "SELECT T1.i32Field, T2.i16Field FROM " + "{oj TestType T1 LEFT OUTER JOIN TestType T2 ON T2.i16Field >= T1.i32Field}"; + + ret = SQLExecDirect(stmt, request, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 20); + + BOOST_CHECK_NE(columnsLen[1], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[1], 40); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 20); + + BOOST_CHECK_NE(columnsLen[1], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[1], 20); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 30); + + BOOST_CHECK_NE(columnsLen[1], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[1], 40); + + ret = SQLFetch(stmt); + BOOST_CHECK(ret == SQL_NO_DATA); +} + +// Checking that the comparison operator in the ON clause can be any of the ODBC +// comparison operators. Corresponds to SQL_OJ_ALL_COMPARISON_OPS flag. +// Operator '!='. +BOOST_AUTO_TEST_CASE(TestOuterJoinOpsNotEqual) +{ + TestType in1; + TestType in2; + + in1.i32Field = 20; + in2.i32Field = 30; + + in1.i16Field = 40; + in2.i16Field = 20; + + testCache.Put(1, in1); + testCache.Put(2, in2); + + SQLINTEGER columns[2]; + SQLLEN columnsLen[2]; + + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &columns[0], 0, &columnsLen[0]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 2, SQL_C_SLONG, &columns[1], 0, &columnsLen[1]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + SQLCHAR request[] = "SELECT T1.i32Field, T2.i16Field FROM " + "{oj TestType T1 LEFT OUTER JOIN TestType T2 ON T2.i16Field != T1.i32Field}"; + + ret = SQLExecDirect(stmt, request, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 20); + + BOOST_CHECK_NE(columnsLen[1], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[1], 40); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 30); + + BOOST_CHECK_NE(columnsLen[1], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[1], 40); + + ret = SQLFetch(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[0], 30); + + BOOST_CHECK_NE(columnsLen[1], SQL_NULL_DATA); + BOOST_CHECK_EQUAL(columns[1], 20); + + ret = SQLFetch(stmt); + BOOST_CHECK(ret == SQL_NO_DATA); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp index 69b4bfa49662b..657b8540721be 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp @@ -117,17 +117,17 @@ namespace ignite ret = SQLBindCol(stmt, 1, type, column, bufSize, resSize); if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ; + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); ret = SQLExecDirect(stmt, reinterpret_cast(const_cast(request)), SQL_NTS); if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ; + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); ret = SQLFetch(stmt); if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ; + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); ret = SQLFetch(stmt); BOOST_CHECK(ret == SQL_NO_DATA) ; diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp index 744a88e741f96..ca8d1a0102210 100644 --- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp +++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp @@ -298,9 +298,7 @@ namespace ignite #ifdef SQL_OJ_CAPABILITIES // Bitmask enumerating the types of outer joins supported by the // driver and data source. - intParams[SQL_OJ_CAPABILITIES] = SQL_OJ_LEFT | SQL_OJ_RIGHT | - SQL_OJ_FULL | SQL_OJ_NESTED | SQL_OJ_INNER | - SQL_OJ_ALL_COMPARISON_OPS; + intParams[SQL_OJ_CAPABILITIES] = SQL_OJ_LEFT | SQL_OJ_NOT_ORDERED | SQL_OJ_ALL_COMPARISON_OPS; #endif // SQL_OJ_CAPABILITIES #ifdef SQL_POS_OPERATIONS From f8ae67456703e63e3afc9bb5c21d81d576d59448 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Sun, 4 Sep 2016 17:09:08 +0300 Subject: [PATCH 085/487] IGNITE-3651 IGFS: Implemented "usedSpaceSize()" operation for local secondary file system. This closes #1009. --- .../local/LocalIgfsSecondaryFileSystem.java | 15 ++++- .../local/LocalFileSystemSizeVisitor.java | 60 +++++++++++++++++++ ...condaryFileSystemDualAbstractSelfTest.java | 43 +++++++++++++ 3 files changed, 117 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemSizeVisitor.java diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java index 519f4728c5464..36080f2cbf802 100644 --- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java +++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java @@ -27,6 +27,7 @@ import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable; import org.apache.ignite.internal.processors.igfs.secondary.local.LocalFileSystemIgfsFile; +import org.apache.ignite.internal.processors.igfs.secondary.local.LocalFileSystemSizeVisitor; import org.apache.ignite.internal.processors.igfs.secondary.local.LocalIgfsSecondaryFileSystemPositionedReadable; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; @@ -42,6 +43,7 @@ import java.io.OutputStream; import java.nio.file.Files; import java.nio.file.LinkOption; +import java.nio.file.Path; import java.nio.file.attribute.BasicFileAttributes; import java.util.Collection; import java.util.Collections; @@ -301,7 +303,18 @@ private boolean mkdirs0(@Nullable File dir) { /** {@inheritDoc} */ @Override public long usedSpaceSize() { - throw new UnsupportedOperationException("usedSpaceSize operation is not yet supported."); + Path p = fileForPath(new IgfsPath("/")).toPath(); + + try { + LocalFileSystemSizeVisitor visitor = new LocalFileSystemSizeVisitor(); + + Files.walkFileTree(p, visitor); + + return visitor.size(); + } + catch (IOException e) { + throw new IgfsException("Failed to calculate used space size.", e); + } } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemSizeVisitor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemSizeVisitor.java new file mode 100644 index 0000000000000..222b749805487 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemSizeVisitor.java @@ -0,0 +1,60 @@ +/* + * 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.igfs.secondary.local; + +import java.io.IOException; +import java.nio.file.FileVisitResult; +import java.nio.file.FileVisitor; +import java.nio.file.Path; +import java.nio.file.attribute.BasicFileAttributes; + +/** + * File visitor to get occupied file system size. + */ +public class LocalFileSystemSizeVisitor implements FileVisitor { + /** File size accumulator. */ + private long size; + + /** {@inheritDoc} */ + @Override public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) throws IOException { + return FileVisitResult.CONTINUE; + } + + /** {@inheritDoc} */ + @Override public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { + size += attrs.size(); + return FileVisitResult.CONTINUE; + } + + /** {@inheritDoc} */ + @Override public FileVisitResult visitFileFailed(Path file, IOException exc) throws IOException { + return FileVisitResult.CONTINUE; + } + + /** {@inheritDoc} */ + @Override public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { + return FileVisitResult.CONTINUE; + } + + /** + * @return Total size of visited files. + */ + public long size() { + return size; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java index 1d1ce8d76cf7a..df7d782f706bc 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java @@ -24,6 +24,7 @@ import org.apache.ignite.igfs.secondary.local.LocalIgfsSecondaryFileSystem; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiInClosure; import org.jetbrains.annotations.Nullable; import java.io.File; @@ -32,6 +33,7 @@ import java.io.OutputStream; import java.nio.file.Files; import java.util.Collection; +import java.util.concurrent.atomic.AtomicLong; /** * Abstract test for Hadoop 1.0 file system stack. @@ -166,6 +168,47 @@ public void testSymlinkToFile() throws Exception { checkFileContent(igfs, new IgfsPath("/file"), chunk); } + /** + * + * @throws Exception If failed. + */ + public void testUsedSpaceSize() throws Exception { + final int DIRS_COUNT = 5; + final int DIRS_MAX_DEEP = 3; + final int FILES_COUNT = 10; + final AtomicLong totalSize = new AtomicLong(); + + IgniteBiInClosure createHierarchy = new IgniteBiInClosure() { + @Override public void apply(Integer level, IgfsPath levelDir) { + try { + for (int i = 0; i < FILES_COUNT; ++i) { + IgfsPath filePath = new IgfsPath(levelDir, "file" + Integer.toString(i)); + + createFile(igfs, filePath, true, chunk); + + totalSize.getAndAdd(chunk.length); + } + + if (level < DIRS_MAX_DEEP) { + for (int dir = 0; dir < DIRS_COUNT; dir++) { + IgfsPath dirPath = new IgfsPath(levelDir, "dir" + Integer.toString(dir)); + + igfs.mkdirs(dirPath); + + apply(level + 1, dirPath); + } + } + } catch (Exception e) { + fail(e.getMessage()); + } + } + }; + + createHierarchy.apply(1, new IgfsPath("/dir")); + + assertEquals(totalSize.get(), igfs.metrics().secondarySpaceSize()); + } + /** * * @throws Exception If failed. From fc2fe7bf1905675258d40932a4ff649156c17488 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Sun, 4 Sep 2016 17:12:35 +0300 Subject: [PATCH 086/487] IGNITE-3646: IGFS: Added test for symlinked mkdirs on local secondary file system. This closes #1013. --- ...condaryFileSystemDualAbstractSelfTest.java | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java index df7d782f706bc..8baaf4a4dff8d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java @@ -61,7 +61,9 @@ public abstract class IgfsLocalSecondaryFileSystemDualAbstractSelfTest extends I private final File fileLinkSrc = new File(FS_WORK_DIR + File.separatorChar + "file"); - /** Constructor. + /** + * Constructor. + * * @param mode IGFS mode. */ public IgfsLocalSecondaryFileSystemDualAbstractSelfTest(IgfsMode mode) { @@ -168,6 +170,22 @@ public void testSymlinkToFile() throws Exception { checkFileContent(igfs, new IgfsPath("/file"), chunk); } + /** + * + * @throws Exception If failed. + */ + public void testMkdirsInsideSymlink() throws Exception { + if (U.isWindows()) + return; + + createSymlinks(); + + igfs.mkdirs(SUBSUBDIR); + + assertTrue(Files.isDirectory(dirLinkDest.toPath().resolve("subdir/subsubdir"))); + assertTrue(Files.isDirectory(dirLinkSrc.toPath().resolve("subdir/subsubdir"))); + } + /** * * @throws Exception If failed. From 3aa13f716934a6ccfe49f8bf99ec3b654e263900 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 5 Sep 2016 10:19:48 +0300 Subject: [PATCH 087/487] IGNITE-3829: Additional fix. --- .../cache/binary/CacheObjectBinaryProcessorImpl.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java index ecd27f7a19af7..82e67aca1518a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java @@ -709,10 +709,15 @@ public Object affinityKey(BinaryObject po) { if (meta != null) { String name = meta.affinityKeyFieldName(); - affKeyFields.putIfAbsent(meta.typeId(), new T1<>(meta.field(name))); + if (name != null) { + BinaryField field = meta.field(name); - if (name != null) - return po.field(name); + affKeyFields.putIfAbsent(meta.typeId(), new T1<>(field)); + + return field.value(po); + } + else + affKeyFields.putIfAbsent(meta.typeId(), new T1(null)); } else if (po instanceof BinaryObjectEx) { int typeId = ((BinaryObjectEx)po).typeId(); From a760918757bee71ab28495496f94e9067ef17888 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Mon, 5 Sep 2016 10:36:38 +0300 Subject: [PATCH 088/487] IGNITE-3750: ODBC: Added tests for date/time types. This closes #1002. --- .../processors/odbc/OdbcMessageParser.java | 10 +- modules/platforms/cpp/odbc-test/Makefile.am | 1 + .../include/sql_test_suite_fixture.h | 6 + .../odbc-test/project/vs/odbc-test.vcxproj | 1 + .../project/vs/odbc-test.vcxproj.filters | 3 + .../cpp/odbc-test/src/queries_test.cpp | 1 + .../src/sql_date_time_functions_test.cpp | 213 ++++++++++++++++++ .../odbc-test/src/sql_test_suite_fixture.cpp | 17 ++ modules/platforms/cpp/odbc/src/column.cpp | 14 +- .../cpp/odbc/src/config/connection_info.cpp | 16 +- 10 files changed, 261 insertions(+), 21 deletions(-) create mode 100644 modules/platforms/cpp/odbc-test/src/sql_date_time_functions_test.cpp diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java index a751eb211a98f..3accf742f9dc3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java @@ -232,8 +232,14 @@ else if (res0 instanceof OdbcQueryFetchResult) { writer.writeInt(row.size()); - for (Object obj : row) - writer.writeObjectDetached(obj); + for (Object obj : row) { + if (obj instanceof java.sql.Timestamp) + writer.writeTimestamp((java.sql.Timestamp)obj); + else if (obj instanceof java.util.Date) + writer.writeDate((java.util.Date)obj); + else + writer.writeObjectDetached(obj); + } } } } diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index c3dd86abf70f9..a22e247c92f72 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -70,6 +70,7 @@ ignite_odbc_tests_SOURCES = \ src/sql_operators_test.cpp \ src/sql_value_expressions_test.cpp \ src/sql_types_test.cpp \ + src/sql_date_time_functions_test.cpp \ src/sql_outer_join_test.cpp \ ../odbc/src/cursor.cpp \ ../odbc/src/config/connection_info.cpp \ diff --git a/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h b/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h index 9e482da4cd449..6d26818b5c936 100644 --- a/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h +++ b/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h @@ -186,6 +186,12 @@ namespace ignite template<> void SqlTestSuiteFixture::CheckSingleResult(const char* request); + + template<> + void SqlTestSuiteFixture::CheckSingleResult(const char* request); + + template<> + void SqlTestSuiteFixture::CheckSingleResult(const char* request); } #endif //_IGNITE_ODBC_TEST_SQL_TEST_SUIT_FIXTURE diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index b85f1e6d6ef87..98a1e587c6553 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -171,6 +171,7 @@ + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index ee5df76b1e25f..f348ee7dd5ee6 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -109,6 +109,9 @@ Code + + Code + diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 4ba3a634daee0..7c10527b4818f 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -121,6 +121,7 @@ struct QueriesTestSuiteFixture cfg.jvmOpts.push_back("-Djava.compiler=NONE"); cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005"); cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError"); + cfg.jvmOpts.push_back("-Duser.timezone=GMT"); #ifdef IGNITE_TESTS_32 cfg.jvmInitMem = 256; diff --git a/modules/platforms/cpp/odbc-test/src/sql_date_time_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_date_time_functions_test.cpp new file mode 100644 index 0000000000000..f89cc3d0f9718 --- /dev/null +++ b/modules/platforms/cpp/odbc-test/src/sql_date_time_functions_test.cpp @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include "sql_test_suite_fixture.h" + +using namespace ignite; + +using namespace boost::unit_test; + +BOOST_FIXTURE_TEST_SUITE(SqlDateTimeFunctionTestSuite, ignite::SqlTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(TestCurrentDate) +{ + CheckSingleResult("SELECT {fn CURRENT_DATE()}"); +} + +BOOST_AUTO_TEST_CASE(TestCurdate) +{ + CheckSingleResult("SELECT {fn CURDATE()}"); +} + +BOOST_AUTO_TEST_CASE(TestCurrentTime) +{ + CheckSingleResult("SELECT {fn CURRENT_TIME()}"); +} + +BOOST_AUTO_TEST_CASE(TestCurtime) +{ + CheckSingleResult("SELECT {fn CURTIME()}"); +} + +BOOST_AUTO_TEST_CASE(TestCurrentTimestamp) +{ + CheckSingleResult("SELECT {fn CURRENT_TIMESTAMP()}"); +} + +BOOST_AUTO_TEST_CASE(TestDayname) +{ + TestType in; + + in.dateField = impl::binary::BinaryUtils::MakeDateGmt(2016, 8, 29); + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn DAYNAME(dateField)} FROM TestType", "Monday"); +} + +BOOST_AUTO_TEST_CASE(TestDayofmonth) +{ + TestType in; + + in.dateField = impl::binary::BinaryUtils::MakeDateGmt(2016, 8, 29); + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn DAYOFMONTH(dateField)} FROM TestType", 29); + CheckSingleResult("SELECT {fn DAY_OF_MONTH(dateField)} FROM TestType", 29); +} + +BOOST_AUTO_TEST_CASE(TestDayofweek) +{ + TestType in; + + in.dateField = impl::binary::BinaryUtils::MakeDateGmt(2016, 8, 29); + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn DAYOFWEEK(dateField)} FROM TestType", 2); + CheckSingleResult("SELECT {fn DAY_OF_WEEK(dateField)} FROM TestType", 2); +} + +BOOST_AUTO_TEST_CASE(TestDayofyear) +{ + TestType in; + + in.dateField = impl::binary::BinaryUtils::MakeDateGmt(2016, 8, 29); + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn DAYOFYEAR(dateField)} FROM TestType", 242); + CheckSingleResult("SELECT {fn DAY_OF_YEAR(dateField)} FROM TestType", 242); +} + +BOOST_AUTO_TEST_CASE(TestExtract) +{ + TestType in; + + in.timestampField = impl::binary::BinaryUtils::MakeTimestampGmt(2016, 2, 24, 13, 45, 23, 580695103); + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn EXTRACT(YEAR FROM timestampField)} FROM TestType", 2016); + CheckSingleResult("SELECT {fn EXTRACT(MONTH FROM timestampField)} FROM TestType", 2); + CheckSingleResult("SELECT {fn EXTRACT(DAY FROM timestampField)} FROM TestType", 24); + CheckSingleResult("SELECT {fn EXTRACT(HOUR FROM timestampField)} FROM TestType", 13); + CheckSingleResult("SELECT {fn EXTRACT(MINUTE FROM timestampField)} FROM TestType", 45); + CheckSingleResult("SELECT {fn EXTRACT(SECOND FROM timestampField)} FROM TestType", 23); +} + +BOOST_AUTO_TEST_CASE(TestHour) +{ + TestType in; + + in.timestampField = impl::binary::BinaryUtils::MakeTimestampGmt(2016, 2, 24, 13, 45, 23, 580695103); + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn HOUR(timestampField)} FROM TestType", 13); +} + +BOOST_AUTO_TEST_CASE(TestMinute) +{ + TestType in; + + in.timestampField = impl::binary::BinaryUtils::MakeTimestampGmt(2016, 2, 24, 13, 45, 23, 580695103); + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn MINUTE(timestampField)} FROM TestType", 45); +} + +BOOST_AUTO_TEST_CASE(TestMonth) +{ + TestType in; + + in.timestampField = impl::binary::BinaryUtils::MakeTimestampGmt(2016, 2, 24, 13, 45, 23, 580695103); + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn MONTH(timestampField)} FROM TestType", 2); +} + +BOOST_AUTO_TEST_CASE(TestMonthname) +{ + TestType in; + + in.timestampField = impl::binary::BinaryUtils::MakeTimestampGmt(2016, 2, 24, 13, 45, 23, 580695103); + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn MONTHNAME(timestampField)} FROM TestType", "February"); +} + +BOOST_AUTO_TEST_CASE(TestNow) +{ + CheckSingleResult("SELECT {fn NOW()}"); +} + +BOOST_AUTO_TEST_CASE(TestQuarter) +{ + TestType in; + + in.timestampField = impl::binary::BinaryUtils::MakeTimestampGmt(2016, 2, 24, 13, 45, 23, 580695103); + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn QUARTER(timestampField)} FROM TestType", 1); +} + +BOOST_AUTO_TEST_CASE(TestSecond) +{ + TestType in; + + in.timestampField = impl::binary::BinaryUtils::MakeTimestampGmt(2016, 2, 24, 13, 45, 23, 580695103); + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn SECOND(timestampField)} FROM TestType", 23); +} + +BOOST_AUTO_TEST_CASE(TestWeek) +{ + TestType in; + + in.timestampField = impl::binary::BinaryUtils::MakeTimestampGmt(2016, 2, 24, 13, 45, 23, 580695103); + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn WEEK(timestampField)} FROM TestType", 9); +} + +BOOST_AUTO_TEST_CASE(TestYear) +{ + TestType in; + + in.timestampField = impl::binary::BinaryUtils::MakeTimestampGmt(2016, 2, 24, 13, 45, 23, 580695103); + + testCache.Put(1, in); + + CheckSingleResult("SELECT {fn YEAR(timestampField)} FROM TestType", 2016); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp index 657b8540721be..e9a8fc5770231 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp @@ -34,6 +34,7 @@ namespace ignite cfg.jvmOpts.push_back("-Djava.compiler=NONE"); cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005"); cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError"); + cfg.jvmOpts.push_back("-Duser.timezone=GMT"); #ifdef IGNITE_TESTS_32 cfg.jvmInitMem = 256; @@ -268,4 +269,20 @@ namespace ignite CheckSingleResult0(request, SQL_C_DOUBLE, &res, 0, 0); } + + template<> + void SqlTestSuiteFixture::CheckSingleResult(const char* request) + { + SQL_DATE_STRUCT res; + + CheckSingleResult0(request, SQL_C_DATE, &res, 0, 0); + } + + template<> + void SqlTestSuiteFixture::CheckSingleResult(const char* request) + { + SQL_TIMESTAMP_STRUCT res; + + CheckSingleResult0(request, SQL_C_TIMESTAMP, &res, 0, 0); + } } diff --git a/modules/platforms/cpp/odbc/src/column.cpp b/modules/platforms/cpp/odbc/src/column.cpp index ec779ac15ef43..b076a12ee8c5e 100644 --- a/modules/platforms/cpp/odbc/src/column.cpp +++ b/modules/platforms/cpp/odbc/src/column.cpp @@ -58,7 +58,7 @@ namespace * complex type. * @return Column type header. */ - int8_t ReadColumnHeader(ignite::impl::interop::InteropInputStream& stream) + int8_t ReadColumnHeader(InteropInputStream& stream) { using namespace ignite::impl::binary; @@ -130,10 +130,10 @@ namespace ignite // No-op. } - Column::Column(ignite::impl::binary::BinaryReaderImpl& reader) : + Column::Column(BinaryReaderImpl& reader) : type(0), startPos(-1), endPos(-1), offset(0), size(0) { - ignite::impl::interop::InteropInputStream* stream = reader.GetStream(); + InteropInputStream* stream = reader.GetStream(); if (!stream) return; @@ -294,12 +294,8 @@ namespace ignite size = sizeTmp; } - SqlResult Column::ReadToBuffer(ignite::impl::binary::BinaryReaderImpl& reader, - app::ApplicationDataBuffer& dataBuf) + SqlResult Column::ReadToBuffer(BinaryReaderImpl& reader, app::ApplicationDataBuffer& dataBuf) { - using namespace ignite::impl::binary; - using namespace ignite::impl::interop; - if (!IsValid()) return SQL_RESULT_ERROR; @@ -310,7 +306,7 @@ namespace ignite return SQL_RESULT_NO_DATA; } - ignite::impl::interop::InteropInputStream* stream = reader.GetStream(); + InteropInputStream* stream = reader.GetStream(); if (!stream) return SQL_RESULT_ERROR; diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp index ca8d1a0102210..ee2c22b668c75 100644 --- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp +++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp @@ -254,7 +254,11 @@ namespace ignite #ifdef SQL_TIMEDATE_FUNCTIONS // Bitmask enumerating the scalar date and time functions supported // by the driver and associated data source. - intParams[SQL_TIMEDATE_FUNCTIONS] = 0; + intParams[SQL_TIMEDATE_FUNCTIONS] = SQL_FN_TD_CURRENT_DATE | SQL_FN_TD_CURRENT_TIME | + SQL_FN_TD_CURRENT_TIMESTAMP | SQL_FN_TD_CURDATE | SQL_FN_TD_CURTIME | SQL_FN_TD_DAYNAME | + SQL_FN_TD_DAYOFMONTH | SQL_FN_TD_DAYOFWEEK | SQL_FN_TD_DAYOFYEAR | SQL_FN_TD_EXTRACT | + SQL_FN_TD_HOUR | SQL_FN_TD_MINUTE | SQL_FN_TD_MONTH | SQL_FN_TD_MONTHNAME | SQL_FN_TD_NOW | + SQL_FN_TD_QUARTER | SQL_FN_TD_SECOND | SQL_FN_TD_WEEK | SQL_FN_TD_YEAR; #endif // SQL_TIMEDATE_FUNCTIONS #ifdef SQL_TIMEDATE_ADD_INTERVALS @@ -272,15 +276,7 @@ namespace ignite #ifdef SQL_DATETIME_LITERALS // Bitmask enumerating the SQL-92 datetime literals supported by // the data source. - intParams[SQL_DATETIME_LITERALS] = SQL_DL_SQL92_INTERVAL_HOUR | - SQL_DL_SQL92_DATE | SQL_DL_SQL92_INTERVAL_MINUTE_TO_SECOND | - SQL_DL_SQL92_TIME | SQL_DL_SQL92_INTERVAL_HOUR_TO_SECOND | - SQL_DL_SQL92_TIMESTAMP | SQL_DL_SQL92_INTERVAL_HOUR_TO_MINUTE | - SQL_DL_SQL92_INTERVAL_YEAR | SQL_DL_SQL92_INTERVAL_DAY_TO_SECOND | - SQL_DL_SQL92_INTERVAL_MONTH | SQL_DL_SQL92_INTERVAL_DAY_TO_HOUR | - SQL_DL_SQL92_INTERVAL_DAY | SQL_DL_SQL92_INTERVAL_DAY_TO_MINUTE | - SQL_DL_SQL92_INTERVAL_MINUTE | SQL_DL_SQL92_INTERVAL_SECOND | - SQL_DL_SQL92_INTERVAL_YEAR_TO_MONTH; + intParams[SQL_DATETIME_LITERALS] = SQL_DL_SQL92_DATE | SQL_DL_SQL92_TIME | SQL_DL_SQL92_TIMESTAMP; #endif // SQL_DATETIME_LITERALS #ifdef SQL_SYSTEM_FUNCTIONS From afac3fab5f22250b830383a9f10336d396c83ad9 Mon Sep 17 00:00:00 2001 From: Saikat Maitra Date: Mon, 5 Sep 2016 15:06:39 +0700 Subject: [PATCH 089/487] IGNITE-1952 Visorcmd: add a command for reset (clear) of metrics. Fixes #1029. --- .../ignite/visor/commands/VisorConsole.scala | 1 + .../commands/cache/VisorCacheCommand.scala | 33 ++++- .../cache/VisorCacheResetCommand.scala | 129 ++++++++++++++++++ .../cache/VisorCacheResetCommandSpec.scala | 114 ++++++++++++++++ 4 files changed, 271 insertions(+), 6 deletions(-) create mode 100644 modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheResetCommand.scala create mode 100644 modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/cache/VisorCacheResetCommandSpec.scala diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala index b4d78b5eb80ec..ad8c2edaaf8dd 100644 --- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala +++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala @@ -62,6 +62,7 @@ class VisorConsole { org.apache.ignite.visor.commands.ack.VisorAckCommand org.apache.ignite.visor.commands.alert.VisorAlertCommand org.apache.ignite.visor.commands.cache.VisorCacheClearCommand + org.apache.ignite.visor.commands.cache.VisorCacheResetCommand org.apache.ignite.visor.commands.cache.VisorCacheCommand org.apache.ignite.visor.commands.cache.VisorCacheSwapCommand org.apache.ignite.visor.commands.config.VisorConfigurationCommand 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 68465de062ac3..1fa185f3ffb5f 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 @@ -58,6 +58,9 @@ import scala.language.{implicitConversions, reflectiveCalls} * +-----------------------------------------------------------------------------------------+ * | cache -stop | Stop cache with specified name. | * +-----------------------------------------------------------------------------------------+ + * | cache -reset | Reset metrics for cache with specified name. | + * +-----------------------------------------------------------------------------------------+ + * * }}} * * ====Specification==== @@ -69,6 +72,7 @@ import scala.language.{implicitConversions, reflectiveCalls} * cache -scan -c= {-id=|id8=} {-p=} {-system} * cache -swap {-c=} {-id=|id8=} * cache -stop -c= + * cache -reset -c= * }}} * * ====Arguments==== @@ -113,6 +117,8 @@ import scala.language.{implicitConversions, reflectiveCalls} * Swaps backup entries in cache. * -stop * Stop cache with specified name. + * -reset + * Reset metrics for cache with specified name. * -p= * Number of object to fetch from cache at once. * Valid range from 1 to 100. @@ -153,6 +159,9 @@ import scala.language.{implicitConversions, reflectiveCalls} * Swaps entries in cache with name taken from 'c0' memory variable. * cache -stop -c=cache * Stops cache with name 'cache'. + * cache -reset -c=cache + * Reset metrics for cache with name 'cache'. + * * }}} */ class VisorCacheCommand { @@ -210,6 +219,9 @@ class VisorCacheCommand { *
          * cache -stop -c=@c0 * Stop cache with name taken from 'c0' memory variable. + *
          + * cache -reset -c=@c0 + * Reset metrics for cache with name taken from 'c0' memory variable. * * @param args Command arguments. */ @@ -260,9 +272,9 @@ class VisorCacheCommand { // Get cache stats data from all nodes. val aggrData = cacheData(node, cacheName, showSystem) - if (hasArgFlagIn("clear", "swap", "scan", "stop")) { + if (hasArgFlagIn("clear", "swap", "scan", "stop", "reset")) { if (cacheName.isEmpty) - askForCache("Select cache from:", node, showSystem && !hasArgFlagIn("clear", "swap", "stop"), aggrData) match { + askForCache("Select cache from:", node, showSystem && !hasArgFlagIn("clear", "swap", "stop", "reset"), aggrData) match { case Some(name) => argLst = argLst ++ Seq("c" -> name) @@ -282,6 +294,8 @@ class VisorCacheCommand { VisorCacheSwapCommand().swap(argLst, node) else if (hasArgFlag("stop", argLst)) VisorCacheStopCommand().stop(argLst, node) + else if (hasArgFlag("reset", argLst)) + VisorCacheResetCommand().reset(argLst, node) } else { if (hasArgFlag("clear", argLst)) @@ -290,6 +304,8 @@ class VisorCacheCommand { warn("Backup swapping of system cache is not allowed: " + name) else if (hasArgFlag("stop", argLst)) warn("Stopping of system cache is not allowed: " + name) + else if (hasArgFlag("reset", argLst)) + warn("Reset metrics of system cache is not allowed: " + name) } } }) @@ -716,8 +732,9 @@ object VisorCacheCommand { "cache -clear {-c=} {-id=|id8=}", "cache -scan -c= {-id=|id8=} {-p=}", "cache -swap {-c=} {-id=|id8=}", - "cache -stop -c=" - ), + "cache -stop -c=", + "cache -reset -c=" + ), args = Seq( "-id8=" -> Seq( "ID8 of the node to get cache statistics from.", @@ -749,7 +766,10 @@ object VisorCacheCommand { "Swaps backup entries in cache." ), "-stop" -> Seq( - "Stop cache with specified name" + "Stop cache with specified name." + ), + "-reset" -> Seq( + "Reset metrics of cache with specified name." ), "-s=hi|mi|rd|wr|cn" -> Seq( "Defines sorting type. Sorted by:", @@ -809,7 +829,8 @@ object VisorCacheCommand { "cache -swap" -> "Swaps entries in interactively selected cache.", "cache -swap -c=cache" -> "Swaps entries in cache with name 'cache'.", "cache -swap -c=@c0" -> "Swaps entries in cache with name taken from 'c0' memory variable.", - "cache -stop -c=@c0" -> "Stop cache with name taken from 'c0' memory variable." + "cache -stop -c=@c0" -> "Stop cache with name taken from 'c0' memory variable.", + "cache -reset -c=@c0" -> "Reset metrics for cache with name taken from 'c0' memory variable." ), emptyArgs = cmd.cache, withArgs = cmd.cache diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheResetCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheResetCommand.scala new file mode 100644 index 0000000000000..b59155b62cb85 --- /dev/null +++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheResetCommand.scala @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.visor.commands.cache + +import org.apache.ignite.cluster.{ClusterGroupEmptyException, ClusterNode} +import org.apache.ignite.internal.visor.cache.VisorCacheResetMetricsTask +import org.apache.ignite.internal.visor.util.VisorTaskUtils._ +import org.apache.ignite.visor.visor._ + +import scala.language.reflectiveCalls + +/** + * ==Overview== + * Visor 'reset' command implementation. + * + * ====Specification==== + * {{{ + * cache -reset -c= + * }}} + * + * ====Arguments==== + * {{{ + * + * Name of the cache. + * }}} + * + * ====Examples==== + * {{{ + * cache -reset -c=@c0 + * Reset metrics for cache with name taken from 'c0' memory variable. + * }}} + */ +class VisorCacheResetCommand { + /** + * Prints error message and advise. + * + * @param errMsgs Error messages. + */ + private def scold(errMsgs: Any*) { + assert(errMsgs != null) + + warn(errMsgs: _*) + warn("Type 'help cache' to see how to use this command.") + } + + private def error(e: Exception) { + var cause: Throwable = e + + while (cause.getCause != null) + cause = cause.getCause + + scold(cause.getMessage) + } + + /** + * ===Command=== + * Reset metrics for cache with specified name. + * + * ===Examples=== + * cache -c=cache -reset + * Reset metrics for cache with name 'cache'. + * + * @param argLst Command arguments. + */ + def reset(argLst: ArgList, node: Option[ClusterNode]) { + val cacheArg = argValue("c", argLst) + + val cacheName = cacheArg match { + case None => null // default cache. + + case Some(s) if s.startsWith("@") => + warn("Can't find cache variable with specified name: " + s, + "Type 'cache' to see available cache variables." + ) + + return + + case Some(name) => name + } + + val grp = try { + groupForDataNode(node, cacheName) + } + catch { + case _: ClusterGroupEmptyException => + scold(messageNodeNotFound(node, cacheName)) + + return + } + + try { + executeRandom(grp, classOf[VisorCacheResetMetricsTask], cacheName) + + println("Visor successfully reset metrics for cache: " + escapeName(cacheName)) + } + catch { + case _: ClusterGroupEmptyException => scold(messageNodeNotFound(node, cacheName)) + case e: Exception => error(e) + } + } +} + +/** + * Companion object that does initialization of the command. + */ +object VisorCacheResetCommand { + /** Singleton command. */ + private val cmd = new VisorCacheResetCommand + + /** + * Singleton. + */ + def apply() = cmd +} diff --git a/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/cache/VisorCacheResetCommandSpec.scala b/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/cache/VisorCacheResetCommandSpec.scala new file mode 100644 index 0000000000000..18f728c311dca --- /dev/null +++ b/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/cache/VisorCacheResetCommandSpec.scala @@ -0,0 +1,114 @@ +/* + * 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.visor.commands.cache + +import org.apache.ignite.Ignition +import org.apache.ignite.cache.CacheAtomicityMode._ +import org.apache.ignite.cache.CacheMode._ +import org.apache.ignite.configuration.{CacheConfiguration, IgniteConfiguration} +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder +import org.apache.ignite.visor.commands.cache.VisorCacheCommand._ +import org.apache.ignite.visor.{VisorRuntimeBaseSpec, visor} +import org.jetbrains.annotations.Nullable + +import scala.collection.JavaConversions._ + +/** + * Unit test for 'reset' command. + */ +class VisorCacheResetCommandSpec extends VisorRuntimeBaseSpec(2) { + /** IP finder. */ + val ipFinder = new TcpDiscoveryVmIpFinder(true) + + /** + * Creates grid configuration for provided grid host. + * + * @param name Grid name. + * @return Grid configuration. + */ + override def config(name: String): IgniteConfiguration = { + val cfg = new IgniteConfiguration + + cfg.setGridName(name) + cfg.setLocalHost("127.0.0.1") + cfg.setCacheConfiguration(cacheConfig(null), cacheConfig("cache")) + + val discoSpi = new TcpDiscoverySpi() + + discoSpi.setIpFinder(ipFinder) + + cfg.setDiscoverySpi(discoSpi) + + cfg + } + + /** + * @param name Cache name. + * @return Cache Configuration. + */ + def cacheConfig(@Nullable name: String): CacheConfiguration[Object, Object] = { + val cfg = new CacheConfiguration[Object, Object] + + cfg.setCacheMode(REPLICATED) + cfg.setAtomicityMode(TRANSACTIONAL) + cfg.setName(name) + + cfg + } + + describe("A 'reset' visor command") { + it("should show correct result for default cache") { + Ignition.ignite("node-1").cache[Int, Int](null).putAll(Map(1 -> 1, 2 -> 2, 3 -> 3)) + + val lock = Ignition.ignite("node-1").cache[Int, Int](null).lock(1) + + lock.lock() + + VisorCacheResetCommand().reset(Nil, None) + + lock.unlock() + + VisorCacheResetCommand().reset(Nil, None) + } + + it("should show correct result for named cache") { + Ignition.ignite("node-1").cache[Int, Int]("cache").putAll(Map(1 -> 1, 2 -> 2, 3 -> 3)) + + val lock = Ignition.ignite("node-1").cache[Int, Int]("cache").lock(1) + + lock.lock() + + visor.cache("-reset -c=cache") + + lock.unlock() + + visor.cache("-reset -c=cache") + } + + it("should show correct help") { + VisorCacheCommand + + visor.help("cache") + } + + it("should show empty projection error message") { + visor.cache("-reset -c=wrong") + } + } +} From d65228e42ec9c84182b8c9c9c8d06a0056d5eed2 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 5 Sep 2016 11:20:26 +0300 Subject: [PATCH 090/487] IGNITE-2974: ODBC: Added "socketSendBufferSize" and "socketReceiveBufferSize" configuration parameters. This closes #994. --- .../configuration/OdbcConfiguration.java | 64 +++++++++++++++++++ .../processors/odbc/OdbcProcessor.java | 8 +-- .../odbc/OdbcProcessorValidationSelfTest.java | 21 +++++- 3 files changed, 86 insertions(+), 7 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/OdbcConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/OdbcConfiguration.java index 3746995c06a81..c098e091a2143 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/OdbcConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/OdbcConfiguration.java @@ -17,6 +17,8 @@ package org.apache.ignite.configuration; +import org.apache.ignite.internal.util.typedef.internal.S; + /** * ODBC configuration. */ @@ -30,12 +32,21 @@ public class OdbcConfiguration { /** Default maximum TCP port range value. */ public static final int DFLT_TCP_PORT_TO = 10810; + /** Default socket send and receive buffer size. */ + public static final int DFLT_SOCK_BUF_SIZE = 0; + /** Default max number of open cursors per connection. */ public static final int DFLT_MAX_OPEN_CURSORS = 128; /** Endpoint address. */ private String endpointAddr; + /** Socket send buffer size. */ + private int sockSndBufSize = DFLT_SOCK_BUF_SIZE; + + /** Socket receive buffer size. */ + private int sockRcvBufSize = DFLT_SOCK_BUF_SIZE; + /** Max number of opened cursors per connection. */ private int maxOpenCursors = DFLT_MAX_OPEN_CURSORS; @@ -57,6 +68,8 @@ public OdbcConfiguration(OdbcConfiguration cfg) { endpointAddr = cfg.getEndpointAddress(); maxOpenCursors = cfg.getMaxOpenCursors(); + sockRcvBufSize = cfg.getSocketReceiveBufferSize(); + sockSndBufSize = cfg.getSocketSendBufferSize(); } /** @@ -115,4 +128,55 @@ public OdbcConfiguration setMaxOpenCursors(int maxOpenCursors) { return this; } + + /** + * Gets socket send buffer size. When set to zero, operation system default will be used. + *

          + * Defaults to {@link #DFLT_SOCK_BUF_SIZE} + * + * @return Socket send buffer size in bytes. + */ + public int getSocketSendBufferSize() { + return sockSndBufSize; + } + + /** + * Sets socket send buffer size. See {@link #getSocketSendBufferSize()} for more information. + * + * @param sockSndBufSize Socket send buffer size in bytes. + * @return This instance for chaining. + */ + public OdbcConfiguration setSocketSendBufferSize(int sockSndBufSize) { + this.sockSndBufSize = sockSndBufSize; + + return this; + } + + /** + * Gets socket receive buffer size. When set to zero, operation system default will be used. + *

          + * Defaults to {@link #DFLT_SOCK_BUF_SIZE}. + * + * @return Socket receive buffer size in bytes. + */ + public int getSocketReceiveBufferSize() { + return sockRcvBufSize; + } + + /** + * Sets socket receive buffer size. See {@link #getSocketReceiveBufferSize()} for more information. + * + * @param sockRcvBufSize Socket receive buffer size in bytes. + * @return This instance for chaining. + */ + public OdbcConfiguration setSocketReceiveBufferSize(int sockRcvBufSize) { + this.sockRcvBufSize = sockRcvBufSize; + + return this; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(OdbcConfiguration.class, this); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java index ead890119d42d..adfdc22f0e1c6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java @@ -46,9 +46,6 @@ public class OdbcProcessor extends GridProcessorAdapter { /** Default TCP direct buffer flag. */ private static final boolean DFLT_TCP_DIRECT_BUF = false; - /** Default socket send and receive buffer size. */ - private static final int DFLT_SOCK_BUF_SIZE = 32 * 1024; - /** Busy lock. */ private final GridSpinBusyLock busyLock = new GridSpinBusyLock(); @@ -113,11 +110,10 @@ public OdbcProcessor(GridKernalContext ctx) { .tcpNoDelay(DFLT_TCP_NODELAY) .directBuffer(DFLT_TCP_DIRECT_BUF) .byteOrder(ByteOrder.nativeOrder()) - .socketSendBufferSize(DFLT_SOCK_BUF_SIZE) - .socketReceiveBufferSize(DFLT_SOCK_BUF_SIZE) + .socketSendBufferSize(odbcCfg.getSocketSendBufferSize()) + .socketReceiveBufferSize(odbcCfg.getSocketReceiveBufferSize()) .filters(new GridNioCodecFilter(new OdbcBufferedParser(), log, false)) .directMode(false) - .idleTimeout(Long.MAX_VALUE) .build(); srv0.start(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcProcessorValidationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcProcessorValidationSelfTest.java index 751f0aea8bba1..bb08c6c105987 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcProcessorValidationSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcProcessorValidationSelfTest.java @@ -94,7 +94,7 @@ public void testAddressInvalidHost() throws Exception { /** * Test start with invalid address format. * - * @throws Exception + * @throws Exception If failed. */ public void testAddressInvalidFormat() throws Exception { check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:"), false); @@ -114,6 +114,25 @@ public void testAddressInvalidFormat() throws Exception { check(new OdbcConfiguration().setEndpointAddress(":9999..10000"), false); } + /** + * Test connection parameters: sendBufferSize, receiveBufferSize, connectionTimeout. + * + * @throws Exception If failed. + */ + public void testConnectionParams() throws Exception { + check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9998..10000") + .setSocketSendBufferSize(4 * 1024), true); + + check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9998..10000") + .setSocketReceiveBufferSize(4 * 1024), true); + + check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9998..10000") + .setSocketSendBufferSize(-64 * 1024), false); + + check(new OdbcConfiguration().setEndpointAddress("127.0.0.1:9998..10000") + .setSocketReceiveBufferSize(-64 * 1024), false); + } + /** * Perform check. * From 40d4b6ac6a71ed541d20018cf7deb2fb9b9bbb9b Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 5 Sep 2016 11:35:26 +0300 Subject: [PATCH 091/487] IGNITE-3834: Fixed a problem with BinaryMarshaller handles resolution. --- .../internal/binary/BinaryReaderHandles.java | 2 +- .../binary/BinaryMarshallerSelfTest.java | 38 +++++++++++++++++++ 2 files changed, 39 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderHandles.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderHandles.java index fddb8aa331d9c..881060ff87957 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderHandles.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderHandles.java @@ -57,7 +57,7 @@ public class BinaryReaderHandles { return null; case MODE_SINGLE: - return (T)data; + return pos == singlePos ? (T)data : null; default: assert mode == MODE_MULTIPLE; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java index f4c1bf75850a9..b347ec0590030 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java @@ -2895,6 +2895,21 @@ public void testDuplicateFields() throws Exception { assertEquals(obj.xB(), deserialized.xB()); } + /** + * @throws Exception If failed. + */ + public void testSingleHandle() throws Exception { + SingleHandleA a = new SingleHandleA(new SingleHandleB()); + + BinaryObjectImpl bo = marshal(a, binaryMarshaller()); + + Map map = bo.field("map"); + + BinaryObject innerBo = map.get("key"); + + assertEquals(SingleHandleB.class, innerBo.deserialize().getClass()); + } + /** * */ @@ -4847,4 +4862,27 @@ public Object getValue() { return value; } } + + /** + */ + private static class SingleHandleA { + /** */ + private SingleHandleB b; + + /** */ + private Map map = new HashMap<>(); + + /** + * @param b B. + */ + SingleHandleA(SingleHandleB b) { + this.b = b; + + map.put("key", b); + } + } + + /** + */ + private static class SingleHandleB {} } From 42963e6c99e9b282972e6ad67a813a8038cf580f Mon Sep 17 00:00:00 2001 From: Alexander Paschenko Date: Mon, 5 Sep 2016 14:10:39 +0300 Subject: [PATCH 092/487] IGNITE-3817: Fixed binary object re-build with missing schema. --- .../binary/builder/BinaryObjectBuilderImpl.java | 2 +- .../BinaryObjectBuilderAdditionalSelfTest.java | 14 ++++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java index 16c51b0442533..086da5c689057 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java @@ -199,7 +199,7 @@ void serializeTo(BinaryWriterExImpl writer, BinaryBuilderSerializer serializer) Map fieldsMeta = null; - if (reader != null) { + if (reader != null && BinaryUtils.hasSchema(flags)) { BinarySchema schema = reader.schema(); Map assignedFldsById; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java index f999ad351e982..24806cbf2bf01 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java @@ -56,6 +56,7 @@ import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl; import org.apache.ignite.internal.binary.mutabletest.GridBinaryMarshalerAwareTestClass; import org.apache.ignite.internal.binary.mutabletest.GridBinaryTestClasses; +import org.apache.ignite.internal.binary.test.GridBinaryTestClass2; import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; import org.apache.ignite.internal.processors.cache.binary.IgniteBinaryImpl; import org.apache.ignite.internal.util.lang.GridMapEntry; @@ -1388,6 +1389,19 @@ public void testSameBinaryKey() throws Exception { assertNotNull(partitionedCache.get(key)); } + /** + * Ensure that object w/o schema can be re-built. + */ + public void testBuildFromObjectWithoutSchema() { + BinaryObjectBuilderImpl binBuilder = wrap(new GridBinaryTestClass2()); + + BinaryObject binObj = binBuilder.build(); + + BinaryObjectBuilderImpl binBuilder2 = wrap(binObj); + + binBuilder2.build(); + } + /** * @param obj Object. * @return Object in binary format. From df8163f1ad3a390bb8d51b0eb2f378b5b3663025 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 5 Sep 2016 14:15:59 +0300 Subject: [PATCH 093/487] IGNITE-3743: ODBC: Added procedure call escape sequence support. This closes #1008. --- .../odbc/escape/OdbcEscapeType.java | 4 + .../odbc/escape/OdbcEscapeUtils.java | 5 + .../odbc/OdbcEscapeSequenceSelfTest.java | 125 ++++++++++++++++++ 3 files changed, 134 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java index 3bf0324b7b2cb..c7e32346a63f2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java @@ -27,6 +27,9 @@ public enum OdbcEscapeType { /** Outer join. */ OUTER_JOIN("oj", true, false), + /** Stored procedure call */ + CALL("call", true, false), + /** Date. */ DATE("d", true, false), @@ -47,6 +50,7 @@ public enum OdbcEscapeType { SCALAR_FUNCTION, // Assume that scalar functions are very frequent. DATE, TIMESTAMP, // Date and timestamp are relatively frequent as well; also TS must go before T. OUTER_JOIN, // Joins are less frequent, + CALL, // Procedure calls are less frequent than joins. LIKE, TIME, GUID // LIKE, TIME and GUID are even less frequent. }; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java index 48d4296e3d1e2..88afc526076c1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java @@ -278,6 +278,11 @@ private static String parseStandardExpression(String text, int startPos, int len case OUTER_JOIN: return parseExpression(text, startPos0, len0); + case CALL: { + String val = parseExpression(text, startPos0, len0); + + return "CALL " + val; + } default: throw new IgniteException("Unsupported escape sequence token [text=" + substring(text, startPos, len) + ", token=" + token.type().body() + ']'); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java index 3fec7d3a2e368..26221ea87b7cd 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java @@ -540,6 +540,131 @@ public void testNonEscapeSequence() throws Exception { checkFail("{fn func(arg')}"); } + + /** + * Test escape sequence series. + */ + public void testSimpleCallProc() throws Exception { + check( + "CALL test()", + "{call test()}" + ); + + check( + "select CALL test()", + "select {call test()}" + ); + + check( + "select CALL test() from table;", + "select {call test()} from table;" + ); + + check( + "CALL func(field1) CALL func(field2)", + "{call func(field1)} {call func(field2)}" + ); + + check( + "select CALL func(field1), CALL func(field2)", + "select {call func(field1)}, {call func(field2)}" + ); + + check( + "select CALL func(field1), CALL func(field2) from table;", + "select {call func(field1)}, {call func(field2)} from table;" + ); + } + + /** + * Test simple nested escape sequences. Depth = 2. + */ + public void testNestedCallProc() throws Exception { + check( + "CALL func1(field1, CALL func2(field2))", + "{call func1(field1, {call func2(field2)})}" + ); + + check( + "select CALL func1(field1, CALL func2(field2))", + "select {call func1(field1, {call func2(field2)})}" + ); + + check( + "select CALL func1(field1, CALL func2(field2), field3) from SomeTable;", + "select {call func1(field1, {call func2(field2)}, field3)} from SomeTable;" + ); + } + + /** + * Test nested escape sequences. Depth > 2. + */ + public void testDeepNestedCallProc() { + check( + "CALL func1(CALL func2(CALL func3(field1)))", + "{call func1({call func2({call func3(field1)})})}" + ); + + check( + "CALL func1(CALL func2(CALL func3(CALL func4(field1))))", + "{call func1({call func2({call func3({call func4(field1)})})})}" + ); + + check( + "select CALL func1(field1, CALL func2(CALL func3(field2), field3))", + "select {call func1(field1, {call func2({call func3(field2)}, field3)})}" + ); + + check( + "select CALL func1(field1, CALL func2(CALL func3(field2), field3)) from SomeTable;", + "select {call func1(field1, {call func2({call func3(field2)}, field3)})} from SomeTable;" + ); + } + + /** + * Test series of nested escape sequences. + */ + public void testNestedCallProcMixed() { + check( + "CALL func1(CALL func2(field1), CALL func3(field2))", + "{call func1({call func2(field1)}, {call func3(field2)})}" + ); + + check( + "select CALL func1(CALL func2(field1), CALL func3(field2)) from table;", + "select {call func1({call func2(field1)}, {call func3(field2)})} from table;" + ); + + check( + "CALL func1(CALL func2(CALL func3(field1))) CALL func1(CALL func2(field2))", + "{call func1({call func2({call func3(field1)})})} {call func1({call func2(field2)})}" + ); + } + + /** + * Test invalid escape sequence. + */ + public void testFailedOnInvalidCallProcSequence() { + checkFail("{callfunc1()}"); + + checkFail("select {call func1(field1, {call func2(field2), field3)} from SomeTable;"); + + checkFail("select {call func1(field1, call func2(field2)}, field3)} from SomeTable;"); + } + + /** + * Test escape sequences with additional whitespace characters + */ + public void testCallProcEscapeSequenceWithWhitespaces() throws Exception { + check("CALL func1()", "{ call func1()}"); + + check("CALL func1()", "{ call func1()}"); + + check("CALL func1()", "{ \n call\nfunc1()}"); + + checkFail("{ \n func1()}"); + } + /** * Check parsing logic. * From e3533010b584ba986196f9c7dbc36359aebd829e Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Mon, 5 Sep 2016 15:02:07 +0300 Subject: [PATCH 094/487] IGNITE-3819: ODBC: Improved error logging. This closes #1024. --- .../processors/odbc/OdbcRequestHandler.java | 31 ++++++++++++++----- 1 file changed, 23 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java index 3f7d505403668..69bbc7482c54e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java @@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteProductVersion; @@ -97,16 +98,16 @@ public OdbcResponse handle(long reqId, OdbcRequest req) { return executeQuery(reqId, (OdbcQueryExecuteRequest)req); case FETCH_SQL_QUERY: - return fetchQuery((OdbcQueryFetchRequest)req); + return fetchQuery(reqId, (OdbcQueryFetchRequest)req); case CLOSE_SQL_QUERY: - return closeQuery((OdbcQueryCloseRequest)req); + return closeQuery(reqId, (OdbcQueryCloseRequest)req); case GET_COLUMNS_META: - return getColumnsMeta((OdbcQueryGetColumnsMetaRequest)req); + return getColumnsMeta(reqId, (OdbcQueryGetColumnsMetaRequest)req); case GET_TABLES_META: - return getTablesMeta((OdbcQueryGetTablesMetaRequest)req); + return getTablesMeta(reqId, (OdbcQueryGetTablesMetaRequest)req); } return new OdbcResponse(OdbcResponse.STATUS_FAILED, "Unsupported ODBC request: " + req); @@ -189,6 +190,8 @@ private OdbcResponse executeQuery(long reqId, OdbcQueryExecuteRequest req) { catch (Exception e) { qryCursors.remove(qryId); + U.error(log, "Failed to execute SQL query [reqId=" + reqId + ", req=" + req + ']', e); + return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage()); } } @@ -196,10 +199,11 @@ private OdbcResponse executeQuery(long reqId, OdbcQueryExecuteRequest req) { /** * {@link OdbcQueryCloseRequest} command handler. * + * @param reqId Request ID. * @param req Execute query request. * @return Response. */ - private OdbcResponse closeQuery(OdbcQueryCloseRequest req) { + private OdbcResponse closeQuery(long reqId, OdbcQueryCloseRequest req) { try { QueryCursor cur = qryCursors.get(req.queryId()).get1(); @@ -217,6 +221,8 @@ private OdbcResponse closeQuery(OdbcQueryCloseRequest req) { catch (Exception e) { qryCursors.remove(req.queryId()); + U.error(log, "Failed to close SQL query [reqId=" + reqId + ", req=" + req.queryId() + ']', e); + return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage()); } } @@ -224,10 +230,11 @@ private OdbcResponse closeQuery(OdbcQueryCloseRequest req) { /** * {@link OdbcQueryFetchRequest} command handler. * + * @param reqId Request ID. * @param req Execute query request. * @return Response. */ - private OdbcResponse fetchQuery(OdbcQueryFetchRequest req) { + private OdbcResponse fetchQuery(long reqId, OdbcQueryFetchRequest req) { try { Iterator cur = qryCursors.get(req.queryId()).get2(); @@ -244,6 +251,8 @@ private OdbcResponse fetchQuery(OdbcQueryFetchRequest req) { return new OdbcResponse(res); } catch (Exception e) { + U.error(log, "Failed to fetch SQL query result [reqId=" + reqId + ", req=" + req + ']', e); + return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage()); } } @@ -251,10 +260,11 @@ private OdbcResponse fetchQuery(OdbcQueryFetchRequest req) { /** * {@link OdbcQueryGetColumnsMetaRequest} command handler. * + * @param reqId Request ID. * @param req Get columns metadata request. * @return Response. */ - private OdbcResponse getColumnsMeta(OdbcQueryGetColumnsMetaRequest req) { + private OdbcResponse getColumnsMeta(long reqId, OdbcQueryGetColumnsMetaRequest req) { try { List meta = new ArrayList<>(); @@ -298,6 +308,8 @@ private OdbcResponse getColumnsMeta(OdbcQueryGetColumnsMetaRequest req) { return new OdbcResponse(res); } catch (Exception e) { + U.error(log, "Failed to get columns metadata [reqId=" + reqId + ", req=" + req + ']', e); + return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage()); } } @@ -305,10 +317,11 @@ private OdbcResponse getColumnsMeta(OdbcQueryGetColumnsMetaRequest req) { /** * {@link OdbcQueryGetTablesMetaRequest} command handler. * + * @param reqId Request ID. * @param req Get tables metadata request. * @return Response. */ - private OdbcResponse getTablesMeta(OdbcQueryGetTablesMetaRequest req) { + private OdbcResponse getTablesMeta(long reqId, OdbcQueryGetTablesMetaRequest req) { try { List meta = new ArrayList<>(); @@ -340,6 +353,8 @@ private OdbcResponse getTablesMeta(OdbcQueryGetTablesMetaRequest req) { return new OdbcResponse(res); } catch (Exception e) { + U.error(log, "Failed to get tables metadata [reqId=" + reqId + ", req=" + req + ']', e); + return new OdbcResponse(OdbcResponse.STATUS_FAILED, e.getMessage()); } } From 008cf64429f40635e396a71f2c0aaf184077ff2b Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 5 Sep 2016 15:17:53 +0300 Subject: [PATCH 095/487] IGNITE-3741: ODBC: Added character escape support to expression parser. This closes #1004. --- .../odbc/escape/OdbcEscapeType.java | 9 +- .../odbc/escape/OdbcEscapeUtils.java | 53 +++++++----- .../odbc/OdbcEscapeSequenceSelfTest.java | 85 ++++++++++++++++++- 3 files changed, 119 insertions(+), 28 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java index c7e32346a63f2..44d836147603b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java @@ -42,8 +42,11 @@ public enum OdbcEscapeType { /** GUID. */ GUID("guid", true, false), - /** LIKE clause. */ - LIKE("\'", false, true); + /** LIKE escape character clause. */ + ESCAPE_WO_TOKEN("\'", false, false), + + /** LIKE escape character clause. */ + ESCAPE("escape", true, false); /** Values in convenient order. */ private static final OdbcEscapeType[] VALS = new OdbcEscapeType[] { @@ -51,7 +54,7 @@ public enum OdbcEscapeType { DATE, TIMESTAMP, // Date and timestamp are relatively frequent as well; also TS must go before T. OUTER_JOIN, // Joins are less frequent, CALL, // Procedure calls are less frequent than joins. - LIKE, TIME, GUID // LIKE, TIME and GUID are even less frequent. + ESCAPE_WO_TOKEN, ESCAPE, TIME, GUID // LIKE, TIME and GUID are even less frequent. }; /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java index 88afc526076c1..bbf19c75331b8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java @@ -76,12 +76,9 @@ private static OdbcEscapeParseResult parse0(String text, int startPos, boolean e while (curPos < text.length()) { char curChar = text.charAt(curPos); - if (curChar == '\'') { - if (!insideLiteral) - insideLiteral = true; - else if (text.charAt(curPos - 1) != '\\') - insideLiteral = false; - } + if (curChar == '\'') + /* Escaped quote in odbc is two successive singe quotes. They'll flip flag twice without side-effect. */ + insideLiteral = !insideLiteral; else if (!insideLiteral) { if (curChar == '{') { if (openPos == -1) { @@ -173,11 +170,7 @@ private static String parseEscapeSequence(String text, int startPos, int len) { OdbcEscapeToken token = parseToken(text, startPos, len); - if (token.type().standard()) - return parseStandardExpression(text, startPos, len, token); - else - throw new IgniteException("Unsupported escape sequence token [text=" + - substring(text, startPos, len) + ", token=" + token.type().body() + ']'); + return parseEscapeSequence(text, startPos, len, token); } else { // Nothing to escape, return original string. @@ -209,20 +202,17 @@ private static OdbcEscapeToken parseToken(String text, int startPos, int len) { for (OdbcEscapeType typ : OdbcEscapeType.sortedValues()) { if (text.startsWith(typ.body(), pos)) { - pos += typ.body().length(); + if (typ.standard()) + pos += typ.body().length(); - if (typ == OdbcEscapeType.LIKE) - throw new IgniteException("LIKE escape sequence is not supported yet."); - else { - empty = (startPos + len == pos + 1); + empty = (startPos + len == pos + 1); - if (!empty && typ.standard()) { - char charAfter = text.charAt(pos); + if (!empty && typ.standard()) { + char charAfter = text.charAt(pos); - if (!Character.isWhitespace(charAfter)) - throw new IgniteException("Unexpected escape sequence token: " + - substring(text, startPos, len)); - } + if (!Character.isWhitespace(charAfter)) + throw new IgniteException("Unexpected escape sequence token: " + + substring(text, startPos, len)); } curTyp = typ; @@ -249,7 +239,7 @@ private static OdbcEscapeToken parseToken(String text, int startPos, int len) { * @param token Token. * @return Result. */ - private static String parseStandardExpression(String text, int startPos, int len, OdbcEscapeToken token) { + private static String parseEscapeSequence(String text, int startPos, int len, OdbcEscapeToken token) { assert validSubstring(text, startPos, len); // Get expression borders. @@ -283,6 +273,11 @@ private static String parseStandardExpression(String text, int startPos, int len return "CALL " + val; } + + case ESCAPE: + case ESCAPE_WO_TOKEN: + return parseLikeEscCharacterExpression(text, startPos0, len0); + default: throw new IgniteException("Unsupported escape sequence token [text=" + substring(text, startPos, len) + ", token=" + token.type().body() + ']'); @@ -301,6 +296,18 @@ private static String parseExpression(String text, int startPos, int len) { return substring(text, startPos, len).trim(); } + /** + * Parse LIKE escape character expression. + * + * @param text Text. + * @param startPos Start position. + * @param len Length. + * @return Parsed expression. + */ + private static String parseLikeEscCharacterExpression(String text, int startPos, int len) { + return "ESCAPE " + substring(text, startPos, len).trim(); + } + /** * Parse expression and validate against ODBC specification with regex pattern. * diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java index 26221ea87b7cd..5303c6e76367d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java @@ -528,9 +528,10 @@ public void testNonEscapeSequence() throws Exception { "select '{' + {fn func()} + '}' from table;" ); + // quoted two single quotes should be interpret as apostrophe check( - "select '{\\'{fn test()}\\'}' from table;", - "select '{\\'{fn test()}\\'}' from table;" + "select '{''{fn test()}''}' from table;", + "select '{''{fn test()}''}' from table;" ); checkFail("'{fn test()}"); @@ -665,6 +666,86 @@ public void testCallProcEscapeSequenceWithWhitespaces() throws Exception { checkFail("{ \n func1()}"); } + /** + * Test escape sequence series. + */ + public void testLikeEscapeSequence() throws Exception { + check( + "ESCAPE '\\'", + "{'\\'}" + ); + + check( + "ESCAPE '\\'", + "{escape '\\'}" + ); + + check( + "ESCAPE ''", + "{''}" + ); + + check( + "ESCAPE ''", + "{escape ''}" + ); + + check( + "select * from t where value LIKE '\\%AAA%' ESCAPE '\\'", + "select * from t where value LIKE '\\%AAA%' {'\\'}" + ); + + check( + "select * from t where value LIKE '\\%AAA%' ESCAPE '\\'", + "select * from t where value LIKE '\\%AAA%' {escape '\\'}" + ); + + check( + "select * from t where value LIKE '\\%AAA%' ESCAPE '\\' ORDER BY id;", + "select * from t where value LIKE '\\%AAA%' {'\\'} ORDER BY id;" + ); + + check( + "select * from t where value LIKE '\\%AAA%' ESCAPE '\\' ORDER BY id;", + "select * from t where value LIKE '\\%AAA%' {escape '\\'} ORDER BY id;" + ); + + check( + "select * from t where value LIKE '\\%AAA''s%' ESCAPE '\\'", + "select * from t where value LIKE '\\%AAA''s%' {escape '\\'}" + ); + } + + /** + * Test escape sequences with additional whitespace characters + */ + public void testLikeEscapeSequenceWithWhitespaces() throws Exception { + check("ESCAPE '\\'", "{ '\\' }"); + check("ESCAPE '\\'", "{ escape '\\'}"); + + check("ESCAPE '\\'", "{ '\\' }"); + check("ESCAPE '\\'", "{ escape '\\' }"); + + check("ESCAPE '\\'", "{ \n '\\' }"); + check("ESCAPE '\\'", "{ \n escape\n'\\' }"); + } + + /** + * Test invalid escape sequence. + */ + public void testLikeOnInvalidLikeEscapeSequence() { + checkFail("LIKE 'AAA's'"); + checkFail("LIKE 'AAA\'s'"); + + checkFail("LIKE '\\%AAA%' {escape'\\' }"); + + checkFail("LIKE '\\%AAA%' {'\\' ORDER BY id"); + checkFail("LIKE '\\%AAA%' {escape '\\' ORDER BY id;"); + + checkFail("LIKE '\\%AAA%' '\\'} ORDER BY id"); + checkFail("LIKE '\\%AAA%' escape '\\'} ORDER BY id;"); + } + /** * Check parsing logic. * From bf9371a3ddb020209a6b031e11282706e19c58cb Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 5 Sep 2016 17:05:05 +0300 Subject: [PATCH 096/487] IGNITE-3611: IGFS: Slight refactoring to listPaths() and listFiles() methods. --- .../internal/processors/igfs/IgfsImpl.java | 73 ++++++++++--------- .../igfs/IgfsProcessorSelfTest.java | 11 +-- .../processors/igfs/IgfsStreamsSelfTest.java | 2 +- 3 files changed, 46 insertions(+), 40 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index c704e00555e5d..3b25c82e13846 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal.processors.igfs; -import java.util.Set; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -68,7 +67,6 @@ import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.future.GridFutureAdapter; -import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.A; @@ -791,7 +789,7 @@ else if (val) IgfsMode mode = resolveMode(path); - Collection files = new HashSet<>(); + Collection files = new HashSet<>(); if (IgfsUtils.isDualMode(mode)) { assert secondaryFs != null; @@ -799,8 +797,10 @@ else if (val) try { Collection children = secondaryFs.listPaths(path); - for (IgfsPath child : children) - files.add(child.name()); + files.addAll(children); + + if (!modeRslvr.hasPrimaryChild(path)) + return files; } catch (Exception e) { U.error(log, "List paths in DUAL mode failed [path=" + path + ']', e); @@ -809,20 +809,17 @@ else if (val) } } - if (!IgfsUtils.isDualMode(mode) || modeRslvr.hasPrimaryChild(path)) { - IgniteUuid fileId = meta.fileId(path); + IgfsEntryInfo info = primaryInfoForListing(path); - if (fileId != null) - files.addAll(meta.directoryListing(fileId).keySet()); - else if (mode == PRIMARY) - throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path); + if (info != null) { + // Perform the listing. + for (String child : info.listing().keySet()) + files.add(new IgfsPath(path, child)); } + else if (mode == PRIMARY) + throw new IgfsPathNotFoundException("Failed to list paths (path not found): " + path); - return F.viewReadOnly(files, new C1() { - @Override public IgfsPath apply(String e) { - return new IgfsPath(path, e); - } - }); + return files; } }); } @@ -841,7 +838,7 @@ else if (mode == PRIMARY) IgfsMode mode = resolveMode(path); - Set files = new HashSet<>(); + Collection files = new HashSet<>(); if (IgfsUtils.isDualMode(mode)) { assert secondaryFs != null; @@ -865,27 +862,22 @@ else if (mode == PRIMARY) } } - IgniteUuid fileId = meta.fileId(path); - - if (fileId != null) { - IgfsEntryInfo info = meta.info(fileId); + IgfsEntryInfo info = primaryInfoForListing(path); - // Handle concurrent deletion. - if (info != null) { - if (info.isFile()) - // If this is a file, return its description. - return Collections.singleton(new IgfsFileImpl(path, info, - data.groupBlockSize())); + if (info != null) { + if (info.isFile()) + // If this is a file, return its description. + return Collections.singleton(new IgfsFileImpl(path, info, + data.groupBlockSize())); - // Perform the listing. - for (Map.Entry e : info.listing().entrySet()) { - IgfsEntryInfo childInfo = meta.info(e.getValue().fileId()); + // Perform the listing. + for (Map.Entry e : info.listing().entrySet()) { + IgfsEntryInfo childInfo = meta.info(e.getValue().fileId()); - if (childInfo != null) { - IgfsPath childPath = new IgfsPath(path, e.getKey()); + if (childInfo != null) { + IgfsPath childPath = new IgfsPath(path, e.getKey()); - files.add(new IgfsFileImpl(childPath, childInfo, data.groupBlockSize())); - } + files.add(new IgfsFileImpl(childPath, childInfo, data.groupBlockSize())); } } } @@ -897,6 +889,19 @@ else if (mode == PRIMARY) }); } + /** + * Get primary file system info for listing operation. + * + * @param path Path. + * @return Info or {@code null} if not found. + * @throws IgniteCheckedException If failed. + */ + private IgfsEntryInfo primaryInfoForListing(IgfsPath path) throws IgniteCheckedException { + IgniteUuid fileId = meta.fileId(path); + + return fileId != null ? meta.info(fileId) : null; + } + /** {@inheritDoc} */ @Override public long usedSpaceSize() { return metrics().localSpaceSize(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java index 269706e5c9392..c8549705ee162 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java @@ -400,12 +400,12 @@ public void testBasicOps() throws Exception { igfs.delete(path("/A1/B1/C3"), false); assertNull(igfs.info(path("/A1/B1/C3"))); - assertEquals(Collections.emptyList(), igfs.listPaths(path("/A1/B1"))); + assertTrue(F.isEmpty(igfs.listPaths(path("/A1/B1")))); igfs.delete(path("/A2/B2"), true); assertNull(igfs.info(path("/A2/B2"))); - assertEquals(Collections.emptyList(), igfs.listPaths(path("/A2"))); + assertTrue(F.isEmpty(igfs.listPaths(path("/A2")))); assertEquals(Arrays.asList(path("/A"), path("/A1"), path("/A2")), sorted(igfs.listPaths(path("/")))); @@ -416,13 +416,14 @@ public void testBasicOps() throws Exception { igfs.delete(path("/A"), true); igfs.delete(path("/A1"), true); igfs.delete(path("/A2"), true); - assertEquals(Collections.emptyList(), igfs.listPaths(path("/"))); + + assertTrue(F.isEmpty(igfs.listPaths(path("/")))); // Delete root when it is empty: igfs.delete(path("/"), false); igfs.delete(path("/"), true); - assertEquals(Collections.emptyList(), igfs.listPaths(path("/"))); + assertTrue(F.isEmpty(igfs.listPaths(path("/")))); for (Cache.Entry e : metaCache) info("Entry in cache [key=" + e.getKey() + ", val=" + e.getValue() + ']'); @@ -603,7 +604,7 @@ public Object call() throws Exception { // Cleanup. igfs.format(); - assertEquals(Collections.emptyList(), igfs.listPaths(root)); + assertTrue(F.isEmpty(igfs.listPaths(root))); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java index 2af8c5c986af3..44560e3821fd1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java @@ -198,7 +198,7 @@ public void testCreateFile() throws Exception { long max = 100L * CFG_BLOCK_SIZE / WRITING_THREADS_CNT; for (long size = 0; size <= max; size = size * 15 / 10 + 1) { - assertEquals(Collections.emptyList(), fs.listPaths(root)); + assertTrue(F.isEmpty(fs.listPaths(root))); testCreateFile(path, size, new Random().nextInt()); } From 2c397d2fb8851b25aa5f0a5589ad1deffbe7eee9 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 5 Sep 2016 17:38:47 +0300 Subject: [PATCH 097/487] IGNITE-2629: ODBC: Requests are now processed asynchronously in separate thread pool. This closes #996. --- .../configuration/OdbcConfiguration.java | 34 ++++++++++++++- .../processors/odbc/OdbcProcessor.java | 41 +++++++++++++++++-- .../odbc/OdbcProcessorValidationSelfTest.java | 16 +++++++- 3 files changed, 83 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/OdbcConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/OdbcConfiguration.java index c098e091a2143..8fe166563a923 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/OdbcConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/OdbcConfiguration.java @@ -38,6 +38,9 @@ public class OdbcConfiguration { /** Default max number of open cursors per connection. */ public static final int DFLT_MAX_OPEN_CURSORS = 128; + /** Default size of thread pool. */ + public static final int DFLT_THREAD_POOL_SIZE = IgniteConfiguration.DFLT_PUBLIC_THREAD_CNT; + /** Endpoint address. */ private String endpointAddr; @@ -50,6 +53,9 @@ public class OdbcConfiguration { /** Max number of opened cursors per connection. */ private int maxOpenCursors = DFLT_MAX_OPEN_CURSORS; + /** Thread pool size. */ + private int threadPoolSize = DFLT_THREAD_POOL_SIZE; + /** * Creates ODBC server configuration with all default values. */ @@ -58,8 +64,7 @@ public OdbcConfiguration() { } /** - * Creates ODBC server configuration by copying all properties from - * given configuration. + * Creates ODBC server configuration by copying all properties from given configuration. * * @param cfg ODBC server configuration. */ @@ -70,6 +75,7 @@ public OdbcConfiguration(OdbcConfiguration cfg) { maxOpenCursors = cfg.getMaxOpenCursors(); sockRcvBufSize = cfg.getSocketReceiveBufferSize(); sockSndBufSize = cfg.getSocketSendBufferSize(); + threadPoolSize = cfg.getThreadPoolSize(); } /** @@ -175,6 +181,30 @@ public OdbcConfiguration setSocketReceiveBufferSize(int sockRcvBufSize) { return this; } + /** + * Size of thread pool that is in charge of processing ODBC tasks. + *

          + * Defaults {@link #DFLT_THREAD_POOL_SIZE}. + * + * @return Thread pool that is in charge of processing ODBC tasks. + */ + public int getThreadPoolSize() { + return threadPoolSize; + } + + /** + * Sets thread pool that is in charge of processing ODBC tasks. See {@link #getThreadPoolSize()} for more + * information. + * + * @param threadPoolSize Thread pool that is in charge of processing ODBC tasks. + * @return This instance for chaining. + */ + public OdbcConfiguration setThreadPoolSize(int threadPoolSize) { + this.threadPoolSize = threadPoolSize; + + return this; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(OdbcConfiguration.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java index adfdc22f0e1c6..a672d7cf6ac94 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java @@ -18,20 +18,27 @@ package org.apache.ignite.internal.processors.odbc; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.OdbcConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.processors.GridProcessorAdapter; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.HostAndPortRange; +import org.apache.ignite.internal.util.nio.GridNioAsyncNotifyFilter; import org.apache.ignite.internal.util.nio.GridNioCodecFilter; +import org.apache.ignite.internal.util.nio.GridNioFilter; import org.apache.ignite.internal.util.nio.GridNioServer; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.spi.IgnitePortProtocol; +import org.apache.ignite.thread.IgniteThreadPoolExecutor; import java.net.InetAddress; import java.nio.ByteOrder; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; /** * ODBC processor. @@ -49,9 +56,12 @@ public class OdbcProcessor extends GridProcessorAdapter { /** Busy lock. */ private final GridSpinBusyLock busyLock = new GridSpinBusyLock(); - /** OBCD TCP Server. */ + /** ODBC TCP Server. */ private GridNioServer srv; + /** ODBC executor service. */ + private ExecutorService odbcExecSvc; + /** * @param ctx Kernal context. */ @@ -61,11 +71,13 @@ public OdbcProcessor(GridKernalContext ctx) { /** {@inheritDoc} */ @Override public void start() throws IgniteCheckedException { - OdbcConfiguration odbcCfg = ctx.config().getOdbcConfiguration(); + IgniteConfiguration cfg = ctx.config(); + + OdbcConfiguration odbcCfg = cfg.getOdbcConfiguration(); if (odbcCfg != null) { try { - Marshaller marsh = ctx.config().getMarshaller(); + Marshaller marsh = cfg.getMarshaller(); if (marsh != null && !(marsh instanceof BinaryMarshaller)) throw new IgniteCheckedException("ODBC can only be used with BinaryMarshaller (please set it " + @@ -87,6 +99,16 @@ public OdbcProcessor(GridKernalContext ctx) { ); } + assertParameter(odbcCfg.getThreadPoolSize() > 0, "threadPoolSize > 0"); + + odbcExecSvc = new IgniteThreadPoolExecutor( + "odbc", + cfg.getGridName(), + odbcCfg.getThreadPoolSize(), + odbcCfg.getThreadPoolSize(), + 0, + new LinkedBlockingQueue()); + InetAddress host; try { @@ -100,6 +122,11 @@ public OdbcProcessor(GridKernalContext ctx) { for (int port = hostPort.portFrom(); port <= hostPort.portTo(); port++) { try { + GridNioFilter[] filters = new GridNioFilter[] { + new GridNioAsyncNotifyFilter(ctx.gridName(), odbcExecSvc, log), + new GridNioCodecFilter(new OdbcBufferedParser(), log, false) + }; + GridNioServer srv0 = GridNioServer.builder() .address(host) .port(port) @@ -112,7 +139,7 @@ public OdbcProcessor(GridKernalContext ctx) { .byteOrder(ByteOrder.nativeOrder()) .socketSendBufferSize(odbcCfg.getSocketSendBufferSize()) .socketReceiveBufferSize(odbcCfg.getSocketReceiveBufferSize()) - .filters(new GridNioCodecFilter(new OdbcBufferedParser(), log, false)) + .filters(filters) .directMode(false) .build(); @@ -154,6 +181,12 @@ public OdbcProcessor(GridKernalContext ctx) { ctx.ports().deregisterPorts(getClass()); + if (odbcExecSvc != null) { + U.shutdownNow(getClass(), odbcExecSvc, log); + + odbcExecSvc = null; + } + if (log.isDebugEnabled()) log.debug("ODBC processor stopped."); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcProcessorValidationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcProcessorValidationSelfTest.java index bb08c6c105987..aaee2a957950b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcProcessorValidationSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcProcessorValidationSelfTest.java @@ -133,12 +133,23 @@ public void testConnectionParams() throws Exception { .setSocketReceiveBufferSize(-64 * 1024), false); } + /** + * Test thread pool size. + * + * @throws Exception If failed. + */ + public void testThreadPoolSize() throws Exception { + check(new OdbcConfiguration().setThreadPoolSize(0), false); + check(new OdbcConfiguration().setThreadPoolSize(-1), false); + + check(new OdbcConfiguration().setThreadPoolSize(4), true); + } + /** * Perform check. * * @param odbcCfg ODBC configuration. - * @param success Success flag. - * @throws Exception If failed. + * @param success Success flag. * @throws Exception If failed. */ @SuppressWarnings("ThrowableResultOfMethodCallIgnored") private void check(OdbcConfiguration odbcCfg, boolean success) throws Exception { @@ -160,4 +171,5 @@ private void check(OdbcConfiguration odbcCfg, boolean success) throws Exception }, IgniteException.class, null); } } + } From 355082958b476009179254df9be20e225179cb7c Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 5 Sep 2016 18:06:27 +0300 Subject: [PATCH 098/487] IGNITE-2833: GridCacheTtlManager pending queue retention size optimization. --- .../processors/cache/GridCacheTtlManager.java | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java index ae2895e19253b..8ff035813528e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.cache; import java.util.concurrent.atomic.AtomicLongFieldUpdater; + import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.configuration.CacheConfiguration; @@ -106,7 +107,8 @@ public void addTrackedEntry(GridCacheMapEntry entry) { break; } - } else + } + else break; } } @@ -156,10 +158,11 @@ public void expire() { if (log.isTraceEnabled()) log.trace("Trying to remove expired entry from cache: " + e); - GridCacheEntryEx entry = e.entry; boolean touch = false; + GridCacheEntryEx entry = e.ctx.cache().entryEx(e.key); + while (true) { try { if (entry.onTtlExpired(obsoleteVer)) @@ -278,8 +281,11 @@ private static class EntryWrapper implements Comparable { /** Entry expire time. */ private final long expireTime; - /** Entry. */ - private final GridCacheMapEntry entry; + /** Cache Object Context */ + private final GridCacheContext ctx; + + /** Cache Object Key */ + private final CacheObject key; /** * @param entry Cache entry to create wrapper for. @@ -289,7 +295,8 @@ private EntryWrapper(GridCacheMapEntry entry) { assert expireTime != 0; - this.entry = entry; + this.ctx = entry.context(); + this.key = entry.key(); } /** {@inheritDoc} */ @@ -297,7 +304,7 @@ private EntryWrapper(GridCacheMapEntry entry) { int res = Long.compare(expireTime, o.expireTime); if (res == 0) - res = compareKeys(entry.context(), entry.key(), o.entry.context(), o.entry.key()); + res = compareKeys(ctx, key, o.ctx, o.key); return res; } @@ -312,15 +319,14 @@ private EntryWrapper(GridCacheMapEntry entry) { EntryWrapper that = (EntryWrapper)o; - return expireTime == that.expireTime && - compareKeys(entry.context(), entry.key(), that.entry.context(), that.entry.key()) == 0; + return expireTime == that.expireTime && compareKeys(ctx, key, that.ctx, that.key) == 0; } /** {@inheritDoc} */ @Override public int hashCode() { int res = (int)(expireTime ^ (expireTime >>> 32)); - res = 31 * res + entry.key().hashCode(); + res = 31 * res + key.hashCode(); return res; } From 8970b3e3a36b18274b451fddf365627ba23459d6 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Mon, 5 Sep 2016 18:15:11 +0300 Subject: [PATCH 099/487] IGNITE-3628: ODBC: Added ability to configure page size in DSN, changed default value from 32 to 1024. This closes #1014. --- .../cpp/common/include/ignite/ignite_error.h | 2 +- .../platforms/cpp/common/src/ignite_error.cpp | 2 +- .../cpp/odbc-test/src/configuration_test.cpp | 14 ++++- .../odbc-test/src/sql_test_suite_fixture.cpp | 2 +- .../ignite/odbc/config/configuration.h | 59 +++++++++++++++---- .../odbc/include/ignite/odbc/result_page.h | 3 - .../odbc/system/ui/dsn_configuration_window.h | 8 +++ .../system/ui/dsn_configuration_window.cpp | 29 +++++++-- .../cpp/odbc/src/config/configuration.cpp | 32 +++------- modules/platforms/cpp/odbc/src/dsn_config.cpp | 12 +++- .../cpp/odbc/src/query/data_query.cpp | 2 +- 11 files changed, 113 insertions(+), 52 deletions(-) diff --git a/modules/platforms/cpp/common/include/ignite/ignite_error.h b/modules/platforms/cpp/common/include/ignite/ignite_error.h index 4c0e263113458..a41a42f9aedf5 100644 --- a/modules/platforms/cpp/common/include/ignite/ignite_error.h +++ b/modules/platforms/cpp/common/include/ignite/ignite_error.h @@ -199,7 +199,7 @@ namespace ignite * * @param err Error. */ - static void ThrowIfNeeded(IgniteError& err); + static void ThrowIfNeeded(const IgniteError& err); /** * Default constructor. diff --git a/modules/platforms/cpp/common/src/ignite_error.cpp b/modules/platforms/cpp/common/src/ignite_error.cpp index 5acbed2ce683f..8179184be8a8a 100644 --- a/modules/platforms/cpp/common/src/ignite_error.cpp +++ b/modules/platforms/cpp/common/src/ignite_error.cpp @@ -24,7 +24,7 @@ using namespace ignite::java; namespace ignite { - void IgniteError::ThrowIfNeeded(IgniteError& err) + void IgniteError::ThrowIfNeeded(const IgniteError& err) { if (err.code != IGNITE_SUCCESS) throw err; diff --git a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp index 0fd3277ed6613..bfdb220e46be2 100644 --- a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp @@ -37,6 +37,7 @@ namespace const uint16_t testServerPort = 4242; const std::string testCacheName = "TestCache"; const std::string testDsn = "Ignite DSN"; + const int32_t testPageSize = 4321; const std::string testAddress = testServerHost + ':' + ignite::common::LexicalCast(testServerPort); } @@ -76,12 +77,14 @@ void CheckConnectionConfig(const Configuration& cfg) BOOST_CHECK_EQUAL(cfg.GetAddress(), testAddress); BOOST_CHECK_EQUAL(cfg.GetCache(), testCacheName); BOOST_CHECK_EQUAL(cfg.GetDsn(), std::string()); + BOOST_CHECK_EQUAL(cfg.GetPageSize(), testPageSize); std::stringstream constructor; constructor << "address=" << testAddress << ';' << "cache=" << testCacheName << ';' - << "driver={" << testDriverName << "};"; + << "driver={" << testDriverName << "};" + << "page_size=" << testPageSize << ';'; const std::string& expectedStr = constructor.str(); @@ -96,6 +99,7 @@ void CheckDsnConfig(const Configuration& cfg) BOOST_CHECK_EQUAL(cfg.GetAddress(), Configuration::DefaultValue::address); BOOST_CHECK_EQUAL(cfg.GetHost(), std::string()); BOOST_CHECK_EQUAL(cfg.GetTcpPort(), Configuration::DefaultValue::port); + BOOST_CHECK_EQUAL(cfg.GetPageSize(), Configuration::DefaultValue::pageSize); } BOOST_AUTO_TEST_SUITE(ConfigurationTestSuite) @@ -107,6 +111,7 @@ BOOST_AUTO_TEST_CASE(CheckTestValuesNotEquealDefault) BOOST_CHECK_NE(testServerPort, Configuration::DefaultValue::port); BOOST_CHECK_NE(testCacheName, Configuration::DefaultValue::cache); BOOST_CHECK_NE(testDsn, Configuration::DefaultValue::dsn); + BOOST_CHECK_NE(testPageSize, Configuration::DefaultValue::pageSize); } BOOST_AUTO_TEST_CASE(TestConnectStringUppercase) @@ -117,7 +122,8 @@ BOOST_AUTO_TEST_CASE(TestConnectStringUppercase) constructor << "DRIVER={" << testDriverName << "};" << "ADDRESS=" << testAddress << ';' - << "CACHE=" << testCacheName; + << "CACHE=" << testCacheName << ';' + << "PAGE_SIZE=" << testPageSize; const std::string& connectStr = constructor.str(); @@ -134,6 +140,7 @@ BOOST_AUTO_TEST_CASE(TestConnectStringLowercase) constructor << "driver={" << testDriverName << "};" << "address=" << testAddress << ';' + << "page_size=" << testPageSize << ';' << "cache=" << testCacheName << ';'; const std::string& connectStr = constructor.str(); @@ -151,6 +158,7 @@ BOOST_AUTO_TEST_CASE(TestConnectStringZeroTerminated) constructor << "driver={" << testDriverName << "};" << "address=" << testAddress << ';' + << "page_size=" << testPageSize << ';' << "cache=" << testCacheName << ';'; const std::string& connectStr = constructor.str(); @@ -168,6 +176,7 @@ BOOST_AUTO_TEST_CASE(TestConnectStringMixed) constructor << "Driver={" << testDriverName << "};" << "Address=" << testAddress << ';' + << "Page_Size=" << testPageSize << ';' << "Cache=" << testCacheName << ';'; const std::string& connectStr = constructor.str(); @@ -185,6 +194,7 @@ BOOST_AUTO_TEST_CASE(TestConnectStringWhitepaces) constructor << "DRIVER = {" << testDriverName << "} ;\n" << " ADDRESS =" << testAddress << "; " + << " PAGE_SIZE= " << testPageSize << ';' << "CACHE = \n\r" << testCacheName << ';'; const std::string& connectStr = constructor.str(); diff --git a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp index e9a8fc5770231..29d1d1892d801 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp @@ -131,7 +131,7 @@ namespace ignite BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); ret = SQLFetch(stmt); - BOOST_CHECK(ret == SQL_NO_DATA) ; + BOOST_CHECK(ret == SQL_NO_DATA); } template<> diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h index f90fa2d6a642f..b5f385efb279e 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h @@ -64,6 +64,9 @@ namespace ignite /** Connection attribute keyword for protocol version attribute. */ static const std::string protocolVersion; + + /** Connection attribute keyword for fetch results page size attribute. */ + static const std::string pageSize; }; /** Default values for configuration. */ @@ -89,6 +92,9 @@ namespace ignite /** Default value for port attribute. */ static const uint16_t port; + + /** Default value for fetch results page size attribute. */ + static const int32_t pageSize; }; /** @@ -126,7 +132,10 @@ namespace ignite * * @param str Connect string. */ - void FillFromConnectString(const std::string& str); + void FillFromConnectString(const std::string& str) + { + FillFromConnectString(str.data(), str.size()); + } /** * Convert configure to connect string. @@ -158,7 +167,10 @@ namespace ignite * * @param port Server port. */ - void SetTcpPort(uint16_t port); + void SetTcpPort(uint16_t port) + { + arguments[Key::port] = common::LexicalCast(port); + } /** * Get DSN. @@ -251,28 +263,51 @@ namespace ignite } /** - * Get argument map. + * Get protocol version. * - * @return Argument map. + * @return Protocol version. */ - const ArgumentMap& GetMap() const + ProtocolVersion GetProtocolVersion() const; + + /** + * Set protocol version. + * + * @param version Version to set. + */ + void SetProtocolVersion(const std::string& version) { - return arguments; + arguments[Key::protocolVersion] = version; } /** - * Get protocol version. + * Get fetch results page size. * - * @return Protocol version. + * @return Fetch results page size. */ - ProtocolVersion GetProtocolVersion() const; + int32_t GetPageSize() const + { + return static_cast(GetIntValue(Key::pageSize, DefaultValue::pageSize)); + } /** - * Set protocol version. + * Set fetch results page size. * - * @param version Version to set. + * @param size Fetch results page size. */ - void SetProtocolVersion(const std::string& version); + void SetPageSize(int32_t size) + { + arguments[Key::pageSize] = common::LexicalCast(size); + } + + /** + * Get argument map. + * + * @return Argument map. + */ + const ArgumentMap& GetMap() const + { + return arguments; + } /** * Get string value from the config. diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/result_page.h b/modules/platforms/cpp/odbc/include/ignite/odbc/result_page.h index 3533229d5f113..c17548723c171 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/result_page.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/result_page.h @@ -37,9 +37,6 @@ namespace ignite enum { DEFAULT_ALLOCATED_MEMORY = 1024 }; public: - // Default result page size. - enum { DEFAULT_SIZE = 32 }; - /** * Constructor. */ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h b/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h index 034de82c5d2f6..f034a8bc71747 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h @@ -46,6 +46,8 @@ namespace ignite ID_ADDRESS_LABEL, ID_CACHE_EDIT, ID_CACHE_LABEL, + ID_PAGE_SIZE_EDIT, + ID_PAGE_SIZE_LABEL, ID_OK_BUTTON, ID_CANCEL_BUTTON }; @@ -116,6 +118,12 @@ namespace ignite /** DSN cache edit field. */ std::auto_ptr cacheEdit; + /** DSN fetch page size edit field label. */ + std::auto_ptr pageSizeLabel; + + /** DSN fetch page size edit field. */ + std::auto_ptr pageSizeEdit; + /** Ok button. */ std::auto_ptr okButton; diff --git a/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp index 76132bd1a4b0b..49f87d824459f 100644 --- a/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp +++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp @@ -30,7 +30,7 @@ namespace ignite DsnConfigurationWindow::DsnConfigurationWindow(Window* parent, config::Configuration& config): CustomWindow(parent, "IgniteConfigureDsn", "Configure Apache Ignite DSN"), width(360), - height(160), + height(200), connectionSettingsGroupBox(), nameLabel(), nameEdit(), @@ -38,6 +38,8 @@ namespace ignite addressEdit(), cacheLabel(), cacheEdit(), + pageSizeLabel(), + pageSizeEdit(), okButton(), cancelButton(), config(config), @@ -111,6 +113,16 @@ namespace ignite cacheLabel = CreateLabel(labelPosX, rowPos, labelSizeX, rowSize, "Cache name:", ID_CACHE_LABEL); cacheEdit = CreateEdit(editPosX, rowPos, editSizeX, rowSize, val, ID_CACHE_EDIT); + rowPos += interval + rowSize; + + std::string tmp = common::LexicalCast(config.GetPageSize()); + val = tmp.c_str(); + pageSizeLabel = CreateLabel(labelPosX, rowPos, labelSizeX, + rowSize, "Page size:", ID_PAGE_SIZE_LABEL); + + pageSizeEdit = CreateEdit(editPosX, rowPos, editSizeX, + rowSize, val, ID_PAGE_SIZE_EDIT, ES_NUMBER); + rowPos += interval * 2 + rowSize; connectionSettingsGroupBox = CreateGroupBox(margin, sectionBegin, width - 2 * margin, @@ -186,18 +198,26 @@ namespace ignite std::string dsn; std::string address; std::string cache; + std::string pageSizeStr; nameEdit->GetText(dsn); addressEdit->GetText(address); cacheEdit->GetText(cache); + pageSizeEdit->GetText(pageSizeStr); + + int32_t pageSize = common::LexicalCast(pageSizeStr); + + if (pageSize <= 0) + pageSize = config.GetPageSize(); common::StripSurroundingWhitespaces(address); common::StripSurroundingWhitespaces(dsn); LOG_MSG("Retriving arguments:\n"); - LOG_MSG("DSN: %s\n", dsn.c_str()); - LOG_MSG("Address: %s\n", address.c_str()); - LOG_MSG("Cache: %s\n", cache.c_str()); + LOG_MSG("DSN: %s\n", dsn.c_str()); + LOG_MSG("Address: %s\n", address.c_str()); + LOG_MSG("Cache: %s\n", cache.c_str()); + LOG_MSG("Page size: %d\n", pageSize); if (dsn.empty()) throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, "DSN name can not be empty."); @@ -205,6 +225,7 @@ namespace ignite cfg.SetDsn(dsn); cfg.SetAddress(address); cfg.SetCache(cache); + cfg.SetPageSize(pageSize); } } } diff --git a/modules/platforms/cpp/odbc/src/config/configuration.cpp b/modules/platforms/cpp/odbc/src/config/configuration.cpp index f40c74f104eb2..74ccaaf5c05c0 100644 --- a/modules/platforms/cpp/odbc/src/config/configuration.cpp +++ b/modules/platforms/cpp/odbc/src/config/configuration.cpp @@ -39,18 +39,19 @@ namespace ignite const std::string Configuration::Key::server = "server"; const std::string Configuration::Key::port = "port"; const std::string Configuration::Key::protocolVersion = "protocol_version"; + const std::string Configuration::Key::pageSize = "page_size"; - const std::string Configuration::DefaultValue::dsn = "Apache Ignite DSN"; - const std::string Configuration::DefaultValue::driver = "Apache Ignite"; - const std::string Configuration::DefaultValue::cache = ""; - const std::string Configuration::DefaultValue::address = ""; - const std::string Configuration::DefaultValue::server = ""; + const std::string Configuration::DefaultValue::dsn = "Apache Ignite DSN"; + const std::string Configuration::DefaultValue::driver = "Apache Ignite"; + const std::string Configuration::DefaultValue::cache = ""; + const std::string Configuration::DefaultValue::address = ""; + const std::string Configuration::DefaultValue::server = ""; - const uint16_t Configuration::DefaultValue::port = 10800; + const uint16_t Configuration::DefaultValue::port = 10800; + const int32_t Configuration::DefaultValue::pageSize = 1024; const ProtocolVersion& Configuration::DefaultValue::protocolVersion = ProtocolVersion::GetCurrent(); - Configuration::Configuration() : arguments() { @@ -91,11 +92,6 @@ namespace ignite } } - void Configuration::FillFromConnectString(const std::string& str) - { - FillFromConnectString(str.data(), str.size()); - } - std::string Configuration::ToConnectString() const { std::stringstream connect_string_buffer; @@ -145,11 +141,6 @@ namespace ignite } } - void Configuration::SetTcpPort(uint16_t port) - { - arguments[Key::port] = common::LexicalCast(port); - } - ProtocolVersion Configuration::GetProtocolVersion() const { ArgumentMap::const_iterator it = arguments.find(Key::protocolVersion); @@ -160,11 +151,6 @@ namespace ignite return DefaultValue::protocolVersion; } - void Configuration::SetProtocolVersion(const std::string& version) - { - arguments[Key::protocolVersion] = version; - } - const std::string& Configuration::GetStringValue(const std::string& key, const std::string& dflt) const { ArgumentMap::const_iterator it = arguments.find(common::ToLower(key)); @@ -193,8 +179,6 @@ namespace ignite return dflt; } - - void Configuration::ParseAttributeList(const char * str, size_t len, char delimeter, ArgumentMap & args) { std::string connect_str(str, len); diff --git a/modules/platforms/cpp/odbc/src/dsn_config.cpp b/modules/platforms/cpp/odbc/src/dsn_config.cpp index 99635dcd4d928..356bcaa3d49a7 100644 --- a/modules/platforms/cpp/odbc/src/dsn_config.cpp +++ b/modules/platforms/cpp/odbc/src/dsn_config.cpp @@ -91,21 +91,27 @@ namespace ignite void ReadDsnConfiguration(const char* dsn, Configuration& config) { std::string address = ReadDsnString(dsn, Configuration::Key::address, config.GetAddress().c_str()); + std::string server = ReadDsnString(dsn, Configuration::Key::server, config.GetHost().c_str()); + uint16_t port = ReadDsnInt(dsn, Configuration::Key::port, config.GetTcpPort()); + std::string cache = ReadDsnString(dsn, Configuration::Key::cache, config.GetCache().c_str()); + std::string version = ReadDsnString(dsn, Configuration::Key::protocolVersion, config.GetProtocolVersion().ToString().c_str()); - LOG_MSG("%d\n", __LINE__); + int32_t pageSize = ReadDsnInt(dsn, Configuration::Key::pageSize, config.GetPageSize()); + + if (pageSize <= 0) + pageSize = config.GetPageSize(); config.SetAddress(address); config.SetHost(server); config.SetTcpPort(port); config.SetCache(cache); config.SetProtocolVersion(version); - - LOG_MSG("%d\n", __LINE__); + config.SetPageSize(pageSize); } } } \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/src/query/data_query.cpp b/modules/platforms/cpp/odbc/src/query/data_query.cpp index 183bbb564046f..7a25ccb4ba5fb 100644 --- a/modules/platforms/cpp/odbc/src/query/data_query.cpp +++ b/modules/platforms/cpp/odbc/src/query/data_query.cpp @@ -241,7 +241,7 @@ namespace ignite { std::auto_ptr resultPage(new ResultPage()); - QueryFetchRequest req(cursor->GetQueryId(), ResultPage::DEFAULT_SIZE); + QueryFetchRequest req(cursor->GetQueryId(), connection.GetConfiguration().GetPageSize()); QueryFetchResponse rsp(*resultPage); try From e23a94fbab4040f60b7a8ef3638aaebe4b9ba5f6 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 6 Sep 2016 11:34:14 +0300 Subject: [PATCH 100/487] Fixed GridQueryParsingTest. --- .../processors/query/h2/sql/GridQueryParsingTest.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java index cf000e93c7466..d559d2edb9a38 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java @@ -292,7 +292,10 @@ private T parse(String sql) throws Exception { * @param sql2 Sql 2. */ private void assertSqlEquals(String sql1, String sql2) { - assertEquals(normalizeSql(sql1), normalizeSql(sql2)); + String nsql1 = normalizeSql(sql1); + String nsql2 = normalizeSql(sql2); + + assertEquals(nsql1, nsql2); } /** @@ -301,7 +304,7 @@ private void assertSqlEquals(String sql1, String sql2) { private static String normalizeSql(String sql) { return sql.toLowerCase() .replaceAll("/\\*(?:.|\r|\n)*?\\*/", " ") - .replaceAll("\\s*on\\s+1\\s*=\\s*1\\s*", " ") + .replaceAll("\\s*on\\s+1\\s*=\\s*1\\s*", " on true ") .replaceAll("\\s+", " ") .replaceAll("\\( +", "(") .replaceAll(" +\\)", ")") @@ -314,9 +317,9 @@ private static String normalizeSql(String sql) { private void checkQuery(String qry) throws Exception { Prepared prepared = parse(qry); - GridSqlQueryParser ses = new GridSqlQueryParser(); + GridSqlQuery gQry = new GridSqlQueryParser().parse(prepared); - String res = ses.parse(prepared).getSQL(); + String res = gQry.getSQL(); System.out.println(normalizeSql(res)); From 385355a157993f4d78ced551bbaa168181748a7d Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 6 Sep 2016 12:09:38 +0300 Subject: [PATCH 101/487] GridNearOptimisticTxPrepareFuture: fixed javadoc warning. --- .../distributed/near/GridNearOptimisticTxPrepareFuture.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java index 5a300ff51ec12..e17a76c44b731 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java @@ -198,7 +198,7 @@ void onError(Throwable e, boolean discoThread) { } /** - * @return Keys for which {@link MiniFuture} isn't completed. + * @return Keys for which MiniFuture isn't completed. */ @SuppressWarnings("ForLoopReplaceableByForEach") public Set requestedKeys() { From bdc1b10cabcefb6684adcdb22bb6106e44148516 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 6 Sep 2016 14:45:01 +0300 Subject: [PATCH 102/487] IGNITE-2208 Queries with object arguments doesn't work wth BinaryMarshaller: use 'toBinary' for arguments conversion. --- .../processors/cache/IgniteCacheProxy.java | 12 ++++----- .../IgniteBinaryObjectQueryArgumentsTest.java | 25 ++++++++++--------- 2 files changed, 19 insertions(+), 18 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java index 8b2e6058c7f8a..81d4b49f882e6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java @@ -777,11 +777,13 @@ private void convertToBinary(final Query qry) { final SqlQuery sqlQry = (SqlQuery) qry; convertToBinary(sqlQry.getArgs()); - } else if (qry instanceof SpiQuery) { + } + else if (qry instanceof SpiQuery) { final SpiQuery spiQry = (SpiQuery) qry; convertToBinary(spiQry.getArgs()); - } else if (qry instanceof SqlFieldsQuery) { + } + else if (qry instanceof SqlFieldsQuery) { final SqlFieldsQuery fieldsQry = (SqlFieldsQuery) qry; convertToBinary(fieldsQry.getArgs()); @@ -798,10 +800,8 @@ private void convertToBinary(final Object[] args) { if (args == null) return; - for (int i = 0; i < args.length; i++) { - if (args[i] != null && !BinaryUtils.isBinaryType(args[i].getClass())) - args[i] = ctx.toCacheObject(args[i]); - } + for (int i = 0; i < args.length; i++) + args[i] = ctx.cacheObjects().binary().toBinary(args[i]); } /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java index 8a0c5c89adb22..d87b8b5b084a9 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteBinaryObjectQueryArgumentsTest.java @@ -56,31 +56,31 @@ public class IgniteBinaryObjectQueryArgumentsTest extends GridCommonAbstractTest private static final int NODES = 3; /** */ - public static final String PRIM_CACHE = "prim-cache"; + private static final String PRIM_CACHE = "prim-cache"; /** */ - public static final String STR_CACHE = "str-cache"; + private static final String STR_CACHE = "str-cache"; /** */ - public static final String ENUM_CACHE = "enum-cache"; + private static final String ENUM_CACHE = "enum-cache"; /** */ - public static final String UUID_CACHE = "uuid-cache"; + private static final String UUID_CACHE = "uuid-cache"; /** */ - public static final String DATE_CACHE = "date-cache"; + private static final String DATE_CACHE = "date-cache"; /** */ - public static final String TIMESTAMP_CACHE = "timestamp-cache"; + private static final String TIMESTAMP_CACHE = "timestamp-cache"; /** */ - public static final String BIG_DECIMAL_CACHE = "decimal-cache"; + private static final String BIG_DECIMAL_CACHE = "decimal-cache"; /** */ - public static final String OBJECT_CACHE = "obj-cache"; + private static final String OBJECT_CACHE = "obj-cache"; /** */ - public static final String FIELD_CACHE = "field-cache"; + private static final String FIELD_CACHE = "field-cache"; /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { @@ -110,6 +110,7 @@ protected CacheMemoryMode memoryMode() { } /** + * @param cacheName Cache name. * @return Cache config. */ protected CacheConfiguration getCacheConfiguration(final String cacheName) { @@ -368,7 +369,7 @@ private void testValQuery(final String cacheName, final T val1, final T val2 } /** - * @throws Exception + * @throws Exception If failed. */ public void testFieldSearch() throws Exception { final IgniteCache cache = ignite(0).cache(FIELD_CACHE); @@ -455,7 +456,7 @@ public static class TestKey { /** * @param id Key. */ - public TestKey(int id) { + TestKey(int id) { this.id = id; } @@ -536,7 +537,7 @@ private static class SearchValue { * @param person Person. * @param enumKey Enum. */ - public SearchValue( + SearchValue( final UUID uuid, final String str, final BigDecimal decimal, From 224cae1dd5dc5d95843c6ea1a72f61f27ea35cb4 Mon Sep 17 00:00:00 2001 From: sboikov Date: Wed, 7 Sep 2016 10:18:20 +0300 Subject: [PATCH 103/487] Fixed IgniteCacheExpiryPolicyAbstractTest.testZeroOnAccess. --- .../CacheLateAffinityAssignmentTest.java | 4 ---- .../IgniteCacheExpiryPolicyAbstractTest.java | 14 +++++++++++++- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java index 4fb17e83ef953..7e37450dd5621 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java @@ -1743,10 +1743,6 @@ public void testNoForceKeysRequests() throws Exception { final List ccfgs = new ArrayList<>(); - ccfgs.add(cacheConfiguration("ac1", ATOMIC, 0)); - ccfgs.add(cacheConfiguration("ac2", ATOMIC, 1)); - ccfgs.add(cacheConfiguration("ac3", ATOMIC, 2)); - ccfgs.add(cacheConfiguration("tc1", TRANSACTIONAL, 0)); ccfgs.add(cacheConfiguration("tc2", TRANSACTIONAL, 1)); ccfgs.add(cacheConfiguration("tc3", TRANSACTIONAL, 2)); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java index 794519a0dae48..f22ca6d30b9c4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java @@ -230,7 +230,7 @@ public void testZeroOnAccess() throws Exception { zeroOnAccess(key); } - IgniteCache cache = jcache(0); + final IgniteCache cache = jcache(0); Integer key = primaryKey(cache); @@ -240,12 +240,24 @@ public void testZeroOnAccess() throws Exception { cache.get(key); // Access using get. + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return !cache.iterator().hasNext(); + } + }, 1000); + assertFalse(cache.iterator().hasNext()); cache0.put(key, 1); assertNotNull(cache.iterator().next()); // Access using iterator. + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return !cache.iterator().hasNext(); + } + }, 1000); + assertFalse(cache.iterator().hasNext()); } From b5121adff860d96e5954d3466137854f8fae4c27 Mon Sep 17 00:00:00 2001 From: sboikov Date: Wed, 7 Sep 2016 12:44:24 +0300 Subject: [PATCH 104/487] IGNITE-2629: ODBC: GridNioAsyncNotifyFilter should process onSessionOpened synchronously --- .../ignite/internal/processors/odbc/OdbcProcessor.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java index a672d7cf6ac94..9388a8eca682a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.util.nio.GridNioCodecFilter; import org.apache.ignite.internal.util.nio.GridNioFilter; import org.apache.ignite.internal.util.nio.GridNioServer; +import org.apache.ignite.internal.util.nio.GridNioSession; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; @@ -123,7 +124,11 @@ public OdbcProcessor(GridKernalContext ctx) { for (int port = hostPort.portFrom(); port <= hostPort.portTo(); port++) { try { GridNioFilter[] filters = new GridNioFilter[] { - new GridNioAsyncNotifyFilter(ctx.gridName(), odbcExecSvc, log), + new GridNioAsyncNotifyFilter(ctx.gridName(), odbcExecSvc, log) { + @Override public void onSessionOpened(GridNioSession ses) throws IgniteCheckedException { + proceedSessionOpened(ses); + } + }, new GridNioCodecFilter(new OdbcBufferedParser(), log, false) }; From d672f2989582fe7259260a25f0a8ed26f31f5b09 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Wed, 7 Sep 2016 14:28:06 +0300 Subject: [PATCH 105/487] IGNITE-3705: CPP: Fixed warnings for MSVC 14.0. This closes #1038. --- .../platforms/cpp/common/include/ignite/ignite_error.h | 9 +++++++++ .../platforms/cpp/odbc/os/win/src/system/ui/window.cpp | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/modules/platforms/cpp/common/include/ignite/ignite_error.h b/modules/platforms/cpp/common/include/ignite/ignite_error.h index a41a42f9aedf5..17a24dc2535ff 100644 --- a/modules/platforms/cpp/common/include/ignite/ignite_error.h +++ b/modules/platforms/cpp/common/include/ignite/ignite_error.h @@ -72,6 +72,11 @@ throw ignite::IgniteError(code, stream.str().c_str()); \ } +#ifdef _MSC_VER +# pragma warning(push) +# pragma warning(disable : 4275) +#endif //_MSC_VER + namespace ignite { namespace java @@ -282,4 +287,8 @@ namespace ignite }; } +#ifdef _MSC_VER +# pragma warning(pop) +#endif //_MSC_VER + #endif //_IGNITE_IGNITE_ERROR \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp b/modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp index 1143f01321600..aca23eb99deaf 100644 --- a/modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp +++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp @@ -89,7 +89,7 @@ namespace ignite width, height, parent ? parent->GetHandle() : NULL, - reinterpret_cast(id), + reinterpret_cast(static_cast(id)), GetHInstance(), this ); From 121b89b07990068a31d384da8d6f3c781cf0efb8 Mon Sep 17 00:00:00 2001 From: sboikov Date: Wed, 7 Sep 2016 14:36:51 +0300 Subject: [PATCH 106/487] Fixed NPE in GridCacheAbstractFullApiSelfTest in multi jvm tests. --- .../processors/cache/GridCacheAbstractFullApiSelfTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java index 3f4d812efaaed..a31c82ef72bca 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java @@ -296,7 +296,7 @@ protected CacheMemoryMode memoryMode() { for (int i = 0; i < gridCount(); i++) { Boolean clientMode = grid(i).configuration().isClientMode(); - if (clientMode) + if (clientMode != null && clientMode) // Can be null in multi jvm tests. continue; grid(0).services(grid(0).cluster()).deployNodeSingleton(SERVICE_NAME1, new DummyServiceImpl()); From 2703648f380037a833e55a52a6c33e844a9e48fc Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Wed, 7 Sep 2016 17:12:34 +0300 Subject: [PATCH 107/487] IGNITE-3423 IGFS: Fixed incorrect result of IgfsGlobalSpaceTask for IGFS of unlimited size. This closes #1032. --- .../processors/igfs/IgfsDataManager.java | 2 +- .../processors/igfs/IgfsMaxSizeSelfTest.java | 122 ++++++++++++++++++ .../testsuites/IgniteIgfsTestSuite.java | 3 + 3 files changed, 126 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMaxSizeSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java index 1397e4e2e79f0..5e2c6b26f3dd9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java @@ -250,7 +250,7 @@ public long spaceSize() { * @return Maximum number of bytes for IGFS data cache. */ public long maxSpaceSize() { - return dataCachePrj.igfsDataSpaceMax(); + return (igfsCtx.configuration().getMaxSpaceSize() <= 0) ? 0 : dataCachePrj.igfsDataSpaceMax(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMaxSizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMaxSizeSelfTest.java new file mode 100644 index 0000000000000..c376c52d6126d --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMaxSizeSelfTest.java @@ -0,0 +1,122 @@ +/* + * 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.igfs; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.FileSystemConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.internal.IgniteEx; + +/** + * Check max size limit. + */ +@SuppressWarnings("ConstantConditions") +public class IgfsMaxSizeSelfTest extends IgfsCommonAbstractTest { + /** Work directory. */ + private static long maxSize; + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + FileSystemConfiguration igfsCfg = new FileSystemConfiguration(); + + igfsCfg.setDataCacheName("dataCache"); + igfsCfg.setMetaCacheName("metaCache"); + igfsCfg.setName("test"); + + if (maxSize > 0) + igfsCfg.setMaxSpaceSize(maxSize); + + CacheConfiguration dataCacheCfg = defaultCacheConfiguration(); + + dataCacheCfg.setName("dataCache"); + dataCacheCfg.setCacheMode(CacheMode.PARTITIONED); + dataCacheCfg.setNearConfiguration(null); + dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(2)); + dataCacheCfg.setBackups(0); + dataCacheCfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + dataCacheCfg.setOffHeapMaxMemory(0); + + CacheConfiguration metaCacheCfg = defaultCacheConfiguration(); + + metaCacheCfg.setName("metaCache"); + metaCacheCfg.setNearConfiguration(null); + metaCacheCfg.setCacheMode(CacheMode.REPLICATED); + metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + metaCacheCfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + + cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg); + cfg.setFileSystemConfiguration(igfsCfg); + cfg.setGridName(gridName); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testDefaultOrZero() throws Exception { + IgniteEx ig = startGrid(0); + + try { + assertEquals(0, ((IgfsImpl)ig.igfsx("test")).globalSpace().spaceTotal()); + } + finally { + stopAllGrids(); + } + } + + /** + * @throws Exception If failed. + */ + public void testNegative() throws Exception { + maxSize = -1; + + IgniteEx ig = startGrid(0); + + try { + assertEquals(0, ((IgfsImpl)ig.igfsx("test")).globalSpace().spaceTotal()); + } + finally { + stopAllGrids(); + } + } + + /** + * @throws Exception If failed. + */ + public void testPositive() throws Exception { + maxSize = 1 << 20; + + IgniteEx ig = startGrid(0); + + try { + assertEquals(maxSize, ((IgfsImpl)ig.igfsx("test")).globalSpace().spaceTotal()); + } + finally { + stopAllGrids(); + } + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java index 44199d4b149fa..0241068bcaac4 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsCacheSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsDualAsyncClientSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsDualSyncClientSelfTest; +import org.apache.ignite.internal.processors.igfs.IgfsMaxSizeSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsPrimaryClientSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsDataManagerSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsDualAsyncSelfTest; @@ -148,6 +149,8 @@ public static TestSuite suite() throws Exception { // TODO: Enable when IGFS failover is fixed. //suite.addTestSuite(IgfsBackupFailoverSelfTest.class); + suite.addTestSuite(IgfsMaxSizeSelfTest.class); + return suite; } } \ No newline at end of file From 88d027d280ba437a69d8100db1e0e6a5f034188c Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Wed, 7 Sep 2016 17:20:25 +0300 Subject: [PATCH 108/487] IGNITE-3851 IGFS: Support direct PROXY mode invocation in method: exists. This closes #1039. --- .../internal/processors/igfs/IgfsImpl.java | 25 +++++++++------ .../igfs/IgfsAbstractBaseSelfTest.java | 7 ++-- .../processors/igfs/IgfsProxySelfTest.java | 32 +++++++++++++++++++ 3 files changed, 51 insertions(+), 13 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProxySelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 3b25c82e13846..642352ebcf443 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -236,7 +236,17 @@ public final class IgfsImpl implements IgfsEx { modes = new ArrayList<>(cfgModes.size()); for (Map.Entry mode : cfgModes.entrySet()) { - IgfsMode mode0 = secondaryFs == null ? mode.getValue() == PROXY ? PROXY : PRIMARY : mode.getValue(); + IgfsMode mode0; + + if (mode.getValue() == PROXY) { + if (secondaryFs == null) + throw new IgniteCheckedException("Mode cannot be PROXY if secondary file system hasn't" + + " been defined: " + mode.getKey()); + + mode0 = PROXY; + } + else + mode0 = secondaryFs == null ? PRIMARY : mode.getValue(); try { modes.add(new T2<>(new IgfsPath(mode.getKey()), mode0)); @@ -539,8 +549,10 @@ else if (val) break; - default: - assert false : "Unknown mode."; + case PROXY: + res = secondaryFs.exists(path); + + break; } return res; @@ -1741,12 +1753,7 @@ private IgfsGlobalSpaceTask(@Nullable String igfsName) { * @return Mode. */ private IgfsMode resolveMode(IgfsPath path) { - IgfsMode mode = modeRslvr.resolveMode(path); - - if (mode == PROXY) - throw new IgfsInvalidPathException("PROXY mode cannot be used in IGFS directly: " + path); - - return mode; + return modeRslvr.resolveMode(path); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java index 9575bd04abd88..03f24a4334056 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java @@ -64,8 +64,9 @@ import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED; import static org.apache.ignite.cache.CacheMode.PARTITIONED; import static org.apache.ignite.cache.CacheMode.REPLICATED; +import static org.apache.ignite.igfs.IgfsMode.DUAL_ASYNC; +import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC; import static org.apache.ignite.igfs.IgfsMode.PRIMARY; -import static org.apache.ignite.igfs.IgfsMode.PROXY; /** * Test fo regular igfs operations. @@ -205,12 +206,10 @@ protected IgfsAbstractBaseSelfTest(IgfsMode mode) { * @param memoryMode Memory mode. */ protected IgfsAbstractBaseSelfTest(IgfsMode mode, CacheMemoryMode memoryMode) { - assert mode != null && mode != PROXY; - this.mode = mode; this.memoryMode = memoryMode; - dual = mode != PRIMARY; + dual = (mode == DUAL_SYNC || mode == DUAL_ASYNC); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProxySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProxySelfTest.java new file mode 100644 index 0000000000000..3b8c606592dac --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProxySelfTest.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.igfs; + +import static org.apache.ignite.igfs.IgfsMode.PROXY; + +/** + * Tests for PRIMARY mode. + */ +public class IgfsProxySelfTest extends IgfsAbstractSelfTest { + /** + * Constructor. + */ + public IgfsProxySelfTest() { + super(PROXY); + } +} \ No newline at end of file From 216477e5a9badc09133a7fe1f545e048a23a8218 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Wed, 7 Sep 2016 17:30:48 +0300 Subject: [PATCH 109/487] IGNITE-3852 IGFS: Support direct PROXY mode invocation in method: info. This closes #1040. --- .../apache/ignite/internal/processors/igfs/IgfsImpl.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 642352ebcf443..439e9ba7ad613 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -1526,7 +1526,14 @@ private IgfsFileImpl resolveFileInfo(IgfsPath path, IgfsMode mode) throws Except break; default: - assert false : "Unknown mode: " + mode; + assert mode == PROXY : "Unknown mode: " + mode; + + IgfsFile status = secondaryFs.info(path); + + if (status != null) + return new IgfsFileImpl(status, data.groupBlockSize()); + else + return null; } if (info == null) From 9389328afdad8478b16f124f16cb45dde6c33aaf Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Wed, 7 Sep 2016 17:35:45 +0300 Subject: [PATCH 110/487] IGNITE-3853 IGFS: Support direct PROXY mode invocation in method: update. This closes #1041. --- .../internal/processors/igfs/IgfsImpl.java | 52 ++++++++++++------- 1 file changed, 33 insertions(+), 19 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 439e9ba7ad613..27618be2144cb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -628,36 +628,50 @@ else if (val) IgfsMode mode = resolveMode(path); - if (mode != PRIMARY) { - assert IgfsUtils.isDualMode(mode); + switch (mode) { + case PRIMARY: + { + List fileIds = meta.idsForPath(path); - await(path); + IgniteUuid fileId = fileIds.get(fileIds.size() - 1); - IgfsEntryInfo info = meta.updateDual(secondaryFs, path, props); + if (fileId != null) { + IgfsEntryInfo info = meta.updateProperties(fileId, props); - if (info == null) - return null; + if (info != null) { + if (evts.isRecordable(EVT_IGFS_META_UPDATED)) + evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_META_UPDATED, props)); - return new IgfsFileImpl(path, info, data.groupBlockSize()); - } + return new IgfsFileImpl(path, info, data.groupBlockSize()); + } + } - List fileIds = meta.idsForPath(path); + break; + } - IgniteUuid fileId = fileIds.get(fileIds.size() - 1); + case DUAL_ASYNC: + case DUAL_SYNC: + { + await(path); - if (fileId == null) - return null; + IgfsEntryInfo info = meta.updateDual(secondaryFs, path, props); - IgfsEntryInfo info = meta.updateProperties(fileId, props); + if (info != null) + return new IgfsFileImpl(path, info, data.groupBlockSize()); - if (info != null) { - if (evts.isRecordable(EVT_IGFS_META_UPDATED)) - evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_META_UPDATED, props)); + break; + } + + default: + assert mode == PROXY : "Unknown mode: " + mode; - return new IgfsFileImpl(path, info, data.groupBlockSize()); + IgfsFile file = secondaryFs.update(path, props); + + if (file != null) + return new IgfsFileImpl(secondaryFs.update(path, props), data.groupBlockSize()); } - else - return null; + + return null; } }); } From e1c7937dabb3508183ccbd9afa3f76d5aaa67236 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Wed, 7 Sep 2016 17:39:13 +0300 Subject: [PATCH 111/487] IGNITE-3854 IGFS: Support direct PROXY mode invocation in method: rename. This closes #1042. --- .../internal/processors/igfs/IgfsImpl.java | 24 +++++++++++++------ 1 file changed, 17 insertions(+), 7 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 27618be2144cb..12d2830dd7f96 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -710,17 +710,27 @@ else if (val) throw new IgfsInvalidPathException("Cannot move file to a path with different eviction " + "exclude setting (need to copy and remove)"); - if (mode != PRIMARY) { - assert IgfsUtils.isDualMode(mode); // PROXY mode explicit usage is forbidden. + switch (mode) { + case PRIMARY: + meta.move(src, dest); - await(src, dest); + break; - meta.renameDual(secondaryFs, src, dest); + case DUAL_ASYNC: + case DUAL_SYNC: + await(src, dest); - return null; - } + meta.renameDual(secondaryFs, src, dest); - meta.move(src, dest); + break; + + default: + assert mode == PROXY : "Unknown mode: " + mode; + + secondaryFs.rename(src, dest); + + break; + } return null; } From 6af6560a4c90ac24644991ce25081c7548279aa6 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Thu, 8 Sep 2016 14:43:57 +0700 Subject: [PATCH 112/487] Web Console beta-3. --- .../event/VisorGridDiscoveryEventV2.java | 80 ++ .../visor/node/VisorNodeDataCollectorJob.java | 10 +- .../internal/visor/util/VisorEventMapper.java | 96 +- .../internal/visor/util/VisorTaskUtils.java | 22 +- modules/web-console/.gitignore | 6 + modules/web-console/DEVNOTES.txt | 22 +- modules/web-console/backend/.babelrc | 9 + modules/web-console/backend/.eslintrc | 188 ++++ modules/web-console/backend/.gitignore | 8 + .../serve => backend}/agent_dists/README.txt | 0 .../main/js/serve => backend/app}/agent.js | 97 +- .../{src/main/js/serve => backend/app}/app.js | 19 + .../main/js/serve => backend/app}/browser.js | 62 +- .../js/serve => backend/app}/configure.js | 8 +- .../main/js/serve.js => backend/app/index.js} | 44 +- .../main/js/serve => backend/app}/mongo.js | 99 +- modules/web-console/backend/app/nconf.js | 48 + modules/web-console/backend/app/routes.js | 64 ++ .../main/js/serve => backend/app}/settings.js | 30 +- .../config/settings.json.sample | 16 +- .../backend/errors/AppErrorException.js | 36 + .../backend/errors/AuthFailedException.js | 30 + .../errors/DuplicateKeyException.js} | 19 +- .../errors/IllegalAccessError.js} | 20 +- .../errors/IllegalArgumentException.js | 29 + .../errors/MissingResourceException.js | 30 + .../backend/errors/ServerErrorException.js | 36 + modules/web-console/backend/errors/index.js | 39 + .../form/field/field.css => backend/index.js} | 8 +- .../injector.js} | 23 +- .../middlewares/api.js} | 48 +- .../web-console/backend/middlewares/host.js | 39 + .../web-console/backend/middlewares/user.js | 36 + modules/web-console/backend/package.json | 71 ++ modules/web-console/backend/routes/admin.js | 84 ++ modules/web-console/backend/routes/agent.js | 53 + modules/web-console/backend/routes/caches.js | 65 ++ .../web-console/backend/routes/clusters.js | 64 ++ .../backend/routes/configuration.js | 41 + .../main/js/serve => backend}/routes/demo.js | 28 +- .../serve => backend}/routes/demo/caches.json | 0 .../routes/demo/clusters.json | 0 .../routes/demo/domains.json | 0 .../serve => backend}/routes/demo/igfss.json | 0 modules/web-console/backend/routes/domains.js | 76 ++ modules/web-console/backend/routes/igfss.js | 65 ++ .../web-console/backend/routes/notebooks.js | 80 ++ modules/web-console/backend/routes/profile.js | 73 ++ .../js/serve => backend}/routes/public.js | 121 +-- .../web-console/backend/services/agents.js | 82 ++ modules/web-console/backend/services/auth.js | 47 + .../web-console/backend/services/caches.js | 144 +++ .../web-console/backend/services/clusters.js | 141 +++ .../backend/services/configurations.js | 59 ++ .../web-console/backend/services/domains.js | 187 ++++ modules/web-console/backend/services/igfss.js | 136 +++ modules/web-console/backend/services/mails.js | 131 +++ .../web-console/backend/services/notebooks.js | 104 ++ .../web-console/backend/services/sessions.js | 63 ++ .../web-console/backend/services/spaces.js | 75 ++ modules/web-console/backend/services/users.js | 229 +++++ .../backend/test/config/settings.json | 20 + .../backend/test/data/accounts.json | 18 + .../web-console/backend/test/data/caches.json | 87 ++ .../backend/test/data/clusters.json | 50 + .../backend/test/data/domains.json | 307 ++++++ .../web-console/backend/test/data/igfss.json | 10 + .../test/injector.js} | 22 +- .../backend/test/unit/CacheService.test.js | 192 ++++ .../backend/test/unit/ClusterService.test.js | 190 ++++ .../backend/test/unit/DomainService.test.js | 198 ++++ .../backend/test/unit/IgfsService.test.js | 190 ++++ .../docker/compose/backend/.dockerignore | 1 + .../docker/compose/backend/Dockerfile | 30 + .../docker/compose/backend/build.sh | 57 ++ .../docker/compose/docker-compose.yml | 59 ++ .../docker/compose/frontend/.dockerignore | 3 + .../docker/compose/frontend/Dockerfile | 32 + .../docker/compose/frontend/DockerfileBuild | 30 + .../docker/compose/frontend/build.sh | 59 ++ .../docker/compose/frontend/nginx/nginx.conf | 57 ++ .../compose/frontend/nginx/web-console.conf | 59 ++ .../docker/standalone/.dockerignore | 2 + .../web-console/docker/standalone/Dockerfile | 87 ++ .../web-console/docker/standalone/build.sh | 59 ++ .../docker/standalone/docker-compose.yml | 41 + .../docker/standalone/entrypoint.sh | 23 + .../docker/standalone/nginx/nginx.conf | 55 + .../docker/standalone/nginx/web-console.conf | 54 + modules/web-console/frontend/.babelrc | 9 + .../{src/main/js => frontend}/.eslintrc | 4 +- .../{src/main/js => frontend}/.gitignore | 2 - .../main/js => frontend}/app/app.config.js | 0 .../{src/main/js => frontend}/app/app.js | 38 +- .../app/controllers/auth.controller.js | 0 .../controllers/reset-password.controller.js | 11 +- .../main/js => frontend}/app/data/colors.json | 0 .../js => frontend}/app/data/countries.json | 0 .../js => frontend}/app/data/demo-info.json | 0 .../js => frontend}/app/data/event-types.json | 0 .../app/data/getting-started.json | 0 .../app/data/java-classes.json | 1 + .../app/data/java-keywords.json | 0 .../app/data/java-primitives.json | 0 .../app/data/pom-dependencies.json | 0 .../js => frontend}/app/decorator/select.js | 0 .../js => frontend}/app/decorator/tooltip.js | 0 .../app/directives/auto-focus.directive.js | 0 .../directives/bs-affix-update.directive.js | 0 .../app/directives/centered/centered.css | 0 .../directives/centered/centered.directive.js | 0 .../directives/copy-to-clipboard.directive.js | 0 .../hide-on-state-change.directive.js | 0 .../information/information.directive.js | 0 .../directives/information/information.jade | 0 .../directives/information/information.scss | 0 .../app/directives/match.directive.js | 0 .../directives/on-click-focus.directive.js | 0 .../on-enter-focus-move.directive.js | 0 .../app/directives/on-enter.directive.js | 0 .../app/directives/on-escape.directive.js | 0 .../ui-ace-docker/ui-ace-docker.controller.js | 0 .../ui-ace-docker/ui-ace-docker.directive.js | 0 .../ui-ace-docker/ui-ace-docker.jade | 0 .../ui-ace-java/ui-ace-java.controller.js | 0 .../ui-ace-java/ui-ace-java.directive.js | 16 +- .../directives/ui-ace-java/ui-ace-java.jade | 0 .../ui-ace-pojos/ui-ace-pojos.controller.js | 0 .../ui-ace-pojos/ui-ace-pojos.directive.js | 0 .../directives/ui-ace-pojos/ui-ace-pojos.jade | 0 .../ui-ace-pom/ui-ace-pom.controller.js | 0 .../ui-ace-pom/ui-ace-pom.directive.js | 0 .../app/directives/ui-ace-pom/ui-ace-pom.jade | 0 .../app/directives/ui-ace-tabs.directive.js | 1 + .../ui-ace-xml/ui-ace-xml.controller.js | 0 .../ui-ace-xml/ui-ace-xml.directive.js | 16 +- .../app/directives/ui-ace-xml/ui-ace-xml.jade | 0 .../app/filters/byName.filter.js | 0 .../app/filters/domainsValidation.filter.js | 0 .../frontend/app/filters/duration.filter.js | 38 + .../app/filters/hasPojo.filter.js | 0 .../app/helpers/jade/form.jade} | 17 +- .../jade/form/form-field-checkbox.jade | 38 + .../jade/form/form-field-datalist.jade | 51 + .../helpers/jade/form/form-field-down.jade} | 7 +- .../jade/form/form-field-dropdown.jade | 50 + .../jade/form/form-field-feedback.jade | 29 + .../helpers/jade/form/form-field-label.jade} | 19 +- .../helpers/jade/form/form-field-number.jade | 52 + .../helpers/jade/form/form-field-text.jade | 47 + .../app/helpers/jade/form/form-field-up.jade} | 3 +- .../app/helpers/jade/form/form-group.jade | 23 + .../app/helpers/jade/mixins.jade | 511 +++++----- .../app/modules/Demo/Demo.module.js | 4 +- .../js => frontend}/app/modules/ace.module.js | 0 .../app/modules/agent/agent.module.js | 46 +- .../app/modules/branding/branding.module.js | 0 .../app/modules/branding/branding.provider.js | 0 .../modules/branding/features.directive.js | 0 .../app/modules/branding/footer.directive.js | 0 .../modules/branding/header-logo.directive.js | 0 .../app/modules/branding/header-logo.jade | 2 +- .../branding/header-title.directive.js | 0 .../branding/powered-by-apache.directive.js | 0 .../modules/branding/powered-by-apache.jade | 0 .../app/modules/branding/terms.directive.js | 0 .../configuration/EventGroups.provider.js | 0 .../modules/configuration/Sidebar.provider.js | 0 .../configuration/configuration.module.js | 0 .../configuration/generator/Docker.service.js | 0 .../configuration/generator/Java.service.js | 0 .../configuration/generator/Pom.service.js | 32 +- .../configuration/generator/Xml.service.js | 0 .../configuration/sidebar.directive.js | 0 .../dialog/dialog-content.directive.js | 0 .../modules/dialog/dialog-title.directive.js | 0 .../app/modules/dialog/dialog.controller.js | 0 .../app/modules/dialog/dialog.directive.js | 0 .../app/modules/dialog/dialog.factory.js | 0 .../app/modules/dialog/dialog.jade | 0 .../app/modules/dialog/dialog.module.js | 0 .../field/bs-select-placeholder.directive.js | 0 .../app/modules/form/field/down.directive.js | 26 +- .../app/modules/form/field/feedback.scss | 37 + .../app/modules/form/field/field.scss | 43 + .../field/form-control-feedback.directive.js | 2 +- .../form/field/input/autofocus.directive.js | 4 +- .../app/modules/form/field/input/select.scss} | 15 +- .../app/modules/form/field/input/text.scss} | 0 .../app/modules/form/field/label.directive.js | 4 +- .../modules/form/field/tooltip.directive.js | 0 .../app/modules/form/field/up.directive.js | 25 +- .../app/modules/form/form.module.js | 29 +- .../app/modules/form/group/add.directive.js | 2 +- .../modules/form/group/tooltip.directive.js | 2 +- .../modules/form/panel/chevron.directive.js | 0 .../app/modules/form/panel/field.directive.js | 69 ++ .../app/modules/form/panel/panel.directive.js | 0 .../modules/form/panel/revert.directive.js | 1 + .../form/validator/ipaddress.directive.js | 18 +- .../java-built-in-class.directive.js | 0 .../validator/java-identifier.directive.js | 0 .../form/validator/java-keywords.directive.js | 0 .../validator/java-package-name.directive.js | 0 .../java-package-specified.directive.js | 0 .../validator/property-unique.directive.js | 0 .../property-value-specified.directive.js | 0 .../form/validator/unique.directive.js | 0 .../modules/form/validator/uuid.directive.js} | 22 +- .../GettingStarted.provider.js | 0 .../app/modules/loading/loading.css | 0 .../app/modules/loading/loading.directive.js | 0 .../app/modules/loading/loading.jade | 0 .../app/modules/loading/loading.module.js | 0 .../app/modules/loading/loading.service.js | 0 .../app/modules/navbar/Navbar.provider.js | 0 .../app/modules/navbar/Userbar.provider.js | 0 .../app/modules/navbar/navbar.directive.js | 0 .../app/modules/navbar/navbar.module.js | 0 .../app/modules/navbar/userbar.directive.js | 14 +- .../app/modules/socket.module.js | 0 .../frontend/app/modules/sql/Notebook.data.js | 157 +++ .../app/modules/sql/Notebook.service.js | 74 ++ .../app/modules/sql/notebook.controller.js | 60 ++ .../app/modules/sql/scan-filter-input.jade | 39 + .../modules/sql/scan-filter-input.service.js | 51 + .../app/modules/sql/sql.controller.js} | 584 ++++++----- .../frontend/app/modules/sql/sql.module.js | 60 ++ .../app/modules/states/admin.state.js | 3 +- .../app/modules/states/configuration.state.js | 97 ++ .../configuration/Configuration.resource.js | 42 + .../configuration/caches/concurrency.jade | 22 +- .../states/configuration/caches/general.jade | 23 +- .../states/configuration/caches/memory.jade | 42 +- .../configuration/caches/node-filter.jade | 108 ++ .../states/configuration/caches/query.jade | 66 +- .../configuration/caches/rebalance.jade | 18 +- .../caches/server-near-cache.jade | 16 +- .../configuration/caches/statistics.jade | 8 +- .../states/configuration/caches/store.jade | 181 ++-- .../states/configuration/clusters/atomic.jade | 10 +- .../configuration/clusters/attributes.jade | 5 +- .../states/configuration/clusters/binary.jade | 45 +- .../configuration/clusters/cache-key-cfg.jade | 53 + .../configuration/clusters/collision.jade | 24 +- .../clusters/collision/custom.jade | 2 +- .../clusters/collision/fifo-queue.jade | 5 +- .../clusters/collision/job-stealing.jade | 17 +- .../clusters/collision/priority-queue.jade | 15 +- .../configuration/clusters/communication.jade | 57 +- .../configuration/clusters/connector.jade | 44 +- .../configuration/clusters/deployment.jade | 84 +- .../configuration/clusters/discovery.jade | 56 +- .../states/configuration/clusters/events.jade | 4 +- .../configuration/clusters/failover.jade | 50 +- .../configuration/clusters/general.jade | 73 ++ .../clusters/general/discovery/cloud.jade | 115 ++- .../clusters/general/discovery/google.jade | 8 +- .../clusters/general/discovery/jdbc.jade | 32 + .../clusters/general/discovery/multicast.jade | 74 +- .../clusters/general/discovery/s3.jade | 2 +- .../clusters/general/discovery/shared.jade | 2 +- .../clusters/general/discovery/vm.jade | 79 ++ .../clusters/general/discovery/zookeeper.jade | 35 +- .../bounded-exponential-backoff.jade | 6 +- .../zookeeper/retrypolicy/custom.jade | 2 +- .../retrypolicy/exponential-backoff.jade | 6 +- .../zookeeper/retrypolicy/forever.jade | 2 +- .../zookeeper/retrypolicy/n-times.jade | 4 +- .../zookeeper/retrypolicy/one-time.jade | 2 +- .../zookeeper/retrypolicy/until-elapsed.jade | 4 +- .../states/configuration/clusters/igfs.jade | 6 +- .../states/configuration/clusters/logger.jade | 19 +- .../configuration/clusters/logger/custom.jade | 3 +- .../configuration/clusters/logger/log4j.jade | 7 +- .../configuration/clusters/logger/log4j2.jade | 5 +- .../configuration/clusters/marshaller.jade | 24 +- .../configuration/clusters/metrics.jade | 13 +- .../states/configuration/clusters/ssl.jade | 89 +- .../states/configuration/clusters/swap.jade | 22 +- .../states/configuration/clusters/thread.jade | 12 +- .../states/configuration/clusters/time.jade | 10 +- .../configuration/clusters/transactions.jade | 28 +- .../states/configuration/domains/general.jade | 8 +- .../states/configuration/domains/query.jade | 9 +- .../states/configuration/domains/store.jade | 10 +- .../states/configuration/igfs/dual.jade | 8 +- .../configuration/igfs/fragmentizer.jade | 10 +- .../states/configuration/igfs/general.jade | 11 +- .../states/configuration/igfs/ipc.jade | 17 +- .../states/configuration/igfs/misc.jade | 28 +- .../states/configuration/igfs/secondary.jade | 10 +- .../configuration/preview-panel.directive.js | 0 .../summary/summary-tabs.directive.js | 0 .../summary/summary.controller.js | 42 +- .../app/modules/states/errors.state.js | 43 + .../app/modules/states/logout.state.js | 7 +- .../app/modules/states/password.state.js | 0 .../app/modules/states/profile.state.js | 3 +- .../app/modules/states/signin.state.js} | 35 +- .../app/modules/user/AclRoute.provider.js} | 41 +- .../frontend/app/modules/user/Auth.service.js | 56 ++ .../app/modules/user/User.service.js | 48 +- .../app/modules/user/permissions.js} | 19 +- .../frontend/app/modules/user/user.module.js | 73 ++ .../app/modules/version}/Version.provider.js | 0 .../app/services/ChartColors.service.js | 0 .../app/services/Clone.service.js | 0 .../app/services/Confirm.service.js | 20 +- .../app/services/ConfirmBatch.service.js | 0 .../app/services/CopyToClipboard.service.js | 0 .../app/services/Countries.service.js | 0 .../app/services/ErrorPopover.service.js | 126 +++ .../app/services/Focus.service.js | 0 .../app/services/FormUtils.service.js | 435 ++++++++ .../app/services/InetAddress.service.js | 0 .../app/services/JavaTypes.service.js | 15 +- .../app/services/LegacyTable.service.js | 209 ++++ .../app/services/LegacyUtils.service.js | 572 +++++++++++ .../app/services/Messages.service.js | 0 .../app/services/ModelNormalizer.service.js | 0 .../services/UnsavedChangesGuard.service.js | 0 .../{src/main/js => frontend}/app/vendor.js | 1 + .../controllers/admin-controller.js | 9 +- .../controllers/caches-controller.js | 164 ++- .../controllers/clusters-controller.js | 167 ++- .../controllers/domains-controller.js | 236 +++-- .../controllers/igfs-controller.js | 83 +- .../controllers/profile-controller.js | 5 +- .../generator/generator-common.js | 3 +- .../generator/generator-java.js | 164 ++- .../generator/generator-optional.js | 0 .../generator/generator-properties.js | 107 +- .../generator/generator-readme.js | 0 .../generator/generator-xml.js | 187 +++- .../gulpfile.babel.js/index.js | 0 .../gulpfile.babel.js/paths.js | 8 +- .../gulpfile.babel.js/tasks/build.js | 0 .../gulpfile.babel.js/tasks/bundle.js | 0 .../gulpfile.babel.js/tasks/clean.js | 0 .../gulpfile.babel.js/tasks/copy.js | 0 .../gulpfile.babel.js/tasks/ignite-modules.js | 0 .../gulpfile.babel.js/tasks/jade.js | 0 .../frontend/gulpfile.babel.js/tasks/test.js | 92 ++ .../gulpfile.babel.js/tasks/watch.js | 0 .../gulpfile.babel.js/webpack/common.js | 21 +- .../webpack/environments/development.js | 17 +- .../webpack/environments/production.js | 0 .../gulpfile.babel.js/webpack/index.js | 0 .../webpack/plugins/progress.js | 0 .../js => frontend}/ignite_modules/README.txt | 0 .../js => frontend}/ignite_modules/index.js | 0 .../{src/main/js => frontend}/package.json | 45 +- .../main/js => frontend}/public/favicon.ico | Bin .../js => frontend}/public/images/cache.png | Bin .../js => frontend}/public/images/cluster.png | Bin .../js => frontend}/public/images/docker.png | Bin .../js => frontend}/public/images/domains.png | Bin .../js => frontend}/public/images/igfs.png | Bin .../public/images/ignite-logo.png | Bin .../public/images/ignite-logo@2x.png | Bin .../public/images/ignite-puzzle.png | Bin .../js => frontend}/public/images/java.png | Bin .../public/images/pb-ignite.png | Bin .../public/images/pb-ignite@2x.png | Bin .../public/images/query-chart.png | Bin .../public/images/query-metadata.png | Bin .../public/images/query-table.png | Bin .../js => frontend}/public/images/summary.png | Bin .../js => frontend}/public/images/xml.png | Bin .../public/stylesheets/_bootstrap-custom.scss | 0 .../stylesheets/_bootstrap-variables.scss | 4 +- .../stylesheets/_font-awesome-custom.scss | 2 +- .../public/stylesheets/blocks/error.scss} | 22 +- .../public/stylesheets/style.scss | 163 +-- .../public/stylesheets/variables.scss | 0 .../frontend/test/e2e/exampe.test.js | 40 + .../web-console/frontend/test/karma.conf.js | 113 +++ .../frontend/test/protractor.conf.js | 50 + .../frontend/test/unit/JavaTypes.test.js | 69 ++ .../test/unit/UserAuth.test.js} | 26 +- modules/web-console/frontend/views/403.jade | 22 + modules/web-console/frontend/views/404.jade | 22 + .../{src/main/js => frontend}/views/base.jade | 0 .../views/configuration/caches.jade | 19 +- .../views/configuration/clusters.jade | 66 ++ .../views/configuration/domains-import.jade | 48 +- .../views/configuration/domains.jade | 8 +- .../views/configuration/igfs.jade | 14 +- .../views/configuration/sidebar.jade | 0 .../summary-project-structure.jade | 0 .../views/configuration/summary-tabs.jade | 0 .../views/configuration/summary.jade | 62 +- .../views/includes/footer.jade | 0 .../views/includes/header.jade | 6 +- .../main/js => frontend}/views/index.jade | 1 - .../main/js => frontend}/views/reset.jade | 0 .../js => frontend}/views/settings/admin.jade | 0 .../views/settings/profile.jade | 0 .../main/js => frontend}/views/signin.jade | 0 .../views/sql/cache-metadata.jade | 0 .../views/sql/chart-settings.jade | 0 .../views/sql/notebook-new.jade | 4 +- .../views/sql/paragraph-rate.jade | 0 .../main/js => frontend}/views/sql/sql.jade | 78 +- .../views/templates/agent-download.jade | 0 .../views/templates/alert.jade | 2 +- .../views/templates/batch-confirm.jade | 0 .../views/templates/clone.jade | 8 +- .../views/templates/confirm.jade | 0 .../views/templates/demo-info.jade | 0 .../views/templates/dropdown.jade | 11 +- .../views/templates/getting-started.jade | 0 .../views/templates/message.jade | 0 .../views/templates/pagination.jade | 0 .../views/templates/select.jade | 0 .../views/templates/validation-error.jade | 0 modules/web-console/pom.xml | 33 +- modules/web-console/src/main/js/.babelrc | 3 - .../app/controllers/notebooks.controller.js | 69 -- .../modules/form/field/dropdown.directive.js | 83 -- .../js/app/modules/form/field/dropdown.jade | 61 -- .../main/js/app/modules/form/field/field.jade | 27 - .../form/field/input/checkbox.directive.js | 66 -- .../form/field/input/datalist.directive.js | 122 --- .../modules/form/field/input/datalist.jade | 51 - .../form/field/input/number.directive.js | 76 -- .../app/modules/form/field/input/number.jade | 50 - .../form/field/input/text.directive.js | 126 --- .../js/app/modules/form/field/input/text.jade | 48 - .../app/modules/form/group/group.directive.js | 81 -- .../query-notebooks/query-notebooks.module.js | 115 --- .../app/modules/states/configuration.state.js | 226 ----- .../configuration/caches/query.directive.js | 27 - .../caches/rebalance.directive.js | 27 - .../caches/server-near-cache.directive.js | 27 - .../caches/statistics.directive.js | 27 - .../configuration/caches/store.directive.js | 27 - .../clusters/atomic.directive.js | 27 - .../clusters/attributes.directive.js | 27 - .../clusters/binary.directive.js | 27 - .../clusters/collision/custom.directive.js | 27 - .../collision/fifo-queue.directive.js | 27 - .../collision/job-stealing.directive.js | 27 - .../collision/priority-queue.directive.js | 27 - .../clusters/communication.directive.js | 27 - .../clusters/deployment.directive.js | 27 - .../clusters/events.directive.js | 27 - .../clusters/failover.directive.js | 27 - .../clusters/general.directive.js | 27 - .../configuration/clusters/general.jade | 68 -- .../general/discovery/cloud.directive.js | 27 - .../general/discovery/google.directive.js | 27 - .../general/discovery/jdbc.directive.js | 27 - .../general/discovery/multicast.directive.js | 27 - .../general/discovery/s3.directive.js | 27 - .../general/discovery/shared.directive.js | 27 - .../general/discovery/vm.directive.js | 27 - .../clusters/general/discovery/vm.jade | 90 -- .../general/discovery/zookeeper.directive.js | 27 - .../bounded-exponential-backoff.directive.js | 27 - .../zookeeper/retrypolicy/custom.directive.js | 27 - .../exponential-backoff.directive.js | 27 - .../retrypolicy/forever.directive.js | 27 - .../retrypolicy/n-times.directive.js | 27 - .../retrypolicy/one-time.directive.js | 27 - .../retrypolicy/until-elapsed.directive.js | 27 - .../clusters/logger.directive.js | 27 - .../clusters/logger/custom.directive.js | 27 - .../clusters/logger/log4j.directive.js | 27 - .../clusters/logger/log4j2.directive.js | 27 - .../clusters/marshaller.directive.js | 27 - .../clusters/metrics.directive.js | 27 - .../configuration/clusters/ssl.directive.js | 27 - .../configuration/clusters/swap.directive.js | 27 - .../clusters/thread.directive.js | 27 - .../configuration/clusters/time.directive.js | 27 - .../clusters/transactions.directive.js | 27 - .../domains/general.directive.js | 27 - .../configuration/domains/query.directive.js | 27 - .../configuration/domains/store.directive.js | 27 - .../configuration/igfs/dual.directive.js | 27 - .../igfs/fragmentizer.directive.js | 27 - .../configuration/igfs/general.directive.js | 27 - .../configuration/igfs/ipc.directive.js | 27 - .../configuration/igfs/misc.directive.js | 27 - .../configuration/igfs/secondary.directive.js | 27 - .../js/app/modules/states/signin.state.js | 53 - .../main/js/app/modules/user/Auth.service.js | 76 -- .../js/app/services/LegacyTable.service.js | 205 ---- .../js/app/services/LegacyUtils.service.js | 948 ------------------ modules/web-console/src/main/js/serve/mail.js | 75 -- .../src/main/js/serve/routes/admin.js | 126 --- .../src/main/js/serve/routes/agent.js | 81 -- .../src/main/js/serve/routes/caches.js | 132 --- .../src/main/js/serve/routes/clusters.js | 146 --- .../src/main/js/serve/routes/domains.js | 195 ---- .../src/main/js/serve/routes/igfs.js | 122 --- .../src/main/js/serve/routes/notebooks.js | 121 --- .../src/main/js/serve/routes/profile.js | 102 -- .../src/main/js/serve/routes/routes.js | 103 -- .../main/js/views/configuration/clusters.jade | 64 -- .../web-console/src/test/js/routes/agent.js | 94 -- .../{ => web-console}/web-agent/.gitignore | 0 .../{ => web-console}/web-agent/README.txt | 0 .../web-agent/assembly/release-web-agent.xml | 8 +- .../web-agent/bin/ignite-web-agent.bat | 0 .../web-agent/bin/ignite-web-agent.sh | 0 .../web-agent/demo/README.txt | 0 .../web-agent/demo/db-init.sql | 0 .../web-agent/jdbc-drivers/README.txt | 0 .../web-agent/logs/README.txt | 0 modules/{ => web-console}/web-agent/pom.xml | 12 +- .../console/agent/AgentConfiguration.java | 0 .../ignite/console/agent/AgentLauncher.java | 0 .../ignite/console/agent/AgentUtils.java | 0 .../agent/handlers/AbstractHandler.java | 0 .../agent/handlers/DatabaseHandler.java | 0 .../console/agent/handlers/RestHandler.java | 0 .../ignite/console/demo/AgentClusterDemo.java | 3 + .../console/demo/AgentMetadataDemo.java | 0 .../apache/ignite/console/demo/model/Car.java | 0 .../ignite/console/demo/model/Country.java | 0 .../ignite/console/demo/model/Department.java | 0 .../ignite/console/demo/model/Employee.java | 0 .../ignite/console/demo/model/Parking.java | 0 .../src/main/resources/log4j.properties | 0 pom.xml | 2 +- 528 files changed, 11276 insertions(+), 7933 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/event/VisorGridDiscoveryEventV2.java create mode 100644 modules/web-console/.gitignore create mode 100644 modules/web-console/backend/.babelrc create mode 100644 modules/web-console/backend/.eslintrc create mode 100644 modules/web-console/backend/.gitignore rename modules/web-console/{src/main/js/serve => backend}/agent_dists/README.txt (100%) rename modules/web-console/{src/main/js/serve => backend/app}/agent.js (87%) rename modules/web-console/{src/main/js/serve => backend/app}/app.js (69%) rename modules/web-console/{src/main/js/serve => backend/app}/browser.js (87%) rename modules/web-console/{src/main/js/serve => backend/app}/configure.js (90%) rename modules/web-console/{src/main/js/serve.js => backend/app/index.js} (79%) rename modules/web-console/{src/main/js/serve => backend/app}/mongo.js (92%) create mode 100644 modules/web-console/backend/app/nconf.js create mode 100644 modules/web-console/backend/app/routes.js rename modules/web-console/{src/main/js/serve => backend/app}/settings.js (69%) rename modules/web-console/{src/main/js/serve => backend}/config/settings.json.sample (60%) create mode 100644 modules/web-console/backend/errors/AppErrorException.js create mode 100644 modules/web-console/backend/errors/AuthFailedException.js rename modules/web-console/{src/main/js/app/modules/states/configuration/caches/concurrency.directive.js => backend/errors/DuplicateKeyException.js} (78%) rename modules/web-console/{src/main/js/app/modules/states/configuration/clusters/connector.directive.js => backend/errors/IllegalAccessError.js} (76%) create mode 100644 modules/web-console/backend/errors/IllegalArgumentException.js create mode 100644 modules/web-console/backend/errors/MissingResourceException.js create mode 100644 modules/web-console/backend/errors/ServerErrorException.js create mode 100644 modules/web-console/backend/errors/index.js rename modules/web-console/{src/main/js/app/modules/form/field/field.css => backend/index.js} (89%) rename modules/web-console/{src/main/js/app/modules/form/group/table.directive.js => backend/injector.js} (73%) rename modules/web-console/{src/main/js/app/modules/form/field/field.directive.js => backend/middlewares/api.js} (56%) create mode 100644 modules/web-console/backend/middlewares/host.js create mode 100644 modules/web-console/backend/middlewares/user.js create mode 100644 modules/web-console/backend/package.json create mode 100644 modules/web-console/backend/routes/admin.js create mode 100644 modules/web-console/backend/routes/agent.js create mode 100644 modules/web-console/backend/routes/caches.js create mode 100644 modules/web-console/backend/routes/clusters.js create mode 100644 modules/web-console/backend/routes/configuration.js rename modules/web-console/{src/main/js/serve => backend}/routes/demo.js (88%) rename modules/web-console/{src/main/js/serve => backend}/routes/demo/caches.json (100%) rename modules/web-console/{src/main/js/serve => backend}/routes/demo/clusters.json (100%) rename modules/web-console/{src/main/js/serve => backend}/routes/demo/domains.json (100%) rename modules/web-console/{src/main/js/serve => backend}/routes/demo/igfss.json (100%) create mode 100644 modules/web-console/backend/routes/domains.js create mode 100644 modules/web-console/backend/routes/igfss.js create mode 100644 modules/web-console/backend/routes/notebooks.js create mode 100644 modules/web-console/backend/routes/profile.js rename modules/web-console/{src/main/js/serve => backend}/routes/public.js (50%) create mode 100644 modules/web-console/backend/services/agents.js create mode 100644 modules/web-console/backend/services/auth.js create mode 100644 modules/web-console/backend/services/caches.js create mode 100644 modules/web-console/backend/services/clusters.js create mode 100644 modules/web-console/backend/services/configurations.js create mode 100644 modules/web-console/backend/services/domains.js create mode 100644 modules/web-console/backend/services/igfss.js create mode 100644 modules/web-console/backend/services/mails.js create mode 100644 modules/web-console/backend/services/notebooks.js create mode 100644 modules/web-console/backend/services/sessions.js create mode 100644 modules/web-console/backend/services/spaces.js create mode 100644 modules/web-console/backend/services/users.js create mode 100644 modules/web-console/backend/test/config/settings.json create mode 100644 modules/web-console/backend/test/data/accounts.json create mode 100644 modules/web-console/backend/test/data/caches.json create mode 100644 modules/web-console/backend/test/data/clusters.json create mode 100644 modules/web-console/backend/test/data/domains.json create mode 100644 modules/web-console/backend/test/data/igfss.json rename modules/web-console/{src/main/js/app/modules/states/configuration/caches/general.directive.js => backend/test/injector.js} (70%) create mode 100644 modules/web-console/backend/test/unit/CacheService.test.js create mode 100644 modules/web-console/backend/test/unit/ClusterService.test.js create mode 100644 modules/web-console/backend/test/unit/DomainService.test.js create mode 100644 modules/web-console/backend/test/unit/IgfsService.test.js create mode 100644 modules/web-console/docker/compose/backend/.dockerignore create mode 100644 modules/web-console/docker/compose/backend/Dockerfile create mode 100644 modules/web-console/docker/compose/backend/build.sh create mode 100644 modules/web-console/docker/compose/docker-compose.yml create mode 100644 modules/web-console/docker/compose/frontend/.dockerignore create mode 100644 modules/web-console/docker/compose/frontend/Dockerfile create mode 100644 modules/web-console/docker/compose/frontend/DockerfileBuild create mode 100644 modules/web-console/docker/compose/frontend/build.sh create mode 100644 modules/web-console/docker/compose/frontend/nginx/nginx.conf create mode 100644 modules/web-console/docker/compose/frontend/nginx/web-console.conf create mode 100644 modules/web-console/docker/standalone/.dockerignore create mode 100644 modules/web-console/docker/standalone/Dockerfile create mode 100644 modules/web-console/docker/standalone/build.sh create mode 100644 modules/web-console/docker/standalone/docker-compose.yml create mode 100644 modules/web-console/docker/standalone/entrypoint.sh create mode 100644 modules/web-console/docker/standalone/nginx/nginx.conf create mode 100644 modules/web-console/docker/standalone/nginx/web-console.conf create mode 100644 modules/web-console/frontend/.babelrc rename modules/web-console/{src/main/js => frontend}/.eslintrc (99%) rename modules/web-console/{src/main/js => frontend}/.gitignore (66%) rename modules/web-console/{src/main/js => frontend}/app/app.config.js (100%) rename modules/web-console/{src/main/js => frontend}/app/app.js (91%) rename modules/web-console/{src/main/js => frontend}/app/controllers/auth.controller.js (100%) rename modules/web-console/{src/main/js => frontend}/app/controllers/reset-password.controller.js (93%) rename modules/web-console/{src/main/js => frontend}/app/data/colors.json (100%) rename modules/web-console/{src/main/js => frontend}/app/data/countries.json (100%) rename modules/web-console/{src/main/js => frontend}/app/data/demo-info.json (100%) rename modules/web-console/{src/main/js => frontend}/app/data/event-types.json (100%) rename modules/web-console/{src/main/js => frontend}/app/data/getting-started.json (100%) rename modules/web-console/{src/main/js => frontend}/app/data/java-classes.json (93%) rename modules/web-console/{src/main/js => frontend}/app/data/java-keywords.json (100%) rename modules/web-console/{src/main/js => frontend}/app/data/java-primitives.json (100%) rename modules/web-console/{src/main/js => frontend}/app/data/pom-dependencies.json (100%) rename modules/web-console/{src/main/js => frontend}/app/decorator/select.js (100%) rename modules/web-console/{src/main/js => frontend}/app/decorator/tooltip.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/auto-focus.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/bs-affix-update.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/centered/centered.css (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/centered/centered.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/copy-to-clipboard.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/hide-on-state-change/hide-on-state-change.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/information/information.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/information/information.jade (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/information/information.scss (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/match.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/on-click-focus.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/on-enter-focus-move.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/on-enter.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/on-escape.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-docker/ui-ace-docker.controller.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-docker/ui-ace-docker.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-docker/ui-ace-docker.jade (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-java/ui-ace-java.controller.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-java/ui-ace-java.directive.js (88%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-java/ui-ace-java.jade (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-pojos/ui-ace-pojos.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-pojos/ui-ace-pojos.jade (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-pom/ui-ace-pom.controller.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-pom/ui-ace-pom.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-pom/ui-ace-pom.jade (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-tabs.directive.js (97%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-xml/ui-ace-xml.controller.js (100%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-xml/ui-ace-xml.directive.js (88%) rename modules/web-console/{src/main/js => frontend}/app/directives/ui-ace-xml/ui-ace-xml.jade (100%) rename modules/web-console/{src/main/js => frontend}/app/filters/byName.filter.js (100%) rename modules/web-console/{src/main/js => frontend}/app/filters/domainsValidation.filter.js (100%) create mode 100644 modules/web-console/frontend/app/filters/duration.filter.js rename modules/web-console/{src/main/js => frontend}/app/filters/hasPojo.filter.js (100%) rename modules/web-console/{src/main/js/app/modules/states/configuration/clusters/general/discovery/jdbc.jade => frontend/app/helpers/jade/form.jade} (68%) create mode 100644 modules/web-console/frontend/app/helpers/jade/form/form-field-checkbox.jade create mode 100644 modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade rename modules/web-console/{src/main/js/app/modules/form/group/group.jade => frontend/app/helpers/jade/form/form-field-down.jade} (85%) create mode 100644 modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade create mode 100644 modules/web-console/frontend/app/helpers/jade/form/form-field-feedback.jade rename modules/web-console/{src/main/js/app/modules/form/field/input/checkbox.jade => frontend/app/helpers/jade/form/form-field-label.jade} (74%) create mode 100644 modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade create mode 100644 modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade rename modules/web-console/{src/main/js/app/modules/form/group/table.jade => frontend/app/helpers/jade/form/form-field-up.jade} (85%) create mode 100644 modules/web-console/frontend/app/helpers/jade/form/form-group.jade rename modules/web-console/{src/main/js => frontend}/app/helpers/jade/mixins.jade (60%) rename modules/web-console/{src/main/js => frontend}/app/modules/Demo/Demo.module.js (97%) rename modules/web-console/{src/main/js => frontend}/app/modules/ace.module.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/agent/agent.module.js (86%) rename modules/web-console/{src/main/js => frontend}/app/modules/branding/branding.module.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/branding/branding.provider.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/branding/features.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/branding/footer.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/branding/header-logo.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/branding/header-logo.jade (97%) rename modules/web-console/{src/main/js => frontend}/app/modules/branding/header-title.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/branding/powered-by-apache.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/branding/powered-by-apache.jade (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/branding/terms.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/configuration/EventGroups.provider.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/configuration/Sidebar.provider.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/configuration/configuration.module.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/configuration/generator/Docker.service.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/configuration/generator/Java.service.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/configuration/generator/Pom.service.js (87%) rename modules/web-console/{src/main/js => frontend}/app/modules/configuration/generator/Xml.service.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/configuration/sidebar.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/dialog/dialog-content.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/dialog/dialog-title.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/dialog/dialog.controller.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/dialog/dialog.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/dialog/dialog.factory.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/dialog/dialog.jade (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/dialog/dialog.module.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/field/bs-select-placeholder.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/field/down.directive.js (69%) create mode 100644 modules/web-console/frontend/app/modules/form/field/feedback.scss create mode 100644 modules/web-console/frontend/app/modules/form/field/field.scss rename modules/web-console/{src/main/js => frontend}/app/modules/form/field/form-control-feedback.directive.js (96%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/field/input/autofocus.directive.js (89%) rename modules/web-console/{src/main/js/app/modules/user/user.module.js => frontend/app/modules/form/field/input/select.scss} (81%) rename modules/web-console/{src/main/js/app/modules/form/field/input/text.css => frontend/app/modules/form/field/input/text.scss} (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/field/label.directive.js (93%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/field/tooltip.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/field/up.directive.js (69%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/form.module.js (81%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/group/add.directive.js (96%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/group/tooltip.directive.js (96%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/panel/chevron.directive.js (100%) create mode 100644 modules/web-console/frontend/app/modules/form/panel/field.directive.js rename modules/web-console/{src/main/js => frontend}/app/modules/form/panel/panel.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/panel/revert.directive.js (97%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/validator/ipaddress.directive.js (82%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/validator/java-built-in-class.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/validator/java-identifier.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/validator/java-keywords.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/validator/java-package-name.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/validator/java-package-specified.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/validator/property-unique.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/validator/property-value-specified.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/form/validator/unique.directive.js (100%) rename modules/web-console/{src/main/js/app/modules/states/configuration/clusters/collision.directive.js => frontend/app/modules/form/validator/uuid.directive.js} (61%) rename modules/web-console/{src/main/js => frontend}/app/modules/getting-started/GettingStarted.provider.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/loading/loading.css (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/loading/loading.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/loading/loading.jade (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/loading/loading.module.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/loading/loading.service.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/navbar/Navbar.provider.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/navbar/Userbar.provider.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/navbar/navbar.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/navbar/navbar.module.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/navbar/userbar.directive.js (75%) rename modules/web-console/{src/main/js => frontend}/app/modules/socket.module.js (100%) create mode 100644 modules/web-console/frontend/app/modules/sql/Notebook.data.js create mode 100644 modules/web-console/frontend/app/modules/sql/Notebook.service.js create mode 100644 modules/web-console/frontend/app/modules/sql/notebook.controller.js create mode 100644 modules/web-console/frontend/app/modules/sql/scan-filter-input.jade create mode 100644 modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js rename modules/web-console/{src/main/js/controllers/sql-controller.js => frontend/app/modules/sql/sql.controller.js} (78%) create mode 100644 modules/web-console/frontend/app/modules/sql/sql.module.js rename modules/web-console/{src/main/js => frontend}/app/modules/states/admin.state.js (89%) create mode 100644 modules/web-console/frontend/app/modules/states/configuration.state.js create mode 100644 modules/web-console/frontend/app/modules/states/configuration/Configuration.resource.js rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/caches/concurrency.jade (80%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/caches/general.jade (80%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/caches/memory.jade (69%) create mode 100644 modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/caches/query.jade (60%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/caches/rebalance.jade (87%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/caches/server-near-cache.jade (70%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/caches/statistics.jade (80%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/caches/store.jade (66%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/atomic.jade (88%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/attributes.jade (95%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/binary.jade (70%) create mode 100644 modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/collision.jade (75%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/collision/custom.jade (88%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/collision/fifo-queue.jade (87%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/collision/job-stealing.jade (89%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/collision/priority-queue.jade (79%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/communication.jade (60%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/connector.jade (76%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/deployment.jade (56%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/discovery.jade (58%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/events.jade (94%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/failover.jade (56%) create mode 100644 modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/general/discovery/cloud.jade (51%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/general/discovery/google.jade (76%) create mode 100644 modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.jade rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/general/discovery/multicast.jade (54%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/general/discovery/s3.jade (88%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/general/discovery/shared.jade (91%) create mode 100644 modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.jade rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade (70%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade (73%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade (88%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade (73%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade (87%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade (85%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade (87%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade (81%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/igfs.jade (92%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/logger.jade (85%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/logger/custom.jade (87%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/logger/log4j.jade (87%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/logger/log4j2.jade (87%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/marshaller.jade (77%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/metrics.jade (84%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/ssl.jade (51%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/swap.jade (78%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/thread.jade (85%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/time.jade (88%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/clusters/transactions.jade (67%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/domains/general.jade (85%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/domains/query.jade (97%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/domains/store.jade (95%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/igfs/dual.jade (83%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/igfs/fragmentizer.jade (74%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/igfs/general.jade (90%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/igfs/ipc.jade (70%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/igfs/misc.jade (78%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/igfs/secondary.jade (80%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/preview-panel.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/summary/summary-tabs.directive.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/configuration/summary/summary.controller.js (90%) create mode 100644 modules/web-console/frontend/app/modules/states/errors.state.js rename modules/web-console/{src/main/js => frontend}/app/modules/states/logout.state.js (85%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/password.state.js (100%) rename modules/web-console/{src/main/js => frontend}/app/modules/states/profile.state.js (89%) rename modules/web-console/{src/main/js/app/modules/states/sql.state.js => frontend/app/modules/states/signin.state.js} (63%) rename modules/web-console/{src/main/js/app/modules/states/configuration/summary/summary.resource.js => frontend/app/modules/user/AclRoute.provider.js} (52%) create mode 100644 modules/web-console/frontend/app/modules/user/Auth.service.js rename modules/web-console/{src/main/js => frontend}/app/modules/user/User.service.js (58%) rename modules/web-console/{src/main/js/app/modules/states/configuration/caches/memory.directive.js => frontend/app/modules/user/permissions.js} (77%) create mode 100644 modules/web-console/frontend/app/modules/user/user.module.js rename modules/web-console/{src/main/js/app/modules/Version => frontend/app/modules/version}/Version.provider.js (100%) rename modules/web-console/{src/main/js => frontend}/app/services/ChartColors.service.js (100%) rename modules/web-console/{src/main/js => frontend}/app/services/Clone.service.js (100%) rename modules/web-console/{src/main/js => frontend}/app/services/Confirm.service.js (83%) rename modules/web-console/{src/main/js => frontend}/app/services/ConfirmBatch.service.js (100%) rename modules/web-console/{src/main/js => frontend}/app/services/CopyToClipboard.service.js (100%) rename modules/web-console/{src/main/js => frontend}/app/services/Countries.service.js (100%) create mode 100644 modules/web-console/frontend/app/services/ErrorPopover.service.js rename modules/web-console/{src/main/js => frontend}/app/services/Focus.service.js (100%) create mode 100644 modules/web-console/frontend/app/services/FormUtils.service.js rename modules/web-console/{src/main/js => frontend}/app/services/InetAddress.service.js (100%) rename modules/web-console/{src/main/js => frontend}/app/services/JavaTypes.service.js (85%) create mode 100644 modules/web-console/frontend/app/services/LegacyTable.service.js create mode 100644 modules/web-console/frontend/app/services/LegacyUtils.service.js rename modules/web-console/{src/main/js => frontend}/app/services/Messages.service.js (100%) rename modules/web-console/{src/main/js => frontend}/app/services/ModelNormalizer.service.js (100%) rename modules/web-console/{src/main/js => frontend}/app/services/UnsavedChangesGuard.service.js (100%) rename modules/web-console/{src/main/js => frontend}/app/vendor.js (98%) rename modules/web-console/{src/main/js => frontend}/controllers/admin-controller.js (94%) rename modules/web-console/{src/main/js => frontend}/controllers/caches-controller.js (71%) rename modules/web-console/{src/main/js => frontend}/controllers/clusters-controller.js (74%) rename modules/web-console/{src/main/js => frontend}/controllers/domains-controller.js (88%) rename modules/web-console/{src/main/js => frontend}/controllers/igfs-controller.js (83%) rename modules/web-console/{src/main/js => frontend}/controllers/profile-controller.js (93%) rename modules/web-console/{src/main/js => frontend}/generator/generator-common.js (99%) rename modules/web-console/{src/main/js => frontend}/generator/generator-java.js (96%) rename modules/web-console/{src/main/js => frontend}/generator/generator-optional.js (100%) rename modules/web-console/{src/main/js => frontend}/generator/generator-properties.js (62%) rename modules/web-console/{src/main/js => frontend}/generator/generator-readme.js (100%) rename modules/web-console/{src/main/js => frontend}/generator/generator-xml.js (92%) rename modules/web-console/{src/main/js => frontend}/gulpfile.babel.js/index.js (100%) rename modules/web-console/{src/main/js => frontend}/gulpfile.babel.js/paths.js (85%) rename modules/web-console/{src/main/js => frontend}/gulpfile.babel.js/tasks/build.js (100%) rename modules/web-console/{src/main/js => frontend}/gulpfile.babel.js/tasks/bundle.js (100%) rename modules/web-console/{src/main/js => frontend}/gulpfile.babel.js/tasks/clean.js (100%) rename modules/web-console/{src/main/js => frontend}/gulpfile.babel.js/tasks/copy.js (100%) rename modules/web-console/{src/main/js => frontend}/gulpfile.babel.js/tasks/ignite-modules.js (100%) rename modules/web-console/{src/main/js => frontend}/gulpfile.babel.js/tasks/jade.js (100%) create mode 100644 modules/web-console/frontend/gulpfile.babel.js/tasks/test.js rename modules/web-console/{src/main/js => frontend}/gulpfile.babel.js/tasks/watch.js (100%) rename modules/web-console/{src/main/js => frontend}/gulpfile.babel.js/webpack/common.js (92%) rename modules/web-console/{src/main/js => frontend}/gulpfile.babel.js/webpack/environments/development.js (82%) rename modules/web-console/{src/main/js => frontend}/gulpfile.babel.js/webpack/environments/production.js (100%) rename modules/web-console/{src/main/js => frontend}/gulpfile.babel.js/webpack/index.js (100%) rename modules/web-console/{src/main/js => frontend}/gulpfile.babel.js/webpack/plugins/progress.js (100%) rename modules/web-console/{src/main/js => frontend}/ignite_modules/README.txt (100%) rename modules/web-console/{src/main/js => frontend}/ignite_modules/index.js (100%) rename modules/web-console/{src/main/js => frontend}/package.json (77%) rename modules/web-console/{src/main/js => frontend}/public/favicon.ico (100%) rename modules/web-console/{src/main/js => frontend}/public/images/cache.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/cluster.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/docker.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/domains.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/igfs.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/ignite-logo.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/ignite-logo@2x.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/ignite-puzzle.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/java.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/pb-ignite.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/pb-ignite@2x.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/query-chart.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/query-metadata.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/query-table.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/summary.png (100%) rename modules/web-console/{src/main/js => frontend}/public/images/xml.png (100%) rename modules/web-console/{src/main/js => frontend}/public/stylesheets/_bootstrap-custom.scss (100%) rename modules/web-console/{src/main/js => frontend}/public/stylesheets/_bootstrap-variables.scss (99%) rename modules/web-console/{src/main/js => frontend}/public/stylesheets/_font-awesome-custom.scss (94%) rename modules/web-console/{src/main/js/app/modules/states/configuration/clusters/igfs.directive.js => frontend/public/stylesheets/blocks/error.scss} (81%) rename modules/web-console/{src/main/js => frontend}/public/stylesheets/style.scss (94%) rename modules/web-console/{src/main/js => frontend}/public/stylesheets/variables.scss (100%) create mode 100644 modules/web-console/frontend/test/e2e/exampe.test.js create mode 100644 modules/web-console/frontend/test/karma.conf.js create mode 100644 modules/web-console/frontend/test/protractor.conf.js create mode 100644 modules/web-console/frontend/test/unit/JavaTypes.test.js rename modules/web-console/{src/main/js/app/modules/states/configuration/clusters/discovery.directive.js => frontend/test/unit/UserAuth.test.js} (67%) create mode 100644 modules/web-console/frontend/views/403.jade create mode 100644 modules/web-console/frontend/views/404.jade rename modules/web-console/{src/main/js => frontend}/views/base.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/configuration/caches.jade (68%) create mode 100644 modules/web-console/frontend/views/configuration/clusters.jade rename modules/web-console/{src/main/js => frontend}/views/configuration/domains-import.jade (87%) rename modules/web-console/{src/main/js => frontend}/views/configuration/domains.jade (91%) rename modules/web-console/{src/main/js => frontend}/views/configuration/igfs.jade (80%) rename modules/web-console/{src/main/js => frontend}/views/configuration/sidebar.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/configuration/summary-project-structure.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/configuration/summary-tabs.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/configuration/summary.jade (68%) rename modules/web-console/{src/main/js => frontend}/views/includes/footer.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/includes/header.jade (89%) rename modules/web-console/{src/main/js => frontend}/views/index.jade (99%) rename modules/web-console/{src/main/js => frontend}/views/reset.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/settings/admin.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/settings/profile.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/signin.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/sql/cache-metadata.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/sql/chart-settings.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/sql/notebook-new.jade (92%) rename modules/web-console/{src/main/js => frontend}/views/sql/paragraph-rate.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/sql/sql.jade (77%) rename modules/web-console/{src/main/js => frontend}/views/templates/agent-download.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/templates/alert.jade (96%) rename modules/web-console/{src/main/js => frontend}/views/templates/batch-confirm.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/templates/clone.jade (79%) rename modules/web-console/{src/main/js => frontend}/views/templates/confirm.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/templates/demo-info.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/templates/dropdown.jade (63%) rename modules/web-console/{src/main/js => frontend}/views/templates/getting-started.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/templates/message.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/templates/pagination.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/templates/select.jade (100%) rename modules/web-console/{src/main/js => frontend}/views/templates/validation-error.jade (100%) delete mode 100644 modules/web-console/src/main/js/.babelrc delete mode 100644 modules/web-console/src/main/js/app/controllers/notebooks.controller.js delete mode 100644 modules/web-console/src/main/js/app/modules/form/field/dropdown.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/form/field/dropdown.jade delete mode 100644 modules/web-console/src/main/js/app/modules/form/field/field.jade delete mode 100644 modules/web-console/src/main/js/app/modules/form/field/input/checkbox.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/form/field/input/datalist.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/form/field/input/datalist.jade delete mode 100644 modules/web-console/src/main/js/app/modules/form/field/input/number.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/form/field/input/number.jade delete mode 100644 modules/web-console/src/main/js/app/modules/form/field/input/text.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/form/field/input/text.jade delete mode 100644 modules/web-console/src/main/js/app/modules/form/group/group.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/query-notebooks/query-notebooks.module.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration.state.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/caches/query.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/caches/rebalance.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/caches/server-near-cache.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/caches/statistics.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/caches/store.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/atomic.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/attributes.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/binary.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/custom.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/fifo-queue.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/job-stealing.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/priority-queue.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/communication.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/deployment.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/events.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/failover.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general.jade delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/cloud.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/google.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/jdbc.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/multicast.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/s3.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/shared.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/vm.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/vm.jade delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/custom.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/log4j.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/log4j2.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/marshaller.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/metrics.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/ssl.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/swap.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/thread.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/time.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/clusters/transactions.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/domains/general.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/domains/query.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/domains/store.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/igfs/dual.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/igfs/fragmentizer.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/igfs/general.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/igfs/ipc.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/igfs/misc.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/configuration/igfs/secondary.directive.js delete mode 100644 modules/web-console/src/main/js/app/modules/states/signin.state.js delete mode 100644 modules/web-console/src/main/js/app/modules/user/Auth.service.js delete mode 100644 modules/web-console/src/main/js/app/services/LegacyTable.service.js delete mode 100644 modules/web-console/src/main/js/app/services/LegacyUtils.service.js delete mode 100644 modules/web-console/src/main/js/serve/mail.js delete mode 100644 modules/web-console/src/main/js/serve/routes/admin.js delete mode 100644 modules/web-console/src/main/js/serve/routes/agent.js delete mode 100644 modules/web-console/src/main/js/serve/routes/caches.js delete mode 100644 modules/web-console/src/main/js/serve/routes/clusters.js delete mode 100644 modules/web-console/src/main/js/serve/routes/domains.js delete mode 100644 modules/web-console/src/main/js/serve/routes/igfs.js delete mode 100644 modules/web-console/src/main/js/serve/routes/notebooks.js delete mode 100644 modules/web-console/src/main/js/serve/routes/profile.js delete mode 100644 modules/web-console/src/main/js/serve/routes/routes.js delete mode 100644 modules/web-console/src/main/js/views/configuration/clusters.jade delete mode 100644 modules/web-console/src/test/js/routes/agent.js rename modules/{ => web-console}/web-agent/.gitignore (100%) rename modules/{ => web-console}/web-agent/README.txt (100%) rename modules/{ => web-console}/web-agent/assembly/release-web-agent.xml (91%) rename modules/{ => web-console}/web-agent/bin/ignite-web-agent.bat (100%) rename modules/{ => web-console}/web-agent/bin/ignite-web-agent.sh (100%) rename modules/{ => web-console}/web-agent/demo/README.txt (100%) rename modules/{ => web-console}/web-agent/demo/db-init.sql (100%) rename modules/{ => web-console}/web-agent/jdbc-drivers/README.txt (100%) rename modules/{ => web-console}/web-agent/logs/README.txt (100%) rename modules/{ => web-console}/web-agent/pom.xml (94%) rename modules/{ => web-console}/web-agent/src/main/java/org/apache/ignite/console/agent/AgentConfiguration.java (100%) rename modules/{ => web-console}/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java (100%) rename modules/{ => web-console}/web-agent/src/main/java/org/apache/ignite/console/agent/AgentUtils.java (100%) rename modules/{ => web-console}/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/AbstractHandler.java (100%) rename modules/{ => web-console}/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/DatabaseHandler.java (100%) rename modules/{ => web-console}/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/RestHandler.java (100%) rename modules/{ => web-console}/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java (99%) rename modules/{ => web-console}/web-agent/src/main/java/org/apache/ignite/console/demo/AgentMetadataDemo.java (100%) rename modules/{ => web-console}/web-agent/src/main/java/org/apache/ignite/console/demo/model/Car.java (100%) rename modules/{ => web-console}/web-agent/src/main/java/org/apache/ignite/console/demo/model/Country.java (100%) rename modules/{ => web-console}/web-agent/src/main/java/org/apache/ignite/console/demo/model/Department.java (100%) rename modules/{ => web-console}/web-agent/src/main/java/org/apache/ignite/console/demo/model/Employee.java (100%) rename modules/{ => web-console}/web-agent/src/main/java/org/apache/ignite/console/demo/model/Parking.java (100%) rename modules/{ => web-console}/web-agent/src/main/resources/log4j.properties (100%) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/event/VisorGridDiscoveryEventV2.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/event/VisorGridDiscoveryEventV2.java new file mode 100644 index 0000000000000..b66aacf6281ab --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/event/VisorGridDiscoveryEventV2.java @@ -0,0 +1,80 @@ +/* + * 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.visor.event; + +import java.util.UUID; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +/** + * Lightweight counterpart for {@link org.apache.ignite.events.DiscoveryEvent}. + */ +public class VisorGridDiscoveryEventV2 extends VisorGridDiscoveryEvent { + /** */ + private static final long serialVersionUID = 0L; + + /** Topology version. */ + private final long topVer; + + /** + * Create event with given parameters. + * + * @param typeId Event type. + * @param id Event id. + * @param name Event name. + * @param nid Event node ID. + * @param ts Event timestamp. + * @param msg Event message. + * @param shortDisplay Shortened version of {@code toString()} result. + * @param evtNodeId Event node id. + * @param addr Event node address. + * @param isDaemon If event node is daemon on not. + * @param topVer Topology version. + */ + public VisorGridDiscoveryEventV2( + int typeId, + IgniteUuid id, + String name, + UUID nid, + long ts, + @Nullable String msg, + String shortDisplay, + UUID evtNodeId, + String addr, + boolean isDaemon, + long topVer + ) { + super(typeId, id, name, nid, ts, msg, shortDisplay, evtNodeId, addr, isDaemon); + + this.topVer = topVer; + } + + /** + * @return Topology version or {@code 0} if configured discovery SPI implementation + * does not support versioning. + **/ + public long topologyVersion() { + return topVer; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorGridDiscoveryEventV2.class, this); + } +} 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 611dcde8b5aff..abe1364a0b660 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 @@ -41,6 +41,8 @@ import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isIgfsCache; import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isSystemCache; import static org.apache.ignite.internal.visor.compute.VisorComputeMonitoringHolder.COMPUTE_MONITORING_HOLDER_KEY; +import static org.apache.ignite.internal.visor.util.VisorTaskUtils.EVT_MAPPER; +import static org.apache.ignite.internal.visor.util.VisorTaskUtils.EVT_MAPPER_V2; import static org.apache.ignite.internal.visor.util.VisorTaskUtils.VISOR_TASK_EVTS; import static org.apache.ignite.internal.visor.util.VisorTaskUtils.checkExplicitTaskMonitoring; import static org.apache.ignite.internal.visor.util.VisorTaskUtils.collectEvents; @@ -62,6 +64,9 @@ public class VisorNodeDataCollectorJob extends VisorJob { */ protected VisorGridEvent map(Event evt, int type, IgniteUuid id, String name, UUID nid, long ts, String msg, String shortDisplay) { - if (evt instanceof TaskEvent) { - TaskEvent te = (TaskEvent)evt; + if (evt instanceof TaskEvent) + return taskEvent((TaskEvent)evt, type, id, name, nid, ts, msg, shortDisplay); - return new VisorGridTaskEvent(type, id, name, nid, ts, msg, shortDisplay, - te.taskName(), te.taskClassName(), te.taskSessionId(), te.internal()); - } + if (evt instanceof JobEvent) + return jobEvent((JobEvent)evt, type, id, name, nid, ts, msg, shortDisplay); - if (evt instanceof JobEvent) { - JobEvent je = (JobEvent)evt; + if (evt instanceof DeploymentEvent) + return deploymentEvent((DeploymentEvent)evt, type, id, name, nid, ts, msg, shortDisplay); - return new VisorGridJobEvent(type, id, name, nid, ts, msg, shortDisplay, - je.taskName(), je.taskClassName(), je.taskSessionId(), je.jobId()); - } + if (evt instanceof DiscoveryEvent) + return discoveryEvent((DiscoveryEvent)evt, type, id, name, nid, ts, msg, shortDisplay); - if (evt instanceof DeploymentEvent) { - DeploymentEvent de = (DeploymentEvent)evt; - - return new VisorGridDeploymentEvent(type, id, name, nid, ts, msg, shortDisplay, de.alias()); - } + return null; + } - if (evt instanceof DiscoveryEvent) { - DiscoveryEvent de = (DiscoveryEvent)evt; + /** + * @param te Task event. + * @param type Event's type. + * @param id Event id. + * @param name Event name. + * @param nid Event node ID. + * @param ts Event timestamp. + * @param msg Event message. + * @param shortDisplay Shortened version of {@code toString()} result. + * @return Visor data transfer object for event. + */ + protected VisorGridEvent taskEvent(TaskEvent te, int type, IgniteUuid id, String name, UUID nid, long ts, + String msg, String shortDisplay) { + return new VisorGridTaskEvent(type, id, name, nid, ts, msg, shortDisplay, + te.taskName(), te.taskClassName(), te.taskSessionId(), te.internal()); + } - ClusterNode node = de.eventNode(); + /** + * @param je Job event. + * @param type Event's type. + * @param id Event id. + * @param name Event name. + * @param nid Event node ID. + * @param ts Event timestamp. + * @param msg Event message. + * @param shortDisplay Shortened version of {@code toString()} result. + * @return Visor data transfer object for event. + */ + protected VisorGridEvent jobEvent(JobEvent je, int type, IgniteUuid id, String name, UUID nid, long ts, + String msg, String shortDisplay) { + return new VisorGridJobEvent(type, id, name, nid, ts, msg, shortDisplay, je.taskName(), je.taskClassName(), + je.taskSessionId(), je.jobId()); + } - String addr = F.first(node.addresses()); + /** + * @param de Deployment event. + * @param type Event's type. + * @param id Event id. + * @param name Event name. + * @param nid Event node ID. + * @param ts Event timestamp. + * @param msg Event message. + * @param shortDisplay Shortened version of {@code toString()} result. + * @return Visor data transfer object for event. + */ + protected VisorGridEvent deploymentEvent(DeploymentEvent de, int type, IgniteUuid id, String name, UUID nid, + long ts, String msg, String shortDisplay) { + return new VisorGridDeploymentEvent(type, id, name, nid, ts, msg, shortDisplay, de.alias()); + } - return new VisorGridDiscoveryEvent(type, id, name, nid, ts, msg, shortDisplay, - node.id(), addr, node.isDaemon()); - } + /** + * @param de Discovery event. + * @param type Event's type. + * @param id Event id. + * @param name Event name. + * @param nid Event node ID. + * @param ts Event timestamp. + * @param msg Event message. + * @param shortDisplay Shortened version of {@code toString()} result. + * @return Visor data transfer object for event. + */ + protected VisorGridEvent discoveryEvent(DiscoveryEvent de, int type, IgniteUuid id, String name, UUID nid, + long ts, String msg, String shortDisplay) { + ClusterNode node = de.eventNode(); - return null; + return new VisorGridDiscoveryEvent(type, id, name, nid, ts, msg, shortDisplay, node.id(), + F.first(node.addresses()), node.isDaemon()); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java index 2721be4ee32ca..25aaab50a92ab 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java @@ -41,6 +41,7 @@ import java.util.List; import java.util.Map; import java.util.SortedMap; +import java.util.UUID; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.zip.ZipEntry; @@ -54,18 +55,21 @@ import org.apache.ignite.cache.eviction.lru.LruEvictionPolicyMBean; import org.apache.ignite.cache.eviction.random.RandomEvictionPolicyMBean; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.Event; import org.apache.ignite.internal.processors.igfs.IgfsEx; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.event.VisorGridDiscoveryEventV2; import org.apache.ignite.internal.visor.event.VisorGridEvent; import org.apache.ignite.internal.visor.event.VisorGridEventsLost; import org.apache.ignite.internal.visor.file.VisorFileBlock; import org.apache.ignite.internal.visor.log.VisorLogFile; import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.lang.IgniteUuid; import org.jetbrains.annotations.Nullable; import static java.lang.System.getProperty; @@ -382,6 +386,17 @@ public static boolean checkExplicitTaskMonitoring(Ignite ignite) { /** Mapper from grid event to Visor data transfer object. */ public static final VisorEventMapper EVT_MAPPER = new VisorEventMapper(); + /** Mapper from grid event to Visor data transfer object. */ + public static final VisorEventMapper EVT_MAPPER_V2 = new VisorEventMapper() { + @Override protected VisorGridEvent discoveryEvent(DiscoveryEvent de, int type, IgniteUuid id, String name, + UUID nid, long ts, String msg, String shortDisplay) { + ClusterNode node = de.eventNode(); + + return new VisorGridDiscoveryEventV2(type, id, name, nid, ts, msg, shortDisplay, node.id(), + F.first(node.addresses()), node.isDaemon(), de.topologyVersion()); + } + }; + /** * Grabs local events and detects if events was lost since last poll. * @@ -389,17 +404,18 @@ public static boolean checkExplicitTaskMonitoring(Ignite ignite) { * @param evtOrderKey Unique key to take last order key from node local map. * @param evtThrottleCntrKey Unique key to take throttle count from node local map. * @param all If {@code true} then collect all events otherwise collect only non task events. + * @param evtMapper Closure to map grid events to Visor data transfer objects. * @return Collections of node events */ public static Collection collectEvents(Ignite ignite, String evtOrderKey, String evtThrottleCntrKey, - final boolean all) { + boolean all, IgniteClosure evtMapper) { int[] evtTypes = all ? VISOR_ALL_EVTS : VISOR_NON_TASK_EVTS; // Collect discovery events for Web Console. if (evtOrderKey.startsWith("CONSOLE_")) evtTypes = concat(evtTypes, EVTS_DISCOVERY); - return collectEvents(ignite, evtOrderKey, evtThrottleCntrKey, evtTypes, EVT_MAPPER); + return collectEvents(ignite, evtOrderKey, evtThrottleCntrKey, evtTypes, evtMapper); } /** @@ -413,7 +429,7 @@ public static Collection collectEvents(Ignite ignite, String evt * @return Collections of node events */ public static Collection collectEvents(Ignite ignite, String evtOrderKey, String evtThrottleCntrKey, - final int[] evtTypes, IgniteClosure evtMapper) { + int[] evtTypes, IgniteClosure evtMapper) { assert ignite != null; assert evtTypes != null && evtTypes.length > 0; diff --git a/modules/web-console/.gitignore b/modules/web-console/.gitignore new file mode 100644 index 0000000000000..9ddddc4776d8c --- /dev/null +++ b/modules/web-console/.gitignore @@ -0,0 +1,6 @@ +docker/standalone/backend/build +docker/standalone/frontend/build +docker/standalone/data +docker/compose/backend/build +docker/compose/frontend/build +docker/dev/data diff --git a/modules/web-console/DEVNOTES.txt b/modules/web-console/DEVNOTES.txt index 3732b78c30be7..27211aa36792b 100644 --- a/modules/web-console/DEVNOTES.txt +++ b/modules/web-console/DEVNOTES.txt @@ -13,22 +13,26 @@ How to deploy locally: npm install -g npm-windows-upgrade npm-windows-upgrade See: https://github.com/felixrieseberg/npm-windows-upgrade - Check npm version: "npm --version". -5. Run "npm install --no-optional" in terminal for download dependencies. -6. Build ignite-web-agent module follow instructions from 'modules/web-agent/README.txt'. -7. Copy ignite-web-agent-.zip from target of ignite-web-agent module to 'modules/web-console/src/main/js/serve/agent_dists' folder. + Check npm version: "npm --version", it should be 3.x. +5. Change directory to '$IGNITE_HOME/modules/web-console/backend' and + run "npm install --no-optional" for download backend dependencies. +6. Change directory to '$IGNITE_HOME/modules/web-console/frontend' and + run "npm install --no-optional" for download frontend dependencies. +7. Build ignite-web-agent module follow instructions from 'modules/web-agent/README.txt'. +8. Copy ignite-web-agent-.zip from '$IGNITE_HOME/modules/web-console/web-agent/target' + to '$IGNITE_HOME/modules/web-console/backend/agent_dists' folder. -Steps 1 - 7 should be executed once. +Steps 1 - 8 should be executed once. How to run console in development mode: 1. Configure MongoDB to run as service or in terminal change dir to $MONGO_INSTALL_DIR/server/3.0/bin and start MongoDB by executing "mongod". -2. In new terminal change directory to '$IGNITE_HOME/modules/web-console/src/main/js'. - If needed run "npm install --no-optional" (if dependencies changed) and run "node serve" to start backend. +2. In new terminal change directory to '$IGNITE_HOME/modules/web-console/backend'. + If needed run "npm install --no-optional" (if dependencies changed) and run "npm start" to start backend. -3. In new terminal change directory to '$IGNITE_HOME/modules/web-console/src/main/js' - and start webpack in development mode "npm run dev" . +3. In new terminal change directory to '$IGNITE_HOME/modules/web-console/frontend'. + If needed run "npm install --no-optional" (if dependencies changed) and start webpack in development mode "npm run dev". 4. In browser open: http://localhost:9000 diff --git a/modules/web-console/backend/.babelrc b/modules/web-console/backend/.babelrc new file mode 100644 index 0000000000000..7eb36f493578a --- /dev/null +++ b/modules/web-console/backend/.babelrc @@ -0,0 +1,9 @@ +{ + "presets": ["es2015", "stage-1"], + "plugins": [[ + "transform-builtin-extend", { + "globals": ["Error", "Array"], + "approximate": true + } + ]] +} diff --git a/modules/web-console/backend/.eslintrc b/modules/web-console/backend/.eslintrc new file mode 100644 index 0000000000000..c0c772b421e0c --- /dev/null +++ b/modules/web-console/backend/.eslintrc @@ -0,0 +1,188 @@ +parser: "babel-eslint" + +env: + es6: true + node: true + mocha: true + +ecmaFeatures: + arrowFunctions: true + blockBindings: true + classes: true + defaultParams: true + destructuring: true + module: true + objectLiteralComputedProperties: true + objectLiteralShorthandMethods: true + objectLiteralShorthandProperties: true + spread: true + templateStrings: true + experimentalObjectRestSpread: true + +globals: + _: true + io: true + +rules: + arrow-parens: [1, "always"] + arrow-spacing: [1, { "before": true, "after": true }] + accessor-pairs: 2 + block-scoped-var: 2 + brace-style: [0, "1tbs"] + comma-dangle: [2, "never"] + comma-spacing: [2, {"before": false, "after": true}] + comma-style: [2, "last"] + complexity: [1, 40] + computed-property-spacing: [2, "never"] + consistent-return: 0 + consistent-this: [0, "that"] + constructor-super: 2 + curly: [2, "multi-or-nest"] + default-case: 2 + dot-location: 0 + dot-notation: [2, { "allowKeywords": true }] + eol-last: 2 + eqeqeq: 2 + func-names: 0 + func-style: [0, "declaration"] + generator-star-spacing: 0 + guard-for-in: 1 + handle-callback-err: 0 + id-length: [2, {"min": 1, "max": 60}] + indent: [2, 4, {"SwitchCase": 1}] + key-spacing: [2, { "beforeColon": false, "afterColon": true }] + lines-around-comment: 0 + linebreak-style: [0, "unix"] + max-depth: [0, 4] + max-len: [0, 120, 4] + max-nested-callbacks: [1, 4] + max-params: [0, 3] + max-statements: [0, 10] + new-cap: 2 + new-parens: 2 + no-alert: 2 + no-array-constructor: 2 + no-bitwise: 0 + no-caller: 2 + no-catch-shadow: 2 + no-cond-assign: 2 + no-console: 0 + no-constant-condition: 2 + no-continue: 0 + no-class-assign: 2 + no-const-assign: 2 + no-control-regex: 2 + no-debugger: 2 + no-delete-var: 2 + no-div-regex: 0 + no-dupe-keys: 2 + no-dupe-args: 2 + no-duplicate-case: 2 + no-else-return: 2 + no-empty: 2 + no-empty-character-class: 2 + no-eq-null: 2 + no-eval: 2 + no-ex-assign: 2 + no-extend-native: 2 + no-extra-bind: 2 + no-extra-boolean-cast: 2 + no-extra-parens: 0 + no-extra-semi: 2 + no-fallthrough: 2 + no-floating-decimal: 1 + no-func-assign: 2 + no-implied-eval: 2 + no-inline-comments: 0 + no-inner-declarations: [2, "functions"] + no-invalid-regexp: 2 + no-irregular-whitespace: 2 + no-iterator: 2 + no-label-var: 2 + no-labels: 2 + no-lone-blocks: 2 + no-lonely-if: 2 + no-implicit-coercion: [2, {"boolean": false, "number": true, "string": true}] + no-loop-func: 2 + no-mixed-requires: [0, false] + no-mixed-spaces-and-tabs: [2, true] + no-multi-spaces: 2 + no-multi-str: 2 + no-multiple-empty-lines: [0, {"max": 2}] + no-native-reassign: 2 + no-negated-in-lhs: 2 + no-nested-ternary: 0 + no-new: 2 + no-new-func: 2 + no-new-object: 2 + no-new-require: 0 + no-new-wrappers: 2 + no-obj-calls: 2 + no-octal: 2 + no-octal-escape: 2 + no-param-reassign: 0 + no-path-concat: 0 + no-plusplus: 0 + no-process-env: 0 + no-process-exit: 1 + no-proto: 2 + no-redeclare: 2 + no-regex-spaces: 1 + no-restricted-modules: 0 + no-script-url: 0 + no-self-compare: 2 + no-sequences: 2 + no-shadow: 2 + no-shadow-restricted-names: 2 + no-spaced-func: 2 + no-sparse-arrays: 1 + no-sync: 0 + no-ternary: 0 + no-trailing-spaces: 2 + no-throw-literal: 0 + no-this-before-super: 2 + no-unexpected-multiline: 2 + no-undef: 2 + no-undef-init: 2 + no-undefined: 2 + no-unneeded-ternary: 2 + no-unreachable: 2 + no-unused-expressions: [2, { allowShortCircuit: true }] + no-unused-vars: [2, {"vars": "all", "args": "after-used"}] + no-use-before-define: 2 + no-useless-call: 2 + no-void: 0 + no-var: 2 + no-warning-comments: 0 + no-with: 2 + newline-after-var: 0 + object-shorthand: [2, "always"] + one-var: [2, "never"] + operator-assignment: [2, "always"] + operator-linebreak: 0 + padded-blocks: 0 + prefer-const: 1 + prefer-spread: 2 + quote-props: [2, "as-needed"] + quotes: [2, "single"] + radix: 1 + semi: [2, "always"] + semi-spacing: [2, {"before": false, "after": true}] + sort-vars: 0 + keyword-spacing: 2 + space-before-blocks: [2, "always"] + space-before-function-paren: [2, "never"] + space-in-parens: 0 + space-infix-ops: 2 + space-unary-ops: [2, { "words": true, "nonwords": false }] + spaced-comment: [1, "always"] + use-isnan: 2 + valid-jsdoc: 0 + valid-typeof: 2 + vars-on-top: 2 + wrap-iife: 0 + wrap-regex: 0 + yoda: [2, "never"] + +parserOptions: + sourceType: module diff --git a/modules/web-console/backend/.gitignore b/modules/web-console/backend/.gitignore new file mode 100644 index 0000000000000..f95e2bf0a9aa7 --- /dev/null +++ b/modules/web-console/backend/.gitignore @@ -0,0 +1,8 @@ +*.idea +*.log +.npmrc +node_modules +serve/config/*.json +serve/agent_dists/*.zip +agent_dists/*.zip +config/*.json diff --git a/modules/web-console/src/main/js/serve/agent_dists/README.txt b/modules/web-console/backend/agent_dists/README.txt similarity index 100% rename from modules/web-console/src/main/js/serve/agent_dists/README.txt rename to modules/web-console/backend/agent_dists/README.txt diff --git a/modules/web-console/src/main/js/serve/agent.js b/modules/web-console/backend/app/agent.js similarity index 87% rename from modules/web-console/src/main/js/serve/agent.js rename to modules/web-console/backend/app/agent.js index a529e94049455..a1858fddb1182 100644 --- a/modules/web-console/src/main/js/serve/agent.js +++ b/modules/web-console/backend/app/agent.js @@ -222,59 +222,64 @@ module.exports.factory = function(_, ws, fs, path, JSZip, socketio, settings, mo } /** - * * @param {Boolean} demo Is need run command on demo node. + * @param {String} nid Node id. * @param {String} cacheName Cache name. * @param {String} query Query. + * @param {Boolean} local Flag whether to execute query locally. * @param {int} pageSize Page size. * @returns {Promise} */ - fieldsQuery(demo, cacheName, query, pageSize) { - const cmd = new Command(demo, 'qryfldexe') - .addParam('cacheName', cacheName) - .addParam('qry', query) - .addParam('pageSize', pageSize); - - return this.executeRest(cmd); - } - - /** - * - * @param {Boolean} demo Is need run command on demo node. - * @param {String} cacheName Cache name. - * @param {int} pageSize Page size. - * @returns {Promise} - */ - scan(demo, cacheName, pageSize) { - const cmd = new Command(demo, 'qryscanexe') - .addParam('cacheName', cacheName) - .addParam('pageSize', pageSize); + fieldsQuery(demo, nid, cacheName, query, local, pageSize) { + const cmd = new Command(demo, 'exe') + .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask') + .addParam('p1', nid) + .addParam('p2', 'org.apache.ignite.internal.visor.query.VisorQueryTask') + .addParam('p3', 'org.apache.ignite.internal.visor.query.VisorQueryArg') + .addParam('p4', cacheName) + .addParam('p5', query) + .addParam('p6', local) + .addParam('p7', pageSize); return this.executeRest(cmd); } /** * @param {Boolean} demo Is need run command on demo node. + * @param {String} nid Node id. * @param {int} queryId Query Id. * @param {int} pageSize Page size. * @returns {Promise} */ - queryFetch(demo, queryId, pageSize) { - const cmd = new Command(demo, 'qryfetch') - .addParam('qryId', queryId) - .addParam('pageSize', pageSize); + queryFetch(demo, nid, queryId, pageSize) { + const cmd = new Command(demo, 'exe') + .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask') + .addParam('p1', nid) + .addParam('p2', 'org.apache.ignite.internal.visor.query.VisorQueryNextPageTask') + .addParam('p3', 'org.apache.ignite.lang.IgniteBiTuple') + .addParam('p4', 'java.lang.String') + .addParam('p5', 'java.lang.Integer') + .addParam('p6', queryId) + .addParam('p7', pageSize); return this.executeRest(cmd); } /** * @param {Boolean} demo Is need run command on demo node. + * @param {String} nid Node id. * @param {int} queryId Query Id. * @returns {Promise} */ - queryClose(demo, queryId) { - const cmd = new Command(demo, 'qrycls') - .addParam('qryId', queryId); + queryClose(demo, nid, queryId) { + const cmd = new Command(demo, 'exe') + .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask') + .addParam('p1', '') + .addParam('p2', 'org.apache.ignite.internal.visor.query.VisorQueryCleanupTask') + .addParam('p3', 'java.util.Map') + .addParam('p4', 'java.util.UUID') + .addParam('p5', 'java.util.Set') + .addParam('p6', `${nid}=${queryId}`); return this.executeRest(cmd); } @@ -312,6 +317,40 @@ module.exports.factory = function(_, ws, fs, path, JSZip, socketio, settings, mo return this.executeRest(cmd); } + /** + * @param {Boolean} demo Is need run command on demo node. + * @param {String} nid Node id. + * @returns {Promise} + */ + collectNodeConfiguration(demo, nid) { + const cmd = new Command(demo, 'exe') + .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask') + .addParam('p1', nid) + .addParam('p2', 'org.apache.ignite.internal.visor.node.VisorNodeConfigurationCollectorTask') + .addParam('p3', 'java.lang.Void') + .addParam('p4', null); + + return this.executeRest(cmd); + } + + /** + * @param {Boolean} demo Is need run command on demo node. + * @param {String} nid Node id. + * @param {Array.} caches Caches deployment IDs to collect configuration. + * @returns {Promise} + */ + collectCacheConfigurations(demo, nid, caches) { + const cmd = new Command(demo, 'exe') + .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask') + .addParam('p1', nid) + .addParam('p2', 'org.apache.ignite.internal.visor.cache.VisorCacheConfigurationCollectorTask') + .addParam('p3', 'java.util.Collection') + .addParam('p4', 'org.apache.ignite.lang.IgniteUuid') + .addParam('p5', caches); + + return this.executeRest(cmd); + } + /** * @param {Boolean} demo Is need run command on demo node. * @param {String} nid Node id. @@ -595,7 +634,7 @@ module.exports.factory = function(_, ws, fs, path, JSZip, socketio, settings, mo cb(); }) // TODO IGNITE-1379 send error to web master. - .catch((err) => cb('Agent is failed to authenticate. Please check agent\'s tokens')); + .catch(() => cb('Agent is failed to authenticate. Please check agent\'s tokens')); }); }); } diff --git a/modules/web-console/src/main/js/serve/app.js b/modules/web-console/backend/app/app.js similarity index 69% rename from modules/web-console/src/main/js/serve/app.js rename to modules/web-console/backend/app/app.js index 5d6b2cf6a056b..1bbfd2c23db36 100644 --- a/modules/web-console/src/main/js/serve/app.js +++ b/modules/web-console/backend/app/app.js @@ -36,6 +36,25 @@ module.exports.factory = function(Express, configure, routes) { routes.register(app); + // Catch 404 and forward to error handler. + app.use((req, res, next) => { + const err = new Error('Not Found: ' + req.originalUrl); + + err.status = 404; + + next(err); + }); + + // Production error handler: no stacktraces leaked to user. + app.use((err, req, res) => { + res.status(err.status || 500); + + res.render('error', { + message: err.message, + error: {} + }); + }); + srv.addListener('request', app); } }; diff --git a/modules/web-console/src/main/js/serve/browser.js b/modules/web-console/backend/app/browser.js similarity index 87% rename from modules/web-console/src/main/js/serve/browser.js rename to modules/web-console/backend/app/browser.js index 8a6d33e3f6e8e..3256b6a2c1875 100644 --- a/modules/web-console/src/main/js/serve/browser.js +++ b/modules/web-console/backend/app/browser.js @@ -90,53 +90,53 @@ module.exports.factory = (_, socketio, agentMgr, configure) => { }); // Close query on node. - socket.on('node:query:close', (queryId, cb) => { + socket.on('node:query:close', (nid, queryId, cb) => { agentMgr.findAgent(accountId()) - .then((agent) => agent.queryClose(demo, queryId)) + .then((agent) => agent.queryClose(demo, nid, queryId)) .then(() => cb()) .catch((err) => cb(_errorToJson(err))); }); // Execute query on node and return first page to browser. - socket.on('node:query', (cacheName, pageSize, query, cb) => { + socket.on('node:query', (nid, cacheName, query, local, pageSize, cb) => { agentMgr.findAgent(accountId()) - .then((agent) => { - if (query === null) - return agent.scan(demo, cacheName, pageSize); - - return agent.fieldsQuery(demo, cacheName, query, pageSize); - }) + .then((agent) => agent.fieldsQuery(demo, nid, cacheName, query, local, pageSize)) .then((res) => cb(null, res)) .catch((err) => cb(_errorToJson(err))); }); // Fetch next page for query and return result to browser. - socket.on('node:query:fetch', (queryId, pageSize, cb) => { + socket.on('node:query:fetch', (nid, queryId, pageSize, cb) => { agentMgr.findAgent(accountId()) - .then((agent) => agent.queryFetch(demo, queryId, pageSize)) + .then((agent) => agent.queryFetch(demo, nid, queryId, pageSize)) .then((res) => cb(null, res)) .catch((err) => cb(_errorToJson(err))); }); // Execute query on node and return full result to browser. - socket.on('node:query:getAll', (cacheName, query, cb) => { + socket.on('node:query:getAll', (nid, cacheName, query, local, cb) => { // Set page size for query. const pageSize = 1024; agentMgr.findAgent(accountId()) .then((agent) => { - const firstPage = query === null ? agent.scan(demo, cacheName, pageSize) - : agent.fieldsQuery(demo, cacheName, query, pageSize); + const firstPage = agent.fieldsQuery(demo, nid, cacheName, query, local, pageSize) + .then(({result}) => { + if (result.key) + return Promise.reject(result.key); + + return result.value; + }); const fetchResult = (acc) => { - if (acc.last) + if (!acc.hasMore) return acc; - return agent.queryFetch(demo, acc.queryId, pageSize) + return agent.queryFetch(demo, acc.responseNodeId, acc.queryId, pageSize) .then((res) => { - acc.items = acc.items.concat(res.items); + acc.rows = acc.rows.concat(res.rows); - acc.last = res.last; + acc.hasMore = res.hasMore; return fetchResult(acc); }); @@ -259,6 +259,32 @@ module.exports.factory = (_, socketio, agentMgr, configure) => { .catch((err) => cb(_errorToJson(err))); }); + // Gets node configuration for specified node. + socket.on('node:configuration', (nid, cb) => { + agentMgr.findAgent(accountId()) + .then((agent) => agent.collectNodeConfiguration(demo, nid)) + .then((data) => { + if (data.finished) + return cb(null, data.result); + + cb(_errorToJson(data.error)); + }) + .catch((err) => cb(_errorToJson(err))); + }); + + // Gets cache configurations for specified node and caches deployment IDs. + socket.on('cache:configuration', (nid, caches, cb) => { + agentMgr.findAgent(accountId()) + .then((agent) => agent.collectCacheConfigurations(demo, nid, caches)) + .then((data) => { + if (data.finished) + return cb(null, data.result); + + cb(_errorToJson(data.error)); + }) + .catch((err) => cb(_errorToJson(err))); + }); + // Swap backups specified caches on specified node and return result to browser. socket.on('node:cache:swap:backups', (nid, cacheNames, cb) => { agentMgr.findAgent(accountId()) diff --git a/modules/web-console/src/main/js/serve/configure.js b/modules/web-console/backend/app/configure.js similarity index 90% rename from modules/web-console/src/main/js/serve/configure.js rename to modules/web-console/backend/app/configure.js index 9671d66925342..7624bdd4babbe 100644 --- a/modules/web-console/src/main/js/serve/configure.js +++ b/modules/web-console/backend/app/configure.js @@ -24,11 +24,11 @@ */ module.exports = { implements: 'configure', - inject: ['require(morgan)', 'require(cookie-parser)', 'require(body-parser)', - 'require(express-session)', 'require(connect-mongo)', 'require(passport)', 'require(passport.socketio)', 'settings', 'mongo'] + inject: ['require(lodash)', 'require(morgan)', 'require(cookie-parser)', 'require(body-parser)', + 'require(express-session)', 'require(connect-mongo)', 'require(passport)', 'require(passport.socketio)', 'settings', 'mongo', 'middlewares:*'] }; -module.exports.factory = function(logger, cookieParser, bodyParser, session, connectMongo, passport, passportSocketIo, settings, mongo) { +module.exports.factory = function(_, logger, cookieParser, bodyParser, session, connectMongo, passport, passportSocketIo, settings, mongo, apis) { const _sessionStore = new (connectMongo(session))({mongooseConnection: mongo.connection}); return { @@ -37,6 +37,8 @@ module.exports.factory = function(logger, cookieParser, bodyParser, session, con skip: (req, res) => res.statusCode < 400 })); + _.forEach(apis, (api) => app.use(api)); + app.use(cookieParser(settings.sessionSecret)); app.use(bodyParser.json({limit: '50mb'})); diff --git a/modules/web-console/src/main/js/serve.js b/modules/web-console/backend/app/index.js similarity index 79% rename from modules/web-console/src/main/js/serve.js rename to modules/web-console/backend/app/index.js index 891855c071932..57963186b314b 100644 --- a/modules/web-console/src/main/js/serve.js +++ b/modules/web-console/backend/app/index.js @@ -17,9 +17,24 @@ 'use strict'; -const http = require('http'), - https = require('https'), - path = require('path'); +import fs from 'fs'; +import path from 'path'; +import http from 'http'; +import https from 'https'; + +const igniteModules = process.env.IGNITE_MODULES || './ignite_modules'; + +let injector; + +try { + const igniteModulesInjector = path.resolve(path.join(igniteModules, 'backend', 'injector.js')); + + fs.accessSync(igniteModulesInjector, fs.F_OK); + + injector = require(igniteModulesInjector); +} catch (ignore) { + injector = require(path.join(__dirname, '../injector')); +} /** * Event listener for HTTP server "error" event. @@ -28,7 +43,7 @@ const _onError = (port, error) => { if (error.syscall !== 'listen') throw error; - var bind = typeof port === 'string' ? 'Pipe ' + port : 'Port ' + port; + const bind = typeof port === 'string' ? 'Pipe ' + port : 'Port ' + port; // Handle specific listen errors with friendly messages. switch (error.code) { @@ -51,28 +66,13 @@ const _onError = (port, error) => { * Event listener for HTTP server "listening" event. */ const _onListening = (addr) => { - var bind = typeof addr === 'string' ? 'pipe ' + addr : 'port ' + addr.port; + const bind = typeof addr === 'string' ? 'pipe ' + addr : 'port ' + addr.port; console.log('Start listening on ' + bind); }; -const igniteModules = (process.env.IGNITE_MODULES && path.relative(__dirname, process.env.IGNITE_MODULES)) || './ignite_modules'; - -const fireUp = require('fire-up').newInjector({ - basePath: __dirname, - modules: [ - './serve/**/*.js', - `${igniteModules}/**/*.js` - ] -}); - -Promise.all([fireUp('settings'), fireUp('app'), fireUp('agent-manager'), fireUp('browser-manager')]) - .then((values) => { - const settings = values[0]; - const app = values[1]; - const agentMgr = values[2]; - const browserMgr = values[3]; - +Promise.all([injector('settings'), injector('app'), injector('agent-manager'), injector('browser-manager')]) + .then(([settings, app, agentMgr, browserMgr]) => { // Start rest server. const server = settings.server.SSLOptions ? https.createServer(settings.server.SSLOptions) : http.createServer(); diff --git a/modules/web-console/src/main/js/serve/mongo.js b/modules/web-console/backend/app/mongo.js similarity index 92% rename from modules/web-console/src/main/js/serve/mongo.js rename to modules/web-console/backend/app/mongo.js index 8fb0a20e397b5..7fe39f037bf73 100644 --- a/modules/web-console/src/main/js/serve/mongo.js +++ b/modules/web-console/backend/app/mongo.js @@ -24,17 +24,13 @@ */ module.exports = { implements: 'mongo', - inject: ['require(mongoose-deep-populate)', 'require(passport-local-mongoose)', 'settings', 'ignite_modules/mongo:*'] + inject: ['require(passport-local-mongoose)', 'settings', 'ignite_modules/mongo:*', 'require(mongoose)'] }; -module.exports.factory = function(deepPopulatePlugin, passportMongo, settings, pluginMongo) { - const mongoose = require('mongoose'); - +module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose) { // Use native promises mongoose.Promise = global.Promise; - const deepPopulate = deepPopulatePlugin(mongoose); - // Connect to mongoDB database. mongoose.connect(settings.mongoUrl, {server: {poolSize: 4}}); @@ -80,6 +76,10 @@ module.exports.factory = function(deepPopulatePlugin, passportMongo, settings, p } }); + result.errCodes = { + DUPLICATE_KEY_ERROR: 11000, + DUPLICATE_KEY_UPDATE_ERROR: 11001 + }; // Define Account model. result.Account = mongoose.model('Account', AccountSchema); @@ -103,7 +103,7 @@ module.exports.factory = function(deepPopulatePlugin, passportMongo, settings, p databaseSchema: String, databaseTable: String, keyType: String, - valueType: String, + valueType: {type: String}, keyFields: [{ databaseFieldName: String, databaseFieldType: String, @@ -126,18 +126,36 @@ module.exports.factory = function(deepPopulatePlugin, passportMongo, settings, p demo: Boolean }); + DomainModelSchema.index({valueType: 1, space: 1}, {unique: true}); + // Define model of Domain models. result.DomainModel = mongoose.model('DomainModel', DomainModelSchema); // Define Cache schema. const CacheSchema = new Schema({ space: {type: ObjectId, ref: 'Space', index: true}, - name: String, + name: {type: String}, clusters: [{type: ObjectId, ref: 'Cluster'}], domains: [{type: ObjectId, ref: 'DomainModel'}], cacheMode: {type: String, enum: ['PARTITIONED', 'REPLICATED', 'LOCAL']}, atomicityMode: {type: String, enum: ['ATOMIC', 'TRANSACTIONAL']}, + nodeFilter: { + kind: {type: String, enum: ['Default', 'Exclude', 'IGFS', 'OnNodes', 'Custom']}, + Exclude: { + nodeId: String + }, + IGFS: { + igfs: {type: ObjectId, ref: 'Igfs'} + }, + OnNodes: { + nodeIds: [String] + }, + Custom: { + className: String + } + }, + backups: Number, memoryMode: {type: String, enum: ['ONHEAP_TIERED', 'OFFHEAP_TIERED', 'OFFHEAP_VALUES']}, offHeapMaxMemory: Number, @@ -255,17 +273,14 @@ module.exports.factory = function(deepPopulatePlugin, passportMongo, settings, p demo: Boolean }); - // Install deep populate plugin. - CacheSchema.plugin(deepPopulate, { - whitelist: ['domains'] - }); + CacheSchema.index({name: 1, space: 1}, {unique: true}); // Define Cache model. result.Cache = mongoose.model('Cache', CacheSchema); const IgfsSchema = new Schema({ space: {type: ObjectId, ref: 'Space', index: true}, - name: String, + name: {type: String}, clusters: [{type: ObjectId, ref: 'Cluster'}], affinnityGroupSize: Number, blockSize: Number, @@ -285,7 +300,8 @@ module.exports.factory = function(deepPopulatePlugin, passportMongo, settings, p host: String, port: Number, memorySize: Number, - tokenDirectoryPath: String + tokenDirectoryPath: String, + threadCount: Number }, ipcEndpointEnabled: Boolean, maxSpaceSize: Number, @@ -307,13 +323,15 @@ module.exports.factory = function(deepPopulatePlugin, passportMongo, settings, p relaxedConsistency: Boolean }); + IgfsSchema.index({name: 1, space: 1}, {unique: true}); + // Define IGFS model. result.Igfs = mongoose.model('Igfs', IgfsSchema); // Define Cluster schema. const ClusterSchema = new Schema({ space: {type: ObjectId, ref: 'Space', index: true}, - name: String, + name: {type: String}, localHost: String, discovery: { localAddress: String, @@ -370,7 +388,12 @@ module.exports.factory = function(deepPopulatePlugin, passportMongo, settings, p addrReqAttempts: String }, Jdbc: { - initSchema: Boolean + initSchema: Boolean, + dataSourceBean: String, + dialect: { + type: String, + enum: ['Generic', 'Oracle', 'DB2', 'SQLServer', 'MySQL', 'PostgreSQL', 'H2'] + } }, SharedFs: { path: String @@ -596,23 +619,18 @@ module.exports.factory = function(deepPopulatePlugin, passportMongo, settings, p Custom: { class: String } - } + }, + cacheKeyConfiguration: [{ + typeName: String, + affinityKeyFieldName: String + }] }); - // Install deep populate plugin. - ClusterSchema.plugin(deepPopulate, { - whitelist: [ - 'caches', - 'caches.domains', - 'igfss' - ] - }); + ClusterSchema.index({name: 1, space: 1}, {unique: true}); // Define Cluster model. result.Cluster = mongoose.model('Cluster', ClusterSchema); - result.ClusterDefaultPopulate = ''; - // Define Notebook schema. const NotebookSchema = new Schema({ space: {type: ObjectId, ref: 'Space', index: true}, @@ -635,6 +653,8 @@ module.exports.factory = function(deepPopulatePlugin, passportMongo, settings, p }] }); + NotebookSchema.index({name: 1, space: 1}, {unique: true}); + // Define Notebook model. result.Notebook = mongoose.model('Notebook', NotebookSchema); @@ -643,33 +663,10 @@ module.exports.factory = function(deepPopulatePlugin, passportMongo, settings, p res.status(err.code || 500).send(err.message); }; - /** - * Query for user spaces. - * - * @param userId User ID. - * @param {Boolean} demo Is need use demo space. - * @returns {Promise} - */ - result.spaces = function(userId, demo) { - return result.Space.find({owner: userId, demo: !!demo}).lean().exec(); - }; - - /** - * Extract IDs from user spaces. - * - * @param userId User ID. - * @param {Boolean} demo Is need use demo space. - * @returns {Promise} - */ - result.spaceIds = function(userId, demo) { - return result.spaces(userId, demo) - .then((spaces) => spaces.map((space) => space._id)); - }; - // Registering the routes of all plugin modules for (const name in pluginMongo) { if (pluginMongo.hasOwnProperty(name)) - pluginMongo[name].register(mongoose, deepPopulate, result); + pluginMongo[name].register(mongoose, result); } return result; diff --git a/modules/web-console/backend/app/nconf.js b/modules/web-console/backend/app/nconf.js new file mode 100644 index 0000000000000..c585ac6cf410c --- /dev/null +++ b/modules/web-console/backend/app/nconf.js @@ -0,0 +1,48 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +/** + * Module with server-side configuration. + */ +module.exports = { + implements: 'nconf', + inject: ['require(nconf)', 'require(fs)'] +}; + +module.exports.factory = function(nconf, fs) { + const default_config = './config/settings.json'; + const file = process.env.SETTINGS || default_config; + + nconf.env({separator: '_'}); + + try { + fs.accessSync(file, fs.F_OK); + + nconf.file({file}); + } catch (ignore) { + nconf.file({file: default_config}); + } + + if (process.env.CONFIG_PATH && fs.existsSync(process.env.CONFIG_PATH)) + nconf.file({file: process.env.CONFIG_PATH}); + + return nconf; +}; diff --git a/modules/web-console/backend/app/routes.js b/modules/web-console/backend/app/routes.js new file mode 100644 index 0000000000000..6961173d03eb9 --- /dev/null +++ b/modules/web-console/backend/app/routes.js @@ -0,0 +1,64 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'routes', + inject: ['routes/public', 'routes/admin', 'routes/profiles', 'routes/demo', 'routes/clusters', 'routes/domains', + 'routes/caches', 'routes/igfss', 'routes/notebooks', 'routes/agents', 'routes/configurations'] +}; + +module.exports.factory = function(publicRoute, adminRoute, profilesRoute, demoRoute, + clustersRoute, domainsRoute, cachesRoute, igfssRoute, notebooksRoute, agentsRoute, configurationsRoute) { + return { + register: (app) => { + const _mustAuthenticated = (req, res, next) => { + if (req.isAuthenticated()) + return next(); + + res.status(401).send('Access denied. You are not authorized to access this page.'); + }; + + const _adminOnly = (req, res, next) => { + if (req.isAuthenticated() && req.user.admin) + return next(); + + res.status(401).send('Access denied. You are not authorized to access this page.'); + }; + + // Registering the standard routes + app.use('/', publicRoute); + app.use('/admin', _mustAuthenticated, _adminOnly, adminRoute); + app.use('/profile', _mustAuthenticated, profilesRoute); + app.use('/demo', _mustAuthenticated, demoRoute); + + app.all('/configuration/*', _mustAuthenticated); + + app.use('/configuration', configurationsRoute); + app.use('/configuration/clusters', clustersRoute); + app.use('/configuration/domains', domainsRoute); + app.use('/configuration/caches', cachesRoute); + app.use('/configuration/igfs', igfssRoute); + + app.use('/notebooks', _mustAuthenticated, notebooksRoute); + app.use('/agent', _mustAuthenticated, agentsRoute); + } + }; +}; diff --git a/modules/web-console/src/main/js/serve/settings.js b/modules/web-console/backend/app/settings.js similarity index 69% rename from modules/web-console/src/main/js/serve/settings.js rename to modules/web-console/backend/app/settings.js index 5b14bccc888ba..b3609e86c7e26 100644 --- a/modules/web-console/src/main/js/serve/settings.js +++ b/modules/web-console/backend/app/settings.js @@ -24,12 +24,10 @@ */ module.exports = { implements: 'settings', - inject: ['require(nconf)', 'require(fs)'] + inject: ['nconf', 'require(fs)'] }; module.exports.factory = function(nconf, fs) { - nconf.file({file: './serve/config/settings.json'}); - /** * Normalize a port into a number, string, or false. */ @@ -47,15 +45,17 @@ module.exports.factory = function(nconf, fs) { return false; }; + const mailConfig = nconf.get('mail') || {}; + return { agent: { - dists: 'serve/agent_dists', - port: _normalizePort(nconf.get('agent-server:port') || 3002), - legacyPort: _normalizePort(nconf.get('agent-server:legacyPort')), - SSLOptions: nconf.get('agent-server:ssl') && { - key: fs.readFileSync(nconf.get('agent-server:key')), - cert: fs.readFileSync(nconf.get('agent-server:cert')), - passphrase: nconf.get('agent-server:keyPassphrase') + dists: 'agent_dists', + port: _normalizePort(nconf.get('agentServer:port') || 3002), + legacyPort: _normalizePort(nconf.get('agentServer:legacyPort')), + SSLOptions: nconf.get('agentServer:ssl') && { + key: fs.readFileSync(nconf.get('agentServer:key')), + cert: fs.readFileSync(nconf.get('agentServer:cert')), + passphrase: nconf.get('agentServer:keyPassphrase') } }, server: { @@ -69,16 +69,12 @@ module.exports.factory = function(nconf, fs) { } }, smtp: { - service: nconf.get('smtp:service'), - username: nconf.get('smtp:username'), - sign: nconf.get('smtp:sign'), - email: nconf.get('smtp:email'), - password: nconf.get('smtp:password'), + ...mailConfig, address: (username, email) => username ? '"' + username + '" <' + email + '>' : email }, - mongoUrl: nconf.get('mongoDB:url') || 'mongodb://localhost/console', + mongoUrl: nconf.get('mongodb:url') || 'mongodb://localhost/console', cookieTTL: 3600000 * 24 * 30, - sessionSecret: 'keyboard cat', + sessionSecret: nconf.get('server:sessionSecret') || 'keyboard cat', tokenLength: 20 }; }; diff --git a/modules/web-console/src/main/js/serve/config/settings.json.sample b/modules/web-console/backend/config/settings.json.sample similarity index 60% rename from modules/web-console/src/main/js/serve/config/settings.json.sample rename to modules/web-console/backend/config/settings.json.sample index 94dd9f79cc51c..41f1152a411b7 100644 --- a/modules/web-console/src/main/js/serve/config/settings.json.sample +++ b/modules/web-console/backend/config/settings.json.sample @@ -1,26 +1,30 @@ { "server": { "port": 3000, + "sessionSecret": "CHANGE ME", "ssl": false, "key": "serve/keys/test.key", "cert": "serve/keys/test.crt", "keyPassphrase": "password" }, - "mongoDB": { + "mongodb": { "url": "mongodb://localhost/console" }, - "agent-server": { + "agentServer": { "port": 3001, "ssl": false, "key": "serve/keys/test.key", "cert": "serve/keys/test.crt", "keyPassphrase": "password" }, - "smtp": { + "mail": { "service": "", - "username": "Apache Ignite Web Console", "sign": "Kind regards,
          Apache Ignite Team", - "email": "", - "password": "" + "greeting": "Apache Ignite Web Console", + "from": "Apache Ignite Web Console ", + "auth": { + "user": "someusername@somecompany.tld", + "pass": "" + } } } diff --git a/modules/web-console/backend/errors/AppErrorException.js b/modules/web-console/backend/errors/AppErrorException.js new file mode 100644 index 0000000000000..208b09bdb0e25 --- /dev/null +++ b/modules/web-console/backend/errors/AppErrorException.js @@ -0,0 +1,36 @@ +/* + * 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. + */ + +'use strict'; + +class AppErrorException extends Error { + constructor(message) { + super(message); + + this.name = this.constructor.name; + this.code = 400; + this.httpCode = 400; + this.message = message; + + if (typeof Error.captureStackTrace === 'function') + Error.captureStackTrace(this, this.constructor); + else + this.stack = (new Error(message)).stack; + } +} + +module.exports = AppErrorException; diff --git a/modules/web-console/backend/errors/AuthFailedException.js b/modules/web-console/backend/errors/AuthFailedException.js new file mode 100644 index 0000000000000..3208e1d9e642e --- /dev/null +++ b/modules/web-console/backend/errors/AuthFailedException.js @@ -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. + */ + +'use strict'; + +import AppErrorException from './AppErrorException'; + +class AuthFailedException extends AppErrorException { + constructor(message) { + super(message); + + this.httpCode = 401; + } +} + +module.exports = AuthFailedException; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/concurrency.directive.js b/modules/web-console/backend/errors/DuplicateKeyException.js similarity index 78% rename from modules/web-console/src/main/js/app/modules/states/configuration/caches/concurrency.directive.js rename to modules/web-console/backend/errors/DuplicateKeyException.js index 9438679d762d5..b228d0c77ce4c 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/concurrency.directive.js +++ b/modules/web-console/backend/errors/DuplicateKeyException.js @@ -15,13 +15,14 @@ * limitations under the License. */ -import templateUrl from './concurrency.jade'; +'use strict'; -export default ['igniteConfigurationCachesConcurrency', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; +import AppErrorException from './AppErrorException'; + +class DuplicateKeyException extends AppErrorException { + constructor(message) { + super(message); + } +} + +module.exports = DuplicateKeyException; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/connector.directive.js b/modules/web-console/backend/errors/IllegalAccessError.js similarity index 76% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/connector.directive.js rename to modules/web-console/backend/errors/IllegalAccessError.js index ea0b04df2bad3..4fcd2d40c4e57 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/connector.directive.js +++ b/modules/web-console/backend/errors/IllegalAccessError.js @@ -15,13 +15,15 @@ * limitations under the License. */ -import templateUrl from './connector.jade'; +'use strict'; -export default ['igniteConfigurationClustersConnector', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; +import AppErrorException from './AppErrorException'; + +class IllegalAccessError extends AppErrorException { + constructor(message) { + super(message); + this.httpCode = 401; + } +} + +module.exports = IllegalAccessError; diff --git a/modules/web-console/backend/errors/IllegalArgumentException.js b/modules/web-console/backend/errors/IllegalArgumentException.js new file mode 100644 index 0000000000000..0487d05358984 --- /dev/null +++ b/modules/web-console/backend/errors/IllegalArgumentException.js @@ -0,0 +1,29 @@ +/* + * 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. + */ + +'use strict'; + +import AppErrorException from './AppErrorException'; + +class IllegalArgumentException extends AppErrorException { + constructor(message) { + super(message); + this.httpCode = 400; + } +} + +module.exports = IllegalArgumentException; diff --git a/modules/web-console/backend/errors/MissingResourceException.js b/modules/web-console/backend/errors/MissingResourceException.js new file mode 100644 index 0000000000000..799775b796561 --- /dev/null +++ b/modules/web-console/backend/errors/MissingResourceException.js @@ -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. + */ + +'use strict'; + +import AppErrorException from './AppErrorException'; + +class MissingResourceException extends AppErrorException { + constructor(message) { + super(message); + + this.httpCode = 404; + } +} + +module.exports = MissingResourceException; diff --git a/modules/web-console/backend/errors/ServerErrorException.js b/modules/web-console/backend/errors/ServerErrorException.js new file mode 100644 index 0000000000000..439755ef5e660 --- /dev/null +++ b/modules/web-console/backend/errors/ServerErrorException.js @@ -0,0 +1,36 @@ +/* + * 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. + */ + +'use strict'; + +class ServerErrorException extends Error { + constructor(message) { + super(message); + + this.name = this.constructor.name; + this.code = 500; + this.httpCode = 500; + this.message = message; + + if (typeof Error.captureStackTrace === 'function') + Error.captureStackTrace(this, this.constructor); + else + this.stack = (new Error(message)).stack; + } +} + +module.exports = ServerErrorException; diff --git a/modules/web-console/backend/errors/index.js b/modules/web-console/backend/errors/index.js new file mode 100644 index 0000000000000..0af5cd5dd6086 --- /dev/null +++ b/modules/web-console/backend/errors/index.js @@ -0,0 +1,39 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +import AppErrorException from './AppErrorException'; +import IllegalArgumentException from './IllegalArgumentException'; +import DuplicateKeyException from './DuplicateKeyException'; +import ServerErrorException from './ServerErrorException'; +import MissingResourceException from './MissingResourceException'; +import AuthFailedException from './AuthFailedException'; + +module.exports = { + implements: 'errors', + factory: () => ({ + AppErrorException, + IllegalArgumentException, + DuplicateKeyException, + ServerErrorException, + MissingResourceException, + AuthFailedException + }) +}; diff --git a/modules/web-console/src/main/js/app/modules/form/field/field.css b/modules/web-console/backend/index.js similarity index 89% rename from modules/web-console/src/main/js/app/modules/form/field/field.css rename to modules/web-console/backend/index.js index 3ea64f45e1ed4..dcb3f411c3ab4 100644 --- a/modules/web-console/src/main/js/app/modules/form/field/field.css +++ b/modules/web-console/backend/index.js @@ -15,9 +15,5 @@ * limitations under the License. */ -.indexField { - float: left; - line-height: 28px; - margin-right: 5px; - color: #ec1c24; -} +require('babel-core/register'); +require('./app/index.js'); diff --git a/modules/web-console/src/main/js/app/modules/form/group/table.directive.js b/modules/web-console/backend/injector.js similarity index 73% rename from modules/web-console/src/main/js/app/modules/form/group/table.directive.js rename to modules/web-console/backend/injector.js index 8c4d0ed4e7738..62f89808a427e 100644 --- a/modules/web-console/src/main/js/app/modules/form/group/table.directive.js +++ b/modules/web-console/backend/injector.js @@ -15,15 +15,16 @@ * limitations under the License. */ -import templateUrl from './table.jade'; +import fireUp from 'fire-up'; -export default ['igniteFormGroupTable', [() => { - return { - restrict: 'E', - scope: {}, - templateUrl, - replace: true, - transclude: true, - require: ['^form', '^igniteFormGroup'] - }; -}]]; +module.exports = fireUp.newInjector({ + basePath: __dirname, + modules: [ + './app/**/*.js', + './config/**/*.js', + './errors/**/*.js', + './middlewares/**/*.js', + './routes/**/*.js', + './services/**/*.js' + ] +}); diff --git a/modules/web-console/src/main/js/app/modules/form/field/field.directive.js b/modules/web-console/backend/middlewares/api.js similarity index 56% rename from modules/web-console/src/main/js/app/modules/form/field/field.directive.js rename to modules/web-console/backend/middlewares/api.js index 630f74f64053d..9c6395e7e9609 100644 --- a/modules/web-console/src/main/js/app/modules/form/field/field.directive.js +++ b/modules/web-console/backend/middlewares/api.js @@ -15,30 +15,30 @@ * limitations under the License. */ -import templateUrl from './field.jade'; -import './field.css'; +'use strict'; -export default ['igniteFormField', [() => { - const controller = [function() { - const ctrl = this; +// Fire me up! - ctrl.type = ctrl.type || 'external'; - }]; +module.exports = { + implements: 'middlewares:api', + factory: () => { + return (req, res, next) => { + res.api = { + error(err) { + // TODO: removed code from error + res.status(err.httpCode || err.code || 500).send(err.message); + }, + ok(data) { + res.status(200).json(data); + }, + serverError(err) { + err.httpCode = 500; - return { - restrict: 'E', - scope: {}, - bindToController: { - for: '@', - label: '@', - type: '@', - name: '@' - }, - templateUrl, - controller, - controllerAs: 'field', - replace: true, - transclude: true, - require: '^form' - }; -}]]; + res.api.error(err); + } + }; + + next(); + }; + } +}; diff --git a/modules/web-console/backend/middlewares/host.js b/modules/web-console/backend/middlewares/host.js new file mode 100644 index 0000000000000..5ddd918f5e64d --- /dev/null +++ b/modules/web-console/backend/middlewares/host.js @@ -0,0 +1,39 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'middlewares:host', + factory: () => { + return (req, res, next) => { + req.origin = function() { + if (req.headers.origin) + return req.headers.origin; + + if (req.headers['x-forwarded-server']) + return `${req.headers['x-forwarded-proto'] || 'http'}://${req.headers['x-forwarded-server']}`; + + return `${req.protocol}://${req.get('host')}`; + }; + + next(); + }; + } +}; diff --git a/modules/web-console/backend/middlewares/user.js b/modules/web-console/backend/middlewares/user.js new file mode 100644 index 0000000000000..8923211fb7f1d --- /dev/null +++ b/modules/web-console/backend/middlewares/user.js @@ -0,0 +1,36 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'middlewares:user', + factory: () => { + return (req, res, next) => { + req.currentUserId = function() { + if (req.session.viewedUser && req.user.admin) + return req.session.viewedUser._id; + + return req.user._id; + }; + + next(); + }; + } +}; diff --git a/modules/web-console/backend/package.json b/modules/web-console/backend/package.json new file mode 100644 index 0000000000000..598dc0030e10b --- /dev/null +++ b/modules/web-console/backend/package.json @@ -0,0 +1,71 @@ +{ + "name": "ignite-web-console", + "version": "1.0.0", + "description": "Interactive Web console for configuration, executing SQL queries and monitoring of Apache Ignite Cluster", + "private": true, + "scripts": { + "ci-test": "cross-env NODE_ENV=test CONFIG_PATH='./test/config/settings.json' mocha -u tdd --require babel-core/register --reporter mocha-teamcity-reporter --recursive ./test/unit", + "test": "cross-env NODE_ENV=test CONFIG_PATH='./test/config/settings.json' mocha -u tdd --require babel-core/register --recursive ./test/unit", + "eslint": "eslint --env node --format node_modules/eslint-friendly-formatter ./ -- --eff-by-issue", + "start": "node ./index.js" + }, + "author": "", + "contributors": [ + { + "name": "", + "email": "" + } + ], + "license": "Apache-2.0", + "keywords": "grid", + "homepage": "https://ignite.apache.org/", + "engines": { + "npm": "^3.x.x", + "node": "^4.x.x" + }, + "os": [ + "darwin", + "linux", + "win32" + ], + "dependencies": { + "body-parser": "^1.15.0", + "connect-mongo": "^1.1.0", + "cookie-parser": "~1.4.0", + "es6-promise": "^3.0.2", + "express": "^4.14.0", + "express-session": "^1.12.0", + "fire-up": "^1.0.0", + "glob": "^7.0.3", + "jszip": "^3.0.0", + "lodash": "^4.8.2", + "mongoose": "^4.4.11", + "morgan": "^1.7.0", + "nconf": "^0.8.2", + "nodemailer": "^2.3.0", + "passport": "^0.3.2", + "passport-local": "^1.0.0", + "passport-local-mongoose": "^4.0.0", + "passport.socketio": "^3.6.1", + "socket.io": "^1.4.5", + "ws": "^0.8.0" + }, + "devDependencies": { + "babel-core": "^6.7.6", + "babel-eslint": "^6.0.4", + "babel-plugin-add-module-exports": "^0.2.1", + "babel-plugin-transform-builtin-extend": "^1.1.0", + "babel-plugin-transform-runtime": "^6.7.5", + "babel-polyfill": "^6.7.4", + "babel-preset-es2015": "^6.9.0", + "babel-preset-stage-1": "^6.5.0", + "babel-runtime": "^6.6.1", + "chai": "^3.5.0", + "cross-env": "^1.0.7", + "eslint": "^2.9.0", + "eslint-friendly-formatter": "^2.0.5", + "jasmine-core": "^2.4.1", + "mocha": "~2.5.3", + "mocha-teamcity-reporter": "^1.0.0" + } +} diff --git a/modules/web-console/backend/routes/admin.js b/modules/web-console/backend/routes/admin.js new file mode 100644 index 0000000000000..70736d00ea43a --- /dev/null +++ b/modules/web-console/backend/routes/admin.js @@ -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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'routes/admin', + inject: ['require(lodash)', 'require(express)', 'settings', 'mongo', 'services/spaces', 'services/mails', 'services/sessions', 'services/users'] +}; + +/** + * @param _ + * @param express + * @param settings + * @param mongo + * @param spacesService + * @param {MailsService} mailsService + * @param {SessionsService} sessionsService + * @param {UsersService} usersService + * @returns {Promise} + */ +module.exports.factory = function(_, express, settings, mongo, spacesService, mailsService, sessionsService, usersService) { + return new Promise((factoryResolve) => { + const router = new express.Router(); + + /** + * Get list of user accounts. + */ + router.post('/list', (req, res) => { + usersService.list() + .then(res.api.ok) + .catch(res.api.error); + }); + + // Remove user. + router.post('/remove', (req, res) => { + usersService.remove(req.origin(), req.body.userId) + .then(res.api.ok) + .catch(res.api.error); + }); + + // Save user. + router.post('/save', (req, res) => { + const params = req.body; + + mongo.Account.findByIdAndUpdate(params.userId, {admin: params.adminFlag}).exec() + .then(res.api.ok) + .catch(res.api.error); + }); + + // Become user. + router.get('/become', (req, res) => { + sessionsService.become(req.session, req.query.viewedUserId) + .then(res.api.ok) + .catch(res.api.error); + }); + + // Revert to your identity. + router.get('/revert/identity', (req, res) => { + sessionsService.revert(req.session) + .then(res.api.ok) + .catch(res.api.error); + }); + + factoryResolve(router); + }); +}; + diff --git a/modules/web-console/backend/routes/agent.js b/modules/web-console/backend/routes/agent.js new file mode 100644 index 0000000000000..3f90fbd072fc8 --- /dev/null +++ b/modules/web-console/backend/routes/agent.js @@ -0,0 +1,53 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'routes/agents', + inject: ['require(lodash)', 'require(express)', 'services/agents'] +}; + +/** + * @param _ + * @param express + * @param {AgentsService} agentsService + * @returns {Promise} + */ +module.exports.factory = function(_, express, agentsService) { + return new Promise((resolveFactory) => { + const router = new express.Router(); + + /* Get grid topology. */ + router.get('/download/zip', (req, res) => { + const host = req.hostname.match(/:/g) ? req.hostname.slice(0, req.hostname.indexOf(':')) : req.hostname; + + agentsService.getArchive(host, req.user.token) + .then(({fileName, buffer}) => { + // Set the archive name. + res.attachment(fileName); + + res.send(buffer); + }) + .catch(res.api.error); + }); + + resolveFactory(router); + }); +}; diff --git a/modules/web-console/backend/routes/caches.js b/modules/web-console/backend/routes/caches.js new file mode 100644 index 0000000000000..e040fdab3578b --- /dev/null +++ b/modules/web-console/backend/routes/caches.js @@ -0,0 +1,65 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'routes/caches', + inject: ['require(lodash)', 'require(express)', 'mongo', 'services/caches'] +}; + +module.exports.factory = function(_, express, mongo, cachesService) { + return new Promise((factoryResolve) => { + const router = new express.Router(); + + /** + * Save cache. + */ + router.post('/save', (req, res) => { + const cache = req.body; + + cachesService.merge(cache) + .then((savedCache) => res.api.ok(savedCache._id)) + .catch(res.api.error); + }); + + /** + * Remove cache by ._id. + */ + router.post('/remove', (req, res) => { + const cacheId = req.body._id; + + cachesService.remove(cacheId) + .then(res.api.ok) + .catch(res.api.error); + }); + + /** + * Remove all caches. + */ + router.post('/remove/all', (req, res) => { + cachesService.removeAll(req.currentUserId(), req.header('IgniteDemoMode')) + .then(res.api.ok) + .catch(res.api.error); + }); + + factoryResolve(router); + }); +}; + diff --git a/modules/web-console/backend/routes/clusters.js b/modules/web-console/backend/routes/clusters.js new file mode 100644 index 0000000000000..97a446a0e264e --- /dev/null +++ b/modules/web-console/backend/routes/clusters.js @@ -0,0 +1,64 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'routes/clusters', + inject: ['require(lodash)', 'require(express)', 'mongo', 'services/clusters'] +}; + +module.exports.factory = function(_, express, mongo, clustersService) { + return new Promise((factoryResolve) => { + const router = new express.Router(); + + /** + * Save cluster. + */ + router.post('/save', (req, res) => { + const cluster = req.body; + + clustersService.merge(cluster) + .then((savedCluster) => res.api.ok(savedCluster._id)) + .catch(res.api.error); + }); + + /** + * Remove cluster by ._id. + */ + router.post('/remove', (req, res) => { + const clusterId = req.body._id; + + clustersService.remove(clusterId) + .then(res.api.ok) + .catch(res.api.error); + }); + + /** + * Remove all clusters. + */ + router.post('/remove/all', (req, res) => { + clustersService.removeAll(req.currentUserId(), req.header('IgniteDemoMode')) + .then(res.api.ok) + .catch(res.api.error); + }); + + factoryResolve(router); + }); +}; diff --git a/modules/web-console/backend/routes/configuration.js b/modules/web-console/backend/routes/configuration.js new file mode 100644 index 0000000000000..c3ff5d6a482b5 --- /dev/null +++ b/modules/web-console/backend/routes/configuration.js @@ -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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'routes/configurations', + inject: ['require(lodash)', 'require(express)', 'mongo', 'services/configurations'] +}; + +module.exports.factory = function(_, express, mongo, configurationsService) { + return new Promise((factoryResolve) => { + const router = new express.Router(); + /** + * Get all user configuration in current space. + */ + router.get('/list', (req, res) => { + configurationsService.list(req.currentUserId(), req.header('IgniteDemoMode')) + .then(res.api.ok) + .catch(res.api.error); + }); + + factoryResolve(router); + }); +}; diff --git a/modules/web-console/src/main/js/serve/routes/demo.js b/modules/web-console/backend/routes/demo.js similarity index 88% rename from modules/web-console/src/main/js/serve/routes/demo.js rename to modules/web-console/backend/routes/demo.js index dd47eb990131a..724b5c1ecff36 100644 --- a/modules/web-console/src/main/js/serve/routes/demo.js +++ b/modules/web-console/backend/routes/demo.js @@ -19,21 +19,17 @@ // Fire me up! +import clusters from './demo/clusters.json'; +import caches from './demo/caches.json'; +import domains from './demo/domains.json'; +import igfss from './demo/igfss.json'; + module.exports = { - implements: 'demo-routes', - inject: [ - 'require(lodash)', - 'require(express)', - 'settings', - 'mongo', - 'require(./demo/domains.json)', - 'require(./demo/caches.json)', - 'require(./demo/igfss.json)', - 'require(./demo/clusters.json)' - ] + implements: 'routes/demo', + inject: ['require(lodash)', 'require(express)', 'settings', 'mongo', 'services/spaces', 'errors'] }; -module.exports.factory = (_, express, settings, mongo, domains, caches, igfss, clusters) => { +module.exports.factory = (_, express, settings, mongo, spacesService, errors) => { return new Promise((factoryResolve) => { const router = new express.Router(); @@ -41,7 +37,7 @@ module.exports.factory = (_, express, settings, mongo, domains, caches, igfss, c * Reset demo configuration. */ router.post('/reset', (req, res) => { - mongo.spaces(req.user._id, true) + spacesService.spaces(req.user._id, true) .then((spaces) => { if (spaces.length) { const spaceIds = spaces.map((space) => space._id); @@ -53,8 +49,10 @@ module.exports.factory = (_, express, settings, mongo, domains, caches, igfss, c mongo.Igfs.remove({space: {$in: spaceIds}}).exec() ]).then(() => spaces[0]); } - - return new mongo.Space({name: 'Demo space', owner: req.user._id, demo: true}).save(); + }) + .catch((err) => { + if (err instanceof errors.MissingResourceException) + return spacesService.createDemoSpace(req.user._id); }) .then((space) => { return Promise.all(_.map(clusters, (cluster) => { diff --git a/modules/web-console/src/main/js/serve/routes/demo/caches.json b/modules/web-console/backend/routes/demo/caches.json similarity index 100% rename from modules/web-console/src/main/js/serve/routes/demo/caches.json rename to modules/web-console/backend/routes/demo/caches.json diff --git a/modules/web-console/src/main/js/serve/routes/demo/clusters.json b/modules/web-console/backend/routes/demo/clusters.json similarity index 100% rename from modules/web-console/src/main/js/serve/routes/demo/clusters.json rename to modules/web-console/backend/routes/demo/clusters.json diff --git a/modules/web-console/src/main/js/serve/routes/demo/domains.json b/modules/web-console/backend/routes/demo/domains.json similarity index 100% rename from modules/web-console/src/main/js/serve/routes/demo/domains.json rename to modules/web-console/backend/routes/demo/domains.json diff --git a/modules/web-console/src/main/js/serve/routes/demo/igfss.json b/modules/web-console/backend/routes/demo/igfss.json similarity index 100% rename from modules/web-console/src/main/js/serve/routes/demo/igfss.json rename to modules/web-console/backend/routes/demo/igfss.json diff --git a/modules/web-console/backend/routes/domains.js b/modules/web-console/backend/routes/domains.js new file mode 100644 index 0000000000000..db1d8928ab6f4 --- /dev/null +++ b/modules/web-console/backend/routes/domains.js @@ -0,0 +1,76 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'routes/domains', + inject: ['require(lodash)', 'require(express)', 'mongo', 'services/domains'] +}; + +module.exports.factory = (_, express, mongo, domainsService) => { + return new Promise((factoryResolve) => { + const router = new express.Router(); + + /** + * Save domain model. + */ + router.post('/save', (req, res) => { + const domain = req.body; + + domainsService.batchMerge([domain]) + .then(res.api.ok) + .catch(res.api.error); + }); + + /** + * Batch save domain models. + */ + router.post('/save/batch', (req, res) => { + const domains = req.body; + + domainsService.batchMerge(domains) + .then(res.api.ok) + .catch(res.api.error); + }); + + /** + * Remove domain model by ._id. + */ + router.post('/remove', (req, res) => { + const domainId = req.body._id; + + domainsService.remove(domainId) + .then(res.api.ok) + .catch(res.api.error); + }); + + /** + * Remove all domain models. + */ + router.post('/remove/all', (req, res) => { + domainsService.removeAll(req.currentUserId(), req.header('IgniteDemoMode')) + .then(res.api.ok) + .catch(res.api.error); + }); + + factoryResolve(router); + }); +}; + diff --git a/modules/web-console/backend/routes/igfss.js b/modules/web-console/backend/routes/igfss.js new file mode 100644 index 0000000000000..c88d6270a1803 --- /dev/null +++ b/modules/web-console/backend/routes/igfss.js @@ -0,0 +1,65 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'routes/igfss', + inject: ['require(lodash)', 'require(express)', 'mongo', 'services/igfss'] +}; + +module.exports.factory = function(_, express, mongo, igfssService) { + return new Promise((factoryResolve) => { + const router = new express.Router(); + + /** + * Save IGFS. + */ + router.post('/save', (req, res) => { + const igfs = req.body; + + igfssService.merge(igfs) + .then((savedIgfs) => res.api.ok(savedIgfs._id)) + .catch(res.api.error); + }); + + /** + * Remove IGFS by ._id. + */ + router.post('/remove', (req, res) => { + const igfsId = req.body._id; + + igfssService.remove(igfsId) + .then(res.api.ok) + .catch(res.api.error); + }); + + /** + * Remove all IGFSs. + */ + router.post('/remove/all', (req, res) => { + igfssService.removeAll(req.currentUserId(), req.header('IgniteDemoMode')) + .then(res.api.ok) + .catch(res.api.error); + }); + + factoryResolve(router); + }); +}; + diff --git a/modules/web-console/backend/routes/notebooks.js b/modules/web-console/backend/routes/notebooks.js new file mode 100644 index 0000000000000..c33080947997b --- /dev/null +++ b/modules/web-console/backend/routes/notebooks.js @@ -0,0 +1,80 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'routes/notebooks', + inject: ['require(lodash)', 'require(express)', 'mongo', 'services/spaces', 'services/notebooks'] +}; + +module.exports.factory = (_, express, mongo, spacesService, notebooksService) => { + return new Promise((factoryResolve) => { + const router = new express.Router(); + + /** + * Get notebooks names accessed for user account. + * + * @param req Request. + * @param res Response. + */ + router.get('/', (req, res) => { + return spacesService.spaces(req.currentUserId()) + .then((spaces) => _.map(spaces, (space) => space._id)) + .then((spaceIds) => notebooksService.listBySpaces(spaceIds)) + .then(res.api.ok) + .catch(res.api.error); + }); + + /** + * Save notebook accessed for user account. + * + * @param req Request. + * @param res Response. + */ + router.post('/save', (req, res) => { + const notebook = req.body; + + spacesService.spaceIds(req.currentUserId()) + .then((spaceIds) => { + notebook.space = notebook.space || spaceIds[0]; + + return notebooksService.merge(notebook); + }) + .then(res.api.ok) + .catch(res.api.error); + }); + + /** + * Remove notebook by ._id. + * + * @param req Request. + * @param res Response. + */ + router.post('/remove', (req, res) => { + const notebookId = req.body._id; + + notebooksService.remove(notebookId) + .then(res.api.ok) + .catch(res.api.error); + }); + + factoryResolve(router); + }); +}; diff --git a/modules/web-console/backend/routes/profile.js b/modules/web-console/backend/routes/profile.js new file mode 100644 index 0000000000000..4d01cda1e44ae --- /dev/null +++ b/modules/web-console/backend/routes/profile.js @@ -0,0 +1,73 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'routes/profiles', + inject: ['require(lodash)', 'require(express)', 'mongo', 'services/users'] +}; + +/** + * + * @param _ Lodash module + * @param express Express module + * @param mongo + * @param {UsersService} usersService + * @returns {Promise} + */ +module.exports.factory = function(_, express, mongo, usersService) { + return new Promise((resolveFactory) => { + const router = new express.Router(); + + /** + * Save user profile. + */ + router.post('/save', (req, res) => { + if (req.body.password && _.isEmpty(req.body.password)) + return res.status(500).send('Wrong value for new password!'); + + usersService.save(req.body) + .then((user) => { + const becomeUsed = req.session.viewedUser && user.admin; + + if (becomeUsed) { + req.session.viewedUser = user; + + return user; + } + + return new Promise((resolve, reject) => { + req.logout(); + + req.logIn(user, {}, (errLogIn) => { + if (errLogIn) + return reject(errLogIn); + + return resolve(user); + }); + }); + }) + .then(res.api.ok) + .catch(res.api.error); + }); + + resolveFactory(router); + }); +}; diff --git a/modules/web-console/src/main/js/serve/routes/public.js b/modules/web-console/backend/routes/public.js similarity index 50% rename from modules/web-console/src/main/js/serve/routes/public.js rename to modules/web-console/backend/routes/public.js index 207289a0821f4..5aad11a406ad0 100644 --- a/modules/web-console/src/main/js/serve/routes/public.js +++ b/modules/web-console/backend/routes/public.js @@ -20,11 +20,21 @@ // Fire me up! module.exports = { - implements: 'public-routes', - inject: ['require(express)', 'require(passport)', 'require(nodemailer)', 'settings', 'mail', 'mongo'] + implements: 'routes/public', + inject: ['require(express)', 'require(passport)', 'settings', 'mongo', 'services/mails', 'services/users'] }; -module.exports.factory = function(express, passport, nodemailer, settings, mail, mongo) { +/** + * + * @param express + * @param passport + * @param settings + * @param mongo + * @param mailsService + * @param {UsersService} usersService + * @returns {Promise} + */ +module.exports.factory = function(express, passport, settings, mongo, mailsService, usersService) { return new Promise((factoryResolve) => { const router = new express.Router(); @@ -42,90 +52,26 @@ module.exports.factory = function(express, passport, nodemailer, settings, mail, // GET user. router.post('/user', (req, res) => { - const becomeUsed = req.session.viewedUser && req.user.admin; - - let user = req.user; - - if (becomeUsed) { - user = req.session.viewedUser; - - user.becomeUsed = true; - } - else if (user) - user = user.toJSON(); - else - return res.json(user); - - mongo.Space.findOne({owner: user._id, demo: true}).exec() - .then((demoSpace) => { - if (user && demoSpace) - user.demoCreated = true; - - res.json(user); - }) - .catch((err) => { - res.status(401).send(err.message); - }); + usersService.get(req.user, req.session.viewedUser) + .then(res.api.ok) + .catch(res.api.error); }); /** * Register new account. */ router.post('/signup', (req, res) => { - mongo.Account.count().exec() - .then((cnt) => { - req.body.admin = cnt === 0; - - req.body.token = _randomString(); - - return new mongo.Account(req.body); - }) - .then((account) => { - return new Promise((resolve, reject) => { - mongo.Account.register(account, req.body.password, (err, _account) => { - if (err) - reject(err); + usersService.create(req.origin(), req.body) + .then((user) => new Promise((resolve, reject) => { + req.logIn(user, {}, (err) => { + if (err) + reject(err); - if (!_account) - reject(new Error('Failed to create account.')); - - resolve(_account); - }); + resolve(user); }); - }) - .then((account) => new mongo.Space({name: 'Personal space', owner: account._id}).save() - .then(() => account) - ) - .then((account) => { - return new Promise((resolve, reject) => { - req.logIn(account, {}, (err) => { - if (err) - reject(err); - - resolve(account); - }); - }); - }) - .then((account) => { - res.sendStatus(200); - - account.resetPasswordToken = _randomString(); - - return account.save() - .then(() => { - const resetLink = `http://${req.headers.host}/password/reset?token=${account.resetPasswordToken}`; - - mail.send(account, `Thanks for signing up for ${settings.smtp.username}.`, - `Hello ${account.firstName} ${account.lastName}!

          ` + - `You are receiving this email because you have signed up to use ${settings.smtp.username}.

          ` + - 'If you have not done the sign up and do not know what this email is about, please ignore it.
          ' + - 'You may reset the password by clicking on the following link, or paste this into your browser:

          ' + - `${resetLink}`); - }); - }) - .catch((err) => { - res.status(401).send(err.message); - }); + })) + .then(res.api.ok) + .catch(res.api.error); }); /** @@ -170,17 +116,7 @@ module.exports.factory = function(express, passport, nodemailer, settings, mail, return user.save(); }) - .then((user) => { - const resetLink = `http://${req.headers.host}/password/reset?token=${user.resetPasswordToken}`; - - mail.send(user, 'Password Reset', - `Hello ${user.firstName} ${user.lastName}!

          ` + - 'You are receiving this because you (or someone else) have requested the reset of the password for your account.

          ' + - 'Please click on the following link, or paste this into your browser to complete the process:

          ' + - `${resetLink}

          ` + - 'If you did not request this, please ignore this email and your password will remain unchanged.', - 'Failed to send email with reset link!'); - }) + .then((user) => mailsService.emailUserResetLink(req.origin(), user)) .then(() => res.status(200).send('An email has been sent with further instructions.')) .catch((err) => { // TODO IGNITE-843 Send email to admin @@ -208,10 +144,7 @@ module.exports.factory = function(express, passport, nodemailer, settings, mail, }); }); }) - .then((user) => mail.send(user, 'Your password has been changed', - `Hello ${user.firstName} ${user.lastName}!

          ` + - `This is a confirmation that the password for your account on ${settings.smtp.username} has just been changed.

          `, - 'Password was changed, but failed to send confirmation email!')) + .then((user) => mailsService.emailPasswordChanged(req.origin(), user)) .then((user) => res.status(200).send(user.email)) .catch((err) => res.status(401).send(err.message)); }); diff --git a/modules/web-console/backend/services/agents.js b/modules/web-console/backend/services/agents.js new file mode 100644 index 0000000000000..8a657395f8538 --- /dev/null +++ b/modules/web-console/backend/services/agents.js @@ -0,0 +1,82 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'services/agents', + inject: ['require(lodash)', 'require(fs)', 'require(path)', 'require(jszip)', 'settings', 'agent-manager', 'errors'] +}; + +/** + * @param _ + * @param fs + * @param path + * @param JSZip + * @param settings + * @param agentMgr + * @param errors + * @returns {AgentsService} + */ +module.exports.factory = (_, fs, path, JSZip, settings, agentMgr, errors) => { + class AgentsService { + /** + * Get agent archive with user agent configuration. + * @returns {*} - readable stream for further piping. (http://stuk.github.io/jszip/documentation/api_jszip/generate_node_stream.html) + */ + static getArchive(host, token) { + const latest = agentMgr.supportedAgents.latest; + + if (_.isEmpty(latest)) + throw new errors.MissingResourceException('Missing agent zip on server. Please ask webmaster to upload agent zip!'); + + const filePath = latest.filePath; + const fileName = latest.fileName; + + const folder = path.basename(latest.fileName, '.zip'); + + // Read a zip file. + return new Promise((resolve, reject) => { + fs.readFile(filePath, (errFs, data) => { + if (errFs) + reject(new errors.ServerErrorException(errFs)); + + JSZip.loadAsync(data) + .then((zip) => { + const prop = []; + + prop.push('tokens=' + token); + prop.push('server-uri=' + (settings.agent.SSLOptions ? 'https' : 'http') + '://' + host + ':' + settings.agent.port); + prop.push('#Uncomment following options if needed:'); + prop.push('#node-uri=http://localhost:8080'); + prop.push('#driver-folder=./jdbc-drivers'); + + zip.file(folder + '/default.properties', prop.join('\n')); + + return zip.generateAsync({type: 'nodebuffer', platform: 'UNIX'}) + .then((buffer) => resolve({filePath, fileName, buffer})); + }) + .catch(reject); + }); + }); + } + } + + return AgentsService; +}; diff --git a/modules/web-console/backend/services/auth.js b/modules/web-console/backend/services/auth.js new file mode 100644 index 0000000000000..9f7d77dd9dc0e --- /dev/null +++ b/modules/web-console/backend/services/auth.js @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'services/auth', + inject: ['require(lodash)', 'mongo', 'services/spaces', 'errors'] +}; + +/** + * @param _ + * @param mongo + * @param {SpacesService} spacesService + * @param errors + * @returns {AuthService} + */ +module.exports.factory = (_, mongo, spacesService, errors) => { + class AuthService { + // TODO IGNITE-3774: move implementation from public router. + static resetPassword() { + + } + + static validateResetToken() { + + } + } + + return AuthService; +}; diff --git a/modules/web-console/backend/services/caches.js b/modules/web-console/backend/services/caches.js new file mode 100644 index 0000000000000..f8cc2edd0790b --- /dev/null +++ b/modules/web-console/backend/services/caches.js @@ -0,0 +1,144 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'services/caches', + inject: ['require(lodash)', 'mongo', 'services/spaces', 'errors'] +}; + +/** + * @param _ + * @param mongo + * @param {SpacesService} spaceService + * @param errors + * @returns {CachesService} + */ +module.exports.factory = (_, mongo, spaceService, errors) => { + /** + * Convert remove status operation to own presentation. + * @param {RemoveResult} result - The results of remove operation. + */ + const convertRemoveStatus = ({result}) => ({rowsAffected: result.n}); + + /** + * Update existing cache + * @param {Object} cache - The cache for updating + * @returns {Promise.} that resolves cache id + */ + const update = (cache) => { + const cacheId = cache._id; + + return mongo.Cache.update({_id: cacheId}, cache, {upsert: true}).exec() + .then(() => mongo.Cluster.update({_id: {$in: cache.clusters}}, {$addToSet: {caches: cacheId}}, {multi: true}).exec()) + .then(() => mongo.Cluster.update({_id: {$nin: cache.clusters}}, {$pull: {caches: cacheId}}, {multi: true}).exec()) + .then(() => mongo.DomainModel.update({_id: {$in: cache.domains}}, {$addToSet: {caches: cacheId}}, {multi: true}).exec()) + .then(() => mongo.DomainModel.update({_id: {$nin: cache.domains}}, {$pull: {caches: cacheId}}, {multi: true}).exec()) + .then(() => cache) + .catch((err) => { + if (err.code === mongo.errCodes.DUPLICATE_KEY_UPDATE_ERROR || err.code === mongo.errCodes.DUPLICATE_KEY_ERROR) + throw new errors.DuplicateKeyException('Cache with name: "' + cache.name + '" already exist.'); + }); + }; + + /** + * Create new cache. + * @param {Object} cache - The cache for creation. + * @returns {Promise.} that resolves cache id. + */ + const create = (cache) => { + return mongo.Cache.create(cache) + .then((savedCache) => + mongo.Cluster.update({_id: {$in: savedCache.clusters}}, {$addToSet: {caches: savedCache._id}}, {multi: true}).exec() + .then(() => mongo.DomainModel.update({_id: {$in: savedCache.domains}}, {$addToSet: {caches: savedCache._id}}, {multi: true}).exec()) + .then(() => savedCache) + ) + .catch((err) => { + if (err.code === mongo.errCodes.DUPLICATE_KEY_ERROR) + throw new errors.DuplicateKeyException('Cache with name: "' + cache.name + '" already exist.'); + }); + }; + + /** + * Remove all caches by space ids. + * @param {Number[]} spaceIds - The space ids for cache deletion. + * @returns {Promise.} - that resolves results of remove operation. + */ + const removeAllBySpaces = (spaceIds) => { + return mongo.Cluster.update({space: {$in: spaceIds}}, {caches: []}, {multi: true}).exec() + .then(() => mongo.DomainModel.update({space: {$in: spaceIds}}, {caches: []}, {multi: true}).exec()) + .then(() => mongo.Cache.remove({space: {$in: spaceIds}}).exec()); + }; + + /** + * Service for manipulate Cache entities. + */ + class CachesService { + /** + * Create or update cache. + * @param {Object} cache - The cache. + * @returns {Promise.} that resolves cache id of merge operation. + */ + static merge(cache) { + if (cache._id) + return update(cache); + + return create(cache); + } + + /** + * Get caches by spaces. + * @param {mongo.ObjectId|String} spaceIds - The spaces ids that own caches. + * @returns {Promise.} - contains requested caches. + */ + static listBySpaces(spaceIds) { + return mongo.Cache.find({space: {$in: spaceIds}}).sort('name').lean().exec(); + } + + /** + * Remove cache. + * @param {mongo.ObjectId|String} cacheId - The cache id for remove. + * @returns {Promise.<{rowsAffected}>} - The number of affected rows. + */ + static remove(cacheId) { + if (_.isNil(cacheId)) + return Promise.reject(new errors.IllegalArgumentException('Cache id can not be undefined or null')); + + return mongo.Cluster.update({caches: {$in: [cacheId]}}, {$pull: {caches: cacheId}}, {multi: true}).exec() + .then(() => mongo.DomainModel.update({caches: {$in: [cacheId]}}, {$pull: {caches: cacheId}}, {multi: true}).exec()) + .then(() => mongo.Cache.remove({_id: cacheId}).exec()) + .then(convertRemoveStatus); + } + + /** + * Remove all caches by user. + * @param {mongo.ObjectId|String} userId - The user id that own caches. + * @param {Boolean} demo - The flag indicates that need lookup in demo space. + * @returns {Promise.<{rowsAffected}>} - The number of affected rows. + */ + static removeAll(userId, demo) { + return spaceService.spaceIds(userId, demo) + .then(removeAllBySpaces) + .then(convertRemoveStatus); + } + } + + return CachesService; +}; diff --git a/modules/web-console/backend/services/clusters.js b/modules/web-console/backend/services/clusters.js new file mode 100644 index 0000000000000..6c2722b9b5fb0 --- /dev/null +++ b/modules/web-console/backend/services/clusters.js @@ -0,0 +1,141 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'services/clusters', + inject: ['require(lodash)', 'mongo', 'services/spaces', 'errors'] +}; + +/** + * @param _ + * @param mongo + * @param {SpacesService} spacesService + * @param errors + * @returns {ClustersService} + */ +module.exports.factory = (_, mongo, spacesService, errors) => { + /** + * Convert remove status operation to own presentation. + * @param {RemoveResult} result - The results of remove operation. + */ + const convertRemoveStatus = ({result}) => ({rowsAffected: result.n}); + + /** + * Update existing cluster + * @param {Object} cluster - The cluster for updating + * @returns {Promise.} that resolves cluster id + */ + const update = (cluster) => { + const clusterId = cluster._id; + + return mongo.Cluster.update({_id: clusterId}, cluster, {upsert: true}).exec() + .then(() => mongo.Cache.update({_id: {$in: cluster.caches}}, {$addToSet: {clusters: clusterId}}, {multi: true}).exec()) + .then(() => mongo.Cache.update({_id: {$nin: cluster.caches}}, {$pull: {clusters: clusterId}}, {multi: true}).exec()) + .then(() => mongo.Igfs.update({_id: {$in: cluster.igfss}}, {$addToSet: {clusters: clusterId}}, {multi: true}).exec()) + .then(() => mongo.Igfs.update({_id: {$nin: cluster.igfss}}, {$pull: {clusters: clusterId}}, {multi: true}).exec()) + .then(() => cluster) + .catch((err) => { + if (err.code === mongo.errCodes.DUPLICATE_KEY_UPDATE_ERROR || err.code === mongo.errCodes.DUPLICATE_KEY_ERROR) + throw new errors.DuplicateKeyException('Cluster with name: "' + cluster.name + '" already exist.'); + }); + }; + + /** + * Create new cluster. + * @param {Object} cluster - The cluster for creation. + * @returns {Promise.} that resolves cluster id. + */ + const create = (cluster) => { + return mongo.Cluster.create(cluster) + .then((savedCluster) => + mongo.Cache.update({_id: {$in: savedCluster.caches}}, {$addToSet: {clusters: savedCluster._id}}, {multi: true}).exec() + .then(() => mongo.Igfs.update({_id: {$in: savedCluster.igfss}}, {$addToSet: {clusters: savedCluster._id}}, {multi: true}).exec()) + .then(() => savedCluster) + ) + .catch((err) => { + if (err.code === mongo.errCodes.DUPLICATE_KEY_ERROR) + throw new errors.DuplicateKeyException('Cluster with name: "' + cluster.name + '" already exist.'); + }); + }; + + /** + * Remove all caches by space ids. + * @param {Number[]} spaceIds - The space ids for cache deletion. + * @returns {Promise.} - that resolves results of remove operation. + */ + const removeAllBySpaces = (spaceIds) => { + return mongo.Cache.update({space: {$in: spaceIds}}, {clusters: []}, {multi: true}).exec() + .then(() => mongo.Igfs.update({space: {$in: spaceIds}}, {clusters: []}, {multi: true}).exec()) + .then(() => mongo.Cluster.remove({space: {$in: spaceIds}}).exec()); + }; + + class ClustersService { + /** + * Create or update cluster. + * @param {Object} cluster - The cluster + * @returns {Promise.} that resolves cluster id of merge operation. + */ + static merge(cluster) { + if (cluster._id) + return update(cluster); + + return create(cluster); + } + + /** + * Get clusters and linked objects by space. + * @param {mongo.ObjectId|String} spaceIds - The spaces id that own cluster. + * @returns {Promise.<[mongo.Cache[], mongo.Cluster[], mongo.DomainModel[], mongo.Space[]]>} - contains requested caches and array of linked objects: clusters, domains, spaces. + */ + static listBySpaces(spaceIds) { + return mongo.Cluster.find({space: {$in: spaceIds}}).sort('name').lean().exec(); + } + + /** + * Remove cluster. + * @param {mongo.ObjectId|String} clusterId - The cluster id for remove. + * @returns {Promise.<{rowsAffected}>} - The number of affected rows. + */ + static remove(clusterId) { + if (_.isNil(clusterId)) + return Promise.reject(new errors.IllegalArgumentException('Cluster id can not be undefined or null')); + + return mongo.Cache.update({clusters: {$in: [clusterId]}}, {$pull: {clusters: clusterId}}, {multi: true}).exec() + .then(() => mongo.Igfs.update({clusters: {$in: [clusterId]}}, {$pull: {clusters: clusterId}}, {multi: true}).exec()) + .then(() => mongo.Cluster.remove({_id: clusterId}).exec()) + .then(convertRemoveStatus); + } + + /** + * Remove all clusters by user. + * @param {mongo.ObjectId|String} userId - The user id that own cluster. + * @param {Boolean} demo - The flag indicates that need lookup in demo space. + * @returns {Promise.<{rowsAffected}>} - The number of affected rows. + */ + static removeAll(userId, demo) { + return spacesService.spaceIds(userId, demo) + .then(removeAllBySpaces) + .then(convertRemoveStatus); + } + } + + return ClustersService; +}; diff --git a/modules/web-console/backend/services/configurations.js b/modules/web-console/backend/services/configurations.js new file mode 100644 index 0000000000000..7eef8a230c5ea --- /dev/null +++ b/modules/web-console/backend/services/configurations.js @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'services/configurations', + inject: ['require(lodash)', 'mongo', 'services/spaces', 'services/clusters', 'services/caches', 'services/domains', 'services/igfss'] +}; + +/** + * @param _ + * @param mongo + * @param {SpacesService} spacesService + * @param {ClustersService} clustersService + * @param {CachesService} cachesService + * @param {DomainsService} domainsService + * @param {IgfssService} igfssService + * @returns {ConfigurationsService} + */ +module.exports.factory = (_, mongo, spacesService, clustersService, cachesService, domainsService, igfssService) => { + class ConfigurationsService { + static list(userId, demo) { + let spaces; + + return spacesService.spaces(userId, demo) + .then((_spaces) => { + spaces = _spaces; + + return spaces.map((space) => space._id); + }) + .then((spaceIds) => Promise.all([ + clustersService.listBySpaces(spaceIds), + domainsService.listBySpaces(spaceIds), + cachesService.listBySpaces(spaceIds), + igfssService.listBySpaces(spaceIds) + ])) + .then(([clusters, domains, caches, igfss]) => ({clusters, domains, caches, igfss, spaces})); + } + } + + return ConfigurationsService; +}; diff --git a/modules/web-console/backend/services/domains.js b/modules/web-console/backend/services/domains.js new file mode 100644 index 0000000000000..3e4e12963434a --- /dev/null +++ b/modules/web-console/backend/services/domains.js @@ -0,0 +1,187 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'services/domains', + inject: ['require(lodash)', 'mongo', 'services/spaces', 'services/caches', 'errors'] +}; + +/** + * @param _ + * @param mongo + * @param {SpacesService} spacesService + * @param {CachesService} cachesService + * @param errors + * @returns {DomainsService} + */ +module.exports.factory = (_, mongo, spacesService, cachesService, errors) => { + /** + * Convert remove status operation to own presentation. + * @param {RemoveResult} result - The results of remove operation. + */ + const convertRemoveStatus = ({result}) => ({rowsAffected: result.n}); + + const _updateCacheStore = (cacheStoreChanges) => + Promise.all(_.map(cacheStoreChanges, (change) => mongo.Cache.update({_id: {$eq: change.cacheId}}, change.change, {}).exec())); + + /** + * Update existing domain + * @param {Object} domain - The domain for updating + * @param savedDomains List of saved domains. + * @returns {Promise.} that resolves domain id + */ + const update = (domain, savedDomains) => { + const domainId = domain._id; + + return mongo.DomainModel.update({_id: domainId}, domain, {upsert: true}).exec() + .then(() => mongo.Cache.update({_id: {$in: domain.caches}}, {$addToSet: {domains: domainId}}, {multi: true}).exec()) + .then(() => mongo.Cache.update({_id: {$nin: domain.caches}}, {$pull: {domains: domainId}}, {multi: true}).exec()) + .then(() => { + savedDomains.push(domain); + + return _updateCacheStore(domain.cacheStoreChanges); + }) + .catch((err) => { + if (err.code === mongo.errCodes.DUPLICATE_KEY_UPDATE_ERROR || err.code === mongo.errCodes.DUPLICATE_KEY_ERROR) + throw new errors.DuplicateKeyException('Domain model with value type: "' + domain.valueType + '" already exist.'); + }); + }; + + /** + * Create new domain. + * @param {Object} domain - The domain for creation. + * @param savedDomains List of saved domains. + * @returns {Promise.} that resolves cluster id. + */ + const create = (domain, savedDomains) => { + return mongo.DomainModel.create(domain) + .then((createdDomain) => { + savedDomains.push(createdDomain); + + return mongo.Cache.update({_id: {$in: domain.caches}}, {$addToSet: {domains: createdDomain._id}}, {multi: true}).exec() + .then(() => _updateCacheStore(domain.cacheStoreChanges)); + }) + .catch((err) => { + if (err.code === mongo.errCodes.DUPLICATE_KEY_ERROR) + throw new errors.DuplicateKeyException('Domain model with value type: "' + domain.valueType + '" already exist.'); + }); + }; + + const _saveDomainModel = (domain, savedDomains) => { + const domainId = domain._id; + + if (domainId) + return update(domain, savedDomains); + + return create(domain, savedDomains); + }; + + const _save = (domains) => { + if (_.isEmpty(domains)) + throw new errors.IllegalArgumentException('Nothing to save!'); + + const savedDomains = []; + const generatedCaches = []; + + const promises = _.map(domains, (domain) => { + if (domain.newCache) { + return mongo.Cache.findOne({space: domain.space, name: domain.newCache.name}).exec() + .then((cache) => { + if (cache) + return Promise.resolve(cache); + + // If cache not found, then create it and associate with domain model. + const newCache = domain.newCache; + newCache.space = domain.space; + + return cachesService.merge(newCache); + }) + .then((cache) => { + domain.caches = [cache._id]; + + return _saveDomainModel(domain, savedDomains); + }); + } + + return _saveDomainModel(domain, savedDomains); + }); + + return Promise.all(promises).then(() => ({savedDomains, generatedCaches})); + }; + + /** + * Remove all caches by space ids. + * @param {Array.} spaceIds - The space ids for cache deletion. + * @returns {Promise.} - that resolves results of remove operation. + */ + const removeAllBySpaces = (spaceIds) => { + return mongo.Cache.update({space: {$in: spaceIds}}, {domains: []}, {multi: true}).exec() + .then(() => mongo.DomainModel.remove({space: {$in: spaceIds}}).exec()); + }; + + class DomainsService { + /** + * Batch merging domains. + * @param {Array.} domains + */ + static batchMerge(domains) { + return _save(domains); + } + + /** + * Get domain and linked objects by space. + * @param {mongo.ObjectId|String} spaceIds - The space id that own domain. + * @returns {Promise.<[mongo.Cache[], mongo.Cluster[], mongo.DomainModel[], mongo.Space[]]>} + * contains requested domains and array of linked objects: caches, spaces. + */ + static listBySpaces(spaceIds) { + return mongo.DomainModel.find({space: {$in: spaceIds}}).sort('valueType').lean().exec(); + } + + /** + * Remove domain. + * @param {mongo.ObjectId|String} domainId - The domain id for remove. + * @returns {Promise.<{rowsAffected}>} - The number of affected rows. + */ + static remove(domainId) { + if (_.isNil(domainId)) + return Promise.reject(new errors.IllegalArgumentException('Domain id can not be undefined or null')); + + return mongo.Cache.update({domains: {$in: [domainId]}}, {$pull: {domains: domainId}}, {multi: true}).exec() + .then(() => mongo.DomainModel.remove({_id: domainId}).exec()) + .then(convertRemoveStatus); + } + + /** + * Remove all domains by user. + * @param {mongo.ObjectId|String} userId - The user id that own domain. + * @param {Boolean} demo - The flag indicates that need lookup in demo space. + * @returns {Promise.<{rowsAffected}>} - The number of affected rows. + */ + static removeAll(userId, demo) { + return spacesService.spaceIds(userId, demo) + .then(removeAllBySpaces) + .then(convertRemoveStatus); + } + } + + return DomainsService; +}; diff --git a/modules/web-console/backend/services/igfss.js b/modules/web-console/backend/services/igfss.js new file mode 100644 index 0000000000000..20f0121fa3599 --- /dev/null +++ b/modules/web-console/backend/services/igfss.js @@ -0,0 +1,136 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'services/igfss', + inject: ['require(lodash)', 'mongo', 'services/spaces', 'errors'] +}; + +/** + * @param _ + * @param mongo + * @param {SpacesService} spacesService + * @param errors + * @returns {IgfssService} + */ +module.exports.factory = (_, mongo, spacesService, errors) => { + /** + * Convert remove status operation to own presentation. + * @param {RemoveResult} result - The results of remove operation. + */ + const convertRemoveStatus = ({result}) => ({rowsAffected: result.n}); + + /** + * Update existing IGFS + * @param {Object} igfs - The IGFS for updating + * @returns {Promise.} that resolves IGFS id + */ + const update = (igfs) => { + const igfsId = igfs._id; + + return mongo.Igfs.update({_id: igfsId}, igfs, {upsert: true}).exec() + .then(() => mongo.Cluster.update({_id: {$in: igfs.clusters}}, {$addToSet: {igfss: igfsId}}, {multi: true}).exec()) + .then(() => mongo.Cluster.update({_id: {$nin: igfs.clusters}}, {$pull: {igfss: igfsId}}, {multi: true}).exec()) + .then(() => igfs) + .catch((err) => { + if (err.code === mongo.errCodes.DUPLICATE_KEY_UPDATE_ERROR || err.code === mongo.errCodes.DUPLICATE_KEY_ERROR) + throw new errors.DuplicateKeyException('IGFS with name: "' + igfs.name + '" already exist.'); + }); + }; + + /** + * Create new IGFS. + * @param {Object} igfs - The IGFS for creation. + * @returns {Promise.} that resolves IGFS id. + */ + const create = (igfs) => { + return mongo.Igfs.create(igfs) + .then((savedIgfs) => + mongo.Cluster.update({_id: {$in: savedIgfs.clusters}}, {$addToSet: {igfss: savedIgfs._id}}, {multi: true}).exec() + .then(() => savedIgfs) + ) + .catch((err) => { + if (err.code === mongo.errCodes.DUPLICATE_KEY_ERROR) + throw new errors.DuplicateKeyException('IGFS with name: "' + igfs.name + '" already exist.'); + }); + }; + + /** + * Remove all IGFSs by space ids. + * @param {Number[]} spaceIds - The space ids for IGFS deletion. + * @returns {Promise.} - that resolves results of remove operation. + */ + const removeAllBySpaces = (spaceIds) => { + return mongo.Cluster.update({space: {$in: spaceIds}}, {igfss: []}, {multi: true}).exec() + .then(() => mongo.Igfs.remove({space: {$in: spaceIds}}).exec()); + }; + + class IgfssService { + /** + * Create or update IGFS. + * @param {Object} igfs - The IGFS + * @returns {Promise.} that resolves IGFS id of merge operation. + */ + static merge(igfs) { + if (igfs._id) + return update(igfs); + + return create(igfs); + } + + /** + * Get IGFS by spaces. + * @param {mongo.ObjectId|String} spacesIds - The spaces ids that own IGFSs. + * @returns {Promise.} - contains requested IGFSs. + */ + static listBySpaces(spacesIds) { + return mongo.Igfs.find({space: {$in: spacesIds}}).sort('name').lean().exec(); + } + + /** + * Remove IGFS. + * @param {mongo.ObjectId|String} igfsId - The IGFS id for remove. + * @returns {Promise.<{rowsAffected}>} - The number of affected rows. + */ + static remove(igfsId) { + if (_.isNil(igfsId)) + return Promise.reject(new errors.IllegalArgumentException('IGFS id can not be undefined or null')); + + return mongo.Cluster.update({igfss: {$in: [igfsId]}}, {$pull: {igfss: igfsId}}, {multi: true}).exec() + .then(() => mongo.Igfs.remove({_id: igfsId}).exec()) + .then(convertRemoveStatus); + } + + /** + * Remove all IGFSes by user. + * @param {mongo.ObjectId|String} userId - The user id that own IGFS. + * @param {Boolean} demo - The flag indicates that need lookup in demo space. + * @returns {Promise.<{rowsAffected}>} - The number of affected rows. + */ + static removeAll(userId, demo) { + return spacesService.spaceIds(userId, demo) + .then(removeAllBySpaces) + .then(convertRemoveStatus); + } + } + + return IgfssService; +}; diff --git a/modules/web-console/backend/services/mails.js b/modules/web-console/backend/services/mails.js new file mode 100644 index 0000000000000..0700985fedfb2 --- /dev/null +++ b/modules/web-console/backend/services/mails.js @@ -0,0 +1,131 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'services/mails', + inject: ['require(lodash)', 'require(nodemailer)', 'settings'] +}; + +/** + * @param _ + * @param nodemailer + * @param settings + * @returns {MailsService} + */ +module.exports.factory = (_, nodemailer, settings) => { + /** + * Send mail to user. + * + * @param {Account} user + * @param {String} subject + * @param {String} html + * @param {String} sendErr + * @throws {Error} + * @return {Promise} + */ + const send = (user, subject, html, sendErr) => { + return new Promise((resolve, reject) => { + const transportConfig = settings.smtp; + + if (_.isEmpty(transportConfig.service) || _.isEmpty(transportConfig.auth.user) || _.isEmpty(transportConfig.auth.pass)) + throw new Error('Failed to send email. SMTP server is not configured. Please ask webmaster to setup SMTP server!'); + + const mailer = nodemailer.createTransport(transportConfig); + + const sign = settings.smtp.sign ? `

          --------------
          ${settings.smtp.sign}
          ` : ''; + + const mail = { + from: settings.smtp.from, + to: settings.smtp.address(`${user.firstName} ${user.lastName}`, user.email), + subject, + html: html + sign + }; + + mailer.sendMail(mail, (err) => { + if (err) + return reject(sendErr ? new Error(sendErr) : err); + + resolve(user); + }); + }); + }; + + class MailsService { + /** + * Send email to user for password reset. + * @param host + * @param user + */ + static emailUserSignUp(host, user) { + const resetLink = `${host}/password/reset?token=${user.resetPasswordToken}`; + + return send(user, `Thanks for signing up for ${settings.smtp.greeting}.`, + `Hello ${user.firstName} ${user.lastName}!

          ` + + `You are receiving this email because you have signed up to use ${settings.smtp.greeting}.

          ` + + 'If you have not done the sign up and do not know what this email is about, please ignore it.
          ' + + 'You may reset the password by clicking on the following link, or paste this into your browser:

          ' + + `${resetLink}`); + } + + /** + * Send email to user for password reset. + * @param host + * @param user + */ + static emailUserResetLink(host, user) { + const resetLink = `${host}/password/reset?token=${user.resetPasswordToken}`; + + return send(user, 'Password Reset', + `Hello ${user.firstName} ${user.lastName}!

          ` + + 'You are receiving this because you (or someone else) have requested the reset of the password for your account.

          ' + + 'Please click on the following link, or paste this into your browser to complete the process:

          ' + + `${resetLink}

          ` + + 'If you did not request this, please ignore this email and your password will remain unchanged.', + 'Failed to send email with reset link!'); + } + + /** + * Send email to user for password reset. + * @param host + * @param user + */ + static emailPasswordChanged(host, user) { + return send(user, 'Your password has been changed', + `Hello ${user.firstName} ${user.lastName}!

          ` + + `This is a confirmation that the password for your account on ${settings.smtp.greeting} has just been changed.

          `, + 'Password was changed, but failed to send confirmation email!'); + } + + /** + * Send email to user when it was deleted. + * @param host + * @param user + */ + static emailUserDeletion(host, user) { + return send(user, 'Your account was removed', + `Hello ${user.firstName} ${user.lastName}!

          ` + + `You are receiving this email because your account for ${settings.smtp.greeting} was removed.`, + 'Account was removed, but failed to send email notification to user!'); + } + } + + return MailsService; +}; diff --git a/modules/web-console/backend/services/notebooks.js b/modules/web-console/backend/services/notebooks.js new file mode 100644 index 0000000000000..8846d8e365dd7 --- /dev/null +++ b/modules/web-console/backend/services/notebooks.js @@ -0,0 +1,104 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'services/notebooks', + inject: ['require(lodash)', 'mongo', 'services/spaces', 'errors'] +}; + +/** + * @param _ + * @param mongo + * @param {SpacesService} spacesService + * @param errors + * @returns {NotebooksService} + */ +module.exports.factory = (_, mongo, spacesService, errors) => { + /** + * Convert remove status operation to own presentation. + * @param {RemoveResult} result - The results of remove operation. + */ + const convertRemoveStatus = ({result}) => ({rowsAffected: result.n}); + + /** + * Update existing notebook + * @param {Object} notebook - The notebook for updating + * @returns {Promise.} that resolves cache id + */ + const update = (notebook) => { + return mongo.Notebook.findOneAndUpdate({_id: notebook._id}, notebook, {new: true, upsert: true}).exec() + .catch((err) => { + if (err.code === mongo.errCodes.DUPLICATE_KEY_UPDATE_ERROR || err.code === mongo.errCodes.DUPLICATE_KEY_ERROR) + throw new errors.DuplicateKeyException('Notebook with name: "' + notebook.name + '" already exist.'); + }); + }; + + /** + * Create new notebook. + * @param {Object} notebook - The notebook for creation. + * @returns {Promise.} that resolves cache id. + */ + const create = (notebook) => { + return mongo.Notebook.create(notebook) + .catch((err) => { + if (err.code === mongo.errCodes.DUPLICATE_KEY_ERROR) + throw new errors.DuplicateKeyException('Notebook with name: "' + notebook.name + '" already exist.'); + }); + }; + + class NotebooksService { + /** + * Create or update Notebook. + * @param {Object} notebook - The Notebook + * @returns {Promise.} that resolves Notebook id of merge operation. + */ + static merge(notebook) { + if (notebook._id) + return update(notebook); + + return create(notebook); + } + + /** + * Get caches by spaces. + * @param {mongo.ObjectId|String} spaceIds - The spaces ids that own caches. + * @returns {Promise.} - contains requested caches. + */ + static listBySpaces(spaceIds) { + return mongo.Notebook.find({space: {$in: spaceIds}}).sort('name').lean().exec(); + } + + /** + * Remove Notebook. + * @param {mongo.ObjectId|String} notebookId - The Notebook id for remove. + * @returns {Promise.<{rowsAffected}>} - The number of affected rows. + */ + static remove(notebookId) { + if (_.isNil(notebookId)) + return Promise.reject(new errors.IllegalArgumentException('Notebook id can not be undefined or null')); + + return mongo.Notebook.remove({_id: notebookId}).exec() + .then(convertRemoveStatus); + } + } + + return NotebooksService; +}; diff --git a/modules/web-console/backend/services/sessions.js b/modules/web-console/backend/services/sessions.js new file mode 100644 index 0000000000000..4fa95a3bb286c --- /dev/null +++ b/modules/web-console/backend/services/sessions.js @@ -0,0 +1,63 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'services/sessions', + inject: ['require(lodash)', 'mongo', 'errors'] +}; + +/** + * @param _ + * @param mongo + * @param errors + * @returns {SessionsService} + */ +module.exports.factory = (_, mongo, errors) => { + class SessionsService { + /** + * Become user. + * @param {Session} session - current session of user. + * @param {mongo.ObjectId|String} viewedUserId - id of user to become. + */ + static become(session, viewedUserId) { + return mongo.Account.findById(viewedUserId).exec() + .then((viewedUser) => { + if (!session.req.user.admin) + throw new errors.IllegalAccessError('Became this user is not permitted. Only administrators can perform this actions.'); + + session.viewedUser = viewedUser; + }); + } + + /** + * Revert to your identity. + */ + static revert(session) { + return new Promise((resolve) => { + delete session.viewedUser; + + resolve(); + }); + } + } + + return SessionsService; +}; diff --git a/modules/web-console/backend/services/spaces.js b/modules/web-console/backend/services/spaces.js new file mode 100644 index 0000000000000..863d57cc9c74e --- /dev/null +++ b/modules/web-console/backend/services/spaces.js @@ -0,0 +1,75 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'services/spaces', + inject: ['mongo', 'errors'] +}; + +/** + * @param mongo + * @param errors + * @returns {SpacesService} + */ +module.exports.factory = (mongo, errors) => { + class SpacesService { + /** + * Query for user spaces. + * + * @param {mongo.ObjectId|String} userId User ID. + * @param {Boolean} demo Is need use demo space. + * @returns {Promise} + */ + static spaces(userId, demo) { + return mongo.Space.find({owner: userId, demo: !!demo}).lean().exec() + .then((spaces) => { + if (!spaces.length) + throw new errors.MissingResourceException('Failed to find space'); + + return spaces; + }); + } + + /** + * Extract IDs from user spaces. + * + * @param {mongo.ObjectId|String} userId User ID. + * @param {Boolean} demo Is need use demo space. + * @returns {Promise} + */ + static spaceIds(userId, demo) { + return this.spaces(userId, demo) + .then((spaces) => spaces.map((space) => space._id)); + } + + /** + * Create demo space for user + * @param userId - user id + * @returns {Promise} that resolves created demo space for user + */ + static createDemoSpace(userId) { + return new mongo.Space({name: 'Demo space', owner: userId, demo: true}).save(); + } + } + + return SpacesService; +}; + diff --git a/modules/web-console/backend/services/users.js b/modules/web-console/backend/services/users.js new file mode 100644 index 0000000000000..8058b25f9fed6 --- /dev/null +++ b/modules/web-console/backend/services/users.js @@ -0,0 +1,229 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'services/users', + inject: ['require(lodash)', 'mongo', 'settings', 'services/spaces', 'services/mails', 'agent-manager', 'errors'] +}; + +/** + * @param _ + * @param mongo + * @param settings + * @param {SpacesService} spacesService + * @param {MailsService} mailsService + * @param agentMgr + * @param errors + * @returns {UsersService} + */ +module.exports.factory = (_, mongo, settings, spacesService, mailsService, agentMgr, errors) => { + const _randomString = () => { + const possible = 'ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789'; + const possibleLen = possible.length; + + let res = ''; + + for (let i = 0; i < settings.tokenLength; i++) + res += possible.charAt(Math.floor(Math.random() * possibleLen)); + + return res; + }; + + class UsersService { + /** + * Save profile information. + * @param {String} host - The host + * @param {Object} user - The user + * @returns {Promise.} that resolves account id of merge operation. + */ + static create(host, user) { + return mongo.Account.count().exec() + .then((cnt) => { + user.admin = cnt === 0; + + user.token = _randomString(); + + return new mongo.Account(user); + }) + .then((created) => { + return new Promise((resolve, reject) => { + mongo.Account.register(created, user.password, (err, registered) => { + if (err) + reject(err); + + if (!registered) + reject(new errors.ServerErrorException('Failed to register user.')); + + resolve(registered); + }); + }); + }) + .then((registered) => { + registered.resetPasswordToken = _randomString(); + + return registered.save() + .then(() => mongo.Space.create({name: 'Personal space', owner: registered._id})) + .then(() => { + mailsService.emailUserSignUp(host, registered) + .catch((err) => console.error(err)); + + return registered; + }); + }); + } + + /** + * Save user. + * @param {Object} changed - The user + * @returns {Promise.} that resolves account id of merge operation. + */ + static save(changed) { + return mongo.Account.findById(changed._id).exec() + .then((user) => { + if (!changed.password) + return Promise.resolve(user); + + return new Promise((resolve, reject) => { + user.setPassword(changed.password, (err, _user) => { + if (err) + return reject(err); + + delete changed.password; + + resolve(_user); + }); + }); + }) + .then((user) => { + if (!changed.email || user.email === changed.email) + return Promise.resolve(user); + + return new Promise((resolve, reject) => { + mongo.Account.findOne({email: changed.email}, (err, _user) => { + // TODO send error to admin + if (err) + reject(new Error('Failed to check email!')); + + if (_user && _user._id !== user._id) + reject(new Error('User with this email already registered!')); + + resolve(user); + }); + }); + }) + .then((user) => { + if (changed.token && user.token !== changed.token) + agentMgr.close(user._id, user.token); + + _.extend(user, changed); + + return user.save(); + }); + } + + /** + * Get list of user accounts and summary information. + * @returns {mongo.Account[]} - returns all accounts with counters object + */ + static list() { + return Promise.all([ + mongo.Space.aggregate([ + {$match: {demo: false}}, + {$lookup: {from: 'clusters', localField: '_id', foreignField: 'space', as: 'clusters'}}, + {$lookup: {from: 'caches', localField: '_id', foreignField: 'space', as: 'caches'}}, + {$lookup: {from: 'domainmodels', localField: '_id', foreignField: 'space', as: 'domainmodels'}}, + {$lookup: {from: 'igfs', localField: '_id', foreignField: 'space', as: 'igfs'}}, + { + $project: { + owner: 1, + clusters: {$size: '$clusters'}, + models: {$size: '$domainmodels'}, + caches: {$size: '$caches'}, + igfs: {$size: '$igfs'} + } + } + ]).exec(), + mongo.Account.find({}).sort('firstName lastName').lean().exec() + ]) + .then(([counters, users]) => { + const countersMap = _.keyBy(counters, 'owner'); + + _.forEach(users, (user) => { + user.counters = _.omit(countersMap[user._id], '_id', 'owner'); + }); + + return users; + }); + } + + /** + * Remove account. + * @param {String} host. + * @param {mongo.ObjectId|String} userId - The account id for remove. + * @returns {Promise.<{rowsAffected}>} - The number of affected rows. + */ + static remove(host, userId) { + return mongo.Account.findByIdAndRemove(userId).exec() + .then((user) => { + return spacesService.spaceIds(userId) + .then((spaceIds) => Promise.all([ + mongo.Cluster.remove({space: {$in: spaceIds}}).exec(), + mongo.Cache.remove({space: {$in: spaceIds}}).exec(), + mongo.DomainModel.remove({space: {$in: spaceIds}}).exec(), + mongo.Igfs.remove({space: {$in: spaceIds}}).exec(), + mongo.Notebook.remove({space: {$in: spaceIds}}).exec(), + mongo.Space.remove({owner: userId}).exec() + ])) + .catch((err) => console.error(`Failed to cleanup spaces [user=${user.username}, err=${err}`)) + .then(() => user); + }) + .then((user) => mailsService.emailUserDeletion(host, user).catch((err) => console.error(err))); + } + + /** + * Get account information. + */ + static get(user, viewedUser) { + if (_.isNil(user)) + return Promise.reject(new errors.AuthFailedException('The user profile service failed the sign in. User profile cannot be loaded.')); + + const becomeUsed = viewedUser && user.admin; + + if (becomeUsed) { + user = viewedUser; + + user.becomeUsed = true; + } + else + user = user.toJSON(); + + return mongo.Space.findOne({owner: user._id, demo: true}).exec() + .then((demoSpace) => { + if (user && demoSpace) + user.demoCreated = true; + + return user; + }); + } + } + + return UsersService; +}; diff --git a/modules/web-console/backend/test/config/settings.json b/modules/web-console/backend/test/config/settings.json new file mode 100644 index 0000000000000..a17a777a5a428 --- /dev/null +++ b/modules/web-console/backend/test/config/settings.json @@ -0,0 +1,20 @@ +{ + "server": { + "port": 3000, + "ssl": false + }, + "mongodb": { + "url": "mongodb://localhost/console-test" + }, + "agentServer": { + "port": 3001, + "ssl": false + }, + "mail": { + "service": "", + "sign": "Kind regards,
          Apache Ignite Team", + "from": "Apache Ignite Web Console ", + "user": "someusername@somecompany.tld", + "pass": "" + } +} diff --git a/modules/web-console/backend/test/data/accounts.json b/modules/web-console/backend/test/data/accounts.json new file mode 100644 index 0000000000000..e5b7f98967a5f --- /dev/null +++ b/modules/web-console/backend/test/data/accounts.json @@ -0,0 +1,18 @@ +[ + { + "_id" : "57725443e6d604c05dab9ded", + "salt" : "ca8b49c2eacd498a0973de30c0873c166ed99fa0605981726aedcc85bee17832", + "hash" : "c052c87e454cd0875332719e1ce085ccd92bedb73c8f939ba45d387f724da97128280643ad4f841d929d48de802f48f4a27b909d2dc806d957d38a1a4049468ce817490038f00ac1416aaf9f8f5a5c476730b46ea22d678421cd269869d4ba9d194f73906e5d5a4fec5229459e20ebda997fb95298067126f6c15346d886d44b67def03bf3ffe484b2e4fa449985de33a0c12e4e1da4c7d71fe7af5d138433f703d8c7eeebbb3d57f1a89659010a1f1d3cd4fbc524abab07860daabb08f08a28b8bfc64ecde2ea3c103030d0d54fc24d9c02f92ee6b3aa1bcd5c70113ab9a8045faea7dd2dc59ec4f9f69fcf634232721e9fb44012f0e8c8fdf7c6bf642db6867ef8e7877123e1bc78af7604fee2e34ad0191f8b97613ea458e0fca024226b7055e08a4bdb256fabf0a203a1e5b6a6c298fb0c60308569cefba779ce1e41fb971e5d1745959caf524ab0bedafce67157922f9c505cea033f6ed28204791470d9d08d31ce7e8003df8a3a05282d4d60bfe6e2f7de06f4b18377dac0fe764ed683c9b2553e75f8280c748aa166fef6f89190b1c6d369ab86422032171e6f9686de42ac65708e63bf018a043601d85bc5c820c7ad1d51ded32e59cdaa629a3f7ae325bbc931f9f21d90c9204effdbd53721a60c8b180dd8c236133e287a47ccc9e5072eb6593771e435e4d5196d50d6ddb32c226651c6503387895c5ad025f69fd3", + "email" : "a@a", + "firstName" : "TestFirstName", + "lastName" : "TestLastName", + "company" : "TestCompany", + "country" : "Canada", + "admin" : true, + "token" : "ppw4tPI3JUOGHva8CODO", + "attempts" : 0, + "lastLogin" : "2016-06-28T10:41:07.463Z", + "__v" : 0, + "resetPasswordToken" : "892rnLbEnVp1FP75Jgpi" + } +] \ No newline at end of file diff --git a/modules/web-console/backend/test/data/caches.json b/modules/web-console/backend/test/data/caches.json new file mode 100644 index 0000000000000..f7a8690ce7fa7 --- /dev/null +++ b/modules/web-console/backend/test/data/caches.json @@ -0,0 +1,87 @@ +[ + { + "name": "CarCache", + "cacheMode": "PARTITIONED", + "atomicityMode": "ATOMIC", + "readThrough": true, + "writeThrough": true, + "sqlFunctionClasses": [], + "cacheStoreFactory": { + "kind": "CacheJdbcPojoStoreFactory", + "CacheJdbcPojoStoreFactory": { + "dataSourceBean": "dsH2", + "dialect": "H2" + } + }, + "domains": [], + "clusters": [] + }, + { + "name": "ParkingCache", + "cacheMode": "PARTITIONED", + "atomicityMode": "ATOMIC", + "readThrough": true, + "writeThrough": true, + "sqlFunctionClasses": [], + "cacheStoreFactory": { + "kind": "CacheJdbcPojoStoreFactory", + "CacheJdbcPojoStoreFactory": { + "dataSourceBean": "dsH2", + "dialect": "H2" + } + }, + "domains": [], + "clusters": [] + }, + { + "name": "CountryCache", + "cacheMode": "PARTITIONED", + "atomicityMode": "ATOMIC", + "readThrough": true, + "writeThrough": true, + "sqlFunctionClasses": [], + "cacheStoreFactory": { + "kind": "CacheJdbcPojoStoreFactory", + "CacheJdbcPojoStoreFactory": { + "dataSourceBean": "dsH2", + "dialect": "H2" + } + }, + "domains": [], + "clusters": [] + }, + { + "name": "DepartmentCache", + "cacheMode": "PARTITIONED", + "atomicityMode": "ATOMIC", + "readThrough": true, + "writeThrough": true, + "sqlFunctionClasses": [], + "cacheStoreFactory": { + "kind": "CacheJdbcPojoStoreFactory", + "CacheJdbcPojoStoreFactory": { + "dataSourceBean": "dsH2", + "dialect": "H2" + } + }, + "domains": [], + "clusters": [] + }, + { + "name": "EmployeeCache", + "cacheMode": "PARTITIONED", + "atomicityMode": "ATOMIC", + "readThrough": true, + "writeThrough": true, + "sqlFunctionClasses": [], + "cacheStoreFactory": { + "kind": "CacheJdbcPojoStoreFactory", + "CacheJdbcPojoStoreFactory": { + "dataSourceBean": "dsH2", + "dialect": "H2" + } + }, + "domains": [], + "clusters": [] + } +] diff --git a/modules/web-console/backend/test/data/clusters.json b/modules/web-console/backend/test/data/clusters.json new file mode 100644 index 0000000000000..014b51938f1dc --- /dev/null +++ b/modules/web-console/backend/test/data/clusters.json @@ -0,0 +1,50 @@ +[ + { + "name": "cluster-igfs", + "connector": { + "noDelay": true + }, + "communication": { + "tcpNoDelay": true + }, + "igfss": [], + "caches": [], + "binaryConfiguration": { + "compactFooter": true, + "typeConfigurations": [] + }, + "discovery": { + "kind": "Multicast", + "Multicast": { + "addresses": ["127.0.0.1:47500..47510"] + }, + "Vm": { + "addresses": ["127.0.0.1:47500..47510"] + } + } + }, + { + "name": "cluster-caches", + "connector": { + "noDelay": true + }, + "communication": { + "tcpNoDelay": true + }, + "igfss": [], + "caches": [], + "binaryConfiguration": { + "compactFooter": true, + "typeConfigurations": [] + }, + "discovery": { + "kind": "Multicast", + "Multicast": { + "addresses": ["127.0.0.1:47500..47510"] + }, + "Vm": { + "addresses": ["127.0.0.1:47500..47510"] + } + } + } +] diff --git a/modules/web-console/backend/test/data/domains.json b/modules/web-console/backend/test/data/domains.json new file mode 100644 index 0000000000000..980d8d1059a94 --- /dev/null +++ b/modules/web-console/backend/test/data/domains.json @@ -0,0 +1,307 @@ +[ + { + "keyType": "Integer", + "valueType": "model.Parking", + "queryMetadata": "Configuration", + "databaseSchema": "CARS", + "databaseTable": "PARKING", + "indexes": [], + "aliases": [], + "fields": [ + { + "name": "name", + "className": "String" + }, + { + "name": "capacity", + "className": "Integer" + } + ], + "valueFields": [ + { + "databaseFieldName": "NAME", + "databaseFieldType": "VARCHAR", + "javaFieldName": "name", + "javaFieldType": "String" + }, + { + "databaseFieldName": "CAPACITY", + "databaseFieldType": "INTEGER", + "javaFieldName": "capacity", + "javaFieldType": "int" + } + ], + "keyFields": [ + { + "databaseFieldName": "ID", + "databaseFieldType": "INTEGER", + "javaFieldName": "id", + "javaFieldType": "int" + } + ], + "caches": [] + }, + { + "keyType": "Integer", + "valueType": "model.Department", + "queryMetadata": "Configuration", + "databaseSchema": "PUBLIC", + "databaseTable": "DEPARTMENT", + "indexes": [], + "aliases": [], + "fields": [ + { + "name": "countryId", + "className": "Integer" + }, + { + "name": "name", + "className": "String" + } + ], + "valueFields": [ + { + "databaseFieldName": "COUNTRY_ID", + "databaseFieldType": "INTEGER", + "javaFieldName": "countryId", + "javaFieldType": "int" + }, + { + "databaseFieldName": "NAME", + "databaseFieldType": "VARCHAR", + "javaFieldName": "name", + "javaFieldType": "String" + } + ], + "keyFields": [ + { + "databaseFieldName": "ID", + "databaseFieldType": "INTEGER", + "javaFieldName": "id", + "javaFieldType": "int" + } + ], + "caches": [] + }, + { + "keyType": "Integer", + "valueType": "model.Employee", + "queryMetadata": "Configuration", + "databaseSchema": "PUBLIC", + "databaseTable": "EMPLOYEE", + "indexes": [ + { + "name": "EMP_NAMES", + "indexType": "SORTED", + "fields": [ + { + "name": "firstName", + "direction": true + }, + { + "name": "lastName", + "direction": true + } + ] + }, + { + "name": "EMP_SALARY", + "indexType": "SORTED", + "fields": [ + { + "name": "salary", + "direction": true + } + ] + } + ], + "aliases": [], + "fields": [ + { + "name": "departmentId", + "className": "Integer" + }, + { + "name": "managerId", + "className": "Integer" + }, + { + "name": "firstName", + "className": "String" + }, + { + "name": "lastName", + "className": "String" + }, + { + "name": "email", + "className": "String" + }, + { + "name": "phoneNumber", + "className": "String" + }, + { + "name": "hireDate", + "className": "Date" + }, + { + "name": "job", + "className": "String" + }, + { + "name": "salary", + "className": "Double" + } + ], + "valueFields": [ + { + "databaseFieldName": "DEPARTMENT_ID", + "databaseFieldType": "INTEGER", + "javaFieldName": "departmentId", + "javaFieldType": "int" + }, + { + "databaseFieldName": "MANAGER_ID", + "databaseFieldType": "INTEGER", + "javaFieldName": "managerId", + "javaFieldType": "Integer" + }, + { + "databaseFieldName": "FIRST_NAME", + "databaseFieldType": "VARCHAR", + "javaFieldName": "firstName", + "javaFieldType": "String" + }, + { + "databaseFieldName": "LAST_NAME", + "databaseFieldType": "VARCHAR", + "javaFieldName": "lastName", + "javaFieldType": "String" + }, + { + "databaseFieldName": "EMAIL", + "databaseFieldType": "VARCHAR", + "javaFieldName": "email", + "javaFieldType": "String" + }, + { + "databaseFieldName": "PHONE_NUMBER", + "databaseFieldType": "VARCHAR", + "javaFieldName": "phoneNumber", + "javaFieldType": "String" + }, + { + "databaseFieldName": "HIRE_DATE", + "databaseFieldType": "DATE", + "javaFieldName": "hireDate", + "javaFieldType": "Date" + }, + { + "databaseFieldName": "JOB", + "databaseFieldType": "VARCHAR", + "javaFieldName": "job", + "javaFieldType": "String" + }, + { + "databaseFieldName": "SALARY", + "databaseFieldType": "DOUBLE", + "javaFieldName": "salary", + "javaFieldType": "Double" + } + ], + "keyFields": [ + { + "databaseFieldName": "ID", + "databaseFieldType": "INTEGER", + "javaFieldName": "id", + "javaFieldType": "int" + } + ], + "caches": [] + }, + { + "keyType": "Integer", + "valueType": "model.Country", + "queryMetadata": "Configuration", + "databaseSchema": "PUBLIC", + "databaseTable": "COUNTRY", + "indexes": [], + "aliases": [], + "fields": [ + { + "name": "name", + "className": "String" + }, + { + "name": "population", + "className": "Integer" + } + ], + "valueFields": [ + { + "databaseFieldName": "NAME", + "databaseFieldType": "VARCHAR", + "javaFieldName": "name", + "javaFieldType": "String" + }, + { + "databaseFieldName": "POPULATION", + "databaseFieldType": "INTEGER", + "javaFieldName": "population", + "javaFieldType": "int" + } + ], + "keyFields": [ + { + "databaseFieldName": "ID", + "databaseFieldType": "INTEGER", + "javaFieldName": "id", + "javaFieldType": "int" + } + ], + "caches": [] + }, + { + "keyType": "Integer", + "valueType": "model.Car", + "queryMetadata": "Configuration", + "databaseSchema": "CARS", + "databaseTable": "CAR", + "indexes": [], + "aliases": [], + "fields": [ + { + "name": "parkingId", + "className": "Integer" + }, + { + "name": "name", + "className": "String" + } + ], + "valueFields": [ + { + "databaseFieldName": "PARKING_ID", + "databaseFieldType": "INTEGER", + "javaFieldName": "parkingId", + "javaFieldType": "int" + }, + { + "databaseFieldName": "NAME", + "databaseFieldType": "VARCHAR", + "javaFieldName": "name", + "javaFieldType": "String" + } + ], + "keyFields": [ + { + "databaseFieldName": "ID", + "databaseFieldType": "INTEGER", + "javaFieldName": "id", + "javaFieldType": "int" + } + ], + "caches": [] + } +] diff --git a/modules/web-console/backend/test/data/igfss.json b/modules/web-console/backend/test/data/igfss.json new file mode 100644 index 0000000000000..cd128a68bc6fb --- /dev/null +++ b/modules/web-console/backend/test/data/igfss.json @@ -0,0 +1,10 @@ +[ + { + "ipcEndpointEnabled": true, + "fragmentizerEnabled": true, + "name": "igfs", + "dataCacheName": "igfs-data", + "metaCacheName": "igfs-meta", + "clusters": [] + } +] diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/general.directive.js b/modules/web-console/backend/test/injector.js similarity index 70% rename from modules/web-console/src/main/js/app/modules/states/configuration/caches/general.directive.js rename to modules/web-console/backend/test/injector.js index 3edc1a33e6db1..8d44d310e3f03 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/general.directive.js +++ b/modules/web-console/backend/test/injector.js @@ -15,13 +15,17 @@ * limitations under the License. */ -import templateUrl from './general.jade'; +import path from 'path'; +import fireUp from 'fire-up'; -export default ['igniteConfigurationCachesGeneral', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; +module.exports = fireUp.newInjector({ + basePath: path.join(__dirname, '../'), + modules: [ + './app/**/*.js', + './config/**/*.js', + './errors/**/*.js', + './middlewares/**/*.js', + './routes/**/*.js', + './services/**/*.js' + ] +}); diff --git a/modules/web-console/backend/test/unit/CacheService.test.js b/modules/web-console/backend/test/unit/CacheService.test.js new file mode 100644 index 0000000000000..14427757d1796 --- /dev/null +++ b/modules/web-console/backend/test/unit/CacheService.test.js @@ -0,0 +1,192 @@ +/* + * 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. + */ + + +import {assert} from 'chai'; +import injector from '../injector'; +import testCaches from '../data/caches.json'; +import testAccounts from '../data/accounts.json'; + +let cacheService; +let mongo; +let errors; + +suite('CacheServiceTestsSuite', () => { + const prepareUserSpaces = () => { + return mongo.Account.create(testAccounts) + .then((accounts) => { + return Promise.all( + accounts.map((account) => mongo.Space.create( + [ + {name: 'Personal space', owner: account._id, demo: false}, + {name: 'Demo space', owner: account._id, demo: true} + ] + ))) + .then((spaces) => [accounts, spaces]); + }); + }; + + suiteSetup(() => { + return Promise.all([injector('services/caches'), + injector('mongo'), + injector('errors')]) + .then(([_cacheService, _mongo, _errors]) => { + mongo = _mongo; + cacheService = _cacheService; + errors = _errors; + }); + }); + + setup(() => { + return Promise.all([ + mongo.Cache.remove().exec(), + mongo.Account.remove().exec(), + mongo.Space.remove().exec() + ]); + }); + + test('Create new cache', (done) => { + cacheService.merge(testCaches[0]) + .then((cache) => { + assert.isNotNull(cache._id); + + return cache._id; + }) + .then((cacheId) => mongo.Cache.findById(cacheId)) + .then((cache) => { + assert.isNotNull(cache); + }) + .then(done) + .catch(done); + }); + + test('Update existed cache', (done) => { + const newName = 'NewUniqueName'; + + cacheService.merge(testCaches[0]) + .then((cache) => { + const cacheBeforeMerge = {...testCaches[0], _id: cache._id, name: newName}; + + return cacheService.merge(cacheBeforeMerge); + }) + .then((cache) => mongo.Cache.findById(cache._id)) + .then((cacheAfterMerge) => { + assert.equal(cacheAfterMerge.name, newName); + }) + .then(done) + .catch(done); + }); + + test('Create duplicated cache', (done) => { + cacheService.merge(testCaches[0]) + .then(() => cacheService.merge(testCaches[0])) + .catch((err) => { + assert.instanceOf(err, errors.DuplicateKeyException); + + done(); + }); + }); + + test('Remove existed cache', (done) => { + cacheService.merge(testCaches[0]) + .then((createdCache) => { + return mongo.Cache.findById(createdCache._id) + .then((foundCache) => foundCache._id) + .then(cacheService.remove) + .then(({rowsAffected}) => { + assert.equal(rowsAffected, 1); + }) + .then(() => mongo.Cache.findById(createdCache._id)) + .then((notFoundCache) => { + assert.isNull(notFoundCache); + }); + }) + .then(done) + .catch(done); + }); + + test('Remove cache without identifier', (done) => { + cacheService.merge(testCaches[0]) + .then(() => cacheService.remove()) + .catch((err) => { + assert.instanceOf(err, errors.IllegalArgumentException); + + done(); + }); + }); + + test('Remove missed cache', (done) => { + const validNoExistingId = 'FFFFFFFFFFFFFFFFFFFFFFFF'; + + cacheService.merge(testCaches[0]) + .then(() => cacheService.remove(validNoExistingId)) + .then(({rowsAffected}) => { + assert.equal(rowsAffected, 0); + }) + .then(done) + .catch(done); + }); + + test('Remove all caches in space', (done) => { + prepareUserSpaces() + .then(([accounts, spaces]) => { + const currentUser = accounts[0]; + const userCache = {...testCaches[0], space: spaces[0][0]._id}; + + return cacheService.merge(userCache) + .then(() => cacheService.removeAll(currentUser._id, false)); + }) + .then(({rowsAffected}) => { + assert.equal(rowsAffected, 1); + }) + .then(done) + .catch(done); + }); + + test('Get all caches by space', (done) => { + prepareUserSpaces() + .then(([accounts, spaces]) => { + const userCache = {...testCaches[0], space: spaces[0][0]._id}; + + return cacheService.merge(userCache) + .then((cache) => { + return cacheService.listBySpaces(spaces[0][0]._id) + .then((caches) => { + assert.equal(caches.length, 1); + assert.equal(caches[0]._id.toString(), cache._id.toString()); + }); + }); + }) + .then(done) + .catch(done); + }); + + test('Update linked entities on update cache', (done) => { + // TODO IGNITE-3262 Add test. + done(); + }); + + test('Update linked entities on remove cache', (done) => { + // TODO IGNITE-3262 Add test. + done(); + }); + + test('Update linked entities on remove all caches in space', (done) => { + // TODO IGNITE-3262 Add test. + done(); + }); +}); diff --git a/modules/web-console/backend/test/unit/ClusterService.test.js b/modules/web-console/backend/test/unit/ClusterService.test.js new file mode 100644 index 0000000000000..ab0e912b639e1 --- /dev/null +++ b/modules/web-console/backend/test/unit/ClusterService.test.js @@ -0,0 +1,190 @@ +/* + * 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. + */ + + +import {assert} from 'chai'; +import injector from '../injector'; +import testClusters from '../data/clusters.json'; +import testAccounts from '../data/accounts.json'; + +let clusterService; +let mongo; +let errors; + +suite('ClusterServiceTestsSuite', () => { + const prepareUserSpaces = () => { + return mongo.Account.create(testAccounts) + .then((accounts) => { + return Promise.all(accounts.map((account) => mongo.Space.create( + [ + {name: 'Personal space', owner: account._id, demo: false}, + {name: 'Demo space', owner: account._id, demo: true} + ] + ))) + .then((spaces) => [accounts, spaces]); + }); + }; + + suiteSetup(() => { + return Promise.all([injector('services/clusters'), + injector('mongo'), + injector('errors')]) + .then(([_clusterService, _mongo, _errors]) => { + mongo = _mongo; + clusterService = _clusterService; + errors = _errors; + }); + }); + + setup(() => { + return Promise.all([ + mongo.Cluster.remove().exec(), + mongo.Account.remove().exec(), + mongo.Space.remove().exec() + ]); + }); + + test('Create new cluster', (done) => { + clusterService.merge(testClusters[0]) + .then((cluster) => { + assert.isNotNull(cluster._id); + + return cluster._id; + }) + .then((clusterId) => mongo.Cluster.findById(clusterId)) + .then((cluster) => { + assert.isNotNull(cluster); + }) + .then(done) + .catch(done); + }); + + test('Update existed cluster', (done) => { + const newName = 'NewUniqueName'; + + clusterService.merge(testClusters[0]) + .then((cluster) => { + const clusterBeforeMerge = {...testClusters[0], _id: cluster._id, name: newName}; + + return clusterService.merge(clusterBeforeMerge); + }) + .then((cluster) => mongo.Cluster.findById(cluster._id)) + .then((clusterAfterMerge) => { + assert.equal(clusterAfterMerge.name, newName); + }) + .then(done) + .catch(done); + }); + + test('Create duplicated cluster', (done) => { + clusterService.merge(testClusters[0]) + .then(() => clusterService.merge(testClusters[0])) + .catch((err) => { + assert.instanceOf(err, errors.DuplicateKeyException); + + done(); + }); + }); + + test('Remove existed cluster', (done) => { + clusterService.merge(testClusters[0]) + .then((existCluster) => { + return mongo.Cluster.findById(existCluster._id) + .then((foundCluster) => clusterService.remove(foundCluster._id)) + .then(({rowsAffected}) => { + assert.equal(rowsAffected, 1); + }) + .then(() => mongo.Cluster.findById(existCluster._id)) + .then((notFoundCluster) => { + assert.isNull(notFoundCluster); + }); + }) + .then(done) + .catch(done); + }); + + test('Remove cluster without identifier', (done) => { + clusterService.merge(testClusters[0]) + .then(() => clusterService.remove()) + .catch((err) => { + assert.instanceOf(err, errors.IllegalArgumentException); + + done(); + }); + }); + + test('Remove missed cluster', (done) => { + const validNoExistingId = 'FFFFFFFFFFFFFFFFFFFFFFFF'; + + clusterService.merge(testClusters[0]) + .then(() => clusterService.remove(validNoExistingId)) + .then(({rowsAffected}) => { + assert.equal(rowsAffected, 0); + }) + .then(done) + .catch(done); + }); + + test('Remove all clusters in space', (done) => { + prepareUserSpaces() + .then(([accounts, spaces]) => { + const currentUser = accounts[0]; + const userCluster = {...testClusters[0], space: spaces[0][0]._id}; + + return clusterService.merge(userCluster) + .then(() => clusterService.removeAll(currentUser._id, false)); + }) + .then(({rowsAffected}) => { + assert.equal(rowsAffected, 1); + }) + .then(done) + .catch(done); + }); + + test('Get all clusters by space', (done) => { + prepareUserSpaces() + .then(([accounts, spaces]) => { + const userCluster = {...testClusters[0], space: spaces[0][0]._id}; + + return clusterService.merge(userCluster) + .then((existCluster) => { + return clusterService.listBySpaces(spaces[0][0]._id) + .then((clusters) => { + assert.equal(clusters.length, 1); + assert.equal(clusters[0]._id.toString(), existCluster._id.toString()); + }); + }); + }) + .then(done) + .catch(done); + }); + + test('Update linked entities on update cluster', (done) => { + // TODO IGNITE-3262 Add test. + done(); + }); + + test('Update linked entities on remove cluster', (done) => { + // TODO IGNITE-3262 Add test. + done(); + }); + + test('Update linked entities on remove all clusters in space', (done) => { + // TODO IGNITE-3262 Add test. + done(); + }); +}); diff --git a/modules/web-console/backend/test/unit/DomainService.test.js b/modules/web-console/backend/test/unit/DomainService.test.js new file mode 100644 index 0000000000000..477b45418d4bb --- /dev/null +++ b/modules/web-console/backend/test/unit/DomainService.test.js @@ -0,0 +1,198 @@ +/* + * 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. + */ + + +import {assert} from 'chai'; +import injector from '../injector'; +import testDomains from '../data/domains.json'; +import testAccounts from '../data/accounts.json'; + +let domainService; +let mongo; +let errors; + +suite('DomainsServiceTestsSuite', () => { + const prepareUserSpaces = () => { + return mongo.Account.create(testAccounts) + .then((accounts) => { + return Promise.all(accounts.map((account) => mongo.Space.create( + [ + {name: 'Personal space', owner: account._id, demo: false}, + {name: 'Demo space', owner: account._id, demo: true} + ] + ))) + .then((spaces) => [accounts, spaces]); + }); + }; + + suiteSetup(() => { + return Promise.all([injector('services/domains'), + injector('mongo'), + injector('errors')]) + .then(([_domainService, _mongo, _errors]) => { + mongo = _mongo; + domainService = _domainService; + errors = _errors; + }); + }); + + setup(() => { + return Promise.all([ + mongo.DomainModel.remove().exec(), + mongo.Account.remove().exec(), + mongo.Space.remove().exec() + ]); + }); + + test('Create new domain', (done) => { + domainService.batchMerge([testDomains[0]]) + .then((results) => { + const domain = results.savedDomains[0]; + + assert.isNotNull(domain._id); + + return domain._id; + }) + .then((domainId) => mongo.DomainModel.findById(domainId)) + .then((domain) => { + assert.isNotNull(domain); + }) + .then(done) + .catch(done); + }); + + test('Update existed domain', (done) => { + const newValType = 'value.Type'; + + domainService.batchMerge([testDomains[0]]) + .then((results) => { + const domain = results.savedDomains[0]; + + const domainBeforeMerge = {...testDomains[0], _id: domain._id, valueType: newValType}; + + return domainService.batchMerge([domainBeforeMerge]); + }) + .then((results) => mongo.DomainModel.findById(results.savedDomains[0]._id)) + .then((domainAfterMerge) => { + assert.equal(domainAfterMerge.valueType, newValType); + }) + .then(done) + .catch(done); + }); + + test('Create duplicated domain', (done) => { + domainService.batchMerge([testDomains[0]]) + .then(() => domainService.batchMerge([testDomains[0]])) + .catch((err) => { + assert.instanceOf(err, errors.DuplicateKeyException); + + done(); + }); + }); + + test('Remove existed domain', (done) => { + domainService.batchMerge([testDomains[0]]) + .then((results) => { + const domain = results.savedDomains[0]; + + return mongo.DomainModel.findById(domain._id) + .then((foundDomain) => domainService.remove(foundDomain._id)) + .then(({rowsAffected}) => { + assert.equal(rowsAffected, 1); + }) + .then(() => mongo.DomainModel.findById(domain._id)) + .then((notFoundDomain) => { + assert.isNull(notFoundDomain); + }); + }) + .then(done) + .catch(done); + }); + + test('Remove domain without identifier', (done) => { + domainService.batchMerge([testDomains[0]]) + .then(() => domainService.remove()) + .catch((err) => { + assert.instanceOf(err, errors.IllegalArgumentException); + + done(); + }); + }); + + test('Remove missed domain', (done) => { + const validNoExistingId = 'FFFFFFFFFFFFFFFFFFFFFFFF'; + + domainService.batchMerge([testDomains[0]]) + .then(() => domainService.remove(validNoExistingId)) + .then(({rowsAffected}) => { + assert.equal(rowsAffected, 0); + }) + .then(done) + .catch(done); + }); + + test('Remove all domains in space', (done) => { + prepareUserSpaces() + .then(([accounts, spaces]) => { + const currentUser = accounts[0]; + const userDomain = {...testDomains[0], space: spaces[0][0]._id}; + + return domainService.batchMerge([userDomain]) + .then(() => domainService.removeAll(currentUser._id, false)); + }) + .then(({rowsAffected}) => { + assert.equal(rowsAffected, 1); + }) + .then(done) + .catch(done); + }); + + test('Get all domains by space', (done) => { + prepareUserSpaces() + .then(([accounts, spaces]) => { + const userDomain = {...testDomains[0], space: spaces[0][0]._id}; + + return domainService.batchMerge([userDomain]) + .then((results) => { + const domain = results.savedDomains[0]; + + return domainService.listBySpaces(spaces[0][0]._id) + .then((domains) => { + assert.equal(domains.length, 1); + assert.equal(domains[0]._id.toString(), domain._id.toString()); + }); + }); + }) + .then(done) + .catch(done); + }); + + test('Update linked entities on update domain', (done) => { + // TODO IGNITE-3262 Add test. + done(); + }); + + test('Update linked entities on remove domain', (done) => { + // TODO IGNITE-3262 Add test. + done(); + }); + + test('Update linked entities on remove all domains in space', (done) => { + // TODO IGNITE-3262 Add test. + done(); + }); +}); diff --git a/modules/web-console/backend/test/unit/IgfsService.test.js b/modules/web-console/backend/test/unit/IgfsService.test.js new file mode 100644 index 0000000000000..3d78148ce3cbb --- /dev/null +++ b/modules/web-console/backend/test/unit/IgfsService.test.js @@ -0,0 +1,190 @@ +/* + * 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. + */ + + +import {assert} from 'chai'; +import injector from '../injector'; +import testIgfss from '../data/igfss.json'; +import testAccounts from '../data/accounts.json'; + +let igfsService; +let mongo; +let errors; + +suite('IgfsServiceTestsSuite', () => { + const prepareUserSpaces = () => { + return mongo.Account.create(testAccounts) + .then((accounts) => { + return Promise.all(accounts.map((account) => mongo.Space.create( + [ + {name: 'Personal space', owner: account._id, demo: false}, + {name: 'Demo space', owner: account._id, demo: true} + ] + ))) + .then((spaces) => [accounts, spaces]); + }); + }; + + suiteSetup(() => { + return Promise.all([injector('services/igfss'), + injector('mongo'), + injector('errors')]) + .then(([_igfsService, _mongo, _errors]) => { + mongo = _mongo; + igfsService = _igfsService; + errors = _errors; + }); + }); + + setup(() => { + return Promise.all([ + mongo.Igfs.remove().exec(), + mongo.Account.remove().exec(), + mongo.Space.remove().exec() + ]); + }); + + test('Create new igfs', (done) => { + igfsService.merge(testIgfss[0]) + .then((igfs) => { + assert.isNotNull(igfs._id); + + return igfs._id; + }) + .then((igfsId) => mongo.Igfs.findById(igfsId)) + .then((igfs) => { + assert.isNotNull(igfs); + }) + .then(done) + .catch(done); + }); + + test('Update existed igfs', (done) => { + const newName = 'NewUniqueName'; + + igfsService.merge(testIgfss[0]) + .then((existIgfs) => { + const igfsBeforeMerge = {...testIgfss[0], _id: existIgfs._id, name: newName}; + + return igfsService.merge(igfsBeforeMerge); + }) + .then((igfs) => mongo.Igfs.findById(igfs._id)) + .then((igfsAfterMerge) => { + assert.equal(igfsAfterMerge.name, newName); + }) + .then(done) + .catch(done); + }); + + test('Create duplicated igfs', (done) => { + igfsService.merge(testIgfss[0]) + .then(() => igfsService.merge(testIgfss[0])) + .catch((err) => { + assert.instanceOf(err, errors.DuplicateKeyException); + + done(); + }); + }); + + test('Remove existed igfs', (done) => { + igfsService.merge(testIgfss[0]) + .then((existIgfs) => { + return mongo.Igfs.findById(existIgfs._id) + .then((foundIgfs) => igfsService.remove(foundIgfs._id)) + .then(({rowsAffected}) => { + assert.equal(rowsAffected, 1); + }) + .then(() => mongo.Igfs.findById(existIgfs._id)) + .then((notFoundIgfs) => { + assert.isNull(notFoundIgfs); + }); + }) + .then(done) + .catch(done); + }); + + test('Remove igfs without identifier', (done) => { + igfsService.merge(testIgfss[0]) + .then(() => igfsService.remove()) + .catch((err) => { + assert.instanceOf(err, errors.IllegalArgumentException); + + done(); + }); + }); + + test('Remove missed igfs', (done) => { + const validNoExistingId = 'FFFFFFFFFFFFFFFFFFFFFFFF'; + + igfsService.merge(testIgfss[0]) + .then(() => igfsService.remove(validNoExistingId)) + .then(({rowsAffected}) => { + assert.equal(rowsAffected, 0); + }) + .then(done) + .catch(done); + }); + + test('Remove all igfss in space', (done) => { + prepareUserSpaces() + .then(([accounts, spaces]) => { + const currentUser = accounts[0]; + const userIgfs = {...testIgfss[0], space: spaces[0][0]._id}; + + return igfsService.merge(userIgfs) + .then(() => igfsService.removeAll(currentUser._id, false)); + }) + .then(({rowsAffected}) => { + assert.equal(rowsAffected, 1); + }) + .then(done) + .catch(done); + }); + + test('Get all igfss by space', (done) => { + prepareUserSpaces() + .then(([accounts, spaces]) => { + const userIgfs = {...testIgfss[0], space: spaces[0][0]._id}; + + return igfsService.merge(userIgfs) + .then((existIgfs) => { + return igfsService.listBySpaces(spaces[0][0]._id) + .then((igfss) => { + assert.equal(igfss.length, 1); + assert.equal(igfss[0]._id.toString(), existIgfs._id.toString()); + }); + }); + }) + .then(done) + .catch(done); + }); + + test('Update linked entities on update igfs', (done) => { + // TODO IGNITE-3262 Add test. + done(); + }); + + test('Update linked entities on remove igfs', (done) => { + // TODO IGNITE-3262 Add test. + done(); + }); + + test('Update linked entities on remove all igfss in space', (done) => { + // TODO IGNITE-3262 Add test. + done(); + }); +}); diff --git a/modules/web-console/docker/compose/backend/.dockerignore b/modules/web-console/docker/compose/backend/.dockerignore new file mode 100644 index 0000000000000..6fadfa5eb31ae --- /dev/null +++ b/modules/web-console/docker/compose/backend/.dockerignore @@ -0,0 +1 @@ +build/node_modules diff --git a/modules/web-console/docker/compose/backend/Dockerfile b/modules/web-console/docker/compose/backend/Dockerfile new file mode 100644 index 0000000000000..b4f7c9ddf4c39 --- /dev/null +++ b/modules/web-console/docker/compose/backend/Dockerfile @@ -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. +# + +FROM node:4 + +RUN mkdir -p /opt/web-console-backend + +WORKDIR /opt/web-console-backend + +COPY build . + +RUN npm -g update npm && npm install --no-optional + +EXPOSE 3000 3001 + +CMD ["npm", "start"] diff --git a/modules/web-console/docker/compose/backend/build.sh b/modules/web-console/docker/compose/backend/build.sh new file mode 100644 index 0000000000000..f925bd795b9a6 --- /dev/null +++ b/modules/web-console/docker/compose/backend/build.sh @@ -0,0 +1,57 @@ +#!/bin/bash +# +# 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. +# + +if [ -z "$IGNITE_HOME" ]; then + echo "Ignite source folder is not found or IGNITE_HOME environment variable is not valid." + + exit 1 +fi + +WORK_DIR=`cd "$(dirname "$0")"; pwd` + +BUILD_DIR="$WORK_DIR/build" + +IGNITE_WEB_CONSOLE_BACKEND_DIR="$IGNITE_HOME/modules/web-console/backend" +DOCKER_IMAGE_NAME="ignite/web-console-backend" + +echo "Receiving version..." +VERSION=`cd $IGNITE_HOME && mvn org.apache.maven.plugins:maven-help-plugin:evaluate -Dexpression=project.version| grep -Ev '(^\[|Download\w+:)'` +RELEASE_VERSION=${VERSION%-SNAPSHOT} + +echo "Building $DOCKER_IMAGE_NAME:$RELEASE_VERSION" +echo "Step 1. Prepare build temp paths." +cd $WORK_DIR +rm -Rf $BUILD_DIR +docker rmi -f $DOCKER_IMAGE_NAME:$RELEASE_VERSION + +echo "Step 2. Build ignite web agent." +cd $IGNITE_HOME +mvn versions:set -DnewVersion=$RELEASE_VERSION -DgenerateBackupPoms=false -Pweb-console -DartifactId='*' +mvn clean package -pl :ignite-web-agent -am -P web-console -DskipTests=true +mvn versions:set -DnewVersion=$VERSION -DgenerateBackupPoms=false -Pweb-console -DartifactId='*' + +echo "Step 3. Copy sources." +cd $WORK_DIR +cp -r $IGNITE_WEB_CONSOLE_BACKEND_DIR/. $BUILD_DIR +cp $IGNITE_HOME/modules/web-console/web-agent/target/ignite-web-agent*.zip $BUILD_DIR/agent_dists/. + +echo "Step 4. Build docker image." +docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION . + +echo "Step 5. Cleanup." +rm -Rf $BUILD_DIR diff --git a/modules/web-console/docker/compose/docker-compose.yml b/modules/web-console/docker/compose/docker-compose.yml new file mode 100644 index 0000000000000..bacd769b87f10 --- /dev/null +++ b/modules/web-console/docker/compose/docker-compose.yml @@ -0,0 +1,59 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +mongodb: + image: mongo:latest + volumes: + # External volume for persisting data. (HOST_PATH:CONTAINER_PATH). + - ./data/mongo:/data/db + +backend: + image: ignite/web-console-backend:1.7.0 + links: + # Link mongodb container as with mongodb hostname. + - mongodb:mongodb + ports: + # Proxy 3001 port from docker container to 3001 port host machine. (HOST_PORT:DOCKER_PORT) + - 3001:3001 + # Restart on crash. + restart: always + environment: + # Port for serving frontend API + - server_port=3000 + # Cookie session secret + - server_sessionSecret="CHANGE ME" + # URL for mongodb connection + - mongodb_url=mongodb://mongodb/console + # Port for web-agent. + - agentServer_port=3001 + # Mail connection settings. Leave empty if no needed. See also settings, https://github.com/nodemailer/nodemailer + - mail_service="" + - mail_sign="" + - mail_greeting="" + - mail_from="" + - mail_auth_user="" + - mail_auth_pass="" + +frontend: + image: ignite/web-console-frontend:1.7.0 + links: + # Link backend container to proxy backend requests throught nginx container. + - backend:backend + + ports: + # Proxy HTTP nginx port (HOST_PORT:DOCKER_PORT) + - 80:80 diff --git a/modules/web-console/docker/compose/frontend/.dockerignore b/modules/web-console/docker/compose/frontend/.dockerignore new file mode 100644 index 0000000000000..caf0e8e6e2df1 --- /dev/null +++ b/modules/web-console/docker/compose/frontend/.dockerignore @@ -0,0 +1,3 @@ +src/build +src/ignite_modules_temp +src/node_modules diff --git a/modules/web-console/docker/compose/frontend/Dockerfile b/modules/web-console/docker/compose/frontend/Dockerfile new file mode 100644 index 0000000000000..1b578d17cc828 --- /dev/null +++ b/modules/web-console/docker/compose/frontend/Dockerfile @@ -0,0 +1,32 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +FROM nginx + +RUN mkdir -p /data/www + +WORKDIR /data/www + +COPY ./build . + +COPY nginx/nginx.conf /etc/nginx/nginx.conf +COPY nginx/web-console.conf /etc/nginx/web-console.conf + +VOLUME /etc/nginx +VOLUME /data/www + +EXPOSE 80 diff --git a/modules/web-console/docker/compose/frontend/DockerfileBuild b/modules/web-console/docker/compose/frontend/DockerfileBuild new file mode 100644 index 0000000000000..277991f7f9a36 --- /dev/null +++ b/modules/web-console/docker/compose/frontend/DockerfileBuild @@ -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. +# + +FROM node:4 + +RUN mkdir -p /opt/web-console-frontend + +WORKDIR /opt/web-console-frontend + +COPY src . + +RUN npm update -g npm && npm install --no-optional + +VOLUME /opt/web-console-frontend/build + +CMD ["npm", "run", "build"] diff --git a/modules/web-console/docker/compose/frontend/build.sh b/modules/web-console/docker/compose/frontend/build.sh new file mode 100644 index 0000000000000..4dfa57af5abe0 --- /dev/null +++ b/modules/web-console/docker/compose/frontend/build.sh @@ -0,0 +1,59 @@ +#!/bin/bash +# +# 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. +# + +if [ -z "$IGNITE_HOME" ]; then + echo "Ignite source folder is not found or IGNITE_HOME environment variable is not valid." + + exit 1 +fi + +WORK_DIR=`cd "$(dirname "$0")"; pwd` + +SOURCE_DIR=$WORK_DIR/src +BUILD_DIR=$WORK_DIR/build + +DOCKER_BUILD_CONTAINER=web-console-frontend-builder +DOCKER_BUILD_IMAGE_NAME=ignite/$DOCKER_BUILD_CONTAINER +DOCKER_IMAGE_NAME=ignite/web-console-frontend + +echo "Receiving version..." +VERSION=`cd $IGNITE_HOME && mvn org.apache.maven.plugins:maven-help-plugin:evaluate -Dexpression=project.version| grep -Ev '(^\[|Download\w+:)'` +RELEASE_VERSION=${VERSION%-SNAPSHOT} + +echo "Building $DOCKER_IMAGE_NAME:$RELEASE_VERSION" +echo "Step 1. Build frontend SPA" +cd $WORK_DIR + +rm -Rf $SOURCE_DIR +rm -Rf $BUILD_DIR +mkdir -p $SOURCE_DIR +mkdir -p $BUILD_DIR + +cp -r $IGNITE_HOME/modules/web-console/frontend/. $SOURCE_DIR + +docker build -f=./DockerfileBuild -t $DOCKER_BUILD_IMAGE_NAME:latest . +docker run -it -v $BUILD_DIR:/opt/web-console-frontend/build --name $DOCKER_BUILD_CONTAINER $DOCKER_BUILD_IMAGE_NAME + +echo "Step 2. Build NGINX container with SPA and proxy configuration" +docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION . + +echo "Step 3. Cleanup" +docker rm -f $DOCKER_BUILD_CONTAINER +docker rmi -f $DOCKER_BUILD_IMAGE_NAME +rm -r $SOURCE_DIR +rm -r $BUILD_DIR diff --git a/modules/web-console/docker/compose/frontend/nginx/nginx.conf b/modules/web-console/docker/compose/frontend/nginx/nginx.conf new file mode 100644 index 0000000000000..dc208f51ed630 --- /dev/null +++ b/modules/web-console/docker/compose/frontend/nginx/nginx.conf @@ -0,0 +1,57 @@ +# +# 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. +# + +user nginx; +worker_processes 1; + +error_log /var/log/nginx/error.log warn; +pid /var/run/nginx.pid; + +events { + worker_connections 128; +} + +http { + server_tokens off; + sendfile on; + aio on; + tcp_nopush on; + + keepalive_timeout 60; + tcp_nodelay on; + + client_max_body_size 100m; + + #access log + log_format main '$http_host $remote_addr - $remote_user [$time_local] ' + '"$request" $status $bytes_sent ' + '"$http_referer" "$http_user_agent" ' + '"$gzip_ratio"'; + + include /etc/nginx/mime.types; + default_type application/octet-stream; + + gzip on; + gzip_disable "msie6"; + gzip_types text/plain text/css text/xml text/javascript application/json application/x-javascript application/xml application/xml+rss application/javascript; + gzip_vary on; + gzip_comp_level 5; + + access_log /var/log/nginx/access.log main; + #conf.d + include web-console.conf; +} diff --git a/modules/web-console/docker/compose/frontend/nginx/web-console.conf b/modules/web-console/docker/compose/frontend/nginx/web-console.conf new file mode 100644 index 0000000000000..d80a7f9a46765 --- /dev/null +++ b/modules/web-console/docker/compose/frontend/nginx/web-console.conf @@ -0,0 +1,59 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +upstream backend-api { + server backend:3000; +} + +server { + listen 80; + server_name _; + + set $ignite_console_dir /data/www; + set $maintenance $ignite_console_dir/maintenance.file; + + root $ignite_console_dir; + + error_page 500 502 503 504 /50x.html; + + location / { + if (-f $maintenance) { + return 503; + } + + try_files $uri /index.html = 404; + } + + location /api/v1 { + rewrite /api/v1/(.*) /$1 break; + proxy_set_header Host $http_host; + proxy_pass http://backend-api; + } + + location /socket.io { + proxy_set_header Upgrade $http_upgrade; + proxy_set_header Connection "upgrade"; + proxy_http_version 1.1; + proxy_set_header X-Forwarded-For $proxy_add_x_forwarded_for; + proxy_set_header Host $host; + proxy_pass http://backend-api; + } + + location = /50x.html { + root $ignite_console_dir/error_page; + } +} diff --git a/modules/web-console/docker/standalone/.dockerignore b/modules/web-console/docker/standalone/.dockerignore new file mode 100644 index 0000000000000..35b7244e3f7af --- /dev/null +++ b/modules/web-console/docker/standalone/.dockerignore @@ -0,0 +1,2 @@ +build/frontend/node_modules +build/backend/node_modules diff --git a/modules/web-console/docker/standalone/Dockerfile b/modules/web-console/docker/standalone/Dockerfile new file mode 100644 index 0000000000000..785f1099d5256 --- /dev/null +++ b/modules/web-console/docker/standalone/Dockerfile @@ -0,0 +1,87 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +FROM ubuntu:14.04 + +ENV NPM_CONFIG_LOGLEVEL info +ENV NODE_VERSION 4.4.7 + +# Before package list update. +RUN set -ex && \ + for key in \ + 9554F04D7259F04124DE6B476D5A82AC7E37093B \ + 94AE36675C464D64BAFA68DD7434390BDBE9B9C5 \ + 0034A06D9D9B0064CE8ADF6BF1747F4AD2306D93 \ + FD3A5288F042B6850C66B31F09FE44734EB7990E \ + 71DCFD284A79C3B38668286BC97EC7A07EDE3FC1 \ + DD8F2338BAE7501E3DD5AC78C273792F7D83545D \ + B9AE9905FFD7803F25714661B63B535A4C206CA9 \ + C4F0DFFF4E8C1A8236409D08E73BC641CC11F4C8 \ + ; do \ + gpg --keyserver ha.pool.sks-keyservers.net --recv-keys "$key"; \ + done + +RUN apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv EA312927 && \ + echo "deb http://repo.mongodb.org/apt/ubuntu trusty/mongodb-org/3.2 multiverse" | sudo tee /etc/apt/sources.list.d/mongodb-org-3.2.list + +# Update package list & install. +RUN apt-get update && \ + apt-get install -y nginx-light mongodb-org-server curl xz-utils + +# Install Node JS. +RUN curl -SLO "https://nodejs.org/dist/v$NODE_VERSION/node-v$NODE_VERSION-linux-x64.tar.xz" && \ + curl -SLO "https://nodejs.org/dist/v$NODE_VERSION/SHASUMS256.txt.asc" && \ + gpg --batch --decrypt --output SHASUMS256.txt SHASUMS256.txt.asc && \ + grep " node-v$NODE_VERSION-linux-x64.tar.xz\$" SHASUMS256.txt | sha256sum -c - && \ + tar -xJf "node-v$NODE_VERSION-linux-x64.tar.xz" -C /usr/local --strip-components=1 && \ + rm "node-v$NODE_VERSION-linux-x64.tar.xz" SHASUMS256.txt.asc SHASUMS256.txt + +# Install global node packages. +RUN npm upgrade -g npm && npm install -g pm2 + +# Install frontend & backend apps. +RUN mkdir -p /opt/web-console + +# Copy source. +WORKDIR /opt/web-console +COPY build . + +# Install node modules. +RUN cd /opt/web-console/frontend && npm install --no-optional && npm run build +RUN cd /opt/web-console/backend && npm install --no-optional + +# Returns to base path. +WORKDIR /opt/web-console + +# Copy nginx config. +COPY ./nginx/nginx.conf /etc/nginx/nginx.conf +COPY ./nginx/web-console.conf /etc/nginx/web-console.conf + +# Setup entrypoint. +COPY ./entrypoint.sh . +RUN chmod 755 /opt/web-console/entrypoint.sh + +# Clean up. +RUN apt-get clean && rm -rf /var/lib/apt/lists/* /tmp/* /var/tmp/* + +VOLUME ["/etc/nginx"] +VOLUME ["/var/lib/mongodb"] +VOLUME ["/opt/web-console/serve/agent_dists"] + +EXPOSE 80 3001 + +ENTRYPOINT ["/opt/web-console/entrypoint.sh"] diff --git a/modules/web-console/docker/standalone/build.sh b/modules/web-console/docker/standalone/build.sh new file mode 100644 index 0000000000000..4365dec3f7535 --- /dev/null +++ b/modules/web-console/docker/standalone/build.sh @@ -0,0 +1,59 @@ +#!/bin/bash +# +# 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. +# + +if [ -z "$IGNITE_HOME" ]; then + echo "Ignite source folder is not found or IGNITE_HOME environment variable is not valid." + + exit 1 +fi + +WORK_DIR=`cd "$(dirname "$0")"; pwd` + +BUILD_DIR="$WORK_DIR/build" + +IGNITE_WEB_CONSOLE_DIR="$IGNITE_HOME/modules/web-console" +DOCKER_IMAGE_NAME="ignite/web-console-standalone" + +echo "Receiving version..." +VERSION=`cd $IGNITE_HOME && mvn org.apache.maven.plugins:maven-help-plugin:evaluate -Dexpression=project.version| grep -Ev '(^\[|Download\w+:)'` +RELEASE_VERSION=${VERSION%-SNAPSHOT} + +echo "Building $DOCKER_IMAGE_NAME:$RELEASE_VERSION" +echo "Step 1. Prepare build temp paths." +cd $WORK_DIR +rm -Rf $BUILD_DIR +docker rmi -f $DOCKER_IMAGE_NAME:$RELEASE_VERSION +mkdir -p $BUILD_DIR/frontend $BUILD_DIR/backend + +echo "Step 2. Build ignite web agent." +cd $IGNITE_HOME +mvn versions:set -DnewVersion=$RELEASE_VERSION -DgenerateBackupPoms=false -Pweb-console -DartifactId='*' +mvn clean package -pl :ignite-web-agent -am -P web-console -DskipTests=true +mvn versions:set -DnewVersion=$VERSION -DgenerateBackupPoms=false -Pweb-console -DartifactId='*' + +echo "Step 3. Copy sources." +cd $WORK_DIR +cp -r $IGNITE_WEB_CONSOLE_DIR/frontend/. $BUILD_DIR/frontend +cp -r $IGNITE_WEB_CONSOLE_DIR/backend/. $BUILD_DIR/backend +cp $IGNITE_HOME/modules/web-console/web-agent/target/ignite-web-agent*.zip $BUILD_DIR/backend/agent_dists/. + +echo "Step 4. Build docker image." +docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION . + +echo "Step 5. Cleanup." +rm -Rf $BUILD_DIR diff --git a/modules/web-console/docker/standalone/docker-compose.yml b/modules/web-console/docker/standalone/docker-compose.yml new file mode 100644 index 0000000000000..561c88d6b9c64 --- /dev/null +++ b/modules/web-console/docker/standalone/docker-compose.yml @@ -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. +# + +webconsole: + image: ignite/web-console-standalone + ports: + - 3080:80 + - 3000:3000 + - 3001:3001 + - 27017:27017 + restart: always + environment: + # Port for serving frontend API + - server_port=3000 + # Cookie session secret + - server_sessionSecret="CHANGE ME" + # URL for mongodb connection + - mongodb_url=mongodb://127.0.0.1/console + # Port for web-agent. + - agentServer_port=3001 + # Mail connection settings. Leave empty if no needed. See also settings, https://github.com/nodemailer/nodemailer + - mail_service="" + - mail_sign="" + - mail_greeting="" + - mail_from="" + - mail_auth_user="" + - mail_auth_pass="" diff --git a/modules/web-console/docker/standalone/entrypoint.sh b/modules/web-console/docker/standalone/entrypoint.sh new file mode 100644 index 0000000000000..3882f06a2ed5e --- /dev/null +++ b/modules/web-console/docker/standalone/entrypoint.sh @@ -0,0 +1,23 @@ +#!/bin/bash +# +# 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. +# + +/usr/bin/mongod --fork --config=/etc/mongod.conf + +service nginx start + +cd backend && pm2 start ./index.js --no-daemon diff --git a/modules/web-console/docker/standalone/nginx/nginx.conf b/modules/web-console/docker/standalone/nginx/nginx.conf new file mode 100644 index 0000000000000..dbc79d7d77bf8 --- /dev/null +++ b/modules/web-console/docker/standalone/nginx/nginx.conf @@ -0,0 +1,55 @@ +# +# 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. +# + +user www-data; +worker_processes 1; + +error_log /var/log/nginx/error.log warn; +pid /var/run/nginx.pid; + +events { + worker_connections 128; +} + +http { + server_tokens off; + sendfile on; + tcp_nopush on; + + keepalive_timeout 60; + tcp_nodelay on; + + client_max_body_size 100m; + + #access log + log_format main '$http_host $remote_addr - $remote_user [$time_local] ' + '"$request" $status $bytes_sent ' + '"$http_referer" "$http_user_agent" ' + '"$gzip_ratio"'; + + include /etc/nginx/mime.types; + default_type application/octet-stream; + gzip on; + gzip_disable "msie6"; + gzip_types text/plain text/css text/xml text/javascript application/json application/x-javascript application/xml application/xml+rss application/javascript; + gzip_vary on; + gzip_comp_level 5; + + access_log /var/log/nginx/access.log main; + #conf.d + include web-console.conf ; +} diff --git a/modules/web-console/docker/standalone/nginx/web-console.conf b/modules/web-console/docker/standalone/nginx/web-console.conf new file mode 100644 index 0000000000000..3de544f44f422 --- /dev/null +++ b/modules/web-console/docker/standalone/nginx/web-console.conf @@ -0,0 +1,54 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +upstream backend-api { + server localhost:3000; +} + +server { + listen 80; + server_name _; + + set $ignite_console_dir /opt/web-console/frontend/build; + + root $ignite_console_dir; + + error_page 500 502 503 504 /50x.html; + + location / { + try_files $uri /index.html = 404; + } + + location /api/v1 { + rewrite /api/v1/(.*) /$1 break; + proxy_set_header Host $http_host; + proxy_pass http://backend-api; + } + + location /socket.io { + proxy_set_header Upgrade $http_upgrade; + proxy_set_header Connection "upgrade"; + proxy_http_version 1.1; + proxy_set_header X-Forwarded-For $proxy_add_x_forwarded_for; + proxy_set_header Host $host; + proxy_pass http://backend-api; + } + + location = /50x.html { + root $ignite_console_dir/error_page; + } +} diff --git a/modules/web-console/frontend/.babelrc b/modules/web-console/frontend/.babelrc new file mode 100644 index 0000000000000..a1d48c1f3597f --- /dev/null +++ b/modules/web-console/frontend/.babelrc @@ -0,0 +1,9 @@ +{ + "presets": ["angular"], + "plugins": [[ + "transform-builtin-extend", { + "globals": ["Error", "Array"], + "approximate": true + } + ]] +} diff --git a/modules/web-console/src/main/js/.eslintrc b/modules/web-console/frontend/.eslintrc similarity index 99% rename from modules/web-console/src/main/js/.eslintrc rename to modules/web-console/frontend/.eslintrc index 1882b447ba014..a8a3bb8ff7fa3 100644 --- a/modules/web-console/src/main/js/.eslintrc +++ b/modules/web-console/frontend/.eslintrc @@ -46,7 +46,7 @@ rules: comma-dangle: [2, "never"] comma-spacing: [2, {"before": false, "after": true}] comma-style: [2, "last"] - complexity: [1, 11] + complexity: [1, 40] computed-property-spacing: [2, "never"] consistent-return: 0 consistent-this: [0, "that"] @@ -138,7 +138,7 @@ rules: no-path-concat: 0 no-plusplus: 0 no-process-env: 0 - no-process-exit: 2 + no-process-exit: 1 no-proto: 2 no-redeclare: 2 no-regex-spaces: 1 diff --git a/modules/web-console/src/main/js/.gitignore b/modules/web-console/frontend/.gitignore similarity index 66% rename from modules/web-console/src/main/js/.gitignore rename to modules/web-console/frontend/.gitignore index a6af5e16e822a..46bca1383a528 100644 --- a/modules/web-console/src/main/js/.gitignore +++ b/modules/web-console/frontend/.gitignore @@ -5,5 +5,3 @@ build/* node_modules ignite_modules_temp/* public/stylesheets/*.css -serve/config/*.json -serve/agent_dists/*.zip diff --git a/modules/web-console/src/main/js/app/app.config.js b/modules/web-console/frontend/app/app.config.js similarity index 100% rename from modules/web-console/src/main/js/app/app.config.js rename to modules/web-console/frontend/app/app.config.js diff --git a/modules/web-console/src/main/js/app/app.js b/modules/web-console/frontend/app/app.js similarity index 91% rename from modules/web-console/src/main/js/app/app.js rename to modules/web-console/frontend/app/app.js index 00ae3eb5ea480..45851a25cfe57 100644 --- a/modules/web-console/src/main/js/app/app.js +++ b/modules/web-console/frontend/app/app.js @@ -16,6 +16,7 @@ */ import '../public/stylesheets/style.scss'; +import './helpers/jade/mixins.jade'; import './app.config'; @@ -24,16 +25,16 @@ import './decorator/tooltip'; import './modules/form/form.module'; import './modules/agent/agent.module.js'; -import './modules/query-notebooks/query-notebooks.module'; +import './modules/sql/sql.module'; import './modules/Demo/Demo.module.js'; import './modules/states/signin.state'; import './modules/states/logout.state'; import './modules/states/password.state'; import './modules/states/configuration.state'; -import './modules/states/sql.state'; import './modules/states/profile.state'; import './modules/states/admin.state'; +import './modules/states/errors.state'; // ignite:modules import './modules/user/user.module'; @@ -42,7 +43,7 @@ import './modules/navbar/navbar.module'; import './modules/configuration/configuration.module'; import './modules/getting-started/GettingStarted.provider'; import './modules/dialog/dialog.module'; -import './modules/Version/Version.provider'; +import './modules/version/Version.provider'; import './modules/ace.module'; import './modules/socket.module'; import './modules/loading/loading.module'; @@ -80,6 +81,8 @@ import JavaTypes from './services/JavaTypes.service'; import Messages from './services/Messages.service'; import ModelNormalizer from './services/ModelNormalizer.service.js'; import LegacyTable from './services/LegacyTable.service'; +import ErrorPopover from './services/ErrorPopover.service'; +import FormUtils from './services/FormUtils.service'; import LegacyUtils from './services/LegacyUtils.service'; import UnsavedChangesGuard from './services/UnsavedChangesGuard.service'; @@ -89,6 +92,7 @@ import UnsavedChangesGuard from './services/UnsavedChangesGuard.service'; import byName from './filters/byName.filter'; import domainsValidation from './filters/domainsValidation.filter'; import hasPojo from './filters/hasPojo.filter'; +import duration from './filters/duration.filter'; // Generators import $generatorCommon from 'generator/generator-common'; @@ -112,9 +116,7 @@ import clusters from 'controllers/clusters-controller'; import domains from 'controllers/domains-controller'; import igfs from 'controllers/igfs-controller'; import profile from 'controllers/profile-controller'; -import sql from 'controllers/sql-controller'; import auth from './controllers/auth.controller'; -import notebooks from './controllers/notebooks.controller'; import resetPassword from './controllers/reset-password.controller'; // Inject external modules. @@ -150,16 +152,16 @@ angular 'ignite-console.branding', 'ignite-console.socket', 'ignite-console.agent', - 'ignite-console.query-notebooks', + 'ignite-console.sql', 'ignite-console.demo', // States. 'ignite-console.states.login', 'ignite-console.states.logout', 'ignite-console.states.password', 'ignite-console.states.configuration', - 'ignite-console.states.sql', 'ignite-console.states.profile', 'ignite-console.states.admin', + 'ignite-console.states.errors', // Common modules. 'ignite-console.dialog', 'ignite-console.navbar', @@ -203,23 +205,24 @@ angular .service(...Messages) .service(...ModelNormalizer) .service(...LegacyTable) +.service('IgniteErrorPopover', ErrorPopover) +.service(...FormUtils) .service(...LegacyUtils) .service(...UnsavedChangesGuard) // Controllers. .controller(...admin) .controller(...auth) -.controller(...notebooks) .controller(...resetPassword) .controller(...caches) .controller(...clusters) .controller(...domains) .controller(...igfs) .controller(...profile) -.controller(...sql) // Filters. .filter(...hasPojo) .filter(...domainsValidation) .filter(...byName) +.filter(...duration) .config(['$stateProvider', '$locationProvider', '$urlRouterProvider', ($stateProvider, $locationProvider, $urlRouterProvider) => { // Set up the states. $stateProvider @@ -234,8 +237,7 @@ angular templateUrl: baseTemplate }); - $urlRouterProvider.otherwise('/'); - + $urlRouterProvider.otherwise('/404'); $locationProvider.html5Mode(true); }]) .run(['$rootScope', '$state', 'MetaTags', 'gettingStarted', ($root, $state, $meta, gettingStarted) => { @@ -244,12 +246,8 @@ angular $root.$meta = $meta; $root.gettingStarted = gettingStarted; }]) -.run(['$rootScope', 'Auth', 'User', 'IgniteAgentMonitor', ($root, Auth, User, agentMonitor) => { - if (Auth.authorized) { - User.read() - .then((user) => $root.$broadcast('user', user)) - .then(() => Auth.authorized && agentMonitor.init()); - } +.run(['$rootScope', 'IgniteAgentMonitor', ($root, agentMonitor) => { + $root.$on('user', () => agentMonitor.init()); }]) .run(['$rootScope', ($root) => { $root.$on('$stateChangeStart', () => { @@ -259,9 +257,8 @@ angular .run(['$rootScope', '$http', '$state', 'IgniteMessages', 'User', ($root, $http, $state, Messages, User) => { // eslint-disable-line no-shadow $root.revertIdentity = () => { - $http - .get('/api/v1/admin/revert/identity') - .then(User.read) + $http.get('/api/v1/admin/revert/identity') + .then(User.load) .then((user) => { $root.$broadcast('user', user); @@ -271,4 +268,3 @@ angular }; } ]); - diff --git a/modules/web-console/src/main/js/app/controllers/auth.controller.js b/modules/web-console/frontend/app/controllers/auth.controller.js similarity index 100% rename from modules/web-console/src/main/js/app/controllers/auth.controller.js rename to modules/web-console/frontend/app/controllers/auth.controller.js diff --git a/modules/web-console/src/main/js/app/controllers/reset-password.controller.js b/modules/web-console/frontend/app/controllers/reset-password.controller.js similarity index 93% rename from modules/web-console/src/main/js/app/controllers/reset-password.controller.js rename to modules/web-console/frontend/app/controllers/reset-password.controller.js index f3cee81dc2995..da0c37b05c428 100644 --- a/modules/web-console/src/main/js/app/controllers/reset-password.controller.js +++ b/modules/web-console/frontend/app/controllers/reset-password.controller.js @@ -35,17 +35,16 @@ export default ['resetPassword', [ $scope.resetPassword = (reset_info) => { $http.post('/api/v1/password/reset', reset_info) .success(() => { - Messages.showInfo('Password successfully changed'); + $state.go('signin'); - $state.go('base.configuration.clusters'); + Messages.showInfo('Password successfully changed'); }) .error((err, state) => { - Messages.showError(err); - if (state === 503) - $state.go('base.configuration.clusters'); + $state.go('signin'); + + Messages.showError(err); }); }; - } ]]; diff --git a/modules/web-console/src/main/js/app/data/colors.json b/modules/web-console/frontend/app/data/colors.json similarity index 100% rename from modules/web-console/src/main/js/app/data/colors.json rename to modules/web-console/frontend/app/data/colors.json diff --git a/modules/web-console/src/main/js/app/data/countries.json b/modules/web-console/frontend/app/data/countries.json similarity index 100% rename from modules/web-console/src/main/js/app/data/countries.json rename to modules/web-console/frontend/app/data/countries.json diff --git a/modules/web-console/src/main/js/app/data/demo-info.json b/modules/web-console/frontend/app/data/demo-info.json similarity index 100% rename from modules/web-console/src/main/js/app/data/demo-info.json rename to modules/web-console/frontend/app/data/demo-info.json diff --git a/modules/web-console/src/main/js/app/data/event-types.json b/modules/web-console/frontend/app/data/event-types.json similarity index 100% rename from modules/web-console/src/main/js/app/data/event-types.json rename to modules/web-console/frontend/app/data/event-types.json diff --git a/modules/web-console/src/main/js/app/data/getting-started.json b/modules/web-console/frontend/app/data/getting-started.json similarity index 100% rename from modules/web-console/src/main/js/app/data/getting-started.json rename to modules/web-console/frontend/app/data/getting-started.json diff --git a/modules/web-console/src/main/js/app/data/java-classes.json b/modules/web-console/frontend/app/data/java-classes.json similarity index 93% rename from modules/web-console/src/main/js/app/data/java-classes.json rename to modules/web-console/frontend/app/data/java-classes.json index e2cdff90f586b..b0ec9fb4b7559 100644 --- a/modules/web-console/src/main/js/app/data/java-classes.json +++ b/modules/web-console/frontend/app/data/java-classes.json @@ -4,6 +4,7 @@ {"short": "Byte", "full": "java.lang.Byte"}, {"short": "Character", "full": "java.lang.Character"}, {"short": "Date", "full": "java.sql.Date"}, + {"short": "java.util.Date", "full": "java.util.Date"}, {"short": "Double", "full": "java.lang.Double"}, {"short": "Float", "full": "java.lang.Float"}, {"short": "Integer", "full": "java.lang.Integer"}, diff --git a/modules/web-console/src/main/js/app/data/java-keywords.json b/modules/web-console/frontend/app/data/java-keywords.json similarity index 100% rename from modules/web-console/src/main/js/app/data/java-keywords.json rename to modules/web-console/frontend/app/data/java-keywords.json diff --git a/modules/web-console/src/main/js/app/data/java-primitives.json b/modules/web-console/frontend/app/data/java-primitives.json similarity index 100% rename from modules/web-console/src/main/js/app/data/java-primitives.json rename to modules/web-console/frontend/app/data/java-primitives.json diff --git a/modules/web-console/src/main/js/app/data/pom-dependencies.json b/modules/web-console/frontend/app/data/pom-dependencies.json similarity index 100% rename from modules/web-console/src/main/js/app/data/pom-dependencies.json rename to modules/web-console/frontend/app/data/pom-dependencies.json diff --git a/modules/web-console/src/main/js/app/decorator/select.js b/modules/web-console/frontend/app/decorator/select.js similarity index 100% rename from modules/web-console/src/main/js/app/decorator/select.js rename to modules/web-console/frontend/app/decorator/select.js diff --git a/modules/web-console/src/main/js/app/decorator/tooltip.js b/modules/web-console/frontend/app/decorator/tooltip.js similarity index 100% rename from modules/web-console/src/main/js/app/decorator/tooltip.js rename to modules/web-console/frontend/app/decorator/tooltip.js diff --git a/modules/web-console/src/main/js/app/directives/auto-focus.directive.js b/modules/web-console/frontend/app/directives/auto-focus.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/auto-focus.directive.js rename to modules/web-console/frontend/app/directives/auto-focus.directive.js diff --git a/modules/web-console/src/main/js/app/directives/bs-affix-update.directive.js b/modules/web-console/frontend/app/directives/bs-affix-update.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/bs-affix-update.directive.js rename to modules/web-console/frontend/app/directives/bs-affix-update.directive.js diff --git a/modules/web-console/src/main/js/app/directives/centered/centered.css b/modules/web-console/frontend/app/directives/centered/centered.css similarity index 100% rename from modules/web-console/src/main/js/app/directives/centered/centered.css rename to modules/web-console/frontend/app/directives/centered/centered.css diff --git a/modules/web-console/src/main/js/app/directives/centered/centered.directive.js b/modules/web-console/frontend/app/directives/centered/centered.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/centered/centered.directive.js rename to modules/web-console/frontend/app/directives/centered/centered.directive.js diff --git a/modules/web-console/src/main/js/app/directives/copy-to-clipboard.directive.js b/modules/web-console/frontend/app/directives/copy-to-clipboard.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/copy-to-clipboard.directive.js rename to modules/web-console/frontend/app/directives/copy-to-clipboard.directive.js diff --git a/modules/web-console/src/main/js/app/directives/hide-on-state-change/hide-on-state-change.directive.js b/modules/web-console/frontend/app/directives/hide-on-state-change/hide-on-state-change.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/hide-on-state-change/hide-on-state-change.directive.js rename to modules/web-console/frontend/app/directives/hide-on-state-change/hide-on-state-change.directive.js diff --git a/modules/web-console/src/main/js/app/directives/information/information.directive.js b/modules/web-console/frontend/app/directives/information/information.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/information/information.directive.js rename to modules/web-console/frontend/app/directives/information/information.directive.js diff --git a/modules/web-console/src/main/js/app/directives/information/information.jade b/modules/web-console/frontend/app/directives/information/information.jade similarity index 100% rename from modules/web-console/src/main/js/app/directives/information/information.jade rename to modules/web-console/frontend/app/directives/information/information.jade diff --git a/modules/web-console/src/main/js/app/directives/information/information.scss b/modules/web-console/frontend/app/directives/information/information.scss similarity index 100% rename from modules/web-console/src/main/js/app/directives/information/information.scss rename to modules/web-console/frontend/app/directives/information/information.scss diff --git a/modules/web-console/src/main/js/app/directives/match.directive.js b/modules/web-console/frontend/app/directives/match.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/match.directive.js rename to modules/web-console/frontend/app/directives/match.directive.js diff --git a/modules/web-console/src/main/js/app/directives/on-click-focus.directive.js b/modules/web-console/frontend/app/directives/on-click-focus.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/on-click-focus.directive.js rename to modules/web-console/frontend/app/directives/on-click-focus.directive.js diff --git a/modules/web-console/src/main/js/app/directives/on-enter-focus-move.directive.js b/modules/web-console/frontend/app/directives/on-enter-focus-move.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/on-enter-focus-move.directive.js rename to modules/web-console/frontend/app/directives/on-enter-focus-move.directive.js diff --git a/modules/web-console/src/main/js/app/directives/on-enter.directive.js b/modules/web-console/frontend/app/directives/on-enter.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/on-enter.directive.js rename to modules/web-console/frontend/app/directives/on-enter.directive.js diff --git a/modules/web-console/src/main/js/app/directives/on-escape.directive.js b/modules/web-console/frontend/app/directives/on-escape.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/on-escape.directive.js rename to modules/web-console/frontend/app/directives/on-escape.directive.js diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-docker/ui-ace-docker.controller.js b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/ui-ace-docker/ui-ace-docker.controller.js rename to modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-docker/ui-ace-docker.directive.js b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/ui-ace-docker/ui-ace-docker.directive.js rename to modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.directive.js diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-docker/ui-ace-docker.jade b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade similarity index 100% rename from modules/web-console/src/main/js/app/directives/ui-ace-docker/ui-ace-docker.jade rename to modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-java/ui-ace-java.controller.js b/modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.controller.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/ui-ace-java/ui-ace-java.controller.js rename to modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.controller.js diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-java/ui-ace-java.directive.js b/modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.directive.js similarity index 88% rename from modules/web-console/src/main/js/app/directives/ui-ace-java/ui-ace-java.directive.js rename to modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.directive.js index 17d3a004e74a9..fbb143135899c 100644 --- a/modules/web-console/src/main/js/app/directives/ui-ace-java/ui-ace-java.directive.js +++ b/modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.directive.js @@ -77,6 +77,7 @@ export default ['igniteUiAceJava', ['GeneratorJava', (generator) => { break; case 'cacheStore': + case 'cacheQuery': ctrl.generator = (cache) => { const domains = _.reduce(scope.detail, (acc, domain) => { if (_.includes(cache.domains, domain.value)) @@ -85,7 +86,20 @@ export default ['igniteUiAceJava', ['GeneratorJava', (generator) => { return acc; }, []); - return generator.cacheStore(cache, domains).asString(); + return generator[method](cache, domains).asString(); + }; + + break; + + case 'cacheNodeFilter': + ctrl.generator = (cache) => { + const igfss = _.reduce(scope.detail, (acc, igfs) => { + acc.push(igfs.igfs); + + return acc; + }, []); + + return generator.cacheNodeFilter(cache, igfss).asString(); }; break; diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-java/ui-ace-java.jade b/modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.jade similarity index 100% rename from modules/web-console/src/main/js/app/directives/ui-ace-java/ui-ace-java.jade rename to modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.jade diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js rename to modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-pojos/ui-ace-pojos.directive.js b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/ui-ace-pojos/ui-ace-pojos.directive.js rename to modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.directive.js diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-pojos/ui-ace-pojos.jade b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.jade similarity index 100% rename from modules/web-console/src/main/js/app/directives/ui-ace-pojos/ui-ace-pojos.jade rename to modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.jade diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-pom/ui-ace-pom.controller.js b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/ui-ace-pom/ui-ace-pom.controller.js rename to modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-pom/ui-ace-pom.directive.js b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/ui-ace-pom/ui-ace-pom.directive.js rename to modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.directive.js diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-pom/ui-ace-pom.jade b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.jade similarity index 100% rename from modules/web-console/src/main/js/app/directives/ui-ace-pom/ui-ace-pom.jade rename to modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.jade diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-tabs.directive.js b/modules/web-console/frontend/app/directives/ui-ace-tabs.directive.js similarity index 97% rename from modules/web-console/src/main/js/app/directives/ui-ace-tabs.directive.js rename to modules/web-console/frontend/app/directives/ui-ace-tabs.directive.js index a9649c8bba7ea..2b90a72ebddc8 100644 --- a/modules/web-console/src/main/js/app/directives/ui-ace-tabs.directive.js +++ b/modules/web-console/frontend/app/directives/ui-ace-tabs.directive.js @@ -17,6 +17,7 @@ export default ['igniteUiAceTabs', [() => { return { + scope: true, restrict: 'AE', controller: _.noop }; diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-xml/ui-ace-xml.controller.js b/modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.controller.js similarity index 100% rename from modules/web-console/src/main/js/app/directives/ui-ace-xml/ui-ace-xml.controller.js rename to modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.controller.js diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-xml/ui-ace-xml.directive.js b/modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.directive.js similarity index 88% rename from modules/web-console/src/main/js/app/directives/ui-ace-xml/ui-ace-xml.directive.js rename to modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.directive.js index 3bae4de4965ea..3bd834f14366a 100644 --- a/modules/web-console/src/main/js/app/directives/ui-ace-xml/ui-ace-xml.directive.js +++ b/modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.directive.js @@ -77,6 +77,7 @@ export default ['igniteUiAceXml', ['GeneratorXml', (generator) => { break; case 'cacheStore': + case 'cacheQuery': ctrl.generator = (cache) => { const domains = _.reduce(scope.detail, (acc, domain) => { if (_.includes(cache.domains, domain.value)) @@ -85,7 +86,20 @@ export default ['igniteUiAceXml', ['GeneratorXml', (generator) => { return acc; }, []); - return generator.cacheStore(cache, domains).asString(); + return generator[method](cache, domains).asString(); + }; + + break; + + case 'cacheNodeFilter': + ctrl.generator = (cache) => { + const igfss = _.reduce(scope.detail, (acc, igfs) => { + acc.push(igfs.igfs); + + return acc; + }, []); + + return generator.cacheNodeFilter(cache, igfss).asString(); }; break; diff --git a/modules/web-console/src/main/js/app/directives/ui-ace-xml/ui-ace-xml.jade b/modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.jade similarity index 100% rename from modules/web-console/src/main/js/app/directives/ui-ace-xml/ui-ace-xml.jade rename to modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.jade diff --git a/modules/web-console/src/main/js/app/filters/byName.filter.js b/modules/web-console/frontend/app/filters/byName.filter.js similarity index 100% rename from modules/web-console/src/main/js/app/filters/byName.filter.js rename to modules/web-console/frontend/app/filters/byName.filter.js diff --git a/modules/web-console/src/main/js/app/filters/domainsValidation.filter.js b/modules/web-console/frontend/app/filters/domainsValidation.filter.js similarity index 100% rename from modules/web-console/src/main/js/app/filters/domainsValidation.filter.js rename to modules/web-console/frontend/app/filters/domainsValidation.filter.js diff --git a/modules/web-console/frontend/app/filters/duration.filter.js b/modules/web-console/frontend/app/filters/duration.filter.js new file mode 100644 index 0000000000000..deeedd729d459 --- /dev/null +++ b/modules/web-console/frontend/app/filters/duration.filter.js @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +export default ['duration', [() => { + /** + * @param {Number} t Time in ms. + */ + return (t) => { + const a = (i, suffix) => i && i !== '00' ? i + suffix + ' ' : ''; + + const cd = 24 * 60 * 60 * 1000; + const ch = 60 * 60 * 1000; + const cm = 60 * 1000; + const cs = 1000; + + const d = Math.floor(t / cd); + const h = Math.floor((t - d * cd) / ch); + const m = Math.floor((t - d * cd - h * ch) / cm); + const s = Math.floor((t - d * cd - h * ch - m * cm) / cs); + const ms = t % 1000; + + return a(d, 'd') + a(h, 'h') + a(m, 'm') + a(s, 's') + (t < cm ? ms + 'ms' : ''); + }; +}]]; diff --git a/modules/web-console/src/main/js/app/filters/hasPojo.filter.js b/modules/web-console/frontend/app/filters/hasPojo.filter.js similarity index 100% rename from modules/web-console/src/main/js/app/filters/hasPojo.filter.js rename to modules/web-console/frontend/app/filters/hasPojo.filter.js diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/jdbc.jade b/modules/web-console/frontend/app/helpers/jade/form.jade similarity index 68% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/jdbc.jade rename to modules/web-console/frontend/app/helpers/jade/form.jade index fbca508a12b4e..4b017ea46133a 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/jdbc.jade +++ b/modules/web-console/frontend/app/helpers/jade/form.jade @@ -14,11 +14,14 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../app/helpers/jade/mixins.jade +include ./form/form-field-feedback.jade +include ./form/form-field-label.jade +include ./form/form-field-text.jade +include ./form/form-field-dropdown.jade +include ./form/form-field-datalist.jade +include ./form/form-field-checkbox.jade +include ./form/form-field-number.jade +include ./form/form-field-up.jade +include ./form/form-field-down.jade --var model = 'backupItem.discovery.Jdbc' - -div - .details-row - +checkbox('DB schema should be initialized by Ignite', model + '.initSchema', 'initSchema', - 'Flag indicating whether DB schema should be initialized by Ignite or was explicitly created by user') +include ./form/form-group.jade diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-checkbox.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-checkbox.jade new file mode 100644 index 0000000000000..ef5cb37fa8222 --- /dev/null +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-checkbox.jade @@ -0,0 +1,38 @@ +//- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +mixin form-field-checkbox(label, model, name, disabled, required, tip) + div.checkbox.col-xs-12.col-sm-12.col-md-12 + label(id='{{ #{name} }}Label') + .input-tip + if block + block + else + input( + id='{{ #{name} }}Input' + name='{{ #{name} }}' + type='checkbox' + + data-ng-model=model + data-ng-required=required && '#{required}' + data-ng-disabled=disabled && '#{disabled}' + + data-ng-focus='tableReset()' + + data-ignite-form-panel-field='' + ) + span #{label} + i.tipLabel.fa.fa-question-circle(bs-tooltip='' data-title=tip) diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade new file mode 100644 index 0000000000000..98a0b9a02d5e7 --- /dev/null +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade @@ -0,0 +1,51 @@ +//- + 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. + +mixin form-field-datalist(label, model, name, disabled, required, placeholder, options, tip) + -var errLbl = label.substring(0, label.length - 1) + + mixin form-field-input() + input.form-control( + id='{{ #{name} }}Input' + name='{{ #{name} }}' + placeholder=placeholder + + data-ng-model=model + + data-ng-required=required && '#{required}' + data-ng-disabled=disabled && '#{disabled}' || '!#{options}.length' + + bs-typeahead + bs-options='item for item in #{options}' + container='body' + data-min-length='1' + ignite-retain-selection + + data-ignite-form-panel-field='' + )&attributes(attributes.attributes) + + div + +ignite-form-field-label(label, name, required) + .col-xs-8.col-sm-8.col-md-8 + i.tipField.fa.fa-question-circle(bs-tooltip data-title=tip) + + +form-field-feedback(name, 'required', errLbl + ' could not be empty!') + + if block + block + + .input-tip + +form-field-input(attributes=attributes) diff --git a/modules/web-console/src/main/js/app/modules/form/group/group.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-down.jade similarity index 85% rename from modules/web-console/src/main/js/app/modules/form/group/group.jade rename to modules/web-console/frontend/app/helpers/jade/form/form-field-down.jade index ba3a8f27d5254..cd10ebee1f889 100644 --- a/modules/web-console/src/main/js/app/modules/form/group/group.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-down.jade @@ -14,8 +14,5 @@ See the License for the specific language governing permissions and limitations under the License. -.group-section - .group - .group-legend - label {{::group.label}} - div(ng-transclude='') +mixin ignite-form-field-down() + i.tipField.fa.fa-arrow-down(ignite-form-field-down ng-click="$ctrl.down()")&attributes(attributes) diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade new file mode 100644 index 0000000000000..a8496bc7698d8 --- /dev/null +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade @@ -0,0 +1,50 @@ +//- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +mixin ignite-form-field-dropdown(label, model, name, disabled, required, multiple, placeholder, placeholderEmpty, options, tip) + mixin form-field-input() + button.select-toggle.form-control( + id='{{ #{name} }}Input' + name='{{ #{name} }}' + + data-placeholder=placeholderEmpty ? '{{ #{options}.length > 0 ? "#{placeholder}" : "#{placeholderEmpty}" }}' : placeholder + + data-ng-model=model + + data-ng-required=required && '#{required}' + data-ng-disabled=disabled && '#{disabled}' || '!#{options}.length' + + bs-select + bs-options='item.value as item.label for item in #{options}' + + data-multiple=multiple ? '1' : false + data-container='body > .wrapper' + + tabindex='0' + + data-ignite-form-panel-field='' + )&attributes(attributes.attributes) + + div + +ignite-form-field-label(label, name, required) + .col-xs-8.col-sm-8.col-md-8 + i.tipField.fa.fa-question-circle(bs-tooltip data-title=tip) + + if block + block + + .input-tip + +form-field-input(attributes=attributes) diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-feedback.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-feedback.jade new file mode 100644 index 0000000000000..bf012db9f7504 --- /dev/null +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-feedback.jade @@ -0,0 +1,29 @@ +//- + 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. + +mixin form-field-feedback(name, error, message) + -var __field = form + '[' + name + ']' + -var __error = __field + '.$error.' + error + -var __pristine = __field + '.$pristine' + + i.fa.fa-exclamation-triangle.form-field-feedback( + ng-if='!#{__pristine} && #{__error}' + bs-tooltip + data-title=message + ignite-error=error + ignite-error-message=message + name='{{ #{name} }}' + ) diff --git a/modules/web-console/src/main/js/app/modules/form/field/input/checkbox.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-label.jade similarity index 74% rename from modules/web-console/src/main/js/app/modules/form/field/input/checkbox.jade rename to modules/web-console/frontend/app/helpers/jade/form/form-field-label.jade index 477d4b2eb9440..cd5f8fca14e0e 100644 --- a/modules/web-console/src/main/js/app/modules/form/field/input/checkbox.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-label.jade @@ -14,17 +14,10 @@ See the License for the specific language governing permissions and limitations under the License. -.input-tip - input( - id='{{ id }}' - name='{{ name }}' - type='checkbox' - - data-ng-model='value' - data-ng-required='required || false' - data-ng-disabled='disabled || false' - - data-ng-focus='tableReset()' +mixin ignite-form-field-label(label, name, required) + label.col-xs-4.col-sm-4.col-md-4( + id='{{ #{name} }}Label' + for='{{ #{name} }}Input' + class="{{ #{required} ? 'required' : '' }}" ) - - span(ng-transclude='') + span !{label} diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade new file mode 100644 index 0000000000000..c32d3d9f02cb7 --- /dev/null +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade @@ -0,0 +1,52 @@ +//- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +mixin ignite-form-field-number(label, model, name, disabled, required, placeholder, min, max, step, tip) + mixin form-field-input() + input.form-control( + id='{{ #{name} }}Input' + name='{{ #{name} }}' + placeholder=placeholder + type='number' + + min=min ? min : '0' + max=max ? max : '{{ Number.MAX_VALUE }}' + step=step ? step : '1' + + data-ng-model=model + + data-ng-required=required && '#{required}' + data-ng-disabled=disabled && '#{disabled}' + data-ng-focus='tableReset()' + + data-ignite-form-panel-field='' + )&attributes(attributes.attributes) + + div + +ignite-form-field-label(label, name, required) + .col-xs-8.col-sm-8.col-md-8 + i.tipField.fa.fa-question-circle(bs-tooltip data-title=tip) + + +form-field-feedback(name, 'required', 'This field could not be empty') + +form-field-feedback(name, 'min', 'Value is less than allowable minimum: '+ min || 0) + +form-field-feedback(name, 'max', 'Value is more than allowable maximum: '+ max) + +form-field-feedback(name, 'number', 'Only numbers allowed') + + if block + block + + .input-tip + +form-field-input(attributes=attributes) diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade new file mode 100644 index 0000000000000..796e641de1966 --- /dev/null +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade @@ -0,0 +1,47 @@ +//- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +mixin ignite-form-field-input(name, model, disabled, required, placeholder) + input.form-control( + id='{{ #{name} }}Input' + name='{{ #{name} }}' + placeholder=placeholder + type='text' + + data-ng-model=model + + data-ng-required=required && '#{required}' + data-ng-disabled=disabled && '#{disabled}' + data-ng-focus='tableReset()' + + data-ignite-form-panel-field='' + )&attributes(attributes ? attributes.attributes ? attributes.attributes : attributes : {}) + +mixin ignite-form-field-text(label, model, name, disabled, required, placeholder, tip) + -var errLbl = label.substring(0, label.length - 1) + + div + +ignite-form-field-label(label, name, required) + .col-xs-8.col-sm-8.col-md-8 + i.tipField.fa.fa-question-circle(bs-tooltip data-title=tip) + + if block + block + + +form-field-feedback(name, 'required', errLbl + ' could not be empty!') + + .input-tip + +ignite-form-field-input(name, model, disabled, required, placeholder)(attributes=attributes) diff --git a/modules/web-console/src/main/js/app/modules/form/group/table.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-up.jade similarity index 85% rename from modules/web-console/src/main/js/app/modules/form/group/table.jade rename to modules/web-console/frontend/app/helpers/jade/form/form-field-up.jade index 6f9486d4efd64..c66cd0e6021c4 100644 --- a/modules/web-console/src/main/js/app/modules/form/group/table.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-up.jade @@ -14,4 +14,5 @@ See the License for the specific language governing permissions and limitations under the License. -div \ No newline at end of file +mixin ignite-form-field-up() + i.tipField.fa.fa-arrow-up.ng-scope(ignite-form-field-up ng-click="$ctrl.up()")&attributes(attributes) diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-group.jade b/modules/web-console/frontend/app/helpers/jade/form/form-group.jade new file mode 100644 index 0000000000000..8fb7b1f3e5e9f --- /dev/null +++ b/modules/web-console/frontend/app/helpers/jade/form/form-group.jade @@ -0,0 +1,23 @@ +//- + 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. + +mixin ignite-form-group() + .group-section(ignite-form-group)&attributes(attributes) + .group(ng-if='true' ng-init='group = {}') + .group-legend + label {{::group.label}} + if block + block diff --git a/modules/web-console/src/main/js/app/helpers/jade/mixins.jade b/modules/web-console/frontend/app/helpers/jade/mixins.jade similarity index 60% rename from modules/web-console/src/main/js/app/helpers/jade/mixins.jade rename to modules/web-console/frontend/app/helpers/jade/mixins.jade index 08ebe877cf36e..c37ab157c28ce 100644 --- a/modules/web-console/src/main/js/app/helpers/jade/mixins.jade +++ b/modules/web-console/frontend/app/helpers/jade/mixins.jade @@ -14,10 +14,12 @@ See the License for the specific language governing permissions and limitations under the License. +include ./form.jade + //- Mixin for advanced options toggle. mixin advanced-options-toggle(click, cond, showMessage, hideMessage) .advanced-options - i.fa(ng-click=click ng-class='#{cond} ? "fa-chevron-circle-down" : "fa-chevron-circle-right"') + i.fa(ng-click='#{click}' ng-class='#{cond} ? "fa-chevron-circle-down" : "fa-chevron-circle-right"') a(ng-click=click) {{#{cond} ? '#{hideMessage}' : '#{showMessage}'}} //- Mixin for advanced options toggle with default settings. @@ -76,274 +78,170 @@ mixin error-feedback(visible, error, errorMessage, name) ) //- Mixin for feedback on unique violation. -mixin unique-feedback(visible, errorMessage) - +error-feedback(visible, 'igniteUnique', errorMessage) +mixin unique-feedback(name, errorMessage) + +form-field-feedback(name, 'igniteUnique', errorMessage) //- Mixin for feedback on IP address violation. -mixin ipaddress-feedback(visible, name) - +error-feedback(visible, 'ipaddress', 'Invalid address!', name) +mixin ipaddress-feedback(name) + +form-field-feedback(name, 'ipaddress', 'Invalid address!') //- Mixin for feedback on port of IP address violation. -mixin ipaddress-port-feedback(visible, name) - +error-feedback(visible, 'ipaddressPort', 'Invalid port!', name) +mixin ipaddress-port-feedback(name) + +form-field-feedback(name, 'ipaddressPort', 'Invalid port!') //- Mixin for feedback on port range violation. -mixin ipaddress-port-range-feedback(visible, name) - +error-feedback(visible, 'ipaddressPortRange', 'Invalid port range!', name) +mixin ipaddress-port-range-feedback(name) + +form-field-feedback(name, 'ipaddressPortRange', 'Invalid port range!') + +//- Mixin for feedback on UUID violation. +mixin uuid-feedback(name) + +form-field-feedback(name, 'uuid', 'Invalid node ID!') //- Mixin for checkbox. mixin checkbox(lbl, model, name, tip) - ignite-form-field.checkbox - ignite-form-field-input-checkbox( - data-id=name - data-name=name - data-ng-model=model - ) - | #{lbl} - ignite-form-field-tooltip - | !{tip} + +form-field-checkbox(lbl, model, name, false, false, tip) //- Mixin for checkbox with enabled condition. mixin checkbox-enabled(lbl, model, name, enabled, tip) - ignite-form-field.checkbox - ignite-form-field-input-checkbox( - data-id=name - data-name=name - data-ng-model=model - data-ng-disabled='!(#{enabled})' - ) - | #{lbl} - ignite-form-field-tooltip - | !{tip} + if enabled === false || enabled === true + -var disabled = !enabled + else + -var disabled = '!('+enabled+')' + + +form-field-checkbox(lbl, model, name, disabled, false, tip) //- Mixin for java name field with enabled condition. mixin java-class(lbl, model, name, enabled, required, tip) -var errLbl = lbl.substring(0, lbl.length - 1) - ignite-form-field - ignite-form-field-label - | #{lbl} - ignite-form-field-tooltip - | !{tip} - ignite-form-field-input-text( - data-id=name - data-name=name - data-ng-model=model - data-ng-disabled='!(#{enabled})' - data-ng-required=required - data-placeholder='Enter fully qualified class name' + if enabled === false || enabled === true + -var disabled = !enabled + else + -var disabled = '!('+enabled+')' - data-java-identifier='true' - data-java-package-specified='true' - data-java-keywords='true' - data-java-built-in-class='true' - ) - +error-feedback('form[ngModelName].$error.javaBuiltInClass', 'javaBuiltInClass', errLbl + ' should not be the Java built-in class!', name) - +error-feedback('form[ngModelName].$error.javaKeywords', 'javaKeywords', errLbl + ' could not contains reserved Java keyword!', name) - +error-feedback('form[ngModelName].$error.javaPackageSpecified', 'javaPackageSpecified', errLbl + ' does not have package specified!', name) - +error-feedback('form[ngModelName].$error.javaIdentifier', 'javaIdentifier', errLbl + ' is invalid Java identifier!', name) + +ignite-form-field-text(lbl, model, name, disabled, required, 'Enter fully qualified class name', tip)( + data-java-identifier='true' + data-java-package-specified='true' + data-java-keywords='true' + data-java-built-in-class='true' + ) + if block + block + + +form-field-feedback(name, 'javaBuiltInClass', errLbl + ' should not be the Java built-in class!') + +form-field-feedback(name, 'javaKeywords', errLbl + ' could not contains reserved Java keyword!') + +form-field-feedback(name, 'javaPackageSpecified', errLbl + ' does not have package specified!') + +form-field-feedback(name, 'javaIdentifier', errLbl + ' is invalid Java identifier!') + + //- Mixin for text field with enabled condition with options. mixin java-class-typeahead(lbl, model, name, options, enabled, required, placeholder, tip) -var errLbl = lbl.substring(0, lbl.length - 1) - ignite-form-field - ignite-form-field-label - | #{lbl} - ignite-form-field-tooltip - | !{tip} - ignite-form-field-input-datalist( - data-id=name - data-name=name - data-ng-model=model - data-options=options - data-placeholder=placeholder - data-ng-disabled='!(#{enabled})' - data-ng-required=required - - data-java-identifier='true' - data-java-package-specified='allow-built-in' - data-java-keywords='true' - ) - +error-feedback('form[ngModelName].$error.javaKeywords', 'javaKeywords', errLbl + ' could not contains reserved Java keyword!', name) - +error-feedback('form[ngModelName].$error.javaPackageSpecified', 'javaPackageSpecified', errLbl + ' does not have package specified!', name) - +error-feedback('form[ngModelName].$error.javaIdentifier', 'javaIdentifier', errLbl + ' is invalid Java identifier!', name) + +form-field-datalist(lbl, model, name, '!('+enabled+')', required, placeholder, options, tip)( + data-java-identifier='true' + data-java-package-specified='allow-built-in' + data-java-keywords='true' + ) + +form-field-feedback(name, 'javaKeywords', errLbl + ' could not contains reserved Java keyword!') + +form-field-feedback(name, 'javaPackageSpecified', errLbl + ' does not have package specified!') + +form-field-feedback(name, 'javaIdentifier', errLbl + ' is invalid Java identifier!') //- Mixin for text field with IP address check. mixin text-ip-address(lbl, model, name, enabled, placeholder, tip) - ignite-form-field - ignite-form-field-label - | #{lbl} - ignite-form-field-tooltip - | !{tip} - ignite-form-field-input-text( - data-id=name - data-name=name - data-ng-model=model - data-ipaddress='true' - data-ng-disabled='!(#{enabled})' - data-placeholder=placeholder - ) - +ipaddress-feedback('form[ngModelName].$error.ipaddress', name) - + +ignite-form-field-text(lbl, model, name, '!('+enabled+')', false, placeholder, tip)(data-ipaddress='true') + +ipaddress-feedback(name) //- Mixin for text field with IP address and port check. mixin text-ip-address-with-port(lbl, model, name, enabled, placeholder, tip) - ignite-form-field - ignite-form-field-label - | #{lbl} - ignite-form-field-tooltip - | !{tip} - ignite-form-field-input-text( - data-id=name - data-name=name - data-ng-model=model - data-ipaddress='true' - data-ipaddress-with-port='true' - data-ng-disabled='!(#{enabled})' - data-placeholder=placeholder - ) - +ipaddress-feedback('form[ngModelName].$error.ipaddress', name) - +ipaddress-port-feedback('form[ngModelName].$error.ipaddressPort', name) + +ignite-form-field-text(lbl, model, name, '!('+enabled+')', false, placeholder, tip)(data-ipaddress='true' data-ipaddress-with-port='true') + +ipaddress-feedback(name) + +ipaddress-port-feedback(name) //- Mixin for text field. mixin text-enabled(lbl, model, name, enabled, required, placeholder, tip) - ignite-form-field - ignite-form-field-label - | #{lbl} - ignite-form-field-tooltip - | !{tip} - ignite-form-field-input-text( - data-id=name - data-name=name - data-ng-model=model - data-placeholder=placeholder - data-ng-disabled='!(#{enabled})' - data-ng-required=required - ) + if enabled === false || enabled === true + -var disabled = !enabled + else + -var disabled = '!('+enabled+')' + + +ignite-form-field-text(lbl, model, name, disabled, required, placeholder, tip) + if block + block //- Mixin for text field. mixin text(lbl, model, name, required, placeholder, tip) - +text-enabled(lbl, model, name, 'true', required, placeholder, tip) + +ignite-form-field-text(lbl, model, name, false, required, placeholder, tip) + if block + block //- Mixin for text field with enabled condition with options. mixin text-options(lbl, model, name, options, enabled, required, placeholder, tip) - ignite-form-field - ignite-form-field-label - | #{lbl} - ignite-form-field-tooltip - | !{tip} - ignite-form-field-input-datalist( - data-id=name - data-name=name - data-ng-model=model - data-options=options - data-placeholder=placeholder - data-ng-disabled='!(#{enabled})' - data-ng-required=required - ) + +form-field-datalist(lbl, model, name, '!('+enabled+')', required, placeholder, options, tip) //- Mixin for required numeric field. mixin number-required(lbl, model, name, enabled, required, placeholder, min, tip) - ignite-form-field - ignite-form-field-label - | #{lbl} - ignite-form-field-tooltip - | !{tip} - ignite-form-field-input-number( - data-id=name - data-name=name - data-ng-model=model - data-ng-required=required - data-ng-disabled='!(#{enabled})' - data-placeholder=placeholder - data-min=min - ) + +ignite-form-field-number(lbl, model, name, '!('+enabled+')', required, placeholder, min, false, false, tip) //- Mixin for required numeric field with maximum and minimum limit. mixin number-min-max(lbl, model, name, enabled, placeholder, min, max, tip) - +number-min-max-step(lbl, model, name, enabled, placeholder, min, max, '1', tip) + +ignite-form-field-number(lbl, model, name, '!('+enabled+')', false, placeholder, min, max, '1', tip) //- Mixin for required numeric field with maximum and minimum limit. mixin number-min-max-step(lbl, model, name, enabled, placeholder, min, max, step, tip) - ignite-form-field - ignite-form-field-label - | #{lbl} - ignite-form-field-tooltip - | !{tip} - ignite-form-field-input-number( - data-id=name - data-name=name - data-ng-model=model - data-ng-disabled='!(#{enabled})' - data-placeholder=placeholder - data-min=min - data-max=max - data-step=step - ) + +ignite-form-field-number(lbl, model, name, '!('+enabled+')', false, placeholder, min, max, step, tip) //- Mixin for numeric field. mixin number(lbl, model, name, enabled, placeholder, min, tip) - +number-required(lbl, model, name, enabled, 'false', placeholder, min, tip) + +ignite-form-field-number(lbl, model, name, '!('+enabled+')', false, placeholder, min, false, false, tip) //- Mixin for required dropdown field. mixin dropdown-required-empty(lbl, model, name, enabled, required, placeholder, placeholderEmpty, options, tip) - ignite-form-field - ignite-form-field-label - | #{lbl} - ignite-form-field-tooltip - | !{tip} - ignite-form-field-dropdown( - data-id=name - data-name=name - data-options=options - data-ng-model=model - data-ng-disabled='!(#{enabled})' - data-ng-required=required - data-placeholder='{{#{options}.length > 0 ? "#{placeholder}" : "#{placeholderEmpty}"}}' - ) - + if enabled === false || enabled === true + -var disabled = !enabled + else + -var disabled = '!('+enabled+')' + + +ignite-form-field-dropdown(lbl, model, name, disabled, required, false, placeholder, placeholderEmpty, options, tip) + if block + block + //- Mixin for required dropdown field. mixin dropdown-required(lbl, model, name, enabled, required, placeholder, options, tip) - ignite-form-field - ignite-form-field-label - | #{lbl} - ignite-form-field-tooltip - | !{tip} - ignite-form-field-dropdown( - data-id=name - data-name=name - data-options=options - data-ng-model=model - data-ng-disabled='!(#{enabled})' - data-ng-required=required - data-placeholder=placeholder - ) + if enabled === false || enabled === true + -var disabled = !enabled + else + -var disabled = '!('+enabled+')' + + +ignite-form-field-dropdown(lbl, model, name, disabled, required, false, placeholder, '', options, tip) + if block + block //- Mixin for dropdown field. mixin dropdown(lbl, model, name, enabled, placeholder, options, tip) - +dropdown-required(lbl, model, name, enabled, 'false', placeholder, options, tip) + if enabled === false || enabled === true + -var disabled = !enabled + else + -var disabled = '!('+enabled+')' + + +ignite-form-field-dropdown(lbl, model, name, disabled, false, false, placeholder, '', options, tip) + if block + block //- Mixin for dropdown-multiple field. mixin dropdown-multiple(lbl, model, name, enabled, placeholder, placeholderEmpty, options, tip) - ignite-form-field - ignite-form-field-label - | !{lbl} - ignite-form-field-tooltip - | !{tip} - ignite-form-field-dropdown( - data-id=name - data-name=name - data-multiple='true' - data-options=options - data-ng-model=model - data-ng-disabled='!(#{enabled})' - data-placeholder='{{ #{options}.length > 0 ? "#{placeholder}" : "#{placeholderEmpty}" }}' - data-disabled='!#{options}.length' - ) + if enabled === false || enabled === true + -var disabled = !enabled + else + -var disabled = '!('+enabled+')' + + +ignite-form-field-dropdown(lbl, model, name, disabled, false, true, placeholder, placeholderEmpty, options, tip) + if block + block //- Mixin for table text field. -mixin table-text-field(field, items, valid, save, placeholder, newItem) +mixin table-text-field(name, model, items, valid, save, placeholder, newItem) -var resetOnEnter = newItem ? '(stopblur = true) && (group.add = [{}])' : '(field.edit = false)' -var onEnter = valid + ' && (' + save + '); ' + valid + ' && ' + resetOnEnter + ';' @@ -352,21 +250,22 @@ mixin table-text-field(field, items, valid, save, placeholder, newItem) -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false' -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';' - ignite-form-field-input-text( - data-name='#{field}{{ $index || "" }}' - data-ng-model=field - data-ng-required='true' - data-placeholder=placeholder - data-ignite-unique=items - data-ignite-form-field-input-autofocus='true' - ignite-on-enter=onEnter - ignite-on-escape=onEscape - ng-blur=onBlur - ) + if block block + .input-tip + +ignite-form-field-input(name, model, false, 'true', placeholder)( + data-ignite-unique=items + data-ignite-form-field-input-autofocus='true' + ng-blur=onBlur + ignite-on-enter=onEnter + ignite-on-escape=onEscape + ) + //- Mixin for table java class field. -mixin table-java-class-field(lbl, field, items, valid, save, newItem) +mixin table-java-class-field(lbl, name, model, items, valid, save, newItem) + -var errLbl = lbl.substring(0, lbl.length - 1) + -var resetOnEnter = newItem ? '(stopblur = true) && (group.add = [{}])' : '(field.edit = false)' -var onEnter = valid + ' && (' + save + '); ' + valid + ' && ' + resetOnEnter + ';' @@ -375,33 +274,34 @@ mixin table-java-class-field(lbl, field, items, valid, save, newItem) -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false' -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';' - ignite-form-field-input-text( - data-name='#{field}{{ $index || "" }}' - data-ng-model=field - data-ng-required='true' - data-ignite-unique=items - data-ignite-form-field-input-autofocus='true' + if block + block - data-java-identifier='true' - data-java-package-specified='true' - data-java-keywords='true' - data-java-built-in-class='true' + +form-field-feedback(name, 'javaBuiltInClass', errLbl + ' should not be the Java built-in class!') + +form-field-feedback(name, 'javaKeywords', errLbl + ' could not contains reserved Java keyword!') + +form-field-feedback(name, 'javaPackageSpecified', errLbl + ' does not have package specified!') + +form-field-feedback(name, 'javaIdentifier', errLbl + ' is invalid Java identifier!') - data-placeholder='Enter fully qualified class name' + if block + block - ignite-on-enter=onEnter - ignite-on-escape=onEscape - ng-blur=onBlur - ) - +error-feedback('form[ngModelName].$error.javaBuiltInClass', 'javaBuiltInClass', lbl + ' should not be the Java built-in class!') - +error-feedback('form[ngModelName].$error.javaKeywords', 'javaKeywords', lbl + ' could not contains reserved Java keyword!') - +error-feedback('form[ngModelName].$error.javaPackageSpecified', 'javaPackageSpecified', lbl + ' does not have package specified!') - +error-feedback('form[ngModelName].$error.javaIdentifier', 'javaIdentifier', lbl + ' is invalid Java identifier!') + .input-tip + +ignite-form-field-input(name, model, false, 'true', 'Enter fully qualified class name')( + data-java-identifier='true' + data-java-package-specified='true' + data-java-keywords='true' + data-java-built-in-class='true' - block + data-ignite-unique=items + data-ignite-form-field-input-autofocus='true' + + ng-blur=onBlur + ignite-on-enter=onEnter + ignite-on-escape=onEscape + ) //- Mixin for table java package field. -mixin table-java-package-field(field, items, valid, save, newItem) +mixin table-java-package-field(name, model, items, valid, save, newItem) -var resetOnEnter = newItem ? '(stopblur = true) && (group.add = [{}])' : '(field.edit = false)' -var onEnter = valid + ' && (' + save + '); ' + valid + ' && ' + resetOnEnter + ';' @@ -410,23 +310,28 @@ mixin table-java-package-field(field, items, valid, save, newItem) -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false' -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';' - ignite-form-field-input-text( - data-name='#{field}{{ $index || "" }}' - data-ng-model=field - data-ng-required='true' - data-placeholder='Enter package name' - data-java-keywords='true' - data-java-package-name='true' - data-ignite-unique=items - data-ignite-form-field-input-autofocus='true' - ignite-on-enter=onEnter - ignite-on-escape=onEscape - ng-blur=onBlur - ) + +form-field-feedback(name, 'javaKeywords', 'Package name could not contains reserved Java keyword!') + +form-field-feedback(name, 'javaPackageName', 'Package name is invalid!') + + if block block + .input-tip + +ignite-form-field-input(name, model, false, 'true', 'Enter package name')( + data-java-keywords='true' + data-java-package-name='true' + + data-ignite-unique=items + data-ignite-form-field-input-autofocus='true' + + ng-blur=onBlur + ignite-on-enter=onEnter + ignite-on-escape=onEscape + ) + + //- Mixin for table address field. -mixin table-address-field(field, items, valid, save, newItem, portRange) +mixin table-address-field(name, model, items, valid, save, newItem, portRange) -var resetOnEnter = newItem ? '(stopblur = true) && (group.add = [{}])' : '(field.edit = false)' -var onEnter = valid + ' && (' + save + '); ' + valid + ' && ' + resetOnEnter + ';' @@ -435,28 +340,54 @@ mixin table-address-field(field, items, valid, save, newItem, portRange) -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false' -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';' - ignite-form-field-input-text( - data-name='#{field}{{ $index || "" }}' - data-ng-model=field - data-ng-required='true' - data-placeholder='IP address:port' - data-ipaddress='true' - data-ipaddress-with-port='true' - data-ipaddress-with-port-range=portRange ? 'true' : null - data-ignite-unique=items - data-ignite-form-field-input-autofocus='true' - ignite-on-enter=onEnter - ignite-on-escape=onEscape - ng-blur=onBlur - ) + +ipaddress-feedback(name) + +ipaddress-port-feedback(name) + +ipaddress-port-range-feedback(name) + + if block + block + + .input-tip + +ignite-form-field-input(name, model, false, 'true', 'IP address:port')( + data-ipaddress='true' + data-ipaddress-with-port='true' + data-ipaddress-with-port-range=portRange ? 'true' : null + data-ignite-unique=items + data-ignite-form-field-input-autofocus='true' + ng-blur=onBlur + ignite-on-enter=onEnter + ignite-on-escape=onEscape + ) + +//- Mixin for table UUID field. +mixin table-uuid-field(name, model, items, valid, save, newItem, portRange) + -var resetOnEnter = newItem ? '(stopblur = true) && (group.add = [{}])' : '(field.edit = false)' + -var onEnter = valid + ' && (' + save + '); ' + valid + ' && ' + resetOnEnter + ';' + + -var onEscape = newItem ? 'group.add = []' : 'field.edit = false' + + -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false' + -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';' + + if block block + .input-tip + +ignite-form-field-input(name, model, false, 'true', 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx')( + data-uuid='true' + data-ignite-unique=items + data-ignite-form-field-input-autofocus='true' + ng-blur=onBlur + ignite-on-enter=onEnter + ignite-on-escape=onEscape + ) + //- Mixin for table save button. "||" used instead of "&&" to workaround escaping of "&&" to "&&" mixin table-save-button(valid, save, newItem) -var reset = newItem ? 'group.add = []' : 'field.edit = false' - i.fa.fa-floppy-o( + i.fa.fa-floppy-o.form-field-save( ng-show=valid ng-click='!(#{valid}) || (#{save}); !(#{valid}) || (#{reset});' bs-tooltip @@ -486,9 +417,9 @@ mixin cacheMode(lbl, model, name, placeholder) ]', 'Cache modes:\

            \ -
          • Partitioned - in this mode the overall key set will be divided into partitions and all partitions will be split equally between participating nodes
          • \ -
          • Replicated - in this mode all the keys are distributed to all participating nodes
          • \ -
          • Local - in this mode caches residing on different grid nodes will not know about each other
          • \ +
          • PARTITIONED - in this mode the overall key set will be divided into partitions and all partitions will be split equally between participating nodes
          • \ +
          • REPLICATED - in this mode all the keys are distributed to all participating nodes
          • \ +
          • LOCAL - in this mode caches residing on different grid nodes will not know about each other
          • \
          ' ) @@ -497,41 +428,41 @@ mixin evictionPolicy(model, name, enabled, required, tip) -var kind = model + '.kind' -var policy = model + '[' + kind + ']' - +dropdown-required('Eviction policy:', kind, name + 'Kind', enabled, required, 'Not set', + +dropdown-required('Eviction policy:', kind, name + '+ "Kind"', enabled, required, 'Not set', '[\ {value: "LRU", label: "LRU"},\ {value: "FIFO", label: "FIFO"},\ {value: "SORTED", label: "Sorted"},\ {value: undefined, label: "Not set"}\ ]', tip) - span(ng-if=kind) + span(ng-if=kind ng-init='__ = {};') a.customize(ng-show='__.expanded' ng-click='__.expanded = false') Hide settings a.customize(ng-hide='__.expanded' ng-click='__.expanded = true') Show settings .panel-details(ng-if='__.expanded') .details-row - +number('Batch size', policy + '.batchSize', name + 'batchSize', enabled, '1', '0', + +number('Batch size', policy + '.batchSize', name + '+ "batchSize"', enabled, '1', '0', 'Number of entries to remove on shrink') .details-row - +number('Max memory size', policy + '.maxMemorySize', name + 'maxMemorySize', enabled, '0', '0', + +number('Max memory size', policy + '.maxMemorySize', name + '+ "maxMemorySize"', enabled, '0', '0', 'Maximum allowed cache size in bytes') .details-row - +number('Max size', policy + '.maxSize', name + 'maxSize', enabled, '100000', '0', + +number('Max size', policy + '.maxSize', name + '+ "maxSize"', enabled, '100000', '0', 'Maximum allowed size of cache before entry will start getting evicted') //- Mixin for clusters dropdown. mixin clusters(model, tip) +dropdown-multiple('Clusters:' + ' (add)', - model + '.clusters', 'clusters', 'true', 'Choose clusters', 'No clusters configured', 'clusters', tip) + model + '.clusters', '"clusters"', true, 'Choose clusters', 'No clusters configured', 'clusters', tip) //- Mixin for caches dropdown. mixin caches(model, tip) +dropdown-multiple('Caches:' + ' (add)', - model + '.caches', 'caches', 'true', 'Choose caches', 'No caches configured', 'caches', tip) + model + '.caches', '"caches"', true, 'Choose caches', 'No caches configured', 'caches', tip) //- Mixin for XML and Java preview. mixin preview-xml-java(master, generator, detail) ignite-ui-ace-tabs - .preview-panel(ng-init='mode = false') + .preview-panel .preview-legend a(ng-class='{active: !mode, inactive: mode}' ng-click='mode = false') XML |   @@ -586,3 +517,25 @@ mixin table-pair-edit(tbl, prefix, keyPlaceholder, valPlaceholder, keyJavaBuiltI input.form-control(id=valFocusId type='text' ng-model=valModel placeholder=valPlaceholder bs-typeahead container='body' ignite-retain-selection data-min-length='1' bs-options='javaClass for javaClass in javaBuiltInClasses' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()') else input.form-control(id=valFocusId type='text' ng-model=valModel placeholder=valPlaceholder ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()') + +//- Mixin for DB dialect. +mixin dialect(lbl, model, name, required, tipTitle, genericDialectName, placeholder) + +dropdown(lbl, model, name, required, placeholder, '[\ + {value: "Generic", label: "' + genericDialectName + '"},\ + {value: "Oracle", label: "Oracle"},\ + {value: "DB2", label: "IBM DB2"},\ + {value: "SQLServer", label: "Microsoft SQL Server"},\ + {value: "MySQL", label: "MySQL"},\ + {value: "PostgreSQL", label: "PostgreSQL"},\ + {value: "H2", label: "H2 database"}\ + ]', + tipTitle + + '
            \ +
          • ' + genericDialectName + '
          • \ +
          • Oracle database
          • \ +
          • IBM DB2
          • \ +
          • Microsoft SQL Server
          • \ +
          • MySQL
          • \ +
          • PostgreSQL
          • \ +
          • H2 database
          • \ +
          ') diff --git a/modules/web-console/src/main/js/app/modules/Demo/Demo.module.js b/modules/web-console/frontend/app/modules/Demo/Demo.module.js similarity index 97% rename from modules/web-console/src/main/js/app/modules/Demo/Demo.module.js rename to modules/web-console/frontend/app/modules/Demo/Demo.module.js index 68aca9deb2404..83d55ed596da7 100644 --- a/modules/web-console/src/main/js/app/modules/Demo/Demo.module.js +++ b/modules/web-console/frontend/app/modules/Demo/Demo.module.js @@ -41,8 +41,8 @@ angular url: '/demo/reset', controller: ['$state', '$http', 'IgniteMessages', ($state, $http, Messages) => { $http.post('/api/v1/demo/reset') - .then(() => $state.go('base.configuration.clusters')) - .catch((err) => { + .success(() => $state.go('base.configuration.clusters')) + .error((err) => { $state.go('base.configuration.clusters'); Messages.showError(err); diff --git a/modules/web-console/src/main/js/app/modules/ace.module.js b/modules/web-console/frontend/app/modules/ace.module.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/ace.module.js rename to modules/web-console/frontend/app/modules/ace.module.js diff --git a/modules/web-console/src/main/js/app/modules/agent/agent.module.js b/modules/web-console/frontend/app/modules/agent/agent.module.js similarity index 86% rename from modules/web-console/src/main/js/app/modules/agent/agent.module.js rename to modules/web-console/frontend/app/modules/agent/agent.module.js index ca95166116279..22ced13600f67 100644 --- a/modules/web-console/src/main/js/app/modules/agent/agent.module.js +++ b/modules/web-console/frontend/app/modules/agent/agent.module.js @@ -18,6 +18,8 @@ import angular from 'angular'; import io from 'socket.io-client'; // eslint-disable-line no-unused-vars +const maskNull = (val) => _.isEmpty(val) ? 'null' : val; + class IgniteAgentMonitor { constructor(socketFactory, $root, $q, $state, $modal, Messages) { this._scope = $root.$new(); @@ -124,6 +126,9 @@ class IgniteAgentMonitor { } init() { + if (this._socket) + return; + this._socket = this._socketFactory(); const disconnectFn = () => { @@ -260,47 +265,60 @@ class IgniteAgentMonitor { } /** + * @param {String} nid Node id. * @param {int} [queryId] * @returns {Promise} */ - queryClose(queryId) { - return this._rest('node:query:close', queryId); + queryClose(nid, queryId) { + return this._rest('node:query:close', nid, queryId); } /** + * @param {String} nid Node id. * @param {String} cacheName Cache name. - * @param {int} pageSize * @param {String} [query] Query if null then scan query. + * @param {Boolean} local Flag whether to execute query locally. + * @param {int} pageSize * @returns {Promise} */ - query(cacheName, pageSize, query) { - return this._rest('node:query', _.isEmpty(cacheName) ? null : cacheName, pageSize, query); + query(nid, cacheName, query, local, pageSize) { + return this._rest('node:query', nid, maskNull(cacheName), maskNull(query), local, pageSize) + .then(({result}) => { + if (_.isEmpty(result.key)) + return result.value; + + return Promise.reject(result.key); + }); } /** + * @param {String} nid Node id. * @param {String} cacheName Cache name. * @param {String} [query] Query if null then scan query. + * @param {Boolean} local Flag whether to execute query locally. * @returns {Promise} */ - queryGetAll(cacheName, query) { - return this._rest('node:query:getAll', _.isEmpty(cacheName) ? null : cacheName, query); + queryGetAll(nid, cacheName, query, local) { + return this._rest('node:query:getAll', nid, maskNull(cacheName), maskNull(query), local); } /** - * @param {String} [cacheName] Cache name. + * @param {String} nid Node id. + * @param {int} queryId + * @param {int} pageSize * @returns {Promise} */ - metadata(cacheName) { - return this._rest('node:cache:metadata', _.isEmpty(cacheName) ? null : cacheName); + next(nid, queryId, pageSize) { + return this._rest('node:query:fetch', nid, queryId, pageSize) + .then(({result}) => result); } /** - * @param {int} queryId - * @param {int} pageSize + * @param {String} [cacheName] Cache name. * @returns {Promise} */ - next(queryId, pageSize) { - return this._rest('node:query:fetch', queryId, pageSize); + metadata(cacheName) { + return this._rest('node:cache:metadata', maskNull(cacheName)); } stopWatch() { diff --git a/modules/web-console/src/main/js/app/modules/branding/branding.module.js b/modules/web-console/frontend/app/modules/branding/branding.module.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/branding/branding.module.js rename to modules/web-console/frontend/app/modules/branding/branding.module.js diff --git a/modules/web-console/src/main/js/app/modules/branding/branding.provider.js b/modules/web-console/frontend/app/modules/branding/branding.provider.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/branding/branding.provider.js rename to modules/web-console/frontend/app/modules/branding/branding.provider.js diff --git a/modules/web-console/src/main/js/app/modules/branding/features.directive.js b/modules/web-console/frontend/app/modules/branding/features.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/branding/features.directive.js rename to modules/web-console/frontend/app/modules/branding/features.directive.js diff --git a/modules/web-console/src/main/js/app/modules/branding/footer.directive.js b/modules/web-console/frontend/app/modules/branding/footer.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/branding/footer.directive.js rename to modules/web-console/frontend/app/modules/branding/footer.directive.js diff --git a/modules/web-console/src/main/js/app/modules/branding/header-logo.directive.js b/modules/web-console/frontend/app/modules/branding/header-logo.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/branding/header-logo.directive.js rename to modules/web-console/frontend/app/modules/branding/header-logo.directive.js diff --git a/modules/web-console/src/main/js/app/modules/branding/header-logo.jade b/modules/web-console/frontend/app/modules/branding/header-logo.jade similarity index 97% rename from modules/web-console/src/main/js/app/modules/branding/header-logo.jade rename to modules/web-console/frontend/app/modules/branding/header-logo.jade index b80792115aa81..b58f6706e48bf 100644 --- a/modules/web-console/src/main/js/app/modules/branding/header-logo.jade +++ b/modules/web-console/frontend/app/modules/branding/header-logo.jade @@ -14,5 +14,5 @@ See the License for the specific language governing permissions and limitations under the License. -a(href='/') +a(ui-sref='signin') img.navbar-brand(ng-src='{{logo.url}}' height='40') diff --git a/modules/web-console/src/main/js/app/modules/branding/header-title.directive.js b/modules/web-console/frontend/app/modules/branding/header-title.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/branding/header-title.directive.js rename to modules/web-console/frontend/app/modules/branding/header-title.directive.js diff --git a/modules/web-console/src/main/js/app/modules/branding/powered-by-apache.directive.js b/modules/web-console/frontend/app/modules/branding/powered-by-apache.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/branding/powered-by-apache.directive.js rename to modules/web-console/frontend/app/modules/branding/powered-by-apache.directive.js diff --git a/modules/web-console/src/main/js/app/modules/branding/powered-by-apache.jade b/modules/web-console/frontend/app/modules/branding/powered-by-apache.jade similarity index 100% rename from modules/web-console/src/main/js/app/modules/branding/powered-by-apache.jade rename to modules/web-console/frontend/app/modules/branding/powered-by-apache.jade diff --git a/modules/web-console/src/main/js/app/modules/branding/terms.directive.js b/modules/web-console/frontend/app/modules/branding/terms.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/branding/terms.directive.js rename to modules/web-console/frontend/app/modules/branding/terms.directive.js diff --git a/modules/web-console/src/main/js/app/modules/configuration/EventGroups.provider.js b/modules/web-console/frontend/app/modules/configuration/EventGroups.provider.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/configuration/EventGroups.provider.js rename to modules/web-console/frontend/app/modules/configuration/EventGroups.provider.js diff --git a/modules/web-console/src/main/js/app/modules/configuration/Sidebar.provider.js b/modules/web-console/frontend/app/modules/configuration/Sidebar.provider.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/configuration/Sidebar.provider.js rename to modules/web-console/frontend/app/modules/configuration/Sidebar.provider.js diff --git a/modules/web-console/src/main/js/app/modules/configuration/configuration.module.js b/modules/web-console/frontend/app/modules/configuration/configuration.module.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/configuration/configuration.module.js rename to modules/web-console/frontend/app/modules/configuration/configuration.module.js diff --git a/modules/web-console/src/main/js/app/modules/configuration/generator/Docker.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/configuration/generator/Docker.service.js rename to modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js diff --git a/modules/web-console/src/main/js/app/modules/configuration/generator/Java.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Java.service.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/configuration/generator/Java.service.js rename to modules/web-console/frontend/app/modules/configuration/generator/Java.service.js diff --git a/modules/web-console/src/main/js/app/modules/configuration/generator/Pom.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js similarity index 87% rename from modules/web-console/src/main/js/app/modules/configuration/generator/Pom.service.js rename to modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js index 674c16e2e3613..627a1e34d132f 100644 --- a/modules/web-console/src/main/js/app/modules/configuration/generator/Pom.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js @@ -126,6 +126,19 @@ class GeneratorPom { res.endBlock(''); } + /** + * Add dependency for specified store factory if not exist. + * @param storeDeps Already added dependencies. + * @param storeFactory Store factory to add dependency. + */ + storeFactoryDependency(storeDeps, storeFactory) { + if (storeFactory.dialect && (!storeFactory.connectVia || storeFactory.connectVia === 'DataSource')) { + const dep = POM_DEPENDENCIES[storeFactory.dialect]; + + this.addDependency(storeDeps, dep.groupId, dep.artifactId, dep.version, dep.jar); + } + } + /** * Generate pom.xml. * @@ -146,15 +159,11 @@ class GeneratorPom { res = $generatorCommon.builder(); _.forEach(caches, (cache) => { - if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) { - const storeFactory = cache.cacheStoreFactory[cache.cacheStoreFactory.kind]; + if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) + this.storeFactoryDependency(storeDeps, cache.cacheStoreFactory[cache.cacheStoreFactory.kind]); - if (storeFactory.dialect && (!storeFactory.connectVia || storeFactory.connectVia === 'DataSource')) { - const dep = POM_DEPENDENCIES[storeFactory.dialect]; - - this.addDependency(storeDeps, dep.groupId, dep.artifactId, dep.version, dep.jar); - } - } + if (_.get(cache, 'nodeFilter.kind') === 'Exclude') + this.addDependency(deps, 'org.apache.ignite', 'ignite-extdata-p2p', igniteVersion); }); res.line(''); @@ -184,6 +193,13 @@ class GeneratorPom { if (dep) this.addDependency(deps, 'org.apache.ignite', dep.artifactId, igniteVersion); + if (cluster.discovery.kind === 'Jdbc') { + const store = cluster.discovery.Jdbc; + + if (store.dataSourceBean && store.dialect) + this.storeFactoryDependency(storeDeps, cluster.discovery.Jdbc); + } + if (_.find(cluster.igfss, (igfs) => igfs.secondaryFileSystemEnabled)) this.addDependency(deps, 'org.apache.ignite', 'ignite-hadoop', igniteVersion); diff --git a/modules/web-console/src/main/js/app/modules/configuration/generator/Xml.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Xml.service.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/configuration/generator/Xml.service.js rename to modules/web-console/frontend/app/modules/configuration/generator/Xml.service.js diff --git a/modules/web-console/src/main/js/app/modules/configuration/sidebar.directive.js b/modules/web-console/frontend/app/modules/configuration/sidebar.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/configuration/sidebar.directive.js rename to modules/web-console/frontend/app/modules/configuration/sidebar.directive.js diff --git a/modules/web-console/src/main/js/app/modules/dialog/dialog-content.directive.js b/modules/web-console/frontend/app/modules/dialog/dialog-content.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/dialog/dialog-content.directive.js rename to modules/web-console/frontend/app/modules/dialog/dialog-content.directive.js diff --git a/modules/web-console/src/main/js/app/modules/dialog/dialog-title.directive.js b/modules/web-console/frontend/app/modules/dialog/dialog-title.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/dialog/dialog-title.directive.js rename to modules/web-console/frontend/app/modules/dialog/dialog-title.directive.js diff --git a/modules/web-console/src/main/js/app/modules/dialog/dialog.controller.js b/modules/web-console/frontend/app/modules/dialog/dialog.controller.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/dialog/dialog.controller.js rename to modules/web-console/frontend/app/modules/dialog/dialog.controller.js diff --git a/modules/web-console/src/main/js/app/modules/dialog/dialog.directive.js b/modules/web-console/frontend/app/modules/dialog/dialog.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/dialog/dialog.directive.js rename to modules/web-console/frontend/app/modules/dialog/dialog.directive.js diff --git a/modules/web-console/src/main/js/app/modules/dialog/dialog.factory.js b/modules/web-console/frontend/app/modules/dialog/dialog.factory.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/dialog/dialog.factory.js rename to modules/web-console/frontend/app/modules/dialog/dialog.factory.js diff --git a/modules/web-console/src/main/js/app/modules/dialog/dialog.jade b/modules/web-console/frontend/app/modules/dialog/dialog.jade similarity index 100% rename from modules/web-console/src/main/js/app/modules/dialog/dialog.jade rename to modules/web-console/frontend/app/modules/dialog/dialog.jade diff --git a/modules/web-console/src/main/js/app/modules/dialog/dialog.module.js b/modules/web-console/frontend/app/modules/dialog/dialog.module.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/dialog/dialog.module.js rename to modules/web-console/frontend/app/modules/dialog/dialog.module.js diff --git a/modules/web-console/src/main/js/app/modules/form/field/bs-select-placeholder.directive.js b/modules/web-console/frontend/app/modules/form/field/bs-select-placeholder.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/form/field/bs-select-placeholder.directive.js rename to modules/web-console/frontend/app/modules/form/field/bs-select-placeholder.directive.js diff --git a/modules/web-console/src/main/js/app/modules/form/field/down.directive.js b/modules/web-console/frontend/app/modules/form/field/down.directive.js similarity index 69% rename from modules/web-console/src/main/js/app/modules/form/field/down.directive.js rename to modules/web-console/frontend/app/modules/form/field/down.directive.js index 91be945a6ad01..659933ebfcec9 100644 --- a/modules/web-console/src/main/js/app/modules/form/field/down.directive.js +++ b/modules/web-console/frontend/app/modules/form/field/down.directive.js @@ -15,29 +15,25 @@ * limitations under the License. */ -const template = ''; - export default ['igniteFormFieldDown', ['$tooltip', ($tooltip) => { - const link = (scope, $element) => { + const controller = ['$element', function($element) { $tooltip($element, { title: 'Move item down' }); - scope.down = () => { - const i = scope.models.indexOf(scope.model); - scope.models.splice(i, 1); - scope.models.splice(i + 1, 0, scope.model); + this.down = () => { + const i = this.models.indexOf(this.model); + + this.models.splice(i, 1); + this.models.splice(i + 1, 0, this.model); }; - }; + }]; return { - restrict: 'E', - scope: { + restrict: 'A', + bindToController: { model: '=ngModel', models: '=models' }, - template, - link, - replace: true, - transclude: true, - require: '^form' + controller, + controllerAs: '$ctrl' }; }]]; diff --git a/modules/web-console/frontend/app/modules/form/field/feedback.scss b/modules/web-console/frontend/app/modules/form/field/feedback.scss new file mode 100644 index 0000000000000..08d0aefd8b8f2 --- /dev/null +++ b/modules/web-console/frontend/app/modules/form/field/feedback.scss @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +@import "../../../../public/stylesheets/variables"; + +.form-field-feedback { + position: relative; + width: 0; + height: 28px; + float: right; + z-index: 2; + + color: $brand-primary; + line-height: $input-height; + pointer-events: initial; + text-align: center; + + &:before { + position: absolute; + right: 0; + width: 38px; + } +} diff --git a/modules/web-console/frontend/app/modules/form/field/field.scss b/modules/web-console/frontend/app/modules/form/field/field.scss new file mode 100644 index 0000000000000..57177664546e3 --- /dev/null +++ b/modules/web-console/frontend/app/modules/form/field/field.scss @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +@import "../../../../public/stylesheets/variables"; + +.indexField { + float: left; + line-height: 28px; + margin-right: 5px; + color: $brand-primary; +} + +.form-field-save { + position: relative; + width: 0; + height: 28px; + float: right; + z-index: 2; + + line-height: $input-height; + pointer-events: initial; + text-align: center; + + &:before { + position: absolute; + right: 0; + width: 38px; + } +} diff --git a/modules/web-console/src/main/js/app/modules/form/field/form-control-feedback.directive.js b/modules/web-console/frontend/app/modules/form/field/form-control-feedback.directive.js similarity index 96% rename from modules/web-console/src/main/js/app/modules/form/field/form-control-feedback.directive.js rename to modules/web-console/frontend/app/modules/form/field/form-control-feedback.directive.js index 058bcc32d0157..797ba6988fb4f 100644 --- a/modules/web-console/src/main/js/app/modules/form/field/form-control-feedback.directive.js +++ b/modules/web-console/frontend/app/modules/form/field/form-control-feedback.directive.js @@ -15,7 +15,7 @@ * limitations under the License. */ -export default ['formControlFeedback', [() => { +export default ['formFieldFeedback', [() => { const link = ($scope, $element, $attrs, [form]) => { let name = $scope.name; diff --git a/modules/web-console/src/main/js/app/modules/form/field/input/autofocus.directive.js b/modules/web-console/frontend/app/modules/form/field/input/autofocus.directive.js similarity index 89% rename from modules/web-console/src/main/js/app/modules/form/field/input/autofocus.directive.js rename to modules/web-console/frontend/app/modules/form/field/input/autofocus.directive.js index c963cc1393816..8ffc9a052c307 100644 --- a/modules/web-console/src/main/js/app/modules/form/field/input/autofocus.directive.js +++ b/modules/web-console/frontend/app/modules/form/field/input/autofocus.directive.js @@ -15,12 +15,12 @@ * limitations under the License. */ -export default ['igniteFormFieldInputAutofocus', [() => { +export default ['igniteFormFieldInputAutofocus', ['$timeout', ($timeout) => { const link = (scope, el, attrs) => { if (_.isUndefined(attrs.igniteFormFieldInputAutofocus) || attrs.igniteFormFieldInputAutofocus !== 'true') return; - el.focus(); + $timeout(() => el.focus(), 100); }; return { diff --git a/modules/web-console/src/main/js/app/modules/user/user.module.js b/modules/web-console/frontend/app/modules/form/field/input/select.scss similarity index 81% rename from modules/web-console/src/main/js/app/modules/user/user.module.js rename to modules/web-console/frontend/app/modules/form/field/input/select.scss index 2387f208c9b15..55bbd58ccc7bd 100644 --- a/modules/web-console/src/main/js/app/modules/user/user.module.js +++ b/modules/web-console/frontend/app/modules/form/field/input/select.scss @@ -15,14 +15,7 @@ * limitations under the License. */ -import angular from 'angular'; - -import Auth from './Auth.service'; -import User from './User.service'; - -angular -.module('ignite-console.user', [ - -]) -.service(...Auth) -.service(...User); +.select.dropdown-menu.ng-leave { + transition: none !important; /* disable transitions */ + animation: none 0s !important; /* disable keyframe animations */ +} diff --git a/modules/web-console/src/main/js/app/modules/form/field/input/text.css b/modules/web-console/frontend/app/modules/form/field/input/text.scss similarity index 100% rename from modules/web-console/src/main/js/app/modules/form/field/input/text.css rename to modules/web-console/frontend/app/modules/form/field/input/text.scss diff --git a/modules/web-console/src/main/js/app/modules/form/field/label.directive.js b/modules/web-console/frontend/app/modules/form/field/label.directive.js similarity index 93% rename from modules/web-console/src/main/js/app/modules/form/field/label.directive.js rename to modules/web-console/frontend/app/modules/form/field/label.directive.js index 9b812d7e684cf..97ba5987f859f 100644 --- a/modules/web-console/src/main/js/app/modules/form/field/label.directive.js +++ b/modules/web-console/frontend/app/modules/form/field/label.directive.js @@ -27,9 +27,9 @@ export default ['igniteFormFieldLabel', [() => { if (/(.*):$/.test(text)) field.name = /(.*):$/.exec(text)[1]; - const $label = $element.parent().parent().find('label'); + const $label = $element.parent().parent().find('.group-legend > label, .ignite-field > label'); - if ($element[0].id) { + if ($label[0] && $element[0].id) { const id = $element[0].id; $label[0].id = id.indexOf('+') >= 0 ? $scope.$eval(id) : id; diff --git a/modules/web-console/src/main/js/app/modules/form/field/tooltip.directive.js b/modules/web-console/frontend/app/modules/form/field/tooltip.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/form/field/tooltip.directive.js rename to modules/web-console/frontend/app/modules/form/field/tooltip.directive.js diff --git a/modules/web-console/src/main/js/app/modules/form/field/up.directive.js b/modules/web-console/frontend/app/modules/form/field/up.directive.js similarity index 69% rename from modules/web-console/src/main/js/app/modules/form/field/up.directive.js rename to modules/web-console/frontend/app/modules/form/field/up.directive.js index d31bdc51b1df4..aba1cbe4b21b4 100644 --- a/modules/web-console/src/main/js/app/modules/form/field/up.directive.js +++ b/modules/web-console/frontend/app/modules/form/field/up.directive.js @@ -15,30 +15,25 @@ * limitations under the License. */ -const template = ''; - export default ['igniteFormFieldUp', ['$tooltip', ($tooltip) => { - const link = (scope, $element) => { + const controller = ['$element', function($element) { $tooltip($element, { title: 'Move item up' }); - scope.up = () => { - const idx = scope.models.indexOf(scope.model); + this.up = () => { + const idx = this.models.indexOf(this.model); - scope.models.splice(idx, 1); - scope.models.splice(idx - 1, 0, scope.model); + this.models.splice(idx, 1); + this.models.splice(idx - 1, 0, this.model); }; - }; + }]; return { - restrict: 'E', - scope: { + restrict: 'A', + bindToController: { model: '=ngModel', models: '=models' }, - template, - link, - replace: true, - transclude: true, - require: '^form' + controller, + controllerAs: '$ctrl' }; }]]; diff --git a/modules/web-console/src/main/js/app/modules/form/form.module.js b/modules/web-console/frontend/app/modules/form/form.module.js similarity index 81% rename from modules/web-console/src/main/js/app/modules/form/form.module.js rename to modules/web-console/frontend/app/modules/form/form.module.js index 57a92fa6ce9b4..23eafcd6de831 100644 --- a/modules/web-console/src/main/js/app/modules/form/form.module.js +++ b/modules/web-console/frontend/app/modules/form/form.module.js @@ -17,25 +17,24 @@ import angular from 'angular'; +// Fields styles. +import './field/field.scss'; +import './field/feedback.scss'; +import './field/input/text.scss'; +import './field/input/select.scss'; + // Panel. import igniteFormPanel from './panel/panel.directive'; +import igniteFormPanelField from './panel/field.directive'; import igniteFormPanelChevron from './panel/chevron.directive'; import igniteFormRevert from './panel/revert.directive'; // Field. -import igniteFormField from './field/field.directive'; import igniteFormFieldLabel from './field/label.directive'; import igniteFormFieldTooltip from './field/tooltip.directive'; -import igniteFormFieldDropdown from './field/dropdown.directive'; -import igniteFormFieldInputNumber from './field/input/number.directive'; -import igniteFormFieldInputText from './field/input/text.directive'; -import igniteFormFieldInputCheckbox from './field/input/checkbox.directive'; -import igniteFormFieldInputDatalist from './field/input/datalist.directive'; - import placeholder from './field/bs-select-placeholder.directive'; // Group. -import igniteFormGroup from './group/group.directive'; import igniteFormGroupAdd from './group/add.directive'; import igniteFormGroupTooltip from './group/tooltip.directive'; @@ -49,12 +48,13 @@ import javaPackageName from './validator/java-package-name.directive'; import propertyValueSpecified from './validator/property-value-specified.directive'; import propertyUnique from './validator/property-unique.directive'; import unique from './validator/unique.directive'; +import uuid from './validator/uuid.directive'; // Helpers. import igniteFormFieldInputAutofocus from './field/input/autofocus.directive'; +import igniteFormControlFeedback from './field/form-control-feedback.directive'; import igniteFormFieldUp from './field/up.directive'; import igniteFormFieldDown from './field/down.directive'; -import igniteFormControlFeedback from './field/form-control-feedback.directive'; angular .module('ignite-console.Form', [ @@ -62,20 +62,14 @@ angular ]) // Panel. .directive(...igniteFormPanel) +.directive(...igniteFormPanelField) .directive(...igniteFormPanelChevron) .directive(...igniteFormRevert) // Field. -.directive(...igniteFormField) .directive(...igniteFormFieldLabel) .directive(...igniteFormFieldTooltip) -.directive(...igniteFormFieldDropdown) -.directive(...igniteFormFieldInputNumber) -.directive(...igniteFormFieldInputText) -.directive(...igniteFormFieldInputCheckbox) -.directive(...igniteFormFieldInputDatalist) .directive(...placeholder) // Group. -.directive(...igniteFormGroup) .directive(...igniteFormGroupAdd) .directive(...igniteFormGroupTooltip) // Validators. @@ -88,11 +82,12 @@ angular .directive(...propertyValueSpecified) .directive(...propertyUnique) .directive(...unique) +.directive(...uuid) // Helpers. .directive(...igniteFormFieldInputAutofocus) +.directive(...igniteFormControlFeedback) .directive(...igniteFormFieldUp) .directive(...igniteFormFieldDown) -.directive(...igniteFormControlFeedback) // Generator of globally unique identifier. .factory('IgniteFormGUID', [() => { let guid = 0; diff --git a/modules/web-console/src/main/js/app/modules/form/group/add.directive.js b/modules/web-console/frontend/app/modules/form/group/add.directive.js similarity index 96% rename from modules/web-console/src/main/js/app/modules/form/group/add.directive.js rename to modules/web-console/frontend/app/modules/form/group/add.directive.js index 98560b59cdf06..7e9a50c02717e 100644 --- a/modules/web-console/src/main/js/app/modules/form/group/add.directive.js +++ b/modules/web-console/frontend/app/modules/form/group/add.directive.js @@ -35,6 +35,6 @@ export default ['igniteFormGroupAdd', ['$tooltip', ($tooltip) => { link, replace: true, transclude: true, - require: ['^form', '^igniteFormGroup'] + require: ['^form'] }; }]]; diff --git a/modules/web-console/src/main/js/app/modules/form/group/tooltip.directive.js b/modules/web-console/frontend/app/modules/form/group/tooltip.directive.js similarity index 96% rename from modules/web-console/src/main/js/app/modules/form/group/tooltip.directive.js rename to modules/web-console/frontend/app/modules/form/group/tooltip.directive.js index 2202e7b712e94..3e470e1c37c81 100644 --- a/modules/web-console/src/main/js/app/modules/form/group/tooltip.directive.js +++ b/modules/web-console/frontend/app/modules/form/group/tooltip.directive.js @@ -35,6 +35,6 @@ export default ['igniteFormGroupTooltip', ['$tooltip', ($tooltip) => { link, replace: true, transclude: true, - require: ['^form', '^igniteFormGroup'] + require: ['^form'] }; }]]; diff --git a/modules/web-console/src/main/js/app/modules/form/panel/chevron.directive.js b/modules/web-console/frontend/app/modules/form/panel/chevron.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/form/panel/chevron.directive.js rename to modules/web-console/frontend/app/modules/form/panel/chevron.directive.js diff --git a/modules/web-console/frontend/app/modules/form/panel/field.directive.js b/modules/web-console/frontend/app/modules/form/panel/field.directive.js new file mode 100644 index 0000000000000..5dc7b079dc2b8 --- /dev/null +++ b/modules/web-console/frontend/app/modules/form/panel/field.directive.js @@ -0,0 +1,69 @@ +/* + * 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. + */ + +export default ['igniteFormPanelField', ['$parse', 'IgniteLegacyTable', ($parse, LegacyTable) => { + const link = (scope, element, attrs, [ngModelCtrl, formCtrl]) => { + formCtrl.$defaults = formCtrl.$defaults || {}; + + const { name, ngModel } = attrs; + const getter = () => $parse(ngModel)(scope); + + const saveDefault = () => { + formCtrl.$defaults[name] = _.cloneDeep(getter()); + }; + + const resetDefault = () => { + ngModelCtrl.$viewValue = formCtrl.$defaults[name]; + + ngModelCtrl.$valid = true; + ngModelCtrl.$invalid = false; + ngModelCtrl.$error = {}; + ngModelCtrl.$render(); + }; + + if (!(_.isNull(formCtrl.$defaults[name]) || _.isUndefined(formCtrl.$defaults[name]))) + resetDefault(); + else + saveDefault(); + + scope.tableReset = () => { + if (!LegacyTable.tableSaveAndReset()) + LegacyTable.tableReset(); + }; + + scope.$watch(() => formCtrl.$pristine, () => { + if (!formCtrl.$pristine) + return; + + saveDefault(); + resetDefault(); + }); + + scope.$watch(() => ngModelCtrl.$modelValue, () => { + if (!formCtrl.$pristine) + return; + + saveDefault(); + }); + }; + + return { + restrict: 'A', + link, + require: ['ngModel', '^form'] + }; +}]]; diff --git a/modules/web-console/src/main/js/app/modules/form/panel/panel.directive.js b/modules/web-console/frontend/app/modules/form/panel/panel.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/form/panel/panel.directive.js rename to modules/web-console/frontend/app/modules/form/panel/panel.directive.js diff --git a/modules/web-console/src/main/js/app/modules/form/panel/revert.directive.js b/modules/web-console/frontend/app/modules/form/panel/revert.directive.js similarity index 97% rename from modules/web-console/src/main/js/app/modules/form/panel/revert.directive.js rename to modules/web-console/frontend/app/modules/form/panel/revert.directive.js index d60efb8f20dff..2076b0d428217 100644 --- a/modules/web-console/src/main/js/app/modules/form/panel/revert.directive.js +++ b/modules/web-console/frontend/app/modules/form/panel/revert.directive.js @@ -32,6 +32,7 @@ export default ['igniteFormRevert', ['$tooltip', 'IgniteLegacyTable', ($tooltip, const field = form[name]; if (field) { + field.$viewValue = value; field.$setViewValue(value); field.$setPristine(); field.$render(); diff --git a/modules/web-console/src/main/js/app/modules/form/validator/ipaddress.directive.js b/modules/web-console/frontend/app/modules/form/validator/ipaddress.directive.js similarity index 82% rename from modules/web-console/src/main/js/app/modules/form/validator/ipaddress.directive.js rename to modules/web-console/frontend/app/modules/form/validator/ipaddress.directive.js index 2ddc78620c84b..77e63f6034f6f 100644 --- a/modules/web-console/src/main/js/app/modules/form/validator/ipaddress.directive.js +++ b/modules/web-console/frontend/app/modules/form/validator/ipaddress.directive.js @@ -35,14 +35,14 @@ export default ['ipaddress', ['IgniteInetAddress', (InetAddress) => { const portRange = !_.isNil(attrs.ipaddressWithPortRange); if (attrs.ipaddressWithPort) { - ngModel.$validators.ipaddressPort = (modelValue, viewValue) => { - if (isEmpty(modelValue) || viewValue.indexOf(':') === -1) + ngModel.$validators.ipaddressPort = (modelValue) => { + if (isEmpty(modelValue) || modelValue.indexOf(':') === -1) return true; - if ((viewValue.match(/:/g) || []).length > 1) + if ((modelValue.match(/:/g) || []).length > 1) return false; - const {ports} = parse(viewValue); + const {ports} = parse(modelValue); if (ports.length !== 1) return portRange; @@ -52,11 +52,11 @@ export default ['ipaddress', ['IgniteInetAddress', (InetAddress) => { } if (portRange) { - ngModel.$validators.ipaddressPortRange = (modelValue, viewValue) => { - if (isEmpty(modelValue) || viewValue.indexOf('..') === -1) + ngModel.$validators.ipaddressPortRange = (modelValue) => { + if (isEmpty(modelValue) || modelValue.indexOf('..') === -1) return true; - const {ports} = parse(viewValue); + const {ports} = parse(modelValue); if (ports.length !== 2) return false; @@ -65,11 +65,11 @@ export default ['ipaddress', ['IgniteInetAddress', (InetAddress) => { }; } - ngModel.$validators.ipaddress = (modelValue, viewValue) => { + ngModel.$validators.ipaddress = (modelValue) => { if (isEmpty(modelValue)) return true; - const {ipOrHost, ports} = parse(viewValue); + const {ipOrHost, ports} = parse(modelValue); if (attrs.ipaddressWithPort || attrs.ipaddressWithPortRange || ports.length === 0) return InetAddress.validHost(ipOrHost); diff --git a/modules/web-console/src/main/js/app/modules/form/validator/java-built-in-class.directive.js b/modules/web-console/frontend/app/modules/form/validator/java-built-in-class.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/form/validator/java-built-in-class.directive.js rename to modules/web-console/frontend/app/modules/form/validator/java-built-in-class.directive.js diff --git a/modules/web-console/src/main/js/app/modules/form/validator/java-identifier.directive.js b/modules/web-console/frontend/app/modules/form/validator/java-identifier.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/form/validator/java-identifier.directive.js rename to modules/web-console/frontend/app/modules/form/validator/java-identifier.directive.js diff --git a/modules/web-console/src/main/js/app/modules/form/validator/java-keywords.directive.js b/modules/web-console/frontend/app/modules/form/validator/java-keywords.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/form/validator/java-keywords.directive.js rename to modules/web-console/frontend/app/modules/form/validator/java-keywords.directive.js diff --git a/modules/web-console/src/main/js/app/modules/form/validator/java-package-name.directive.js b/modules/web-console/frontend/app/modules/form/validator/java-package-name.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/form/validator/java-package-name.directive.js rename to modules/web-console/frontend/app/modules/form/validator/java-package-name.directive.js diff --git a/modules/web-console/src/main/js/app/modules/form/validator/java-package-specified.directive.js b/modules/web-console/frontend/app/modules/form/validator/java-package-specified.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/form/validator/java-package-specified.directive.js rename to modules/web-console/frontend/app/modules/form/validator/java-package-specified.directive.js diff --git a/modules/web-console/src/main/js/app/modules/form/validator/property-unique.directive.js b/modules/web-console/frontend/app/modules/form/validator/property-unique.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/form/validator/property-unique.directive.js rename to modules/web-console/frontend/app/modules/form/validator/property-unique.directive.js diff --git a/modules/web-console/src/main/js/app/modules/form/validator/property-value-specified.directive.js b/modules/web-console/frontend/app/modules/form/validator/property-value-specified.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/form/validator/property-value-specified.directive.js rename to modules/web-console/frontend/app/modules/form/validator/property-value-specified.directive.js diff --git a/modules/web-console/src/main/js/app/modules/form/validator/unique.directive.js b/modules/web-console/frontend/app/modules/form/validator/unique.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/form/validator/unique.directive.js rename to modules/web-console/frontend/app/modules/form/validator/unique.directive.js diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision.directive.js b/modules/web-console/frontend/app/modules/form/validator/uuid.directive.js similarity index 61% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision.directive.js rename to modules/web-console/frontend/app/modules/form/validator/uuid.directive.js index b8e0b43a8e25d..070417563bc91 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision.directive.js +++ b/modules/web-console/frontend/app/modules/form/validator/uuid.directive.js @@ -15,13 +15,23 @@ * limitations under the License. */ -import templateUrl from './collision.jade'; +export default ['uuid', ['JavaTypes', (JavaTypes) => { + const link = (scope, el, attrs, [ngModel]) => { + const isEmpty = (modelValue) => { + return ngModel.$isEmpty(modelValue) || _.isUndefined(attrs.uuid) || attrs.uuid !== 'true'; + }; + + ngModel.$validators.uuid = (modelValue) => { + if (isEmpty(modelValue)) + return true; + + return JavaTypes.validUUID(modelValue); + }; + }; -export default ['igniteConfigurationClustersCollision', [() => { return { - scope: true, - restrict: 'E', - templateUrl, - replace: true + restrict: 'A', + link, + require: ['ngModel'] }; }]]; diff --git a/modules/web-console/src/main/js/app/modules/getting-started/GettingStarted.provider.js b/modules/web-console/frontend/app/modules/getting-started/GettingStarted.provider.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/getting-started/GettingStarted.provider.js rename to modules/web-console/frontend/app/modules/getting-started/GettingStarted.provider.js diff --git a/modules/web-console/src/main/js/app/modules/loading/loading.css b/modules/web-console/frontend/app/modules/loading/loading.css similarity index 100% rename from modules/web-console/src/main/js/app/modules/loading/loading.css rename to modules/web-console/frontend/app/modules/loading/loading.css diff --git a/modules/web-console/src/main/js/app/modules/loading/loading.directive.js b/modules/web-console/frontend/app/modules/loading/loading.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/loading/loading.directive.js rename to modules/web-console/frontend/app/modules/loading/loading.directive.js diff --git a/modules/web-console/src/main/js/app/modules/loading/loading.jade b/modules/web-console/frontend/app/modules/loading/loading.jade similarity index 100% rename from modules/web-console/src/main/js/app/modules/loading/loading.jade rename to modules/web-console/frontend/app/modules/loading/loading.jade diff --git a/modules/web-console/src/main/js/app/modules/loading/loading.module.js b/modules/web-console/frontend/app/modules/loading/loading.module.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/loading/loading.module.js rename to modules/web-console/frontend/app/modules/loading/loading.module.js diff --git a/modules/web-console/src/main/js/app/modules/loading/loading.service.js b/modules/web-console/frontend/app/modules/loading/loading.service.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/loading/loading.service.js rename to modules/web-console/frontend/app/modules/loading/loading.service.js diff --git a/modules/web-console/src/main/js/app/modules/navbar/Navbar.provider.js b/modules/web-console/frontend/app/modules/navbar/Navbar.provider.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/navbar/Navbar.provider.js rename to modules/web-console/frontend/app/modules/navbar/Navbar.provider.js diff --git a/modules/web-console/src/main/js/app/modules/navbar/Userbar.provider.js b/modules/web-console/frontend/app/modules/navbar/Userbar.provider.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/navbar/Userbar.provider.js rename to modules/web-console/frontend/app/modules/navbar/Userbar.provider.js diff --git a/modules/web-console/src/main/js/app/modules/navbar/navbar.directive.js b/modules/web-console/frontend/app/modules/navbar/navbar.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/navbar/navbar.directive.js rename to modules/web-console/frontend/app/modules/navbar/navbar.directive.js diff --git a/modules/web-console/src/main/js/app/modules/navbar/navbar.module.js b/modules/web-console/frontend/app/modules/navbar/navbar.module.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/navbar/navbar.module.js rename to modules/web-console/frontend/app/modules/navbar/navbar.module.js diff --git a/modules/web-console/src/main/js/app/modules/navbar/userbar.directive.js b/modules/web-console/frontend/app/modules/navbar/userbar.directive.js similarity index 75% rename from modules/web-console/src/main/js/app/modules/navbar/userbar.directive.js rename to modules/web-console/frontend/app/modules/navbar/userbar.directive.js index 0e9406369913e..af70eb1b1ec02 100644 --- a/modules/web-console/src/main/js/app/modules/navbar/userbar.directive.js +++ b/modules/web-console/frontend/app/modules/navbar/userbar.directive.js @@ -18,24 +18,24 @@ export default ['igniteUserbar', [function() { return { restrict: 'A', - controller: ['$rootScope', 'IgniteUserbar', function($root, IgniteUserbar) { + controller: ['$rootScope', 'IgniteUserbar', 'AclService', function($root, IgniteUserbar, AclService) { const ctrl = this; ctrl.items = [ {text: 'Profile', sref: 'settings.profile'}, - {text: 'Getting Started', click: 'gettingStarted.tryShow(true)'} + {text: 'Getting started', click: 'gettingStarted.tryShow(true)'} ]; - const _rebuildSettings = (event, user) => { + const _rebuildSettings = () => { ctrl.items.splice(2); - if (!user.becomeUsed && user.admin) - ctrl.items.push({text: 'Admin Panel', sref: 'settings.admin'}); + if (AclService.can('admin_page')) + ctrl.items.push({text: 'Admin panel', sref: 'settings.admin'}); ctrl.items.push(...IgniteUserbar); - if (!user.becomeUsed) - ctrl.items.push({text: 'Log Out', sref: 'logout'}); + if (AclService.can('logout')) + ctrl.items.push({text: 'Log out', sref: 'logout'}); }; if ($root.user) diff --git a/modules/web-console/src/main/js/app/modules/socket.module.js b/modules/web-console/frontend/app/modules/socket.module.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/socket.module.js rename to modules/web-console/frontend/app/modules/socket.module.js diff --git a/modules/web-console/frontend/app/modules/sql/Notebook.data.js b/modules/web-console/frontend/app/modules/sql/Notebook.data.js new file mode 100644 index 0000000000000..aef72ebce508f --- /dev/null +++ b/modules/web-console/frontend/app/modules/sql/Notebook.data.js @@ -0,0 +1,157 @@ +/* + * 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. + */ + +const DEMO_NOTEBOOK = { + name: 'SQL demo', + paragraphs: [ + { + name: 'Query with refresh rate', + cacheName: 'CarCache', + pageSize: 50, + query: [ + 'SELECT count(*)', + 'FROM "CarCache".Car' + ].join('\n'), + result: 'bar', + timeLineSpan: '1', + rate: { + value: 3, + unit: 1000, + installed: true + } + }, + { + name: 'Simple query', + cacheName: 'CarCache', + pageSize: 50, + query: 'SELECT * FROM "CarCache".Car', + result: 'table', + timeLineSpan: '1', + rate: { + value: 30, + unit: 1000, + installed: false + } + }, + { + name: 'Query with aggregates', + cacheName: 'CarCache', + pageSize: 50, + query: [ + 'SELECT p.name, count(*) AS cnt', + 'FROM "ParkingCache".Parking p', + 'INNER JOIN "CarCache".Car c', + ' ON (p.id) = (c.parkingId)', + 'GROUP BY P.NAME' + ].join('\n'), + result: 'table', + timeLineSpan: '1', + rate: { + value: 30, + unit: 1000, + installed: false + } + } + ], + expandedParagraphs: [0, 1, 2] +}; + +export default class NotebookData { + static $inject = ['$rootScope', '$http', '$q']; + + constructor($root, $http, $q) { + this.demo = $root.IgniteDemoMode; + + this.initLatch = null; + this.notebooks = null; + + this.$http = $http; + this.$q = $q; + } + + read() { + if (!_.isNil(this.initLatch)) + return this.initLatch; + + if (this.demo) + return this.initLatch = this.$q.when(this.notebooks = [DEMO_NOTEBOOK]); + + return this.initLatch = this.$http.get('/api/v1/notebooks') + .then(({data}) => this.notebooks = data) + .catch(({data}) => Promise.reject(data)); + } + + find(_id) { + return this.read() + .then(() => { + const notebook = this.demo ? this.notebooks[0] : _.find(this.notebooks, {_id}); + + if (_.isNil(notebook)) + return this.$q.reject('Failed to load notebook.'); + + return notebook; + }); + } + + findIndex(notebook) { + return this.read() + .then(() => _.findIndex(this.notebooks, {_id: notebook._id})); + } + + save(notebook) { + if (this.demo) + return this.$q.when(DEMO_NOTEBOOK); + + return this.$http.post('/api/v1/notebooks/save', notebook) + .then(({data}) => { + const idx = _.findIndex(this.notebooks, {_id: data._id}); + + if (idx >= 0) + this.notebooks[idx] = data; + else + this.notebooks.push(data); + + return data; + }) + .catch(({data}) => Promise.reject(data)); + } + + remove(notebook) { + if (this.demo) + return this.$q.reject(`Removing "${notebook.name}" notebook is not supported.`); + + const key = {_id: notebook._id}; + + return this.$http.post('/api/v1/notebooks/remove', key) + .then(() => { + const idx = _.findIndex(this.notebooks, key); + + if (idx >= 0) { + this.notebooks.splice(idx, 1); + + if (idx < this.notebooks.length) + return this.notebooks[idx]; + } + + if (this.notebooks.length > 0) + return this.notebooks[this.notebooks.length - 1]; + + return null; + }) + .catch(({data}) => Promise.reject(data)); + } +} diff --git a/modules/web-console/frontend/app/modules/sql/Notebook.service.js b/modules/web-console/frontend/app/modules/sql/Notebook.service.js new file mode 100644 index 0000000000000..12730be686611 --- /dev/null +++ b/modules/web-console/frontend/app/modules/sql/Notebook.service.js @@ -0,0 +1,74 @@ +/* + * 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. + */ + +export default class Notebook { + static $inject = ['$state', 'IgniteConfirm', 'IgniteMessages', 'IgniteNotebookData']; + + /** + * @param $state + * @param confirmModal + * @param Messages + * @param {NotebookData} NotebookData + */ + constructor($state, confirmModal, Messages, NotebookData) { + this.$state = $state; + this.confirmModal = confirmModal; + this.Messages = Messages; + this.NotebookData = NotebookData; + } + + read() { + return this.NotebookData.read(); + } + + create(name) { + return this.NotebookData.save({name}); + } + + save(notebook) { + return this.NotebookData.save(notebook); + } + + find(_id) { + return this.NotebookData.find(_id); + } + + _openNotebook(idx) { + return this.NotebookData.read() + .then((notebooks) => { + const nextNotebook = notebooks.length > idx ? notebooks[idx] : _.last(notebooks); + + if (nextNotebook) + this.$state.go('base.sql.notebook', {noteId: nextNotebook._id}); + else + this.$state.go('base.configuration.clusters'); + }); + } + + remove(notebook) { + return this.confirmModal.confirm(`Are you sure you want to remove: "${notebook.name}"?`) + .then(() => this.NotebookData.findIndex(notebook)) + .then((idx) => { + this.NotebookData.remove(notebook) + .then(() => { + if (this.$state.includes('base.sql.notebook') && this.$state.params.noteId === notebook._id) + return this._openNotebook(idx); + }) + .catch(this.Messages.showError); + }); + } +} diff --git a/modules/web-console/frontend/app/modules/sql/notebook.controller.js b/modules/web-console/frontend/app/modules/sql/notebook.controller.js new file mode 100644 index 0000000000000..f10a4d005da4e --- /dev/null +++ b/modules/web-console/frontend/app/modules/sql/notebook.controller.js @@ -0,0 +1,60 @@ +/* + * 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. + */ + +// Controller that load notebooks in navigation bar . +export default ['$scope', '$modal', '$state', 'IgniteMessages', 'IgniteNotebook', + (scope, $modal, $state, Messages, Notebook) => { + // Pre-fetch modal dialogs. + const nameModal = $modal({scope, templateUrl: '/sql/notebook-new.html', show: false}); + + scope.create = (name) => { + return Notebook.create(name) + .then((notebook) => { + nameModal.hide(); + + $state.go('base.sql.notebook', {noteId: notebook._id}); + }) + .catch(Messages.showError); + }; + + scope.createNotebook = () => nameModal.$promise.then(nameModal.show); + + Notebook.read() + .then((notebooks) => { + scope.$watchCollection(() => notebooks, (changed) => { + if (!changed.length) + return scope.notebooks = []; + + scope.notebooks = [ + {text: 'Create new notebook', click: scope.createNotebook}, + {divider: true} + ]; + + _.forEach(changed, (notebook) => scope.notebooks.push({ + data: notebook, + action: { + icon: 'fa-trash', + click: (item) => Notebook.remove(item) + }, + text: notebook.name, + sref: `base.sql.notebook({noteId:"${notebook._id}"})` + })); + }); + }) + .catch(Messages.showError); + } +]; diff --git a/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade b/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade new file mode 100644 index 0000000000000..addc5f3f06a2a --- /dev/null +++ b/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade @@ -0,0 +1,39 @@ +//- + 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. + +include ../../helpers/jade/mixins.jade + +.modal(tabindex='-1' role='dialog') + .modal-dialog + .modal-content + .modal-header + button.close(ng-click='$hide()') × + h4.modal-title Scan filter + form.form-horizontal.modal-body.row(name='ui.inputForm' novalidate) + .settings-row + .col-sm-2 + label.required.labelFormField Filter:  + .col-sm-10 + .input-tip + +ignite-form-field-input('"filter"', 'ui.filter', false, 'true', 'Enter filter')( + data-ignite-form-field-input-autofocus='true' + ignite-on-enter='form.$valid && ok()' + ) + .settings-row + +checkbox('Case sensitive search', 'ui.caseSensitive', '"caseSensitive"', 'Select this checkbox for case sensitive search') + .modal-footer + button.btn.btn-default(id='btn-cancel' ng-click='$hide()') Cancel + button.btn.btn-primary(id='btn-scan' ng-disabled='ui.inputForm.$invalid' ng-click='ok()') Scan diff --git a/modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js b/modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js new file mode 100644 index 0000000000000..18ba3baa2107f --- /dev/null +++ b/modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js @@ -0,0 +1,51 @@ +/* + * 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. + */ + +export default class ScanFilter { + static $inject = ['$rootScope', '$q', '$modal']; + + constructor($root, $q, $modal) { + this.deferred = null; + this.$q = $q; + + const scope = $root.$new(); + + scope.ui = {}; + + scope.ok = () => { + this.deferred.resolve({filter: scope.ui.filter, caseSensitive: !!scope.ui.caseSensitive}); + + this.modal.hide(); + }; + + scope.$hide = () => { + this.modal.hide(); + + this.deferred.reject(); + }; + + this.modal = $modal({templateUrl: '/scan-filter-input.html', scope, placement: 'center', show: false}); + } + + open() { + this.deferred = this.$q.defer(); + + this.modal.$promise.then(this.modal.show); + + return this.deferred.promise; + } +} diff --git a/modules/web-console/src/main/js/controllers/sql-controller.js b/modules/web-console/frontend/app/modules/sql/sql.controller.js similarity index 78% rename from modules/web-console/src/main/js/controllers/sql-controller.js rename to modules/web-console/frontend/app/modules/sql/sql.controller.js index 7a851c38b9d8f..92eb7be39bdae 100644 --- a/modules/web-console/src/main/js/controllers/sql-controller.js +++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js @@ -15,10 +15,147 @@ * limitations under the License. */ +// Time line X axis descriptor. +const TIME_LINE = {value: -1, type: 'java.sql.Date', label: 'TIME_LINE'}; + +// Row index X axis descriptor. +const ROW_IDX = {value: -2, type: 'java.lang.Integer', label: 'ROW_IDX'}; + +/** Prefix for node local key for SCAN near queries. */ +const SCAN_CACHE_WITH_FILTER = 'VISOR_SCAN_CACHE_WITH_FILTER'; + +/** Prefix for node local key for SCAN near queries. */ +const SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE = 'VISOR_SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE'; + +const _fullColName = (col) => { + const res = []; + + if (col.schemaName) + res.push(col.schemaName); + + if (col.typeName) + res.push(col.typeName); + + res.push(col.fieldName); + + return res.join('.'); +}; + +let paragraphId = 0; + +class Paragraph { + constructor($animate, $timeout, paragraph) { + const self = this; + + self.id = 'paragraph-' + paragraphId++; + + _.assign(this, paragraph); + + Object.defineProperty(this, 'gridOptions', {value: { + enableGridMenu: false, + enableColumnMenus: false, + flatEntityAccess: true, + fastWatch: true, + rebuildColumns() { + if (_.isNil(this.api)) + return; + + this.columnDefs = _.reduce(self.meta, (cols, col, idx) => { + if (self.columnFilter(col)) { + cols.push({ + displayName: col.fieldName, + headerTooltip: _fullColName(col), + field: idx.toString(), + minWidth: 50, + cellClass: 'cell-left' + }); + } + + return cols; + }, []); + + $timeout(() => this.api.core.notifyDataChange('column')); + }, + adjustHeight() { + if (_.isNil(this.api)) + return; + + this.data = self.rows; + + const height = Math.min(self.rows.length, 15) * 30 + 47; + + // Remove header height. + this.api.grid.element.css('height', height + 'px'); + + $timeout(() => this.api.core.handleWindowResize()); + }, + onRegisterApi(api) { + $animate.enabled(api.grid.element, false); + + this.api = api; + + this.rebuildColumns(); + + this.adjustHeight(); + } + }}); + + Object.defineProperty(this, 'chartHistory', {value: []}); + } + + resultType() { + if (_.isNil(this.queryArgs)) + return null; + + if (!_.isEmpty(this.errMsg)) + return 'error'; + + if (_.isEmpty(this.rows)) + return 'empty'; + + return this.result === 'table' ? 'table' : 'chart'; + } + + nonRefresh() { + return _.isNil(this.rate) || _.isNil(this.rate.stopTime); + } + + table() { + return this.result === 'table'; + } + + chart() { + return this.result !== 'table' && this.result !== 'none'; + } + + nonEmpty() { + return this.rows && this.rows.length > 0; + } + + queryExecuted() { + return !_.isEmpty(this.meta); + } + + scanExplain() { + return this.queryExecuted() && this.queryArgs.type !== 'QUERY'; + } + + timeLineSupported() { + return this.result !== 'pie'; + } + + chartColumnsConfigured() { + return !_.isEmpty(this.chartKeyCols) && !_.isEmpty(this.chartValCols); + } + + chartTimeLineEnabled() { + return !_.isEmpty(this.chartKeyCols) && _.eq(this.chartKeyCols[0], TIME_LINE); + } +} + // Controller for SQL notebook screen. -export default ['sqlController', [ - '$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', '$animate', '$location', '$anchorScroll', '$state', '$modal', '$popover', 'IgniteLoading', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteAgentMonitor', 'IgniteChartColors', 'QueryNotebooks', 'uiGridConstants', 'uiGridExporterConstants', - function($root, $scope, $http, $q, $timeout, $interval, $animate, $location, $anchorScroll, $state, $modal, $popover, Loading, LegacyUtils, Messages, Confirm, agentMonitor, IgniteChartColors, QueryNotebooks, uiGridConstants, uiGridExporterConstants) { +export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', '$animate', '$location', '$anchorScroll', '$state', '$modal', '$popover', 'IgniteLoading', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteAgentMonitor', 'IgniteChartColors', 'IgniteNotebook', 'IgniteScanFilterInput', 'uiGridExporterConstants', + function($root, $scope, $http, $q, $timeout, $interval, $animate, $location, $anchorScroll, $state, $modal, $popover, Loading, LegacyUtils, Messages, Confirm, agentMonitor, IgniteChartColors, Notebook, ScanFilterInput, uiGridExporterConstants) { let stopTopology = null; const _tryStopRefresh = function(paragraph) { @@ -77,12 +214,6 @@ export default ['sqlController', [ $scope.maskCacheName = (cacheName) => _.isEmpty(cacheName) ? '' : cacheName; - // Time line X axis descriptor. - const TIME_LINE = {value: -1, type: 'java.sql.Date', label: 'TIME_LINE'}; - - // Row index X axis descriptor. - const ROW_IDX = {value: -2, type: 'java.lang.Integer', label: 'ROW_IDX'}; - // We need max 1800 items to hold history for 30 mins in case of refresh every second. const HISTORY_LENGTH = 1800; @@ -604,8 +735,8 @@ export default ['sqlController', [ }); }; - $scope.scrollToParagraph = function(paragraphId) { - const idx = _.findIndex($scope.notebook.paragraphs, {id: paragraphId}); + $scope.scrollToParagraph = (id) => { + const idx = _.findIndex($scope.notebook.paragraphs, {id}); if (idx >= 0) { if (!_.includes($scope.notebook.expandedParagraphs, idx)) @@ -616,7 +747,7 @@ export default ['sqlController', [ }); } - $location.hash(paragraphId); + $location.hash(id); $anchorScroll(); }; @@ -625,96 +756,6 @@ export default ['sqlController', [ const _allColumn = () => true; - let paragraphId = 0; - - const _fullColName = function(col) { - const res = []; - - if (col.schemaName) - res.push(col.schemaName); - - if (col.typeName) - res.push(col.typeName); - - res.push(col.fieldName); - - return res.join('.'); - }; - - function enhanceParagraph(paragraph) { - paragraph.nonEmpty = function() { - return this.rows && this.rows.length > 0; - }; - - paragraph.chart = function() { - return this.result !== 'table' && this.result !== 'none'; - }; - - paragraph.queryExecuted = () => - paragraph.queryArgs && paragraph.queryArgs.query && !paragraph.queryArgs.query.startsWith('EXPLAIN '); - - paragraph.table = function() { - return this.result === 'table'; - }; - - paragraph.chartColumnsConfigured = function() { - return !_.isEmpty(this.chartKeyCols) && !_.isEmpty(this.chartValCols); - }; - - paragraph.chartTimeLineEnabled = function() { - return !_.isEmpty(this.chartKeyCols) && angular.equals(this.chartKeyCols[0], TIME_LINE); - }; - - paragraph.timeLineSupported = function() { - return this.result !== 'pie'; - }; - - paragraph.refreshExecuting = function() { - return paragraph.rate && paragraph.rate.stopTime; - }; - - Object.defineProperty(paragraph, 'gridOptions', { value: { - enableGridMenu: false, - enableColumnMenus: false, - flatEntityAccess: true, - fastWatch: true, - updateColumns(cols) { - this.columnDefs = _.map(cols, (col) => { - return { - displayName: col.fieldName, - headerTooltip: _fullColName(col), - field: col.field, - minWidth: 50, - cellClass: 'cell-left' - }; - }); - - $timeout(() => this.api.core.notifyDataChange(uiGridConstants.dataChange.COLUMN)); - }, - updateRows(rows) { - const sizeChanged = this.data.length !== rows.length; - - this.data = rows; - - if (sizeChanged) { - const height = Math.min(rows.length, 15) * 30 + 47; - - // Remove header height. - this.api.grid.element.css('height', height + 'px'); - - $timeout(() => this.api.core.handleWindowResize()); - } - }, - onRegisterApi(api) { - $animate.enabled(api.grid.element, false); - - this.api = api; - } - }}); - - Object.defineProperty(paragraph, 'chartHistory', {value: []}); - } - $scope.aceInit = function(paragraph) { return function(editor) { editor.setAutoScrollEditorIntoView(true); @@ -735,74 +776,46 @@ export default ['sqlController', [ }; }; - const _setActiveCache = function() { - if ($scope.caches.length > 0) { - _.forEach($scope.notebook.paragraphs, (paragraph) => { - if (!_.find($scope.caches, {name: paragraph.cacheName})) - paragraph.cacheName = $scope.caches[0].name; - }); - } - }; - - const _updateTopology = () => + /** + * Update caches list. + * @private + */ + const _refreshFn = () => agentMonitor.topology() .then((clusters) => { - agentMonitor.checkModal(); - - const caches = _.flattenDeep(clusters.map((cluster) => cluster.caches)); - - $scope.caches = _.sortBy(_.map(_.uniqBy(_.reject(caches, {mode: 'LOCAL'}), 'name'), (cache) => { - cache.label = $scope.maskCacheName(cache.name); - - return cache; - }), 'label'); + $scope.caches = _.sortBy(_.reduce(clusters, (items, cluster) => { + _.forEach(cluster.caches, (cache) => { + let item = _.find(items, {name: cache.name}); - _setActiveCache(); - }) - .catch((err) => { - if (err.code === 2) - return agentMonitor.showNodeError('Agent is failed to authenticate in grid. Please check agent\'s login and password.'); - - agentMonitor.showNodeError(err); - }); - - const _startTopologyRefresh = () => { - Loading.start('sqlLoading'); - - agentMonitor.awaitAgent() - .then(_updateTopology) - .finally(() => { - if ($root.IgniteDemoMode) - _.forEach($scope.notebook.paragraphs, $scope.execute); - - Loading.finish('sqlLoading'); - - stopTopology = $interval(_updateTopology, 5000, 0, false); - }); - }; + if (_.isNil(item)) { + cache.label = $scope.maskCacheName(cache.name); - const loadNotebook = function(notebook) { - $scope.notebook = notebook; + cache.nodeIds = []; - $scope.notebook_name = notebook.name; + items.push(item = cache); + } - if (!$scope.notebook.expandedParagraphs) - $scope.notebook.expandedParagraphs = []; + item.nodeIds.push(cluster.nodeId); + }); - if (!$scope.notebook.paragraphs) - $scope.notebook.paragraphs = []; + return items; + }, []), 'label'); - _.forEach(notebook.paragraphs, (paragraph) => { - paragraph.id = 'paragraph-' + paragraphId++; + if (_.isEmpty($scope.caches)) + return; - enhanceParagraph(paragraph); - }); + // Reset to first cache in case of stopped selected. + const cacheNames = _.map($scope.caches, (cache) => cache.name); - if (!notebook.paragraphs || notebook.paragraphs.length === 0) - $scope.addParagraph(); - else - $scope.rebuildScrollParagraphs(); + _.forEach($scope.notebook.paragraphs, (paragraph) => { + if (!_.includes(cacheNames, paragraph.cacheName)) + paragraph.cacheName = _.head(cacheNames); + }); + }) + .then(() => agentMonitor.checkModal()) + .catch((err) => agentMonitor.showNodeError(err)); + const _startWatch = () => agentMonitor.startWatch({ state: 'base.configuration.clusters', text: 'Back to Configuration', @@ -810,21 +823,46 @@ export default ['sqlController', [ onDisconnect: () => { _stopTopologyRefresh(); - _startTopologyRefresh(); + _startWatch(); } }) - .then(_startTopologyRefresh); - }; + .then(() => Loading.start('sqlLoading')) + .then(_refreshFn) + .then(() => Loading.finish('sqlLoading')) + .then(() => { + $root.IgniteDemoMode && _.forEach($scope.notebook.paragraphs, $scope.execute); + + stopTopology = $interval(_refreshFn, 5000, 0, false); + }); + + Notebook.find($state.params.noteId) + .then((notebook) => { + $scope.notebook = _.cloneDeep(notebook); + + $scope.notebook_name = $scope.notebook.name; - QueryNotebooks.read($state.params.noteId) - .then(loadNotebook) + if (!$scope.notebook.expandedParagraphs) + $scope.notebook.expandedParagraphs = []; + + if (!$scope.notebook.paragraphs) + $scope.notebook.paragraphs = []; + + $scope.notebook.paragraphs = _.map($scope.notebook.paragraphs, + (paragraph) => new Paragraph($animate, $timeout, paragraph)); + + if (_.isEmpty($scope.notebook.paragraphs)) + $scope.addParagraph(); + else + $scope.rebuildScrollParagraphs(); + }) + .then(_startWatch) .catch(() => { $scope.notebookLoadFailed = true; Loading.finish('sqlLoading'); }); - $scope.renameNotebook = function(name) { + $scope.renameNotebook = (name) => { if (!name) return; @@ -833,20 +871,8 @@ export default ['sqlController', [ $scope.notebook.name = name; - QueryNotebooks.save($scope.notebook) - .then(function() { - const idx = _.findIndex($root.notebooks, function(item) { - return item._id === $scope.notebook._id; - }); - - if (idx >= 0) { - $root.notebooks[idx].name = name; - - $root.rebuildDropdown(); - } - - $scope.notebook.edit = false; - }) + Notebook.save($scope.notebook) + .then(() => $scope.notebook.edit = false) .catch((err) => { $scope.notebook.name = prevName; @@ -857,19 +883,7 @@ export default ['sqlController', [ $scope.notebook.edit = false; }; - $scope.removeNotebook = function() { - Confirm.confirm('Are you sure you want to remove: "' + $scope.notebook.name + '"?') - .then(function() { - return QueryNotebooks.remove($scope.notebook); - }) - .then(function(notebook) { - if (notebook) - $state.go('base.sql.notebook', {noteId: notebook._id}); - else - $state.go('base.configuration.clusters'); - }) - .catch(Messages.showError); - }; + $scope.removeNotebook = (notebook) => Notebook.remove(notebook); $scope.renameParagraph = function(paragraph, newName) { if (!newName) @@ -880,7 +894,7 @@ export default ['sqlController', [ $scope.rebuildScrollParagraphs(); - QueryNotebooks.save($scope.notebook) + Notebook.save($scope.notebook) .then(() => paragraph.edit = false) .catch(Messages.showError); } @@ -891,8 +905,7 @@ export default ['sqlController', [ $scope.addParagraph = function() { const sz = $scope.notebook.paragraphs.length; - const paragraph = { - id: 'paragraph-' + paragraphId++, + const paragraph = new Paragraph($animate, $timeout, { name: 'Query' + (sz === 0 ? '' : sz), query: '', pageSize: $scope.pageSizes[0], @@ -903,9 +916,7 @@ export default ['sqlController', [ unit: 60000, installed: false } - }; - - enhanceParagraph(paragraph); + }); if ($scope.caches && $scope.caches.length > 0) paragraph.cacheName = $scope.caches[0].name; @@ -918,9 +929,9 @@ export default ['sqlController', [ $location.hash(paragraph.id); - $anchorScroll(); + $timeout(() => { + $anchorScroll(); - setTimeout(function() { paragraph.ace.focus(); }); }; @@ -958,8 +969,6 @@ export default ['sqlController', [ if (paragraph.chart()) _chartApplySettings(paragraph, true); - else - $timeout(() => paragraph.gridOptions.api.core.handleWindowResize()); }; $scope.resultEq = function(paragraph, result) { @@ -986,7 +995,7 @@ export default ['sqlController', [ $scope.rebuildScrollParagraphs(); - QueryNotebooks.save($scope.notebook) + Notebook.save($scope.notebook) .catch(Messages.showError); }); }; @@ -1060,25 +1069,15 @@ export default ['sqlController', [ }); }); - const cols = []; - - _.forEach(paragraph.meta, (col, idx) => { - if (paragraph.columnFilter(col)) { - col.field = paragraph.queryArgs.query ? idx.toString() : col.fieldName; - - cols.push(col); - } - }); - - paragraph.gridOptions.updateColumns(cols); + paragraph.gridOptions.rebuildColumns(); - paragraph.chartColumns = _.reduce(cols, (acc, col) => { - if (_notObjectType(col.fieldTypeName)) { + paragraph.chartColumns = _.reduce(paragraph.meta, (acc, col, idx) => { + if (paragraph.columnFilter(col) && _notObjectType(col.fieldTypeName)) { acc.push({ label: col.fieldName, type: col.fieldTypeName, aggFx: $scope.aggregateFxs[0], - value: col.field + value: idx.toString() }); } @@ -1114,14 +1113,14 @@ export default ['sqlController', [ /** * @param {Object} paragraph Query - * @param {{fieldsMetadata: Array, items: Array, queryId: int, last: Boolean}} res Query results. + * @param {{columns: Array, rows: Array, responseNodeId: String, queryId: int, hasMore: Boolean}} res Query results. * @private */ - const _processQueryResult = function(paragraph, res) { + const _processQueryResult = (paragraph, res) => { const prevKeyCols = paragraph.chartKeyCols; const prevValCols = paragraph.chartValCols; - if (!_.eq(paragraph.meta, res.fieldsMetadata)) { + if (!_.eq(paragraph.meta, res.columns)) { paragraph.meta = []; paragraph.chartColumns = []; @@ -1132,17 +1131,17 @@ export default ['sqlController', [ if (!LegacyUtils.isDefined(paragraph.chartValCols)) paragraph.chartValCols = []; - if (res.fieldsMetadata.length <= 2) { - const _key = _.find(res.fieldsMetadata, {fieldName: '_KEY'}); - const _val = _.find(res.fieldsMetadata, {fieldName: '_VAL'}); + if (res.columns.length <= 2) { + const _key = _.find(res.columns, {fieldName: '_KEY'}); + const _val = _.find(res.columns, {fieldName: '_VAL'}); - paragraph.disabledSystemColumns = (res.fieldsMetadata.length === 2 && _key && _val) || - (res.fieldsMetadata.length === 1 && (_key || _val)); + paragraph.disabledSystemColumns = (res.columns.length === 2 && _key && _val) || + (res.columns.length === 1 && (_key || _val)); } paragraph.columnFilter = _columnFilter(paragraph); - paragraph.meta = res.fieldsMetadata; + paragraph.meta = res.columns; _rebuildColumns(paragraph); } @@ -1151,24 +1150,28 @@ export default ['sqlController', [ paragraph.total = 0; - paragraph.queryId = res.last ? null : res.queryId; + paragraph.duration = res.duration; + + paragraph.queryId = res.hasMore ? res.queryId : null; + + paragraph.resNodeId = res.responseNodeId; delete paragraph.errMsg; // Prepare explain results for display in table. - if (paragraph.queryArgs.query && paragraph.queryArgs.query.startsWith('EXPLAIN') && res.items) { + if (paragraph.queryArgs.query && paragraph.queryArgs.query.startsWith('EXPLAIN') && res.rows) { paragraph.rows = []; - res.items.forEach(function(row, i) { - const line = res.items.length - 1 === i ? row[0] : row[0] + '\n'; + res.rows.forEach((row, i) => { + const line = res.rows.length - 1 === i ? row[0] : row[0] + '\n'; line.replace(/\"/g, '').split('\n').forEach((ln) => paragraph.rows.push([ln])); }); } else - paragraph.rows = res.items; + paragraph.rows = res.rows; - paragraph.gridOptions.updateRows(paragraph.rows); + paragraph.gridOptions.adjustHeight(paragraph.rows.length); const chartHistory = paragraph.chartHistory; @@ -1192,7 +1195,7 @@ export default ['sqlController', [ _showLoading(paragraph, false); - if (paragraph.result === 'none' || !paragraph.queryExecuted()) + if (_.isNil(paragraph.result) || paragraph.result === 'none' || paragraph.scanExplain()) paragraph.result = 'table'; else if (paragraph.chart()) { let resetCharts = queryChanged; @@ -1216,12 +1219,18 @@ export default ['sqlController', [ return queryId ? agentMonitor.queryClose(queryId) : $q.when(); }; + const cacheNode = (name) => { + const cache = _.find($scope.caches, {name}); + + return cache.nodeIds[_.random(0, cache.nodeIds.length - 1)]; + }; + const _executeRefresh = (paragraph) => { const args = paragraph.queryArgs; agentMonitor.awaitAgent() .then(() => _closeOldQuery(paragraph)) - .then(() => agentMonitor.query(args.cacheName, args.pageSize, args.query)) + .then(() => agentMonitor.query(cacheNode(args.cacheName), args.cacheName, args.query, false, args.pageSize)) .then(_processQueryResult.bind(this, paragraph)) .catch((err) => paragraph.errMsg = err.message); }; @@ -1240,8 +1249,11 @@ export default ['sqlController', [ } }; - $scope.execute = function(paragraph) { - QueryNotebooks.save($scope.notebook) + $scope.execute = (paragraph) => { + if (!$scope.actionAvailable(paragraph, true)) + return; + + Notebook.save($scope.notebook) .catch(Messages.showError); paragraph.prevQuery = paragraph.queryArgs ? paragraph.queryArgs.query : paragraph.query; @@ -1249,16 +1261,17 @@ export default ['sqlController', [ _showLoading(paragraph, true); _closeOldQuery(paragraph) - .then(function() { + .then(() => { const args = paragraph.queryArgs = { cacheName: paragraph.cacheName, pageSize: paragraph.pageSize, - query: paragraph.query + query: paragraph.query, + type: 'QUERY' }; - return agentMonitor.query(args.cacheName, args.pageSize, args.query); + return agentMonitor.query(cacheNode(paragraph.cacheName), args.cacheName, args.query, false, args.pageSize); }) - .then(function(res) { + .then((res) => { _processQueryResult(paragraph, res); _tryStartRefresh(paragraph); @@ -1270,14 +1283,10 @@ export default ['sqlController', [ $scope.stopRefresh(paragraph); }) - .finally(() => paragraph.ace.focus()); - }; - - $scope.queryExecuted = function(paragraph) { - return LegacyUtils.isDefined(paragraph.queryArgs); + .then(() => paragraph.ace.focus()); }; - const _cancelRefresh = function(paragraph) { + const _cancelRefresh = (paragraph) => { if (paragraph.rate && paragraph.rate.stopTime) { delete paragraph.queryArgs; @@ -1289,8 +1298,11 @@ export default ['sqlController', [ } }; - $scope.explain = function(paragraph) { - QueryNotebooks.save($scope.notebook) + $scope.explain = (paragraph) => { + if (!$scope.actionAvailable(paragraph, true)) + return; + + Notebook.save($scope.notebook) .catch(Messages.showError); _cancelRefresh(paragraph); @@ -1298,14 +1310,15 @@ export default ['sqlController', [ _showLoading(paragraph, true); _closeOldQuery(paragraph) - .then(function() { + .then(() => { const args = paragraph.queryArgs = { cacheName: paragraph.cacheName, pageSize: paragraph.pageSize, - query: 'EXPLAIN ' + paragraph.query + query: 'EXPLAIN ' + paragraph.query, + type: 'EXPLAIN' }; - return agentMonitor.query(args.cacheName, args.pageSize, args.query); + return agentMonitor.query(cacheNode(paragraph.cacheName), args.cacheName, args.query, false, args.pageSize); }) .then(_processQueryResult.bind(this, paragraph)) .catch((err) => { @@ -1313,11 +1326,14 @@ export default ['sqlController', [ _showLoading(paragraph, false); }) - .finally(() => paragraph.ace.focus()); + .then(() => paragraph.ace.focus()); }; - $scope.scan = function(paragraph) { - QueryNotebooks.save($scope.notebook) + $scope.scan = (paragraph, query = null) => { + if (!$scope.actionAvailable(paragraph, false)) + return; + + Notebook.save($scope.notebook) .catch(Messages.showError); _cancelRefresh(paragraph); @@ -1328,10 +1344,12 @@ export default ['sqlController', [ .then(() => { const args = paragraph.queryArgs = { cacheName: paragraph.cacheName, - pageSize: paragraph.pageSize + pageSize: paragraph.pageSize, + query, + type: 'SCAN' }; - return agentMonitor.query(args.cacheName, args.pageSize); + return agentMonitor.query(cacheNode(paragraph.cacheName), args.cacheName, query, false, args.pageSize); }) .then(_processQueryResult.bind(this, paragraph)) .catch((err) => { @@ -1339,7 +1357,19 @@ export default ['sqlController', [ _showLoading(paragraph, false); }) - .finally(() => paragraph.ace.focus()); + .then(() => paragraph.ace.focus()); + }; + + $scope.scanWithFilter = (paragraph) => { + if (!$scope.actionAvailable(paragraph, false)) + return; + + ScanFilterInput.open() + .then(({filter, caseSensitive}) => { + const prefix = caseSensitive ? SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE : SCAN_CACHE_WITH_FILTER; + + $scope.scan(paragraph, `${prefix}${filter}`); + }); }; function _updatePieChartsWithData(paragraph, newDatum) { @@ -1359,18 +1389,20 @@ export default ['sqlController', [ }); } - $scope.nextPage = function(paragraph) { + $scope.nextPage = (paragraph) => { _showLoading(paragraph, true); paragraph.queryArgs.pageSize = paragraph.pageSize; - agentMonitor.next(paragraph.queryId, paragraph.pageSize) - .then(function(res) { + agentMonitor.next(paragraph.resNodeId, paragraph.queryId, paragraph.pageSize) + .then((res) => { paragraph.page++; paragraph.total += paragraph.rows.length; - paragraph.rows = res.items; + paragraph.duration = res.duration; + + paragraph.rows = res.rows; if (paragraph.chart()) { if (paragraph.result === 'pie') @@ -1379,11 +1411,11 @@ export default ['sqlController', [ _updateChartsWithData(paragraph, _chartDatum(paragraph)); } - paragraph.gridOptions.updateRows(paragraph.rows); + paragraph.gridOptions.adjustHeight(paragraph.rows.length); _showLoading(paragraph, false); - if (res.last) + if (!res.hasMore) delete paragraph.queryId; }) .catch((err) => { @@ -1391,7 +1423,7 @@ export default ['sqlController', [ _showLoading(paragraph, false); }) - .finally(() => paragraph.ace.focus()); + .then(() => paragraph.ace.focus()); }; const _export = (fileName, columnFilter, meta, rows) => { @@ -1451,10 +1483,10 @@ export default ['sqlController', [ $scope.exportCsvAll = function(paragraph) { const args = paragraph.queryArgs; - agentMonitor.queryGetAll(args.cacheName, args.query) - .then((res) => _export(paragraph.name + '-all.csv', paragraph.columnFilter, res.fieldsMetadata, res.items)) + agentMonitor.queryGetAll(cacheNode(args.cacheName), args.cacheName, args.query, false) + .then((res) => _export(paragraph.name + '-all.csv', paragraph.columnFilter, res.columns, res.rows)) .catch(Messages.showError) - .finally(() => paragraph.ace.focus()); + .then(() => paragraph.ace.focus()); }; // $scope.exportPdfAll = function(paragraph) { @@ -1560,17 +1592,29 @@ export default ['sqlController', [ }), 'name'); }) .catch(Messages.showError) - .finally(() => Loading.finish('loadingCacheMetadata')); + .then(() => Loading.finish('loadingCacheMetadata')); }; $scope.showResultQuery = function(paragraph) { - if (LegacyUtils.isDefined(paragraph)) { + if (!_.isNil(paragraph)) { const scope = $scope.$new(); if (_.isNil(paragraph.queryArgs.query)) { scope.title = 'SCAN query'; scope.content = [`SCAN query for cache: ${$scope.maskCacheName(paragraph.queryArgs.cacheName)}`]; } + else if (paragraph.queryArgs.query.startsWith(SCAN_CACHE_WITH_FILTER)) { + scope.title = 'SCAN query'; + + let filter = ''; + + if (paragraph.queryArgs.query.startsWith(SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE)) + filter = paragraph.queryArgs.query.substr(SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE.length); + else + filter = paragraph.queryArgs.query.substr(SCAN_CACHE_WITH_FILTER.length); + + scope.content = [`SCAN query for cache: ${$scope.maskCacheName(paragraph.queryArgs.cacheName)} with filter: ${filter}`]; + } else if (paragraph.queryArgs.query .startsWith('EXPLAIN ')) { scope.title = 'Explain query'; scope.content = [paragraph.queryArgs.query]; @@ -1585,4 +1629,4 @@ export default ['sqlController', [ } }; } -]]; +]; diff --git a/modules/web-console/frontend/app/modules/sql/sql.module.js b/modules/web-console/frontend/app/modules/sql/sql.module.js new file mode 100644 index 0000000000000..d615d28e108c2 --- /dev/null +++ b/modules/web-console/frontend/app/modules/sql/sql.module.js @@ -0,0 +1,60 @@ +/* + * 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. + */ + +import angular from 'angular'; + +import NotebookData from './Notebook.data'; +import Notebook from './Notebook.service'; +import ScanFilterInput from './scan-filter-input.service'; +import notebook from './notebook.controller'; +import sql from './sql.controller'; + +angular.module('ignite-console.sql', [ + 'ui.router' +]) + .config(['$stateProvider', 'AclRouteProvider', + ($stateProvider, AclRoute) => { + // set up the states + $stateProvider + .state('base.sql', { + url: '/sql', + abstract: true, + template: '' + }) + .state('base.sql.notebook', { + url: '/notebook/{noteId}', + templateUrl: '/sql/sql.html', + onEnter: AclRoute.checkAccess('query'), + metaTags: { + title: 'Query notebook' + } + }) + .state('base.sql.demo', { + url: '/demo', + templateUrl: '/sql/sql.html', + onEnter: AclRoute.checkAccess('query'), + metaTags: { + title: 'SQL demo' + } + }); + }] + ) + .service('IgniteNotebookData', NotebookData) + .service('IgniteNotebook', Notebook) + .service('IgniteScanFilterInput', ScanFilterInput) + .controller('notebookController', notebook) + .controller('sqlController', sql); diff --git a/modules/web-console/src/main/js/app/modules/states/admin.state.js b/modules/web-console/frontend/app/modules/states/admin.state.js similarity index 89% rename from modules/web-console/src/main/js/app/modules/states/admin.state.js rename to modules/web-console/frontend/app/modules/states/admin.state.js index af1fbdeef3d37..c3151e1f442a8 100644 --- a/modules/web-console/src/main/js/app/modules/states/admin.state.js +++ b/modules/web-console/frontend/app/modules/states/admin.state.js @@ -21,12 +21,13 @@ angular .module('ignite-console.states.admin', [ 'ui.router' ]) -.config(['$stateProvider', function($stateProvider) { +.config(['$stateProvider', 'AclRouteProvider', function($stateProvider, AclRoute) { // set up the states $stateProvider .state('settings.admin', { url: '/admin', templateUrl: '/settings/admin.html', + onEnter: AclRoute.checkAccess('admin_page'), metaTags: { title: 'List of registered users' } diff --git a/modules/web-console/frontend/app/modules/states/configuration.state.js b/modules/web-console/frontend/app/modules/states/configuration.state.js new file mode 100644 index 0000000000000..7fd7541f28582 --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration.state.js @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import angular from 'angular'; + +// Common directives. +import previewPanel from './configuration/preview-panel.directive.js'; + +// Summary screen. +import ConfigurationSummaryCtrl from './configuration/summary/summary.controller'; +import ConfigurationResource from './configuration/Configuration.resource'; +import summaryTabs from './configuration/summary/summary-tabs.directive'; + +angular.module('ignite-console.states.configuration', ['ui.router']) + .directive(...previewPanel) + // Summary screen + .directive(...summaryTabs) + // Services. + .service('igniteConfigurationResource', ConfigurationResource) + // Configure state provider. + .config(['$stateProvider', 'AclRouteProvider', ($stateProvider, AclRoute) => { + // Setup the states. + $stateProvider + .state('base.configuration', { + url: '/configuration', + templateUrl: '/configuration/sidebar.html', + abstract: true + }) + .state('base.configuration.clusters', { + url: '/clusters', + templateUrl: '/configuration/clusters.html', + onEnter: AclRoute.checkAccess('configuration'), + params: { + linkId: null + }, + metaTags: { + title: 'Configure Clusters' + } + }) + .state('base.configuration.caches', { + url: '/caches', + templateUrl: '/configuration/caches.html', + onEnter: AclRoute.checkAccess('configuration'), + params: { + linkId: null + }, + metaTags: { + title: 'Configure Caches' + } + }) + .state('base.configuration.domains', { + url: '/domains', + templateUrl: '/configuration/domains.html', + onEnter: AclRoute.checkAccess('configuration'), + params: { + linkId: null + }, + metaTags: { + title: 'Configure Domain Model' + } + }) + .state('base.configuration.igfs', { + url: '/igfs', + templateUrl: '/configuration/igfs.html', + onEnter: AclRoute.checkAccess('configuration'), + params: { + linkId: null + }, + metaTags: { + title: 'Configure IGFS' + } + }) + .state('base.configuration.summary', { + url: '/summary', + templateUrl: '/configuration/summary.html', + onEnter: AclRoute.checkAccess('configuration'), + controller: ConfigurationSummaryCtrl, + controllerAs: 'ctrl', + metaTags: { + title: 'Configurations Summary' + } + }); + }]); diff --git a/modules/web-console/frontend/app/modules/states/configuration/Configuration.resource.js b/modules/web-console/frontend/app/modules/states/configuration/Configuration.resource.js new file mode 100644 index 0000000000000..0582d5c6e8f76 --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/Configuration.resource.js @@ -0,0 +1,42 @@ +/* + * 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. + */ + +export default ['$http', ($http) => { + return { + read() { + return $http.get('/api/v1/configuration/list') + .then(({data}) => data) + .catch(({data}) => Promise.reject(data)); + }, + populate({spaces, clusters, caches, igfss, domains}) { + _.forEach(clusters, (cluster) => { + cluster.caches = _.filter(caches, ({_id}) => _.includes(cluster.caches, _id)); + + _.forEach(cluster.caches, (cache) => { + cache.domains = _.filter(domains, ({_id}) => _.includes(cache.domains, _id)); + + if (_.get(cache, 'nodeFilter.kind') === 'IGFS') + cache.nodeFilter.IGFS.instance = _.find(igfss, {_id: cache.nodeFilter.IGFS.igfs}); + }); + + cluster.igfss = _.filter(igfss, ({_id}) => _.includes(cluster.igfss, _id)); + }); + + return Promise.resolve({spaces, clusters, caches, igfss, domains}); + } + }; +}]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/concurrency.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/concurrency.jade similarity index 80% rename from modules/web-console/src/main/js/app/modules/states/configuration/caches/concurrency.jade rename to modules/web-console/frontend/app/modules/states/configuration/caches/concurrency.jade index 7cf8371f16fe3..37bd88d5b4898 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/concurrency.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/concurrency.jade @@ -19,43 +19,43 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'concurrency' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Concurrency control ignite-form-field-tooltip.tipLabel - | Cache concurrent usage settings + | Cache concurrent asynchronous operations settings ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +number('Max async operations:', model + '.maxConcurrentAsyncOperations', 'maxConcurrentAsyncOperations', 'true', '500', '0', + +number('Max async operations:', model + '.maxConcurrentAsyncOperations', '"maxConcurrentAsyncOperations"', 'true', '500', '0', 'Maximum number of allowed concurrent asynchronous operations
          \ - If 0 then number of concurrent asynchronous operations is unlimited') + If 0 then number of concurrent asynchronous operations is unlimited') .settings-row - +number('Default lock timeout:', model + '.defaultLockTimeout', 'defaultLockTimeout', 'true', '0', '0', - 'Default lock acquisition timeout
          \ - If 0 then lock acquisition will never timeout') + +number('Default lock timeout:', model + '.defaultLockTimeout', '"defaultLockTimeout"', 'true', '0', '0', + 'Default lock acquisition timeout in milliseconds
          \ + If 0 then lock acquisition will never timeout') .settings-row(ng-hide='#{model}.atomicityMode === "TRANSACTIONAL"') - +dropdown('Entry versioning:', model + '.atomicWriteOrderMode', 'atomicWriteOrderMode', 'true', 'Choose versioning', + +dropdown('Entry versioning:', model + '.atomicWriteOrderMode', '"atomicWriteOrderMode"', 'true', 'Choose versioning', '[\ {value: "CLOCK", label: "CLOCK"},\ {value: "PRIMARY", label: "PRIMARY"}\ ]', - 'Write ordering mode determines which node assigns the write version, sender or the primary node:\ + 'Write ordering mode determines which node assigns the write version, sender or the primary node\
            \
          • CLOCK - in this mode write versions are assigned on a sender node which generally leads to better performance
          • \
          • PRIMARY - in this mode version is assigned only on primary node. This means that sender will only send write request to primary node, which in turn will assign write version and forward it to backups
          • \
          ') .settings-row - +dropdown('Write synchronization mode:', model + '.writeSynchronizationMode', 'writeSynchronizationMode', 'true', 'PRIMARY_SYNC', + +dropdown('Write synchronization mode:', model + '.writeSynchronizationMode', '"writeSynchronizationMode"', 'true', 'PRIMARY_SYNC', '[\ {value: "FULL_SYNC", label: "FULL_SYNC"},\ {value: "FULL_ASYNC", label: "FULL_ASYNC"},\ {value: "PRIMARY_SYNC", label: "PRIMARY_SYNC"}\ ]', - 'Write synchronization mode:\ + 'Write synchronization mode\
            \
          • FULL_SYNC - Ignite will wait for write or commit replies from all nodes
          • \
          • FULL_ASYNC - Ignite will not wait for write or commit responses from participating nodes
          • \ diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/general.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade similarity index 80% rename from modules/web-console/src/main/js/app/modules/states/configuration/caches/general.jade rename to modules/web-console/frontend/app/modules/states/configuration/caches/general.jade index 44304bfa3c00b..e3147b19e60a4 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/general.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade @@ -16,9 +16,10 @@ include ../../../../../app/helpers/jade/mixins.jade +-var form = 'general' -var model = 'backupItem' -form.panel.panel-default(name='general' novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle) ignite-form-panel-chevron label General @@ -27,38 +28,38 @@ form.panel.panel-default(name='general' novalidate) .panel-body .col-sm-6 .settings-row - +text('Name:', model + '.name', 'cacheName', 'true', 'Input name', 'Cache name') + +text('Name:', model + '.name', '"cacheName"', 'true', 'Input name', 'Cache name') .settings-row +clusters(model, 'Associate clusters with the current cache') .settings-row +dropdown-multiple('Domain models: (add)', - model + '.domains', 'domains', 'true', 'Choose domain models', 'No domain models configured', 'domains', + model + '.domains', '"domains"', true, 'Choose domain models', 'No valid domain models configured', 'domains', 'Select domain models to describe types in cache') .settings-row - +cacheMode('Mode:', model + '.cacheMode', 'cacheMode', 'PARTITIONED') + +cacheMode('Mode:', model + '.cacheMode', '"cacheMode"', 'PARTITIONED') .settings-row - +dropdown('Atomicity:', model + '.atomicityMode', 'atomicityMode', 'true', 'ATOMIC', + +dropdown('Atomicity:', model + '.atomicityMode', '"atomicityMode"', 'true', 'ATOMIC', '[\ {value: "ATOMIC", label: "ATOMIC"},\ {value: "TRANSACTIONAL", label: "TRANSACTIONAL"}\ ]', 'Atomicity:\
              \ -
            • Atomic - in this mode distributed transactions and distributed locking are not supported
            • \ -
            • Transactional - in this mode specified fully ACID-compliant transactional cache behavior
            • \ +
            • ATOMIC - in this mode distributed transactions and distributed locking are not supported
            • \ +
            • TRANSACTIONAL - in this mode specified fully ACID-compliant transactional cache behavior
            • \
            ') .settings-row(data-ng-show='#{model}.cacheMode === "PARTITIONED"') - +number('Backups:', model + '.backups', 'backups', 'true', '0', '0', 'Number of nodes used to back up single partition for partitioned cache') + +number('Backups:', model + '.backups', '"backups"', 'true', '0', '0', 'Number of nodes used to back up single partition for partitioned cache') .settings-row(data-ng-show='#{model}.cacheMode === "PARTITIONED" && #{model}.backups') - +checkbox('Read from backup', model + '.readFromBackup', 'readFromBackup', + +checkbox('Read from backup', model + '.readFromBackup', '"readFromBackup"', 'Flag indicating whether data can be read from backup
            \ If not set then always get data from primary node (never from backup)') .settings-row - +checkbox('Copy on read', model + '.copyOnRead', 'copyOnRead', + +checkbox('Copy on read', model + '.copyOnRead', '"copyOnRead"', 'Flag indicating whether copy of the value stored in cache should be created for cache operation implying return value
            \ Also if this flag is set copies are created for values passed to CacheInterceptor and to CacheEntryProcessor') .settings-row(ng-show='#{model}.cacheMode === "PARTITIONED" && #{model}.atomicityMode === "TRANSACTIONAL"') - +checkbox('Invalidate near cache', model + '.invalidate', 'invalidate', + +checkbox('Invalidate near cache', model + '.invalidate', '"invalidate"', 'Invalidation flag for near cache entries in transaction
            \ If set then values will be invalidated (nullified) upon commit in near cache') .col-sm-6 diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/memory.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade similarity index 69% rename from modules/web-console/src/main/js/app/modules/states/configuration/caches/memory.jade rename to modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade index c862071858f04..debbe0d794b22 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/memory.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'memory' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Memory @@ -30,13 +30,13 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +dropdown('Mode:', model + '.memoryMode', 'memoryMode', 'true', 'ONHEAP_TIERED', + +dropdown('Mode:', model + '.memoryMode', '"memoryMode"', 'true', 'ONHEAP_TIERED', '[\ {value: "ONHEAP_TIERED", label: "ONHEAP_TIERED"},\ {value: "OFFHEAP_TIERED", label: "OFFHEAP_TIERED"},\ {value: "OFFHEAP_VALUES", label: "OFFHEAP_VALUES"}\ ]', - 'Memory modes:\ + 'Memory modes control whether value is stored in on-heap memory, off-heap memory, or swap space\
              \
            • \ ONHEAP_TIERED - entries are cached on heap memory first
              \ @@ -61,28 +61,42 @@ form.panel.panel-default(name=form novalidate)
            • \
            ') .settings-row(data-ng-show=model + '.memoryMode !== "OFFHEAP_VALUES"') - +number-required('Off-heap max memory:', model + '.offHeapMaxMemory', 'offHeapMaxMemory', 'true', - model + '.memoryMode === "OFFHEAP_TIERED"', '-1', '-1', - 'Sets maximum amount of memory available to off-heap storage
            \ - Possible values are:\ + +dropdown-required('Off-heap memory:', model + '.offHeapMode', '"offHeapMode"', 'true', + model + '.memoryMode === "OFFHEAP_TIERED"', + 'Disabled', + '[\ + {value: -1, label: "Disabled"},\ + {value: 1, label: "Limited"},\ + {value: 0, label: "Unlimited"}\ + ]', + 'Off-heap storage mode\
              \ -
            • -1 - means that off-heap storage is disabled
            • \ -
            • 0 - Ignite will not limit off-heap storage (it is up to user to properly add and remove entries from cache to ensure that off-heap storage does not grow infinitely)
            • \ -
            • Any positive value specifies the limit of off-heap storage in bytes
            • \ +
            • Disabled - Off-heap storage is disabled
            • \ +
            • Limited - Off-heap storage has limited size
            • \ +
            • Unlimited - Off-heap storage grow infinitely (it is up to user to properly add and remove entries from cache to ensure that off-heap storage does not grow infinitely)
            • \
            ') + .settings-row(data-ng-if=model + '.offHeapMode === 1 && ' + model + '.memoryMode !== "OFFHEAP_VALUES"') + +number-required('Off-heap memory max size:', model + '.offHeapMaxMemory', '"offHeapMaxMemory"', 'true', + model + '.offHeapMode === 1', '', 1, + 'Sets maximum amount of memory available to off-heap storage in bytes') .settings-row -var onHeapTired = model + '.memoryMode === "ONHEAP_TIERED"' -var swapEnabled = model + '.swapEnabled' -var offHeapMaxMemory = model + '.offHeapMaxMemory' - +evictionPolicy(model + '.evictionPolicy', 'evictionPolicy', 'true', + +evictionPolicy(model + '.evictionPolicy', '"evictionPolicy"', 'true', onHeapTired + ' && (' + swapEnabled + '|| _.isNumber(' + offHeapMaxMemory + ') &&' + offHeapMaxMemory + ' >= 0)', 'Optional cache eviction policy
            \ - Must be set for entries to be evicted from on - heap to off - heap or swap') + Must be set for entries to be evicted from on-heap to off-heap or swap\ +
              \ +
            • Least Recently Used(LRU) - Eviction policy based on LRU algorithm and supports batch eviction
            • \ +
            • First In First Out (FIFO) - Eviction policy based on FIFO algorithm and supports batch eviction
            • \ +
            • SORTED - Eviction policy which will select the minimum cache entry for eviction
            • \ +
            ') .settings-row - +number('Start size:', model + '.startSize', 'startSize', 'true', '1500000', '0', + +number('Start size:', model + '.startSize', '"startSize"', 'true', '1500000', '0', 'Initial cache size which will be used to pre-create internal hash table after start') .settings-row - +checkbox('Swap enabled', model + '.swapEnabled', 'swapEnabled', 'Flag indicating whether swap storage is enabled or not for this cache') + +checkbox('Swap enabled', model + '.swapEnabled', '"swapEnabled"', 'Flag indicating whether swap storage is enabled or not for this cache') .col-sm-6 +preview-xml-java(model, 'cacheMemory') diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade new file mode 100644 index 0000000000000..ee28c876e0ee7 --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade @@ -0,0 +1,108 @@ +//- + 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. + +include ../../../../../app/helpers/jade/mixins.jade + +-var form = 'nodeFilter' +-var model = 'backupItem' + +.panel.panel-default(ng-form=form novalidate) + .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') + ignite-form-panel-chevron + label(id='nodeFilter-title') Node filter + ignite-form-field-tooltip.tipLabel + | Determines on what nodes the cache should be started + ignite-form-revert + .panel-collapse(role='tabpanel' bs-collapse-target id=form) + .panel-body(ng-if='ui.isPanelLoaded("#{form}")') + .col-sm-6 + .settings-row + -var nodeFilter = model + '.nodeFilter'; + -var nodeFilterKind = nodeFilter + '.kind'; + + +dropdown('Node filter:', nodeFilterKind, '"nodeFilter"', 'true', 'Not set', + '[\ + {value: "IGFS", label: "IGFS nodes"},\ + {value: "OnNodes", label: "Specified nodes"},\ + {value: "Custom", label: "Custom"},\ + {value: undefined, label: "Not set"}\ + ]', + 'Node filter variant' + ) + .settings-row(ng-show=nodeFilterKind) + div(ng-show='#{nodeFilterKind} === "IGFS"') + -var igfsNodeFilter = nodeFilter + '.IGFS' + -var required = nodeFilterKind + ' === "IGFS"' + + //(lbl, model, name, enabled, required, placeholder, options, tip) + +dropdown-required-empty('IGFS:', igfsNodeFilter + '.igfs', '"igfsNodeFilter"', 'true', required, + 'Choose IGFS', 'No IGFS configured', 'igfss', 'Select IGFS to filter nodes') + div(ng-show='#{nodeFilterKind} === "Custom"') + -var customNodeFilter = nodeFilter + '.Custom' + -var required = nodeFilterKind + ' === "Custom"' + + +java-class('Class name:', customNodeFilter + '.className', '"customNodeFilter"', + 'true', required, 'Class name of custom node filter implementation') + div(ng-show='#{nodeFilterKind} === "OnNodes"') + -var nodeSetFilter = nodeFilter + '.OnNodes.nodeIds' + + +ignite-form-group(ng-form=form ng-model=nodeSetFilter) + -var uniqueTip = 'Such node ID already exists!' + + ignite-form-field-label + | Node IDs + ignite-form-group-tooltip + | Set of node IDs to deploy cache + ignite-form-group-add(ng-click='group.add = [{}]') + | Add new node ID + + .group-content(ng-if='#{nodeSetFilter}.length') + -var model = 'obj.model'; + -var name = '"edit" + $index' + -var valid = form + '[' + name + '].$valid' + -var save = nodeSetFilter + '[$index] = ' + model + + div(ng-repeat='model in #{nodeSetFilter} track by $index' ng-init='obj = {}') + label.col-xs-12.col-sm-12.col-md-12 + .indexField + | {{ $index+1 }}) + +table-remove-button(nodeSetFilter, 'Remove node ID') + + span(ng-hide='field.edit') + a.labelFormField(ng-click='field.edit = true; #{model} = model;') {{ model }} + span(ng-if='field.edit' ng-init='#{field} = model') + +table-uuid-field(name, model, nodeSetFilter, valid, save, false, true) + +table-save-button(valid, save, false) + +unique-feedback(name, uniqueTip) + +uuid-feedback(name) + + .group-content(ng-repeat='field in group.add') + -var model = 'new'; + -var name = '"new"' + -var valid = form + '[' + name + '].$valid' + -var save = nodeSetFilter + '.push(' + model + ')' + + div + label.col-xs-12.col-sm-12.col-md-12 + +table-uuid-field(name, model, nodeSetFilter, valid, save, true, true) + +table-save-button(valid, save, true) + +unique-feedback(name, uniqueTip) + +uuid-feedback(name) + .group-content-empty(id='nodeSetFilter' ng-if='!(#{nodeSetFilter}.length) && !group.add.length') + | Not defined + + .col-sm-6 + +preview-xml-java(model, 'cacheNodeFilter', 'igfss') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/query.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade similarity index 60% rename from modules/web-console/src/main/js/app/modules/states/configuration/caches/query.jade rename to modules/web-console/frontend/app/modules/states/configuration/caches/query.jade index 2a6495bcff07b..c83114b6dbf38 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/query.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'query' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Queries & Indexing @@ -30,18 +30,18 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +text('SQL schema name:', model + '.sqlSchema', 'sqlSchema', 'false', 'Input schema name', 'Schema name for cache according to SQL ANSI-99') + +text('SQL schema name:', model + '.sqlSchema', '"sqlSchema"', 'false', 'Input schema name', 'Schema name for cache according to SQL ANSI-99') .settings-row - +number('On-heap cache for off-heap indexes:', model + '.sqlOnheapRowCacheSize', 'sqlOnheapRowCacheSize', 'true', '10240', '1', + +number('On-heap cache for off-heap indexes:', model + '.sqlOnheapRowCacheSize', '"sqlOnheapRowCacheSize"', 'true', '10240', '1', 'Number of SQL rows which will be cached onheap to avoid deserialization on each SQL index access') .settings-row - +number('Long query timeout:', model + '.longQueryWarningTimeout', 'longQueryWarningTimeout', 'true', '3000', '0', + +number('Long query timeout:', model + '.longQueryWarningTimeout', '"longQueryWarningTimeout"', 'true', '3000', '0', 'Timeout in milliseconds after which long query warning will be printed') .settings-row - -var sqlFunctionClassesForm = 'querySqlFunctionClasses'; + -var form = 'querySqlFunctionClasses'; -var sqlFunctionClasses = model + '.sqlFunctionClasses'; - ignite-form-group(ng-model=sqlFunctionClasses ng-form=sqlFunctionClassesForm) + +ignite-form-group(ng-form=form ng-model=sqlFunctionClasses) ignite-form-field-label | SQL functions ignite-form-group-tooltip @@ -51,43 +51,45 @@ form.panel.panel-default(name=form novalidate) -var uniqueTip = 'SQL function with such class name already exists!' - .group-content(ng-if=sqlFunctionClasses + '.length') - -var field = 'edit' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.igniteUnique' - -var save = sqlFunctionClasses + '[$index] = ' + field + .group-content(ng-if='#{sqlFunctionClasses}.length') + -var model = 'obj.model'; + -var name = '"edit" + $index' + -var valid = form + '[' + name + '].$valid' + -var save = sqlFunctionClasses + '[$index] = ' + model - ignite-form-field(ng-repeat='model in #{sqlFunctionClasses} track by $index' type='internal' name='SQL function') - .indexField - | {{ $index+1 }}) - +table-remove-button(sqlFunctionClasses, 'Remove user-defined function') + div(ng-repeat='model in #{sqlFunctionClasses} track by $index' ng-init='obj = {}') + label.col-xs-12.col-sm-12.col-md-12 + .indexField + | {{ $index+1 }}) + +table-remove-button(sqlFunctionClasses, 'Remove user-defined function') - span(ng-hide='field.edit') - a.labelFormField(ng-click='field.edit = true') {{ model }} - span(ng-if='field.edit' ng-init='#{field} = model') - +table-java-class-field('SQL function', field, sqlFunctionClasses, valid, save, false) - +table-save-button(valid, save, false) - +unique-feedback(unique, uniqueTip) + span(ng-hide='field.edit') + a.labelFormField(ng-click='field.edit = true; #{model} = model;') {{ model }} + span(ng-if='field.edit') + +table-java-class-field('SQL function', name, model, sqlFunctionClasses, valid, save, false) + +table-save-button(valid, save, false) + +unique-feedback(name, uniqueTip) .group-content(ng-repeat='field in group.add') - -var field = 'new' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.igniteUnique' - -var save = sqlFunctionClasses + '.push(' + field + ')' + -var model = 'new'; + -var name = '"new"' + -var valid = form + '[' + name + '].$valid' + -var save = sqlFunctionClasses + '.push(' + model + ')' - ignite-form-field(type='internal' name='SQL function') - +table-java-class-field('SQL function', field, sqlFunctionClasses, valid, save, true) - +table-save-button(valid, save, true) - +unique-feedback(unique, uniqueTip) + div + label.col-xs-12.col-sm-12.col-md-12 + +table-java-class-field('SQL function', name, model, sqlFunctionClasses, valid, save, true) + +table-save-button(valid, save, true) + +unique-feedback(name, uniqueTip) .group-content-empty(ng-if='!(#{sqlFunctionClasses}.length) && !group.add.length') | Not defined .settings-row - +checkbox('Snapshotable index', model + '.snapshotableIndex', 'snapshotableIndex', + +checkbox('Snapshotable index', model + '.snapshotableIndex', '"snapshotableIndex"', 'Flag indicating whether SQL indexes should support snapshots') .settings-row - +checkbox('Escape table and filed names', model + '.sqlEscapeAll', 'sqlEscapeAll', + +checkbox('Escape table and filed names', model + '.sqlEscapeAll', '"sqlEscapeAll"', 'If set then all the SQL table and field names will be escaped with double quotes
            \ This enforces case sensitivity for field names and also allows having special characters in table and field names') .col-sm-6 - +preview-xml-java(model, 'cacheQuery') + +preview-xml-java(model, 'cacheQuery', 'domains') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/rebalance.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade similarity index 87% rename from modules/web-console/src/main/js/app/modules/states/configuration/caches/rebalance.jade rename to modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade index 88026a25f0733..6cf2d33f93f66 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/rebalance.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'rebalance' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate ng-hide='#{model}.cacheMode === "LOCAL"') +.panel.panel-default(ng-form=form novalidate ng-hide='#{model}.cacheMode === "LOCAL"') .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Rebalance @@ -30,36 +30,36 @@ form.panel.panel-default(name=form novalidate ng-hide='#{model}.cacheMode === "L .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +dropdown('Mode:', model + '.rebalanceMode', 'rebalanceMode', 'true', 'ASYNC', + +dropdown('Mode:', model + '.rebalanceMode', '"rebalanceMode"', 'true', 'ASYNC', '[\ {value: "SYNC", label: "SYNC"},\ {value: "ASYNC", label: "ASYNC"},\ {value: "NONE", label: "NONE"}\ ]', - 'Rebalance modes:\ + 'Rebalance modes\
              \
            • Synchronous - in this mode distributed caches will not start until all necessary data is loaded from other available grid nodes
            • \
            • Asynchronous - in this mode distributed caches will start immediately and will load all necessary data from other available grid nodes in the background
            • \
            • None - in this mode no rebalancing will take place which means that caches will be either loaded on demand from persistent store whenever data is accessed, or will be populated explicitly
            • \
            ') .settings-row - +number('Batch size:', model + '.rebalanceBatchSize', 'rebalanceBatchSize', 'true', '512 * 1024', '1', + +number('Batch size:', model + '.rebalanceBatchSize', '"rebalanceBatchSize"', 'true', '512 * 1024', '1', 'Size (in bytes) to be loaded within a single rebalance message
            \ Rebalancing algorithm will split total data set on every node into multiple batches prior to sending data') .settings-row - +number('Batches prefetch count:', model + '.rebalanceBatchesPrefetchCount', 'rebalanceBatchesPrefetchCount', 'true', '2', '1', + +number('Batches prefetch count:', model + '.rebalanceBatchesPrefetchCount', '"rebalanceBatchesPrefetchCount"', 'true', '2', '1', 'Number of batches generated by supply node at rebalancing start') .settings-row - +number('Order:', model + '.rebalanceOrder', 'rebalanceOrder', 'true', '0', Number.MIN_SAFE_INTEGER, + +number('Order:', model + '.rebalanceOrder', '"rebalanceOrder"', 'true', '0', Number.MIN_SAFE_INTEGER, 'If cache rebalance order is positive, rebalancing for this cache will be started only when rebalancing for all caches with smaller rebalance order (except caches with rebalance order 0) will be completed') .settings-row - +number('Delay:', model + '.rebalanceDelay', 'rebalanceDelay', 'true', '0', '0', + +number('Delay:', model + '.rebalanceDelay', '"rebalanceDelay"', 'true', '0', '0', 'Delay in milliseconds upon a node joining or leaving topology (or crash) after which rebalancing should be started automatically') .settings-row - +number('Timeout:', model + '.rebalanceTimeout', 'rebalanceTimeout', 'true', '10000', '0', + +number('Timeout:', model + '.rebalanceTimeout', '"rebalanceTimeout"', 'true', '10000', '0', 'Rebalance timeout in milliseconds') .settings-row - +number('Throttle:', model + '.rebalanceThrottle', 'rebalanceThrottle', 'true', '0', '0', + +number('Throttle:', model + '.rebalanceThrottle', '"rebalanceThrottle"', 'true', '0', '0', 'Time in milliseconds to wait between rebalance messages to avoid overloading of CPU or network') .col-sm-6 +preview-xml-java(model, 'cacheRebalance') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/server-near-cache.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/server-near-cache.jade similarity index 70% rename from modules/web-console/src/main/js/app/modules/states/configuration/caches/server-near-cache.jade rename to modules/web-console/frontend/app/modules/states/configuration/caches/server-near-cache.jade index 2b1ca07ca7ef3..74f500bac4960 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/server-near-cache.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/server-near-cache.jade @@ -16,10 +16,10 @@ include ../../../../../app/helpers/jade/mixins.jade --var form = 'server-near-cache' +-var form = 'serverNearCache' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate ng-show='#{model}.cacheMode === "PARTITIONED"') +.panel.panel-default(ng-form=form novalidate ng-show='#{model}.cacheMode === "PARTITIONED"') .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Server near cache @@ -35,11 +35,17 @@ form.panel.panel-default(name=form novalidate ng-show='#{model}.cacheMode === "P -var nearCfg = model + '.nearConfiguration' .settings-row - +checkbox('Enabled', enabled, 'nearCacheEnabled', 'Flag indicating whether to configure near cache') + +checkbox('Enabled', enabled, '"nearCacheEnabled"', 'Flag indicating whether to configure near cache') .settings-row - +number('Start size:', nearCfg + '.nearStartSize', 'nearStartSize', enabled, '375000', '0', + +number('Start size:', nearCfg + '.nearStartSize', '"nearStartSize"', enabled, '375000', '0', 'Initial cache size for near cache which will be used to pre-create internal hash table after start') .settings-row - +evictionPolicy(model + '.nearConfiguration.nearEvictionPolicy', 'nearCacheEvictionPolicy', enabled, 'false', 'Near cache eviction policy') + +evictionPolicy(model + '.nearConfiguration.nearEvictionPolicy', '"nearCacheEvictionPolicy"', enabled, 'false', + 'Near cache eviction policy\ +
              \ +
            • Least Recently Used (LRU) - Eviction policy based on LRU algorithm and supports batch eviction
            • \ +
            • First In First Out (FIFO) - Eviction policy based on FIFO algorithm and supports batch eviction
            • \ +
            • SORTED - Eviction policy which will select the minimum cache entry for eviction
            • \ +
            ') .col-sm-6 +preview-xml-java(model, 'cacheServerNearCache') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/statistics.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/statistics.jade similarity index 80% rename from modules/web-console/src/main/js/app/modules/states/configuration/caches/statistics.jade rename to modules/web-console/frontend/app/modules/states/configuration/caches/statistics.jade index cc2310af2f66d..027a2bd82aed1 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/statistics.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/statistics.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'statistics' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Statistics @@ -30,8 +30,10 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +checkbox('Statistics enabled', model + '.statisticsEnabled', 'statisticsEnabled', 'Flag indicating whether statistics gathering is enabled on a cache') + +checkbox('Statistics enabled', model + '.statisticsEnabled', '"statisticsEnabled"', 'Flag indicating whether statistics gathering is enabled on this cache') .settings-row - +checkbox('Management enabled', model + '.managementEnabled', 'managementEnabled', 'Flag indicating whether management is enabled on this cache') + +checkbox('Management enabled', model + '.managementEnabled', '"managementEnabled"', + 'Flag indicating whether management is enabled on this cache
            \ + If enabled the CacheMXBean for each cache is registered in the platform MBean server') .col-sm-6 +preview-xml-java(model, 'cacheStatistics') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/store.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade similarity index 66% rename from modules/web-console/src/main/js/app/modules/states/configuration/caches/store.jade rename to modules/web-console/frontend/app/modules/states/configuration/caches/store.jade index 6361e283b75fa..84752d6330214 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/store.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade @@ -19,38 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'store' -var model = 'backupItem' -//- Mixin for DB dialect. -mixin dialect(lbl, model, name, required, tipTitle, genericDialectName, placeholder) - ignite-form-field - ignite-form-field-label - | #{lbl} - ignite-form-field-tooltip - | #{tipTitle} - ul: li #{genericDialectName} - li Oracle database - li IBM DB2 - li Microsoft SQL Server - li MySQL - li PostgreSQL - li H2 database - ignite-form-field-dropdown( - data-id=name - data-name=name - data-options='[\ - {value: "Generic", label: "#{genericDialectName}"},\ - {value: "Oracle", label: "Oracle"},\ - {value: "DB2", label: "IBM DB2"},\ - {value: "SQLServer", label: "Microsoft SQL Server"},\ - {value: "MySQL", label: "MySQL"},\ - {value: "PostgreSQL", label: "PostgreSQL"},\ - {value: "H2", label: "H2 database"}\ - ]' - data-ng-model=model - data-ng-required=required - data-placeholder=placeholder - ) - -mixin hibernateField(items, field, valid, save, newItem) +mixin hibernateField(name, model, items, valid, save, newItem) -var reset = newItem ? 'group.add = []' : 'field.edit = false' -var resetOnEnter = newItem ? '(stopblur = true) && (group.add = [{}])' : '(field.edit = false)' @@ -59,21 +28,21 @@ mixin hibernateField(items, field, valid, save, newItem) -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false' -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';' - ignite-form-field-input-text( - data-name='#{field}{{ $index || "" }}' - data-ng-model=field - data-ng-required='true' - data-placeholder='key=value' - data-ignite-property-unique=items - data-ignite-property-value-specified - data-ignite-form-field-input-autofocus='true' - ignite-on-enter=onEnter - ignite-on-escape=reset - ng-blur=onBlur - ) + if block block -form.panel.panel-default(name=form novalidate) + .input-tip + +ignite-form-field-input(name, model, false, 'true', 'key=value')( + data-ignite-property-unique=items + data-ignite-property-value-specified + data-ignite-form-field-input-autofocus='true' + + ng-blur=onBlur + ignite-on-enter=onEnter + ignite-on-escape=onEscape + ) + +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Store @@ -87,14 +56,19 @@ form.panel.panel-default(name=form novalidate) -var storeFactory = model + '.cacheStoreFactory'; -var storeFactoryKind = storeFactory + '.kind'; - +dropdown('Store factory:', storeFactoryKind, 'cacheStoreFactory', 'true', 'Not set', + +dropdown('Store factory:', storeFactoryKind, '"cacheStoreFactory"', 'true', 'Not set', '[\ {value: "CacheJdbcPojoStoreFactory", label: "JDBC POJO store factory"},\ {value: "CacheJdbcBlobStoreFactory", label: "JDBC BLOB store factory"},\ {value: "CacheHibernateBlobStoreFactory", label: "Hibernate BLOB store factory"},\ {value: undefined, label: "Not set"}\ ]', - 'Factory for persistent storage for cache data' + 'Factory for persistent storage for cache data\ +
              \ +
            • JDBC POJO store factory - Objects are stored in underlying database by using java beans mapping description via reflection backed by JDBC
            • \ +
            • JDBC BLOB store factory - Objects are stored in underlying database in BLOB format backed by JDBC
            • \ +
            • Hibernate BLOB store factory - Objects are stored in underlying database in BLOB format backed by Hibernate
            • \ +
            ' ) span(ng-show=storeFactoryKind ng-init='__.expanded = true') a.customize(ng-show='__.expanded' ng-click='__.expanded = false') Hide settings @@ -106,10 +80,10 @@ form.panel.panel-default(name=form novalidate) .details-row +text('Data source bean name:', pojoStoreFactory + '.dataSourceBean', - 'pojoDataSourceBean', required, 'Input bean name', + '"pojoDataSourceBean"', required, 'Input bean name', 'Name of the data source bean in Spring context') .details-row - +dialect('Dialect:', pojoStoreFactory + '.dialect', 'pojoDialect', required, + +dialect('Dialect:', pojoStoreFactory + '.dialect', '"pojoDialect"', required, 'Dialect of SQL implemented by a particular RDBMS:', 'Generic JDBC dialect', 'Choose JDBC dialect') div(ng-show='#{storeFactoryKind} === "CacheJdbcBlobStoreFactory"') @@ -117,7 +91,7 @@ form.panel.panel-default(name=form novalidate) -var blobStoreFactoryVia = blobStoreFactory + '.connectVia' .details-row - +dropdown('Connect via:', blobStoreFactoryVia, 'connectVia', 'true', 'Choose connection method', + +dropdown('Connect via:', blobStoreFactoryVia, '"connectVia"', 'true', 'Choose connection method', '[\ {value: "URL", label: "URL"},\ {value: "DataSource", label: "Data source"}\ @@ -131,41 +105,41 @@ form.panel.panel-default(name=form novalidate) -var required = storeFactoryKind + ' === "CacheJdbcBlobStoreFactory" && ' + blobStoreFactoryVia + ' === "URL"' .details-row - +text('Connection URL:', blobStoreFactory + '.connectionUrl', 'connectionUrl', required, 'Input URL', + +text('Connection URL:', blobStoreFactory + '.connectionUrl', '"connectionUrl"', required, 'Input URL', 'URL for database access, for example: jdbc:h2:mem:myDatabase') .details-row - +text('User:', blobStoreFactory + '.user', 'user', required, 'Input user name', 'User name for database access') + +text('User:', blobStoreFactory + '.user', '"user"', required, 'Input user name', 'User name for database access') .details-row label Note, password will be generated as stub div(ng-show='#{blobStoreFactoryVia} !== "URL"') -var required = storeFactoryKind + ' === "CacheJdbcBlobStoreFactory" && ' + blobStoreFactoryVia + '!== "URL"' .details-row - +text('Data source bean name:', blobStoreFactory + '.dataSourceBean', 'blobDataSourceBean', required, 'Input bean name', + +text('Data source bean name:', blobStoreFactory + '.dataSourceBean', '"blobDataSourceBean"', required, 'Input bean name', 'Name of the data source bean in Spring context') .details-row - +dialect('Database:', blobStoreFactory + '.dialect', 'blobDialect', required, 'Supported databases:', 'Generic database', 'Choose database') + +dialect('Database:', blobStoreFactory + '.dialect', '"blobDialect"', required, 'Supported databases:', 'Generic database', 'Choose database') .details-row - +checkbox('Init schema', blobStoreFactory + '.initSchema', 'initSchema', + +checkbox('Init schema', blobStoreFactory + '.initSchema', '"initSchema"', 'Flag indicating whether DB schema should be initialized by Ignite (default behaviour) or was explicitly created by user') .details-row - +text('Create query:', blobStoreFactory + '.createTableQuery', 'createTableQuery', 'false', 'SQL for table creation', + +text('Create query:', blobStoreFactory + '.createTableQuery', '"createTableQuery"', 'false', 'SQL for table creation', 'Query for table creation in underlying database
            \ Default value: create table if not exists ENTRIES (key binary primary key, val binary)') .details-row - +text('Load query:', blobStoreFactory + '.loadQuery', 'loadQuery', 'false', 'SQL for load entry', + +text('Load query:', blobStoreFactory + '.loadQuery', '"loadQuery"', 'false', 'SQL for load entry', 'Query for entry load from underlying database
            \ Default value: select * from ENTRIES where key=?') .details-row - +text('Insert query:', blobStoreFactory + '.insertQuery', 'insertQuery', 'false', 'SQL for insert entry', + +text('Insert query:', blobStoreFactory + '.insertQuery', '"insertQuery"', 'false', 'SQL for insert entry', 'Query for insert entry into underlying database
            \ Default value: insert into ENTRIES (key, val) values (?, ?)') .details-row - +text('Update query:', blobStoreFactory + '.updateQuery', 'updateQuery', 'false', 'SQL for update entry', + +text('Update query:', blobStoreFactory + '.updateQuery', '"updateQuery"', 'false', 'SQL for update entry', 'Query for update entry in underlying database
            \ Default value: update ENTRIES set val=? where key=?') .details-row - +text('Delete query:', blobStoreFactory + '.deleteQuery', 'deleteQuery', 'false', 'SQL for delete entry', + +text('Delete query:', blobStoreFactory + '.deleteQuery', '"deleteQuery"', 'false', 'SQL for delete entry', 'Query for delete entry from underlying database
            \ Default value: delete from ENTRIES where key=?') @@ -174,7 +148,7 @@ form.panel.panel-default(name=form novalidate) -var hibernateProperties = hibernateStoreFactory + '.hibernateProperties' .details-row - ignite-form-group(ng-model=hibernateProperties ng-form=form) + +ignite-form-group(ng-form=form ng-model=hibernateProperties) ignite-form-field-label | Hibernate properties ignite-form-group-tooltip @@ -187,50 +161,49 @@ form.panel.panel-default(name=form novalidate) -var tipPropertySpecified = 'Property should be present in format key=value!' .group-content(ng-if='#{hibernateProperties}.length') - -var field = 'edit' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.ignitePropertyUnique' - -var prop = 'form[ngModelName].$error.ignitePropertyValueSpecified' - -var save = hibernateProperties + '[$index] = ' + field - - ignite-form-field(ng-repeat='model in #{hibernateProperties} track by $index' type='internal' name='Hibernate properties') - .indexField - | {{ $index+1 }}) - +table-remove-button(hibernateProperties, 'Remove Hibernate property') - - span(ng-hide='field.edit') - a.labelFormField(ng-click='field.edit = true') {{ model }} - span(ng-if='field.edit' ng-init='#{field} = model') - +hibernateField(hibernateProperties, field, valid, save, false) - +table-save-button(valid, save, false) - +error-feedback(unique, 'ignitePropertyUnique', tipUnique) - +error-feedback(prop, 'ignitePropertyValueSpecified', tipPropertySpecified) + -var model = 'obj.model'; + -var name = '"edit" + $index' + -var valid = form + '[' + name + '].$valid' + -var save = hibernateProperties + '[$index] = ' + model + + div(ng-repeat='model in #{hibernateProperties} track by $index' ng-init='obj = {}') + label.col-xs-12.col-sm-12.col-md-12 + .indexField + | {{ $index+1 }}) + +table-remove-button(hibernateProperties, 'Remove Hibernate property') + + span(ng-hide='field.edit') + a.labelFormField(ng-click='field.edit = true; #{model} = model;') {{ model }} + span(ng-if='field.edit') + +hibernateField(name, model, hibernateProperties, valid, save, false) + +table-save-button(valid, save, false) + +form-field-feedback(name, 'ignitePropertyUnique', tipUnique) + +form-field-feedback(name, 'ignitePropertyValueSpecified', tipPropertySpecified) .group-content(ng-repeat='field in group.add') - -var field = 'new' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.ignitePropertyUnique' - -var prop = 'form[ngModelName].$error.ignitePropertyValueSpecified' - -var save = hibernateProperties + '.push(' + field + ')' - - ignite-form-field(type='internal' name='Hibernate property') - +hibernateField(hibernateProperties, field, valid, save, true) - +table-save-button(valid, save, true) - +error-feedback(unique, 'ignitePropertyUnique', tipUnique) - +error-feedback(prop, 'ignitePropertyValueSpecified', tipPropertySpecified) + -var model = 'new'; + -var name = '"new"' + -var valid = form + '[' + name + '].$valid' + -var save = hibernateProperties + '.push(' + model + ')' + + div + label.col-xs-12.col-sm-12.col-md-12 + +hibernateField(name, model, hibernateProperties, valid, save, true) + +table-save-button(valid, save, true) + +form-field-feedback(name, 'ignitePropertyUnique', tipUnique) + +form-field-feedback(name, 'ignitePropertyValueSpecified', tipPropertySpecified) .group-content-empty(ng-if='!(#{hibernateProperties}.length) && !group.add.length') | Not defined .settings-row - +checkbox('Keep binary in store', model + '.storeKeepBinary', 'storeKeepBinary', + +checkbox('Keep binary in store', model + '.storeKeepBinary', '"storeKeepBinary"', 'Flag indicating that CacheStore implementation is working with binary objects instead of Java objects') .settings-row - +checkbox('Load previous value', model + '.loadPreviousValue', 'loadPreviousValue', + +checkbox('Load previous value', model + '.loadPreviousValue', '"loadPreviousValue"', 'Flag indicating whether value should be loaded from store if it is not in the cache for following cache operations: \
              \
            • IgniteCache.putIfAbsent()
            • \
            • IgniteCache.replace()
            • \ -
            • IgniteCache.replace()
            • \
            • IgniteCache.remove()
            • \
            • IgniteCache.getAndPut()
            • \
            • IgniteCache.getAndRemove()
            • \ @@ -238,34 +211,34 @@ form.panel.panel-default(name=form novalidate)
            • IgniteCache.getAndPutIfAbsent()
            • \
            ') .settings-row - +checkbox('Read-through', model + '.readThrough', 'readThrough', 'Flag indicating whether read-through caching should be used') + +checkbox('Read-through', model + '.readThrough', '"readThrough"', 'Flag indicating whether read-through caching should be used') .settings-row - +checkbox('Write-through', model + '.writeThrough', 'writeThrough', 'Flag indicating whether write-through caching should be used') + +checkbox('Write-through', model + '.writeThrough', '"writeThrough"', 'Flag indicating whether write-through caching should be used') .settings-row - ignite-form-group + +ignite-form-group ignite-form-field-label - | Write behind + | Write-behind ignite-form-group-tooltip - | Cache write behind settings#[br] + | Cache write-behind settings#[br] | Write-behind is a special mode when updates to cache accumulated and then asynchronously flushed to persistent store as a bulk operation .group-content -var enabled = model + '.writeBehindEnabled' .details-row - +checkbox('Enabled', enabled, 'writeBehindEnabled', 'Flag indicating whether Ignite should use write-behind behaviour for the cache store') + +checkbox('Enabled', enabled, '"writeBehindEnabled"', 'Flag indicating whether Ignite should use write-behind behaviour for the cache store') .details-row - +number('Batch size:', model + '.writeBehindBatchSize', 'writeBehindBatchSize', enabled, '512', '1', - 'Maximum batch size for write - behind cache store operations
            \ + +number('Batch size:', model + '.writeBehindBatchSize', '"writeBehindBatchSize"', enabled, '512', '1', + 'Maximum batch size for write-behind cache store operations
            \ Store operations(get or remove) are combined in a batch of this size to be passed to cache store') .details-row - +number('Flush size:', model + '.writeBehindFlushSize', 'writeBehindFlushSize', enabled, '10240', '1', + +number('Flush size:', model + '.writeBehindFlushSize', '"writeBehindFlushSize"', enabled, '10240', '0', 'Maximum size of the write-behind cache
            \ If cache size exceeds this value, all cached items are flushed to the cache store and write cache is cleared') .details-row - +number('Flush frequency:', model + '.writeBehindFlushFrequency', 'writeBehindFlushFrequency', enabled, '5000', '1', + +number('Flush frequency:', model + '.writeBehindFlushFrequency', '"writeBehindFlushFrequency"', enabled, '5000', '0', 'Frequency with which write-behind cache is flushed to the cache store in milliseconds') .details-row - +number('Flush threads count:', model + '.writeBehindFlushThreadCount', 'writeBehindFlushThreadCount', enabled, '1', '1', + +number('Flush threads count:', model + '.writeBehindFlushThreadCount', '"writeBehindFlushThreadCount"', enabled, '1', '1', 'Number of threads that will perform cache flushing') .col-sm-6 +preview-xml-java(model, 'cacheStore', 'domains') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/atomic.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade similarity index 88% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/atomic.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade index 3927b720d5fb5..c4ef88ee2f569 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/atomic.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'atomics' -var model = 'backupItem.atomicConfiguration' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Atomic configuration @@ -31,7 +31,7 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +dropdown('Cache mode:', model + '.cacheMode', 'cacheMode', 'true', 'PARTITIONED', + +dropdown('Cache mode:', model + '.cacheMode', '"cacheMode"', 'true', 'PARTITIONED', '[\ {value: "LOCAL", label: "LOCAL"},\ {value: "REPLICATED", label: "REPLICATED"},\ @@ -44,10 +44,10 @@ form.panel.panel-default(name=form novalidate)
          • Local - in this mode caches residing on different grid nodes will not know about each other
          • \
          ') .settings-row - +number('Sequence reserve:', model + '.atomicSequenceReserveSize', 'atomicSequenceReserveSize', 'true', '1000', '0', + +number('Sequence reserve:', model + '.atomicSequenceReserveSize', '"atomicSequenceReserveSize"', 'true', '1000', '0', 'Default number of sequence values reserved for IgniteAtomicSequence instances
          \ After a certain number has been reserved, consequent increments of sequence will happen locally, without communication with other nodes, until the next reservation has to be made') - .settings-row(ng-if='!(#{model}.cacheMode && #{model}.cacheMode != "PARTITIONED")') - +number('Backups:', model + '.backups', 'backups', 'true', '0', '0', 'Number of backup nodes') + .settings-row(ng-show='!(#{model}.cacheMode && #{model}.cacheMode != "PARTITIONED")') + +number('Backups:', model + '.backups', '"backups"', 'true', '0', '0', 'Number of backup nodes') .col-sm-6 +preview-xml-java(model, 'clusterAtomics') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/attributes.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade similarity index 95% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/attributes.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade index e6ffd50560fab..29e7a79b41fb7 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/attributes.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade @@ -18,10 +18,9 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'attributes' -var model = 'backupItem' --var types = model + '.typeConfigurations' -var userAttributes = model + '.attributes' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label User attributes @@ -32,7 +31,7 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - ignite-form-group(ng-model='#{userAttributes}' ng-form='#{form}') + +ignite-form-group(ng-model='#{userAttributes}' ng-form='#{form}') ignite-form-field-label | User attributes ignite-form-group-tooltip diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/binary.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade similarity index 70% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/binary.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade index 77caa36704419..c63e2d9e8836c 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/binary.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade @@ -22,55 +22,32 @@ include ../../../../../app/helpers/jade/mixins.jade //- Mixin for java name field with enabled condition. mixin binary-types-java-class(lbl, model, name, enabled, required, remove, autofocus, tip) - -var errLbl = lbl.substring(0, lbl.length - 1) - - ignite-form-field - ignite-form-field-label - | #{lbl} - ignite-form-field-tooltip - | !{tip} + +java-class(lbl, model, name, enabled, required, tip) if (remove) +table-remove-button(types, 'Remove type configuration') - ignite-form-field-input-text( - data-id='{{#{name}}}' - data-name='{{#{name}}}' - data-ng-model=model - data-ng-disabled='!(#{enabled})' - data-ng-required=required - data-placeholder='Enter fully qualified class name' - - data-ignite-form-field-input-autofocus=autofocus - data-java-identifier='true' - data-java-package-specified='true' - data-java-keywords='true' - data-java-built-in-class='true' - ) - +error-feedback('form[ngModelName].$error.javaBuiltInClass', 'javaBuiltInClass', lbl + ' should not be the Java built-in class!', '{{' + name + '}}') - +error-feedback('form[ngModelName].$error.javaKeywords', 'javaKeywords', lbl + ' could not contains reserved Java keyword!', '{{' + name + '}}') - +error-feedback('form[ngModelName].$error.javaPackageSpecified', 'javaPackageSpecified', lbl + ' does not have package specified!', '{{' + name + '}}') - +error-feedback('form[ngModelName].$error.javaIdentifier', 'javaIdentifier', lbl + ' is invalid Java identifier!', '{{' + name + '}}') - -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Binary configuration ignite-form-field-tooltip.tipLabel - | Configuration for Ignite Binary Objects + | Configuration of specific binary types ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +java-class('ID mapper:', model + '.idMapper', 'idMapper', 'true', 'false', + +java-class('ID mapper:', model + '.idMapper', '"idMapper"', 'true', 'false', 'Maps given from BinaryNameMapper type and filed name to ID that will be used by Ignite in internals
          \ Ignite never writes full strings for field or type names. Instead, for performance reasons, Ignite writes integer hash codes for type/class and field names. It has been tested that hash code conflicts for the type/class names or the field names within the same type are virtually non - existent and, to gain performance, it is safe to work with hash codes. For the cases when hash codes for different types or fields actually do collide BinaryIdMapper allows to override the automatically generated hash code IDs for the type and field names') .settings-row - +java-class('Name mapper:', model + '.nameMapper', 'nameMapper', 'true', 'false', 'Maps type/class and field names to different names') + +java-class('Name mapper:', model + '.nameMapper', '"nameMapper"', 'true', 'false', 'Maps type/class and field names to different names') .settings-row - +java-class('Serializer:', model + '.serializer', 'serializer', 'true', 'false', 'Class with custom serialization logic for binary objects') + +java-class('Serializer:', model + '.serializer', '"serializer"', 'true', 'false', 'Class with custom serialization logic for binary objects') .settings-row - ignite-form-group(ng-form='#{form}TypeConfigurations' ng-model='#{types}') + -var form = 'binaryTypeConfigurations'; + + +ignite-form-group() ignite-form-field-label | Type configurations ignite-form-group-tooltip @@ -86,7 +63,7 @@ form.panel.panel-default(name=form novalidate) .settings-row +binary-types-java-class('ID mapper:', 'model.idMapper', '"idMapper" + $index', 'true', 'false', false, 'false', 'Maps given from BinaryNameMapper type and filed name to ID that will be used by Ignite in internals
          \ - Ignite never writes full strings for field or type/class names. Instead, for performance reasons, Ignite writes integer hash codes for type/class and field names. It has been tested that hash code conflicts for the type/class names or the field names within the same type are virtually non - existent and, to gain performance, it is safe to work with hash codes. For the cases when hash codes for different types or fields actually do collide #[b BinaryIdMapper] allows to override the automatically generated hash code IDs for the type and field names') + Ignite never writes full strings for field or type/class names. Instead, for performance reasons, Ignite writes integer hash codes for type/class and field names. It has been tested that hash code conflicts for the type/class names or the field names within the same type are virtually non - existent and, to gain performance, it is safe to work with hash codes. For the cases when hash codes for different types or fields actually do collide BinaryIdMapper allows to override the automatically generated hash code IDs for the type and field names') .settings-row +binary-types-java-class('Name mapper:', 'model.nameMapper', '"nameMapper" + $index', 'true', 'false', false, 'false', 'Maps type/class and field names to different names') .settings-row @@ -95,6 +72,6 @@ form.panel.panel-default(name=form novalidate) +checkbox('Enum', 'model.enum', 'enum', 'Flag indicating that this type is the enum') .settings-row - +checkbox('Compact footer', model + '.compactFooter', 'compactFooter', 'When enabled, Ignite will not write fields metadata when serializing objects(this will increase serialization performance), because internally #[b BinaryMarshaller] already distribute metadata inside cluster') + +checkbox('Compact footer', model + '.compactFooter', '"compactFooter"', 'When enabled, Ignite will not write fields metadata when serializing objects(this will increase serialization performance), because internally #[b BinaryMarshaller] already distribute metadata inside cluster') .col-sm-6 +preview-xml-java(model, 'clusterBinary') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade new file mode 100644 index 0000000000000..a078bf149bf1a --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade @@ -0,0 +1,53 @@ +//- + 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. + +include ../../../../../app/helpers/jade/mixins.jade + +-var form = 'cacheKeyCfg' +-var model = 'backupItem.cacheKeyConfiguration' +-var items = model; + +.panel.panel-default(ng-form=form novalidate) + .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') + ignite-form-panel-chevron + label Cache key configuration + ignite-form-field-tooltip.tipLabel + | Cache key configuration allows to collocate objects in a partitioned cache based on field in cache key without explicit usage of annotations on user classes. + ignite-form-revert + .panel-collapse(role='tabpanel' bs-collapse-target id=form) + .panel-body(ng-if='ui.isPanelLoaded("#{form}")') + .col-sm-6 + .settings-row + -var form = form + 'TypeConfigurations' + + +ignite-form-group(ng-form=form ng-model=model) + ignite-form-field-label + | Cache key configuration + ignite-form-group-tooltip + | Cache key configuration + ignite-form-group-add(ng-click='#{model}.push({})') + | Add new cache key configuration + .group-content-empty(ng-if='!#{model}.length') + | Not defined + .group-content(ng-repeat='model in #{model} track by $index') + hr(ng-if='$index !== 0') + .settings-row + +java-class('Type name:', 'model.typeName', '"cacheKeyTypeName" + $index', 'true', 'true', 'Type name') + +table-remove-button(items, 'Remove cache key configuration') + .settings-row + +text('Affinity key field name:', 'model.affinityKeyFieldName', '"affinityKeyFieldName" + $index', true, 'Enter field name', 'Affinity key field name') + .col-sm-6 + +preview-xml-java(model, 'clusterCacheKeyConfiguration') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade similarity index 75% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade index 10e51f04891e6..491e4f19a379e 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade @@ -18,8 +18,9 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'collision' -var model = 'backupItem.collision' +-var modelCollisionKind = model + '.kind'; -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Collision configuration @@ -30,7 +31,7 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +dropdown('CollisionSpi:', model + '.kind', 'collision', 'true', '', + +dropdown('CollisionSpi:', modelCollisionKind, '"collisionKind"', 'true', '', '[\ {value: "JobStealing", label: "Job stealing"},\ {value: "FifoQueue", label: "FIFO queue"},\ @@ -46,15 +47,16 @@ form.panel.panel-default(name=form novalidate)
        • Custom - custom CollisionSpi implementation
        • \
        • Default - jobs are activated immediately on arrival to mapped node
        • \ ') - .settings-row(ng-show='#{model}.kind !== "Noop"') + .settings-row(ng-show='#{modelCollisionKind} !== "Noop"') .panel-details - ignite-configuration-clusters-collision-job-stealing( - ng-show='#{model}.kind === "JobStealing"') - ignite-configuration-clusters-collision-fifo-queue( - ng-show='#{model}.kind === "FifoQueue"') - ignite-configuration-clusters-collision-priority-queue( - ng-show='#{model}.kind === "PriorityQueue"') - ignite-configuration-clusters-collision-custom( - ng-show='#{model}.kind === "Custom"') + div(ng-show='#{modelCollisionKind} === "JobStealing"') + include ./collision/job-stealing.jade + div(ng-show='#{modelCollisionKind} === "FifoQueue"') + include ./collision/fifo-queue.jade + div(ng-show='#{modelCollisionKind} === "PriorityQueue"') + include ./collision/priority-queue.jade + div(ng-show='#{modelCollisionKind} === "Custom"') + include ./collision/custom.jade .col-sm-6 + -var model = 'backupItem.collision' +preview-xml-java(model, 'clusterCollision') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade similarity index 88% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/custom.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade index b666f54de31ed..923891764c06f 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/custom.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade @@ -21,4 +21,4 @@ include ../../../../../../app/helpers/jade/mixins.jade div .details-row - +java-class('Class:', model + '.class', 'collisionCustom', 'true', required, 'CollisionSpi implementation class') + +java-class('Class:', model + '.class', '"collisionCustom"', 'true', required, 'CollisionSpi implementation class') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/fifo-queue.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/fifo-queue.jade similarity index 87% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/fifo-queue.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/collision/fifo-queue.jade index 9c9d315340af7..f16363d54fa87 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/fifo-queue.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/fifo-queue.jade @@ -17,12 +17,11 @@ include ../../../../../../app/helpers/jade/mixins.jade -var model = 'backupItem.collision.FifoQueue' --var form = 'collisionFifoQueue' div .details-row - +number('Parallel jobs number:', model + '.parallelJobsNumber', 'fifoParallelJobsNumber', 'true', 'availableProcessors * 2', '1', + +number('Parallel jobs number:', model + '.parallelJobsNumber', '"fifoParallelJobsNumber"', 'true', 'availableProcessors * 2', '1', 'Number of jobs that can be executed in parallel') .details-row - +number('Wait jobs number:', model + '.waitingJobsNumber', 'fifoWaitingJobsNumber', 'true', 'Integer.MAX_VALUE', '0', + +number('Wait jobs number:', model + '.waitingJobsNumber', '"fifoWaitingJobsNumber"', 'true', 'Integer.MAX_VALUE', '0', 'Maximum number of jobs that are allowed to wait in waiting queue') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/job-stealing.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade similarity index 89% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/job-stealing.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade index 17392c0e6e729..3e6d42823798b 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/job-stealing.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade @@ -17,30 +17,29 @@ include ../../../../../../app/helpers/jade/mixins.jade -var model = 'backupItem.collision.JobStealing' --var form = 'collisionJobStealing' -var stealingAttributes = model + '.stealingAttributes' div .details-row - +number('Active jobs threshold:', model + '.activeJobsThreshold', 'jsActiveJobsThreshold', 'true', '95', '0', + +number('Active jobs threshold:', model + '.activeJobsThreshold', '"jsActiveJobsThreshold"', 'true', '95', '0', 'Number of jobs that can be executed in parallel') .details-row - +number('Wait jobs threshold:', model + '.waitJobsThreshold', 'jsWaitJobsThreshold', 'true', '0', '0', + +number('Wait jobs threshold:', model + '.waitJobsThreshold', '"jsWaitJobsThreshold"', 'true', '0', '0', 'Job count threshold at which this node will start stealing jobs from other nodes') .details-row - +number('Message expire time:', model + '.messageExpireTime', 'jsMessageExpireTime', 'true', '1000', '1', + +number('Message expire time:', model + '.messageExpireTime', '"jsMessageExpireTime"', 'true', '1000', '1', 'Message expire time in ms') .details-row - +number('Maximum stealing attempts:', model + '.maximumStealingAttempts', 'jsMaximumStealingAttempts', 'true', '5', '1', + +number('Maximum stealing attempts:', model + '.maximumStealingAttempts', '"jsMaximumStealingAttempts"', 'true', '5', '1', 'Maximum number of attempts to steal job by another node') .details-row - +checkbox('Stealing enabled', model + '.stealingEnabled', 'jsStealingEnabled', + +checkbox('Stealing enabled', model + '.stealingEnabled', '"jsStealingEnabled"', 'Node should attempt to steal jobs from other nodes') .details-row - +java-class('External listener:', model + '.externalCollisionListener', 'jsExternalCollisionListener', 'true', 'false', + +java-class('External listener:', model + '.externalCollisionListener', '"jsExternalCollisionListener"', 'true', 'false', 'Listener to be set for notification of external collision events') .details-row - ignite-form-group(ng-model='#{stealingAttributes}' ng-form='#{form}') + +ignite-form-group(ng-model='#{stealingAttributes}' ng-form='#{form}') ignite-form-field-label | Stealing attributes ignite-form-group-tooltip @@ -52,7 +51,7 @@ div .group-content(ng-show='(#{stealingAttributes} && #{stealingAttributes}.length > 0) || tableNewItemActive(stealingAttributesTbl)') table.links-edit(id='attributes' st-table=stealingAttributes) tbody - tr(ng-repeat='item in #{stealingAttributes}') + tr(ng-repeat='item in #{stealingAttributes} track by $index') td.col-sm-12(ng-show='!tableEditing(stealingAttributesTbl, $index)') a.labelFormField(ng-click='tableStartEdit(backupItem, stealingAttributesTbl, $index)') {{item.name}} = {{item.value}} +btn-remove('tableRemove(backupItem, stealingAttributesTbl, $index)', '"Remove attribute"') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/priority-queue.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/priority-queue.jade similarity index 79% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/priority-queue.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/collision/priority-queue.jade index 208c12b50c949..bdd1daca60701 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/priority-queue.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/priority-queue.jade @@ -17,27 +17,26 @@ include ../../../../../../app/helpers/jade/mixins.jade -var model = 'backupItem.collision.PriorityQueue' --var form = 'collisionPriorityQueue' div .details-row - +number('Parallel jobs number:', model + '.parallelJobsNumber', 'priorityParallelJobsNumber', 'true', 'availableProcessors * 2', '1', + +number('Parallel jobs number:', model + '.parallelJobsNumber', '"priorityParallelJobsNumber"', 'true', 'availableProcessors * 2', '1', 'Number of jobs that can be executed in parallel') .details-row - +number('Waiting jobs number:', model + '.waitingJobsNumber', 'priorityWaitingJobsNumber', 'true', 'Integer.MAX_VALUE', '0', + +number('Waiting jobs number:', model + '.waitingJobsNumber', '"priorityWaitingJobsNumber"', 'true', 'Integer.MAX_VALUE', '0', 'Maximum number of jobs that are allowed to wait in waiting queue') .details-row - +text('Priority attribute key:', model + '.priorityAttributeKey', 'priorityPriorityAttributeKey', 'false', 'grid.task.priority', + +text('Priority attribute key:', model + '.priorityAttributeKey', '"priorityPriorityAttributeKey"', 'false', 'grid.task.priority', 'Task priority attribute key') .details-row - +text('Job priority attribute key:', model + '.jobPriorityAttributeKey', 'priorityJobPriorityAttributeKey', 'false', 'grid.job.priority', + +text('Job priority attribute key:', model + '.jobPriorityAttributeKey', '"priorityJobPriorityAttributeKey"', 'false', 'grid.job.priority', 'Job priority attribute key') .details-row - +number('Default priority:', model + '.defaultPriority', 'priorityDefaultPriority', 'true', '0', '0', + +number('Default priority:', model + '.defaultPriority', '"priorityDefaultPriority"', 'true', '0', '0', 'Default priority to use if a job does not have priority attribute set') .details-row - +number('Starvation increment:', model + '.starvationIncrement', 'priorityStarvationIncrement', 'true', '1', '0', + +number('Starvation increment:', model + '.starvationIncrement', '"priorityStarvationIncrement"', 'true', '1', '0', 'Value to increment job priority by every time a lower priority job gets behind a higher priority job') .details-row - +checkbox('Starvation prevention enabled', model + '.starvationPreventionEnabled', 'priorityStarvationPreventionEnabled', + +checkbox('Starvation prevention enabled', model + '.starvationPreventionEnabled', '"priorityStarvationPreventionEnabled"', 'Job starvation prevention is enabled') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/communication.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade similarity index 60% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/communication.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade index 7073f27341e67..0643555c3a1bd 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/communication.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade @@ -20,58 +20,61 @@ include ../../../../../app/helpers/jade/mixins.jade -var model = 'backupItem' -var communication = model + '.communication' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Communication ignite-form-field-tooltip.tipLabel - | Cluster communication network properties + | Configuration of communication with other nodes by TCP/IP + | Provide basic plumbing to send and receive grid messages and is utilized for all distributed grid operations ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +number('Timeout:', model + '.networkTimeout', 'commNetworkTimeout', 'true', '5000', '1', 'Maximum timeout in milliseconds for network requests') + +number('Timeout:', model + '.networkTimeout', '"commNetworkTimeout"', 'true', '5000', '1', 'Maximum timeout in milliseconds for network requests') .settings-row - +number('Send retry delay:', model + '.networkSendRetryDelay', 'networkSendRetryDelay', 'true', '1000', '1', 'Interval in milliseconds between message send retries') + +number('Send retry delay:', model + '.networkSendRetryDelay', '"networkSendRetryDelay"', 'true', '1000', '1', 'Interval in milliseconds between message send retries') .settings-row - +number('Send retry count:', model + '.networkSendRetryCount', 'networkSendRetryCount', 'true', '3', '1', 'Message send retries count') + +number('Send retry count:', model + '.networkSendRetryCount', '"networkSendRetryCount"', 'true', '3', '1', 'Message send retries count') .settings-row - +number('Discovery startup delay:', model + '.discoveryStartupDelay', 'discoveryStartupDelay', 'true', '60000', '1', 'This value is used to expire messages from waiting list whenever node discovery discrepancies happen') + +number('Discovery startup delay:', model + '.discoveryStartupDelay', '"discoveryStartupDelay"', 'true', '60000', '1', 'This value is used to expire messages from waiting list whenever node discovery discrepancies happen') .settings-row - +java-class('Communication listener:', communication + '.listener', 'comListener', 'true', 'false', 'Communication listener') + +java-class('Communication listener:', communication + '.listener', '"comListener"', 'true', 'false', 'Listener of communication events') .settings-row - +text-ip-address('Local IP address:', communication + '.localAddress', 'comLocalAddress', 'true', '0.0.0.0', 'Local host address for socket binding') + +text-ip-address('Local IP address:', communication + '.localAddress', '"comLocalAddress"', 'true', '0.0.0.0', + 'Local host address for socket binding
          \ + If not specified use all available addres on local host') .settings-row - +number-min-max('Local port:', communication + '.localPort', 'comLocalPort', 'true', '47100', '1024', '65535', 'Local port for socket binding') + +number-min-max('Local port:', communication + '.localPort', '"comLocalPort"', 'true', '47100', '1024', '65535', 'Local port for socket binding') .settings-row - +number('Local port range:', communication + '.localPortRange', 'comLocalPortRange', 'true', '100', '1', 'Local port range for local host ports') + +number('Local port range:', communication + '.localPortRange', '"comLocalPortRange"', 'true', '100', '1', 'Local port range for local host ports') .settings-row - +number-min-max('Shared memory port:', communication + '.sharedMemoryPort', 'sharedMemoryPort', 'true', '48100', '-1', '65535', + +number-min-max('Shared memory port:', communication + '.sharedMemoryPort', '"sharedMemoryPort"', 'true', '48100', '-1', '65535', 'Local port to accept shared memory connections
          \ If set to #[b -1] shared memory communication will be disabled') .settings-row - +number('Idle connection timeout:', communication + '.idleConnectionTimeout', 'idleConnectionTimeout', 'true', '30000', '1', + +number('Idle connection timeout:', communication + '.idleConnectionTimeout', '"idleConnectionTimeout"', 'true', '30000', '1', 'Maximum idle connection timeout upon which a connection to client will be closed') .settings-row - +number('Connect timeout:', communication + '.connectTimeout', 'connectTimeout', 'true', '5000', '0', 'Connect timeout used when establishing connection with remote nodes') + +number('Connect timeout:', communication + '.connectTimeout', '"connectTimeout"', 'true', '5000', '0', 'Connect timeout used when establishing connection with remote nodes') .settings-row - +number('Maximum connect timeout:', communication + '.maxConnectTimeout', 'maxConnectTimeout', 'true', '600000', '0', 'Maximum connect timeout') + +number('Maximum connect timeout:', communication + '.maxConnectTimeout', '"maxConnectTimeout"', 'true', '600000', '0', 'Maximum connect timeout') .settings-row - +number('Reconnect count:', communication + '.reconnectCount', 'comReconnectCount', 'true', '10', '1', + +number('Reconnect count:', communication + '.reconnectCount', '"comReconnectCount"', 'true', '10', '1', 'Maximum number of reconnect attempts used when establishing connection with remote nodes') .settings-row - +number('Socket send buffer:', communication + '.socketSendBuffer', 'socketSendBuffer', 'true', '32768', '0', 'Send buffer size for sockets created or accepted by this SPI') + +number('Socket send buffer:', communication + '.socketSendBuffer', '"socketSendBuffer"', 'true', '32768', '0', 'Send buffer size for sockets created or accepted by this SPI') .settings-row - +number('Socket receive buffer:', communication + '.socketReceiveBuffer', 'socketReceiveBuffer', 'true', '32768', '0', 'Receive buffer size for sockets created or accepted by this SPI') + +number('Socket receive buffer:', communication + '.socketReceiveBuffer', '"socketReceiveBuffer"', 'true', '32768', '0', 'Receive buffer size for sockets created or accepted by this SPI') .settings-row - +number('Slow client queue limit:', communication + '.slowClientQueueLimit', 'slowClientQueueLimit', 'true', '0', '0', 'Slow client queue limit') + +number('Slow client queue limit:', communication + '.slowClientQueueLimit', '"slowClientQueueLimit"', 'true', '0', '0', 'Slow client queue limit') .settings-row - +number('Ack send threshold:', communication + '.ackSendThreshold', 'ackSendThreshold', 'true', '16', '1', 'Number of received messages per connection to node after which acknowledgment message is sent') + +number('Ack send threshold:', communication + '.ackSendThreshold', '"ackSendThreshold"', 'true', '16', '1', 'Number of received messages per connection to node after which acknowledgment message is sent') .settings-row - +number('Message queue limit:', communication + '.messageQueueLimit', 'messageQueueLimit', 'true', '1024', '0', 'Message queue limit for incoming and outgoing messages') + +number('Message queue limit:', communication + '.messageQueueLimit', '"messageQueueLimit"', 'true', '1024', '0', 'Message queue limit for incoming and outgoing messages') .settings-row - +number('Unacknowledged messages:', communication + '.unacknowledgedMessagesBufferSize', 'unacknowledgedMessagesBufferSize', 'true', '0', '0', + +number('Unacknowledged messages:', communication + '.unacknowledgedMessagesBufferSize', '"unacknowledgedMessagesBufferSize"', 'true', '0', '0', 'Maximum number of stored unacknowledged messages per connection to node
          \ If specified non zero value it should be\
            \ @@ -79,18 +82,18 @@ form.panel.panel-default(name=form novalidate)
          • At least message queue limit * 5
          • \
          ') .settings-row - +number('Socket write timeout:', communication + '.socketWriteTimeout', 'socketWriteTimeout', 'true', '2000', '0', 'Socket write timeout') + +number('Socket write timeout:', communication + '.socketWriteTimeout', '"socketWriteTimeout"', 'true', '2000', '0', 'Socket write timeout') .settings-row - +number('Selectors count:', communication + '.selectorsCount', 'selectorsCount', 'true', 'min(4, availableProcessors)', '1', 'Count of selectors te be used in TCP server') + +number('Selectors count:', communication + '.selectorsCount', '"selectorsCount"', 'true', 'min(4, availableProcessors)', '1', 'Count of selectors te be used in TCP server') .settings-row - +java-class('Address resolver:', communication + '.addressResolver', 'comAddressResolver', 'true', 'false', 'Address resolver') + +java-class('Address resolver:', communication + '.addressResolver', '"comAddressResolver"', 'true', 'false', 'Provides resolution between external and internal addresses') .settings-row - +checkbox('Direct buffer', communication + '.directBuffer', 'directBuffer', + +checkbox('Direct buffer', communication + '.directBuffer', '"directBuffer"', 'If value is true, then SPI will use ByteBuffer.allocateDirect(int) call
          \ Otherwise, SPI will use ByteBuffer.allocate(int) call.') .settings-row - +checkbox('Direct send buffer', communication + '.directSendBuffer', 'directSendBuffer', 'Flag defining whether direct send buffer should be used') + +checkbox('Direct send buffer', communication + '.directSendBuffer', '"directSendBuffer"', 'Flag defining whether direct send buffer should be used') .settings-row - +checkbox('TCP_NODELAY option', communication + '.tcpNoDelay', 'tcpNoDelay', 'Value for TCP_NODELAY socket option') + +checkbox('TCP_NODELAY option', communication + '.tcpNoDelay', '"tcpNoDelay"', 'Value for TCP_NODELAY socket option') .col-sm-6 +preview-xml-java(model, 'clusterCommunication') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/connector.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade similarity index 76% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/connector.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade index 95f6bb30f3b1d..baec54fe2ed00 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/connector.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade @@ -21,83 +21,83 @@ include ../../../../../app/helpers/jade/mixins.jade -var enabled = model + '.enabled' -var sslEnabled = enabled + ' && ' + model + '.sslEnabled' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Connector configuration ignite-form-field-tooltip.tipLabel - | REST access configuration + | Configure HTTP REST configuration to enable HTTP server features ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +checkbox('Enabled', enabled, 'restEnabled', 'Flag indicating whether to configure connector configuration') + +checkbox('Enabled', enabled, '"restEnabled"', 'Flag indicating whether to configure connector configuration') .settings-row - +text-enabled('Jetty configuration path:', model + '.jettyPath', 'connectorJettyPath', enabled, 'false', 'Input path to Jetty configuration', + +text-enabled('Jetty configuration path:', model + '.jettyPath', '"connectorJettyPath"', enabled, 'false', 'Input path to Jetty configuration', 'Path, either absolute or relative to IGNITE_HOME, to Jetty XML configuration file
          \ Jetty is used to support REST over HTTP protocol for accessing Ignite APIs remotely
          \ If not provided, Jetty instance with default configuration will be started picking IgniteSystemProperties.IGNITE_JETTY_HOST and IgniteSystemProperties.IGNITE_JETTY_PORT as host and port respectively') .settings-row - +text-ip-address('TCP host:', model + '.host', 'connectorHost', enabled, 'IgniteConfiguration#getLocalHost()', + +text-ip-address('TCP host:', model + '.host', '"connectorHost"', enabled, 'IgniteConfiguration#getLocalHost()', 'Host for TCP binary protocol server
          \ This can be either an IP address or a domain name
          \ If not defined, system - wide local address will be used IgniteConfiguration#getLocalHost()
          \ You can also use "0.0.0.0" value to bind to all locally - available IP addresses') .settings-row - +number-min-max('TCP port:', model + '.port', 'connectorPort', enabled, '11211', '1024', '65535', 'Port for TCP binary protocol server') + +number-min-max('TCP port:', model + '.port', '"connectorPort"', enabled, '11211', '1024', '65535', 'Port for TCP binary protocol server') .settings-row - +number('TCP port range:', model + '.portRange', 'connectorPortRange', enabled, '100', '1', 'Number of ports for TCP binary protocol server to try if configured port is already in use') + +number('TCP port range:', model + '.portRange', '"connectorPortRange"', enabled, '100', '1', 'Number of ports for TCP binary protocol server to try if configured port is already in use') .settings-row - +number('Idle query cursor timeout:', model + '.idleQueryCursorTimeout', 'connectorIdleQueryCursorTimeout', enabled, '600000', '0', + +number('Idle query cursor timeout:', model + '.idleQueryCursorTimeout', '"connectorIdleQueryCursorTimeout"', enabled, '600000', '0', 'Reject open query cursors that is not used timeout
          \ If no fetch query request come within idle timeout, it will be removed on next check for old query cursors') .settings-row - +number('Idle query cursor check frequency:', model + '.idleQueryCursorCheckFrequency', 'connectorIdleQueryCursorCheckFrequency', enabled, '60000', '0', + +number('Idle query cursor check frequency:', model + '.idleQueryCursorCheckFrequency', '"connectorIdleQueryCursorCheckFrequency"', enabled, '60000', '0', 'Idle query cursors check frequency
          \ This setting is used to reject open query cursors that is not used') .settings-row - +number('Idle timeout:', model + '.idleTimeout', 'connectorIdleTimeout', enabled, '7000', '0', + +number('Idle timeout:', model + '.idleTimeout', '"connectorIdleTimeout"', enabled, '7000', '0', 'Idle timeout for REST server
          \ This setting is used to reject half - opened sockets
          \ If no packets come within idle timeout, the connection is closed') .settings-row - +number('Receive buffer size:', model + '.receiveBufferSize', 'connectorReceiveBufferSize', enabled, '32768', '0', 'REST TCP server receive buffer size') + +number('Receive buffer size:', model + '.receiveBufferSize', '"connectorReceiveBufferSize"', enabled, '32768', '0', 'REST TCP server receive buffer size') .settings-row - +number('Send buffer size:', model + '.sendBufferSize', 'connectorSendBufferSize', enabled, '32768', '0', 'REST TCP server send buffer size') + +number('Send buffer size:', model + '.sendBufferSize', '"connectorSendBufferSize"', enabled, '32768', '0', 'REST TCP server send buffer size') .settings-row - +number('Send queue limit:', model + '.sendQueueLimit', 'connectorSendQueueLimit', enabled, 'unlimited', '0', + +number('Send queue limit:', model + '.sendQueueLimit', '"connectorSendQueueLimit"', enabled, 'unlimited', '0', 'REST TCP server send queue limit
          \ If the limit exceeds, all successive writes will block until the queue has enough capacity') .settings-row - +checkbox-enabled('Direct buffer', model + '.directBuffer', 'connectorDirectBuffer', enabled, + +checkbox-enabled('Direct buffer', model + '.directBuffer', '"connectorDirectBuffer"', enabled, 'Flag indicating whether REST TCP server should use direct buffers
          \ A direct buffer is a buffer that is allocated and accessed using native system calls, without using JVM heap
          \ Enabling direct buffer may improve performance and avoid memory issues(long GC pauses due to huge buffer size)') .settings-row - +checkbox-enabled('TCP_NODELAY option', model + '.noDelay', 'connectorNoDelay', enabled, + +checkbox-enabled('TCP_NODELAY option', model + '.noDelay', '"connectorNoDelay"', enabled, 'Flag indicating whether TCP_NODELAY option should be set for accepted client connections
          \ Setting this option reduces network latency and should be enabled in majority of cases
          \ For more information, see Socket#setTcpNoDelay(boolean)') .settings-row - +number('Selector count:', model + '.selectorCount', 'connectorSelectorCount', enabled, 'min(4, availableProcessors)', '1', + +number('Selector count:', model + '.selectorCount', '"connectorSelectorCount"', enabled, 'min(4, availableProcessors)', '1', 'Number of selector threads in REST TCP server
          \ Higher value for this parameter may increase throughput, but also increases context switching') .settings-row - +number('Thread pool size:', model + '.threadPoolSize', 'connectorThreadPoolSize', enabled, 'max(8, availableProcessors) * 2', '1', + +number('Thread pool size:', model + '.threadPoolSize', '"connectorThreadPoolSize"', enabled, 'max(8, availableProcessors) * 2', '1', 'Thread pool size to use for processing of client messages (REST requests)') .settings-row - +java-class('Message interceptor:', model + '.messageInterceptor', 'connectorMessageInterceptor', enabled, 'false', + +java-class('Message interceptor:', model + '.messageInterceptor', '"connectorMessageInterceptor"', enabled, 'false', 'Interceptor allows to transform all objects exchanged via REST protocol
          \ For example if you use custom serialisation on client you can write interceptor to transform binary representations received from client to Java objects and later access them from java code directly') .settings-row - +text-enabled('Secret key:', model + '.secretKey', 'connectorSecretKey', enabled, 'false', 'Specify to enable authentication', 'Secret key to authenticate REST requests') + +text-enabled('Secret key:', model + '.secretKey', '"connectorSecretKey"', enabled, 'false', 'Specify to enable authentication', 'Secret key to authenticate REST requests') .settings-row - +checkbox-enabled('Enable SSL', model + '.sslEnabled', 'connectorSslEnabled', enabled, 'Enables/disables SSL for REST TCP binary protocol') + +checkbox-enabled('Enable SSL', model + '.sslEnabled', '"connectorSslEnabled"', enabled, 'Enables/disables SSL for REST TCP binary protocol') .settings-row - +checkbox-enabled('Enable SSL client auth', model + '.sslClientAuth', 'connectorSslClientAuth', sslEnabled, 'Flag indicating whether or not SSL client authentication is required') + +checkbox-enabled('Enable SSL client auth', model + '.sslClientAuth', '"connectorSslClientAuth"', sslEnabled, 'Flag indicating whether or not SSL client authentication is required') .settings-row - +java-class('SSL factory:', model + '.sslFactory', 'connectorSslFactory', sslEnabled, sslEnabled, + +java-class('SSL factory:', model + '.sslFactory', '"connectorSslFactory"', sslEnabled, sslEnabled, 'Instance of Factory that will be used to create an instance of SSLContext for Secure Socket Layer on TCP binary protocol') .col-sm-6 +preview-xml-java(model, 'clusterConnector') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/deployment.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade similarity index 56% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/deployment.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade index 98e7f617c9e1e..6cfa82d8984b7 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/deployment.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade @@ -21,7 +21,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var exclude = model + '.peerClassLoadingLocalClassPathExclude' -var enabled = 'backupItem.peerClassLoadingEnabled' -form.panel.panel-default(name='deployment' novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Class deployment @@ -32,7 +32,7 @@ form.panel.panel-default(name='deployment' novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +dropdown('Deployment mode:', model + '.deploymentMode', 'deploymentMode', 'true', 'SHARED', + +dropdown('Deployment mode:', model + '.deploymentMode', '"deploymentMode"', 'true', 'SHARED', '[\ {value: "PRIVATE", label: "PRIVATE"},\ {value: "ISOLATED", label: "ISOLATED"}, \ @@ -48,15 +48,17 @@ form.panel.panel-default(name='deployment' novalidate)
        • CONTINUOUS - same as SHARED deployment mode, but resources will not be undeployed even after all master nodes left grid
        • \ ') .settings-row - +checkbox('Enable peer class loading', model + '.peerClassLoadingEnabled', 'peerClassLoadingEnabled', 'Enables/disables peer class loading') + +checkbox('Enable peer class loading', model + '.peerClassLoadingEnabled', '"peerClassLoadingEnabled"', 'Enables/disables peer class loading') .settings-row - +number('Missed resources cache size:', model + '.peerClassLoadingMissedResourcesCacheSize', 'peerClassLoadingMissedResourcesCacheSize', enabled, '100', '0', + +number('Missed resources cache size:', model + '.peerClassLoadingMissedResourcesCacheSize', '"peerClassLoadingMissedResourcesCacheSize"', enabled, '100', '0', 'If size greater than 0, missed resources will be cached and next resource request ignored
          \ If size is 0, then request for the resource will be sent to the remote node every time this resource is requested') .settings-row - +number('Pool size:', model + '.peerClassLoadingThreadPoolSize', 'peerClassLoadingThreadPoolSize', enabled, '2', '1', 'Thread pool size to use for peer class loading') + +number('Pool size:', model + '.peerClassLoadingThreadPoolSize', '"peerClassLoadingThreadPoolSize"', enabled, '2', '1', 'Thread pool size to use for peer class loading') .settings-row - ignite-form-group(ng-model=exclude ng-form=form) + +ignite-form-group(ng-model=exclude ng-form=form) + -var uniqueTip = 'Such package already exists' + ignite-form-field-label | Local class path exclude ignite-form-group-tooltip @@ -65,52 +67,44 @@ form.panel.panel-default(name='deployment' novalidate) ignite-form-group-add(ng-show='#{enabled}' ng-click='(group.add = [{}])') | Add package name. - -var uniqueTip = 'Such package already exists' - -var tipJavaPackageName = 'Package name is invalid' - -var tipJavaKeyWord = 'Package name could not contains reserved java keyword' - .group-content(ng-if=exclude + '.length') - -var field = 'edit' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.igniteUnique' - -var javaPackageName = 'form[ngModelName].$error.javaPackageName' - -var javaKeywords = 'form[ngModelName].$error.javaKeywords' - -var save = exclude + '[$index] = ' + field + -var model = 'obj.model'; + -var name = '"edit" + $index' + -var valid = form + '[' + name + '].$valid' + -var save = exclude + '[$index] = ' + model div(ng-show=enabled) - ignite-form-field(ng-repeat='model in #{exclude} track by $index' type='internal' name='Package name') - .indexField - | {{ $index+1 }}) - +table-remove-button(exclude, 'Remove package name') - span(ng-hide='field.edit') - a.labelFormField(ng-click='#{enabled} && (field.edit = true)') {{ model }} - span(ng-if='field.edit' ng-init='#{field} = model') - +table-java-package-field(field, exclude, valid, save, false) - +table-save-button(valid, save, false) - +unique-feedback(unique, uniqueTip) - +error-feedback(javaPackageName, 'javaPackageName', tipJavaPackageName) - +error-feedback(javaKeywords, 'javaKeywords', tipJavaKeyWord) + div(ng-repeat='model in #{exclude} track by $index' ng-init='obj = {}') + label.col-xs-12.col-sm-12.col-md-12 + .indexField + | {{ $index+1 }}) + +table-remove-button(exclude, 'Remove package name') + span(ng-hide='field.edit') + a.labelFormField(ng-click='#{enabled} && (field.edit = true) && (#{model} = model)') {{ model }} + span(ng-if='field.edit') + +table-java-package-field(name, model, exclude, valid, save, false) + +table-save-button(valid, save, false) + +unique-feedback(name, uniqueTip) + div(ng-hide=enabled) - ignite-form-field(ng-repeat='model in #{exclude} track by $index' type='internal' name='Package name') - .labelFormField.labelField - | {{ $index+1 }}) - span.labelFormField - | {{ model }} + div(ng-repeat='model in #{exclude} track by $index') + label.col-xs-12.col-sm-12.col-md-12 + .labelFormField.labelField + | {{ $index+1 }}) + span.labelFormField + | {{ model }} .group-content(ng-repeat='field in group.add') - -var field = 'new' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.igniteUnique' - -var javaPackageName = 'form[ngModelName].$error.javaPackageName' - -var javaKeywords = 'form[ngModelName].$error.javaKeywords' - -var save = exclude + '.push(' + field + ')' + -var model = 'new'; + -var name = '"new"' + -var valid = form + '[' + name + '].$valid' + -var save = exclude + '.push(' + model + ')' - ignite-form-field(type='internal' name='Package name') - +table-java-package-field(field, exclude, valid, save, true) - +table-save-button(valid, save, true) - +unique-feedback(unique, uniqueTip) - +error-feedback(javaPackageName, 'javaPackageName', tipJavaPackageName) - +error-feedback(javaKeywords, 'javaKeywords', tipJavaKeyWord) + div(type='internal' name='Package name') + label.col-xs-12.col-sm-12.col-md-12 + +table-java-package-field(name, model, exclude, valid, save, true) + +table-save-button(valid, save, true) + +unique-feedback(name, uniqueTip) .group-content-empty(ng-if='!(#{exclude}.length) && !group.add.length') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/discovery.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade similarity index 58% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/discovery.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade index 15b70657d0bfe..1fdcbec44d998 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/discovery.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade @@ -19,65 +19,69 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'discovery' -var model = 'backupItem.discovery' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Discovery ignite-form-field-tooltip.tipLabel - | Discovery properties configuration + | TCP/IP discovery configuration ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +text-ip-address('Local address:', model + '.localAddress', 'discoLocalAddress', 'true', '228.1.2.4', 'Local address') + +text-ip-address('Local address:', model + '.localAddress', '"discoLocalAddress"', 'true', '228.1.2.4', + 'Local host IP address that discovery SPI uses
          \ + If not provided a first found non-loopback address will be used') .settings-row - +number-min-max('Local port:', model + '.localPort', 'discoLocalPort', 'true', '47500', '1024', '65535', 'Local port which node uses') + +number-min-max('Local port:', model + '.localPort', '"discoLocalPort"', 'true', '47500', '1024', '65535', 'Local port which node uses') .settings-row - +number('Local port range:', model + '.localPortRange', 'discoLocalPortRange', 'true', '100', '1', 'Local port range') + +number('Local port range:', model + '.localPortRange', '"discoLocalPortRange"', 'true', '100', '1', 'Local port range') .settings-row - +java-class('Address resolver:', model + '.addressResolver', 'discoAddressResolver', 'true', 'false', - 'Class name of resolution between external and internal addresses provider') + +java-class('Address resolver:', model + '.addressResolver', '"discoAddressResolver"', 'true', 'false', + 'Provides resolution between external and internal addresses') .settings-row - +number('Socket timeout:', model + '.socketTimeout', 'socketTimeout', 'true', '5000', '0', 'Socket operations timeout') + +number('Socket timeout:', model + '.socketTimeout', '"socketTimeout"', 'true', '5000', '0', 'Socket operations timeout') .settings-row - +number('Acknowledgement timeout:', model + '.ackTimeout', 'ackTimeout', 'true', '5000', '0', 'Message acknowledgement timeout') + +number('Acknowledgement timeout:', model + '.ackTimeout', '"ackTimeout"', 'true', '5000', '0', 'Message acknowledgement timeout') .settings-row - +number('Max acknowledgement timeout:', model + '.maxAckTimeout', 'maxAckTimeout', 'true', '600000', '0', 'Maximum message acknowledgement timeout') + +number('Max acknowledgement timeout:', model + '.maxAckTimeout', '"maxAckTimeout"', 'true', '600000', '0', 'Maximum message acknowledgement timeout') .settings-row - +number('Network timeout:', model + '.networkTimeout', 'discoNetworkTimeout', 'true', '5000', '1', 'Network timeout') + +number('Network timeout:', model + '.networkTimeout', '"discoNetworkTimeout"', 'true', '5000', '1', 'Timeout to use for network operations') .settings-row - +number('Join timeout:', model + '.joinTimeout', 'joinTimeout', 'true', '0', '0', + +number('Join timeout:', model + '.joinTimeout', '"joinTimeout"', 'true', '0', '0', 'Join timeout
          ' + '0 means wait forever') .settings-row - +number('Thread priority:', model + '.threadPriority', 'threadPriority', 'true', '10', '1', 'Thread priority for all threads started by SPI') + +number('Thread priority:', model + '.threadPriority', '"threadPriority"', 'true', '10', '1', 'Thread priority for all threads started by SPI') .settings-row - +number('Heartbeat frequency:', model + '.heartbeatFrequency', 'heartbeatFrequency', 'true', '2000', '1', 'Heartbeat messages issuing frequency') + +number('Heartbeat frequency:', model + '.heartbeatFrequency', '"heartbeatFrequency"', 'true', '2000', '1', 'Heartbeat messages issuing frequency') .settings-row - +number('Max heartbeats miss w/o init:', model + '.maxMissedHeartbeats', 'maxMissedHeartbeats', 'true', '1', '1', + +number('Max heartbeats miss w/o init:', model + '.maxMissedHeartbeats', '"maxMissedHeartbeats"', 'true', '1', '1', 'Max heartbeats count node can miss without initiating status check') .settings-row - +number('Max heartbeats miss w/o failing client node:', model + '.maxMissedClientHeartbeats', 'maxMissedClientHeartbeats', 'true', '5', '1') + +number('Max missed client heartbeats:', model + '.maxMissedClientHeartbeats', '"maxMissedClientHeartbeats"', 'true', '5', '1', + 'Max heartbeats count node can miss without failing client node') .settings-row - +number('Topology history:', model + '.topHistorySize', 'topHistorySize', 'true', '1000', '0', 'Size of topology snapshots history') + +number('Topology history:', model + '.topHistorySize', '"topHistorySize"', 'true', '1000', '0', 'Size of topology snapshots history') .settings-row - +java-class('Discovery listener:', model + '.listener', 'discoListener', 'true', 'false', 'Grid discovery listener') + +java-class('Discovery listener:', model + '.listener', '"discoListener"', 'true', 'false', 'Listener for grid node discovery events') .settings-row - +java-class('Data exchange:', model + '.dataExchange', 'dataExchange', 'true', 'false', 'Class name of handler for initial data exchange between Ignite nodes') + +java-class('Data exchange:', model + '.dataExchange', '"dataExchange"', 'true', 'false', 'Class name of handler for initial data exchange between Ignite nodes') .settings-row - +java-class('Metrics provider:', model + '.metricsProvider', 'metricsProvider', 'true', 'false', 'Class name of metric provider to discovery SPI') + +java-class('Metrics provider:', model + '.metricsProvider', '"metricsProvider"', 'true', 'false', 'Class name of metric provider to discovery SPI') .settings-row - +number('Reconnect count:', model + '.reconnectCount', 'discoReconnectCount', 'true', '10', '1', 'Reconnect attempts count') + +number('Reconnect count:', model + '.reconnectCount', '"discoReconnectCount"', 'true', '10', '1', 'Reconnect attempts count') .settings-row - +number('Statistics frequency:', model + '.statisticsPrintFrequency', 'statisticsPrintFrequency', 'true', '0', '1', 'Statistics print frequency') + +number('Statistics frequency:', model + '.statisticsPrintFrequency', '"statisticsPrintFrequency"', 'true', '0', '1', 'Statistics print frequency') .settings-row - +number('IP finder clean frequency:', model + '.ipFinderCleanFrequency', 'ipFinderCleanFrequency', 'true', '60000', '1', 'IP finder clean frequency') + +number('IP finder clean frequency:', model + '.ipFinderCleanFrequency', '"ipFinderCleanFrequency"', 'true', '60000', '1', 'IP finder clean frequency') .settings-row - +java-class('Node authenticator:', model + '.authenticator', 'authenticator', 'true', 'false', 'Node authenticator') + +java-class('Node authenticator:', model + '.authenticator', '"authenticator"', 'true', 'false', 'Class name of node authenticator implementation') .settings-row - +checkbox('Force server mode', model + '.forceServerMode', 'forceServerMode', 'Force server mode') + +checkbox('Force server mode', model + '.forceServerMode', '"forceServerMode"', 'Force start TCP/IP discovery in server mode') .settings-row - +checkbox('Client reconnect disabled', model + '.clientReconnectDisabled', 'clientReconnectDisabled', 'Client reconnect disabled') + +checkbox('Client reconnect disabled', model + '.clientReconnectDisabled', '"clientReconnectDisabled"', + 'Disable try of client to reconnect after server detected client node failure') .col-sm-6 +preview-xml-java(model, 'clusterDiscovery') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/events.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade similarity index 94% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/events.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade index 42582acf93b9f..412714c55657e 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/events.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'events' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Events @@ -30,7 +30,7 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +dropdown-multiple('Include type:', model + '.includeEventTypes', 'includeEventTypes', 'true', 'Choose recorded event types', '', 'eventGroups', + +dropdown-multiple('Include type:', model + '.includeEventTypes', '"includeEventTypes"', true, 'Choose recorded event types', '', 'eventGroups', 'Array of event types, which will be recorded by GridEventStorageManager#record(Event)
          \ Note, that either the include event types or the exclude event types can be established') .col-sm-6 diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/failover.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade similarity index 56% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/failover.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade index a973aeb7c7f99..85f0f5439c9eb 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/failover.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade @@ -21,7 +21,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var failoverSpi = model + '.failoverSpi' -var failoverCustom = 'failover.kind === "Custom"' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Failover configuration @@ -32,7 +32,7 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row(ng-init='failoverSpiTbl={type: "failoverSpi", model: "failoverSpi", focusId: "kind", ui: "failover-table"}') - ignite-form-group(ng-model='#{failoverSpi}' ng-form=form) + +ignite-form-group() ignite-form-field-label | Failover SPI configurations ignite-form-group-tooltip @@ -44,39 +44,29 @@ form.panel.panel-default(name=form novalidate) .group-content(ng-show='#{failoverSpi} && #{failoverSpi}.length > 0' ng-repeat='failover in #{failoverSpi} track by $index') hr(ng-if='$index != 0') .settings-row - ignite-form-field - ignite-form-field-label - | Failover SPI - i.tipField.fa.fa-remove(bs-tooltip='"Remove Failover SPI"' ng-click='removeFailoverConfiguration($index)') - ignite-form-field-tooltip - | Provides ability to supply custom logic for handling failed execution of a grid job - ul - li Job stealing - Supports job stealing from over-utilized nodes to under-utilized nodes - li Never - Jobs are ordered as they arrived - li Always - Jobs are first ordered by their priority - li Custom - Jobs are activated immediately on arrival to mapped node - li Default - Default FailoverSpi implementation - ignite-form-field-dropdown( - data-id='failoverKind{{$index}}' - data-name='failoverKind{{$index}}' - data-options='[\ - {value: "JobStealing", label: "Job stealing"},\ - {value: "Never", label: "Never"},\ - {value: "Always", label: "Always"},\ - {value: "Custom", label: "Custom"}\ - ]' - data-ng-model='failover.kind' - data-ng-required='true' - data-placeholder='Choose Failover SPI' - ) + +dropdown('Failover SPI:', 'failover.kind', '"failoverKind" + $index', 'true', 'Choose Failover SPI', '[\ + {value: "JobStealing", label: "Job stealing"},\ + {value: "Never", label: "Never"},\ + {value: "Always", label: "Always"},\ + {value: "Custom", label: "Custom"}\ + ]', 'Provides ability to supply custom logic for handling failed execution of a grid job\ +
            \ +
          • Job stealing - Supports job stealing from over-utilized nodes to under-utilized nodes
          • \ +
          • Never - Jobs are ordered as they arrived
          • \ +
          • Always - Jobs are first ordered by their priority
          • \ +
          • Custom - Jobs are activated immediately on arrival to mapped node
          • \ +
          • Default - Default FailoverSpi implementation
          • \ +
          ') + + +table-remove-button(failoverSpi, 'Remove Failover SPI') .settings-row(ng-show='failover.kind === "JobStealing"') - +number('Maximum failover attempts:', 'failover.JobStealing.maximumFailoverAttempts', 'jsMaximumFailoverAttempts{{$index}}', 'true', '5', '0', + +number('Maximum failover attempts:', 'failover.JobStealing.maximumFailoverAttempts', '"jsMaximumFailoverAttempts" + $index', 'true', '5', '0', 'Maximum number of attempts to execute a failed job on another node') .settings-row(ng-show='failover.kind === "Always"') - +number('Maximum failover attempts:', 'failover.Always.maximumFailoverAttempts', 'alwaysMaximumFailoverAttempts{{$index}}', 'true', '5', '0', + +number('Maximum failover attempts:', 'failover.Always.maximumFailoverAttempts', '"alwaysMaximumFailoverAttempts" + $index', 'true', '5', '0', 'Maximum number of attempts to execute a failed job on another node') .settings-row(ng-show=failoverCustom) - +java-class('SPI implementation', 'failover.Custom.class', 'failoverSpiClass{{$index}}', 'true', failoverCustom, + +java-class('SPI implementation', 'failover.Custom.class', '"failoverSpiClass" + $index', 'true', failoverCustom, 'Custom FailoverSpi implementation class name.') .col-sm-6 +preview-xml-java(model, 'clusterFailover') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade new file mode 100644 index 0000000000000..d0d390f820495 --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade @@ -0,0 +1,73 @@ +//- + 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. + +include ../../../../../app/helpers/jade/mixins.jade + +-var form = 'general' +-var model = 'backupItem' +-var modelDiscoveryKind = model + '.discovery.kind' + +.panel.panel-default(ng-form=form novalidate) + .panel-heading(bs-collapse-toggle) + ignite-form-panel-chevron + label General + ignite-form-revert + .panel-collapse(role='tabpanel' bs-collapse-target id=form) + .panel-body + .col-sm-6 + .settings-row + +text('Name:', model + '.name', '"clusterName"', 'true', 'Input name', 'Grid name allows to indicate to what grid this particular grid instance belongs to') + .settings-row + +caches(model, 'Select caches to start in cluster or add a new cache') + .settings-row + +text-ip-address('Local host:', model + '.localHost', '"localHost"', 'true', '0.0.0.0', + 'System-wide local address or host for all Ignite components to bind to
          \ + If not defined then Ignite tries to use local wildcard address
          \ + That means that all services will be available on all network interfaces of the host machine') + .settings-row + +dropdown('Discovery:', model + '.discovery.kind', '"discovery"', 'true', 'Choose discovery', 'discoveries', + 'Discovery allows to discover remote nodes in grid\ +
            \ +
          • Static IPs - IP Finder which works only with pre configured list of IP addresses specified
          • \ +
          • Multicast - Multicast based IP finder
          • \ +
          • AWS S3 - AWS S3 based IP finder that automatically discover cluster nodes on Amazon EC2 cloud
          • \ +
          • Apache jclouds - Apache jclouds multi cloud toolkit based IP finder for cloud platforms with unstable IP addresses
          • \ +
          • Google cloud storage - Google Cloud Storage based IP finder that automatically discover cluster nodes on Google Compute Engine cluster
          • \ +
          • JDBC - JDBC based IP finder that use database to store node IP addres
          • \ +
          • Shared filesystem - Shared filesystem based IP finder that use file to store node IP address
          • \ +
          • Apache ZooKeeper - Apache ZooKeeper based IP finder when you use ZooKeeper to coordinate your distributed environment
          • \ +
          ') + .settings-row + .panel-details + div(ng-if='#{modelDiscoveryKind} === "Cloud"') + include ./general/discovery/cloud.jade + div(ng-if='#{modelDiscoveryKind} === "GoogleStorage"') + include ./general/discovery/google.jade + div(ng-if='#{modelDiscoveryKind} === "Jdbc"') + include ./general/discovery/jdbc.jade + div(ng-if='#{modelDiscoveryKind} === "Multicast"') + include ./general/discovery/multicast.jade + div(ng-if='#{modelDiscoveryKind} === "S3"') + include ./general/discovery/s3.jade + div(ng-if='#{modelDiscoveryKind} === "SharedFs"') + include ./general/discovery/shared.jade + div(ng-if='#{modelDiscoveryKind} === "Vm"') + include ./general/discovery/vm.jade + div(ng-if='#{modelDiscoveryKind} === "ZooKeeper"') + include ./general/discovery/zookeeper.jade + .col-sm-6 + -var model = 'backupItem' + +preview-xml-java(model, 'clusterCaches', 'caches') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/cloud.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/cloud.jade similarity index 51% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/cloud.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/cloud.jade index 3a6565d1dfe4a..640c78c5aafee 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/cloud.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/cloud.jade @@ -26,21 +26,22 @@ include ../../../../../../../app/helpers/jade/mixins.jade div .details-row - +text('Credential:', model + '.credential', 'credential', 'false', 'Input cloud credential', + +text('Credential:', model + '.credential', '"credential"', 'false', 'Input cloud credential', 'Credential that is used during authentication on the cloud
          \ Depending on a cloud platform it can be a password or access key') .details-row - +text('Path to credential:', model + '.credentialPath', 'credentialPath', 'false', 'Input pathto credential', + +text('Path to credential:', model + '.credentialPath', '"credentialPath"', 'false', 'Input pathto credential', 'Path to a credential that is used during authentication on the cloud
          \ Access key or private key should be stored in a plain or PEM file without a passphrase') .details-row - +text('Identity:', model + '.identity', discoveryKind + 'Identity', required, 'Input identity', + +text('Identity:', model + '.identity', '"' + discoveryKind + 'Identity"', required, 'Input identity', 'Identity that is used as a user name during a connection to the cloud
          \ Depending on a cloud platform it can be an email address, user name, etc') .details-row - +text('Provider:', model + '.provider', discoveryKind + 'Provider', required, 'Input provider', 'Cloud provider to use') + +text('Provider:', model + '.provider', '"' + discoveryKind + 'Provider"', required, 'Input provider', 'Cloud provider to use') .details-row - ignite-form-group(ng-model=regions ng-form=formRegions) + -var form = formRegions; + +ignite-form-group(ng-model=regions ng-form=form) -var uniqueTip = 'Such region already exists!' ignite-form-field-label @@ -53,37 +54,40 @@ div | Add new region .group-content(ng-if='#{regions}.length') - -var field = 'edit' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.igniteUnique' - -var save = regions + '[$index] = ' + field - - ignite-form-field(ng-repeat='model in #{regions} track by $index' type='internal' name='Region') - .indexField - | {{ $index+1 }}) - +table-remove-button(regions, 'Remove region') - span(ng-hide='field.edit') - a.labelFormField(ng-click='field.edit = true') {{ model }} - span(ng-if='field.edit' ng-init='#{field} = model') - +table-text-field(field, regions, valid, save, 'Region name', false) - +table-save-button(valid, save, false) - +unique-feedback(unique, uniqueTip) + -var model = 'field.model'; + -var name = '"edit" + $index' + -var valid = form + '[' + name + '].$valid' + -var save = regions + '[$index] = ' + model + + div(ng-repeat='model in #{regions} track by $index') + label.col-xs-12.col-sm-12.col-md-12(ng-init='field = {}') + .indexField + | {{ $index+1 }}) + +table-remove-button(regions, 'Remove region') + span(ng-hide='field.edit') + a.labelFormField(ng-click='field.edit = true; #{model} = model;') {{ model }} + span(ng-if='field.edit') + +table-text-field(name, model, regions, valid, save, 'Region name', false) + +table-save-button(valid, save, false) + +unique-feedback(name, uniqueTip) .group-content(ng-repeat='field in group.add') - -var field = 'new' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.igniteUnique' - -var save = regions + '.push(' + field + ')' - - ignite-form-field(type='internal' name='Region') - +table-text-field(field, regions, valid, save, 'Region name', true) - +table-save-button(valid, save, true) - +unique-feedback(unique, uniqueTip) + -var model = 'field.new'; + -var name = '"new"' + -var valid = form + '[' + name + '].$valid' + -var save = regions + '.push(' + model + ')' + + div + label.col-xs-12.col-sm-12.col-md-12 + +table-text-field(name, model, regions, valid, save, 'Region name', true) + +table-save-button(valid, save, false) + +unique-feedback(name, uniqueTip) .group-content-empty(ng-if='!(#{regions}.length) && !group.add.length') | Not defined .details-row - ignite-form-group(ng-model=zones ng-form=formZones) + -var form = formZones; + +ignite-form-group(ng-model=zones ng-form=form) -var uniqueTip = 'Such zone already exists!' ignite-form-field-label @@ -95,33 +99,36 @@ div ignite-form-group-add(ng-click='group.add = [{}]') | Add new zone + -var form = formZones; .group-content(ng-if='#{zones}.length') - -var field = 'edit' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.igniteUnique' - -var save = zones + '[$index] = ' + field - - ignite-form-field(ng-repeat='model in #{zones} track by $index' type='internal' name='Zone') - .indexField - | {{ $index+1 }}) - +table-remove-button(zones, 'Remove zone') - span(ng-hide='field.edit') - a.labelFormField(ng-click='field.edit = true') {{ model }} - span(ng-if='field.edit' ng-init='#{field} = model') - +table-text-field(field, zones, valid, save, 'Zone name', false) - +table-save-button(valid, save, false) - +unique-feedback(unique, uniqueTip) + -var model = 'field.model'; + -var name = '"edit" + $index' + -var valid = form + '[' + name + '].$valid' + -var save = zones + '[$index] = ' + model + + div(ng-repeat='model in #{zones} track by $index') + label.col-xs-12.col-sm-12.col-md-12(ng-init='field = {}') + .indexField + | {{ $index+1 }}) + +table-remove-button(zones, 'Remove zone') + span(ng-hide='field.edit') + a.labelFormField(ng-click='field.edit = true; #{model} = model;') {{ model }} + span(ng-if='field.edit') + +table-text-field(name, model, zones, valid, save, 'Zone name', false) + +table-save-button(valid, save, false) + +unique-feedback(name, uniqueTip) .group-content(ng-repeat='field in group.add') - -var field = 'new' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.igniteUnique' - -var save = zones + '.push(' + field + ')' - - ignite-form-field(type='internal' name='Zone') - +table-text-field(field, zones, valid, save, 'Zone name', true) - +table-save-button(valid, save, true) - +unique-feedback(unique, uniqueTip) + -var model = 'field.new'; + -var name = '"new"' + -var valid = form + '[' + name + '].$valid' + -var save = zones + '.push(' + model + ')' + + div + label.col-xs-12.col-sm-12.col-md-12 + +table-text-field(name, model, zones, valid, save, 'Zone name', true) + +table-save-button(valid, save, true) + +unique-feedback(name, uniqueTip) .group-content-empty(ng-if='!(#{zones}.length) && !group.add.length') | Not defined diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/google.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/google.jade similarity index 76% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/google.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/google.jade index 2a651dff091ec..b1a595806f934 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/google.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/google.jade @@ -23,16 +23,16 @@ include ../../../../../../../app/helpers/jade/mixins.jade div .details-row - +text('Project name:', model + '.projectName', discoveryKind + 'ProjectName', required, 'Input project name', '' + + +text('Project name:', model + '.projectName', '"' + discoveryKind + 'ProjectName"', required, 'Input project name', '' + 'Google Cloud Platforms project name
          \ Usually this is an auto generated project number(ex. 208709979073) that can be found in "Overview" section of Google Developer Console') .details-row - +text('Bucket name:', model + '.bucketName', discoveryKind + 'BucketName', required, 'Input bucket name', + +text('Bucket name:', model + '.bucketName', '"' + discoveryKind + 'BucketName"', required, 'Input bucket name', 'Google Cloud Storage bucket name
          \ If the bucket does not exist Ignite will automatically create it
          \ However the name must be unique across whole Google Cloud Storage and Service Account Id must be authorized to perform this operation') .details-row - +text('Private key path:', model + '.serviceAccountP12FilePath', discoveryKind + 'ServiceAccountP12FilePath', required, 'Input private key path', + +text('Private key path:', model + '.serviceAccountP12FilePath', '"' + discoveryKind + 'ServiceAccountP12FilePath"', required, 'Input private key path', 'Full path to the private key in PKCS12 format of the Service Account') .details-row - +text('Account id:', model + '.serviceAccountId', discoveryKind + 'ServiceAccountId', required, 'Input account id', 'Service account ID (typically an e-mail address)') + +text('Account id:', model + '.serviceAccountId', '"' + discoveryKind + 'ServiceAccountId"', required, 'Input account id', 'Service account ID (typically an e-mail address)') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.jade new file mode 100644 index 0000000000000..42cf69707ac23 --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.jade @@ -0,0 +1,32 @@ +//- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +include ../../../../../../../app/helpers/jade/mixins.jade +include ../../../../../../../app/helpers/jade/mixins.jade + +-var model = 'backupItem.discovery.Jdbc' +-var required = 'backupItem.discovery.kind === "Jdbc"' + +div + .details-row + +checkbox('DB schema should be initialized by Ignite', model + '.initSchema', '"initSchema"', + 'Flag indicating whether DB schema should be initialized by Ignite or was explicitly created by user') + .details-row + +text('Data source bean name:', model + '.dataSourceBean', + '"dataSourceBean"', required, 'Input bean name', 'Name of the data source bean in Spring context') + .details-row + +dialect('Dialect:', model + '.dialect', '"dialect"', required, + 'Dialect of SQL implemented by a particular RDBMS:', 'Generic JDBC dialect', 'Choose JDBC dialect') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/multicast.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/multicast.jade similarity index 54% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/multicast.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/multicast.jade index 2c9dd503c4314..829fbaa4cf72e 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/multicast.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/multicast.jade @@ -22,25 +22,25 @@ include ../../../../../../../app/helpers/jade/mixins.jade div .details-row - +text-ip-address('IP address:', model + '.multicastGroup', 'multicastGroup', 'true', '228.1.2.4', 'IP address of multicast group') + +text-ip-address('IP address:', model + '.multicastGroup', '"multicastGroup"', 'true', '228.1.2.4', 'IP address of multicast group') .details-row - +number-min-max('Port number:', model + '.multicastPort', 'multicastPort', 'true', '47400', '0', '65535', 'Port number which multicast messages are sent to') + +number-min-max('Port number:', model + '.multicastPort', '"multicastPort"', 'true', '47400', '0', '65535', 'Port number which multicast messages are sent to') .details-row - +number('Waits for reply:', model + '.responseWaitTime', 'responseWaitTime', 'true', '500', '0', + +number('Waits for reply:', model + '.responseWaitTime', '"responseWaitTime"', 'true', '500', '0', 'Time in milliseconds IP finder waits for reply to multicast address request') .details-row - +number('Attempts count:', model + '.addressRequestAttempts', 'addressRequestAttempts', 'true', '2', '0', + +number('Attempts count:', model + '.addressRequestAttempts', '"addressRequestAttempts"', 'true', '2', '0', 'Number of attempts to send multicast address request
          \ IP finder re - sends request only in case if no reply for previous request is received') .details-row - +text-ip-address('Local address:', model + '.localAddress', 'localAddress', 'true', '0.0.0.0', + +text-ip-address('Local address:', model + '.localAddress', '"localAddress"', 'true', '0.0.0.0', 'Local host address used by this IP finder
          \ If provided address is non - loopback then multicast socket is bound to this interface
          \ If local address is not set or is any local address then IP finder creates multicast sockets for all found non - loopback addresses') .details-row -var form = 'discoveryMulticastAddresses'; - ignite-form-group(ng-model=addresses ng-form=form) + +ignite-form-group(ng-model=addresses ng-form=form) -var uniqueTip = 'Such IP address already exists!' -var ipAddressTip = 'Invalid IP address!' @@ -62,48 +62,38 @@ div | Add new address .group-content(ng-if='#{addresses}.length') - -var field = 'edit' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.igniteUnique' - -var ipaddress = 'form[ngModelName].$error.ipaddress' - -var ipaddressPort = 'form[ngModelName].$error.ipaddressPort' - -var ipaddressPortRange = 'form[ngModelName].$error.ipaddressPortRange' - -var save = addresses + '[$index] = ' + field + -var model = 'obj.model'; + -var name = '"edit" + $index' + -var valid = form + '[' + name + '].$valid' + -var save = addresses + '[$index] = ' + model - ignite-form-field(ng-repeat='model in #{addresses} track by $index' type='internal' name='Address') - .indexField - | {{ $index+1 }}) - +table-remove-button(addresses, 'Remove address') + div(ng-repeat='model in #{addresses} track by $index' ng-init='obj = {}') + label.col-xs-12.col-sm-12.col-md-12 + .indexField + | {{ $index+1 }}) + +table-remove-button(addresses, 'Remove address') - ignite-form-field-down(ng-if='!$last' ng-hide='field.edit' data-ng-model='model' data-models=addresses) - ignite-form-field-up(ng-if='!$first' ng-hide='field.edit' data-ng-model='model' data-models=addresses) + +ignite-form-field-down(ng-if='!$last' ng-hide='field.edit' data-ng-model='model' data-models=addresses) + +ignite-form-field-up(ng-if='!$first' ng-hide='field.edit' data-ng-model='model' data-models=addresses) - span(ng-hide='field.edit') - a.labelFormField(ng-click='field.edit = true') {{ model }} - span(ng-if='field.edit' ng-init='#{field} = model') - +table-address-field(field, addresses, valid, save, false, true) - +table-save-button(valid, save, false) - +unique-feedback(unique, uniqueTip) - +ipaddress-feedback(ipaddress) - +ipaddress-port-feedback(ipaddressPort) - +ipaddress-port-range-feedback(ipaddressPortRange) + span(ng-hide='field.edit') + a.labelFormField(ng-click='field.edit = true; #{model} = model;') {{ model }} + span(ng-if='field.edit') + +table-address-field(name, model, addresses, valid, save, false, true) + +table-save-button(valid, save, false) + +unique-feedback(name, uniqueTip) .group-content(ng-repeat='field in group.add') - -var field = 'new' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.igniteUnique' - -var ipaddress = 'form[ngModelName].$error.ipaddress' - -var ipaddressPort = 'form[ngModelName].$error.ipaddressPort' - -var ipaddressPortRange = 'form[ngModelName].$error.ipaddressPortRange' - -var save = addresses + '.push(' + field + ')' + -var model = 'new'; + -var name = '"new"' + -var valid = form + '[' + name + '].$valid' + -var save = addresses + '.push(' + model + ')' - ignite-form-field(type='internal' name='Address') - +table-address-field(field, addresses, valid, save, true, true) - +table-save-button(valid, save, true) - +unique-feedback(unique, uniqueTip) - +ipaddress-feedback(ipaddress) - +ipaddress-port-feedback(ipaddressPort) - +ipaddress-port-range-feedback(ipaddressPortRange) + div + label.col-xs-12.col-sm-12.col-md-12 + +table-address-field(name, model, addresses, valid, save, true, true) + +table-save-button(valid, save, true) + +unique-feedback(name, uniqueTip) .group-content-empty(ng-if='!(#{addresses}.length) && !group.add.length') | Not defined diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/s3.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.jade similarity index 88% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/s3.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.jade index c2e29be389408..e255f20547508 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/s3.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.jade @@ -22,6 +22,6 @@ include ../../../../../../../app/helpers/jade/mixins.jade div .details-row - +text('Bucket name:', model + '.bucketName', discoveryKind + 'BucketName', required, 'Input bucket name', 'Bucket name for IP finder') + +text('Bucket name:', model + '.bucketName', '"' + discoveryKind + 'BucketName"', required, 'Input bucket name', 'Bucket name for IP finder') .details-row label Note, AWS credentials will be generated as stub diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/shared.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/shared.jade similarity index 91% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/shared.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/shared.jade index 5ae808a7a2ad2..2a949a5cbc363 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/shared.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/shared.jade @@ -20,4 +20,4 @@ include ../../../../../../../app/helpers/jade/mixins.jade div .details-row - +text('File path:', model + '.path', 'path', 'false', 'disco/tcp', 'Shared path') + +text('File path:', model + '.path', '"path"', 'false', 'disco/tcp', 'Shared path') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.jade new file mode 100644 index 0000000000000..6588e6a9e1b0c --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.jade @@ -0,0 +1,79 @@ +//- + 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. + +include ../../../../../../../app/helpers/jade/mixins.jade + +-var model = 'backupItem.discovery.Vm' +-var addresses = model + '.addresses' +-var form = 'discoveryVmAddresses' + +.details-row + +ignite-form-group(ng-form=form ng-model=addresses) + -var uniqueTip = 'Such IP address already exists!' + + ignite-form-field-label + | Addresses + ignite-form-group-tooltip + | Addresses may be represented as follows: + ul: li IP address (e.g. 127.0.0.1, 9.9.9.9, etc) + li IP address and port (e.g. 127.0.0.1:47500, 9.9.9.9:47501, etc) + li IP address and port range (e.g. 127.0.0.1:47500..47510, 9.9.9.9:47501..47504, etc) + li Hostname (e.g. host1.com, host2, etc) + li Hostname and port (e.g. host1.com:47500, host2:47502, etc) + li Hostname and port range (e.g. host1.com:47500..47510, host2:47502..47508, etc) + | If port is 0 or not provided then default port will be used (depends on discovery SPI configuration)#[br] + | If port range is provided (e.g. host:port1..port2) the following should be considered: + ul: li port1 < port2 should be true + li Both port1 and port2 should be greater than 0 + ignite-form-group-add(ng-click='group.add = [{}]') + | Add new address + + .group-content(ng-if='#{addresses}.length') + -var model = 'obj.model'; + -var name = '"edit" + $index' + -var valid = form + '[' + name + '].$valid' + -var save = addresses + '[$index] = ' + model + + div(ng-repeat='model in #{addresses} track by $index' ng-init='obj = {}') + label.col-xs-12.col-sm-12.col-md-12 + .indexField + | {{ $index+1 }}) + +table-remove-button(addresses, 'Remove address') + + +ignite-form-field-down(ng-if='!$last' ng-hide='field.edit' data-ng-model='model' data-models=addresses) + +ignite-form-field-up(ng-if='!$first' ng-hide='field.edit' data-ng-model='model' data-models=addresses) + + span(ng-hide='field.edit') + a.labelFormField(ng-click='field.edit = true; #{model} = model;') {{ model }} + span(ng-if='field.edit') + +table-address-field(name, model, addresses, valid, save, false, true) + +table-save-button(valid, save, false) + +unique-feedback(name, uniqueTip) + + .group-content(ng-repeat='field in group.add') + -var model = 'new'; + -var name = '"new"' + -var valid = form + '[' + name + '].$valid' + -var save = addresses + '.push(' + model + ')' + + div + label.col-xs-12.col-sm-12.col-md-12 + +table-address-field(name, model, addresses, valid, save, true, true) + +table-save-button(valid, save, true) + +unique-feedback(name, uniqueTip) + + .group-content-empty(id='addresses' ng-if='!(#{addresses}.length) && !group.add.length') + | Not defined diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade similarity index 70% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade index 72f0678ef3f23..afd3ecd87c3ad 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade @@ -16,21 +16,23 @@ include ../../../../../../../app/helpers/jade/mixins.jade +-var form = 'general' -var discoveryKind = 'ZooKeeper' -var required = 'backupItem.discovery.kind == "' + discoveryKind + '"' -var model = 'backupItem.discovery.ZooKeeper' +-var modelRetryPolicyKind = model + '.retryPolicy.kind' div .details-row - +java-class('Curator:', model + '.curator', 'curator', 'true', 'false', + +java-class('Curator:', model + '.curator', '"curator"', 'true', 'false', 'The Curator framework in use
          \ By default generates curator of org.apache.curator. framework.imps.CuratorFrameworkImpl\ class with configured connect string, retry policy, and default session and connection timeouts') .details-row - +text('Connect string:', model + '.zkConnectionString', ZooKeeper + 'ConnectionString', required, 'host:port[chroot][,host:port[chroot]]', + +text('Connect string:', model + '.zkConnectionString', '"' + discoveryKind + 'ConnectionString"', required, 'host:port[chroot][,host:port[chroot]]', 'When "IGNITE_ZK_CONNECTION_STRING" system property is not configured this property will be used') .details-row - +dropdown('Retry policy:', model + '.retryPolicy.kind', 'retryPolicy', 'true', 'Default', + +dropdown('Retry policy:', model + '.retryPolicy.kind', '"retryPolicy"', 'true', 'Default', '[\ {value: "ExponentialBackoff", label: "Exponential backoff"},\ {value: "BoundedExponentialBackoff", label: "Bounded exponential backoff"},\ @@ -54,21 +56,28 @@ div ') .details-row(ng-show='#{model}.retryPolicy.kind') .panel-details - ignite-configuration-clusters-general-discovery-zookeeper-exponential(ng-show='#{model}.retryPolicy.kind === "ExponentialBackoff"') - ignite-configuration-clusters-general-discovery-zookeeper-bounded-exponential(ng-show='#{model}.retryPolicy.kind === "BoundedExponentialBackoff"') - ignite-configuration-clusters-general-discovery-zookeeper-until-elapsed(ng-show='#{model}.retryPolicy.kind === "UntilElapsed"') - ignite-configuration-clusters-general-discovery-zookeeper-n-times(ng-show='#{model}.retryPolicy.kind === "NTimes"') - ignite-configuration-clusters-general-discovery-zookeeper-one-time(ng-show='#{model}.retryPolicy.kind === "OneTime"') - ignite-configuration-clusters-general-discovery-zookeeper-forever(ng-show='#{model}.retryPolicy.kind === "Forever"') - ignite-configuration-clusters-general-discovery-zookeeper-custom(ng-show='#{model}.retryPolicy.kind === "Custom"') + div(ng-show='#{modelRetryPolicyKind} === "ExponentialBackoff"') + include ./zookeeper/retrypolicy/exponential-backoff.jade + div(ng-show='#{modelRetryPolicyKind} === "BoundedExponentialBackoff"') + include ./zookeeper/retrypolicy/bounded-exponential-backoff.jade + div(ng-show='#{modelRetryPolicyKind} === "UntilElapsed"') + include ./zookeeper/retrypolicy/until-elapsed.jade + div(ng-show='#{modelRetryPolicyKind} === "NTimes"') + include ./zookeeper/retrypolicy/n-times.jade + div(ng-show='#{modelRetryPolicyKind} === "OneTime"') + include ./zookeeper/retrypolicy/one-time.jade + div(ng-show='#{modelRetryPolicyKind} === "Forever"') + include ./zookeeper/retrypolicy/forever.jade + div(ng-show='#{modelRetryPolicyKind} === "Custom"') + include ./zookeeper/retrypolicy/custom.jade .details-row - +text('Base path:', model + '.basePath', 'basePath', 'false', '/services', 'Base path for service registration') + +text('Base path:', model + '.basePath', '"basePath"', 'false', '/services', 'Base path for service registration') .details-row - +text('Service name:', model + '.serviceName', 'serviceName', 'false', 'ignite', + +text('Service name:', model + '.serviceName', '"serviceName"', 'false', 'ignite', 'Service name to use, as defined by Curator's ServiceDiscovery recipe
          \ In physical ZooKeeper terms, it represents the node under basePath, under which services will be registered') .details-row - +checkbox('Allow duplicate registrations', model + '.allowDuplicateRegistrations', 'allowDuplicateRegistrations', + +checkbox('Allow duplicate registrations', model + '.allowDuplicateRegistrations', '"allowDuplicateRegistrations"', 'Whether to register each node only once, or if duplicate registrations are allowed
          \ Nodes will attempt to register themselves, plus those they know about
          \ By default, duplicate registrations are not allowed, but you might want to set this property to true if you have multiple network interfaces or if you are facing troubles') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade similarity index 73% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade index 6f6e035064d32..5e4bbda3e44d7 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade @@ -20,8 +20,8 @@ include ../../../../../../../../../app/helpers/jade/mixins.jade div .details-row - +number('Base interval:', model + '.baseSleepTimeMs', 'beBaseSleepTimeMs', 'true', '1000', '0', 'Initial amount of time in ms to wait between retries') + +number('Base interval:', model + '.baseSleepTimeMs', '"beBaseSleepTimeMs"', 'true', '1000', '0', 'Initial amount of time in ms to wait between retries') .details-row - +number('Max interval:', model + '.maxSleepTimeMs', 'beMaxSleepTimeMs', 'true', 'Integer.MAX_VALUE', '0', 'Max time in ms to sleep on each retry') + +number('Max interval:', model + '.maxSleepTimeMs', '"beMaxSleepTimeMs"', 'true', 'Integer.MAX_VALUE', '0', 'Max time in ms to sleep on each retry') .details-row - +number-min-max('Max retries:', model + '.maxRetries', 'beMaxRetries', 'true', '10', '0', '29', 'Max number of times to retry') + +number-min-max('Max retries:', model + '.maxRetries', '"beMaxRetries"', 'true', '10', '0', '29', 'Max number of times to retry') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade similarity index 88% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade index 6e631ab9e6ea3..3dcf514a1b94c 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade @@ -21,4 +21,4 @@ include ../../../../../../../../../app/helpers/jade/mixins.jade -var required = 'backupItem.discovery.kind === "ZooKeeper" && backupItem.discovery.ZooKeeper.retryPolicy.kind === "Custom"' .details-row - +java-class('Class name:', retry + '.className', 'customClassName', 'true', required, 'Custom retry policy implementation class name') + +java-class('Class name:', retry + '.className', '"customClassName"', 'true', required, 'Custom retry policy implementation class name') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade similarity index 73% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade index 9be2a714edc11..618683cee7a86 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade @@ -20,8 +20,8 @@ include ../../../../../../../../../app/helpers/jade/mixins.jade div .details-row - +number('Base interval:', model + '.baseSleepTimeMs', 'expBaseSleepTimeMs', 'true', '1000', '0', 'Initial amount of time in ms to wait between retries') + +number('Base interval:', model + '.baseSleepTimeMs', '"expBaseSleepTimeMs"', 'true', '1000', '0', 'Initial amount of time in ms to wait between retries') .details-row - +number-min-max('Max retries:', model + '.maxRetries', 'expMaxRetries', 'true', '10', '0', '29', 'Max number of times to retry') + +number-min-max('Max retries:', model + '.maxRetries', '"expMaxRetries"', 'true', '10', '0', '29', 'Max number of times to retry') .details-row - +number('Max interval:', model + '.maxSleepMs', 'expMaxSleepMs', 'true', 'Integer.MAX_VALUE', '0', 'Max time in ms to sleep on each retry') + +number('Max interval:', model + '.maxSleepMs', '"expMaxSleepMs"', 'true', 'Integer.MAX_VALUE', '0', 'Max time in ms to sleep on each retry') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade similarity index 87% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade index f4045ebb84c95..10e4fb19d62b9 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade @@ -19,4 +19,4 @@ include ../../../../../../../../../app/helpers/jade/mixins.jade -var model = 'backupItem.discovery.ZooKeeper.retryPolicy.Forever' .details-row - +number('Interval:', model + '.retryIntervalMs', 'feRetryIntervalMs', 'true', '1000', '0', 'Time in ms between retry attempts') + +number('Interval:', model + '.retryIntervalMs', '"feRetryIntervalMs"', 'true', '1000', '0', 'Time in ms between retry attempts') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade similarity index 85% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade index a4083b7962097..329f693e01b13 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade @@ -20,6 +20,6 @@ include ../../../../../../../../../app/helpers/jade/mixins.jade div .details-row - +number('Retries:', model + '.n', 'n', 'true', '10', '0', 'Number of times to retry') + +number('Retries:', model + '.n', '"n"', 'true', '10', '0', 'Number of times to retry') .details-row - +number('Interval:', model + '.sleepMsBetweenRetries', 'ntSleepMsBetweenRetries', 'true', '1000', '0', 'Time in ms between retry attempts') + +number('Interval:', model + '.sleepMsBetweenRetries', '"ntSleepMsBetweenRetries"', 'true', '1000', '0', 'Time in ms between retry attempts') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade similarity index 87% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade index f259630868cc8..938eceadb1307 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade @@ -20,4 +20,4 @@ include ../../../../../../../../../app/helpers/jade/mixins.jade div .details-row - +number('Interval:', model + '.sleepMsBetweenRetry', 'oneSleepMsBetweenRetry', 'true', '1000', '0', 'Time in ms to retry attempt') + +number('Interval:', model + '.sleepMsBetweenRetry', '"oneSleepMsBetweenRetry"', 'true', '1000', '0', 'Time in ms to retry attempt') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade similarity index 81% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade index 24a884e9c2b3d..513da6a14476e 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade @@ -20,6 +20,6 @@ include ../../../../../../../../../app/helpers/jade/mixins.jade div .details-row - +number('Total time:', model + '.maxElapsedTimeMs', 'ueMaxElapsedTimeMs', 'true', '60000', '0', 'Total time in ms for execution of retry attempt') + +number('Total time:', model + '.maxElapsedTimeMs', '"ueMaxElapsedTimeMs"', 'true', '60000', '0', 'Total time in ms for execution of retry attempt') .details-row - +number('Interval:', model + '.sleepMsBetweenRetries', 'ueSleepMsBetweenRetries', 'true', '1000', '0', 'Time in ms between retry attempts') + +number('Interval:', model + '.sleepMsBetweenRetries', '"ueSleepMsBetweenRetries"', 'true', '1000', '0', 'Time in ms between retry attempts') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/igfs.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade similarity index 92% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/igfs.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade index de20dfecd6cfa..181b44fdfddac 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/igfs.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade @@ -19,19 +19,19 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'igfs' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label IGFS ignite-form-field-tooltip.tipLabel - | IGFS (Ignite In-Memory File System) configurations + | IGFS (Ignite In-Memory File System) configurations assigned to cluster ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row +dropdown-multiple('IGFS: (add)', - model + '.igfss', 'igfss', 'true', 'Choose IGFS', 'No IGFS configured', 'igfss', + model + '.igfss', '"igfss"', true, 'Choose IGFS', 'No IGFS configured', 'igfss', 'Select IGFS to start in cluster or add a new IGFS') .col-sm-6 +preview-xml-java(model, 'igfss', 'igfss') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger.jade similarity index 85% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/logger.jade index 3406ff5b31bcd..31f87a9006dc4 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger.jade @@ -16,11 +16,11 @@ include ../../../../../app/helpers/jade/mixins.jade --var model = 'backupItem.logger' -var form = 'logger' +-var model = 'backupItem.logger' -var kind = model + '.kind' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Logger configuration @@ -31,7 +31,7 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +dropdown('Logger:', kind, 'logger', 'true', 'Default', + +dropdown('Logger:', kind, '"logger"', 'true', 'Default', '[\ {value: "Log4j", label: "Apache Log4j"},\ {value: "Log4j2", label: "Apache Log4j 2"},\ @@ -55,11 +55,12 @@ form.panel.panel-default(name=form novalidate) ') .settings-row(ng-show='#{kind} && (#{kind} === "Log4j2" || #{kind} === "Log4j" || #{kind} === "Custom")') .panel-details - ignite-configuration-clusters-logger-log4j2( - ng-show='#{kind} === "Log4j2"') - ignite-configuration-clusters-logger-log4j( - ng-show='#{kind} === "Log4j"') - ignite-configuration-clusters-logger-custom( - ng-show='#{kind} === "Custom"') + div(ng-show='#{kind} === "Log4j2"') + include ./logger/log4j2.jade + div(ng-show='#{kind} === "Log4j"') + include ./logger/log4j.jade + div(ng-show='#{kind} === "Custom"') + include ./logger/custom.jade .col-sm-6 + -var model = 'backupItem.logger' +preview-xml-java(model, 'clusterLogger') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade similarity index 87% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/custom.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade index 58dba159577ec..df80af0a98b31 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/custom.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade @@ -16,9 +16,10 @@ include ../../../../../../app/helpers/jade/mixins.jade +-var form = 'logger' -var model = 'backupItem.logger.Custom' -var required = 'backupItem.logger.kind === "Custom"' div .details-row - +java-class('Class:', model + '.class', 'customLogger', 'true', required, 'Logger implementation class') + +java-class('Class:', model + '.class', '"customLogger"', 'true', required, 'Logger implementation class name') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/log4j.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j.jade similarity index 87% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/log4j.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j.jade index 97f407d677400..cf556ecb0dfe8 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/log4j.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j.jade @@ -16,12 +16,13 @@ include ../../../../../../app/helpers/jade/mixins.jade +-var form = 'logger' -var model = 'backupItem.logger.Log4j' -var pathRequired = model + '.mode === "Path" && backupItem.logger.kind === "Log4j"' div .details-row - +dropdown('Level:', model + '.level', 'log4jLevel', 'true', 'Default', + +dropdown('Level:', model + '.level', '"log4jLevel"', 'true', 'Default', '[\ {value: "OFF", label: "OFF"},\ {value: "FATAL", label: "FATAL"},\ @@ -35,7 +36,7 @@ div ]', 'Level for internal log4j implementation') .details-row - +dropdown-required('Logger configuration:', model + '.mode', 'log4jMode', 'true', 'true', 'Choose logger mode', + +dropdown-required('Logger configuration:', model + '.mode', '"log4jMode"', 'true', 'true', 'Choose logger mode', '[\ {value: "Default", label: "Default"},\ {value: "Path", label: "Path"}\ @@ -46,4 +47,4 @@ div
        • Path - path or URI to XML configuration
        • \ ') .details-row(ng-show=pathRequired) - +text('Path:', model + '.path', 'log4jPath', pathRequired, 'Input path', 'Path or URI to XML configuration') + +text('Path:', model + '.path', '"log4jPath"', pathRequired, 'Input path', 'Path or URI to XML configuration') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/log4j2.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j2.jade similarity index 87% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/log4j2.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j2.jade index b4cea903bc4f9..8b9d3e197c12b 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/log4j2.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j2.jade @@ -16,12 +16,13 @@ include ../../../../../../app/helpers/jade/mixins.jade +-var form = 'logger' -var model = 'backupItem.logger.Log4j2' -var log4j2Required = 'backupItem.logger.kind === "Log4j2"' div .details-row - +dropdown('Level:', model + '.level', 'log4j2Level', 'true', 'Default', + +dropdown('Level:', model + '.level', '"log4j2Level"', 'true', 'Default', '[\ {value: "OFF", label: "OFF"},\ {value: "FATAL", label: "FATAL"},\ @@ -35,4 +36,4 @@ div ]', 'Level for internal log4j2 implementation') .details-row - +text('Path:', model + '.path', 'log4j2Path', log4j2Required, 'Input path', 'Path or URI to XML configuration') + +text('Path:', model + '.path', '"log4j2Path"', log4j2Required, 'Input path', 'Path or URI to XML configuration') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/marshaller.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade similarity index 77% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/marshaller.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade index fbeb11ac5cb54..1fc3ce7c3f7d3 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/marshaller.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade @@ -21,34 +21,39 @@ include ../../../../../app/helpers/jade/mixins.jade -var marshaller = model + '.marshaller' -var optMarshaller = marshaller + '.OptimizedMarshaller' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Marshaller ignite-form-field-tooltip.tipLabel | Marshaller allows to marshal or unmarshal objects in grid#[br] | It provides serialization/deserialization mechanism for all instances that are sent across networks or are otherwise serialized + | By default BinaryMarshaller will be used ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +dropdown('Marshaller:', marshaller + '.kind', 'kind', 'true', 'Choose marshaller', + +dropdown('Marshaller:', marshaller + '.kind', '"kind"', 'true', 'Choose marshaller', '[\ {value: "OptimizedMarshaller", label: "OptimizedMarshaller"},\ {value: "JdkMarshaller", label: "JdkMarshaller"},\ {value: undefined, label: "Not set"}\ ]', 'Instance of marshaller to use in grid
          \ - If not provided, BinaryMarshaller will be used') +
            \ +
          • OptimizedMarshaller - Optimized implementation of marshaller
          • \ +
          • JdkMarshaller - Marshaller based on JDK serialization mechanism
          • \ +
          • Not set - BinaryMarshaller serialize and deserialize all objects in the binary format
          • \ +
          ') a.customize( ng-if='#{marshaller}.kind && #{marshaller}.kind === "OptimizedMarshaller"' ng-click='#{marshaller}.expanded = !#{marshaller}.expanded' ) {{ #{marshaller}.expanded ? "Hide settings" : "Show settings"}} .settings-row - .panel-details(ng-if='#{marshaller}.expanded && #{marshaller}.kind === "OptimizedMarshaller"') + .panel-details(ng-show='#{marshaller}.expanded && #{marshaller}.kind === "OptimizedMarshaller"') .details-row - +number('Streams pool size:', optMarshaller + '.poolSize', 'poolSize', 'true', '0', '0', + +number('Streams pool size:', optMarshaller + '.poolSize', '"poolSize"', 'true', '0', '0', 'Specifies size of cached object streams used by marshaller
          \ Object streams are cached for performance reason to avoid costly recreation for every serialization routine
          \ If 0 (default), pool is not used and each thread has its own cached object stream which it keeps reusing
          \ @@ -56,14 +61,15 @@ form.panel.panel-default(name=form novalidate) Consider using pool in this case. This will limit number of streams that can be created and, therefore, decrease memory consumption
          \ NOTE: Using streams pool can decrease performance since streams will be shared between different threads which will lead to more frequent context switching') .details-row - +checkbox('Require serializable', optMarshaller + '.requireSerializable', 'requireSerializable', + +checkbox('Require serializable', optMarshaller + '.requireSerializable', '"requireSerializable"', 'Whether marshaller should require Serializable interface or not') .settings-row - +checkbox('Marshal local jobs', model + '.marshalLocalJobs', 'marshalLocalJobs', 'If this flag is enabled, jobs mapped to local node will be marshalled as if it was remote node') + +checkbox('Marshal local jobs', model + '.marshalLocalJobs', '"marshalLocalJobs"', 'If this flag is enabled, jobs mapped to local node will be marshalled as if it was remote node') .settings-row - +number('Keep alive time:', model + '.marshallerCacheKeepAliveTime', 'marshallerCacheKeepAliveTime', 'true', '10000', '0') + +number('Keep alive time:', model + '.marshallerCacheKeepAliveTime', '"marshallerCacheKeepAliveTime"', 'true', '10000', '0', + 'Keep alive time of thread pool that is in charge of processing marshaller messages') .settings-row - +number('Pool size:', model + '.marshallerCacheThreadPoolSize', 'marshallerCacheThreadPoolSize', 'true', 'max(8, availableProcessors) * 2', '0', + +number('Pool size:', model + '.marshallerCacheThreadPoolSize', '"marshallerCacheThreadPoolSize"', 'true', 'max(8, availableProcessors) * 2', '1', 'Default size of thread pool that is in charge of processing marshaller messages') .col-sm-6 +preview-xml-java(model, 'clusterMarshaller') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/metrics.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/metrics.jade similarity index 84% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/metrics.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/metrics.jade index 4d41d1072d5c5..2b0dfd6ab6fd2 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/metrics.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/metrics.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'metrics' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Metrics @@ -30,16 +30,17 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +number('Elapsed time:', model + '.metricsExpireTime', 'metricsExpireTime', 'true', 'Long.MAX_VALUE', '1', + +number('Elapsed time:', model + '.metricsExpireTime', '"metricsExpireTime"', 'true', 'Long.MAX_VALUE', '1', 'Time in milliseconds after which a certain metric value is considered expired') .settings-row - +number('History size:', model + '.metricsHistorySize', 'metricsHistorySize', 'true', '10000', '1', + +number('History size:', model + '.metricsHistorySize', '"metricsHistorySize"', 'true', '10000', '1', 'Number of metrics kept in history to compute totals and averages') .settings-row - +number('Log frequency:', model + '.metricsLogFrequency', 'metricsLogFrequency', 'true', '60000', '0', - 'Frequency of metrics log print out. To disable set to 0') + +number('Log frequency:', model + '.metricsLogFrequency', '"metricsLogFrequency"', 'true', '60000', '0', + 'Frequency of metrics log print out
          \ ' + + 'When 0 log print of metrics is disabled') .settings-row - +number('Update frequency:', model + '.metricsUpdateFrequency', 'metricsUpdateFrequency', 'true', '2000', '0', + +number('Update frequency:', model + '.metricsUpdateFrequency', '"metricsUpdateFrequency"', 'true', '2000', '0', 'Job metrics update frequency in milliseconds\
            \
          • If set to -1 job metrics are never updated
          • \ diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/ssl.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade similarity index 51% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/ssl.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade index 0a25e7977e20a..726438672136b 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/ssl.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade @@ -22,35 +22,35 @@ include ../../../../../app/helpers/jade/mixins.jade -var model = cluster + '.sslContextFactory' -var trust = model + '.trustManagers' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label(id='sslConfiguration-title') SSL configuration ignite-form-field-tooltip.tipLabel - | Settings for SSL configuration + | Settings for SSL configuration for creating a secure socket layer ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +checkbox('Enabled', enabled, 'sslEnabled', 'Flag indicating whether to configure SSL configuration') + +checkbox('Enabled', enabled, '"sslEnabled"', 'Flag indicating whether to configure SSL configuration') .settings-row - +text-options('Algorithm to create a key manager:', model + '.keyAlgorithm', 'keyAlgorithm', '["SumX509", "X509"]', enabled, 'false', 'SumX509', + +text-options('Algorithm to create a key manager:', model + '.keyAlgorithm', '"keyAlgorithm"', '["SumX509", "X509"]', enabled, 'false', 'SumX509', 'Sets key manager algorithm that will be used to create a key manager
            \ Notice that in most cased default value suites well, however, on Android platform this value need to be set to X509') .settings-row - +text-enabled('Key store file:', model + '.keyStoreFilePath', 'keyStoreFilePath', enabled, enabled, 'Path to the key store file', + +text-enabled('Key store file:', model + '.keyStoreFilePath', '"keyStoreFilePath"', enabled, enabled, 'Path to the key store file', 'Path to the key store file
            \ This is a mandatory parameter since ssl context could not be initialized without key manager') .settings-row - +text-options('Key store type:', model + '.keyStoreType', 'keyStoreType', '["JKS", "PCKS11", "PCKS12"]', enabled, 'false', 'JKS', + +text-options('Key store type:', model + '.keyStoreType', '"keyStoreType"', '["JKS", "PCKS11", "PCKS12"]', enabled, 'false', 'JKS', 'Key store type used in context initialization') .settings-row - +text-options('Protocol:', model + '.protocol', 'protocol', '["TSL", "SSL"]', enabled, 'false', 'TSL', 'Protocol for secure transport') + +text-options('Protocol:', model + '.protocol', '"protocol"', '["TSL", "SSL"]', enabled, 'false', 'TSL', 'Protocol for secure transport') .settings-row - -var trustManagersForm = 'trustManagers' + -var form = 'trustManagers' - ignite-form-group(ng-form='trustManagers' ng-model='#{trust}') + +ignite-form-group(ng-form=form ng-model=trust) -var uniqueTip = 'Such trust manager already exists!' ignite-form-field-label @@ -61,48 +61,49 @@ form.panel.panel-default(name=form novalidate) | Add new trust manager. .group-content(ng-if='#{trust}.length') - -var field = 'edit' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.igniteUnique' - -var save = trust + '[$index] = ' + field - -var reset = 'field.edit = false' - + -var model = 'obj.model'; + -var name = '"edit" + $index' + -var valid = form + '[' + name + '].$valid' + -var save = trust + '[$index] = ' + model + div(ng-show=enabled) - ignite-form-field(ng-repeat='model in #{trust} track by $index' type='internal' name='Trust manager') - .indexField - | {{ $index+1 }}) - +table-remove-conditional-button(trust, enabled, 'Remove trust manager') - span(ng-hide='field.edit') - a.labelFormField(ng-click='#{enabled} && (field.edit = true)') {{ model }} - span(ng-if='field.edit' ng-init='#{field} = model') - +table-java-class-field('Trust manager', field, trust, valid, save, false) - +table-save-button(valid, save, false) - +unique-feedback(unique, uniqueTip) + div(ng-repeat='model in #{trust} track by $index' ng-init='obj = {}') + label.col-xs-12.col-sm-12.col-md-12 + .indexField + | {{ $index+1 }}) + +table-remove-conditional-button(trust, enabled, 'Remove trust manager') + span(ng-hide='field.edit') + a.labelFormField(ng-click='#{enabled} && (field.edit = true) && (#{model} = model)') {{ model }} + span(ng-if='field.edit') + +table-java-class-field('Trust manager:', name, model, trust, valid, save, false) + +table-save-button(valid, save, false) + +unique-feedback(name, uniqueTip) div(ng-hide=enabled) - ignite-form-field(ng-repeat='model in #{trust} track by $index' type='internal' name='Trust manager') - .labelFormField.labelField - | {{ $index+1 }}) - span.labelFormField - | {{ model }} + div(ng-repeat='model in #{trust} track by $index') + label.col-xs-12.col-sm-12.col-md-12 + .labelFormField.labelField + | {{ $index+1 }}) + span.labelFormField + | {{ model }} .group-content(ng-repeat='field in group.add') - -var field = 'new' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.igniteUnique' - -var save = trust + '.push(' + field + ')' - - ignite-form-field(type='internal' name='Trust manager') - +table-java-class-field('Trust manager', field, trust, valid, save, true) - +table-save-button(valid, save, true) - +unique-feedback(unique, uniqueTip) + -var model = 'new'; + -var name = '"new"' + -var valid = form + '[' + name + '].$valid' + -var save = trust + '.push(' + model + ')' + + div + label.col-xs-12.col-sm-12.col-md-12 + +table-java-class-field('Trust manager', name, model, trust, valid, save, true) + +table-save-button(valid, save, true) + +unique-feedback(name, uniqueTip) .group-content-empty(ng-if='!(#{trust}.length) && !group.add.length') | Not defined - .settings-row(ng-if='!#{trust}.length') - +text-enabled('Trust store file:', model + '.trustStoreFilePath', 'trustStoreFilePath', enabled, 'false', 'Path to the trust store file', 'Path to the trust store file') - .settings-row(ng-if='!#{trust}.length') - +text-options('Trust store type:', model + '.trustStoreType', 'trustStoreType', '["JKS", "PCKS11", "PCKS12"]', enabled, 'false', 'JKS', - 'Trust store type used in context initialization') + .settings-row(ng-show='!#{trust}.length') + +text-enabled('Trust store file:', model + '.trustStoreFilePath', '"trustStoreFilePath"', enabled, 'false', 'Path to the trust store file', 'Path to the trust store file') + .settings-row(ng-show='!#{trust}.length') + +text-options('Trust store type:', model + '.trustStoreType', '"trustStoreType"', '["JKS", "PCKS11", "PCKS12"]', enabled, 'false', 'JKS', 'Trust store type used in context initialization') .col-sm-6 +preview-xml-java(cluster, 'clusterSsl') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/swap.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade similarity index 78% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/swap.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade index fd62fe89fae69..1c75c4f33443c 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/swap.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade @@ -21,7 +21,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var swapModel = model + '.swapSpaceSpi' -var fileSwapModel = swapModel + '.FileSwapSpaceSpi' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Swap @@ -32,35 +32,39 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +dropdown('Swap space SPI:', swapModel + '.kind', 'swapSpaceSpi', 'true', 'Choose swap SPI', + +dropdown('Swap space SPI:', swapModel + '.kind', '"swapSpaceSpi"', 'true', 'Choose swap SPI', '[\ {value: "FileSwapSpaceSpi", label: "File-based swap"},\ {value: undefined, label: "Not set"}\ ]', 'Provides a mechanism in grid for storing data on disk
            \ - Ignite cache uses swap space to overflow data to disk if it cannot fit in memory') + Ignite cache uses swap space to overflow data to disk if it cannot fit in memory\ +
              \ +
            • File-based swap - File-based swap space SPI implementation which holds keys in memory
            • \ +
            • Not set - File-based swap space SPI with default configuration when it needed
            • \ +
            ') a.customize( ng-if='#{swapModel}.kind' ng-click='#{swapModel}.expanded = !#{swapModel}.expanded' ) {{ #{swapModel}.expanded ? "Hide settings" : "Show settings"}} .settings-row - .panel-details(ng-if='#{swapModel}.expanded && #{swapModel}.kind') + .panel-details(ng-show='#{swapModel}.expanded && #{swapModel}.kind') .details-row - +text('Base directory:', fileSwapModel + '.baseDirectory', 'baseDirectory', 'false', 'swapspace', + +text('Base directory:', fileSwapModel + '.baseDirectory', '"baseDirectory"', 'false', 'swapspace', 'Base directory where to write files') .details-row - +number('Read stripe size:', fileSwapModel + '.readStripesNumber', 'readStripesNumber', 'true', 'availableProcessors', '0', + +number('Read stripe size:', fileSwapModel + '.readStripesNumber', '"readStripesNumber"', 'true', 'availableProcessors', '0', 'Read stripe size defines number of file channels to be used concurrently') .details-row - +number-min-max-step('Maximum sparsity:', fileSwapModel + '.maximumSparsity', 'maximumSparsity', 'true', '0.5', '0', '0.999', '0.05', + +number-min-max-step('Maximum sparsity:', fileSwapModel + '.maximumSparsity', '"maximumSparsity"', 'true', '0.5', '0', '0.999', '0.05', 'This property defines maximum acceptable wasted file space to whole file size ratio
            \ When this ratio becomes higher than specified number compacting thread starts working') .details-row - +number('Max write queue size:', fileSwapModel + '.maxWriteQueueSize', 'maxWriteQueueSize', 'true', '1024 * 1024', '0', + +number('Max write queue size:', fileSwapModel + '.maxWriteQueueSize', '"maxWriteQueueSize"', 'true', '1024 * 1024', '0', 'Max write queue size in bytes
            \ If there are more values are waiting for being written to disk then specified size, SPI will block on store operation') .details-row - +number('Write buffer size:', fileSwapModel + '.writeBufferSize', 'writeBufferSize', 'true', '64 * 1024', '0', + +number('Write buffer size:', fileSwapModel + '.writeBufferSize', '"writeBufferSize"', 'true', '64 * 1024', '0', 'Write buffer size in bytes
            \ Write to disk occurs only when this buffer is full') .col-sm-6 diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/thread.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/thread.jade similarity index 85% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/thread.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/thread.jade index 20803aae04c3c..9669a87b2791a 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/thread.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/thread.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'pools' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Thread pools size @@ -30,19 +30,19 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +number('Public:', model + '.publicThreadPoolSize', 'publicThreadPoolSize', 'true', 'max(8, availableProcessors) * 2', '1', + +number('Public:', model + '.publicThreadPoolSize', '"publicThreadPoolSize"', 'true', 'max(8, availableProcessors) * 2', '1', 'Thread pool that is in charge of processing ComputeJob, GridJobs and user messages sent to node') .settings-row - +number('System:', model + '.systemThreadPoolSize', 'systemThreadPoolSize', 'true', 'max(8, availableProcessors) * 2', '1', + +number('System:', model + '.systemThreadPoolSize', '"systemThreadPoolSize"', 'true', 'max(8, availableProcessors) * 2', '1', 'Thread pool that is in charge of processing internal system messages') .settings-row - +number('Management:', model + '.managementThreadPoolSize', 'managementThreadPoolSize', 'true', '4', '1', + +number('Management:', model + '.managementThreadPoolSize', '"managementThreadPoolSize"', 'true', '4', '1', 'Thread pool that is in charge of processing internal and Visor ComputeJob, GridJobs') .settings-row - +number('IGFS:', model + '.igfsThreadPoolSize', 'igfsThreadPoolSize', 'true', 'availableProcessors', '1', + +number('IGFS:', model + '.igfsThreadPoolSize', '"igfsThreadPoolSize"', 'true', 'availableProcessors', '1', 'Thread pool that is in charge of processing outgoing IGFS messages') .settings-row - +number('Rebalance:', model + '.rebalanceThreadPoolSize', 'rebalanceThreadPoolSize', 'true', '1', '1', + +number('Rebalance:', model + '.rebalanceThreadPoolSize', '"rebalanceThreadPoolSize"', 'true', '1', '1', 'Max count of threads can be used at rebalancing') .col-sm-6 +preview-xml-java(model, 'clusterPools') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/time.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/time.jade similarity index 88% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/time.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/time.jade index 714a36c9ca189..813948dfb35ac 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/time.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/time.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'time' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Time configuration @@ -30,18 +30,18 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +number('Samples size:', model + '.clockSyncSamples', 'clockSyncSamples', 'true', '8', '0', + +number('Samples size:', model + '.clockSyncSamples', '"clockSyncSamples"', 'true', '8', '0', 'Number of samples used to synchronize clocks between different nodes
            \ Clock synchronization is used for cache version assignment in CLOCK order mode') .settings-row - +number('Frequency:', model + '.clockSyncFrequency', 'clockSyncFrequency', 'true', '120000', '0', + +number('Frequency:', model + '.clockSyncFrequency', '"clockSyncFrequency"', 'true', '120000', '0', 'Frequency at which clock is synchronized between nodes, in milliseconds
            \ Clock synchronization is used for cache version assignment in CLOCK order mode') .settings-row - +number-min-max('Port base:', model + '.timeServerPortBase', 'timeServerPortBase', 'true', '31100', '0', '65535', + +number-min-max('Port base:', model + '.timeServerPortBase', '"timeServerPortBase"', 'true', '31100', '0', '65535', 'Time server provides clock synchronization between nodes
            \ Base UPD port number for grid time server. Time server will be started on one of free ports in range') .settings-row - +number('Port range:', model + '.timeServerPortRange', 'timeServerPortRange', 'true', '100', '1', 'Time server port range') + +number('Port range:', model + '.timeServerPortRange', '"timeServerPortRange"', 'true', '100', '1', 'Time server port range') .col-sm-6 +preview-xml-java(model, 'clusterTime') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/transactions.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade similarity index 67% rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/transactions.jade rename to modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade index 3b0fef74d48cd..d9611a5f01509 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/transactions.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'transactions' -var model = 'backupItem.transactionConfiguration' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Transactions @@ -30,30 +30,40 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +dropdown('Concurrency:', model + '.defaultTxConcurrency', 'defaultTxConcurrency', 'true', 'PESSIMISTIC', + +dropdown('Concurrency:', model + '.defaultTxConcurrency', '"defaultTxConcurrency"', 'true', 'PESSIMISTIC', '[\ {value: "OPTIMISTIC", label: "OPTIMISTIC"},\ {value: "PESSIMISTIC", label: "PESSIMISTIC"}\ ]', - 'Cache transaction concurrency to use when one is not explicitly specified') + 'Cache transaction concurrency to use when one is not explicitly specified\ +
              \ +
            • OPTIMISTIC - All cache operations are not distributed to other nodes until commit is called
            • \ +
            • PESSIMISTIC - A lock is acquired on all cache operations with exception of read operations in READ_COMMITTED mode
            • \ +
            \ + ') .settings-row - +dropdown('Isolation:', model + '.defaultTxIsolation', 'defaultTxIsolation', 'true', 'REPEATABLE_READ', + +dropdown('Isolation:', model + '.defaultTxIsolation', '"defaultTxIsolation"', 'true', 'REPEATABLE_READ', '[\ {value: "READ_COMMITTED", label: "READ_COMMITTED"},\ {value: "REPEATABLE_READ", label: "REPEATABLE_READ"},\ {value: "SERIALIZABLE", label: "SERIALIZABLE"}\ ]', - 'Default transaction isolation') + 'Default transaction isolation\ +
              \ +
            • READ_COMMITTED - Always a committed value will be provided for read operations
            • \ +
            • REPEATABLE_READ - If a value was read once within transaction, then all consecutive reads will provide the same in-transaction value
            • \ +
            • SERIALIZABLE - All transactions occur in a completely isolated fashion, as if all transactions in the system had executed serially, one after the other.
            • \ +
            ') .settings-row - +number('Default timeout:', model + '.defaultTxTimeout', 'defaultTxTimeout', 'true', '0', '0', 'Default transaction timeout') + +number('Default timeout:', model + '.defaultTxTimeout', '"defaultTxTimeout"', 'true', '0', '0', 'Default transaction timeout') .settings-row - +number('Pessimistic log cleanup delay:', model + '.pessimisticTxLogLinger', 'pessimisticTxLogLinger', 'true', '10000', '0', + +number('Pessimistic log cleanup delay:', model + '.pessimisticTxLogLinger', '"pessimisticTxLogLinger"', 'true', '10000', '0', 'Delay, in milliseconds, after which pessimistic recovery entries will be cleaned up for failed node') .settings-row - +number('Pessimistic log size:', model + '.pessimisticTxLogSize', 'pessimisticTxLogSize', 'true', '0', '0', + +number('Pessimistic log size:', model + '.pessimisticTxLogSize', '"pessimisticTxLogSize"', 'true', '0', '0', 'Size of pessimistic transactions log stored on node in order to recover transaction commit if originating node has left grid before it has sent all messages to transaction nodes') .settings-row - +java-class('Manager factory:', model + '.txManagerFactory', 'txManagerFactory', 'true', 'false', + +java-class('Manager factory:', model + '.txManagerFactory', '"txManagerFactory"', 'true', 'false', 'Class name of transaction manager factory for integration with JEE app servers') .col-sm-6 +preview-xml-java(model, 'clusterTransactions') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/domains/general.jade b/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade similarity index 85% rename from modules/web-console/src/main/js/app/modules/states/configuration/domains/general.jade rename to modules/web-console/frontend/app/modules/states/configuration/domains/general.jade index c7d25b4415aa9..5c55e0cfcac23 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/domains/general.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade - var form = 'general' - var model = 'backupItem' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle) ignite-form-panel-chevron label General @@ -32,15 +32,15 @@ form.panel.panel-default(name=form novalidate) .settings-row +caches(model, 'Select caches to associate domain model with cache') .settings-row - +dropdown-required('Query metadata:', model + '.queryMetadata', 'queryMetadata', 'true', 'true', '', 'queryMetadataVariants', + +dropdown-required('Query metadata:', model + '.queryMetadata', '"queryMetadata"', 'true', 'true', '', 'queryMetadataVariants', 'Query metadata configured with:\
              \
            • Java annotations like @QuerySqlField
            • \
            • Configuration via QueryEntity class
            • \
            ') .settings-row - +java-class-typeahead('Key type:', model + '.keyType', 'keyType', 'javaBuiltInClasses', 'true', 'true', 'Full class name for Key', 'Key class used to store key in cache') + +java-class-typeahead('Key type:', model + '.keyType', '"keyType"', 'javaBuiltInClasses', 'true', 'true', 'Full class name for Key', 'Key class used to store key in cache') .settings-row - +java-class('Value type:', model + '.valueType', 'valueType', 'true', 'true', 'Value class used to store value in cache') + +java-class('Value type:', model + '.valueType', '"valueType"', 'true', 'true', 'Value class used to store value in cache') .col-sm-6 +preview-xml-java(model, 'domainModelGeneral') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/domains/query.jade b/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade similarity index 97% rename from modules/web-console/src/main/js/app/modules/states/configuration/domains/query.jade rename to modules/web-console/frontend/app/modules/states/configuration/domains/query.jade index 9ac0d00a8abb1..33c358a28024e 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/domains/query.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade @@ -16,6 +16,7 @@ include ../../../../../app/helpers/jade/mixins.jade +- var form = 'query' - var model = 'backupItem' - var queryFields = model + '.fields' - var queryAliases = model + '.aliases' @@ -49,7 +50,7 @@ mixin table-index-item-edit(prefix, index, sortAvailable, idAddition) .input-tip input.form-control(id='{{::"#{fieldName}" + #{idAddition}}}' type='text' ng-model=fieldNameModel placeholder='Field name' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()') -form.panel.panel-default(name='query' novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle) ignite-form-panel-chevron label(id='query-title') Domain model for SQL query @@ -63,7 +64,7 @@ form.panel.panel-default(name='query' novalidate) label Not available for annotated types div(ng-if='#{model}.queryMetadata === "Configuration"') .settings-row - ignite-form-group(ng-model='#{queryFields}' ng-form='#{queryFieldsForm}') + +ignite-form-group(ng-model='#{queryFields}' ng-form='#{queryFieldsForm}') ignite-form-field-label(id='queryFields') | Fields ignite-form-group-tooltip @@ -86,7 +87,7 @@ form.panel.panel-default(name='query' novalidate) td.col-sm-12 +table-pair-edit('queryFieldsTbl', 'new', 'Field name', 'Field full class name', false, true, '{{::queryFieldsTbl.focusId + $index}}', '-1', '/') .settings-row - ignite-form-group(ng-model='#{queryAliases}' ng-form='#{queryAliasesForm}') + +ignite-form-group(ng-model='#{queryAliases}' ng-form='#{queryAliasesForm}') ignite-form-field-label | Aliases ignite-form-group-tooltip @@ -110,7 +111,7 @@ form.panel.panel-default(name='query' novalidate) td.col-sm-12 +table-pair-edit('aliasesTbl', 'new', 'Field name', 'Field Alias', false, false, '{{::aliasesTbl.focusId + $index}}', '-1', '→') .settings-row(ng-init='indexesTbl={type: "table-indexes", model: "indexes", focusId: "IndexName", ui: "table-indexes"}') - ignite-form-group(ng-model='#{queryIndexes}' ng-form='#{queryIndexesForm}') + +ignite-form-group(ng-model='#{queryIndexes}' ng-form='#{queryIndexesForm}') ignite-form-field-label | Indexes ignite-form-group-tooltip diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/domains/store.jade b/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade similarity index 95% rename from modules/web-console/src/main/js/app/modules/states/configuration/domains/store.jade rename to modules/web-console/frontend/app/modules/states/configuration/domains/store.jade index 8d3c65b2b213b..96913bbab0bed 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/domains/store.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade @@ -61,7 +61,7 @@ mixin table-db-field-edit(tbl, prefix, focusId, index) .input-tip button.select-toggle.form-control(id=javaTypeId ng-model=javaTypeModel data-placeholder='Java type' ng-class='{placeholder: !#{javaTypeModel}}' bs-select bs-options='item.value as item.label for item in {{supportedJavaTypes}}' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()' tabindex='0') -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Domain model for cache store @@ -72,11 +72,11 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +text('Database schema:', model + '.databaseSchema', 'databaseSchema', 'false', 'Input DB schema name', 'Schema name in database') + +text('Database schema:', model + '.databaseSchema', '"databaseSchema"', 'false', 'Input DB schema name', 'Schema name in database') .settings-row - +text('Database table:', model + '.databaseTable', 'databaseTable', 'false', 'Input DB table name', 'Table name in database') + +text('Database table:', model + '.databaseTable', '"databaseTable"', 'false', 'Input DB table name', 'Table name in database') .settings-row(ng-init='keysTbl={type: "table-db-fields", model: "keyFields", focusId: "KeyField", ui: "table-db-fields"}') - ignite-form-group(ng-model='#{keyFields}' ng-form='#{keyFieldsForm}') + +ignite-form-group(ng-form=keyFieldsForm ng-model=keyFields) ignite-form-field-label(id='keyFields') | Key fields ignite-form-group-tooltip @@ -99,7 +99,7 @@ form.panel.panel-default(name=form novalidate) td +table-db-field-edit('keysTbl', 'new', 'KeyField', '-1') .settings-row(ng-init='valuesTbl={type: "table-db-fields", model: "valueFields", focusId: "ValueField", ui: "table-db-fields"}') - ignite-form-group(ng-model='#{valueFields}' ng-form='#{valueFieldsForm}') + +ignite-form-group(ng-form=valueFieldsForm ng-model=valueFields) ignite-form-field-label(id='valueFields') | Value fields ignite-form-group-tooltip diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/dual.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/dual.jade similarity index 83% rename from modules/web-console/src/main/js/app/modules/states/configuration/igfs/dual.jade rename to modules/web-console/frontend/app/modules/states/configuration/igfs/dual.jade index 8129f0d18c6ff..1b3a00a979eb4 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/dual.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/dual.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'dualMode' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Dual mode @@ -31,12 +31,12 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +number('Maximum pending puts size:', model + '.dualModeMaxPendingPutsSize', 'dualModeMaxPendingPutsSize', 'true', '0', 'Number.MIN_SAFE_INTEGER', + +number('Maximum pending puts size:', model + '.dualModeMaxPendingPutsSize', '"dualModeMaxPendingPutsSize"', 'true', '0', 'Number.MIN_SAFE_INTEGER', 'Maximum amount of pending data read from the secondary file system and waiting to be written to data cache
            \ Zero or negative value stands for unlimited size') .settings-row - +java-class('Put executor service:', model + '.dualModePutExecutorService', 'dualModePutExecutorService', 'true', 'false', 'DUAL mode put operation executor service') + +java-class('Put executor service:', model + '.dualModePutExecutorService', '"dualModePutExecutorService"', 'true', 'false', 'DUAL mode put operation executor service') .settings-row - +checkbox('Put executor service shutdown', model + '.dualModePutExecutorServiceShutdown', 'dualModePutExecutorServiceShutdown', 'DUAL mode put operation executor service shutdown flag') + +checkbox('Put executor service shutdown', model + '.dualModePutExecutorServiceShutdown', '"dualModePutExecutorServiceShutdown"', 'DUAL mode put operation executor service shutdown flag') .col-sm-6 +preview-xml-java(model, 'igfsDualMode') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/fragmentizer.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/fragmentizer.jade similarity index 74% rename from modules/web-console/src/main/js/app/modules/states/configuration/igfs/fragmentizer.jade rename to modules/web-console/frontend/app/modules/states/configuration/igfs/fragmentizer.jade index fa8c244f0e095..edc235200e98a 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/fragmentizer.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/fragmentizer.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'fragmentizer' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Fragmentizer @@ -32,12 +32,12 @@ form.panel.panel-default(name=form novalidate) -var enabled = model + '.fragmentizerEnabled' .settings-row - +checkbox('Enabled', enabled, 'fragmentizerEnabled', 'Fragmentizer enabled flag') + +checkbox('Enabled', enabled, '"fragmentizerEnabled"', 'Fragmentizer enabled flag') .settings-row - +number('Concurrent files:', model + '.fragmentizerConcurrentFiles', 'fragmentizerConcurrentFiles', enabled, '0', '0', 'Number of files to process concurrently by fragmentizer') + +number('Concurrent files:', model + '.fragmentizerConcurrentFiles', '"fragmentizerConcurrentFiles"', enabled, '0', '0', 'Number of files to process concurrently by fragmentizer') .settings-row - +number('Throttling block length:', model + '.fragmentizerThrottlingBlockLength', 'fragmentizerThrottlingBlockLength', enabled, '16777216', '1', 'Length of file chunk to transmit before throttling is delayed') + +number('Throttling block length:', model + '.fragmentizerThrottlingBlockLength', '"fragmentizerThrottlingBlockLength"', enabled, '16777216', '1', 'Length of file chunk to transmit before throttling is delayed') .settings-row - +number('Throttling delay:', model + '.fragmentizerThrottlingDelay', 'fragmentizerThrottlingDelay', enabled, '200', '0', 'Delay in milliseconds for which fragmentizer is paused') + +number('Throttling delay:', model + '.fragmentizerThrottlingDelay', '"fragmentizerThrottlingDelay"', enabled, '200', '0', 'Delay in milliseconds for which fragmentizer is paused') .col-sm-6 +preview-xml-java(model, 'igfsFragmentizer') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/general.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade similarity index 90% rename from modules/web-console/src/main/js/app/modules/states/configuration/igfs/general.jade rename to modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade index e81dd9bf78c94..b0871208e18ef 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/general.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade @@ -16,9 +16,10 @@ include ../../../../../app/helpers/jade/mixins.jade +-var form = 'general' -var model = 'backupItem' -form.panel.panel-default(name='general' novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle) ignite-form-panel-chevron label General @@ -27,18 +28,18 @@ form.panel.panel-default(name='general' novalidate) .panel-body .col-sm-6 .settings-row - +text('Name:', model + '.name', 'igfsName', 'true', 'Input name', 'IGFS name') + +text('Name:', model + '.name', '"igfsName"', 'true', 'Input name', 'IGFS name') .settings-row +clusters(model, 'Associate clusters with the current IGFS') .settings-row - +dropdown('IGFS mode:', model + '.defaultMode', 'defaultMode', 'true', 'DUAL_ASYNC', + +dropdown('IGFS mode:', model + '.defaultMode', '"defaultMode"', 'true', 'DUAL_ASYNC', '[\ {value: "PRIMARY", label: "PRIMARY"},\ {value: "PROXY", label: "PROXY"},\ {value: "DUAL_SYNC", label: "DUAL_SYNC"},\ {value: "DUAL_ASYNC", label: "DUAL_ASYNC"}\ ]', - 'Mode to specify how IGFS interacts with Hadoop file system:\ + 'Mode to specify how IGFS interacts with Hadoop file system\
              \
            • PRIMARY - in this mode IGFS will not delegate to secondary Hadoop file system and will cache all the files in memory only
            • \
            • PROXY - in this mode IGFS will not cache any files in memory and will only pass them through to secondary file system
            • \ @@ -46,7 +47,7 @@ form.panel.panel-default(name='general' novalidate)
            • DUAL_ASYNC - in this mode IGFS will cache files locally and also asynchronously write them through to secondary file system
            • \
            ') .settings-row - +number('Group size:', model + '.affinnityGroupSize', 'affinnityGroupSize', 'true', '512', '1', + +number('Group size:', model + '.affinnityGroupSize', '"affinnityGroupSize"', 'true', '512', '1', 'Size of the group in blocks
            \ Required for construction of affinity mapper in IGFS data cache') .col-sm-6 diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/ipc.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade similarity index 70% rename from modules/web-console/src/main/js/app/modules/states/configuration/igfs/ipc.jade rename to modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade index 361036a844137..bb5e00b54b724 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/ipc.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'ipc' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label IPC @@ -33,9 +33,9 @@ form.panel.panel-default(name=form novalidate) -var enabled = model + '.ipcEndpointEnabled' .settings-row - +checkbox('Enabled', enabled, 'ipcEndpointEnabled', 'IPC endpoint enabled flag') + +checkbox('Enabled', enabled, '"ipcEndpointEnabled"', 'IPC endpoint enabled flag') .settings-row - +dropdown('Type:', ipcEndpointConfiguration + '.type', 'ipcEndpointConfigurationType', enabled, 'TCP', + +dropdown('Type:', ipcEndpointConfiguration + '.type', '"ipcEndpointConfigurationType"', enabled, 'TCP', '[\ {value: "SHMEM", label: "SHMEM"},\ {value: "TCP", label: "TCP"}\ @@ -46,12 +46,15 @@ form.panel.panel-default(name=form novalidate)
          • TCP - TCP endpoint
          • \
          ') .settings-row - +text-ip-address('Host:', ipcEndpointConfiguration + '.host', 'ipcEndpointConfigurationHost', enabled, '127.0.0.1', 'Host name') + +text-ip-address('Host:', ipcEndpointConfiguration + '.host', '"ipcEndpointConfigurationHost"', enabled, '127.0.0.1', 'Host endpoint is bound to') .settings-row - +number-min-max('Port:', ipcEndpointConfiguration + '.port', 'ipcEndpointConfigurationPort', enabled, '10500', '1', '65535', 'Port number') + +number-min-max('Port:', ipcEndpointConfiguration + '.port', '"ipcEndpointConfigurationPort"', enabled, '10500', '1', '65535', 'Port endpoint is bound to') .settings-row - +number('Memory size:', ipcEndpointConfiguration + '.memorySize', 'ipcEndpointConfigurationMemorySize', enabled, '262144', '1', 'Shared memory size in bytes allocated for endpoint communication') + +number('Memory size:', ipcEndpointConfiguration + '.memorySize', '"ipcEndpointConfigurationMemorySize"', enabled, '262144', '1', 'Shared memory size in bytes allocated for endpoint communication') .settings-row - +text-enabled('Token directory:', ipcEndpointConfiguration + '.tokenDirectoryPath', 'ipcEndpointConfigurationTokenDirectoryPath', enabled, 'false', 'ipc/shmem', 'Directory where shared memory tokens are stored') + +text-enabled('Token directory:', ipcEndpointConfiguration + '.tokenDirectoryPath', '"ipcEndpointConfigurationTokenDirectoryPath"', enabled, 'false', 'ipc/shmem', 'Directory where shared memory tokens are stored') + .settings-row + +number('Thread count:', ipcEndpointConfiguration + '.threadCount', 'ipcEndpointConfigurationThreadCount', enabled, 'availableProcessors', '1', + 'Number of threads used by this endpoint to process incoming requests') .col-sm-6 +preview-xml-java(model, 'igfsIPC') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/misc.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade similarity index 78% rename from modules/web-console/src/main/js/app/modules/states/configuration/igfs/misc.jade rename to modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade index dc48d07a4d135..cb4687addf5a7 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/misc.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade @@ -42,7 +42,7 @@ mixin table-igfs-path-mode-edit(prefix, focusId, index) .input-tip button.select-toggle.form-control(id=valFocusId bs-select ng-model=valModel data-placeholder='Mode' bs-options='item.value as item.label for item in igfsModes' tabindex='0' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()') -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Miscellaneous @@ -53,34 +53,34 @@ form.panel.panel-default(name=form novalidate) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +number('Block size:', model + '.blockSize', 'blockSize', 'true', '65536', '0', 'File data block size in bytes') + +number('Block size:', model + '.blockSize', '"blockSize"', 'true', '65536', '0', 'File data block size in bytes') .settings-row - +number('Stream buffer size:', model + '.streamBufferSize', 'streamBufferSize', 'true', '65536', '0', 'File data block size in bytes') + +number('Stream buffer size:', model + '.streamBufferSize', '"streamBufferSize"', 'true', '65536', '0', 'Read/write buffer size for IGFS stream operations in bytes') .settings-row - +number('Maximum space size:', model + '.maxSpaceSize', 'maxSpaceSize', 'true', '0', '0', 'Maximum space available for data cache to store file system entries') + +number('Maximum space size:', model + '.maxSpaceSize', '"maxSpaceSize"', 'true', '0', '0', 'Maximum space available for data cache to store file system entries') .settings-row - +number('Maximum task range length:', model + '.maximumTaskRangeLength', 'maximumTaskRangeLength', 'true', '0', '0', 'Maximum default range size of a file being split during IGFS task execution') + +number('Maximum task range length:', model + '.maximumTaskRangeLength', '"maximumTaskRangeLength"', 'true', '0', '0', 'Maximum default range size of a file being split during IGFS task execution') .settings-row - +number-min-max('Management port:', model + '.managementPort', 'managementPort', 'true', '11400', '0', '65535', 'Port number for management endpoint') + +number-min-max('Management port:', model + '.managementPort', '"managementPort"', 'true', '11400', '0', '65535', 'Port number for management endpoint') .settings-row - +number('Per node batch size:', model + '.perNodeBatchSize', 'perNodeBatchSize', 'true', '100', '0', 'Number of file blocks collected on local node before sending batch to remote node') + +number('Per node batch size:', model + '.perNodeBatchSize', '"perNodeBatchSize"', 'true', '100', '0', 'Number of file blocks collected on local node before sending batch to remote node') .settings-row - +number('Per node parallel batch count:', model + '.perNodeParallelBatchCount', 'perNodeParallelBatchCount', 'true', '8', '0', 'Number of file block batches that can be concurrently sent to remote node') + +number('Per node parallel batch count:', model + '.perNodeParallelBatchCount', '"perNodeParallelBatchCount"', 'true', '8', '0', 'Number of file block batches that can be concurrently sent to remote node') .settings-row - +number('Prefetch blocks:', model + '.prefetchBlocks', 'prefetchBlocks', 'true', '0', '0', 'Number of pre-fetched blocks if specific file chunk is requested') + +number('Prefetch blocks:', model + '.prefetchBlocks', '"prefetchBlocks"', 'true', '0', '0', 'Number of pre-fetched blocks if specific file chunk is requested') .settings-row - +number('Sequential reads before prefetch:', model + '.sequentialReadsBeforePrefetch', 'sequentialReadsBeforePrefetch', 'true', '0', '0', 'Amount of sequential block reads before prefetch is triggered') + +number('Sequential reads before prefetch:', model + '.sequentialReadsBeforePrefetch', '"sequentialReadsBeforePrefetch"', 'true', '0', '0', 'Amount of sequential block reads before prefetch is triggered') .settings-row - +number('Trash purge timeout:', model + '.trashPurgeTimeout', 'trashPurgeTimeout', 'true', '1000', '0', 'Maximum timeout awaiting for trash purging in case data cache oversize is detected') + +number('Trash purge timeout:', model + '.trashPurgeTimeout', '"trashPurgeTimeout"', 'true', '1000', '0', 'Maximum timeout awaiting for trash purging in case data cache oversize is detected') .settings-row - +checkbox('Colocate metadata', model + '.colocateMetadata', 'colocateMetadata', 'Whether to co-locate metadata on a single node') + +checkbox('Colocate metadata', model + '.colocateMetadata', '"colocateMetadata"', 'Whether to co-locate metadata on a single node') .settings-row - +checkbox('Relaxed consistency', model + '.relaxedConsistency', 'relaxedConsistency', + +checkbox('Relaxed consistency', model + '.relaxedConsistency', '"relaxedConsistency"', 'If value of this flag is true, IGFS will skip expensive consistency checks
          \ It is recommended to set this flag to false if your application has conflicting\ operations, or you do not know how exactly users will use your system') .settings-row - ignite-form-group(ng-model=pathModes ng-form=pathModesForm) + +ignite-form-group(ng-model=pathModes ng-form=pathModesForm) ignite-form-field-label | Path modes ignite-form-group-tooltip diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/secondary.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade similarity index 80% rename from modules/web-console/src/main/js/app/modules/states/configuration/igfs/secondary.jade rename to modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade index 333fad9c5920f..0649527222759 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/secondary.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade @@ -19,7 +19,7 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'secondaryFileSystem' -var model = 'backupItem' -form.panel.panel-default(name=form novalidate) +.panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label(id="secondaryFileSystem-title") Secondary file system @@ -33,12 +33,12 @@ form.panel.panel-default(name=form novalidate) -var secondaryFileSystem = model + '.secondaryFileSystem' .settings-row - +checkbox('Enabled', enabled, 'secondaryFileSystemEnabled', 'Secondary file system enabled flag') + +checkbox('Enabled', enabled, '"secondaryFileSystemEnabled"', 'Secondary file system enabled flag') .settings-row - +text-enabled('URI:', secondaryFileSystem + '.uri', 'hadoopURI', enabled, 'false', 'hdfs://[namenodehost]:[port]/[path]', 'URI of file system') + +text-enabled('URI:', secondaryFileSystem + '.uri', '"hadoopURI"', enabled, 'false', 'hdfs://[namenodehost]:[port]/[path]', 'URI of file system') .settings-row - +text-enabled('Config path:', secondaryFileSystem + '.cfgPath', 'cfgPath', enabled, 'false', 'Path to additional config', 'Additional path to Hadoop configuration') + +text-enabled('Config path:', secondaryFileSystem + '.cfgPath', '"cfgPath"', enabled, 'false', 'Path to additional config', 'Additional path to Hadoop configuration') .settings-row - +text-enabled('User name:', secondaryFileSystem + '.userName', 'userName', enabled, 'false', 'Input user name', 'User name') + +text-enabled('User name:', secondaryFileSystem + '.userName', '"userName"', enabled, 'false', 'Input user name', 'User name') .col-sm-6 +preview-xml-java(model, 'igfsSecondFS') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/preview-panel.directive.js b/modules/web-console/frontend/app/modules/states/configuration/preview-panel.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/states/configuration/preview-panel.directive.js rename to modules/web-console/frontend/app/modules/states/configuration/preview-panel.directive.js diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/summary/summary-tabs.directive.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary-tabs.directive.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/states/configuration/summary/summary-tabs.directive.js rename to modules/web-console/frontend/app/modules/states/configuration/summary/summary-tabs.directive.js diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/summary/summary.controller.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js similarity index 90% rename from modules/web-console/src/main/js/app/modules/states/configuration/summary/summary.controller.js rename to modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js index 1b19e3a4369e2..f0cb842c8120c 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/summary/summary.controller.js +++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js @@ -20,41 +20,44 @@ import JSZip from 'jszip'; import saver from 'file-saver'; export default [ - '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteLoading', '$filter', 'ConfigurationSummaryResource', 'JavaTypes', 'IgniteVersion', 'GeneratorDocker', 'GeneratorPom', - function($root, $scope, $http, LegacyUtils, Loading, $filter, Resource, JavaTypes, IgniteVersion, docker, pom) { + '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteLoading', '$filter', 'igniteConfigurationResource', 'JavaTypes', 'IgniteVersion', 'GeneratorDocker', 'GeneratorPom', 'IgniteFormUtils', + function($root, $scope, $http, LegacyUtils, Messages, Loading, $filter, Resource, JavaTypes, IgniteVersion, docker, pom, FormUtils) { const ctrl = this; $scope.ui = { ready: false }; Loading.start('summaryPage'); - Resource.read().then(({clusters}) => { - $scope.clusters = clusters; - $scope.clustersMap = {}; - $scope.clustersView = _.map(clusters, (item) => { - const { _id, name } = item; + Resource.read() + .then(Resource.populate) + .then(({clusters}) => { + $scope.clusters = clusters; + $scope.clustersMap = {}; + $scope.clustersView = _.map(clusters, (item) => { + const {_id, name} = item; - $scope.clustersMap[_id] = item; + $scope.clustersMap[_id] = item; - return { _id, name }; - }); + return {_id, name}; + }); - Loading.finish('summaryPage'); + Loading.finish('summaryPage'); - $scope.ui.ready = true; + $scope.ui.ready = true; - if (!_.isEmpty(clusters)) { - const idx = sessionStorage.summarySelectedId || 0; + if (!_.isEmpty(clusters)) { + const idx = sessionStorage.summarySelectedId || 0; - $scope.selectItem(clusters[idx]); - } - }); + $scope.selectItem(clusters[idx]); + } + }) + .catch(Messages.showError); $scope.contentVisible = (rows, row) => { return !row || !row._id || _.findIndex(rows, (item) => item._id === row._id) >= 0; }; - $scope.widthIsSufficient = LegacyUtils.widthIsSufficient; + $scope.widthIsSufficient = FormUtils.widthIsSufficient; $scope.dialects = {}; $scope.projectStructureOptions = { @@ -229,6 +232,9 @@ export default [ if ($generatorJava.isDemoConfigured(cluster, $root.IgniteDemoMode)) javaFolder.children.push(demoFolder); + if (cluster.discovery.kind === 'Jdbc' && cluster.discovery.Jdbc.dialect) + $scope.dialects[cluster.discovery.Jdbc.dialect] = true; + _.forEach(cluster.caches, (cache) => { if (cache.cacheStoreFactory) { const store = cache.cacheStoreFactory[cache.cacheStoreFactory.kind]; diff --git a/modules/web-console/frontend/app/modules/states/errors.state.js b/modules/web-console/frontend/app/modules/states/errors.state.js new file mode 100644 index 0000000000000..2bdb80a432461 --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/errors.state.js @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import angular from 'angular'; +import templateNotFoundPage from '../../../views/404.jade'; +import templateNotAuthorizedPage from '../../../views/403.jade'; + +angular + .module('ignite-console.states.errors', [ + 'ui.router' + ]) + .config(['$stateProvider', 'AclRouteProvider', function($stateProvider) { + // set up the states + $stateProvider + .state('404', { + url: '/404', + templateUrl: templateNotFoundPage, + metaTags: { + title: 'Page not found' + } + }) + .state('403', { + url: '/403', + templateUrl: templateNotAuthorizedPage, + metaTags: { + title: 'Not authorized' + } + }); + }]); diff --git a/modules/web-console/src/main/js/app/modules/states/logout.state.js b/modules/web-console/frontend/app/modules/states/logout.state.js similarity index 85% rename from modules/web-console/src/main/js/app/modules/states/logout.state.js rename to modules/web-console/frontend/app/modules/states/logout.state.js index 7f24a457a4dc4..42795ea46a41f 100644 --- a/modules/web-console/src/main/js/app/modules/states/logout.state.js +++ b/modules/web-console/frontend/app/modules/states/logout.state.js @@ -21,14 +21,13 @@ angular .module('ignite-console.states.logout', [ 'ui.router' ]) -.config(['$stateProvider', function($stateProvider) { +.config(['$stateProvider', 'AclRouteProvider', function($stateProvider, AclRoute) { // set up the states $stateProvider .state('logout', { url: '/logout', - controller: ['Auth', function(Auth) { - Auth.logout(); - }], + onEnter: AclRoute.checkAccess('logout'), + controller: ['Auth', (Auth) => Auth.logout()], metaTags: { title: 'Logout' } diff --git a/modules/web-console/src/main/js/app/modules/states/password.state.js b/modules/web-console/frontend/app/modules/states/password.state.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/states/password.state.js rename to modules/web-console/frontend/app/modules/states/password.state.js diff --git a/modules/web-console/src/main/js/app/modules/states/profile.state.js b/modules/web-console/frontend/app/modules/states/profile.state.js similarity index 89% rename from modules/web-console/src/main/js/app/modules/states/profile.state.js rename to modules/web-console/frontend/app/modules/states/profile.state.js index 8b6cdfe1cfd77..9c31340a9569b 100644 --- a/modules/web-console/src/main/js/app/modules/states/profile.state.js +++ b/modules/web-console/frontend/app/modules/states/profile.state.js @@ -21,12 +21,13 @@ angular .module('ignite-console.states.profile', [ 'ui.router' ]) -.config(['$stateProvider', function($stateProvider) { +.config(['$stateProvider', 'AclRouteProvider', function($stateProvider, AclRoute) { // set up the states $stateProvider .state('settings.profile', { url: '/profile', templateUrl: '/settings/profile.html', + onEnter: AclRoute.checkAccess('profile'), metaTags: { title: 'User profile' } diff --git a/modules/web-console/src/main/js/app/modules/states/sql.state.js b/modules/web-console/frontend/app/modules/states/signin.state.js similarity index 63% rename from modules/web-console/src/main/js/app/modules/states/sql.state.js rename to modules/web-console/frontend/app/modules/states/signin.state.js index 1aa4f47e284cd..14ebc1baa6aab 100644 --- a/modules/web-console/src/main/js/app/modules/states/sql.state.js +++ b/modules/web-console/frontend/app/modules/states/signin.state.js @@ -16,31 +16,28 @@ */ import angular from 'angular'; +import templateUrl from 'views/signin.jade'; angular -.module('ignite-console.states.sql', [ - 'ui.router' +.module('ignite-console.states.login', [ + 'ui.router', + // services + 'ignite-console.user' ]) -.config(['$stateProvider', function($stateProvider) { +.config(['$stateProvider', 'AclRouteProvider', function($stateProvider) { // set up the states $stateProvider - .state('base.sql', { - url: '/sql', - abstract: true, - template: '' - }) - .state('base.sql.notebook', { - url: '/notebook/{noteId}', - templateUrl: '/sql/sql.html', + .state('signin', { + url: '/', + templateUrl, + resolve: { + user: ['$state', 'User', ($state, User) => { + return User.read() + .then(() => $state.go('base.configuration.clusters')) + .catch(() => {}); + }] + }, metaTags: { - title: 'Query notebook' - } - }) - .state('base.sql.demo', { - url: '/demo', - templateUrl: '/sql/sql.html', - metaTags: { - title: 'SQL demo' } }); }]); diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/summary/summary.resource.js b/modules/web-console/frontend/app/modules/user/AclRoute.provider.js similarity index 52% rename from modules/web-console/src/main/js/app/modules/states/configuration/summary/summary.resource.js rename to modules/web-console/frontend/app/modules/user/AclRoute.provider.js index 0ef53ac59cf7c..40abea5ad872e 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/summary/summary.resource.js +++ b/modules/web-console/frontend/app/modules/user/AclRoute.provider.js @@ -15,26 +15,33 @@ * limitations under the License. */ -export default ['ConfigurationSummaryResource', ['$q', '$http', ($q, $http) => { - const api = '/api/v1/configuration/clusters/list'; +export default [() => { + class AclRoute { + static checkAccess = (permissions, failState) => { + failState = failState || '403'; - return { - read() { - return $http - .post(api) - .then(({data}) => data) - .then(({clusters, caches, igfss}) => { - if (!clusters || !clusters.length) - return {}; + return ['$state', 'AclService', 'User', ($state, AclService, User) => { + User.read() + .then(() => { + if (AclService.can(permissions)) + return; + + return $state.go(failState); + }) + .catch(() => { + User.clean(); - _.forEach(clusters, (cluster) => { - cluster.igfss = _.filter(igfss, ({_id}) => _.includes(cluster.igfss, _id)); - cluster.caches = _.filter(caches, ({_id}) => _.includes(cluster.caches, _id)); + if ($state.current.name !== 'signin') + $state.go('signin'); }); + }]; + } + } - return {clusters}; - }) - .catch((err) => $q.reject(err)); + return { + checkAccess: AclRoute.checkAccess, + $get: () => { + return AclRoute; } }; -}]]; +}]; diff --git a/modules/web-console/frontend/app/modules/user/Auth.service.js b/modules/web-console/frontend/app/modules/user/Auth.service.js new file mode 100644 index 0000000000000..43e2f9216071e --- /dev/null +++ b/modules/web-console/frontend/app/modules/user/Auth.service.js @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +export default ['Auth', ['$http', '$rootScope', '$state', '$window', 'IgniteErrorPopover', 'IgniteMessages', 'gettingStarted', 'User', 'IgniteAgentMonitor', + ($http, $root, $state, $window, ErrorPopover, Messages, gettingStarted, User, agentMonitor) => { + return { + forgotPassword(userInfo) { + $http.post('/api/v1/password/forgot', userInfo) + .success(() => $state.go('password.send')) + .error((err) => ErrorPopover.show('forgot_email', Messages.errorMessage(null, err))); + }, + auth(action, userInfo) { + $http.post('/api/v1/' + action, userInfo) + .catch(({data}) => Promise.reject(data)) + .then(() => { + if (action === 'password/forgot') + return; + + User.read() + .then((user) => { + $root.$broadcast('user', user); + + $state.go('base.configuration.clusters'); + + agentMonitor.init(); + + $root.gettingStarted.tryShow(); + }); + }) + .catch((err) => ErrorPopover.show(action + '_email', Messages.errorMessage(null, err))); + }, + logout() { + $http.post('/api/v1/logout') + .success(() => { + User.clean(); + + $window.open($state.href('signin'), '_self'); + }) + .error(Messages.showError); + } + }; + }]]; diff --git a/modules/web-console/src/main/js/app/modules/user/User.service.js b/modules/web-console/frontend/app/modules/user/User.service.js similarity index 58% rename from modules/web-console/src/main/js/app/modules/user/User.service.js rename to modules/web-console/frontend/app/modules/user/User.service.js index d1f8083fffefd..8b9a1e71245c1 100644 --- a/modules/web-console/src/main/js/app/modules/user/User.service.js +++ b/modules/web-console/frontend/app/modules/user/User.service.js @@ -16,47 +16,33 @@ */ export default ['User', ['$q', '$injector', '$rootScope', '$state', '$http', function($q, $injector, $root, $state, $http) { - let _user; - - try { - _user = JSON.parse(localStorage.user); - - if (_user) - $root.user = _user; - } - catch (ignore) { - // No-op. - } + let user; return { - read() { - return $http.post('/api/v1/user').then(({data}) => { - if (_.isEmpty(data)) { - const Auth = $injector.get('Auth'); - - Auth.authorized = false; + load() { + return user = $http.post('/api/v1/user') + .then(({data}) => { + $root.user = data; - this.clean(); + $root.$broadcast('user', $root.user); - if ($state.current.name !== 'signin') - $state.go('signin'); - } + return $root.user; + }) + .catch(({data}) => { + user = null; - try { - localStorage.user = JSON.stringify(data); - } - catch (ignore) { - // No-op. - } + return $q.reject(data); + }); + }, + read() { + if (user) + return user; - return _user = $root.user = data; - }); + return this.load(); }, clean() { delete $root.user; - delete localStorage.user; - delete $root.IgniteDemoMode; sessionStorage.removeItem('IgniteDemoMode'); diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/memory.directive.js b/modules/web-console/frontend/app/modules/user/permissions.js similarity index 77% rename from modules/web-console/src/main/js/app/modules/states/configuration/caches/memory.directive.js rename to modules/web-console/frontend/app/modules/user/permissions.js index c160b184cbf10..e13509ceaf719 100644 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/memory.directive.js +++ b/modules/web-console/frontend/app/modules/user/permissions.js @@ -15,13 +15,14 @@ * limitations under the License. */ -import templateUrl from './memory.jade'; +const guest = ['login']; +const becomed = ['profile', 'configuration', 'query']; +const user = becomed.concat(['logout']); +const admin = user.concat(['admin_page']); -export default ['igniteConfigurationCachesMemory', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; +export default { + guest, + user, + admin, + becomed +}; diff --git a/modules/web-console/frontend/app/modules/user/user.module.js b/modules/web-console/frontend/app/modules/user/user.module.js new file mode 100644 index 0000000000000..11798d0746d62 --- /dev/null +++ b/modules/web-console/frontend/app/modules/user/user.module.js @@ -0,0 +1,73 @@ +/* + * 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. + */ + +import angular from 'angular'; +import aclData from './permissions'; + +import Auth from './Auth.service'; +import User from './User.service'; +import AclRouteProvider from './AclRoute.provider'; + +angular +.module('ignite-console.user', [ + 'mm.acl', + 'ignite-console.config' +]) +.factory('sessionRecoverer', ['$injector', '$q', ($injector, $q) => { + return { + responseError: (response) => { + // Session has expired + if (response.status === 401) { + $injector.get('User').clean(); + + const $state = $injector.get('$state'); + + if ($state.current.name !== 'signin') + $state.go('signin'); + } + + return $q.reject(response); + } + }; +}]) +.config(['$httpProvider', ($httpProvider) => { + $httpProvider.interceptors.push('sessionRecoverer'); +}]) +.service(...Auth) +.service(...User) +.provider('AclRoute', AclRouteProvider) +.run(['$rootScope', 'AclService', ($root, AclService) => { + AclService.setAbilities(aclData); + AclService.attachRole('guest'); + + $root.$on('user', (event, user) => { + if (!user) + return; + + AclService.flushRoles(); + + let role = 'user'; + + if (user.admin) + role = 'admin'; + + if (user.becomeUsed) + role = 'becomed'; + + AclService.attachRole(role); + }); +}]); diff --git a/modules/web-console/src/main/js/app/modules/Version/Version.provider.js b/modules/web-console/frontend/app/modules/version/Version.provider.js similarity index 100% rename from modules/web-console/src/main/js/app/modules/Version/Version.provider.js rename to modules/web-console/frontend/app/modules/version/Version.provider.js diff --git a/modules/web-console/src/main/js/app/services/ChartColors.service.js b/modules/web-console/frontend/app/services/ChartColors.service.js similarity index 100% rename from modules/web-console/src/main/js/app/services/ChartColors.service.js rename to modules/web-console/frontend/app/services/ChartColors.service.js diff --git a/modules/web-console/src/main/js/app/services/Clone.service.js b/modules/web-console/frontend/app/services/Clone.service.js similarity index 100% rename from modules/web-console/src/main/js/app/services/Clone.service.js rename to modules/web-console/frontend/app/services/Clone.service.js diff --git a/modules/web-console/src/main/js/app/services/Confirm.service.js b/modules/web-console/frontend/app/services/Confirm.service.js similarity index 83% rename from modules/web-console/src/main/js/app/services/Confirm.service.js rename to modules/web-console/frontend/app/services/Confirm.service.js index c4e25a89f7f99..8208ea2a15454 100644 --- a/modules/web-console/src/main/js/app/services/Confirm.service.js +++ b/modules/web-console/frontend/app/services/Confirm.service.js @@ -19,30 +19,30 @@ export default ['IgniteConfirm', ['$rootScope', '$q', '$modal', '$animate', ($root, $q, $modal, $animate) => { const scope = $root.$new(); - const modal = $modal({templateUrl: '/templates/confirm.html', scope, placement: 'center', show: false}); + const modal = $modal({templateUrl: '/templates/confirm.html', scope, placement: 'center', show: false, backdrop: true}); - let deferred; - - const _hide = (animate) => { - $animate.enabled(modal.$element, animate); + const _hide = () => { + $animate.enabled(modal.$element, false); modal.hide(); }; + let deferred; + scope.confirmYes = () => { - _hide(scope.animate); + _hide(); deferred.resolve(true); }; scope.confirmNo = () => { - _hide(scope.animate); + _hide(); deferred.resolve(false); }; scope.confirmCancel = () => { - _hide(true); + _hide(); deferred.reject('cancelled'); }; @@ -51,11 +51,9 @@ export default ['IgniteConfirm', ['$rootScope', '$q', '$modal', '$animate', ($ro * * @param {String } content * @param {Boolean} [yesNo] - * @param {Boolean} [animate] * @returns {Promise} */ - modal.confirm = (content, yesNo, animate) => { - scope.animate = !!animate; + modal.confirm = (content, yesNo) => { scope.content = content || 'Confirm?'; scope.yesNo = !!yesNo; diff --git a/modules/web-console/src/main/js/app/services/ConfirmBatch.service.js b/modules/web-console/frontend/app/services/ConfirmBatch.service.js similarity index 100% rename from modules/web-console/src/main/js/app/services/ConfirmBatch.service.js rename to modules/web-console/frontend/app/services/ConfirmBatch.service.js diff --git a/modules/web-console/src/main/js/app/services/CopyToClipboard.service.js b/modules/web-console/frontend/app/services/CopyToClipboard.service.js similarity index 100% rename from modules/web-console/src/main/js/app/services/CopyToClipboard.service.js rename to modules/web-console/frontend/app/services/CopyToClipboard.service.js diff --git a/modules/web-console/src/main/js/app/services/Countries.service.js b/modules/web-console/frontend/app/services/Countries.service.js similarity index 100% rename from modules/web-console/src/main/js/app/services/Countries.service.js rename to modules/web-console/frontend/app/services/Countries.service.js diff --git a/modules/web-console/frontend/app/services/ErrorPopover.service.js b/modules/web-console/frontend/app/services/ErrorPopover.service.js new file mode 100644 index 0000000000000..85e4fdab3d820 --- /dev/null +++ b/modules/web-console/frontend/app/services/ErrorPopover.service.js @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +export default class ErrorPopover { + static $inject = ['$popover', '$anchorScroll', '$location', '$timeout', 'IgniteFormUtils']; + + /** + * @param $popover + * @param $anchorScroll + * @param $location + * @param $timeout + * @param FormUtils + */ + constructor($popover, $anchorScroll, $location, $timeout, FormUtils) { + this.$popover = $popover; + this.$anchorScroll = $anchorScroll; + this.$location = $location; + this.$timeout = $timeout; + this.FormUtils = FormUtils; + + this.$anchorScroll.yOffset = 55; + + this._popover = null; + } + + /** + * Check that element is document area. + * + * @param el Element to check. + * @returns {boolean} True when element in document area. + */ + static _isElementInViewport(el) { + const rect = el.getBoundingClientRect(); + + return ( + rect.top >= 0 && + rect.left >= 0 && + rect.bottom <= (window.innerHeight || document.documentElement.clientHeight) && + rect.right <= (window.innerWidth || document.documentElement.clientWidth) + ); + } + + /** + * Internal show popover message with detected properties. + * + * @param id Id element to show popover message. + * @param message Message to show. + * @param showTime Time before popover will be hidden. + */ + _show(id, message, showTime = 5000) { + const body = $('body'); + + let el = body.find('#' + id); + + if (!el || el.length === 0) + el = body.find('[name="' + id + '"]'); + + if (el && el.length > 0) { + if (!ErrorPopover._isElementInViewport(el[0])) { + this.$location.hash(el[0].id); + + this.$anchorScroll(); + } + + const newPopover = this.$popover(el, {content: message}); + + this._popover = newPopover; + + this.$timeout(() => newPopover.$promise.then(() => { + newPopover.show(); + + // Workaround to fix popover location when content is longer than content template. + // https://github.com/mgcrea/angular-strap/issues/1497 + this.$timeout(newPopover.$applyPlacement); + }), 400); + this.$timeout(() => newPopover.hide(), showTime); + } + } + + /** + * Show popover message. + * + * @param {String} id ID of element to show popover. + * @param {String} message Message to show. + * @param {Object} [ui] Form UI object. When specified extend section with that name. + * @param {String} [panelId] ID of element owner panel. When specified focus element with that ID. + * @param {Number} [showTime] Time before popover will be hidden. 5 sec when not specified. + * @returns {boolean} False always. + */ + show(id, message, ui, panelId, showTime) { + if (this._popover) + this._popover.hide(); + + if (ui) { + this.FormUtils.ensureActivePanel(ui, panelId, id); + + this.$timeout(() => this._show(id, message, showTime), ui.isPanelLoaded(panelId) ? 200 : 500); + } + else + this._show(id, message); + + return false; + } + + /** + * Hide popover message. + */ + hide() { + if (this._popover) + this._popover.hide(); + } +} diff --git a/modules/web-console/src/main/js/app/services/Focus.service.js b/modules/web-console/frontend/app/services/Focus.service.js similarity index 100% rename from modules/web-console/src/main/js/app/services/Focus.service.js rename to modules/web-console/frontend/app/services/Focus.service.js diff --git a/modules/web-console/frontend/app/services/FormUtils.service.js b/modules/web-console/frontend/app/services/FormUtils.service.js new file mode 100644 index 0000000000000..5e7943a015faf --- /dev/null +++ b/modules/web-console/frontend/app/services/FormUtils.service.js @@ -0,0 +1,435 @@ +/* + * 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. + */ + +export default ['IgniteFormUtils', ['$window', 'IgniteFocus', ($window, Focus) => { + function ensureActivePanel(ui, pnl, focusId) { + if (ui) { + const collapses = $('div.panel-collapse'); + + ui.loadPanel(pnl); + + const idx = _.findIndex(collapses, function(collapse) { + return collapse.id === pnl; + }); + + if (idx >= 0) { + const activePanels = ui.activePanels; + + if (!_.includes(ui.topPanels, idx)) { + ui.expanded = true; + + const customExpanded = ui[pnl]; + + if (customExpanded) + ui[customExpanded] = true; + } + + if (!activePanels || activePanels.length < 1) + ui.activePanels = [idx]; + else if (!_.includes(activePanels, idx)) { + const newActivePanels = angular.copy(activePanels); + + newActivePanels.push(idx); + + ui.activePanels = newActivePanels; + } + } + + if (!_.isNil(focusId)) + Focus.move(focusId); + } + } + + let context = null; + + /** + * Calculate width of specified text in body's font. + * + * @param text Text to calculate width. + * @returns {Number} Width of text in pixels. + */ + function measureText(text) { + if (!context) { + const canvas = document.createElement('canvas'); + + context = canvas.getContext('2d'); + + const style = window.getComputedStyle(document.getElementsByTagName('body')[0]); + + context.font = style.fontSize + ' ' + style.fontFamily; + } + + return context.measureText(text).width; + } + + /** + * Compact java full class name by max number of characters. + * + * @param names Array of class names to compact. + * @param nameLength Max available width in characters for simple name. + * @returns {*} Array of compacted class names. + */ + function compactByMaxCharts(names, nameLength) { + for (let nameIx = 0; nameIx < names.length; nameIx++) { + const s = names[nameIx]; + + if (s.length > nameLength) { + let totalLength = s.length; + + const packages = s.split('.'); + + const packageCnt = packages.length - 1; + + for (let i = 0; i < packageCnt && totalLength > nameLength; i++) { + if (packages[i].length > 0) { + totalLength -= packages[i].length - 1; + + packages[i] = packages[i][0]; + } + } + + if (totalLength > nameLength) { + const className = packages[packageCnt]; + + const classNameLen = className.length; + + let remains = Math.min(nameLength - totalLength + classNameLen, classNameLen); + + if (remains < 3) + remains = Math.min(3, classNameLen); + + packages[packageCnt] = className.substring(0, remains) + '...'; + } + + let result = packages[0]; + + for (let i = 1; i < packages.length; i++) + result += '.' + packages[i]; + + names[nameIx] = result; + } + } + + return names; + } + + /** + * Compact java full class name by max number of pixels. + * + * @param names Array of class names to compact. + * @param nameLength Max available width in characters for simple name. Used for calculation optimization. + * @param nameWidth Maximum available width in pixels for simple name. + * @returns {*} Array of compacted class names. + */ + function compactByMaxPixels(names, nameLength, nameWidth) { + if (nameWidth <= 0) + return names; + + const fitted = []; + + const widthByName = []; + + const len = names.length; + + let divideTo = len; + + for (let nameIx = 0; nameIx < len; nameIx++) { + fitted[nameIx] = false; + + widthByName[nameIx] = nameWidth; + } + + // Try to distribute space from short class names to long class names. + let remains = 0; + + do { + for (let nameIx = 0; nameIx < len; nameIx++) { + if (!fitted[nameIx]) { + const curNameWidth = measureText(names[nameIx]); + + if (widthByName[nameIx] > curNameWidth) { + fitted[nameIx] = true; + + remains += widthByName[nameIx] - curNameWidth; + + divideTo -= 1; + + widthByName[nameIx] = curNameWidth; + } + } + } + + const remainsByName = remains / divideTo; + + for (let nameIx = 0; nameIx < len; nameIx++) { + if (!fitted[nameIx]) + widthByName[nameIx] += remainsByName; + } + } + while (remains > 0); + + // Compact class names to available for each space. + for (let nameIx = 0; nameIx < len; nameIx++) { + const s = names[nameIx]; + + if (s.length > (nameLength / 2 | 0)) { + let totalWidth = measureText(s); + + if (totalWidth > widthByName[nameIx]) { + const packages = s.split('.'); + + const packageCnt = packages.length - 1; + + for (let i = 0; i < packageCnt && totalWidth > widthByName[nameIx]; i++) { + if (packages[i].length > 1) { + totalWidth -= measureText(packages[i].substring(1, packages[i].length)); + + packages[i] = packages[i][0]; + } + } + + let shortPackage = ''; + + for (let i = 0; i < packageCnt; i++) + shortPackage += packages[i] + '.'; + + const className = packages[packageCnt]; + + const classLen = className.length; + + let minLen = Math.min(classLen, 3); + + totalWidth = measureText(shortPackage + className); + + // Compact class name if shorten package path is very long. + if (totalWidth > widthByName[nameIx]) { + let maxLen = classLen; + let middleLen = (minLen + (maxLen - minLen) / 2 ) | 0; + + while (middleLen !== minLen && middleLen !== maxLen) { + const middleLenPx = measureText(shortPackage + className.substr(0, middleLen) + '...'); + + if (middleLenPx > widthByName[nameIx]) + maxLen = middleLen; + else + minLen = middleLen; + + middleLen = (minLen + (maxLen - minLen) / 2 ) | 0; + } + + names[nameIx] = shortPackage + className.substring(0, middleLen) + '...'; + } + else + names[nameIx] = shortPackage + className; + } + } + } + + return names; + } + + /** + * Compact any string by max number of pixels. + * + * @param label String to compact. + * @param nameWidth Maximum available width in pixels for simple name. + * @returns {*} Compacted string. + */ + function compactLabelByPixels(label, nameWidth) { + if (nameWidth <= 0) + return label; + + const totalWidth = measureText(label); + + if (totalWidth > nameWidth) { + let maxLen = label.length; + let minLen = Math.min(maxLen, 3); + let middleLen = (minLen + (maxLen - minLen) / 2 ) | 0; + + while (middleLen !== minLen && middleLen !== maxLen) { + const middleLenPx = measureText(label.substr(0, middleLen) + '...'); + + if (middleLenPx > nameWidth) + maxLen = middleLen; + else + minLen = middleLen; + + middleLen = (minLen + (maxLen - minLen) / 2 ) | 0; + } + + return label.substring(0, middleLen) + '...'; + } + + return label; + } + + /** + * Calculate available width for text in link to edit element. + * + * @param index Showed index of element for calculation of maximum width in pixels. + * @param id Id of contains link table. + * @returns {*[]} First element is length of class for single value, second element is length for pair vlaue. + */ + function availableWidth(index, id) { + const idElem = $('#' + id); + + let width = 0; + + switch (idElem.prop('tagName')) { + // Detection of available width in presentation table row. + case 'TABLE': + const cont = $(idElem.find('tr')[index - 1]).find('td')[0]; + + width = cont.clientWidth; + + if (width > 0) { + const children = $(cont).children(':not("a")'); + + _.forEach(children, function(child) { + if ('offsetWidth' in child) + width -= $(child).outerWidth(true); + }); + } + + break; + + // Detection of available width in dropdown row. + case 'A': + width = idElem.width(); + + $(idElem).children(':not("span")').each(function(ix, child) { + if ('offsetWidth' in child) + width -= child.offsetWidth; + }); + + break; + + default: + } + + return width | 0; + } + + return { + /** + * Cut class name by width in pixel or width in symbol count. + * + * @param id Id of parent table. + * @param index Row number in table. + * @param maxLength Maximum length in symbols for all names. + * @param names Array of class names to compact. + * @param divider String to visualy divide items. + * @returns {*} Array of compacted class names. + */ + compactJavaName(id, index, maxLength, names, divider) { + divider = ' ' + divider + ' '; + + const prefix = index + ') '; + + const nameCnt = names.length; + + const nameLength = ((maxLength - 3 * (nameCnt - 1)) / nameCnt) | 0; + + try { + const nameWidth = (availableWidth(index, id) - measureText(prefix) - (nameCnt - 1) * measureText(divider)) / + nameCnt | 0; + + // HTML5 calculation of showed message width. + names = compactByMaxPixels(names, nameLength, nameWidth); + } + catch (err) { + names = compactByMaxCharts(names, nameLength); + } + + let result = prefix + names[0]; + + for (let nameIx = 1; nameIx < names.length; nameIx++) + result += divider + names[nameIx]; + + return result; + }, + /** + * Compact text by width in pixels or symbols count. + * + * @param id Id of parent table. + * @param index Row number in table. + * @param maxLength Maximum length in symbols for all names. + * @param label Text to compact. + * @returns Compacted label text. + */ + compactTableLabel(id, index, maxLength, label) { + label = index + ') ' + label; + + try { + const nameWidth = availableWidth(index, id) | 0; + + // HTML5 calculation of showed message width. + label = compactLabelByPixels(label, nameWidth); + } + catch (err) { + const nameLength = maxLength - 3 | 0; + + label = label.length > maxLength ? label.substr(0, nameLength) + '...' : label; + } + + return label; + }, + widthIsSufficient(id, index, text) { + try { + const available = availableWidth(index, id); + + const required = measureText(text); + + return !available || available >= Math.floor(required); + } + catch (err) { + return true; + } + }, + ensureActivePanel(panels, id, focusId) { + ensureActivePanel(panels, id, focusId); + }, + confirmUnsavedChanges(dirty, selectFunc) { + if (dirty) { + if ($window.confirm('You have unsaved changes.\n\nAre you sure you want to discard them?')) + selectFunc(); + } + else + selectFunc(); + }, + saveBtnTipText(dirty, objectName) { + if (dirty) + return 'Save ' + objectName; + + return 'Nothing to save'; + }, + formUI() { + return { + ready: false, + expanded: false, + loadedPanels: [], + loadPanel(pnl) { + if (!_.includes(this.loadedPanels, pnl)) + this.loadedPanels.push(pnl); + }, + isPanelLoaded(pnl) { + return _.includes(this.loadedPanels, pnl); + } + }; + } + }; +}]]; diff --git a/modules/web-console/src/main/js/app/services/InetAddress.service.js b/modules/web-console/frontend/app/services/InetAddress.service.js similarity index 100% rename from modules/web-console/src/main/js/app/services/InetAddress.service.js rename to modules/web-console/frontend/app/services/InetAddress.service.js diff --git a/modules/web-console/src/main/js/app/services/JavaTypes.service.js b/modules/web-console/frontend/app/services/JavaTypes.service.js similarity index 85% rename from modules/web-console/src/main/js/app/services/JavaTypes.service.js rename to modules/web-console/frontend/app/services/JavaTypes.service.js index d34d669e9f428..e8d49030f8355 100644 --- a/modules/web-console/src/main/js/app/services/JavaTypes.service.js +++ b/modules/web-console/frontend/app/services/JavaTypes.service.js @@ -16,12 +16,12 @@ */ // Java built-in class names. -import JAVA_CLASSES from 'app/data/java-classes.json'; +import JAVA_CLASSES from '../data/java-classes.json'; // Java build-in primitive. -import JAVA_PRIMITIVES from 'app/data/java-primitives.json'; +import JAVA_PRIMITIVES from '../data/java-primitives.json'; -import JAVA_KEYWORDS from 'app/data/java-keywords.json'; +import JAVA_KEYWORDS from '../data/java-keywords.json'; export default ['JavaTypes', function() { return { @@ -59,6 +59,15 @@ export default ['JavaTypes', function() { return value === '' || regexp.test(value); }, + /** + * @param {String} value text to check. + * @returns boolean 'true' if given text is valid Java UUID value. + */ + validUUID(value) { + const regexp = /^[0-9a-f]{8}-[0-9a-f]{4}-[1-5][0-9a-f]{3}-[89ab][0-9a-f]{3}-[0-9a-f]{12}$/igm; + + return value === '' || regexp.test(value); + }, /** * @param {String} value text to check. * @returns boolean 'true' if given text is a Java type with package. diff --git a/modules/web-console/frontend/app/services/LegacyTable.service.js b/modules/web-console/frontend/app/services/LegacyTable.service.js new file mode 100644 index 0000000000000..5d9ec9d436215 --- /dev/null +++ b/modules/web-console/frontend/app/services/LegacyTable.service.js @@ -0,0 +1,209 @@ +/* + * 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. + */ + +// TODO: Refactor this service for legacy tables with more than one input field. +export default ['IgniteLegacyTable', + ['IgniteLegacyUtils', 'IgniteFocus', 'IgniteErrorPopover', (LegacyUtils, Focus, ErrorPopover) => { + function _model(item, field) { + return LegacyUtils.getModel(item, field); + } + + const table = {name: 'none', editIndex: -1}; + + function _tableReset() { + delete table.field; + table.name = 'none'; + table.editIndex = -1; + + ErrorPopover.hide(); + } + + function _tableSaveAndReset() { + const field = table.field; + + const save = LegacyUtils.isDefined(field) && LegacyUtils.isDefined(field.save); + + if (!save || !LegacyUtils.isDefined(field) || field.save(field, table.editIndex, true)) { + _tableReset(); + + return true; + } + + return false; + } + + function _tableState(field, editIndex, specName) { + table.field = field; + table.name = specName || field.model; + table.editIndex = editIndex; + } + + function _tableUI(field) { + const ui = field.ui; + + return ui ? ui : field.type; + } + + function _tableFocus(focusId, index) { + Focus.move((index < 0 ? 'new' : 'cur') + focusId + (index >= 0 ? index : '')); + } + + function _tablePairValue(filed, index) { + return index < 0 ? {key: filed.newKey, value: filed.newValue} : { + key: filed.curKey, + value: filed.curValue + }; + } + + function _tableStartEdit(item, tbl, index, save) { + _tableState(tbl, index); + + const val = _.get(_model(item, tbl), tbl.model)[index]; + + const ui = _tableUI(tbl); + + tbl.save = save; + + if (ui === 'table-pair') { + tbl.curKey = val[tbl.keyName]; + tbl.curValue = val[tbl.valueName]; + + _tableFocus('Key' + tbl.focusId, index); + } + else if (ui === 'table-db-fields') { + tbl.curDatabaseFieldName = val.databaseFieldName; + tbl.curDatabaseFieldType = val.databaseFieldType; + tbl.curJavaFieldName = val.javaFieldName; + tbl.curJavaFieldType = val.javaFieldType; + + _tableFocus('DatabaseFieldName' + tbl.focusId, index); + } + else if (ui === 'table-indexes') { + tbl.curIndexName = val.name; + tbl.curIndexType = val.indexType; + tbl.curIndexFields = val.fields; + + _tableFocus(tbl.focusId, index); + } + } + + function _tableNewItem(tbl) { + _tableState(tbl, -1); + + const ui = _tableUI(tbl); + + if (ui === 'table-pair') { + tbl.newKey = null; + tbl.newValue = null; + + _tableFocus('Key' + tbl.focusId, -1); + } + else if (ui === 'table-db-fields') { + tbl.newDatabaseFieldName = null; + tbl.newDatabaseFieldType = null; + tbl.newJavaFieldName = null; + tbl.newJavaFieldType = null; + + _tableFocus('DatabaseFieldName' + tbl.focusId, -1); + } + else if (ui === 'table-indexes') { + tbl.newIndexName = null; + tbl.newIndexType = 'SORTED'; + tbl.newIndexFields = null; + + _tableFocus(tbl.focusId, -1); + } + } + + return { + tableState: _tableState, + tableReset: _tableReset, + tableSaveAndReset: _tableSaveAndReset, + tableNewItem: _tableNewItem, + tableNewItemActive(tbl) { + return table.name === tbl.model && table.editIndex < 0; + }, + tableEditing(tbl, index) { + return table.name === tbl.model && table.editIndex === index; + }, + tableEditedRowIndex() { + return table.editIndex; + }, + tableField() { + return table.field; + }, + tableStartEdit: _tableStartEdit, + tableRemove(item, field, index) { + _tableReset(); + + _.get(_model(item, field), field.model).splice(index, 1); + }, + tablePairValue: _tablePairValue, + tablePairSave(pairValid, item, field, index, stopEdit) { + const valid = pairValid(item, field, index, stopEdit); + + if (valid) { + const pairValue = _tablePairValue(field, index); + + let pairModel = {}; + + const container = _.get(item, field.model); + + if (index < 0) { + pairModel[field.keyName] = pairValue.key; + pairModel[field.valueName] = pairValue.value; + + if (container) + container.push(pairModel); + else + _.set(item, field.model, [pairModel]); + + if (!stopEdit) + _tableNewItem(field); + } + else { + pairModel = container[index]; + + pairModel[field.keyName] = pairValue.key; + pairModel[field.valueName] = pairValue.value; + + if (!stopEdit) { + if (index < container.length - 1) + _tableStartEdit(item, field, index + 1); + else + _tableNewItem(field); + } + } + } + + return valid; + }, + tablePairSaveVisible(field, index) { + const pairValue = _tablePairValue(field, index); + + return !LegacyUtils.isEmptyString(pairValue.key) && !LegacyUtils.isEmptyString(pairValue.value); + }, + tableFocusInvalidField(index, id) { + _tableFocus(id, index); + + return false; + }, + tableFieldId(index, id) { + return (index < 0 ? 'new' : 'cur') + id + (index >= 0 ? index : ''); + } + }; + }]]; diff --git a/modules/web-console/frontend/app/services/LegacyUtils.service.js b/modules/web-console/frontend/app/services/LegacyUtils.service.js new file mode 100644 index 0000000000000..ed555a1ff50f0 --- /dev/null +++ b/modules/web-console/frontend/app/services/LegacyUtils.service.js @@ -0,0 +1,572 @@ +/* + * 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. + */ + +// TODO: Refactor this service for legacy tables with more than one input field. +export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { + function isDefined(v) { + return !_.isNil(v); + } + + function isEmptyString(s) { + if (isDefined(s)) + return s.trim().length === 0; + + return true; + } + + const javaBuiltInClasses = [ + 'BigDecimal', + 'Boolean', + 'Byte', + 'Date', + 'Double', + 'Float', + 'Integer', + 'Long', + 'Object', + 'Short', + 'String', + 'Time', + 'Timestamp', + 'UUID' + ]; + + const javaBuiltInTypes = [ + 'BigDecimal', + 'boolean', + 'Boolean', + 'byte', + 'Byte', + 'Date', + 'double', + 'Double', + 'float', + 'Float', + 'int', + 'Integer', + 'long', + 'Long', + 'Object', + 'short', + 'Short', + 'String', + 'Time', + 'Timestamp', + 'UUID' + ]; + + const javaBuiltInFullNameClasses = [ + 'java.math.BigDecimal', + 'java.lang.Boolean', + 'java.lang.Byte', + 'java.sql.Date', + 'java.lang.Double', + 'java.lang.Float', + 'java.lang.Integer', + 'java.lang.Long', + 'java.lang.Object', + 'java.lang.Short', + 'java.lang.String', + 'java.sql.Time', + 'java.sql.Timestamp', + 'java.util.UUID' + ]; + + /** + * @param clsName Class name to check. + * @returns {Boolean} 'true' if given class name is a java build-in type. + */ + function isJavaBuiltInClass(clsName) { + if (isEmptyString(clsName)) + return false; + + return _.includes(javaBuiltInClasses, clsName) || _.includes(javaBuiltInFullNameClasses, clsName); + } + + const SUPPORTED_JDBC_TYPES = [ + 'BIGINT', + 'BIT', + 'BOOLEAN', + 'BLOB', + 'CHAR', + 'CLOB', + 'DATE', + 'DECIMAL', + 'DOUBLE', + 'FLOAT', + 'INTEGER', + 'LONGNVARCHAR', + 'LONGVARCHAR', + 'NCHAR', + 'NUMERIC', + 'NVARCHAR', + 'REAL', + 'SMALLINT', + 'TIME', + 'TIMESTAMP', + 'TINYINT', + 'VARCHAR' + ]; + + const ALL_JDBC_TYPES = [ + {dbName: 'BIT', dbType: -7, javaType: 'Boolean', primitiveType: 'boolean'}, + {dbName: 'TINYINT', dbType: -6, javaType: 'Byte', primitiveType: 'byte'}, + {dbName: 'SMALLINT', dbType: 5, javaType: 'Short', primitiveType: 'short'}, + {dbName: 'INTEGER', dbType: 4, javaType: 'Integer', primitiveType: 'int'}, + {dbName: 'BIGINT', dbType: -5, javaType: 'Long', primitiveType: 'long'}, + {dbName: 'FLOAT', dbType: 6, javaType: 'Float', primitiveType: 'float'}, + {dbName: 'REAL', dbType: 7, javaType: 'Double', primitiveType: 'double'}, + {dbName: 'DOUBLE', dbType: 8, javaType: 'Double', primitiveType: 'double'}, + {dbName: 'NUMERIC', dbType: 2, javaType: 'BigDecimal'}, + {dbName: 'DECIMAL', dbType: 3, javaType: 'BigDecimal'}, + {dbName: 'CHAR', dbType: 1, javaType: 'String'}, + {dbName: 'VARCHAR', dbType: 12, javaType: 'String'}, + {dbName: 'LONGVARCHAR', dbType: -1, javaType: 'String'}, + {dbName: 'DATE', dbType: 91, javaType: 'Date'}, + {dbName: 'TIME', dbType: 92, javaType: 'Time'}, + {dbName: 'TIMESTAMP', dbType: 93, javaType: 'Timestamp'}, + {dbName: 'BINARY', dbType: -2, javaType: 'Object'}, + {dbName: 'VARBINARY', dbType: -3, javaType: 'Object'}, + {dbName: 'LONGVARBINARY', dbType: -4, javaType: 'Object'}, + {dbName: 'NULL', dbType: 0, javaType: 'Object'}, + {dbName: 'OTHER', dbType: 1111, javaType: 'Object'}, + {dbName: 'JAVA_OBJECT', dbType: 2000, javaType: 'Object'}, + {dbName: 'DISTINCT', dbType: 2001, javaType: 'Object'}, + {dbName: 'STRUCT', dbType: 2002, javaType: 'Object'}, + {dbName: 'ARRAY', dbType: 2003, javaType: 'Object'}, + {dbName: 'BLOB', dbType: 2004, javaType: 'Object'}, + {dbName: 'CLOB', dbType: 2005, javaType: 'String'}, + {dbName: 'REF', dbType: 2006, javaType: 'Object'}, + {dbName: 'DATALINK', dbType: 70, javaType: 'Object'}, + {dbName: 'BOOLEAN', dbType: 16, javaType: 'Boolean', primitiveType: 'boolean'}, + {dbName: 'ROWID', dbType: -8, javaType: 'Object'}, + {dbName: 'NCHAR', dbType: -15, javaType: 'String'}, + {dbName: 'NVARCHAR', dbType: -9, javaType: 'String'}, + {dbName: 'LONGNVARCHAR', dbType: -16, javaType: 'String'}, + {dbName: 'NCLOB', dbType: 2011, javaType: 'String'}, + {dbName: 'SQLXML', dbType: 2009, javaType: 'Object'} + ]; + + /*eslint-disable */ + const JAVA_KEYWORDS = [ + 'abstract', + 'assert', + 'boolean', + 'break', + 'byte', + 'case', + 'catch', + 'char', + 'class', + 'const', + 'continue', + 'default', + 'do', + 'double', + 'else', + 'enum', + 'extends', + 'false', + 'final', + 'finally', + 'float', + 'for', + 'goto', + 'if', + 'implements', + 'import', + 'instanceof', + 'int', + 'interface', + 'long', + 'native', + 'new', + 'null', + 'package', + 'private', + 'protected', + 'public', + 'return', + 'short', + 'static', + 'strictfp', + 'super', + 'switch', + 'synchronized', + 'this', + 'throw', + 'throws', + 'transient', + 'true', + 'try', + 'void', + 'volatile', + 'while' + ]; + /*eslint-enable */ + + const VALID_JAVA_IDENTIFIER = new RegExp('^[a-zA-Z_$][a-zA-Z\\d_$]*$'); + + function isValidJavaIdentifier(msg, ident, elemId, panels, panelId) { + if (isEmptyString(ident)) + return ErrorPopover.show(elemId, msg + ' is invalid!', panels, panelId); + + if (_.includes(JAVA_KEYWORDS, ident)) + return ErrorPopover.show(elemId, msg + ' could not contains reserved java keyword: "' + ident + '"!', panels, panelId); + + if (!VALID_JAVA_IDENTIFIER.test(ident)) + return ErrorPopover.show(elemId, msg + ' contains invalid identifier: "' + ident + '"!', panels, panelId); + + return true; + } + + function getModel(obj, field) { + let path = field.path; + + if (!isDefined(path) || !isDefined(obj)) + return obj; + + path = path.replace(/\[(\w+)\]/g, '.$1'); // convert indexes to properties + path = path.replace(/^\./, ''); // strip a leading dot + + const segs = path.split('.'); + let root = obj; + + while (segs.length > 0) { + const pathStep = segs.shift(); + + if (typeof root[pathStep] === 'undefined') + root[pathStep] = {}; + + root = root[pathStep]; + } + + return root; + } + + /** + * Extract datasource from cache or cluster. + * + * @param object Cache or cluster to extract datasource. + * @returns {*} Datasource object or null if not set. + */ + function extractDataSource(object) { + // Extract from cluster object + if (_.get(object, 'discovery.kind') === 'Jdbc') { + const datasource = object.discovery.Jdbc; + + if (datasource.dataSourceBean && datasource.dialect) + return datasource; + } // Extract from cache object + else if (_.get(object, 'cacheStoreFactory.kind')) { + const storeFactory = object.cacheStoreFactory[object.cacheStoreFactory.kind]; + + if (storeFactory.dialect || (storeFactory.connectVia === 'DataSource')) + return storeFactory; + } + + return null; + } + + const cacheStoreJdbcDialects = [ + {value: 'Generic', label: 'Generic JDBC'}, + {value: 'Oracle', label: 'Oracle'}, + {value: 'DB2', label: 'IBM DB2'}, + {value: 'SQLServer', label: 'Microsoft SQL Server'}, + {value: 'MySQL', label: 'MySQL'}, + {value: 'PostgreSQL', label: 'PostgreSQL'}, + {value: 'H2', label: 'H2 database'} + ]; + + function domainForStoreConfigured(domain) { + const isEmpty = !isDefined(domain) || (isEmptyString(domain.databaseSchema) && + isEmptyString(domain.databaseTable) && + _.isEmpty(domain.keyFields) && + _.isEmpty(domain.valueFields)); + + return !isEmpty; + } + + const DS_CHECK_SUCCESS = {checked: true}; + + /** + * Compare datasources of caches or clusters. + * + * @param firstObj First cache or cluster. + * @param secondObj Second cache or cluster. + * @returns {*} Check result object. + */ + function compareDataSources(firstObj, secondObj) { + const firstDs = extractDataSource(firstObj); + const secondDs = extractDataSource(secondObj); + + if (firstDs && secondDs) { + const firstDB = firstDs.dialect; + const secondDB = secondDs.dialect; + + if (firstDs.dataSourceBean === secondDs.dataSourceBean && firstDB !== secondDB) + return {checked: false, firstObj, firstDB, secondObj, secondDB}; + } + + return DS_CHECK_SUCCESS; + } + + function compareSQLSchemaNames(firstCache, secondCache) { + const firstName = firstCache.sqlSchema; + const secondName = secondCache.sqlSchema; + + if (firstName && secondName && (firstName === secondName)) + return {checked: false, firstCache, secondCache}; + + return DS_CHECK_SUCCESS; + } + + function toJavaName(prefix, name) { + const javaName = name ? name.replace(/[^A-Za-z_0-9]+/g, '_') : 'dflt'; + + return prefix + javaName.charAt(0).toLocaleUpperCase() + javaName.slice(1); + } + + return { + getModel, + mkOptions(options) { + return _.map(options, (option) => { + return {value: option, label: isDefined(option) ? option : 'Not set'}; + }); + }, + isDefined, + hasProperty(obj, props) { + for (const propName in props) { + if (props.hasOwnProperty(propName)) { + if (obj[propName]) + return true; + } + } + + return false; + }, + isEmptyString, + SUPPORTED_JDBC_TYPES, + findJdbcType(jdbcType) { + const res = _.find(ALL_JDBC_TYPES, (item) => item.dbType === jdbcType); + + return res ? res : {dbName: 'Unknown', javaType: 'Unknown'}; + }, + javaBuiltInClasses, + javaBuiltInTypes, + isJavaBuiltInClass, + isValidJavaIdentifier, + isValidJavaClass(msg, ident, allowBuiltInClass, elemId, packageOnly, panels, panelId) { + if (isEmptyString(ident)) + return ErrorPopover.show(elemId, msg + ' could not be empty!', panels, panelId); + + const parts = ident.split('.'); + + const len = parts.length; + + if (!allowBuiltInClass && isJavaBuiltInClass(ident)) + return ErrorPopover.show(elemId, msg + ' should not be the Java build-in class!', panels, panelId); + + if (len < 2 && !isJavaBuiltInClass(ident) && !packageOnly) + return ErrorPopover.show(elemId, msg + ' does not have package specified!', panels, panelId); + + for (let i = 0; i < parts.length; i++) { + const part = parts[i]; + + if (!isValidJavaIdentifier(msg, part, elemId, panels, panelId)) + return false; + } + + return true; + }, + domainForQueryConfigured(domain) { + const isEmpty = !isDefined(domain) || (_.isEmpty(domain.fields) && + _.isEmpty(domain.aliases) && + _.isEmpty(domain.indexes)); + + return !isEmpty; + }, + domainForStoreConfigured, + download(type, name, data) { + const file = document.createElement('a'); + + file.setAttribute('href', 'data:' + type + ';charset=utf-8,' + data); + file.setAttribute('download', name); + file.setAttribute('target', '_self'); + + file.style.display = 'none'; + + document.body.appendChild(file); + + file.click(); + + document.body.removeChild(file); + }, + getQueryVariable(name) { + const attrs = window.location.search.substring(1).split('&'); + const attr = _.find(attrs, (a) => a === name || (a.indexOf('=') >= 0 && a.substr(0, a.indexOf('=')) === name)); + + if (!isDefined(attr)) + return null; + + if (attr === name) + return true; + + return attr.substr(attr.indexOf('=') + 1); + }, + cacheStoreJdbcDialects, + cacheStoreJdbcDialectsLabel(dialect) { + const found = _.find(cacheStoreJdbcDialects, (dialectVal) => dialectVal.value === dialect); + + return found ? found.label : null; + }, + checkDataSources(cluster, caches, checkCacheExt) { + let res = DS_CHECK_SUCCESS; + + _.find(caches, (curCache, curIx) => { + res = compareDataSources(curCache, cluster); + + if (!res.checked) + return true; + + if (isDefined(checkCacheExt)) { + if (checkCacheExt._id !== curCache._id) { + res = compareDataSources(checkCacheExt, curCache); + + return !res.checked; + } + + return false; + } + + return _.find(caches, (checkCache, checkIx) => { + if (checkIx < curIx) { + res = compareDataSources(checkCache, curCache); + + return !res.checked; + } + + return false; + }); + }); + + return res; + }, + checkCacheSQLSchemas(caches, checkCacheExt) { + let res = DS_CHECK_SUCCESS; + + _.find(caches, (curCache, curIx) => { + if (isDefined(checkCacheExt)) { + if (checkCacheExt._id !== curCache._id) { + res = compareSQLSchemaNames(checkCacheExt, curCache); + + return !res.checked; + } + + return false; + } + + return _.find(caches, (checkCache, checkIx) => { + if (checkIx < curIx) { + res = compareSQLSchemaNames(checkCache, curCache); + + return !res.checked; + } + + return false; + }); + }); + + return res; + }, + autoCacheStoreConfiguration(cache, domains) { + const cacheStoreFactory = isDefined(cache.cacheStoreFactory) && + isDefined(cache.cacheStoreFactory.kind); + + if (!cacheStoreFactory && _.findIndex(domains, domainForStoreConfigured) >= 0) { + const dflt = !cache.readThrough && !cache.writeThrough; + + return { + cacheStoreFactory: { + kind: 'CacheJdbcPojoStoreFactory', + CacheJdbcPojoStoreFactory: { + dataSourceBean: toJavaName('ds', cache.name), + dialect: 'Generic' + }, + CacheJdbcBlobStoreFactory: {connectVia: 'DataSource'} + }, + readThrough: dflt || cache.readThrough, + writeThrough: dflt || cache.writeThrough + }; + } + }, + autoClusterSwapSpiConfiguration(cluster, caches) { + const swapConfigured = cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind; + + if (!swapConfigured && _.find(caches, (cache) => cache.swapEnabled)) + return {swapSpaceSpi: {kind: 'FileSwapSpaceSpi'}}; + + return null; + }, + randomString(len) { + const possible = 'ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789'; + const possibleLen = possible.length; + + let res = ''; + + for (let i = 0; i < len; i++) + res += possible.charAt(Math.floor(Math.random() * possibleLen)); + + return res; + }, + checkFieldValidators(ui) { + const form = ui.inputForm; + const errors = form.$error; + const errKeys = Object.keys(errors); + + if (errKeys && errKeys.length > 0) { + const firstErrorKey = errKeys[0]; + + const firstError = errors[firstErrorKey][0]; + const actualError = firstError.$error[firstErrorKey][0]; + + const errNameFull = actualError.$name; + const errNameShort = errNameFull.endsWith('TextInput') ? errNameFull.substring(0, errNameFull.length - 9) : errNameFull; + + const extractErrorMessage = (errName) => { + try { + return errors[firstErrorKey][0].$errorMessages[errName][firstErrorKey]; + } + catch (ignored) { + try { + return form[firstError.$name].$errorMessages[errName][firstErrorKey]; + } + catch (ignited) { + return false; + } + } + }; + + const msg = extractErrorMessage(errNameFull) || extractErrorMessage(errNameShort) || 'Invalid value!'; + + return ErrorPopover.show(errNameFull, msg, ui, firstError.$name); + } + + return true; + } + }; +}]]; diff --git a/modules/web-console/src/main/js/app/services/Messages.service.js b/modules/web-console/frontend/app/services/Messages.service.js similarity index 100% rename from modules/web-console/src/main/js/app/services/Messages.service.js rename to modules/web-console/frontend/app/services/Messages.service.js diff --git a/modules/web-console/src/main/js/app/services/ModelNormalizer.service.js b/modules/web-console/frontend/app/services/ModelNormalizer.service.js similarity index 100% rename from modules/web-console/src/main/js/app/services/ModelNormalizer.service.js rename to modules/web-console/frontend/app/services/ModelNormalizer.service.js diff --git a/modules/web-console/src/main/js/app/services/UnsavedChangesGuard.service.js b/modules/web-console/frontend/app/services/UnsavedChangesGuard.service.js similarity index 100% rename from modules/web-console/src/main/js/app/services/UnsavedChangesGuard.service.js rename to modules/web-console/frontend/app/services/UnsavedChangesGuard.service.js diff --git a/modules/web-console/src/main/js/app/vendor.js b/modules/web-console/frontend/app/vendor.js similarity index 98% rename from modules/web-console/src/main/js/app/vendor.js rename to modules/web-console/frontend/app/vendor.js index a8eeea7a1f062..0322887de9f68 100644 --- a/modules/web-console/src/main/js/app/vendor.js +++ b/modules/web-console/frontend/app/vendor.js @@ -17,6 +17,7 @@ import 'jquery'; import 'angular'; +import 'angular-acl'; import 'angular-animate'; import 'angular-sanitize'; import 'angular-strap'; diff --git a/modules/web-console/src/main/js/controllers/admin-controller.js b/modules/web-console/frontend/controllers/admin-controller.js similarity index 94% rename from modules/web-console/src/main/js/controllers/admin-controller.js rename to modules/web-console/frontend/controllers/admin-controller.js index 9e5aea7fbf06b..57a39b2e87b6b 100644 --- a/modules/web-console/src/main/js/controllers/admin-controller.js +++ b/modules/web-console/frontend/controllers/admin-controller.js @@ -23,8 +23,8 @@ export default ['adminController', [ const _reloadUsers = () => { $http.post('/api/v1/admin/list') - .then(({data}) => { - $scope.users = data; + .success((users) => { + $scope.users = users; _.forEach($scope.users, (user) => { user.userName = user.firstName + ' ' + user.lastName; @@ -33,14 +33,15 @@ export default ['adminController', [ (user.company || '') + ' ' + (user.countryCode || ''); }); }) - .catch(Messages.showError); + .error(Messages.showError); }; _reloadUsers(); $scope.becomeUser = function(user) { $http.get('/api/v1/admin/become', { params: {viewedUserId: user._id}}) - .then(User.read) + .catch(({data}) => Promise.reject(data)) + .then(User.load) .then((becomeUser) => { $rootScope.$broadcast('user', becomeUser); diff --git a/modules/web-console/src/main/js/controllers/caches-controller.js b/modules/web-console/frontend/controllers/caches-controller.js similarity index 71% rename from modules/web-console/src/main/js/controllers/caches-controller.js rename to modules/web-console/frontend/controllers/caches-controller.js index cbd681e2a32ad..98730515c3444 100644 --- a/modules/web-console/src/main/js/controllers/caches-controller.js +++ b/modules/web-console/frontend/controllers/caches-controller.js @@ -17,8 +17,8 @@ // Controller for Caches screen. export default ['cachesController', [ - '$scope', '$http', '$state', '$filter', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', - function($scope, $http, $state, $filter, $timeout, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard) { + '$scope', '$http', '$state', '$filter', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'igniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', + function($scope, $http, $state, $filter, $timeout, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, Resource, ErrorPopover, FormUtils) { UnsavedChangesGuard.install($scope); const emptyCache = {empty: true}; @@ -27,22 +27,25 @@ export default ['cachesController', [ const blank = { evictionPolicy: {}, - cacheStoreFactory: {}, - nearConfiguration: {} + cacheStoreFactory: { + CacheHibernateBlobStoreFactory: { + hibernateProperties: [] + } + }, + nearConfiguration: {}, + sqlFunctionClasses: [] }; // We need to initialize backupItem with empty object in order to properly used from angular directives. $scope.backupItem = emptyCache; - $scope.ui = LegacyUtils.formUI(); + $scope.ui = FormUtils.formUI(); $scope.ui.activePanels = [0]; $scope.ui.topPanels = [0, 1, 2, 3]; - $scope.hidePopover = LegacyUtils.hidePopover; - $scope.saveBtnTipText = LegacyUtils.saveBtnTipText; - $scope.widthIsSufficient = LegacyUtils.widthIsSufficient; - - const showPopoverMessage = LegacyUtils.showPopoverMessage; + $scope.saveBtnTipText = FormUtils.saveBtnTipText; + $scope.widthIsSufficient = FormUtils.widthIsSufficient; + $scope.offHeapMode = 'DISABLED'; $scope.contentVisible = function() { const item = $scope.backupItem; @@ -53,7 +56,7 @@ export default ['cachesController', [ $scope.toggleExpanded = function() { $scope.ui.expanded = !$scope.ui.expanded; - LegacyUtils.hidePopover(); + ErrorPopover.hide(); }; $scope.caches = []; @@ -77,34 +80,52 @@ export default ['cachesController', [ }, []); } + const setOffHeapMode = (item) => { + if (_.isNil(item.offHeapMaxMemory)) + return; + + return item.offHeapMode = Math.sign(item.offHeapMaxMemory); + }; + + const setOffHeapMaxMemory = (value) => { + const item = $scope.backupItem; + + if (_.isNil(value) || value <= 0) + return item.offHeapMaxMemory = value; + + item.offHeapMaxMemory = item.offHeapMaxMemory > 0 ? item.offHeapMaxMemory : null; + }; + Loading.start('loadingCachesScreen'); // When landing on the page, get caches and show them. - $http.post('/api/v1/configuration/caches/list') - .success(function(data) { + Resource.read() + .then(({spaces, clusters, caches, domains, igfss}) => { const validFilter = $filter('domainsValidation'); - $scope.spaces = data.spaces; - $scope.caches = data.caches; + $scope.spaces = spaces; + $scope.caches = caches; + $scope.igfss = _.map(igfss, (igfs) => ({ + label: igfs.name, + value: igfs._id, + igfs + })); _.forEach($scope.caches, (cache) => cache.label = _cacheLbl(cache)); - $scope.clusters = _.map(data.clusters, function(cluster) { - return { - value: cluster._id, - label: cluster.name, - caches: cluster.caches - }; - }); + $scope.clusters = _.map(clusters, (cluster) => ({ + value: cluster._id, + label: cluster.name, + discovery: cluster.discovery, + caches: cluster.caches + })); - $scope.domains = _.sortBy(_.map(validFilter(data.domains, true, false), function(domain) { - return { - value: domain._id, - label: domain.valueType, - kind: domain.kind, - meta: domain - }; - }), 'label'); + $scope.domains = _.sortBy(_.map(validFilter(domains, true, false), (domain) => ({ + label: domain.valueType, + value: domain._id, + kind: domain.kind, + meta: domain + })), 'label'); if ($state.params.linkId) $scope.createItem($state.params.linkId); @@ -134,18 +155,28 @@ export default ['cachesController', [ }); $scope.$watch('backupItem', function(val) { + if (!$scope.ui.inputForm) + return; + const form = $scope.ui.inputForm; - if (form.$pristine || (form.$valid && ModelNormalizer.isEqual(__original_value, val))) + if (form.$valid && ModelNormalizer.isEqual(__original_value, val)) form.$setPristine(); else form.$setDirty(); }, true); + + $scope.$watch('backupItem.offHeapMode', setOffHeapMaxMemory); + + $scope.$watch('ui.activePanels.length', () => { + ErrorPopover.hide(); + }); }) .catch(Messages.showError) - .finally(function() { + .then(() => { $scope.ui.ready = true; - $scope.ui.inputForm.$setPristine(); + $scope.ui.inputForm && $scope.ui.inputForm.$setPristine(); + Loading.finish('loadingCachesScreen'); }); @@ -175,13 +206,20 @@ export default ['cachesController', [ $scope.backupItem = angular.merge({}, blank, $scope.backupItem); + if ($scope.ui.inputForm) { + $scope.ui.inputForm.$error = {}; + $scope.ui.inputForm.$setPristine(); + } + + setOffHeapMode($scope.backupItem); + __original_value = ModelNormalizer.normalize($scope.backupItem); if (LegacyUtils.getQueryVariable('new')) $state.go('base.configuration.caches'); } - LegacyUtils.confirmUnsavedChanges($scope.backupItem && $scope.ui.inputForm.$dirty, selectItem); + FormUtils.confirmUnsavedChanges($scope.backupItem && $scope.ui.inputForm && $scope.ui.inputForm.$dirty, selectItem); }; $scope.linkId = () => $scope.backupItem._id ? $scope.backupItem._id : 'create'; @@ -202,7 +240,7 @@ export default ['cachesController', [ // Add new cache. $scope.createItem = function(linkId) { - $timeout(() => LegacyUtils.ensureActivePanel($scope.ui, 'general', 'cacheName')); + $timeout(() => FormUtils.ensureActivePanel($scope.ui, 'general', 'cacheNameInput')); $scope.selectItem(null, prepareNewItem(linkId)); }; @@ -228,17 +266,27 @@ export default ['cachesController', [ const failCluster = _.find(clusters, (cluster) => { const caches = clusterCaches(cluster); - checkRes = LegacyUtils.checkCachesDataSources(caches, $scope.backupItem); + checkRes = LegacyUtils.checkDataSources(cluster, caches, $scope.backupItem); return !checkRes.checked; }); if (!checkRes.checked) { - return showPopoverMessage($scope.ui, 'store', checkRes.firstCache.cacheStoreFactory.kind === 'CacheJdbcPojoStoreFactory' ? 'pojoDialect' : 'blobDialect', - 'Found cache "' + checkRes.secondCache.name + '" in cluster "' + failCluster.label + '" ' + - 'with the same data source bean name "' + checkRes.firstCache.cacheStoreFactory[checkRes.firstCache.cacheStoreFactory.kind].dataSourceBean + + if (_.get(checkRes.secondObj, 'discovery.kind') === 'Jdbc') { + return ErrorPopover.show(checkRes.firstObj.cacheStoreFactory.kind === 'CacheJdbcPojoStoreFactory' ? 'pojoDialectInput' : 'blobDialectInput', + 'Found cluster "' + failCluster.label + '" with the same data source bean name "' + + checkRes.secondObj.discovery.Jdbc.dataSourceBean + '" and different database: "' + + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.firstDB) + '" in current cache and "' + + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.secondDB) + '" in"' + checkRes.secondObj.label + '" cluster', + $scope.ui, 'store', 10000); + } + + return ErrorPopover.show(checkRes.firstObj.cacheStoreFactory.kind === 'CacheJdbcPojoStoreFactory' ? 'pojoDialectInput' : 'blobDialectInput', + 'Found cache "' + checkRes.secondObj.name + '" in cluster "' + failCluster.label + '" ' + + 'with the same data source bean name "' + checkRes.firstObj.cacheStoreFactory[checkRes.firstObj.cacheStoreFactory.kind].dataSourceBean + '" and different database: "' + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.firstDB) + '" in current cache and "' + - LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.secondDB) + '" in "' + checkRes.secondCache.name + '"', 10000); + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.secondDB) + '" in "' + checkRes.secondObj.name + '" cache', + $scope.ui, 'store', 10000); } return true; @@ -258,9 +306,10 @@ export default ['cachesController', [ }); if (!checkRes.checked) { - return showPopoverMessage($scope.ui, 'query', 'sqlSchema', + return ErrorPopover.show('sqlSchemaInput', 'Found cache "' + checkRes.secondCache.name + '" in cluster "' + failCluster.label + '" ' + - 'with the same SQL schema name "' + checkRes.firstCache.sqlSchema + '"', 10000); + 'with the same SQL schema name "' + checkRes.firstCache.sqlSchema + '"', + $scope.ui, 'query', 10000); } return true; @@ -288,35 +337,35 @@ export default ['cachesController', [ } if ((item.readThrough || item.writeThrough) && !cacheStoreFactorySelected) - return showPopoverMessage($scope.ui, 'store', 'cacheStoreFactory', (item.readThrough ? 'Read' : 'Write') + ' through are enabled but store is not configured!'); + return ErrorPopover.show('cacheStoreFactoryInput', (item.readThrough ? 'Read' : 'Write') + ' through are enabled but store is not configured!', $scope.ui, 'store'); if (item.writeBehindEnabled && !cacheStoreFactorySelected) - return showPopoverMessage($scope.ui, 'store', 'cacheStoreFactory', 'Write behind enabled but store is not configured!'); + return ErrorPopover.show('cacheStoreFactoryInput', 'Write behind enabled but store is not configured!', $scope.ui, 'store'); if (cacheStoreFactorySelected && !item.readThrough && !item.writeThrough) - return showPopoverMessage($scope.ui, 'store', 'readThroughTooltip', 'Store is configured but read/write through are not enabled!'); + return ErrorPopover.show('readThroughLabel', 'Store is configured but read/write through are not enabled!', $scope.ui, 'store'); return true; } // Check cache logical consistency. function validate(item) { - LegacyUtils.hidePopover(); + ErrorPopover.hide(); if (LegacyUtils.isEmptyString(item.name)) - return showPopoverMessage($scope.ui, 'general', 'cacheName', 'Cache name should not be empty!'); + return ErrorPopover.show('cacheNameInput', 'Cache name should not be empty!', $scope.ui, 'general'); if (item.memoryMode === 'ONHEAP_TIERED' && item.offHeapMaxMemory > 0 && !LegacyUtils.isDefined(item.evictionPolicy.kind)) - return showPopoverMessage($scope.ui, 'memory', 'evictionPolicyKind', 'Eviction policy should not be configured!'); + return ErrorPopover.show('evictionPolicyKindInput', 'Eviction policy should be configured!', $scope.ui, 'memory'); if (!LegacyUtils.checkFieldValidators($scope.ui)) return false; if (item.memoryMode === 'OFFHEAP_VALUES' && !_.isEmpty(item.domains)) - return showPopoverMessage($scope.ui, 'memory', 'memoryMode', 'Query indexing could not be enabled while values are stored off-heap!'); + return ErrorPopover.show('memoryModeInput', 'Query indexing could not be enabled while values are stored off-heap!', $scope.ui, 'memory'); - if (item.memoryMode === 'OFFHEAP_TIERED' && (!LegacyUtils.isDefined(item.offHeapMaxMemory) || item.offHeapMaxMemory < 0)) - return showPopoverMessage($scope.ui, 'memory', 'offHeapMaxMemory', 'Off-heap max memory should be specified!'); + if (item.memoryMode === 'OFFHEAP_TIERED' && item.offHeapMaxMemory === -1) + return ErrorPopover.show('offHeapModeInput', 'Invalid value!', $scope.ui, 'memory'); if (!checkSQLSchemas()) return false; @@ -325,7 +374,10 @@ export default ['cachesController', [ return false; if (item.writeBehindFlushSize === 0 && item.writeBehindFlushFrequency === 0) - return showPopoverMessage($scope.ui, 'store', 'writeBehindFlushSize', 'Both "Flush frequency" and "Flush size" are not allowed as 0!'); + return ErrorPopover.show('writeBehindFlushSizeInput', 'Both "Flush frequency" and "Flush size" are not allowed as 0!', $scope.ui, 'store'); + + if (item.nodeFilter && item.nodeFilter.kind === 'OnNodes' && _.isEmpty(item.nodeFilter.OnNodes.nodeIds)) + return ErrorPopover.show('nodeFilter-title', 'At least one node ID should be specified!', $scope.ui, 'nodeFilter'); return true; } @@ -424,12 +476,12 @@ export default ['cachesController', [ if (idx >= 0) { caches.splice(idx, 1); + $scope.ui.inputForm.$setPristine(); + if (caches.length > 0) $scope.selectItem(caches[0]); - else { + else $scope.backupItem = emptyCache; - $scope.ui.inputForm.$setPristine(); - } _.forEach($scope.clusters, (cluster) => _.remove(cluster.caches, (id) => id === _id)); _.forEach($scope.domains, (domain) => _.remove(domain.meta.caches, (id) => id === _id)); @@ -453,6 +505,7 @@ export default ['cachesController', [ _.forEach($scope.domains, (domain) => domain.meta.caches = []); $scope.backupItem = emptyCache; + $scope.ui.inputForm.$error = {}; $scope.ui.inputForm.$setPristine(); }) .error(Messages.showError); @@ -463,6 +516,7 @@ export default ['cachesController', [ Confirm.confirm('Are you sure you want to undo all changes for current cache?') .then(function() { $scope.backupItem = $scope.selectedItem ? angular.copy($scope.selectedItem) : prepareNewItem(); + $scope.ui.inputForm.$error = {}; $scope.ui.inputForm.$setPristine(); }); }; diff --git a/modules/web-console/src/main/js/controllers/clusters-controller.js b/modules/web-console/frontend/controllers/clusters-controller.js similarity index 74% rename from modules/web-console/src/main/js/controllers/clusters-controller.js rename to modules/web-console/frontend/controllers/clusters-controller.js index 5f86d08711fe6..5a3c7e2717602 100644 --- a/modules/web-console/src/main/js/controllers/clusters-controller.js +++ b/modules/web-console/frontend/controllers/clusters-controller.js @@ -17,8 +17,8 @@ // Controller for Clusters screen. export default ['clustersController', [ - '$rootScope', '$scope', '$http', '$state', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'igniteEventGroups', 'DemoInfo', 'IgniteLegacyTable', - function($root, $scope, $http, $state, $timeout, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, igniteEventGroups, DemoInfo, LegacyTable) { + '$rootScope', '$scope', '$http', '$state', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'igniteEventGroups', 'DemoInfo', 'IgniteLegacyTable', 'igniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', + function($root, $scope, $http, $state, $timeout, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, igniteEventGroups, DemoInfo, LegacyTable, Resource, ErrorPopover, FormUtils) { UnsavedChangesGuard.install($scope); const emptyCluster = {empty: true}; @@ -28,11 +28,19 @@ export default ['clustersController', [ const blank = { atomicConfiguration: {}, binaryConfiguration: {}, + cacheKeyConfiguration: [], communication: {}, connector: {}, - discovery: {}, + discovery: { + Cloud: { + regions: [], + zones: [] + } + }, marshaller: {}, - sslContextFactory: {}, + sslContextFactory: { + trustManagers: [] + }, swapSpaceSpi: {}, transactionConfiguration: {}, collision: {} @@ -43,9 +51,7 @@ export default ['clustersController', [ 'collision.JobStealing.stealingAttributes': {id: 'CAttribute', idPrefix: 'Key', searchCol: 'name', valueCol: 'key', dupObjName: 'name', group: 'collision'} }; - const showPopoverMessage = LegacyUtils.showPopoverMessage; - - $scope.tablePairValid = function(item, field, index) { + $scope.tablePairValid = function(item, field, index, stopEdit) { const pairField = pairFields[field.model]; const pairValue = LegacyTable.tablePairValue(field, index); @@ -59,8 +65,12 @@ export default ['clustersController', [ }); // Found duplicate by key. - if (idx >= 0 && idx !== index) - return showPopoverMessage($scope.ui, pairField.group, LegacyTable.tableFieldId(index, pairField.idPrefix + pairField.id), 'Attribute with such ' + pairField.dupObjName + ' already exists!'); + if (idx >= 0 && idx !== index) { + if (stopEdit) + return false; + + return ErrorPopover.show(LegacyTable.tableFieldId(index, pairField.idPrefix + pairField.id), 'Attribute with such ' + pairField.dupObjName + ' already exists!', $scope.ui, pairField.group); + } } } @@ -142,13 +152,12 @@ export default ['clustersController', [ // We need to initialize backupItem with empty object in order to properly used from angular directives. $scope.backupItem = emptyCluster; - $scope.ui = LegacyUtils.formUI(); + $scope.ui = FormUtils.formUI(); $scope.ui.activePanels = [0]; $scope.ui.topPanels = [0]; - $scope.hidePopover = LegacyUtils.hidePopover; - $scope.saveBtnTipText = LegacyUtils.saveBtnTipText; - $scope.widthIsSufficient = LegacyUtils.widthIsSufficient; + $scope.saveBtnTipText = FormUtils.saveBtnTipText; + $scope.widthIsSufficient = FormUtils.widthIsSufficient; $scope.contentVisible = function() { const item = $scope.backupItem; @@ -159,7 +168,7 @@ export default ['clustersController', [ $scope.toggleExpanded = function() { $scope.ui.expanded = !$scope.ui.expanded; - LegacyUtils.hidePopover(); + ErrorPopover.hide(); }; $scope.discoveries = [ @@ -194,12 +203,13 @@ export default ['clustersController', [ Loading.start('loadingClustersScreen'); // When landing on the page, get clusters and show them. - $http.post('/api/v1/configuration/clusters/list') - .success(function(data) { - $scope.spaces = data.spaces; - $scope.clusters = data.clusters; - $scope.caches = _.map(data.caches, (cache) => ({value: cache._id, label: cache.name, cache})); - $scope.igfss = _.map(data.igfss, (igfs) => ({value: igfs._id, label: igfs.name, igfs})); + Resource.read() + .then(({spaces, clusters, caches, igfss}) => { + $scope.spaces = spaces; + $scope.clusters = clusters; + + $scope.caches = _.map(caches, (cache) => ({value: cache._id, label: cache.name, cache})); + $scope.igfss = _.map(igfss, (igfs) => ({value: igfs._id, label: igfs.name, igfs})); _.forEach($scope.clusters, (cluster) => { cluster.label = _clusterLbl(cluster); @@ -240,14 +250,21 @@ export default ['clustersController', [ }); $scope.$watch('backupItem', function(val) { + if (!$scope.ui.inputForm) + return; + const form = $scope.ui.inputForm; - if (form.$pristine || (form.$valid && ModelNormalizer.isEqual(__original_value, val))) + if (form.$valid && ModelNormalizer.isEqual(__original_value, val)) form.$setPristine(); else form.$setDirty(); }, true); + $scope.$watch('ui.activePanels.length', () => { + ErrorPopover.hide(); + }); + if ($root.IgniteDemoMode && sessionStorage.showDemoInfo !== 'true') { sessionStorage.showDemoInfo = 'true'; @@ -255,9 +272,10 @@ export default ['clustersController', [ } }) .catch(Messages.showError) - .finally(function() { + .then(() => { $scope.ui.ready = true; - $scope.ui.inputForm.$setPristine(); + $scope.ui.inputForm && $scope.ui.inputForm.$setPristine(); + Loading.finish('loadingClustersScreen'); }); @@ -284,13 +302,18 @@ export default ['clustersController', [ $scope.backupItem = angular.merge({}, blank, $scope.backupItem); + if ($scope.ui.inputForm) { + $scope.ui.inputForm.$error = {}; + $scope.ui.inputForm.$setPristine(); + } + __original_value = ModelNormalizer.normalize($scope.backupItem); if (LegacyUtils.getQueryVariable('new')) $state.go('base.configuration.clusters'); } - LegacyUtils.confirmUnsavedChanges($scope.backupItem && $scope.ui.inputForm.$dirty, selectItem); + FormUtils.confirmUnsavedChanges($scope.backupItem && $scope.ui.inputForm && $scope.ui.inputForm.$dirty, selectItem); }; $scope.linkId = () => $scope.backupItem._id ? $scope.backupItem._id : 'create'; @@ -298,7 +321,12 @@ export default ['clustersController', [ function prepareNewItem(linkId) { return angular.merge({}, blank, { space: $scope.spaces[0]._id, - discovery: {kind: 'Multicast', Vm: {addresses: ['127.0.0.1:47500..47510']}, Multicast: {addresses: ['127.0.0.1:47500..47510']}}, + discovery: { + kind: 'Multicast', + Vm: {addresses: ['127.0.0.1:47500..47510']}, + Multicast: {addresses: ['127.0.0.1:47500..47510']}, + Jdbc: {initSchema: true} + }, binaryConfiguration: {typeConfigurations: [], compactFooter: true}, communication: {tcpNoDelay: true}, connector: {noDelay: true}, @@ -312,7 +340,7 @@ export default ['clustersController', [ // Add new cluster. $scope.createItem = function(linkId) { - $timeout(() => LegacyUtils.ensureActivePanel($scope.ui, 'general', 'clusterName')); + $timeout(() => FormUtils.ensureActivePanel($scope.ui, 'general', 'clusterNameInput')); $scope.selectItem(null, prepareNewItem(linkId)); }; @@ -329,14 +357,24 @@ export default ['clustersController', [ function checkCacheDatasources(item) { const caches = clusterCaches(item); - const checkRes = LegacyUtils.checkCachesDataSources(caches); + const checkRes = LegacyUtils.checkDataSources(item, caches); if (!checkRes.checked) { - return showPopoverMessage($scope.ui, 'general', 'caches', - 'Found caches "' + checkRes.firstCache.name + '" and "' + checkRes.secondCache.name + '" ' + - 'with the same data source bean name "' + checkRes.firstCache.cacheStoreFactory[checkRes.firstCache.cacheStoreFactory.kind].dataSourceBean + - '" and different databases: "' + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.firstDB) + '" in "' + checkRes.firstCache.name + '" and "' + - LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.secondDB) + '" in "' + checkRes.secondCache.name + '"', 10000); + if (_.get(checkRes.secondObj, 'discovery.kind') === 'Jdbc') { + return ErrorPopover.show('dialectInput', + 'Found cache "' + checkRes.firstObj.name + '" with the same data source bean name "' + + item.discovery.Jdbc.dataSourceBean + '" and different database: "' + + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.secondDB) + '" in current cluster and "' + + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.firstDB) + '" in "' + checkRes.firstObj.name + '" cache', + $scope.ui, 'general', 10000); + } + + return ErrorPopover.show('cachesInput', + 'Found caches "' + checkRes.firstObj.name + '" and "' + checkRes.secondObj.name + '" ' + + 'with the same data source bean name "' + checkRes.firstObj.cacheStoreFactory[checkRes.firstObj.cacheStoreFactory.kind].dataSourceBean + + '" and different databases: "' + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.firstDB) + '" in "' + checkRes.firstObj.name + '" and "' + + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.secondDB) + '" in "' + checkRes.secondObj.name + '" cache', + $scope.ui, 'general', 10000); } return true; @@ -348,9 +386,10 @@ export default ['clustersController', [ const checkRes = LegacyUtils.checkCacheSQLSchemas(caches); if (!checkRes.checked) { - return showPopoverMessage($scope.ui, 'general', 'caches', + return ErrorPopover.show('cachesInput', 'Found caches "' + checkRes.firstCache.name + '" and "' + checkRes.secondCache.name + '" ' + - 'with the same SQL schema name "' + checkRes.firstCache.sqlSchema + '"', 10000); + 'with the same SQL schema name "' + checkRes.firstCache.sqlSchema + '"', + $scope.ui, 'general', 10000); } return true; @@ -365,10 +404,10 @@ export default ['clustersController', [ const type = b.typeConfigurations[typeIx]; if (LegacyUtils.isEmptyString(type.typeName)) - return showPopoverMessage($scope.ui, 'binary', 'typeName' + typeIx, 'Type name should be specified!'); + return ErrorPopover.show('typeName' + typeIx + 'Input', 'Type name should be specified!', $scope.ui, 'binary'); if (_.find(b.typeConfigurations, (t, ix) => ix < typeIx && t.typeName === type.typeName)) - return showPopoverMessage($scope.ui, 'binary', 'typeName' + typeIx, 'Type with such name is already specified!'); + return ErrorPopover.show('typeName' + typeIx + 'Input', 'Type with such name is already specified!', $scope.ui, 'binary'); } } } @@ -376,20 +415,39 @@ export default ['clustersController', [ return true; } + function checkCacheKeyConfiguration(item) { + const cfgs = item.cacheKeyConfiguration; + + if (_.isEmpty(cfgs)) + return true; + + for (let typeIx = 0; typeIx < cfgs.length; typeIx++) { + const type = cfgs[typeIx]; + + if (LegacyUtils.isEmptyString(type.typeName)) + return ErrorPopover.show('cacheKeyTypeName' + typeIx + 'Input', 'Cache type configuration name should be specified!', $scope.ui, 'cacheKeyCfg'); + + if (_.find(cfgs, (t, ix) => ix < typeIx && t.typeName === type.typeName)) + return ErrorPopover.show('cacheKeyTypeName' + typeIx + 'Input', 'Cache type configuration with such name is already specified!', $scope.ui, 'cacheKeyCfg'); + } + + return true; + } + function checkCommunicationConfiguration(item) { const c = item.communication; if (LegacyUtils.isDefined(c)) { if (LegacyUtils.isDefined(c.unacknowledgedMessagesBufferSize)) { if (LegacyUtils.isDefined(c.messageQueueLimit) && c.unacknowledgedMessagesBufferSize < 5 * c.messageQueueLimit) - return showPopoverMessage($scope.ui, 'communication', 'unacknowledgedMessagesBufferSize', 'Maximum number of stored unacknowledged messages should be at least 5 * message queue limit!'); + return ErrorPopover.show('unacknowledgedMessagesBufferSizeInput', 'Maximum number of stored unacknowledged messages should be at least 5 * message queue limit!', $scope.ui, 'communication'); if (LegacyUtils.isDefined(c.ackSendThreshold) && c.unacknowledgedMessagesBufferSize < 5 * c.ackSendThreshold) - return showPopoverMessage($scope.ui, 'communication', 'unacknowledgedMessagesBufferSize', 'Maximum number of stored unacknowledged messages should be at least 5 * ack send threshold!'); + return ErrorPopover.show('unacknowledgedMessagesBufferSizeInput', 'Maximum number of stored unacknowledged messages should be at least 5 * ack send threshold!', $scope.ui, 'communication'); } if (c.sharedMemoryPort === 0) - return showPopoverMessage($scope.ui, 'communication', 'sharedMemoryPort', 'Shared memory port should be more than "0" or equals to "-1"!'); + return ErrorPopover.show('sharedMemoryPortInput', 'Shared memory port should be more than "0" or equals to "-1"!', $scope.ui, 'communication'); } return true; @@ -400,10 +458,10 @@ export default ['clustersController', [ if (d) { if ((_.isNil(d.maxAckTimeout) ? 600000 : d.maxAckTimeout) < (d.ackTimeout || 5000)) - return showPopoverMessage($scope.ui, 'discovery', 'ackTimeout', 'Acknowledgement timeout should be less than max acknowledgement timeout!'); + return ErrorPopover.show('ackTimeoutInput', 'Acknowledgement timeout should be less than max acknowledgement timeout!', $scope.ui, 'discovery'); if (d.kind === 'Vm' && d.Vm && d.Vm.addresses.length === 0) - return showPopoverMessage($scope.ui, 'general', 'addresses', 'Addresses are not specified!'); + return ErrorPopover.show('addresses', 'Addresses are not specified!', $scope.ui, 'general'); } return true; @@ -418,12 +476,12 @@ export default ['clustersController', [ const sparsity = swap.maximumSparsity; if (LegacyUtils.isDefined(sparsity) && (sparsity < 0 || sparsity >= 1)) - return showPopoverMessage($scope.ui, 'swap', 'maximumSparsity', 'Maximum sparsity should be more or equal 0 and less than 1!'); + return ErrorPopover.show('maximumSparsityInput', 'Maximum sparsity should be more or equal 0 and less than 1!', $scope.ui, 'swap'); const readStripesNumber = swap.readStripesNumber; if (readStripesNumber && !(readStripesNumber === -1 || (readStripesNumber & (readStripesNumber - 1)) === 0)) - return showPopoverMessage($scope.ui, 'swap', 'readStripesNumber', 'Read stripe size must be positive and power of two!'); + return ErrorPopover.show('readStripesNumberInput', 'Read stripe size must be positive and power of two!', $scope.ui, 'swap'); } return true; @@ -434,15 +492,15 @@ export default ['clustersController', [ if (LegacyUtils.isDefined(r)) { if (r.sslEnabled && LegacyUtils.isEmptyString(r.sslFactory)) - return showPopoverMessage($scope.ui, 'connector', 'connectorSslFactory', 'SSL factory should not be empty!'); + return ErrorPopover.show('connectorSslFactoryInput', 'SSL factory should not be empty!', $scope.ui, 'connector'); } if (item.sslEnabled) { if (!LegacyUtils.isDefined(item.sslContextFactory) || LegacyUtils.isEmptyString(item.sslContextFactory.keyStoreFilePath)) - return showPopoverMessage($scope.ui, 'sslConfiguration', 'keyStoreFilePath', 'Key store file should not be empty!'); + return ErrorPopover.show('keyStoreFilePathInput', 'Key store file should not be empty!', $scope.ui, 'sslConfiguration'); if (LegacyUtils.isEmptyString(item.sslContextFactory.trustStoreFilePath) && _.isEmpty(item.sslContextFactory.trustManagers)) - return showPopoverMessage($scope.ui, 'sslConfiguration', 'sslConfiguration-title', 'Trust storage file or managers should be configured!'); + return ErrorPopover.show('sslConfiguration-title', 'Trust storage file or managers should be configured!', $scope.ui, 'sslConfiguration'); } return true; @@ -450,17 +508,17 @@ export default ['clustersController', [ function checkPoolSizes(item) { if (item.rebalanceThreadPoolSize && item.systemThreadPoolSize && item.systemThreadPoolSize <= item.rebalanceThreadPoolSize) - return showPopoverMessage($scope.ui, 'pools', 'rebalanceThreadPoolSize', 'Rebalance thread pool size exceed or equals System thread pool size!'); + return ErrorPopover.show('rebalanceThreadPoolSizeInput', 'Rebalance thread pool size exceed or equals System thread pool size!', $scope.ui, 'pools'); return true; } // Check cluster logical consistency. function validate(item) { - LegacyUtils.hidePopover(); + ErrorPopover.hide(); if (LegacyUtils.isEmptyString(item.name)) - return showPopoverMessage($scope.ui, 'general', 'clusterName', 'Cluster name should not be empty!'); + return ErrorPopover.show('clusterNameInput', 'Cluster name should not be empty!', $scope.ui, 'general'); if (!LegacyUtils.checkFieldValidators($scope.ui)) return false; @@ -474,6 +532,9 @@ export default ['clustersController', [ if (!checkBinaryConfiguration(item)) return false; + if (!checkCacheKeyConfiguration(item)) + return false; + if (!checkCommunicationConfiguration(item)) return false; @@ -580,12 +641,12 @@ export default ['clustersController', [ if (idx >= 0) { clusters.splice(idx, 1); + $scope.ui.inputForm.$setPristine(); + if (clusters.length > 0) $scope.selectItem(clusters[0]); - else { + else $scope.backupItem = emptyCluster; - $scope.ui.inputForm.$setPristine(); - } _.forEach($scope.caches, (cache) => _.remove(cache.cache.clusters, (id) => id === _id)); _.forEach($scope.igfss, (igfs) => _.remove(igfs.igfs.clusters, (id) => id === _id)); @@ -609,6 +670,7 @@ export default ['clustersController', [ _.forEach($scope.igfss, (igfs) => igfs.igfs.clusters = []); $scope.backupItem = emptyCluster; + $scope.ui.inputForm.$error = {}; $scope.ui.inputForm.$setPristine(); }) .error(Messages.showError); @@ -619,6 +681,7 @@ export default ['clustersController', [ Confirm.confirm('Are you sure you want to undo all changes for current cluster?') .then(function() { $scope.backupItem = $scope.selectedItem ? angular.copy($scope.selectedItem) : prepareNewItem(); + $scope.ui.inputForm.$error = {}; $scope.ui.inputForm.$setPristine(); }); }; diff --git a/modules/web-console/src/main/js/controllers/domains-controller.js b/modules/web-console/frontend/controllers/domains-controller.js similarity index 88% rename from modules/web-console/src/main/js/controllers/domains-controller.js rename to modules/web-console/frontend/controllers/domains-controller.js index cfb539d1a2140..e520494bbcf15 100644 --- a/modules/web-console/src/main/js/controllers/domains-controller.js +++ b/modules/web-console/frontend/controllers/domains-controller.js @@ -17,8 +17,8 @@ // Controller for Domain model screen. export default ['domainsController', [ - '$rootScope', '$scope', '$http', '$state', '$filter', '$timeout', '$modal', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteConfirmBatch', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteAgentMonitor', 'IgniteLegacyTable', - function($root, $scope, $http, $state, $filter, $timeout, $modal, LegacyUtils, Messages, Focus, Confirm, ConfirmBatch, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, IgniteAgentMonitor, LegacyTable) { + '$rootScope', '$scope', '$http', '$state', '$filter', '$timeout', '$modal', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteConfirmBatch', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteAgentMonitor', 'IgniteLegacyTable', 'igniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', + function($root, $scope, $http, $state, $filter, $timeout, $modal, LegacyUtils, Messages, Focus, Confirm, ConfirmBatch, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, IgniteAgentMonitor, LegacyTable, Resource, ErrorPopover, FormUtils) { UnsavedChangesGuard.install($scope); const emptyDomain = {empty: true}; @@ -30,7 +30,7 @@ export default ['domainsController', [ // We need to initialize backupItem with empty object in order to properly used from angular directives. $scope.backupItem = emptyDomain; - $scope.ui = LegacyUtils.formUI(); + $scope.ui = FormUtils.formUI(); $scope.ui.activePanels = [0, 1]; $scope.ui.topPanels = [0, 1, 2]; @@ -47,8 +47,17 @@ export default ['domainsController', [ return name ? name.replace(/[^A-Za-z_0-9/.]+/g, '_') : 'org'; }; - $scope.ui.packageNameUserInput = $scope.ui.packageName = - _toJavaPackage($root.user.email.replace('@', '.').split('.').reverse().join('.') + '.model'); + const _packageNameUpdate = (event, user) => { + if (_.isNil(user)) + return; + + $scope.ui.packageNameUserInput = _toJavaPackage(user.email.replace('@', '.').split('.').reverse().join('.') + '.model'); + }; + + _packageNameUpdate(null, $root.user); + + $scope.$on('$destroy', $root.$on('user', _packageNameUpdate)); + $scope.ui.builtinKeys = true; $scope.ui.usePrimitives = true; $scope.ui.generateAliases = true; @@ -68,9 +77,9 @@ export default ['domainsController', [ $scope.getModel = LegacyUtils.getModel; $scope.javaBuiltInClasses = LegacyUtils.javaBuiltInClasses; - $scope.compactJavaName = LegacyUtils.compactJavaName; - $scope.widthIsSufficient = LegacyUtils.widthIsSufficient; - $scope.saveBtnTipText = LegacyUtils.saveBtnTipText; + $scope.compactJavaName = FormUtils.compactJavaName; + $scope.widthIsSufficient = FormUtils.widthIsSufficient; + $scope.saveBtnTipText = FormUtils.saveBtnTipText; $scope.tableSave = function(field, index, stopEdit) { if (LegacyTable.tableEditing({model: 'table-index-fields'}, LegacyTable.tableEditedRowIndex())) { @@ -173,10 +182,6 @@ export default ['domainsController', [ 'It may be a result of import tables from database without primary keys
          ' + 'Key field for such key types should be configured manually'; - $scope.hidePopover = LegacyUtils.hidePopover; - - const showPopoverMessage = LegacyUtils.showPopoverMessage; - $scope.indexType = LegacyUtils.mkOptions(['SORTED', 'FULLTEXT', 'GEOSPATIAL']); const _dbPresets = [ @@ -325,7 +330,9 @@ export default ['domainsController', [ $scope.selectAllSchemas = function() { const allSelected = $scope.importDomain.allSchemasSelected; - _.forEach($scope.importDomain.displayedSchemas, (schema) => schema.use = allSelected); + _.forEach($scope.importDomain.displayedSchemas, (schema) => { + schema.use = allSelected; + }); }; $scope.selectSchema = function() { @@ -378,7 +385,7 @@ export default ['domainsController', [ $scope.showImportDomainModal = function() { LegacyTable.tableReset(); - LegacyUtils.confirmUnsavedChanges($scope.ui.inputForm.$dirty, function() { + FormUtils.confirmUnsavedChanges($scope.ui.inputForm.$dirty, function() { if ($scope.ui.inputForm.$dirty) $scope.backupItem = $scope.selectedItem ? angular.copy($scope.selectedItem) : prepareNewItem(); @@ -399,9 +406,9 @@ export default ['domainsController', [ $scope.importDomain.loadingOptions = LOADING_JDBC_DRIVERS; IgniteAgentMonitor.startWatch({text: 'Back to Domain models', goal: 'import domain model from database'}) - .then(function() { - importDomainModal.$promise.then(importDomainModal.show); - + .then(importDomainModal.$promise) + .then(importDomainModal.show) + .then(() => { if (demo) { $scope.ui.packageNameUserInput = $scope.ui.packageName; $scope.ui.packageName = 'model'; @@ -416,13 +423,13 @@ export default ['domainsController', [ $scope.ui.selectedJdbcDriverJar = {}; return IgniteAgentMonitor.drivers() - .then(function(drivers) { + .then((drivers) => { $scope.ui.packageName = $scope.ui.packageNameUserInput; if (drivers && drivers.length > 0) { drivers = _.sortBy(drivers, 'jdbcDriverJar'); - _.forEach(drivers, function(drv) { + _.forEach(drivers, (drv) => { $scope.jdbcDriverJars.push({ label: drv.jdbcDriverJar, value: { @@ -434,13 +441,11 @@ export default ['domainsController', [ $scope.ui.selectedJdbcDriverJar = $scope.jdbcDriverJars[0].value; - LegacyUtils.confirmUnsavedChanges($scope.ui.inputForm.$dirty, function() { - importDomainModal.$promise.then(() => { - $scope.importDomain.action = 'connect'; - $scope.importDomain.tables = []; + FormUtils.confirmUnsavedChanges($scope.ui.inputForm.$dirty, () => { + $scope.importDomain.action = 'connect'; + $scope.importDomain.tables = []; - Focus.move('jdbcUrl'); - }); + Focus.move('jdbcUrl'); }); } else { @@ -448,7 +453,7 @@ export default ['domainsController', [ $scope.importDomain.button = 'Cancel'; } }) - .finally(function() { + .then(() => { $scope.importDomain.info = INFO_CONNECT_TO_DB; Loading.finish('importDomainFromDb'); @@ -488,7 +493,7 @@ export default ['domainsController', [ $scope.importDomain.info = INFO_SELECT_SCHEMAS; }) .catch(Messages.showError) - .finally(() => Loading.finish('importDomainFromDb')); + .then(() => Loading.finish('importDomainFromDb')); } const DFLT_PARTITIONED_CACHE = { @@ -625,7 +630,7 @@ export default ['domainsController', [ $scope.importDomain.info = INFO_SELECT_TABLES; }) .catch(Messages.showError) - .finally(() => Loading.finish('importDomainFromDb')); + .then(() => Loading.finish('importDomainFromDb')); } $scope.applyDefaults = function() { @@ -725,18 +730,25 @@ export default ['domainsController', [ } function _saveDomainModel() { - if (LegacyUtils.isEmptyString($scope.ui.packageName)) - return LegacyUtils.showPopoverMessage(null, null, 'domainPackageName', 'Package could not be empty'); + if (LegacyUtils.isEmptyString($scope.ui.packageName)) { + ErrorPopover.show('domainPackageNameInput', 'Package could not be empty'); + + Focus.move('domainPackageNameInput'); + + return false; + } + + if (!LegacyUtils.isValidJavaClass('Package', $scope.ui.packageName, false, 'domainPackageNameInput', true)) { + Focus.move('domainPackageNameInput'); - if (!LegacyUtils.isValidJavaClass('Package', $scope.ui.packageName, false, 'domainPackageName', true)) return false; + } const batch = []; - const tables = []; const checkedCaches = []; - let dupCnt = 0; let containKey = true; + let containDup = false; function queryField(name, jdbcType) { return {name: toJavaName(name), className: jdbcType.javaType}; @@ -753,7 +765,7 @@ export default ['domainsController', [ }; } - _.forEach($scope.importDomain.tables, function(table) { + _.forEach($scope.importDomain.tables, function(table, curIx) { if (table.use) { const qryFields = []; const indexes = []; @@ -762,13 +774,15 @@ export default ['domainsController', [ const aliases = []; const tableName = table.tbl; + let typeName = toJavaClassName(tableName); - const dup = tables.indexOf(tableName) >= 0; + if (_.find($scope.importDomain.tables, + (tbl, ix) => tbl.use && ix !== curIx && tableName === tbl.tbl)) { + typeName = typeName + '_' + toJavaClassName(table.schema); - if (dup) - dupCnt++; + containDup = true; + } - const typeName = toJavaClassName(tableName); const valType = _toJavaPackage($scope.ui.packageName) + '.' + typeName; let _containKey = false; @@ -829,10 +843,8 @@ export default ['domainsController', [ newDomain.confirm = true; } - const dupSfx = (dup ? '_' + dupCnt : ''); - - newDomain.keyType = valType + 'Key' + dupSfx; - newDomain.valueType = valType + dupSfx; + newDomain.keyType = valType + 'Key'; + newDomain.valueType = valType; newDomain.queryMetadata = 'Configuration'; newDomain.databaseSchema = table.schema; newDomain.databaseTable = tableName; @@ -912,7 +924,6 @@ export default ['domainsController', [ } batch.push(newDomain); - tables.push(tableName); } }); @@ -941,12 +952,22 @@ export default ['domainsController', [ _saveBatch(batch); } + function checkDuplicate() { + if (containDup) { + Confirm.confirm('Some tables have the same name.
          ' + + 'Name of types for that tables will contain schema name too.') + .then(() => checkOverwrite()); + } + else + checkOverwrite(); + } + if (containKey) - checkOverwrite(); + checkDuplicate(); else { Confirm.confirm('Some tables have no primary key.
          ' + 'You will need to configure key type and key fields for such tables after import complete.') - .then(() => checkOverwrite()); + .then(() => checkDuplicate()); } } @@ -976,6 +997,12 @@ export default ['domainsController', [ if (act === 'drivers' && $scope.importDomain.jdbcDriversNotFound) return 'Resolve issue with JDBC drivers
          Close this dialog and try again'; + if (act === 'connect' && _.isNil($scope.selectedPreset.jdbcDriverClass)) + return 'Input valid JDBC driver class name'; + + if (act === 'connect' && _.isNil($scope.selectedPreset.jdbcUrl)) + return 'Input valid JDBC URL'; + if (act === 'connect' || act === 'drivers') return 'Click to load list of schemas from database'; @@ -1005,23 +1032,19 @@ export default ['domainsController', [ }; $scope.importDomainNextAvailable = function() { - let res = true; - switch ($scope.importDomain.action) { - case 'schemas': - res = _.isEmpty($scope.importDomain.schemas) || _.find($scope.importDomain.schemas, {use: true}); + case 'connect': + return !_.isNil($scope.selectedPreset.jdbcDriverClass) && !_.isNil($scope.selectedPreset.jdbcUrl); - break; + case 'schemas': + return _.isEmpty($scope.importDomain.schemas) || _.find($scope.importDomain.schemas, {use: true}); case 'tables': - res = _.find($scope.importDomain.tables, {use: true}); - - break; + return _.find($scope.importDomain.tables, {use: true}); default: + return true; } - - return res; }; $scope.importDomainPrev = function() { @@ -1061,21 +1084,17 @@ export default ['domainsController', [ // When landing on the page, get domain models and show them. Loading.start('loadingDomainModelsScreen'); - $http.post('/api/v1/configuration/domains/list') - .success(function(data) { - $scope.spaces = data.spaces; - $scope.clusters = _.map(data.clusters, function(cluster) { - return { - value: cluster._id, - label: cluster.name - }; - }); - $scope.caches = _mapCaches(data.caches); - $scope.domains = data.domains; + Resource.read() + .then(({spaces, clusters, caches, domains}) => { + $scope.spaces = spaces; + $scope.clusters = _.map(clusters, (cluster) => ({ + label: cluster.name, + value: cluster._id + })); + $scope.caches = _mapCaches(caches); + $scope.domains = _.sortBy(domains, 'valueType'); - _.forEach($scope.clusters, function(cluster) { - $scope.ui.generatedCachesClusters.push(cluster.value); - }); + _.forEach($scope.clusters, (cluster) => $scope.ui.generatedCachesClusters.push(cluster.value)); if (!_.isEmpty($scope.caches)) { $scope.importActions.push({ @@ -1117,23 +1136,33 @@ export default ['domainsController', [ }); $scope.$watch('backupItem', function(val) { + if (!$scope.ui.inputForm) + return; + const form = $scope.ui.inputForm; - if (form.$pristine || (form.$valid && ModelNormalizer.isEqual(__original_value, val))) + if (form.$valid && ModelNormalizer.isEqual(__original_value, val)) form.$setPristine(); else form.$setDirty(); }, true); + + $scope.$watch('ui.activePanels.length', () => { + ErrorPopover.hide(); + }); }) .catch(Messages.showError) - .finally(() => { + .then(() => { $scope.ui.ready = true; - $scope.ui.inputForm.$setPristine(); + $scope.ui.inputForm && $scope.ui.inputForm.$setPristine(); Loading.finish('loadingDomainModelsScreen'); }); const clearFormDefaults = (ngFormCtrl) => { + if (!ngFormCtrl) + return; + ngFormCtrl.$defaults = {}; _.forOwn(ngFormCtrl, (value, key) => { @@ -1169,6 +1198,11 @@ export default ['domainsController', [ $scope.backupItem = angular.merge({}, blank, $scope.backupItem); + if ($scope.ui.inputForm) { + $scope.ui.inputForm.$error = {}; + $scope.ui.inputForm.$setPristine(); + } + __original_value = ModelNormalizer.normalize($scope.backupItem); if (LegacyUtils.isDefined($scope.backupItem) && !LegacyUtils.isDefined($scope.backupItem.queryMetadata)) @@ -1181,15 +1215,15 @@ export default ['domainsController', [ $state.go('base.configuration.domains'); } - LegacyUtils.confirmUnsavedChanges($scope.ui.inputForm.$dirty, selectItem); + FormUtils.confirmUnsavedChanges($scope.backupItem && $scope.ui.inputForm && $scope.ui.inputForm.$dirty, selectItem); }; // Add new domain model. $scope.createItem = function(cacheId) { if ($scope.tableReset(true)) { $timeout(() => { - LegacyUtils.ensureActivePanel($scope.ui, 'query'); - LegacyUtils.ensureActivePanel($scope.ui, 'general', 'keyType'); + FormUtils.ensureActivePanel($scope.ui, 'query'); + FormUtils.ensureActivePanel($scope.ui, 'general', 'keyTypeInput'); }); $scope.selectItem(null, prepareNewItem(cacheId)); @@ -1199,14 +1233,14 @@ export default ['domainsController', [ function checkQueryConfiguration(item) { if (item.queryMetadata === 'Configuration' && LegacyUtils.domainForQueryConfigured(item)) { if (_.isEmpty(item.fields)) - return showPopoverMessage($scope.ui, 'query', 'queryFields', 'Query fields should not be empty'); + return ErrorPopover.show('queryFields', 'Query fields should not be empty', $scope.ui, 'query'); const indexes = item.indexes; if (indexes && indexes.length > 0) { if (_.find(indexes, function(index, i) { if (_.isEmpty(index.fields)) - return !showPopoverMessage($scope.ui, 'query', 'indexes' + i, 'Index fields are not specified'); + return !ErrorPopover.show('indexes' + i, 'Index fields are not specified', $scope.ui, 'query'); })) return false; } @@ -1218,19 +1252,19 @@ export default ['domainsController', [ function checkStoreConfiguration(item) { if (LegacyUtils.domainForStoreConfigured(item)) { if (LegacyUtils.isEmptyString(item.databaseSchema)) - return showPopoverMessage($scope.ui, 'store', 'databaseSchema', 'Database schema should not be empty'); + return ErrorPopover.show('databaseSchemaInput', 'Database schema should not be empty', $scope.ui, 'store'); if (LegacyUtils.isEmptyString(item.databaseTable)) - return showPopoverMessage($scope.ui, 'store', 'databaseTable', 'Database table should not be empty'); + return ErrorPopover.show('databaseTableInput', 'Database table should not be empty', $scope.ui, 'store'); if (_.isEmpty(item.keyFields)) - return showPopoverMessage($scope.ui, 'store', 'keyFields', 'Key fields are not specified'); + return ErrorPopover.show('keyFields', 'Key fields are not specified', $scope.ui, 'store'); if (LegacyUtils.isJavaBuiltInClass(item.keyType) && item.keyFields.length !== 1) - return showPopoverMessage($scope.ui, 'store', 'keyFields', 'Only one field should be specified in case when key type is a Java built-in type'); + return ErrorPopover.show('keyFields', 'Only one field should be specified in case when key type is a Java built-in type', $scope.ui, 'store'); if (_.isEmpty(item.valueFields)) - return showPopoverMessage($scope.ui, 'store', 'valueFields', 'Value fields are not specified'); + return ErrorPopover.show('valueFields', 'Value fields are not specified', $scope.ui, 'store'); } return true; @@ -1248,7 +1282,7 @@ export default ['domainsController', [ return false; if (!LegacyUtils.domainForStoreConfigured(item) && !LegacyUtils.domainForQueryConfigured(item) && item.queryMetadata === 'Configuration') - return showPopoverMessage($scope.ui, 'query', 'query-title', 'SQL query domain model should be configured'); + return ErrorPopover.show('query-title', 'SQL query domain model should be configured', $scope.ui, 'query'); return true; } @@ -1332,7 +1366,7 @@ export default ['domainsController', [ } function _newNameIsValidJavaClass(newName) { - return LegacyUtils.isValidJavaClass('New name for value type', newName, false, 'copy-new-name'); + return LegacyUtils.isValidJavaClass('New name for value type', newName, false, 'copy-new-nameInput'); } // Save domain model with new name. @@ -1372,12 +1406,12 @@ export default ['domainsController', [ if (idx >= 0) { domains.splice(idx, 1); + $scope.ui.inputForm.$setPristine(); + if (domains.length > 0) $scope.selectItem(domains[0]); - else { + else $scope.backupItem = emptyDomain; - $scope.ui.inputForm.$setPristine(); - } _.forEach($scope.caches, (cache) => _.remove(cache.cache.domains, (id) => id === _id)); } @@ -1402,9 +1436,10 @@ export default ['domainsController', [ _.forEach($scope.caches, (cache) => cache.cache.domains = []); - $scope.ui.inputForm.$setPristine(); $scope.backupItem = emptyDomain; $scope.ui.showValid = true; + $scope.ui.inputForm.$error = {}; + $scope.ui.inputForm.$setPristine(); }) .error(Messages.showError); }); @@ -1440,7 +1475,7 @@ export default ['domainsController', [ aliases: {id: 'Alias', idPrefix: 'Value', searchCol: 'alias', valueCol: 'value', dupObjName: 'alias'} }; - $scope.tablePairValid = function(item, field, index) { + $scope.tablePairValid = function(item, field, index, stopEdit) { const pairField = pairFields[field.model]; const pairValue = LegacyTable.tablePairValue(field, index); @@ -1454,12 +1489,20 @@ export default ['domainsController', [ }); // Found duplicate by key. - if (idx >= 0 && idx !== index) - return showPopoverMessage($scope.ui, 'query', LegacyTable.tableFieldId(index, pairField.idPrefix + pairField.id), 'Field with such ' + pairField.dupObjName + ' already exists!'); + if (idx >= 0 && idx !== index) { + if (stopEdit) + return false; + + return ErrorPopover.show(LegacyTable.tableFieldId(index, pairField.idPrefix + pairField.id), 'Field with such ' + pairField.dupObjName + ' already exists!', $scope.ui, 'query'); + } } - if (pairField.classValidation && !LegacyUtils.isValidJavaClass(pairField.msg, pairValue.value, true, LegacyTable.tableFieldId(index, 'Value' + pairField.id), false, $scope.ui, 'query')) + if (pairField.classValidation && !LegacyUtils.isValidJavaClass(pairField.msg, pairValue.value, true, LegacyTable.tableFieldId(index, 'Value' + pairField.id), false, $scope.ui, 'query')) { + if (stopEdit) + return false; + return LegacyTable.tableFocusInvalidField(index, 'Value' + pairField.id); + } } return true; @@ -1513,7 +1556,7 @@ export default ['domainsController', [ // Found duplicate. if (idx >= 0 && index !== idx) - return showPopoverMessage($scope.ui, 'store', LegacyTable.tableFieldId(index, 'DatabaseFieldName' + dbFieldTable.id), 'Field with such database name already exists!'); + return ErrorPopover.show(LegacyTable.tableFieldId(index, 'DatabaseFieldName' + dbFieldTable.id), 'Field with such database name already exists!', $scope.ui, 'store'); idx = _.findIndex(model, function(dbMeta) { return dbMeta.javaFieldName === dbFieldValue.javaFieldName; @@ -1521,7 +1564,7 @@ export default ['domainsController', [ // Found duplicate. if (idx >= 0 && index !== idx) - return showPopoverMessage($scope.ui, 'store', LegacyTable.tableFieldId(index, 'JavaFieldName' + dbFieldTable.id), 'Field with such java name already exists!'); + return ErrorPopover.show(LegacyTable.tableFieldId(index, 'JavaFieldName' + dbFieldTable.id), 'Field with such java name already exists!', $scope.ui, 'store'); if (index < 0) model.push(dbFieldValue); @@ -1582,7 +1625,7 @@ export default ['domainsController', [ // Found duplicate. if (idx >= 0 && idx !== curIdx) - return showPopoverMessage($scope.ui, 'query', LegacyTable.tableFieldId(curIdx, 'IndexName'), 'Index with such name already exists!'); + return ErrorPopover.show(LegacyTable.tableFieldId(curIdx, 'IndexName'), 'Index with such name already exists!', $scope.ui, 'query'); } LegacyTable.tableReset(); @@ -1697,7 +1740,7 @@ export default ['domainsController', [ // Found duplicate. if (idx >= 0 && idx !== curIdx) - return showPopoverMessage($scope.ui, 'query', LegacyTable.tableFieldId(curIdx, 'FieldName' + (index.indexType === 'SORTED' ? 'S' : '') + indexIdx + (curIdx >= 0 ? '-' : '')), 'Field with such name already exists in index!'); + return ErrorPopover.show(LegacyTable.tableFieldId(curIdx, 'FieldName' + (index.indexType === 'SORTED' ? 'S' : '') + indexIdx + (curIdx >= 0 ? '-' : '')), 'Field with such name already exists in index!', $scope.ui, 'query'); } LegacyTable.tableReset(); @@ -1739,6 +1782,7 @@ export default ['domainsController', [ Confirm.confirm('Are you sure you want to undo all changes for current domain model?') .then(function() { $scope.backupItem = $scope.selectedItem ? angular.copy($scope.selectedItem) : prepareNewItem(); + $scope.ui.inputForm.$error = {}; $scope.ui.inputForm.$setPristine(); }); }; diff --git a/modules/web-console/src/main/js/controllers/igfs-controller.js b/modules/web-console/frontend/controllers/igfs-controller.js similarity index 83% rename from modules/web-console/src/main/js/controllers/igfs-controller.js rename to modules/web-console/frontend/controllers/igfs-controller.js index 5f5f0bb2c7c2f..761771207dc32 100644 --- a/modules/web-console/src/main/js/controllers/igfs-controller.js +++ b/modules/web-console/frontend/controllers/igfs-controller.js @@ -17,8 +17,8 @@ // Controller for IGFS screen. export default ['igfsController', [ - '$scope', '$http', '$state', '$filter', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteLegacyTable', - function($scope, $http, $state, $filter, $timeout, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, LegacyTable) { + '$scope', '$http', '$state', '$filter', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteLegacyTable', 'igniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', + function($scope, $http, $state, $filter, $timeout, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, LegacyTable, Resource, ErrorPopover, FormUtils) { UnsavedChangesGuard.install($scope); const emptyIgfs = {empty: true}; @@ -33,15 +33,13 @@ export default ['igfsController', [ // We need to initialize backupItem with empty object in order to properly used from angular directives. $scope.backupItem = emptyIgfs; - $scope.ui = LegacyUtils.formUI(); + $scope.ui = FormUtils.formUI(); $scope.ui.activePanels = [0]; $scope.ui.topPanels = [0]; - $scope.compactJavaName = LegacyUtils.compactJavaName; - $scope.widthIsSufficient = LegacyUtils.widthIsSufficient; - $scope.saveBtnTipText = LegacyUtils.saveBtnTipText; - - const showPopoverMessage = LegacyUtils.showPopoverMessage; + $scope.compactJavaName = FormUtils.compactJavaName; + $scope.widthIsSufficient = FormUtils.widthIsSufficient; + $scope.saveBtnTipText = FormUtils.saveBtnTipText; $scope.tableSave = function(field, index, stopEdit) { if (field.type === 'pathModes' && LegacyTable.tablePairSaveVisible(field, index)) @@ -82,7 +80,7 @@ export default ['igfsController', [ LegacyTable.tableRemove(item, field, index); }; - $scope.tablePairValid = function(item, field, index) { + $scope.tablePairValid = function(item, field, index, stopEdit) { const pairValue = LegacyTable.tablePairValue(field, index); const model = item[field.model]; @@ -93,8 +91,12 @@ export default ['igfsController', [ }); // Found duplicate. - if (idx >= 0 && idx !== index) - return showPopoverMessage($scope.ui, 'misc', LegacyTable.tableFieldId(index, 'KeyPathMode'), 'Such path already exists!'); + if (idx >= 0 && idx !== index) { + if (stopEdit) + return false; + + return ErrorPopover.show(LegacyTable.tableFieldId(index, 'KeyPathMode'), 'Such path already exists!', $scope.ui, 'misc'); + } } return true; @@ -121,7 +123,7 @@ export default ['igfsController', [ $scope.toggleExpanded = function() { $scope.ui.expanded = !$scope.ui.expanded; - LegacyUtils.hidePopover(); + ErrorPopover.hide(); }; $scope.igfss = []; @@ -135,11 +137,11 @@ export default ['igfsController', [ Loading.start('loadingIgfsScreen'); // When landing on the page, get IGFSs and show them. - $http.post('/api/v1/configuration/igfs/list') - .success(function(data) { - $scope.spaces = data.spaces; + Resource.read() + .then(({spaces, clusters, igfss}) => { + $scope.spaces = spaces; - $scope.igfss = data.igfss || []; + $scope.igfss = igfss || []; // For backward compatibility set colocateMetadata and relaxedConsistency default values. _.forEach($scope.igfss, (igfs) => { @@ -150,12 +152,10 @@ export default ['igfsController', [ igfs.relaxedConsistency = true; }); - $scope.clusters = _.map(data.clusters || [], function(cluster) { - return { - value: cluster._id, - label: cluster.name - }; - }); + $scope.clusters = _.map(clusters || [], (cluster) => ({ + label: cluster.name, + value: cluster._id + })); if ($state.params.linkId) $scope.createItem($state.params.linkId); @@ -185,18 +185,26 @@ export default ['igfsController', [ }); $scope.$watch('backupItem', function(val) { + if (!$scope.ui.inputForm) + return; + const form = $scope.ui.inputForm; - if (form.$pristine || (form.$valid && ModelNormalizer.isEqual(__original_value, val))) + if (form.$valid && ModelNormalizer.isEqual(__original_value, val)) form.$setPristine(); else form.$setDirty(); }, true); + + $scope.$watch('ui.activePanels.length', () => { + ErrorPopover.hide(); + }); }) .catch(Messages.showError) - .finally(function() { + .then(() => { $scope.ui.ready = true; - $scope.ui.inputForm.$setPristine(); + $scope.ui.inputForm && $scope.ui.inputForm.$setPristine(); + Loading.finish('loadingIgfsScreen'); }); @@ -225,13 +233,18 @@ export default ['igfsController', [ $scope.backupItem = angular.merge({}, blank, $scope.backupItem); + if ($scope.ui.inputForm) { + $scope.ui.inputForm.$error = {}; + $scope.ui.inputForm.$setPristine(); + } + __original_value = ModelNormalizer.normalize($scope.backupItem); if (LegacyUtils.getQueryVariable('new')) $state.go('base.configuration.igfs'); } - LegacyUtils.confirmUnsavedChanges($scope.backupItem && $scope.ui.inputForm.$dirty, selectItem); + FormUtils.confirmUnsavedChanges($scope.backupItem && $scope.ui.inputForm && $scope.ui.inputForm.$dirty, selectItem); }; $scope.linkId = () => $scope.backupItem._id ? $scope.backupItem._id : 'create'; @@ -251,7 +264,7 @@ export default ['igfsController', [ // Add new IGFS. $scope.createItem = function(linkId) { if ($scope.tableReset(true)) { - $timeout(() => LegacyUtils.ensureActivePanel($scope.ui, 'general', 'igfsName')); + $timeout(() => FormUtils.ensureActivePanel($scope.ui, 'general', 'igfsNameInput')); $scope.selectItem(null, prepareNewItem(linkId)); } @@ -259,21 +272,21 @@ export default ['igfsController', [ // Check IGFS logical consistency. function validate(item) { - LegacyUtils.hidePopover(); + ErrorPopover.hide(); if (LegacyUtils.isEmptyString(item.name)) - return showPopoverMessage($scope.ui, 'general', 'igfsName', 'IGFS name should not be empty!'); + return ErrorPopover.show('igfsNameInput', 'IGFS name should not be empty!', $scope.ui, 'general'); if (!LegacyUtils.checkFieldValidators($scope.ui)) return false; if (!item.secondaryFileSystemEnabled && (item.defaultMode === 'PROXY')) - return showPopoverMessage($scope.ui, 'secondaryFileSystem', 'secondaryFileSystem-title', 'Secondary file system should be configured for "PROXY" IGFS mode!'); + return ErrorPopover.show('secondaryFileSystem-title', 'Secondary file system should be configured for "PROXY" IGFS mode!', $scope.ui, 'secondaryFileSystem'); if (item.pathModes) { for (let pathIx = 0; pathIx < item.pathModes.length; pathIx++) { if (!item.secondaryFileSystemEnabled && item.pathModes[pathIx].mode === 'PROXY') - return showPopoverMessage($scope.ui, 'secondaryFileSystem', 'secondaryFileSystem-title', 'Secondary file system should be configured for "PROXY" path mode!'); + return ErrorPopover.show('secondaryFileSystem-title', 'Secondary file system should be configured for "PROXY" path mode!', $scope.ui, 'secondaryFileSystem'); } } @@ -358,12 +371,12 @@ export default ['igfsController', [ if (idx >= 0) { igfss.splice(idx, 1); + $scope.ui.inputForm.$setPristine(); + if (igfss.length > 0) $scope.selectItem(igfss[0]); - else { + else $scope.backupItem = emptyIgfs; - $scope.ui.inputForm.$setPristine(); - } } }) .error(Messages.showError); @@ -382,6 +395,7 @@ export default ['igfsController', [ $scope.igfss = []; $scope.backupItem = emptyIgfs; + $scope.ui.inputForm.$error = {}; $scope.ui.inputForm.$setPristine(); }) .error(Messages.showError); @@ -394,6 +408,7 @@ export default ['igfsController', [ Confirm.confirm('Are you sure you want to undo all changes for current IGFS?') .then(function() { $scope.backupItem = $scope.selectedItem ? angular.copy($scope.selectedItem) : prepareNewItem(); + $scope.ui.inputForm.$error = {}; $scope.ui.inputForm.$setPristine(); }); }; diff --git a/modules/web-console/src/main/js/controllers/profile-controller.js b/modules/web-console/frontend/controllers/profile-controller.js similarity index 93% rename from modules/web-console/src/main/js/controllers/profile-controller.js rename to modules/web-console/frontend/controllers/profile-controller.js index 6b8f541af9c56..9499db5e96f0a 100644 --- a/modules/web-console/src/main/js/controllers/profile-controller.js +++ b/modules/web-console/frontend/controllers/profile-controller.js @@ -73,6 +73,7 @@ export default ['profileController', [ $scope.saveUser = () => { $http.post('/api/v1/profile/save', $scope.user) + .catch(({data}) => Promise.reject(data)) .then(User.read) .then(() => { if ($scope.expandedPassword) @@ -84,8 +85,10 @@ export default ['profileController', [ Messages.showInfo('Profile saved.'); Focus.move('profile-username'); + + $root.$broadcast('user', $scope.user); }) - .catch(({data}) => Messages.showError(Messages.errorMessage('Failed to save profile: ', data))); + .catch((err) => Messages.showError(Messages.errorMessage('Failed to save profile: ', err))); }; } ]]; diff --git a/modules/web-console/src/main/js/generator/generator-common.js b/modules/web-console/frontend/generator/generator-common.js similarity index 99% rename from modules/web-console/src/main/js/generator/generator-common.js rename to modules/web-console/frontend/generator/generator-common.js index 10bd299848783..124a1b73c0c47 100644 --- a/modules/web-console/src/main/js/generator/generator-common.js +++ b/modules/web-console/frontend/generator/generator-common.js @@ -484,7 +484,8 @@ $generatorCommon.IGFS_IPC_CONFIGURATION = { host: {dflt: '127.0.0.1'}, port: {dflt: 10500}, memorySize: {dflt: 262144}, - tokenDirectoryPath: {dflt: 'ipc/shmem'} + tokenDirectoryPath: {dflt: 'ipc/shmem'}, + threadCount: null } }; diff --git a/modules/web-console/src/main/js/generator/generator-java.js b/modules/web-console/frontend/generator/generator-java.js similarity index 96% rename from modules/web-console/src/main/js/generator/generator-java.js rename to modules/web-console/frontend/generator/generator-java.js index d9425e0b7f191..6cc3323b6580a 100644 --- a/modules/web-console/src/main/js/generator/generator-java.js +++ b/modules/web-console/frontend/generator/generator-java.js @@ -286,9 +286,11 @@ $generatorJava.listProperty = function(res, varName, obj, propName, dataType, se * @param args Array with arguments. * @param startBlock Optional start block string. * @param endBlock Optional end block string. + * @param startQuote Start quote string. + * @param endQuote End quote string. */ -$generatorJava.fxVarArgs = function(res, fx, quote, args, startBlock = '(', endBlock = ')') { - const quoteArg = (arg) => quote ? '"' + arg + '"' : arg; +$generatorJava.fxVarArgs = function(res, fx, quote, args, startBlock = '(', endBlock = ')', startQuote = '"', endQuote = '"') { + const quoteArg = (arg) => quote ? startQuote + arg + endQuote : arg; if (args.length === 1) res.append(fx + startBlock + quoteArg(args[0]) + endBlock + ';'); @@ -539,8 +541,20 @@ $generatorJava.clusterGeneral = function(cluster, clientNearCfg, res) { break; case 'Jdbc': - $generatorJava.beanProperty(res, 'discovery', d.Jdbc, 'ipFinder', 'ipFinder', - 'org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder', {initSchema: null}, true); + $generatorJava.declareVariable(res, 'ipFinder', + 'org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder'); + $generatorJava.property(res, 'ipFinder', d.Jdbc, 'initSchema'); + + const datasource = d.Jdbc; + if (datasource.dataSourceBean && datasource.dialect) { + res.needEmptyLine = !datasource.initSchema; + + res.line('ipFinder.setDataSource(DataSources.INSTANCE_' + datasource.dataSourceBean + ');'); + } + + res.needEmptyLine = true; + + res.line('discovery.setIpFinder(ipFinder);'); break; @@ -729,6 +743,35 @@ $generatorJava.clusterBinary = function(binary, res) { return res; }; +// Generate cache key configurations. +$generatorJava.clusterCacheKeyConfiguration = function(keyCfgs, res) { + if (!res) + res = $generatorCommon.builder(); + + keyCfgs = _.filter(keyCfgs, (cfg) => cfg.typeName && cfg.affinityKeyFieldName); + + if (_.isEmpty(keyCfgs)) + return res; + + $generatorJava.declareVariableArray(res, 'keyConfigurations', 'org.apache.ignite.cache.CacheKeyConfiguration', keyCfgs.length); + + const cacheKeyCfg = res.importClass('org.apache.ignite.cache.CacheKeyConfiguration'); + + _.forEach(keyCfgs, (cfg, idx) => { + res.needEmptyLine = true; + + res.line(`keyConfigurations[${idx}] = new ${cacheKeyCfg}("${cfg.typeName}", "${cfg.affinityKeyFieldName}");`); + + res.needEmptyLine = true; + }); + + res.line('cfg.setCacheKeyConfiguration(keyConfigurations);'); + + res.needEmptyLine = true; + + return res; +}; + // TODO IGNITE-2269 Remove specified methods after implamentation of extended constructors. // Construct binary type configuration factory method name. $generatorJava.binaryTypeFunctionName = function(typeName) { @@ -1341,7 +1384,9 @@ $generatorJava.cacheMemory = function(cache, varName, res) { varName = $generatorJava.nextVariableName('cache', cache); $generatorJava.enumProperty(res, varName, cache, 'memoryMode', 'org.apache.ignite.cache.CacheMemoryMode', null, 'ONHEAP_TIERED'); - $generatorJava.property(res, varName, cache, 'offHeapMaxMemory', null, null, -1); + + if (cache.memoryMode !== 'OFFHEAP_VALUES') + $generatorJava.property(res, varName, cache, 'offHeapMaxMemory', null, null, -1); res.softEmptyLine(); @@ -1356,7 +1401,7 @@ $generatorJava.cacheMemory = function(cache, varName, res) { }; // Generate cache query & indexing group. -$generatorJava.cacheQuery = function(cache, varName, res) { +$generatorJava.cacheQuery = function(cache, domains, varName, res) { if (!res) res = $generatorCommon.builder(); @@ -1367,6 +1412,21 @@ $generatorJava.cacheQuery = function(cache, varName, res) { $generatorJava.property(res, varName, cache, 'sqlOnheapRowCacheSize', null, null, 10240); $generatorJava.property(res, varName, cache, 'longQueryWarningTimeout', null, null, 3000); + const indexedTypes = _.reduce(domains, (acc, domain) => { + if (domain.queryMetadata === 'Annotations') { + acc.push(domain.keyType); + acc.push(domain.valueType); + } + + return acc; + }, []); + + if (indexedTypes.length > 0) { + res.softEmptyLine(); + + $generatorJava.multiparamProperty(res, varName, {indexedTypes}, 'indexedTypes', 'class'); + } + res.softEmptyLine(); $generatorJava.multiparamProperty(res, varName, cache, 'sqlFunctionClasses', 'class'); @@ -1456,7 +1516,7 @@ $generatorJava.cacheStoreDataSource = function(storeFactory, res) { return null; }; -$generatorJava.clusterDataSources = function(caches, res) { +$generatorJava.clusterDataSources = function(cluster, res) { if (!res) res = $generatorCommon.builder(); @@ -1464,7 +1524,16 @@ $generatorJava.clusterDataSources = function(caches, res) { let storeFound = false; - _.forEach(caches, function(cache) { + function startSourcesFunction() { + if (!storeFound) { + res.line('/** Helper class for datasource creation. */'); + res.startBlock('public static class DataSources {'); + + storeFound = true; + } + } + + _.forEach(cluster.caches, function(cache) { const factoryKind = cache.cacheStoreFactory.kind; const storeFactory = cache.cacheStoreFactory[factoryKind]; @@ -1476,12 +1545,7 @@ $generatorJava.clusterDataSources = function(caches, res) { datasources.push(beanClassName); if (factoryKind === 'CacheJdbcPojoStoreFactory' || factoryKind === 'CacheJdbcBlobStoreFactory') { - if (!storeFound) { - res.line('/** Helper class for datasource creation. */'); - res.startBlock('public static class DataSources {'); - - storeFound = true; - } + startSourcesFunction(); $generatorJava.cacheStoreDataSource(storeFactory, res); } @@ -1489,6 +1553,20 @@ $generatorJava.clusterDataSources = function(caches, res) { } }); + if (cluster.discovery.kind === 'Jdbc') { + const datasource = cluster.discovery.Jdbc; + + if (datasource.dataSourceBean && datasource.dialect) { + const beanClassName = $generatorJava.dataSourceClassName(res, datasource); + + if (beanClassName && !_.includes(datasources, beanClassName)) { + startSourcesFunction(); + + $generatorJava.cacheStoreDataSource(datasource, res); + } + } + } + if (storeFound) res.endBlock('}'); @@ -1637,6 +1715,52 @@ $generatorJava.cacheStore = function(cache, domains, cacheVarName, res) { return res; }; +// Generate cache node filter group. +$generatorJava.cacheNodeFilter = function(cache, igfss, varName, res) { + if (!res) + res = $generatorCommon.builder(); + + if (!varName) + varName = $generatorJava.nextVariableName('cache', cache); + + switch (_.get(cache, 'nodeFilter.kind')) { + case 'IGFS': + const foundIgfs = _.find(igfss, (igfs) => igfs._id === cache.nodeFilter.IGFS.igfs); + + if (foundIgfs) { + const predClsName = res.importClass('org.apache.ignite.internal.processors.igfs.IgfsNodePredicate'); + + res.line(`${varName}.setNodeFilter(new ${predClsName}("${foundIgfs.name}"));`); + } + + break; + + case 'OnNodes': + const nodes = cache.nodeFilter.OnNodes.nodeIds; + + if ($generatorCommon.isDefinedAndNotEmpty(nodes)) { + const startQuote = res.importClass('java.util.UUID') + '.fromString("'; + + $generatorJava.fxVarArgs(res, varName + '.setNodeFilter(new ' + + res.importClass('org.apache.ignite.internal.util.lang.GridNodePredicate'), true, nodes, '(', '))', + startQuote, '")'); + } + + break; + + case 'Custom': + res.line(varName + '.setNodeFilter(new ' + res.importClass(cache.nodeFilter.Custom.className) + '());'); + + break; + + default: break; + } + + res.needEmptyLine = true; + + return res; +}; + // Generate cache concurrency group. $generatorJava.cacheConcurrency = function(cache, varName, res) { if (!res) @@ -1648,7 +1772,7 @@ $generatorJava.cacheConcurrency = function(cache, varName, res) { $generatorJava.property(res, varName, cache, 'maxConcurrentAsyncOperations', null, null, 500); $generatorJava.property(res, varName, cache, 'defaultLockTimeout', null, null, 0); $generatorJava.enumProperty(res, varName, cache, 'atomicWriteOrderMode', 'org.apache.ignite.cache.CacheAtomicWriteOrderMode'); - $generatorJava.enumProperty(res, varName, cache, 'writeSynchronizationMode', 'org.apache.ignite.cache.CacheWriteSynchronizationMode', null, null, 'PRIMARY_SYNC'); + $generatorJava.enumProperty(res, varName, cache, 'writeSynchronizationMode', 'org.apache.ignite.cache.CacheWriteSynchronizationMode', null, 'PRIMARY_SYNC'); res.needEmptyLine = true; @@ -1980,8 +2104,12 @@ $generatorJava.cache = function(cache, varName, res) { $generatorJava.cacheGeneral(cache, varName, res); $generatorJava.cacheMemory(cache, varName, res); - $generatorJava.cacheQuery(cache, varName, res); + $generatorJava.cacheQuery(cache, cache.domains, varName, res); $generatorJava.cacheStore(cache, cache.domains, varName, res); + + const igfs = _.get(cache, 'nodeFilter.IGFS.instance'); + + $generatorJava.cacheNodeFilter(cache, igfs ? [igfs] : [], varName, res); $generatorJava.cacheConcurrency(cache, varName, res); $generatorJava.cacheRebalance(cache, varName, res); $generatorJava.cacheServerNearCache(cache, varName, res); @@ -2728,6 +2856,8 @@ $generatorJava.clusterConfiguration = function(cluster, clientNearCfg, res) { $generatorJava.clusterBinary(cluster.binaryConfiguration, res); + $generatorJava.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, res); + $generatorJava.clusterCollision(cluster.collision, res); $generatorJava.clusterCommunication(cluster, res); @@ -2814,7 +2944,7 @@ $generatorJava.cluster = function(cluster, pkg, javaClass, clientNearCfg) { $generatorJava.tryLoadSecretProperties(cluster, res); - $generatorJava.clusterDataSources(cluster.caches, res); + $generatorJava.clusterDataSources(cluster, res); res.line('/**'); res.line(' * Configure grid.'); diff --git a/modules/web-console/src/main/js/generator/generator-optional.js b/modules/web-console/frontend/generator/generator-optional.js similarity index 100% rename from modules/web-console/src/main/js/generator/generator-optional.js rename to modules/web-console/frontend/generator/generator-optional.js diff --git a/modules/web-console/src/main/js/generator/generator-properties.js b/modules/web-console/frontend/generator/generator-properties.js similarity index 62% rename from modules/web-console/src/main/js/generator/generator-properties.js rename to modules/web-console/frontend/generator/generator-properties.js index 4773f22aa0af9..9d1e7e32c2285 100644 --- a/modules/web-console/src/main/js/generator/generator-properties.js +++ b/modules/web-console/frontend/generator/generator-properties.js @@ -38,63 +38,80 @@ $generatorProperties.jdbcUrlTemplate = function(dialect) { return 'jdbc:your_database'; }; +$generatorProperties.createBuilder = function() { + const res = $generatorCommon.builder(); + + res.line('# ' + $generatorCommon.mainComment()); + + return res; +}; + /** * Generate properties file with properties stubs for stores data sources. * - * @param cluster Configuration to process. * @param res Resulting output with generated properties. + * @param datasources Already added datasources. + * @param storeFactory Current datasource factory. + * @param dialect Current dialect. * @returns {string} Generated content. */ -$generatorProperties.dataSourcesProperties = function(cluster, res) { - const datasources = []; +$generatorProperties.dataSourceProperties = function(res, datasources, storeFactory, dialect) { + const beanId = storeFactory.dataSourceBean; - if (cluster.caches && cluster.caches.length > 0) { - _.forEach(cluster.caches, function(cache) { - if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) { - const storeFactory = cache.cacheStoreFactory[cache.cacheStoreFactory.kind]; + const dsClsName = $generatorCommon.dataSourceClassName(dialect); - const dialect = storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : null) : storeFactory.dialect; + const varType = res.importClass(dsClsName); - const connectViaUrl = cache.cacheStoreFactory.kind === 'CacheJdbcBlobStoreFactory' && storeFactory.connectVia === 'URL'; + const beanClassName = $generatorCommon.toJavaName(varType, storeFactory.dataSourceBean); - if (!res && (dialect || connectViaUrl)) { - res = $generatorCommon.builder(); + if (!_.includes(datasources, beanClassName)) { + datasources.push(beanClassName); - res.line('# ' + $generatorCommon.mainComment()); - } + res.needEmptyLine = true; - if (dialect) { - const beanId = storeFactory.dataSourceBean; + switch (dialect) { + case 'DB2': + res.line(beanId + '.jdbc.server_name=YOUR_DATABASE_SERVER_NAME'); + res.line(beanId + '.jdbc.port_number=YOUR_JDBC_PORT_NUMBER'); + res.line(beanId + '.jdbc.driver_type=YOUR_JDBC_DRIVER_TYPE'); + res.line(beanId + '.jdbc.database_name=YOUR_DATABASE_NAME'); - const dsClsName = $generatorCommon.dataSourceClassName(dialect); + break; - const varType = res.importClass(dsClsName); + default: + res.line(beanId + '.jdbc.url=' + $generatorProperties.jdbcUrlTemplate(dialect)); + } - const beanClassName = $generatorCommon.toJavaName(varType, storeFactory.dataSourceBean); + res.line(beanId + '.jdbc.username=YOUR_USER_NAME'); + res.line(beanId + '.jdbc.password=YOUR_PASSWORD'); + res.line(''); + } +}; - if (!_.includes(datasources, beanClassName)) { - datasources.push(beanClassName); +/** + * Generate properties file with properties stubs for stores data sources. + * + * @param cluster Configuration to process. + * @param res Resulting output with generated properties. + * @returns {string} Generated content. + */ +$generatorProperties.dataSourcesProperties = function(cluster, res) { + const datasources = []; - res.needEmptyLine = true; + if (cluster.caches && cluster.caches.length > 0) { + _.forEach(cluster.caches, function(cache) { + if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) { + const storeFactory = cache.cacheStoreFactory[cache.cacheStoreFactory.kind]; - switch (dialect) { - case 'DB2': - res.line(beanId + '.jdbc.server_name=YOUR_DATABASE_SERVER_NAME'); - res.line(beanId + '.jdbc.port_number=YOUR_JDBC_PORT_NUMBER'); - res.line(beanId + '.jdbc.driver_type=YOUR_JDBC_DRIVER_TYPE'); - res.line(beanId + '.jdbc.database_name=YOUR_DATABASE_NAME'); + const dialect = storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : null) : storeFactory.dialect; - break; + const connectViaUrl = cache.cacheStoreFactory.kind === 'CacheJdbcBlobStoreFactory' && storeFactory.connectVia === 'URL'; - default: - res.line(beanId + '.jdbc.url=' + $generatorProperties.jdbcUrlTemplate(dialect)); - } + if (!res && (dialect || connectViaUrl)) + res = $generatorProperties.createBuilder(); - res.line(beanId + '.jdbc.username=YOUR_USER_NAME'); - res.line(beanId + '.jdbc.password=YOUR_PASSWORD'); - res.line(''); - } - } + if (dialect) + $generatorProperties.dataSourceProperties(res, datasources, storeFactory, dialect); if (connectViaUrl) res.line('ds.' + storeFactory.user + '.password=YOUR_PASSWORD'); @@ -102,6 +119,17 @@ $generatorProperties.dataSourcesProperties = function(cluster, res) { }); } + if (cluster.discovery.kind === 'Jdbc') { + const ds = cluster.discovery.Jdbc; + + if (ds.dataSourceBean && ds.dialect) { + if (!res) + res = $generatorProperties.createBuilder(); + + $generatorProperties.dataSourceProperties(res, datasources, ds, ds.dialect); + } + } + return res; }; @@ -114,11 +142,8 @@ $generatorProperties.dataSourcesProperties = function(cluster, res) { */ $generatorProperties.sslProperties = function(cluster, res) { if (cluster.sslEnabled && cluster.sslContextFactory) { - if (!res) { - res = $generatorCommon.builder(); - - res.line('# ' + $generatorCommon.mainComment()); - } + if (!res) + res = $generatorProperties.createBuilder(); res.needEmptyLine = true; diff --git a/modules/web-console/src/main/js/generator/generator-readme.js b/modules/web-console/frontend/generator/generator-readme.js similarity index 100% rename from modules/web-console/src/main/js/generator/generator-readme.js rename to modules/web-console/frontend/generator/generator-readme.js diff --git a/modules/web-console/src/main/js/generator/generator-xml.js b/modules/web-console/frontend/generator/generator-xml.js similarity index 92% rename from modules/web-console/src/main/js/generator/generator-xml.js rename to modules/web-console/frontend/generator/generator-xml.js index efc8df8947ef8..b49b052efdcda 100644 --- a/modules/web-console/src/main/js/generator/generator-xml.js +++ b/modules/web-console/frontend/generator/generator-xml.js @@ -350,8 +350,22 @@ $generatorXml.clusterGeneral = function(cluster, res) { case 'Jdbc': res.startBlock(''); - if (d.Jdbc) - res.line(''); + if (d.Jdbc) { + const datasource = d.Jdbc; + + res.line(''); + + if (datasource.dataSourceBean && datasource.dialect) { + res.line(''); + + if (_.findIndex(res.datasources, (ds) => ds.dataSourceBean === datasource.dataSourceBean) < 0) { + res.datasources.push({ + dataSourceBean: datasource.dataSourceBean, + dialect: datasource.dialect + }); + } + } + } res.endBlock(''); @@ -555,6 +569,34 @@ $generatorXml.clusterBinary = function(binary, res) { return res; }; +// Generate cache key configurations. +$generatorXml.clusterCacheKeyConfiguration = function(keyCfgs, res) { + if (!res) + res = $generatorCommon.builder(); + + keyCfgs = _.filter(keyCfgs, (cfg) => cfg.typeName && cfg.affinityKeyFieldName); + + if (_.isEmpty(keyCfgs)) + return res; + + res.startBlock(''); + res.startBlock(''); + + _.forEach(keyCfgs, (cfg) => { + res.startBlock(''); + + $generatorXml.constructorArg(res, -1, cfg, 'typeName'); + $generatorXml.constructorArg(res, -1, cfg, 'affinityKeyFieldName'); + + res.endBlock(''); + }); + + res.endBlock(''); + res.endBlock(''); + + return res; +}; + // Generate collision group. $generatorXml.clusterCollision = function(collision, res) { if (!res) @@ -1081,7 +1123,9 @@ $generatorXml.cacheMemory = function(cache, res) { res = $generatorCommon.builder(); $generatorXml.property(res, cache, 'memoryMode', null, 'ONHEAP_TIERED'); - $generatorXml.property(res, cache, 'offHeapMaxMemory', null, -1); + + if (cache.memoryMode !== 'OFFHEAP_VALUES') + $generatorXml.property(res, cache, 'offHeapMaxMemory', null, -1); res.softEmptyLine(); @@ -1098,7 +1142,7 @@ $generatorXml.cacheMemory = function(cache, res) { }; // Generate cache query & indexing group. -$generatorXml.cacheQuery = function(cache, res) { +$generatorXml.cacheQuery = function(cache, domains, res) { if (!res) res = $generatorCommon.builder(); @@ -1106,9 +1150,11 @@ $generatorXml.cacheQuery = function(cache, res) { $generatorXml.property(res, cache, 'sqlOnheapRowCacheSize', null, 10240); $generatorXml.property(res, cache, 'longQueryWarningTimeout', null, 3000); - const indexedTypes = _.filter(cache.domains, (domain) => domain.queryMetadata === 'Annotations'); + const indexedTypes = _.filter(domains, (domain) => domain.queryMetadata === 'Annotations'); if (indexedTypes.length > 0) { + res.softEmptyLine(); + res.startBlock(''); res.startBlock(''); @@ -1213,7 +1259,6 @@ $generatorXml.cacheStore = function(cache, domains, res) { if (_.findIndex(res.datasources, (ds) => ds.dataSourceBean === storeFactory.dataSourceBean) < 0) { res.datasources.push({ dataSourceBean: storeFactory.dataSourceBean, - className: $generatorCommon.DATA_SOURCES[storeFactory.dialect], dialect: storeFactory.dialect }); } @@ -1243,6 +1288,68 @@ $generatorXml.cacheStore = function(cache, domains, res) { return res; }; +// Generate cache node filter group. +$generatorXml.cacheNodeFilter = function(cache, igfss, res) { + if (!res) + res = $generatorCommon.builder(); + + const kind = _.get(cache, 'nodeFilter.kind'); + + if (_.isNil(kind) || _.isNil(cache.nodeFilter[kind])) + return res; + + switch (kind) { + case 'IGFS': + const foundIgfs = _.find(igfss, (igfs) => igfs._id === cache.nodeFilter.IGFS.igfs); + + if (foundIgfs) { + res.startBlock(''); + res.startBlock(''); + res.line(''); + res.endBlock(''); + res.endBlock(''); + } + + break; + + case 'OnNodes': + const nodes = cache.nodeFilter.OnNodes.nodeIds; + + if ($generatorCommon.isDefinedAndNotEmpty(nodes)) { + res.startBlock(''); + res.startBlock(''); + res.startBlock(''); + res.startBlock(''); + + _.forEach(nodes, (nodeId) => { + res.startBlock(''); + res.line(''); + res.endBlock(''); + }); + + res.endBlock(''); + res.endBlock(''); + res.endBlock(''); + res.endBlock(''); + } + + break; + + case 'Custom': + res.startBlock(''); + res.line(''); + res.endBlock(''); + + break; + + default: break; + } + + res.needEmptyLine = true; + + return res; +}; + // Generate cache concurrency group. $generatorXml.cacheConcurrency = function(cache, res) { if (!res) @@ -1584,8 +1691,12 @@ $generatorXml.cacheConfiguration = function(cache, res) { $generatorXml.cacheGeneral(cache, res); $generatorXml.cacheMemory(cache, res); - $generatorXml.cacheQuery(cache, res); + $generatorXml.cacheQuery(cache, cache.domains, res); $generatorXml.cacheStore(cache, cache.domains, res); + + const igfs = _.get(cache, 'nodeFilter.IGFS.instance'); + + $generatorXml.cacheNodeFilter(cache, igfs ? [igfs] : [], res); $generatorXml.cacheConcurrency(cache, res); $generatorXml.cacheRebalance(cache, res); $generatorXml.cacheServerNearCache(cache, res); @@ -1813,50 +1924,52 @@ $generatorXml.generateDataSources = function(datasources, res) { if (datasources.length > 0) { res.line(''); - _.forEach(datasources, function(item) { - const beanId = item.dataSourceBean; + _.forEach(datasources, (datasource) => $generatorXml.generateDataSource(datasource, res)); - res.startBlock(''); + res.needEmptyLine = true; - switch (item.dialect) { - case 'Generic': - res.line(''); + res.emptyLineIfNeeded(); + } - break; + return res; +}; - case 'DB2': - res.line(''); - res.line(''); - res.line(''); - res.line(''); +$generatorXml.generateDataSource = function(datasource, res) { + const beanId = datasource.dataSourceBean; - break; + res.startBlock(''); - case 'PostgreSQL': - res.line(''); + switch (datasource.dialect) { + case 'Generic': + res.line(''); - break; + break; - default: - res.line(''); - } + case 'DB2': + res.line(''); + res.line(''); + res.line(''); + res.line(''); - res.line(''); - res.line(''); + break; - res.endBlock(''); + case 'PostgreSQL': + res.line(''); - res.needEmptyLine = true; + break; - res.emptyLineIfNeeded(); - }); + default: + res.line(''); + } - res.needEmptyLine = true; + res.line(''); + res.line(''); - res.emptyLineIfNeeded(); - } + res.endBlock(''); - return res; + res.needEmptyLine = true; + + res.emptyLineIfNeeded(); }; $generatorXml.clusterConfiguration = function(cluster, clientNearCfg, res) { @@ -1874,6 +1987,8 @@ $generatorXml.clusterConfiguration = function(cluster, clientNearCfg, res) { $generatorXml.clusterBinary(cluster.binaryConfiguration, res); + $generatorXml.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, res); + $generatorXml.clusterCollision(cluster.collision, res); $generatorXml.clusterCommunication(cluster, res); diff --git a/modules/web-console/src/main/js/gulpfile.babel.js/index.js b/modules/web-console/frontend/gulpfile.babel.js/index.js similarity index 100% rename from modules/web-console/src/main/js/gulpfile.babel.js/index.js rename to modules/web-console/frontend/gulpfile.babel.js/index.js diff --git a/modules/web-console/src/main/js/gulpfile.babel.js/paths.js b/modules/web-console/frontend/gulpfile.babel.js/paths.js similarity index 85% rename from modules/web-console/src/main/js/gulpfile.babel.js/paths.js rename to modules/web-console/frontend/gulpfile.babel.js/paths.js index cca36432636ba..9134e446b069c 100644 --- a/modules/web-console/src/main/js/gulpfile.babel.js/paths.js +++ b/modules/web-console/frontend/gulpfile.babel.js/paths.js @@ -21,12 +21,15 @@ const rootDir = path.resolve('./'); const srcDir = path.resolve('app'); const destDir = path.resolve('build'); -const igniteModulesDir = process.env.IGNITE_MODULES ? path.normalize(process.env.IGNITE_MODULES) : './ignite_modules'; +const igniteModulesDir = process.env.IGNITE_MODULES ? path.join(path.normalize(process.env.IGNITE_MODULES), 'frontend') : './ignite_modules'; const igniteModulesTemp = path.resolve('ignite_modules_temp'); const jadePaths = [ './views/*.jade', - './views/**/*.jade' + './views/**/*.jade', + './app/helpers/**/*.jade', + './app/modules/states/configuration/**/*.jade', + './app/modules/sql/*.jade' ]; const resourcePaths = [ @@ -46,6 +49,7 @@ const appModulePaths = [ igniteModulesDir + '/**/app/modules/**/*.js', igniteModulesDir + '/**/app/modules/**/*.jade', igniteModulesDir + '/**/app/**/*.css', + igniteModulesDir + '/**/app/**/*.scss', igniteModulesDir + '/**/app/data/*.json' ]; diff --git a/modules/web-console/src/main/js/gulpfile.babel.js/tasks/build.js b/modules/web-console/frontend/gulpfile.babel.js/tasks/build.js similarity index 100% rename from modules/web-console/src/main/js/gulpfile.babel.js/tasks/build.js rename to modules/web-console/frontend/gulpfile.babel.js/tasks/build.js diff --git a/modules/web-console/src/main/js/gulpfile.babel.js/tasks/bundle.js b/modules/web-console/frontend/gulpfile.babel.js/tasks/bundle.js similarity index 100% rename from modules/web-console/src/main/js/gulpfile.babel.js/tasks/bundle.js rename to modules/web-console/frontend/gulpfile.babel.js/tasks/bundle.js diff --git a/modules/web-console/src/main/js/gulpfile.babel.js/tasks/clean.js b/modules/web-console/frontend/gulpfile.babel.js/tasks/clean.js similarity index 100% rename from modules/web-console/src/main/js/gulpfile.babel.js/tasks/clean.js rename to modules/web-console/frontend/gulpfile.babel.js/tasks/clean.js diff --git a/modules/web-console/src/main/js/gulpfile.babel.js/tasks/copy.js b/modules/web-console/frontend/gulpfile.babel.js/tasks/copy.js similarity index 100% rename from modules/web-console/src/main/js/gulpfile.babel.js/tasks/copy.js rename to modules/web-console/frontend/gulpfile.babel.js/tasks/copy.js diff --git a/modules/web-console/src/main/js/gulpfile.babel.js/tasks/ignite-modules.js b/modules/web-console/frontend/gulpfile.babel.js/tasks/ignite-modules.js similarity index 100% rename from modules/web-console/src/main/js/gulpfile.babel.js/tasks/ignite-modules.js rename to modules/web-console/frontend/gulpfile.babel.js/tasks/ignite-modules.js diff --git a/modules/web-console/src/main/js/gulpfile.babel.js/tasks/jade.js b/modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js similarity index 100% rename from modules/web-console/src/main/js/gulpfile.babel.js/tasks/jade.js rename to modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js diff --git a/modules/web-console/frontend/gulpfile.babel.js/tasks/test.js b/modules/web-console/frontend/gulpfile.babel.js/tasks/test.js new file mode 100644 index 0000000000000..ad4108dae1e21 --- /dev/null +++ b/modules/web-console/frontend/gulpfile.babel.js/tasks/test.js @@ -0,0 +1,92 @@ +/* + * 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. + */ + +import gulp from 'gulp'; +import karmaBabelPreprocessor from 'karma-babel-preprocessor'; +import karmaPhantomjsLauncher from 'karma-phantomjs-launcher'; +import karmaWebpack from 'karma-webpack'; +import karmaJasmine from 'karma-jasmine'; + +import {Server} from 'karma'; + +import {rootDir} from '../paths'; + +gulp.task('test', (cb) => { + new Server({ + // Base path that will be used to resolve all patterns (eg. files, exclude). + basePath: rootDir, + + // Frameworks to use available frameworks: https://npmjs.org/browse/keyword/karma-adapter + frameworks: ['jasmine'], + + // List of files / patterns to load in the browser. + files: [ + 'test/**/*.test.js' + ], + + plugins: [ + karmaBabelPreprocessor, + karmaPhantomjsLauncher, + karmaWebpack, + karmaJasmine + ], + + // Preprocess matching files before serving them to the browser + // available preprocessors: https://npmjs.org/browse/keyword/karma-preprocessor. + preprocessors: { + 'test/**/*.js': ['webpack'] + }, + + webpack: { + devtool: 'inline-source-map', + module: { + loaders: [ + {test: /\.js/, loaders: ['babel'], exclude: /node_modules/} + ] + }, + resolve: { + extensions: ['', '.js'] + } + }, + + // Test results reporter to use + // possible values: 'dots', 'progress' + // available reporters: https://npmjs.org/browse/keyword/karma-reporter. + reporters: ['progress'], + + // web server port + port: 9876, + + // enable / disable colors in the output (reporters and logs) + colors: true, + + // enable / disable watching file and executing tests whenever any file changes + autoWatch: true, + + // start these browsers + // available browser launchers: https://npmjs.org/browse/keyword/karma-launcher + browsers: ['PhantomJS'], + + // Continuous Integration mode + // if true, Karma captures browsers, runs the tests and exits + singleRun: true, + + // Concurrency level + // how many browser should be started simultaneous + concurrency: Infinity + }, cb).start(); +}); diff --git a/modules/web-console/src/main/js/gulpfile.babel.js/tasks/watch.js b/modules/web-console/frontend/gulpfile.babel.js/tasks/watch.js similarity index 100% rename from modules/web-console/src/main/js/gulpfile.babel.js/tasks/watch.js rename to modules/web-console/frontend/gulpfile.babel.js/tasks/watch.js diff --git a/modules/web-console/src/main/js/gulpfile.babel.js/webpack/common.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js similarity index 92% rename from modules/web-console/src/main/js/gulpfile.babel.js/webpack/common.js rename to modules/web-console/frontend/gulpfile.babel.js/webpack/common.js index 237f2e7bb1d4b..e3d88b3106945 100644 --- a/modules/web-console/src/main/js/gulpfile.babel.js/webpack/common.js +++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js @@ -26,7 +26,7 @@ import eslintFormatter from 'eslint-friendly-formatter'; import ExtractTextPlugin from 'extract-text-webpack-plugin'; import HtmlWebpackPlugin from 'html-webpack-plugin'; -import {srcDir, destDir, rootDir} from '../paths'; +import {srcDir, destDir, rootDir, igniteModulesDir} from '../paths'; const NODE_ENV = process.env.NODE_ENV || 'production'; const development = NODE_ENV === 'development'; @@ -34,18 +34,19 @@ const node_modules_path = path.resolve('node_modules'); const cssLoader = 'css-loader?sourceMap!postcss-loader'; const stylesLoader = cssLoader + '!sass-loader?outputStyle=expanded&sourceMap=true&sourceMapContents=true'; -let favicon; +let favicon = 'build/ignite_modules/favicon.ico'; try { - fs.accessSync('build/ignite_modules/favicon.ico', fs.F_OK); - - favicon = 'build/ignite_modules/favicon.ico'; + fs.accessSync(path.join(igniteModulesDir, 'favicon.ico'), fs.F_OK); } catch (ignore) { favicon = 'build/favicon.ico'; } export default () => { + const assetsLoader = development ? 'url-loader' : 'file-loader'; + return { + cache: true, node: { fs: 'empty' }, @@ -101,11 +102,6 @@ export default () => { 'jade-html-loader' ] }, - { - test: /\.js$/, - exclude: [node_modules_path], - loaders: ['ng-annotate-loader'] - }, { test: /\.js$/, exclude: [node_modules_path], @@ -129,12 +125,12 @@ export default () => { { test: /\.(woff2|woff|ttf|eot|svg)?(\?v=[0-9]\.[0-9]\.[0-9])?$/, loaders: [ - 'file-loader?name=assets/fonts/[name].[ext]' + `${assetsLoader}?name=assets/fonts/[name].[ext]` ] }, { test: /\.(jpe?g|png|gif)$/i, - loaders: ['file-loader?name=assets/images/[name]_[hash].[ext]'] + loaders: [`${assetsLoader}?name=assets/images/[name]_[hash].[ext]`] }, { test: require.resolve('jquery'), @@ -178,6 +174,7 @@ export default () => { chunks: ['vendor', 'app'] }), new webpack.optimize.AggressiveMergingPlugin({moveToParents: true}), + new webpack.optimize.OccurenceOrderPlugin(), new ExtractTextPlugin('assets/css/[name]' + (development ? '' : '.[chunkhash]') + '.css', {allChunks: true}), new HtmlWebpackPlugin({ filename: 'index.html', diff --git a/modules/web-console/src/main/js/gulpfile.babel.js/webpack/environments/development.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js similarity index 82% rename from modules/web-console/src/main/js/gulpfile.babel.js/webpack/environments/development.js rename to modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js index aad4b37ff4d4c..229760e8af067 100644 --- a/modules/web-console/src/main/js/gulpfile.babel.js/webpack/environments/development.js +++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js @@ -15,8 +15,10 @@ * limitations under the License. */ +import path from 'path'; import webpack from 'webpack'; -import {destDir, rootDir} from '../../paths'; + +import {destDir, rootDir, srcDir} from '../../paths'; export default () => { const plugins = [ @@ -24,11 +26,16 @@ export default () => { ]; return { + entry: { + webpack: 'webpack-dev-server/client?http://localhost:9000/', + app: [path.join(srcDir, 'app.js'), 'webpack/hot/only-dev-server'] + }, context: rootDir, debug: true, devtool: 'source-map', watch: true, devServer: { + compress: true, historyApiFallback: true, publicPath: '/', contentBase: destDir, @@ -44,16 +51,14 @@ export default () => { '/api/v1/*': { target: 'http://localhost:3000', changeOrigin: true, - rewrite: (req) => { - req.url = req.url.replace(/^\/api\/v1/, ''); - - return req; + pathRewrite: { + '^/api/v1': '' } } }, watchOptions: { aggregateTimeout: 1000, - poll: 1000 + poll: 2000 }, stats: {colors: true}, port: 9000 diff --git a/modules/web-console/src/main/js/gulpfile.babel.js/webpack/environments/production.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js similarity index 100% rename from modules/web-console/src/main/js/gulpfile.babel.js/webpack/environments/production.js rename to modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js diff --git a/modules/web-console/src/main/js/gulpfile.babel.js/webpack/index.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/index.js similarity index 100% rename from modules/web-console/src/main/js/gulpfile.babel.js/webpack/index.js rename to modules/web-console/frontend/gulpfile.babel.js/webpack/index.js diff --git a/modules/web-console/src/main/js/gulpfile.babel.js/webpack/plugins/progress.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/plugins/progress.js similarity index 100% rename from modules/web-console/src/main/js/gulpfile.babel.js/webpack/plugins/progress.js rename to modules/web-console/frontend/gulpfile.babel.js/webpack/plugins/progress.js diff --git a/modules/web-console/src/main/js/ignite_modules/README.txt b/modules/web-console/frontend/ignite_modules/README.txt similarity index 100% rename from modules/web-console/src/main/js/ignite_modules/README.txt rename to modules/web-console/frontend/ignite_modules/README.txt diff --git a/modules/web-console/src/main/js/ignite_modules/index.js b/modules/web-console/frontend/ignite_modules/index.js similarity index 100% rename from modules/web-console/src/main/js/ignite_modules/index.js rename to modules/web-console/frontend/ignite_modules/index.js diff --git a/modules/web-console/src/main/js/package.json b/modules/web-console/frontend/package.json similarity index 77% rename from modules/web-console/src/main/js/package.json rename to modules/web-console/frontend/package.json index b2c648d488eed..7ea289346cddf 100644 --- a/modules/web-console/src/main/js/package.json +++ b/modules/web-console/frontend/package.json @@ -6,7 +6,8 @@ "scripts": { "dev": "cross-env NODE_ENV=development gulp watch", "build": "cross-env NODE_ENV=production gulp build", - "eslint": "eslint --env node --format node_modules/eslint-friendly-formatter gulpfile.babel.js/ serve/ app/ controllers/ generator/ ignite_modules/ ignite_modules_temp/" + "test": "karma start ./test/karma.conf.js", + "eslint": "eslint --format node_modules/eslint-friendly-formatter gulpfile.babel.js/ app/ controllers/ generator/ ignite_modules/ ignite_modules_temp/ -- --eff-by-issue" }, "author": "", "contributors": [ @@ -29,6 +30,7 @@ ], "dependencies": { "angular": "^1.5.5", + "angular-acl": "^0.1.7", "angular-animate": "^1.5.5", "angular-aria": "^1.5.5", "angular-cookies": "^1.5.5", @@ -45,36 +47,21 @@ "angular-tree-control": "^0.2.26", "angular-ui-grid": "^3.1.1", "angular-ui-router": "^0.3.1", - "body-parser": "^1.15.0", "bootstrap-sass": "^3.3.6", "brace": "^0.8.0", - "connect-mongo": "^1.1.0", - "cookie-parser": "~1.4.0", "es6-promise": "^3.0.2", - "express": "^4.14.0", - "express-session": "^1.12.0", "file-saver": "^1.3.2", - "fire-up": "^1.0.0", "font-awesome": "^4.6.3", "glob": "^7.0.3", "jquery": "^3.0.0", "jszip": "^3.0.0", "lodash": "^4.8.2", - "mongoose": "^4.4.11", - "mongoose-deep-populate": "^3.0.0", - "nconf": "^0.8.2", - "nodemailer": "^2.3.0", "nvd3": "^1.8.3", - "passport": "^0.3.2", - "passport-local": "^1.0.0", - "passport-local-mongoose": "^4.0.0", - "passport.socketio": "^3.6.1", "query-command-supported": "^1.0.0", - "require-dir": "^0.3.0", - "socket.io": "^1.4.5", + "raleway-webfont": "^3.0.1", + "roboto-font": "^0.1.0", "socket.io-client": "^1.4.6", - "ui-router-metatags": "^1.0.3", - "ws": "^0.8.0" + "ui-router-metatags": "^1.0.3" }, "devDependencies": { "assets-webpack-plugin": "^3.2.0", @@ -83,11 +70,13 @@ "babel-eslint": "^6.0.4", "babel-loader": "^6.2.4", "babel-plugin-add-module-exports": "^0.2.1", + "babel-plugin-transform-builtin-extend": "^1.1.0", "babel-plugin-transform-runtime": "^6.7.5", "babel-polyfill": "^6.7.4", "babel-preset-angular": "^6.0.15", + "babel-preset-es2015": "^6.9.0", "babel-runtime": "^6.6.1", - "bootstrap-loader": "^1.0.10", + "chai": "^3.5.0", "cross-env": "^1.0.7", "css-loader": "^0.23.0", "eslint": "^2.9.0", @@ -105,24 +94,32 @@ "gulp-util": "^3.0.7", "html-loader": "^0.4.3", "html-webpack-plugin": "^2.21.0", - "imagemin-pngquant": "^5.0.0", "jade": "^1.11.0", "jade-html-loader": "0.0.3", + "jasmine-core": "^2.4.1", "json-loader": "^0.5.4", + "karma": "^0.13.22", + "karma-babel-preprocessor": "^6.0.1", + "karma-jasmine": "^1.0.2", + "karma-mocha": "^1.0.1", + "karma-phantomjs-launcher": "^1.0.0", + "karma-teamcity-reporter": "^1.0.0", + "karma-webpack": "^1.7.0", "mocha": "~2.5.3", + "mocha-teamcity-reporter": "^1.0.0", "morgan": "^1.7.0", - "ng-annotate-loader": "0.1.0", "ngtemplate-loader": "^1.3.1", "node-sass": "^3.4.2", + "phantomjs-prebuilt": "^2.1.7", "postcss-loader": "^0.9.1", + "require-dir": "^0.3.0", "resolve-url-loader": "^1.4.3", "sass-loader": "^3.1.1", "should": "^9.0.2", "style-loader": "^0.13.1", - "supertest": "^1.1.0", "url": "^0.11.0", "url-loader": "^0.5.6", "webpack": "^1.13.1", - "webpack-dev-server": "^1.14.1" + "webpack-dev-server": "^1.15.0" } } diff --git a/modules/web-console/src/main/js/public/favicon.ico b/modules/web-console/frontend/public/favicon.ico similarity index 100% rename from modules/web-console/src/main/js/public/favicon.ico rename to modules/web-console/frontend/public/favicon.ico diff --git a/modules/web-console/src/main/js/public/images/cache.png b/modules/web-console/frontend/public/images/cache.png similarity index 100% rename from modules/web-console/src/main/js/public/images/cache.png rename to modules/web-console/frontend/public/images/cache.png diff --git a/modules/web-console/src/main/js/public/images/cluster.png b/modules/web-console/frontend/public/images/cluster.png similarity index 100% rename from modules/web-console/src/main/js/public/images/cluster.png rename to modules/web-console/frontend/public/images/cluster.png diff --git a/modules/web-console/src/main/js/public/images/docker.png b/modules/web-console/frontend/public/images/docker.png similarity index 100% rename from modules/web-console/src/main/js/public/images/docker.png rename to modules/web-console/frontend/public/images/docker.png diff --git a/modules/web-console/src/main/js/public/images/domains.png b/modules/web-console/frontend/public/images/domains.png similarity index 100% rename from modules/web-console/src/main/js/public/images/domains.png rename to modules/web-console/frontend/public/images/domains.png diff --git a/modules/web-console/src/main/js/public/images/igfs.png b/modules/web-console/frontend/public/images/igfs.png similarity index 100% rename from modules/web-console/src/main/js/public/images/igfs.png rename to modules/web-console/frontend/public/images/igfs.png diff --git a/modules/web-console/src/main/js/public/images/ignite-logo.png b/modules/web-console/frontend/public/images/ignite-logo.png similarity index 100% rename from modules/web-console/src/main/js/public/images/ignite-logo.png rename to modules/web-console/frontend/public/images/ignite-logo.png diff --git a/modules/web-console/src/main/js/public/images/ignite-logo@2x.png b/modules/web-console/frontend/public/images/ignite-logo@2x.png similarity index 100% rename from modules/web-console/src/main/js/public/images/ignite-logo@2x.png rename to modules/web-console/frontend/public/images/ignite-logo@2x.png diff --git a/modules/web-console/src/main/js/public/images/ignite-puzzle.png b/modules/web-console/frontend/public/images/ignite-puzzle.png similarity index 100% rename from modules/web-console/src/main/js/public/images/ignite-puzzle.png rename to modules/web-console/frontend/public/images/ignite-puzzle.png diff --git a/modules/web-console/src/main/js/public/images/java.png b/modules/web-console/frontend/public/images/java.png similarity index 100% rename from modules/web-console/src/main/js/public/images/java.png rename to modules/web-console/frontend/public/images/java.png diff --git a/modules/web-console/src/main/js/public/images/pb-ignite.png b/modules/web-console/frontend/public/images/pb-ignite.png similarity index 100% rename from modules/web-console/src/main/js/public/images/pb-ignite.png rename to modules/web-console/frontend/public/images/pb-ignite.png diff --git a/modules/web-console/src/main/js/public/images/pb-ignite@2x.png b/modules/web-console/frontend/public/images/pb-ignite@2x.png similarity index 100% rename from modules/web-console/src/main/js/public/images/pb-ignite@2x.png rename to modules/web-console/frontend/public/images/pb-ignite@2x.png diff --git a/modules/web-console/src/main/js/public/images/query-chart.png b/modules/web-console/frontend/public/images/query-chart.png similarity index 100% rename from modules/web-console/src/main/js/public/images/query-chart.png rename to modules/web-console/frontend/public/images/query-chart.png diff --git a/modules/web-console/src/main/js/public/images/query-metadata.png b/modules/web-console/frontend/public/images/query-metadata.png similarity index 100% rename from modules/web-console/src/main/js/public/images/query-metadata.png rename to modules/web-console/frontend/public/images/query-metadata.png diff --git a/modules/web-console/src/main/js/public/images/query-table.png b/modules/web-console/frontend/public/images/query-table.png similarity index 100% rename from modules/web-console/src/main/js/public/images/query-table.png rename to modules/web-console/frontend/public/images/query-table.png diff --git a/modules/web-console/src/main/js/public/images/summary.png b/modules/web-console/frontend/public/images/summary.png similarity index 100% rename from modules/web-console/src/main/js/public/images/summary.png rename to modules/web-console/frontend/public/images/summary.png diff --git a/modules/web-console/src/main/js/public/images/xml.png b/modules/web-console/frontend/public/images/xml.png similarity index 100% rename from modules/web-console/src/main/js/public/images/xml.png rename to modules/web-console/frontend/public/images/xml.png diff --git a/modules/web-console/src/main/js/public/stylesheets/_bootstrap-custom.scss b/modules/web-console/frontend/public/stylesheets/_bootstrap-custom.scss similarity index 100% rename from modules/web-console/src/main/js/public/stylesheets/_bootstrap-custom.scss rename to modules/web-console/frontend/public/stylesheets/_bootstrap-custom.scss diff --git a/modules/web-console/src/main/js/public/stylesheets/_bootstrap-variables.scss b/modules/web-console/frontend/public/stylesheets/_bootstrap-variables.scss similarity index 99% rename from modules/web-console/src/main/js/public/stylesheets/_bootstrap-variables.scss rename to modules/web-console/frontend/public/stylesheets/_bootstrap-variables.scss index fe0ac187b2466..07e8c516cd94f 100644 --- a/modules/web-console/src/main/js/public/stylesheets/_bootstrap-variables.scss +++ b/modules/web-console/frontend/public/stylesheets/_bootstrap-variables.scss @@ -60,8 +60,8 @@ $link-hover-decoration: underline !default; // //## Font, line-height, and color for body text, headings, and more. -$font-family-sans-serif: Roboto Slab, sans-serif !default; -$font-family-serif: Roboto Slab, serif !default; +$font-family-sans-serif: Roboto_slab, sans-serif !default; +$font-family-serif: Roboto_slab, serif !default; //** Default monospace fonts for ``, ``, and `
          `.
           $font-family-monospace:   Menlo, Monaco, Consolas, "Courier New", monospace !default;
           $font-family-base:        $font-family-sans-serif !default;
          diff --git a/modules/web-console/src/main/js/public/stylesheets/_font-awesome-custom.scss b/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss
          similarity index 94%
          rename from modules/web-console/src/main/js/public/stylesheets/_font-awesome-custom.scss
          rename to modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss
          index 514acc2a6fef9..1c8f325790cd9 100644
          --- a/modules/web-console/src/main/js/public/stylesheets/_font-awesome-custom.scss
          +++ b/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss
          @@ -17,7 +17,7 @@
           
           @import '~font-awesome/scss/variables';
           
          -$fa-font-path: 'https://maxcdn.bootstrapcdn.com/font-awesome/4.6.3/fonts';
          +$fa-font-path: '~font-awesome/fonts';
           
           @import '~font-awesome/scss/mixins';
           @import '~font-awesome/scss/path';
          diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/igfs.directive.js b/modules/web-console/frontend/public/stylesheets/blocks/error.scss
          similarity index 81%
          rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/igfs.directive.js
          rename to modules/web-console/frontend/public/stylesheets/blocks/error.scss
          index 74d4826271e59..4e16989914489 100644
          --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/igfs.directive.js
          +++ b/modules/web-console/frontend/public/stylesheets/blocks/error.scss
          @@ -15,13 +15,17 @@
            * limitations under the License.
            */
           
          -import templateUrl from './igfs.jade';
          +.error-page {
          +    text-align: center;
          +    min-height: 300px;
           
          -export default ['igniteConfigurationClustersIgfs', [() => {
          -    return {
          -        scope: true,
          -        restrict: 'E',
          -        templateUrl,
          -        replace: true
          -    };
          -}]];
          +    &__title {
          +        margin-top: 150px;
          +        font-weight: 200;
          +    }
          +
          +    &__description {
          +        margin-top: 30px;
          +        font-weight: 500;
          +    }
          +}
          diff --git a/modules/web-console/src/main/js/public/stylesheets/style.scss b/modules/web-console/frontend/public/stylesheets/style.scss
          similarity index 94%
          rename from modules/web-console/src/main/js/public/stylesheets/style.scss
          rename to modules/web-console/frontend/public/stylesheets/style.scss
          index 9f35637bd56c7..4db712786b881 100644
          --- a/modules/web-console/src/main/js/public/stylesheets/style.scss
          +++ b/modules/web-console/frontend/public/stylesheets/style.scss
          @@ -18,22 +18,9 @@
           @import "./font-awesome-custom";
           @import "./bootstrap-custom";
           @import "./variables";
          -
          +@import "~roboto-font/css/fonts.css";
           @import "./../../app/directives/information/information.scss";
          -
          -@font-face {
          -    font-family: 'Roboto Slab';
          -    font-style: normal;
          -    font-weight: 400;
          -    src: local('Roboto Slab Regular'), local('RobotoSlab-Regular'), url(https://fonts.gstatic.com/s/robotoslab/v6/y7lebkjgREBJK96VQi37ZiwlidHJgAgmTjOEEzwu1L8.ttf) format('truetype');
          -}
          -
          -@font-face {
          -    font-family: 'Roboto Slab';
          -    font-style: normal;
          -    font-weight: 700;
          -    src: local('Roboto Slab Bold'), local('RobotoSlab-Bold'), url(https://fonts.gstatic.com/s/robotoslab/v6/dazS1PrQQuCxC3iOAJFEJTdGNerWpg2Hn6A-BxWgZ_I.ttf) format('truetype');
          -}
          +@import "./blocks/error";
           
           hr {
               margin: 20px 0;
          @@ -183,11 +170,11 @@ ul.navbar-nav, .sidebar-nav {
               overflow: auto;
               overflow-x: hidden;
               outline-style: none;
          +    margin-top: 0;
           
               li > a {
                   display: block;
           
          -        //cursor: default;
                   padding: 3px 10px;
           
                   overflow: hidden;
          @@ -204,6 +191,60 @@ ul.navbar-nav, .sidebar-nav {
                   }
               }
           
          +    li > div {
          +        display: block;
          +        overflow: hidden;
          +
          +        i {
          +            float: right;
          +            color: $text-color;
          +            background-color: transparent;
          +            line-height: $line-height-base;
          +            margin: 0 10px 0 0;
          +            padding: 6px 0;
          +        }
          +
          +        div {
          +            overflow: hidden;
          +            text-overflow: ellipsis;
          +        }
          +    }
          +
          +    // Hover/Focus state
          +    li > div a {
          +        float: left;
          +        display: block;
          +        width: 100%;
          +        padding: 3px 10px;
          +        color: $dropdown-link-color;
          +
          +        overflow: hidden;
          +        white-space: nowrap;
          +        text-overflow: ellipsis;
          +        
          +        &:hover,
          +        &:focus {
          +            text-decoration: none;
          +            color: $dropdown-link-hover-color;
          +            background-color: $dropdown-link-hover-bg;
          +        }
          +    }
          +
          +    // Active state
          +    .active > div a {
          +        cursor: default;
          +        pointer-events: none;
          +
          +        &,
          +        &:hover,
          +        &:focus {
          +            color: $dropdown-link-active-color;
          +            text-decoration: none;
          +            outline: 0;
          +            background-color: $dropdown-link-active-bg;
          +        }
          +    }
          +
               li.divider {
                   margin: 3px 0;
               }
          @@ -576,17 +617,6 @@ button.form-control {
               }
           }
           
          -.theme-line .sql-notebooks {
          -    li.custom > a {
          -        color: $brand-info;
          -        font-weight: bold;
          -    }
          -
          -    li.custom > a:hover {
          -        color: darken($brand-info, 15%);
          -    }
          -}
          -
           .theme-line .paragraphs {
               .panel-group .panel + .panel {
                   margin-top: 30px;
          @@ -612,29 +642,6 @@ button.form-control {
                   }
               }
           
          -    .sql-controls {
          -        margin: 10px 0;
          -        padding: 0 10px;
          -    }
          -
          -    .sql-table-total {
          -        padding: 0 10px;
          -
          -        label, b {
          -            display: inline-block;
          -
          -            padding-top: 5px;
          -
          -            height: 27px;
          -        }
          -
          -        margin-bottom: 10px;
          -    }
          -
          -    .sql-table {
          -        height: 400px;
          -    }
          -
               table thead {
                   background-color: white;
               }
          @@ -653,38 +660,45 @@ button.form-control {
                   line-height: 55px;
               }
           
          -    .sql-error-result {
          -        padding: 10px 0;
          +    .sql-controls {
          +        border-top: 1px solid $ignite-border-color;
           
          -        text-align: center;
          -        color: $brand-primary;
          +        padding: 10px 10px;
          +    }
           
          +    .sql-result {
                   border-top: 1px solid $ignite-border-color;
          -    }
           
          -    .sql-empty-result {
          -        margin-top: 10px;
          -        margin-bottom: 10px;
          -        text-align: center;
          -        color: $ignite-placeholder-color;
          -    }
          +        .error {
          +            padding: 10px 10px;
           
          -    .sql-next {
          -        float: right;
          +            text-align: center;
          +            color: $brand-primary;
          +        }
           
          -        .disabled {
          -            cursor: default;
          -            text-decoration: none;
          +        .empty {
          +            padding: 10px 10px;
          +
          +            text-align: center;
          +            color: $ignite-placeholder-color;
                   }
           
          -        a {
          -            margin-right: 5px;
          -            margin-bottom: 5px;
          +        .total {
          +            padding: 10px 10px;
                   }
           
          -        i {
          -            margin-top: 3px;
          -            margin-right: 10px;
          +        .table {
          +            margin: 0
          +        }
          +
          +        .chart {
          +            margin: 0
          +        }
          +
          +        .footer {
          +            border-top: 1px solid $ignite-border-color;
          +
          +            padding: 5px 10px;
                   }
               }
           }
          @@ -1143,6 +1157,7 @@ button.form-control {
           .theme-line .popover.cache-metadata {
               @extend .popover.settings;
           
          +    position: absolute;
               z-index: 1030;
               min-width: 305px;
               max-width: 450px;
          @@ -2030,7 +2045,7 @@ treecontrol.tree-classic {
           
                   text-align: center;
                   text-decoration: none;
          -        font-family: 'Roboto Slab', sans-serif;
          +        font-family: $font-family-sans-serif;
                   font-size: 20px;
                   font-weight: 500;
           
          @@ -2153,4 +2168,4 @@ html,body,.splash-screen {
                   margin: 0;
                   padding: 0;
               }
          -}
          \ No newline at end of file
          +}
          diff --git a/modules/web-console/src/main/js/public/stylesheets/variables.scss b/modules/web-console/frontend/public/stylesheets/variables.scss
          similarity index 100%
          rename from modules/web-console/src/main/js/public/stylesheets/variables.scss
          rename to modules/web-console/frontend/public/stylesheets/variables.scss
          diff --git a/modules/web-console/frontend/test/e2e/exampe.test.js b/modules/web-console/frontend/test/e2e/exampe.test.js
          new file mode 100644
          index 0000000000000..c778c797e61b7
          --- /dev/null
          +++ b/modules/web-console/frontend/test/e2e/exampe.test.js
          @@ -0,0 +1,40 @@
          +/*
          + * 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.
          + */
          +
          +suite('ExampleTestSuite', () => {
          +    setup(() => {
          +        // browser.get('http://localhost:9000/');
          +    });
          +
          +    test('initially has a greeting', (done) => {
          +        done();
          +
          +        // element(by.model('ui.email')).sendKeys('jhon@doe.com');
          +    });
          +
          +    test('initially has a greeting', (done) => {
          +        done();
          +
          +        // element(by.model('ui.email')).sendKeys('jhon@doe.com');
          +    });
          +
          +    test('initially has a greeting', (done) => {
          +        done();
          +
          +        // element(by.model('ui.email')).sendKeys('jhon@doe.com');
          +    });
          +});
          \ No newline at end of file
          diff --git a/modules/web-console/frontend/test/karma.conf.js b/modules/web-console/frontend/test/karma.conf.js
          new file mode 100644
          index 0000000000000..e13ba00031908
          --- /dev/null
          +++ b/modules/web-console/frontend/test/karma.conf.js
          @@ -0,0 +1,113 @@
          +/*
          + * 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.
          + */
          +
          +const path = require('path');
          +const webpack = require('webpack');
          +
          +const basePath = path.resolve('./');
          +
          +module.exports = function(config) {
          +    config.set({
          +        // Base path that will be used to resolve all patterns (eg. files, exclude).
          +        basePath: basePath,
          +
          +        // Frameworks to use available frameworks: https://npmjs.org/browse/keyword/karma-adapter
          +        frameworks: ['mocha'],
          +
          +        // List of files / patterns to load in the browser.
          +        files: [
          +            'test/**/*.test.js'
          +        ],
          +
          +        plugins: [
          +            require('karma-phantomjs-launcher'),
          +            require('karma-teamcity-reporter'),
          +            require('karma-webpack'),
          +            require('karma-mocha')
          +        ],
          +
          +        // Preprocess matching files before serving them to the browser
          +        // available preprocessors: https://npmjs.org/browse/keyword/karma-preprocessor.
          +        preprocessors: {
          +            'test/**/*.js': ['webpack']
          +        },
          +
          +        webpack: {
          +            module: {
          +                loaders: [
          +                    {
          +                        test: /\.json$/,
          +                        loader: 'json'
          +                    },
          +                    {
          +                        test: /\.js$/,
          +                        loader: 'babel',
          +                        exclude: /node_modules/
          +                    }
          +                ]
          +            },
          +            resolve: {
          +                extensions: ["", ".js"]
          +            },
          +            plugins: [
          +                new webpack.ProvidePlugin({
          +                    _: 'lodash'
          +                })
          +            ]
          +        },
          +
          +        webpackMiddleware: {
          +            noInfo: true
          +        },
          +
          +        // Test results reporter to use
          +        // possible values: 'dots', 'progress'
          +        // available reporters: https://npmjs.org/browse/keyword/karma-reporter.
          +        reporters: ['teamcity'],
          +
          +        // web server port
          +        port: 9876,
          +
          +        // enable / disable colors in the output (reporters and logs)
          +        colors: true,
          +
          +        // level of logging
          +        // possible values: config.LOG_DISABLE || config.LOG_ERROR || config.LOG_WARN || config.LOG_INFO || config.LOG_DEBUG
          +        logLevel: config.LOG_INFO,
          +
          +        // enable / disable watching file and executing tests whenever any file changes
          +        autoWatch: true,
          +
          +        // start these browsers
          +        // available browser launchers: https://npmjs.org/browse/keyword/karma-launcher
          +        browsers: ['PhantomJS'],
          +
          +        // Continuous Integration mode
          +        // if true, Karma captures browsers, runs the tests and exits
          +        singleRun: true,
          +
          +        // Concurrency level
          +        // how many browser should be started simultaneous
          +        concurrency: Infinity,
          +
          +        client: {
          +            mocha: {
          +                ui: 'tdd'
          +            }
          +        }
          +    });
          +};
          diff --git a/modules/web-console/frontend/test/protractor.conf.js b/modules/web-console/frontend/test/protractor.conf.js
          new file mode 100644
          index 0000000000000..3386e66b9f115
          --- /dev/null
          +++ b/modules/web-console/frontend/test/protractor.conf.js
          @@ -0,0 +1,50 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one or more
          + * contributor license agreements.  See the NOTICE file distributed with
          + * this work for additional information regarding copyright ownership.
          + * The ASF licenses this file to You under the Apache License, Version 2.0
          + * (the "License"); you may not use this file except in compliance with
          + * the License.  You may obtain a copy of the License at
          + *
          + *      http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +// exports.config = {
          +//   specs: ['test/e2e/*.js'],
          +//   capabilities: {
          +
          +//   }
          +// };
          +
          +exports.config = {
          +  seleniumAddress: 'http://localhost:4444/wd/hub',
          +
          +  capabilities: {
          +    'browserName': 'chrome'
          +    // 'browserName': 'phantomjs',
          +
          +    // /*
          +    //  * Can be used to specify the phantomjs binary path.
          +    //  * This can generally be ommitted if you installed phantomjs globally.
          +    //  */
          +    // 'phantomjs.binary.path': require('phantomjs').path,
          +
          +    // /*
          +    //  * Command line args to pass to ghostdriver, phantomjs's browser driver.
          +    //  * See https://github.com/detro/ghostdriver#faq
          +    //  */
          +    // 'phantomjs.ghostdriver.cli.args': ['--loglevel=DEBUG']
          +  },
          +
          +  specs: ['test/e2e/*.js'],
          +
          +  jasmineNodeOpts: {
          +    showColors: true
          +  }
          +};
          diff --git a/modules/web-console/frontend/test/unit/JavaTypes.test.js b/modules/web-console/frontend/test/unit/JavaTypes.test.js
          new file mode 100644
          index 0000000000000..25c0f679edd54
          --- /dev/null
          +++ b/modules/web-console/frontend/test/unit/JavaTypes.test.js
          @@ -0,0 +1,69 @@
          +/*
          + * 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.
          + */
          +
          +import JavaTypes from '../../app/services/JavaTypes.service.js';
          +
          +import { assert } from 'chai';
          +
          +const { nonBuiltInClass, fullClassName, validIdentifier, validPackage, packageSpecified, isKeywords, isJavaPrimitive} = JavaTypes[1]();
          +
          +suite('JavaTypesTestsSuite', () => {
          +    test('nonBuiltInClass', () => {
          +        assert.equal(nonBuiltInClass('BigDecimal'), false);
          +        assert.equal(nonBuiltInClass('java.math.BigDecimal'), false);
          +
          +        assert.equal(nonBuiltInClass('String'), false);
          +        assert.equal(nonBuiltInClass('java.lang.String'), false);
          +
          +        assert.equal(nonBuiltInClass('Timestamp'), false);
          +        assert.equal(nonBuiltInClass('java.sql.Timestamp'), false);
          +
          +        assert.equal(nonBuiltInClass('Date'), false);
          +        assert.equal(nonBuiltInClass('java.sql.Date'), false);
          +
          +        assert.equal(nonBuiltInClass('Date'), false);
          +        assert.equal(nonBuiltInClass('java.util.Date'), false);
          +
          +        assert.equal(nonBuiltInClass('CustomClass'), true);
          +        assert.equal(nonBuiltInClass('java.util.CustomClass'), true);
          +        assert.equal(nonBuiltInClass('my.package.CustomClass'), true);
          +    });
          +
          +    test('fullClassName', () => {
          +        assert.equal(fullClassName('BigDecimal'), 'java.math.BigDecimal');
          +    });
          +
          +    test('validIdentifier', () => {
          +        assert.equal(validIdentifier('java.math.BigDecimal'), true);
          +    });
          +
          +    test('validPackage', () => {
          +        assert.equal(validPackage('java.math.BigDecimal'), true);
          +    });
          +
          +    test('packageSpecified', () => {
          +        assert.equal(packageSpecified('java.math.BigDecimal'), true);
          +    });
          +
          +    test('isKeywords', () => {
          +        assert.equal(isKeywords('abstract'), true);
          +    });
          +
          +    test('isJavaPrimitive', () => {
          +        assert.equal(isJavaPrimitive('boolean'), true);
          +    });
          +});
          diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/discovery.directive.js b/modules/web-console/frontend/test/unit/UserAuth.test.js
          similarity index 67%
          rename from modules/web-console/src/main/js/app/modules/states/configuration/clusters/discovery.directive.js
          rename to modules/web-console/frontend/test/unit/UserAuth.test.js
          index 80286ac4e2c54..dbba1f63fa747 100644
          --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/discovery.directive.js
          +++ b/modules/web-console/frontend/test/unit/UserAuth.test.js
          @@ -15,13 +15,21 @@
            * limitations under the License.
            */
           
          -import templateUrl from './discovery.jade';
          +import AuthService from '../../app/modules/user/Auth.service';
           
          -export default ['igniteConfigurationClustersDiscovery', [() => {
          -    return {
          -        scope: true,
          -        restrict: 'E',
          -        templateUrl,
          -        replace: true
          -    };
          -}]];
          +suite('AuthServiceTestsSuite', () => {
          +    test('SignIn', (done) => {
          +        // TODO IGNITE-3262 Add test.
          +        done();
          +    });
          +
          +    test('SignUp', (done) => {
          +        // TODO IGNITE-3262 Add test.
          +        done();
          +    });
          +
          +    test('Logout', (done) => {
          +        // TODO IGNITE-3262 Add test.
          +        done();
          +    });
          +});
          diff --git a/modules/web-console/frontend/views/403.jade b/modules/web-console/frontend/views/403.jade
          new file mode 100644
          index 0000000000000..38d8bb5257bb9
          --- /dev/null
          +++ b/modules/web-console/frontend/views/403.jade
          @@ -0,0 +1,22 @@
          +//-
          +    Licensed to the Apache Software Foundation (ASF) under one or more
          +    contributor license agreements.  See the NOTICE file distributed with
          +    this work for additional information regarding copyright ownership.
          +    The ASF licenses this file to You under the Apache License, Version 2.0
          +    (the "License"); you may not use this file except in compliance with
          +    the License.  You may obtain a copy of the License at
          +
          +         http://www.apache.org/licenses/LICENSE-2.0
          +
          +    Unless required by applicable law or agreed to in writing, software
          +    distributed under the License is distributed on an "AS IS" BASIS,
          +    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          +    See the License for the specific language governing permissions and
          +    limitations under the License.
          +
          +include includes/header
          +
          +.error-page
          +    .container
          +        h1.error-page__title 403
          +        h2.error-page__description You are not authorized
          diff --git a/modules/web-console/frontend/views/404.jade b/modules/web-console/frontend/views/404.jade
          new file mode 100644
          index 0000000000000..7d2fc55055f5f
          --- /dev/null
          +++ b/modules/web-console/frontend/views/404.jade
          @@ -0,0 +1,22 @@
          +//-
          +    Licensed to the Apache Software Foundation (ASF) under one or more
          +    contributor license agreements.  See the NOTICE file distributed with
          +    this work for additional information regarding copyright ownership.
          +    The ASF licenses this file to You under the Apache License, Version 2.0
          +    (the "License"); you may not use this file except in compliance with
          +    the License.  You may obtain a copy of the License at
          +
          +         http://www.apache.org/licenses/LICENSE-2.0
          +
          +    Unless required by applicable law or agreed to in writing, software
          +    distributed under the License is distributed on an "AS IS" BASIS,
          +    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          +    See the License for the specific language governing permissions and
          +    limitations under the License.
          +
          +include includes/header
          +
          +.error-page
          +    .container
          +        h1.error-page__title 404
          +        h2.error-page__description Page not found
          diff --git a/modules/web-console/src/main/js/views/base.jade b/modules/web-console/frontend/views/base.jade
          similarity index 100%
          rename from modules/web-console/src/main/js/views/base.jade
          rename to modules/web-console/frontend/views/base.jade
          diff --git a/modules/web-console/src/main/js/views/configuration/caches.jade b/modules/web-console/frontend/views/configuration/caches.jade
          similarity index 68%
          rename from modules/web-console/src/main/js/views/configuration/caches.jade
          rename to modules/web-console/frontend/views/configuration/caches.jade
          index a5b331f9e2dba..a1218ec7b83f2 100644
          --- a/modules/web-console/src/main/js/views/configuration/caches.jade
          +++ b/modules/web-console/frontend/views/configuration/caches.jade
          @@ -34,19 +34,20 @@ include ../../app/helpers/jade/mixins.jade
                           hr
                       .bs-affix-fix
                       div(bs-collapse='' data-allow-multiple='true' ng-model='ui.activePanels')
          -                form.form-horizontal(name='ui.inputForm' ng-show='contentVisible()' novalidate)
          +                form.form-horizontal(name='ui.inputForm' novalidate ng-if='contentVisible()')
                               .panel-group
          -                        ignite-configuration-caches-general
          -                        ignite-configuration-caches-memory
          -                        ignite-configuration-caches-query
          -                        ignite-configuration-caches-store
          +                        include ../../app/modules/states/configuration/caches/general.jade
          +                        include ../../app/modules/states/configuration/caches/memory.jade
          +                        include ../../app/modules/states/configuration/caches/query.jade
          +                        include ../../app/modules/states/configuration/caches/store.jade
           
                                   +advanced-options-toggle-default
           
                                   div(ng-show='ui.expanded')
          -                            ignite-configuration-caches-concurrency
          -                            ignite-configuration-caches-rebalance
          -                            ignite-configuration-caches-server-near-cache
          -                            ignite-configuration-caches-statistics
          +                            include ../../app/modules/states/configuration/caches/concurrency.jade
          +                            include ../../app/modules/states/configuration/caches/node-filter.jade
          +                            include ../../app/modules/states/configuration/caches/rebalance.jade
          +                            include ../../app/modules/states/configuration/caches/server-near-cache.jade
          +                            include ../../app/modules/states/configuration/caches/statistics.jade
           
                                       +advanced-options-toggle-default
          diff --git a/modules/web-console/frontend/views/configuration/clusters.jade b/modules/web-console/frontend/views/configuration/clusters.jade
          new file mode 100644
          index 0000000000000..b10a477448ceb
          --- /dev/null
          +++ b/modules/web-console/frontend/views/configuration/clusters.jade
          @@ -0,0 +1,66 @@
          +//-
          +    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.
          +
          +include ../../app/helpers/jade/mixins.jade
          +
          +.docs-header
          +    h1 Configure Ignite Clusters
          +.docs-body(ng-controller='clustersController')
          +    ignite-information
          +        ul
          +            li Configure #[a(href='https://apacheignite.readme.io/docs/clustering' target='_blank') clusters] properties
          +            li Associate clusters with caches and in-memory file systems
          +    div(ignite-loading='loadingClustersScreen' ignite-loading-text='Loading clusters...' ignite-loading-position='top')
          +        div(ng-show='ui.ready')
          +            hr
          +            +main-table('clusters', 'clusters', 'clusterName', 'selectItem(row)', '{{$index + 1}}) {{row.label}}', 'label')
          +            .padding-top-dflt(bs-affix)
          +                .panel-tip-container(data-placement='bottom' bs-tooltip='' data-title='Create new cluster')
          +                    button.btn.btn-primary(id='new-item' ng-click='createItem()') Add cluster
          +                +save-remove-clone-undo-buttons('cluster')
          +                hr
          +            .bs-affix-fix
          +            div(bs-collapse='' data-allow-multiple='true' ng-model='ui.activePanels')
          +                form.form-horizontal(name='ui.inputForm' novalidate ng-if='contentVisible()')
          +                    .panel-group
          +                        include ../../app/modules/states/configuration/clusters/general.jade
          +
          +                        +advanced-options-toggle-default
          +
          +                        div(ng-show='ui.expanded')
          +                            
          +                            include ../../app/modules/states/configuration/clusters/atomic.jade
          +                            include ../../app/modules/states/configuration/clusters/binary.jade
          +                            include ../../app/modules/states/configuration/clusters/cache-key-cfg.jade
          +                            include ../../app/modules/states/configuration/clusters/collision.jade
          +                            include ../../app/modules/states/configuration/clusters/communication.jade
          +                            include ../../app/modules/states/configuration/clusters/connector.jade
          +                            include ../../app/modules/states/configuration/clusters/deployment.jade
          +                            include ../../app/modules/states/configuration/clusters/discovery.jade
          +                            include ../../app/modules/states/configuration/clusters/events.jade
          +                            include ../../app/modules/states/configuration/clusters/failover.jade
          +                            include ../../app/modules/states/configuration/clusters/igfs.jade
          +                            include ../../app/modules/states/configuration/clusters/logger.jade
          +                            include ../../app/modules/states/configuration/clusters/marshaller.jade
          +                            include ../../app/modules/states/configuration/clusters/metrics.jade
          +                            include ../../app/modules/states/configuration/clusters/ssl.jade
          +                            include ../../app/modules/states/configuration/clusters/swap.jade
          +                            include ../../app/modules/states/configuration/clusters/thread.jade
          +                            include ../../app/modules/states/configuration/clusters/time.jade
          +                            include ../../app/modules/states/configuration/clusters/transactions.jade
          +                            include ../../app/modules/states/configuration/clusters/attributes.jade
          +
          +                            +advanced-options-toggle-default
          diff --git a/modules/web-console/src/main/js/views/configuration/domains-import.jade b/modules/web-console/frontend/views/configuration/domains-import.jade
          similarity index 87%
          rename from modules/web-console/src/main/js/views/configuration/domains-import.jade
          rename to modules/web-console/frontend/views/configuration/domains-import.jade
          index ccaeb27bf6fd0..d95e98ab626ba 100644
          --- a/modules/web-console/src/main/js/views/configuration/domains-import.jade
          +++ b/modules/web-console/frontend/views/configuration/domains-import.jade
          @@ -38,7 +38,7 @@ mixin td-ellipses-lbl(w, lbl)
                                   li Agent failed to find JDBC drivers
                                   li Copy required JDBC drivers into agent 'jdbc-drivers' folder and try again
                                   li Refer to agent README.txt for more information
          -                .import-domain-model-wizard-page(ng-show='importDomain.action == "connect" && importDomain.demo')
          +                .import-domain-model-wizard-page(ng-if='importDomain.action == "connect" && importDomain.demo')
                               div(ng-if='demoConnection.db == "H2"')
                                   label Demo description:
                                   ul
          @@ -52,26 +52,40 @@ mixin td-ellipses-lbl(w, lbl)
                                           li Agent failed to resolve H2 database jar
                                           li Copy h2-x.x.x.jar into agent 'jdbc-drivers' folder and try again
                                           li Refer to agent README.txt for more information
          -                .import-domain-model-wizard-page(ng-show='importDomain.action == "connect" && !importDomain.demo')
          -                    form.form-horizontal(name='connectForm' novalidate)
          +                .import-domain-model-wizard-page(ng-if='importDomain.action == "connect" && !importDomain.demo')
          +                    - var form = 'connectForm'
          +
          +                    form.form-horizontal(name=form novalidate)
                                   .settings-row
          -                            label.col-xs-4.col-sm-2.col-md-2 Driver JAR:
          +                            label.col-xs-4.col-sm-2.col-md-2.required Driver JAR:
                                       .col-xs-8.col-sm-10.col-md-10
                                           i.tipField.fa.fa-question-circle(bs-tooltip='' data-title='Select appropriate JAR with JDBC driver
          To add another driver you need to place it into "/jdbc-drivers" folder of Ignite Web Agent
          Refer to Ignite Web Agent README.txt for for more information') .input-tip button.select-toggle.form-control(id='jdbcDriverJar' bs-select data-container='.modal-domain-import' ng-model='ui.selectedJdbcDriverJar' ng-class='{placeholder: !(jdbcDriverJars && jdbcDriverJars.length > 0)}' placeholder='Choose JDBC driver' bs-options='item.value as item.label for item in jdbcDriverJars') .settings-row - label.col-xs-4.col-sm-2.col-md-2 JDBC Driver: + label.col-xs-4.col-sm-2.col-md-2.required JDBC driver: .col-xs-8.col-sm-10.col-md-10 + - var name = '"jdbcDriverClass"' i.tipField.fa.fa-question-circle(bs-tooltip='' data-title='Fully qualified class name of JDBC driver that will be used to connect to database') + + +form-field-feedback(name, 'javaBuiltInClass', 'JDBC Driver should not be the Java built-in class!') + +form-field-feedback(name, 'javaKeywords', 'JDBC Driver could not contains reserved Java keyword!') + +form-field-feedback(name, 'javaPackageSpecified', 'JDBC Driver does not have package specified!') + +form-field-feedback(name, 'javaIdentifier', 'JDBC Driver is invalid Java identifier!') + .input-tip - input.form-control(id='jdbcDriverClass' type='text' ng-model='selectedPreset.jdbcDriverClass' placeholder='JDBC driver fully qualified class name' required=true) + +ignite-form-field-input(name, 'selectedPreset.jdbcDriverClass', false, true, 'Enter fully qualified class name')( + data-java-identifier='true' + data-java-package-specified='true' + data-java-keywords='true' + data-java-built-in-class='true' + ) .settings-row - label.col-xs-4.col-sm-2.col-md-2 JDBC URL: + label.col-xs-4.col-sm-2.col-md-2.required JDBC URL: .col-xs-8.col-sm-10.col-md-10 i.tipField.fa.fa-question-circle(bs-tooltip='' data-title='JDBC URL for connecting to database
          Refer to your database documentation for details') .input-tip - input.form-control(id='jdbcUrl' type='text' ng-model='selectedPreset.jdbcUrl' placeholder='JDBC URL' required=true) + +ignite-form-field-input('jdbcUrl', 'selectedPreset.jdbcUrl', false, true, 'JDBC URL') .settings-row label.col-xs-4.col-sm-2.col-md-2 User: .col-xs-8.col-sm-10.col-md-10 @@ -160,26 +174,24 @@ mixin td-ellipses-lbl(w, lbl) .col-sm-1(style='padding-left: 5px') button.btn.btn-primary(ng-click='applyDefaults()') Apply .import-domain-model-wizard-page(ng-show='importDomain.action == "options"') + - var form = 'optionsForm' form.form-horizontal(name='optionsForm' novalidate) .settings-row .col-xs-3.col-sm-2.col-md-2.required label.required Package: .col-xs-9.col-sm-10.col-md-10 + - var name = '"domainPackageName"' i.tipField.fa.fa-question-circle(bs-tooltip='' data-title='Package that will be used for POJOs generation') + + +form-field-feedback(name, 'javaPackageName', 'Package name is invalid') + +form-field-feedback(name, 'javaKeywords', 'Package name could not contains reserved java keyword') + .input-tip - ignite-form-field-input-text( - data-id='domainPackageName' - data-name='domainPackageName' - data-ng-model='ui.packageName' - data-ignite-label-name='Package' - data-ng-required='true' - data-placeholder='Enter package name' + +ignite-form-field-input(name, 'ui.packageName', false, true, 'Enter package name')( data-java-keywords='true' data-java-package-name='package-only' ng-model-options='{allowInvalid: true}' ) - +error-feedback('optionsForm.$error.javaPackageName', 'javaPackageName', 'Package name is invalid') - +error-feedback('optionsForm.$error.javaKeywords', 'javaKeywords', 'Package name could not contains reserved java keyword') .settings-row .checkbox label @@ -197,7 +209,7 @@ mixin td-ellipses-lbl(w, lbl) label input(id='domainGenerateAliases' type='checkbox' ng-model='ui.generateAliases') | Generate aliases for query fields - i.tipLabel.fa.fa-question-circle(bs-tooltip='' data-title='Generate aliases for query fields with names from database fields') + i.tipLabel.fa.fa-question-circle(bs-tooltip='' data-title='Generate aliases for query fields with database field names when database field name differ from Java field name') .settings-row .col-xs-3.col-sm-2.col-md-2.required label Clusters: diff --git a/modules/web-console/src/main/js/views/configuration/domains.jade b/modules/web-console/frontend/views/configuration/domains.jade similarity index 91% rename from modules/web-console/src/main/js/views/configuration/domains.jade rename to modules/web-console/frontend/views/configuration/domains.jade index 7f3253e2989e5..591234333aa10 100644 --- a/modules/web-console/src/main/js/views/configuration/domains.jade +++ b/modules/web-console/frontend/views/configuration/domains.jade @@ -59,8 +59,8 @@ include ../../app/helpers/jade/mixins.jade hr .bs-affix-fix div(bs-collapse='' data-allow-multiple='true' ng-model='ui.activePanels') - form.form-horizontal(name='ui.inputForm' ng-show='contentVisible()' novalidate) + form.form-horizontal(name='ui.inputForm' novalidate ng-if='contentVisible()') .panel-group - ignite-configuration-domains-general - ignite-configuration-domains-query - ignite-configuration-domains-store + include ../../app/modules/states/configuration/domains/general.jade + include ../../app/modules/states/configuration/domains/query.jade + include ../../app/modules/states/configuration/domains/store.jade diff --git a/modules/web-console/src/main/js/views/configuration/igfs.jade b/modules/web-console/frontend/views/configuration/igfs.jade similarity index 80% rename from modules/web-console/src/main/js/views/configuration/igfs.jade rename to modules/web-console/frontend/views/configuration/igfs.jade index b889a97158861..16e9f280ac3b8 100644 --- a/modules/web-console/src/main/js/views/configuration/igfs.jade +++ b/modules/web-console/frontend/views/configuration/igfs.jade @@ -35,17 +35,17 @@ include ../../app/helpers/jade/mixins.jade hr .bs-affix-fix div(bs-collapse='' data-allow-multiple='true' ng-model='ui.activePanels') - form.form-horizontal(name='ui.inputForm' ng-show='contentVisible()' novalidate) + form.form-horizontal(name='ui.inputForm' novalidate ng-if='contentVisible()') .panel-group - ignite-configuration-igfs-general + include ../../app/modules/states/configuration/igfs/general.jade +advanced-options-toggle-default div(ng-show='ui.expanded') - ignite-configuration-igfs-secondary - ignite-configuration-igfs-ipc - ignite-configuration-igfs-fragmentizer - ignite-configuration-igfs-dual - ignite-configuration-igfs-misc + include ../../app/modules/states/configuration/igfs/secondary.jade + include ../../app/modules/states/configuration/igfs/ipc.jade + include ../../app/modules/states/configuration/igfs/fragmentizer.jade + include ../../app/modules/states/configuration/igfs/dual.jade + include ../../app/modules/states/configuration/igfs/misc.jade +advanced-options-toggle-default diff --git a/modules/web-console/src/main/js/views/configuration/sidebar.jade b/modules/web-console/frontend/views/configuration/sidebar.jade similarity index 100% rename from modules/web-console/src/main/js/views/configuration/sidebar.jade rename to modules/web-console/frontend/views/configuration/sidebar.jade diff --git a/modules/web-console/src/main/js/views/configuration/summary-project-structure.jade b/modules/web-console/frontend/views/configuration/summary-project-structure.jade similarity index 100% rename from modules/web-console/src/main/js/views/configuration/summary-project-structure.jade rename to modules/web-console/frontend/views/configuration/summary-project-structure.jade diff --git a/modules/web-console/src/main/js/views/configuration/summary-tabs.jade b/modules/web-console/frontend/views/configuration/summary-tabs.jade similarity index 100% rename from modules/web-console/src/main/js/views/configuration/summary-tabs.jade rename to modules/web-console/frontend/views/configuration/summary-tabs.jade diff --git a/modules/web-console/src/main/js/views/configuration/summary.jade b/modules/web-console/frontend/views/configuration/summary.jade similarity index 68% rename from modules/web-console/src/main/js/views/configuration/summary.jade rename to modules/web-console/frontend/views/configuration/summary.jade index f367e671a18d4..0d30df86eafc6 100644 --- a/modules/web-console/src/main/js/views/configuration/summary.jade +++ b/modules/web-console/frontend/views/configuration/summary.jade @@ -70,37 +70,25 @@ mixin hard-link(ref, txt) label Client .panel-collapse(id='client' role='tabpanel' bs-collapse-target) - form(name='clientForm' novalidate) + -var form = 'clientForm' + form(name=form novalidate) -var nearCfg = 'ctrl.cluster.clientNearCfg' -var nearCfgEvictionPolicy = nearCfg + '.nearEvictionPolicy[' + nearCfg + '.nearEvictionPolicy.kind]' .group-content - .settings-row - ignite-form-field.col-xs-8.col-sm-8.col-md-7(data-label='Near cache start size') - ignite-form-field-tooltip - | Initial cache size for near cache which will be used to pre-create internal hash table after start - ignite-form-field-input-number( - data-name='nearStartSize' - data-ng-model='#{nearCfg}.nearStartSize' - data-placeholder='375000' - ) + .settings-row(ng-if='true') + .col-xs-8.col-sm-8.col-md-7 + +ignite-form-field-number('Near cache start size:', nearCfg + '.nearStartSize', '"nearStartSize"', false, false, '375000', false, false, false, 'Initial cache size for near cache which will be used to pre-create internal hash table after start') - .settings-row - ignite-form-field.col-xs-8.col-sm-8.col-md-7(data-label='Near cache eviction policy') - ignite-form-field-tooltip - | Near cache eviction policy - ignite-form-field-dropdown( - data-id='evictionPolicies' - data-name='evictionPolicies' - data-placeholder='Not set' - data-options='[\ + .settings-row(ng-if='true') + .col-xs-8.col-sm-8.col-md-7 + +ignite-form-field-dropdown('Near cache eviction policy', nearCfg + '.nearEvictionPolicy.kind', '"evictionPolicies"', false, false, false, 'Not set', false, '[\ {value: "LRU", label: "LRU"},\ {value: "FIFO", label: "FIFO"},\ {value: "SORTED", label: "Sorted"},\ {value: undefined, label: "Not set"}\ - ]' - data-ng-model='#{nearCfg}.nearEvictionPolicy.kind' - ) + ]', 'Near cache eviction policy') + span(ng-if='#{nearCfg}.nearEvictionPolicy.kind') a.customize( ng-show='ctrl.__form.expanded' @@ -114,32 +102,14 @@ mixin hard-link(ref, txt) .settings-row .panel-details.col-xs-12.col-sm-12.col-md-7(ng-if='ctrl.__form.expanded && #{nearCfg}.nearEvictionPolicy.kind') .details-row - ignite-form-field(data-label='Batch size') - ignite-form-field-tooltip - | Number of entries to remove on shrink - ignite-form-field-input-number( - data-name='batchSize' - data-ng-model='#{nearCfgEvictionPolicy}.batchSize' - data-placeholder='1' - ) + +ignite-form-field-number('Batch size:', nearCfgEvictionPolicy + '.batchSize', '"batchSize"', false, false, '1', false, false, false, 'Number of entries to remove on shrink') + .details-row - ignite-form-field(data-label='Max memory size') - ignite-form-field-tooltip - | Maximum allowed cache size in bytes - ignite-form-field-input-number( - data-name='maxMemorySize' - data-ng-model='#{nearCfgEvictionPolicy}.maxMemorySize' - data-placeholder='0' - ) + +ignite-form-field-number('Max memory size:', nearCfgEvictionPolicy + '.maxMemorySize', '"maxMemorySize"', false, false, '0', false, false, false, 'Maximum allowed cache size in bytes') + .details-row - ignite-form-field(data-label='Max size') - ignite-form-field-tooltip - | Maximum allowed size of cache before entry will start getting evicted - ignite-form-field-input-number( - data-name='maxSize' - data-ng-model='#{nearCfgEvictionPolicy}.maxSize' - data-placeholder='100000' - ) + +ignite-form-field-number('Max size:', nearCfgEvictionPolicy + '.maxSize', '"maxSize"', false, false, '100000', false, false, false, 'Maximum allowed size of cache before entry will start getting evicted') + .summary-tabs(ignite-ui-ace-tabs) div(bs-tabs data-bs-active-pane="tabsClient.activeTab" template='configuration/summary-tabs.html') div(bs-pane title='XML') diff --git a/modules/web-console/src/main/js/views/includes/footer.jade b/modules/web-console/frontend/views/includes/footer.jade similarity index 100% rename from modules/web-console/src/main/js/views/includes/footer.jade rename to modules/web-console/frontend/views/includes/footer.jade diff --git a/modules/web-console/src/main/js/views/includes/header.jade b/modules/web-console/frontend/views/includes/header.jade similarity index 89% rename from modules/web-console/src/main/js/views/includes/header.jade rename to modules/web-console/frontend/views/includes/header.jade index e224f64ff4991..9ef09aa71c304 100644 --- a/modules/web-console/src/main/js/views/includes/header.jade +++ b/modules/web-console/frontend/views/includes/header.jade @@ -27,14 +27,14 @@ header#header.header a.dropdown-toggle Configuration span.caret - ul.nav.navbar-nav(ng-controller='notebooks') + ul.nav.navbar-nav(ng-controller='notebookController') li.sql-notebooks(ng-if='IgniteDemoMode' ng-class='{active: $state.includes("base.sql")}') a(ui-sref='base.sql.demo') SQL li.sql-notebooks(ng-if='!IgniteDemoMode && !notebooks.length' ng-class='{active: $state.includes("base.sql")}') - a(ng-click='inputNotebookName()') SQL + a(ng-click='createNotebook()') SQL - li.sql-notebooks(ng-if='!IgniteDemoMode && notebooks.length' ng-class='{active: $state.includes("base.sql")}' bs-dropdown='notebookDropdown' data-placement='bottom-left' data-trigger='hover focus' data-container='self' data-animation='' ng-click='$event.stopPropagation()') + li.sql-notebooks(ng-if='!IgniteDemoMode && notebooks.length' ng-class='{active: $state.includes("base.sql")}' bs-dropdown='notebooks' data-placement='bottom-left' data-trigger='hover focus' data-container='self' data-animation='' ng-click='$event.stopPropagation()' aria-haspopup="true" aria-expanded="false") a.dropdown-toggle SQL span.caret diff --git a/modules/web-console/src/main/js/views/index.jade b/modules/web-console/frontend/views/index.jade similarity index 99% rename from modules/web-console/src/main/js/views/index.jade rename to modules/web-console/frontend/views/index.jade index 707bfac3b4ba7..c561c3e9c9564 100644 --- a/modules/web-console/src/main/js/views/index.jade +++ b/modules/web-console/frontend/views/index.jade @@ -40,7 +40,6 @@ html(ng-app='ignite-console' id='app' ng-strict-di) .splash-wellcome Loading... - .ribbon-wrapper.right(ng-cloak) .ribbon(ng-style='IgniteDemoMode && {"background": "#1b6d88"}') label {{IgniteDemoMode ? "Demo" : "Beta" }} diff --git a/modules/web-console/src/main/js/views/reset.jade b/modules/web-console/frontend/views/reset.jade similarity index 100% rename from modules/web-console/src/main/js/views/reset.jade rename to modules/web-console/frontend/views/reset.jade diff --git a/modules/web-console/src/main/js/views/settings/admin.jade b/modules/web-console/frontend/views/settings/admin.jade similarity index 100% rename from modules/web-console/src/main/js/views/settings/admin.jade rename to modules/web-console/frontend/views/settings/admin.jade diff --git a/modules/web-console/src/main/js/views/settings/profile.jade b/modules/web-console/frontend/views/settings/profile.jade similarity index 100% rename from modules/web-console/src/main/js/views/settings/profile.jade rename to modules/web-console/frontend/views/settings/profile.jade diff --git a/modules/web-console/src/main/js/views/signin.jade b/modules/web-console/frontend/views/signin.jade similarity index 100% rename from modules/web-console/src/main/js/views/signin.jade rename to modules/web-console/frontend/views/signin.jade diff --git a/modules/web-console/src/main/js/views/sql/cache-metadata.jade b/modules/web-console/frontend/views/sql/cache-metadata.jade similarity index 100% rename from modules/web-console/src/main/js/views/sql/cache-metadata.jade rename to modules/web-console/frontend/views/sql/cache-metadata.jade diff --git a/modules/web-console/src/main/js/views/sql/chart-settings.jade b/modules/web-console/frontend/views/sql/chart-settings.jade similarity index 100% rename from modules/web-console/src/main/js/views/sql/chart-settings.jade rename to modules/web-console/frontend/views/sql/chart-settings.jade diff --git a/modules/web-console/src/main/js/views/sql/notebook-new.jade b/modules/web-console/frontend/views/sql/notebook-new.jade similarity index 92% rename from modules/web-console/src/main/js/views/sql/notebook-new.jade rename to modules/web-console/frontend/views/sql/notebook-new.jade index 51200e801ff33..09b2daed618a7 100644 --- a/modules/web-console/src/main/js/views/sql/notebook-new.jade +++ b/modules/web-console/frontend/views/sql/notebook-new.jade @@ -25,7 +25,7 @@ .col-sm-2 label.required.labelFormField Name:  .col-sm-10 - input.form-control(id='create-notebook' type='text' ng-model='name' required ignite-on-enter='ui.inputForm.$valid && createNewNotebook(name)' ignite-auto-focus) + input.form-control(id='create-notebook' type='text' ng-model='name' required ignite-on-enter='ui.inputForm.$valid && create(name)' ignite-auto-focus) .modal-footer button.btn.btn-default(id='copy-btn-cancel' ng-click='$hide()') Cancel - button.btn.btn-primary(id='copy-btn-confirm' ng-disabled='ui.inputForm.$invalid' ng-click='createNewNotebook(name)') Create + button.btn.btn-primary(id='copy-btn-confirm' ng-disabled='ui.inputForm.$invalid' ng-click='create(name)') Create diff --git a/modules/web-console/src/main/js/views/sql/paragraph-rate.jade b/modules/web-console/frontend/views/sql/paragraph-rate.jade similarity index 100% rename from modules/web-console/src/main/js/views/sql/paragraph-rate.jade rename to modules/web-console/frontend/views/sql/paragraph-rate.jade diff --git a/modules/web-console/src/main/js/views/sql/sql.jade b/modules/web-console/frontend/views/sql/sql.jade similarity index 77% rename from modules/web-console/src/main/js/views/sql/sql.jade rename to modules/web-console/frontend/views/sql/sql.jade index a5e7fae303052..81acdfd5b53cb 100644 --- a/modules/web-console/src/main/js/views/sql/sql.jade +++ b/modules/web-console/frontend/views/sql/sql.jade @@ -28,8 +28,8 @@ mixin result-toolbar +btn-toolbar-data('fa-line-chart', 'line', 'Show line chart
          By default first column - X values, second column - Y values
          In case of one column it will be treated as Y values') +btn-toolbar-data('fa-area-chart', 'area', 'Show area chart
          By default first column - X values, second column - Y values
          In case of one column it will be treated as Y values') -mixin chart-settings(mdl) - .row +mixin chart-settings + .total.row .col-xs-4 .chart-settings-link(ng-show='paragraph.chart && paragraph.chartColumns.length > 0') a(title='Click to show chart settings dialog' ng-click='$event.stopPropagation()' bs-popover data-template-url='/sql/chart-settings.html' data-placement='bottom' data-auto-close='1' data-trigger='click') @@ -48,7 +48,7 @@ mixin notebook-rename label(style='max-width: calc(100% - 60px)') {{notebook.name}} .btn-group(ng-if='!demo') +btn-toolbar('fa-pencil', 'notebook.edit = true;notebook.editName = notebook.name', 'Rename notebook') - +btn-toolbar('fa-trash', 'removeNotebook()', 'Remove notebook') + +btn-toolbar('fa-trash', 'removeNotebook(notebook)', 'Remove notebook') h1.col-sm-6(ng-show='notebook.edit') i.btn.fa.fa-floppy-o(ng-show='notebook.editName' ng-click='renameNotebook(notebook.editName)' bs-tooltip data-title='Save notebook name' data-trigger='hover') .input-tip @@ -84,11 +84,15 @@ mixin paragraph-rename mixin query-controls .sql-controls - a.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='actionAvailable(paragraph, true) && execute(paragraph)' data-placement='bottom' bs-tooltip data-title='{{actionTooltip(paragraph, "execute", true)}}') Execute - a.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='actionAvailable(paragraph, true) && explain(paragraph)' data-placement='bottom' bs-tooltip data-title='{{actionTooltip(paragraph, "explain", true)}}') Explain - a.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' ng-click='actionAvailable(paragraph, false) && scan(paragraph)' data-placement='bottom' bs-tooltip data-title='{{actionTooltip(paragraph, "execute scan", false)}}') Scan + a.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph)' data-placement='bottom' bs-tooltip='' data-title='{{actionTooltip(paragraph, "execute", true)}}') Execute + a.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='explain(paragraph)' data-placement='bottom' bs-tooltip='' data-title='{{actionTooltip(paragraph, "explain", true)}}') Explain + .btn-group(bs-tooltip='' data-title='{{actionTooltip(paragraph, "execute scan", false)}}' data-placement='bottom') + a.btn.btn-primary.fieldButton(ng-disabled='!actionAvailable(paragraph, false)' ng-click='scan(paragraph)') Scan + a.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' data-toggle='dropdown' data-container='body' bs-dropdown='[{ text: "Scan with filter", click: "actionAvailable(paragraph, false) && scanWithFilter(paragraph)" }]') + span.caret + .pull-right - labelHide System columns: + label.tipLabel System columns: a.btn.btn-default.fa.fa-bars.tipLabel(ng-class='{"btn-info": paragraph.systemColumns}' ng-click='toggleSystemColumns(paragraph)' ng-disabled='paragraph.disabledSystemColumns' bs-tooltip data-title='Show "_KEY", "_VAL" columns') label.tipLabel Refresh rate: button.btn.btn-default.fa.fa-clock-o.tipLabel(title='Click to show refresh rate dialog' ng-class='{"btn-info": paragraph.rate && paragraph.rate.installed}' bs-popover data-template-url='/sql/paragraph-rate.html' data-placement='left' data-auto-close='1' data-trigger='click') {{rateAsString(paragraph)}} @@ -96,10 +100,11 @@ mixin query-controls button.select-toggle.fieldButton.btn.btn-default(ng-model='paragraph.pageSize' bs-options='item for item in pageSizes' bs-select bs-tooltip data-placement='bottom-right' data-title='Max number of rows to show in query result as one page') mixin table-result - .sql-table-total.row + .total.row .col-xs-4 - label(style='margin-right: 10px;') Page: #[b {{paragraph.page}}] - label Results so far: #[b {{paragraph.rows.length + paragraph.total}}] + label Page: #[b {{paragraph.page}}] + label.margin-left-dflt Results so far: #[b {{paragraph.rows.length + paragraph.total}}] + label.margin-left-dflt Duration: #[b {{paragraph.duration | duration}}] .col-xs-4 +result-toolbar .col-xs-4 @@ -110,33 +115,21 @@ mixin table-result .grid(ui-grid='paragraph.gridOptions' ui-grid-resize-columns ui-grid-exporter) mixin chart-result - div(ng-show='paragraph.queryExecuted()') + div(ng-hide='paragraph.scanExplain()') +chart-settings - div(ng-show='paragraph.chartColumns.length > 0 && !paragraph.chartColumnsConfigured()') - .sql-empty-result Cannot display chart. Please configure axis using #[b Chart settings] - div(ng-show='paragraph.chartColumns.length == 0') - .sql-empty-result Cannot display chart. Result set must contain Java build-in type columns. Please change query and execute it again. + .empty(ng-show='paragraph.chartColumns.length > 0 && !paragraph.chartColumnsConfigured()') Cannot display chart. Please configure axis using #[b Chart settings] + .empty(ng-show='paragraph.chartColumns.length == 0') Cannot display chart. Result set must contain Java build-in type columns. Please change query and execute it again. div(ng-show='paragraph.chartColumnsConfigured()') div(ng-show='paragraph.timeLineSupported() || !paragraph.chartTimeLineEnabled()') div(ng-repeat='chart in paragraph.charts') nvd3(options='chart.options' data='chart.data' api='chart.api') - .sql-empty-result(ng-show='!paragraph.timeLineSupported() && paragraph.chartTimeLineEnabled()') Pie chart does not support 'TIME_LINE' column for X-axis. Please use another column for X-axis or switch to another chart. - .sql-empty-result(ng-hide='paragraph.queryExecuted()') + .empty(ng-show='!paragraph.timeLineSupported() && paragraph.chartTimeLineEnabled()') Pie chart does not support 'TIME_LINE' column for X-axis. Please use another column for X-axis or switch to another chart. + .empty(ng-show='paragraph.scanExplain()') .row .col-xs-4.col-xs-offset-4 +result-toolbar label.margin-top-dflt Charts do not support #[b Explain] and #[b Scan] query -mixin footer-controls - hr(style='margin-top: 0; margin-bottom: 5px') - a(style='float: left; margin-left: 10px; margin-bottom: 5px' ng-click='showResultQuery(paragraph)') Show query - - -var nextVisibleCondition = 'paragraph.queryId && (paragraph.table() || paragraph.chart() && (paragraph.timeLineSupported() || !paragraph.chartTimeLineEnabled()))' - - .sql-next(ng-show=nextVisibleCondition) - i.fa.fa-chevron-circle-right(ng-class='{disabled: paragraph.loading}' ng-click='!paragraph.loading && nextPage(paragraph)') - a(ng-class='{disabled: paragraph.loading}' ng-click='!paragraph.loading && nextPage(paragraph)') Next - .row(ng-controller='sqlController') .docs-content .row(ng-if='notebook' bs-affix style='margin-bottom: 20px;') @@ -155,8 +148,8 @@ mixin footer-controls div(ng-if='notebook' ignite-loading='sqlLoading' ignite-loading-text='{{ loadingText }}' ignite-loading-position='top') .docs-body.paragraphs .panel-group(bs-collapse ng-model='notebook.expandedParagraphs' data-allow-multiple='true' data-start-collapsed='false') - .panel.panel-default(ng-repeat='paragraph in notebook.paragraphs') - .panel-heading(id='{{paragraph.id}}' bs-collapse-toggle) + .panel.panel-default(ng-repeat='paragraph in notebook.paragraphs' id='{{paragraph.id}}') + .panel-heading(bs-collapse-toggle) .row +paragraph-rename .panel-collapse(role='tabpanel' bs-collapse-target) @@ -168,7 +161,7 @@ mixin footer-controls .col-xs-4.col-sm-3 div(ng-show='caches.length > 0' style='padding: 5px 10px' st-table='displayedCaches' st-safe-src='caches') lable.labelField.labelFormField Caches: - i.fa.fa-database.tipField(title='Click to show cache types metadata dialog' bs-popover data-template-url='/sql/cache-metadata.html', data-placement='bottom', data-trigger='click') + i.fa.fa-database.tipField(title='Click to show cache types metadata dialog' bs-popover data-template-url='/sql/cache-metadata.html' data-placement='bottom' data-trigger='click' data-container='#{{ paragraph.id}}') .input-tip input.form-control(type='text' st-search='label' placeholder='Filter caches...') table.links @@ -181,21 +174,20 @@ mixin footer-controls label Wrong caches filter .empty-caches(ng-show='caches.length == 0') label No caches - .col-sm-12 - hr(style='margin: 0') .col-sm-12 +query-controls - .col-sm-12.sql-error-result(ng-show='paragraph.errMsg') Error: {{paragraph.errMsg}} - .col-sm-12(ng-show='!paragraph.errMsg && paragraph.queryArgs') - hr(style='margin-top: 0; margin-bottom: 10px') - - .sql-empty-result(ng-show='!paragraph.nonEmpty()') Result set is empty - - div(ng-show='paragraph.table() && paragraph.nonEmpty()') + .col-sm-12.sql-result(ng-if='paragraph.queryExecuted()' ng-switch='paragraph.resultType()') + .error(ng-switch-when='error') Error: {{paragraph.errMsg}} + .empty(ng-switch-when='empty') Result set is empty + .table(ng-switch-when='table') +table-result - - div(ng-show='paragraph.chart() && paragraph.nonEmpty()') + .chart(ng-switch-when='chart') +chart-result + .footer.clearfix(ng-show='paragraph.nonRefresh()') + a.pull-left(ng-click='showResultQuery(paragraph)') Show query + + -var nextVisibleCondition = 'paragraph.queryId && (paragraph.table() || paragraph.chart() && (paragraph.timeLineSupported() || !paragraph.chartTimeLineEnabled()))' - div(ng-show='!paragraph.refreshExecuting()') - +footer-controls + .pull-right(ng-show=nextVisibleCondition ng-class='{disabled: paragraph.loading}' ng-click='!paragraph.loading && nextPage(paragraph)') + i.fa.fa-chevron-circle-right + a Next diff --git a/modules/web-console/src/main/js/views/templates/agent-download.jade b/modules/web-console/frontend/views/templates/agent-download.jade similarity index 100% rename from modules/web-console/src/main/js/views/templates/agent-download.jade rename to modules/web-console/frontend/views/templates/agent-download.jade diff --git a/modules/web-console/src/main/js/views/templates/alert.jade b/modules/web-console/frontend/views/templates/alert.jade similarity index 96% rename from modules/web-console/src/main/js/views/templates/alert.jade rename to modules/web-console/frontend/views/templates/alert.jade index 0ab3dd9c8a608..182ba997d06b3 100644 --- a/modules/web-console/src/main/js/views/templates/alert.jade +++ b/modules/web-console/frontend/views/templates/alert.jade @@ -17,5 +17,5 @@ .alert(ng-show='type' ng-class='[type ? "alert-" + type : null]') button.close(type='button', ng-if='dismissable', ng-click='$hide()') × i.alert-icon.fa(ng-if='icon' ng-class='[icon]') - span.alert-title(ng-bind='title') + span.alert-title(ng-bind-html='title') span.alert-content(ng-bind-html='content') diff --git a/modules/web-console/src/main/js/views/templates/batch-confirm.jade b/modules/web-console/frontend/views/templates/batch-confirm.jade similarity index 100% rename from modules/web-console/src/main/js/views/templates/batch-confirm.jade rename to modules/web-console/frontend/views/templates/batch-confirm.jade diff --git a/modules/web-console/src/main/js/views/templates/clone.jade b/modules/web-console/frontend/views/templates/clone.jade similarity index 79% rename from modules/web-console/src/main/js/views/templates/clone.jade rename to modules/web-console/frontend/views/templates/clone.jade index 25e99e79ca073..d68bf458d134c 100644 --- a/modules/web-console/src/main/js/views/templates/clone.jade +++ b/modules/web-console/frontend/views/templates/clone.jade @@ -14,6 +14,8 @@ See the License for the specific language governing permissions and limitations under the License. +include ../../app/helpers/jade/mixins.jade + .modal(tabindex='-1' role='dialog') .modal-dialog .modal-content @@ -25,7 +27,11 @@ .col-sm-2 label.required.labelFormField New name:  .col-sm-10 - input.form-control(id='copy-new-name' type='text' ng-model='newName' ignite-on-enter='form.$valid && ok(newName)' required ignite-auto-focus) + .input-tip + +ignite-form-field-input('"copy-new-name"','newName', false, 'true', 'Enter new name')( + data-ignite-form-field-input-autofocus='true' + ignite-on-enter='form.$valid && ok(newName)' + ) .modal-footer button.btn.btn-default(id='copy-btn-cancel' ng-click='$hide()') Cancel button.btn.btn-primary(id='copy-btn-confirm' ng-disabled='ui.inputForm.$invalid' ng-click='ok(newName)') Confirm diff --git a/modules/web-console/src/main/js/views/templates/confirm.jade b/modules/web-console/frontend/views/templates/confirm.jade similarity index 100% rename from modules/web-console/src/main/js/views/templates/confirm.jade rename to modules/web-console/frontend/views/templates/confirm.jade diff --git a/modules/web-console/src/main/js/views/templates/demo-info.jade b/modules/web-console/frontend/views/templates/demo-info.jade similarity index 100% rename from modules/web-console/src/main/js/views/templates/demo-info.jade rename to modules/web-console/frontend/views/templates/demo-info.jade diff --git a/modules/web-console/src/main/js/views/templates/dropdown.jade b/modules/web-console/frontend/views/templates/dropdown.jade similarity index 63% rename from modules/web-console/src/main/js/views/templates/dropdown.jade rename to modules/web-console/frontend/views/templates/dropdown.jade index 96cc43d4fa49e..2ee8616a61ee8 100644 --- a/modules/web-console/src/main/js/views/templates/dropdown.jade +++ b/modules/web-console/frontend/views/templates/dropdown.jade @@ -15,7 +15,10 @@ limitations under the License. ul.dropdown-menu(tabindex='-1' role='menu' ng-show='content && content.length') - li(role='presentation' ui-sref-active='active' ng-class='{divider: item.divider, active: item.active, custom: item.custom}' ng-repeat='item in content') - a(role='menuitem' tabindex='-1' ui-sref='{{item.sref}}' ng-if='!item.divider && item.sref' ng-bind='item.text') - a(role='menuitem' tabindex='-1' ng-href='{{item.href}}' ng-if='!item.divider && item.href' target='{{item.target || ""}}' ng-bind='item.text') - a(role='menuitem' tabindex='-1' href='javascript:void(0)' ng-if='!item.divider && item.click' ng-click='$eval(item.click);$hide()' ng-bind='item.text') + li(role='presentation' ui-sref-active='active' ng-class='{divider: item.divider, active: item.active}' ng-repeat='item in content') + a(role='menuitem' tabindex='-1' ui-sref='{{item.sref}}' ng-if='!item.action && !item.divider && item.sref' ng-bind='item.text') + a(role='menuitem' tabindex='-1' ng-href='{{item.href}}' ng-if='!item.action && !item.divider && item.href' target='{{item.target || ""}}' ng-bind='item.text') + a(role='menuitem' tabindex='-1' href='javascript:void(0)' ng-if='!item.action && !item.divider && item.click' ng-click='$eval(item.click);$hide()' ng-bind='item.text') + div(role='menuitem' ng-if='item.action') + i.fa.pull-right(class='{{ item.action.icon }}' ng-click='item.action.click(item.data)' bs-tooltip data-title='{{ item.action.tooltip }}' data-trigger='hover' data-placement='bottom') + div: a(ui-sref='{{ item.sref }}' ng-bind='item.text') diff --git a/modules/web-console/src/main/js/views/templates/getting-started.jade b/modules/web-console/frontend/views/templates/getting-started.jade similarity index 100% rename from modules/web-console/src/main/js/views/templates/getting-started.jade rename to modules/web-console/frontend/views/templates/getting-started.jade diff --git a/modules/web-console/src/main/js/views/templates/message.jade b/modules/web-console/frontend/views/templates/message.jade similarity index 100% rename from modules/web-console/src/main/js/views/templates/message.jade rename to modules/web-console/frontend/views/templates/message.jade diff --git a/modules/web-console/src/main/js/views/templates/pagination.jade b/modules/web-console/frontend/views/templates/pagination.jade similarity index 100% rename from modules/web-console/src/main/js/views/templates/pagination.jade rename to modules/web-console/frontend/views/templates/pagination.jade diff --git a/modules/web-console/src/main/js/views/templates/select.jade b/modules/web-console/frontend/views/templates/select.jade similarity index 100% rename from modules/web-console/src/main/js/views/templates/select.jade rename to modules/web-console/frontend/views/templates/select.jade diff --git a/modules/web-console/src/main/js/views/templates/validation-error.jade b/modules/web-console/frontend/views/templates/validation-error.jade similarity index 100% rename from modules/web-console/src/main/js/views/templates/validation-error.jade rename to modules/web-console/frontend/views/templates/validation-error.jade diff --git a/modules/web-console/pom.xml b/modules/web-console/pom.xml index 5c5cac0a0a05a..e66252c48a875 100644 --- a/modules/web-console/pom.xml +++ b/modules/web-console/pom.xml @@ -32,16 +32,17 @@ ignite-web-console 1.7.0-SNAPSHOT + http://ignite.apache.org com.github.eirslett frontend-maven-plugin - 0.0.23 + 1.0 - src/main/js + frontend @@ -51,8 +52,9 @@ install-node-and-npm - v4.4.4 - 2.14.20 + v4.4.7 + 3.8.6 + frontend @@ -61,9 +63,32 @@ npm + + + install --no-optional + + + + + gulp build + + gulp + + + + build + + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + diff --git a/modules/web-console/src/main/js/.babelrc b/modules/web-console/src/main/js/.babelrc deleted file mode 100644 index af0f0c3d35213..0000000000000 --- a/modules/web-console/src/main/js/.babelrc +++ /dev/null @@ -1,3 +0,0 @@ -{ - "presets": ["es2015"] -} \ No newline at end of file diff --git a/modules/web-console/src/main/js/app/controllers/notebooks.controller.js b/modules/web-console/src/main/js/app/controllers/notebooks.controller.js deleted file mode 100644 index 0440c4659b5d7..0000000000000 --- a/modules/web-console/src/main/js/app/controllers/notebooks.controller.js +++ /dev/null @@ -1,69 +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. - */ - -// Controller that load notebooks in navigation bar . -export default ['notebooks', [ - '$rootScope', '$scope', '$modal', '$state', '$http', 'IgniteMessages', - ($root, $scope, $modal, $state, $http, Messages) => { - $root.notebooks = []; - - // Pre-fetch modal dialogs. - const _notebookNewModal = $modal({scope: $scope, templateUrl: '/sql/notebook-new.html', show: false}); - - $root.rebuildDropdown = function() { - $scope.notebookDropdown = [ - {text: 'Create new notebook', click: 'inputNotebookName()'}, - {divider: true} - ]; - - _.forEach($root.notebooks, (notebook) => $scope.notebookDropdown.push({ - text: notebook.name, - sref: 'base.sql.notebook({noteId:"' + notebook._id + '"})' - })); - }; - - $root.reloadNotebooks = function() { - // When landing on the page, get clusters and show them. - $http.post('/api/v1/notebooks/list') - .success((data) => { - $root.notebooks = data; - - $root.rebuildDropdown(); - }) - .error(Messages.showError); - }; - - $root.inputNotebookName = () => { - _notebookNewModal.$promise.then(_notebookNewModal.show); - }; - - $root.createNewNotebook = (name) => { - $http.post('/api/v1/notebooks/new', {name}) - .success((noteId) => { - _notebookNewModal.hide(); - - $root.reloadNotebooks(); - - $state.go('base.sql.notebook', {noteId}); - }) - .error(Messages.showError); - }; - - $root.reloadNotebooks(); - - } -]]; diff --git a/modules/web-console/src/main/js/app/modules/form/field/dropdown.directive.js b/modules/web-console/src/main/js/app/modules/form/field/dropdown.directive.js deleted file mode 100644 index 23c900a954cae..0000000000000 --- a/modules/web-console/src/main/js/app/modules/form/field/dropdown.directive.js +++ /dev/null @@ -1,83 +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. - */ - -import templateUrl from './dropdown.jade'; - -export default ['igniteFormFieldDropdown', ['IgniteFormGUID', 'IgniteLegacyTable', (guid, LegacyTable) => { - const controller = () => {}; - - const link = (scope, $element, attrs, [form, label]) => { - const {id, name} = scope; - - scope.id = id || guid(); - - if (label) { - label.for = scope.id; - - scope.label = label; - - scope.$watch('required', (required) => { - label.required = required || false; - }); - } - - form.$defaults = form.$defaults || {}; - form.$defaults[name] = _.cloneDeep(scope.value); - - const setAsDefault = () => { - if (!form.$pristine) return; - - form.$defaults = form.$defaults || {}; - form.$defaults[name] = _.cloneDeep(scope.value); - }; - - scope.$watch(() => form.$pristine, setAsDefault); - scope.$watch('value', setAsDefault); - - scope.tableReset = () => { - LegacyTable.tableSaveAndReset(); - }; - }; - - return { - restrict: 'E', - scope: { - id: '@', - name: '@', - required: '=ngRequired', - value: '=ngModel', - - focus: '=ngFocus', - - onEnter: '@' - }, - bindToController: { - value: '=ngModel', - placeholder: '@', - options: '=', - ngDisabled: '=', - multiple: '=' - }, - link, - templateUrl, - controller, - controllerAs: 'dropdown', - replace: true, - transclude: true, - require: ['^form', '?^igniteFormField'] - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/form/field/dropdown.jade b/modules/web-console/src/main/js/app/modules/form/field/dropdown.jade deleted file mode 100644 index 5b7cdf6693189..0000000000000 --- a/modules/web-console/src/main/js/app/modules/form/field/dropdown.jade +++ /dev/null @@ -1,61 +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. - -.input-tip - button.select-toggle.form-control( - ng-if='dropdown.multiple' - id='{{ id }}' - name='{{ name }}' - data-placeholder='{{ dropdown.placeholder }}' - - bs-select - bs-options='item.value as item.label for item in dropdown.options' - data-multiple='1' - - ng-model='dropdown.value' - ng-disabled='dropdown.ngDisabled || !dropdown.options || !dropdown.options.length' - - data-ng-required='required || false' - - ignite-on-enter='{{ onEnter }}' - - tabindex='0' - - data-ng-focus='tableReset()' - ) - - button.select-toggle.form-control( - ng-if='!dropdown.multiple' - id='{{ id }}' - name='{{ name }}' - data-placeholder='{{ dropdown.placeholder }}' - - bs-select - bs-options='item.value as item.label for item in dropdown.options' - - ng-model='dropdown.value' - ng-disabled='dropdown.ngDisabled || !dropdown.options || !dropdown.options.length' - - data-ng-required='required || false' - - ignite-on-enter='{{ onEnter }}' - - tabindex='0' - - data-ng-focus='tableReset()' - ) - - span(ng-transclude='') diff --git a/modules/web-console/src/main/js/app/modules/form/field/field.jade b/modules/web-console/src/main/js/app/modules/form/field/field.jade deleted file mode 100644 index 08250ac4eca0d..0000000000000 --- a/modules/web-console/src/main/js/app/modules/form/field/field.jade +++ /dev/null @@ -1,27 +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. - -div - div(ng-if='field.type == "external"') - label.col-xs-4.col-sm-4.col-md-4( - id='{{::field.for}}Label' - for='{{::field.for}}' - class='{{ field.required ? "required" : "" }}' - ) - span(ng-if='field.label') {{::field.label}}: - .col-xs-8.col-sm-8.col-md-8(ng-transclude='') - div(ng-if='field.type == "internal"') - label.col-xs-12.col-sm-12.col-md-12(ng-transclude id='{{::field.for}}Label') diff --git a/modules/web-console/src/main/js/app/modules/form/field/input/checkbox.directive.js b/modules/web-console/src/main/js/app/modules/form/field/input/checkbox.directive.js deleted file mode 100644 index 3d84b4eb97300..0000000000000 --- a/modules/web-console/src/main/js/app/modules/form/field/input/checkbox.directive.js +++ /dev/null @@ -1,66 +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. - */ - -import templateUrl from './checkbox.jade'; - -export default ['igniteFormFieldInputCheckbox', ['IgniteFormGUID', 'IgniteLegacyTable', (guid, LegacyTable) => { - const link = (scope, el, attrs, [form, label]) => { - const {id, name} = scope; - const field = form[name]; - - scope.field = field; - label.for = scope.id = id || guid(); - - label.type = 'internal'; - - form.$defaults = form.$defaults || {}; - form.$defaults[name] = _.cloneDeep(scope.value); - - const setAsDefault = () => { - if (!form.$pristine) return; - - form.$defaults = form.$defaults || {}; - form.$defaults[name] = _.cloneDeep(scope.value); - }; - - scope.$watch(() => form.$pristine, setAsDefault); - scope.$watch('value', setAsDefault); - - scope.tableReset = () => { - LegacyTable.tableSaveAndReset(); - }; - }; - - return { - restrict: 'E', - scope: { - id: '@', - name: '@', - required: '=ngRequired', - disabled: '=ngDisabled', - - focus: '=ngFocus', - - value: '=ngModel' - }, - link, - templateUrl, - replace: true, - transclude: true, - require: ['^form', '?^igniteFormField'] - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/form/field/input/datalist.directive.js b/modules/web-console/src/main/js/app/modules/form/field/input/datalist.directive.js deleted file mode 100644 index 6c43a2a98b255..0000000000000 --- a/modules/web-console/src/main/js/app/modules/form/field/input/datalist.directive.js +++ /dev/null @@ -1,122 +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. - */ - -import templateUrl from './datalist.jade'; - -export default ['igniteFormFieldInputDatalist', ['IgniteFormGUID', 'IgniteLegacyTable', (guid, LegacyTable) => { - const link = (scope, element, attrs, [ngModel, form, label], transclude) => { - const {id, ngModelName} = scope; - - const name = ngModelName; - - scope.id = id || guid(); - scope.form = form; - scope.name = ngModelName + 'TextInput'; - scope.ngModel = ngModel; - - Object.defineProperty(scope, 'field', { - get: () => scope.form[scope.name] - }); - - if (label) { - label.for = scope.id; - - scope.label = label; - - scope.$watch('required', (required) => { - label.required = required || false; - }); - } - - form.$defaults = form.$defaults || {}; - - if (form.$pristine) { - if (!(_.isNull(form.$defaults[name]) || _.isUndefined(form.$defaults[name]))) { - scope.value = form.$defaults[name]; - ngModel.$setViewValue(scope.value); - } else - form.$defaults[name] = _.cloneDeep(scope.value); - } - - const setAsDefault = () => { - if (!form.$pristine) return; - - form.$defaults = form.$defaults || {}; - form.$defaults[name] = _.cloneDeep(scope.value); - }; - - scope.$watch(() => form.$pristine, setAsDefault); - scope.$watch('value', setAsDefault); - - const checkValid = () => { - const input = element.find('input'); - - const invalid = ngModel.$invalid || (input[0].required && !input[0].value); - - input.removeClass(invalid ? 'ng-valid' : 'ng-invalid'); - input.addClass(invalid ? 'ng-invalid' : 'ng-valid'); - }; - - scope.ngChange = () => { - ngModel.$setViewValue(scope.value); - - if (_.isEqual(scope.value, form.$defaults[name])) - ngModel.$setPristine(); - else - ngModel.$setDirty(); - - setTimeout(checkValid, 100); // Use setTimeout() workaround of problem of two controllers. - }; - - ngModel.$render = () => { - scope.value = ngModel.$modelValue; - }; - - scope.tableReset = () => { - LegacyTable.tableSaveAndReset(); - }; - - transclude(scope.$parent, function(clone, tscope) { - tscope.form = form; - tscope.ngModelName = ngModelName; - - element.find('.transclude-here').append(clone); - }); - }; - - return { - restrict: 'E', - scope: { - id: '@', - ngModelName: '@name', - placeholder: '@', - required: '=ngRequired', - disabled: '=ngDisabled', - ngBlur: '&', - - options: '=', - - focus: '=ngFocus', - autofocus: '=igniteFormFieldInputAutofocus' - }, - link, - templateUrl, - replace: true, - transclude: true, - require: ['ngModel', '^form', '?^igniteFormField'] - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/form/field/input/datalist.jade b/modules/web-console/src/main/js/app/modules/form/field/input/datalist.jade deleted file mode 100644 index 7ae1411857cc6..0000000000000 --- a/modules/web-console/src/main/js/app/modules/form/field/input/datalist.jade +++ /dev/null @@ -1,51 +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. - -mixin feedback(isCheckPristine, error, errorMessage) - -var checkPristine = isCheckPristine ? '!field.$pristine && ' : '' - - i.fa.fa-exclamation-triangle.form-control-feedback( - ng-if='#{checkPristine}field.$error.#{error}' - bs-tooltip='"#{errorMessage}"' - ignite-error=error - ignite-error-message=errorMessage - ) - -.input-tip - input.form-control( - id='{{ id }}' - name='{{ name }}' - placeholder='{{ placeholder }}' - - data-ng-model='value' - - data-ng-blur='ngBlur()' - data-ng-change='ngChange()' - data-ng-required='required || false' - data-ng-disabled='disabled || false' - - data-ignite-form-field-input-autofocus='{{autofocus}}' - - bs-typeahead - bs-options='item for item in options' - container='body' - data-min-length='1' - ignite-retain-selection - ) - - +feedback(true, 'required', '{{ label.name }} could not be empty!') - - span.transclude-here diff --git a/modules/web-console/src/main/js/app/modules/form/field/input/number.directive.js b/modules/web-console/src/main/js/app/modules/form/field/input/number.directive.js deleted file mode 100644 index 0d21f5b1b4d3b..0000000000000 --- a/modules/web-console/src/main/js/app/modules/form/field/input/number.directive.js +++ /dev/null @@ -1,76 +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. - */ - -import templateUrl from './number.jade'; - -export default ['igniteFormFieldInputNumber', ['IgniteFormGUID', 'IgniteLegacyTable', (guid, LegacyTable) => { - const link = (scope, el, attrs, [form, label]) => { - const {id, name} = scope; - const field = form[name]; - - scope.id = id || guid(); - scope.field = field; - - if (label) { - label.for = scope.id; - - scope.$watch('required', (required) => { - label.required = required || false; - }); - } - - form.$defaults = form.$defaults || {}; - form.$defaults[name] = _.cloneDeep(scope.value); - - const setAsDefault = () => { - if (!form.$pristine) return; - - form.$defaults = form.$defaults || {}; - form.$defaults[name] = _.cloneDeep(scope.value); - }; - - scope.$watch(() => form.$pristine, setAsDefault); - scope.$watch('value', setAsDefault); - - scope.tableReset = () => { - LegacyTable.tableSaveAndReset(); - }; - }; - - return { - restrict: 'E', - scope: { - id: '@', - name: '@', - placeholder: '@', - required: '=ngRequired', - disabled: '=ngDisabled', - - focus: '=ngFocus', - - min: '@', - max: '@', - step: '@', - value: '=ngModel' - }, - link, - templateUrl, - replace: true, - transclude: true, - require: ['^form', '?^igniteFormField'] - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/form/field/input/number.jade b/modules/web-console/src/main/js/app/modules/form/field/input/number.jade deleted file mode 100644 index e72a1d05cbae2..0000000000000 --- a/modules/web-console/src/main/js/app/modules/form/field/input/number.jade +++ /dev/null @@ -1,50 +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. - -mixin feedback(isCheckPristine, error, errorMessage) - -var checkPristine = isCheckPristine ? '!field.$pristine && ' : '' - - i.fa.fa-exclamation-triangle.form-control-feedback( - ng-if='#{checkPristine}field.$error.#{error}' - bs-tooltip='"#{errorMessage}"' - ignite-error=error - ignite-error-message=errorMessage - ) - -.input-tip - input.form-control( - id='{{ id }}' - name='{{ name }}' - placeholder='{{ placeholder }}' - type='number' - min='{{ min || 0 }}' - max='{{ max || Number.MAX_VALUE }}' - step='{{ step || 1 }}' - - data-ng-model='value' - - data-ng-required='required || false' - data-ng-disabled='disabled || false' - - data-ng-focus='tableReset()' - ) - - +feedback(true, 'required', 'This field could not be empty') - +feedback(false, 'min', 'Value is less than allowable minimum: {{ min || 0 }}') - +feedback(false, 'max', 'Value is more than allowable maximum: {{ max }}') - +feedback(false, 'number', 'Only numbers allowed') - - span(ng-transclude='') diff --git a/modules/web-console/src/main/js/app/modules/form/field/input/text.directive.js b/modules/web-console/src/main/js/app/modules/form/field/input/text.directive.js deleted file mode 100644 index 56c02b5ff6624..0000000000000 --- a/modules/web-console/src/main/js/app/modules/form/field/input/text.directive.js +++ /dev/null @@ -1,126 +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. - */ - -import templateUrl from './text.jade'; -import './text.css'; - -export default ['igniteFormFieldInputText', ['IgniteFormGUID', 'IgniteLegacyTable', (guid, LegacyTable) => { - const link = (scope, element, attrs, [ngModel, form, label], transclude) => { - const {id, ngModelName} = scope; - - const name = ngModelName; - - scope.id = id || guid(); - scope.form = form; - scope.name = ngModelName + 'TextInput'; - scope.ngModel = ngModel; - - Object.defineProperty(scope, 'field', { - get: () => scope.form[scope.name] - }); - - if (label) { - label.for = scope.id; - - scope.label = label; - scope.labelName = label.name; - - scope.$watch('required', (required) => { - label.required = required || false; - }); - } - else - scope.labelName = attrs.igniteLabelName || 'Value'; - - form.$defaults = form.$defaults || {}; - - if (form.$pristine) { - if (!(_.isNull(form.$defaults[name]) || _.isUndefined(form.$defaults[name]))) { - scope.value = form.$defaults[name]; - ngModel.$setViewValue(scope.value); - } else - form.$defaults[name] = _.cloneDeep(scope.value); - } - - const setAsDefault = () => { - if (!form.$pristine) return; - - form.$defaults = form.$defaults || {}; - form.$defaults[name] = _.cloneDeep(scope.value); - }; - - scope.$watch(() => form.$pristine, setAsDefault); - scope.$watch('value', setAsDefault); - - const checkValid = () => { - const input = element.find('input'); - - const invalid = ngModel.$invalid || (input[0].required && !input[0].value); - - input.removeClass(invalid ? 'ng-valid' : 'ng-invalid'); - input.addClass(invalid ? 'ng-invalid' : 'ng-valid'); - }; - - scope.ngChange = () => { - ngModel.$setViewValue(scope.value); - - if (_.isEqual(scope.value, form.$defaults[name])) - ngModel.$setPristine(); - else - ngModel.$setDirty(); - - setTimeout(checkValid, 100); // Use setTimeout() workaround of problem of two controllers. - }; - - ngModel.$render = () => { - scope.value = ngModel.$modelValue; - }; - - scope.tableReset = () => { - LegacyTable.tableSaveAndReset(); - }; - - transclude(scope.$parent, function(clone, tscope) { - tscope.form = form; - tscope.ngModelName = ngModelName; - - element.find('.transclude-here').append(clone); - }); - }; - - return { - restrict: 'E', - scope: { - id: '@', - ngModelName: '@name', - placeholder: '@', - required: '=ngRequired', - disabled: '=ngDisabled', - - focus: '=ngFocus', - - ngBlur: '&', - - autofocus: '=igniteFormFieldInputAutofocus' - }, - link, - templateUrl, - replace: true, - transclude: true, - require: ['ngModel', '^form', '?^igniteFormField'] - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/form/field/input/text.jade b/modules/web-console/src/main/js/app/modules/form/field/input/text.jade deleted file mode 100644 index 8a1dfc209bf26..0000000000000 --- a/modules/web-console/src/main/js/app/modules/form/field/input/text.jade +++ /dev/null @@ -1,48 +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. - -mixin feedback(isCheckPristine, error, errorMessage) - -var checkPristine = isCheckPristine ? '!field.$pristine && ' : '' - - i.fa.fa-exclamation-triangle.form-control-feedback( - ng-if='#{checkPristine}field.$error.#{error}' - bs-tooltip='"#{errorMessage}"' - ignite-error=error - ignite-error-message=errorMessage - ) - -.input-tip - input.form-control( - id='{{ id }}' - name='{{ name }}' - placeholder='{{ placeholder }}' - type='text' - - data-ng-model='value' - - data-ng-blur='ngBlur()' - data-ng-change='ngChange()' - data-ng-required='required || false' - data-ng-disabled='disabled || false' - - data-ignite-form-field-input-autofocus='{{autofocus}}' - - data-ng-focus='tableReset()' - ) - - +feedback(true, 'required', '{{ labelName }} could not be empty!') - - span.transclude-here diff --git a/modules/web-console/src/main/js/app/modules/form/group/group.directive.js b/modules/web-console/src/main/js/app/modules/form/group/group.directive.js deleted file mode 100644 index 20cad22b09ca5..0000000000000 --- a/modules/web-console/src/main/js/app/modules/form/group/group.directive.js +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import templateUrl from './group.jade'; - -export default ['igniteFormGroup', [() => { - const controller = [function() { }]; - - const link = (scope, el, attrs, [ngModelCtrl, ownFormCtrl, parentFormCtrl]) => { - if (!ownFormCtrl) - return; - - const name = attrs.ngForm; - ngModelCtrl.$name = name; - - parentFormCtrl.$addControl(ngModelCtrl); - parentFormCtrl.$removeControl(ownFormCtrl); - - scope.ngModel = scope.ngModel || []; - parentFormCtrl.$defaults = parentFormCtrl.$defaults || {}; - - if (parentFormCtrl.$pristine) { - if (!(_.isNull(parentFormCtrl.$defaults[name]) || _.isUndefined(parentFormCtrl.$defaults[name]))) - scope.ngModel = parentFormCtrl.$defaults[name]; - else - parentFormCtrl.$defaults[name] = _.cloneDeep(scope.ngModel); - } - - const setAsDefault = () => { - if (!parentFormCtrl.$pristine) - return; - - scope.ngModel = scope.ngModel || []; - parentFormCtrl.$defaults = parentFormCtrl.$defaults || {}; - parentFormCtrl.$defaults[name] = _.cloneDeep(scope.ngModel); - }; - - const setAsDirty = () => { - if (_.isEqual(scope.ngModel, parentFormCtrl.$defaults[name])) - ngModelCtrl.$setPristine(); - else - ngModelCtrl.$setDirty(); - }; - - scope.$watch(() => parentFormCtrl.$pristine, setAsDefault); - - scope.$watch('ngModel', setAsDefault); - scope.$watch('ngModel', setAsDirty, true); - }; - - return { - restrict: 'E', - scope: { - ngModel: '=ngModel' - }, - bindToController: { - label: '@' - }, - link, - templateUrl, - controller, - controllerAs: 'group', - replace: true, - transclude: true, - require: ['?ngModel', '?form', '^^form'] - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/query-notebooks/query-notebooks.module.js b/modules/web-console/src/main/js/app/modules/query-notebooks/query-notebooks.module.js deleted file mode 100644 index 59999484c1e96..0000000000000 --- a/modules/web-console/src/main/js/app/modules/query-notebooks/query-notebooks.module.js +++ /dev/null @@ -1,115 +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. - */ - -import angular from 'angular'; - -angular - .module('ignite-console.query-notebooks', [ - - ]) - .provider('QueryNotebooks', function() { - const _demoNotebook = { - name: 'SQL demo', - paragraphs: [ - { - name: 'Query with refresh rate', - cacheName: 'CarCache', - pageSize: 50, - query: 'SELECT count(*)\nFROM "CarCache".Car', - result: 'bar', - timeLineSpan: '1', - rate: { - value: 3, - unit: 1000, - installed: true - } - }, - { - name: 'Simple query', - cacheName: 'CarCache', - pageSize: 50, - query: 'SELECT * FROM "CarCache".Car', - result: 'table', - timeLineSpan: '1', - rate: { - value: 30, - unit: 1000, - installed: false - } - }, - { - name: 'Query with aggregates', - cacheName: 'CarCache', - pageSize: 50, - query: 'SELECT p.name, count(*) AS cnt\nFROM "ParkingCache".Parking p\nINNER JOIN "CarCache".Car c\n ON (p.id) = (c.parkingId)\nGROUP BY P.NAME', - result: 'table', - timeLineSpan: '1', - rate: { - value: 30, - unit: 1000, - installed: false - } - } - ], - expandedParagraphs: [0, 1, 2] - }; - - this.$get = ['$q', '$http', '$rootScope', ($q, $http, $root) => { - return { - read(noteId) { - if ($root.IgniteDemoMode) - return $q.when(angular.copy(_demoNotebook)); - - return $http.post('/api/v1/notebooks/get', {noteId}) - .then(({data}) => data) - .catch(({data}) => $q.reject(data)); - }, - save(notebook) { - if ($root.IgniteDemoMode) - return $q.when(); - - return $http.post('/api/v1/notebooks/save', notebook) - .then(({data}) => data) - .catch(({data}) => $q.reject(data)); - }, - remove(notebook) { - if ($root.IgniteDemoMode) - return $q.reject(`Removing "${notebook.name}" notebook is not supported.`); - - return $http.post('/api/v1/notebooks/remove', {_id: notebook._id}) - .then(() => { - const idx = _.findIndex($root.notebooks, (item) => { - return item._id === notebook._id; - }); - - if (idx >= 0) { - $root.notebooks.splice(idx, 1); - - $root.rebuildDropdown(); - - if (idx < $root.notebooks.length) - return $root.notebooks[idx]; - } - - if ($root.notebooks.length > 0) - return $root.notebooks[$root.notebooks.length - 1]; - }) - .catch(({data}) => $q.reject(data)); - } - }; - }]; - }); diff --git a/modules/web-console/src/main/js/app/modules/states/configuration.state.js b/modules/web-console/src/main/js/app/modules/states/configuration.state.js deleted file mode 100644 index 7e5e95e5aaaa2..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration.state.js +++ /dev/null @@ -1,226 +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. - */ - -import angular from 'angular'; - -// Common directives. -import previewPanel from './configuration/preview-panel.directive.js'; - -// Clusters screen. -import clustersGeneral from './configuration/clusters/general.directive'; - -import clustersGeneralDiscoveryCloud from './configuration/clusters/general/discovery/cloud.directive'; -import clustersGeneralDiscoveryGoogle from './configuration/clusters/general/discovery/google.directive'; -import clustersGeneralDiscoveryJdbc from './configuration/clusters/general/discovery/jdbc.directive'; -import clustersGeneralDiscoveryMulticast from './configuration/clusters/general/discovery/multicast.directive'; -import clustersGeneralDiscoveryS3 from './configuration/clusters/general/discovery/s3.directive'; -import clustersGeneralDiscoveryShared from './configuration/clusters/general/discovery/shared.directive'; -import clustersGeneralDiscoveryVm from './configuration/clusters/general/discovery/vm.directive'; - -import clustersGeneralDiscoveryZookeeper from './configuration/clusters/general/discovery/zookeeper.directive'; -import clustersGeneralDiscoveryZookeeperRetryExponential from './configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.directive'; -import clustersGeneralDiscoveryZookeeperRetryBoundedExponential from './configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.directive'; -import clustersGeneralDiscoveryZookeeperRetryUntilElapsed from './configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.directive'; -import clustersGeneralDiscoveryZookeeperRetryNTimes from './configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.directive'; -import clustersGeneralDiscoveryZookeeperRetryOneTime from './configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.directive'; -import clustersGeneralDiscoveryZookeeperRetryForever from './configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.directive'; -import clustersGeneralDiscoveryZookeeperRetryCustom from './configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.directive'; - -import clustersAtomic from './configuration/clusters/atomic.directive'; -import clustersBinary from './configuration/clusters/binary.directive'; -import clustersCommunication from './configuration/clusters/communication.directive'; -import clustersConnector from './configuration/clusters/connector.directive'; -import clustersDeployment from './configuration/clusters/deployment.directive'; -import clustersDiscovery from './configuration/clusters/discovery.directive'; -import clustersEvents from './configuration/clusters/events.directive'; -import clustersIgfs from './configuration/clusters/igfs.directive'; -import clustersMarshaller from './configuration/clusters/marshaller.directive'; -import clustersMetrics from './configuration/clusters/metrics.directive'; -import clustersSsl from './configuration/clusters/ssl.directive'; -import clustersSwap from './configuration/clusters/swap.directive'; -import clustersTime from './configuration/clusters/time.directive'; -import clustersThread from './configuration/clusters/thread.directive'; -import clustersTransactions from './configuration/clusters/transactions.directive'; -import clustersUserAttributes from './configuration/clusters/attributes.directive'; -import clustersCollision from './configuration/clusters/collision.directive'; -import clustersFailover from './configuration/clusters/failover.directive'; -import clustersLogger from './configuration/clusters/logger.directive'; - -import clustersCollisionJobStealing from './configuration/clusters/collision/job-stealing.directive'; -import clustersCollisionFifoQueue from './configuration/clusters/collision/fifo-queue.directive'; -import clustersCollisionPriorityQueue from './configuration/clusters/collision/priority-queue.directive'; -import clustersCollisionCustom from './configuration/clusters/collision/custom.directive'; - -import clustersLoggerLog4j2 from './configuration/clusters/logger/log4j2.directive'; -import clustersLoggerLog4j from './configuration/clusters/logger/log4j.directive'; -import clustersLoggerCustom from './configuration/clusters/logger/custom.directive'; - -// Domains screen. -import domainsGeneral from './configuration/domains/general.directive'; -import domainsQuery from './configuration/domains/query.directive'; -import domainsStore from './configuration/domains/store.directive'; - -// Caches screen. -import cachesGeneral from './configuration/caches/general.directive'; -import cachesMemory from './configuration/caches/memory.directive'; -import cachesQuery from './configuration/caches/query.directive'; -import cachesStore from './configuration/caches/store.directive'; -import cachesConcurrency from './configuration/caches/concurrency.directive'; -import cachesRebalance from './configuration/caches/rebalance.directive'; -import cachesServerNearCache from './configuration/caches/server-near-cache.directive'; -import cachesStatistics from './configuration/caches/statistics.directive'; - -// IGFS screen. -import igfsGeneral from './configuration/igfs/general.directive'; -import igfsIpc from './configuration/igfs/ipc.directive'; -import igfsFragmentizer from './configuration/igfs/fragmentizer.directive'; -import igfsDual from './configuration/igfs/dual.directive'; -import igfsSecondary from './configuration/igfs/secondary.directive'; -import igfsMisc from './configuration/igfs/misc.directive'; - -// Summary screen. -import ConfigurationSummaryCtrl from './configuration/summary/summary.controller'; -import ConfigurationSummaryResource from './configuration/summary/summary.resource'; -import summaryTabs from './configuration/summary/summary-tabs.directive'; - -angular.module('ignite-console.states.configuration', ['ui.router']) - // Clusters screen. - .directive(...previewPanel) - .directive(...clustersLoggerCustom) - .directive(...clustersLoggerLog4j) - .directive(...clustersLoggerLog4j2) - .directive(...clustersLogger) - .directive(...clustersFailover) - .directive(...clustersCollisionCustom) - .directive(...clustersCollisionPriorityQueue) - .directive(...clustersCollisionFifoQueue) - .directive(...clustersCollisionJobStealing) - .directive(...clustersCollision) - .directive(...clustersUserAttributes) - .directive(...clustersTransactions) - .directive(...clustersThread) - .directive(...clustersTime) - .directive(...clustersSwap) - .directive(...clustersSsl) - .directive(...clustersMetrics) - .directive(...clustersMarshaller) - .directive(...clustersIgfs) - .directive(...clustersEvents) - .directive(...clustersDiscovery) - .directive(...clustersDeployment) - .directive(...clustersConnector) - .directive(...clustersCommunication) - .directive(...clustersBinary) - .directive(...clustersAtomic) - .directive(...clustersGeneral) - .directive(...clustersGeneralDiscoveryCloud) - .directive(...clustersGeneralDiscoveryGoogle) - .directive(...clustersGeneralDiscoveryJdbc) - .directive(...clustersGeneralDiscoveryMulticast) - .directive(...clustersGeneralDiscoveryS3) - .directive(...clustersGeneralDiscoveryShared) - .directive(...clustersGeneralDiscoveryVm) - .directive(...clustersGeneralDiscoveryZookeeper) - .directive(...clustersGeneralDiscoveryZookeeperRetryExponential) - .directive(...clustersGeneralDiscoveryZookeeperRetryBoundedExponential) - .directive(...clustersGeneralDiscoveryZookeeperRetryUntilElapsed) - .directive(...clustersGeneralDiscoveryZookeeperRetryNTimes) - .directive(...clustersGeneralDiscoveryZookeeperRetryOneTime) - .directive(...clustersGeneralDiscoveryZookeeperRetryForever) - .directive(...clustersGeneralDiscoveryZookeeperRetryCustom) - // Domains screen - .directive(...domainsGeneral) - .directive(...domainsQuery) - .directive(...domainsStore) - // Caches screen - .directive(...cachesGeneral) - .directive(...cachesMemory) - .directive(...cachesQuery) - .directive(...cachesStore) - .directive(...cachesConcurrency) - .directive(...cachesRebalance) - .directive(...cachesServerNearCache) - .directive(...cachesStatistics) - // IGFS screen - .directive(...igfsGeneral) - .directive(...igfsIpc) - .directive(...igfsFragmentizer) - .directive(...igfsDual) - .directive(...igfsSecondary) - .directive(...igfsMisc) - // Summary screen - .directive(...summaryTabs) - // Services. - .service(...ConfigurationSummaryResource) - // Configure state provider. - .config(['$stateProvider', ($stateProvider) => { - // Setup the states. - $stateProvider - .state('base.configuration', { - url: '/configuration', - templateUrl: '/configuration/sidebar.html' - }) - .state('base.configuration.clusters', { - url: '/clusters', - templateUrl: '/configuration/clusters.html', - params: { - linkId: null - }, - metaTags: { - title: 'Configure Clusters' - } - }) - .state('base.configuration.caches', { - url: '/caches', - templateUrl: '/configuration/caches.html', - params: { - linkId: null - }, - metaTags: { - title: 'Configure Caches' - } - }) - .state('base.configuration.domains', { - url: '/domains', - templateUrl: '/configuration/domains.html', - params: { - linkId: null - }, - metaTags: { - title: 'Configure Domain Model' - } - }) - .state('base.configuration.igfs', { - url: '/igfs', - templateUrl: '/configuration/igfs.html', - params: { - linkId: null - }, - metaTags: { - title: 'Configure IGFS' - } - }) - .state('base.configuration.summary', { - url: '/summary', - templateUrl: '/configuration/summary.html', - controller: ConfigurationSummaryCtrl, - controllerAs: 'ctrl', - metaTags: { - title: 'Configurations Summary' - } - }); - }]); diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/query.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/caches/query.directive.js deleted file mode 100644 index 9d8264e62c03e..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/query.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './query.jade'; - -export default ['igniteConfigurationCachesQuery', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/rebalance.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/caches/rebalance.directive.js deleted file mode 100644 index 508157861bb28..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/rebalance.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './rebalance.jade'; - -export default ['igniteConfigurationCachesRebalance', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/server-near-cache.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/caches/server-near-cache.directive.js deleted file mode 100644 index c2e83aa021dfa..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/server-near-cache.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './server-near-cache.jade'; - -export default ['igniteConfigurationCachesServerNearCache', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/statistics.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/caches/statistics.directive.js deleted file mode 100644 index ee0da3de4c599..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/statistics.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './statistics.jade'; - -export default ['igniteConfigurationCachesStatistics', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/caches/store.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/caches/store.directive.js deleted file mode 100644 index 80528e5e35b3d..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/caches/store.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './store.jade'; - -export default ['igniteConfigurationCachesStore', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/atomic.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/atomic.directive.js deleted file mode 100644 index 64b80a0da4d45..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/atomic.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './atomic.jade'; - -export default ['igniteConfigurationClustersAtomic', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/attributes.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/attributes.directive.js deleted file mode 100644 index f5f431c700326..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/attributes.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './attributes.jade'; - -export default ['igniteConfigurationUserAttributes', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/binary.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/binary.directive.js deleted file mode 100644 index 7d701bc143ae5..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/binary.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './binary.jade'; - -export default ['igniteConfigurationClustersBinary', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/custom.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/custom.directive.js deleted file mode 100644 index 22991338edca3..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/custom.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './custom.jade'; - -export default ['igniteConfigurationClustersCollisionCustom', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/fifo-queue.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/fifo-queue.directive.js deleted file mode 100644 index f14c1bba189a8..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/fifo-queue.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './fifo-queue.jade'; - -export default ['igniteConfigurationClustersCollisionFifoQueue', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/job-stealing.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/job-stealing.directive.js deleted file mode 100644 index 0cc9cfae3f560..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/job-stealing.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './job-stealing.jade'; - -export default ['igniteConfigurationClustersCollisionJobStealing', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/priority-queue.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/priority-queue.directive.js deleted file mode 100644 index 95ebd5ed5a09f..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/collision/priority-queue.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './priority-queue.jade'; - -export default ['igniteConfigurationClustersCollisionPriorityQueue', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/communication.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/communication.directive.js deleted file mode 100644 index e578dc67042fa..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/communication.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './communication.jade'; - -export default ['igniteConfigurationClustersCommunication', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/deployment.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/deployment.directive.js deleted file mode 100644 index 392ed7e47c8a6..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/deployment.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './deployment.jade'; - -export default ['igniteConfigurationClustersDeployment', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/events.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/events.directive.js deleted file mode 100644 index ee25ae2c79ffe..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/events.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './events.jade'; - -export default ['igniteConfigurationClustersEvents', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/failover.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/failover.directive.js deleted file mode 100644 index b9f59ba3e0806..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/failover.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './failover.jade'; - -export default ['igniteConfigurationClustersFailover', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general.directive.js deleted file mode 100644 index ccfadbe7f6abe..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './general.jade'; - -export default ['igniteConfigurationClustersGeneral', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general.jade b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general.jade deleted file mode 100644 index 2ed0db032ffca..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general.jade +++ /dev/null @@ -1,68 +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. - -include ../../../../../app/helpers/jade/mixins.jade - --var form = 'general' --var model = 'backupItem' - -form.panel.panel-default(name=form novalidate) - .panel-heading(bs-collapse-toggle) - ignite-form-panel-chevron - label General - ignite-form-revert - .panel-collapse(role='tabpanel' bs-collapse-target id=form) - .panel-body - .col-sm-6 - .settings-row - +text('Name:', model + '.name', 'clusterName', 'true', 'Input name', 'Grid name') - .settings-row - +caches(model, 'Select caches to start in cluster or add a new cache') - .settings-row - +text-ip-address('Local host:', model + '.localHost', 'localHost', 'true', '0.0.0.0', 'System-wide local address or host for all Ignite components to bind to') - .settings-row - +dropdown('Discovery:', model + '.discovery.kind', 'discovery', 'true', 'Choose discovery', 'discoveries', - 'Discovery allows to discover remote nodes in grid\ -
            \ -
          • Static IPs - IP Finder which works only with pre configured list of IP addresses specified
          • \ -
          • Multicast - Multicast based IP finder
          • \ -
          • AWS S3 - AWS S3 based IP finder
          • \ -
          • Apache jclouds - Apache jclouds multi cloud toolkit based IP finder
          • \ -
          • Google cloud storage - Google Cloud Storage based IP finder
          • \ -
          • JDBC - JDBC based IP finder
          • \ -
          • Shared filesystem - Shared filesystem based IP finder
          • \ -
          • Apache ZooKeeper - Apache ZooKeeper based IP finder
          • \ -
          ') - .settings-row - .panel-details - ignite-configuration-clusters-general-discovery-cloud( - ng-show='#{model}.discovery.kind === "Cloud"') - ignite-configuration-clusters-general-discovery-google( - ng-show='#{model}.discovery.kind === "GoogleStorage"') - ignite-configuration-clusters-general-discovery-jdbc( - ng-show='#{model}.discovery.kind === "Jdbc"') - ignite-configuration-clusters-general-discovery-multicast( - ng-show='#{model}.discovery.kind === "Multicast"') - ignite-configuration-clusters-general-discovery-s3( - ng-show='#{model}.discovery.kind === "S3"') - ignite-configuration-clusters-general-discovery-shared( - ng-show='#{model}.discovery.kind === "SharedFs"') - ignite-configuration-clusters-general-discovery-vm( - ng-show='#{model}.discovery.kind === "Vm"') - ignite-configuration-clusters-general-discovery-zookeeper( - ng-show='#{model}.discovery.kind === "ZooKeeper"') - .col-sm-6 - +preview-xml-java(model, 'clusterCaches', 'caches') diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/cloud.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/cloud.directive.js deleted file mode 100644 index 8fa7edc73025a..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/cloud.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './cloud.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryCloud', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/google.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/google.directive.js deleted file mode 100644 index a409946136179..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/google.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './google.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryGoogle', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/jdbc.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/jdbc.directive.js deleted file mode 100644 index 6edd6ed356d93..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/jdbc.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './jdbc.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryJdbc', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/multicast.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/multicast.directive.js deleted file mode 100644 index 67775d2f89bb2..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/multicast.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './multicast.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryMulticast', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/s3.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/s3.directive.js deleted file mode 100644 index 25fd90a7149e0..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/s3.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './s3.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryS3', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/shared.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/shared.directive.js deleted file mode 100644 index da1ee0cdc8fcf..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/shared.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './shared.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryShared', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/vm.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/vm.directive.js deleted file mode 100644 index d07adc8812891..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/vm.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './vm.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryVm', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/vm.jade b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/vm.jade deleted file mode 100644 index 1f206136489e0..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/vm.jade +++ /dev/null @@ -1,90 +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. - -include ../../../../../../../app/helpers/jade/mixins.jade - --var model = 'backupItem.discovery.Vm' --var addresses = model + '.addresses' --var form = 'discoveryVmAddresses' - -.details-row - ignite-form-group(ng-model=addresses ng-form=form) - -var uniqueTip = 'Such IP address already exists!' - -var ipAddressTip = 'Invalid IP address!' - - ignite-form-field-label - | Addresses - ignite-form-group-tooltip - | Addresses may be represented as follows: - ul: li IP address (e.g. 127.0.0.1, 9.9.9.9, etc) - li IP address and port (e.g. 127.0.0.1:47500, 9.9.9.9:47501, etc) - li IP address and port range (e.g. 127.0.0.1:47500..47510, 9.9.9.9:47501..47504, etc) - li Hostname (e.g. host1.com, host2, etc) - li Hostname and port (e.g. host1.com:47500, host2:47502, etc) - li Hostname and port range (e.g. host1.com:47500..47510, host2:47502..47508, etc) - | If port is 0 or not provided then default port will be used (depends on discovery SPI configuration)#[br] - | If port range is provided (e.g. host:port1..port2) the following should be considered: - ul: li port1 < port2 should be true - li Both port1 and port2 should be greater than 0 - ignite-form-group-add(ng-click='group.add = [{}]') - | Add new address - - .group-content(ng-if='#{addresses}.length') - -var field = 'edit' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.igniteUnique' - -var ipaddress = 'form[ngModelName].$error.ipaddress' - -var ipaddressPort = 'form[ngModelName].$error.ipaddressPort' - -var ipaddressPortRange = 'form[ngModelName].$error.ipaddressPortRange' - -var save = addresses + '[$index] = ' + field - - ignite-form-field(ng-repeat='model in #{addresses} track by $index' type='internal' name='Address') - .indexField - | {{ $index+1 }}) - +table-remove-button(addresses, 'Remove address') - - ignite-form-field-down(ng-if='!$last' ng-hide='field.edit' data-ng-model='model' data-models=addresses) - ignite-form-field-up(ng-if='!$first' ng-hide='field.edit' data-ng-model='model' data-models=addresses) - - span(ng-hide='field.edit') - a.labelFormField(ng-click='field.edit = true') {{ model }} - span(ng-if='field.edit' ng-init='#{field} = model') - +table-address-field(field, addresses, valid, save, false, true) - +table-save-button(valid, save, false) - +unique-feedback(unique, uniqueTip) - +ipaddress-feedback(ipaddress) - +ipaddress-port-feedback(ipaddressPort) - +ipaddress-port-range-feedback(ipaddressPortRange) - - .group-content(ng-repeat='field in group.add') - -var field = 'new' - -var valid = 'form[ngModelName].$valid' - -var unique = 'form[ngModelName].$error.igniteUnique' - -var ipaddress = 'form[ngModelName].$error.ipaddress' - -var ipaddressPort = 'form[ngModelName].$error.ipaddressPort' - -var ipaddressPortRange = 'form[ngModelName].$error.ipaddressPortRange' - -var save = addresses + '.push(' + field + ')' - - ignite-form-field(type='internal' name='Address') - +table-address-field(field, addresses, valid, save, true, true) - +table-save-button(valid, save, true) - +unique-feedback(unique, uniqueTip) - +ipaddress-feedback(ipaddress) - +ipaddress-port-feedback(ipaddressPort) - +ipaddress-port-range-feedback(ipaddressPortRange) - - .group-content-empty(id='addresses' ng-if='!(#{addresses}.length) && !group.add.length') - | Not defined diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper.directive.js deleted file mode 100644 index 308144326181f..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './zookeeper.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryZookeeper', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.directive.js deleted file mode 100644 index 42fb2293b16e0..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './bounded-exponential-backoff.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryZookeeperBoundedExponential', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.directive.js deleted file mode 100644 index 969356b2ea1aa..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './custom.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryZookeeperCustom', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.directive.js deleted file mode 100644 index e68dff662db7e..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './exponential-backoff.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryZookeeperExponential', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.directive.js deleted file mode 100644 index 38bc2d8d6353f..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './forever.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryZookeeperForever', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.directive.js deleted file mode 100644 index f369af8e6dcc9..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './n-times.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryZookeeperNTimes', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.directive.js deleted file mode 100644 index e9e114da4b4c5..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './one-time.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryZookeeperOneTime', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.directive.js deleted file mode 100644 index 4ab76b2839a0a..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './until-elapsed.jade'; - -export default ['igniteConfigurationClustersGeneralDiscoveryZookeeperUntilElapsed', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger.directive.js deleted file mode 100644 index 8aec165c89804..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './logger.jade'; - -export default ['igniteConfigurationClustersLogger', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/custom.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/custom.directive.js deleted file mode 100644 index 999a3ca34f5ad..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/custom.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './custom.jade'; - -export default ['igniteConfigurationClustersLoggerCustom', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/log4j.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/log4j.directive.js deleted file mode 100644 index 32f55e96f9ae0..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/log4j.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './log4j.jade'; - -export default ['igniteConfigurationClustersLoggerLog4j', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/log4j2.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/log4j2.directive.js deleted file mode 100644 index 81ae296fdf259..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/logger/log4j2.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './log4j2.jade'; - -export default ['igniteConfigurationClustersLoggerLog4j2', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/marshaller.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/marshaller.directive.js deleted file mode 100644 index b4175b036f79a..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/marshaller.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './marshaller.jade'; - -export default ['igniteConfigurationClustersMarshaller', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/metrics.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/metrics.directive.js deleted file mode 100644 index 9b8ddc3b303ea..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/metrics.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './metrics.jade'; - -export default ['igniteConfigurationClustersMetrics', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/ssl.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/ssl.directive.js deleted file mode 100644 index ded5bf08784fc..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/ssl.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './ssl.jade'; - -export default ['igniteConfigurationClustersSsl', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/swap.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/swap.directive.js deleted file mode 100644 index c340c4bc840b1..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/swap.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './swap.jade'; - -export default ['igniteConfigurationClustersSwap', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/thread.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/thread.directive.js deleted file mode 100644 index f504e67c45ed4..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/thread.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './thread.jade'; - -export default ['igniteConfigurationClustersThread', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/time.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/time.directive.js deleted file mode 100644 index 8ab3e8940fa41..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/time.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './time.jade'; - -export default ['igniteConfigurationClustersTime', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/transactions.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/clusters/transactions.directive.js deleted file mode 100644 index faca0fab5dc83..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/clusters/transactions.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './transactions.jade'; - -export default ['igniteConfigurationClustersTransactions', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/domains/general.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/domains/general.directive.js deleted file mode 100644 index 9ca02f9b58bd2..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/domains/general.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './general.jade'; - -export default ['igniteConfigurationDomainsGeneral', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/domains/query.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/domains/query.directive.js deleted file mode 100644 index 2033394bdebc4..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/domains/query.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './query.jade'; - -export default ['igniteConfigurationDomainsQuery', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/domains/store.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/domains/store.directive.js deleted file mode 100644 index a69b025fdd250..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/domains/store.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './store.jade'; - -export default ['igniteConfigurationDomainsStore', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/dual.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/igfs/dual.directive.js deleted file mode 100644 index c5afc4e405f7b..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/dual.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './dual.jade'; - -export default ['igniteConfigurationIgfsDual', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/fragmentizer.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/igfs/fragmentizer.directive.js deleted file mode 100644 index d0089336befc0..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/fragmentizer.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './fragmentizer.jade'; - -export default ['igniteConfigurationIgfsFragmentizer', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/general.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/igfs/general.directive.js deleted file mode 100644 index bda54184cd0ae..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/general.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './general.jade'; - -export default ['igniteConfigurationIgfsGeneral', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/ipc.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/igfs/ipc.directive.js deleted file mode 100644 index eb52e51a902bc..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/ipc.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './ipc.jade'; - -export default ['igniteConfigurationIgfsIpc', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/misc.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/igfs/misc.directive.js deleted file mode 100644 index 810944f242b08..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/misc.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './misc.jade'; - -export default ['igniteConfigurationIgfsMisc', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/secondary.directive.js b/modules/web-console/src/main/js/app/modules/states/configuration/igfs/secondary.directive.js deleted file mode 100644 index 69179c0cb65fd..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/configuration/igfs/secondary.directive.js +++ /dev/null @@ -1,27 +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. - */ - -import templateUrl from './secondary.jade'; - -export default ['igniteConfigurationIgfsSecondary', [() => { - return { - scope: true, - restrict: 'E', - templateUrl, - replace: true - }; -}]]; diff --git a/modules/web-console/src/main/js/app/modules/states/signin.state.js b/modules/web-console/src/main/js/app/modules/states/signin.state.js deleted file mode 100644 index a23a496f9799d..0000000000000 --- a/modules/web-console/src/main/js/app/modules/states/signin.state.js +++ /dev/null @@ -1,53 +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. - */ - -import angular from 'angular'; -import templateUrl from 'views/signin.jade'; - -angular -.module('ignite-console.states.login', [ - 'ui.router', - // services - 'ignite-console.user' -]) -.config(['$stateProvider', function($stateProvider) { - // set up the states - $stateProvider - .state('signin', { - url: '/', - templateUrl, - metaTags: { - } - }); -}]) -.run(['$rootScope', '$state', 'Auth', 'IgniteBranding', function($root, $state, Auth, branding) { - $root.$on('$stateChangeStart', function(event, toState) { - if (toState.name === branding.termsState) - return; - - if (!Auth.authorized && (toState.name !== 'signin' && !_.startsWith(toState.name, 'password.'))) { - event.preventDefault(); - - $state.go('signin'); - } - else if (Auth.authorized && toState.name === 'signin') { - event.preventDefault(); - - $state.go('base.configuration.clusters'); - } - }); -}]); diff --git a/modules/web-console/src/main/js/app/modules/user/Auth.service.js b/modules/web-console/src/main/js/app/modules/user/Auth.service.js deleted file mode 100644 index 080f4afc58c0c..0000000000000 --- a/modules/web-console/src/main/js/app/modules/user/Auth.service.js +++ /dev/null @@ -1,76 +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. - */ - -export default ['Auth', ['$http', '$rootScope', '$state', '$window', 'IgniteLegacyUtils', 'IgniteMessages', 'gettingStarted', 'User', 'IgniteAgentMonitor', - ($http, $root, $state, $window, LegacyUtils, Messages, gettingStarted, User, agentMonitor) => { - let _auth = false; - - try { - _auth = localStorage.authorized === 'true'; - } - catch (ignore) { - // No-op. - } - - function _authorized(value) { - try { - return _auth = localStorage.authorized = !!value; - } catch (ignore) { - return _auth = !!value; - } - } - - return { - get authorized() { - return _auth; - }, - set authorized(auth) { - _authorized(auth); - }, - forgotPassword(userInfo) { - return $http.post('/api/v1/password/forgot', userInfo) - .success(() => $state.go('password.send')) - .error((err) => LegacyUtils.showPopoverMessage(null, null, 'forgot_email', Messages.errorMessage(null, err))); - }, - auth(action, userInfo) { - return $http.post('/api/v1/' + action, userInfo) - .success(() => { - return User.read().then((user) => { - _authorized(true); - - $root.$broadcast('user', user); - - $state.go('base.configuration.clusters'); - - $root.gettingStarted.tryShow(); - - agentMonitor.init(); - }); - }) - .error((err) => LegacyUtils.showPopoverMessage(null, null, action + '_email', Messages.errorMessage(null, err))); - }, - logout() { - return $http.post('/api/v1/logout') - .then(() => { - User.clean(); - - $window.open($state.href('signin'), '_self'); - }) - .catch(Messages.showError); - } - }; - }]]; diff --git a/modules/web-console/src/main/js/app/services/LegacyTable.service.js b/modules/web-console/src/main/js/app/services/LegacyTable.service.js deleted file mode 100644 index 8f3b79158ab05..0000000000000 --- a/modules/web-console/src/main/js/app/services/LegacyTable.service.js +++ /dev/null @@ -1,205 +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. - */ - -// TODO: Refactor this service for legacy tables with more than one input field. -export default ['IgniteLegacyTable', ['IgniteLegacyUtils', 'IgniteFocus', (LegacyUtils, Focus) => { - function _model(item, field) { - return LegacyUtils.getModel(item, field); - } - - const table = {name: 'none', editIndex: -1}; - - function _tableReset() { - delete table.field; - table.name = 'none'; - table.editIndex = -1; - - LegacyUtils.hidePopover(); - } - - function _tableSaveAndReset() { - const field = table.field; - - const save = LegacyUtils.isDefined(field) && LegacyUtils.isDefined(field.save); - - if (!save || !LegacyUtils.isDefined(field) || field.save(field, table.editIndex, true)) { - _tableReset(); - - return true; - } - - return false; - } - - function _tableState(field, editIndex, specName) { - table.field = field; - table.name = specName || field.model; - table.editIndex = editIndex; - } - - function _tableUI(field) { - const ui = field.ui; - - return ui ? ui : field.type; - } - - function _tableFocus(focusId, index) { - Focus.move((index < 0 ? 'new' : 'cur') + focusId + (index >= 0 ? index : '')); - } - - function _tablePairValue(filed, index) { - return index < 0 ? {key: filed.newKey, value: filed.newValue} : {key: filed.curKey, value: filed.curValue}; - } - - function _tableStartEdit(item, tbl, index, save) { - _tableState(tbl, index); - - const val = _.get(_model(item, tbl), tbl.model)[index]; - - const ui = _tableUI(tbl); - - tbl.save = save; - - if (ui === 'table-pair') { - tbl.curKey = val[tbl.keyName]; - tbl.curValue = val[tbl.valueName]; - - _tableFocus('Key' + tbl.focusId, index); - } - else if (ui === 'table-db-fields') { - tbl.curDatabaseFieldName = val.databaseFieldName; - tbl.curDatabaseFieldType = val.databaseFieldType; - tbl.curJavaFieldName = val.javaFieldName; - tbl.curJavaFieldType = val.javaFieldType; - - _tableFocus('DatabaseFieldName' + tbl.focusId, index); - } - else if (ui === 'table-indexes') { - tbl.curIndexName = val.name; - tbl.curIndexType = val.indexType; - tbl.curIndexFields = val.fields; - - _tableFocus(tbl.focusId, index); - } - } - - function _tableNewItem(tbl) { - _tableState(tbl, -1); - - const ui = _tableUI(tbl); - - if (ui === 'table-pair') { - tbl.newKey = null; - tbl.newValue = null; - - _tableFocus('Key' + tbl.focusId, -1); - } - else if (ui === 'table-db-fields') { - tbl.newDatabaseFieldName = null; - tbl.newDatabaseFieldType = null; - tbl.newJavaFieldName = null; - tbl.newJavaFieldType = null; - - _tableFocus('DatabaseFieldName' + tbl.focusId, -1); - } - else if (ui === 'table-indexes') { - tbl.newIndexName = null; - tbl.newIndexType = 'SORTED'; - tbl.newIndexFields = null; - - _tableFocus(tbl.focusId, -1); - } - } - - return { - tableState: _tableState, - tableReset: _tableReset, - tableSaveAndReset: _tableSaveAndReset, - tableNewItem: _tableNewItem, - tableNewItemActive(tbl) { - return table.name === tbl.model && table.editIndex < 0; - }, - tableEditing(tbl, index) { - return table.name === tbl.model && table.editIndex === index; - }, - tableEditedRowIndex() { - return table.editIndex; - }, - tableField() { - return table.field; - }, - tableStartEdit: _tableStartEdit, - tableRemove(item, field, index) { - _tableReset(); - - _.get(_model(item, field), field.model).splice(index, 1); - }, - tablePairValue: _tablePairValue, - tablePairSave(pairValid, item, field, index, stopEdit) { - const valid = pairValid(item, field, index); - - if (valid) { - const pairValue = _tablePairValue(field, index); - - let pairModel = {}; - - const container = _.get(item, field.model); - - if (index < 0) { - pairModel[field.keyName] = pairValue.key; - pairModel[field.valueName] = pairValue.value; - - if (container) - container.push(pairModel); - else - _.set(item, field.model, [pairModel]); - - if (!stopEdit) - _tableNewItem(field); - } - else { - pairModel = container[index]; - - pairModel[field.keyName] = pairValue.key; - pairModel[field.valueName] = pairValue.value; - - if (!stopEdit) { - if (index < container.length - 1) - _tableStartEdit(item, field, index + 1); - else - _tableNewItem(field); - } - } - } - - return valid; - }, - tablePairSaveVisible(field, index) { - const pairValue = _tablePairValue(field, index); - - return !LegacyUtils.isEmptyString(pairValue.key) && !LegacyUtils.isEmptyString(pairValue.value); - }, - tableFocusInvalidField(index, id) { - _tableFocus(id, index); - - return false; - }, - tableFieldId(index, id) { - return (index < 0 ? 'new' : 'cur') + id + (index >= 0 ? index : ''); - } - }; -}]]; diff --git a/modules/web-console/src/main/js/app/services/LegacyUtils.service.js b/modules/web-console/src/main/js/app/services/LegacyUtils.service.js deleted file mode 100644 index c4c83e93acc31..0000000000000 --- a/modules/web-console/src/main/js/app/services/LegacyUtils.service.js +++ /dev/null @@ -1,948 +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. - */ - -// TODO: Refactor this service for legacy tables with more than one input field. -export default ['IgniteLegacyUtils', [ - '$alert', '$popover', '$anchorScroll', '$location', '$timeout', '$window', 'IgniteFocus', - ($alert, $popover, $anchorScroll, $location, $timeout, $window, Focus) => { - $anchorScroll.yOffset = 55; - - function isDefined(v) { - return !_.isNil(v); - } - - function isEmptyString(s) { - if (isDefined(s)) - return s.trim().length === 0; - - return true; - } - - const javaBuiltInClasses = [ - 'BigDecimal', 'Boolean', 'Byte', 'Date', 'Double', 'Float', 'Integer', 'Long', 'Object', 'Short', 'String', 'Time', 'Timestamp', 'UUID' - ]; - - const javaBuiltInTypes = [ - 'BigDecimal', 'boolean', 'Boolean', 'byte', 'Byte', 'Date', 'double', 'Double', 'float', 'Float', - 'int', 'Integer', 'long', 'Long', 'Object', 'short', 'Short', 'String', 'Time', 'Timestamp', 'UUID' - ]; - - const javaBuiltInFullNameClasses = [ - 'java.math.BigDecimal', 'java.lang.Boolean', 'java.lang.Byte', 'java.sql.Date', 'java.lang.Double', - 'java.lang.Float', 'java.lang.Integer', 'java.lang.Long', 'java.lang.Object', 'java.lang.Short', - 'java.lang.String', 'java.sql.Time', 'java.sql.Timestamp', 'java.util.UUID' - ]; - - /** - * @param clsName Class name to check. - * @returns {Boolean} 'true' if given class name is a java build-in type. - */ - function isJavaBuiltInClass(clsName) { - if (isEmptyString(clsName)) - return false; - - return _.includes(javaBuiltInClasses, clsName) || _.includes(javaBuiltInFullNameClasses, clsName); - } - - const SUPPORTED_JDBC_TYPES = [ - 'BIGINT', - 'BIT', - 'BOOLEAN', - 'BLOB', - 'CHAR', - 'CLOB', - 'DATE', - 'DECIMAL', - 'DOUBLE', - 'FLOAT', - 'INTEGER', - 'LONGNVARCHAR', - 'LONGVARCHAR', - 'NCHAR', - 'NUMERIC', - 'NVARCHAR', - 'REAL', - 'SMALLINT', - 'TIME', - 'TIMESTAMP', - 'TINYINT', - 'VARCHAR' - ]; - - const ALL_JDBC_TYPES = [ - {dbName: 'BIT', dbType: -7, javaType: 'Boolean', primitiveType: 'boolean'}, - {dbName: 'TINYINT', dbType: -6, javaType: 'Byte', primitiveType: 'byte'}, - {dbName: 'SMALLINT', dbType: 5, javaType: 'Short', primitiveType: 'short'}, - {dbName: 'INTEGER', dbType: 4, javaType: 'Integer', primitiveType: 'int'}, - {dbName: 'BIGINT', dbType: -5, javaType: 'Long', primitiveType: 'long'}, - {dbName: 'FLOAT', dbType: 6, javaType: 'Float', primitiveType: 'float'}, - {dbName: 'REAL', dbType: 7, javaType: 'Double', primitiveType: 'double'}, - {dbName: 'DOUBLE', dbType: 8, javaType: 'Double', primitiveType: 'double'}, - {dbName: 'NUMERIC', dbType: 2, javaType: 'BigDecimal'}, - {dbName: 'DECIMAL', dbType: 3, javaType: 'BigDecimal'}, - {dbName: 'CHAR', dbType: 1, javaType: 'String'}, - {dbName: 'VARCHAR', dbType: 12, javaType: 'String'}, - {dbName: 'LONGVARCHAR', dbType: -1, javaType: 'String'}, - {dbName: 'DATE', dbType: 91, javaType: 'Date'}, - {dbName: 'TIME', dbType: 92, javaType: 'Time'}, - {dbName: 'TIMESTAMP', dbType: 93, javaType: 'Timestamp'}, - {dbName: 'BINARY', dbType: -2, javaType: 'Object'}, - {dbName: 'VARBINARY', dbType: -3, javaType: 'Object'}, - {dbName: 'LONGVARBINARY', dbType: -4, javaType: 'Object'}, - {dbName: 'NULL', dbType: 0, javaType: 'Object'}, - {dbName: 'OTHER', dbType: 1111, javaType: 'Object'}, - {dbName: 'JAVA_OBJECT', dbType: 2000, javaType: 'Object'}, - {dbName: 'DISTINCT', dbType: 2001, javaType: 'Object'}, - {dbName: 'STRUCT', dbType: 2002, javaType: 'Object'}, - {dbName: 'ARRAY', dbType: 2003, javaType: 'Object'}, - {dbName: 'BLOB', dbType: 2004, javaType: 'Object'}, - {dbName: 'CLOB', dbType: 2005, javaType: 'String'}, - {dbName: 'REF', dbType: 2006, javaType: 'Object'}, - {dbName: 'DATALINK', dbType: 70, javaType: 'Object'}, - {dbName: 'BOOLEAN', dbType: 16, javaType: 'Boolean', primitiveType: 'boolean'}, - {dbName: 'ROWID', dbType: -8, javaType: 'Object'}, - {dbName: 'NCHAR', dbType: -15, javaType: 'String'}, - {dbName: 'NVARCHAR', dbType: -9, javaType: 'String'}, - {dbName: 'LONGNVARCHAR', dbType: -16, javaType: 'String'}, - {dbName: 'NCLOB', dbType: 2011, javaType: 'String'}, - {dbName: 'SQLXML', dbType: 2009, javaType: 'Object'} - ]; - - /*eslint-disable */ - const JAVA_KEYWORDS = [ - 'abstract', 'assert', 'boolean', 'break', 'byte', - 'case', 'catch', 'char', 'class', 'const', - 'continue', 'default', 'do', 'double', 'else', - 'enum', 'extends', 'false', 'final', 'finally', - 'float', 'for', 'goto', 'if', 'implements', - 'import', 'instanceof', 'int', 'interface', 'long', - 'native', 'new', 'null', 'package', 'private', - 'protected', 'public', 'return', 'short', 'static', - 'strictfp', 'super', 'switch', 'synchronized', 'this', - 'throw', 'throws', 'transient', 'true', 'try', - 'void', 'volatile', 'while' - ]; - /*eslint-enable */ - - const VALID_JAVA_IDENTIFIER = new RegExp('^[a-zA-Z_$][a-zA-Z\\d_$]*$'); - - let popover = null; - - function isElementInViewport(el) { - const rect = el.getBoundingClientRect(); - - return ( - rect.top >= 0 && - rect.left >= 0 && - rect.bottom <= (window.innerHeight || document.documentElement.clientHeight) && - rect.right <= (window.innerWidth || document.documentElement.clientWidth) - ); - } - - const _showPopoverMessage = (id, message, showTime) => { - const body = $('body'); - - let el = body.find('#' + id); - - if (!el || el.length === 0) - el = body.find('[name="' + id + '"]'); - - if (el && el.length > 0) { - if (!isElementInViewport(el[0])) { - $location.hash(el[0].id); - - $anchorScroll(); - } - - const newPopover = $popover(el, {content: message}); - - popover = newPopover; - - $timeout(() => newPopover.$promise.then(() => { - newPopover.show(); - - // Workaround to fix popover location when content is longer than content template. - // https://github.com/mgcrea/angular-strap/issues/1497 - $timeout(newPopover.$applyPlacement); - }), 400); - $timeout(() => newPopover.hide(), showTime || 5000); - } - }; - - function ensureActivePanel(ui, pnl, focusId) { - if (ui) { - const collapses = $('div.panel-collapse'); - - ui.loadPanel(pnl); - - const idx = _.findIndex(collapses, function(collapse) { - return collapse.id === pnl; - }); - - if (idx >= 0) { - const activePanels = ui.activePanels; - - if (!_.includes(ui.topPanels, idx)) { - ui.expanded = true; - - const customExpanded = ui[pnl]; - - if (customExpanded) - ui[customExpanded] = true; - } - - if (!activePanels || activePanels.length < 1) - ui.activePanels = [idx]; - else if (!_.includes(activePanels, idx)) { - const newActivePanels = angular.copy(activePanels); - - newActivePanels.push(idx); - - ui.activePanels = newActivePanels; - } - } - - if (isDefined(focusId)) - Focus.move(focusId); - } - } - - function showPopoverMessage(ui, panelId, id, message, showTime) { - if (popover) - popover.hide(); - - if (ui) { - ensureActivePanel(ui, panelId); - - $timeout(() => _showPopoverMessage(id, message, showTime), ui.isPanelLoaded(panelId) ? 200 : 500); - } - else - _showPopoverMessage(id, message, showTime); - - return false; - } - - function isValidJavaIdentifier(msg, ident, elemId, panels, panelId) { - if (isEmptyString(ident)) - return showPopoverMessage(panels, panelId, elemId, msg + ' is invalid!'); - - if (_.includes(JAVA_KEYWORDS, ident)) - return showPopoverMessage(panels, panelId, elemId, msg + ' could not contains reserved java keyword: "' + ident + '"!'); - - if (!VALID_JAVA_IDENTIFIER.test(ident)) - return showPopoverMessage(panels, panelId, elemId, msg + ' contains invalid identifier: "' + ident + '"!'); - - return true; - } - - let context = null; - - /** - * Calculate width of specified text in body's font. - * - * @param text Text to calculate width. - * @returns {Number} Width of text in pixels. - */ - function measureText(text) { - if (!context) { - const canvas = document.createElement('canvas'); - - context = canvas.getContext('2d'); - - const style = window.getComputedStyle(document.getElementsByTagName('body')[0]); - - context.font = style.fontSize + ' ' + style.fontFamily; - } - - return context.measureText(text).width; - } - - /** - * Compact java full class name by max number of characters. - * - * @param names Array of class names to compact. - * @param nameLength Max available width in characters for simple name. - * @returns {*} Array of compacted class names. - */ - function compactByMaxCharts(names, nameLength) { - for (let nameIx = 0; nameIx < names.length; nameIx++) { - const s = names[nameIx]; - - if (s.length > nameLength) { - let totalLength = s.length; - - const packages = s.split('.'); - - const packageCnt = packages.length - 1; - - for (let i = 0; i < packageCnt && totalLength > nameLength; i++) { - if (packages[i].length > 0) { - totalLength -= packages[i].length - 1; - - packages[i] = packages[i][0]; - } - } - - if (totalLength > nameLength) { - const className = packages[packageCnt]; - - const classNameLen = className.length; - - let remains = Math.min(nameLength - totalLength + classNameLen, classNameLen); - - if (remains < 3) - remains = Math.min(3, classNameLen); - - packages[packageCnt] = className.substring(0, remains) + '...'; - } - - let result = packages[0]; - - for (let i = 1; i < packages.length; i++) - result += '.' + packages[i]; - - names[nameIx] = result; - } - } - - return names; - } - - /** - * Compact java full class name by max number of pixels. - * - * @param names Array of class names to compact. - * @param nameLength Max available width in characters for simple name. Used for calculation optimization. - * @param nameWidth Maximum available width in pixels for simple name. - * @returns {*} Array of compacted class names. - */ - function compactByMaxPixels(names, nameLength, nameWidth) { - if (nameWidth <= 0) - return names; - - const fitted = []; - - const widthByName = []; - - const len = names.length; - - let divideTo = len; - - for (let nameIx = 0; nameIx < len; nameIx++) { - fitted[nameIx] = false; - - widthByName[nameIx] = nameWidth; - } - - // Try to distribute space from short class names to long class names. - let remains = 0; - - do { - for (let nameIx = 0; nameIx < len; nameIx++) { - if (!fitted[nameIx]) { - const curNameWidth = measureText(names[nameIx]); - - if (widthByName[nameIx] > curNameWidth) { - fitted[nameIx] = true; - - remains += widthByName[nameIx] - curNameWidth; - - divideTo -= 1; - - widthByName[nameIx] = curNameWidth; - } - } - } - - const remainsByName = remains / divideTo; - - for (let nameIx = 0; nameIx < len; nameIx++) { - if (!fitted[nameIx]) - widthByName[nameIx] += remainsByName; - } - } - while (remains > 0); - - // Compact class names to available for each space. - for (let nameIx = 0; nameIx < len; nameIx++) { - const s = names[nameIx]; - - if (s.length > (nameLength / 2 | 0)) { - let totalWidth = measureText(s); - - if (totalWidth > widthByName[nameIx]) { - const packages = s.split('.'); - - const packageCnt = packages.length - 1; - - for (let i = 0; i < packageCnt && totalWidth > widthByName[nameIx]; i++) { - if (packages[i].length > 1) { - totalWidth -= measureText(packages[i].substring(1, packages[i].length)); - - packages[i] = packages[i][0]; - } - } - - let shortPackage = ''; - - for (let i = 0; i < packageCnt; i++) - shortPackage += packages[i] + '.'; - - const className = packages[packageCnt]; - - const classLen = className.length; - - let minLen = Math.min(classLen, 3); - - totalWidth = measureText(shortPackage + className); - - // Compact class name if shorten package path is very long. - if (totalWidth > widthByName[nameIx]) { - let maxLen = classLen; - let middleLen = (minLen + (maxLen - minLen) / 2 ) | 0; - - while (middleLen !== minLen && middleLen !== maxLen) { - const middleLenPx = measureText(shortPackage + className.substr(0, middleLen) + '...'); - - if (middleLenPx > widthByName[nameIx]) - maxLen = middleLen; - else - minLen = middleLen; - - middleLen = (minLen + (maxLen - minLen) / 2 ) | 0; - } - - names[nameIx] = shortPackage + className.substring(0, middleLen) + '...'; - } - else - names[nameIx] = shortPackage + className; - } - } - } - - return names; - } - - /** - * Compact any string by max number of pixels. - * - * @param label String to compact. - * @param nameWidth Maximum available width in pixels for simple name. - * @returns {*} Compacted string. - */ - function compactLabelByPixels(label, nameWidth) { - if (nameWidth <= 0) - return label; - - const totalWidth = measureText(label); - - if (totalWidth > nameWidth) { - let maxLen = label.length; - let minLen = Math.min(maxLen, 3); - let middleLen = (minLen + (maxLen - minLen) / 2 ) | 0; - - while (middleLen !== minLen && middleLen !== maxLen) { - const middleLenPx = measureText(label.substr(0, middleLen) + '...'); - - if (middleLenPx > nameWidth) - maxLen = middleLen; - else - minLen = middleLen; - - middleLen = (minLen + (maxLen - minLen) / 2 ) | 0; - } - - return label.substring(0, middleLen) + '...'; - } - - return label; - } - - /** - * Calculate available width for text in link to edit element. - * - * @param index Showed index of element for calculation of maximum width in pixels. - * @param id Id of contains link table. - * @returns {*[]} First element is length of class for single value, second element is length for pair vlaue. - */ - function availableWidth(index, id) { - const idElem = $('#' + id); - - let width = 0; - - switch (idElem.prop('tagName')) { - // Detection of available width in presentation table row. - case 'TABLE': - const cont = $(idElem.find('tr')[index - 1]).find('td')[0]; - - width = cont.clientWidth; - - if (width > 0) { - const children = $(cont).children(':not("a")'); - - _.forEach(children, function(child) { - if ('offsetWidth' in child) - width -= $(child).outerWidth(true); - }); - } - - break; - - // Detection of available width in dropdown row. - case 'A': - width = idElem.width(); - - $(idElem).children(':not("span")').each(function(ix, child) { - if ('offsetWidth' in child) - width -= child.offsetWidth; - }); - - break; - - default: - } - - return width | 0; - } - - function getModel(obj, field) { - let path = field.path; - - if (!isDefined(path) || !isDefined(obj)) - return obj; - - path = path.replace(/\[(\w+)\]/g, '.$1'); // convert indexes to properties - path = path.replace(/^\./, ''); // strip a leading dot - - const segs = path.split('.'); - let root = obj; - - while (segs.length > 0) { - const pathStep = segs.shift(); - - if (typeof root[pathStep] === 'undefined') - root[pathStep] = {}; - - root = root[pathStep]; - } - - return root; - } - - function extractDataSource(cache) { - if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) { - const storeFactory = cache.cacheStoreFactory[cache.cacheStoreFactory.kind]; - - if (storeFactory.dialect || (storeFactory.connectVia === 'DataSource')) - return storeFactory; - } - - return null; - } - - const cacheStoreJdbcDialects = [ - {value: 'Generic', label: 'Generic JDBC'}, - {value: 'Oracle', label: 'Oracle'}, - {value: 'DB2', label: 'IBM DB2'}, - {value: 'SQLServer', label: 'Microsoft SQL Server'}, - {value: 'MySQL', label: 'MySQL'}, - {value: 'PostgreSQL', label: 'PostgreSQL'}, - {value: 'H2', label: 'H2 database'} - ]; - - function domainForStoreConfigured(domain) { - const isEmpty = !isDefined(domain) || (isEmptyString(domain.databaseSchema) && - isEmptyString(domain.databaseTable) && - _.isEmpty(domain.keyFields) && - _.isEmpty(domain.valueFields)); - - return !isEmpty; - } - - const DS_CHECK_SUCCESS = { checked: true }; - - function compareDataSources(firstCache, secondCache) { - const firstDs = extractDataSource(firstCache); - const secondDs = extractDataSource(secondCache); - - if (firstDs && secondDs) { - const firstDB = firstDs.dialect; - const secondDB = secondDs.dialect; - - if (firstDs.dataSourceBean === secondDs.dataSourceBean && firstDB !== secondDB) - return {checked: false, firstCache, firstDB, secondCache, secondDB}; - } - - return DS_CHECK_SUCCESS; - } - - function compareSQLSchemaNames(firstCache, secondCache) { - const firstName = firstCache.sqlSchema; - const secondName = secondCache.sqlSchema; - - if (firstName && secondName && (firstName === secondName)) - return {checked: false, firstCache, secondCache}; - - return DS_CHECK_SUCCESS; - } - - function toJavaName(prefix, name) { - const javaName = name ? name.replace(/[^A-Za-z_0-9]+/g, '_') : 'dflt'; - - return prefix + javaName.charAt(0).toLocaleUpperCase() + javaName.slice(1); - } - - return { - getModel, - mkOptions(options) { - return _.map(options, (option) => { - return {value: option, label: isDefined(option) ? option : 'Not set'}; - }); - }, - isDefined, - hasProperty(obj, props) { - for (const propName in props) { - if (props.hasOwnProperty(propName)) { - if (obj[propName]) - return true; - } - } - - return false; - }, - isEmptyString, - SUPPORTED_JDBC_TYPES, - findJdbcType(jdbcType) { - const res = _.find(ALL_JDBC_TYPES, function(item) { - return item.dbType === jdbcType; - }); - - return res ? res : {dbName: 'Unknown', javaType: 'Unknown'}; - }, - javaBuiltInClasses, - javaBuiltInTypes, - isJavaBuiltInClass, - isValidJavaIdentifier, - isValidJavaClass(msg, ident, allowBuiltInClass, elemId, packageOnly, panels, panelId) { - if (isEmptyString(ident)) - return showPopoverMessage(panels, panelId, elemId, msg + ' could not be empty!'); - - const parts = ident.split('.'); - - const len = parts.length; - - if (!allowBuiltInClass && isJavaBuiltInClass(ident)) - return showPopoverMessage(panels, panelId, elemId, msg + ' should not be the Java build-in class!'); - - if (len < 2 && !isJavaBuiltInClass(ident) && !packageOnly) - return showPopoverMessage(panels, panelId, elemId, msg + ' does not have package specified!'); - - for (let i = 0; i < parts.length; i++) { - const part = parts[i]; - - if (!isValidJavaIdentifier(msg, part, elemId, panels, panelId)) - return false; - } - - return true; - }, - domainForQueryConfigured(domain) { - const isEmpty = !isDefined(domain) || (_.isEmpty(domain.fields) && - _.isEmpty(domain.aliases) && - _.isEmpty(domain.indexes)); - - return !isEmpty; - }, - domainForStoreConfigured, - /** - * Cut class name by width in pixel or width in symbol count. - * - * @param id Id of parent table. - * @param index Row number in table. - * @param maxLength Maximum length in symbols for all names. - * @param names Array of class names to compact. - * @param divider String to visualy divide items. - * @returns {*} Array of compacted class names. - */ - compactJavaName(id, index, maxLength, names, divider) { - divider = ' ' + divider + ' '; - - const prefix = index + ') '; - - const nameCnt = names.length; - - const nameLength = ((maxLength - 3 * (nameCnt - 1)) / nameCnt) | 0; - - try { - const nameWidth = (availableWidth(index, id) - measureText(prefix) - (nameCnt - 1) * measureText(divider)) / - nameCnt | 0; - - // HTML5 calculation of showed message width. - names = compactByMaxPixels(names, nameLength, nameWidth); - } - catch (err) { - names = compactByMaxCharts(names, nameLength); - } - - let result = prefix + names[0]; - - for (let nameIx = 1; nameIx < names.length; nameIx++) - result += divider + names[nameIx]; - - return result; - }, - /** - * Compact text by width in pixels or symbols count. - * - * @param id Id of parent table. - * @param index Row number in table. - * @param maxLength Maximum length in symbols for all names. - * @param label Text to compact. - * @returns Compacted label text. - */ - compactTableLabel(id, index, maxLength, label) { - label = index + ') ' + label; - - try { - const nameWidth = availableWidth(index, id) | 0; - - // HTML5 calculation of showed message width. - label = compactLabelByPixels(label, nameWidth); - } - catch (err) { - const nameLength = maxLength - 3 | 0; - - label = label.length > maxLength ? label.substr(0, nameLength) + '...' : label; - } - - return label; - }, - widthIsSufficient(id, index, text) { - try { - const available = availableWidth(index, id); - - const required = measureText(text); - - return !available || available >= Math.floor(required); - } - catch (err) { - return true; - } - }, - ensureActivePanel(panels, id, focusId) { - ensureActivePanel(panels, id, focusId); - }, - showPopoverMessage, - hidePopover() { - if (popover) - popover.hide(); - }, - confirmUnsavedChanges(dirty, selectFunc) { - if (dirty) { - if ($window.confirm('You have unsaved changes.\n\nAre you sure you want to discard them?')) - selectFunc(); - } - else - selectFunc(); - }, - saveBtnTipText(dirty, objectName) { - if (dirty) - return 'Save ' + objectName; - - return 'Nothing to save'; - }, - download(type, name, data) { - const file = document.createElement('a'); - - file.setAttribute('href', 'data:' + type + ';charset=utf-8,' + data); - file.setAttribute('download', name); - file.setAttribute('target', '_self'); - - file.style.display = 'none'; - - document.body.appendChild(file); - - file.click(); - - document.body.removeChild(file); - }, - formUI() { - return { - ready: false, - expanded: false, - loadedPanels: [], - loadPanel(pnl) { - if (!_.includes(this.loadedPanels, pnl)) - this.loadedPanels.push(pnl); - }, - isPanelLoaded(pnl) { - return _.includes(this.loadedPanels, pnl); - } - }; - }, - getQueryVariable(name) { - const attrs = window.location.search.substring(1).split('&'); - const attr = _.find(attrs, (a) => a === name || (a.indexOf('=') >= 0 && a.substr(0, a.indexOf('=')) === name)); - - if (!isDefined(attr)) - return null; - - if (attr === name) - return true; - - return attr.substr(attr.indexOf('=') + 1); - }, - cacheStoreJdbcDialects, - cacheStoreJdbcDialectsLabel(dialect) { - const found = _.find(cacheStoreJdbcDialects, function(dialectVal) { - return dialectVal.value === dialect; - }); - - return found ? found.label : null; - }, - checkCachesDataSources(caches, checkCacheExt) { - let res = DS_CHECK_SUCCESS; - - _.find(caches, function(curCache, curIx) { - if (isDefined(checkCacheExt)) { - if (checkCacheExt._id !== curCache._id) { - res = compareDataSources(checkCacheExt, curCache); - - return !res.checked; - } - - return false; - } - - return _.find(caches, function(checkCache, checkIx) { - if (checkIx < curIx) { - res = compareDataSources(checkCache, curCache); - - return !res.checked; - } - - return false; - }); - }); - - return res; - }, - checkCacheSQLSchemas(caches, checkCacheExt) { - let res = DS_CHECK_SUCCESS; - - _.find(caches, (curCache, curIx) => { - if (isDefined(checkCacheExt)) { - if (checkCacheExt._id !== curCache._id) { - res = compareSQLSchemaNames(checkCacheExt, curCache); - - return !res.checked; - } - - return false; - } - - return _.find(caches, function(checkCache, checkIx) { - if (checkIx < curIx) { - res = compareSQLSchemaNames(checkCache, curCache); - - return !res.checked; - } - - return false; - }); - }); - - return res; - }, - autoCacheStoreConfiguration(cache, domains) { - const cacheStoreFactory = isDefined(cache.cacheStoreFactory) && - isDefined(cache.cacheStoreFactory.kind); - - if (!cacheStoreFactory && _.findIndex(domains, domainForStoreConfigured) >= 0) { - const dflt = !cache.readThrough && !cache.writeThrough; - - return { - cacheStoreFactory: { - kind: 'CacheJdbcPojoStoreFactory', - CacheJdbcPojoStoreFactory: { - dataSourceBean: toJavaName('ds', cache.name), - dialect: 'Generic' - }, - CacheJdbcBlobStoreFactory: { connectVia: 'DataSource' } - }, - readThrough: dflt || cache.readThrough, - writeThrough: dflt || cache.writeThrough - }; - } - }, - autoClusterSwapSpiConfiguration(cluster, caches) { - const swapConfigured = cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind; - - if (!swapConfigured && _.find(caches, (cache) => cache.swapEnabled)) - return {swapSpaceSpi: {kind: 'FileSwapSpaceSpi'}}; - - return null; - }, - randomString(len) { - const possible = 'ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789'; - const possibleLen = possible.length; - - let res = ''; - - for (let i = 0; i < len; i++) - res += possible.charAt(Math.floor(Math.random() * possibleLen)); - - return res; - }, - checkFieldValidators(ui) { - const form = ui.inputForm; - const errors = form.$error; - const errKeys = Object.keys(errors); - - if (errKeys && errKeys.length > 0) { - const firstErrorKey = errKeys[0]; - - const firstError = errors[firstErrorKey][0]; - const actualError = firstError.$error[firstErrorKey][0]; - - const errNameFull = actualError.$name; - const errNameShort = errNameFull.endsWith('TextInput') ? errNameFull.substring(0, errNameFull.length - 9) : errNameFull; - - const extractErrorMessage = function(errName) { - try { - return errors[firstErrorKey][0].$errorMessages[errName][firstErrorKey]; - } - catch (ignored) { - try { - return form[firstError.$name].$errorMessages[errName][firstErrorKey]; - } - catch (ignited) { - return false; - } - } - }; - - const msg = extractErrorMessage(errNameFull) || extractErrorMessage(errNameShort) || 'Invalid value!'; - - return showPopoverMessage(ui, firstError.$name, errNameFull, msg); - } - - return true; - } - }; - } -]]; diff --git a/modules/web-console/src/main/js/serve/mail.js b/modules/web-console/src/main/js/serve/mail.js deleted file mode 100644 index 2c67276f6e21f..0000000000000 --- a/modules/web-console/src/main/js/serve/mail.js +++ /dev/null @@ -1,75 +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. - */ - -'use strict'; - -// Fire me up! - -/** - * Module for send email. - */ -module.exports = { - implements: 'mail', - inject: ['require(nodemailer)', 'settings'] -}; - -module.exports.factory = function(nodemailer, settings) { - return { - /** - * Send mail to user. - * - * @param {Account} user - * @param {String} subject - * @param {String} html - * @param {String} sendErr - * @throws {Error} - * @return {Promise} - */ - send: (user, subject, html, sendErr) => { - const transporter = { - service: settings.smtp.service, - auth: { - user: settings.smtp.email, - pass: settings.smtp.password - } - }; - - if (transporter.service === '' || transporter.auth.user === '' || transporter.auth.pass === '') - throw new Error('Failed to send email. SMTP server is not configured. Please ask webmaster to setup SMTP server!'); - - const mailer = nodemailer.createTransport(transporter); - - const sign = settings.smtp.sign ? `

          --------------
          ${settings.smtp.sign}
          ` : ''; - - const mail = { - from: settings.smtp.address(settings.smtp.username, settings.smtp.email), - to: settings.smtp.address(`${user.firstName} ${user.lastName}`, user.email), - subject, - html: html + sign - }; - - return new Promise((resolve, reject) => { - mailer.sendMail(mail, (err) => { - if (err) - return reject(sendErr ? new Error(sendErr) : err); - - resolve(user); - }); - }); - } - }; -}; diff --git a/modules/web-console/src/main/js/serve/routes/admin.js b/modules/web-console/src/main/js/serve/routes/admin.js deleted file mode 100644 index 3c2e7280cae25..0000000000000 --- a/modules/web-console/src/main/js/serve/routes/admin.js +++ /dev/null @@ -1,126 +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. - */ - -'use strict'; - -// Fire me up! - -module.exports = { - implements: 'admin-routes', - inject: ['require(lodash)', 'require(express)', 'settings', 'mail', 'mongo'] -}; - -module.exports.factory = function(_, express, settings, mail, mongo) { - return new Promise((factoryResolve) => { - const router = new express.Router(); - - /** - * Get list of user accounts. - */ - router.post('/list', (req, res) => { - - Promise.all([ - mongo.Space.aggregate([ - {$match: {demo: false}}, - {$lookup: {from: 'clusters', localField: '_id', foreignField: 'space', as: 'clusters'}}, - {$lookup: {from: 'caches', localField: '_id', foreignField: 'space', as: 'caches'}}, - {$lookup: {from: 'domainmodels', localField: '_id', foreignField: 'space', as: 'domainmodels'}}, - {$lookup: {from: 'igfs', localField: '_id', foreignField: 'space', as: 'igfs'}}, - {$project: { - owner: 1, - clusters: {$size: '$clusters'}, - models: {$size: '$domainmodels'}, - caches: {$size: '$caches'}, - igfs: {$size: '$igfs'} - }} - ]).exec(), - mongo.Account.find({}).sort('firstName lastName').lean().exec() - ]) - .then((values) => { - const counters = _.keyBy(values[0], 'owner'); - const accounts = values[1]; - - return accounts.map((account) => { - account.counters = _.omit(counters[account._id], '_id', 'owner'); - - return account; - }); - }) - .then((users) => res.json(users)) - .catch((err) => mongo.handleError(res, err)); - }); - - // Remove user. - router.post('/remove', (req, res) => { - const userId = req.body.userId; - - mongo.Account.findByIdAndRemove(userId).exec() - .then((user) => { - res.sendStatus(200); - - return mongo.spaceIds(userId) - .then((spaceIds) => Promise.all([ - mongo.Cluster.remove({space: {$in: spaceIds}}).exec(), - mongo.Cache.remove({space: {$in: spaceIds}}).exec(), - mongo.DomainModel.remove({space: {$in: spaceIds}}).exec(), - mongo.Igfs.remove({space: {$in: spaceIds}}).exec(), - mongo.Notebook.remove({space: {$in: spaceIds}}).exec(), - mongo.Space.remove({owner: userId}).exec() - ])) - .then(() => user) - .catch((err) => console.error(`Failed to cleanup spaces [user=${user.username}, err=${err}`)); - }) - .then((user) => - mail.send(user, 'Your account was deleted', - `Hello ${user.firstName} ${user.lastName}!

          ` + - `You are receiving this email because your account for ${settings.smtp.username} was removed.`, - 'Account was removed, but failed to send email notification to user!') - ) - .catch((err) => mongo.handleError(res, err)); - }); - - // Save user. - router.post('/save', (req, res) => { - const params = req.body; - - mongo.Account.findByIdAndUpdate(params.userId, {admin: params.adminFlag}).exec() - .then(() => res.sendStatus(200)) - .catch((err) => mongo.handleError(res, err)); - }); - - // Become user. - router.get('/become', (req, res) => { - mongo.Account.findById(req.query.viewedUserId).exec() - .then((viewedUser) => { - req.session.viewedUser = viewedUser; - - res.sendStatus(200); - }) - .catch(() => res.sendStatus(404)); - }); - - // Revert to your identity. - router.get('/revert/identity', (req, res) => { - req.session.viewedUser = null; - - return res.sendStatus(200); - }); - - factoryResolve(router); - }); -}; - diff --git a/modules/web-console/src/main/js/serve/routes/agent.js b/modules/web-console/src/main/js/serve/routes/agent.js deleted file mode 100644 index 48ec131173688..0000000000000 --- a/modules/web-console/src/main/js/serve/routes/agent.js +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -'use strict'; - -// Fire me up! - -module.exports = { - implements: 'agent-routes', - inject: ['require(lodash)', 'require(express)', 'require(fs)', 'require(jszip)', 'settings', 'agent-manager'] -}; - -/** - * @param _ - * @param express - * @param fs - * @param JSZip - * @param settings - * @param {AgentManager} agentMgr - * @returns {Promise} - */ -module.exports.factory = function(_, express, fs, JSZip, settings, agentMgr) { - return new Promise((resolveFactory) => { - const router = new express.Router(); - - /* Get grid topology. */ - router.get('/download/zip', (req, res) => { - const latest = agentMgr.supportedAgents.latest; - - if (_.isEmpty(latest)) - return res.status(500).send('Missing agent zip on server. Please ask webmaster to upload agent zip!'); - - const agentFld = latest.fileName.substr(0, latest.fileName.length - 4); - const agentZip = latest.fileName; - const agentPathZip = latest.filePath; - - // Read a zip file. - fs.readFile(agentPathZip, (errFs, data) => { - if (errFs) - return res.download(agentPathZip, agentZip); - - // Set the archive name. - res.attachment(agentZip); - - JSZip.loadAsync(data) - .then((zip) => { - const prop = []; - - const host = req.hostname.match(/:/g) ? req.hostname.slice(0, req.hostname.indexOf(':')) : req.hostname; - - prop.push('tokens=' + req.user.token); - prop.push('server-uri=' + (settings.agent.SSLOptions ? 'https' : 'http') + '://' + host + ':' + settings.agent.port); - prop.push('#Uncomment following options if needed:'); - prop.push('#node-uri=http://localhost:8080'); - prop.push('#driver-folder=./jdbc-drivers'); - - zip.file(agentFld + '/default.properties', prop.join('\n')); - - zip.generateAsync({type: 'nodebuffer', platform: 'UNIX'}) - .then((buffer) => res.send(buffer)); - }); - }); - }); - - resolveFactory(router); - }); -}; diff --git a/modules/web-console/src/main/js/serve/routes/caches.js b/modules/web-console/src/main/js/serve/routes/caches.js deleted file mode 100644 index ed1f257ccf7a5..0000000000000 --- a/modules/web-console/src/main/js/serve/routes/caches.js +++ /dev/null @@ -1,132 +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. - */ - -'use strict'; - -// Fire me up! - -module.exports = { - implements: 'caches-routes', - inject: ['require(lodash)', 'require(express)', 'mongo'] -}; - -module.exports.factory = function(_, express, mongo) { - return new Promise((factoryResolve) => { - const router = new express.Router(); - - /** - * Get spaces and caches accessed for user account. - * - * @param req Request. - * @param res Response. - */ - router.post('/list', (req, res) => { - const result = {}; - let spaceIds = []; - - // Get owned space and all accessed space. - mongo.spaces(req.currentUserId(), req.header('IgniteDemoMode')) - .then((spaces) => { - result.spaces = spaces; - spaceIds = spaces.map((space) => space._id); - - return mongo.Cluster.find({space: {$in: spaceIds}}).sort('name').lean().exec(); - }) - .then((clusters) => { - result.clusters = clusters; - - return mongo.DomainModel.find({space: {$in: spaceIds}}).sort('name').lean().exec(); - }) - .then((domains) => { - result.domains = domains; - - return mongo.Cache.find({space: {$in: spaceIds}}).sort('name').lean().exec(); - }) - .then((caches) => { - result.caches = caches; - - res.json(result); - }) - .catch((err) => mongo.handleError(res, err)); - }); - - /** - * Save cache. - */ - router.post('/save', (req, res) => { - const params = req.body; - const clusters = params.clusters; - const domains = params.domains; - - mongo.Cache.findOne({space: params.space, name: params.name}).exec() - .then((existingCache) => { - const cacheId = params._id; - - if (existingCache && cacheId !== existingCache._id.toString()) - return res.status(500).send('Cache with name: "' + existingCache.name + '" already exist.'); - - if (cacheId) { - return mongo.Cache.update({_id: cacheId}, params, {upsert: true}).exec() - .then(() => mongo.Cluster.update({_id: {$in: clusters}}, {$addToSet: {caches: cacheId}}, {multi: true}).exec()) - .then(() => mongo.Cluster.update({_id: {$nin: clusters}}, {$pull: {caches: cacheId}}, {multi: true}).exec()) - .then(() => mongo.DomainModel.update({_id: {$in: domains}}, {$addToSet: {caches: cacheId}}, {multi: true}).exec()) - .then(() => mongo.DomainModel.update({_id: {$nin: domains}}, {$pull: {caches: cacheId}}, {multi: true}).exec()) - .then(() => res.send(cacheId)); - } - - return (new mongo.Cache(params)).save() - .then((cache) => - mongo.Cluster.update({_id: {$in: clusters}}, {$addToSet: {caches: cache._id}}, {multi: true}).exec() - .then(() => mongo.DomainModel.update({_id: {$in: domains}}, {$addToSet: {caches: cache._id}}, {multi: true}).exec()) - .then(() => res.send(cache._id)) - ); - }) - .catch((err) => mongo.handleError(res, err)); - }); - - /** - * Remove cache by ._id. - */ - router.post('/remove', (req, res) => { - const params = req.body; - const cacheId = params._id; - - mongo.Cluster.update({caches: {$in: [cacheId]}}, {$pull: {caches: cacheId}}, {multi: true}).exec() - .then(() => mongo.DomainModel.update({caches: {$in: [cacheId]}}, {$pull: {caches: cacheId}}, {multi: true}).exec()) - .then(() => mongo.Cache.remove(params).exec()) - .then(() => res.sendStatus(200)) - .catch((err) => mongo.handleError(res, err)); - }); - - /** - * Remove all caches. - */ - router.post('/remove/all', (req, res) => { - mongo.spaceIds(req.currentUserId(), req.header('IgniteDemoMode')) - .then((spaceIds) => - mongo.Cluster.update({space: {$in: spaceIds}}, {caches: []}, {multi: true}).exec() - .then(() => mongo.DomainModel.update({space: {$in: spaceIds}}, {caches: []}, {multi: true}).exec()) - .then(() => mongo.Cache.remove({space: {$in: spaceIds}}).exec()) - ) - .then(() => res.sendStatus(200)) - .catch((err) => mongo.handleError(res, err)); - }); - - factoryResolve(router); - }); -}; - diff --git a/modules/web-console/src/main/js/serve/routes/clusters.js b/modules/web-console/src/main/js/serve/routes/clusters.js deleted file mode 100644 index 9d13990b03cb4..0000000000000 --- a/modules/web-console/src/main/js/serve/routes/clusters.js +++ /dev/null @@ -1,146 +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. - */ - -'use strict'; - -// Fire me up! - -module.exports = { - implements: 'clusters-routes', - inject: ['require(lodash)', 'require(express)', 'mongo'] -}; - -module.exports.factory = function(_, express, mongo) { - return new Promise((factoryResolve) => { - const router = new express.Router(); - - /** - * Get spaces and clusters accessed for user account. - * - * @param req Request. - * @param res Response. - */ - router.post('/list', (req, res) => { - const result = {}; - let spaceIds = []; - let domains = {}; - - mongo.spaces(req.currentUserId(), req.header('IgniteDemoMode')) - .then((spaces) => { - result.spaces = spaces; - spaceIds = spaces.map((space) => space._id); - - return mongo.DomainModel.find({space: {$in: spaceIds}}).lean().exec(); - }) - .then((_domains) => { - domains = _domains.reduce((map, obj) => { - map[obj._id] = obj; - - return map; - }, {}); - - return mongo.Cache.find({space: {$in: spaceIds}}).sort('name').lean().exec(); - }) - .then((caches) => { - _.forEach(caches, (cache) => { - cache.domains = _.map(cache.domains, (domainId) => domains[domainId]); - }); - - result.caches = caches; - - return mongo.Igfs.find({space: {$in: spaceIds}}).sort('name').lean().exec(); - }) - .then((igfss) => { - result.igfss = igfss; - - return mongo.Cluster.find({space: {$in: spaceIds}}).sort('name').deepPopulate(mongo.ClusterDefaultPopulate).lean().exec(); - }) - .then((clusters) => { - result.clusters = clusters; - - res.json(result); - }) - .catch((err) => mongo.handleError(res, err)); - }); - - /** - * Save cluster. - */ - router.post('/save', (req, res) => { - const params = req.body; - const caches = params.caches; - const igfss = params.igfss; - - mongo.Cluster.findOne({space: params.space, name: params.name}).exec() - .then((existingCluster) => { - const clusterId = params._id; - - if (existingCluster && clusterId !== existingCluster._id.toString()) - throw new Error('Cluster with name: "' + existingCluster.name + '" already exist.'); - - if (clusterId) { - return mongo.Cluster.update({_id: clusterId}, params, {upsert: true}).exec() - .then(() => mongo.Cache.update({_id: {$in: caches}}, {$addToSet: {clusters: clusterId}}, {multi: true}).exec()) - .then(() => mongo.Cache.update({_id: {$nin: caches}}, {$pull: {clusters: clusterId}}, {multi: true}).exec()) - .then(() => mongo.Igfs.update({_id: {$in: igfss}}, {$addToSet: {clusters: clusterId}}, {multi: true}).exec()) - .then(() => mongo.Igfs.update({_id: {$nin: igfss}}, {$pull: {clusters: clusterId}}, {multi: true}).exec()) - .then(() => res.send(clusterId)); - } - - return (new mongo.Cluster(params)).save() - .then((cluster) => - mongo.Cache.update({_id: {$in: caches}}, {$addToSet: {clusters: cluster._id}}, {multi: true}).exec() - .then(() => mongo.Cache.update({_id: {$nin: caches}}, {$pull: {clusters: cluster._id}}, {multi: true}).exec()) - .then(() => mongo.Igfs.update({_id: {$in: igfss}}, {$addToSet: {clusters: cluster._id}}, {multi: true}).exec()) - .then(() => mongo.Igfs.update({_id: {$nin: igfss}}, {$pull: {clusters: cluster._id}}, {multi: true}).exec()) - .then(() => res.send(cluster._id)) - ); - }) - .catch((err) => mongo.handleError(res, err)); - }); - - /** - * Remove cluster by ._id. - */ - router.post('/remove', (req, res) => { - const params = req.body; - const clusterId = params._id; - - mongo.Cache.update({clusters: {$in: [clusterId]}}, {$pull: {clusters: clusterId}}, {multi: true}).exec() - .then(() => mongo.Igfs.update({clusters: {$in: [clusterId]}}, {$pull: {clusters: clusterId}}, {multi: true}).exec()) - .then(() => mongo.Cluster.remove(params).exec()) - .then(() => res.sendStatus(200)) - .catch((err) => mongo.handleError(res, err)); - }); - - /** - * Remove all clusters. - */ - router.post('/remove/all', (req, res) => { - // Get owned space and all accessed space. - mongo.spaceIds(req.currentUserId(), req.header('IgniteDemoMode')) - .then((spaceIds) => mongo.Cache.update({space: {$in: spaceIds}}, {clusters: []}, {multi: true}).exec() - .then(() => mongo.Igfs.update({space: {$in: spaceIds}}, {clusters: []}, {multi: true}).exec()) - .then(() => mongo.Cluster.remove({space: {$in: spaceIds}}).exec()) - ) - .then(() => res.sendStatus(200)) - .catch((err) => mongo.handleError(res, err)); - }); - - factoryResolve(router); - }); -}; diff --git a/modules/web-console/src/main/js/serve/routes/domains.js b/modules/web-console/src/main/js/serve/routes/domains.js deleted file mode 100644 index 9dbf4186e09c1..0000000000000 --- a/modules/web-console/src/main/js/serve/routes/domains.js +++ /dev/null @@ -1,195 +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. - */ - -'use strict'; - -// Fire me up! - -module.exports = { - implements: 'domains-routes', - inject: ['require(lodash)', 'require(express)', 'mongo'] -}; - -module.exports.factory = (_, express, mongo) => { - return new Promise((factoryResolve) => { - const router = new express.Router(); - - /** - * Get spaces and domain models accessed for user account. - * - * @param req Request. - * @param res Response. - */ - router.post('/list', (req, res) => { - const result = {}; - let spacesIds = []; - - mongo.spaces(req.currentUserId(), req.header('IgniteDemoMode')) - .then((spaces) => { - result.spaces = spaces; - spacesIds = spaces.map((space) => space._id); - - return mongo.Cluster.find({space: {$in: spacesIds}}, '_id name').sort('name').lean().exec(); - }) - .then((clusters) => { - result.clusters = clusters; - - return mongo.Cache.find({space: {$in: spacesIds}}).sort('name').lean().exec(); - }) - .then((caches) => { - result.caches = caches; - - return mongo.DomainModel.find({space: {$in: spacesIds}}).sort('valueType').lean().exec(); - }) - .then((domains) => { - result.domains = domains; - - res.json(result); - }) - .catch((err) => mongo.handleError(res, err)); - }); - - function _updateCacheStore(cacheStoreChanges) { - const promises = []; - - _.forEach(cacheStoreChanges, (change) => - promises.push(mongo.Cache.update({_id: {$eq: change.cacheId}}, change.change, {}).exec()) - ); - - return Promise.all(promises); - } - - const _saveDomainModel = (domain, savedDomains) => { - const caches = domain.caches; - const cacheStoreChanges = domain.cacheStoreChanges; - const domainId = domain._id; - - return mongo.DomainModel.findOne({space: domain.space, valueType: domain.valueType}).exec() - .then((_domain) => { - if (_domain && domainId !== _domain._id.toString()) - throw new Error('Domain model with value type: "' + _domain.valueType + '" already exist.'); - - if (domainId) { - return mongo.DomainModel.update({_id: domain._id}, domain, {upsert: true}).exec() - .then(() => mongo.Cache.update({_id: {$in: caches}}, {$addToSet: {domains: domainId}}, {multi: true}).exec()) - .then(() => mongo.Cache.update({_id: {$nin: caches}}, {$pull: {domains: domainId}}, {multi: true}).exec()) - .then(() => { - savedDomains.push(domain); - - return _updateCacheStore(cacheStoreChanges); - }); - } - - return (new mongo.DomainModel(domain)).save() - .then((savedDomain) => { - savedDomains.push(savedDomain); - - return mongo.Cache.update({_id: {$in: caches}}, {$addToSet: {domains: savedDomain._id}}, {multi: true}).exec(); - }) - .then(() => _updateCacheStore(cacheStoreChanges)); - }); - }; - - const _save = (domains, res) => { - if (domains && domains.length > 0) { - const savedDomains = []; - const generatedCaches = []; - const promises = []; - - _.forEach(domains, (domain) => { - if (domain.newCache) { - promises.push( - mongo.Cache.findOne({space: domain.space, name: domain.newCache.name}).exec() - .then((cache) => { - if (cache) - return Promise.resolve(cache); - - // If cache not found, then create it and associate with domain model. - const newCache = domain.newCache; - newCache.space = domain.space; - - return (new mongo.Cache(newCache)).save() - .then((_cache) => { - generatedCaches.push(_cache); - - return mongo.Cluster.update({_id: {$in: _cache.clusters}}, {$addToSet: {caches: _cache._id}}, {multi: true}).exec() - .then(() => Promise.resolve(_cache)); - }); - }) - .then((cache) => { - domain.caches = [cache._id]; - - return _saveDomainModel(domain, savedDomains); - }) - .catch((err) => mongo.handleError(res, err)) - ); - } - else - promises.push(_saveDomainModel(domain, savedDomains)); - }); - - Promise.all(promises) - .then(() => res.send({savedDomains, generatedCaches})) - .catch((err) => mongo.handleError(res, err)); - } - else - res.status(500).send('Nothing to save!'); - }; - - /** - * Save domain model. - */ - router.post('/save', (req, res) => { - _save([req.body], res); - }); - - /** - * Batch save domain models. - */ - router.post('/save/batch', (req, res) => { - _save(req.body, res); - }); - - /** - * Remove domain model by ._id. - */ - router.post('/remove', (req, res) => { - const params = req.body; - const domainId = params._id; - - mongo.DomainModel.findOne(params).exec() - .then((domain) => mongo.Cache.update({_id: {$in: domain.caches}}, {$pull: {domains: domainId}}, {multi: true}).exec()) - .then(() => mongo.DomainModel.remove(params).exec()) - .then(() => res.sendStatus(200)) - .catch((err) => mongo.handleError(res, err)); - }); - - /** - * Remove all domain models. - */ - router.post('/remove/all', (req, res) => { - mongo.spaceIds(req.currentUserId(), req.header('IgniteDemoMode')) - .then((spaceIds) => mongo.Cache.update({space: {$in: spaceIds}}, {domains: []}, {multi: true}).exec() - .then(() => mongo.DomainModel.remove({space: {$in: spaceIds}}).exec())) - .then(() => res.sendStatus(200)) - .catch((err) => mongo.handleError(res, err)); - }); - - factoryResolve(router); - }); -}; - diff --git a/modules/web-console/src/main/js/serve/routes/igfs.js b/modules/web-console/src/main/js/serve/routes/igfs.js deleted file mode 100644 index f59027378fde5..0000000000000 --- a/modules/web-console/src/main/js/serve/routes/igfs.js +++ /dev/null @@ -1,122 +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. - */ - -'use strict'; - -// Fire me up! - -module.exports = { - implements: 'igfs-routes', - inject: ['require(lodash)', 'require(express)', 'mongo'] -}; - -module.exports.factory = function(_, express, mongo) { - return new Promise((factoryResolve) => { - const router = new express.Router(); - - /** - * Get spaces and IGFSs accessed for user account. - * - * @param req Request. - * @param res Response. - */ - router.post('/list', (req, res) => { - const result = {}; - let spaceIds = []; - - // Get owned space and all accessed space. - mongo.spaces(req.currentUserId(), req.header('IgniteDemoMode')) - .then((spaces) => { - result.spaces = spaces; - spaceIds = spaces.map((space) => space._id); - - return mongo.Cluster.find({space: {$in: spaceIds}}, '_id name').sort('name').lean().exec(); - }) - .then((clusters) => { - result.clusters = clusters; - - return mongo.Igfs.find({space: {$in: spaceIds}}).sort('name').lean().exec(); - }) - .then((igfss) => { - result.igfss = igfss; - - res.json(result); - }) - .catch((err) => mongo.handleError(res, err)); - }); - - /** - * Save IGFS. - */ - router.post('/save', (req, res) => { - const params = req.body; - const clusters = params.clusters; - - mongo.Igfs.findOne({space: params.space, name: params.name}).exec() - .then((_igfs) => { - const igfsId = params._id; - - if (_igfs && igfsId !== _igfs._id.toString()) - return res.status(500).send('IGFS with name: "' + params.name + '" already exist.'); - - if (params._id) { - return mongo.Igfs.update({_id: igfsId}, params, {upsert: true}).exec() - .then(() => mongo.Cluster.update({_id: {$in: clusters}}, {$addToSet: {igfss: igfsId}}, {multi: true}).exec()) - .then(() => mongo.Cluster.update({_id: {$nin: clusters}}, {$pull: {igfss: igfsId}}, {multi: true}).exec()) - .then(() => res.send(igfsId)); - } - - return (new mongo.Igfs(params)).save() - .then((igfs) => - mongo.Cluster.update({_id: {$in: clusters}}, {$addToSet: {igfss: igfs._id}}, {multi: true}).exec() - .then(() => res.send(igfs._id)) - ); - }) - .catch((err) => mongo.handleError(res, err)); - }); - - /** - * Remove IGFS by ._id. - */ - router.post('/remove', (req, res) => { - const params = req.body; - const igfsId = params._id; - - mongo.Cluster.update({igfss: {$in: [igfsId]}}, {$pull: {igfss: igfsId}}, {multi: true}).exec() - .then(() => mongo.Igfs.remove(params).exec()) - .then(() => res.sendStatus(200)) - .catch((err) => mongo.handleError(res, err)); - }); - - /** - * Remove all IGFSs. - */ - router.post('/remove/all', (req, res) => { - // Get owned space and all accessed space. - mongo.spaceIds(req.currentUserId(), req.header('IgniteDemoMode')) - .then((spaceIds) => - mongo.Cluster.update({space: {$in: spaceIds}}, {igfss: []}, {multi: true}).exec() - .then(() => mongo.Igfs.remove({space: {$in: spaceIds}}).exec()) - ) - .then(() => res.sendStatus(200)) - .catch((err) => mongo.handleError(res, err)); - }); - - factoryResolve(router); - }); -}; - diff --git a/modules/web-console/src/main/js/serve/routes/notebooks.js b/modules/web-console/src/main/js/serve/routes/notebooks.js deleted file mode 100644 index 37665bfa09201..0000000000000 --- a/modules/web-console/src/main/js/serve/routes/notebooks.js +++ /dev/null @@ -1,121 +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. - */ - -'use strict'; - -// Fire me up! - -module.exports = { - implements: 'notebooks-routes', - inject: ['require(express)', 'mongo'] -}; - -module.exports.factory = function(express, mongo) { - return new Promise((factoryResolve) => { - const router = new express.Router(); - - /** - * Get notebooks names accessed for user account. - * - * @param req Request. - * @param res Response. - */ - router.post('/list', (req, res) => { - mongo.spaces(req.currentUserId()) - .then((spaces) => mongo.Notebook.find({space: {$in: spaces.map((value) => value._id)}}).select('_id name').sort('name').lean().exec()) - .then((notebooks) => res.json(notebooks)) - .catch((err) => mongo.handleError(res, err)); - - }); - - /** - * Get notebook accessed for user account. - * - * @param req Request. - * @param res Response. - */ - router.post('/get', (req, res) => { - mongo.spaces(req.currentUserId()) - .then((spaces) => mongo.Notebook.findOne({space: {$in: spaces.map((value) => value._id)}, _id: req.body.noteId}).lean().exec()) - .then((notebook) => res.json(notebook)) - .catch((err) => mongo.handleError(res, err)); - }); - - /** - * Save notebook accessed for user account. - * - * @param req Request. - * @param res Response. - */ - router.post('/save', (req, res) => { - const note = req.body; - - mongo.Notebook.findOne({space: note.space, name: note.name}).exec() - .then((notebook) => { - const noteId = note._id; - - if (notebook && noteId !== notebook._id.toString()) - throw new Error('Notebook with name: "' + notebook.name + '" already exist.'); - - if (noteId) { - return mongo.Notebook.update({_id: noteId}, note, {upsert: true}).exec() - .then(() => res.send(noteId)) - .catch((err) => mongo.handleError(res, err)); - } - - return (new mongo.Notebook(req.body)).save(); - }) - .then((notebook) => res.send(notebook._id)) - .catch((err) => mongo.handleError(res, err)); - }); - - /** - * Remove notebook by ._id. - * - * @param req Request. - * @param res Response. - */ - router.post('/remove', (req, res) => { - mongo.Notebook.remove(req.body).exec() - .then(() => res.sendStatus(200)) - .catch((err) => mongo.handleError(res, err)); - }); - - /** - * Create new notebook for user account. - * - * @param req Request. - * @param res Response. - */ - router.post('/new', (req, res) => { - mongo.spaceIds(req.currentUserId()) - .then((spaceIds) => - mongo.Notebook.findOne({space: spaceIds[0], name: req.body.name}) - .then((notebook) => { - if (notebook) - throw new Error('Notebook with name: "' + notebook.name + '" already exist.'); - - return spaceIds; - })) - .then((spaceIds) => (new mongo.Notebook({space: spaceIds[0], name: req.body.name})).save()) - .then((notebook) => res.send(notebook._id)) - .catch((err) => mongo.handleError(res, err)); - }); - - factoryResolve(router); - }); -}; diff --git a/modules/web-console/src/main/js/serve/routes/profile.js b/modules/web-console/src/main/js/serve/routes/profile.js deleted file mode 100644 index 5563a2bcd1113..0000000000000 --- a/modules/web-console/src/main/js/serve/routes/profile.js +++ /dev/null @@ -1,102 +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. - */ - -'use strict'; - -// Fire me up! - -module.exports = { - implements: 'profile-routes', - inject: ['require(lodash)', 'require(express)', 'mongo', 'agent-manager'] -}; - -/** - * - * @param _ Lodash module - * @param express Express module - * @param mongo - * @param {AgentManager} agentMgr - * @returns {Promise} - */ -module.exports.factory = function(_, express, mongo, agentMgr) { - return new Promise((resolveFactory) => { - const router = new express.Router(); - - /** - * Save user profile. - */ - router.post('/save', (req, res) => { - const params = req.body; - - if (params.password && _.isEmpty(params.password)) - return res.status(500).send('Wrong value for new password!'); - - mongo.Account.findById(params._id).exec() - .then((user) => { - if (!params.password) - return Promise.resolve(user); - - return new Promise((resolve, reject) => { - user.setPassword(params.password, (err, _user) => { - if (err) - return reject(err); - - delete params.password; - - resolve(_user); - }); - }); - }) - .then((user) => { - if (!params.email || user.email === params.email) - return Promise.resolve(user); - - return new Promise((resolve, reject) => { - mongo.Account.findOne({email: params.email}, (err, _user) => { - // TODO send error to admin - if (err) - reject(new Error('Failed to check email!')); - - if (_user && _user._id !== user._id) - reject(new Error('User with this email already registered!')); - - resolve(user); - }); - }); - }) - .then((user) => { - if (params.token && user.token !== params.token) - agentMgr.close(user._id, user.token); - - _.extend(user, params); - - return user.save(); - }) - .then((user) => { - const becomeUsed = req.session.viewedUser && req.user.admin; - - if (becomeUsed) - req.session.viewedUser = user; - - res.sendStatus(200); - }) - .catch((err) => mongo.handleError(res, err)); - }); - - resolveFactory(router); - }); -}; diff --git a/modules/web-console/src/main/js/serve/routes/routes.js b/modules/web-console/src/main/js/serve/routes/routes.js deleted file mode 100644 index cbee8bb314626..0000000000000 --- a/modules/web-console/src/main/js/serve/routes/routes.js +++ /dev/null @@ -1,103 +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. - */ - -'use strict'; - -// Fire me up! - -module.exports = { - implements: 'routes', - inject: [ - 'public-routes', - 'admin-routes', - 'profile-routes', - 'demo-routes', - 'clusters-routes', - 'domains-routes', - 'caches-routes', - 'igfs-routes', - 'notebooks-routes', - 'agent-routes', - 'ignite_modules/routes:*' // Loads all routes modules of all plugins - ] -}; - -module.exports.factory = function(publicRoutes, adminRoutes, profileRoutes, demoRoutes, - clusterRoutes, domainRoutes, cacheRoutes, igfsRoutes, notebookRoutes, agentRoutes, pluginRoutes) { - return { - register: (app) => { - app.all('*', (req, res, next) => { - req.currentUserId = () => { - if (!req.user) - return null; - - if (req.session.viewedUser && req.user.admin) - return req.session.viewedUser._id; - - return req.user._id; - }; - - next(); - }); - - const _mustAuthenticated = (req, res, next) => req.isAuthenticated() ? next() : res.redirect('/'); - - const _adminOnly = (req, res, next) => req.isAuthenticated() && req.user.admin ? next() : res.sendStatus(403); - - // Registering the standard routes - app.use('/', publicRoutes); - app.use('/admin', _mustAuthenticated, _adminOnly, adminRoutes); - app.use('/profile', _mustAuthenticated, profileRoutes); - app.use('/demo', _mustAuthenticated, demoRoutes); - - app.all('/configuration/*', _mustAuthenticated); - - app.use('/configuration/clusters', clusterRoutes); - app.use('/configuration/domains', domainRoutes); - app.use('/configuration/caches', cacheRoutes); - app.use('/configuration/igfs', igfsRoutes); - - app.use('/notebooks', _mustAuthenticated, notebookRoutes); - app.use('/agent', _mustAuthenticated, agentRoutes); - - // Registering the routes of all plugin modules - for (const name in pluginRoutes) { - if (pluginRoutes.hasOwnProperty(name)) - pluginRoutes[name].register(app, _mustAuthenticated, _adminOnly); - } - - // Catch 404 and forward to error handler. - app.use((req, res, next) => { - const err = new Error('Not Found: ' + req.originalUrl); - - err.status = 404; - - next(err); - }); - - // Production error handler: no stacktraces leaked to user. - app.use((err, req, res) => { - res.status(err.status || 500); - - res.render('error', { - message: err.message, - error: {} - }); - }); - } - }; -}; diff --git a/modules/web-console/src/main/js/views/configuration/clusters.jade b/modules/web-console/src/main/js/views/configuration/clusters.jade deleted file mode 100644 index 6450163675539..0000000000000 --- a/modules/web-console/src/main/js/views/configuration/clusters.jade +++ /dev/null @@ -1,64 +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. - -include ../../app/helpers/jade/mixins.jade - -.docs-header - h1 Configure Ignite Clusters -.docs-body(ng-controller='clustersController') - ignite-information - ul - li Configure #[a(href='https://apacheignite.readme.io/docs/clustering' target='_blank') clusters] properties - li Associate clusters with caches and in-memory file systems - div(ignite-loading='loadingClustersScreen' ignite-loading-text='Loading clusters...' ignite-loading-position='top') - div(ng-show='ui.ready') - hr - +main-table('clusters', 'clusters', 'clusterName', 'selectItem(row)', '{{$index + 1}}) {{row.label}}', 'label') - .padding-top-dflt(bs-affix) - .panel-tip-container(data-placement='bottom' bs-tooltip='' data-title='Create new cluster') - button.btn.btn-primary(id='new-item' ng-click='createItem()') Add cluster - +save-remove-clone-undo-buttons('cluster') - hr - .bs-affix-fix - div(bs-collapse='' data-allow-multiple='true' ng-model='ui.activePanels') - form.form-horizontal(name='ui.inputForm' ng-show='contentVisible()' novalidate) - .panel-group - ignite-configuration-clusters-general - - +advanced-options-toggle-default - - div(ng-show='ui.expanded') - ignite-configuration-clusters-atomic - ignite-configuration-clusters-binary - ignite-configuration-clusters-collision - ignite-configuration-clusters-communication - ignite-configuration-clusters-connector - ignite-configuration-clusters-deployment - ignite-configuration-clusters-discovery - ignite-configuration-clusters-events - ignite-configuration-clusters-failover - ignite-configuration-clusters-igfs - ignite-configuration-clusters-logger - ignite-configuration-clusters-marshaller - ignite-configuration-clusters-metrics - ignite-configuration-clusters-ssl - ignite-configuration-clusters-swap - ignite-configuration-clusters-thread - ignite-configuration-clusters-time - ignite-configuration-clusters-transactions - ignite-configuration-user-attributes - - +advanced-options-toggle-default diff --git a/modules/web-console/src/test/js/routes/agent.js b/modules/web-console/src/test/js/routes/agent.js deleted file mode 100644 index 1c4aff5d4ac1b..0000000000000 --- a/modules/web-console/src/test/js/routes/agent.js +++ /dev/null @@ -1,94 +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. - */ - -var request = require('supertest'), - should = require('should'), - app = require('../../app'), - fs = require('fs'), - https = require('https'), - config = require('../../helpers/configuration-loader.js'), - agentManager = require('../../agents/agent-manager'); - -/** - * Create HTTP server. - */ -/** - * Start agent server. - */ -var agentServer = https.createServer({ - key: fs.readFileSync(config.get('monitor:server:key')), - cert: fs.readFileSync(config.get('monitor:server:cert')), - passphrase: config.get('monitor:server:keyPassphrase') -}); - -agentServer.listen(config.get('monitor:server:port')); - -agentManager.createManager(agentServer); - -describe('request from agent', function() { - var agent = request.agent(app); - - before(function (done) { - this.timeout(10000); - - agent - .post('/login') - .send({email: 'test@test.com', password: 'test'}) - .expect(302) - .end(function (err) { - if (error) - throw error; - - setTimeout(done, 5000); - }); - }); - - it('should return topology snapshot', function(done){ - agent - .post('/agent/topology') - .send({}) - .end(function(err, nodes) { - if (error) { - console.log(error.response.text); - - throw error; - } - - console.log(nodes); - - done(); - }); - }); - - //it('should query result', function(done){ - // agent - // .post('/agent/query') - // .send({ - // username: 'nva', - // password: 'nva.141', - // host: 'localhost', - // port: '5432', - // dbName: 'ggmonitor' - // }) - // .end(function(err, res) { - // if (err) - // throw err; - // - // done(); - // }); - //}); -}); diff --git a/modules/web-agent/.gitignore b/modules/web-console/web-agent/.gitignore similarity index 100% rename from modules/web-agent/.gitignore rename to modules/web-console/web-agent/.gitignore diff --git a/modules/web-agent/README.txt b/modules/web-console/web-agent/README.txt similarity index 100% rename from modules/web-agent/README.txt rename to modules/web-console/web-agent/README.txt diff --git a/modules/web-agent/assembly/release-web-agent.xml b/modules/web-console/web-agent/assembly/release-web-agent.xml similarity index 91% rename from modules/web-agent/assembly/release-web-agent.xml rename to modules/web-console/web-agent/assembly/release-web-agent.xml index aa85b5964a84b..bb994c005af01 100644 --- a/modules/web-agent/assembly/release-web-agent.xml +++ b/modules/web-console/web-agent/assembly/release-web-agent.xml @@ -34,6 +34,7 @@ jdbc-drivers/README* demo/README* demo/*.sql + logs/README* README* LICENSE* NOTICE* @@ -54,13 +55,6 @@ **/*.sh - - ${basedir}/logs - /logs - - README* - - ${project.build.directory} / diff --git a/modules/web-agent/bin/ignite-web-agent.bat b/modules/web-console/web-agent/bin/ignite-web-agent.bat similarity index 100% rename from modules/web-agent/bin/ignite-web-agent.bat rename to modules/web-console/web-agent/bin/ignite-web-agent.bat diff --git a/modules/web-agent/bin/ignite-web-agent.sh b/modules/web-console/web-agent/bin/ignite-web-agent.sh similarity index 100% rename from modules/web-agent/bin/ignite-web-agent.sh rename to modules/web-console/web-agent/bin/ignite-web-agent.sh diff --git a/modules/web-agent/demo/README.txt b/modules/web-console/web-agent/demo/README.txt similarity index 100% rename from modules/web-agent/demo/README.txt rename to modules/web-console/web-agent/demo/README.txt diff --git a/modules/web-agent/demo/db-init.sql b/modules/web-console/web-agent/demo/db-init.sql similarity index 100% rename from modules/web-agent/demo/db-init.sql rename to modules/web-console/web-agent/demo/db-init.sql diff --git a/modules/web-agent/jdbc-drivers/README.txt b/modules/web-console/web-agent/jdbc-drivers/README.txt similarity index 100% rename from modules/web-agent/jdbc-drivers/README.txt rename to modules/web-console/web-agent/jdbc-drivers/README.txt diff --git a/modules/web-agent/logs/README.txt b/modules/web-console/web-agent/logs/README.txt similarity index 100% rename from modules/web-agent/logs/README.txt rename to modules/web-console/web-agent/logs/README.txt diff --git a/modules/web-agent/pom.xml b/modules/web-console/web-agent/pom.xml similarity index 94% rename from modules/web-agent/pom.xml rename to modules/web-console/web-agent/pom.xml index d87084f6cdda8..530a27289e8aa 100644 --- a/modules/web-agent/pom.xml +++ b/modules/web-console/web-agent/pom.xml @@ -27,11 +27,13 @@ org.apache.ignite ignite-parent 1 - ../../parent + ../../../parent ignite-web-agent + jar 1.7.0-SNAPSHOT + http://ignite.apache.org yyMMddHHmmss @@ -184,6 +186,14 @@ + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + diff --git a/modules/web-agent/src/main/java/org/apache/ignite/console/agent/AgentConfiguration.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentConfiguration.java similarity index 100% rename from modules/web-agent/src/main/java/org/apache/ignite/console/agent/AgentConfiguration.java rename to modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentConfiguration.java diff --git a/modules/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java similarity index 100% rename from modules/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java rename to modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java diff --git a/modules/web-agent/src/main/java/org/apache/ignite/console/agent/AgentUtils.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentUtils.java similarity index 100% rename from modules/web-agent/src/main/java/org/apache/ignite/console/agent/AgentUtils.java rename to modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentUtils.java diff --git a/modules/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/AbstractHandler.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/AbstractHandler.java similarity index 100% rename from modules/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/AbstractHandler.java rename to modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/AbstractHandler.java diff --git a/modules/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/DatabaseHandler.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/DatabaseHandler.java similarity index 100% rename from modules/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/DatabaseHandler.java rename to modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/DatabaseHandler.java diff --git a/modules/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/RestHandler.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/RestHandler.java similarity index 100% rename from modules/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/RestHandler.java rename to modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/handlers/RestHandler.java diff --git a/modules/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java similarity index 99% rename from modules/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java rename to modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java index bf0903aea92da..09189b5992398 100644 --- a/modules/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java +++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java @@ -54,6 +54,7 @@ import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi; import org.apache.ignite.transactions.Transaction; import org.apache.log4j.Logger; @@ -363,6 +364,8 @@ private static IgniteConfiguration igniteConfiguration(int gridIdx, boolean cli cfg.setCacheConfiguration(cacheCountry(), cacheDepartment(), cacheEmployee(), cacheParking(), cacheCar()); + cfg.setSwapSpaceSpi(new FileSwapSpaceSpi()); + return cfg; } diff --git a/modules/web-agent/src/main/java/org/apache/ignite/console/demo/AgentMetadataDemo.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentMetadataDemo.java similarity index 100% rename from modules/web-agent/src/main/java/org/apache/ignite/console/demo/AgentMetadataDemo.java rename to modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentMetadataDemo.java diff --git a/modules/web-agent/src/main/java/org/apache/ignite/console/demo/model/Car.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/model/Car.java similarity index 100% rename from modules/web-agent/src/main/java/org/apache/ignite/console/demo/model/Car.java rename to modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/model/Car.java diff --git a/modules/web-agent/src/main/java/org/apache/ignite/console/demo/model/Country.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/model/Country.java similarity index 100% rename from modules/web-agent/src/main/java/org/apache/ignite/console/demo/model/Country.java rename to modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/model/Country.java diff --git a/modules/web-agent/src/main/java/org/apache/ignite/console/demo/model/Department.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/model/Department.java similarity index 100% rename from modules/web-agent/src/main/java/org/apache/ignite/console/demo/model/Department.java rename to modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/model/Department.java diff --git a/modules/web-agent/src/main/java/org/apache/ignite/console/demo/model/Employee.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/model/Employee.java similarity index 100% rename from modules/web-agent/src/main/java/org/apache/ignite/console/demo/model/Employee.java rename to modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/model/Employee.java diff --git a/modules/web-agent/src/main/java/org/apache/ignite/console/demo/model/Parking.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/model/Parking.java similarity index 100% rename from modules/web-agent/src/main/java/org/apache/ignite/console/demo/model/Parking.java rename to modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/model/Parking.java diff --git a/modules/web-agent/src/main/resources/log4j.properties b/modules/web-console/web-agent/src/main/resources/log4j.properties similarity index 100% rename from modules/web-agent/src/main/resources/log4j.properties rename to modules/web-console/web-agent/src/main/resources/log4j.properties diff --git a/pom.xml b/pom.xml index 2c7bad19c745c..070b5c8034e9a 100644 --- a/pom.xml +++ b/pom.xml @@ -798,8 +798,8 @@ web-console - modules/web-agent modules/web-console + modules/web-console/web-agent modules/schema-import-db From 492ab120b71a2bd044bb3631560f689c80d8bf95 Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Thu, 8 Sep 2016 15:16:17 +0700 Subject: [PATCH 113/487] Web Console beta-3. Added execution bit for docker build files. --- modules/web-console/docker/compose/backend/build.sh | 0 modules/web-console/docker/compose/frontend/build.sh | 0 modules/web-console/docker/standalone/build.sh | 0 modules/web-console/docker/standalone/entrypoint.sh | 0 4 files changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 modules/web-console/docker/compose/backend/build.sh mode change 100644 => 100755 modules/web-console/docker/compose/frontend/build.sh mode change 100644 => 100755 modules/web-console/docker/standalone/build.sh mode change 100644 => 100755 modules/web-console/docker/standalone/entrypoint.sh diff --git a/modules/web-console/docker/compose/backend/build.sh b/modules/web-console/docker/compose/backend/build.sh old mode 100644 new mode 100755 diff --git a/modules/web-console/docker/compose/frontend/build.sh b/modules/web-console/docker/compose/frontend/build.sh old mode 100644 new mode 100755 diff --git a/modules/web-console/docker/standalone/build.sh b/modules/web-console/docker/standalone/build.sh old mode 100644 new mode 100755 diff --git a/modules/web-console/docker/standalone/entrypoint.sh b/modules/web-console/docker/standalone/entrypoint.sh old mode 100644 new mode 100755 From 4e1a8ffe27fc7ce5293c756df28d7ca5b3f90167 Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Thu, 8 Sep 2016 15:30:55 +0700 Subject: [PATCH 114/487] Web Console beta-3. Updated Ignite version. --- .../frontend/app/modules/version/Version.provider.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/web-console/frontend/app/modules/version/Version.provider.js b/modules/web-console/frontend/app/modules/version/Version.provider.js index fe503abc11610..31ff8d066bdb5 100644 --- a/modules/web-console/frontend/app/modules/version/Version.provider.js +++ b/modules/web-console/frontend/app/modules/version/Version.provider.js @@ -21,7 +21,7 @@ angular .module('ignite-console.version', []) .provider('IgniteVersion', function() { const version = { - version: '1.6.0' + version: '1.7.0' }; this.update = (newVersion) => { From 769e0be12b61497d456a113307c8cff86128ba50 Mon Sep 17 00:00:00 2001 From: Maxim Afanasiev Date: Thu, 8 Sep 2016 16:22:45 +0700 Subject: [PATCH 115/487] Fixed containers version in docker compose file. --- modules/web-console/docker/compose/docker-compose.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/web-console/docker/compose/docker-compose.yml b/modules/web-console/docker/compose/docker-compose.yml index bacd769b87f10..4f83ba05fcc2d 100644 --- a/modules/web-console/docker/compose/docker-compose.yml +++ b/modules/web-console/docker/compose/docker-compose.yml @@ -22,7 +22,7 @@ mongodb: - ./data/mongo:/data/db backend: - image: ignite/web-console-backend:1.7.0 + image: ignite/web-console-backend links: # Link mongodb container as with mongodb hostname. - mongodb:mongodb @@ -49,7 +49,7 @@ backend: - mail_auth_pass="" frontend: - image: ignite/web-console-frontend:1.7.0 + image: ignite/web-console-frontend links: # Link backend container to proxy backend requests throught nginx container. - backend:backend From 2a117fec92d3455e53933cd3992ac73d2b2c0d3a Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Thu, 8 Sep 2016 16:53:11 +0700 Subject: [PATCH 116/487] Web Console beta-3. Fixed import from database. --- .../frontend/controllers/domains-controller.js | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/modules/web-console/frontend/controllers/domains-controller.js b/modules/web-console/frontend/controllers/domains-controller.js index e520494bbcf15..6cb3540c82391 100644 --- a/modules/web-console/frontend/controllers/domains-controller.js +++ b/modules/web-console/frontend/controllers/domains-controller.js @@ -385,8 +385,10 @@ export default ['domainsController', [ $scope.showImportDomainModal = function() { LegacyTable.tableReset(); - FormUtils.confirmUnsavedChanges($scope.ui.inputForm.$dirty, function() { - if ($scope.ui.inputForm.$dirty) + const dirty = $scope.ui.inputForm && $scope.ui.inputForm.$dirty; + + FormUtils.confirmUnsavedChanges(dirty, function() { + if (dirty) $scope.backupItem = $scope.selectedItem ? angular.copy($scope.selectedItem) : prepareNewItem(); const demo = $root.IgniteDemoMode; @@ -441,7 +443,7 @@ export default ['domainsController', [ $scope.ui.selectedJdbcDriverJar = $scope.jdbcDriverJars[0].value; - FormUtils.confirmUnsavedChanges($scope.ui.inputForm.$dirty, () => { + FormUtils.confirmUnsavedChanges(dirty, () => { $scope.importDomain.action = 'connect'; $scope.importDomain.tables = []; From c85b7021270730e72369dd97aacc3c3dd017851d Mon Sep 17 00:00:00 2001 From: iveselovskiy Date: Thu, 8 Sep 2016 15:10:11 +0300 Subject: [PATCH 117/487] IGNITE-3775: IGFS: Uncommented deadlock tests. This closes #1044. --- .../GridCacheAbstractFullApiSelfTest.java | 2 +- .../processors/igfs/IgfsAbstractSelfTest.java | 31 +++++++++---------- 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java index 3f4d812efaaed..a31c82ef72bca 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java @@ -296,7 +296,7 @@ protected CacheMemoryMode memoryMode() { for (int i = 0; i < gridCount(); i++) { Boolean clientMode = grid(i).configuration().isClientMode(); - if (clientMode) + if (clientMode != null && clientMode) // Can be null in multi jvm tests. continue; grid(0).services(grid(0).cluster()).deployNodeSingleton(SERVICE_NAME1, new DummyServiceImpl()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java index c9b08d9c83bab..89979adc5f63b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java @@ -2290,22 +2290,21 @@ public void testDeadlocks() throws Exception { private void checkDeadlocksRepeat(final int lvlCnt, final int childrenDirPerLvl, final int childrenFilePerLvl, int primaryLvlCnt, int renCnt, int delCnt, int updateCnt, int mkdirsCnt, int createCnt) throws Exception { - // TODO: Enable. -// if (relaxedConsistency()) -// return; -// -// for (int i = 0; i < REPEAT_CNT; i++) { -// try { -// checkDeadlocks(lvlCnt, childrenDirPerLvl, childrenFilePerLvl, primaryLvlCnt, renCnt, delCnt, -// updateCnt, mkdirsCnt, createCnt); -// -// if (i % 10 == 0) -// X.println(" - " + i); -// } -// finally { -// clear(igfs, igfsSecondary); -// } -// } + if (relaxedConsistency()) + return; + + for (int i = 0; i < REPEAT_CNT; i++) { + try { + checkDeadlocks(lvlCnt, childrenDirPerLvl, childrenFilePerLvl, primaryLvlCnt, renCnt, delCnt, + updateCnt, mkdirsCnt, createCnt); + + if (i % 10 == 0) + X.println(" - " + i); + } + finally { + clear(igfs, igfsSecondary); + } + } } /** From b3bdca7ad9b394fba0f263990bd762ba04aef80b Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Thu, 8 Sep 2016 15:13:31 +0300 Subject: [PATCH 118/487] IGNITE-3855: IGFS: Support direct PROXY mode invocation in method: delete. This closes #1046. --- .../org/apache/ignite/internal/processors/igfs/IgfsImpl.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 12d2830dd7f96..fd32745234279 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -754,7 +754,10 @@ else if (val) IgfsMode mode = resolveMode(path); - boolean dual = IgfsUtils.isDualMode(mode);; + if (mode == PROXY) + return secondaryFs.delete(path, recursive); + + boolean dual = IgfsUtils.isDualMode(mode); if (dual) await(path); From cc595021491a79d6ccf343a156d6e21389b68ff8 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Thu, 8 Sep 2016 15:15:22 +0300 Subject: [PATCH 119/487] IGNITE-3856: IGFS: Support direct PROXY mode invocation in method: mkdirs. This closes #1047. --- .../internal/processors/igfs/IgfsImpl.java | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index fd32745234279..a6d5b777765e9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -798,14 +798,24 @@ else if (val) IgfsMode mode = resolveMode(path); - if (mode == PRIMARY) - meta.mkdirs(path, props0); - else { - assert IgfsUtils.isDualMode(mode);; + switch (mode) { + case PRIMARY: + meta.mkdirs(path, props0); - await(path); + break; + + case DUAL_ASYNC: + case DUAL_SYNC: + await(path); + + meta.mkdirsDual(secondaryFs, path, props0); + + break; - meta.mkdirsDual(secondaryFs, path, props0); + case PROXY: + secondaryFs.mkdirs(path, props0); + + break; } return null; From 1cc502d64ca058d97b071f7db0ee18f3aac3dfd7 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Thu, 8 Sep 2016 15:17:26 +0300 Subject: [PATCH 120/487] IGNITE-3857: IGFS: Support direct PROXY mode invocation in methods: listPaths / listFiles. This closes #1048. --- .../apache/ignite/internal/processors/igfs/IgfsImpl.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index a6d5b777765e9..636b4a97b502e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -840,7 +840,7 @@ else if (val) Collection files = new HashSet<>(); - if (IgfsUtils.isDualMode(mode)) { + if (mode != PRIMARY) { assert secondaryFs != null; try { @@ -848,7 +848,7 @@ else if (val) files.addAll(children); - if (!modeRslvr.hasPrimaryChild(path)) + if (mode == PROXY || !modeRslvr.hasPrimaryChild(path)) return files; } catch (Exception e) { @@ -889,7 +889,7 @@ else if (mode == PRIMARY) Collection files = new HashSet<>(); - if (IgfsUtils.isDualMode(mode)) { + if (mode != PRIMARY) { assert secondaryFs != null; try { @@ -901,7 +901,7 @@ else if (mode == PRIMARY) files.add(impl); } - if (!modeRslvr.hasPrimaryChild(path)) + if (mode == PROXY || !modeRslvr.hasPrimaryChild(path)) return files; } catch (Exception e) { From 79745ee6d549b0eca8f29608ce0c4d0c2efcfd6c Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Fri, 9 Sep 2016 13:21:26 +0700 Subject: [PATCH 121/487] Web Console beta-3. Fixed typo in docker. --- .../web-console/docker/compose/docker-compose.yml | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/modules/web-console/docker/compose/docker-compose.yml b/modules/web-console/docker/compose/docker-compose.yml index 4f83ba05fcc2d..a2c2f8bd9c691 100644 --- a/modules/web-console/docker/compose/docker-compose.yml +++ b/modules/web-console/docker/compose/docker-compose.yml @@ -35,18 +35,18 @@ backend: # Port for serving frontend API - server_port=3000 # Cookie session secret - - server_sessionSecret="CHANGE ME" + - server_sessionSecret=CHANGE ME # URL for mongodb connection - mongodb_url=mongodb://mongodb/console # Port for web-agent. - agentServer_port=3001 # Mail connection settings. Leave empty if no needed. See also settings, https://github.com/nodemailer/nodemailer - - mail_service="" - - mail_sign="" - - mail_greeting="" - - mail_from="" - - mail_auth_user="" - - mail_auth_pass="" + - mail_service= + - mail_sign= + - mail_greeting= + - mail_from= + - mail_auth_user= + - mail_auth_pass= frontend: image: ignite/web-console-frontend From b595991fbc8eeef4bf7f8bf1e4e096c6c1f88488 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Fri, 9 Sep 2016 12:13:26 +0300 Subject: [PATCH 122/487] IGNITE-3868: ODBC: Fix for driver installation. This closes #1051. --- .../GridCacheAbstractFullApiSelfTest.java | 2 +- .../cpp/odbc/install/ignite-odbc-install.ini | 1 + .../cpp/odbc/install/install_amd64.cmd | 22 ++++++++++--------- .../cpp/odbc/install/install_x86.cmd | 11 +++++----- 4 files changed, 20 insertions(+), 16 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java index 3f4d812efaaed..a31c82ef72bca 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java @@ -296,7 +296,7 @@ protected CacheMemoryMode memoryMode() { for (int i = 0; i < gridCount(); i++) { Boolean clientMode = grid(i).configuration().isClientMode(); - if (clientMode) + if (clientMode != null && clientMode) // Can be null in multi jvm tests. continue; grid(0).services(grid(0).cluster()).deployNodeSingleton(SERVICE_NAME1, new DummyServiceImpl()); diff --git a/modules/platforms/cpp/odbc/install/ignite-odbc-install.ini b/modules/platforms/cpp/odbc/install/ignite-odbc-install.ini index 2de265a2e73e3..f512fb4869100 100644 --- a/modules/platforms/cpp/odbc/install/ignite-odbc-install.ini +++ b/modules/platforms/cpp/odbc/install/ignite-odbc-install.ini @@ -1,5 +1,6 @@ [Apache Ignite] Description=Apache Ignite Driver=/usr/local/lib/libignite-odbc.so +Setup=/usr/local/lib/libignite-odbc.so DriverODBCVer=03.00 FileUsage=0 \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/install/install_amd64.cmd b/modules/platforms/cpp/odbc/install/install_amd64.cmd index 8a0391fcf11bc..3b4b63f9b32c8 100644 --- a/modules/platforms/cpp/odbc/install/install_amd64.cmd +++ b/modules/platforms/cpp/odbc/install/install_amd64.cmd @@ -7,17 +7,18 @@ if [%ODBC_AMD64%] == [] ( echo error: 64-bit driver is not specified. Call format: install_amd64 abs_path_to_64_bit_driver [abs_path_to_32_bit_driver] pause exit /b 1 -) else if exist "%ODBC_AMD64%" ( - if exist "%ODBC_AMD64%\" ( +) else if exist %ODBC_AMD64% ( + if exist %ODBC_AMD64%\ ( echo warning: The path you have specified seems to be a directory. Note that you have to specify path to driver file itself instead. ) - echo Installing 64-bit driver: "%ODBC_AMD64%" + echo Installing 64-bit driver: %ODBC_AMD64% reg add "HKEY_LOCAL_MACHINE\SOFTWARE\ODBC\ODBCINST.INI\Apache Ignite" /v DriverODBCVer /t REG_SZ /d "03.80" /f reg add "HKEY_LOCAL_MACHINE\SOFTWARE\ODBC\ODBCINST.INI\Apache Ignite" /v UsageCount /t REG_DWORD /d 00000001 /f - reg add "HKEY_LOCAL_MACHINE\SOFTWARE\ODBC\ODBCINST.INI\Apache Ignite" /v Driver /t REG_SZ /d "%ODBC_AMD64%" /f + reg add "HKEY_LOCAL_MACHINE\SOFTWARE\ODBC\ODBCINST.INI\Apache Ignite" /v Driver /t REG_SZ /d %ODBC_AMD64% /f + reg add "HKEY_LOCAL_MACHINE\SOFTWARE\ODBC\ODBCINST.INI\Apache Ignite" /v Setup /t REG_SZ /d %ODBC_AMD64% /f reg add "HKEY_LOCAL_MACHINE\SOFTWARE\ODBC\ODBCINST.INI\ODBC Drivers" /v "Apache Ignite" /t REG_SZ /d "Installed" /f ) else ( - echo 64-bit driver can not be found: "%ODBC_AMD64%" + echo 64-bit driver can not be found: %ODBC_AMD64% echo Call format: install_amd64 abs_path_to_64_bit_driver [abs_path_to_32_bit_driver] pause exit /b 1 @@ -25,16 +26,17 @@ if [%ODBC_AMD64%] == [] ( if [%ODBC_X86%] == [] ( echo warning: 32-bit driver is not specified. If you want to install 32-bit driver please specify path to it as a second argument. -) else if exist "%ODBC_X86%" ( - if exist "%ODBC_X86%\" ( +) else if exist %ODBC_X86% ( + if exist %ODBC_X86%\ ( echo warning: The path you have specified seems to be a directory. Note that you have to specify path to driver file itself instead. ) - echo Installing 32-bit driver: "%ODBC_X86%" + echo Installing 32-bit driver: %ODBC_X86% reg add "HKEY_LOCAL_MACHINE\SOFTWARE\Wow6432Node\ODBC\ODBCINST.INI\Apache Ignite" /v DriverODBCVer /t REG_SZ /d "03.80" /f reg add "HKEY_LOCAL_MACHINE\SOFTWARE\Wow6432Node\ODBC\ODBCINST.INI\Apache Ignite" /v UsageCount /t REG_DWORD /d 00000001 /f - reg add "HKEY_LOCAL_MACHINE\SOFTWARE\Wow6432Node\ODBC\ODBCINST.INI\Apache Ignite" /v Driver /t REG_SZ /d "%ODBC_X86%" /f + reg add "HKEY_LOCAL_MACHINE\SOFTWARE\Wow6432Node\ODBC\ODBCINST.INI\Apache Ignite" /v Driver /t REG_SZ /d %ODBC_X86% /f + reg add "HKEY_LOCAL_MACHINE\SOFTWARE\Wow6432Node\ODBC\ODBCINST.INI\Apache Ignite" /v Setup /t REG_SZ /d %ODBC_X86% /f reg add "HKEY_LOCAL_MACHINE\SOFTWARE\Wow6432Node\ODBC\ODBCINST.INI\ODBC Drivers" /v "Apache Ignite" /t REG_SZ /d "Installed" /f ) else ( - echo warning: 32-bit driver can not be found: "%ODBC_X86%" + echo warning: 32-bit driver can not be found: %ODBC_X86% ) diff --git a/modules/platforms/cpp/odbc/install/install_x86.cmd b/modules/platforms/cpp/odbc/install/install_x86.cmd index 74eea0ab9bb91..93ced4b9010eb 100644 --- a/modules/platforms/cpp/odbc/install/install_x86.cmd +++ b/modules/platforms/cpp/odbc/install/install_x86.cmd @@ -5,17 +5,18 @@ set ODBC=%1 if [%ODBC%] == [] ( echo error: driver is not specified. Call format: install_x86 abs_path_to_driver. pause -) else if exist "%ODBC%" ( - if exist "%ODBC%\" ( +) else if exist %ODBC% ( + if exist %ODBC%\ ( echo warning: The path you have specified seems to be a directory. Note that you have to specify path to driver file itself instead. ) - echo Installing driver: "%ODBC%" + echo Installing driver: %ODBC% reg add "HKEY_LOCAL_MACHINE\SOFTWARE\ODBC\ODBCINST.INI\Apache Ignite" /v DriverODBCVer /t REG_SZ /d "03.80" /f reg add "HKEY_LOCAL_MACHINE\SOFTWARE\ODBC\ODBCINST.INI\Apache Ignite" /v UsageCount /t REG_DWORD /d 00000001 /f - reg add "HKEY_LOCAL_MACHINE\SOFTWARE\ODBC\ODBCINST.INI\Apache Ignite" /v Driver /t REG_SZ /d "%ODBC%" /f + reg add "HKEY_LOCAL_MACHINE\SOFTWARE\ODBC\ODBCINST.INI\Apache Ignite" /v Driver /t REG_SZ /d %ODBC% /f + reg add "HKEY_LOCAL_MACHINE\SOFTWARE\ODBC\ODBCINST.INI\Apache Ignite" /v Setup /t REG_SZ /d %ODBC% /f reg add "HKEY_LOCAL_MACHINE\SOFTWARE\ODBC\ODBCINST.INI\ODBC Drivers" /v "Apache Ignite" /t REG_SZ /d "Installed" /f ) else ( - echo Driver can not be found: "%ODBC%" + echo Driver can not be found: %ODBC% echo Call format: install_x86 abs_path_to_driver pause ) From 527a2996fea51b81267d0fc96da0f4d6076ed436 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 9 Sep 2016 16:51:15 +0700 Subject: [PATCH 123/487] IGNITE-3846 Fixed wrong recording of JobEvent for @GridInternal task. Added test. --- .../processors/task/GridTaskWorker.java | 2 +- .../internal/GridEventStorageSelfTest.java | 97 ++++++++++++++++++- 2 files changed, 97 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java index 00ea29e279e65..79d1a2cf4e35d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java @@ -1500,7 +1500,7 @@ private void recordTaskEvent(int evtType, String msg) { * @param msg Event message. */ private void recordJobEvent(int evtType, IgniteUuid jobId, ClusterNode evtNode, String msg) { - if (ctx.event().isRecordable(evtType)) { + if (!internal && ctx.event().isRecordable(evtType)) { JobEvent evt = new JobEvent(); evt.message(msg); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridEventStorageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridEventStorageSelfTest.java index a3b9608a35211..4f98b0c271171 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridEventStorageSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridEventStorageSelfTest.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal; import java.io.Serializable; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -31,12 +32,17 @@ import org.apache.ignite.compute.ComputeJobResult; import org.apache.ignite.compute.ComputeTaskSplitAdapter; import org.apache.ignite.events.Event; +import org.apache.ignite.events.JobEvent; +import org.apache.ignite.events.TaskEvent; +import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.testframework.junits.common.GridCommonTest; import static org.apache.ignite.events.EventType.EVTS_ALL_MINUS_METRIC_UPDATE; +import static org.apache.ignite.events.EventType.EVTS_JOB_EXECUTION; +import static org.apache.ignite.events.EventType.EVTS_TASK_EXECUTION; import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.events.EventType.EVT_TASK_STARTED; @@ -192,6 +198,53 @@ public void testRemoteAndLocalNodeEventStorage() throws Exception { assert remEvts.isEmpty(); } + /** + * Checks that specified event is not task or job event. + * + * @param evt Event to check. + */ + private void checkGridInternalEvent(Event evt) { + assertFalse("Found TASK event for task marked with @GridInternal [evtType=" + evt.type() + "]", evt instanceof TaskEvent); + assertFalse("Found JOB event for task marked with @GridInternal [evtType=" + evt.type() + "]", evt instanceof JobEvent); + } + + /** + * @throws Exception In case of error. + */ + public void testGridInternalEvents() throws Exception { + IgnitePredicate lsnr = new IgnitePredicate() { + @Override public boolean apply(Event evt) { + checkGridInternalEvent(evt); + + return true; + } + }; + + ignite1.events().localListen(lsnr, EVTS_TASK_EXECUTION); + ignite1.events().localListen(lsnr, EVTS_JOB_EXECUTION); + ignite2.events().localListen(lsnr, EVTS_TASK_EXECUTION); + ignite2.events().localListen(lsnr, EVTS_JOB_EXECUTION); + + executeGridInternalTask(ignite1); + + Collection evts1 = ignite1.events().localQuery(F.alwaysTrue()); + Collection evts2 = ignite2.events().localQuery(F.alwaysTrue()); + + assert evts1 != null; + assert evts2 != null; + + for (Event evt : evts1) + checkGridInternalEvent(evt); + + for (Event evt : evts2) + checkGridInternalEvent(evt); + + assert ignite1.events().stopLocalListen(lsnr, EVTS_TASK_EXECUTION); + assert ignite1.events().stopLocalListen(lsnr, EVTS_JOB_EXECUTION); + assert ignite2.events().stopLocalListen(lsnr, EVTS_TASK_EXECUTION); + assert ignite2.events().stopLocalListen(lsnr, EVTS_JOB_EXECUTION); + } + /** * Create events in grid. * @@ -203,6 +256,15 @@ private void generateEvents(Ignite ignite) { ignite.compute().execute(GridEventTestTask.class.getName(), null); } + /** + * Execute task marged with {@code GridInternal} annotation. + * + * @param ignite Grid. + */ + private void executeGridInternalTask(Ignite ignite) { + ignite.compute().execute(GridInternalTestTask.class.getName(), null); + } + /** * Test task. */ @@ -231,6 +293,39 @@ private static class GridEventTestJob extends ComputeJobAdapter { } } + /** + * Test task marked with @GridInternal. + */ + @GridInternal + private static class GridInternalTestTask extends ComputeTaskSplitAdapter { + /** {@inheritDoc} */ + @Override protected Collection split(int gridSize, Object arg) { + Collection jobs = new ArrayList<>(gridSize); + + for (int i = 0; i < gridSize; i++) + jobs.add(new GridInternalTestJob()); + + return jobs; + } + + /** {@inheritDoc} */ + @Override public Serializable reduce(List results) { + assert results != null; + + return "GridInternalTestTask-result."; + } + } + + /** + * Test job. + */ + private static class GridInternalTestJob extends ComputeJobAdapter { + /** {@inheritDoc} */ + @Override public String execute() { + return "GridInternalTestJob-result."; + } + } + /** * Test event listener. */ @@ -274,4 +369,4 @@ private static class TestEventFilter implements IgnitePredicate { return evt.type() == EVT_TASK_STARTED; } } -} \ No newline at end of file +} From 59527c5649afd3576d944d42299cf49416972562 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Fri, 9 Sep 2016 12:27:55 +0300 Subject: [PATCH 124/487] IGNITE-2649: Ensured correct local Ignite instance processing during serialization and deserialization. --- .../ignite/internal/GridLoggerProxy.java | 3 +- .../apache/ignite/internal/IgnitionEx.java | 17 +- .../ignite/internal/binary/BinaryContext.java | 3 + .../internal/binary/BinaryMarshaller.java | 21 +- .../internal/binary/BinaryReaderExImpl.java | 24 ++ .../internal/binary/BinaryWriterExImpl.java | 18 + .../client/GridClientConfiguration.java | 1 - .../internal/cluster/ClusterGroupAdapter.java | 2 +- .../cluster/ClusterNodeLocalMapImpl.java | 3 +- .../processors/cache/GridCacheAdapter.java | 2 +- .../processors/cache/GridCacheContext.java | 2 +- .../processors/cache/GridCacheProcessor.java | 5 +- .../ignite/internal/util/IgniteUtils.java | 60 ++- .../ignite/marshaller/AbstractMarshaller.java | 41 +- .../AbstractNodeNameAwareMarshaller.java | 142 +++++++ .../ignite/marshaller/MarshallerUtils.java | 58 +++ .../ignite/marshaller/jdk/JdkMarshaller.java | 40 +- .../optimized/OptimizedMarshaller.java | 12 +- .../sharedfs/SharedFsCheckpointSpi.java | 7 +- .../ignite/spi/discovery/tcp/ClientImpl.java | 11 +- .../ignite/spi/discovery/tcp/ServerImpl.java | 29 +- .../spi/discovery/tcp/TcpDiscoverySpi.java | 20 +- .../ignite/stream/socket/SocketStreamer.java | 18 +- .../ignite/internal/ClusterGroupSelfTest.java | 2 +- .../GridLocalIgniteSerializationTest.java | 378 ++++++++++++++++++ .../discovery/tcp/TcpDiscoverySelfTest.java | 2 +- .../testsuites/IgniteBasicTestSuite.java | 2 + 27 files changed, 818 insertions(+), 105 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/marshaller/AbstractNodeNameAwareMarshaller.java create mode 100644 modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalIgniteSerializationTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridLoggerProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/GridLoggerProxy.java index 295ee1c83af66..b26921bbd8d95 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridLoggerProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridLoggerProxy.java @@ -202,10 +202,9 @@ protected Object readResolve() throws ObjectStreamException { try { IgniteBiTuple t = stash.get(); - String gridNameR = t.get1(); Object ctgrR = t.get2(); - IgniteLogger log = IgnitionEx.gridx(gridNameR).log(); + IgniteLogger log = IgnitionEx.localIgnite().log(); return ctgrR != null ? log.getLogger(ctgrR) : log; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index 30f1d15c3f08f..b54c17d1794a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -77,6 +77,7 @@ import org.apache.ignite.logger.LoggerNodeIdAware; import org.apache.ignite.logger.java.JavaLogger; import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.mxbean.IgnitionMXBean; import org.apache.ignite.plugin.segmentation.SegmentationPolicy; @@ -1278,17 +1279,21 @@ public static Ignite grid(@Nullable String name) throws IgniteIllegalStateExcept } /** - * Gets the grid, which is owner of current thread. An Exception is thrown if - * current thread is not an {@link IgniteThread}. + * Gets a name of the grid from thread local config, which is owner of current thread. * * @return Grid instance related to current thread * @throws IllegalArgumentException Thrown to indicate, that current thread is not an {@link IgniteThread}. */ public static IgniteKernal localIgnite() throws IllegalArgumentException { - if (Thread.currentThread() instanceof IgniteThread) + String name = U.getCurrentIgniteName(); + + if (U.isCurrentIgniteNameSet(name)) + return gridx(name); + else if (Thread.currentThread() instanceof IgniteThread) return gridx(((IgniteThread)Thread.currentThread()).getGridName()); else - throw new IllegalArgumentException("This method should be accessed under " + IgniteThread.class.getName()); + throw new IllegalArgumentException("Ignite grid name thread local must be set or" + + " this method should be accessed under " + IgniteThread.class.getName()); } /** @@ -1297,7 +1302,7 @@ public static IgniteKernal localIgnite() throws IllegalArgumentException { * @param name Grid name. * @return Grid instance. */ - public static IgniteKernal gridx(@Nullable String name) { + private static IgniteKernal gridx(@Nullable String name) { IgniteNamedInstance grid = name != null ? grids.get(name) : dfltGrid; IgniteKernal res; @@ -1929,6 +1934,8 @@ private IgniteConfiguration initializeConfiguration(IgniteConfiguration cfg) marsh = new BinaryMarshaller(); } + MarshallerUtils.setNodeName(marsh, cfg.getGridName()); + myCfg.setMarshaller(marsh); if (myCfg.getPeerClassLoadingLocalClassPathExclude() == null) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java index 97afef1292e8b..0d66970fe9fb3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java @@ -82,6 +82,7 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.marshaller.MarshallerContext; +import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.marshaller.optimized.OptimizedMarshaller; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -251,6 +252,8 @@ public BinaryContext(BinaryMetadataHandler metaHnd, IgniteConfiguration igniteCf assert metaHnd != null; assert igniteCfg != null; + MarshallerUtils.setNodeName(optmMarsh, igniteCfg.getGridName()); + this.metaHnd = metaHnd; this.igniteCfg = igniteCfg; this.log = log; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java index 39015e5312e69..168c61abee373 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java @@ -24,7 +24,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.marshaller.AbstractMarshaller; +import org.apache.ignite.marshaller.AbstractNodeNameAwareMarshaller; import org.apache.ignite.marshaller.MarshallerContext; import org.jetbrains.annotations.Nullable; import sun.misc.Unsafe; @@ -33,7 +33,7 @@ * Implementation of {@link org.apache.ignite.marshaller.Marshaller} that lets to serialize and deserialize all objects * in the binary format. */ -public class BinaryMarshaller extends AbstractMarshaller { +public class BinaryMarshaller extends AbstractNodeNameAwareMarshaller { /** */ private GridBinaryMarshaller impl; @@ -66,15 +66,6 @@ public static boolean available() { } } - /** - * Returns currently set {@link MarshallerContext}. - * - * @return Marshaller context. - */ - public MarshallerContext getContext() { - return ctx; - } - /** * Sets {@link BinaryContext}. *

          @@ -88,12 +79,12 @@ private void setBinaryContext(BinaryContext ctx, IgniteConfiguration cfg) { } /** {@inheritDoc} */ - @Override public byte[] marshal(@Nullable Object obj) throws IgniteCheckedException { + @Override protected byte[] marshal0(@Nullable Object obj) throws IgniteCheckedException { return impl.marshal(obj); } /** {@inheritDoc} */ - @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException { + @Override protected void marshal0(@Nullable Object obj, OutputStream out) throws IgniteCheckedException { byte[] arr = marshal(obj); try { @@ -105,12 +96,12 @@ private void setBinaryContext(BinaryContext ctx, IgniteConfiguration cfg) { } /** {@inheritDoc} */ - @Override public T unmarshal(byte[] bytes, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { + @Override protected T unmarshal0(byte[] bytes, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { return impl.deserialize(bytes, clsLdr); } /** {@inheritDoc} */ - @Override public T unmarshal(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { + @Override protected T unmarshal0(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { ByteArrayOutputStream buf = new ByteArrayOutputStream(); // we have to fully read the InputStream because GridBinaryMarshaller requires support of a method that diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java index 194b1beea610d..775f237787cfc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java @@ -34,6 +34,7 @@ import org.apache.ignite.binary.BinaryRawReader; import org.apache.ignite.binary.BinaryReader; import org.apache.ignite.internal.binary.streams.BinaryInputStream; +import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.typedef.internal.SB; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -1442,6 +1443,22 @@ else if (flag == NULL) * @throws BinaryObjectException If failed. */ @Nullable Object deserialize() throws BinaryObjectException { + String newName = ctx.configuration().getGridName(); + String oldName = IgniteUtils.setCurrentIgniteName(newName); + + try { + return deserialize0(); + } + finally { + IgniteUtils.restoreOldIgniteName(oldName, newName); + } + } + + /** + * @return Deserialized object. + * @throws BinaryObjectException If failed. + */ + @Nullable private Object deserialize0() throws BinaryObjectException { Object obj; byte flag = in.readByte(); @@ -2049,6 +2066,13 @@ private void streamPositionRandom(int pos) { // No-op. } + /** + * @return Binary context. + */ + public BinaryContext context() { + return ctx; + } + /** * Flag. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java index 94504829a32cd..21fb2bfe5033d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java @@ -23,6 +23,7 @@ import org.apache.ignite.binary.BinaryWriter; import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream; import org.apache.ignite.internal.binary.streams.BinaryOutputStream; +import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.typedef.internal.A; import org.jetbrains.annotations.Nullable; @@ -138,6 +139,23 @@ void marshal(Object obj) throws BinaryObjectException { * @throws org.apache.ignite.binary.BinaryObjectException In case of error. */ void marshal(Object obj, boolean enableReplace) throws BinaryObjectException { + String newName = ctx.configuration().getGridName(); + String oldName = IgniteUtils.setCurrentIgniteName(newName); + + try { + marshal0(obj, enableReplace); + } + finally { + IgniteUtils.restoreOldIgniteName(oldName, newName); + } + } + + /** + * @param obj Object. + * @param enableReplace Object replacing enabled flag. + * @throws org.apache.ignite.binary.BinaryObjectException In case of error. + */ + private void marshal0(Object obj, boolean enableReplace) throws BinaryObjectException { assert obj != null; Class cls = obj.getClass(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java index e9d2958541665..cac1bb3f47eb1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java @@ -157,7 +157,6 @@ public GridClientConfiguration(GridClientConfiguration cfg) { tcpNoDelay = cfg.isTcpNoDelay(); topRefreshFreq = cfg.getTopologyRefreshFrequency(); daemon = cfg.isDaemon(); - marshaller = cfg.getMarshaller(); setDataConfigurations(cfg.getDataConfigurations()); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java index c664f1e4c6097..648c86dc1f633 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java @@ -727,7 +727,7 @@ private boolean contains(UUID id) { */ protected Object readResolve() throws ObjectStreamException { try { - IgniteKernal g = IgnitionEx.gridx(gridName); + IgniteKernal g = IgnitionEx.localIgnite(); return ids != null ? new ClusterGroupAdapter(g.context(), subjId, ids) : new ClusterGroupAdapter(g.context(), subjId, p); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterNodeLocalMapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterNodeLocalMapImpl.java index 4890231f8bbe0..3147b120ccb50 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterNodeLocalMapImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterNodeLocalMapImpl.java @@ -26,6 +26,7 @@ import java.util.concurrent.ConcurrentMap; import org.apache.ignite.IgniteCluster; import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.IgnitionEx; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -89,7 +90,7 @@ public ClusterNodeLocalMapImpl() { */ protected Object readResolve() throws ObjectStreamException { try { - return IgnitionEx.gridx(stash.get()).cluster().nodeLocalMap(); + return IgnitionEx.localIgnite().cluster().nodeLocalMap(); } catch (IllegalStateException e) { throw U.withCause(new InvalidObjectException(e.getMessage()), e); 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 1ac94a46d022c..fe6bb1be27c89 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 @@ -4639,7 +4639,7 @@ protected Object readResolve() throws ObjectStreamException { try { IgniteBiTuple t = stash.get(); - return IgnitionEx.gridx(t.get1()).cachex(t.get2()); + return IgnitionEx.localIgnite().cachex(t.get2()); } catch (IllegalStateException e) { throw U.withCause(new InvalidObjectException(e.getMessage()), e); 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 ba923df1cd839..dc9c766dcd993 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 @@ -2019,7 +2019,7 @@ protected Object readResolve() throws ObjectStreamException { try { IgniteBiTuple t = stash.get(); - IgniteKernal grid = IgnitionEx.gridx(t.get1()); + IgniteKernal grid = IgnitionEx.localIgnite(); GridCacheAdapter cache = grid.internalCache(t.get2()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index e104b87c95d91..6640db8c7bcef 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -116,6 +116,7 @@ import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.lifecycle.LifecycleAware; import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.spi.IgniteNodeValidationResult; import org.jetbrains.annotations.Nullable; @@ -185,7 +186,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { private IdentityHashMap sesHolders = new IdentityHashMap<>(); /** Must use JDK marshaller since it is used by discovery to fire custom events. */ - private Marshaller marshaller = new JdkMarshaller(); + private final Marshaller marshaller; /** Count down latch for caches. */ private final CountDownLatch cacheStartedLatch = new CountDownLatch(1); @@ -205,6 +206,8 @@ public GridCacheProcessor(GridKernalContext ctx) { caches = new ConcurrentHashMap<>(); jCacheProxies = new ConcurrentHashMap<>(); stopSeq = new LinkedList<>(); + + marshaller = MarshallerUtils.jdkMarshaller(ctx.gridName()); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index a480b8761cf9c..cdaeab1abe552 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -254,7 +254,7 @@ /** * Collection of utility methods used throughout the system. */ -@SuppressWarnings({"UnusedReturnValue", "UnnecessaryFullyQualifiedName"}) +@SuppressWarnings({"UnusedReturnValue", "UnnecessaryFullyQualifiedName", "RedundantStringConstructorCall"}) public abstract class IgniteUtils { /** {@code True} if {@code unsafe} should be used for array copy. */ private static final boolean UNSAFE_BYTE_ARR_CP = unsafeByteArrayCopyAvailable(); @@ -490,6 +490,16 @@ public abstract class IgniteUtils { /** Object.toString() */ private static Method toStringMtd; + /** Empty local Ignite name. */ + public static final String LOC_IGNITE_NAME_EMPTY = new String(); + + /** Local Ignite name thread local. */ + private static final ThreadLocal LOC_IGNITE_NAME = new ThreadLocal() { + @Override protected String initialValue() { + return LOC_IGNITE_NAME_EMPTY; + } + }; + /** * Initializes enterprise check. */ @@ -9612,4 +9622,52 @@ public static String threadName(long threadId) { public static > T max(T t0, T t1) { return t0.compareTo(t1) > 0 ? t0 : t1; } + + /** + * Get current Ignite name. + * + * @return Current Ignite name. + */ + @Nullable public static String getCurrentIgniteName() { + return LOC_IGNITE_NAME.get(); + } + + /** + * Check if current Ignite name is set. + * + * @param name Name to check. + * @return {@code True} if set. + */ + @SuppressWarnings("StringEquality") + public static boolean isCurrentIgniteNameSet(@Nullable String name) { + return name != LOC_IGNITE_NAME_EMPTY; + } + + /** + * Set current Ignite name. + * + * @param newName New name. + * @return Old name. + */ + @SuppressWarnings("StringEquality") + @Nullable public static String setCurrentIgniteName(@Nullable String newName) { + String oldName = LOC_IGNITE_NAME.get(); + + if (oldName != newName) + LOC_IGNITE_NAME.set(newName); + + return oldName; + } + + /** + * Restore old Ignite name. + * + * @param oldName Old name. + * @param curName Current name. + */ + @SuppressWarnings("StringEquality") + public static void restoreOldIgniteName(@Nullable String oldName, @Nullable String curName) { + if (oldName != curName) + LOC_IGNITE_NAME.set(oldName); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/AbstractMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/AbstractMarshaller.java index dd5bad0c5802a..6c3428e3924f7 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/AbstractMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/AbstractMarshaller.java @@ -17,12 +17,9 @@ package org.apache.ignite.marshaller; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.util.GridByteArrayList; import org.apache.ignite.internal.util.io.GridByteArrayInputStream; import org.apache.ignite.internal.util.io.GridByteArrayOutputStream; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.jetbrains.annotations.Nullable; /** * Base class for marshallers. Provides default implementations of methods @@ -37,7 +34,6 @@ public abstract class AbstractMarshaller implements Marshaller { /** Context. */ protected MarshallerContext ctx; - /** * Undeployment callback invoked when class loader is being undeployed. * @@ -47,38 +43,15 @@ public abstract class AbstractMarshaller implements Marshaller { */ public abstract void onUndeploy(ClassLoader ldr); - /** {@inheritDoc} */ - @Override public void setContext(MarshallerContext ctx) { - this.ctx = ctx; - } - - /** {@inheritDoc} */ - @Override public byte[] marshal(@Nullable Object obj) throws IgniteCheckedException { - GridByteArrayOutputStream out = null; - - try { - out = new GridByteArrayOutputStream(DFLT_BUFFER_SIZE); - - marshal(obj, out); - - return out.toByteArray(); - } - finally { - U.close(out, null); - } + /** + * @return Marshaller context. + */ + public MarshallerContext getContext() { + return ctx; } /** {@inheritDoc} */ - @Override public T unmarshal(byte[] arr, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { - GridByteArrayInputStream in = null; - - try { - in = new GridByteArrayInputStream(arr, 0, arr.length); - - return unmarshal(in, clsLdr); - } - finally { - U.close(in, null); - } + @Override public void setContext(MarshallerContext ctx) { + this.ctx = ctx; } } diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/AbstractNodeNameAwareMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/AbstractNodeNameAwareMarshaller.java new file mode 100644 index 0000000000000..559ac73d8a7d8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/AbstractNodeNameAwareMarshaller.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.marshaller; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.Ignition; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import java.io.InputStream; +import java.io.OutputStream; + +/** + * Marshaller allowing for {@link Ignition#localIgnite()} calls. + */ +public abstract class AbstractNodeNameAwareMarshaller extends AbstractMarshaller { + /** Whether node name is set. */ + private volatile boolean nodeNameSet; + + /** Node name. */ + private volatile String nodeName = U.LOC_IGNITE_NAME_EMPTY; + + /** + * Set node name. + * + * @param nodeName Node name. + */ + @SuppressWarnings("unchecked") + public void nodeName(@Nullable String nodeName) { + if (!nodeNameSet) { + this.nodeName = nodeName; + + nodeNameSet = true; + } + } + + /** {@inheritDoc} */ + @Override public byte[] marshal(@Nullable Object obj) throws IgniteCheckedException { + String oldNodeName = IgniteUtils.setCurrentIgniteName(nodeName); + + try { + return marshal0(obj); + } + finally { + IgniteUtils.restoreOldIgniteName(oldNodeName, nodeName); + } + } + + /** {@inheritDoc} */ + @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException { + String oldNodeName = IgniteUtils.setCurrentIgniteName(nodeName); + + try { + marshal0(obj, out); + } + finally { + IgniteUtils.restoreOldIgniteName(oldNodeName, nodeName); + } + } + + /** {@inheritDoc} */ + @Override public T unmarshal(byte[] arr, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { + String oldNodeName = IgniteUtils.setCurrentIgniteName(nodeName); + + try { + return unmarshal0(arr, clsLdr); + } + finally { + IgniteUtils.restoreOldIgniteName(oldNodeName, nodeName); + } + } + + /** {@inheritDoc} */ + @Override public T unmarshal(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { + String oldNodeName = IgniteUtils.setCurrentIgniteName(nodeName); + + try { + return unmarshal0(in, clsLdr); + } + finally { + IgniteUtils.restoreOldIgniteName(oldNodeName, nodeName); + } + } + + /** + * Marshals object to the output stream. This method should not close + * given output stream. + * + * @param obj Object to marshal. + * @param out Output stream to marshal into. + * @throws IgniteCheckedException If marshalling failed. + */ + protected abstract void marshal0(@Nullable Object obj, OutputStream out) throws IgniteCheckedException; + + /** + * Marshals object to byte array. + * + * @param obj Object to marshal. + * @return Byte array. + * @throws IgniteCheckedException If marshalling failed. + */ + protected abstract byte[] marshal0(@Nullable Object obj) throws IgniteCheckedException; + + /** + * Unmarshals object from the input stream using given class loader. + * This method should not close given input stream. + * + * @param Type of unmarshalled object. + * @param in Input stream. + * @param clsLdr Class loader to use. + * @return Unmarshalled object. + * @throws IgniteCheckedException If unmarshalling failed. + */ + protected abstract T unmarshal0(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException; + + /** + * Unmarshals object from byte array using given class loader. + * + * @param Type of unmarshalled object. + * @param arr Byte array. + * @param clsLdr Class loader to use. + * @return Unmarshalled object. + * @throws IgniteCheckedException If unmarshalling failed. + */ + protected abstract T unmarshal0(byte[] arr, @Nullable ClassLoader clsLdr) throws IgniteCheckedException; +} diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java new file mode 100644 index 0000000000000..9668bafb1c7ff --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java @@ -0,0 +1,58 @@ +/* + * 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.marshaller; + +import org.apache.ignite.marshaller.jdk.JdkMarshaller; +import org.jetbrains.annotations.Nullable; + +/** + * Utility marshaller methods. + */ +public class MarshallerUtils { + /** + * Set node name to marshaller context if possible. + * + * @param marsh Marshaller instance. + * @param nodeName Node name. + */ + public static void setNodeName(Marshaller marsh, @Nullable String nodeName) { + if (marsh instanceof AbstractNodeNameAwareMarshaller) + ((AbstractNodeNameAwareMarshaller)marsh).nodeName(nodeName); + } + + /** + * Create JDK marshaller with provided node name. + * + * @param nodeName Node name. + * @return JDK marshaller. + */ + public static JdkMarshaller jdkMarshaller(@Nullable String nodeName) { + JdkMarshaller marsh = new JdkMarshaller(); + + setNodeName(new JdkMarshaller(), nodeName); + + return marsh; + } + + /** + * Private constructor. + */ + private MarshallerUtils() { + // No-op. + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java index deb3953065c34..7a130d3686931 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java @@ -23,9 +23,11 @@ import java.io.ObjectOutputStream; import java.io.OutputStream; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.util.io.GridByteArrayInputStream; +import org.apache.ignite.internal.util.io.GridByteArrayOutputStream; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.marshaller.AbstractMarshaller; +import org.apache.ignite.marshaller.AbstractNodeNameAwareMarshaller; import org.jetbrains.annotations.Nullable; /** @@ -63,9 +65,9 @@ *
          * For information about Spring framework visit www.springframework.org */ -public class JdkMarshaller extends AbstractMarshaller { +public class JdkMarshaller extends AbstractNodeNameAwareMarshaller { /** {@inheritDoc} */ - @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException { + @Override protected void marshal0(@Nullable Object obj, OutputStream out) throws IgniteCheckedException { assert out != null; ObjectOutputStream objOut = null; @@ -86,9 +88,25 @@ public class JdkMarshaller extends AbstractMarshaller { } } + /** {@inheritDoc} */ + @Override protected byte[] marshal0(@Nullable Object obj) throws IgniteCheckedException { + GridByteArrayOutputStream out = null; + + try { + out = new GridByteArrayOutputStream(DFLT_BUFFER_SIZE); + + marshal(obj, out); + + return out.toByteArray(); + } + finally { + U.close(out, null); + } + } + /** {@inheritDoc} */ @SuppressWarnings({"unchecked"}) - @Override public T unmarshal(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { + @Override protected T unmarshal0(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { assert in != null; if (clsLdr == null) @@ -114,6 +132,20 @@ public class JdkMarshaller extends AbstractMarshaller { } } + /** {@inheritDoc} */ + @Override protected T unmarshal0(byte[] arr, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { + GridByteArrayInputStream in = null; + + try { + in = new GridByteArrayInputStream(arr, 0, arr.length); + + return unmarshal(in, clsLdr); + } + finally { + U.close(in, null); + } + } + /** {@inheritDoc} */ @Override public void onUndeploy(ClassLoader ldr) { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java index b3caca2fbff83..37f7acb30d318 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java @@ -27,7 +27,7 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.marshaller.AbstractMarshaller; +import org.apache.ignite.marshaller.AbstractNodeNameAwareMarshaller; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; import sun.misc.Unsafe; @@ -82,7 +82,7 @@ *
          * For information about Spring framework visit www.springframework.org */ -public class OptimizedMarshaller extends AbstractMarshaller { +public class OptimizedMarshaller extends AbstractNodeNameAwareMarshaller { /** Use default {@code serialVersionUID} for {@link Serializable} classes. */ public static final boolean USE_DFLT_SUID = IgniteSystemProperties.getBoolean(IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID, false); @@ -158,7 +158,7 @@ public void setPoolSize(int poolSize) { } /** {@inheritDoc} */ - @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException { + @Override protected void marshal0(@Nullable Object obj, OutputStream out) throws IgniteCheckedException { assert out != null; OptimizedObjectOutputStream objOut = null; @@ -181,7 +181,7 @@ public void setPoolSize(int poolSize) { } /** {@inheritDoc} */ - @Override public byte[] marshal(@Nullable Object obj) throws IgniteCheckedException { + @Override protected byte[] marshal0(@Nullable Object obj) throws IgniteCheckedException { OptimizedObjectOutputStream objOut = null; try { @@ -203,7 +203,7 @@ public void setPoolSize(int poolSize) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public T unmarshal(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { + @Override protected T unmarshal0(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { assert in != null; OptimizedObjectInputStream objIn = null; @@ -232,7 +232,7 @@ public void setPoolSize(int poolSize) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public T unmarshal(byte[] arr, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { + @Override protected T unmarshal0(byte[] arr, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { assert arr != null; OptimizedObjectInputStream objIn = null; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java index 36a4ea62a2c0b..092c864947177 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java @@ -35,6 +35,7 @@ import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.LoggerResource; @@ -214,8 +215,10 @@ public void setDirectoryPaths(Collection dirPaths) { this.gridName = gridName; - marsh = ignite.configuration().getMarshaller() instanceof BinaryMarshaller ? new JdkMarshaller() : - ignite.configuration().getMarshaller(); + if (ignite.configuration().getMarshaller() instanceof BinaryMarshaller) + marsh = MarshallerUtils.jdkMarshaller(ignite.name()); + else + marsh = ignite.configuration().getMarshaller(); folder = getNextSharedPath(); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 1e71888f7fd62..bf7f519b6ca7f 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -64,6 +64,7 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.spi.IgniteSpiContext; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.IgniteSpiOperationTimeoutHelper; @@ -430,7 +431,7 @@ else if (state == DISCONNECTED) { try { sockWriter.sendMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt, - spi.marsh.marshal(evt))); + spi.marshaller().marshal(evt))); } catch (IgniteCheckedException e) { throw new IgniteSpiException("Failed to marshal custom event: " + evt, e); @@ -684,7 +685,7 @@ private void marshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException Map attrs = new HashMap<>(node.getAttributes()); attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS, - spi.marsh.marshal(attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS))); + spi.marshaller().marshal(attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS))); node.setAttributes(attrs); } @@ -885,7 +886,7 @@ public void setSocket(SocketStream sockStream, UUID rmtNodeId) { TcpDiscoveryAbstractMessage msg; try { - msg = spi.marsh.unmarshal(in, U.resolveClassLoader(spi.ignite().configuration())); + msg = spi.marshaller().unmarshal(in, U.resolveClassLoader(spi.ignite().configuration())); } catch (IgniteCheckedException e) { if (log.isDebugEnabled()) @@ -1215,7 +1216,7 @@ public void cancel() { List msgs = null; while (!isInterrupted()) { - TcpDiscoveryAbstractMessage msg = spi.marsh.unmarshal(in, + TcpDiscoveryAbstractMessage msg = spi.marshaller().unmarshal(in, U.resolveClassLoader(spi.ignite().configuration())); if (msg instanceof TcpDiscoveryClientReconnectMessage) { @@ -1976,7 +1977,7 @@ private void processCustomMessage(TcpDiscoveryCustomEventMessage msg) { if (node != null && node.visible()) { try { - DiscoverySpiCustomMessage msgObj = msg.message(spi.marsh, + DiscoverySpiCustomMessage msgObj = msg.message(spi.marshaller(), U.resolveClassLoader(spi.ignite().configuration())); notifyDiscovery(EVT_DISCOVERY_CUSTOM_EVT, topVer, node, allVisibleNodes(), msgObj); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 6db1e8796d038..135a737587bc6 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -743,7 +743,8 @@ private void interruptPing(TcpDiscoveryNode node) { /** {@inheritDoc} */ @Override public void sendCustomEvent(DiscoverySpiCustomMessage evt) { try { - msgWorker.addMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt, spi.marsh.marshal(evt))); + msgWorker.addMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt, + spi.marshaller().marshal(evt))); } catch (IgniteCheckedException e) { throw new IgniteSpiException("Failed to marshal custom event: " + evt, e); @@ -826,7 +827,7 @@ private void joinTopology() throws IgniteSpiException { Map attrs = new HashMap<>(locNode.attributes()); - attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, spi.marsh.marshal(subj)); + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, spi.marshaller().marshal(subj)); attrs.remove(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS); locNode.setAttributes(attrs); @@ -1242,7 +1243,7 @@ private void marshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException Map attrs = new HashMap<>(node.getAttributes()); attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS, - spi.marsh.marshal(attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS))); + spi.marshaller().marshal(attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS))); node.setAttributes(attrs); } @@ -1265,7 +1266,7 @@ private SecurityCredentials unmarshalCredentials(TcpDiscoveryNode node) throws I if (credBytes == null) return null; - return spi.marsh.unmarshal(credBytes, null); + return spi.marshaller().unmarshal(credBytes, null); } catch (IgniteCheckedException e) { throw new IgniteSpiException("Failed to unmarshal node security credentials: " + node.id(), e); @@ -2379,7 +2380,7 @@ private void sendMessageToClients(TcpDiscoveryAbstractMessage msg) { for (ClientMessageWorker clientMsgWorker : clientMsgWorkers.values()) { if (msgBytes == null) { try { - msgBytes = spi.marsh.marshal(msg); + msgBytes = spi.marshaller().marshal(msg); } catch (IgniteCheckedException e) { U.error(log, "Failed to marshal message: " + msg, e); @@ -2398,7 +2399,7 @@ private void sendMessageToClients(TcpDiscoveryAbstractMessage msg) { if (clientMsgWorker.clientNodeId.equals(node.id())) { try { - msg0 = spi.marsh.unmarshal(msgBytes, + msg0 = spi.marshaller().unmarshal(msgBytes, U.resolveClassLoader(spi.ignite().configuration())); prepareNodeAddedMessage(msg0, clientMsgWorker.clientNodeId, null, null, null); @@ -3156,7 +3157,7 @@ else if (log.isDebugEnabled()) // Stick in authentication subject to node (use security-safe attributes for copy). Map attrs = new HashMap<>(node.getAttributes()); - attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, spi.marsh.marshal(subj)); + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, spi.marshaller().marshal(subj)); node.setAttributes(attrs); } @@ -3804,7 +3805,7 @@ else if (!locNodeId.equals(node.id()) && ring.node(node.id()) != null) { else { SecurityContext subj = spi.nodeAuth.authenticateNode(node, cred); - SecurityContext coordSubj = spi.marsh.unmarshal( + SecurityContext coordSubj = spi.marshaller().unmarshal( node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT), U.resolveClassLoader(spi.ignite().configuration())); @@ -4859,7 +4860,7 @@ private void processCustomMessage(TcpDiscoveryCustomEventMessage msg) { DiscoverySpiCustomMessage msgObj = null; try { - msgObj = msg.message(spi.marsh, U.resolveClassLoader(spi.ignite().configuration())); + msgObj = msg.message(spi.marshaller(), U.resolveClassLoader(spi.ignite().configuration())); } catch (Throwable e) { U.error(log, "Failed to unmarshal discovery custom message.", e); @@ -4871,7 +4872,7 @@ private void processCustomMessage(TcpDiscoveryCustomEventMessage msg) { if (nextMsg != null) { try { TcpDiscoveryCustomEventMessage ackMsg = new TcpDiscoveryCustomEventMessage( - getLocalNodeId(), nextMsg, spi.marsh.marshal(nextMsg)); + getLocalNodeId(), nextMsg, spi.marshaller().marshal(nextMsg)); ackMsg.topologyVersion(msg.topologyVersion()); @@ -5005,7 +5006,7 @@ private void notifyDiscoveryListener(TcpDiscoveryCustomEventMessage msg) { if (node != null) { try { - DiscoverySpiCustomMessage msgObj = msg.message(spi.marsh, + DiscoverySpiCustomMessage msgObj = msg.message(spi.marshaller(), U.resolveClassLoader(spi.ignite().configuration())); lsnr.onDiscovery(DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT, @@ -5016,7 +5017,7 @@ private void notifyDiscoveryListener(TcpDiscoveryCustomEventMessage msg) { msgObj); if (msgObj.isMutable()) - msg.message(msgObj, spi.marsh.marshal(msgObj)); + msg.message(msgObj, spi.marshaller().marshal(msgObj)); } catch (Throwable e) { U.error(log, "Failed to unmarshal discovery custom message.", e); @@ -5454,7 +5455,7 @@ else if (e.hasCause(ObjectStreamException.class) || while (!isInterrupted()) { try { - TcpDiscoveryAbstractMessage msg = spi.marsh.unmarshal(in, + TcpDiscoveryAbstractMessage msg = spi.marshaller().unmarshal(in, U.resolveClassLoader(spi.ignite().configuration())); msg.senderNodeId(nodeId); @@ -5945,7 +5946,7 @@ public void addMessage(TcpDiscoveryAbstractMessage msg, @Nullable byte[] msgByte byte[] msgBytes = msgT.get2(); if (msgBytes == null) - msgBytes = spi.marsh.marshal(msg); + msgBytes = spi.marshaller().marshal(msg); if (msg instanceof TcpDiscoveryClientAckResponse) { if (clientVer == null) { diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java index 3d6df89cb0248..426eb8ef039e5 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java @@ -64,6 +64,7 @@ import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.LoggerResource; @@ -342,7 +343,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T protected volatile long gridStartTime; /** Marshaller. */ - protected final Marshaller marsh = new JdkMarshaller(); + private final Marshaller marsh = new JdkMarshaller(); /** Statistics. */ protected final TcpDiscoveryStatistics stats = new TcpDiscoveryStatistics(); @@ -1379,7 +1380,7 @@ protected void writeToSocket(Socket sock, IgniteCheckedException err = null; try { - marsh.marshal(msg, out); + marshaller().marshal(msg, out); } catch (IgniteCheckedException e) { err = e; @@ -1463,7 +1464,7 @@ protected T readMessage(Socket sock, @Nullable InputStream in, long timeout) try { sock.setSoTimeout((int)timeout); - T res = marsh.unmarshal(in == null ? sock.getInputStream() : in, + T res = marshaller().unmarshal(in == null ? sock.getInputStream() : in, U.resolveClassLoader(ignite.configuration())); return res; @@ -1681,7 +1682,7 @@ protected Map collectExchangeData(UUID nodeId) { for (Map.Entry entry : data.entrySet()) { try { - byte[] bytes = marsh.marshal(entry.getValue()); + byte[] bytes = marshaller().marshal(entry.getValue()); data0.put(entry.getKey(), bytes); } @@ -1712,7 +1713,7 @@ protected void onExchange(UUID joiningNodeID, for (Map.Entry entry : data.entrySet()) { try { - Serializable compData = marsh.unmarshal(entry.getValue(), clsLdr); + Serializable compData = marshaller().unmarshal(entry.getValue(), clsLdr); data0.put(entry.getKey(), compData); } @@ -1989,6 +1990,15 @@ public void brakeConnection() { impl.brakeConnection(); } + /** + * @return Marshaller. + */ + protected Marshaller marshaller() { + MarshallerUtils.setNodeName(marsh, gridName); + + return marsh; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(TcpDiscoverySpi.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java b/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java index 21204c70b5a9c..a5b89e47b7ae3 100644 --- a/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java +++ b/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java @@ -35,7 +35,8 @@ import org.apache.ignite.internal.util.nio.GridNioSession; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; -import org.apache.ignite.marshaller.jdk.JdkMarshaller; +import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.stream.StreamAdapter; import org.apache.ignite.stream.StreamTupleExtractor; import org.jetbrains.annotations.Nullable; @@ -173,7 +174,7 @@ public void start() { new GridDelimitedParser(delim, directMode); if (converter == null) - converter = new DefaultConverter<>(); + converter = new DefaultConverter<>(getIgnite().name()); GridNioFilter codec = new GridNioCodecFilter(parser, log, directMode); @@ -216,12 +217,21 @@ public void stop() { */ private static class DefaultConverter implements SocketMessageConverter { /** Marshaller. */ - private static final JdkMarshaller MARSH = new JdkMarshaller(); + private final Marshaller marsh; + + /** + * Constructor. + * + * @param gridName Grid name. + */ + private DefaultConverter(@Nullable String gridName) { + marsh = MarshallerUtils.jdkMarshaller(gridName); + } /** {@inheritDoc} */ @Override public T convert(byte[] msg) { try { - return MARSH.unmarshal(msg, null); + return marsh.unmarshal(msg, null); } catch (IgniteCheckedException e) { throw new IgniteException(e); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java index 18eb3b773d0c2..9c74bd834a1b5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java @@ -211,7 +211,7 @@ public void testForPredicate() throws Exception { * @throws Exception If failed. */ public void testAgeClusterGroupSerialization() throws Exception { - Marshaller marshaller = getConfiguration().getMarshaller(); + Marshaller marshaller = ignite.configuration().getMarshaller(); ClusterGroup grp = ignite.cluster().forYoungest(); ClusterNode node = grp.node(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalIgniteSerializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalIgniteSerializationTest.java new file mode 100644 index 0000000000000..9737d1217c8af --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalIgniteSerializationTest.java @@ -0,0 +1,378 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryReader; +import org.apache.ignite.binary.BinaryWriter; +import org.apache.ignite.binary.Binarylizable; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.binary.BinaryMarshaller; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.marshaller.AbstractNodeNameAwareMarshaller; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectInputStream; +import java.io.ObjectOutput; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.concurrent.Callable; + +/** + * Test for local Ignite instance processing during serialization/deserialization. + */ +public class GridLocalIgniteSerializationTest extends GridCommonAbstractTest { + /** */ + private static final String CACHE_NAME = "cache_name"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception { + final IgniteConfiguration cfg = super.getConfiguration(gridName); + + if (gridName != null && gridName.startsWith("binary")) + cfg.setMarshaller(new BinaryMarshaller()); + + return cfg; + } + + /** + * Test that calling {@link Ignition#localIgnite()} + * is safe for binary marshaller. + * + * @throws Exception If failed. + */ + public void testPutGetSimple() throws Exception { + checkPutGet(new SimpleTestObject("one"), null); + } + + /** + * @throws Exception If failed. + */ + public void testPutGetSerializable() throws Exception { + checkPutGet(new SerializableTestObject("test"), null); + } + + /** + * @throws Exception If failed. + */ + public void testPutGetExternalizable() throws Exception { + checkPutGet(new ExternalizableTestObject("test"), null); + } + + /** + * @throws Exception If failed. + */ + public void testPutGetBinarylizable() throws Exception { + checkPutGet(new BinarylizableTestObject("test"), "binaryIgnite"); + } + + /** + * @throws Exception If failed. + */ + private void checkPutGet(final TestObject obj, final String gridName) throws Exception { + + + // Run async to emulate user thread. + GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + try (final Ignite ignite = startGrid(gridName)) { + if (ignite.configuration().getMarshaller() instanceof AbstractNodeNameAwareMarshaller) { + final IgniteCache cache = ignite.getOrCreateCache(CACHE_NAME); + + assertNull(obj.ignite()); + + cache.put(1, obj); + + assertNotNull(obj.ignite()); + + final TestObject loadedObj = cache.get(1); + + assertNotNull(loadedObj.ignite()); + + assertEquals(obj, loadedObj); + } + } + + return null; + } + }).get(); + } + + /** + * + */ + private interface TestObject { + /** + * @return Ignite instance. + */ + Ignite ignite(); + } + + /** + * Test object. + */ + private static class SimpleTestObject implements TestObject { + /** */ + private final String val; + + /** */ + private transient Ignite ignite; + + /** */ + private SimpleTestObject(final String val) { + this.val = val; + } + + /** + * @return Object. + */ + @SuppressWarnings("unused") + private Object readResolve() { + ignite = Ignition.localIgnite(); + + return this; + } + + /** + * @return Object. + */ + @SuppressWarnings("unused") + private Object writeReplace() { + ignite = Ignition.localIgnite(); + + return this; + } + + /** */ + @Override public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + final SimpleTestObject simpleTestObj = (SimpleTestObject) o; + + return val != null ? val.equals(simpleTestObj.val) : simpleTestObj.val == null; + + } + + /** */ + @Override public int hashCode() { + return val != null ? val.hashCode() : 0; + } + + /** {@inheritDoc} */ + @Override public Ignite ignite() { + return ignite; + } + } + + /** + * + */ + private static class SerializableTestObject implements Serializable, TestObject { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private String val; + + /** */ + private transient Ignite ignite; + + /** + * + */ + public SerializableTestObject() { + } + + /** + * @param val Value + */ + public SerializableTestObject(final String val) { + this.val = val; + } + + /** + * @param out Object output. + * @throws IOException If fail. + */ + private void writeObject(ObjectOutputStream out) throws IOException { + U.writeString(out, val); + + ignite = Ignition.localIgnite(); + } + + /** + * @param in Object input. + * @throws IOException If fail. + */ + private void readObject(ObjectInputStream in) throws IOException { + val = U.readString(in); + + ignite = Ignition.localIgnite(); + } + + /** {@inheritDoc} */ + @Override public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + final SerializableTestObject that = (SerializableTestObject) o; + + return val != null ? val.equals(that.val) : that.val == null; + + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return val != null ? val.hashCode() : 0; + } + + /** {@inheritDoc} */ + @Override public Ignite ignite() { + return ignite; + } + } + + /** + * + */ + private static class ExternalizableTestObject implements Externalizable, TestObject { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private String val; + + /** */ + private transient Ignite ignite; + + /** + * + */ + public ExternalizableTestObject() { + } + + /** + * @param val Value. + */ + public ExternalizableTestObject(final String val) { + this.val = val; + } + + /** {@inheritDoc} */ + @Override public void writeExternal(final ObjectOutput out) throws IOException { + U.writeString(out, val); + + ignite = Ignition.localIgnite(); + } + + /** {@inheritDoc} */ + @Override public void readExternal(final ObjectInput in) throws IOException, ClassNotFoundException { + val = U.readString(in); + + ignite = Ignition.localIgnite(); + } + + /** {@inheritDoc} */ + @Override public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + final ExternalizableTestObject that = (ExternalizableTestObject) o; + + return val != null ? val.equals(that.val) : that.val == null; + + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return val != null ? val.hashCode() : 0; + } + + /** {@inheritDoc} */ + @Override public Ignite ignite() { + return ignite; + } + } + + /** + * + */ + private static class BinarylizableTestObject implements Binarylizable, TestObject { + /** */ + private String val; + + /** */ + private transient Ignite ignite; + + /** + * + */ + public BinarylizableTestObject() { + } + + /** + * @param val Value. + */ + public BinarylizableTestObject(final String val) { + this.val = val; + } + + /** {@inheritDoc} */ + @Override public void writeBinary(final BinaryWriter writer) throws BinaryObjectException { + writer.rawWriter().writeString(val); + + ignite = Ignition.localIgnite(); + } + + /** {@inheritDoc} */ + @Override public void readBinary(final BinaryReader reader) throws BinaryObjectException { + val = reader.rawReader().readString(); + + ignite = Ignition.localIgnite(); + } + + /** {@inheritDoc} */ + @Override public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + final BinarylizableTestObject that = (BinarylizableTestObject) o; + + return val != null ? val.equals(that.val) : that.val == null; + + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return val != null ? val.hashCode() : 0; + } + + /** {@inheritDoc} */ + @Override public Ignite ignite() { + return ignite; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java index d5ea46c9628f5..035c6fefbe47c 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java @@ -2061,7 +2061,7 @@ private static class TestCustomerEventAckSpi extends TcpDiscoverySpi { if (msg instanceof TcpDiscoveryCustomEventMessage) { try { DiscoveryCustomMessage custMsg = GridTestUtils.getFieldValue( - ((TcpDiscoveryCustomEventMessage)msg).message(marsh, U.gridClassLoader()), "delegate"); + ((TcpDiscoveryCustomEventMessage)msg).message(marshaller(), U.gridClassLoader()), "delegate"); if (custMsg instanceof StartRoutineAckDiscoveryMessage) { log.info("Skip message send and stop node: " + msg); diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java index 6bb2c11bc00b6..62c2eb3b7a383 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java @@ -35,6 +35,7 @@ import org.apache.ignite.internal.IgniteSlowClientDetectionSelfTest; import org.apache.ignite.internal.MarshallerContextLockingSelfTest; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessorRendezvousSelfTest; +import org.apache.ignite.internal.processors.cache.GridLocalIgniteSerializationTest; import org.apache.ignite.internal.processors.cache.GridProjectionForCachesOnDaemonNodeSelfTest; import org.apache.ignite.internal.processors.cache.IgniteDaemonNodeMarshallerCacheTest; import org.apache.ignite.internal.processors.cache.IgniteMarshallerCacheConcurrentReadWriteTest; @@ -125,6 +126,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { GridTestUtils.addTestIfNeeded(suite, IgniteDaemonNodeMarshallerCacheTest.class, ignoredTests); suite.addTestSuite(IgniteMarshallerCacheConcurrentReadWriteTest.class); suite.addTestSuite(GridNodeMetricsLogSelfTest.class); + suite.addTestSuite(GridLocalIgniteSerializationTest.class); suite.addTestSuite(IgniteExceptionInNioWorkerSelfTest.class); From 6c3993d9d4b2126a4ef9699fdb3c0d296b03dea7 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 9 Sep 2016 13:09:40 +0300 Subject: [PATCH 125/487] IGNITE-3828: Added wrapper for DataStreamerImpl keys to minimize impact of hash code collisions. This closes #1034. --- .../datastreamer/DataStreamerImpl.java | 69 ++++++++++++++----- 1 file changed, 51 insertions(+), 18 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java index a3bae249d13ef..05e6488cc21f3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java @@ -39,6 +39,7 @@ import java.util.concurrent.atomic.AtomicReference; import javax.cache.CacheException; import javax.cache.expiry.ExpiryPolicy; + import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; @@ -139,7 +140,6 @@ public class DataStreamerImpl implements IgniteDataStreamer, Delayed /** Cache name ({@code null} for default cache). */ private final String cacheName; - /** Per-node buffer size. */ @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") private int bufSize = DFLT_PER_NODE_BUFFER_SIZE; @@ -512,7 +512,7 @@ else if (autoFlushFreq == 0) activeFuts.add(resFut); - Collection keys = + Collection keys = new GridConcurrentHashSet<>(entries.size(), U.capacity(entries.size()), 1); Collection entries0 = new ArrayList<>(entries.size()); @@ -521,7 +521,7 @@ else if (autoFlushFreq == 0) KeyCacheObject key = cacheObjProc.toCacheKeyObject(cacheObjCtx, null, entry.getKey(), true); CacheObject val = cacheObjProc.toCacheObject(cacheObjCtx, entry.getValue(), true); - keys.add(key); + keys.add(new KeyCacheObjectWrapper(key)); entries0.add(new DataStreamerEntry(key, val)); } @@ -572,13 +572,13 @@ public IgniteFuture addDataInternal(Collection e activeFuts.add(resFut); - Collection keys = null; + Collection keys = null; if (entries.size() > 1) { keys = new GridConcurrentHashSet<>(entries.size(), U.capacity(entries.size()), 1); for (DataStreamerEntry entry : entries) - keys.add(entry.getKey()); + keys.add(new KeyCacheObjectWrapper(entry.getKey())); } load0(entries, resFut, keys, 0); @@ -641,7 +641,7 @@ public void ioPolicyResolver(IgniteClosure ioPlcRslvr) { private void load0( Collection entries, final GridFutureAdapter resFut, - @Nullable final Collection activeKeys, + @Nullable final Collection activeKeys, final int remaps ) { assert entries != null; @@ -729,7 +729,7 @@ else if (rcvr != null) if (activeKeys != null) { for (DataStreamerEntry e : entriesForNode) - activeKeys.remove(e.getKey()); + activeKeys.remove(new KeyCacheObjectWrapper(e.getKey())); if (activeKeys.isEmpty()) resFut.onDone(); @@ -1103,7 +1103,7 @@ private long nextFlushTime() { * @throws org.apache.ignite.plugin.security.SecurityException If permissions are not enough for streaming. */ private void checkSecurityPermission(SecurityPermission perm) - throws org.apache.ignite.plugin.security.SecurityException{ + throws org.apache.ignite.plugin.security.SecurityException { if (!ctx.security().enabled()) return; @@ -1172,8 +1172,8 @@ private class Buffer { * @param newEntries Infos. * @param topVer Topology version. * @param lsnr Listener for the operation future. - * @throws IgniteInterruptedCheckedException If failed. * @return Future for operation. + * @throws IgniteInterruptedCheckedException If failed. */ @Nullable GridFutureAdapter update(Iterable newEntries, AffinityTopologyVersion topVer, @@ -1221,7 +1221,6 @@ private List newEntries() { /** * @return Future if any submitted. - * * @throws IgniteInterruptedCheckedException If thread has been interrupted. */ @Nullable IgniteInternalFuture flush() throws IgniteInterruptedCheckedException { @@ -1273,13 +1272,12 @@ private List newEntries() { private void incrementActiveTasks() throws IgniteInterruptedCheckedException { if (timeout == DFLT_UNLIMIT_TIMEOUT) U.acquire(sem); - else - if (!U.tryAcquire(sem, timeout, TimeUnit.MILLISECONDS)) { - if (log.isDebugEnabled()) - log.debug("Failed to add parallel operation."); + else if (!U.tryAcquire(sem, timeout, TimeUnit.MILLISECONDS)) { + if (log.isDebugEnabled()) + log.debug("Failed to add parallel operation."); - throw new IgniteDataStreamerTimeoutException("Data streamer exceeded timeout when starts parallel operation."); - } + throw new IgniteDataStreamerTimeoutException("Data streamer exceeded timeout when starts parallel operation."); + } } /** @@ -1307,7 +1305,8 @@ private void submit(final Collection entries, try { incrementActiveTasks(); - } catch (IgniteDataStreamerTimeoutException e) { + } + catch (IgniteDataStreamerTimeoutException e) { curFut.onDone(e); throw e; } @@ -1574,7 +1573,7 @@ private DataStreamerPda(Object... objs) { if (depCls != null) cls0 = depCls; else { - for (Iterator it = objs.iterator(); (cls0 == null || U.isJdk(cls0)) && it.hasNext();) { + for (Iterator it = objs.iterator(); (cls0 == null || U.isJdk(cls0)) && it.hasNext(); ) { Object o = it.next(); if (o != null) @@ -1696,4 +1695,38 @@ private static class DefaultIoPolicyResolver implements IgniteClosure Date: Tue, 13 Sep 2016 14:57:36 +0300 Subject: [PATCH 126/487] IGNITE-3831: CPP: Added distributed joins query example. This closes #1035. --- .../query-example/src/query_example.cpp | 54 +++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/modules/platforms/cpp/examples/query-example/src/query_example.cpp b/modules/platforms/cpp/examples/query-example/src/query_example.cpp index 1bbf21b0c2728..9bf3e52ee873b 100644 --- a/modules/platforms/cpp/examples/query-example/src/query_example.cpp +++ b/modules/platforms/cpp/examples/query-example/src/query_example.cpp @@ -42,6 +42,57 @@ const char* PERSON_CACHE = "Person"; /** Persons type name. */ const char* PERSON_TYPE = "Person"; +/** + * Example for SQL queries based on all employees working for a specific + * organization (query uses distributed join). + */ +void DoSqlQueryWithDistributedJoin() +{ + typedef std::vector< CacheEntry > ResVector; + + Cache cache = Ignition::Get().GetCache(PERSON_CACHE); + + // SQL clause query which joins on 2 types to select people for a specific organization. + std::string joinSql( + "from Person, \"Organization\".Organization as org " + "where Person.orgId = org._key " + "and lower(org.name) = lower(?)"); + + SqlQuery qry("Person", joinSql); + + qry.AddArgument("ApacheIgnite"); + + // Enable distributed joins for query. + qry.SetDistributedJoins(true); + + // Execute queries for find employees for different organizations. + ResVector igniters; + cache.Query(qry).GetAll(igniters); + + // Printing first result set. + std::cout << "Following people are 'ApacheIgnite' employees (distributed join): " << std::endl; + + for (ResVector::const_iterator i = igniters.begin(); i != igniters.end(); ++i) + std::cout << i->GetKey() << " : " << i->GetValue().ToString() << std::endl; + + std::cout << std::endl; + + qry = SqlQuery("Person", joinSql); + + qry.AddArgument("Other"); + + ResVector others; + cache.Query(qry).GetAll(others); + + // Printing second result set. + std::cout << "Following people are 'Other' employees (distributed join): " << std::endl; + + for (ResVector::const_iterator i = others.begin(); i != others.end(); ++i) + std::cout << i->GetKey() << " : " << i->GetValue().ToString() << std::endl; + + std::cout << std::endl; +} + /** * Example for SQL-based fields queries that return only required * fields instead of whole key-value pairs. @@ -374,6 +425,9 @@ int main() // Example for SQL-based querying employees for a given organization (includes SQL join). DoSqlQueryWithJoin(); + // Example for SQL-based querying employees for a given organization (includes distributed SQL join). + DoSqlQueryWithDistributedJoin(); + // Example for TEXT-based querying for a given string in peoples resumes. DoTextQuery(); From 0852baeef4c828e4832b34edf24c0869619c5745 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Tue, 13 Sep 2016 16:50:14 +0300 Subject: [PATCH 127/487] IGNITE-3887: IGFS: IgfsPath API cleanup. --- .../java/org/apache/ignite/igfs/IgfsPath.java | 14 ++++++++++++- .../apache/ignite/igfs/IgfsPathSummary.java | 3 +-- .../local/LocalIgfsSecondaryFileSystem.java | 2 +- .../igfs/common/IgfsControlResponse.java | 19 +++++------------ .../internal/igfs/common/IgfsMarshaller.java | 14 +++---------- .../processors/igfs/IgfsFileImpl.java | 5 +---- .../internal/processors/igfs/IgfsImpl.java | 4 ++-- .../processors/igfs/IgfsModeResolver.java | 3 ++- .../internal/processors/igfs/IgfsPathIds.java | 2 +- .../internal/processors/igfs/IgfsPaths.java | 5 +---- .../internal/processors/igfs/IgfsUtils.java | 21 +++++++++++++++---- .../apache/ignite/igfs/IgfsPathSelfTest.java | 6 ------ .../igfs/IgfsAbstractBaseSelfTest.java | 2 +- .../processors/igfs/IgfsAbstractSelfTest.java | 12 +++++------ .../igfs/IgfsDataManagerSelfTest.java | 6 +++--- .../igfs/IgfsDualAbstractSelfTest.java | 14 ++++++------- .../igfs/IgfsMetaManagerSelfTest.java | 4 ++-- .../igfs/IgfsModeResolverSelfTest.java | 4 ++-- .../processors/igfs/IgfsStreamsSelfTest.java | 4 +--- 19 files changed, 68 insertions(+), 76 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPath.java b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPath.java index c70527490d993..d01e1db050767 100644 --- a/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPath.java +++ b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPath.java @@ -62,9 +62,15 @@ public final class IgfsPath implements Comparable, Externalizable, Bin /** URI representing this path. Should never change after object creation or de-serialization. */ private String path; + /** Root path. */ + public static final IgfsPath ROOT = new IgfsPath(SLASH); + /** * Constructs default root path. + * + * @deprecated Use {@link #ROOT} instead. */ + @Deprecated public IgfsPath() { path = SLASH; } @@ -146,9 +152,11 @@ public String name() { * Returns a root for this path. * * @return Root for this path. + * @deprecated Use {@link #ROOT} instead. */ + @Deprecated public IgfsPath root() { - return new IgfsPath(); + return ROOT; } /** @@ -244,7 +252,10 @@ public boolean isSubDirectoryOf(IgfsPath path) { * * @param path Path to check. * @return {@code True} if paths are identical. + * @deprecated Use {@link #equals(Object)} instead. */ + @SuppressWarnings("unused") + @Deprecated public boolean isSame(IgfsPath path) { A.notNull(path, "path"); @@ -252,6 +263,7 @@ public boolean isSame(IgfsPath path) { } /** {@inheritDoc} */ + @SuppressWarnings("NullableProblems") @Override public int compareTo(IgfsPath o) { return path.compareTo(o.path); } diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPathSummary.java b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPathSummary.java index 7d60b9509f27f..0eff4b73e38c3 100644 --- a/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPathSummary.java +++ b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPathSummary.java @@ -137,8 +137,7 @@ public void path(IgfsPath path) { dirCnt = in.readInt(); totalLen = in.readLong(); - path = new IgfsPath(); - path.readExternal(in); + path = IgfsUtils.readPath(in); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java index 36080f2cbf802..c84fcf2adefb4 100644 --- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java +++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java @@ -303,7 +303,7 @@ private boolean mkdirs0(@Nullable File dir) { /** {@inheritDoc} */ @Override public long usedSpaceSize() { - Path p = fileForPath(new IgfsPath("/")).toPath(); + Path p = fileForPath(IgfsPath.ROOT).toPath(); try { LocalFileSystemSizeVisitor visitor = new LocalFileSystemSizeVisitor(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsControlResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsControlResponse.java index ce7107d5d696c..595f3c4de75d5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsControlResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsControlResponse.java @@ -40,6 +40,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse; import org.apache.ignite.internal.processors.igfs.IgfsInputStreamDescriptor; import org.apache.ignite.internal.processors.igfs.IgfsStatus; +import org.apache.ignite.internal.processors.igfs.IgfsUtils; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.S; @@ -492,13 +493,8 @@ public void readExternal(ObjectInput in) throws IOException, ClassNotFoundExcept case RES_TYPE_IGFS_PATH: { boolean hasVal = in.readBoolean(); - if (hasVal) { - IgfsPath path = new IgfsPath(); - - path.readExternal(in); - - res = path; - } + if (hasVal) + res = IgfsUtils.readPath(in); break; } @@ -603,13 +599,8 @@ public void readExternal(ObjectInput in) throws IOException, ClassNotFoundExcept if (size >= 0) { paths = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - IgfsPath path = new IgfsPath(); - - path.readExternal(in); - - paths.add(path); - } + for (int i = 0; i < size; i++) + paths.add(IgfsUtils.readPath(in)); } res = paths; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java index 1dba21cfbfdbd..91656fd88b1a2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java @@ -23,6 +23,7 @@ import java.util.Arrays; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.igfs.IgfsPath; +import org.apache.ignite.internal.processors.igfs.IgfsUtils; import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; @@ -331,21 +332,12 @@ private void writePath(ObjectOutput out, @Nullable IgfsPath path) throws IOExcep } /** - * Reads IGFS path from data input that was written by {@link #writePath(ObjectOutput, org.apache.ignite.igfs.IgfsPath)} - * method. + * Reads IGFS path from data input that was written by {@link #writePath(ObjectOutput, IgfsPath)} method. * * @param in Data input. * @return Written path or {@code null}. */ @Nullable private IgfsPath readPath(ObjectInput in) throws IOException { - if(in.readBoolean()) { - IgfsPath path = new IgfsPath(); - - path.readExternal(in); - - return path; - } - - return null; + return in.readBoolean() ? IgfsUtils.readPath(in) : null; } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java index 984c8f5b83f84..98777bea1b2c7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java @@ -231,10 +231,7 @@ public IgniteUuid fileId() { * @param in Data input. */ @Override public void readExternal(ObjectInput in) throws IOException { - path = new IgfsPath(); - - path.readExternal(in); - + path = IgfsUtils.readPath(in); blockSize = in.readInt(); grpBlockSize = in.readLong(); len = in.readLong(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 636b4a97b502e..273e67d4a93d3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -388,7 +388,7 @@ private boolean enterBusy() { boolean await = false; for (IgfsPath path : paths) { - if (workerPath.isSubDirectoryOf(path) || workerPath.isSame(path)) { + if (workerPath.isSubDirectoryOf(path) || F.eq(workerPath, path)) { await = true; break; @@ -749,7 +749,7 @@ else if (val) if (log.isDebugEnabled()) log.debug("Deleting file [path=" + path + ", recursive=" + recursive + ']'); - if (IgfsPath.SLASH.equals(path.toString())) + if (F.eq(IgfsPath.ROOT, path)) return false; IgfsMode mode = resolveMode(path); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java index 33b835f9f276b..d4e248d6e3f4f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolver.java @@ -26,6 +26,7 @@ import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap; +import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; import org.jetbrains.annotations.Nullable; @@ -84,7 +85,7 @@ public IgfsMode resolveMode(IgfsPath path) { if (mode == null) { for (T2 entry : modes) { - if (path.isSame(entry.getKey()) || path.isSubDirectoryOf(entry.getKey())) { + if (F.eq(path, entry.getKey()) || path.isSubDirectoryOf(entry.getKey())) { // As modes ordered from most specific to least specific first mode found is ours. mode = entry.getValue(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java index b710ba2ee8072..9b30913d4c032 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java @@ -163,7 +163,7 @@ public boolean isLastIndex(int idx) { * @return Path of the last existing element. */ public IgfsPath lastExistingPath() { - IgfsPath path = new IgfsPath(); + IgfsPath path = IgfsPath.ROOT; for (int i = 1; i <= lastExistingIdx; i++) path = new IgfsPath(path, parts[i]); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java index 2bdb23b1c64b0..72b5a19d8244d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java @@ -24,7 +24,6 @@ import java.io.ObjectInput; import java.io.ObjectOutput; import java.util.ArrayList; -import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.igfs.IgfsMode; @@ -144,9 +143,7 @@ public IgfsMode defaultMode() { pathModes = new ArrayList<>(size); for (int i = 0; i < size; i++) { - IgfsPath path = new IgfsPath(); - - path.readExternal(in); + IgfsPath path = IgfsUtils.readPath(in); pathModes.add(new T2<>(path, IgfsMode.fromOrdinal(in.readByte()))); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java index 2e79a98d8e355..4453dba6fe45f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java @@ -52,6 +52,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.io.ObjectInput; import java.lang.reflect.Constructor; import java.util.ArrayList; import java.util.Collections; @@ -126,9 +127,6 @@ public class IgfsUtils { /** Flag: this is a file. */ private static final byte FLAG_FILE = 0x2; - /** - * Static initializer. - */ static { TRASH_IDS = new IgniteUuid[TRASH_CONCURRENCY]; @@ -749,6 +747,21 @@ public static void writePath(BinaryRawWriter writer, @Nullable IgfsPath path) { return null; } + /** + * Read non-null path from the input. + * + * @param in Input. + * @return IGFS path. + * @throws IOException If failed. + */ + public static IgfsPath readPath(ObjectInput in) throws IOException { + IgfsPath res = new IgfsPath(); + + res.readExternal(in); + + return res; + } + /** * Write IgfsFileAffinityRange. * @@ -878,7 +891,7 @@ public static ArrayList> preparePathModes(final IgfsMode ArrayList> resModes = new ArrayList<>(modes.size() + 1); - resModes.add(new T2<>(new IgfsPath("/"), dfltMode)); + resModes.add(new T2<>(IgfsPath.ROOT, dfltMode)); for (T2 mode : modes) { assert mode.getKey() != null; diff --git a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsPathSelfTest.java b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsPathSelfTest.java index eb0f3832219a8..30923745d629f 100644 --- a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsPathSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsPathSelfTest.java @@ -97,12 +97,6 @@ public void testConstructors() throws Exception { URI uri = URI.create(pathStr); IgfsPath path = new IgfsPath(uri); - assertNotNull(new IgfsPath(uri)); - assertNotNull(new IgfsPath(pathStr)); - assertNotNull(new IgfsPath("/")); - assertNotNull(new IgfsPath(path, pathStr)); - assertNotNull(new IgfsPath()); - Class nullUri = URI.class; Class nullStr = String.class; Class nullPath = IgfsPath.class; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java index 03f24a4334056..3f62cf55013c4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java @@ -342,7 +342,7 @@ protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Except @Override protected void afterTest() throws Exception { clear(igfs, igfsSecondary); - assert igfs.listFiles(new IgfsPath("/")).isEmpty(); + assert igfs.listFiles(IgfsPath.ROOT).isEmpty(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java index 89979adc5f63b..a115e03c7f981 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java @@ -265,7 +265,7 @@ public void testMoveFile() throws Exception { public void testMoveFileDestinationRoot() throws Exception { create(igfs, paths(DIR, SUBDIR), paths(FILE)); - igfs.rename(FILE, new IgfsPath()); + igfs.rename(FILE, IgfsPath.ROOT); checkExist(igfs, igfsSecondary, new IgfsPath("/" + FILE.name())); checkNotExist(igfs, igfsSecondary, FILE); @@ -357,7 +357,7 @@ public void testMoveDirectory() throws Exception { public void testMoveDirectoryDestinationRoot() throws Exception { create(igfs, paths(DIR, SUBDIR, SUBSUBDIR), null); - igfs.rename(SUBSUBDIR, new IgfsPath()); + igfs.rename(SUBSUBDIR, IgfsPath.ROOT); checkExist(igfs, igfsSecondary, new IgfsPath("/" + SUBSUBDIR.name())); checkNotExist(igfs, igfsSecondary, SUBSUBDIR); @@ -745,13 +745,11 @@ public void testRootPropertiesPersistAfterFormat() throws Exception { * @throws Exception If failed. */ private void checkRootPropertyUpdate(String prop, String setVal, String expGetVal) throws Exception { - final IgfsPath rootPath = new IgfsPath("/"); - - igfs.update(rootPath, Collections.singletonMap(prop, setVal)); + igfs.update(IgfsPath.ROOT, Collections.singletonMap(prop, setVal)); igfs.format(); - IgfsFile file = igfs.info(rootPath); + IgfsFile file = igfs.info(IgfsPath.ROOT); assert file != null; @@ -2334,7 +2332,7 @@ private void checkDeadlocks(final int lvlCnt, final int childrenDirPerLvl, final Queue> queue = new ArrayDeque<>(); - queue.add(F.t(0, new IgfsPath())); // Add root directory. + queue.add(F.t(0, IgfsPath.ROOT)); // Add root directory. while (!queue.isEmpty()) { IgniteBiTuple entry = queue.poll(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java index 70d6b99962190..a98c3a1908aa8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java @@ -165,7 +165,7 @@ protected CacheConfiguration cacheConfiguration(String cacheName) { @SuppressWarnings("ConstantConditions") public void testDataStoring() throws Exception { for (int i = 0; i < 10; i++) { - IgfsPath path = new IgfsPath(); + IgfsPath path = IgfsPath.ROOT; long t = System.currentTimeMillis(); @@ -250,7 +250,7 @@ public void testDataStoringRemainder() throws Exception { final int blockSize = IGFS_BLOCK_SIZE; for (int i = 0; i < 10; i++) { - IgfsPath path = new IgfsPath(); + IgfsPath path = IgfsPath.ROOT; long t = System.currentTimeMillis(); @@ -341,7 +341,7 @@ public void testDataStoringFlush() throws Exception { final int writesCnt = 64; for (int i = 0; i < 10; i++) { - IgfsPath path = new IgfsPath(); + IgfsPath path = IgfsPath.ROOT; long t = System.currentTimeMillis(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java index 02027d6b609bc..57bc4f3f087f8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java @@ -23,6 +23,7 @@ import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.U; @@ -334,7 +335,7 @@ public void testMoveFileDestinationRootSourceMissingPartially() throws Exception create(igfsSecondary, paths(DIR, SUBDIR), paths(FILE)); create(igfs, paths(DIR), null); - igfs.rename(FILE, new IgfsPath()); + igfs.rename(FILE, IgfsPath.ROOT); checkExist(igfs, SUBDIR); checkExist(igfs, igfsSecondary, new IgfsPath("/" + FILE.name())); @@ -350,7 +351,7 @@ public void testMoveFileDestinationRootSourceMissing() throws Exception { create(igfsSecondary, paths(DIR, SUBDIR), paths(FILE)); create(igfs, null, null); - igfs.rename(FILE, new IgfsPath()); + igfs.rename(FILE, IgfsPath.ROOT); checkExist(igfs, DIR, SUBDIR); checkExist(igfs, igfsSecondary, new IgfsPath("/" + FILE.name())); @@ -701,7 +702,7 @@ public void testMoveDirectoryDestinationRootSourceMissingPartially() throws Exce create(igfsSecondary, paths(DIR, SUBDIR, SUBSUBDIR), null); create(igfs, paths(DIR), null); - igfs.rename(SUBSUBDIR, new IgfsPath()); + igfs.rename(SUBSUBDIR, IgfsPath.ROOT); checkExist(igfs, SUBDIR); checkExist(igfs, igfsSecondary, new IgfsPath("/" + SUBSUBDIR.name())); @@ -717,7 +718,7 @@ public void testMoveDirectoryDestinationRootSourceMissing() throws Exception { create(igfsSecondary, paths(DIR, SUBDIR, SUBSUBDIR), null); create(igfs, null, null); - igfs.rename(SUBSUBDIR, new IgfsPath()); + igfs.rename(SUBSUBDIR, IgfsPath.ROOT); checkExist(igfs, DIR, SUBDIR); checkExist(igfs, igfsSecondary, new IgfsPath("/" + SUBSUBDIR.name())); @@ -1510,8 +1511,7 @@ private T2 checkParentListingTime(IgfsSecondaryFileSystem fs, IgfsPa T2 t0 = new T2<>(f0.accessTime(), f0.modificationTime()); // Root cannot be seen through the parent listing: - if (!p.isSame(p.root())) { - + if (!F.eq(IgfsPath.ROOT, p)) { assertNotNull(f0); Collection listing = fs.listFiles(p.parent()); @@ -1519,7 +1519,7 @@ private T2 checkParentListingTime(IgfsSecondaryFileSystem fs, IgfsPa IgfsFile f1 = null; for (IgfsFile fi : listing) { - if (fi.path().isSame(p)) { + if (F.eq(fi.path(), p)) { f1 = fi; break; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java index 6053d3bc0f9f3..a4100ea628342 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java @@ -266,7 +266,7 @@ public void testStructure() throws Exception { } // Validate 'file ID' operations. - assertEquals(ROOT_ID, mgr.fileId(new IgfsPath("/"))); + assertEquals(ROOT_ID, mgr.fileId(IgfsPath.ROOT)); assertEquals(a.id(), mgr.fileId(new IgfsPath("/a"))); assertEquals(b.id(), mgr.fileId(new IgfsPath("/a/b"))); assertEquals(f1.id(), mgr.fileId(new IgfsPath("/f1"))); @@ -285,7 +285,7 @@ public void testStructure() throws Exception { assertNull(mgr.fileId(a.id(), "f5")); assertNull(mgr.fileId(b.id(), "f6")); - assertEquals(Arrays.asList(ROOT_ID), mgr.fileIds(new IgfsPath("/"))); + assertEquals(Arrays.asList(ROOT_ID), mgr.fileIds(IgfsPath.ROOT)); assertEquals(Arrays.asList(ROOT_ID, a.id()), mgr.fileIds(new IgfsPath("/a"))); assertEquals(Arrays.asList(ROOT_ID, a.id(), b.id()), mgr.fileIds(new IgfsPath("/a/b"))); assertEquals(Arrays.asList(ROOT_ID, f1.id()), mgr.fileIds(new IgfsPath("/f1"))); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java index bd7e413e1850e..79326604998d3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsModeResolverSelfTest.java @@ -69,7 +69,7 @@ public void testCanContain() throws Exception { * @throws Exception If failed. */ public void testResolve() throws Exception { - assertEquals(DUAL_SYNC, reslvr.resolveMode(new IgfsPath("/"))); + assertEquals(DUAL_SYNC, reslvr.resolveMode(IgfsPath.ROOT)); assertEquals(DUAL_SYNC, reslvr.resolveMode(new IgfsPath("/a"))); assertEquals(DUAL_SYNC, reslvr.resolveMode(new IgfsPath("/a/1"))); @@ -161,7 +161,7 @@ public void testDualParentsWithPrimaryChild() throws Exception { assertEquals(set, new HashSet() {{ add(new IgfsPath("/a/b")); add(new IgfsPath("/a/b/x")); - add(new IgfsPath("/")); + add(IgfsPath.ROOT); }}); set = new HashSet<>(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java index 44560e3821fd1..8c6cf574ce159 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java @@ -47,7 +47,6 @@ import java.io.InputStream; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.List; import java.util.Random; import java.util.UUID; @@ -192,13 +191,12 @@ public void testConfiguration() throws IgniteCheckedException { * @throws Exception In case of exception. */ public void testCreateFile() throws Exception { - IgfsPath root = new IgfsPath("/"); IgfsPath path = new IgfsPath("/asdf"); long max = 100L * CFG_BLOCK_SIZE / WRITING_THREADS_CNT; for (long size = 0; size <= max; size = size * 15 / 10 + 1) { - assertTrue(F.isEmpty(fs.listPaths(root))); + assertTrue(F.isEmpty(fs.listPaths(IgfsPath.ROOT))); testCreateFile(path, size, new Random().nextInt()); } From 43f65fe91d85ba8f3c16e13bdf5ea815d3a48d71 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Tue, 13 Sep 2016 17:30:47 +0300 Subject: [PATCH 128/487] IGNITE-3613 IGFS: Fixed IgfsImpl.size() to take secondary fie system in count. This closes #1023. --- .../internal/processors/igfs/IgfsImpl.java | 69 +++++++++---------- .../igfs/IgfsDualAbstractSelfTest.java | 14 ++++ 2 files changed, 47 insertions(+), 36 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 273e67d4a93d3..2720f2406427d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -598,16 +598,7 @@ else if (val) if (log.isDebugEnabled()) log.debug("Calculating path summary: " + path); - IgniteUuid fileId = meta.fileId(path); - - if (fileId == null) - throw new IgfsPathNotFoundException("Failed to get path summary (path not found): " + path); - - IgfsPathSummary sum = new IgfsPathSummary(path); - - summary0(fileId, sum); - - return sum; + return summary0(path); } }); } @@ -1259,9 +1250,7 @@ private IgfsOutputStream create0( @Override public IgfsMetrics metrics() { return safeOp(new Callable() { @Override public IgfsMetrics call() throws Exception { - IgfsPathSummary sum = new IgfsPathSummary(); - - summary0(IgfsUtils.ROOT_ID, sum); + IgfsPathSummary sum = summary0(IgfsPath.ROOT); long secondarySpaceSize = 0; @@ -1310,44 +1299,52 @@ private IgfsOutputStream create0( return safeOp(new Callable() { @Override public Long call() throws Exception { - IgniteUuid nextId = meta.fileId(path); + return summary0(path).totalLength(); + } + }); + } - if (nextId == null) - return 0L; + /** + * Get summary for path. + * + * @param path Path. + * @return Summary. + * @throws IgniteCheckedException If failed. + */ + private IgfsPathSummary summary0(IgfsPath path) throws IgniteCheckedException { + IgfsFile info = info(path); - IgfsPathSummary sum = new IgfsPathSummary(path); + if (info == null) + throw new IgfsPathNotFoundException("Failed to get path summary (path not found): " + path); - summary0(nextId, sum); + IgfsPathSummary sum = new IgfsPathSummary(path); - return sum.totalLength(); - } - }); + summaryRecursive(info, sum); + + return sum; } /** * Calculates size of directory or file for given ID. * - * @param fileId File ID. + * @param file IGFS File object. * @param sum Summary object that will collect information. * @throws IgniteCheckedException If failed. */ - private void summary0(IgniteUuid fileId, IgfsPathSummary sum) throws IgniteCheckedException { + private void summaryRecursive(IgfsFile file, IgfsPathSummary sum) throws IgniteCheckedException { + assert file != null; assert sum != null; - IgfsEntryInfo info = meta.info(fileId); - - if (info != null) { - if (info.isDirectory()) { - if (!IgfsUtils.ROOT_ID.equals(info.id())) - sum.directoriesCount(sum.directoriesCount() + 1); + if (file.isDirectory()) { + if (!F.eq(IgfsPath.ROOT, file.path())) + sum.directoriesCount(sum.directoriesCount() + 1); - for (IgfsListingEntry entry : info.listing().values()) - summary0(entry.fileId(), sum); - } - else { - sum.filesCount(sum.filesCount() + 1); - sum.totalLength(sum.totalLength() + info.length()); - } + for (IgfsFile childFile : listFiles(file.path())) + summaryRecursive(childFile, sum); + } + else { + sum.filesCount(sum.filesCount() + 1); + sum.totalLength(sum.totalLength() + file.length()); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java index 57bc4f3f087f8..742d20c76ec7d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java @@ -1599,4 +1599,18 @@ public void testSetTimesMissingPartially() throws Exception { // No-op. } } + + /** + * + * @throws Exception If failed. + */ + public void testSecondarySize() throws Exception { + igfs.mkdirs(SUBDIR); + + createFile(igfsSecondary, FILE, chunk); + createFile(igfsSecondary, new IgfsPath(SUBDIR, "file2"), chunk); + + assertEquals(chunk.length, igfs.size(FILE)); + assertEquals(chunk.length * 2, igfs.size(SUBDIR)); + } } \ No newline at end of file From 16c5a715889322d31ed95a2a29206d3a909aa7b7 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Tue, 13 Sep 2016 18:00:31 +0300 Subject: [PATCH 129/487] IGNITE-3890: IGFS: Simplified IgfsInputStream hierarchy. --- .../processors/igfs/IgfsAsyncImpl.java | 7 +-- .../internal/processors/igfs/IgfsEx.java | 10 ---- .../internal/processors/igfs/IgfsImpl.java | 11 ++-- .../igfs/IgfsInputStreamAdapter.java | 51 ------------------- .../processors/igfs/IgfsInputStreamImpl.java | 17 +++++-- .../processors/igfs/IgfsIpcHandler.java | 7 +-- .../igfs/IgfsSecondaryFileSystemImpl.java | 2 +- .../processors/igfs/IgfsMetricsSelfTest.java | 21 +++----- .../internal/processors/igfs/IgfsMock.java | 8 +-- .../hadoop/igfs/HadoopIgfsInProc.java | 12 ++--- .../hadoop/HadoopCommandLineTest.java | 4 +- 11 files changed, 47 insertions(+), 103 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamAdapter.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java index 07b070eae8de2..743601e29974f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java @@ -25,6 +25,7 @@ import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.igfs.IgfsBlockLocation; import org.apache.ignite.igfs.IgfsFile; +import org.apache.ignite.igfs.IgfsInputStream; import org.apache.ignite.igfs.IgfsMetrics; import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsOutputStream; @@ -125,18 +126,18 @@ public IgfsAsyncImpl(IgfsImpl igfs) { } /** {@inheritDoc} */ - @Override public IgfsInputStreamAdapter open(IgfsPath path, int bufSize, + @Override public IgfsInputStream open(IgfsPath path, int bufSize, int seqReadsBeforePrefetch) { return igfs.open(path, bufSize, seqReadsBeforePrefetch); } /** {@inheritDoc} */ - @Override public IgfsInputStreamAdapter open(IgfsPath path) { + @Override public IgfsInputStream open(IgfsPath path) { return igfs.open(path); } /** {@inheritDoc} */ - @Override public IgfsInputStreamAdapter open(IgfsPath path, int bufSize) { + @Override public IgfsInputStream open(IgfsPath path, int bufSize) { return igfs.open(path, bufSize); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java index 9760f435f22d7..05e157d413c7c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java @@ -49,16 +49,6 @@ public interface IgfsEx extends IgniteFileSystem { */ public IgfsPaths proxyPaths(); - /** {@inheritDoc} */ - @Override public IgfsInputStreamAdapter open(IgfsPath path, int bufSize, int seqReadsBeforePrefetch) - throws IgniteException; - - /** {@inheritDoc} */ - @Override public IgfsInputStreamAdapter open(IgfsPath path) throws IgniteException; - - /** {@inheritDoc} */ - @Override public IgfsInputStreamAdapter open(IgfsPath path, int bufSize) throws IgniteException; - /** * Gets global space counters. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 2720f2406427d..2c1f0f3889c77 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -36,6 +36,7 @@ import org.apache.ignite.igfs.IgfsBlockLocation; import org.apache.ignite.igfs.IgfsException; import org.apache.ignite.igfs.IgfsFile; +import org.apache.ignite.igfs.IgfsInputStream; import org.apache.ignite.igfs.IgfsInvalidPathException; import org.apache.ignite.igfs.IgfsMetrics; import org.apache.ignite.igfs.IgfsMode; @@ -948,24 +949,24 @@ private IgfsEntryInfo primaryInfoForListing(IgfsPath path) throws IgniteCheckedE } /** {@inheritDoc} */ - @Override public IgfsInputStreamAdapter open(IgfsPath path) { + @Override public IgfsInputStream open(IgfsPath path) { return open(path, cfg.getStreamBufferSize(), cfg.getSequentialReadsBeforePrefetch()); } /** {@inheritDoc} */ - @Override public IgfsInputStreamAdapter open(IgfsPath path, int bufSize) { + @Override public IgfsInputStream open(IgfsPath path, int bufSize) { return open(path, bufSize, cfg.getSequentialReadsBeforePrefetch()); } /** {@inheritDoc} */ - @Override public IgfsInputStreamAdapter open(final IgfsPath path, final int bufSize, + @Override public IgfsInputStream open(final IgfsPath path, final int bufSize, final int seqReadsBeforePrefetch) { A.notNull(path, "path"); A.ensure(bufSize >= 0, "bufSize >= 0"); A.ensure(seqReadsBeforePrefetch >= 0, "seqReadsBeforePrefetch >= 0"); - return safeOp(new Callable() { - @Override public IgfsInputStreamAdapter call() throws Exception { + return safeOp(new Callable() { + @Override public IgfsInputStream call() throws Exception { if (log.isDebugEnabled()) log.debug("Open file for reading [path=" + path + ", bufSize=" + bufSize + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamAdapter.java deleted file mode 100644 index 07ab051daf927..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamAdapter.java +++ /dev/null @@ -1,51 +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.igfs; - -import org.apache.ignite.igfs.IgfsInputStream; -import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable; - -import java.io.IOException; - -/** - * Implementation adapter providing necessary methods. - */ -public abstract class IgfsInputStreamAdapter extends IgfsInputStream - implements IgfsSecondaryFileSystemPositionedReadable { - /** {@inheritDoc} */ - @Override public long length() { - return fileInfo().length(); - } - - /** - * Gets file info for opened file. - * - * @return File info. - */ - public abstract IgfsEntryInfo fileInfo(); - - /** - * Reads bytes from given position. - * - * @param pos Position to read from. - * @param len Number of bytes to read. - * @return Array of chunks with respect to chunk file representation. - * @throws IOException If read failed. - */ - public abstract byte[][] readChunks(long pos, int len) throws IOException; -} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java index ca2f9f7ca29b8..f20a423fd6a0c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java @@ -46,7 +46,7 @@ /** * Input stream to read data from grid cache with separate blocks. */ -public class IgfsInputStreamImpl extends IgfsInputStreamAdapter { +public class IgfsInputStreamImpl extends IgfsInputStream implements IgfsSecondaryFileSystemPositionedReadable { /** Empty chunks result. */ private static final byte[][] EMPTY_CHUNKS = new byte[0][]; @@ -158,8 +158,8 @@ public synchronized long bytes() { } /** {@inheritDoc} */ - @Override public IgfsEntryInfo fileInfo() { - return fileInfo; + @Override public long length() { + return fileInfo.length(); } /** {@inheritDoc} */ @@ -234,9 +234,16 @@ public synchronized long bytes() { return readFromStore(pos, buf, off, len); } - /** {@inheritDoc} */ + /** + * Reads bytes from given position. + * + * @param pos Position to read from. + * @param len Number of bytes to read. + * @return Array of chunks with respect to chunk file representation. + * @throws IOException If read failed. + */ @SuppressWarnings("IfMayBeConditional") - @Override public synchronized byte[][] readChunks(long pos, int len) throws IOException { + public synchronized byte[][] readChunks(long pos, int len) throws IOException { // Readable bytes in the file, starting from the specified position. long readable = fileInfo.length() - pos; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java index a888aff83eee5..6047604892de9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java @@ -20,6 +20,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.igfs.IgfsInputStream; import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; import org.apache.ignite.igfs.IgfsOutOfSpaceException; import org.apache.ignite.igfs.IgfsOutputStream; @@ -381,7 +382,7 @@ private IgfsMessage processPathControlRequest(final IgfsClientSession ses, final break; case OPEN_READ: { - IgfsInputStreamAdapter igfsIn = !req.flag() ? igfs.open(req.path(), bufSize) : + IgfsInputStream igfsIn = !req.flag() ? igfs.open(req.path(), bufSize) : igfs.open(req.path(), bufSize, req.sequentialReadsBeforePrefetch()); long streamId = registerResource(ses, igfsIn); @@ -390,7 +391,7 @@ private IgfsMessage processPathControlRequest(final IgfsClientSession ses, final log.debug("Opened IGFS input stream for file read [igfsName=" + igfs.name() + ", path=" + req.path() + ", streamId=" + streamId + ", ses=" + ses + ']'); - res.response(new IgfsInputStreamDescriptor(streamId, igfsIn.fileInfo().length())); + res.response(new IgfsInputStreamDescriptor(streamId, igfsIn.length())); break; } @@ -514,7 +515,7 @@ private IgfsMessage processStreamControlRequest(IgfsClientSession ses, IgfsIpcCo long pos = req.position(); int size = req.length(); - IgfsInputStreamAdapter igfsIn = (IgfsInputStreamAdapter)resource(ses, rsrcId); + IgfsInputStreamImpl igfsIn = (IgfsInputStreamImpl)resource(ses, rsrcId); if (igfsIn == null) throw new IgniteCheckedException("Input stream not found (already closed?): " + rsrcId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java index 453682ca58764..526e60db30854 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java @@ -86,7 +86,7 @@ class IgfsSecondaryFileSystemImpl implements IgfsSecondaryFileSystemV2 { /** {@inheritDoc} */ @Override public IgfsSecondaryFileSystemPositionedReadable open(IgfsPath path, int bufSize) throws IgniteException { - return igfs.open(path, bufSize); + return (IgfsSecondaryFileSystemPositionedReadable)igfs.open(path, bufSize); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetricsSelfTest.java index 38cfc00ae0cb0..8d742fb535b6b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetricsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetricsSelfTest.java @@ -116,6 +116,7 @@ private void startPrimary() throws Exception { * @return Configuration. * @throws Exception If failed. */ + @SuppressWarnings("unchecked") private IgniteConfiguration primaryConfiguration(int idx) throws Exception { FileSystemConfiguration igfsCfg = new FileSystemConfiguration(); @@ -172,6 +173,7 @@ private IgniteConfiguration primaryConfiguration(int idx) throws Exception { * * @throws Exception If failed. */ + @SuppressWarnings("unchecked") private void startSecondary() throws Exception { FileSystemConfiguration igfsCfg = new FileSystemConfiguration(); @@ -384,6 +386,7 @@ public void testMultipleClose() throws Exception { * * @throws Exception If failed. */ + @SuppressWarnings({"ResultOfMethodCallIgnored", "ConstantConditions"}) public void testBlockMetrics() throws Exception { IgfsEx igfs = (IgfsEx)igfsPrimary[0]; @@ -424,7 +427,7 @@ public void testBlockMetrics() throws Exception { checkBlockMetrics(initMetrics, igfs.metrics(), 0, 0, 0, 3, 0, blockSize * 3); // Read data from the first file. - IgfsInputStreamAdapter is = igfs.open(file1); + IgfsInputStream is = igfs.open(file1); is.readFully(0, new byte[blockSize * 2]); is.close(); @@ -432,7 +435,7 @@ public void testBlockMetrics() throws Exception { // Read data from the second file with hits. is = igfs.open(file2); - is.readChunks(0, blockSize); + is.read(new byte[blockSize]); is.close(); checkBlockMetrics(initMetrics, igfs.metrics(), 3, 0, blockSize * 3, 3, 0, blockSize * 3); @@ -449,7 +452,7 @@ public void testBlockMetrics() throws Exception { // Read remote file. is = igfs.open(fileRemote); - is.readChunks(0, rmtBlockSize); + is.read(new byte[rmtBlockSize]); is.close(); checkBlockMetrics(initMetrics, igfs.metrics(), 4, 1, blockSize * 3 + rmtBlockSize, 3, 0, blockSize * 3); @@ -459,7 +462,7 @@ public void testBlockMetrics() throws Exception { // Read remote file again. is = igfs.open(fileRemote); - is.readChunks(0, rmtBlockSize); + is.read(new byte[rmtBlockSize]); is.close(); checkBlockMetrics(initMetrics, igfs.metrics(), 5, 1, blockSize * 3 + rmtBlockSize * 2, 3, 0, blockSize * 3); @@ -495,16 +498,6 @@ public void testBlockMetrics() throws Exception { checkBlockMetrics(initMetrics, igfs.metrics(), 5, 1, blockSize * 3 + rmtBlockSize * 2, 5, 1, blockSize * 7 / 2 + rmtBlockSize); - // Now read partial block. - // Read remote file again. - is = igfs.open(file1); - is.seek(blockSize * 2); - is.readChunks(0, blockSize / 2); - is.close(); - - checkBlockMetrics(initMetrics, igfs.metrics(), 6, 1, blockSize * 7 / 2 + rmtBlockSize * 2, 5, 1, - blockSize * 7 / 2 + rmtBlockSize); - igfs.resetMetrics(); metrics = igfs.metrics(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java index 01389079649f9..2b989c5de462a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java @@ -23,6 +23,7 @@ import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.igfs.IgfsBlockLocation; import org.apache.ignite.igfs.IgfsFile; +import org.apache.ignite.igfs.IgfsInputStream; import org.apache.ignite.igfs.IgfsMetrics; import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsOutputStream; @@ -75,21 +76,22 @@ public IgfsMock(@Nullable String name) { } /** {@inheritDoc} */ - @Override public IgfsInputStreamAdapter open(IgfsPath path, int bufSize, int seqReadsBeforePrefetch) throws IgniteException { + @Override public IgfsInputStream open(IgfsPath path, int bufSize, int seqReadsBeforePrefetch) + throws IgniteException { throwUnsupported(); return null; } /** {@inheritDoc} */ - @Override public IgfsInputStreamAdapter open(IgfsPath path) throws IgniteException { + @Override public IgfsInputStream open(IgfsPath path) throws IgniteException { throwUnsupported(); return null; } /** {@inheritDoc} */ - @Override public IgfsInputStreamAdapter open(IgfsPath path, int bufSize) throws IgniteException { + @Override public IgfsInputStream open(IgfsPath path, int bufSize) throws IgniteException { throwUnsupported(); return null; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java index f426243221a6f..322053829653e 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java @@ -28,6 +28,7 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.igfs.IgfsBlockLocation; import org.apache.ignite.igfs.IgfsFile; +import org.apache.ignite.igfs.IgfsInputStream; import org.apache.ignite.igfs.IgfsOutputStream; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.IgfsPathSummary; @@ -35,7 +36,6 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.igfs.IgfsEx; import org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse; -import org.apache.ignite.internal.processors.igfs.IgfsInputStreamAdapter; import org.apache.ignite.internal.processors.igfs.IgfsStatus; import org.apache.ignite.internal.processors.igfs.IgfsUtils; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -316,9 +316,9 @@ public HadoopIgfsInProc(IgfsEx igfs, Log log, String userName) throws IgniteChec try { return IgfsUserContext.doAs(user, new IgniteOutClosure() { @Override public HadoopIgfsStreamDelegate apply() { - IgfsInputStreamAdapter stream = igfs.open(path, bufSize); + IgfsInputStream stream = igfs.open(path, bufSize); - return new HadoopIgfsStreamDelegate(HadoopIgfsInProc.this, stream, stream.fileInfo().length()); + return new HadoopIgfsStreamDelegate(HadoopIgfsInProc.this, stream, stream.length()); } }); } @@ -336,9 +336,9 @@ public HadoopIgfsInProc(IgfsEx igfs, Log log, String userName) throws IgniteChec try { return IgfsUserContext.doAs(user, new IgniteOutClosure() { @Override public HadoopIgfsStreamDelegate apply() { - IgfsInputStreamAdapter stream = igfs.open(path, bufSize, seqReadsBeforePrefetch); + IgfsInputStream stream = igfs.open(path, bufSize, seqReadsBeforePrefetch); - return new HadoopIgfsStreamDelegate(HadoopIgfsInProc.this, stream, stream.fileInfo().length()); + return new HadoopIgfsStreamDelegate(HadoopIgfsInProc.this, stream, stream.length()); } }); } @@ -394,7 +394,7 @@ public HadoopIgfsInProc(IgfsEx igfs, Log log, String userName) throws IgniteChec /** {@inheritDoc} */ @Override public IgniteInternalFuture readData(HadoopIgfsStreamDelegate delegate, long pos, int len, @Nullable byte[] outBuf, int outOff, int outLen) { - IgfsInputStreamAdapter stream = delegate.target(); + IgfsInputStream stream = delegate.target(); try { byte[] res = null; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java index 8dc27172de31b..7ee318a283f9e 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java @@ -34,11 +34,11 @@ import org.apache.ignite.Ignition; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.fs.IgniteHadoopFileSystemCounterWriter; +import org.apache.ignite.igfs.IgfsInputStream; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.IgnitionEx; import org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker; import org.apache.ignite.internal.processors.igfs.IgfsEx; -import org.apache.ignite.internal.processors.igfs.IgfsInputStreamAdapter; import org.apache.ignite.internal.processors.resource.GridSpringResourceContext; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; @@ -398,7 +398,7 @@ private void checkQuery(String expRes, String qry) throws Exception { "location '/result' as " + qry )); - IgfsInputStreamAdapter in = igfs.open(new IgfsPath("/result/000000_0")); + IgfsInputStream in = igfs.open(new IgfsPath("/result/000000_0")); byte[] buf = new byte[(int) in.length()]; From 409f043b3e94f51aa23341b7283233a572be6cd2 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Wed, 14 Sep 2016 11:01:33 +0300 Subject: [PATCH 130/487] IGNITE-3890: IGFS: Simplified IgfsInputStream hierarchy (2). --- .../processors/igfs/IgfsAsyncImpl.java | 5 - .../internal/processors/igfs/IgfsContext.java | 27 ++++ .../processors/igfs/IgfsDataManager.java | 19 +-- .../internal/processors/igfs/IgfsEx.java | 8 - .../internal/processors/igfs/IgfsImpl.java | 143 +----------------- .../processors/igfs/IgfsInputStreamImpl.java | 103 +++++++------ .../processors/igfs/IgfsOutputStreamImpl.java | 8 +- .../internal/processors/igfs/IgfsMock.java | 7 - 8 files changed, 100 insertions(+), 220 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java index 743601e29974f..106ef6014c96e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java @@ -156,11 +156,6 @@ public IgfsAsyncImpl(IgfsImpl igfs) { return igfs.globalSampling(); } - /** {@inheritDoc} */ - @Override public IgfsLocalMetrics localMetrics() { - return igfs.localMetrics(); - } - /** {@inheritDoc} */ @Override public long groupBlockSize() { return igfs.groupBlockSize(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java index a638bf34f6056..3e012469b0022 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java @@ -54,6 +54,12 @@ public class IgfsContext { /** IGFS instance. */ private final IgfsEx igfs; + /** Local metrics holder. */ + private final IgfsLocalMetrics metrics = new IgfsLocalMetrics(); + + /** Local cluster node. */ + private volatile ClusterNode locNode; + /** * @param ctx Kernal context. * @param cfg IGFS configuration. @@ -178,6 +184,27 @@ public boolean igfsNode(ClusterNode node) { return IgfsUtils.isIgfsNode(node, cfg.getName()); } + /** + * Get local metrics. + * + * @return Local metrics. + */ + public IgfsLocalMetrics metrics() { + return metrics; + } + + /** + * Get local node. + * + * @return Local node. + */ + public ClusterNode localNode() { + if (locNode == null) + locNode = ctx.discovery().localNode(); + + return locNode; + } + /** * Adds manager to managers list. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java index 5e2c6b26f3dd9..d2183f927de30 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java @@ -105,9 +105,6 @@ public class IgfsDataManager extends IgfsManager { /** */ private CountDownLatch dataCacheStartLatch; - /** Local IGFS metrics. */ - private IgfsLocalMetrics metrics; - /** Group block size. */ private long grpBlockSize; @@ -201,8 +198,6 @@ else if (msg instanceof IgfsAckMessage) dataCache = (IgniteInternalCache)dataCachePrj; - metrics = igfsCtx.igfs().localMetrics(); - AffinityKeyMapper mapper = igfsCtx.kernalContext().cache() .internalCache(igfsCtx.configuration().getDataCacheName()).configuration().getAffinityMapper(); @@ -388,7 +383,7 @@ private IgniteDataStreamer dataStreamer() { putBlock(fileInfo.blockSize(), key, res); - metrics.addReadBlocks(1, 1); + igfsCtx.metrics().addReadBlocks(1, 1); } catch (IgniteCheckedException e) { rmtReadFut.onDone(e); @@ -405,18 +400,18 @@ private IgniteDataStreamer dataStreamer() { // Wait for existing future to finish and get it's result. res = oldRmtReadFut.get(); - metrics.addReadBlocks(1, 0); + igfsCtx.metrics().addReadBlocks(1, 0); } } else - metrics.addReadBlocks(1, 0); + igfsCtx.metrics().addReadBlocks(1, 0); return res; } }); } else - metrics.addReadBlocks(1, 0); + igfsCtx.metrics().addReadBlocks(1, 0); return fut; } @@ -1308,7 +1303,7 @@ private abstract class BlocksWriter { if (!nodeBlocks.isEmpty()) { processBatch(id, node, nodeBlocks); - metrics.addWriteBlocks(1, 0); + igfsCtx.metrics().addWriteBlocks(1, 0); } return portion; @@ -1350,7 +1345,7 @@ private abstract class BlocksWriter { else nodeBlocks.put(key, portion); - metrics.addWriteBlocks(writtenTotal, writtenSecondary); + igfsCtx.metrics().addWriteBlocks(writtenTotal, writtenSecondary); written += portion.length; } @@ -1359,7 +1354,7 @@ private abstract class BlocksWriter { if (!nodeBlocks.isEmpty()) { processBatch(id, node, nodeBlocks); - metrics.addWriteBlocks(nodeBlocks.size(), 0); + igfsCtx.metrics().addWriteBlocks(nodeBlocks.size(), 0); } assert written == len; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java index 05e157d413c7c..c86969572fdc8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java @@ -19,7 +19,6 @@ import java.net.URI; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteFileSystem; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; @@ -74,13 +73,6 @@ public interface IgfsEx extends IgniteFileSystem { */ @Nullable public Boolean globalSampling(); - /** - * Get local metrics. - * - * @return Local metrics. - */ - public IgfsLocalMetrics localMetrics(); - /** * Gets group block size, i.e. block size multiplied by group size in affinity mapper. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 2c1f0f3889c77..45596a3f4f376 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -24,7 +24,6 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.eviction.EvictionPolicy; import org.apache.ignite.cache.eviction.igfs.IgfsPerBlockLruEvictionPolicy; -import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.compute.ComputeJob; import org.apache.ignite.compute.ComputeJobAdapter; import org.apache.ignite.compute.ComputeJobResult; @@ -48,7 +47,6 @@ import org.apache.ignite.igfs.mapreduce.IgfsRecordResolver; import org.apache.ignite.igfs.mapreduce.IgfsTask; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; -import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager; @@ -72,7 +70,6 @@ import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.LT; -import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteFuture; @@ -83,7 +80,6 @@ import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; -import java.io.IOException; import java.io.OutputStream; import java.net.URI; import java.util.ArrayList; @@ -97,12 +93,10 @@ import java.util.concurrent.Callable; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadFactory; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_DELETED; -import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CLOSED_READ; import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_DELETED; import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_READ; import static org.apache.ignite.events.EventType.EVT_IGFS_META_UPDATED; @@ -142,9 +136,6 @@ public final class IgfsImpl implements IgfsEx { /** Event storage manager. */ private GridEventStorageManager evts; - /** Local node. */ - private ClusterNode locNode; - /** Logger. */ private IgniteLogger log; @@ -285,16 +276,6 @@ public final class IgfsImpl implements IgfsEx { new LinkedBlockingQueue(), new IgfsThreadFactory(cfg.getName()), null) : null; } - /** - * @return Local node. - */ - private ClusterNode localNode() { - if (locNode == null) - locNode = igfsCtx.kernalContext().discovery().localNode(); - - return locNode; - } - /** {@inheritDoc} */ @Override public void stop(boolean cancel) { busyLock.block(); @@ -500,12 +481,6 @@ else if (val) } }); } - - /** {@inheritDoc} */ - @Override public IgfsLocalMetrics localMetrics() { - return metrics; - } - /** {@inheritDoc} */ @Override public long groupBlockSize() { return data.groupBlockSize(); @@ -632,7 +607,7 @@ else if (val) if (info != null) { if (evts.isRecordable(EVT_IGFS_META_UPDATED)) - evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_META_UPDATED, props)); + evts.record(new IgfsEvent(path, igfsCtx.localNode(), EVT_IGFS_META_UPDATED, props)); return new IgfsFileImpl(path, info, data.groupBlockSize()); } @@ -979,8 +954,8 @@ private IgfsEntryInfo primaryInfoForListing(IgfsPath path) throws IgniteCheckedE IgfsSecondaryInputStreamDescriptor desc = meta.openDual(secondaryFs, path, bufSize0); - IgfsEventAwareInputStream os = new IgfsEventAwareInputStream(igfsCtx, path, desc.info(), - cfg.getPrefetchBlocks(), seqReadsBeforePrefetch, desc.reader(), metrics); + IgfsInputStreamImpl os = new IgfsInputStreamImpl(igfsCtx, path, desc.info(), + cfg.getPrefetchBlocks(), seqReadsBeforePrefetch, desc.reader()); IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_READ); @@ -996,8 +971,8 @@ private IgfsEntryInfo primaryInfoForListing(IgfsPath path) throws IgniteCheckedE throw new IgfsPathIsDirectoryException("Failed to open file (not a file): " + path); // Input stream to read data from grid cache with separate blocks. - IgfsEventAwareInputStream os = new IgfsEventAwareInputStream(igfsCtx, path, info, - cfg.getPrefetchBlocks(), seqReadsBeforePrefetch, null, metrics); + IgfsInputStreamImpl os = new IgfsInputStreamImpl(igfsCtx, path, info, + cfg.getPrefetchBlocks(), seqReadsBeforePrefetch, null); IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_READ); @@ -1266,6 +1241,8 @@ private IgfsOutputStream create0( } } + IgfsLocalMetrics metrics = igfsCtx.metrics(); + return new IgfsMetricsAdapter( igfsCtx.data().spaceSize(), igfsCtx.data().maxSpaceSize(), @@ -1288,7 +1265,7 @@ private IgfsOutputStream create0( /** {@inheritDoc} */ @Override public void resetMetrics() { - metrics.reset(); + igfsCtx.metrics().reset(); } /** {@inheritDoc} */ @@ -1592,110 +1569,6 @@ private IgfsFileImpl resolveFileInfo(IgfsPath path, IgfsMode mode) throws Except throw new IllegalStateException("Asynchronous mode is not enabled."); } - /** Detailed file descriptor. */ - private static final class FileDescriptor { - /** Parent file ID. */ - @Nullable - private final IgniteUuid parentId; - - /** File name. */ - private final String fileName; - - /** File ID. */ - private final IgniteUuid fileId; - - /** File is plain data file or directory. */ - private final boolean isFile; - - /** - * Constructs detailed file descriptor. - * - * @param parentId Parent file ID. - * @param fileName File name. - * @param fileId File ID. - * @param isFile {@code True} if file. - */ - private FileDescriptor(@Nullable IgniteUuid parentId, String fileName, IgniteUuid fileId, boolean isFile) { - assert fileName != null; - - this.parentId = parentId; - this.fileName = fileName; - - this.fileId = fileId; - this.isFile = isFile; - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - int res = parentId != null ? parentId.hashCode() : 0; - - res = 31 * res + fileName.hashCode(); - res = 31 * res + fileId.hashCode(); - res = 31 * res + (isFile ? 1231 : 1237); - - return res; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (o == this) - return true; - - if (o == null || getClass() != o.getClass()) - return false; - - FileDescriptor that = (FileDescriptor)o; - - return fileId.equals(that.fileId) && isFile == that.isFile && fileName.equals(that.fileName) && - (parentId == null ? that.parentId == null : parentId.equals(that.parentId)); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(FileDescriptor.class, this); - } - } - - /** - * IGFS input stream extension that fires events. - */ - private class IgfsEventAwareInputStream extends IgfsInputStreamImpl { - /** Close guard. */ - private final AtomicBoolean closeGuard = new AtomicBoolean(false); - - /** - * Constructor. - * - * @param igfsCtx IGFS context. - * @param path Path to stored file. - * @param fileInfo File info. - * @param prefetchBlocks Prefetch blocks. - * @param seqReadsBeforePrefetch Amount of sequential reads before prefetch is triggered. - * @param secReader Optional secondary file system reader. - * @param metrics Metrics. - */ - IgfsEventAwareInputStream(IgfsContext igfsCtx, IgfsPath path, IgfsEntryInfo fileInfo, - int prefetchBlocks, int seqReadsBeforePrefetch, @Nullable IgfsSecondaryFileSystemPositionedReadable secReader, - IgfsLocalMetrics metrics) { - super(igfsCtx, path, fileInfo, prefetchBlocks, seqReadsBeforePrefetch, secReader, metrics); - - metrics.incrementFilesOpenedForRead(); - } - - /** {@inheritDoc} */ - @SuppressWarnings("NonSynchronizedMethodOverridesSynchronizedMethod") - @Override public void close() throws IOException { - if (closeGuard.compareAndSet(false, true)) { - super.close(); - - metrics.decrementFilesOpenedForRead(); - - if (evts.isRecordable(EVT_IGFS_FILE_CLOSED_READ)) - evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_FILE_CLOSED_READ, bytes())); - } - } - } - /** * Space calculation task. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java index f20a423fd6a0c..2f9f2fcab5e1f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java @@ -19,12 +19,14 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.events.IgfsEvent; import org.apache.ignite.igfs.IgfsCorruptedFileException; import org.apache.ignite.igfs.IgfsInputStream; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.IgfsPathNotFoundException; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -43,6 +45,8 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CLOSED_READ; + /** * Input stream to read data from grid cache with separate blocks. */ @@ -50,11 +54,8 @@ public class IgfsInputStreamImpl extends IgfsInputStream implements IgfsSecondar /** Empty chunks result. */ private static final byte[][] EMPTY_CHUNKS = new byte[0][]; - /** Meta manager. */ - private final IgfsMetaManager meta; - - /** Data manager. */ - private final IgfsDataManager data; + /** IGFS context. */ + private final IgfsContext igfsCtx; /** Secondary file system reader. */ @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") @@ -108,9 +109,6 @@ public class IgfsInputStreamImpl extends IgfsInputStream implements IgfsSecondar /** Time consumed on reading. */ private long time; - /** Local IGFS metrics. */ - private final IgfsLocalMetrics metrics; - /** * Constructs file output stream. * @@ -120,24 +118,19 @@ public class IgfsInputStreamImpl extends IgfsInputStream implements IgfsSecondar * @param prefetchBlocks Number of blocks to prefetch. * @param seqReadsBeforePrefetch Amount of sequential reads before prefetch is triggered. * @param secReader Optional secondary file system reader. - * @param metrics Local IGFS metrics. */ IgfsInputStreamImpl(IgfsContext igfsCtx, IgfsPath path, IgfsEntryInfo fileInfo, int prefetchBlocks, - int seqReadsBeforePrefetch, @Nullable IgfsSecondaryFileSystemPositionedReadable secReader, IgfsLocalMetrics metrics) { + int seqReadsBeforePrefetch, @Nullable IgfsSecondaryFileSystemPositionedReadable secReader) { assert igfsCtx != null; assert path != null; assert fileInfo != null; - assert metrics != null; + this.igfsCtx = igfsCtx; this.path = path; this.fileInfo = fileInfo; this.prefetchBlocks = prefetchBlocks; this.seqReadsBeforePrefetch = seqReadsBeforePrefetch; this.secReader = secReader; - this.metrics = metrics; - - meta = igfsCtx.meta(); - data = igfsCtx.data(); log = igfsCtx.kernalContext().log(IgfsInputStream.class); @@ -146,6 +139,8 @@ public class IgfsInputStreamImpl extends IgfsInputStream implements IgfsSecondar locCache = new LinkedHashMap<>(maxLocCacheSize, 1.0f); pendingFuts = new GridConcurrentHashSet<>(prefetchBlocks > 0 ? prefetchBlocks : 1); + + igfsCtx.metrics().incrementFilesOpenedForRead(); } /** @@ -295,46 +290,56 @@ public synchronized byte[][] readChunks(long pos, int len) throws IOException { /** {@inheritDoc} */ @Override public synchronized void close() throws IOException { - try { - if (secReader != null) { - // Close secondary input stream. - secReader.close(); - - // Ensuring local cache futures completion. - for (IgniteInternalFuture fut : locCache.values()) { - try { - fut.get(); - } - catch (IgniteCheckedException ignore) { - // No-op. + if (!closed) { + try { + if (secReader != null) { + // Close secondary input stream. + secReader.close(); + + // Ensuring local cache futures completion. + for (IgniteInternalFuture fut : locCache.values()) { + try { + fut.get(); + } + catch (IgniteCheckedException ignore) { + // No-op. + } } - } - // Ensuring pending evicted futures completion. - while (!pendingFuts.isEmpty()) { - pendingFutsLock.lock(); + // Ensuring pending evicted futures completion. + while (!pendingFuts.isEmpty()) { + pendingFutsLock.lock(); - try { - pendingFutsCond.await(100, TimeUnit.MILLISECONDS); - } - catch (InterruptedException ignore) { - // No-op. - } - finally { - pendingFutsLock.unlock(); + try { + pendingFutsCond.await(100, TimeUnit.MILLISECONDS); + } + catch (InterruptedException ignore) { + // No-op. + } + finally { + pendingFutsLock.unlock(); + } } } } - } - catch (Exception e) { - throw new IOException("File to close the file: " + path, e); - } - finally { - closed = true; + catch (Exception e) { + throw new IOException("File to close the file: " + path, e); + } + finally { + closed = true; + + IgfsLocalMetrics metrics = igfsCtx.metrics(); + + metrics.addReadBytesTime(bytes, time); + metrics.decrementFilesOpenedForRead(); - metrics.addReadBytesTime(bytes, time); + locCache.clear(); - locCache.clear(); + GridEventStorageManager evts = igfsCtx.kernalContext().event(); + + if (evts.isRecordable(EVT_IGFS_FILE_CLOSED_READ)) + evts.record(new IgfsEvent(path, igfsCtx.localNode(), EVT_IGFS_FILE_CLOSED_READ, bytes())); + } } } @@ -408,7 +413,7 @@ private byte[] blockFragmentizerSafe(long blockIdx) throws IOException { // This failure may be caused by file being fragmented. if (fileInfo.fileMap() != null && !fileInfo.fileMap().ranges().isEmpty()) { - IgfsEntryInfo newInfo = meta.info(fileInfo.id()); + IgfsEntryInfo newInfo = igfsCtx.meta().info(fileInfo.id()); // File was deleted. if (newInfo == null) @@ -540,7 +545,7 @@ public void apply(IgniteInternalFuture t) { */ @Nullable protected IgniteInternalFuture dataBlock(IgfsEntryInfo fileInfo, long blockIdx) throws IgniteCheckedException { - return data.dataBlock(fileInfo, path, blockIdx, secReader); + return igfsCtx.data().dataBlock(fileInfo, path, blockIdx, secReader); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java index bbff93b4fc0fa..6dec0c1d9c957 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java @@ -127,7 +127,7 @@ class IgfsOutputStreamImpl extends IgfsOutputStream { writeFut = igfsCtx.data().writeStart(fileInfo.id()); } - igfsCtx.igfs().localMetrics().incrementFilesOpenedForWrite(); + igfsCtx.metrics().incrementFilesOpenedForWrite(); } /** {@inheritDoc} */ @@ -355,8 +355,8 @@ private void flushRemainder() throws IOException { if (err != null) throw err; - igfsCtx.igfs().localMetrics().addWrittenBytesTime(bytes, time); - igfsCtx.igfs().localMetrics().decrementFilesOpenedForWrite(); + igfsCtx.metrics().addWrittenBytesTime(bytes, time); + igfsCtx.metrics().decrementFilesOpenedForWrite(); GridEventStorageManager evts = igfsCtx.kernalContext().event(); @@ -396,7 +396,7 @@ private void sendBufferIfFull() throws IOException { /** * Send local buffer if at least something is stored there. * - * @throws IOException + * @throws IOException If failed. */ private void sendBufferIfNotEmpty() throws IOException { if (buf != null && buf.position() > 0) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java index 2b989c5de462a..04c67dc9b2f1c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMock.java @@ -116,13 +116,6 @@ public IgfsMock(@Nullable String name) { return null; } - /** {@inheritDoc} */ - @Override public IgfsLocalMetrics localMetrics() { - throwUnsupported(); - - return null; - } - /** {@inheritDoc} */ @Override public long groupBlockSize() { throwUnsupported(); From 98914fef1565dee660b1d743c45c91f3c0bf0afe Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Wed, 14 Sep 2016 15:59:42 +0700 Subject: [PATCH 131/487] IGNITE-3815 Added check that key type registered in store or cache. Added test. IGNITE-3816 Fixed column index search logic and added propper exception. --- .../store/jdbc/CacheAbstractJdbcStore.java | 71 +++++++++++++------ .../cache/store/jdbc/CacheJdbcPojoStore.java | 22 +++--- .../CacheJdbcPojoStoreAbstractSelfTest.java | 70 +++++++++++++++--- ...JdbcPojoStoreBinaryMarshallerSelfTest.java | 12 ++-- 4 files changed, 126 insertions(+), 49 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java index aad05e085535d..a33a1e6625f86 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java @@ -574,8 +574,8 @@ private void checkTypeConfiguration(@Nullable String cacheName, TypeKind kind, S try { if (kind == TypeKind.BUILT_IN) { if (flds.length != 1) - throw new CacheException("More than one field for built in type [cache=" + U.maskName(cacheName) + - ", type=" + typeName + " ]"); + throw new CacheException("More than one field for built in type " + + "[cache=" + U.maskName(cacheName) + ", type=" + typeName + " ]"); JdbcTypeField field = flds[0]; @@ -588,16 +588,16 @@ private void checkTypeConfiguration(@Nullable String cacheName, TypeKind kind, S else for (JdbcTypeField field : flds) { if (field.getDatabaseFieldName() == null) - throw new CacheException("Missing database name in mapping description [cache=" + - U.maskName(cacheName) + ", type=" + typeName + " ]"); + throw new CacheException("Missing database name in mapping description " + + "[cache=" + U.maskName(cacheName) + ", type=" + typeName + " ]"); if (field.getJavaFieldName() == null) - throw new CacheException("Missing field name in mapping description [cache=" + - U.maskName(cacheName) + ", type=" + typeName + " ]"); + throw new CacheException("Missing field name in mapping description " + + "[cache=" + U.maskName(cacheName) + ", type=" + typeName + " ]"); if (field.getJavaFieldType() == null) - throw new CacheException("Missing field type in mapping description [cache=" + - U.maskName(cacheName) + ", type=" + typeName + " ]"); + throw new CacheException("Missing field type in mapping description " + + "[cache=" + U.maskName(cacheName) + ", type=" + typeName + " ]"); } } catch (ClassNotFoundException e) { @@ -778,6 +778,23 @@ private EntryMapping entryMapping(String cacheName, Object typeId) throws CacheE return em; } + /** + * Find column index by database name. + * + * @param loadColIdxs Select query columns indexes. + * @param dbName Column name in database. + * @return Column index. + * @throws IllegalStateException if column not found. + */ + protected Integer columnIndex(Map loadColIdxs, String dbName) { + Integer colIdx = loadColIdxs.get(dbName.toUpperCase()); + + if (colIdx == null) + throw new IllegalStateException("Failed to find column index for database field: " + dbName); + + return colIdx; + } + /** {@inheritDoc} */ @Override public void loadCache(final IgniteBiInClosure clo, @Nullable Object... args) throws CacheLoaderException { @@ -800,7 +817,15 @@ private EntryMapping entryMapping(String cacheName, Object typeId) throws CacheE log.debug("Start loading entries from db using user queries from arguments..."); for (int i = 0; i < args.length; i += 2) { - String keyType = args[i].toString(); + final String keyType = args[i].toString(); + + if (!F.exist(mappings.values(), new IgnitePredicate() { + @Override public boolean apply(EntryMapping em) { + return em.keyType().equals(keyType); + } + })) + throw new CacheLoaderException("Provided key type is not found in store or cache configuration " + + "[cache=" + U.maskName(cacheName) + ", key=" + keyType + "]"); String selQry = args[i + 1].toString(); @@ -827,7 +852,7 @@ private EntryMapping entryMapping(String cacheName, Object typeId) throws CacheE if (rs.next()) { if (log.isDebugEnabled()) - log.debug("Multithread loading entries from db [cache=" + U.maskName(cacheName) + + log.debug("Multithread loading entries from db [cache=" + U.maskName(cacheName) + ", keyType=" + em.keyType() + " ]"); int keyCnt = em.keyCols.size(); @@ -856,8 +881,8 @@ private EntryMapping entryMapping(String cacheName, Object typeId) throws CacheE } } catch (SQLException e) { - log.warning("Failed to load entries from db in multithreaded mode [cache=" + U.maskName(cacheName) + - ", keyType=" + em.keyType() + " ]", e); + log.warning("Failed to load entries from db in multithreaded mode " + + "[cache=" + U.maskName(cacheName) + ", keyType=" + em.keyType() + " ]", e); } finally { U.closeQuiet(conn); @@ -865,7 +890,7 @@ private EntryMapping entryMapping(String cacheName, Object typeId) throws CacheE } if (log.isDebugEnabled()) - log.debug("Single thread loading entries from db [cache=" + U.maskName(cacheName) + + log.debug("Single thread loading entries from db [cache=" + U.maskName(cacheName) + ", keyType=" + em.keyType() + " ]"); futs.add(pool.submit(loadCacheFull(em, clo))); @@ -876,7 +901,7 @@ private EntryMapping entryMapping(String cacheName, Object typeId) throws CacheE U.get(fut); if (log.isDebugEnabled()) - log.debug("Cache loaded from db: " + U.maskName(cacheName)); + log.debug("Cache loaded from db: " + U.maskName(cacheName)); } catch (IgniteCheckedException e) { throw new CacheLoaderException("Failed to load cache: " + U.maskName(cacheName), e.getCause()); @@ -1143,7 +1168,7 @@ private void writeUpsert(PreparedStatement insStmt, PreparedStatement updStmt, if (currKeyTypeId == null || !currKeyTypeId.equals(keyTypeId)) { if (mergeStmt != null) { if (log.isDebugEnabled()) - log.debug("Write entries to db [cache=" + U.maskName(cacheName) + + log.debug("Write entries to db [cache=" + U.maskName(cacheName) + ", keyType=" + em.keyType() + ", cnt=" + prepared + "]"); executeBatch(em, mergeStmt, "writeAll", fromIdx, prepared, lazyEntries); @@ -1168,7 +1193,7 @@ private void writeUpsert(PreparedStatement insStmt, PreparedStatement updStmt, if (++prepared % batchSize == 0) { if (log.isDebugEnabled()) - log.debug("Write entries to db [cache=" + U.maskName(cacheName) + + log.debug("Write entries to db [cache=" + U.maskName(cacheName) + ", keyType=" + em.keyType() + ", cnt=" + prepared + "]"); executeBatch(em, mergeStmt, "writeAll", fromIdx, prepared, lazyEntries); @@ -1181,7 +1206,7 @@ private void writeUpsert(PreparedStatement insStmt, PreparedStatement updStmt, if (mergeStmt != null && prepared % batchSize != 0) { if (log.isDebugEnabled()) - log.debug("Write entries to db [cache=" + U.maskName(cacheName) + + log.debug("Write entries to db [cache=" + U.maskName(cacheName) + ", keyType=" + em.keyType() + ", cnt=" + prepared + "]"); executeBatch(em, mergeStmt, "writeAll", fromIdx, prepared, lazyEntries); @@ -1194,8 +1219,8 @@ private void writeUpsert(PreparedStatement insStmt, PreparedStatement updStmt, } else { if (log.isDebugEnabled()) - log.debug("Write entries to db one by one using update and insert statements [cache=" + - U.maskName(cacheName) + ", cnt=" + entries.size() + "]"); + log.debug("Write entries to db one by one using update and insert statements " + + "[cache=" + U.maskName(cacheName) + ", cnt=" + entries.size() + "]"); PreparedStatement insStmt = null; @@ -1359,7 +1384,7 @@ private void executeBatch(EntryMapping em, Statement stmt, String desc, int from if (!currKeyTypeId.equals(keyTypeId)) { if (log.isDebugEnabled()) - log.debug("Delete entries from db [cache=" + U.maskName(cacheName) + + log.debug("Delete entries from db [cache=" + U.maskName(cacheName) + ", keyType=" + em.keyType() + ", cnt=" + prepared + "]"); executeBatch(em, delStmt, "deleteAll", fromIdx, prepared, lazyKeys); @@ -1377,7 +1402,7 @@ private void executeBatch(EntryMapping em, Statement stmt, String desc, int from if (++prepared % batchSize == 0) { if (log.isDebugEnabled()) - log.debug("Delete entries from db [cache=" + U.maskName(cacheName) + + log.debug("Delete entries from db [cache=" + U.maskName(cacheName) + ", keyType=" + em.keyType() + ", cnt=" + prepared + "]"); executeBatch(em, delStmt, "deleteAll", fromIdx, prepared, lazyKeys); @@ -1390,7 +1415,7 @@ private void executeBatch(EntryMapping em, Statement stmt, String desc, int from if (delStmt != null && prepared % batchSize != 0) { if (log.isDebugEnabled()) - log.debug("Delete entries from db [cache=" + U.maskName(cacheName) + + log.debug("Delete entries from db [cache=" + U.maskName(cacheName) + ", keyType=" + em.keyType() + ", cnt=" + prepared + "]"); executeBatch(em, delStmt, "deleteAll", fromIdx, prepared, lazyKeys); @@ -1766,7 +1791,7 @@ public EntryMapping(@Nullable String cacheName, JdbcDialect dialect, JdbcType ty int idx = 1; for (String col : cols) - loadColIdxs.put(col, idx++); + loadColIdxs.put(col.toUpperCase(), idx++); loadCacheQry = dialect.loadCacheQuery(fullTblName, cols); diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java index b9a3118a2838c..798b84a8acba2 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java @@ -151,7 +151,9 @@ private Object buildBuiltinObject(String typeName, JdbcTypeField[] fields, Map hashValues = calcHash ? new ArrayList<>(hashFields.size()) : null; for (JdbcTypeField field : fields) { - Integer colIdx = loadColIdxs.get(field.getDatabaseFieldName()); + Integer colIdx = columnIndex(loadColIdxs, field.getDatabaseFieldName()); Object colVal = getColumnValue(rs, colIdx, field.getJavaFieldType()); @@ -370,8 +372,8 @@ private ClassProperty(Method getter, Method setter, Field field) { * * @param obj Object to get property value from. * @return Property value. - * @throws IllegalAccessException - * @throws InvocationTargetException + * @throws IllegalAccessException If failed to get value from property or failed access to property via reflection. + * @throws InvocationTargetException If failed access to property via reflection. */ private Object get(Object obj) throws IllegalAccessException, InvocationTargetException { if (getter != null) @@ -388,8 +390,8 @@ private Object get(Object obj) throws IllegalAccessException, InvocationTargetEx * * @param obj Object to set property value to. * @param val New property value to set. - * @throws IllegalAccessException - * @throws InvocationTargetException + * @throws IllegalAccessException If failed to set property value or failed access to property via reflection. + * @throws InvocationTargetException If failed access to property via reflection. */ private void set(Object obj, Object val) throws IllegalAccessException, InvocationTargetException { if (setter != null) diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java index 41a6136bec302..e8592d74fc334 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java @@ -26,6 +26,7 @@ import java.sql.Statement; import java.sql.Types; import java.util.Random; +import javax.cache.integration.CacheLoaderException; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.store.jdbc.dialect.H2Dialect; import org.apache.ignite.cache.store.jdbc.model.Person; @@ -77,7 +78,7 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr /** * @return Connection to test in-memory H2 database. - * @throws SQLException + * @throws SQLException if failed to connect. */ protected Connection getConnection() throws SQLException { return DriverManager.getConnection(DFLT_CONN_URL, "sa", ""); @@ -164,9 +165,9 @@ protected JdbcType[] storeTypes() { storeTypes[0].setValueType("org.apache.ignite.cache.store.jdbc.model.Organization" + (noValClasses ? "1" : "")); storeTypes[0].setValueFields( - new JdbcTypeField(Types.INTEGER, "ID", Integer.class, "id"), - new JdbcTypeField(Types.VARCHAR, "NAME", String.class, "name"), - new JdbcTypeField(Types.VARCHAR, "CITY", String.class, "city")); + new JdbcTypeField(Types.INTEGER, "Id", Integer.class, "id"), + new JdbcTypeField(Types.VARCHAR, "Name", String.class, "name"), + new JdbcTypeField(Types.VARCHAR, "City", String.class, "city")); storeTypes[1] = new JdbcType(); storeTypes[1].setCacheName(CACHE_NAME); @@ -273,7 +274,7 @@ protected void fillSampleDatabase(Connection conn) throws SQLException { * @param noKeyCls {@code True} if keys classes are not on class path. * @param noValCls {@code True} if values classes are not on class path. * @param trn {@code True} if cache should be started in transactional mode. - * @throws Exception + * @throws Exception If failed to start grid. */ protected void startTestGrid(boolean builtin, boolean noKeyCls, boolean noValCls, boolean trn) throws Exception { builtinKeys = builtin; @@ -287,7 +288,7 @@ protected void startTestGrid(boolean builtin, boolean noKeyCls, boolean noValCls /** * Check that data was loaded correctly. */ - protected void checkCacheContent() { + protected void checkCacheLoad() { IgniteCache c1 = grid().cache(CACHE_NAME); c1.loadCache(null); @@ -295,13 +296,33 @@ protected void checkCacheContent() { assertEquals(ORGANIZATION_CNT + PERSON_CNT, c1.size()); } + /** + * Check that data was loaded correctly. + */ + protected void checkCacheLoadWithSql() { + IgniteCache c1 = grid().cache(CACHE_NAME); + + c1.loadCache(null, "org.apache.ignite.cache.store.jdbc.model.PersonKey", "select id, org_id, name, birthday from Person"); + + assertEquals(PERSON_CNT, c1.size()); + } + /** * @throws Exception If failed. */ public void testLoadCache() throws Exception { startTestGrid(false, false, false, false); - checkCacheContent(); + checkCacheLoad(); + } + + /** + * @throws Exception If failed. + */ + public void testLoadCacheWithSql() throws Exception { + startTestGrid(false, false, false, false); + + checkCacheLoadWithSql(); } /** @@ -310,7 +331,16 @@ public void testLoadCache() throws Exception { public void testLoadCacheTx() throws Exception { startTestGrid(false, false, false, true); - checkCacheContent(); + checkCacheLoad(); + } + + /** + * @throws Exception If failed. + */ + public void testLoadCacheWithSqlTx() throws Exception { + startTestGrid(false, false, false, true); + + checkCacheLoadWithSql(); } /** @@ -319,7 +349,7 @@ public void testLoadCacheTx() throws Exception { public void testLoadCachePrimitiveKeys() throws Exception { startTestGrid(true, false, false, false); - checkCacheContent(); + checkCacheLoad(); } /** @@ -328,7 +358,7 @@ public void testLoadCachePrimitiveKeys() throws Exception { public void testLoadCachePrimitiveKeysTx() throws Exception { startTestGrid(true, false, false, true); - checkCacheContent(); + checkCacheLoad(); } /** @@ -431,4 +461,24 @@ public void testPutTx() throws Exception { checkPut(); } + + /** + * @throws Exception If failed. + */ + public void testLoadNotRegisteredType() throws Exception { + startTestGrid(false, false, false, false); + + IgniteCache c1 = grid().cache(CACHE_NAME); + + try { + c1.loadCache(null, "PersonKeyWrong", "SELECT * FROM Person"); + } + catch (CacheLoaderException e) { + String msg = e.getMessage(); + + assertTrue("Unexpected exception: " + msg, + ("Provided key type is not found in store or cache configuration " + + "[cache=" + CACHE_NAME + ", key=PersonKeyWrong]").equals(msg)); + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java index b825a24ea5151..f998027cd0fbd 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java @@ -35,7 +35,7 @@ public class CacheJdbcPojoStoreBinaryMarshallerSelfTest extends CacheJdbcPojoSto public void testLoadCacheNoKeyClasses() throws Exception { startTestGrid(false, true, false, false); - checkCacheContent(); + checkCacheLoad(); } /** @@ -44,7 +44,7 @@ public void testLoadCacheNoKeyClasses() throws Exception { public void testLoadCacheNoKeyClassesTx() throws Exception { startTestGrid(false, true, false, true); - checkCacheContent(); + checkCacheLoad(); } /** @@ -53,7 +53,7 @@ public void testLoadCacheNoKeyClassesTx() throws Exception { public void testLoadCacheNoValueClasses() throws Exception { startTestGrid(false, false, true, false); - checkCacheContent(); + checkCacheLoad(); } /** @@ -62,7 +62,7 @@ public void testLoadCacheNoValueClasses() throws Exception { public void testLoadCacheNoValueClassesTx() throws Exception { startTestGrid(false, false, true, true); - checkCacheContent(); + checkCacheLoad(); } /** @@ -71,7 +71,7 @@ public void testLoadCacheNoValueClassesTx() throws Exception { public void testLoadCacheNoKeyAndValueClasses() throws Exception { startTestGrid(false, true, true, false); - checkCacheContent(); + checkCacheLoad(); } /** @@ -80,6 +80,6 @@ public void testLoadCacheNoKeyAndValueClasses() throws Exception { public void testLoadCacheNoKeyAndValueClassesTx() throws Exception { startTestGrid(false, true, true, true); - checkCacheContent(); + checkCacheLoad(); } } From 7cd9c63599f57fd51026a5e04947a624483eca69 Mon Sep 17 00:00:00 2001 From: Ignite Teamcity Date: Thu, 15 Sep 2016 10:18:31 +0300 Subject: [PATCH 132/487] 1.8.0-SNAPSHOT --- examples/pom.xml | 2 +- examples/schema-import/pom.xml | 2 +- modules/aop/pom.xml | 2 +- modules/apache-license-gen/pom.xml | 2 +- modules/aws/pom.xml | 2 +- modules/benchmarks/pom.xml | 2 +- modules/camel/pom.xml | 2 +- modules/cassandra/pom.xml | 2 +- modules/clients/pom.xml | 2 +- modules/cloud/pom.xml | 2 +- modules/codegen/pom.xml | 2 +- modules/core/pom.xml | 2 +- modules/core/src/main/resources/ignite.properties | 2 +- modules/extdata/p2p/pom.xml | 2 +- modules/extdata/uri/modules/uri-dependency/pom.xml | 2 +- modules/extdata/uri/pom.xml | 2 +- modules/flink/pom.xml | 2 +- modules/flume/pom.xml | 2 +- modules/gce/pom.xml | 2 +- modules/geospatial/pom.xml | 2 +- modules/hadoop/pom.xml | 2 +- modules/hibernate/pom.xml | 2 +- modules/indexing/pom.xml | 2 +- modules/jcl/pom.xml | 2 +- modules/jms11/pom.xml | 2 +- modules/jta/pom.xml | 2 +- modules/kafka/pom.xml | 2 +- modules/log4j/pom.xml | 2 +- modules/log4j2/pom.xml | 2 +- modules/mesos/pom.xml | 2 +- modules/mqtt/pom.xml | 2 +- modules/osgi-karaf/pom.xml | 2 +- modules/osgi-paxlogging/pom.xml | 2 +- modules/osgi/pom.xml | 2 +- modules/platforms/cpp/configure.ac | 2 +- modules/platforms/cpp/examples/configure.ac | 2 +- .../dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs | 6 +++--- .../Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs | 6 +++--- .../Properties/AssemblyInfo.cs | 6 +++--- .../Properties/AssemblyInfo.cs | 6 +++--- .../Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs | 6 +++--- .../dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs | 6 +++--- .../dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs | 6 +++--- .../dotnet/Apache.Ignite/Properties/AssemblyInfo.cs | 6 +++--- .../Apache.Ignite.Examples/Properties/AssemblyInfo.cs | 6 +++--- .../Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs | 6 +++--- modules/rest-http/pom.xml | 2 +- modules/scalar-2.10/pom.xml | 2 +- modules/scalar/pom.xml | 2 +- modules/schedule/pom.xml | 2 +- modules/schema-import-db/pom.xml | 2 +- modules/schema-import/pom.xml | 2 +- modules/slf4j/pom.xml | 2 +- modules/spark-2.10/pom.xml | 2 +- modules/spark/pom.xml | 2 +- modules/spring/pom.xml | 2 +- modules/ssh/pom.xml | 2 +- modules/storm/pom.xml | 2 +- modules/tools/pom.xml | 2 +- modules/twitter/pom.xml | 2 +- modules/urideploy/pom.xml | 2 +- modules/visor-console-2.10/pom.xml | 2 +- modules/visor-console/pom.xml | 2 +- modules/visor-plugins/pom.xml | 2 +- modules/web-console/pom.xml | 2 +- modules/web-console/web-agent/pom.xml | 2 +- modules/web/ignite-appserver-test/pom.xml | 2 +- modules/web/ignite-websphere-test/pom.xml | 2 +- modules/web/pom.xml | 2 +- modules/yardstick/pom.xml | 2 +- modules/yarn/pom.xml | 2 +- modules/zookeeper/pom.xml | 2 +- pom.xml | 2 +- 73 files changed, 93 insertions(+), 93 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index 979a46f4a92dc..5c4ead3a8cd4c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ ignite-examples - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT diff --git a/examples/schema-import/pom.xml b/examples/schema-import/pom.xml index b6e36efed3064..8ca92dd70dbe2 100644 --- a/examples/schema-import/pom.xml +++ b/examples/schema-import/pom.xml @@ -35,7 +35,7 @@ ignite-schema-import-demo - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml index eaebaf25c137a..d0314af927805 100644 --- a/modules/aop/pom.xml +++ b/modules/aop/pom.xml @@ -31,7 +31,7 @@ ignite-aop - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml index 53b1a8f9114de..74cf35e22bb59 100644 --- a/modules/apache-license-gen/pom.xml +++ b/modules/apache-license-gen/pom.xml @@ -31,7 +31,7 @@ org.apache.ignite ignite-apache-license-gen - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml index 2aee655a2ddbc..ee300d4acc2fd 100644 --- a/modules/aws/pom.xml +++ b/modules/aws/pom.xml @@ -31,7 +31,7 @@ ignite-aws - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/benchmarks/pom.xml b/modules/benchmarks/pom.xml index 00315a83b7ac2..4a18c1890d0f9 100644 --- a/modules/benchmarks/pom.xml +++ b/modules/benchmarks/pom.xml @@ -31,7 +31,7 @@ ignite-benchmarks - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/camel/pom.xml b/modules/camel/pom.xml index f40ff62e01146..841cfd92ff9e3 100644 --- a/modules/camel/pom.xml +++ b/modules/camel/pom.xml @@ -31,7 +31,7 @@ ignite-camel - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/cassandra/pom.xml b/modules/cassandra/pom.xml index dc8b08ca89892..7aac1165ce8fa 100644 --- a/modules/cassandra/pom.xml +++ b/modules/cassandra/pom.xml @@ -31,7 +31,7 @@ ignite-cassandra - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml index fa25d18902fff..e445229f2558a 100644 --- a/modules/clients/pom.xml +++ b/modules/clients/pom.xml @@ -31,7 +31,7 @@ ignite-clients - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml index 8232d0028d2d9..1bcc4bfb1a147 100644 --- a/modules/cloud/pom.xml +++ b/modules/cloud/pom.xml @@ -29,7 +29,7 @@ ignite-cloud - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml index aff0d3ac9cf25..be287e373088b 100644 --- a/modules/codegen/pom.xml +++ b/modules/codegen/pom.xml @@ -31,7 +31,7 @@ ignite-codegen - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/core/pom.xml b/modules/core/pom.xml index 7eefa4a6b1a43..4c4343a9c41a1 100644 --- a/modules/core/pom.xml +++ b/modules/core/pom.xml @@ -31,7 +31,7 @@ ignite-core - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties index 4d5780a8fb5ef..dcebbe4b5b4fc 100644 --- a/modules/core/src/main/resources/ignite.properties +++ b/modules/core/src/main/resources/ignite.properties @@ -15,7 +15,7 @@ # limitations under the License. # -ignite.version=1.7.0-SNAPSHOT +ignite.version=1.8.0-SNAPSHOT ignite.build=0 ignite.revision=DEV ignite.rel.date=01011970 diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml index 9bebf74d77af4..73a80cf607db7 100644 --- a/modules/extdata/p2p/pom.xml +++ b/modules/extdata/p2p/pom.xml @@ -31,7 +31,7 @@ ignite-extdata-p2p - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT diff --git a/modules/extdata/uri/modules/uri-dependency/pom.xml b/modules/extdata/uri/modules/uri-dependency/pom.xml index 1d9da24d6643f..40518fb117a0b 100644 --- a/modules/extdata/uri/modules/uri-dependency/pom.xml +++ b/modules/extdata/uri/modules/uri-dependency/pom.xml @@ -27,7 +27,7 @@ ignite-extdata-uri-dep jar - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT 4.0.0 diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml index 1cef7f91e7754..ff30d0aa258e7 100644 --- a/modules/extdata/uri/pom.xml +++ b/modules/extdata/uri/pom.xml @@ -32,7 +32,7 @@ ignite-extdata-uri - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT diff --git a/modules/flink/pom.xml b/modules/flink/pom.xml index 118c17575ded4..cd80f165be7a0 100644 --- a/modules/flink/pom.xml +++ b/modules/flink/pom.xml @@ -31,7 +31,7 @@ ignite-flink - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/flume/pom.xml b/modules/flume/pom.xml index f9d36e9f1735c..5e6fa6427710e 100644 --- a/modules/flume/pom.xml +++ b/modules/flume/pom.xml @@ -31,7 +31,7 @@ ignite-flume - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml index 8c7bed46d764d..8340512e876ee 100644 --- a/modules/gce/pom.xml +++ b/modules/gce/pom.xml @@ -31,7 +31,7 @@ ignite-gce - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml index eeef6575bfe7d..e65339bafd715 100644 --- a/modules/geospatial/pom.xml +++ b/modules/geospatial/pom.xml @@ -31,7 +31,7 @@ ignite-geospatial - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml index a3f40e5fe82e2..c02a57405f787 100644 --- a/modules/hadoop/pom.xml +++ b/modules/hadoop/pom.xml @@ -31,7 +31,7 @@ ignite-hadoop - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml index b0ea68db62909..40a3c95586139 100644 --- a/modules/hibernate/pom.xml +++ b/modules/hibernate/pom.xml @@ -31,7 +31,7 @@ ignite-hibernate - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml index d6675317bef61..58e82ed278dd6 100644 --- a/modules/indexing/pom.xml +++ b/modules/indexing/pom.xml @@ -31,7 +31,7 @@ ignite-indexing - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml index fbc94418451ac..37f475be4d1ba 100644 --- a/modules/jcl/pom.xml +++ b/modules/jcl/pom.xml @@ -31,7 +31,7 @@ ignite-jcl - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/jms11/pom.xml b/modules/jms11/pom.xml index a4222a052fc7d..eddeb707a55e1 100644 --- a/modules/jms11/pom.xml +++ b/modules/jms11/pom.xml @@ -31,7 +31,7 @@ ignite-jms11 - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml index 64ad26d3ff5cb..1f22f0276e29b 100644 --- a/modules/jta/pom.xml +++ b/modules/jta/pom.xml @@ -31,7 +31,7 @@ ignite-jta - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/kafka/pom.xml b/modules/kafka/pom.xml index 07d031e79cadd..96e3d0decc949 100644 --- a/modules/kafka/pom.xml +++ b/modules/kafka/pom.xml @@ -31,7 +31,7 @@ ignite-kafka - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml index f74e59bed51e3..24f31076ef0ea 100644 --- a/modules/log4j/pom.xml +++ b/modules/log4j/pom.xml @@ -31,7 +31,7 @@ ignite-log4j - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/log4j2/pom.xml b/modules/log4j2/pom.xml index ac991964fcfb0..99174ea2448b4 100644 --- a/modules/log4j2/pom.xml +++ b/modules/log4j2/pom.xml @@ -31,7 +31,7 @@ ignite-log4j2 - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml index 660b046355972..f407a3f4e84ac 100644 --- a/modules/mesos/pom.xml +++ b/modules/mesos/pom.xml @@ -31,7 +31,7 @@ ignite-mesos - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/mqtt/pom.xml b/modules/mqtt/pom.xml index ae984eb549b83..4b0e1a9635aa7 100644 --- a/modules/mqtt/pom.xml +++ b/modules/mqtt/pom.xml @@ -31,7 +31,7 @@ ignite-mqtt - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/osgi-karaf/pom.xml b/modules/osgi-karaf/pom.xml index f4e01131d963b..c2d455379b157 100644 --- a/modules/osgi-karaf/pom.xml +++ b/modules/osgi-karaf/pom.xml @@ -31,7 +31,7 @@ ignite-osgi-karaf - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT pom diff --git a/modules/osgi-paxlogging/pom.xml b/modules/osgi-paxlogging/pom.xml index 779c9a9a602d3..3cb2a88a49188 100644 --- a/modules/osgi-paxlogging/pom.xml +++ b/modules/osgi-paxlogging/pom.xml @@ -31,7 +31,7 @@ ignite-osgi-paxlogging - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT jar diff --git a/modules/osgi/pom.xml b/modules/osgi/pom.xml index 041fbd1dead7c..5232a10a6cf6c 100644 --- a/modules/osgi/pom.xml +++ b/modules/osgi/pom.xml @@ -31,7 +31,7 @@ ignite-osgi - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/platforms/cpp/configure.ac b/modules/platforms/cpp/configure.ac index 8d7987a7720bd..9dc9f529a067d 100644 --- a/modules/platforms/cpp/configure.ac +++ b/modules/platforms/cpp/configure.ac @@ -19,7 +19,7 @@ # Process this file with autoconf to produce a configure script. AC_PREREQ([2.69]) -AC_INIT([Apache Ignite C++], [1.7.0.11707], [dev@ignite.apache.org], [ignite], [ignite.apache.org]) +AC_INIT([Apache Ignite C++], [1.8.0.14218], [dev@ignite.apache.org], [ignite], [ignite.apache.org]) AC_CANONICAL_HOST AC_CONFIG_MACRO_DIR([m4]) diff --git a/modules/platforms/cpp/examples/configure.ac b/modules/platforms/cpp/examples/configure.ac index 85313ab0d27e7..82a3727fbf4fa 100644 --- a/modules/platforms/cpp/examples/configure.ac +++ b/modules/platforms/cpp/examples/configure.ac @@ -19,7 +19,7 @@ # Process this file with autoconf to produce a configure script. AC_PREREQ([2.69]) -AC_INIT([Apache Ignite C++ Examples], [1.7.0.11707], [dev@ignite.apache.org], [ignite-examples], [ignite.apache.org]) +AC_INIT([Apache Ignite C++ Examples], [1.8.0.14218], [dev@ignite.apache.org], [ignite-examples], [ignite.apache.org]) AC_CANONICAL_HOST AC_CONFIG_MACRO_DIR([m4]) diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs index 0b502fd74dca7..ad61ecd5852a2 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs @@ -32,8 +32,8 @@ [assembly: Guid("13ea96fc-cc83-4164-a7c0-4f30ed797460")] -[assembly: AssemblyVersion("1.7.0.11707")] -[assembly: AssemblyFileVersion("1.7.0.11707")] -[assembly: AssemblyInformationalVersion("1.7.0")] +[assembly: AssemblyVersion("1.8.0.14218")] +[assembly: AssemblyFileVersion("1.8.0.14218")] +[assembly: AssemblyInformationalVersion("1.8.0")] [assembly: CLSCompliant(true)] \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs index 136682903ce47..85af146eba846 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs @@ -31,6 +31,6 @@ [assembly: Guid("8fae8395-7e91-411a-a78f-44d6d3fed0fc")] -[assembly: AssemblyVersion("1.7.0.11707")] -[assembly: AssemblyFileVersion("1.7.0.11707")] -[assembly: AssemblyInformationalVersion("1.7.0")] +[assembly: AssemblyVersion("1.8.0.14218")] +[assembly: AssemblyFileVersion("1.8.0.14218")] +[assembly: AssemblyInformationalVersion("1.8.0")] diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs index fa30ff61b4003..34fca37b4b157 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs @@ -30,6 +30,6 @@ [assembly: ComVisible(false)] [assembly: Guid("134707f6-155d-47f6-9eb2-c67abbf3c009")] -[assembly: AssemblyVersion("1.7.0.11707")] -[assembly: AssemblyFileVersion("1.7.0.11707")] -[assembly: AssemblyInformationalVersion("1.7.0")] +[assembly: AssemblyVersion("1.8.0.14218")] +[assembly: AssemblyFileVersion("1.8.0.14218")] +[assembly: AssemblyInformationalVersion("1.8.0")] diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs index d04fd69356d62..4aa03f1d9035b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs @@ -45,6 +45,6 @@ // You can specify all the values or you can default the Build and Revision Numbers // by using the '*' as shown below: // [assembly: AssemblyVersion("1.0.*")] -[assembly: AssemblyVersion("1.7.0.11707")] -[assembly: AssemblyFileVersion("1.7.0.11707")] -[assembly: AssemblyInformationalVersion("1.7.0")] +[assembly: AssemblyVersion("1.8.0.14218")] +[assembly: AssemblyFileVersion("1.8.0.14218")] +[assembly: AssemblyInformationalVersion("1.8.0")] diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs index f1953e3394814..9eb2e2490e07a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs @@ -31,6 +31,6 @@ [assembly: Guid("de8dd5cc-7c7f-4a09-80d5-7086d9416a7b")] -[assembly: AssemblyVersion("1.7.0.11707")] -[assembly: AssemblyFileVersion("1.7.0.11707")] -[assembly: AssemblyInformationalVersion("1.7.0")] +[assembly: AssemblyVersion("1.8.0.14218")] +[assembly: AssemblyFileVersion("1.8.0.14218")] +[assembly: AssemblyInformationalVersion("1.8.0")] diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs index 5c5dfe5a4f1a7..008ba5cae62a3 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs @@ -33,9 +33,9 @@ [assembly: Guid("97db45a8-f922-456a-a819-7b3c6e5e03ba")] -[assembly: AssemblyVersion("1.7.0.11707")] -[assembly: AssemblyFileVersion("1.7.0.11707")] -[assembly: AssemblyInformationalVersion("1.7.0")] +[assembly: AssemblyVersion("1.8.0.14218")] +[assembly: AssemblyFileVersion("1.8.0.14218")] +[assembly: AssemblyInformationalVersion("1.8.0")] [assembly: CLSCompliant(true)] diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs index 8a4b3421304dd..d47bef9a1d931 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs @@ -33,8 +33,8 @@ // The following GUID is for the ID of the typelib if this project is exposed to COM [assembly: Guid("5b571661-17f4-4f29-8c7d-0edb38ca9b55")] -[assembly: AssemblyVersion("1.7.0.11707")] -[assembly: AssemblyFileVersion("1.7.0.11707")] -[assembly: AssemblyInformationalVersion("1.7.0")] +[assembly: AssemblyVersion("1.8.0.14218")] +[assembly: AssemblyFileVersion("1.8.0.14218")] +[assembly: AssemblyInformationalVersion("1.8.0")] [assembly: CLSCompliant(true)] diff --git a/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs index 48310a7529675..82e27b130de6f 100644 --- a/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs @@ -31,6 +31,6 @@ [assembly: Guid("0f9702ec-da7d-4ce5-b4b7-73310c885355")] -[assembly: AssemblyVersion("1.7.0.11707")] -[assembly: AssemblyFileVersion("1.7.0.11707")] -[assembly: AssemblyInformationalVersion("1.7.0")] +[assembly: AssemblyVersion("1.8.0.14218")] +[assembly: AssemblyFileVersion("1.8.0.14218")] +[assembly: AssemblyInformationalVersion("1.8.0")] diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs index 2423f2d2e39ba..4f5503960de9f 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs @@ -31,6 +31,6 @@ [assembly: Guid("41a0cb95-3435-4c78-b867-900b28e2c9ee")] -[assembly: AssemblyVersion("1.7.0.11707")] -[assembly: AssemblyFileVersion("1.7.0.11707")] -[assembly: AssemblyInformationalVersion("1.7.0")] +[assembly: AssemblyVersion("1.8.0.14218")] +[assembly: AssemblyFileVersion("1.8.0.14218")] +[assembly: AssemblyInformationalVersion("1.8.0")] diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs index 4cb86b51ad7be..471e7e972319c 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs @@ -31,6 +31,6 @@ [assembly: Guid("ce65ec7c-d3cf-41ad-8f45-f90d5af68d77")] -[assembly: AssemblyVersion("1.7.0.11707")] -[assembly: AssemblyFileVersion("1.7.0.11707")] -[assembly: AssemblyInformationalVersion("1.7.0")] +[assembly: AssemblyVersion("1.8.0.14218")] +[assembly: AssemblyFileVersion("1.8.0.14218")] +[assembly: AssemblyInformationalVersion("1.8.0")] diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml index 860c64d6aafaf..14c70eb46c913 100644 --- a/modules/rest-http/pom.xml +++ b/modules/rest-http/pom.xml @@ -31,7 +31,7 @@ ignite-rest-http - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml index 06f43b9085db4..3920e0a8ad3ec 100644 --- a/modules/scalar-2.10/pom.xml +++ b/modules/scalar-2.10/pom.xml @@ -31,7 +31,7 @@ ignite-scalar_2.10 - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml index e42c7106de28b..eb8a24cfd0790 100644 --- a/modules/scalar/pom.xml +++ b/modules/scalar/pom.xml @@ -31,7 +31,7 @@ ignite-scalar - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml index e06f0517119f0..e4e0005e527ad 100644 --- a/modules/schedule/pom.xml +++ b/modules/schedule/pom.xml @@ -31,7 +31,7 @@ ignite-schedule - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/schema-import-db/pom.xml b/modules/schema-import-db/pom.xml index b34281b2548a8..ba6b80ec86e29 100644 --- a/modules/schema-import-db/pom.xml +++ b/modules/schema-import-db/pom.xml @@ -31,7 +31,7 @@ ignite-schema-import-db - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml index 271e513cfa698..c8f341f670d0f 100644 --- a/modules/schema-import/pom.xml +++ b/modules/schema-import/pom.xml @@ -31,7 +31,7 @@ ignite-schema-import - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml index 85195b0624eb8..e9733e0c41456 100644 --- a/modules/slf4j/pom.xml +++ b/modules/slf4j/pom.xml @@ -31,7 +31,7 @@ ignite-slf4j - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml index c02599bc3182c..b39012c458682 100644 --- a/modules/spark-2.10/pom.xml +++ b/modules/spark-2.10/pom.xml @@ -31,7 +31,7 @@ ignite-spark_2.10 - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml index 4bde34fd75b52..25da109ce4a02 100644 --- a/modules/spark/pom.xml +++ b/modules/spark/pom.xml @@ -31,7 +31,7 @@ ignite-spark - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml index 352b51c86d4d9..a746c93252566 100644 --- a/modules/spring/pom.xml +++ b/modules/spring/pom.xml @@ -31,7 +31,7 @@ ignite-spring - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml index aec09484c50c3..9b1a6330d6310 100644 --- a/modules/ssh/pom.xml +++ b/modules/ssh/pom.xml @@ -31,7 +31,7 @@ ignite-ssh - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/storm/pom.xml b/modules/storm/pom.xml index 712f910e3a1b7..17d92cd57794a 100644 --- a/modules/storm/pom.xml +++ b/modules/storm/pom.xml @@ -31,7 +31,7 @@ ignite-storm - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml index 4f65542955d3d..232a1c2f11965 100644 --- a/modules/tools/pom.xml +++ b/modules/tools/pom.xml @@ -31,7 +31,7 @@ ignite-tools - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/twitter/pom.xml b/modules/twitter/pom.xml index 8f0d91739c9c3..4afd8bf3cc43b 100644 --- a/modules/twitter/pom.xml +++ b/modules/twitter/pom.xml @@ -31,7 +31,7 @@ ignite-twitter - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml index a621017e3e152..2042d03f2201b 100644 --- a/modules/urideploy/pom.xml +++ b/modules/urideploy/pom.xml @@ -31,7 +31,7 @@ ignite-urideploy - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/visor-console-2.10/pom.xml b/modules/visor-console-2.10/pom.xml index a1d2987aee76d..71599bada490c 100644 --- a/modules/visor-console-2.10/pom.xml +++ b/modules/visor-console-2.10/pom.xml @@ -31,7 +31,7 @@ ignite-visor-console_2.10 - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml index 3dd9bd86064ee..c8a7c6ebe9668 100644 --- a/modules/visor-console/pom.xml +++ b/modules/visor-console/pom.xml @@ -31,7 +31,7 @@ ignite-visor-console - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml index d0956507b218a..ddbb474017703 100644 --- a/modules/visor-plugins/pom.xml +++ b/modules/visor-plugins/pom.xml @@ -31,7 +31,7 @@ ignite-visor-plugins - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/web-console/pom.xml b/modules/web-console/pom.xml index e66252c48a875..18821b45ccd45 100644 --- a/modules/web-console/pom.xml +++ b/modules/web-console/pom.xml @@ -31,7 +31,7 @@ ignite-web-console - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/web-console/web-agent/pom.xml b/modules/web-console/web-agent/pom.xml index 530a27289e8aa..736136ab97301 100644 --- a/modules/web-console/web-agent/pom.xml +++ b/modules/web-console/web-agent/pom.xml @@ -32,7 +32,7 @@ ignite-web-agent jar - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/web/ignite-appserver-test/pom.xml b/modules/web/ignite-appserver-test/pom.xml index 045bbfa22d304..30b6e1583bb81 100644 --- a/modules/web/ignite-appserver-test/pom.xml +++ b/modules/web/ignite-appserver-test/pom.xml @@ -30,7 +30,7 @@ ignite-appserver-test jar - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/web/ignite-websphere-test/pom.xml b/modules/web/ignite-websphere-test/pom.xml index 912756046940d..5e4c4150916d5 100644 --- a/modules/web/ignite-websphere-test/pom.xml +++ b/modules/web/ignite-websphere-test/pom.xml @@ -30,7 +30,7 @@ ignite-websphere-test war - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/web/pom.xml b/modules/web/pom.xml index 710c803c1c0ac..8235b5bfc64cd 100644 --- a/modules/web/pom.xml +++ b/modules/web/pom.xml @@ -31,7 +31,7 @@ ignite-web - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml index 51d714e84a58d..cf404a700f1b0 100644 --- a/modules/yardstick/pom.xml +++ b/modules/yardstick/pom.xml @@ -31,7 +31,7 @@ ignite-yardstick - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/yarn/pom.xml b/modules/yarn/pom.xml index 884176a37259e..322924406b454 100644 --- a/modules/yarn/pom.xml +++ b/modules/yarn/pom.xml @@ -31,7 +31,7 @@ ignite-yarn - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml index 41c381bac3d55..ec9ff7b0d1ec3 100644 --- a/modules/zookeeper/pom.xml +++ b/modules/zookeeper/pom.xml @@ -31,7 +31,7 @@ ignite-zookeeper - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT http://ignite.apache.org diff --git a/pom.xml b/pom.xml index 070b5c8034e9a..45a2238fe3e04 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ org.apache.ignite apache-ignite - 1.7.0-SNAPSHOT + 1.8.0-SNAPSHOT pom From 75997afc762aff0581e53adc25c3efa329f48c6e Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Thu, 15 Sep 2016 13:23:44 +0300 Subject: [PATCH 133/487] Hadoop: Removed invalid test. --- .../ignite/igfs/HadoopFIleSystemFactorySelfTest.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFIleSystemFactorySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFIleSystemFactorySelfTest.java index 7cf1e4bff9497..5be3a6413c8ee 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFIleSystemFactorySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFIleSystemFactorySelfTest.java @@ -35,14 +35,12 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; -import org.apache.ignite.testframework.GridTestUtils; import org.jetbrains.annotations.Nullable; import java.io.Externalizable; import java.io.File; import java.io.FileOutputStream; import java.net.URI; -import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicInteger; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; @@ -117,14 +115,6 @@ public void testCustomFactory() throws Exception { assert primary.exists(IGFS_PATH_DUAL); assert secondary.exists(IGFS_PATH_DUAL); - GridTestUtils.assertThrows(null, new Callable() { - @Override public Object call() throws Exception { - primary.mkdirs(IGFS_PATH_PROXY); - - return null; - } - }, IgfsInvalidPathException.class, null); - // Create remote instance. FileSystem fs = FileSystem.get(URI.create("igfs://primary:primary@127.0.0.1:10500/"), baseConfiguration()); From 3a94f27e6bdee1672a2a94faf0b5b63ab218b476 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Thu, 15 Sep 2016 13:58:38 +0300 Subject: [PATCH 134/487] IGNITE-3902: Hadoop: fixed incorrect context classloader management. --- .../fs/BasicHadoopFileSystemFactory.java | 17 ++---- .../hadoop/HadoopDefaultJobInfo.java | 1 - .../processors/hadoop/HadoopUtils.java | 53 +++++++++++-------- .../processors/hadoop/v2/HadoopV2Job.java | 32 +++++------ .../hadoop/v2/HadoopV2JobResourceManager.java | 5 +- .../hadoop/v2/HadoopV2TaskContext.java | 15 +++--- 6 files changed, 63 insertions(+), 60 deletions(-) diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java index 06f76c3ba1aaa..a01bfaf690d40 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java @@ -93,20 +93,13 @@ protected FileSystem getWithMappedName(String usrName) throws IOException { // FileSystem.get() might delegate to ServiceLoader to get the list of file system implementation. // And ServiceLoader is known to be sensitive to context classloader. Therefore, we change context // classloader to classloader of current class to avoid strange class-cast-exceptions. - ClassLoader ctxClsLdr = Thread.currentThread().getContextClassLoader(); - ClassLoader clsLdr = getClass().getClassLoader(); + ClassLoader oldLdr = HadoopUtils.setContextClassLoader(getClass().getClassLoader()); - if (ctxClsLdr == clsLdr) + try { return create(usrName); - else { - Thread.currentThread().setContextClassLoader(clsLdr); - - try { - return create(usrName); - } - finally { - Thread.currentThread().setContextClassLoader(ctxClsLdr); - } + } + finally { + HadoopUtils.restoreContextClassLoader(oldLdr); } } catch (InterruptedException e) { diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java index be2d9ca849ce0..1382c1fff84c7 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java @@ -92,7 +92,6 @@ public HadoopDefaultJobInfo(String jobName, String user, boolean hasCombiner, in return constructor.newInstance(jobId, this, log, libNames); } - // NB: java.lang.NoClassDefFoundError may be thrown from Class#getConstructor() call. catch (Throwable t) { if (t instanceof Error) throw (Error)t; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java index 44d871a459954..65d9810ee909a 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java @@ -43,6 +43,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.hadoop.v2.HadoopSplitWrapper; import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; /** * Hadoop utility methods. @@ -326,33 +327,13 @@ public static File taskLocalDir(UUID locNodeId, HadoopTaskInfo info) throws Igni * @return New instance of {@link Configuration}. */ public static Configuration safeCreateConfiguration() { - final ClassLoader cl0 = Thread.currentThread().getContextClassLoader(); - - Thread.currentThread().setContextClassLoader(Configuration.class.getClassLoader()); + final ClassLoader oldLdr = setContextClassLoader(Configuration.class.getClassLoader()); try { return new Configuration(); } finally { - Thread.currentThread().setContextClassLoader(cl0); - } - } - - /** - * Creates {@link JobConf} in a correct class loader context to avoid caching - * of inappropriate class loader in the Configuration object. - * @return New instance of {@link JobConf}. - */ - public static JobConf safeCreateJobConf() { - final ClassLoader cl0 = Thread.currentThread().getContextClassLoader(); - - Thread.currentThread().setContextClassLoader(JobConf.class.getClassLoader()); - - try { - return new JobConf(); - } - finally { - Thread.currentThread().setContextClassLoader(cl0); + restoreContextClassLoader(oldLdr); } } @@ -381,6 +362,33 @@ public static List sortInputSplits(Collection e : jobInfo.properties().entrySet()) - jobConf.set(e.getKey(), e.getValue()); + for (Map.Entry e : jobInfo.properties().entrySet()) + jobConf.set(e.getKey(), e.getValue()); - jobCtx = new JobContextImpl(jobConf, hadoopJobID); + jobCtx = new JobContextImpl(jobConf, hadoopJobID); - rsrcMgr = new HadoopV2JobResourceManager(jobId, jobCtx, log, this); + rsrcMgr = new HadoopV2JobResourceManager(jobId, jobCtx, log, this); + } + finally { + HadoopUtils.setContextClassLoader(oldLdr); + } } /** {@inheritDoc} */ @@ -166,7 +171,7 @@ public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, Ignite /** {@inheritDoc} */ @Override public Collection input() throws IgniteCheckedException { - Thread.currentThread().setContextClassLoader(jobConf.getClassLoader()); + ClassLoader oldLdr = HadoopUtils.setContextClassLoader(jobConf.getClassLoader()); try { String jobDirPath = jobConf.get(MRJobConfig.MAPREDUCE_JOB_DIR); @@ -223,7 +228,7 @@ public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, Ignite } } finally { - Thread.currentThread().setContextClassLoader(null); + HadoopUtils.restoreContextClassLoader(oldLdr); } } @@ -296,16 +301,13 @@ public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, Ignite this.locNodeId = locNodeId; - assert ((HadoopClassLoader)getClass().getClassLoader()).name() - .equals(HadoopClassLoader.nameForJob(this.locNodeId)); - - Thread.currentThread().setContextClassLoader(jobConf.getClassLoader()); + ClassLoader oldLdr = HadoopUtils.setContextClassLoader(getClass().getClassLoader()); try { rsrcMgr.prepareJobEnvironment(!external, jobLocalDir(locNodeId, jobId)); } finally { - Thread.currentThread().setContextClassLoader(null); + HadoopUtils.restoreContextClassLoader(oldLdr); } } diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java index 67ab600bcb7c7..33aef60c63039 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java @@ -37,6 +37,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.processors.hadoop.HadoopJobId; +import org.apache.ignite.internal.processors.hadoop.HadoopUtils; import org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemsUtils; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; @@ -94,7 +95,7 @@ public HadoopV2JobResourceManager(HadoopJobId jobId, JobContextImpl ctx, IgniteL private void setLocalFSWorkingDirectory(File dir) throws IOException { JobConf cfg = ctx.getJobConf(); - Thread.currentThread().setContextClassLoader(cfg.getClassLoader()); + ClassLoader oldLdr = HadoopUtils.setContextClassLoader(cfg.getClassLoader()); try { cfg.set(HadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP, dir.getAbsolutePath()); @@ -103,7 +104,7 @@ private void setLocalFSWorkingDirectory(File dir) throws IOException { FileSystem.getLocal(cfg).setWorkingDirectory(new Path(dir.getAbsolutePath())); } finally { - Thread.currentThread().setContextClassLoader(null); + HadoopUtils.restoreContextClassLoader(oldLdr); } } diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java index 68c9ff8d26d62..4b1121c45e8ed 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java @@ -57,6 +57,7 @@ import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; import org.apache.ignite.internal.processors.hadoop.HadoopTaskType; +import org.apache.ignite.internal.processors.hadoop.HadoopUtils; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounter; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCountersImpl; @@ -158,7 +159,7 @@ public HadoopV2TaskContext(HadoopTaskInfo taskInfo, HadoopJob job, HadoopJobId j this.locNodeId = locNodeId; // Before create JobConf instance we should set new context class loader. - Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); + ClassLoader oldLdr = HadoopUtils.setContextClassLoader(getClass().getClassLoader()); try { JobConf jobConf = new JobConf(); @@ -180,7 +181,7 @@ public HadoopV2TaskContext(HadoopTaskInfo taskInfo, HadoopJob job, HadoopJobId j useNewCombiner = jobConf.getCombinerClass() == null; } finally { - Thread.currentThread().setContextClassLoader(null); + HadoopUtils.restoreContextClassLoader(oldLdr); } } @@ -229,9 +230,9 @@ private HadoopTask createTask() { /** {@inheritDoc} */ @Override public void run() throws IgniteCheckedException { - try { - Thread.currentThread().setContextClassLoader(jobConf().getClassLoader()); + ClassLoader oldLdr = HadoopUtils.setContextClassLoader(jobConf().getClassLoader()); + try { try { task = createTask(); } @@ -258,7 +259,7 @@ private HadoopTask createTask() { finally { task = null; - Thread.currentThread().setContextClassLoader(null); + HadoopUtils.restoreContextClassLoader(oldLdr); } } @@ -289,7 +290,7 @@ private HadoopTask createTask() { locDir = jobLocalDir(locNodeId, taskInfo().jobId()); } - Thread.currentThread().setContextClassLoader(jobConf().getClassLoader()); + ClassLoader oldLdr = HadoopUtils.setContextClassLoader(jobConf().getClassLoader()); try { FileSystem.get(jobConf()); @@ -305,7 +306,7 @@ private HadoopTask createTask() { throw transformException(e); } finally { - Thread.currentThread().setContextClassLoader(null); + HadoopUtils.restoreContextClassLoader(oldLdr); } } From 0dc9713ac3124c0ea34d949271a813c992873656 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Thu, 15 Sep 2016 14:01:19 +0300 Subject: [PATCH 135/487] IGNITE-3906: Hadoop: implemented additional user libs facility. --- .../processors/hadoop/HadoopClassLoader.java | 6 +- .../hadoop/HadoopClasspathMain.java | 2 +- .../hadoop/HadoopClasspathUtils.java | 230 +++++++++++++--- .../processors/hadoop/HadoopTestUtils.java | 73 ++++- .../hadoop/HadoopUserLibsSelfTest.java | 260 ++++++++++++++++++ .../testsuites/IgniteHadoopTestSuite.java | 3 + 6 files changed, 536 insertions(+), 38 deletions(-) create mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopUserLibsSelfTest.java diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java index 389de8cef12c5..2e0e271b78b5d 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java @@ -61,10 +61,8 @@ * unavailable for parent. */ public class HadoopClassLoader extends URLClassLoader implements ClassCache { - /** - * We are very parallel capable. - */ static { + // We are very parallel capable. registerAsParallelCapable(); } @@ -498,7 +496,7 @@ public static Collection hadoopUrls() throws IgniteCheckedException { return hadoopUrls; try { - hadoopUrls = HadoopClasspathUtils.classpathUrls(); + hadoopUrls = HadoopClasspathUtils.classpathForClassLoader(); } catch (IOException e) { throw new IgniteCheckedException("Failed to resolve Hadoop JAR locations: " + e.getMessage(), e); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathMain.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathMain.java index 5279b7deaf2b2..40694967a278a 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathMain.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathMain.java @@ -36,7 +36,7 @@ public static void main(String[] args) throws Exception { StringBuilder sb = new StringBuilder(); - for (String path : HadoopClasspathUtils.classpathForJavaProcess()) + for (String path : HadoopClasspathUtils.classpathForProcess()) sb.append(path).append(separator); System.out.println(sb); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathUtils.java index 121fcab697155..f5c2814d0f3dd 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathUtils.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathUtils.java @@ -27,6 +27,7 @@ import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collection; +import java.util.LinkedList; import java.util.List; /** @@ -36,18 +37,21 @@ public class HadoopClasspathUtils { /** Prefix directory. */ public static final String PREFIX = "HADOOP_PREFIX"; - /** Home directory. */ + /** Hadoop home directory. */ public static final String HOME = "HADOOP_HOME"; - /** Home directory. */ + /** Hadoop common directory. */ public static final String COMMON_HOME = "HADOOP_COMMON_HOME"; - /** Home directory. */ + /** Hadoop HDFS directory. */ public static final String HDFS_HOME = "HADOOP_HDFS_HOME"; - /** Home directory. */ + /** Hadoop mapred directory. */ public static final String MAPRED_HOME = "HADOOP_MAPRED_HOME"; + /** Arbitrary additional dependencies. Compliant with standard Java classpath resolution. */ + public static final String HADOOP_USER_LIBS = "HADOOP_USER_LIBS"; + /** Empty string. */ private static final String EMPTY_STR = ""; @@ -57,16 +61,20 @@ public class HadoopClasspathUtils { * @return List of the class path elements. * @throws IOException If failed. */ - public static List classpathForJavaProcess() throws IOException { + public static List classpathForProcess() throws IOException { List res = new ArrayList<>(); for (final SearchDirectory dir : classpathDirectories()) { - if (dir.hasFilter()) { - for (File file : dir.files()) + File[] files = dir.files(); + + if (dir.useWildcard()) { + if (files.length > 0) + res.add(dir.absolutePath() + File.separator + '*'); + } + else { + for (File file : files) res.add(file.getAbsolutePath()); } - else - res.add(dir.absolutePath() + File.separator + '*'); } return res; @@ -78,7 +86,7 @@ public static List classpathForJavaProcess() throws IOException { * @return List of class path URLs. * @throws IOException If failed. */ - public static List classpathUrls() throws IOException { + public static List classpathForClassLoader() throws IOException { List res = new ArrayList<>(); for (SearchDirectory dir : classpathDirectories()) { @@ -182,17 +190,70 @@ private static Collection classpathDirectories() throws IOExcep Collection res = new ArrayList<>(); - res.add(new SearchDirectory(new File(loc.common(), "lib"), null)); - res.add(new SearchDirectory(new File(loc.hdfs(), "lib"), null)); - res.add(new SearchDirectory(new File(loc.mapred(), "lib"), null)); + res.add(new SearchDirectory(new File(loc.common(), "lib"), AcceptAllDirectoryFilter.INSTANCE)); + res.add(new SearchDirectory(new File(loc.hdfs(), "lib"), AcceptAllDirectoryFilter.INSTANCE)); + res.add(new SearchDirectory(new File(loc.mapred(), "lib"), AcceptAllDirectoryFilter.INSTANCE)); + + res.add(new SearchDirectory(new File(loc.common()), new PrefixDirectoryFilter("hadoop-common-"))); + res.add(new SearchDirectory(new File(loc.common()), new PrefixDirectoryFilter("hadoop-auth-"))); - res.add(new SearchDirectory(new File(loc.common()), "hadoop-common-")); - res.add(new SearchDirectory(new File(loc.common()), "hadoop-auth-")); + res.add(new SearchDirectory(new File(loc.hdfs()), new PrefixDirectoryFilter("hadoop-hdfs-"))); - res.add(new SearchDirectory(new File(loc.hdfs()), "hadoop-hdfs-")); + res.add(new SearchDirectory(new File(loc.mapred()), + new PrefixDirectoryFilter("hadoop-mapreduce-client-common"))); + res.add(new SearchDirectory(new File(loc.mapred()), + new PrefixDirectoryFilter("hadoop-mapreduce-client-core"))); - res.add(new SearchDirectory(new File(loc.mapred()), "hadoop-mapreduce-client-common")); - res.add(new SearchDirectory(new File(loc.mapred()), "hadoop-mapreduce-client-core")); + res.addAll(parseUserLibs()); + + return res; + } + + /** + * Parse user libs. + * + * @return Parsed libs search patterns. + * @throws IOException If failed. + */ + static Collection parseUserLibs() throws IOException { + return parseUserLibs(systemOrEnv(HADOOP_USER_LIBS, null)); + } + + /** + * Parse user libs. + * + * @param str String. + * @return Result. + * @throws IOException If failed. + */ + static Collection parseUserLibs(String str) throws IOException { + Collection res = new LinkedList<>(); + + if (!isEmpty(str)) { + String[] tokens = normalize(str).split(File.pathSeparator); + + for (String token : tokens) { + // Skip empty tokens. + if (isEmpty(token)) + continue; + + File file = new File(token); + File dir = file.getParentFile(); + + if (token.endsWith("*")) { + assert dir != null; + + res.add(new SearchDirectory(dir, AcceptAllDirectoryFilter.INSTANCE, false)); + } + else { + // Met "/" or "C:\" pattern, nothing to do with it. + if (dir == null) + continue; + + res.add(new SearchDirectory(dir, new ExactDirectoryFilter(file.getName()), false)); + } + } + } return res; } @@ -238,58 +299,163 @@ private static boolean isEmpty(String val) { return val == null || val.isEmpty(); } + /** + * NOramlize the string. + * + * @param str String. + * @return Normalized string. + */ + private static String normalize(String str) { + assert str != null; + + return str.trim().toLowerCase(); + } + /** * Simple pair-like structure to hold directory name and a mask assigned to it. */ - private static class SearchDirectory { + static class SearchDirectory { /** File. */ private final File dir; - /** The mask. */ - private final String filter; + /** Filter. */ + private final DirectoryFilter filter; + + /** Whether directory must exist. */ + private final boolean strict; + + /** + * Constructor for directory search with strict rule. + * + * @param dir Directory. + * @param filter Filter. + * @throws IOException If failed. + */ + private SearchDirectory(File dir, DirectoryFilter filter) throws IOException { + this(dir, filter, true); + } /** * Constructor. * * @param dir Directory. * @param filter Filter. + * @param strict Whether directory must exist. + * @throws IOException If failed. */ - private SearchDirectory(File dir, String filter) throws IOException { + private SearchDirectory(File dir, DirectoryFilter filter, boolean strict) throws IOException { this.dir = dir; this.filter = filter; + this.strict = strict; - if (!exists(dir.getAbsolutePath())) + if (strict && !exists(dir.getAbsolutePath())) throw new IOException("Directory cannot be read: " + dir.getAbsolutePath()); } /** * @return Absolute path. */ - private String absolutePath() { + String absolutePath() { return dir.getAbsolutePath(); } /** * @return Child files. */ - private File[] files() throws IOException { + File[] files() throws IOException { File[] files = dir.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { - return filter == null || name.startsWith(filter); + return filter.test(name); } }); - if (files == null) - throw new IOException("Path is not a directory. [dir=" + dir + ']'); + if (files == null) { + if (strict) + throw new IOException("Failed to get directory files [dir=" + dir + ']'); + else + return new File[0]; + } + else + return files; + } - return files; + /** + * @return {@code True} if wildcard can be used. + */ + boolean useWildcard() { + return filter instanceof AcceptAllDirectoryFilter; } + } + /** + * Directory filter interface. + */ + static interface DirectoryFilter { /** - * @return {@code True} if filter exists. + * Test if file with this name should be included. + * + * @param name File name. + * @return {@code True} if passed. */ - private boolean hasFilter() { - return filter != null; + public boolean test(String name); + } + + /** + * Filter to accept all files. + */ + static class AcceptAllDirectoryFilter implements DirectoryFilter { + /** Singleton instance. */ + public static final AcceptAllDirectoryFilter INSTANCE = new AcceptAllDirectoryFilter(); + + /** {@inheritDoc} */ + @Override public boolean test(String name) { + return true; + } + } + + /** + * Filter which uses prefix to filter files. + */ + static class PrefixDirectoryFilter implements DirectoryFilter { + /** Prefix. */ + private final String prefix; + + /** + * Constructor. + * + * @param prefix Prefix. + */ + public PrefixDirectoryFilter(String prefix) { + assert prefix != null; + + this.prefix = normalize(prefix); + } + + /** {@inheritDoc} */ + @Override public boolean test(String name) { + return normalize(name).startsWith(prefix); + } + } + + /** + * Filter which uses exact comparison. + */ + static class ExactDirectoryFilter implements DirectoryFilter { + /** Name. */ + private final String name; + + /** + * Constructor. + * + * @param name Name. + */ + public ExactDirectoryFilter(String name) { + this.name = normalize(name); + } + + /** {@inheritDoc} */ + @Override public boolean test(String name) { + return normalize(name).equals(this.name); } } } \ No newline at end of file diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestUtils.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestUtils.java index 9ebad78e3b42c..da0d922a988f0 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestUtils.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestUtils.java @@ -17,14 +17,18 @@ package org.apache.ignite.internal.processors.hadoop; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + import java.io.BufferedReader; +import java.io.File; import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Map; -import org.apache.ignite.internal.util.typedef.F; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -33,6 +37,41 @@ * Utility class for tests. */ public class HadoopTestUtils { + /** Base test directory. */ + private static final File BASE_TEST_DIR = new File(U.getIgniteHome() + "/work/test/hadoop/"); + + /** + * @return Base directory for tests. + */ + public static File baseTestDir() { + return BASE_TEST_DIR; + } + + /** + * Get test directory. + * + * @param parts Parts. + * @return Directory. + */ + public static File testDir(String... parts) { + File res = BASE_TEST_DIR; + + if (parts != null) { + for (String part : parts) + res = new File(res, part); + } + + return res; + } + + /** + * Clear base test directory. + */ + public static void clearBaseTestDir() { + if (baseTestDir().exists()) + assert delete(baseTestDir()); + } + /** * Checks that job statistics file contains valid strings only. * @@ -40,6 +79,7 @@ public class HadoopTestUtils { * @return Amount of events. * @throws IOException If failed. */ + @SuppressWarnings("ResultOfMethodCallIgnored") public static long simpleCheckJobStatFile(BufferedReader reader) throws IOException { Collection phases = new HashSet<>(); @@ -104,4 +144,35 @@ public static long simpleCheckJobStatFile(BufferedReader reader) throws IOExcept return evtCnt; } + + /** + * Deletes file or directory with all sub-directories and files. + * + * @param file File or directory to delete. + * @return {@code true} if and only if the file or directory is successfully deleted, + * {@code false} otherwise + */ + public static boolean delete(@Nullable File file) { + if (file == null) + return false; + + boolean res = true; + + if (file.isDirectory()) { + File[] files = file.listFiles(); + + if (files != null && files.length > 0) + for (File file1 : files) + if (file1.isDirectory()) + res &= delete(file1); + else + res &= file1.delete(); + + res &= file.delete(); + } + else + res = file.delete(); + + return res; + } } \ No newline at end of file diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopUserLibsSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopUserLibsSelfTest.java new file mode 100644 index 0000000000000..9e3c8f402917a --- /dev/null +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopUserLibsSelfTest.java @@ -0,0 +1,260 @@ +/* + * 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.hadoop; + +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; + +/** + * Tests for user libs parsing. + */ +public class HadoopUserLibsSelfTest extends GridCommonAbstractTest { + /** Directory 1. */ + private static final File DIR_1 = HadoopTestUtils.testDir("dir1"); + + /** File 1 in directory 1. */ + private static final File FILE_1_1 = new File(DIR_1, "file1.jar"); + + /** File 2 in directory 1. */ + private static final File FILE_1_2 = new File(DIR_1, "file2.jar"); + + /** Directory 2. */ + private static final File DIR_2 = HadoopTestUtils.testDir("dir2"); + + /** File 1 in directory 2. */ + private static final File FILE_2_1 = new File(DIR_2, "file1.jar"); + + /** File 2 in directory 2. */ + private static final File FILE_2_2 = new File(DIR_2, "file2.jar"); + + /** Missing directory. */ + private static final File MISSING_DIR = HadoopTestUtils.testDir("missing_dir"); + + /** Missing file. */ + private static final File MISSING_FILE = new File(MISSING_DIR, "file.jar"); + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + HadoopTestUtils.clearBaseTestDir(); + + assert DIR_1.mkdirs(); + assert DIR_2.mkdirs(); + + assert FILE_1_1.createNewFile(); + assert FILE_1_2.createNewFile(); + assert FILE_2_1.createNewFile(); + assert FILE_2_2.createNewFile(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + // Sanity checks before test start. + ensureExists(FILE_1_1); + ensureExists(FILE_1_2); + ensureExists(FILE_2_1); + ensureExists(FILE_2_2); + + ensureNotExists(MISSING_DIR); + ensureNotExists(MISSING_FILE); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + HadoopTestUtils.clearBaseTestDir(); + } + + /** + * Test null or empty user libs. + * + * @throws Exception If failed. + */ + public void testNullOrEmptyUserLibs() throws Exception { + assert parse(null).isEmpty(); + assert parse("").isEmpty(); + } + + /** + * Test single file. + * + * @throws Exception If failed. + */ + public void testSingle() throws Exception { + Collection res = parse(single(FILE_1_1)); + + assert res.size() == 1; + assert res.contains(FILE_1_1); + + res = parse(single(MISSING_FILE)); + + assert res.size() == 0; + } + + /** + * Test multiple files. + * + * @throws Exception If failed. + */ + public void testMultiple() throws Exception { + Collection res = + parse(merge(single(FILE_1_1), single(FILE_1_2), single(FILE_2_1), single(FILE_2_2), single(MISSING_FILE))); + + assert res.size() == 4; + assert res.contains(FILE_1_1); + assert res.contains(FILE_1_2); + assert res.contains(FILE_2_1); + assert res.contains(FILE_2_2); + } + + /** + * Test single wildcard. + * + * @throws Exception If failed. + */ + public void testSingleWildcard() throws Exception { + Collection res = parse(wildcard(DIR_1)); + + assert res.size() == 2; + assert res.contains(FILE_1_1); + assert res.contains(FILE_1_2); + + res = parse(wildcard(MISSING_DIR)); + + assert res.size() == 0; + } + + /** + * Test multiple wildcards. + * + * @throws Exception If failed. + */ + public void testMultipleWildcards() throws Exception { + Collection res = parse(merge(wildcard(DIR_1), wildcard(DIR_2), wildcard(MISSING_DIR))); + + assert res.size() == 4; + assert res.contains(FILE_1_1); + assert res.contains(FILE_1_2); + assert res.contains(FILE_2_1); + assert res.contains(FILE_2_2); + } + + /** + * Test mixed tokens. + * + * @throws Exception If failed. + */ + public void testMixed() throws Exception { + String str = merge( + single(FILE_1_1), + wildcard(DIR_2), + single(MISSING_FILE), + wildcard(MISSING_DIR) + ); + + Collection res = parse(str); + + assert res.size() == 3; + assert res.contains(FILE_1_1); + assert res.contains(FILE_2_1); + assert res.contains(FILE_2_2); + } + /** + * Ensure provided file exists. + * + * @param file File. + */ + private static void ensureExists(File file) { + assert file.exists(); + } + + /** + * Ensure provided file doesn't exist. + * + * @param file File. + */ + private static void ensureNotExists(File file) { + assert !file.exists(); + } + + /** + * Merge string using path separator. + * + * @param vals Values. + * @return Result. + */ + private static String merge(String... vals) { + StringBuilder res = new StringBuilder(); + + if (vals != null) { + boolean first = true; + + for (String val : vals) { + if (first) + first = false; + else + res.append(File.pathSeparatorChar); + + res.append(val); + } + } + + return res.toString(); + } + + /** + * Parse string. + * + * @param str String. + * @return Files. + * @throws IOException If failed. + */ + Collection parse(String str) throws IOException { + Collection dirs = HadoopClasspathUtils.parseUserLibs(str); + + Collection res = new HashSet<>(); + + for (HadoopClasspathUtils.SearchDirectory dir : dirs) + Collections.addAll(res, dir.files()); + + return res; + } + + /** + * Get absolute path to a single file. + * + * @param file File. + * @return Path. + */ + private static String single(File file) { + return file.getAbsolutePath(); + } + + /** + * Create a wildcard. + * + * @param file File. + * @return Wildcard. + */ + private static String wildcard(File file) { + return file.getAbsolutePath() + File.separatorChar + "*"; + } +} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java index 33745479f2a03..603fd5b3c2ad5 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java @@ -68,6 +68,7 @@ import org.apache.ignite.internal.processors.hadoop.HadoopTaskExecutionSelfTest; import org.apache.ignite.internal.processors.hadoop.HadoopTasksV1Test; import org.apache.ignite.internal.processors.hadoop.HadoopTasksV2Test; +import org.apache.ignite.internal.processors.hadoop.HadoopUserLibsSelfTest; import org.apache.ignite.internal.processors.hadoop.HadoopV2JobSelfTest; import org.apache.ignite.internal.processors.hadoop.HadoopValidationSelfTest; import org.apache.ignite.internal.processors.hadoop.HadoopWeightedMapReducePlannerTest; @@ -110,6 +111,8 @@ public static TestSuite suite() throws Exception { TestSuite suite = new TestSuite("Ignite Hadoop MR Test Suite"); + suite.addTest(new TestSuite(ldr.loadClass(HadoopUserLibsSelfTest.class.getName()))); + suite.addTest(new TestSuite(ldr.loadClass(HadoopDefaultMapReducePlannerSelfTest.class.getName()))); suite.addTest(new TestSuite(ldr.loadClass(HadoopWeightedMapReducePlannerTest.class.getName()))); From 2474e2b331da34e16acc7a69c4703335e67b142e Mon Sep 17 00:00:00 2001 From: shtykh_roman Date: Thu, 15 Sep 2016 10:40:38 +0900 Subject: [PATCH 136/487] IGNITE-3421: Add ability to configure maxIterCnt property in GridCacheQueryManager. - Fixes #1055. Signed-off-by: shtykh_roman --- .../configuration/CacheConfiguration.java | 35 +++++++++++++++++-- .../cache/query/GridCacheQueryManager.java | 13 +++---- ...iteCacheReplicatedFieldsQuerySelfTest.java | 6 ++-- .../IgniteCacheReplicatedQuerySelfTest.java | 4 +-- 4 files changed, 42 insertions(+), 16 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java index 64b7e1f0c0f7b..e28aad5919bd1 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java @@ -191,6 +191,9 @@ public class CacheConfiguration extends MutableConfiguration { /** Default batch size for write-behind cache store. */ public static final int DFLT_WRITE_BEHIND_BATCH_SIZE = 512; + /** Default maximum number of query iterators that can be stored. */ + public static final int DFLT_MAX_QUERY_ITERATOR_CNT = 1024; + /** Default value for load previous value flag. */ public static final boolean DFLT_LOAD_PREV_VAL = false; @@ -209,7 +212,7 @@ public class CacheConfiguration extends MutableConfiguration { /** Default size for onheap SQL row cache size. */ public static final int DFLT_SQL_ONHEAP_ROW_CACHE_SIZE = 10 * 1024; - /** Default value for keep binary in store behavior .*/ + /** Default value for keep binary in store behavior . */ @SuppressWarnings({"UnnecessaryBoxing", "BooleanConstructorCall"}) public static final Boolean DFLT_STORE_KEEP_BINARY = new Boolean(false); @@ -334,6 +337,9 @@ public class CacheConfiguration extends MutableConfiguration { /** Maximum batch size for write-behind cache store. */ private int writeBehindBatchSize = DFLT_WRITE_BEHIND_BATCH_SIZE; + /** Maximum number of query iterators that can be stored. */ + private int maxQryIterCnt = DFLT_MAX_QUERY_ITERATOR_CNT; + /** Memory mode. */ private CacheMemoryMode memMode = DFLT_MEMORY_MODE; @@ -886,7 +892,7 @@ public CacheConfiguration setLoadPreviousValue(boolean loadPrevVal) { /** * Sets factory for persistent storage for cache data. - + * * @param storeFactory Cache store factory. * @return {@code this} for chaining. */ @@ -1631,6 +1637,31 @@ public CacheConfiguration setOffHeapMaxMemory(long offHeapMaxMem) { return this; } + /** + * Gets maximum number of query iterators that can be stored. Iterators are stored to + * support query pagination when each page of data is sent to user's node only on demand. + * Increase this property if you are running and processing lots of queries in parallel. + *

          + * Default value is {@link #DFLT_MAX_QUERY_ITERATOR_CNT}. + * + * @return Maximum number of query iterators that can be stored. + */ + public int getMaxQueryIteratorsCount() { + return maxQryIterCnt; + } + + /** + * Sets maximum number of query iterators that can be stored. + * + * @param maxQryIterCnt Maximum number of query iterators that can be stored. + * @return {@code this} for chaining. + */ + public CacheConfiguration setMaxQueryIteratorsCount(int maxQryIterCnt) { + this.maxQryIterCnt = maxQryIterCnt; + + return this; + } + /** * Gets memory mode for cache. Memory mode helps control whether value is stored in on-heap memory, * off-heap memory, or swap space. Refer to {@link CacheMemoryMode} for more info. 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 454ce047130dd..9699f09eacfec 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 @@ -134,9 +134,6 @@ */ @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") public abstract class GridCacheQueryManager extends GridCacheManagerAdapter { - /** */ - public static int MAX_ITERATORS = 1000; - /** */ protected GridQueryProcessor qryProc; @@ -176,7 +173,7 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte @Override public void start0() throws IgniteCheckedException { qryProc = cctx.kernalContext().query(); space = cctx.name(); - maxIterCnt = MAX_ITERATORS; + maxIterCnt = cctx.config().getMaxQueryIteratorsCount(); lsnr = new GridLocalEventListener() { @Override public void onEvent(Event evt) { @@ -557,7 +554,7 @@ private QueryResult executeQuery(GridCacheQueryAdapter qry, throw new IgniteCheckedException("Received next page request after iterator was removed. " + "Consider increasing maximum number of stored iterators (see " + - "GridCacheConfiguration.getMaximumQueryIteratorCount() configuration property)."); + "CacheConfiguration.getMaxQueryIteratorsCount() configuration property)."); } QueryResult res; @@ -698,7 +695,7 @@ private FieldsResult executeFieldsQuery(GridCacheQueryAdapter qry, @Nullable throw new IgniteCheckedException("Received next page request after iterator was removed. " + "Consider increasing maximum number of stored iterators (see " + - "GridCacheConfiguration.getMaximumQueryIteratorCount() configuration property)."); + "CacheConfiguration.getMaxQueryIteratorsCount() configuration property)."); } if (qry.type() == SQL_FIELDS) { @@ -1654,7 +1651,6 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { else data.add(!loc ? new GridCacheQueryResponseEntry<>(key, val) : F.t(key, val)); - if (!loc) { if (++cnt == pageSize || !iter.hasNext()) { boolean finished = !iter.hasNext(); @@ -1815,10 +1811,9 @@ protected GridCloseableIterator scanQueryLocal(final GridCacheQueryAdapter qry, if (qry.keepBinary()) entry = cctx.cache().keepBinary().getEntry(next.getKey()); - else + else entry = cctx.cache().getEntry(next.getKey()); - return transform.apply(entry); } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedFieldsQuerySelfTest.java index a01a3ff5c9a14..4ec840c11ffd6 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedFieldsQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedFieldsQuerySelfTest.java @@ -24,8 +24,8 @@ import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractFieldsQuerySelfTest; -import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; import org.apache.ignite.testframework.GridTestUtils; import static org.apache.ignite.cache.CacheMode.REPLICATED; @@ -52,11 +52,11 @@ public void testLostIterator() throws Exception { QueryCursor> qry = null; - int maximumQueryIteratorCount = GridCacheQueryManager.MAX_ITERATORS; + int maximumQueryIteratorCount = cache.getConfiguration(CacheConfiguration.class).getMaxQueryIteratorsCount(); for (int i = 0; i < maximumQueryIteratorCount + 1; i++) { QueryCursor> q = cache - .query(new SqlFieldsQuery("select _key from Integer where _key >= 0 order by _key")); + .query(new SqlFieldsQuery("select _key from Integer where _key >= 0 order by _key")); assertEquals(0, q.iterator().next().get(0)); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java index e462ccea0aff3..06adb688637d9 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java @@ -37,13 +37,13 @@ import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlQuery; import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.Event; import org.apache.ignite.events.EventType; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractQuerySelfTest; -import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridCloseableIterator; @@ -325,7 +325,7 @@ public void testLostIterator() throws Exception { QueryCursor> fut = null; - for (int i = 0; i < GridCacheQueryManager.MAX_ITERATORS + 1; i++) { + for (int i = 0; i < cache.getConfiguration(CacheConfiguration.class).getMaxQueryIteratorsCount() + 1; i++) { QueryCursor> q = cache.query(new SqlQuery(Integer.class, "_key >= 0 order by _key")); From 3f98797d7fa0cc8a9823bfcf1d8a82a6e0793492 Mon Sep 17 00:00:00 2001 From: shtykh_roman Date: Thu, 15 Sep 2016 10:40:38 +0900 Subject: [PATCH 137/487] IGNITE-3421: Add ability to configure maxIterCnt property in GridCacheQueryManager. - Fixes #1055. Signed-off-by: shtykh_roman --- .../configuration/CacheConfiguration.java | 35 +++++++++++++++++-- .../cache/query/GridCacheQueryManager.java | 13 +++---- ...iteCacheReplicatedFieldsQuerySelfTest.java | 6 ++-- .../IgniteCacheReplicatedQuerySelfTest.java | 4 +-- 4 files changed, 42 insertions(+), 16 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java index 64b7e1f0c0f7b..e28aad5919bd1 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java @@ -191,6 +191,9 @@ public class CacheConfiguration extends MutableConfiguration { /** Default batch size for write-behind cache store. */ public static final int DFLT_WRITE_BEHIND_BATCH_SIZE = 512; + /** Default maximum number of query iterators that can be stored. */ + public static final int DFLT_MAX_QUERY_ITERATOR_CNT = 1024; + /** Default value for load previous value flag. */ public static final boolean DFLT_LOAD_PREV_VAL = false; @@ -209,7 +212,7 @@ public class CacheConfiguration extends MutableConfiguration { /** Default size for onheap SQL row cache size. */ public static final int DFLT_SQL_ONHEAP_ROW_CACHE_SIZE = 10 * 1024; - /** Default value for keep binary in store behavior .*/ + /** Default value for keep binary in store behavior . */ @SuppressWarnings({"UnnecessaryBoxing", "BooleanConstructorCall"}) public static final Boolean DFLT_STORE_KEEP_BINARY = new Boolean(false); @@ -334,6 +337,9 @@ public class CacheConfiguration extends MutableConfiguration { /** Maximum batch size for write-behind cache store. */ private int writeBehindBatchSize = DFLT_WRITE_BEHIND_BATCH_SIZE; + /** Maximum number of query iterators that can be stored. */ + private int maxQryIterCnt = DFLT_MAX_QUERY_ITERATOR_CNT; + /** Memory mode. */ private CacheMemoryMode memMode = DFLT_MEMORY_MODE; @@ -886,7 +892,7 @@ public CacheConfiguration setLoadPreviousValue(boolean loadPrevVal) { /** * Sets factory for persistent storage for cache data. - + * * @param storeFactory Cache store factory. * @return {@code this} for chaining. */ @@ -1631,6 +1637,31 @@ public CacheConfiguration setOffHeapMaxMemory(long offHeapMaxMem) { return this; } + /** + * Gets maximum number of query iterators that can be stored. Iterators are stored to + * support query pagination when each page of data is sent to user's node only on demand. + * Increase this property if you are running and processing lots of queries in parallel. + *

          + * Default value is {@link #DFLT_MAX_QUERY_ITERATOR_CNT}. + * + * @return Maximum number of query iterators that can be stored. + */ + public int getMaxQueryIteratorsCount() { + return maxQryIterCnt; + } + + /** + * Sets maximum number of query iterators that can be stored. + * + * @param maxQryIterCnt Maximum number of query iterators that can be stored. + * @return {@code this} for chaining. + */ + public CacheConfiguration setMaxQueryIteratorsCount(int maxQryIterCnt) { + this.maxQryIterCnt = maxQryIterCnt; + + return this; + } + /** * Gets memory mode for cache. Memory mode helps control whether value is stored in on-heap memory, * off-heap memory, or swap space. Refer to {@link CacheMemoryMode} for more info. 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 4b4275ffd4aba..b1fa5899c55aa 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 @@ -134,9 +134,6 @@ */ @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") public abstract class GridCacheQueryManager extends GridCacheManagerAdapter { - /** */ - public static int MAX_ITERATORS = 1000; - /** */ protected GridQueryProcessor qryProc; @@ -176,7 +173,7 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte @Override public void start0() throws IgniteCheckedException { qryProc = cctx.kernalContext().query(); space = cctx.name(); - maxIterCnt = MAX_ITERATORS; + maxIterCnt = cctx.config().getMaxQueryIteratorsCount(); lsnr = new GridLocalEventListener() { @Override public void onEvent(Event evt) { @@ -538,7 +535,7 @@ private QueryResult executeQuery(GridCacheQueryAdapter qry, throw new IgniteCheckedException("Received next page request after iterator was removed. " + "Consider increasing maximum number of stored iterators (see " + - "GridCacheConfiguration.getMaximumQueryIteratorCount() configuration property)."); + "CacheConfiguration.getMaxQueryIteratorsCount() configuration property)."); } QueryResult res; @@ -660,7 +657,7 @@ private FieldsResult executeFieldsQuery(GridCacheQueryAdapter qry, @Nullable throw new IgniteCheckedException("Received next page request after iterator was removed. " + "Consider increasing maximum number of stored iterators (see " + - "GridCacheConfiguration.getMaximumQueryIteratorCount() configuration property)."); + "CacheConfiguration.getMaxQueryIteratorsCount() configuration property)."); } if (qry.type() == SQL_FIELDS) { @@ -1615,7 +1612,6 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { else data.add(!loc ? new GridCacheQueryResponseEntry<>(key, val) : F.t(key, val)); - if (!loc) { if (++cnt == pageSize || !iter.hasNext()) { boolean finished = !iter.hasNext(); @@ -1776,10 +1772,9 @@ protected GridCloseableIterator scanQueryLocal(final GridCacheQueryAdapter qry, if (qry.keepBinary()) entry = cctx.cache().keepBinary().getEntry(next.getKey()); - else + else entry = cctx.cache().getEntry(next.getKey()); - return transform.apply(entry); } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedFieldsQuerySelfTest.java index a01a3ff5c9a14..4ec840c11ffd6 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedFieldsQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedFieldsQuerySelfTest.java @@ -24,8 +24,8 @@ import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractFieldsQuerySelfTest; -import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; import org.apache.ignite.testframework.GridTestUtils; import static org.apache.ignite.cache.CacheMode.REPLICATED; @@ -52,11 +52,11 @@ public void testLostIterator() throws Exception { QueryCursor> qry = null; - int maximumQueryIteratorCount = GridCacheQueryManager.MAX_ITERATORS; + int maximumQueryIteratorCount = cache.getConfiguration(CacheConfiguration.class).getMaxQueryIteratorsCount(); for (int i = 0; i < maximumQueryIteratorCount + 1; i++) { QueryCursor> q = cache - .query(new SqlFieldsQuery("select _key from Integer where _key >= 0 order by _key")); + .query(new SqlFieldsQuery("select _key from Integer where _key >= 0 order by _key")); assertEquals(0, q.iterator().next().get(0)); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java index e462ccea0aff3..06adb688637d9 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java @@ -37,13 +37,13 @@ import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlQuery; import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.Event; import org.apache.ignite.events.EventType; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractQuerySelfTest; -import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridCloseableIterator; @@ -325,7 +325,7 @@ public void testLostIterator() throws Exception { QueryCursor> fut = null; - for (int i = 0; i < GridCacheQueryManager.MAX_ITERATORS + 1; i++) { + for (int i = 0; i < cache.getConfiguration(CacheConfiguration.class).getMaxQueryIteratorsCount() + 1; i++) { QueryCursor> q = cache.query(new SqlQuery(Integer.class, "_key >= 0 order by _key")); From 886ed64fd6958d7dad1226ef1de7e75e7b29ff2f Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Thu, 18 Aug 2016 18:14:12 +0300 Subject: [PATCH 138/487] IGNITE-2943 .NET: Improve cache error propagation and interop performance --- .../platform/PlatformAbstractTarget.java | 17 +- .../platform/cache/PlatformCache.java | 335 +++++++++--------- .../dotnet/PlatformDotNetCacheStore.java | 12 +- .../platform/utils/PlatformFutureUtils.java | 6 +- .../platform/utils/PlatformUtils.java | 25 ++ .../include/ignite/impl/binary/binary_utils.h | 87 +++++ .../src/impl/binary/binary_reader_impl.cpp | 30 +- .../ignite/impl/interop/interop_target.h | 15 +- .../cpp/core/include/ignite/impl/operations.h | 47 ++- .../cpp/core/src/impl/cache/cache_impl.cpp | 14 +- .../core/src/impl/interop/interop_target.cpp | 41 ++- .../impl/transactions/transactions_impl.cpp | 5 + .../BenchmarkRunner.cs | 5 +- .../Interop/PlatformBenchmarkBase.cs | 2 +- .../Cache/CacheAbstractTest.cs | 2 +- .../Cache/Store/CacheStoreTest.cs | 39 +- .../Cache/Store/CacheTestStore.cs | 50 ++- .../Impl/Cache/CacheImpl.cs | 239 +++++++------ .../Apache.Ignite.Core/Impl/ExceptionUtils.cs | 10 +- .../Apache.Ignite.Core/Impl/PlatformTarget.cs | 80 ++++- .../Impl/Unmanaged/UnmanagedCallbacks.cs | 21 +- 21 files changed, 746 insertions(+), 336 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java index 0cd683d8da87e..0ca44530132c8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java @@ -38,6 +38,9 @@ public abstract class PlatformAbstractTarget implements PlatformTarget { /** Constant: FALSE. */ protected static final int FALSE = 0; + /** Constant: ERROR. */ + protected static final int ERROR = -1; + /** */ private static final int OP_META = -1; @@ -69,7 +72,7 @@ protected PlatformAbstractTarget(PlatformContext platformCtx) { return TRUE; } else - return processInStreamOutLong(type, reader); + return processInStreamOutLong(type, reader, mem); } catch (Exception e) { throw convertException(e); @@ -234,6 +237,18 @@ protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws return throwUnsupported(type); } + /** + * Process IN operation. + * + * @param type Type. + * @param reader Binary reader. + * @return Result. + * @throws IgniteCheckedException In case of exception. + */ + protected long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem) throws IgniteCheckedException { + return processInStreamOutLong(type, reader); + } + /** * Process IN-OUT operation. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index d572e8bc06e95..a7b6e418a9a4d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -42,10 +42,13 @@ import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery; import org.apache.ignite.internal.processors.platform.cache.query.PlatformFieldsQueryCursor; import org.apache.ignite.internal.processors.platform.cache.query.PlatformQueryCursor; +import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; +import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream; import org.apache.ignite.internal.processors.platform.utils.PlatformConfigurationUtils; import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils; import org.apache.ignite.internal.processors.platform.utils.PlatformListenable; import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; +import org.apache.ignite.internal.processors.platform.utils.PlatformWriterClosure; import org.apache.ignite.internal.util.GridConcurrentFactory; import org.apache.ignite.internal.util.future.IgniteFutureImpl; import org.apache.ignite.internal.util.typedef.C1; @@ -290,109 +293,207 @@ public PlatformCache withNoRetries() { } /** {@inheritDoc} */ - @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException { - switch (type) { - case OP_PUT: - cache.put(reader.readObjectDetached(), reader.readObjectDetached()); + @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem) throws IgniteCheckedException { + try { + switch (type) { + case OP_PUT: + cache.put(reader.readObjectDetached(), reader.readObjectDetached()); - return TRUE; + return TRUE; - case OP_REMOVE_BOOL: - return cache.remove(reader.readObjectDetached(), reader.readObjectDetached()) ? TRUE : FALSE; + case OP_GET: + return writeResult(mem, cache.get(reader.readObjectDetached())); - case OP_REMOVE_ALL: - cache.removeAll(PlatformUtils.readSet(reader)); + case OP_REMOVE_BOOL: + return cache.remove(reader.readObjectDetached(), reader.readObjectDetached()) ? TRUE : FALSE; - return TRUE; + case OP_REMOVE_ALL: + cache.removeAll(PlatformUtils.readSet(reader)); - case OP_PUT_ALL: - cache.putAll(PlatformUtils.readMap(reader)); + return TRUE; - return TRUE; + case OP_PUT_ALL: + cache.putAll(PlatformUtils.readMap(reader)); - case OP_LOC_EVICT: - cache.localEvict(PlatformUtils.readCollection(reader)); + return TRUE; - return TRUE; + case OP_LOC_EVICT: + cache.localEvict(PlatformUtils.readCollection(reader)); - case OP_CONTAINS_KEY: - return cache.containsKey(reader.readObjectDetached()) ? TRUE : FALSE; + return TRUE; - case OP_CONTAINS_KEYS: - return cache.containsKeys(PlatformUtils.readSet(reader)) ? TRUE : FALSE; + case OP_CONTAINS_KEY: + return cache.containsKey(reader.readObjectDetached()) ? TRUE : FALSE; - case OP_LOC_PROMOTE: { - cache.localPromote(PlatformUtils.readSet(reader)); + case OP_CONTAINS_KEYS: + return cache.containsKeys(PlatformUtils.readSet(reader)) ? TRUE : FALSE; - break; - } + case OP_LOC_PROMOTE: { + cache.localPromote(PlatformUtils.readSet(reader)); - case OP_REPLACE_3: - return cache.replace(reader.readObjectDetached(), reader.readObjectDetached(), - reader.readObjectDetached()) ? TRUE : FALSE; + return TRUE; + } - case OP_LOC_LOAD_CACHE: - loadCache0(reader, true); + case OP_REPLACE_3: + return cache.replace(reader.readObjectDetached(), reader.readObjectDetached(), + reader.readObjectDetached()) ? TRUE : FALSE; - break; + case OP_LOC_LOAD_CACHE: + loadCache0(reader, true); - case OP_LOAD_CACHE: - loadCache0(reader, false); + return TRUE; - break; + case OP_LOAD_CACHE: + loadCache0(reader, false); - case OP_CLEAR: - cache.clear(reader.readObjectDetached()); + return TRUE; - break; + case OP_CLEAR: + cache.clear(reader.readObjectDetached()); - case OP_CLEAR_ALL: - cache.clearAll(PlatformUtils.readSet(reader)); + return TRUE; - break; + case OP_CLEAR_ALL: + cache.clearAll(PlatformUtils.readSet(reader)); - case OP_LOCAL_CLEAR: - cache.localClear(reader.readObjectDetached()); + return TRUE; - break; + case OP_LOCAL_CLEAR: + cache.localClear(reader.readObjectDetached()); - case OP_LOCAL_CLEAR_ALL: - cache.localClearAll(PlatformUtils.readSet(reader)); + return TRUE; - break; + case OP_LOCAL_CLEAR_ALL: + cache.localClearAll(PlatformUtils.readSet(reader)); - case OP_PUT_IF_ABSENT: { - return cache.putIfAbsent(reader.readObjectDetached(), reader.readObjectDetached()) ? TRUE : FALSE; - } + return TRUE; - case OP_REPLACE_2: { - return cache.replace(reader.readObjectDetached(), reader.readObjectDetached()) ? TRUE : FALSE; - } + case OP_PUT_IF_ABSENT: + return cache.putIfAbsent(reader.readObjectDetached(), reader.readObjectDetached()) ? TRUE : FALSE; - case OP_REMOVE_OBJ: { - return cache.remove(reader.readObjectDetached()) ? TRUE : FALSE; - } + case OP_REPLACE_2: + return cache.replace(reader.readObjectDetached(), reader.readObjectDetached()) ? TRUE : FALSE; + + case OP_REMOVE_OBJ: + return cache.remove(reader.readObjectDetached()) ? TRUE : FALSE; + + case OP_IS_LOCAL_LOCKED: + return cache.isLocalLocked(reader.readObjectDetached(), reader.readBoolean()) ? TRUE : FALSE; + + case OP_LOAD_ALL: { + long futId = reader.readLong(); + boolean replaceExisting = reader.readBoolean(); + + CompletionListenable fut = new CompletionListenable(); + + PlatformFutureUtils.listen(platformCtx, fut, futId, PlatformFutureUtils.TYP_OBJ, null, this); + + cache.loadAll(PlatformUtils.readSet(reader), replaceExisting, fut); + + return TRUE; + } + + case OP_GET_AND_PUT: + return writeResult(mem, cache.getAndPut(reader.readObjectDetached(), reader.readObjectDetached())); + + case OP_GET_AND_REPLACE: + return writeResult(mem, cache.getAndReplace(reader.readObjectDetached(), reader.readObjectDetached())); + + case OP_GET_AND_REMOVE: + return writeResult(mem, cache.getAndRemove(reader.readObjectDetached())); + + case OP_GET_AND_PUT_IF_ABSENT: + return writeResult(mem, cache.getAndPutIfAbsent(reader.readObjectDetached(), reader.readObjectDetached())); + + case OP_PEEK: { + Object key = reader.readObjectDetached(); + + CachePeekMode[] modes = PlatformUtils.decodeCachePeekModes(reader.readInt()); + + return writeResult(mem, cache.localPeek(key, modes)); + } + + case OP_GET_ALL: { + Set keys = PlatformUtils.readSet(reader); + + Map entries = cache.getAll(keys); + + return writeResult(mem, entries, new PlatformWriterClosure() { + @Override public void write(BinaryRawWriterEx writer, Map val) { + PlatformUtils.writeNullableMap(writer, val); + } + }); + } + + case OP_INVOKE: { + Object key = reader.readObjectDetached(); - case OP_IS_LOCAL_LOCKED: - return cache.isLocalLocked(reader.readObjectDetached(), reader.readBoolean()) ? TRUE : FALSE; + CacheEntryProcessor proc = platformCtx.createCacheEntryProcessor(reader.readObjectDetached(), 0); - case OP_LOAD_ALL: { - long futId = reader.readLong(); - boolean replaceExisting = reader.readBoolean(); + return writeResult(mem, cache.invoke(key, proc)); + } - CompletionListenable fut = new CompletionListenable(); + case OP_INVOKE_ALL: { + Set keys = PlatformUtils.readSet(reader); - PlatformFutureUtils.listen(platformCtx, fut, futId, PlatformFutureUtils.TYP_OBJ, null, this); + CacheEntryProcessor proc = platformCtx.createCacheEntryProcessor(reader.readObjectDetached(), 0); - cache.loadAll(PlatformUtils.readSet(reader), replaceExisting, fut); + Map results = cache.invokeAll(keys, proc); - return TRUE; + return writeResult(mem, results, new PlatformWriterClosure() { + @Override public void write(BinaryRawWriterEx writer, Map val) { + writeInvokeAllResult(writer, val); + } + }); + } + + case OP_LOCK: + return registerLock(cache.lock(reader.readObjectDetached())); + + case OP_LOCK_ALL: + return registerLock(cache.lockAll(PlatformUtils.readCollection(reader))); } + } + catch (Exception e) { + PlatformOutputStream out = mem.output(); + BinaryRawWriterEx writer = platformCtx.writer(out); - default: - return super.processInStreamOutLong(type, reader); + Exception err = convertException(e); + + PlatformUtils.writeError(err, writer); + PlatformUtils.writeErrorData(err, writer); + + out.synchronize(); + + return ERROR; } + return super.processInStreamOutLong(type, reader, mem); + } + + /** + * Writes the result to reused stream, if any. + */ + private long writeResult(PlatformMemory mem, Object obj) { + return writeResult(mem, obj, null); + } + + /** + * Writes the result to reused stream, if any. + */ + private long writeResult(PlatformMemory mem, Object obj, PlatformWriterClosure clo) { + if (obj == null) + return FALSE; + + PlatformOutputStream out = mem.output(); + BinaryRawWriterEx writer = platformCtx.writer(out); + + if (clo == null) + writer.writeObjectDetached(obj); + else + clo.write(writer, obj); + + out.synchronize(); return TRUE; } @@ -554,106 +655,6 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc) { } } - /** {@inheritDoc} */ - @SuppressWarnings({"IfMayBeConditional", "ConstantConditions"}) - @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer) - throws IgniteCheckedException { - switch (type) { - case OP_GET: { - writer.writeObjectDetached(cache.get(reader.readObjectDetached())); - - break; - } - - case OP_GET_AND_PUT: { - writer.writeObjectDetached(cache.getAndPut(reader.readObjectDetached(), reader.readObjectDetached())); - - break; - } - - case OP_GET_AND_REPLACE: { - writer.writeObjectDetached(cache.getAndReplace(reader.readObjectDetached(), - reader.readObjectDetached())); - - break; - } - - case OP_GET_AND_REMOVE: { - writer.writeObjectDetached(cache.getAndRemove(reader.readObjectDetached())); - - break; - } - - case OP_GET_AND_PUT_IF_ABSENT: { - writer.writeObjectDetached(cache.getAndPutIfAbsent(reader.readObjectDetached(), reader.readObjectDetached())); - - break; - } - - case OP_PEEK: { - Object key = reader.readObjectDetached(); - - CachePeekMode[] modes = PlatformUtils.decodeCachePeekModes(reader.readInt()); - - writer.writeObjectDetached(cache.localPeek(key, modes)); - - break; - } - - case OP_GET_ALL: { - Set keys = PlatformUtils.readSet(reader); - - Map entries = cache.getAll(keys); - - PlatformUtils.writeNullableMap(writer, entries); - - break; - } - - case OP_INVOKE: { - Object key = reader.readObjectDetached(); - - CacheEntryProcessor proc = platformCtx.createCacheEntryProcessor(reader.readObjectDetached(), 0); - - try { - writer.writeObjectDetached(cache.invoke(key, proc)); - } - catch (EntryProcessorException ex) - { - if (ex.getCause() instanceof PlatformNativeException) - writer.writeObjectDetached(((PlatformNativeException)ex.getCause()).cause()); - else - throw ex; - } - - break; - } - - case OP_INVOKE_ALL: { - Set keys = PlatformUtils.readSet(reader); - - CacheEntryProcessor proc = platformCtx.createCacheEntryProcessor(reader.readObjectDetached(), 0); - - writeInvokeAllResult(writer, cache.invokeAll(keys, proc)); - - break; - } - - case OP_LOCK: - writer.writeLong(registerLock(cache.lock(reader.readObjectDetached()))); - - break; - - case OP_LOCK_ALL: - writer.writeLong(registerLock(cache.lockAll(PlatformUtils.readCollection(reader)))); - - break; - - default: - super.processInStreamOutStream(type, reader, writer); - } - } - /** {@inheritDoc} */ @Override public Exception convertException(Exception e) { if (e instanceof CachePartialUpdateException) @@ -699,7 +700,7 @@ private static void writeInvokeAllResult(BinaryRawWriterEx writer, Map task, @Nullable PlatformCacheStoreCallback cb) + protected void doInvoke(IgniteInClosureX task, @Nullable PlatformCacheStoreCallback cb) throws IgniteCheckedException{ try (PlatformMemory mem = platformCtx.memory().allocate()) { PlatformOutputStream out = mem.output(); @@ -403,7 +402,14 @@ protected int doInvoke(IgniteInClosureX task, @Nullable Platf out.synchronize(); - return platformCtx.gateway().cacheStoreInvoke(ptr, mem.pointer(), cb); + int res = platformCtx.gateway().cacheStoreInvoke(ptr, mem.pointer(), cb); + + if (res != 0) { + // Read error + Object nativeErr = platformCtx.reader(mem.input()).readObjectDetached(); + + throw platformCtx.createNativeException(nativeErr); + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java index 6692a23907783..5985d22ae54ce 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java @@ -25,7 +25,6 @@ import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway; import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream; -import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgniteInClosure; @@ -293,10 +292,7 @@ private static void writeFutureError(final PlatformContext ctx, long futPtr, Thr BinaryRawWriterEx outWriter = ctx.writer(out); - outWriter.writeString(err.getClass().getName()); - outWriter.writeString(err.getMessage()); - outWriter.writeString(X.getFullStackTrace(err)); - + PlatformUtils.writeError(err, outWriter); PlatformUtils.writeErrorData(err, outWriter); out.synchronize(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java index dd90fda4e0f94..ccdd59dcdbf11 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java @@ -574,6 +574,31 @@ private static void writeCacheEntryEvent(BinaryRawWriterEx writer, CacheEntryEve writer.writeObjectDetached(evt.getValue()); } + /** + * Writes error. + * + * @param ex Error. + * @param writer Writer. + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public static void writeError(Throwable ex, BinaryRawWriterEx writer) { + writer.writeObjectDetached(ex.getClass().getName()); + + writer.writeObjectDetached(ex.getMessage()); + + writer.writeObjectDetached(X.getFullStackTrace(ex)); + + PlatformNativeException nativeCause = X.cause(ex, PlatformNativeException.class); + + if (nativeCause != null) { + writer.writeBoolean(true); + + writer.writeObjectDetached(nativeCause.cause()); + } + else + writer.writeBoolean(false); + } + /** * Writer error data. * diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_utils.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_utils.h index 88130d85e2511..3abd651dc59e2 100644 --- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_utils.h +++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_utils.h @@ -26,6 +26,8 @@ #include "ignite/date.h" #include "ignite/timestamp.h" +#include "ignite/binary/binary_type.h" + namespace ignite { namespace impl @@ -538,7 +540,92 @@ namespace ignite */ static Timestamp MakeTimestampLocal(int year = 1900, int month = 1, int day = 1, int hour = 0, int min = 0, int sec = 0, long ns = 0); + + /** + * Get default value for the type. + * + * @return Null value for non primitive types and zeroes for primitives. + */ + template + static T GetDefaultValue() + { + ignite::binary::BinaryType binType; + + return binType.GetNull(); + } }; + + template<> + inline int8_t BinaryUtils::GetDefaultValue() + { + return 0; + } + + template<> + inline int16_t BinaryUtils::GetDefaultValue() + { + return 0; + } + + template<> + inline uint16_t BinaryUtils::GetDefaultValue() + { + return 0; + } + + template<> + inline int32_t BinaryUtils::GetDefaultValue() + { + return 0; + } + + template<> + inline int64_t BinaryUtils::GetDefaultValue() + { + return 0; + } + + template<> + inline bool BinaryUtils::GetDefaultValue() + { + return false; + } + + template<> + inline float BinaryUtils::GetDefaultValue() + { + return 0.0f; + } + + template<> + inline double BinaryUtils::GetDefaultValue() + { + return 0.0; + } + + template<> + inline Guid BinaryUtils::GetDefaultValue() + { + return Guid(); + } + + template<> + inline Date BinaryUtils::GetDefaultValue() + { + return Date(); + } + + template<> + inline Timestamp BinaryUtils::GetDefaultValue() + { + return Timestamp(); + } + + template<> + inline std::string BinaryUtils::GetDefaultValue() + { + return std::string(); + } } } } diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp index 33205e4bfd886..c3f4fcc6dc59c 100644 --- a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp +++ b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp @@ -676,49 +676,57 @@ namespace ignite template <> int8_t BinaryReaderImpl::ReadTopObject() { - return ReadTopObject0(IGNITE_TYPE_BYTE, BinaryUtils::ReadInt8, static_cast(0)); + return ReadTopObject0(IGNITE_TYPE_BYTE, BinaryUtils::ReadInt8, + BinaryUtils::GetDefaultValue()); } template <> bool BinaryReaderImpl::ReadTopObject() { - return ReadTopObject0(IGNITE_TYPE_BOOL, BinaryUtils::ReadBool, static_cast(0)); + return ReadTopObject0(IGNITE_TYPE_BOOL, BinaryUtils::ReadBool, + BinaryUtils::GetDefaultValue()); } template <> int16_t BinaryReaderImpl::ReadTopObject() { - return ReadTopObject0(IGNITE_TYPE_SHORT, BinaryUtils::ReadInt16, static_cast(0)); + return ReadTopObject0(IGNITE_TYPE_SHORT, BinaryUtils::ReadInt16, + BinaryUtils::GetDefaultValue()); } template <> uint16_t BinaryReaderImpl::ReadTopObject() { - return ReadTopObject0(IGNITE_TYPE_CHAR, BinaryUtils::ReadUInt16, static_cast(0)); + return ReadTopObject0(IGNITE_TYPE_CHAR, BinaryUtils::ReadUInt16, + BinaryUtils::GetDefaultValue()); } template <> int32_t BinaryReaderImpl::ReadTopObject() { - return ReadTopObject0(IGNITE_TYPE_INT, BinaryUtils::ReadInt32, static_cast(0)); + return ReadTopObject0(IGNITE_TYPE_INT, BinaryUtils::ReadInt32, + BinaryUtils::GetDefaultValue()); } template <> int64_t BinaryReaderImpl::ReadTopObject() { - return ReadTopObject0(IGNITE_TYPE_LONG, BinaryUtils::ReadInt64, static_cast(0)); + return ReadTopObject0(IGNITE_TYPE_LONG, BinaryUtils::ReadInt64, + BinaryUtils::GetDefaultValue()); } template <> float BinaryReaderImpl::ReadTopObject() { - return ReadTopObject0(IGNITE_TYPE_FLOAT, BinaryUtils::ReadFloat, static_cast(0)); + return ReadTopObject0(IGNITE_TYPE_FLOAT, BinaryUtils::ReadFloat, + BinaryUtils::GetDefaultValue()); } template <> double BinaryReaderImpl::ReadTopObject() { - return ReadTopObject0(IGNITE_TYPE_DOUBLE, BinaryUtils::ReadDouble, static_cast(0)); + return ReadTopObject0(IGNITE_TYPE_DOUBLE, BinaryUtils::ReadDouble, + BinaryUtils::GetDefaultValue()); } template <> @@ -729,7 +737,7 @@ namespace ignite if (typeId == IGNITE_TYPE_UUID) return BinaryUtils::ReadGuid(stream); else if (typeId == IGNITE_HDR_NULL) - return Guid(); + return BinaryUtils::GetDefaultValue(); else { int32_t pos = stream->Position() - 1; @@ -747,7 +755,7 @@ namespace ignite else if (typeId == IGNITE_TYPE_TIMESTAMP) return Date(BinaryUtils::ReadTimestamp(stream).GetMilliseconds()); else if (typeId == IGNITE_HDR_NULL) - return Date(); + return BinaryUtils::GetDefaultValue(); else { int32_t pos = stream->Position() - 1; @@ -763,7 +771,7 @@ namespace ignite if (typeId == IGNITE_TYPE_TIMESTAMP) return BinaryUtils::ReadTimestamp(stream); else if (typeId == IGNITE_HDR_NULL) - return Timestamp(); + return BinaryUtils::GetDefaultValue(); else { int32_t pos = stream->Position() - 1; diff --git a/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h b/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h index 8b6ebb9c47de1..4042fa2d3d9fa 100644 --- a/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h +++ b/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h @@ -68,14 +68,25 @@ namespace ignite /** * Internal out-in operation. + * Uses two independent memory pieces to write and read data. * * @param opType Operation type. * @param inOp Input. * @param outOp Output. * @param err Error. */ - void OutInOp(int32_t opType, InputOperation& inOp, OutputOperation& outOp, - IgniteError* err); + void OutInOp(int32_t opType, InputOperation& inOp, OutputOperation& outOp, IgniteError* err); + + /** + * Internal out-in operation. + * Uses single memory piece to write and read data. + * + * @param opType Operation type. + * @param inOp Input. + * @param outOp Output. + * @param err Error. + */ + void OutInOpX(int32_t opType, InputOperation& inOp, OutputOperation& outOp, IgniteError* err); /** * Get environment shared pointer. diff --git a/modules/platforms/cpp/core/include/ignite/impl/operations.h b/modules/platforms/cpp/core/include/ignite/impl/operations.h index ed01ecec90070..a8fef9308719f 100644 --- a/modules/platforms/cpp/core/include/ignite/impl/operations.h +++ b/modules/platforms/cpp/core/include/ignite/impl/operations.h @@ -27,6 +27,7 @@ #include "ignite/cache/cache_entry.h" #include "ignite/impl/binary/binary_reader_impl.h" #include "ignite/impl/binary/binary_writer_impl.h" +#include "ignite/impl/binary/binary_utils.h" #include "ignite/binary/binary.h" namespace ignite @@ -270,7 +271,12 @@ namespace ignite * * @param reader Reader. */ - virtual void ProcessOutput(ignite::impl::binary::BinaryReaderImpl& reader) = 0; + virtual void ProcessOutput(binary::BinaryReaderImpl& reader) = 0; + + /** + * Sets result to null value. + */ + virtual void SetNull() = 0; }; /** @@ -288,11 +294,16 @@ namespace ignite // No-op. } - virtual void ProcessOutput(ignite::impl::binary::BinaryReaderImpl& reader) + virtual void ProcessOutput(binary::BinaryReaderImpl& reader) { val = reader.ReadTopObject(); } + virtual void SetNull() + { + val = binary::BinaryUtils::GetDefaultValue(); + } + /** * Get value. * @@ -324,12 +335,18 @@ namespace ignite // No-op. } - virtual void ProcessOutput(ignite::impl::binary::BinaryReaderImpl& reader) + virtual void ProcessOutput(binary::BinaryReaderImpl& reader) { val1 = reader.ReadTopObject(); val2 = reader.ReadTopObject(); } + virtual void SetNull() + { + val1 = binary::BinaryUtils::GetDefaultValue(); + val2 = binary::BinaryUtils::GetDefaultValue(); + } + /** * Get value 1. * @@ -375,7 +392,7 @@ namespace ignite // No-op. } - virtual void ProcessOutput(ignite::impl::binary::BinaryReaderImpl& reader) + virtual void ProcessOutput(binary::BinaryReaderImpl& reader) { val1 = reader.ReadTopObject(); val2 = reader.ReadTopObject(); @@ -383,6 +400,14 @@ namespace ignite val4 = reader.ReadTopObject(); } + virtual void SetNull() + { + val1 = binary::BinaryUtils::GetDefaultValue(); + val2 = binary::BinaryUtils::GetDefaultValue(); + val3 = binary::BinaryUtils::GetDefaultValue(); + val4 = binary::BinaryUtils::GetDefaultValue(); + } + /** * Get value 1. * @@ -454,7 +479,7 @@ namespace ignite // No-op. } - virtual void ProcessOutput(ignite::impl::binary::BinaryReaderImpl& reader) + virtual void ProcessOutput(binary::BinaryReaderImpl& reader) { bool exists = reader.GetStream()->ReadBool(); @@ -475,6 +500,11 @@ namespace ignite } } + virtual void SetNull() + { + // No-op. + } + /** * Get value. * @@ -506,7 +536,7 @@ namespace ignite // No-op. } - virtual void ProcessOutput(ignite::impl::binary::BinaryReaderImpl& reader) + virtual void ProcessOutput(binary::BinaryReaderImpl& reader) { int32_t cnt = reader.ReadInt32(); @@ -519,6 +549,11 @@ namespace ignite } } + virtual void SetNull() + { + res->clear(); + } + private: /** Entries. */ std::vector>* res; diff --git a/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp b/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp index e728f55a3ae35..8197187d868d2 100644 --- a/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp +++ b/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp @@ -156,17 +156,17 @@ namespace ignite void CacheImpl::LocalPeek(InputOperation& inOp, OutputOperation& outOp, int32_t peekModes, IgniteError* err) { - OutInOp(OP_LOCAL_PEEK, inOp, outOp, err); + OutInOpX(OP_LOCAL_PEEK, inOp, outOp, err); } void CacheImpl::Get(InputOperation& inOp, OutputOperation& outOp, IgniteError* err) { - OutInOp(OP_GET, inOp, outOp, err); + OutInOpX(OP_GET, inOp, outOp, err); } void CacheImpl::GetAll(InputOperation& inOp, OutputOperation& outOp, IgniteError* err) { - OutInOp(OP_GET_ALL, inOp, outOp, err); + OutInOpX(OP_GET_ALL, inOp, outOp, err); } void CacheImpl::Put(InputOperation& inOp, IgniteError* err) @@ -181,17 +181,17 @@ namespace ignite void CacheImpl::GetAndPut(InputOperation& inOp, OutputOperation& outOp, IgniteError* err) { - OutInOp(OP_GET_AND_PUT, inOp, outOp, err); + OutInOpX(OP_GET_AND_PUT, inOp, outOp, err); } void CacheImpl::GetAndReplace(InputOperation& inOp, OutputOperation& outOp, IgniteError* err) { - OutInOp(OP_GET_AND_REPLACE, inOp, outOp, err); + OutInOpX(OP_GET_AND_REPLACE, inOp, outOp, err); } void CacheImpl::GetAndRemove(InputOperation& inOp, OutputOperation& outOp, IgniteError* err) { - OutInOp(OP_GET_AND_REMOVE, inOp, outOp, err); + OutInOpX(OP_GET_AND_REMOVE, inOp, outOp, err); } bool CacheImpl::PutIfAbsent(InputOperation& inOp, IgniteError* err) @@ -201,7 +201,7 @@ namespace ignite void CacheImpl::GetAndPutIfAbsent(InputOperation& inOp, OutputOperation& outOp, IgniteError* err) { - OutInOp(OP_GET_AND_PUT_IF_ABSENT, inOp, outOp, err); + OutInOpX(OP_GET_AND_PUT_IF_ABSENT, inOp, outOp, err); } bool CacheImpl::Replace(InputOperation& inOp, IgniteError* err) diff --git a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp index 05764c70d9447..5d17214204a71 100644 --- a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp +++ b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp @@ -31,6 +31,21 @@ namespace ignite { namespace interop { + /** + * Operation result. + */ + enum OperationResult + { + /** Null. */ + ResultNull = 0, + + /** Object. */ + ResultObject = 1, + + /** Error. */ + ResultError = -1 + }; + InteropTarget::InteropTarget(SharedPointer env, jobject javaRef) : env(env), javaRef(javaRef) { @@ -116,8 +131,7 @@ namespace ignite return false; } - void InteropTarget::OutInOp(int32_t opType, InputOperation& inOp, OutputOperation& outOp, - IgniteError* err) + void InteropTarget::OutInOp(int32_t opType, InputOperation& inOp, OutputOperation& outOp, IgniteError* err) { JniErrorInfo jniErr; @@ -137,6 +151,29 @@ namespace ignite ReadFrom(inMem.Get(), outOp); } } + + void InteropTarget::OutInOpX(int32_t opType, InputOperation& inOp, OutputOperation& outOp, IgniteError* err) + { + JniErrorInfo jniErr; + + SharedPointer outInMem = env.Get()->AllocateMemory(); + + int64_t outInPtr = WriteTo(outInMem.Get(), inOp, err); + + if (outInPtr) + { + int64_t res = env.Get()->Context()->TargetInStreamOutLong(javaRef, opType, outInPtr, &jniErr); + + IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); + + if (jniErr.code == IGNITE_JNI_ERR_SUCCESS && res == ResultObject) + ReadFrom(outInMem.Get(), outOp); + else if (res == ResultNull) + outOp.SetNull(); + + //Read and process error if res == ResultError here. + } + } } } } \ No newline at end of file diff --git a/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp b/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp index 6c01332c666a6..fed43fcf38b54 100644 --- a/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp +++ b/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp @@ -145,6 +145,11 @@ namespace ignite val = TransactionMetrics(commitTime, rollbackTime, commits, rollbacks); } + virtual void SetNull() + { + // No-op. + } + /** * Get value. * diff --git a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/BenchmarkRunner.cs b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/BenchmarkRunner.cs index 5d8e78aa10420..40ae01e2f623d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/BenchmarkRunner.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/BenchmarkRunner.cs @@ -21,6 +21,7 @@ namespace Apache.Ignite.Benchmarks using System.Diagnostics; using System.Text; using Apache.Ignite.Benchmarks.Binary; + using Apache.Ignite.Benchmarks.Interop; /// /// Benchmark runner. @@ -35,8 +36,8 @@ internal class BenchmarkRunner public static void Main(string[] args) { args = new[] { - typeof(BinarizableReadBenchmark).FullName, - "-ConfigPath", @"modules\platforms\dotnet\Apache.Ignite.Benchmarks\Config\benchmark.xml", + typeof(GetBenchmark).FullName, + "-ConfigPath", @"C:\W\incubator-ignite\modules\platforms\dotnet\Apache.Ignite.Benchmarks\Config\benchmark.xml", "-Threads", "1", "-Warmup", "0", "-Duration", "60", diff --git a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Interop/PlatformBenchmarkBase.cs b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Interop/PlatformBenchmarkBase.cs index eeebed01b496e..f437eb876857d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Interop/PlatformBenchmarkBase.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Interop/PlatformBenchmarkBase.cs @@ -66,7 +66,7 @@ protected override void OnStarted() "-DIGNITE_QUIET=false", "-DIGNITE_NO_SHUTDOWN_HOOK=true" }, - JvmClasspath = Classpath ?? Core.Impl.Common.Classpath.CreateClasspath(), + JvmClasspath = Classpath ?? Core.Impl.Common.Classpath.CreateClasspath(forceTestClasspath: true), JvmDllPath = DllPath, SpringConfigUrl = ConfigPath }; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs index 62dc2df5f78c0..7627ce067c1b5 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs @@ -2906,7 +2906,7 @@ private static void AssertThrowsCacheEntryProcessorException(Action action, stri Assert.IsInstanceOf(ex); if (string.IsNullOrEmpty(containsText)) - Assert.AreEqual(ex.InnerException.Message, AddArgCacheEntryProcessor.ExceptionText); + Assert.AreEqual(AddArgCacheEntryProcessor.ExceptionText, ex.InnerException.Message); else Assert.IsTrue(ex.ToString().Contains(containsText)); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs index 8061e9f6bb41a..d39ccde391a96 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs @@ -178,13 +178,15 @@ public void BeforeTest() [TearDown] public void AfterTest() { + CacheTestStore.Reset(); + var cache = GetCache(); cache.Clear(); - Assert.IsTrue(cache.IsEmpty(), "Cache is not empty: " + cache.GetSize()); - - CacheTestStore.Reset(); + Assert.IsTrue(cache.IsEmpty(), + "Cache is not empty: " + + string.Join(", ", cache.Select(x => string.Format("[{0}:{1}]", x.Key, x.Value)))); TestUtils.AssertHandleRegistryHasItems(300, _storeCount, Ignition.GetIgnite(GridName)); @@ -210,6 +212,11 @@ public void TestLoadCache() // Test exception in filter Assert.Throws(() => cache.LoadCache(new ExceptionalEntryFilter(), 100, 10)); + + // Test exception in store + CacheTestStore.ThrowError = true; + CheckCustomStoreError(Assert.Throws(() => + cache.LoadCache(new CacheEntryFilter(), 100, 10)).InnerException); } [Test] @@ -262,6 +269,13 @@ public void TestLoadCacheAsync() { Assert.AreEqual("val_" + i, cache.GetAsync(i).Result); } + + // Test errors + CacheTestStore.ThrowError = true; + CheckCustomStoreError( + Assert.Throws( + () => cache.LocalLoadCacheAsync(new CacheEntryFilter(), 100, 10).Wait()) + .InnerException); } [Test] @@ -282,6 +296,13 @@ public void TestPutLoad() Assert.AreEqual("val", cache.Get(1)); Assert.AreEqual(1, cache.GetSize()); + + // Test errors + CacheTestStore.ThrowError = true; + CheckCustomStoreError(Assert.Throws(() => cache.Put(-2, "fail")).InnerException); + + cache.LocalEvict(new[] { 1 }); + CheckCustomStoreError(Assert.Throws(() => cache.Get(1)).InnerException); } [Test] @@ -418,8 +439,6 @@ public void TestTx() using (var tx = cache.Ignite.GetTransactions().TxStart()) { - CacheTestStore.ExpCommit = true; - tx.AddMeta("meta", 100); cache.Put(1, "val"); @@ -549,6 +568,16 @@ private IDictionary StoreMap() return Ignition.GetIgnite(GridName).GetOrCreateCache(cacheName); } + + private static void CheckCustomStoreError(Exception err) + { + var customErr = err as CacheTestStore.CustomStoreException ?? + err.InnerException as CacheTestStore.CustomStoreException; + + Assert.IsNotNull(customErr); + + Assert.AreEqual(customErr.Message, customErr.Details); + } } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestStore.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestStore.cs index b4b1670af5d22..4224835b96335 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestStore.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestStore.cs @@ -23,6 +23,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Store using System.Diagnostics; using System.Diagnostics.CodeAnalysis; using System.Linq; + using System.Runtime.Serialization; using System.Threading; using Apache.Ignite.Core.Cache.Store; using Apache.Ignite.Core.Resource; @@ -32,12 +33,12 @@ public class CacheTestStore : ICacheStore { public static readonly IDictionary Map = new ConcurrentDictionary(); - public static bool ExpCommit; - public static bool LoadMultithreaded; public static bool LoadObjects; + public static bool ThrowError; + [InstanceResource] private IIgnite _grid = null; @@ -54,13 +55,15 @@ public static void Reset() { Map.Clear(); - ExpCommit = false; LoadMultithreaded = false; LoadObjects = false; + ThrowError = false; } public void LoadCache(Action act, params object[] args) { + ThrowIfNeeded(); + Debug.Assert(_grid != null); if (LoadMultithreaded) @@ -91,6 +94,8 @@ public void LoadCache(Action act, params object[] args) public object Load(object key) { + ThrowIfNeeded(); + Debug.Assert(_grid != null); return Map[key]; @@ -98,6 +103,8 @@ public object Load(object key) public IDictionary LoadAll(ICollection keys) { + ThrowIfNeeded(); + Debug.Assert(_grid != null); return keys.OfType().ToDictionary(key => key, key => "val_" + key); @@ -105,6 +112,8 @@ public IDictionary LoadAll(ICollection keys) public void Write(object key, object val) { + ThrowIfNeeded(); + Debug.Assert(_grid != null); Map[key] = val; @@ -112,6 +121,8 @@ public void Write(object key, object val) public void WriteAll(IDictionary map) { + ThrowIfNeeded(); + Debug.Assert(_grid != null); foreach (DictionaryEntry e in map) @@ -120,6 +131,8 @@ public void WriteAll(IDictionary map) public void Delete(object key) { + ThrowIfNeeded(); + Debug.Assert(_grid != null); Map.Remove(key); @@ -127,6 +140,8 @@ public void Delete(object key) public void DeleteAll(ICollection keys) { + ThrowIfNeeded(); + Debug.Assert(_grid != null); foreach (object key in keys) @@ -151,5 +166,34 @@ public string StringProperty get { return stringProperty; } set { stringProperty = value; } } + + private static void ThrowIfNeeded() + { + if (ThrowError) + throw new CustomStoreException("Exception in cache store"); + } + + [Serializable] + public class CustomStoreException : Exception, ISerializable + { + public string Details { get; private set; } + + public CustomStoreException(string message) : base(message) + { + Details = message; + } + + protected CustomStoreException(SerializationInfo info, StreamingContext ctx) : base(info, ctx) + { + Details = info.GetString("details"); + } + + public override void GetObjectData(SerializationInfo info, StreamingContext context) + { + info.AddValue("details", Details); + + base.GetObjectData(info, context); + } + } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs index 32c59de1077ec..8ba3e29b0519c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs @@ -271,7 +271,7 @@ public Task LocalLoadCacheAsync(ICacheEntryFilter p, params object[] arg /// private void LoadCache0(ICacheEntryFilter p, object[] args, int opId) { - DoOutOp(opId, writer => + DoOutInOpX(opId, writer => { if (p != null) { @@ -284,7 +284,7 @@ private void LoadCache0(ICacheEntryFilter p, object[] args, int opId) writer.WriteObject(null); writer.WriteArray(args); - }); + }, ReadException); } /** */ @@ -296,7 +296,7 @@ public void LoadAll(IEnumerable keys, bool replaceExistingValues) /** */ public Task LoadAllAsync(IEnumerable keys, bool replaceExistingValues) { - return GetFuture((futId, futTyp) => DoOutOp((int) CacheOp.LoadAll, writer => + return GetFuture((futId, futTyp) => DoOutOp(CacheOp.LoadAll, writer => { writer.WriteLong(futId); writer.WriteBoolean(replaceExistingValues); @@ -309,7 +309,7 @@ public bool ContainsKey(TK key) { IgniteArgumentCheck.NotNull(key, "key"); - return DoOutOp((int)CacheOp.ContainsKey, key) == True; + return DoOutOp(CacheOp.ContainsKey, key); } /** */ @@ -325,7 +325,7 @@ public bool ContainsKeys(IEnumerable keys) { IgniteArgumentCheck.NotNull(keys, "keys"); - return DoOutOp((int)CacheOp.ContainsKeys, writer => WriteEnumerable(writer, keys)) == True; + return DoOutOp(CacheOp.ContainsKeys, writer => WriteEnumerable(writer, keys)); } /** */ @@ -354,11 +354,14 @@ public bool TryLocalPeek(TK key, out TV value, params CachePeekMode[] modes) { IgniteArgumentCheck.NotNull(key, "key"); - var res = DoOutInOpNullable((int)CacheOp.Peek, writer => - { - writer.Write(key); - writer.WriteInt(EncodePeekModes(modes)); - }); + var res = DoOutInOpX((int)CacheOp.Peek, + w => + { + w.Write(key); + w.WriteInt(EncodePeekModes(modes)); + }, + (s, r) => r == True ? new CacheResult(Unmarshal(s)) : new CacheResult(), + ReadException); value = res.Success ? res.Value : default(TV); @@ -389,19 +392,22 @@ public TV Get(TK key) { IgniteArgumentCheck.NotNull(key, "key"); - var result = DoOutInOpNullable((int) CacheOp.Get, key); - - if (!IsAsync) - { - if (!result.Success) - throw GetKeyNotFoundException(); + return DoOutInOpX((int) CacheOp.Get, + w => w.Write(key), + (stream, res) => + { + if (res == True) // Not null + { + Debug.Assert(!IsAsync); - return result.Value; - } + return Unmarshal(stream); + } - Debug.Assert(!result.Success); + if (!IsAsync) + throw GetKeyNotFoundException(); - return default(TV); + return default(TV); + }, ReadException); } /** */ @@ -426,7 +432,7 @@ public bool TryGet(TK key, out TV value) if (IsAsync) throw new InvalidOperationException("TryGet can't be used in async mode."); - var res = DoOutInOpNullable((int) CacheOp.Get, key); + var res = DoOutInOpNullable(CacheOp.Get, key); value = res.Value; @@ -448,14 +454,10 @@ public Task> TryGetAsync(TK key) { IgniteArgumentCheck.NotNull(keys, "keys"); - return DoOutInOp((int)CacheOp.GetAll, + return DoOutInOpX((int) CacheOp.GetAll, writer => WriteEnumerable(writer, keys), - input => - { - var reader = Marshaller.StartUnmarshal(input, _flagKeepBinary); - - return ReadGetAllDictionary(reader); - }); + (s, r) => r == True ? ReadGetAllDictionary(Marshaller.StartUnmarshal(s, _flagKeepBinary)) : null, + ReadException); } /** */ @@ -473,7 +475,7 @@ public void Put(TK key, TV val) IgniteArgumentCheck.NotNull(val, "val"); - DoOutOp((int)CacheOp.Put, key, val); + DoOutOp(CacheOp.Put, key, val); } /** */ @@ -491,7 +493,7 @@ public CacheResult GetAndPut(TK key, TV val) IgniteArgumentCheck.NotNull(val, "val"); - return DoOutInOpNullable((int)CacheOp.GetAndPut, key, val); + return DoOutInOpNullable(CacheOp.GetAndPut, key, val); } /** */ @@ -509,7 +511,7 @@ public CacheResult GetAndReplace(TK key, TV val) IgniteArgumentCheck.NotNull(val, "val"); - return DoOutInOpNullable((int) CacheOp.GetAndReplace, key, val); + return DoOutInOpNullable(CacheOp.GetAndReplace, key, val); } /** */ @@ -525,7 +527,7 @@ public CacheResult GetAndRemove(TK key) { IgniteArgumentCheck.NotNull(key, "key"); - return DoOutInOpNullable((int)CacheOp.GetAndRemove, key); + return DoOutInOpNullable(CacheOp.GetAndRemove, key); } /** */ @@ -543,7 +545,7 @@ public bool PutIfAbsent(TK key, TV val) IgniteArgumentCheck.NotNull(val, "val"); - return DoOutOp((int) CacheOp.PutIfAbsent, key, val) == True; + return DoOutOp(CacheOp.PutIfAbsent, key, val); } /** */ @@ -561,7 +563,7 @@ public CacheResult GetAndPutIfAbsent(TK key, TV val) IgniteArgumentCheck.NotNull(val, "val"); - return DoOutInOpNullable((int)CacheOp.GetAndPutIfAbsent, key, val); + return DoOutInOpNullable(CacheOp.GetAndPutIfAbsent, key, val); } /** */ @@ -579,7 +581,7 @@ public bool Replace(TK key, TV val) IgniteArgumentCheck.NotNull(val, "val"); - return DoOutOp((int) CacheOp.Replace2, key, val) == True; + return DoOutOp(CacheOp.Replace2, key, val); } /** */ @@ -599,7 +601,7 @@ public bool Replace(TK key, TV oldVal, TV newVal) IgniteArgumentCheck.NotNull(newVal, "newVal"); - return DoOutOp((int)CacheOp.Replace3, key, oldVal, newVal) == True; + return DoOutOp(CacheOp.Replace3, key, oldVal, newVal); } /** */ @@ -615,7 +617,7 @@ public void PutAll(IDictionary vals) { IgniteArgumentCheck.NotNull(vals, "vals"); - DoOutOp((int) CacheOp.PutAll, writer => WriteDictionary(writer, vals)); + DoOutOp(CacheOp.PutAll, writer => WriteDictionary(writer, vals)); } /** */ @@ -631,7 +633,7 @@ public void LocalEvict(IEnumerable keys) { IgniteArgumentCheck.NotNull(keys, "keys"); - DoOutOp((int) CacheOp.LocEvict, writer => WriteEnumerable(writer, keys)); + DoOutOp(CacheOp.LocEvict, writer => WriteEnumerable(writer, keys)); } /** */ @@ -653,7 +655,7 @@ public void Clear(TK key) { IgniteArgumentCheck.NotNull(key, "key"); - DoOutOp((int) CacheOp.Clear, key); + DoOutOp(CacheOp.Clear, key); } /** */ @@ -669,7 +671,7 @@ public void ClearAll(IEnumerable keys) { IgniteArgumentCheck.NotNull(keys, "keys"); - DoOutOp((int)CacheOp.ClearAll, writer => WriteEnumerable(writer, keys)); + DoOutOp(CacheOp.ClearAll, writer => WriteEnumerable(writer, keys)); } /** */ @@ -685,7 +687,7 @@ public void LocalClear(TK key) { IgniteArgumentCheck.NotNull(key, "key"); - DoOutOp((int) CacheOp.LocalClear, key); + DoOutOp(CacheOp.LocalClear, key); } /** */ @@ -693,7 +695,7 @@ public void LocalClearAll(IEnumerable keys) { IgniteArgumentCheck.NotNull(keys, "keys"); - DoOutOp((int)CacheOp.LocalClearAll, writer => WriteEnumerable(writer, keys)); + DoOutOp(CacheOp.LocalClearAll, writer => WriteEnumerable(writer, keys)); } /** */ @@ -701,7 +703,7 @@ public bool Remove(TK key) { IgniteArgumentCheck.NotNull(key, "key"); - return DoOutOp((int) CacheOp.RemoveObj, key) == True; + return DoOutOp(CacheOp.RemoveObj, key); } /** */ @@ -719,7 +721,7 @@ public bool Remove(TK key, TV val) IgniteArgumentCheck.NotNull(val, "val"); - return DoOutOp((int)CacheOp.RemoveBool, key, val) == True; + return DoOutOp(CacheOp.RemoveBool, key, val); } /** */ @@ -735,7 +737,7 @@ public void RemoveAll(IEnumerable keys) { IgniteArgumentCheck.NotNull(keys, "keys"); - DoOutOp((int)CacheOp.RemoveAll, writer => WriteEnumerable(writer, keys)); + DoOutOp(CacheOp.RemoveAll, writer => WriteEnumerable(writer, keys)); } /** */ @@ -798,7 +800,7 @@ public void LocalPromote(IEnumerable keys) { IgniteArgumentCheck.NotNull(keys, "keys"); - DoOutOp((int)CacheOp.LocPromote, writer => WriteEnumerable(writer, keys)); + DoOutOp(CacheOp.LocPromote, writer => WriteEnumerable(writer, keys)); } /** */ @@ -811,12 +813,14 @@ public void LocalPromote(IEnumerable keys) var holder = new CacheEntryProcessorHolder(processor, arg, (e, a) => processor.Process((IMutableCacheEntry)e, (TArg)a), typeof(TK), typeof(TV)); - return DoOutInOp((int)CacheOp.Invoke, writer => - { - writer.Write(key); - writer.Write(holder); - }, - input => GetResultOrThrow(Unmarshal(input))); + return DoOutInOpX((int) CacheOp.Invoke, + writer => + { + writer.Write(key); + writer.Write(holder); + }, + (input, res) => res == True ? Unmarshal(input) : default(TRes), + ReadException); } /** */ @@ -849,17 +853,19 @@ public void LocalPromote(IEnumerable keys) var holder = new CacheEntryProcessorHolder(processor, arg, (e, a) => processor.Process((IMutableCacheEntry)e, (TArg)a), typeof(TK), typeof(TV)); - return DoOutInOp((int) CacheOp.InvokeAll, + return DoOutInOpX((int) CacheOp.InvokeAll, writer => { WriteEnumerable(writer, keys); writer.Write(holder); }, - input => ReadInvokeAllResults(input)); + (input, res) => res == True ? ReadInvokeAllResults(input) : null, + ReadException); } /** */ - public Task>> InvokeAllAsync(IEnumerable keys, ICacheEntryProcessor processor, TArg arg) + public Task>> InvokeAllAsync(IEnumerable keys, + ICacheEntryProcessor processor, TArg arg) { AsyncInstance.InvokeAll(keys, processor, arg); @@ -871,10 +877,8 @@ public ICacheLock Lock(TK key) { IgniteArgumentCheck.NotNull(key, "key"); - return DoOutInOp((int)CacheOp.Lock, writer => - { - writer.Write(key); - }, input => new CacheLock(input.ReadInt(), Target)); + return DoOutInOpX((int) CacheOp.Lock, w => w.Write(key), + (stream, res) => new CacheLock(res, Target), ReadException); } /** */ @@ -882,10 +886,8 @@ public ICacheLock LockAll(IEnumerable keys) { IgniteArgumentCheck.NotNull(keys, "keys"); - return DoOutInOp((int)CacheOp.LockAll, writer => - { - WriteEnumerable(writer, keys); - }, input => new CacheLock(input.ReadInt(), Target)); + return DoOutInOpX((int) CacheOp.LockAll, w => WriteEnumerable(w, keys), + (stream, res) => new CacheLock(res, Target), ReadException); } /** */ @@ -893,11 +895,11 @@ public bool IsLocalLocked(TK key, bool byCurrentThread) { IgniteArgumentCheck.NotNull(key, "key"); - return DoOutOp((int)CacheOp.IsLocalLocked, writer => + return DoOutOp(CacheOp.IsLocalLocked, writer => { writer.Write(key); writer.WriteBoolean(byCurrentThread); - }) == True; + }); } /** */ @@ -1158,22 +1160,6 @@ private static int EncodePeekModes(CachePeekMode[] modes) return modesEncoded; } - /// - /// Unwraps an exception. - /// - /// Result type. - /// Object. - /// Result. - private static T GetResultOrThrow(object obj) - { - var err = obj as Exception; - - if (err != null) - throw err as CacheEntryProcessorException ?? new CacheEntryProcessorException(err); - - return obj == null ? default(T) : (T) obj; - } - /// /// Reads results of InvokeAll operation. /// @@ -1208,9 +1194,11 @@ private static T GetResultOrThrow(object obj) /// /// The stream. /// Exception. - private CacheEntryProcessorException ReadException(IBinaryStream inStream) + private Exception ReadException(IBinaryStream inStream) { - var item = Unmarshal(inStream); + var reader = Marshaller.StartUnmarshal(inStream, _flagKeepBinary); + + var item = reader.ReadObject(); var clsName = item as string; @@ -1219,8 +1207,9 @@ private CacheEntryProcessorException ReadException(IBinaryStream inStream) var msg = Unmarshal(inStream); var trace = Unmarshal(inStream); - - return new CacheEntryProcessorException(ExceptionUtils.GetException(_ignite, clsName, msg, trace)); + var inner = reader.ReadBoolean() ? reader.ReadObject() : null; + + return ExceptionUtils.GetException(_ignite, clsName, msg, trace, reader, inner); } /// @@ -1272,49 +1261,73 @@ private static KeyNotFoundException GetKeyNotFoundException() } /// - /// Perform simple out-in operation accepting single argument. + /// Does the out op. /// - /// Operation type. - /// Value. - /// Result. - private CacheResult DoOutInOpNullable(int type, T1 val) + private bool DoOutOp(CacheOp op, T1 x) { - var res = DoOutInOp(type, val); + return DoOutInOpX((int) op, w => + { + w.Write(x); + }, ReadException); + } - return res == null - ? new CacheResult() - : new CacheResult((TR)res); + /// + /// Does the out op. + /// + private bool DoOutOp(CacheOp op, T1 x, T2 y) + { + return DoOutInOpX((int) op, w => + { + w.Write(x); + w.Write(y); + }, ReadException); } /// - /// Perform out-in operation. + /// Does the out op. /// - /// Operation type. - /// Out action. - /// Result. - private CacheResult DoOutInOpNullable(int type, Action outAction) + private bool DoOutOp(CacheOp op, T1 x, T2 y, T3 z) { - var res = DoOutInOp(type, outAction); + return DoOutInOpX((int) op, w => + { + w.Write(x); + w.Write(y); + w.Write(z); + }, ReadException); + } - return res == null - ? new CacheResult() - : new CacheResult((TR)res); + /// + /// Does the out op. + /// + private bool DoOutOp(CacheOp op, Action write) + { + return DoOutInOpX((int) op, write, ReadException); } /// - /// Perform simple out-in operation accepting single argument. + /// Does the out-in op. /// - /// Operation type. - /// Value. - /// Value. - /// Result. - private CacheResult DoOutInOpNullable(int type, T1 val1, T2 val2) + private CacheResult DoOutInOpNullable(CacheOp cacheOp, TK x) { - var res = DoOutInOp(type, val1, val2); + return DoOutInOpX((int)cacheOp, + w => w.Write(x), + (stream, res) => res == True ? new CacheResult(Unmarshal(stream)) : new CacheResult(), + ReadException); + } - return res == null - ? new CacheResult() - : new CacheResult((TR)res); + /// + /// Does the out-in op. + /// + private CacheResult DoOutInOpNullable(CacheOp cacheOp, T1 x, T2 y) + { + return DoOutInOpX((int)cacheOp, + w => + { + w.Write(x); + w.Write(y); + }, + (stream, res) => res == True ? new CacheResult(Unmarshal(stream)) : new CacheResult(), + ReadException); } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs index 22881c661a93d..461872f8f7929 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs @@ -115,10 +115,14 @@ static ExceptionUtils() /// Exception message. /// Native stack trace. /// Error data reader. + /// Inner exception. /// Exception. - public static Exception GetException(IIgnite ignite, string clsName, string msg, string stackTrace, BinaryReader reader = null) + public static Exception GetException(IIgnite ignite, string clsName, string msg, string stackTrace, + BinaryReader reader = null, Exception innerException = null) { - Exception innerException = string.IsNullOrEmpty(stackTrace) ? null : new JavaException(stackTrace); + // Set JavaException as inner only if there is no InnerException. + if (innerException == null && !string.IsNullOrEmpty(stackTrace)) + innerException = new JavaException(stackTrace); ExceptionFactoryDelegate ctor; @@ -158,7 +162,7 @@ public static Exception GetException(IIgnite ignite, string clsName, string msg, /// Reader. /// CachePartialUpdateException. [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes")] - private static Exception ProcessCachePartialUpdateException(IIgnite ignite, string msg, string stackTrace, + private static Exception ProcessCachePartialUpdateException(IIgnite ignite, string msg, string stackTrace, BinaryReader reader) { if (reader == null) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs index 26b6033b5b7e9..5f24e43e60c25 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs @@ -42,6 +42,9 @@ internal abstract class PlatformTarget /** */ protected const int True = 1; + /** */ + protected const int Error = -1; + /** */ private const int OpMeta = -1; @@ -470,7 +473,82 @@ protected TR DoOutInOp(int type, Action outAction, Func + /// Perform out-in operation with a single stream. + /// + /// The type of the r. + /// Operation type. + /// Out action. + /// In action. + /// The action to read an error. + /// + /// Result. + /// + protected TR DoOutInOpX(int type, Action outAction, Func inAction, + Func inErrorAction) + { + Debug.Assert(inErrorAction != null); + + using (var stream = IgniteManager.Memory.Allocate().GetStream()) + { + var writer = _marsh.StartMarshal(stream); + + outAction(writer); + + FinishMarshal(writer); + + var res = UU.TargetInStreamOutLong(_target, type, stream.SynchronizeOutput()); + + if (res != Error && inAction == null) + return default(TR); // quick path for void operations + + stream.SynchronizeInput(); + + stream.Seek(0, SeekOrigin.Begin); + + if (res != Error) + return inAction != null ? inAction(stream, res) : default(TR); + + throw inErrorAction(stream); + } + } + + /// + /// Perform out-in operation with a single stream. + /// + /// Operation type. + /// Out action. + /// The action to read an error. + /// + /// Result. + /// + protected bool DoOutInOpX(int type, Action outAction, + Func inErrorAction) + { + Debug.Assert(inErrorAction != null); + + using (var stream = IgniteManager.Memory.Allocate().GetStream()) + { + var writer = _marsh.StartMarshal(stream); + + outAction(writer); + + FinishMarshal(writer); + + var res = UU.TargetInStreamOutLong(_target, type, stream.SynchronizeOutput()); + + if (res != Error) + return res == True; + + stream.SynchronizeInput(); + + stream.Seek(0, SeekOrigin.Begin); + + throw inErrorAction(stream); + } + } + /// /// Perform out-in operation. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs index b1c840f26a49e..fd52c8a5d905e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs @@ -21,6 +21,8 @@ namespace Apache.Ignite.Core.Impl.Unmanaged using System.Collections.Generic; using System.Diagnostics; using System.Diagnostics.CodeAnalysis; + using System.Globalization; + using System.IO; using System.Runtime.InteropServices; using System.Threading; using Apache.Ignite.Core.Cache.Affinity; @@ -312,6 +314,7 @@ private long CacheStoreCreate(void* target, long memPtr) }, true); } + [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes")] [SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope")] private int CacheStoreInvoke(void* target, long objPtr, long memPtr, void* cb) { @@ -326,7 +329,18 @@ private int CacheStoreInvoke(void* target, long objPtr, long memPtr, void* cb) using (PlatformMemoryStream stream = IgniteManager.Memory.Get(memPtr).GetStream()) { - return t.Invoke(stream, cb0, _ignite); + try + { + return t.Invoke(stream, cb0, _ignite); + } + catch (Exception e) + { + stream.Seek(0, SeekOrigin.Begin); + + _ignite.Marshaller.StartMarshal(stream).WriteObject(e); + + return -1; + } } }); } @@ -756,8 +770,9 @@ private void FutureError(void* target, long futPtr, long memPtr) string errCls = reader.ReadString(); string errMsg = reader.ReadString(); string stackTrace = reader.ReadString(); + Exception inner = reader.ReadBoolean() ? reader.ReadObject() : null; - Exception err = ExceptionUtils.GetException(_ignite, errCls, errMsg, stackTrace, reader); + Exception err = ExceptionUtils.GetException(_ignite, errCls, errMsg, stackTrace, reader, inner); ProcessFuture(futPtr, fut => { fut.OnError(err); }); } @@ -1084,7 +1099,7 @@ private void OnStop(void* target) // Stream disposal intentionally omitted: IGNITE-1598 var stream = new PlatformRawMemory(errData, errDataLen).GetStream(); - throw ExceptionUtils.GetException(_ignite, errCls, errMsg, stackTrace, + throw ExceptionUtils.GetException(_ignite, errCls, errMsg, stackTrace, _ignite.Marshaller.StartUnmarshal(stream)); } From ebf354c568d0802b7eed1cc6b9d251941dbce014 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Fri, 16 Sep 2016 14:32:13 +0300 Subject: [PATCH 139/487] IGNITE-3907 Fixed "Incorrect initialization CQ when node filter configured for cache" --- .../internal/GridEventConsumeHandler.java | 5 - .../internal/GridMessageListenHandler.java | 5 - .../CacheContinuousQueryHandler.java | 5 - .../continuous/GridContinuousHandler.java | 8 - .../continuous/GridContinuousProcessor.java | 33 +--- ...ontinuousQueryMultiNodesFilteringTest.java | 161 ++++++++++++++++++ 6 files changed, 170 insertions(+), 47 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java index b4b1e58b93727..ed6998d5b8ab1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java @@ -261,11 +261,6 @@ public GridEventConsumeHandler() { return RegisterStatus.REGISTERED; } - /** {@inheritDoc} */ - @Override public void onListenerRegistered(UUID routineId, GridKernalContext ctx) { - // No-op. - } - /** {@inheritDoc} */ @Override public void unregister(UUID routineId, GridKernalContext ctx) { assert routineId != null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java index 2b8041d5a7ab1..1bca85cacfd7d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java @@ -138,11 +138,6 @@ public GridMessageListenHandler(GridMessageListenHandler orig) { return RegisterStatus.REGISTERED; } - /** {@inheritDoc} */ - @Override public void onListenerRegistered(UUID routineId, GridKernalContext ctx) { - // No-op. - } - /** {@inheritDoc} */ @Override public void unregister(UUID routineId, GridKernalContext ctx) { ctx.io().removeUserMessageListener(topic, pred); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 7b3b47b5a9dfb..a5752ed47eb92 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -563,11 +563,6 @@ public void waitTopologyFuture(GridKernalContext ctx) throws IgniteCheckedExcept } } - /** {@inheritDoc} */ - @Override public void onListenerRegistered(UUID routineId, GridKernalContext ctx) { - // No-op. - } - /** {@inheritDoc} */ @Override public void unregister(UUID routineId, GridKernalContext ctx) { assert routineId != null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java index c90746d759e6a..f14b450303f2a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java @@ -56,14 +56,6 @@ public enum RegisterStatus { */ public RegisterStatus register(UUID nodeId, UUID routineId, GridKernalContext ctx) throws IgniteCheckedException; - /** - * Callback called after listener is registered and acknowledgement is sent. - * - * @param routineId Routine ID. - * @param ctx Kernal context. - */ - public void onListenerRegistered(UUID routineId, GridKernalContext ctx); - /** * Unregisters listener. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java index 5f61051b984be..ad7ad4fd70db8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java @@ -478,11 +478,9 @@ public void unlockStopping() { // Register handler only if local node passes projection predicate. if ((item.prjPred == null || item.prjPred.apply(ctx.discovery().localNode())) && - !locInfos.containsKey(item.routineId)) { - if (registerHandler(data.nodeId, item.routineId, item.hnd, item.bufSize, item.interval, - item.autoUnsubscribe, false)) - item.hnd.onListenerRegistered(item.routineId, ctx); - } + !locInfos.containsKey(item.routineId)) + registerHandler(data.nodeId, item.routineId, item.hnd, item.bufSize, item.interval, + item.autoUnsubscribe, false); if (!item.autoUnsubscribe) // Register routine locally. @@ -509,14 +507,13 @@ public void unlockStopping() { ctx.resource().injectGeneric(info.prjPred); if (info.prjPred == null || info.prjPred.apply(ctx.discovery().localNode())) { - if (registerHandler(clientNodeId, + registerHandler(clientNodeId, routineId, info.hnd, info.bufSize, info.interval, info.autoUnsubscribe, - false)) - info.hnd.onListenerRegistered(routineId, ctx); + false); } } catch (IgniteCheckedException err) { @@ -555,9 +552,6 @@ public void onCacheStart(GridCacheContext ctx) throws IgniteCheckedException { GridContinuousHandler.RegisterStatus status = hnd.register(rmtInfo.nodeId, routineId, this.ctx); assert status != GridContinuousHandler.RegisterStatus.DELAYED; - - if (status == GridContinuousHandler.RegisterStatus.REGISTERED) - hnd.onListenerRegistered(routineId, this.ctx); } } } @@ -649,8 +643,6 @@ public IgniteInternalFuture startRoutine(GridContinuousHandler hnd, try { registerHandler(ctx.localNodeId(), routineId, hnd, bufSize, interval, autoUnsubscribe, true); - hnd.onListenerRegistered(routineId, ctx); - return new GridFinishedFuture<>(routineId); } catch (IgniteCheckedException e) { @@ -700,9 +692,8 @@ public IgniteInternalFuture startRoutine(GridContinuousHandler hnd, startFuts.put(routineId, fut); try { - if (locIncluded - && registerHandler(ctx.localNodeId(), routineId, hnd, bufSize, interval, autoUnsubscribe, true)) - hnd.onListenerRegistered(routineId, ctx); + if (locIncluded || hnd.isQuery()) + registerHandler(ctx.localNodeId(), routineId, hnd, bufSize, interval, autoUnsubscribe, true); ctx.discovery().sendCustomEvent(new StartRoutineDiscoveryMessage(routineId, reqData, reqData.handler().keepBinary())); @@ -1020,8 +1011,6 @@ private void processStartRequest(ClusterNode node, StartRoutineDiscoveryMessage data.autoUnsubscribe())); } - boolean registered = false; - if (err == null) { try { IgnitePredicate prjPred = data.projectionPredicate(); @@ -1030,10 +1019,9 @@ private void processStartRequest(ClusterNode node, StartRoutineDiscoveryMessage ctx.resource().injectGeneric(prjPred); if ((prjPred == null || prjPred.apply(ctx.discovery().node(ctx.localNodeId()))) && - !locInfos.containsKey(routineId)) { - registered = registerHandler(node.id(), routineId, hnd0, data.bufferSize(), data.interval(), + !locInfos.containsKey(routineId)) + registerHandler(node.id(), routineId, hnd0, data.bufferSize(), data.interval(), data.autoUnsubscribe(), false); - } if (!data.autoUnsubscribe()) // Register routine locally. @@ -1061,9 +1049,6 @@ private void processStartRequest(ClusterNode node, StartRoutineDiscoveryMessage if (err != null) req.addError(ctx.localNodeId(), err); - - if (registered) - hnd0.onListenerRegistered(routineId, ctx); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryMultiNodesFilteringTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryMultiNodesFilteringTest.java index 700044659880a..cf0c0d9575441 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryMultiNodesFilteringTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryMultiNodesFilteringTest.java @@ -17,9 +17,17 @@ package org.apache.ignite.internal.processors.cache.query.continuous; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Pattern; import javax.cache.configuration.Factory; import javax.cache.configuration.MutableCacheEntryListenerConfiguration; import javax.cache.event.CacheEntryCreatedListener; @@ -33,9 +41,12 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; import org.apache.ignite.cache.query.ContinuousQuery; +import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.typedef.PA; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; @@ -45,8 +56,10 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jsr166.ConcurrentHashMap8; +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; import static org.apache.ignite.cache.CacheMode.PARTITIONED; import static org.apache.ignite.cache.CacheMode.REPLICATED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; /** */ @SuppressWarnings("unchecked") @@ -57,13 +70,21 @@ public class GridCacheContinuousQueryMultiNodesFilteringTest extends GridCommonA /** */ private static final int SERVER_GRIDS_COUNT = 6; + /** */ + public static final int KEYS = 2_000; + /** Cache entry operations' counts. */ private static final ConcurrentMap opCounts = new ConcurrentHashMap8<>(); + /** Client. */ + private static boolean client = false; + /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); + client = false; + super.afterTest(); } @@ -122,6 +143,108 @@ public void testFiltersAndListeners() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testWithNodeFilter() throws Exception { + List qryCursors = new ArrayList<>(); + + final int nodesCnt = 3; + + startGridsMultiThreaded(nodesCnt); + + awaitPartitionMapExchange(); + + CacheConfiguration ccfg = cacheConfiguration(new NodeFilterByRegexp(".*(0|1)$")); + + grid(0).createCache(ccfg); + + final AtomicInteger cntr = new AtomicInteger(); + + final ConcurrentMap> maps = new ConcurrentHashMap<>(); + + final AtomicBoolean doubleNtfFail = new AtomicBoolean(false); + + CacheEntryUpdatedListener lsnr = new CacheEntryUpdatedListener() { + @Override public void onUpdated(Iterable> evts) + throws CacheEntryListenerException { + for (CacheEntryEvent e : evts) { + cntr.incrementAndGet(); + + ClusterNode node = ((Ignite)e.getSource().unwrap(Ignite.class)).cluster().localNode(); + + Set set = maps.get(node); + + if (set == null) { + set = new ConcurrentSkipListSet<>(); + + Set oldVal = maps.putIfAbsent(node, set); + + set = oldVal != null ? oldVal : set; + } + + if (!set.add(e.getValue())) + doubleNtfFail.set(false); + } + } + }; + + for (int i = 0; i < nodesCnt; i++) { + ContinuousQuery qry = new ContinuousQuery<>(); + + qry.setLocalListener(lsnr); + + Ignite ignite = grid(i); + + log.info("Try to start CQ on node: " + ignite.cluster().localNode().id()); + + qryCursors.add(ignite.cache(ccfg.getName()).query(qry)); + + log.info("CQ started on node: " + ignite.cluster().localNode().id()); + } + + client = true; + + startGrid(nodesCnt); + + awaitPartitionMapExchange(); + + ContinuousQuery qry = new ContinuousQuery<>(); + + qry.setLocalListener(lsnr); + + qryCursors.add(grid(nodesCnt).cache(ccfg.getName()).query(qry)); + + for (int i = 0; i <= nodesCnt; i++) { + for (int key = 0; key < KEYS; key++) { + int val = (i * KEYS) + key; + + grid(i).cache(ccfg.getName()).put(val, val); + } + } + + assertTrue(GridTestUtils.waitForCondition(new PA() { + @Override public boolean apply() { + return cntr.get() >= 2 * (nodesCnt + 1) * KEYS; + } + }, 5000L)); + + assertFalse("Got duplicate", doubleNtfFail.get()); + + for (int i = 0; i < (nodesCnt + 1) * KEYS; i++) { + for (Map.Entry> e : maps.entrySet()) + assertTrue("Lost event on node: " + e.getKey().id() + ", event: " + i, e.getValue().remove(i)); + } + + for (Map.Entry> e : maps.entrySet()) + assertTrue("Unexpected event on node: " + e.getKey(), e.getValue().isEmpty()); + + assertEquals("Not expected count of CQ", nodesCnt + 1, qryCursors.size()); + + for (QueryCursor cur : qryCursors) + cur.close(); + } + /** */ private Ignite startGrid(final int idx, boolean isClientMode) throws Exception { String gridName = getTestGridName(idx); @@ -179,6 +302,28 @@ private Ignite startGrid(final int idx, boolean isClientMode) throws Exception { return node; } + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setClientMode(client); + + return cfg; + } + + /** + * @param filter Node filter. + * @return Cache configuration. + */ + private CacheConfiguration cacheConfiguration(NodeFilterByRegexp filter) { + return new CacheConfiguration("test-cache-cq") + .setBackups(1) + .setNodeFilter(filter) + .setAtomicityMode(ATOMIC) + .setWriteSynchronizationMode(FULL_SYNC) + .setCacheMode(PARTITIONED); + } + /** */ private final static class ListenerConfiguration extends MutableCacheEntryListenerConfiguration { /** Operation. */ @@ -275,4 +420,20 @@ private NodeFilter(int idx) { return ((Integer)clusterNode.attributes().get("idx") % 2) == idx % 2; } } + + /** */ + private final static class NodeFilterByRegexp implements IgnitePredicate { + /** */ + private final Pattern pattern; + + /** */ + private NodeFilterByRegexp(String regExp) { + this.pattern = Pattern.compile(regExp); + } + + /** {@inheritDoc} */ + @Override public boolean apply(ClusterNode clusterNode) { + return pattern.matcher(clusterNode.id().toString()).matches(); + } + } } From 799f1909cea96037a38eee1b4ecb21fab737d092 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Fri, 16 Sep 2016 17:50:24 +0300 Subject: [PATCH 140/487] IGNITE-3199 .NET: Add ASP.NET Session-State Store Provider --- .../ignite/internal/binary/BinaryContext.java | 5 + .../platform/PlatformConfigurationEx.java | 7 + .../platform/PlatformContextImpl.java | 3 - .../platform/PlatformProcessor.java | 10 +- .../platform/PlatformProcessorImpl.java | 72 ++- .../platform/cache/PlatformCache.java | 114 +++- .../cache/PlatformCacheExtension.java | 47 ++ .../cpp/PlatformCppConfigurationEx.java | 7 + .../dotnet/PlatformDotNetConfigurationEx.java | 9 + .../PlatformDotNetSessionCacheExtension.java | 144 +++++ .../websession/PlatformDotNetSessionData.java | 260 +++++++++ .../PlatformDotNetSessionLockProcessor.java | 84 +++ .../PlatformDotNetSessionLockResult.java | 106 ++++ ...ormDotNetSessionSetAndUnlockProcessor.java | 179 ++++++ .../Apache.Ignite.AspNet.Tests.csproj | 77 +++ .../Apache.Ignite.AspNet.Tests.snk | Bin 0 -> 596 bytes .../Apache.Ignite.AspNet.Tests/App.config | 72 +++ .../ExpiryCacheHolderTest.cs | 492 ++++++++++++++++ .../IgniteOutputCacheProviderTest.cs | 14 +- .../IgniteSessionStateItemCollectionTest.cs | 267 +++++++++ .../IgniteSessionStateStoreDataTest.cs | 117 ++++ .../IgniteSessionStateStoreProviderTest.cs | 425 ++++++++++++++ .../Properties/AssemblyInfo.cs | 42 ++ .../packages.config | 22 + .../Apache.Ignite.AspNet.csproj | 6 + .../Apache.Ignite.AspNet.ruleset | 3 + .../IgniteOutputCacheProvider.cs | 120 +--- .../IgniteSessionStateStoreProvider.cs | 494 ++++++++++++++++ .../Apache.Ignite.AspNet/Impl/ConfigUtil.cs | 109 ++++ .../Impl/ExpiryCacheHolder.cs | 113 ++++ .../Impl/IgniteSessionStateItemCollection.cs | 534 ++++++++++++++++++ .../Impl/IgniteSessionStateStoreData.cs | 116 ++++ .../Impl/SessionStateLockResult.cs | 91 +++ .../Properties/AssemblyInfo.cs | 5 +- .../Apache.Ignite.Core.Tests.csproj | 6 - .../Apache.Ignite.Core.Tests/TestUtils.cs | 1 - .../Impl/Cache/CacheImpl.cs | 15 + .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs | 3 +- .../Impl/Cache/ICacheInternal.cs | 14 + .../Impl/Unmanaged/UnmanagedCallbacks.cs | 1 - .../Properties/AssemblyInfo.cs | 1 + modules/platforms/dotnet/Apache.Ignite.sln | 14 + 42 files changed, 4058 insertions(+), 163 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtension.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionCacheExtension.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionData.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionLockProcessor.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionLockResult.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionSetAndUnlockProcessor.java create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.csproj create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.snk create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs rename modules/platforms/dotnet/{Apache.Ignite.Core.Tests/AspNet => Apache.Ignite.AspNet.Tests}/IgniteOutputCacheProviderTest.cs (95%) create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateItemCollectionTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreDataTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/packages.config create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ExpiryCacheHolder.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/IgniteSessionStateItemCollection.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/IgniteSessionStateStoreData.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/SessionStateLockResult.cs diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java index 0d66970fe9fb3..c468a4df1eab9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java @@ -76,6 +76,8 @@ import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdatePropertiesProcessor; import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdateTimesProcessor; import org.apache.ignite.internal.processors.platform.PlatformJavaObjectFactoryProxy; +import org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionData; +import org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionLockResult; import org.apache.ignite.internal.util.lang.GridMapEntry; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; @@ -321,6 +323,9 @@ public BinaryContext(BinaryMetadataHandler metaHnd, IgniteConfiguration igniteCf registerPredefinedType(BinaryMetadata.class, 0); registerPredefinedType(BinaryEnumObjectImpl.class, 0); + registerPredefinedType(PlatformDotNetSessionData.class, 0); + registerPredefinedType(PlatformDotNetSessionLockResult.class, 0); + // IDs range [200..1000] is used by Ignite internal APIs. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformConfigurationEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformConfigurationEx.java index 66eff8b98e39d..b7c8895860a00 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformConfigurationEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformConfigurationEx.java @@ -17,8 +17,10 @@ package org.apache.ignite.internal.processors.platform; +import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension; import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway; import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManagerImpl; +import org.jetbrains.annotations.Nullable; import java.util.Collection; @@ -45,4 +47,9 @@ public interface PlatformConfigurationEx { * @return Warnings to be displayed on grid start. */ public Collection warnings(); + + /** + * @return Available cache extensions. + */ + @Nullable public Collection cacheExtensions(); } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java index d544fff523ded..e7fdb0a2dcc15 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java @@ -118,9 +118,6 @@ public class PlatformContextImpl implements PlatformContext { /** Platform name. */ private final String platform; - /** - * Static initializer. - */ static { Set evtTyps0 = new HashSet<>(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java index fc42b689ba19f..f0201eff67d80 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java @@ -19,7 +19,6 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteException; import org.apache.ignite.internal.processors.GridProcessor; import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheStore; import org.jetbrains.annotations.Nullable; @@ -203,9 +202,8 @@ public interface PlatformProcessor extends GridProcessor { * @param initVal Initial value. * @param create Create flag. * @return Platform atomic long. - * @throws IgniteException */ - public PlatformTarget atomicLong(String name, long initVal, boolean create) throws IgniteException; + public PlatformTarget atomicLong(String name, long initVal, boolean create); /** * Get or create AtomicSequence. @@ -213,9 +211,8 @@ public interface PlatformProcessor extends GridProcessor { * @param initVal Initial value. * @param create Create flag. * @return Platform atomic long. - * @throws IgniteException */ - public PlatformTarget atomicSequence(String name, long initVal, boolean create) throws IgniteException; + public PlatformTarget atomicSequence(String name, long initVal, boolean create); /** * Get or create AtomicReference. @@ -223,9 +220,8 @@ public interface PlatformProcessor extends GridProcessor { * @param memPtr Pointer to a stream with initial value. 0 for null initial value. * @param create Create flag. * @return Platform atomic long. - * @throws IgniteException */ - public PlatformTarget atomicReference(String name, long memPtr, boolean create) throws IgniteException; + public PlatformTarget atomicReference(String name, long memPtr, boolean create); /** * Gets the configuration of the current Ignite instance. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java index 8c9e2057d2468..b364c4a11ca99 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java @@ -35,6 +35,7 @@ import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl; import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicLongImpl; import org.apache.ignite.internal.processors.platform.cache.PlatformCache; +import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension; import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinity; import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheStore; import org.apache.ignite.internal.processors.platform.cluster.PlatformClusterGroup; @@ -59,6 +60,8 @@ import java.util.Collection; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.locks.ReadWriteLock; @@ -79,6 +82,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf private final ReadWriteLock storeLock = new ReentrantReadWriteLock(); /** Logger. */ + @SuppressWarnings("FieldCanBeLocal") private final IgniteLogger log; /** Context. */ @@ -93,6 +97,9 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf /** Whether processor if stopped (or stopping). */ private volatile boolean stopped; + /** Cache extensions. */ + private final PlatformCacheExtension[] cacheExts; + /** * Constructor. * @@ -118,6 +125,9 @@ public PlatformProcessorImpl(GridKernalContext ctx) { } platformCtx = new PlatformContextImpl(ctx, interopCfg.gate(), interopCfg.memory(), interopCfg.platform()); + + // Initialize cache extensions (if any). + cacheExts = prepareCacheExtensions(interopCfg.cacheExtensions()); } /** {@inheritDoc} */ @@ -207,7 +217,7 @@ public PlatformProcessorImpl(GridKernalContext ctx) { if (cache == null) throw new IllegalArgumentException("Cache doesn't exist: " + name); - return new PlatformCache(platformCtx, cache.keepBinary(), false); + return createPlatformCache(cache); } /** {@inheritDoc} */ @@ -216,7 +226,7 @@ public PlatformProcessorImpl(GridKernalContext ctx) { assert cache != null; - return new PlatformCache(platformCtx, cache.keepBinary(), false); + return createPlatformCache(cache); } /** {@inheritDoc} */ @@ -225,7 +235,7 @@ public PlatformProcessorImpl(GridKernalContext ctx) { assert cache != null; - return new PlatformCache(platformCtx, cache.keepBinary(), false); + return createPlatformCache(cache); } /** {@inheritDoc} */ @@ -237,7 +247,7 @@ public PlatformProcessorImpl(GridKernalContext ctx) { ? (IgniteCacheProxy)ctx.grid().createCache(cfg, PlatformConfigurationUtils.readNearConfiguration(reader)) : (IgniteCacheProxy)ctx.grid().createCache(cfg); - return new PlatformCache(platformCtx, cache.keepBinary(), false); + return createPlatformCache(cache); } /** {@inheritDoc} */ @@ -250,7 +260,7 @@ public PlatformProcessorImpl(GridKernalContext ctx) { PlatformConfigurationUtils.readNearConfiguration(reader)) : (IgniteCacheProxy)ctx.grid().getOrCreateCache(cfg); - return new PlatformCache(platformCtx, cache.keepBinary(), false); + return createPlatformCache(cache); } /** {@inheritDoc} */ @@ -404,7 +414,7 @@ public PlatformProcessorImpl(GridKernalContext ctx) { IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().createNearCache(cacheName, cfg); - return new PlatformCache(platformCtx, cache.keepBinary(), false); + return createPlatformCache(cache); } /** {@inheritDoc} */ @@ -413,7 +423,14 @@ public PlatformProcessorImpl(GridKernalContext ctx) { IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().getOrCreateNearCache(cacheName, cfg); - return new PlatformCache(platformCtx, cache.keepBinary(), false); + return createPlatformCache(cache); + } + + /** + * Creates new platform cache. + */ + private PlatformTarget createPlatformCache(IgniteCacheProxy cache) { + return new PlatformCache(platformCtx, cache, false, cacheExts); } /** @@ -446,6 +463,47 @@ private void registerStore0(PlatformCacheStore store, boolean convertBinary) thr throw new IgniteCheckedException("Unsupported interop store: " + store); } + /** + * Prepare cache extensions. + * + * @param cacheExts Original extensions. + * @return Prepared extensions. + */ + private static PlatformCacheExtension[] prepareCacheExtensions(Collection cacheExts) { + if (!F.isEmpty(cacheExts)) { + int maxExtId = 0; + + Map idToExt = new HashMap<>(); + + for (PlatformCacheExtension cacheExt : cacheExts) { + if (cacheExt == null) + throw new IgniteException("Platform cache extension cannot be null."); + + if (cacheExt.id() < 0) + throw new IgniteException("Platform cache extension ID cannot be negative: " + cacheExt); + + PlatformCacheExtension oldCacheExt = idToExt.put(cacheExt.id(), cacheExt); + + if (oldCacheExt != null) + throw new IgniteException("Platform cache extensions cannot have the same ID [" + + "id=" + cacheExt.id() + ", first=" + oldCacheExt + ", second=" + cacheExt + ']'); + + if (cacheExt.id() > maxExtId) + maxExtId = cacheExt.id(); + } + + PlatformCacheExtension[] res = new PlatformCacheExtension[maxExtId + 1]; + + for (PlatformCacheExtension cacheExt : cacheExts) + res[cacheExt.id()]= cacheExt; + + return res; + } + else + //noinspection ZeroLengthArrayAllocation + return new PlatformCacheExtension[0]; + } + /** * Store and manager pair. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index a7b6e418a9a4d..a1f8da9209129 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -19,6 +19,8 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.binary.BinaryRawReader; import org.apache.ignite.cache.CacheEntryProcessor; import org.apache.ignite.cache.CacheMetrics; import org.apache.ignite.cache.CachePartialUpdateException; @@ -28,7 +30,7 @@ import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.SqlQuery; import org.apache.ignite.cache.query.TextQuery; -import org.apache.ignite.configuration.*; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.binary.BinaryRawReaderEx; import org.apache.ignite.internal.binary.BinaryRawWriterEx; @@ -52,12 +54,13 @@ import org.apache.ignite.internal.util.GridConcurrentFactory; import org.apache.ignite.internal.util.future.IgniteFutureImpl; import org.apache.ignite.internal.util.typedef.C1; -import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lang.IgniteFuture; import org.jetbrains.annotations.Nullable; import javax.cache.Cache; +import javax.cache.CacheException; import javax.cache.expiry.Duration; import javax.cache.expiry.ExpiryPolicy; import javax.cache.integration.CompletionListener; @@ -193,20 +196,26 @@ public class PlatformCache extends PlatformAbstractTarget { /** */ public static final int OP_LOAD_ALL = 40; - /** Underlying JCache. */ + /** */ + public static final int OP_EXTENSION = 41; + + /** Underlying JCache in binary mode. */ private final IgniteCacheProxy cache; + /** Initial JCache (not in binary mode). */ + private final IgniteCache rawCache; + /** Whether this cache is created with "keepBinary" flag on the other side. */ private final boolean keepBinary; /** */ - private static final GetAllWriter WRITER_GET_ALL = new GetAllWriter(); + private static final PlatformFutureUtils.Writer WRITER_GET_ALL = new GetAllWriter(); /** */ - private static final EntryProcessorInvokeWriter WRITER_INVOKE = new EntryProcessorInvokeWriter(); + private static final PlatformFutureUtils.Writer WRITER_INVOKE = new EntryProcessorInvokeWriter(); /** */ - private static final EntryProcessorInvokeAllWriter WRITER_INVOKE_ALL = new EntryProcessorInvokeAllWriter(); + private static final PlatformFutureUtils.Writer WRITER_INVOKE_ALL = new EntryProcessorInvokeAllWriter(); /** Map with currently active locks. */ private final ConcurrentMap lockMap = GridConcurrentFactory.newMap(); @@ -214,6 +223,9 @@ public class PlatformCache extends PlatformAbstractTarget { /** Lock ID sequence. */ private static final AtomicLong LOCK_ID_GEN = new AtomicLong(); + /** Extensions. */ + private final PlatformCacheExtension[] exts; + /** * Constructor. * @@ -222,10 +234,29 @@ public class PlatformCache extends PlatformAbstractTarget { * @param keepBinary Keep binary flag. */ public PlatformCache(PlatformContext platformCtx, IgniteCache cache, boolean keepBinary) { + this(platformCtx, cache, keepBinary, new PlatformCacheExtension[0]); + } + + /** + * Constructor. + * + * @param platformCtx Context. + * @param cache Underlying cache. + * @param keepBinary Keep binary flag. + * @param exts Extensions. + */ + public PlatformCache(PlatformContext platformCtx, IgniteCache cache, boolean keepBinary, + PlatformCacheExtension[] exts) { super(platformCtx); - this.cache = (IgniteCacheProxy)cache; + assert cache != null; + assert exts != null; + + rawCache = cache; + + this.cache = (IgniteCacheProxy)cache.withKeepBinary(); this.keepBinary = keepBinary; + this.exts = exts; } /** @@ -237,7 +268,7 @@ public PlatformCache withSkipStore() { if (cache.delegate().skipStore()) return this; - return new PlatformCache(platformCtx, cache.withSkipStore(), keepBinary); + return copy(rawCache.withSkipStore(), keepBinary); } /** @@ -249,7 +280,7 @@ public PlatformCache withKeepBinary() { if (keepBinary) return this; - return new PlatformCache(platformCtx, cache.withKeepBinary(), true); + return copy(rawCache.withKeepBinary(), true); } /** @@ -261,9 +292,9 @@ public PlatformCache withKeepBinary() { * @return Cache. */ public PlatformCache withExpiryPolicy(final long create, final long update, final long access) { - IgniteCache cache0 = cache.withExpiryPolicy(new InteropExpiryPolicy(create, update, access)); + IgniteCache cache0 = rawCache.withExpiryPolicy(new InteropExpiryPolicy(create, update, access)); - return new PlatformCache(platformCtx, cache0, keepBinary); + return copy(cache0, keepBinary); } /** @@ -275,7 +306,7 @@ public PlatformCache withAsync() { if (cache.isAsync()) return this; - return new PlatformCache(platformCtx, (IgniteCache)cache.withAsync(), keepBinary); + return copy(rawCache.withAsync(), keepBinary); } /** @@ -289,11 +320,19 @@ public PlatformCache withNoRetries() { if (opCtx != null && opCtx.noRetries()) return this; - return new PlatformCache(platformCtx, cache.withNoRetries(), keepBinary); + return copy(rawCache.withNoRetries(), keepBinary); + } + + /** + * @return Raw cache. + */ + public IgniteCache rawCache() { + return rawCache; } /** {@inheritDoc} */ - @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem) throws IgniteCheckedException { + @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem) + throws IgniteCheckedException { try { switch (type) { case OP_PUT: @@ -452,6 +491,11 @@ public PlatformCache withNoRetries() { case OP_LOCK_ALL: return registerLock(cache.lockAll(PlatformUtils.readCollection(reader))); + + case OP_EXTENSION: + PlatformCacheExtension ext = extension(reader.readInt()); + + return ext.processInOutStreamLong(this, reader.readInt(), reader, mem); } } catch (Exception e) { @@ -474,14 +518,14 @@ public PlatformCache withNoRetries() { /** * Writes the result to reused stream, if any. */ - private long writeResult(PlatformMemory mem, Object obj) { + public long writeResult(PlatformMemory mem, Object obj) { return writeResult(mem, obj, null); } /** * Writes the result to reused stream, if any. */ - private long writeResult(PlatformMemory mem, Object obj, PlatformWriterClosure clo) { + public long writeResult(PlatformMemory mem, Object obj, PlatformWriterClosure clo) { if (obj == null) return FALSE; @@ -665,7 +709,7 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc) { return new PlatformCachePartialUpdateException((CachePartialUpdateCheckedException)e, platformCtx, keepBinary); if (e.getCause() instanceof EntryProcessorException) - return (EntryProcessorException) e.getCause(); + return (Exception)e.getCause(); return super.convertException(e); } @@ -743,7 +787,7 @@ private static void writeError(BinaryRawWriterEx writer, Exception ex) { * Clears the contents of the cache, without notifying listeners or CacheWriters. * * @throws IllegalStateException if the cache is closed. - * @throws javax.cache.CacheException if there is a problem during the clear + * @throws CacheException if there is a problem during the clear */ public void clear() throws IgniteCheckedException { cache.clear(); @@ -752,7 +796,7 @@ public void clear() throws IgniteCheckedException { /** * Removes all entries. * - * @throws org.apache.ignite.IgniteCheckedException In case of error. + * @throws IgniteCheckedException In case of error. */ public void removeAll() throws IgniteCheckedException { cache.removeAll(); @@ -969,7 +1013,7 @@ private Query readFieldsQuery(BinaryRawReaderEx reader) { /** * Reads text query. */ - private Query readTextQuery(BinaryRawReaderEx reader) { + private Query readTextQuery(BinaryRawReader reader) { boolean loc = reader.readBoolean(); String txt = reader.readString(); String typ = reader.readString(); @@ -1003,6 +1047,34 @@ private Query readScanQuery(BinaryRawReaderEx reader) { return qry; } + /** + * Clones this instance. + * + * @param cache Cache. + * @param keepBinary Keep binary flag. + * @return Cloned instance. + */ + private PlatformCache copy(IgniteCache cache, boolean keepBinary) { + return new PlatformCache(platformCtx, cache, keepBinary, exts); + } + + /** + * Get extension by ID. + * + * @param id ID. + * @return Extension. + */ + private PlatformCacheExtension extension(int id) { + if (exts != null && id < exts.length) { + PlatformCacheExtension ext = exts[id]; + + if (ext != null) + return ext; + } + + throw new IgniteException("Platform cache extension is not registered [id=" + id + ']'); + } + /** * Writes error with EntryProcessorException cause. */ @@ -1088,7 +1160,7 @@ private static class InteropExpiryPolicy implements ExpiryPolicy { * @param update Expiry for update. * @param access Expiry for access. */ - public InteropExpiryPolicy(long create, long update, long access) { + private InteropExpiryPolicy(long create, long update, long access) { this.create = convert(create); this.update = convert(update); this.access = convert(access); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtension.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtension.java new file mode 100644 index 0000000000000..5d2040c6f4b70 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheExtension.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.platform.cache; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.binary.BinaryRawReaderEx; +import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; + +/** + * Platform cache extension. Decouples other modules from cache. + */ +public interface PlatformCacheExtension { + /** + * Get extension ID. Must be unique among all extensions. + * + * @return Extension ID. + */ + public int id(); + + /** + * Invokes in-out operation with long return type. + * + * @param target Target cache. + * @param type Operation type. + * @param reader Reader. + * @param mem Memory. + * @return Result. + * @throws IgniteCheckedException If failed. + */ + long processInOutStreamLong(PlatformCache target, int type, BinaryRawReaderEx reader, PlatformMemory mem) + throws IgniteCheckedException; +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationEx.java index ea11ce9e44492..931a18eb812cb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationEx.java @@ -18,10 +18,12 @@ package org.apache.ignite.internal.processors.platform.cpp; import org.apache.ignite.internal.processors.platform.PlatformConfigurationEx; +import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension; import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway; import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManagerImpl; import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; import org.apache.ignite.platform.cpp.PlatformCppConfiguration; +import org.jetbrains.annotations.Nullable; import java.util.Collection; @@ -73,6 +75,11 @@ public PlatformCppConfigurationEx(PlatformCppConfiguration cfg, PlatformCallback return warns; } + /** {@inheritDoc} */ + @Override @Nullable public Collection cacheExtensions() { + return null; + } + /** * @param warnings Warnings. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java index eaf0997a8a296..78fb755decc00 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java @@ -18,12 +18,16 @@ package org.apache.ignite.internal.processors.platform.dotnet; import org.apache.ignite.internal.processors.platform.PlatformConfigurationEx; +import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension; import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway; import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManagerImpl; import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; +import org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionCacheExtension; import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration; +import org.jetbrains.annotations.Nullable; import java.util.Collection; +import java.util.Collections; /** * Extended .Net configuration. @@ -73,6 +77,11 @@ public PlatformDotNetConfigurationEx(PlatformDotNetConfiguration cfg, PlatformCa return warnings; } + /** {@inheritDoc} */ + @Nullable @Override public Collection cacheExtensions() { + return Collections.singleton(new PlatformDotNetSessionCacheExtension()); + } + /** * @param warnings Warnings. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionCacheExtension.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionCacheExtension.java new file mode 100644 index 0000000000000..aa5f69f6ce58d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionCacheExtension.java @@ -0,0 +1,144 @@ +/* + * 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.platform.websession; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.binary.BinaryRawReaderEx; +import org.apache.ignite.internal.binary.BinaryRawWriterEx; +import org.apache.ignite.internal.processors.platform.cache.PlatformCache; +import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension; +import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; +import org.apache.ignite.internal.processors.platform.utils.PlatformWriterClosure; +import org.apache.ignite.internal.util.typedef.internal.S; + +import java.sql.Timestamp; +import java.util.UUID; + +/** + * Custom entry processor invoker. + */ +public class PlatformDotNetSessionCacheExtension implements PlatformCacheExtension { + /** Extension ID. */ + private static final int EXT_ID = 0; + + /** Operation: session lock. */ + private static final int OP_LOCK = 1; + + /** Operation: session set/unlock. */ + private static final int OP_SET_AND_UNLOCK = 2; + + /** Operation: session get without lock. */ + private static final int OP_GET = 3; + + /** Operation: session put without lock. */ + private static final int OP_PUT = 4; + + /** Operation: session remove without lock. */ + private static final int OP_REMOVE = 5; + + /** {@inheritDoc} */ + @Override public int id() { + return EXT_ID; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public long processInOutStreamLong(PlatformCache target, int type, BinaryRawReaderEx reader, + PlatformMemory mem) throws IgniteCheckedException { + switch (type) { + case OP_LOCK: { + String key = reader.readString(); + UUID lockNodeId = reader.readUuid(); + long lockId = reader.readLong(); + Timestamp lockTime = reader.readTimestamp(); + + final PlatformDotNetSessionLockResult res = (PlatformDotNetSessionLockResult) + target.rawCache().invoke(key, new PlatformDotNetSessionLockProcessor(lockNodeId, lockId, lockTime)); + + return target.writeResult(mem, res, new PlatformWriterClosure() { + @Override public void write(BinaryRawWriterEx writer, Object val) { + res.writeBinary(writer); + } + }); + } + + case OP_SET_AND_UNLOCK: { + String key = reader.readString(); + + PlatformDotNetSessionSetAndUnlockProcessor proc; + + if (reader.readBoolean()) { + PlatformDotNetSessionData data = new PlatformDotNetSessionData(); + + data.readBinary(reader); + + proc = new PlatformDotNetSessionSetAndUnlockProcessor(data); + } + else { + UUID lockNodeId = reader.readUuid(); + long lockId = reader.readLong(); + + proc = new PlatformDotNetSessionSetAndUnlockProcessor(lockNodeId, lockId); + } + + target.rawCache().invoke(key, proc); + + return target.writeResult(mem, null); + } + + case OP_GET: { + String key = reader.readString(); + + final PlatformDotNetSessionData data = (PlatformDotNetSessionData)target.rawCache().get(key); + + return target.writeResult(mem, data, new PlatformWriterClosure() { + @Override public void write(BinaryRawWriterEx writer, Object val) { + data.writeBinary(writer); + } + }); + } + + case OP_PUT: { + String key = reader.readString(); + + PlatformDotNetSessionData data = new PlatformDotNetSessionData(); + + data.readBinary(reader); + + target.rawCache().put(key, data); + + return target.writeResult(mem, null); + } + + case OP_REMOVE: { + String key = reader.readString(); + + target.rawCache().remove(key); + + return target.writeResult(mem, null); + } + } + + throw new IgniteCheckedException("Unsupported operation type: " + type); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(PlatformDotNetSessionCacheExtension.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionData.java new file mode 100644 index 0000000000000..18dbab0687665 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionData.java @@ -0,0 +1,260 @@ +/* + * 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.platform.websession; + +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryRawReader; +import org.apache.ignite.binary.BinaryRawWriter; +import org.apache.ignite.binary.BinaryReader; +import org.apache.ignite.binary.BinaryWriter; +import org.apache.ignite.binary.Binarylizable; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +import java.sql.Timestamp; +import java.util.Map; +import java.util.TreeMap; +import java.util.UUID; + +/** + * Web session state data. + */ +@SuppressWarnings({"ReturnOfDateField", "AssignmentToDateFieldFromParameter"}) +public class PlatformDotNetSessionData implements Binarylizable { + /** Items. */ + private Map items; + + /** Static objects. */ + @GridToStringExclude + private byte[] staticObjects; + + /** Timeout. */ + private int timeout; + + /** Lock ID. */ + private long lockId; + + /** Lock node ID. */ + private UUID lockNodeId; + + /** Lock time. */ + private Timestamp lockTime; + + /** + * @return Items. + */ + public Map items() { + return items; + } + + /** + * @return Static objects. + */ + public byte[] staticObjects() { + return staticObjects; + } + + /** + * @return Timeout. + */ + public int timeout() { + return timeout; + } + + /** + * @return Lock ID. + */ + public long lockId() { + return lockId; + } + + /** + * @return Lock node ID. + */ + public UUID lockNodeId() { + return lockNodeId; + } + + /** + * @return Lock time. + */ + public Timestamp lockTime() { + return lockTime; + } + + /** + * @return {@code True} if locked. + */ + public boolean isLocked() { + return lockTime != null; + } + + /** + * Locks the session state data. + * + * @param lockNodeId Lock node ID. + * @param lockId Lock ID. + * @param lockTime Lock time. + * + * @return Unlocked data copy. + */ + public PlatformDotNetSessionData lock(UUID lockNodeId, long lockId, Timestamp lockTime) { + assert !isLocked(); + + PlatformDotNetSessionData res = copyWithoutLockInfo(); + + res.lockId = lockId; + res.lockNodeId = lockNodeId; + res.lockTime = lockTime; + + return res; + } + + /** + * Unlocks the session state data. + * + * @param lockNodeId Lock node ID. + * @param lockId Lock ID. + * + * @return Unlocked data copy. + */ + public PlatformDotNetSessionData unlock(UUID lockNodeId, long lockId) { + assert isLocked(); + + if (!this.lockNodeId.equals(lockNodeId)) + throw new IllegalStateException("Can not unlock session data: lock node id check failed."); + + if (this.lockId != lockId) + throw new IllegalStateException("Can not unlock session data: lock id check failed."); + + return copyWithoutLockInfo(); + } + + /** + * Update session state and release the lock. + * + * @param lockNodeId Lock node ID. + * @param lockId Lock ID. + * @param items Items. + * @param isDiff Diff flag. + * @param staticObjects Static objects. + * @param timeout Timeout. + * @return Result. + */ + public PlatformDotNetSessionData updateAndUnlock(UUID lockNodeId, long lockId, Map items, + boolean isDiff, byte[] staticObjects, int timeout) { + assert items != null; + + PlatformDotNetSessionData res = unlock(lockNodeId, lockId); + + if (!isDiff) { + // Not a diff: remove all + this.items.clear(); + } + + for (Map.Entry e : items.entrySet()) { + String key = e.getKey(); + byte[] value = e.getValue(); + + if (value != null) + this.items.put(key, value); + else + this.items.remove(key); // Null value indicates removed key. + } + + res.staticObjects = staticObjects; + res.timeout = timeout; + + return res; + } + + /** + * Gets a copy of this instance with non-lock properties set. + * + * @return Copied state data. + */ + private PlatformDotNetSessionData copyWithoutLockInfo() { + PlatformDotNetSessionData res = new PlatformDotNetSessionData(); + + res.staticObjects = staticObjects; + res.items = items; + res.timeout = timeout; + + return res; + } + + /** {@inheritDoc} */ + @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException { + BinaryRawWriter raw = writer.rawWriter(); + + writeBinary(raw); + } + + /** + * Writes to a binary writer. + * + * @param writer Binary writer. + */ + public void writeBinary(BinaryRawWriter writer) { + writer.writeInt(items.size()); + + for (Map.Entry e : items.entrySet()) { + writer.writeString(e.getKey()); + writer.writeByteArray(e.getValue()); + } + + writer.writeByteArray(staticObjects); + + writer.writeInt(timeout); + writer.writeUuid(lockNodeId); + writer.writeLong(lockId); + writer.writeTimestamp(lockTime); + } + + /** {@inheritDoc} */ + @Override public void readBinary(BinaryReader reader) throws BinaryObjectException { + BinaryRawReader raw = reader.rawReader(); + + readBinary(raw); + } + + /** + * Reads from a binary reader. + * + * @param reader Reader. + */ + public void readBinary(BinaryRawReader reader) { + items = new TreeMap<>(); + int count = reader.readInt(); + + for (int i = 0; i < count; i++) + items.put(reader.readString(), reader.readByteArray()); + + staticObjects = reader.readByteArray(); + + timeout = reader.readInt(); + lockNodeId = reader.readUuid(); + lockId = reader.readLong(); + lockTime = reader.readTimestamp(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(PlatformDotNetSessionData.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionLockProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionLockProcessor.java new file mode 100644 index 0000000000000..0e51448782120 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionLockProcessor.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.platform.websession; + +import org.apache.ignite.cache.CacheEntryProcessor; +import org.apache.ignite.internal.util.typedef.internal.S; + +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; +import java.sql.Timestamp; +import java.util.UUID; + +/** + * Entry processor that locks web session data. + */ +@SuppressWarnings("AssignmentToDateFieldFromParameter") +public class PlatformDotNetSessionLockProcessor implements CacheEntryProcessor { + /** */ + private static final long serialVersionUID = 0L; + + /** Lock node id. */ + private UUID lockNodeId; + + /** Lock id. */ + private long lockId; + + /** Lock time. */ + private Timestamp lockTime; + + /** + * Ctor. + * + * @param lockNodeId Lock node id. + * @param lockId Lock id. + * @param lockTime Lock time. + */ + public PlatformDotNetSessionLockProcessor(UUID lockNodeId, long lockId, Timestamp lockTime) { + this.lockNodeId = lockNodeId; + this.lockId = lockId; + this.lockTime = lockTime; + } + + /** {@inheritDoc} */ + @Override public Object process(MutableEntry entry, Object... args) + throws EntryProcessorException { + if (!entry.exists()) + return null; + + PlatformDotNetSessionData data = entry.getValue(); + + assert data != null; + + if (data.isLocked()) + return new PlatformDotNetSessionLockResult(false, null, data.lockTime(), data.lockId()); + + // Not locked: lock and return result + data = data.lock(lockNodeId, lockId, lockTime); + + // Apply. + entry.setValue(data); + + return new PlatformDotNetSessionLockResult(true, data, null, data.lockId()); + } + + /** {@inheritDoc */ + @Override public String toString() { + return S.toString(PlatformDotNetSessionLockProcessor.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionLockResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionLockResult.java new file mode 100644 index 0000000000000..cfa355c03cd97 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionLockResult.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.internal.processors.platform.websession; + +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryRawReader; +import org.apache.ignite.binary.BinaryRawWriter; +import org.apache.ignite.binary.BinaryReader; +import org.apache.ignite.binary.BinaryWriter; +import org.apache.ignite.binary.Binarylizable; +import org.apache.ignite.internal.util.typedef.internal.S; + +import java.sql.Timestamp; + +/** + * Result of the {@link PlatformDotNetSessionLockProcessor} execution. + */ +@SuppressWarnings({"AssignmentToDateFieldFromParameter", "ReturnOfDateField"}) +public class PlatformDotNetSessionLockResult implements Binarylizable { + /** Success flag. */ + private boolean success; + + /** Data. */ + private PlatformDotNetSessionData data; + + /** Lock time. */ + private Timestamp lockTime; + + /** Lock id. */ + private long lockId; + + /** + * Constructor. + * + * @param success Success flag. + * @param data Session data. + * @param lockTime Lock time. + */ + public PlatformDotNetSessionLockResult(boolean success, PlatformDotNetSessionData data, Timestamp lockTime, + long lockId) { + assert success ^ (data == null); + + this.success = success; + this.data = data; + this.lockTime = lockTime; + this.lockId = lockId; + } + + /** {@inheritDoc} */ + @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException { + BinaryRawWriter raw = writer.rawWriter(); + + writeBinary(raw); + } + + /** + * Writes to a binary writer. + * + * @param writer Binary writer. + */ + public void writeBinary(BinaryRawWriter writer) { + writer.writeBoolean(success); + + if (success) + data.writeBinary(writer); + + writer.writeTimestamp(lockTime); + writer.writeLong(lockId); + } + + /** {@inheritDoc} */ + @Override public void readBinary(BinaryReader reader) throws BinaryObjectException { + BinaryRawReader raw = reader.rawReader(); + + success = raw.readBoolean(); + + if (success) { + data = new PlatformDotNetSessionData(); + + data.readBinary(raw); + } + + lockTime = raw.readTimestamp(); + lockId = raw.readLong(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(PlatformDotNetSessionLockResult.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionSetAndUnlockProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionSetAndUnlockProcessor.java new file mode 100644 index 0000000000000..9015c5c252de4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionSetAndUnlockProcessor.java @@ -0,0 +1,179 @@ +/* + * 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.platform.websession; + +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryRawReader; +import org.apache.ignite.binary.BinaryRawWriter; +import org.apache.ignite.binary.BinaryReader; +import org.apache.ignite.binary.BinaryWriter; +import org.apache.ignite.binary.Binarylizable; +import org.apache.ignite.cache.CacheEntryProcessor; +import org.apache.ignite.internal.util.typedef.internal.S; + +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; +import java.util.Map; +import java.util.TreeMap; +import java.util.UUID; + +/** + * Processor to unlock and optionally update the session. + */ +public class PlatformDotNetSessionSetAndUnlockProcessor implements + CacheEntryProcessor, Binarylizable { + /** */ + private static final long serialVersionUID = 0L; + + /** Lock node ID. */ + private UUID lockNodeId; + + /** Lock ID. */ + private long lockId; + + /** Update flag. */ + private boolean update; + + /** Data. */ + private Map items; + + /** Whether items collection represents a diff. */ + private boolean isDiff; + + /** Static data. */ + private byte[] staticData; + + /** Timeout. */ + private int timeout; + + /** + * Constructor for unlock. + * + * @param lockNodeId Lock node ID. + * @param lockId Lock ID. + */ + public PlatformDotNetSessionSetAndUnlockProcessor(UUID lockNodeId, long lockId) { + this(lockNodeId, lockId, false, null, false, null, 0); + } + + /** + * Constructor for unlock/update. + * + * @param data Data. + */ + public PlatformDotNetSessionSetAndUnlockProcessor(PlatformDotNetSessionData data) { + this(data.lockNodeId(), data.lockId(), true, data.items(), true, data.staticObjects(), data.timeout()); + } + + /** + * Constructor. + * + * @param lockNodeId Lock node ID. + * @param lockId Lock ID. + * @param update Whether to perform update. + * @param items Items. + * @param isDiff Whether items is a diff. + * @param staticData Static data. + * @param timeout Timeout. + */ + public PlatformDotNetSessionSetAndUnlockProcessor(UUID lockNodeId, long lockId, boolean update, + Map items, boolean isDiff, byte[] staticData, int timeout) { + this.lockNodeId = lockNodeId; + this.lockId = lockId; + this.update = update; + this.items = items; + this.isDiff = isDiff; + this.staticData = staticData; + this.timeout = timeout; + } + + /** {@inheritDoc} */ + @Override public Void process(MutableEntry entry, Object... args) + throws EntryProcessorException { + assert entry.exists(); + + PlatformDotNetSessionData data = entry.getValue(); + + assert data != null; + + // Unlock and update. + data = update + ? data.updateAndUnlock(lockNodeId, lockId, items, isDiff, staticData, timeout) + : data.unlock(lockNodeId, lockId); + + // Apply. + entry.setValue(data); + + return null; + } + + /** {@inheritDoc} */ + @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException { + BinaryRawWriter raw = writer.rawWriter(); + + raw.writeUuid(lockNodeId); + raw.writeLong(lockId); + raw.writeBoolean(update); + + if (update) { + raw.writeBoolean(isDiff); + raw.writeByteArray(staticData); + raw.writeInt(timeout); + + if (items != null) { + raw.writeInt(items.size()); + + for (Map.Entry e : items.entrySet()) { + raw.writeString(e.getKey()); + raw.writeByteArray(e.getValue()); + } + } + else + raw.writeInt(-1); + } + } + + /** {@inheritDoc} */ + @Override public void readBinary(BinaryReader reader) throws BinaryObjectException { + BinaryRawReader raw = reader.rawReader(); + + lockNodeId = raw.readUuid(); + lockId = raw.readLong(); + update = raw.readBoolean(); + + if (update) { + isDiff = raw.readBoolean(); + staticData = raw.readByteArray(); + timeout = raw.readInt(); + + int cnt = raw.readInt(); + + if (cnt >= 0) { + items = new TreeMap<>(); + + for (int i = 0; i < cnt; i++) + items.put(raw.readString(), raw.readByteArray()); + } + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(PlatformDotNetSessionSetAndUnlockProcessor.class, this); + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.csproj new file mode 100644 index 0000000000000..aed74db42c773 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.csproj @@ -0,0 +1,77 @@ + + + + + Debug + AnyCPU + {18EA4C71-A11D-4AB1-8042-418F7559D84F} + Library + Properties + Apache.Ignite.AspNet.Tests + Apache.Ignite.AspNet.Tests + v4.0 + 512 + + + true + full + false + bin\Debug\ + DEBUG;TRACE + prompt + 4 + true + AllRules.ruleset + + + pdbonly + true + bin\Release\ + TRACE + prompt + 4 + + + true + + + Apache.Ignite.AspNet.Tests.snk + + + + ..\packages\NUnit.Runners.2.6.3\tools\nunit.framework.dll + + + + + + + + + + + + + + + + + + + + + + {13EA96FC-CC83-4164-A7C0-4F30ED797460} + Apache.Ignite.AspNet + + + {6A62F66C-DA5B-4FBB-8CE7-A95F740FDC7A} + Apache.Ignite.Core.Tests + + + {4CD2F726-7E2B-46C4-A5BA-057BB82EECB6} + Apache.Ignite.Core + + + + \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.snk b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.snk new file mode 100644 index 0000000000000000000000000000000000000000..799e7426cc22c75b1d9227ef15b9be16ca3b5d69 GIT binary patch literal 596 zcmV-a0;~N80ssI2Bme+XQ$aES1ONa50098WI1J!kw@cT>G50Q>rWM7Ve$KF)*?C;E z-|_tRIQ@B?Dp;w$LjUnr@8QVJ>$g$jcVqU+r^pQ49l^WwLW5wA@r=z-7yVbVuqIG8 zH^`hNH}heDe>>etCSOfNM-~;Gn3FbH=E~b3Xl|Cz5&3CEDkOJ^w`c>vm0sy;rbFDm)+jiE6CRxxzp2(vm{nRko zyu37iYmu#%&mGZDWup?f4QWHBdodC-*Y734QiR>Qyb!}(u!cVctHhaZjq)FSt$8)W8n>~=n z6{a3iqa@Mc3aidg&2}il<)?CS`n$hwf`Ii z#4eft)WS|@Mu1+5Si4L&6xMNZgtQX{pUmY}Iiz=8Pw{K(rqvYzYsKD=)JAI1*8Vic ihIz#0$MA9g*TUC{Xy9bze(k}}o64o!sUK4b(_ejv91 literal 0 HcmV?d00001 diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config new file mode 100644 index 0000000000000..86ee3d4544845 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config @@ -0,0 +1,72 @@ + + + + + + +
          +
          +
          + + + + + + + + + + + 127.0.0.1:47500 + + + + + + + + + + + + + + 127.0.0.1:47500 + + + + + + + + + + + + + + 127.0.0.1:47500 + + + + + + + + + \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs new file mode 100644 index 0000000000000..c12fe93b3a06f --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs @@ -0,0 +1,492 @@ +/* + * 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. + */ + +// ReSharper disable UnusedAutoPropertyAccessor.Local +namespace Apache.Ignite.AspNet.Tests +{ + using System; + using System.Collections; + using System.Collections.Generic; + using System.Threading.Tasks; + using Apache.Ignite.AspNet.Impl; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Cache.Expiry; + using Apache.Ignite.Core.Cache.Query; + using Apache.Ignite.Core.Cache.Query.Continuous; + using NUnit.Framework; + + /// + /// Tests for . + /// + public class ExpiryCacheHolderTest + { + /// + /// Tests the expiry policy. + /// + [Test] + public void TestExpiryPolicy() + { + var cache = new CacheEx(); + + Assert.IsNull(cache.ExpiryPolicy); + + var holder = new ExpiryCacheHolder(cache); + + // Check same cache for same expiry. + var cache1 = (CacheEx) holder.GetCacheWithExpiry(15); + + CheckExpiry(TimeSpan.FromSeconds(15), cache1); + Assert.AreNotSame(cache, cache1); + Assert.AreSame(cache1, holder.GetCacheWithExpiry(15)); + + // Check rounding. + var cache2 = (CacheEx) holder.GetCacheWithExpiry(DateTime.UtcNow.AddSeconds(15.1)); + Assert.AreSame(cache1, cache2); + + // Check no expiration. + var cache3 = (CacheEx) holder.GetCacheWithExpiry(DateTime.MaxValue); + Assert.AreSame(cache, cache3); + } + + /// + /// Checks the expiry. + /// + private static void CheckExpiry(TimeSpan timeSpan, CacheEx cache) + { + Assert.AreEqual(timeSpan, cache.ExpiryPolicy.GetExpiryForCreate()); + Assert.IsNull(cache.ExpiryPolicy.GetExpiryForUpdate()); + Assert.IsNull(cache.ExpiryPolicy.GetExpiryForAccess()); + } + + /// + /// Test cache implementation. + /// + private class CacheEx : ICache + { + public IExpiryPolicy ExpiryPolicy { get; set; } + + public IEnumerator> GetEnumerator() + { + throw new NotImplementedException(); + } + + IEnumerator IEnumerable.GetEnumerator() + { + return GetEnumerator(); + } + + public string Name { get; private set; } + + public IIgnite Ignite { get; private set; } + + public CacheConfiguration GetConfiguration() + { + throw new NotImplementedException(); + } + + public bool IsEmpty() + { + throw new NotImplementedException(); + } + + public bool IsKeepBinary { get; private set; } + + public ICache WithSkipStore() + { + throw new NotImplementedException(); + } + + public ICache WithExpiryPolicy(IExpiryPolicy plc) + { + return new CacheEx {ExpiryPolicy = plc}; + } + + public ICache WithKeepBinary() + { + throw new NotImplementedException(); + } + + public void LoadCache(ICacheEntryFilter p, params object[] args) + { + throw new NotImplementedException(); + } + + public Task LoadCacheAsync(ICacheEntryFilter p, params object[] args) + { + throw new NotImplementedException(); + } + + public void LocalLoadCache(ICacheEntryFilter p, params object[] args) + { + throw new NotImplementedException(); + } + + public Task LocalLoadCacheAsync(ICacheEntryFilter p, params object[] args) + { + throw new NotImplementedException(); + } + + public void LoadAll(IEnumerable keys, bool replaceExistingValues) + { + throw new NotImplementedException(); + } + + public Task LoadAllAsync(IEnumerable keys, bool replaceExistingValues) + { + throw new NotImplementedException(); + } + + public bool ContainsKey(int key) + { + throw new NotImplementedException(); + } + + public Task ContainsKeyAsync(int key) + { + throw new NotImplementedException(); + } + + public bool ContainsKeys(IEnumerable keys) + { + throw new NotImplementedException(); + } + + public Task ContainsKeysAsync(IEnumerable keys) + { + throw new NotImplementedException(); + } + + public int LocalPeek(int key, params CachePeekMode[] modes) + { + throw new NotImplementedException(); + } + + public bool TryLocalPeek(int key, out int value, params CachePeekMode[] modes) + { + throw new NotImplementedException(); + } + + public int this[int key] + { + get { throw new NotImplementedException(); } + set { throw new NotImplementedException(); } + } + + public int Get(int key) + { + throw new NotImplementedException(); + } + + public Task GetAsync(int key) + { + throw new NotImplementedException(); + } + + public bool TryGet(int key, out int value) + { + throw new NotImplementedException(); + } + + public Task> TryGetAsync(int key) + { + throw new NotImplementedException(); + } + + public IDictionary GetAll(IEnumerable keys) + { + throw new NotImplementedException(); + } + + public Task> GetAllAsync(IEnumerable keys) + { + throw new NotImplementedException(); + } + + public void Put(int key, int val) + { + throw new NotImplementedException(); + } + + public Task PutAsync(int key, int val) + { + throw new NotImplementedException(); + } + + public CacheResult GetAndPut(int key, int val) + { + throw new NotImplementedException(); + } + + public Task> GetAndPutAsync(int key, int val) + { + throw new NotImplementedException(); + } + + public CacheResult GetAndReplace(int key, int val) + { + throw new NotImplementedException(); + } + + public Task> GetAndReplaceAsync(int key, int val) + { + throw new NotImplementedException(); + } + + public CacheResult GetAndRemove(int key) + { + throw new NotImplementedException(); + } + + public Task> GetAndRemoveAsync(int key) + { + throw new NotImplementedException(); + } + + public bool PutIfAbsent(int key, int val) + { + throw new NotImplementedException(); + } + + public Task PutIfAbsentAsync(int key, int val) + { + throw new NotImplementedException(); + } + + public CacheResult GetAndPutIfAbsent(int key, int val) + { + throw new NotImplementedException(); + } + + public Task> GetAndPutIfAbsentAsync(int key, int val) + { + throw new NotImplementedException(); + } + + public bool Replace(int key, int val) + { + throw new NotImplementedException(); + } + + public Task ReplaceAsync(int key, int val) + { + throw new NotImplementedException(); + } + + public bool Replace(int key, int oldVal, int newVal) + { + throw new NotImplementedException(); + } + + public Task ReplaceAsync(int key, int oldVal, int newVal) + { + throw new NotImplementedException(); + } + + public void PutAll(IDictionary vals) + { + throw new NotImplementedException(); + } + + public Task PutAllAsync(IDictionary vals) + { + throw new NotImplementedException(); + } + + public void LocalEvict(IEnumerable keys) + { + throw new NotImplementedException(); + } + + public void Clear() + { + throw new NotImplementedException(); + } + + public Task ClearAsync() + { + throw new NotImplementedException(); + } + + public void Clear(int key) + { + throw new NotImplementedException(); + } + + public Task ClearAsync(int key) + { + throw new NotImplementedException(); + } + + public void ClearAll(IEnumerable keys) + { + throw new NotImplementedException(); + } + + public Task ClearAllAsync(IEnumerable keys) + { + throw new NotImplementedException(); + } + + public void LocalClear(int key) + { + throw new NotImplementedException(); + } + + public void LocalClearAll(IEnumerable keys) + { + throw new NotImplementedException(); + } + + public bool Remove(int key) + { + throw new NotImplementedException(); + } + + public Task RemoveAsync(int key) + { + throw new NotImplementedException(); + } + + public bool Remove(int key, int val) + { + throw new NotImplementedException(); + } + + public Task RemoveAsync(int key, int val) + { + throw new NotImplementedException(); + } + + public void RemoveAll(IEnumerable keys) + { + throw new NotImplementedException(); + } + + public Task RemoveAllAsync(IEnumerable keys) + { + throw new NotImplementedException(); + } + + public void RemoveAll() + { + throw new NotImplementedException(); + } + + public Task RemoveAllAsync() + { + throw new NotImplementedException(); + } + + public int GetLocalSize(params CachePeekMode[] modes) + { + throw new NotImplementedException(); + } + + public int GetSize(params CachePeekMode[] modes) + { + throw new NotImplementedException(); + } + + public Task GetSizeAsync(params CachePeekMode[] modes) + { + throw new NotImplementedException(); + } + + public void LocalPromote(IEnumerable keys) + { + throw new NotImplementedException(); + } + + public IQueryCursor> Query(QueryBase qry) + { + throw new NotImplementedException(); + } + + public IQueryCursor QueryFields(SqlFieldsQuery qry) + { + throw new NotImplementedException(); + } + + public IContinuousQueryHandle QueryContinuous(ContinuousQuery qry) + { + throw new NotImplementedException(); + } + + public IContinuousQueryHandle> QueryContinuous(ContinuousQuery qry, QueryBase initialQry) + { + throw new NotImplementedException(); + } + + public IEnumerable> GetLocalEntries(params CachePeekMode[] peekModes) + { + throw new NotImplementedException(); + } + + public TRes Invoke(int key, ICacheEntryProcessor processor, TArg arg) + { + throw new NotImplementedException(); + } + + public Task InvokeAsync(int key, ICacheEntryProcessor processor, TArg arg) + { + throw new NotImplementedException(); + } + + public IDictionary> InvokeAll(IEnumerable keys, ICacheEntryProcessor processor, TArg arg) + { + throw new NotImplementedException(); + } + + public Task>> InvokeAllAsync(IEnumerable keys, ICacheEntryProcessor processor, TArg arg) + { + throw new NotImplementedException(); + } + + public ICacheLock Lock(int key) + { + throw new NotImplementedException(); + } + + public ICacheLock LockAll(IEnumerable keys) + { + throw new NotImplementedException(); + } + + public bool IsLocalLocked(int key, bool byCurrentThread) + { + throw new NotImplementedException(); + } + + public ICacheMetrics GetMetrics() + { + throw new NotImplementedException(); + } + + public Task Rebalance() + { + throw new NotImplementedException(); + } + + public ICache WithNoRetries() + { + throw new NotImplementedException(); + } + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/AspNet/IgniteOutputCacheProviderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteOutputCacheProviderTest.cs similarity index 95% rename from modules/platforms/dotnet/Apache.Ignite.Core.Tests/AspNet/IgniteOutputCacheProviderTest.cs rename to modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteOutputCacheProviderTest.cs index 51bfdc17e509a..bf2fd7e7d3315 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/AspNet/IgniteOutputCacheProviderTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteOutputCacheProviderTest.cs @@ -15,13 +15,14 @@ * limitations under the License. */ -namespace Apache.Ignite.Core.Tests.AspNet +namespace Apache.Ignite.AspNet.Tests { using System; using System.Collections.Specialized; using System.Threading; - using Apache.Ignite.AspNet; + using Apache.Ignite.Core; using Apache.Ignite.Core.Common; + using Apache.Ignite.Core.Tests; using NUnit.Framework; /// @@ -70,18 +71,17 @@ public void TestInitialization() { var cacheProvider = new IgniteOutputCacheProvider(); - // Not initialized + // Not initialized. Assert.Throws(() => cacheProvider.Get("1")); - // Grid not started + // Invalid section. Assert.Throws(() => cacheProvider.Initialize("testName", new NameValueCollection { - {GridNameAttr, "invalidGridName"}, - {CacheNameAttr, CacheName} + {SectionNameAttr, "invalidSection"}, })); - // Valid grid + // Valid grid. cacheProvider = GetProvider(); cacheProvider.Set("1", 1, DateTime.MaxValue); diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateItemCollectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateItemCollectionTest.cs new file mode 100644 index 0000000000000..137382e7b8ad7 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateItemCollectionTest.cs @@ -0,0 +1,267 @@ +/* + * 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. + */ + +namespace Apache.Ignite.AspNet.Tests +{ + using System; + using System.IO; + using System.Linq; + using Apache.Ignite.AspNet.Impl; + using Apache.Ignite.Core.Impl.Binary; + using Apache.Ignite.Core.Impl.Binary.IO; + using NUnit.Framework; + + /// + /// Tests for . + /// + public class IgniteSessionStateItemCollectionTest + { + /// + /// Tests the empty collection. + /// + [Test] + public void TestEmpty() + { + var col1 = new IgniteSessionStateItemCollection(); + var col2 = SerializeDeserialize(col1); + + foreach (var col in new[] { col1, col2 }) + { + Assert.IsFalse(col.Dirty); + Assert.IsFalse(col.IsSynchronized); + Assert.AreEqual(0, col.Count); + Assert.IsNotNull(col.SyncRoot); + Assert.IsEmpty(col); + Assert.IsEmpty(col.OfType().ToArray()); + Assert.IsEmpty(col.Keys); + Assert.IsNotNull(col.SyncRoot); + + Assert.IsNull(col["key"]); + Assert.Throws(() => col[0] = "x"); + Assert.Throws(() => Assert.AreEqual(0, col[0])); + Assert.Throws(() => col.RemoveAt(0)); + + col.Clear(); + col.Remove("test"); + + Assert.AreEqual(0, col.Count); + + col.Dirty = true; + Assert.IsTrue(col.Dirty); + } + } + + /// + /// Tests the modification. + /// + [Test] + public void TestModification() + { + var col = new IgniteSessionStateItemCollection(); + + // Populate and check. + col["key"] = "val"; + col["1"] = 1; + + Assert.AreEqual("val", col["key"]); + Assert.AreEqual(1, col["1"]); + + Assert.AreEqual(2, col.Count); + Assert.IsTrue(col.Dirty); + + CollectionAssert.AreEquivalent(new[] {"key", "1"}, col); + CollectionAssert.AreEquivalent(new[] {"key", "1"}, col.Keys); + + // Modify using index. + col[0] = "val1"; + col[1] = 2; + + Assert.AreEqual("val1", col["key"]); + Assert.AreEqual(2, col["1"]); + + // Modify using key. + col["1"] = 3; + col["key"] = "val2"; + + Assert.AreEqual("val2", col["key"]); + Assert.AreEqual(3, col["1"]); + + // CopyTo. + var keys = new string[5]; + col.CopyTo(keys, 2); + Assert.AreEqual(new[] {null, null, "key", "1", null}, keys); + + // Remove. + col["2"] = 2; + col["3"] = 3; + + col.Remove("invalid"); + Assert.AreEqual(4, col.Count); + + col.Remove("1"); + + Assert.AreEqual(new[] { "key", "2", "3" }, col.OfType()); + Assert.AreEqual(null, col["1"]); + + Assert.AreEqual("val2", col["key"]); + Assert.AreEqual("val2", col[0]); + + Assert.AreEqual(2, col["2"]); + Assert.AreEqual(2, col[1]); + + Assert.AreEqual(3, col["3"]); + Assert.AreEqual(3, col[2]); + + // RemoveAt. + col.RemoveAt(0); + Assert.AreEqual(new[] { "2", "3" }, col.OfType()); + + // Clear. + Assert.AreEqual(2, col.Count); + + col.Clear(); + Assert.AreEqual(0, col.Count); + + // Set dirty. + var col1 = new IgniteSessionStateItemCollection {Dirty = true}; + Assert.IsTrue(col1.Dirty); + } + + /// + /// Tests dirty tracking. + /// + [Test] + public void TestApplyChanges() + { + Func getCol = () => + { + var res = new IgniteSessionStateItemCollection(); + + res["1"] = 1; + res["2"] = 2; + res["3"] = 3; + + return res; + }; + + var col = getCol(); + + var col0 = SerializeDeserialize(col); + + Assert.AreEqual(3, col0.Count); + + col0.Remove("1"); + col0["2"] = 22; + col0["4"] = 44; + + // Apply non-serialized changes. + col.ApplyChanges(col0); + + Assert.AreEqual(3, col.Count); + Assert.AreEqual(null, col["1"]); + Assert.AreEqual(22, col["2"]); + Assert.AreEqual(3, col["3"]); + Assert.AreEqual(44, col["4"]); + + // Apply serialized changes without WriteChangesOnly. + col = getCol(); + col.ApplyChanges(SerializeDeserialize(col0)); + + Assert.AreEqual(3, col.Count); + Assert.AreEqual(null, col["1"]); + Assert.AreEqual(22, col["2"]); + Assert.AreEqual(3, col["3"]); + Assert.AreEqual(44, col["4"]); + + // Apply serialized changes with WriteChangesOnly. + col = getCol(); + col.ApplyChanges(SerializeDeserialize(col0, true)); + + Assert.AreEqual(3, col.Count); + Assert.AreEqual(null, col["1"]); + Assert.AreEqual(22, col["2"]); + Assert.AreEqual(3, col["3"]); + Assert.AreEqual(44, col["4"]); + + // Remove key then add back. + col0.Remove("2"); + col0.Remove("3"); + col0["2"] = 222; + + col = getCol(); + col.ApplyChanges(SerializeDeserialize(col0)); + + Assert.AreEqual(2, col.Count); + Assert.AreEqual(222, col["2"]); + Assert.AreEqual(44, col["4"]); + + // Remove all. + col0 = SerializeDeserialize(getCol()); + col0.Clear(); + + col = getCol(); + col.ApplyChanges(SerializeDeserialize(col0, true)); + + Assert.AreEqual(0, col.Count); + + // Add to empty. + col0["-1"] = -1; + col0["-2"] = -2; + + col = getCol(); + col.ApplyChanges(SerializeDeserialize(col0)); + + Assert.AreEqual(2, col.Count); + Assert.AreEqual(-1, col0["-1"]); + Assert.AreEqual(-2, col0["-2"]); + + // Remove initial key, then add it back, then remove again. + col0 = SerializeDeserialize(getCol()); + + col0.Remove("1"); + col0.Remove("2"); + col0["1"] = "111"; + col0.Remove("1"); + + col = getCol(); + col.ApplyChanges(SerializeDeserialize(col0, true)); + + Assert.AreEqual(1, col.Count); + Assert.AreEqual(3, col["3"]); + } + + /// + /// Serializes and deserializes back an instance. + /// + private static IgniteSessionStateItemCollection SerializeDeserialize(IgniteSessionStateItemCollection data, + bool changesOnly = false) + { + var marsh = BinaryUtils.Marshaller; + + using (var stream = new BinaryHeapStream(128)) + { + var writer = marsh.StartMarshal(stream); + + data.WriteBinary(writer.GetRawWriter(), changesOnly); + + stream.Seek(0, SeekOrigin.Begin); + + return new IgniteSessionStateItemCollection(marsh.StartUnmarshal(stream)); + } + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreDataTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreDataTest.cs new file mode 100644 index 0000000000000..e8dcd7c53f8a9 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreDataTest.cs @@ -0,0 +1,117 @@ +/* + * 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. + */ + +namespace Apache.Ignite.AspNet.Tests +{ + using System; + using System.IO; + using System.Reflection; + using System.Web; + using Apache.Ignite.AspNet.Impl; + using Apache.Ignite.Core.Impl.Binary; + using Apache.Ignite.Core.Impl.Binary.IO; + using NUnit.Framework; + + /// + /// Tests for . + /// + public class IgniteSessionStateStoreDataTest + { + /// + /// Tests the data. + /// + [Test] + public void TestData() + { + // Modification method is internal. + var statics = new HttpStaticObjectsCollection(); + statics.GetType().GetMethod("Add", BindingFlags.Instance | BindingFlags.NonPublic) + .Invoke(statics, new object[] { "int", typeof(int), false }); + + var data = new IgniteSessionStateStoreData(statics, 44); + + data.Items["key"] = "val"; + + Assert.AreEqual(44, data.Timeout); + Assert.AreEqual(1, data.StaticObjects.Count); + Assert.AreEqual(0, data.StaticObjects["int"]); + Assert.AreEqual("val", data.Items["key"]); + } + + /// + /// Tests the empty data. + /// + [Test] + public void TestEmpty() + { + var data = new IgniteSessionStateStoreData(null, 0); + + Assert.AreEqual(0, data.LockId); + Assert.AreEqual(0, data.Items.Count); + Assert.AreEqual(0, data.Timeout); + Assert.IsNull(data.LockNodeId); + Assert.IsNull(data.LockTime); + Assert.IsNull(data.StaticObjects); + } + + /// + /// Tests the serialization. + /// + [Test] + public void TestSerialization() + { + var data = new IgniteSessionStateStoreData(null, 96) + { + Timeout = 97, + LockId = 11, + LockNodeId = Guid.NewGuid(), + LockTime = DateTime.UtcNow.AddHours(-1), + }; + + data.Items["key1"] = 1; + data.Items["key2"] = 2; + + var data0 = SerializeDeserialize(data); + + Assert.AreEqual(data.Timeout, data0.Timeout); + Assert.AreEqual(data.LockId, data0.LockId); + Assert.AreEqual(data.LockNodeId, data0.LockNodeId); + Assert.AreEqual(data.LockTime, data0.LockTime); + Assert.AreEqual(data.Items.Keys, data0.Items.Keys); + } + + + /// + /// Serializes and deserializes back an instance. + /// + private static IgniteSessionStateStoreData SerializeDeserialize(IgniteSessionStateStoreData data) + { + var marsh = BinaryUtils.Marshaller; + + using (var stream = new BinaryHeapStream(128)) + { + var writer = marsh.StartMarshal(stream); + + data.WriteBinary(writer.GetRawWriter(), false); + + stream.Seek(0, SeekOrigin.Begin); + + return new IgniteSessionStateStoreData(marsh.StartUnmarshal(stream)); + } + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs new file mode 100644 index 0000000000000..fc239add7034e --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs @@ -0,0 +1,425 @@ +/* + * 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. + */ + +namespace Apache.Ignite.AspNet.Tests +{ + using System; + using System.Collections.Specialized; + using System.Linq; + using System.Reflection; + using System.Threading; + using System.Threading.Tasks; + using System.Web; + using System.Web.SessionState; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Common; + using Apache.Ignite.Core.Tests; + using NUnit.Framework; + + /// + /// Tests for . + /// + public class IgniteSessionStateStoreProviderTest + { + /** Grid name XML config attribute. */ + private const string GridNameAttr = "gridName"; + + /** Cache name XML config attribute. */ + private const string CacheNameAttr = "cacheName"; + + /** Section name XML config attribute. */ + private const string SectionNameAttr = "igniteConfigurationSectionName"; + + /** Grid name. */ + private const string GridName = "grid1"; + + /** Cache name. */ + private const string CacheName = "myCache"; + + /** Session id. */ + private const string Id = "1"; + + /** Test context. */ + private static readonly HttpContext HttpContext = + new HttpContext(new HttpRequest(null, "http://tempuri.org", null), new HttpResponse(null)); + + /// + /// Fixture setup. + /// + [TestFixtureSetUp] + public void TestFixtureSetUp() + { + Ignition.Start(new IgniteConfiguration(TestUtils.GetTestConfiguration()) { GridName = GridName }); + } + + /// + /// Fixture teardown. + /// + [TestFixtureTearDown] + public void TestFixtureTearDown() + { + Ignition.StopAll(true); + } + + /// + /// Test teardown. + /// + [TearDown] + public void TearDown() + { + // Clear all caches. + var ignite = Ignition.GetIgnite(GridName); + ignite.GetCacheNames().ToList().ForEach(x => ignite.GetCache(x).RemoveAll()); + } + + /// + /// Test setup. + /// + [SetUp] + public void SetUp() + { + // Make sure caches are empty. + var ignite = Ignition.GetIgnite(GridName); + + foreach (var cache in ignite.GetCacheNames().Select(x => ignite.GetCache(x))) + CollectionAssert.IsEmpty(cache.ToArray()); + } + + /// + /// Tests provider initialization. + /// + [Test] + public void TestInitialization() + { + var stateProvider = new IgniteSessionStateStoreProvider(); + + SessionStateActions actions; + bool locked; + TimeSpan lockAge; + object lockId; + + // Not initialized. + Assert.Throws(() => + stateProvider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions)); + + // Invalid section. + Assert.Throws(() => + stateProvider.Initialize("testName", new NameValueCollection + { + {SectionNameAttr, "invalidSection"}, + {CacheNameAttr, CacheName} + })); + + // Valid grid. + stateProvider = GetProvider(); + + CheckProvider(stateProvider); + + // Same grid once again. + stateProvider = GetProvider(); + + CheckProvider(stateProvider); + } + + /// + /// Tests autostart from web configuration section. + /// + [Test] + public void TestStartFromWebConfigSection() + { + var provider = new IgniteSessionStateStoreProvider(); + + provider.Initialize("testName3", new NameValueCollection + { + {SectionNameAttr, "igniteConfiguration3"}, + {CacheNameAttr, "cacheName3"} + }); + + CheckProvider(provider); + } + + /// + /// Tests the caching. + /// + [Test] + public void TestCaching() + { + bool locked; + TimeSpan lockAge; + object lockId; + SessionStateActions actions; + + var provider = GetProvider(); + + // Not locked, no item. + var res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions); + Assert.IsNull(res); + Assert.IsNull(lockId); + Assert.IsFalse(locked); + Assert.AreEqual(TimeSpan.Zero, lockAge); + Assert.AreEqual(SessionStateActions.None, actions); + + // Exclusive: not locked, no item. + res = provider.GetItemExclusive(HttpContext, Id, out locked, out lockAge, out lockId, out actions); + Assert.IsNull(res); + Assert.IsNull(lockId); + Assert.IsFalse(locked); + Assert.AreEqual(TimeSpan.Zero, lockAge); + Assert.AreEqual(SessionStateActions.None, actions); + + // Add item. + provider.CreateUninitializedItem(HttpContext, Id, 7); + + // Check added item. + res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions); + Assert.IsNotNull(res); + Assert.IsNull(lockId); + Assert.AreEqual(7, res.Timeout); + Assert.IsFalse(locked); + Assert.AreEqual(TimeSpan.Zero, lockAge); + Assert.AreEqual(SessionStateActions.None, actions); + + // Lock and update. + res = provider.GetItemExclusive(HttpContext, Id, out locked, out lockAge, out lockId, out actions); + Assert.IsNotNull(res); + Assert.IsNotNull(lockId); + Assert.IsFalse(locked); + Assert.AreEqual(TimeSpan.Zero, lockAge); + Assert.AreEqual(SessionStateActions.None, actions); + provider.SetAndReleaseItemExclusive(HttpContext, Id, UpdateStoreData(res), lockId, true); + + // Not locked, item present. + res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions); + CheckStoreData(res); + Assert.IsNull(lockId); + Assert.IsFalse(locked); + Assert.AreEqual(TimeSpan.Zero, lockAge); + Assert.AreEqual(SessionStateActions.None, actions); + + // Lock item. + res = provider.GetItemExclusive(HttpContext, Id, out locked, out lockAge, out lockId, out actions); + CheckStoreData(res); + Assert.IsNotNull(lockId); + Assert.IsFalse(locked); + Assert.AreEqual(TimeSpan.Zero, lockAge); + Assert.AreEqual(SessionStateActions.None, actions); + + // Try to get it in a different thread. + Task.Factory.StartNew(() => + { + object lockId1; // do not overwrite lockId + res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId1, out actions); + Assert.IsNull(res); + Assert.IsNotNull(lockId1); + Assert.IsTrue(locked); + Assert.Greater(lockAge, TimeSpan.Zero); + Assert.AreEqual(SessionStateActions.None, actions); + }).Wait(); + + // Try to get it in a different thread. + Task.Factory.StartNew(() => + { + object lockId1; // do not overwrite lockId + res = provider.GetItemExclusive(HttpContext, Id, out locked, out lockAge, out lockId1, out actions); + Assert.IsNull(res); + Assert.IsNotNull(lockId1); + Assert.IsTrue(locked); + Assert.Greater(lockAge, TimeSpan.Zero); + Assert.AreEqual(SessionStateActions.None, actions); + }).Wait(); + + // Release item. + provider.ReleaseItemExclusive(HttpContext, Id, lockId); + + // Make sure it is accessible in a different thread. + Task.Factory.StartNew(() => + { + res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions); + Assert.IsNotNull(res); + Assert.IsFalse(locked); + Assert.AreEqual(TimeSpan.Zero, lockAge); + Assert.AreEqual(SessionStateActions.None, actions); + }).Wait(); + + // Remove item. + provider.RemoveItem(HttpContext, Id, lockId, null); + + // Check removal. + res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions); + Assert.IsNull(res); + Assert.IsFalse(locked); + Assert.AreEqual(TimeSpan.Zero, lockAge); + Assert.AreEqual(SessionStateActions.None, actions); + } + + /// + /// Tests the create new store data. + /// + [Test] + public void TestCreateNewStoreData() + { + var provider = GetProvider(); + + var data = provider.CreateNewStoreData(HttpContext, 56); + + Assert.AreEqual(56, data.Timeout); + Assert.IsEmpty(data.Items); + Assert.IsEmpty(data.StaticObjects); + + // Check that caches are empty. + var ignite = Ignition.GetIgnite(GridName); + Assert.IsFalse(ignite.GetCacheNames().SelectMany(x => ignite.GetCache(x)).Any()); + } + + /// + /// Tests the expiry. + /// + [Test] + [Category(TestUtils.CategoryIntensive)] // Minimum expiration is 1 minute + public void TestExpiry() + { + var provider = GetProvider(); + + bool locked; + TimeSpan lockAge; + object lockId; + SessionStateActions actions; + + // Callbacks are not supported for now. + Assert.IsFalse(GetProvider().SetItemExpireCallback(null)); + + // Check there is no item. + var res = provider.GetItem(HttpContext, "myId", out locked, out lockAge, out lockId, out actions); + Assert.IsNull(res); + + // Put an item. + provider.CreateUninitializedItem(HttpContext, "myId", 1); + + // Check that it is there. + res = provider.GetItem(HttpContext, "myId", out locked, out lockAge, out lockId, out actions); + Assert.IsNotNull(res); + + // Wait a minute and check again. + Thread.Sleep(TimeSpan.FromMinutes(1.05)); + + res = provider.GetItem(HttpContext, "myId", out locked, out lockAge, out lockId, out actions); + Assert.IsNull(res); + } + + /// + /// Tests the create uninitialized item. + /// + [Test] + public void TestCreateUninitializedItem() + { + bool locked; + TimeSpan lockAge; + object lockId; + SessionStateActions actions; + + var provider = GetProvider(); + provider.CreateUninitializedItem(HttpContext, "myId", 45); + + var res = provider.GetItem(HttpContext, "myId", out locked, out lockAge, out lockId, out actions); + Assert.IsNotNull(res); + Assert.AreEqual(45, res.Timeout); + Assert.AreEqual(0, res.Items.Count); + Assert.AreEqual(0, res.StaticObjects.Count); + } + + /// + /// Updates the store data. + /// + private static SessionStateStoreData UpdateStoreData(SessionStateStoreData data) + { + data.Timeout = 8; + + data.Items["name1"] = 1; + data.Items["name2"] = "2"; + + var statics = data.StaticObjects; + + // Modification method is internal. + statics.GetType().GetMethod("Add", BindingFlags.Instance | BindingFlags.NonPublic) + .Invoke(statics, new object[] {"int", typeof(int), false}); + + CheckStoreData(data); + + return data; + } + + /// + /// Checks that store data is the same as returns. + /// + private static void CheckStoreData(SessionStateStoreData data) + { + Assert.IsNotNull(data); + + Assert.AreEqual(8, data.Timeout); + + Assert.AreEqual(1, data.Items["name1"]); + Assert.AreEqual(1, data.Items[0]); + + Assert.AreEqual("2", data.Items["name2"]); + Assert.AreEqual("2", data.Items[1]); + + Assert.AreEqual(0, data.StaticObjects["int"]); + } + + /// + /// Gets the initialized provider. + /// + private static IgniteSessionStateStoreProvider GetProvider() + { + var stateProvider = new IgniteSessionStateStoreProvider(); + + stateProvider.Initialize("testName", new NameValueCollection + { + {GridNameAttr, GridName}, + {CacheNameAttr, CacheName} + }); + + return stateProvider; + } + + /// + /// Checks the provider. + /// + private static void CheckProvider(SessionStateStoreProviderBase provider) + { + bool locked; + TimeSpan lockAge; + object lockId; + SessionStateActions actions; + + provider.InitializeRequest(HttpContext); + provider.CreateUninitializedItem(HttpContext, Id, 42); + + var data = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions); + Assert.IsNotNull(data); + Assert.AreEqual(42, data.Timeout); + Assert.IsFalse(locked); + Assert.AreEqual(TimeSpan.Zero, lockAge); + Assert.IsNull(lockId); + Assert.AreEqual(SessionStateActions.None, actions); + + provider.ResetItemTimeout(HttpContext, Id); + provider.EndRequest(HttpContext); + provider.Dispose(); + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs new file mode 100644 index 0000000000000..afaa6f07669fe --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs @@ -0,0 +1,42 @@ +/* +* 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. +*/ + +using System; +using System.Reflection; +using System.Runtime.InteropServices; + +// General Information about an assembly is controlled through the following +// set of attributes. Change these attribute values to modify the information +// associated with an assembly. +[assembly: AssemblyTitle("Apache.Ignite.AspNet.Tests")] +[assembly: AssemblyDescription("")] +[assembly: AssemblyConfiguration("")] +[assembly: AssemblyCompany("Apache Software Foundation")] +[assembly: AssemblyProduct("Apache Ignite.NET")] +[assembly: AssemblyCopyright("Copyright © 2016")] +[assembly: AssemblyTrademark("")] +[assembly: AssemblyCulture("")] + +[assembly: ComVisible(false)] + +[assembly: Guid("18ea4c71-a11d-4ab1-8042-418f7559d84f")] + +[assembly: AssemblyVersion("1.8.0.14218")] +[assembly: AssemblyFileVersion("1.8.0.14218")] +[assembly: AssemblyInformationalVersion("1.8.0")] + +[assembly: CLSCompliant(true)] \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/packages.config b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/packages.config new file mode 100644 index 0000000000000..c1198cba1a0e0 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/packages.config @@ -0,0 +1,22 @@ + + + + + + + \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj b/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj index 0c273e0d72cdd..1ac452ffd8f35 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj @@ -47,7 +47,13 @@ + + + + + + diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.ruleset b/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.ruleset index bc7683ae8911e..5a77e40502daf 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.ruleset +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.ruleset @@ -2,7 +2,10 @@ + + + diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteOutputCacheProvider.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteOutputCacheProvider.cs index 64216ddaee632..d232726dd790c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteOutputCacheProvider.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteOutputCacheProvider.cs @@ -18,23 +18,20 @@ namespace Apache.Ignite.AspNet { using System; - using System.Collections.Generic; using System.Collections.Specialized; - using System.Configuration; using System.Diagnostics.CodeAnalysis; - using System.Globalization; using System.Web.Caching; + using Apache.Ignite.AspNet.Impl; using Apache.Ignite.Core; using Apache.Ignite.Core.Cache; - using Apache.Ignite.Core.Cache.Expiry; - using Apache.Ignite.Core.Common; /// /// ASP.NET output cache provider that uses Ignite cache as a storage. /// /// You can either start Ignite yourself, and provide gridName attribute, /// or provide igniteConfigurationSectionName attribute to start Ignite automatically from specified - /// configuration section (see ). + /// configuration section (see ) + /// using igniteConfigurationSectionName. /// /// cacheName attribute specifies Ignite cache name to use for data storage. This attribute can be omitted /// if cache name is null. @@ -42,26 +39,7 @@ namespace Apache.Ignite.AspNet public class IgniteOutputCacheProvider : OutputCacheProvider { /** */ - private const string GridName = "gridName"; - - /** */ - private const string CacheName = "cacheName"; - - /** */ - private const string IgniteConfigurationSectionName = "igniteConfigurationSectionName"; - - /** Max number of cached expiry caches. */ - private const int MaxExpiryCaches = 1000; - - /** */ - private volatile ICache _cache; - - /** Cached caches per expiry seconds. */ - private volatile Dictionary> _expiryCaches = - new Dictionary>(); - - /** Sync object. */ - private readonly object _syncRoot = new object(); + private volatile ExpiryCacheHolder _expiryCacheHolder; /// /// Returns a reference to the specified entry in the output cache. @@ -88,7 +66,7 @@ public override object Get(string key) /// public override object Add(string key, object entry, DateTime utcExpiry) { - return GetCacheWithExpiry(utcExpiry).GetAndPutIfAbsent(key, entry).Value; + return _expiryCacheHolder.GetCacheWithExpiry(utcExpiry).GetAndPutIfAbsent(key, entry).Value; } /// @@ -99,7 +77,7 @@ public override object Add(string key, object entry, DateTime utcExpiry) /// The time and date on which the cached expires. public override void Set(string key, object entry, DateTime utcExpiry) { - GetCacheWithExpiry(utcExpiry)[key] = entry; + _expiryCacheHolder.GetCacheWithExpiry(utcExpiry)[key] = entry; } /// @@ -121,46 +99,11 @@ public override void Initialize(string name, NameValueCollection config) { base.Initialize(name, config); - var gridName = config[GridName]; - var cacheName = config[CacheName]; - var cfgSection = config[IgniteConfigurationSectionName]; - - try - { - var grid = cfgSection != null - ? StartFromApplicationConfiguration(cfgSection) - : Ignition.GetIgnite(gridName); + var cache = ConfigUtil.InitializeCache(config, GetType()); - _cache = grid.GetOrCreateCache(cacheName); - } - catch (Exception ex) - { - throw new IgniteException(string.Format(CultureInfo.InvariantCulture, - "Failed to initialize {0}: {1}", GetType(), ex), ex); - } + _expiryCacheHolder = new ExpiryCacheHolder(cache); } - /// - /// Starts Ignite from application configuration. - /// - private static IIgnite StartFromApplicationConfiguration(string sectionName) - { - var section = ConfigurationManager.GetSection(sectionName) as IgniteConfigurationSection; - - if (section == null) - throw new ConfigurationErrorsException(string.Format(CultureInfo.InvariantCulture, - "Could not find {0} with name '{1}'", typeof(IgniteConfigurationSection).Name, sectionName)); - - var config = section.IgniteConfiguration; - - if (string.IsNullOrWhiteSpace(config.IgniteHome)) - { - // IgniteHome not set by user: populate from default directory - config = new IgniteConfiguration(config) {IgniteHome = IgniteWebUtils.GetWebIgniteHome()}; - } - - return Ignition.Start(config); - } /// /// Gets the cache. @@ -169,51 +112,12 @@ private static IIgnite StartFromApplicationConfiguration(string sectionName) { get { - var cache = _cache; - - if (cache == null) - throw new InvalidOperationException("IgniteOutputCacheProvider has not been initialized."); - - return cache; - } - } - - /// - /// Gets the cache with expiry policy according to provided expiration date. - /// - /// The UTC expiry. - /// Cache with expiry policy. - private ICache GetCacheWithExpiry(DateTime utcExpiry) - { - if (utcExpiry == DateTime.MaxValue) - return Cache; - - // Round up to seconds ([OutputCache] duration is in seconds). - var expirySeconds = (long) Math.Round((utcExpiry - DateTime.UtcNow).TotalSeconds); - - if (expirySeconds < 0) - expirySeconds = 0; - - ICache expiryCache; - - if (_expiryCaches.TryGetValue(expirySeconds, out expiryCache)) - return expiryCache; - - lock (_syncRoot) - { - if (_expiryCaches.TryGetValue(expirySeconds, out expiryCache)) - return expiryCache; - - // Copy on write with size limit - _expiryCaches = _expiryCaches.Count > MaxExpiryCaches - ? new Dictionary>() - : new Dictionary>(_expiryCaches); - - expiryCache = Cache.WithExpiryPolicy(new ExpiryPolicy(TimeSpan.FromSeconds(expirySeconds), null, null)); + var holder = _expiryCacheHolder; - _expiryCaches[expirySeconds] = expiryCache; + if (holder == null) + throw new InvalidOperationException(GetType() + " has not been initialized."); - return expiryCache; + return holder.Cache; } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs new file mode 100644 index 0000000000000..1ee6d925b6515 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs @@ -0,0 +1,494 @@ +/* + * 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. + */ + +namespace Apache.Ignite.AspNet +{ + using System; + using System.Collections.Specialized; + using System.Diagnostics; + using System.Diagnostics.CodeAnalysis; + using System.Globalization; + using System.Threading; + using System.Web; + using System.Web.SessionState; + using Apache.Ignite.AspNet.Impl; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Binary; + using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Common; + using Apache.Ignite.Core.Impl.Cache; + + /// + /// ASP.NET Session-State Store Provider that uses Ignite distributed cache as an underlying storage. + /// + /// You can either start Ignite yourself, and provide gridName attribute, + /// or provide igniteConfigurationSectionName attribute to start Ignite automatically from specified + /// configuration section (see ) + /// using igniteConfigurationSectionName. + /// + /// cacheName attribute specifies Ignite cache name to use for data storage. This attribute can be omitted + /// if cache name is null. + /// + /// Optional applicationId attribute allows sharing a single Ignite cache between multiple web applications. + /// + public class IgniteSessionStateStoreProvider : SessionStateStoreProviderBase + { + /** Extension id */ + private const int ExtensionId = 0; + + /// + /// Op codes for . + /// + private enum Op + { + /** Lock the session data. */ + Lock = 1, + + /** Update and unlock the session data. */ + SetAndUnlock = 2, + + /** Get the data without lock. */ + Get = 3, + + /** Put the data without lock. */ + Put = 4, + + /** Remove the data without lock. */ + Remove = 5 + } + + /** Application id config parameter. */ + private const string ApplicationId = "applicationId"; + + /** */ + private volatile string _applicationId; + + /** */ + private volatile ExpiryCacheHolder _expiryCacheHolder; + + /** */ + private static long _lockId; + + /// + /// Initializes the provider. + /// + /// The friendly name of the provider. + /// A collection of the name/value pairs representing the provider-specific attributes + /// specified in the configuration for this provider. + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")] + public override void Initialize(string name, NameValueCollection config) + { + base.Initialize(name, config); + + var cache = ConfigUtil.InitializeCache(config, GetType()); + + _expiryCacheHolder = new ExpiryCacheHolder(cache); + + _applicationId = config[ApplicationId]; + } + + + /// + /// Releases all resources used by the + /// implementation. + /// + public override void Dispose() + { + // No-op. + } + + /// + /// Sets a reference to the + /// delegate for the Session_OnEnd event defined in the Global.asax file. + /// + /// The + /// delegate for the Session_OnEnd event defined in the Global.asax file. + /// + /// true if the session-state store provider supports calling the Session_OnEnd event; otherwise, false. + /// + public override bool SetItemExpireCallback(SessionStateItemExpireCallback expireCallback) + { + // Expiration events not supported for now. + return false; + } + + /// + /// Called by the object + /// for per-request initialization. + /// + /// The for the current request. + public override void InitializeRequest(HttpContext context) + { + // No-op. + } + + /// + /// Returns read-only session-state data from the session data store. + /// + /// The for the current request. + /// The for the + /// current request. + /// When this method returns, contains a Boolean value that is set to true if the + /// requested session item is locked at the session data store; otherwise, false. + /// When this method returns, contains a object that + /// is set to the amount of time that an item in the session data store has been locked. + /// When this method returns, contains an object that is set to the lock identifier + /// for the current request. For details on the lock identifier, see "Locking Session-Store Data" + /// in the class summary. + /// When this method returns, contains one of the + /// values, indicating whether the current + /// session is an uninitialized, cookieless session. + /// + /// A populated with session values and + /// information from the session data store. + /// + public override SessionStateStoreData GetItem(HttpContext context, string id, out bool locked, + out TimeSpan lockAge, out object lockId, + out SessionStateActions actions) + { + actions = SessionStateActions.None; + lockId = null; + lockAge = TimeSpan.Zero; + locked = false; + + var key = GetKey(id); + var data = GetItem(key); + + if (data != null) + { + locked = data.LockNodeId != null; + + if (!locked) + { + return data; + } + + Debug.Assert(data.LockTime != null); + + lockAge = DateTime.UtcNow - data.LockTime.Value; + + lockId = data.LockId; + + return null; + } + + return null; + } + + /// + /// Returns read-only session-state data from the session data store. + /// + /// The for the current request. + /// The for the current + /// request. + /// When this method returns, contains a Boolean value that is set to true if a lock + /// is successfully obtained; otherwise, false. + /// When this method returns, contains a object that + /// is set to the amount of time that an item in the session data store has been locked. + /// When this method returns, contains an object that is set to the lock identifier + /// for the current request. For details on the lock identifier, see "Locking Session-Store Data" in + /// the class summary. + /// When this method returns, contains one of the + /// values, indicating whether the current + /// session is an uninitialized, cookieless session. + /// + /// A populated with session values + /// and information from the session data store. + /// + public override SessionStateStoreData GetItemExclusive(HttpContext context, string id, out bool locked, + out TimeSpan lockAge, + out object lockId, out SessionStateActions actions) + { + actions = SessionStateActions.None; // Our items never need initialization. + lockAge = TimeSpan.Zero; + lockId = null; + + var lockId0 = Interlocked.Increment(ref _lockId); + + var key = GetKey(id); + + var lockResult = LockItem(key, lockId0); + + // No item found. + if (lockResult == null) + { + locked = false; + + return null; + } + + // Item was already locked. + if (!lockResult.Success) + { + locked = true; + + Debug.Assert(lockResult.LockTime != null); + + lockAge = DateTime.UtcNow - lockResult.LockTime.Value; + lockId = lockResult.LockId; + + return null; + } + + // Item found and lock obtained. + locked = false; + lockId = lockId0; + + if (lockId0 != lockResult.Data.LockId) + throw new IgniteException(string.Format(CultureInfo.InvariantCulture, + "Invalid session state lock result, " + + "expected lockId: {0}, actual: {1}", lockId0, lockResult.Data.LockId)); + + return lockResult.Data; + } + + /// + /// Releases a lock on an item in the session data store. + /// + /// The for the current request. + /// The session identifier for the current request. + /// The lock identifier for the current request. + public override void ReleaseItemExclusive(HttpContext context, string id, object lockId) + { + UnlockItem(GetKey(id), (long) lockId); + } + + /// + /// Updates the session-item information in the session-state data store with values from the current request, + /// and clears the lock on the data. + /// + /// The for the current request. + /// The session identifier for the current request. + /// The object that + /// contains the current session values to be stored. + /// The lock identifier for the current request. + /// true to identify the session item as a new item; false to identify the session + /// item as an existing item. + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")] + public override void SetAndReleaseItemExclusive(HttpContext context, string id, SessionStateStoreData item, + object lockId, bool newItem) + { + Debug.Assert(item != null); + + var key = GetKey(id); + + var data = (IgniteSessionStateStoreData) item; + + if (!(lockId is long) || data.LockId != (long) lockId) + throw new IgniteException(string.Format(CultureInfo.InvariantCulture, + "Invalid session release request, expected lockId: {0}, actual: {1}", data.LockId, lockId)); + + SetAndUnlockItem(key, data); + } + + /// + /// Deletes item data from the session data store. + /// + /// The for the current request. + /// The session identifier for the current request. + /// The lock identifier for the current request. + /// The that represents + /// the item to delete from the data store. + public override void RemoveItem(HttpContext context, string id, object lockId, SessionStateStoreData item) + { + RemoveItem(GetKey(id)); + } + + /// + /// Updates the expiration date and time of an item in the session data store. + /// + /// The for the current request. + /// The session identifier for the current request. + public override void ResetItemTimeout(HttpContext context, string id) + { + // No-op. + + // This is not necessary since ResetItemTimeout is called right after SetAndReleaseItemExclusive, + // which itself resets the timeout when the item is inserted into cache. + } + + /// + /// Creates a new object to be used + /// for the current request. + /// + /// The for the current request. + /// The session-state + /// value for the new . + /// + /// A new for the current request. + /// + public override SessionStateStoreData CreateNewStoreData(HttpContext context, int timeout) + { + return new IgniteSessionStateStoreData(SessionStateUtility.GetSessionStaticObjects(context), timeout); + } + + /// + /// Adds a new session-state item to the data store. + /// + /// The for the current request. + /// The + /// for the current request. + /// The session + /// for the current request. + public override void CreateUninitializedItem(HttpContext context, string id, int timeout) + { + var cache = _expiryCacheHolder.GetCacheWithExpiry((long) timeout * 60); + + var key = GetKey(id); + + var data = new IgniteSessionStateStoreData(SessionStateUtility.GetSessionStaticObjects(context), timeout); + + PutItem(key, data, cache); + } + + /// + /// Called by the object at the end of a request. + /// + /// The for the current request. + public override void EndRequest(HttpContext context) + { + // No-op. + } + + /// + /// Gets the cache. + /// + private ICache Cache + { + get + { + var holder = _expiryCacheHolder; + + if (holder == null) + throw new InvalidOperationException(GetType() + " has not been initialized."); + + return holder.Cache; + } + } + + /// + /// Gets the key. + /// + private string GetKey(string sessionId) + { + return _applicationId == null ? sessionId : ApplicationId + "." + sessionId; + } + + /// + /// Writes the lock info. + /// + private void WriteLockInfo(IBinaryRawWriter writer, long lockId, bool writeTime = false) + { + writer.WriteGuid(Cache.Ignite.GetCluster().GetLocalNode().Id); + writer.WriteLong(lockId); + + if (writeTime) + writer.WriteTimestamp(DateTime.UtcNow); + } + + /// + /// Locks the item. + /// + private SessionStateLockResult LockItem(string key, long lockId) + { + return OutInOp(Op.Lock, + w => + { + w.WriteString(key); + WriteLockInfo(w, lockId, true); + }, + r => new SessionStateLockResult(r)); + } + + /// + /// Unlocks the item. + /// + private void UnlockItem(string key, long lockId) + { + OutOp(Op.SetAndUnlock, + w => + { + w.WriteString(key); + w.WriteBoolean(false); // Only unlock. + WriteLockInfo(w, lockId); + }); + } + + /// + /// Sets and unlocks the item. + /// + private void SetAndUnlockItem(string key, IgniteSessionStateStoreData data) + { + var cache = _expiryCacheHolder.GetCacheWithExpiry(data.Timeout * 60); + + OutOp(Op.SetAndUnlock, w => + { + w.WriteString(key); + w.WriteBoolean(true); // Unlock and update. + data.WriteBinary(w, true); + }, cache); + } + + /// + /// Puts the item. + /// + private void PutItem(string key, IgniteSessionStateStoreData data, ICache cache) + { + OutOp(Op.Put, w => + { + w.WriteString(key); + data.WriteBinary(w, false); + }, cache); + } + + /// + /// Gets the item. + /// + private IgniteSessionStateStoreData GetItem(string key) + { + return OutInOp(Op.Get, w => w.WriteString(key), r => new IgniteSessionStateStoreData(r)); + } + + /// + /// Removes the item. + /// + private void RemoveItem(string key) + { + OutOp(Op.Remove, w => w.WriteString(key)); + } + + /// + /// Invokes the extension operation. + /// + private void OutOp(Op op, Action writeAction, + ICache cache = null) + { + OutInOp(op, writeAction, null, cache); + } + + /// + /// Invokes the extension operation. + /// + private T OutInOp(Op op, Action writeAction, Func readFunc, + ICache cache = null) + { + cache = cache ?? Cache; + + return ((ICacheInternal) cache).DoOutInOpExtension(ExtensionId, (int) op, writeAction, readFunc); + } + } +} \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs new file mode 100644 index 0000000000000..3eb3d90c71677 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs @@ -0,0 +1,109 @@ +/* + * 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. + */ + +namespace Apache.Ignite.AspNet.Impl +{ + using System; + using System.Collections.Specialized; + using System.Configuration; + using System.Diagnostics; + using System.Globalization; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Common; + + /// + /// Config utils. + /// + internal static class ConfigUtil + { + /** */ + public const string GridName = "gridName"; + + /** */ + public const string CacheName = "cacheName"; + + /** */ + public const string IgniteConfigurationSectionName = "igniteConfigurationSectionName"; + + /// + /// Initializes the cache from configuration. + /// + public static ICache InitializeCache(NameValueCollection config, Type callerType) + { + Debug.Assert(config != null); + Debug.Assert(callerType != null); + + var gridName = config[GridName]; + var cacheName = config[CacheName]; + var cfgSection = config[IgniteConfigurationSectionName]; + + try + { + var grid = StartFromApplicationConfiguration(cfgSection, gridName); + + var cacheConfiguration = new CacheConfiguration(cacheName); + + return grid.GetOrCreateCache(cacheConfiguration); + } + catch (Exception ex) + { + throw new IgniteException(string.Format(CultureInfo.InvariantCulture, + "Failed to initialize {0}: {1}", callerType, ex), ex); + } + + } + + /// + /// Starts Ignite from application configuration. + /// + private static IIgnite StartFromApplicationConfiguration(string sectionName, string gridName) + { + IgniteConfiguration config; + + if (!string.IsNullOrEmpty(sectionName)) + { + var section = ConfigurationManager.GetSection(sectionName) as IgniteConfigurationSection; + + if (section == null) + throw new ConfigurationErrorsException(string.Format(CultureInfo.InvariantCulture, + "Could not find {0} with name '{1}'", typeof(IgniteConfigurationSection).Name, sectionName)); + + config = section.IgniteConfiguration; + } + else + config = new IgniteConfiguration {GridName = gridName}; + + // Check if already started. + var ignite = Ignition.TryGetIgnite(config.GridName); + + if (ignite != null) + return ignite; + + // Start. + if (string.IsNullOrWhiteSpace(config.IgniteHome)) + { + // IgniteHome not set by user: populate from default directory. + config = new IgniteConfiguration(config) { IgniteHome = IgniteWebUtils.GetWebIgniteHome() }; + } + + return Ignition.Start(config); + } + + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ExpiryCacheHolder.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ExpiryCacheHolder.cs new file mode 100644 index 0000000000000..9678c38d153af --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ExpiryCacheHolder.cs @@ -0,0 +1,113 @@ +/* + * 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. + */ + +namespace Apache.Ignite.AspNet.Impl +{ + using System; + using System.Collections.Generic; + using System.Diagnostics; + using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Expiry; + + /// + /// Holds WithExpiry caches per expiration interval to avoid garbage on frequent WithExpiry calls. + /// + internal class ExpiryCacheHolder + { + /** Max number of cached expiry caches. */ + private const int MaxExpiryCaches = 1000; + + /** */ + private readonly ICache _cache; + + /** Cached caches per expiry seconds. */ + private volatile Dictionary> _expiryCaches = + new Dictionary>(); + + /** Sync object. */ + private readonly object _syncRoot = new object(); + + /// + /// Initializes a new instance of the class. + /// + /// The cache. + public ExpiryCacheHolder(ICache cache) + { + Debug.Assert(cache != null); + + _cache = cache; + } + + /// + /// Gets the cache. + /// + public ICache Cache + { + get { return _cache; } + } + + /// + /// Gets the cache with expiry policy according to provided expiration date. + /// + /// The UTC expiry. + /// Cache with expiry policy. + public ICache GetCacheWithExpiry(DateTime utcExpiry) + { + if (utcExpiry == DateTime.MaxValue) + return _cache; + + Debug.Assert(utcExpiry.Kind == DateTimeKind.Utc); + + // Round up to seconds ([OutputCache] duration is in seconds). + var expirySeconds = (long)Math.Round((utcExpiry - DateTime.UtcNow).TotalSeconds); + + if (expirySeconds < 0) + expirySeconds = 0; + + return GetCacheWithExpiry(expirySeconds); + } + + /// + /// Gets the cache with expiry. + /// + /// The expiration interval (in seconds). + public ICache GetCacheWithExpiry(long expiry) + { + ICache expiryCache; + + if (_expiryCaches.TryGetValue(expiry, out expiryCache)) + return expiryCache; + + lock (_syncRoot) + { + if (_expiryCaches.TryGetValue(expiry, out expiryCache)) + return expiryCache; + + // Copy on write with size limit + _expiryCaches = _expiryCaches.Count > MaxExpiryCaches + ? new Dictionary>() + : new Dictionary>(_expiryCaches); + + expiryCache = Cache.WithExpiryPolicy(new ExpiryPolicy(TimeSpan.FromSeconds(expiry), null, null)); + + _expiryCaches[expiry] = expiryCache; + + return expiryCache; + } + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/IgniteSessionStateItemCollection.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/IgniteSessionStateItemCollection.cs new file mode 100644 index 0000000000000..d1ba5da99c6da --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/IgniteSessionStateItemCollection.cs @@ -0,0 +1,534 @@ +/* + * 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. + */ + +namespace Apache.Ignite.AspNet.Impl +{ + using System; + using System.Collections; + using System.Collections.Generic; + using System.Collections.Specialized; + using System.Diagnostics; + using System.Diagnostics.CodeAnalysis; + using System.IO; + using System.Linq; + using System.Runtime.Serialization.Formatters.Binary; + using System.Web.SessionState; + using Apache.Ignite.Core.Binary; + using Apache.Ignite.Core.Impl.Common; + + /// + /// Binarizable key-value collection with dirty item tracking. + /// + internal class IgniteSessionStateItemCollection : ISessionStateItemCollection + { + /** */ + private readonly Dictionary _dict; + + /** */ + private readonly List _list; + + /** Indicates where this is a new collection, not a deserialized old one. */ + private readonly bool _isNew; + + /** Removed keys. Hash set because keys can be removed multiple times. */ + private HashSet _removedKeys; + + /** Indicates that entire collection is dirty and can't be written as a diff. */ + private bool _dirtyAll; + + /// + /// Initializes a new instance of the class. + /// + /// The binary reader. + internal IgniteSessionStateItemCollection(IBinaryRawReader reader) + { + Debug.Assert(reader != null); + + var count = reader.ReadInt(); + + _dict = new Dictionary(count); + _list = new List(count); + + for (var i = 0; i < count; i++) + { + var key = reader.ReadString(); + + var valBytes = reader.ReadByteArray(); + + if (valBytes != null) + { + var entry = new Entry(key, true, valBytes); + + _dict[key] = _list.Count; + + _list.Add(entry); + } + else + AddRemovedKey(key); + } + + _isNew = false; + } + + /// + /// Initializes a new instance of the class. + /// + public IgniteSessionStateItemCollection() + { + _dict = new Dictionary(); + _list = new List(); + _isNew = true; + } + + /** */ + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "0", + Justification = "Validation is present.")] + public void CopyTo(Array array, int index) + { + IgniteArgumentCheck.NotNull(array, "array"); + IgniteArgumentCheck.Ensure(Count + index < array.Length, "array", + "The number of elements in the source collection is greater than the available space " + + "from specified index to the end of the array."); + + // This should return only keys. + foreach (var entry in _list) + array.SetValue(entry.Key, index++); + } + + /** */ + public IEnumerator GetEnumerator() + { + // This should return only keys. + return _list.Select(x => x.Key).GetEnumerator(); + } + + /** */ + public int Count + { + get { return _dict.Count; } + } + + /** */ + public object SyncRoot + { + get { return _list; } + } + + /** */ + public bool IsSynchronized + { + get { return false; } + } + + /** */ + public object this[string key] + { + get + { + var entry = GetEntry(key); + + if (entry == null) + return null; + + SetDirtyOnRead(entry); + + return entry.Value; + } + set + { + var entry = GetOrCreateDirtyEntry(key); + + entry.Value = value; + } + } + + /** */ + public object this[int index] + { + get + { + var entry = _list[index]; + + SetDirtyOnRead(entry); + + return entry.Value; + } + set + { + var entry = _list[index]; + + entry.IsDirty = true; + + entry.Value = value; + } + } + + /** */ + public NameObjectCollectionBase.KeysCollection Keys + { + get { return new NameObjectCollection(this).Keys; } + } + + + /** */ + public bool Dirty + { + get { return _dirtyAll || _list.Any(x => x.IsDirty); } + set { _dirtyAll = value; } + } + + /// + /// Writes this object to the given writer. + /// + public void WriteBinary(IBinaryRawWriter writer, bool changesOnly) + { + IgniteArgumentCheck.NotNull(writer, "writer"); + + if (_isNew || _dirtyAll || !changesOnly || (_removedKeys == null && _list.All(x => x.IsDirty))) + { + // Write in full mode. + writer.WriteInt(_list.Count); + + foreach (var entry in _list) + { + writer.WriteString(entry.Key); + + // Write as byte array to enable partial deserialization. + writer.WriteByteArray(entry.GetBytes()); + } + } + else + { + // Write in diff mode. + var removed = GetRemovedKeys(); + + var count = _list.Count(x => x.IsDirty) + (removed == null ? 0 : removed.Count); + + writer.WriteInt(count); // reserve count + + // Write removed keys as [key + null]. + if (removed != null) + { + foreach (var removedKey in removed) + { + writer.WriteString(removedKey); + writer.WriteByteArray(null); + } + } + + // Write dirty items. + foreach (var entry in _list) + { + if (!entry.IsDirty) + continue; + + writer.WriteString(entry.Key); + + // Write as byte array to enable partial deserialization. + writer.WriteByteArray(entry.GetBytes()); + } + } + } + + /// + /// Gets the removed keys. + /// + private ICollection GetRemovedKeys() + { + if (_removedKeys == null) + return null; + + // Filter out existing keys. + var removed = new HashSet(_removedKeys); + + foreach (var entry in _list) + removed.Remove(entry.Key); + + return removed; + } + + /// + /// Removes the specified key. + /// + public void Remove(string key) + { + var index = GetIndex(key); + + if (index < 0) + return; + + var entry = _list[index]; + Debug.Assert(key == entry.Key); + + _list.RemoveAt(index); + _dict.Remove(key); + + // Update all indexes. + for (var i = 0; i < _list.Count; i++) + _dict[_list[i].Key] = i; + + if (entry.IsInitial) + AddRemovedKey(key); + } + + /// + /// Removes at specified index. + /// + public void RemoveAt(int index) + { + var entry = _list[index]; + + _list.RemoveAt(index); + _dict.Remove(entry.Key); + + if (entry.IsInitial) + AddRemovedKey(entry.Key); + } + + /// + /// Clears this instance. + /// + public void Clear() + { + foreach (var entry in _list) + { + if (entry.IsInitial) + AddRemovedKey(entry.Key); + } + + _list.Clear(); + _dict.Clear(); + + _dirtyAll = true; + } + + /// + /// Applies the changes. + /// + public void ApplyChanges(IgniteSessionStateItemCollection changes) + { + var removed = changes._removedKeys; + + if (removed != null) + { + foreach (var key in removed) + Remove(key); + } + else + { + // Not a diff: replace all. + Clear(); + } + + foreach (var changedEntry in changes._list) + { + var entry = GetOrCreateDirtyEntry(changedEntry.Key); + + // Copy without deserialization. + changedEntry.CopyTo(entry); + } + } + + /// + /// Adds the removed key. + /// + private void AddRemovedKey(string key) + { + Debug.Assert(!_isNew); + + if (_removedKeys == null) + _removedKeys = new HashSet(); + + _removedKeys.Add(key); + } + + /// + /// Gets or creates an entry. + /// + private Entry GetOrCreateDirtyEntry(string key) + { + var entry = GetEntry(key); + + if (entry == null) + { + entry = new Entry(key, false, null); + + _dict[key] = _list.Count; + _list.Add(entry); + } + + entry.IsDirty = true; + + return entry; + } + + /// + /// Gets the entry. + /// + private Entry GetEntry(string key) + { + IgniteArgumentCheck.NotNull(key, "key"); + + int index; + + return !_dict.TryGetValue(key, out index) ? null : _list[index]; + } + + /// + /// Gets the index. + /// + private int GetIndex(string key) + { + int index; + + return !_dict.TryGetValue(key, out index) ? -1 : index; + } + + /// + /// Sets the dirty on read. + /// + private static void SetDirtyOnRead(Entry entry) + { + var type = entry.Value.GetType(); + + if (IsImmutable(type)) + return; + + entry.IsDirty = true; + } + + /// + /// Determines whether the specified type is immutable. + /// + private static bool IsImmutable(Type type) + { + type = Nullable.GetUnderlyingType(type) ?? type; // Unwrap nullable. + + if (type.IsPrimitive) + return true; + + if (type == typeof(string) || type == typeof(DateTime) || type == typeof(Guid) || type == typeof(decimal)) + return true; + + return false; + } + + /// + /// Inner entry. + /// + private class Entry + { + /** */ + public readonly bool IsInitial; + + /** */ + public readonly string Key; + + /** */ + public bool IsDirty; + + /** */ + private object _value; + + /** */ + private bool _isDeserialized; + + /// + /// Initializes a new instance of the class. + /// + public Entry(string key, bool isInitial, object value) + { + Debug.Assert(key != null); + + Key = key; + IsInitial = isInitial; + _isDeserialized = !isInitial; + _value = value; + } + + /// + /// Gets or sets the value. + /// + public object Value + { + get + { + if (!_isDeserialized) + { + using (var stream = new MemoryStream((byte[])_value)) + { + _value = new BinaryFormatter().Deserialize(stream); + } + + _isDeserialized = true; + } + + return _value; + } + set + { + _value = value; + _isDeserialized = true; + } + } + + /// + /// Copies contents to another entry. + /// + public void CopyTo(Entry entry) + { + Debug.Assert(entry != null); + + entry._isDeserialized = _isDeserialized; + entry._value = _value; + } + + /// + /// Gets the bytes. + /// + public byte[] GetBytes() + { + if (!_isDeserialized) + return (byte[]) _value; + + using (var stream = new MemoryStream()) + { + new BinaryFormatter().Serialize(stream, _value); + + return stream.ToArray(); + } + } + } + + /// + /// NameObjectCollectionBase.KeysCollection has internal constructor. + /// The only way to implement ISessionStateItemCollection.Keys property + /// is to have a NameObjectCollectionBase in hand. + /// + private class NameObjectCollection : NameObjectCollectionBase + { + /// + /// Initializes a new instance of the class. + /// + public NameObjectCollection(IEnumerable keys) + { + foreach (string key in keys) + BaseAdd(key, null); + } + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/IgniteSessionStateStoreData.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/IgniteSessionStateStoreData.cs new file mode 100644 index 0000000000000..32c36b5fe91de --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/IgniteSessionStateStoreData.cs @@ -0,0 +1,116 @@ +/* + * 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. + */ + +namespace Apache.Ignite.AspNet.Impl +{ + using System; + using System.IO; + using System.Web; + using System.Web.SessionState; + using Apache.Ignite.Core.Binary; + + /// + /// Ignite implementation. + /// + internal class IgniteSessionStateStoreData : SessionStateStoreData + { + /// + /// Initializes a new instance of the class. + /// + /// The reader. + public IgniteSessionStateStoreData(IBinaryRawReader reader) : base( + new IgniteSessionStateItemCollection(reader), + DeserializeStaticObjects(reader.ReadByteArray()), reader.ReadInt()) + { + LockNodeId = reader.ReadGuid(); + LockId = reader.ReadLong(); + LockTime = reader.ReadTimestamp(); + } + + /// + /// Initializes a new instance of the class. + /// + /// The static objects. + /// The timeout. + public IgniteSessionStateStoreData(HttpStaticObjectsCollection staticObjects, int timeout) + : base(new IgniteSessionStateItemCollection(), staticObjects, timeout) + { + // No-op. + } + + /// + /// Writes this object to the given writer. + /// + public void WriteBinary(IBinaryRawWriter writer, bool changesOnly) + { + ((IgniteSessionStateItemCollection)Items).WriteBinary(writer, changesOnly); + writer.WriteByteArray(SerializeStaticObjects()); + writer.WriteInt(Timeout); + + writer.WriteGuid(LockNodeId); + writer.WriteLong(LockId); + writer.WriteTimestamp(LockTime); + } + + /// + /// Gets or sets the lock node id. Null means not locked. + /// + public Guid? LockNodeId { get; set; } + + /// + /// Gets or sets the lock id. + /// + public long LockId { get; set; } + + /// + /// Gets or sets the lock time. + /// + public DateTime? LockTime { get; set; } + + /// + /// Deserializes the static objects. + /// + private static HttpStaticObjectsCollection DeserializeStaticObjects(byte[] bytes) + { + if (bytes == null) + return new HttpStaticObjectsCollection(); + + using (var stream = new MemoryStream(bytes)) + using (var reader = new BinaryReader(stream)) + { + return HttpStaticObjectsCollection.Deserialize(reader); + } + } + + /// + /// Serializes the static objects. + /// + private byte[] SerializeStaticObjects() + { + if (StaticObjects == null || StaticObjects.Count == 0) + return null; + + using (var stream = new MemoryStream()) + using (var writer = new BinaryWriter(stream)) + { + StaticObjects.Serialize(writer); + + return stream.ToArray(); + } + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/SessionStateLockResult.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/SessionStateLockResult.cs new file mode 100644 index 0000000000000..ebca8e05a971d --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/SessionStateLockResult.cs @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +namespace Apache.Ignite.AspNet.Impl +{ + using System; + using System.Diagnostics; + using Apache.Ignite.Core.Binary; + + /// + /// Result of the session state lock processor. + /// + internal class SessionStateLockResult + { + /** Success flag. */ + private readonly bool _success; + + /** Session state data. */ + private readonly IgniteSessionStateStoreData _data; + + /** Lock time. */ + private readonly DateTime? _lockTime; + + /** Lock id. */ + private readonly long _lockId; + + /// + /// Initializes a new instance of the class. + /// + /// The reader. + public SessionStateLockResult(IBinaryRawReader reader) + { + _success = reader.ReadBoolean(); + + if (_success) + _data = new IgniteSessionStateStoreData(reader); + + _lockTime = reader.ReadTimestamp(); + _lockId = reader.ReadLong(); + + Debug.Assert(_success ^ (_data == null)); + Debug.Assert(_success ^ (_lockTime != null)); + } + + /// + /// Gets a value indicating whether lock succeeded. + /// + public bool Success + { + get { return _success; } + } + + /// + /// Gets the data. Null when is false. + /// + public IgniteSessionStateStoreData Data + { + get { return _data; } + } + + /// + /// Gets the lock time. Null when is true. + /// + public DateTime? LockTime + { + get { return _lockTime; } + } + + /// + /// Gets the lock identifier. + /// + public long LockId + { + get { return _lockId; } + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs index ad61ecd5852a2..2a7da675982fb 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs @@ -17,6 +17,7 @@ using System; using System.Reflection; +using System.Runtime.CompilerServices; using System.Runtime.InteropServices; [assembly: AssemblyTitle("Apache.Ignite.AspNet")] @@ -36,4 +37,6 @@ [assembly: AssemblyFileVersion("1.8.0.14218")] [assembly: AssemblyInformationalVersion("1.8.0")] -[assembly: CLSCompliant(true)] \ No newline at end of file +[assembly: CLSCompliant(true)] + +[assembly: InternalsVisibleTo("Apache.Ignite.AspNet.Tests, PublicKey=0024000004800000940000000602000000240000525341310004000001000100c9380ce05eb74bd7c531f72e9ea615c59d7eceb09bd9795cb3dff1fcf638fd799c2a58a9be42fff156efe1c8cdebb751e27763f6c9a7c80cdc1dc1bbf44283608ef18ccd5017fd57b2b026503637c89c2537f361807f3bdd49265f4d444716159d989342561d324b1a0961640338bb32eaf67f4ae0c95f1b210f65404b0909c6")] \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj index 95fea8f01bdaa..b1e0dbe3b2d8d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj @@ -49,13 +49,11 @@ - - @@ -167,10 +165,6 @@ - - {13EA96FC-CC83-4164-A7C0-4F30ED797460} - Apache.Ignite.AspNet - {4CD2F726-7E2B-46C4-A5BA-057BB82EECB6} Apache.Ignite.Core diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs index c0b8599193505..6ca2f9df62e7a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs @@ -23,7 +23,6 @@ namespace Apache.Ignite.Core.Tests using System.Collections.Generic; using System.Linq; using System.Threading; - using Apache.Ignite.Core.Discovery; using Apache.Ignite.Core.Discovery.Tcp; using Apache.Ignite.Core.Discovery.Tcp.Static; using Apache.Ignite.Core.Impl; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs index 8ba3e29b0519c..fb47f29e50825 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs @@ -872,6 +872,21 @@ public void LocalPromote(IEnumerable keys) return AsyncInstance.GetTask(CacheOp.InvokeAll, reader => ReadInvokeAllResults(reader.Stream)); } + /** */ + public T DoOutInOpExtension(int extensionId, int opCode, Action writeAction, + Func readFunc) + { + return DoOutInOpX((int) CacheOp.Extension, writer => + { + writer.WriteInt(extensionId); + writer.WriteInt(opCode); + writeAction(writer); + }, + (input, res) => res == True + ? readFunc(Marshaller.StartUnmarshal(input)) + : default(T), ReadException); + } + /** */ public ICacheLock Lock(TK key) { diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs index 4c42bf35003a7..e6ca938c2cf9e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs @@ -61,6 +61,7 @@ internal enum CacheOp Replace2 = 37, Replace3 = 38, GetConfig = 39, - LoadAll = 40 + LoadAll = 40, + Extension = 41 } } \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/ICacheInternal.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/ICacheInternal.cs index a23cf08c7b71b..0349db837607b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/ICacheInternal.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/ICacheInternal.cs @@ -36,5 +36,19 @@ public interface ICacheInternal /// Cursor. /// IQueryCursor QueryFields(SqlFieldsQuery qry, Func readerFunc); + + /// + /// Invokes a cache extension. + /// + /// The type of the result. + /// The extension identifier. + /// The extension op code. + /// The write action. + /// The read action. + /// + /// Result of the processing. + /// + T DoOutInOpExtension(int extensionId, int opCode, Action writeAction, + Func readFunc); } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs index fd52c8a5d905e..4ee67dd0f48b3 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs @@ -21,7 +21,6 @@ namespace Apache.Ignite.Core.Impl.Unmanaged using System.Collections.Generic; using System.Diagnostics; using System.Diagnostics.CodeAnalysis; - using System.Globalization; using System.IO; using System.Runtime.InteropServices; using System.Threading; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs index 008ba5cae62a3..9fcbeb0862325 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs @@ -43,5 +43,6 @@ [assembly: InternalsVisibleTo("Apache.Ignite.Core.Tests, PublicKey=0024000004800000940000000602000000240000525341310004000001000100a5bf8e0062a26bde53ccf0f8c42ef5b122a22052f99aecacb7028adcc163050324ee3c75ff40eb0cbe2d0426fa20eca03726cad90d7eb882ff47f5361567a82b676a27565f88b2479d7b9354ae0a1e526ee781b6e11de943d8f4a49efb53765f8c954022bede0fca86c133fab038af8dc88b67d6b6e5b9796d6ca490e699efab")] [assembly: InternalsVisibleTo("Apache.Ignite.Benchmarks, PublicKey=0024000004800000940000000602000000240000525341310004000001000100a3e0c1df4cbedbd4ed0e88808401c69b69ec12575ed1c056ac9f448e018fb29af19d236b7b03563aad66c48ab2045e72971ed098d4f65d4cdd38d65abcb39b4f84c626b22ccab2754375f0e8c97dc304fa146f0eddad5cc40a71803a8f15b0b0bb0bff0d4bf0ff6a64bb1044e0d71e6e2405b83fd4c1f7b3e2cfc2e9d50823d4")] +[assembly: InternalsVisibleTo("Apache.Ignite.AspNet.Tests, PublicKey=0024000004800000940000000602000000240000525341310004000001000100c9380ce05eb74bd7c531f72e9ea615c59d7eceb09bd9795cb3dff1fcf638fd799c2a58a9be42fff156efe1c8cdebb751e27763f6c9a7c80cdc1dc1bbf44283608ef18ccd5017fd57b2b026503637c89c2537f361807f3bdd49265f4d444716159d989342561d324b1a0961640338bb32eaf67f4ae0c95f1b210f65404b0909c6")] #endif diff --git a/modules/platforms/dotnet/Apache.Ignite.sln b/modules/platforms/dotnet/Apache.Ignite.sln index 2362ce2cf7829..8a3bf041b221e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.sln +++ b/modules/platforms/dotnet/Apache.Ignite.sln @@ -36,6 +36,8 @@ Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.Linq", "Apach EndProject Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.AspNet", "Apache.Ignite.AspNet\Apache.Ignite.AspNet.csproj", "{13EA96FC-CC83-4164-A7C0-4F30ED797460}" EndProject +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.AspNet.Tests", "Apache.Ignite.AspNet.Tests\Apache.Ignite.AspNet.Tests.csproj", "{18EA4C71-A11D-4AB1-8042-418F7559D84F}" +EndProject Global GlobalSection(SolutionConfigurationPlatforms) = preSolution Debug|Any CPU = Debug|Any CPU @@ -174,6 +176,18 @@ Global {13EA96FC-CC83-4164-A7C0-4F30ED797460}.Release|x64.Build.0 = Release|Any CPU {13EA96FC-CC83-4164-A7C0-4F30ED797460}.Release|x86.ActiveCfg = Release|Any CPU {13EA96FC-CC83-4164-A7C0-4F30ED797460}.Release|x86.Build.0 = Release|Any CPU + {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Debug|Any CPU.Build.0 = Debug|Any CPU + {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Debug|x64.ActiveCfg = Debug|Any CPU + {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Debug|x64.Build.0 = Debug|Any CPU + {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Debug|x86.ActiveCfg = Debug|Any CPU + {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Debug|x86.Build.0 = Debug|Any CPU + {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Release|Any CPU.ActiveCfg = Release|Any CPU + {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Release|Any CPU.Build.0 = Release|Any CPU + {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Release|x64.ActiveCfg = Release|Any CPU + {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Release|x64.Build.0 = Release|Any CPU + {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Release|x86.ActiveCfg = Release|Any CPU + {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Release|x86.Build.0 = Release|Any CPU EndGlobalSection GlobalSection(SolutionProperties) = preSolution HideSolutionNode = FALSE From a3e98daf8836225ab3bdc1058c13113108cdaaaa Mon Sep 17 00:00:00 2001 From: AKuznetsov Date: Fri, 16 Sep 2016 22:29:19 +0700 Subject: [PATCH 141/487] Fixed RAT. --- parent/pom.xml | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/parent/pom.xml b/parent/pom.xml index 246c36b47478a..85336d80ac18b 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -894,13 +894,15 @@ **/*.ps1 **/*.json - **/src/main/js/.babelrc - **/src/main/js/.eslintrc - **/src/main/js/build/** - **/src/main/js/node_modules/** - **/src/main/js/ignite_modules/** - **/src/main/js/ignite_modules_temp/** - **/src/main/js/serve/config/settings.json.sample + **/.babelrc + **/.eslintrc + **/.dockerignore + **/backend/config/settings.json.sample + **/backend/node_modules/** + **/frontend/build/** + **/frontend/ignite_modules/** + **/frontend/ignite_modules_temp/** + **/frontend/node_modules/** From c0b2b4797be4f250f6f1304ff27d45c72154608a Mon Sep 17 00:00:00 2001 From: Valentin Kulichenko Date: Fri, 16 Sep 2016 14:59:35 -0700 Subject: [PATCH 142/487] IGNITE-3635 - Fixed StackOverflowError thrown from BinaryObject.toString() --- .../internal/binary/BinaryObjectExImpl.java | 161 ++++++++++++------ .../binary/BinaryObjectToStringSelfTest.java | 75 ++++++++ .../IgniteBinaryObjectsTestSuite.java | 2 + 3 files changed, 190 insertions(+), 48 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectToStringSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java index b4e909eaa4a73..e6df4074bdb02 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java @@ -20,14 +20,16 @@ import java.math.BigDecimal; import java.util.Arrays; import java.util.IdentityHashMap; +import java.util.Iterator; +import java.util.Map; import org.apache.ignite.IgniteException; +import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryObjectBuilder; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryType; import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl; import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory; import org.apache.ignite.internal.util.typedef.internal.SB; -import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.BinaryType; -import org.apache.ignite.binary.BinaryObject; import org.jetbrains.annotations.Nullable; /** @@ -164,6 +166,20 @@ public boolean equals(Object other) { } } + /** {@inheritDoc} */ + @Override public String toString() { + try { + BinaryReaderHandles ctx = new BinaryReaderHandles(); + + ctx.put(start(), this); + + return toString(ctx, new IdentityHashMap()); + } + catch (BinaryObjectException e) { + throw new IgniteException("Failed to create string representation of binary object.", e); + } + } + /** * @param ctx Reader context. * @param handles Handles for already traversed objects. @@ -197,43 +213,7 @@ private String toString(BinaryReaderHandles ctx, IdentityHashMap handles) { + if (val instanceof byte[]) + buf.a(Arrays.toString((byte[]) val)); + else if (val instanceof short[]) + buf.a(Arrays.toString((short[])val)); + else if (val instanceof int[]) + buf.a(Arrays.toString((int[])val)); + else if (val instanceof long[]) + buf.a(Arrays.toString((long[])val)); + else if (val instanceof float[]) + buf.a(Arrays.toString((float[])val)); + else if (val instanceof double[]) + buf.a(Arrays.toString((double[])val)); + else if (val instanceof char[]) + buf.a(Arrays.toString((char[])val)); + else if (val instanceof boolean[]) + buf.a(Arrays.toString((boolean[]) val)); + else if (val instanceof BigDecimal[]) + buf.a(Arrays.toString((BigDecimal[])val)); + else if (val instanceof BinaryObjectExImpl) { + BinaryObjectExImpl po = (BinaryObjectExImpl)val; + + Integer idHash0 = handles.get(val); + + if (idHash0 != null) { // Circular reference. + BinaryType meta0 = po.rawType(); + + assert meta0 != null; + + buf.a(meta0.typeName()).a(" [hash=").a(idHash0).a(", ...]"); + } + else + buf.a(po.toString(ctx, handles)); + } + else if (val instanceof Object[]) { + Object[] arr = (Object[])val; - ctx.put(start(), this); + buf.a('['); - return toString(ctx, new IdentityHashMap()); + for (int i = 0; i < arr.length; i++) { + Object o = arr[i]; + + appendValue(o, buf, ctx, handles); + + if (i < arr.length - 1) + buf.a(", "); + } } - catch (BinaryObjectException e) { - throw new IgniteException("Failed to create string representation of binary object.", e); + else if (val instanceof Iterable) { + Iterable col = (Iterable)val; + + buf.a(col.getClass().getSimpleName()).a(" {"); + + Iterator it = col.iterator(); + + while (it.hasNext()) { + Object o = it.next(); + + appendValue(o, buf, ctx, handles); + + if (it.hasNext()) + buf.a(", "); + } + + buf.a('}'); + } + else if (val instanceof Map) { + Map map = (Map)val; + + buf.a(map.getClass().getSimpleName()).a(" {"); + + Iterator> it = map.entrySet().iterator(); + + while (it.hasNext()) { + Map.Entry e = it.next(); + + appendValue(e.getKey(), buf, ctx, handles); + + buf.a('='); + + appendValue(e.getValue(), buf, ctx, handles); + + if (it.hasNext()) + buf.a(", "); + } + + buf.a('}'); } + else + buf.a(val); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectToStringSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectToStringSelfTest.java new file mode 100644 index 0000000000000..cc6cf8bd226b7 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectToStringSelfTest.java @@ -0,0 +1,75 @@ +package org.apache.ignite.internal.binary; + +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests for {@code BinaryObject.toString()}. + */ +public class BinaryObjectToStringSelfTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setMarshaller(new BinaryMarshaller()); + cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + startGrid(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + public void testToString() throws Exception { + MyObject obj = new MyObject(); + + obj.arr = new Object[] {111, "aaa", obj}; + obj.col = Arrays.asList(222, "bbb", obj); + + obj.map = new HashMap(); + + obj.map.put(10, 333); + obj.map.put(20, "ccc"); + obj.map.put(30, obj); + + BinaryObject bo = grid().binary().toBinary(obj); + + // Check that toString() doesn't fail with StackOverflowError or other exceptions. + bo.toString(); + } + + /** + */ + private static class MyObject { + /** Object array. */ + private Object[] arr; + + /** Collection. */ + private Collection col; + + /** Map. */ + private Map map; + } +} 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 dc0540df5962d..c1d9974837fb3 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 @@ -30,6 +30,7 @@ import org.apache.ignite.internal.binary.BinaryObjectBuilderAdditionalSelfTest; import org.apache.ignite.internal.binary.BinaryObjectBuilderDefaultMappersSelfTest; import org.apache.ignite.internal.binary.BinaryObjectBuilderSimpleNameLowerCaseMappersSelfTest; +import org.apache.ignite.internal.binary.BinaryObjectToStringSelfTest; import org.apache.ignite.internal.binary.BinarySimpleNameTestPropertySelfTest; import org.apache.ignite.internal.binary.BinaryTreeSelfTest; import org.apache.ignite.internal.binary.GridBinaryAffinityKeySelfTest; @@ -102,6 +103,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridSimpleLowerCaseBinaryMappersBinaryMetaDataSelfTest.class); suite.addTestSuite(GridBinaryAffinityKeySelfTest.class); suite.addTestSuite(GridBinaryWildcardsSelfTest.class); + suite.addTestSuite(BinaryObjectToStringSelfTest.class); // Tests for objects with non-compact footers. suite.addTestSuite(BinaryMarshallerNonCompactSelfTest.class); From 147ab9c08f6ac7edecf656b23d8b25bfab91becf Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 19 Sep 2016 13:58:41 +0300 Subject: [PATCH 143/487] IGNITE-3406 - Interceptor and continuous query get correct old value during rebalancing. --- .../processors/cache/GridCacheEntryEx.java | 8 ++ .../processors/cache/GridCacheMapEntry.java | 9 +- .../GridDistributedTxRemoteAdapter.java | 6 ++ .../dht/GridDhtTxPrepareFuture.java | 36 ++++++- .../cache/transactions/IgniteTxEntry.java | 44 ++++++++- .../transactions/IgniteTxLocalAdapter.java | 8 ++ .../cache/GridCacheTestEntryEx.java | 4 + .../IgniteCacheInterceptorSelfTestSuite.java | 5 + ...ntinuousQueryFailoverAbstractSelfTest.java | 99 +++++++++++++++++++ 9 files changed, 213 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java index 616854fc66138..ef6a244fa77d0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java @@ -358,6 +358,8 @@ public boolean evictInternal(boolean swap, GridCacheVersion obsoleteVer, * @param evt Flag to signal event notification. * @param metrics Flag to signal metrics update. * @param keepBinary Keep binary flag. + * @param oldValPresent {@code True} if oldValue present. + * @param oldVal Old value. * @param topVer Topology version. * @param filter Filter. * @param drType DR type. @@ -383,6 +385,8 @@ public GridCacheUpdateTxResult innerSet( boolean evt, boolean metrics, boolean keepBinary, + boolean oldValPresent, + @Nullable CacheObject oldVal, AffinityTopologyVersion topVer, CacheEntryPredicate[] filter, GridDrType drType, @@ -402,6 +406,8 @@ public GridCacheUpdateTxResult innerSet( * @param evt Flag to signal event notification. * @param metrics Flag to signal metrics notification. * @param keepBinary Keep binary flag. + * @param oldValPresent {@code True} if oldValue present. + * @param oldVal Old value. * @param topVer Topology version. * @param filter Filter. * @param drType DR type. @@ -422,6 +428,8 @@ public GridCacheUpdateTxResult innerRemove( boolean evt, boolean metrics, boolean keepBinary, + boolean oldValPresent, + @Nullable CacheObject oldVal, AffinityTopologyVersion topVer, CacheEntryPredicate[] filter, GridDrType drType, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index c760ac17f5639..a9ac1e7191c4f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -1141,6 +1141,8 @@ protected void recordNodeId(UUID nodeId, AffinityTopologyVersion topVer) { boolean evt, boolean metrics, boolean keepBinary, + boolean oldValPresent, + @Nullable CacheObject oldVal, AffinityTopologyVersion topVer, CacheEntryPredicate[] filter, GridDrType drType, @@ -1198,7 +1200,8 @@ protected void recordNodeId(UUID nodeId, AffinityTopologyVersion topVer) { Map lsnrCol = notifyContinuousQueries(tx) ? cctx.continuousQueries().updateListeners(internal, false) : null; - old = (retval || intercept || lsnrCol != null) ? + old = oldValPresent ? oldVal : + (retval || intercept || lsnrCol != null) ? rawGetOrUnmarshalUnlocked(!retval && !isOffHeapValuesOnly()) : this.val; if (intercept) { @@ -1333,6 +1336,8 @@ protected Object keyValue(boolean cpy) { boolean evt, boolean metrics, boolean keepBinary, + boolean oldValPresent, + @Nullable CacheObject oldVal, AffinityTopologyVersion topVer, CacheEntryPredicate[] filter, GridDrType drType, @@ -1403,7 +1408,7 @@ protected Object keyValue(boolean cpy) { Map lsnrCol = notifyContinuousQueries(tx) ? cctx.continuousQueries().updateListeners(internal, false) : null; - old = (retval || intercept || lsnrCol != null) ? + old = oldValPresent ? oldVal : (retval || intercept || lsnrCol != null) ? rawGetOrUnmarshalUnlocked(!retval && !isOffHeapValuesOnly()) : val; if (intercept) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index c56d1f7720cb0..9d9862a392dd9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -542,6 +542,8 @@ else if (conflictCtx.isMerge()) { true, true, txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), topVer, null, replicate ? DR_BACKUP : DR_NONE, @@ -561,6 +563,8 @@ else if (conflictCtx.isMerge()) { true, true, txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), topVer, null, replicate ? DR_BACKUP : DR_NONE, @@ -592,6 +596,8 @@ else if (op == DELETE) { true, true, txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), topVer, null, replicate ? DR_BACKUP : DR_NONE, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java index 1bdd9b8ab7b3e..ec73bff9e564c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java @@ -360,7 +360,12 @@ private void onEntriesLocked() { boolean hasFilters = !F.isEmptyOrNulls(txEntry.filters()) && !F.isAlwaysTrue(txEntry.filters()); - if (hasFilters || retVal || txEntry.op() == DELETE || txEntry.op() == TRANSFORM) { + CacheObject val; + CacheObject oldVal = null; + + boolean readOld = hasFilters || retVal || txEntry.op() == DELETE || txEntry.op() == TRANSFORM; + + if (readOld) { cached.unswap(retVal); boolean readThrough = !txEntry.skipStore() && @@ -375,7 +380,7 @@ private void onEntriesLocked() { final boolean keepBinary = txEntry.keepBinary(); - CacheObject val = cached.innerGet( + val = oldVal = cached.innerGet( null, tx, /*swap*/true, @@ -470,6 +475,33 @@ else if (retVal) else ret.success(txEntry.op() != DELETE || cached.hasValue()); } + + // Send old value in case if rebalancing is not finished. + final boolean sndOldVal = !cacheCtx.isLocal() && !cacheCtx.topology().rebalanceFinished(tx.topologyVersion()); + + if (sndOldVal) { + if (oldVal == null && !readOld) { + oldVal = cached.innerGet( + null, + tx, + /*swap*/true, + /*readThrough*/false, + /*metrics*/false, + /*event*/false, + /*tmp*/false, + /*subjectId*/tx.subjectId(), + /*transformClo*/null, + /*taskName*/null, + /*expiryPlc*/null, + /*keepBinary*/true); + } + + if (oldVal != null) { + oldVal.prepareMarshal(cacheCtx.cacheObjectContext()); + + txEntry.oldValue(oldVal, true); + } + } } catch (IgniteCheckedException e) { U.error(log, "Failed to get result value for cache entry: " + cached, e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java index 87b2525cb2dfa..194208e7dfca9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java @@ -115,6 +115,10 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message { @GridDirectTransient private TxEntryValueHolder prevVal = new TxEntryValueHolder(); + /** Old value before update. */ + @GridToStringInclude + private TxEntryValueHolder oldVal = new TxEntryValueHolder(); + /** Transform. */ @GridToStringInclude @GridDirectTransient @@ -497,7 +501,7 @@ public boolean skipStore() { } /** - * @param oldValOnPrimary {@code True} If old value for 'invoke' operation was non null on primary node. + * @param oldValOnPrimary {@code True} If old value for was non null on primary node. */ public void oldValueOnPrimary(boolean oldValOnPrimary) { setFlag(oldValOnPrimary, OLD_VAL_ON_PRIMARY); @@ -582,6 +586,30 @@ public void cached(GridCacheEntryEx entry) { return val.value(); } + /** + * @return Old value. + */ + @Nullable public CacheObject oldValue() { + return oldVal != null ? oldVal.value() : null; + } + + /** + * @param oldVal Old value. + */ + public void oldValue(CacheObject oldVal, boolean hasOldVal) { + if (this.oldVal == null) + this.oldVal = new TxEntryValueHolder(); + + this.oldVal.value(op(), oldVal, hasOldVal, hasOldVal); + } + + /** + * @return {@code True} if old value present. + */ + public boolean hasOldValue() { + return oldVal != null && oldVal.hasValue(); + } + /** * @return {@code True} if has value explicitly set. */ @@ -1069,6 +1097,11 @@ public void clearEntryReadVersion() { writer.incrementState(); + case 13: + if (!writer.writeMessage("oldVal", oldVal)) + return false; + + writer.incrementState(); } return true; @@ -1186,6 +1219,13 @@ public void clearEntryReadVersion() { reader.incrementState(); + case 13: + oldVal = reader.readMessage("oldVal"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); } return reader.afterMessageRead(IgniteTxEntry.class); @@ -1198,7 +1238,7 @@ public void clearEntryReadVersion() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 13; + return 14; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index ee992cca2e2ba..637f3229b4603 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -809,6 +809,8 @@ else if (conflictCtx.isUseNew()) { evt, metrics, txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), topVer, null, cached.detached() ? DR_NONE : drType, @@ -834,6 +836,8 @@ else if (conflictCtx.isUseNew()) { false, metrics, txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), topVer, CU.empty0(), DR_NONE, @@ -854,6 +858,8 @@ else if (op == DELETE) { evt, metrics, txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), topVer, null, cached.detached() ? DR_NONE : drType, @@ -875,6 +881,8 @@ else if (op == DELETE) { false, metrics, txEntry.keepBinary(), + txEntry.hasOldValue(), + txEntry.oldValue(), topVer, CU.empty0(), DR_NONE, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java index 400fb14a5be05..bf543cb24ec6f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java @@ -477,6 +477,8 @@ public void recheckLock() { boolean evt, boolean metrics, boolean keepBinary, + boolean hasOldVal, + @Nullable CacheObject oldVal, AffinityTopologyVersion topVer, CacheEntryPredicate[] filter, GridDrType drType, @@ -556,6 +558,8 @@ public void recheckLock() { boolean evt, boolean metrics, boolean keepBinary, + boolean oldValPresent, + @Nullable CacheObject oldVal, AffinityTopologyVersion topVer, CacheEntryPredicate[] filter, GridDrType drType, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInterceptorSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInterceptorSelfTestSuite.java index d19ecd708e289..17d88ae4868bc 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInterceptorSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInterceptorSelfTestSuite.java @@ -58,6 +58,11 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheInterceptorPartitionCounterRandomOperationsTest.class); suite.addTestSuite(CacheInterceptorPartitionCounterLocalSanityTest.class); + suite.addTestSuite(GridCacheInterceptorAtomicRebalanceTest.class); + suite.addTestSuite(GridCacheInterceptorTransactionalRebalanceTest.class); + suite.addTestSuite(GridCacheInterceptorAtomicOffheapRebalanceTest.class); + suite.addTestSuite(GridCacheInterceptorTransactionalOffheapRebalanceTest.class); + return suite; } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java index 083367cff7c4d..1376be14e726e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java @@ -57,6 +57,8 @@ import org.apache.ignite.cache.CacheEntryProcessor; import org.apache.ignite.cache.CacheMemoryMode; import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheRebalanceMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cache.affinity.Affinity; import org.apache.ignite.cache.query.ContinuousQuery; import org.apache.ignite.cache.query.QueryCursor; @@ -65,6 +67,7 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.managers.communication.GridIoMessage; @@ -311,6 +314,102 @@ public void testRebalanceVersion() throws Exception { waitRebalanceFinished(ignite3, 5, 0); } + /** + * Test that during rebalancing correct old value passed to continuous query. + * + * @throws Exception If fail. + */ + public void testRebalance() throws Exception { + for (int iter = 0; iter < 5; iter++) { + log.info("Iteration: " + iter); + + final IgniteEx ignite = startGrid(1); + + final CacheConfiguration ccfg = new CacheConfiguration<>("testCache"); + + ccfg.setAtomicityMode(atomicityMode()); + ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + ccfg.setCacheMode(cacheMode()); + ccfg.setRebalanceMode(CacheRebalanceMode.SYNC); + ccfg.setBackups(2); + + final IgniteCache cache = ignite.getOrCreateCache(ccfg); + + final int KEYS = 10_000; + + for (int i = 0; i < KEYS; i++) + cache.put(i, i); + + final ContinuousQuery qry = new ContinuousQuery<>(); + + final AtomicBoolean err = new AtomicBoolean(); + + final AtomicInteger cntr = new AtomicInteger(); + + qry.setLocalListener(new CacheEntryUpdatedListener() { + @Override public void onUpdated( + final Iterable> cacheEntryEvts) { + try { + for (final CacheEntryEvent evt : cacheEntryEvts) { + final Integer oldVal = evt.getOldValue(); + + final Integer val = evt.getValue(); + + assertNotNull("No old value: " + evt, oldVal); + assertEquals("Unexpected old value: " + evt, (Integer)(oldVal + 1), val); + + cntr.incrementAndGet(); + } + } + catch (Throwable e) { + err.set(true); + + error("Unexpected error: " + e, e); + } + } + }); + + final QueryCursor> cur = cache.query(qry); + + final CountDownLatch latch = new CountDownLatch(1); + + final IgniteInternalFuture updFut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + latch.await(); + + for (int i = 0; i < KEYS && !err.get(); i++) + cache.put(i, i + 1); + + return null; + } + }); + + final IgniteInternalFuture rebFut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + latch.await(); + + for (int i = 2; i <= 5 && !err.get(); i++) + startGrid(i); + + return null; + } + }); + + latch.countDown(); + + updFut.get(); + rebFut.get(); + + assertFalse("Unexpected error during test", err.get()); + + assertTrue(cntr.get() > 0); + + cur.close(); + + stopAllGrids(); + } + } + /** * @param ignite Ignite. * @param topVer Topology version. From c3eff6b6c7817f83f07afcff8784ec6aa9473876 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 19 Sep 2016 16:18:14 +0300 Subject: [PATCH 144/487] IGNITE-3406 - Fix incorrect patch. --- ...AbstractCacheInterceptorRebalanceTest.java | 356 ++++++++++++++++++ ...InterceptorAtomicOffheapRebalanceTest.java | 30 ++ ...idCacheInterceptorAtomicRebalanceTest.java | 36 ++ ...ptorTransactionalOffheapRebalanceTest.java | 35 ++ ...InterceptorTransactionalRebalanceTest.java | 36 ++ 5 files changed, 493 insertions(+) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicOffheapRebalanceTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicRebalanceTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalOffheapRebalanceTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalRebalanceTest.java diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java new file mode 100644 index 0000000000000..9405a190e58f9 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java @@ -0,0 +1,356 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import javax.cache.Cache; +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; + +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheInterceptor; +import org.apache.ignite.cache.CacheInterceptorAdapter; +import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.cache.CacheRebalanceMode.SYNC; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * + */ +public abstract class GridAbstractCacheInterceptorRebalanceTest extends GridCommonAbstractTest { + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final String CACHE_NAME = "test_cache"; + + /** */ + private static final int CNT = 10_000; + + /** */ + private static final int TEST_ITERATIONS = 5; + + /** */ + private static final int NODES = 5; + + /** */ + private static volatile boolean failed; + + /** */ + private static CacheInterceptor interceptor; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception { + final IgniteConfiguration cfg = super.getConfiguration(gridName); + + final CacheConfiguration ccfg = new CacheConfiguration<>(CACHE_NAME); + + ccfg.setInterceptor(interceptor); + ccfg.setAtomicityMode(atomicityMode()); + ccfg.setWriteSynchronizationMode(FULL_SYNC); + ccfg.setRebalanceMode(SYNC); + ccfg.setBackups(2); + + cfg.setCacheConfiguration(ccfg); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + + return cfg; + } + + /** + * @return Cache atomicity mode. + */ + protected abstract CacheAtomicityMode atomicityMode(); + + /** + * @return Cache memory mode; + */ + protected abstract CacheMemoryMode memoryMode(); + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + + /** + * @throws Exception If fail. + */ + public void testRebalanceUpdate() throws Exception { + interceptor = new RebalanceUpdateInterceptor(); + + testRebalance(new Operation() { + @Override public void run(final IgniteCache cache, final Integer key, final Integer val) { + cache.put(key, val); + } + }); + } + + /** + * @throws Exception If fail. + */ + public void testRebalanceUpdateInvoke() throws Exception { + interceptor = new RebalanceUpdateInterceptor(); + + final UpdateEntryProcessor proc = new UpdateEntryProcessor(); + + testRebalance(new Operation() { + @Override public void run(final IgniteCache cache, final Integer key, final Integer val) { + cache.invoke(key, proc, val); + } + }); + } + + /** + * @throws Exception If fail. + */ + public void testRebalanceRemoveInvoke() throws Exception { + interceptor = new RebalanceUpdateInterceptor(); + + final RemoveEntryProcessor proc = new RemoveEntryProcessor(); + + testRebalance(new Operation() { + @Override public void run(final IgniteCache cache, final Integer key, final Integer val) { + cache.invoke(key, proc, val); + } + }); + } + + /** + * @throws Exception If fail. + */ + public void testRebalanceRemove() throws Exception { + interceptor = new RebalanceRemoveInterceptor(); + + testRebalance(new Operation() { + @Override public void run(final IgniteCache cache, final Integer key, final Integer val) { + cache.remove(key); + } + }); + } + + /** + * @throws Exception If fail. + */ + public void testPutIfAbsent() throws Exception { + interceptor = new RebalanceUpdateInterceptor(); + + testRebalance(new Operation() { + @Override public void run(final IgniteCache cache, final Integer key, final Integer val) { + cache.putIfAbsent(key, val); + } + }); + } + + /** + * @throws Exception If fail. + */ + public void testGetAndPut() throws Exception { + interceptor = new RebalanceUpdateInterceptor(); + + testRebalance(new Operation() { + @Override public void run(final IgniteCache cache, final Integer key, final Integer val) { + final Integer old = cache.getAndPut(key, val); + + assert val == old + 1 : "Unexpected old value: " + old; + } + }); + } + + /** + * @param operation Operation to be tested. + * @throws Exception If fail. + */ + private void testRebalance(final Operation operation) throws Exception { + interceptor = new RebalanceUpdateInterceptor(); + + for (int iter = 0; iter < TEST_ITERATIONS; iter++) { + log.info("Iteration: " + iter); + + failed = false; + + final IgniteEx ignite = startGrid(1); + + final IgniteCache cache = ignite.cache(CACHE_NAME); + + for (int i = 0; i < CNT; i++) + cache.put(i, i); + + final CountDownLatch latch = new CountDownLatch(1); + + final IgniteInternalFuture updFut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + latch.await(); + + for (int j = 1; j <= 3; j++) { + for (int i = 0; i < CNT; i++) { + if (i % 2 == 0) { + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + operation.run(cache, i, i + j); + + tx.commit(); + } + } + else + operation.run(cache, i, i + j); + } + } + + return null; + } + }); + + final IgniteInternalFuture rebFut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + latch.await(); + + for (int i = 2; i < NODES; i++) + startGrid(i); + + return null; + } + }); + + latch.countDown(); + + updFut.get(); + rebFut.get(); + + stopAllGrids(); + + assertFalse(failed); + } + } + + /** + * + */ + private interface Operation { + /** + * @param cache Cache. + * @param key Key. + * @param val Value. + */ + void run(IgniteCache cache, Integer key, Integer val); + } + + /** + * + */ + private static class UpdateEntryProcessor implements EntryProcessor { + /** {@inheritDoc} */ + @Override public Integer process(final MutableEntry entry, + final Object... arguments) throws EntryProcessorException { + entry.setValue((Integer) arguments[0]); + + return null; + } + } + + /** + * + */ + private static class RemoveEntryProcessor implements EntryProcessor { + /** {@inheritDoc} */ + @Override public Integer process(final MutableEntry entry, + final Object... arguments) throws EntryProcessorException { + entry.remove(); + + return null; + } + } + + /** + * + */ + private static class RebalanceUpdateInterceptor extends CacheInterceptorAdapter { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Nullable @Override public Integer onBeforePut(final Cache.Entry entry, final Integer newVal) { + try { + boolean first = entry.getKey().equals(newVal); + + if (first) + assertNull("Expected null old value: " + entry, entry.getValue()); + else { + Integer old = (Integer)entry.getValue(); + + assertNotNull("Null old value: " + entry, old); + assertEquals("Unexpected old value: " + entry, newVal.intValue(), old + 1); + } + } + catch (Throwable e) { + failed = true; + + System.out.println("Unexpected error: " + e); + e.printStackTrace(System.out); + } + + return newVal; + } + } + + /** + * + */ + private static class RebalanceRemoveInterceptor extends CacheInterceptorAdapter { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Nullable @Override public IgniteBiTuple onBeforeRemove( + final Cache.Entry entry) { + try { + assertNotNull("Null old value: " + entry, entry.getValue()); + assertEquals("Unexpected old value: " + entry, entry.getKey(), entry.getValue()); + } + catch (Throwable t) { + failed = true; + + System.out.println("Unexpected error: " + t); + t.printStackTrace(System.out); + } + + return new T2<>(true, null); + } + } + +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicOffheapRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicOffheapRebalanceTest.java new file mode 100644 index 0000000000000..103322fa6d9c6 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicOffheapRebalanceTest.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; + +import org.apache.ignite.cache.CacheMemoryMode; + +/** + * + */ +public class GridCacheInterceptorAtomicOffheapRebalanceTest extends GridCacheInterceptorAtomicRebalanceTest { + /** {@inheritDoc} */ + @Override protected CacheMemoryMode memoryMode() { + return CacheMemoryMode.OFFHEAP_TIERED; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicRebalanceTest.java new file mode 100644 index 0000000000000..aaeda4b0a9c13 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAtomicRebalanceTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMemoryMode; + +/** + * + */ +public class GridCacheInterceptorAtomicRebalanceTest extends GridAbstractCacheInterceptorRebalanceTest { + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return CacheAtomicityMode.ATOMIC; + } + + /** {@inheritDoc} */ + @Override protected CacheMemoryMode memoryMode() { + return CacheMemoryMode.ONHEAP_TIERED; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalOffheapRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalOffheapRebalanceTest.java new file mode 100644 index 0000000000000..bb90062b791e0 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalOffheapRebalanceTest.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.cache.CacheMemoryMode; + +/** + * + */ +public class GridCacheInterceptorTransactionalOffheapRebalanceTest extends GridCacheInterceptorTransactionalRebalanceTest { + /** {@inheritDoc} */ + @Override protected CacheMemoryMode memoryMode() { + return CacheMemoryMode.OFFHEAP_TIERED; + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 10 * 60_000; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalRebalanceTest.java new file mode 100644 index 0000000000000..bace87cf92d7f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorTransactionalRebalanceTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMemoryMode; + +/** + * + */ +public class GridCacheInterceptorTransactionalRebalanceTest extends GridAbstractCacheInterceptorRebalanceTest { + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return CacheAtomicityMode.TRANSACTIONAL; + } + + /** {@inheritDoc} */ + @Override protected CacheMemoryMode memoryMode() { + return CacheMemoryMode.ONHEAP_TIERED; + } +} From 9b72d18dd94ec1383653f00474c102804c02790a Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Mon, 19 Sep 2016 18:07:20 +0300 Subject: [PATCH 145/487] IGNITE-1525 Return value for cache operation can be lost with onePhaseCommit --- .../apache/ignite/IgniteSystemProperties.java | 12 + .../communication/GridIoMessageFactory.java | 6 + .../GridCacheReturnCompletableWrapper.java | 101 ++++++++ .../cache/GridDeferredAckMessageSender.java | 219 +++++++++++++++++ .../GridDistributedTxRemoteAdapter.java | 59 +++-- .../dht/GridDhtTxFinishFuture.java | 12 +- .../dht/GridDhtTxFinishRequest.java | 33 ++- .../dht/GridDhtTxFinishResponse.java | 52 +++- .../GridDhtTxOnePhaseCommitAckRequest.java | 134 +++++++++++ .../dht/GridDhtTxPrepareFuture.java | 6 +- .../dht/GridDhtTxPrepareRequest.java | 93 +++++-- .../distributed/dht/GridDhtTxRemote.java | 6 +- .../dht/atomic/GridDhtAtomicCache.java | 227 ++++-------------- ...OptimisticSerializableTxPrepareFuture.java | 4 +- .../GridNearOptimisticTxPrepareFuture.java | 7 +- .../GridNearPessimisticTxPrepareFuture.java | 4 +- .../near/GridNearTxFinishFuture.java | 112 +++++++-- .../cache/transactions/IgniteTxAdapter.java | 46 +++- .../cache/transactions/IgniteTxHandler.java | 163 ++++++++++--- .../transactions/IgniteTxLocalAdapter.java | 19 +- .../cache/transactions/IgniteTxManager.java | 154 +++++++++++- .../IgniteCachePutRetryAbstractSelfTest.java | 39 ++- ...iteCachePutRetryTransactionalSelfTest.java | 75 +++++- .../config/benchmark-client-mode.properties | 2 + .../config/benchmark-tx-win.properties | 2 + .../yardstick/config/benchmark-tx.properties | 2 + .../yardstick/config/benchmark-win.properties | 2 + modules/yardstick/config/benchmark.properties | 2 + .../cache/IgniteGetAndPutBenchmark.java | 41 ++++ .../cache/IgniteGetAndPutTxBenchmark.java | 70 ++++++ .../cache/IgniteInvokeTxBenchmark.java | 40 +++ 31 files changed, 1405 insertions(+), 339 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturnCompletableWrapper.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxOnePhaseCommitAckRequest.java create mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutBenchmark.java create mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutTxBenchmark.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 7c428a6c6e8ac..ab6403f1e47a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -289,6 +289,18 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_ATOMIC_DEFERRED_ACK_TIMEOUT = "IGNITE_ATOMIC_DEFERRED_ACK_TIMEOUT"; + /** + * One phase commit deferred ack request timeout. + */ + public static final String IGNITE_DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_TIMEOUT = + "IGNITE_DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_TIMEOUT"; + + /** + * One phase commit deferred ack request buffer size. + */ + public static final String IGNITE_DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_BUFFER_SIZE = + "IGNITE_DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_BUFFER_SIZE"; + /** * If this property set then debug console will be opened for H2 indexing SPI. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java index 5f60215a16222..8b8a7345135b5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java @@ -62,6 +62,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxOnePhaseCommitAckRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnlockRequest; @@ -160,6 +161,11 @@ public GridIoMessageFactory(MessageFactory[] ext) { Message msg = null; switch (type) { + case -27: + msg = new GridDhtTxOnePhaseCommitAckRequest(); + + break; + case -26: msg = new TxLockList(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturnCompletableWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturnCompletableWrapper.java new file mode 100644 index 0000000000000..8ceaf715f9f0d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturnCompletableWrapper.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.jetbrains.annotations.Nullable; + +/** + * Provides initialized GridCacheReturn. + */ +public class GridCacheReturnCompletableWrapper { + /** Completable wrapper upd. */ + private static final AtomicReferenceFieldUpdater COMPLETABLE_WRAPPER_UPD = + AtomicReferenceFieldUpdater.newUpdater(GridCacheReturnCompletableWrapper.class, Object.class, "o"); + + /** */ + private volatile Object o; + + /** Node id. */ + private final UUID nodeId; + + /** + * @param nodeId Node id. + */ + public GridCacheReturnCompletableWrapper(UUID nodeId) { + this.nodeId = nodeId; + } + + /** + * @return ID of node initiated tx or {@code null} if this node is local. + */ + @Nullable public UUID nodeId() { + return nodeId; + } + + /** + * Marks as initialized. + * + * @param ret Return. + */ + public void initialize(GridCacheReturn ret) { + final Object obj = this.o; + + if (obj == null) { + boolean res = COMPLETABLE_WRAPPER_UPD.compareAndSet(this, null, ret); + + if (!res) + initialize(ret); + } + else if (obj instanceof GridFutureAdapter) { + ((GridFutureAdapter)obj).onDone(ret); + + boolean res = COMPLETABLE_WRAPPER_UPD.compareAndSet(this, obj, ret); + + assert res; + } + else + throw new IllegalStateException("GridCacheReturnCompletableWrapper can't be reinitialized"); + } + + /** + * Allows wait for properly initialized value. + */ + public IgniteInternalFuture fut() { + final Object obj = this.o; + + if (obj instanceof GridCacheReturn) + return new GridFinishedFuture<>((GridCacheReturn)obj); + else if (obj instanceof IgniteInternalFuture) + return (IgniteInternalFuture)obj; + else if (obj == null) { + boolean res = COMPLETABLE_WRAPPER_UPD.compareAndSet(this, null, new GridFutureAdapter<>()); + + if (res) + return (IgniteInternalFuture)this.o; + else + return fut(); + } + else + throw new IllegalStateException(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java new file mode 100644 index 0000000000000..7145dc2cd7fee --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDeferredAckMessageSender.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.UUID; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.closure.GridClosureProcessor; +import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; +import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.jsr166.ConcurrentHashMap8; +import org.jsr166.ConcurrentLinkedDeque8; + +/** + * + */ +public abstract class GridDeferredAckMessageSender { + /** Deferred message buffers. */ + private ConcurrentMap deferredAckMsgBuffers = new ConcurrentHashMap8<>(); + + /** Timeout processor. */ + private GridTimeoutProcessor time; + + /** Closure processor. */ + public GridClosureProcessor closure; + + /** + * @param time Time. + * @param closure Closure. + */ + public GridDeferredAckMessageSender(GridTimeoutProcessor time, + GridClosureProcessor closure) { + this.time = time; + this.closure = closure; + } + + /** + * + */ + public abstract int getTimeout(); + + /** + * + */ + public abstract int getBufferSize(); + + /** + * + */ + public abstract void finish(UUID nodeId, ConcurrentLinkedDeque8 vers); + + /** + * + */ + public void stop() { + for (DeferredAckMessageBuffer buf : deferredAckMsgBuffers.values()) + buf.finish0(); + } + + /** + * @param nodeId Node ID to send message to. + * @param ver Version to ack. + */ + public void sendDeferredAckMessage(UUID nodeId, GridCacheVersion ver) { + while (true) { + DeferredAckMessageBuffer buf = deferredAckMsgBuffers.get(nodeId); + + if (buf == null) { + buf = new DeferredAckMessageBuffer(nodeId); + + DeferredAckMessageBuffer old = deferredAckMsgBuffers.putIfAbsent(nodeId, buf); + + if (old == null) { + // We have successfully added buffer to map. + time.addTimeoutObject(buf); + } + else + buf = old; + } + + if (!buf.add(ver)) + // Some thread is sending filled up buffer, we can remove it. + deferredAckMsgBuffers.remove(nodeId, buf); + else + break; + } + } + + /** + * Deferred message buffer. + */ + private class DeferredAckMessageBuffer extends ReentrantReadWriteLock implements GridTimeoutObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Filled atomic flag. */ + private AtomicBoolean guard = new AtomicBoolean(false); + + /** Versions. */ + private ConcurrentLinkedDeque8 vers = new ConcurrentLinkedDeque8<>(); + + /** Node ID. */ + private final UUID nodeId; + + /** Timeout ID. */ + private final IgniteUuid timeoutId; + + /** End time. */ + private final long endTime; + + /** + * @param nodeId Node ID to send message to. + */ + private DeferredAckMessageBuffer(UUID nodeId) { + this.nodeId = nodeId; + + timeoutId = IgniteUuid.fromUuid(nodeId); + + endTime = U.currentTimeMillis() + getTimeout(); + } + + /** {@inheritDoc} */ + @Override public IgniteUuid timeoutId() { + return timeoutId; + } + + /** {@inheritDoc} */ + @Override public long endTime() { + return endTime; + } + + /** {@inheritDoc} */ + @Override public void onTimeout() { + if (guard.compareAndSet(false, true)) { + closure.runLocalSafe(new Runnable() { + @Override public void run() { + writeLock().lock(); + + try { + finish0(); + } + finally { + writeLock().unlock(); + } + } + }); + } + } + + /** + * Adds deferred request to buffer. + * + * @param ver Version to send. + * @return {@code True} if request was handled, {@code false} if this buffer is filled and cannot be used. + */ + public boolean add(GridCacheVersion ver) { + readLock().lock(); + + boolean snd = false; + + try { + if (guard.get()) + return false; + + vers.add(ver); + + if (vers.sizex() > getBufferSize() && guard.compareAndSet(false, true)) + snd = true; + } + finally { + readLock().unlock(); + } + + if (snd) { + // Wait all threads in read lock to finish. + writeLock().lock(); + + try { + finish0(); + + time.removeTimeoutObject(this); + } + finally { + writeLock().unlock(); + } + } + + return true; + } + + /** + * Sends deferred notification message and removes this buffer from pending responses map. + */ + private void finish0() { + finish(nodeId, vers); + + deferredAckMsgBuffers.remove(nodeId, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java index 9d9862a392dd9..4adfa8bf04f52 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java @@ -36,6 +36,8 @@ import org.apache.ignite.internal.processors.cache.GridCacheFilterFailedException; import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.GridCacheOperation; +import org.apache.ignite.internal.processors.cache.GridCacheReturn; +import org.apache.ignite.internal.processors.cache.GridCacheReturnCompletableWrapper; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry; @@ -448,7 +450,25 @@ private void commitIfLocked() throws IgniteCheckedException { Map writeMap = txState.writeMap(); + GridCacheReturnCompletableWrapper wrapper = null; + if (!F.isEmpty(writeMap)) { + GridCacheReturn ret = null; + + if (!near() && !local() && onePhaseCommit()) { + if (needReturnValue()) { + ret = new GridCacheReturn(null, cctx.localNodeId().equals(otherNodeId()), true, null, true); + + UUID origNodeId = otherNodeId(); // Originating node. + + cctx.tm().addCommittedTxReturn(this, + wrapper = new GridCacheReturnCompletableWrapper( + !cctx.localNodeId().equals(origNodeId) ? origNodeId : null)); + } + else + cctx.tm().addCommittedTx(this, this.nearXidVersion(), null); + } + // Register this transaction as completed prior to write-phase to // ensure proper lock ordering for removed entries. cctx.tm().addCommittedTx(this); @@ -457,13 +477,13 @@ private void commitIfLocked() throws IgniteCheckedException { batchStoreCommit(writeMap().values()); - // Node that for near transactions we grab all entries. - for (IgniteTxEntry txEntry : (near() ? allEntries() : writeEntries())) { - GridCacheContext cacheCtx = txEntry.context(); + try { + // Node that for near transactions we grab all entries. + for (IgniteTxEntry txEntry : (near() ? allEntries() : writeEntries())) { + GridCacheContext cacheCtx = txEntry.context(); - boolean replicate = cacheCtx.isDrEnabled(); + boolean replicate = cacheCtx.isDrEnabled(); - try { while (true) { try { GridCacheEntryEx cached = txEntry.cached(); @@ -486,7 +506,7 @@ private void commitIfLocked() throws IgniteCheckedException { txEntry.cached().unswap(false); IgniteBiTuple res = - applyTransformClosures(txEntry, false); + applyTransformClosures(txEntry, false, ret); GridCacheOperation op = res.get1(); CacheObject val = res.get2(); @@ -672,21 +692,26 @@ else if (op == READ) { } } } - catch (Throwable ex) { - // In case of error, we still make the best effort to commit, - // as there is no way to rollback at this point. - err = new IgniteTxHeuristicCheckedException("Commit produced a runtime exception " + - "(all transaction entries will be invalidated): " + CU.txString(this), ex); + } + catch (Throwable ex) { + // In case of error, we still make the best effort to commit, + // as there is no way to rollback at this point. + err = new IgniteTxHeuristicCheckedException("Commit produced a runtime exception " + + "(all transaction entries will be invalidated): " + CU.txString(this), ex); - U.error(log, "Commit failed.", err); + U.error(log, "Commit failed.", err); - uncommit(); + uncommit(); - state(UNKNOWN); + state(UNKNOWN); - if (ex instanceof Error) - throw (Error)ex; - } + if (ex instanceof Error) + throw (Error)ex; + + } + finally { + if (wrapper != null) + wrapper.initialize(ret); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java index d2e26b4415f10..ac2ab41049fe7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java @@ -351,7 +351,9 @@ private boolean rollbackLockTransactions(Collection nodes) { tx.size(), tx.subjectId(), tx.taskNameHash(), - tx.activeCachesDeploymentEnabled()); + tx.activeCachesDeploymentEnabled(), + false, + false); try { cctx.io().send(n, req, tx.ioPolicy()); @@ -448,7 +450,9 @@ private boolean finish(Map dhtMap, Map dhtMap, Map updateIdxs + Collection updateIdxs, + boolean retVal, + boolean waitRemoteTxs ) { this(nearNodeId, futId, miniId, topVer, xidVer, commitVer, threadId, isolation, commit, invalidate, sys, plc, sysInvalidate, syncCommit, syncRollback, baseVer, committedVers, rolledbackVers, pendingVers, txSize, - subjId, taskNameHash, addDepInfo); + subjId, taskNameHash, addDepInfo, retVal, waitRemoteTxs); if (updateIdxs != null && !updateIdxs.isEmpty()) { partUpdateCnt = new GridLongList(updateIdxs.size()); @@ -339,6 +349,23 @@ public void waitRemoteTransactions(boolean waitRemoteTxs) { flags &= ~WAIT_REMOTE_TX_FLAG_MASK; } + /** + * @return Flag indicating whether transaction needs return value. + */ + public boolean needReturnValue() { + return (flags & NEED_RETURN_VALUE_FLAG_MASK) != 0; + } + + /** + * @param retVal Need return value. + */ + public void needReturnValue(boolean retVal) { + if (retVal) + flags = (byte)(flags | NEED_RETURN_VALUE_FLAG_MASK); + else + flags &= ~NEED_RETURN_VALUE_FLAG_MASK; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(GridDhtTxFinishRequest.class, this, super.toString()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java index 78dc16fe69197..06181729a6c1d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java @@ -19,9 +19,10 @@ import java.io.Externalizable; import java.nio.ByteBuffer; - import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.GridDirectTransient; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheReturn; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxFinishResponse; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -51,6 +52,9 @@ public class GridDhtTxFinishResponse extends GridDistributedTxFinishResponse { /** Flag indicating if this is a check-committed response. */ private boolean checkCommitted; + /** Cache return value. */ + private GridCacheReturn retVal; + /** * Empty constructor required by {@link Externalizable}. */ @@ -112,6 +116,14 @@ public void checkCommitted(boolean checkCommitted) { if (checkCommittedErr != null && checkCommittedErrBytes == null) checkCommittedErrBytes = ctx.marshaller().marshal(checkCommittedErr); + + if (retVal != null && retVal.cacheId() != 0) { + GridCacheContext cctx = ctx.cacheContext(retVal.cacheId()); + + assert cctx != null : retVal.cacheId(); + + retVal.prepareMarshal(cctx); + } } /** {@inheritDoc} */ @@ -121,6 +133,28 @@ public void checkCommitted(boolean checkCommitted) { if (checkCommittedErrBytes != null && checkCommittedErr == null) checkCommittedErr = ctx.marshaller().unmarshal(checkCommittedErrBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + + if (retVal != null && retVal.cacheId() != 0) { + GridCacheContext cctx = ctx.cacheContext(retVal.cacheId()); + + assert cctx != null : retVal.cacheId(); + + retVal.finishUnmarshal(cctx, ldr); + } + } + + /** + * @param retVal Return value. + */ + public void returnValue(GridCacheReturn retVal) { + this.retVal = retVal; + } + + /** + * @return Return value. + */ + public GridCacheReturn returnValue() { + return retVal; } /** {@inheritDoc} */ @@ -161,6 +195,12 @@ public void checkCommitted(boolean checkCommitted) { writer.incrementState(); + case 8: + if (!writer.writeMessage("retVal", retVal)) + return false; + + writer.incrementState(); + } return true; @@ -201,6 +241,14 @@ public void checkCommitted(boolean checkCommitted) { reader.incrementState(); + case 8: + retVal = reader.readMessage("retVal"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } return reader.afterMessageRead(GridDhtTxFinishResponse.class); @@ -213,6 +261,6 @@ public void checkCommitted(boolean checkCommitted) { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 8; + return 9; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxOnePhaseCommitAckRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxOnePhaseCommitAckRequest.java new file mode 100644 index 0000000000000..0c8ae69e462c3 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxOnePhaseCommitAckRequest.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.dht; + +import java.nio.ByteBuffer; +import java.util.Collection; +import org.apache.ignite.internal.GridDirectCollection; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * One Phase Commit Near transaction ack request. + */ +public class GridDhtTxOnePhaseCommitAckRequest extends GridCacheMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** Lock or transaction versions. */ + @GridToStringInclude + @GridDirectCollection(GridCacheVersion.class) + protected Collection vers; + + /** + * Default constructor. + */ + public GridDhtTxOnePhaseCommitAckRequest() { + // No-op. + } + + /** + * + * @param vers Near Tx xid Versions. + */ + public GridDhtTxOnePhaseCommitAckRequest(Collection vers) { + this.vers = vers; + } + + /** + * @return Version. + */ + public Collection versions() { + return vers; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridDhtTxOnePhaseCommitAckRequest.class, this, super.toString()); + } + + /** {@inheritDoc} */ + @Override public boolean addDeploymentInfo() { + return addDepInfo; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 3: + if (!writer.writeCollection("vers", vers, MessageCollectionItemType.MSG)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 3: + vers = reader.readCollection("vers", MessageCollectionItemType.MSG); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridDhtTxOnePhaseCommitAckRequest.class); + } + + /** {@inheritDoc} */ + @Override public byte directType() { + return -27; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 4; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java index ec73bff9e564c..1dbda69f30dbc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java @@ -1245,7 +1245,8 @@ private void prepare0() { tx.onePhaseCommit(), tx.subjectId(), tx.taskNameHash(), - tx.activeCachesDeploymentEnabled()); + tx.activeCachesDeploymentEnabled(), + retVal); int idx = 0; @@ -1356,7 +1357,8 @@ private void prepare0() { tx.onePhaseCommit(), tx.subjectId(), tx.taskNameHash(), - tx.activeCachesDeploymentEnabled()); + tx.activeCachesDeploymentEnabled(), + retVal); for (IgniteTxEntry entry : nearMapping.entries()) { if (CU.writes().apply(entry)) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java index 1cdc96fd74c05..a8f20876abd29 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java @@ -52,6 +52,9 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest { /** */ private static final long serialVersionUID = 0L; + /** */ + public static final int NEED_RETURN_VALUE_FLAG_MASK = 0x01; + /** Max order. */ private UUID nearNodeId; @@ -100,6 +103,9 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest { /** Preload keys. */ private BitSet preloadKeys; + /** */ + private byte flags; + /** * Empty constructor required for {@link Externalizable}. */ @@ -118,6 +124,7 @@ public GridDhtTxPrepareRequest() { * @param txNodes Transaction nodes mapping. * @param nearXidVer Near transaction ID. * @param last {@code True} if this is last prepare request for node. + * @param retVal Need return value flag. * @param addDepInfo Deployment info flag. */ public GridDhtTxPrepareRequest( @@ -134,7 +141,8 @@ public GridDhtTxPrepareRequest( boolean onePhaseCommit, UUID subjId, int taskNameHash, - boolean addDepInfo) { + boolean addDepInfo, + boolean retVal) { super(tx, timeout, null, dhtWrites, txNodes, onePhaseCommit, addDepInfo); assert futId != null; @@ -149,11 +157,30 @@ public GridDhtTxPrepareRequest( this.subjId = subjId; this.taskNameHash = taskNameHash; + needReturnValue(retVal); + invalidateNearEntries = new BitSet(dhtWrites == null ? 0 : dhtWrites.size()); nearNodeId = tx.nearNodeId(); } + /** + * @return Flag indicating whether transaction needs return value. + */ + public boolean needReturnValue() { + return (flags & NEED_RETURN_VALUE_FLAG_MASK) != 0; + } + + /** + * @param retVal Need return value. + */ + public void needReturnValue(boolean retVal) { + if (retVal) + flags = (byte)(flags | NEED_RETURN_VALUE_FLAG_MASK); + else + flags &= ~NEED_RETURN_VALUE_FLAG_MASK; + } + /** * @return {@code True} if this is last prepare request for node. */ @@ -348,78 +375,84 @@ public Map owned() { switch (writer.state()) { case 23: - if (!writer.writeIgniteUuid("futId", futId)) + if (!writer.writeByte("flags", flags)) return false; writer.incrementState(); case 24: - if (!writer.writeBitSet("invalidateNearEntries", invalidateNearEntries)) + if (!writer.writeIgniteUuid("futId", futId)) return false; writer.incrementState(); case 25: - if (!writer.writeBoolean("last", last)) + if (!writer.writeBitSet("invalidateNearEntries", invalidateNearEntries)) return false; writer.incrementState(); case 26: - if (!writer.writeIgniteUuid("miniId", miniId)) + if (!writer.writeBoolean("last", last)) return false; writer.incrementState(); case 27: - if (!writer.writeUuid("nearNodeId", nearNodeId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; writer.incrementState(); case 28: - if (!writer.writeCollection("nearWrites", nearWrites, MessageCollectionItemType.MSG)) + if (!writer.writeUuid("nearNodeId", nearNodeId)) return false; writer.incrementState(); case 29: - if (!writer.writeMessage("nearXidVer", nearXidVer)) + if (!writer.writeCollection("nearWrites", nearWrites, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 30: - if (!writer.writeCollection("ownedKeys", ownedKeys, MessageCollectionItemType.MSG)) + if (!writer.writeMessage("nearXidVer", nearXidVer)) return false; writer.incrementState(); case 31: - if (!writer.writeCollection("ownedVals", ownedVals, MessageCollectionItemType.MSG)) + if (!writer.writeCollection("ownedKeys", ownedKeys, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 32: - if (!writer.writeBitSet("preloadKeys", preloadKeys)) + if (!writer.writeCollection("ownedVals", ownedVals, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 33: - if (!writer.writeUuid("subjId", subjId)) + if (!writer.writeBitSet("preloadKeys", preloadKeys)) return false; writer.incrementState(); case 34: - if (!writer.writeInt("taskNameHash", taskNameHash)) + if (!writer.writeUuid("subjId", subjId)) return false; writer.incrementState(); case 35: + if (!writer.writeInt("taskNameHash", taskNameHash)) + return false; + + writer.incrementState(); + + case 36: if (!writer.writeMessage("topVer", topVer)) return false; @@ -442,7 +475,7 @@ public Map owned() { switch (reader.state()) { case 23: - futId = reader.readIgniteUuid("futId"); + flags = reader.readByte("flags"); if (!reader.isLastRead()) return false; @@ -450,7 +483,7 @@ public Map owned() { reader.incrementState(); case 24: - invalidateNearEntries = reader.readBitSet("invalidateNearEntries"); + futId = reader.readIgniteUuid("futId"); if (!reader.isLastRead()) return false; @@ -458,7 +491,7 @@ public Map owned() { reader.incrementState(); case 25: - last = reader.readBoolean("last"); + invalidateNearEntries = reader.readBitSet("invalidateNearEntries"); if (!reader.isLastRead()) return false; @@ -466,7 +499,7 @@ public Map owned() { reader.incrementState(); case 26: - miniId = reader.readIgniteUuid("miniId"); + last = reader.readBoolean("last"); if (!reader.isLastRead()) return false; @@ -474,7 +507,7 @@ public Map owned() { reader.incrementState(); case 27: - nearNodeId = reader.readUuid("nearNodeId"); + miniId = reader.readIgniteUuid("miniId"); if (!reader.isLastRead()) return false; @@ -482,7 +515,7 @@ public Map owned() { reader.incrementState(); case 28: - nearWrites = reader.readCollection("nearWrites", MessageCollectionItemType.MSG); + nearNodeId = reader.readUuid("nearNodeId"); if (!reader.isLastRead()) return false; @@ -490,7 +523,7 @@ public Map owned() { reader.incrementState(); case 29: - nearXidVer = reader.readMessage("nearXidVer"); + nearWrites = reader.readCollection("nearWrites", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -498,7 +531,7 @@ public Map owned() { reader.incrementState(); case 30: - ownedKeys = reader.readCollection("ownedKeys", MessageCollectionItemType.MSG); + nearXidVer = reader.readMessage("nearXidVer"); if (!reader.isLastRead()) return false; @@ -506,7 +539,7 @@ public Map owned() { reader.incrementState(); case 31: - ownedVals = reader.readCollection("ownedVals", MessageCollectionItemType.MSG); + ownedKeys = reader.readCollection("ownedKeys", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -514,7 +547,7 @@ public Map owned() { reader.incrementState(); case 32: - preloadKeys = reader.readBitSet("preloadKeys"); + ownedVals = reader.readCollection("ownedVals", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -522,7 +555,7 @@ public Map owned() { reader.incrementState(); case 33: - subjId = reader.readUuid("subjId"); + preloadKeys = reader.readBitSet("preloadKeys"); if (!reader.isLastRead()) return false; @@ -530,7 +563,7 @@ public Map owned() { reader.incrementState(); case 34: - taskNameHash = reader.readInt("taskNameHash"); + subjId = reader.readUuid("subjId"); if (!reader.isLastRead()) return false; @@ -538,6 +571,14 @@ public Map owned() { reader.incrementState(); case 35: + taskNameHash = reader.readInt("taskNameHash"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 36: topVer = reader.readMessage("topVer"); if (!reader.isLastRead()) @@ -557,6 +598,6 @@ public Map owned() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 36; + return 37; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java index dc27eb147b917..6ad20c7566553 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxRemote.java @@ -189,9 +189,9 @@ public GridDhtTxRemote( commitVer, sys, plc, - concurrency, - isolation, - invalidate, + concurrency, + isolation, + invalidate, timeout, txSize, subjId, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java index 1e45fa7a2bdc9..30a3d57ef896b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java @@ -29,9 +29,6 @@ import java.util.Map; import java.util.Set; import java.util.UUID; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.locks.ReentrantReadWriteLock; import javax.cache.expiry.ExpiryPolicy; import javax.cache.processor.EntryProcessor; import javax.cache.processor.EntryProcessorResult; @@ -60,6 +57,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheOperation; import org.apache.ignite.internal.processors.cache.GridCacheReturn; import org.apache.ignite.internal.processors.cache.GridCacheUpdateAtomicResult; +import org.apache.ignite.internal.processors.cache.GridDeferredAckMessageSender; import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter; @@ -82,7 +80,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx; -import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.future.GridEmbeddedFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -102,11 +99,9 @@ import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteOutClosure; -import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.security.SecurityPermission; import org.apache.ignite.transactions.TransactionIsolation; import org.jetbrains.annotations.Nullable; -import org.jsr166.ConcurrentHashMap8; import org.jsr166.ConcurrentLinkedDeque8; import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_DEFERRED_ACK_BUFFER_SIZE; @@ -144,7 +139,7 @@ public class GridDhtAtomicCache extends GridDhtCacheAdapter { private CI2 updateReplyClos; /** Pending */ - private ConcurrentMap pendingResponses = new ConcurrentHashMap8<>(); + private GridDeferredAckMessageSender deferredUpdateMessageSender; /** */ private GridNearAtomicCache near; @@ -240,6 +235,53 @@ else if (res.error() != null) { @Override public void start() throws IgniteCheckedException { super.start(); + deferredUpdateMessageSender = new GridDeferredAckMessageSender(ctx.time(), ctx.closures()) { + @Override public int getTimeout() { + return DEFERRED_UPDATE_RESPONSE_TIMEOUT; + } + + @Override public int getBufferSize() { + return DEFERRED_UPDATE_RESPONSE_BUFFER_SIZE; + } + + @Override public void finish(UUID nodeId, ConcurrentLinkedDeque8 vers) { + GridDhtAtomicDeferredUpdateResponse msg = new GridDhtAtomicDeferredUpdateResponse(ctx.cacheId(), + vers, ctx.deploymentEnabled()); + + try { + ctx.kernalContext().gateway().readLock(); + + try { + ctx.io().send(nodeId, msg, ctx.ioPolicy()); + + if (msgLog.isDebugEnabled()) { + msgLog.debug("Sent deferred DHT update response [futIds=" + msg.futureVersions() + + ", node=" + nodeId + ']'); + } + } + finally { + ctx.kernalContext().gateway().readUnlock(); + } + } + catch (IllegalStateException ignored) { + if (msgLog.isDebugEnabled()) { + msgLog.debug("Failed to send deferred DHT update response, node is stopping [" + + "futIds=" + msg.futureVersions() + ", node=" + nodeId + ']'); + } + } + catch (ClusterTopologyCheckedException ignored) { + if (msgLog.isDebugEnabled()) { + msgLog.debug("Failed to send deferred DHT update response, node left [" + + "futIds=" + msg.futureVersions() + ", node=" + nodeId + ']'); + } + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to send deferred DHT update response to remote node [" + + "futIds=" + msg.futureVersions() + ", node=" + nodeId + ']', e); + } + } + }; + CacheMetricsImpl m = new CacheMetricsImpl(ctx); if (ctx.dht().near() != null) @@ -405,8 +447,7 @@ else if (res.error() != null) { /** {@inheritDoc} */ @Override public void stop() { - for (DeferredResponseBuffer buf : pendingResponses.values()) - buf.finish(); + deferredUpdateMessageSender.stop(); } /** @@ -3208,28 +3249,7 @@ && writeThrough() && !req.skipStore(), * @param ver Version to ack. */ private void sendDeferredUpdateResponse(UUID nodeId, GridCacheVersion ver) { - while (true) { - DeferredResponseBuffer buf = pendingResponses.get(nodeId); - - if (buf == null) { - buf = new DeferredResponseBuffer(nodeId); - - DeferredResponseBuffer old = pendingResponses.putIfAbsent(nodeId, buf); - - if (old == null) { - // We have successfully added buffer to map. - ctx.time().addTimeoutObject(buf); - } - else - buf = old; - } - - if (!buf.addResponse(ver)) - // Some thread is sending filled up buffer, we can remove it. - pendingResponses.remove(nodeId, buf); - else - break; - } + deferredUpdateMessageSender.sendDeferredAckMessage(nodeId, ver); } /** @@ -3452,149 +3472,4 @@ private FinishedLockFuture(Throwable err) { return Collections.emptyList(); } } - - /** - * Deferred response buffer. - */ - private class DeferredResponseBuffer extends ReentrantReadWriteLock implements GridTimeoutObject { - /** */ - private static final long serialVersionUID = 0L; - - /** Filled atomic flag. */ - private AtomicBoolean guard = new AtomicBoolean(false); - - /** Response versions. */ - private ConcurrentLinkedDeque8 respVers = new ConcurrentLinkedDeque8<>(); - - /** Node ID. */ - private final UUID nodeId; - - /** Timeout ID. */ - private final IgniteUuid timeoutId; - - /** End time. */ - private final long endTime; - - /** - * @param nodeId Node ID to send message to. - */ - private DeferredResponseBuffer(UUID nodeId) { - this.nodeId = nodeId; - - timeoutId = IgniteUuid.fromUuid(nodeId); - - endTime = U.currentTimeMillis() + DEFERRED_UPDATE_RESPONSE_TIMEOUT; - } - - /** {@inheritDoc} */ - @Override public IgniteUuid timeoutId() { - return timeoutId; - } - - /** {@inheritDoc} */ - @Override public long endTime() { - return endTime; - } - - /** {@inheritDoc} */ - @Override public void onTimeout() { - if (guard.compareAndSet(false, true)) { - ctx.closures().runLocalSafe(new Runnable() { - @Override public void run() { - writeLock().lock(); - - try { - finish(); - } - finally { - writeLock().unlock(); - } - } - }); - } - } - - /** - * Adds deferred response to buffer. - * - * @param ver Version to send. - * @return {@code True} if response was handled, {@code false} if this buffer is filled and cannot be used. - */ - public boolean addResponse(GridCacheVersion ver) { - readLock().lock(); - - boolean snd = false; - - try { - if (guard.get()) - return false; - - respVers.add(ver); - - if (respVers.sizex() > DEFERRED_UPDATE_RESPONSE_BUFFER_SIZE && guard.compareAndSet(false, true)) - snd = true; - } - finally { - readLock().unlock(); - } - - if (snd) { - // Wait all threads in read lock to finish. - writeLock().lock(); - - try { - finish(); - - ctx.time().removeTimeoutObject(this); - } - finally { - writeLock().unlock(); - } - } - - return true; - } - - /** - * Sends deferred notification message and removes this buffer from pending responses map. - */ - private void finish() { - GridDhtAtomicDeferredUpdateResponse msg = new GridDhtAtomicDeferredUpdateResponse(ctx.cacheId(), - respVers, ctx.deploymentEnabled()); - - try { - ctx.kernalContext().gateway().readLock(); - - try { - ctx.io().send(nodeId, msg, ctx.ioPolicy()); - - if (msgLog.isDebugEnabled()) { - msgLog.debug("Sent deferred DHT update response [futIds=" + msg.futureVersions() + - ", node=" + nodeId + ']'); - } - } - finally { - ctx.kernalContext().gateway().readUnlock(); - } - } - catch (IllegalStateException ignored) { - if (msgLog.isDebugEnabled()) { - msgLog.debug("Failed to send deferred DHT update response, node is stopping [" + - "futIds=" + msg.futureVersions() + ", node=" + nodeId + ']'); - } - } - catch (ClusterTopologyCheckedException ignored) { - if (msgLog.isDebugEnabled()) { - msgLog.debug("Failed to send deferred DHT update response, node left [" + - "futIds=" + msg.futureVersions() + ", node=" + nodeId + ']'); - } - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to send deferred DHT update response to remote node [" + - "futIds=" + msg.futureVersions() + ", node=" + nodeId + ']', e); - } - - pendingResponses.remove(nodeId, this); - } - } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java index d251528590278..4cbfb27d34857 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java @@ -526,7 +526,9 @@ private void map( ) { GridCacheContext cacheCtx = entry.context(); - List nodes = cacheCtx.affinity().nodes(entry.key(), topVer); + List nodes = cacheCtx.isLocal() ? + cacheCtx.affinity().nodes(entry.key(), topVer) : + cacheCtx.topology().nodes(cacheCtx.affinity().partition(entry.key()), topVer); txMapping.addMapping(nodes); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java index e17a76c44b731..91cfbda6e15d3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java @@ -27,7 +27,6 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; - import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.cluster.ClusterTopologyException; @@ -599,9 +598,11 @@ private GridDistributedTxMapping map( GridCacheEntryEx cached0 = entry.cached(); if (cached0.isDht()) - nodes = cacheCtx.affinity().nodes(cached0.partition(), topVer); + nodes = cacheCtx.topology().nodes(cached0.partition(), topVer); else - nodes = cacheCtx.affinity().nodes(entry.key(), topVer); + nodes = cacheCtx.isLocal() ? + cacheCtx.affinity().nodes(entry.key(), topVer) : + cacheCtx.topology().nodes(cacheCtx.affinity().partition(entry.key()), topVer); txMapping.addMapping(nodes); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java index 34b8281a2ee23..5c09398b4ecb1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java @@ -193,7 +193,9 @@ private void preparePessimistic() { GridCacheContext cacheCtx = txEntry.context(); - List nodes = cacheCtx.affinity().nodes(txEntry.key(), topVer); + List nodes = cacheCtx.isLocal() ? + cacheCtx.affinity().nodes(txEntry.key(), topVer) : + cacheCtx.topology().nodes(cacheCtx.affinity().partition(txEntry.key()), topVer); ClusterNode primary = F.first(nodes); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java index bb5d482ab211a..46604c7524527 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java @@ -34,6 +34,8 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheFuture; +import org.apache.ignite.internal.processors.cache.GridCacheReturn; +import org.apache.ignite.internal.processors.cache.GridCacheReturnCompletableWrapper; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; @@ -75,6 +77,9 @@ public final class GridNearTxFinishFuture extends GridCompoundIdentityFutu /** */ public static final IgniteProductVersion PRIMARY_SYNC_TXS_SINCE = IgniteProductVersion.fromString("1.6.0"); + /** */ + public static final IgniteProductVersion ACK_DHT_ONE_PHASE_SINCE = IgniteProductVersion.fromString("1.6.8"); + /** */ private static final long serialVersionUID = 0L; @@ -251,6 +256,9 @@ public void onResult(UUID nodeId, GridDhtTxFinishResponse res) { assert f.node().id().equals(nodeId); + if (res.returnValue() != null) + tx.implicitSingleResult(res.returnValue()); + f.onDhtFinishResponse(res); } } @@ -432,6 +440,50 @@ void finish(boolean commit) { catch (IgniteCheckedException e) { onDone(e); } + finally { + if (commit && + tx.onePhaseCommit() && + !tx.writeMap().isEmpty()) // Readonly operations require no ack. + ackBackup(); + } + } + + /** + * + */ + private void ackBackup() { + if (mappings.empty()) + return; + + if (!tx.needReturnValue() || !tx.implicit()) + return; // GridCacheReturn was not saved at backup. + + GridDistributedTxMapping mapping = mappings.singleMapping(); + + if (mapping != null) { + UUID nodeId = mapping.node().id(); + + Collection backups = tx.transactionNodes().get(nodeId); + + if (!F.isEmpty(backups)) { + assert backups.size() == 1 : backups; + + UUID backupId = F.first(backups); + + ClusterNode backup = cctx.discovery().node(backupId); + + // Nothing to do if backup has left the grid. + if (backup == null) { + // No-op. + } + else if (backup.isLocal()) + cctx.tm().removeTxReturn(tx.xidVersion()); + else { + if (ACK_DHT_ONE_PHASE_SINCE.compareToIgnoreTimestamp(backup.version()) <= 0) + cctx.tm().sendDeferredAckResponse(backupId, tx.xidVersion()); + } + } + } } /** @@ -475,23 +527,48 @@ private void checkBackup() { readyNearMappingFromBackup(mapping); if (committed) { - if (tx.syncMode() == FULL_SYNC) { - GridCacheVersion nearXidVer = tx.nearXidVersion(); + try { + if (tx.needReturnValue() && tx.implicit()) { + GridCacheReturnCompletableWrapper wrapper = + cctx.tm().getCommittedTxReturn(tx.xidVersion()); - assert nearXidVer != null : tx; + assert wrapper != null : tx.xidVersion(); - IgniteInternalFuture fut = cctx.tm().remoteTxFinishFuture(nearXidVer); + GridCacheReturn retVal = wrapper.fut().get(); - fut.listen(new CI1>() { - @Override public void apply(IgniteInternalFuture fut) { - mini.onDone(tx); - } - }); + assert retVal != null; + + tx.implicitSingleResult(retVal); + } - return; + if (tx.syncMode() == FULL_SYNC) { + GridCacheVersion nearXidVer = tx.nearXidVersion(); + + assert nearXidVer != null : tx; + + IgniteInternalFuture fut = cctx.tm().remoteTxFinishFuture(nearXidVer); + + fut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture fut) { + mini.onDone(tx); + } + }); + + return; + } + + mini.onDone(tx); } + catch (IgniteCheckedException e) { + if (msgLog.isDebugEnabled()) { + msgLog.debug("Near finish fut, failed to finish [" + + "txId=" + tx.nearXidVersion() + + ", node=" + backup.id() + + ", err=" + e + ']'); + } - mini.onDone(tx); + mini.onDone(e); + } } else { ClusterTopologyCheckedException cause = @@ -504,7 +581,7 @@ private void checkBackup() { } } else { - GridDhtTxFinishRequest finishReq = checkCommittedRequest(mini.futureId()); + GridDhtTxFinishRequest finishReq = checkCommittedRequest(mini.futureId(), false); // Preserve old behavior, otherwise response is not sent. if (WAIT_REMOTE_TXS_SINCE.compareTo(backup.version()) > 0) @@ -765,9 +842,10 @@ else if (f.getClass() == CheckRemoteTxMiniFuture.class) { /** * @param miniId Mini future ID. + * @param waitRemoteTxs Wait for remote txs. * @return Finish request. */ - private GridDhtTxFinishRequest checkCommittedRequest(IgniteUuid miniId) { + private GridDhtTxFinishRequest checkCommittedRequest(IgniteUuid miniId, boolean waitRemoteTxs) { GridDhtTxFinishRequest finishReq = new GridDhtTxFinishRequest( cctx.localNodeId(), futureId(), @@ -791,7 +869,9 @@ private GridDhtTxFinishRequest checkCommittedRequest(IgniteUuid miniId) { 0, null, 0, - tx.activeCachesDeploymentEnabled()); + tx.activeCachesDeploymentEnabled(), + !waitRemoteTxs && (tx.needReturnValue() && tx.implicit()), + waitRemoteTxs); finishReq.checkCommitted(true); @@ -872,9 +952,7 @@ boolean onNodeLeft(UUID nodeId, boolean discoThread) { add(mini); - GridDhtTxFinishRequest req = checkCommittedRequest(mini.futureId()); - - req.waitRemoteTransactions(true); + GridDhtTxFinishRequest req = checkCommittedRequest(mini.futureId(), true); for (UUID backupId : backups) { ClusterNode backup = cctx.discovery().node(backupId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java index eb2989e55a400..18c301124405f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java @@ -51,6 +51,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.GridCacheOperation; +import org.apache.ignite.internal.processors.cache.GridCacheReturn; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry; @@ -151,6 +152,9 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement @GridToStringExclude protected GridCacheSharedContext cctx; + /** Need return value. */ + protected boolean needRetVal; + /** * End version (a.k.a. 'tnc' or 'transaction number counter') * assigned to this transaction at the end of write phase. @@ -694,6 +698,20 @@ public void syncMode(CacheWriteSynchronizationMode syncMode) { return startTime; } + /** + * @return Flag indicating whether transaction needs return value. + */ + public boolean needReturnValue() { + return needRetVal; + } + + /** + * @param needRetVal Need return value flag. + */ + public void needReturnValue(boolean needRetVal) { + this.needRetVal = needRetVal; + } + /** * Gets remaining allowed transaction time. * @@ -1285,7 +1303,7 @@ protected void batchStoreCommit(Iterable writeEntries) throws Ign if (intercept || !F.isEmpty(e.entryProcessors())) e.cached().unswap(false); - IgniteBiTuple res = applyTransformClosures(e, false); + IgniteBiTuple res = applyTransformClosures(e, false, null); GridCacheContext cacheCtx = e.context(); @@ -1443,13 +1461,15 @@ else if (log.isDebugEnabled()) /** * @param txEntry Entry to process. * @param metrics {@code True} if metrics should be updated. + * @param ret Optional return value to initialize. * @return Tuple containing transformation results. * @throws IgniteCheckedException If failed to get previous value for transform. * @throws GridCacheEntryRemovedException If entry was concurrently deleted. */ protected IgniteBiTuple applyTransformClosures( IgniteTxEntry txEntry, - boolean metrics) throws GridCacheEntryRemovedException, IgniteCheckedException { + boolean metrics, + @Nullable GridCacheReturn ret) throws GridCacheEntryRemovedException, IgniteCheckedException { GridCacheContext cacheCtx = txEntry.context(); assert cacheCtx != null; @@ -1457,8 +1477,12 @@ protected IgniteBiTuple applyTransformClosures( if (isSystemInvalidate()) return F.t(cacheCtx.writeThrough() ? RELOAD : DELETE, null); - if (F.isEmpty(txEntry.entryProcessors())) + if (F.isEmpty(txEntry.entryProcessors())) { + if (ret != null) + ret.value(cacheCtx, txEntry.value(), txEntry.keepBinary()); + return F.t(txEntry.op(), txEntry.value()); + } else { T2 calcVal = txEntry.entryProcessorCalculatedValue(); @@ -1508,17 +1532,27 @@ protected IgniteBiTuple applyTransformClosures( CacheInvokeEntry invokeEntry = new CacheInvokeEntry<>( txEntry.key(), key, cacheVal, val, ver, keepBinary, txEntry.cached()); + Object procRes = null; + Exception err = null; + try { EntryProcessor processor = t.get1(); - processor.process(invokeEntry, t.get2()); + procRes = processor.process(invokeEntry, t.get2()); val = invokeEntry.getValue(); key = invokeEntry.key(); } - catch (Exception ignore) { - // No-op. + catch (Exception e) { + err = e; + } + + if (ret != null) { + if (err != null || procRes != null) + ret.addEntryProcessResult(txEntry.context(), txEntry.key(), null, procRes, err, keepBinary); + else + ret.invokeResult(true); } modified |= invokeEntry.modified(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index e67e60ff0543c..a5b2202fb983d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -32,6 +32,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheReturnCompletableWrapper; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryFuture; @@ -44,6 +45,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocal; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxOnePhaseCommitAckRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse; @@ -175,6 +177,12 @@ public IgniteTxHandler(GridCacheSharedContext ctx) { } }); + ctx.io().addHandler(0, GridDhtTxOnePhaseCommitAckRequest.class, new CI2() { + @Override public void apply(UUID nodeId, GridCacheMessage msg) { + processDhtTxOnePhaseCommitAckRequest(nodeId, (GridDhtTxOnePhaseCommitAckRequest)msg); + } + }); + ctx.io().addHandler(0, GridDhtTxFinishResponse.class, new CI2() { @Override public void apply(UUID nodeId, GridCacheMessage msg) { processDhtTxFinishResponse(nodeId, (GridDhtTxFinishResponse)msg); @@ -882,7 +890,7 @@ public IgniteInternalFuture finishColocatedLocal(boolean commi * @param nodeId Sender node ID. * @param req Request. */ - protected final void processDhtTxPrepareRequest(UUID nodeId, GridDhtTxPrepareRequest req) { + protected final void processDhtTxPrepareRequest(final UUID nodeId, final GridDhtTxPrepareRequest req) { if (txPrepareMsgLog.isDebugEnabled()) { txPrepareMsgLog.debug("Received dht prepare request [txId=" + req.nearXidVersion() + ", dhtTxId=" + req.version() + @@ -918,14 +926,15 @@ protected final void processDhtTxPrepareRequest(UUID nodeId, GridDhtTxPrepareReq if (dhtTx != null) { dhtTx.onePhaseCommit(true); + dhtTx.needReturnValue(req.needReturnValue()); - finish(nodeId, dhtTx, req); + finish(dhtTx, req); } if (nearTx != null) { nearTx.onePhaseCommit(true); - finish(nodeId, nearTx, req); + finish(nearTx, req); } } } @@ -950,38 +959,60 @@ else if (e instanceof IgniteTxHeuristicCheckedException) { req.deployInfo() != null); } - try { - // Reply back to sender. - ctx.io().send(nodeId, res, req.policy()); + if (req.onePhaseCommit()) { + IgniteInternalFuture completeFut; - if (txPrepareMsgLog.isDebugEnabled()) { - txPrepareMsgLog.debug("Sent dht prepare response [txId=" + req.nearXidVersion() + - ", dhtTxId=" + req.version() + - ", node=" + nodeId + ']'); - } - } - catch (IgniteCheckedException e) { - if (e instanceof ClusterTopologyCheckedException) { - if (txPrepareMsgLog.isDebugEnabled()) { - txPrepareMsgLog.debug("Failed to send dht prepare response, node left [txId=" + req.nearXidVersion() + - ", dhtTxId=" + req.version() + - ", node=" + nodeId + ']'); - } - } - else { - U.warn(log, "Failed to send tx response to remote node (will rollback transaction) [" + - "txId=" + req.nearXidVersion() + - ", dhtTxId=" + req.version() + - ", node=" + nodeId + - ", err=" + e.getMessage() + ']'); + IgniteInternalFuture dhtFin = dhtTx == null ? + null : dhtTx.done() ? null : dhtTx.finishFuture(); + + final IgniteInternalFuture nearFin = nearTx == null ? + null : nearTx.done() ? null : nearTx.finishFuture(); + + if (dhtFin != null && nearFin != null) { + GridCompoundFuture fut = new GridCompoundFuture(); + + fut.add(dhtFin); + fut.add(nearFin); + + fut.markInitialized(); + + completeFut = fut; } + else + completeFut = dhtFin != null ? dhtFin : nearFin; - if (nearTx != null) - nearTx.rollback(); + if (completeFut != null) { + final GridDhtTxPrepareResponse res0 = res; + final GridDhtTxRemote dhtTx0 = dhtTx; + final GridNearTxRemote nearTx0 = nearTx; - if (dhtTx != null) - dhtTx.rollback(); + completeFut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture fut) { + sendReply(nodeId, req, res0, dhtTx0, nearTx0); + } + }); + } + else + sendReply(nodeId, req, res, dhtTx, nearTx); } + else + sendReply(nodeId, req, res, dhtTx, nearTx); + } + + /** + * @param nodeId Node ID. + * @param req Request. + */ + protected final void processDhtTxOnePhaseCommitAckRequest(final UUID nodeId, + final GridDhtTxOnePhaseCommitAckRequest req) { + assert nodeId != null; + assert req != null; + + if (log.isDebugEnabled()) + log.debug("Processing dht tx one phase commit ack request [nodeId=" + nodeId + ", req=" + req + ']'); + + for (GridCacheVersion ver : req.versions()) + ctx.tm().removeTxReturn(ver); } /** @@ -1139,12 +1170,10 @@ else if (log.isDebugEnabled()) } /** - * @param nodeId Node ID. * @param tx Transaction. * @param req Request. */ protected void finish( - UUID nodeId, GridDistributedTxRemoteAdapter tx, GridDhtTxPrepareRequest req) throws IgniteTxHeuristicCheckedException { assert tx != null : "No transaction for one-phase commit prepare request: " + req; @@ -1176,6 +1205,52 @@ protected void finish( } } + /** + * @param nodeId Node id. + * @param req Request. + * @param res Response. + * @param dhtTx Dht tx. + * @param nearTx Near tx. + */ + protected void sendReply(UUID nodeId, + GridDhtTxPrepareRequest req, + GridDhtTxPrepareResponse res, + GridDhtTxRemote dhtTx, + GridNearTxRemote nearTx) { + try { + // Reply back to sender. + ctx.io().send(nodeId, res, req.policy()); + + if (txPrepareMsgLog.isDebugEnabled()) { + txPrepareMsgLog.debug("Sent dht prepare response [txId=" + req.nearXidVersion() + + ", dhtTxId=" + req.version() + + ", node=" + nodeId + ']'); + } + } + catch (IgniteCheckedException e) { + if (e instanceof ClusterTopologyCheckedException) { + if (txPrepareMsgLog.isDebugEnabled()) { + txPrepareMsgLog.debug("Failed to send dht prepare response, node left [txId=" + req.nearXidVersion() + + ", dhtTxId=" + req.version() + + ", node=" + nodeId + ']'); + } + } + else { + U.warn(log, "Failed to send tx response to remote node (will rollback transaction) [" + + "txId=" + req.nearXidVersion() + + ", dhtTxId=" + req.version() + + ", node=" + nodeId + + ", err=" + e.getMessage() + ']'); + } + + if (nearTx != null) + nearTx.rollback(); + + if (dhtTx != null) + dhtTx.rollback(); + } + } + /** * Sends tx finish response to remote node, if response is requested. * @@ -1191,7 +1266,26 @@ protected void sendReply(UUID nodeId, GridDhtTxFinishRequest req, boolean commit if (req.checkCommitted()) { res.checkCommitted(true); - if (!committed) { + if (committed) { + if (req.needReturnValue()) { + try { + GridCacheReturnCompletableWrapper wrapper = ctx.tm().getCommittedTxReturn(req.version()); + + if (wrapper != null) + res.returnValue(wrapper.fut().get()); + else + assert !ctx.discovery().alive(nodeId) : nodeId; + } + catch (IgniteCheckedException e) { + if (txFinishMsgLog.isDebugEnabled()) { + txFinishMsgLog.debug("Failed to gain entry processor return value. [txId=" + nearTxId + + ", dhtTxId=" + req.version() + + ", node=" + nodeId + ']'); + } + } + } + } + else { ClusterTopologyCheckedException cause = new ClusterTopologyCheckedException("Primary node left grid."); @@ -1492,8 +1586,7 @@ private void invalidateNearEntry(GridCacheContext cacheCtx, KeyCacheObject key, * @param req Request. */ protected void processCheckPreparedTxRequest(final UUID nodeId, - final GridCacheTxRecoveryRequest req) - { + final GridCacheTxRecoveryRequest req) { if (txRecoveryMsgLog.isDebugEnabled()) { txRecoveryMsgLog.debug("Received tx recovery request [txId=" + req.nearXidVersion() + ", node=" + nodeId + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index 637f3229b4603..fe695368117c3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -151,9 +151,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig /** Commit error. */ protected volatile Throwable commitErr; - /** Need return value. */ - protected boolean needRetVal; - /** Implicit transaction result. */ protected GridCacheReturn implicitRes; @@ -354,13 +351,6 @@ public void implicitSingleResult(GridCacheReturn ret) { implicitRes = ret; } - /** - * @return Flag indicating whether transaction needs return value. - */ - public boolean needReturnValue() { - return needRetVal; - } - /** * @return {@code True} if transaction participates in a cache that has an interceptor configured. */ @@ -368,13 +358,6 @@ public boolean hasInterceptor() { return txState().hasInterceptor(cctx); } - /** - * @param needRetVal Need return value flag. - */ - public void needReturnValue(boolean needRetVal) { - this.needRetVal = needRetVal; - } - /** * @param snd {@code True} if values in tx entries should be replaced with transformed values and sent * to remote nodes. @@ -703,7 +686,7 @@ else if (cacheCtx.isNear() && txEntry.locallyMapped()) txEntry.cached().unswap(false); IgniteBiTuple res = applyTransformClosures(txEntry, - true); + true, null); GridCacheVersion dhtVer = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java index f9357f9aa755e..a1580a58fa413 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java @@ -39,6 +39,7 @@ import org.apache.ignite.events.Event; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -49,7 +50,10 @@ import org.apache.ignite.internal.processors.cache.GridCacheMessage; import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.GridCacheMvccFuture; +import org.apache.ignite.internal.processors.cache.GridCacheReturnCompletableWrapper; import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.GridDeferredAckMessageSender; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.GridCacheMappedVersion; import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxFinishSync; import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryFuture; @@ -57,6 +61,7 @@ import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxRemoteAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocal; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxOnePhaseCommitAckRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxRemote; import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedLockFuture; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter; @@ -87,8 +92,11 @@ import org.apache.ignite.transactions.TransactionState; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; +import org.jsr166.ConcurrentLinkedDeque8; import org.jsr166.ConcurrentLinkedHashMap; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_BUFFER_SIZE; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_TIMEOUT; import static org.apache.ignite.IgniteSystemProperties.IGNITE_MAX_COMPLETED_TX_COUNT; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SLOW_TX_WARN_TIMEOUT; import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_DEADLOCK_DETECTION_MAX_ITERS; @@ -123,6 +131,14 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { /** Tx salvage timeout (default 3s). */ private static final int TX_SALVAGE_TIMEOUT = Integer.getInteger(IGNITE_TX_SALVAGE_TIMEOUT, 100); + /** One phase commit deferred ack request timeout. */ + public static final int DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_TIMEOUT = + Integer.getInteger(IGNITE_DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_TIMEOUT, 500); + + /** One phase commit deferred ack request buffer size. */ + private static final int DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_BUFFER_SIZE = + Integer.getInteger(IGNITE_DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_BUFFER_SIZE, 256); + /** Version in which deadlock detection introduced. */ public static final IgniteProductVersion TX_DEADLOCK_DETECTION_SINCE = IgniteProductVersion.fromString("1.5.19"); @@ -160,7 +176,7 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { Integer.getInteger(IGNITE_MAX_COMPLETED_TX_COUNT, DFLT_MAX_COMPLETED_TX_CNT)); /** Committed local transactions. */ - private final ConcurrentLinkedHashMap completedVersHashMap = + private final ConcurrentLinkedHashMap completedVersHashMap = new ConcurrentLinkedHashMap<>( Integer.getInteger(IGNITE_MAX_COMPLETED_TX_COUNT, DFLT_MAX_COMPLETED_TX_CNT), 0.75f, @@ -168,6 +184,9 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { Integer.getInteger(IGNITE_MAX_COMPLETED_TX_COUNT, DFLT_MAX_COMPLETED_TX_CNT), PER_SEGMENT_Q); + /** Pending one phase commit ack requests sender. */ + private GridDeferredAckMessageSender deferredAckMessageSender; + /** Transaction finish synchronizer. */ private GridCacheTxFinishSync txFinishSync; @@ -209,6 +228,14 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { for (TxDeadlockFuture fut : deadlockDetectFuts.values()) fut.onNodeLeft(nodeId); + + for (Map.Entry entry : completedVersHashMap.entrySet()) { + Object obj = entry.getValue(); + + if (obj instanceof GridCacheReturnCompletableWrapper && + nodeId.equals(((GridCacheReturnCompletableWrapper)obj).nodeId())) + removeTxReturn(entry.getKey()); + } } }, EVT_NODE_FAILED, EVT_NODE_LEFT); @@ -237,6 +264,33 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { txFinishSync = new GridCacheTxFinishSync<>(cctx); txHnd = new IgniteTxHandler(cctx); + + deferredAckMessageSender = new GridDeferredAckMessageSender(cctx.time(), cctx.kernalContext().closure()) { + @Override public int getTimeout() { + return DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_TIMEOUT; + } + + @Override public int getBufferSize() { + return DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_BUFFER_SIZE; + } + + @Override public void finish(UUID nodeId, ConcurrentLinkedDeque8 vers) { + GridDhtTxOnePhaseCommitAckRequest ackReq = new GridDhtTxOnePhaseCommitAckRequest(vers); + + cctx.kernalContext().gateway().readLock(); + + try { + cctx.io().send(nodeId, ackReq, GridIoPolicy.SYSTEM_POOL); + } + catch (IgniteCheckedException e) { + log.error("Failed to send one phase commit ack to backup node [backup=" + + nodeId + ']', e); + } + finally { + cctx.kernalContext().gateway().readUnlock(); + } + } + }; } /** {@inheritDoc} */ @@ -898,9 +952,13 @@ public void removeCommittedTx(IgniteInternalTx tx) { */ public void addCommittedTx(IgniteInternalTx tx) { addCommittedTx(tx, tx.xidVersion(), tx.nearXidVersion()); + } - if (!tx.local() && !tx.near() && tx.onePhaseCommit()) - addCommittedTx(tx, tx.nearXidVersion(), null); + /** + * @param tx Committed transaction. + */ + public void addCommittedTxReturn(IgniteInternalTx tx, GridCacheReturnCompletableWrapper ret) { + addCommittedTxReturn(tx.nearXidVersion(), null, ret); } /** @@ -925,7 +983,7 @@ public boolean addCommittedTx( if (nearXidVer != null) xidVer = new CommittedVersion(xidVer, nearXidVer); - Boolean committed0 = completedVersHashMap.putIfAbsent(xidVer, true); + Object committed0 = completedVersHashMap.putIfAbsent(xidVer, true); if (committed0 == null && (tx == null || tx.needsCompletedVersions())) { Boolean b = completedVersSorted.putIfAbsent(xidVer, true); @@ -933,7 +991,29 @@ public boolean addCommittedTx( assert b == null; } - return committed0 == null || committed0; + Boolean committed = committed0 != null && !committed0.equals(Boolean.FALSE); + + return committed0 == null || committed; + } + + /** + * @param xidVer Completed transaction version. + * @param nearXidVer Optional near transaction ID. + * @param retVal Invoke result. + */ + private void addCommittedTxReturn( + GridCacheVersion xidVer, + @Nullable GridCacheVersion nearXidVer, + GridCacheReturnCompletableWrapper retVal + ) { + assert retVal != null; + + if (nearXidVer != null) + xidVer = new CommittedVersion(xidVer, nearXidVer); + + Object prev = completedVersHashMap.putIfAbsent(xidVer, retVal); + + assert prev == null || Boolean.FALSE.equals(prev) : prev; // Can be rolled back. } /** @@ -945,7 +1025,7 @@ public boolean addRolledbackTx( IgniteInternalTx tx, GridCacheVersion xidVer ) { - Boolean committed0 = completedVersHashMap.putIfAbsent(xidVer, false); + Object committed0 = completedVersHashMap.putIfAbsent(xidVer, false); if (committed0 == null && (tx == null || tx.needsCompletedVersions())) { Boolean b = completedVersSorted.putIfAbsent(xidVer, false); @@ -953,7 +1033,47 @@ public boolean addRolledbackTx( assert b == null; } - return committed0 == null || !committed0; + Boolean committed = committed0 != null && !committed0.equals(Boolean.FALSE); + + return committed0 == null || !committed; + } + + /** + * @param xidVer xidVer Completed transaction version. + * @return Tx result. + */ + public GridCacheReturnCompletableWrapper getCommittedTxReturn(GridCacheVersion xidVer) { + Object retVal = completedVersHashMap.get(xidVer); + + // Will gain true in regular case or GridCacheReturn in onePhaseCommit case. + if (!Boolean.TRUE.equals(retVal)) { + assert !Boolean.FALSE.equals(retVal); // Method should be used only after 'committed' checked. + + GridCacheReturnCompletableWrapper res = (GridCacheReturnCompletableWrapper)retVal; + + removeTxReturn(xidVer); + + return res; + } + else + return null; + } + + /** + * @param xidVer xidVer Completed transaction version. + */ + public void removeTxReturn(GridCacheVersion xidVer) { + Object prev = completedVersHashMap.get(xidVer); + + if (Boolean.FALSE.equals(prev)) // Tx can be rolled back. + return; + + assert prev instanceof GridCacheReturnCompletableWrapper: + prev + " instead of GridCacheReturnCompletableWrapper"; + + boolean res = completedVersHashMap.replace(xidVer, prev, true); + + assert res; } /** @@ -1086,7 +1206,9 @@ public void commitTx(IgniteInternalTx tx) { * so we don't do it here. */ - Boolean committed = completedVersHashMap.get(tx.xidVersion()); + Object committed0 = completedVersHashMap.get(tx.xidVersion()); + + Boolean committed = committed0 != null && !committed0.equals(Boolean.FALSE); // 1. Make sure that committed version has been recorded. if (!((committed != null && committed) || tx.writeSet().isEmpty() || tx.isSystemInvalidate())) { @@ -1672,12 +1794,12 @@ public IgniteInternalFuture txCommitted(GridCacheVersion xidVer) { boolean committed = false; - for (Map.Entry entry : completedVersHashMap.entrySet()) { + for (Map.Entry entry : completedVersHashMap.entrySet()) { if (entry.getKey() instanceof CommittedVersion) { CommittedVersion comm = (CommittedVersion)entry.getKey(); if (comm.nearVer.equals(xidVer)) { - committed = entry.getValue(); + committed = !entry.getValue().equals(Boolean.FALSE); break; } @@ -1809,8 +1931,8 @@ public IgniteInternalFuture remoteTxFinishFuture(GridCacheVersion nearVer) { // Not all transactions were found. Need to scan committed versions to check // if transaction was already committed. - for (Map.Entry e : completedVersHashMap.entrySet()) { - if (!e.getValue()) + for (Map.Entry e : completedVersHashMap.entrySet()) { + if (e.getValue().equals(Boolean.FALSE)) continue; GridCacheVersion ver = e.getKey(); @@ -2136,6 +2258,14 @@ public void removeFuture(long futId) { deadlockDetectFuts.remove(futId); } + /** + * @param nodeId Node ID to send message to. + * @param ver Version to ack. + */ + public void sendDeferredAckResponse(UUID nodeId, GridCacheVersion ver) { + deferredAckMessageSender.sendDeferredAckMessage(nodeId, ver); + } + /** * @return Collection of active transaction deadlock detection futures. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java index e611723c46fb4..c3d194b0c4f27 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java @@ -21,6 +21,7 @@ import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.Map; +import java.util.Random; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicBoolean; @@ -30,7 +31,6 @@ import javax.cache.integration.CacheWriterException; import javax.cache.processor.EntryProcessorResult; import javax.cache.processor.MutableEntry; - import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheAtomicWriteOrderMode; @@ -43,8 +43,10 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; @@ -59,6 +61,7 @@ import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; import static org.apache.ignite.cache.CacheRebalanceMode.SYNC; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager.DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_TIMEOUT; import static org.apache.ignite.testframework.GridTestUtils.TestMemoryMode; import static org.apache.ignite.testframework.GridTestUtils.runAsync; @@ -70,7 +73,7 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCommonAbst private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); /** */ - private static final long DURATION = 60_000; + protected static final long DURATION = 60_000; /** */ protected static final int GRID_CNT = 4; @@ -78,8 +81,8 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCommonAbst /** * @return Keys count for the test. */ - private int keysCount() { - return 10_000; + protected int keysCount() { + return 2_000; } /** @@ -249,12 +252,17 @@ protected final void checkRetry(Test test, TestMemoryMode memMode, boolean store IgniteInternalFuture fut = runAsync(new Callable() { @Override public Object call() throws Exception { + Random rnd = new Random(); + while (!finished.get()) { stopGrid(3); U.sleep(300); startGrid(3); + + if (rnd.nextBoolean()) // OPC possible only when there is no migration from one backup to another. + awaitPartitionMapExchange(); } return null; @@ -456,6 +464,29 @@ protected final void checkRetry(Test test, TestMemoryMode memMode, boolean store assertTrue("Unexpected atomic futures: " + futs, futs.isEmpty()); } + + checkOnePhaseCommitReturnValuesCleaned(); + } + + /** + * + */ + protected void checkOnePhaseCommitReturnValuesCleaned() throws IgniteInterruptedCheckedException { + U.sleep(DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_TIMEOUT); + + for (int i = 0; i < GRID_CNT; i++) { + IgniteKernal ignite = (IgniteKernal)grid(i); + + IgniteTxManager tm = ignite.context().cache().context().tm(); + + Map completedVersHashMap = U.field(tm, "completedVersHashMap"); + + for (Object o : completedVersHashMap.values()) { + assertTrue("completedVersHashMap contains" + o.getClass() + " instead of boolean. " + + "These values should be replaced by boolean after onePhaseCommit finished. " + + "[node=" + i + "]", o instanceof Boolean); + } + } } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java index 9204bc8826957..9bfde27350e87 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht; import java.util.HashSet; +import java.util.Random; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ThreadLocalRandom; @@ -88,16 +89,6 @@ public void testAtomicLongRetries() throws Exception { } } - /** {@inheritDoc} */ - @Override public void testGetAndPut() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-1525"); - } - - /** {@inheritDoc} */ - @Override public void testInvoke() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-1525"); - } - /** * @throws Exception If failed. */ @@ -216,6 +207,70 @@ public void explicitTransactionRetries(TestMemoryMode memMode, boolean store) th } } + /** + * + */ + public void testOriginatingNodeFailureForcesOnePhaseCommitDataCleanup() throws Exception { + ignite(0).createCache(cacheConfiguration(TestMemoryMode.HEAP, false)); + + final AtomicBoolean finished = new AtomicBoolean(); + + final int keysCnt = keysCount(); + + IgniteInternalFuture fut = runAsync(new Callable() { + @Override public Object call() throws Exception { + Random rnd = new Random(); + + while (!finished.get()) { + stopGrid(0); + + U.sleep(300); + + startGrid(0); + + if (rnd.nextBoolean()) // OPC possible only when there is no migration from one backup to another. + awaitPartitionMapExchange(); + } + + return null; + } + }); + + IgniteInternalFuture fut2 = runAsync(new Callable() { + @Override public Object call() throws Exception { + int iter = 0; + + while (!finished.get()) { + try { + IgniteCache cache = ignite(0).cache(null); + + Integer val = ++iter; + + for (int i = 0; i < keysCnt; i++) + cache.invoke(i, new SetEntryProcessor(val)); + } + catch (Exception e) { + // No-op. + } + } + + return null; + } + }); + + try { + U.sleep(DURATION); + } + finally { + finished.set(true); + + fut.get(); + fut2.get(); + } + + checkOnePhaseCommitReturnValuesCleaned(); + } + /** * Callable to process inside transaction. */ diff --git a/modules/yardstick/config/benchmark-client-mode.properties b/modules/yardstick/config/benchmark-client-mode.properties index ba5525f8509cf..f7c83472cf570 100644 --- a/modules/yardstick/config/benchmark-client-mode.properties +++ b/modules/yardstick/config/benchmark-client-mode.properties @@ -70,6 +70,8 @@ CONFIGS="\ -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 --client -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds ${ver}atomic-put-1-backup,\ -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 --client -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds ${ver}atomic-put-get-1-backup,\ -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 --client -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds ${ver}tx-put-1-backup,\ +-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 --client -sm PRIMARY_SYNC -dn IgniteGetAndPutTxBenchmark -sn IgniteNode -ds ${ver}tx-getAndPut-1-backup,\ +-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 --client -sm PRIMARY_SYNC -dn IgniteInvokeTxBenchmark -sn IgniteNode -ds ${ver}tx-invoke-1-backup,\ -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 --client -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds ${ver}tx-put-get-1-backup,\ -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 --client -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds ${ver}sql-query-1-backup,\ -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 --client -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds ${ver}sql-query-join-1-backup,\ diff --git a/modules/yardstick/config/benchmark-tx-win.properties b/modules/yardstick/config/benchmark-tx-win.properties index 73b857df37e8a..54a40b1244e46 100644 --- a/modules/yardstick/config/benchmark-tx-win.properties +++ b/modules/yardstick/config/benchmark-tx-win.properties @@ -54,6 +54,8 @@ set DRIVER_HOSTS=localhost :: Note that each benchmark is set to run for 300 seconds (5 mins) with warm-up set to 60 seconds (1 minute). set CONFIGS=^ -cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,^ +-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteGetAndPutTxBenchmark -sn IgniteNode -ds tx-getAndPut-1-backup,^ +-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteInvokeTxBenchmark -sn IgniteNode -ds tx-invoke-1-backup,^ -cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxOffHeapBenchmark -sn IgniteNode -ds tx-put-offheap-1-backup,^ -cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxOffHeapValuesBenchmark -sn IgniteNode -ds tx-put-offheap-val-1-backup,^ -cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup,^ diff --git a/modules/yardstick/config/benchmark-tx.properties b/modules/yardstick/config/benchmark-tx.properties index f3dbc24c2c8e1..0d5bb02dd3e1a 100644 --- a/modules/yardstick/config/benchmark-tx.properties +++ b/modules/yardstick/config/benchmark-tx.properties @@ -59,6 +59,8 @@ nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} # Note that each benchmark is set to run for 300 seconds (5 mins) with warm-up set to 60 seconds (1 minute). CONFIGS="\ -cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,\ +-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteGetAndPutTxBenchmark -sn IgniteNode -ds tx-getAndPut-1-backup,\ +-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteInvokeTxBenchmark -sn IgniteNode -ds tx-invoke-1-backup,\ -cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxOffHeapBenchmark -sn IgniteNode -ds tx-put-offheap-1-backup,\ -cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxOffHeapValuesBenchmark -sn IgniteNode -ds tx-put-offheap-val-1-backup,\ -cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup,\ diff --git a/modules/yardstick/config/benchmark-win.properties b/modules/yardstick/config/benchmark-win.properties index b6ecd6730e9c4..b75b5d6d1af78 100644 --- a/modules/yardstick/config/benchmark-win.properties +++ b/modules/yardstick/config/benchmark-win.properties @@ -59,6 +59,8 @@ set CONFIGS=^ -cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds atomic-put-1-backup,^ -cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds atomic-put-get-1-backup,^ -cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,^ +-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteGetAndPutTxBenchmark -sn IgniteNode -ds tx-getAndPut-1-backup,^ +-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteInvokeTxBenchmark -sn IgniteNode -ds tx-invoke-1-backup,^ -cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup,^ -cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds sql-query-1-backup,^ -cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds sql-query-join-1-backup,^ diff --git a/modules/yardstick/config/benchmark.properties b/modules/yardstick/config/benchmark.properties index 67ef5ef3bbf88..cfc14999751d9 100644 --- a/modules/yardstick/config/benchmark.properties +++ b/modules/yardstick/config/benchmark.properties @@ -71,6 +71,8 @@ CONFIGS="\ -cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds ${ver}atomic-put-1-backup,\ -cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds ${ver}atomic-put-get-1-backup,\ -cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds ${ver}tx-put-1-backup,\ +-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteGetAndPutTxBenchmark -sn IgniteNode -ds ${ver}tx-getAndPut-1-backup,\ +-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteInvokeTxBenchmark -sn IgniteNode -ds ${ver}tx-invoke-1-backup,\ -cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds ${ver}tx-put-get-1-backup,\ -cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds ${ver}sql-query-1-backup,\ -cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds ${ver}sql-query-join-1-backup,\ diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutBenchmark.java new file mode 100644 index 0000000000000..40e563cbef32a --- /dev/null +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutBenchmark.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.yardstick.cache; + +import java.util.Map; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.yardstick.cache.model.SampleValue; + +/** + * Ignite benchmark that performs invoke operations. + */ +public class IgniteGetAndPutBenchmark extends IgniteCacheAbstractBenchmark { + /** {@inheritDoc} */ + @Override public boolean test(Map ctx) throws Exception { + int key = nextRandom(args.range()); + + cache.getAndPut(key, new SampleValue(key)); + + return true; + } + + /** {@inheritDoc} */ + @Override protected IgniteCache cache() { + return ignite().cache("atomic"); + } +} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutTxBenchmark.java new file mode 100644 index 0000000000000..49ae985d0d2bf --- /dev/null +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutTxBenchmark.java @@ -0,0 +1,70 @@ +/* + * 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.yardstick.cache; + +import java.util.Map; +import java.util.concurrent.Callable; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.IgniteTransactions; +import org.apache.ignite.yardstick.IgniteBenchmarkUtils; +import org.apache.ignite.yardstick.cache.model.SampleValue; +import org.yardstickframework.BenchmarkConfiguration; + +/** + * Ignite benchmark that performs invoke operations. + */ +public class IgniteGetAndPutTxBenchmark extends IgniteCacheAbstractBenchmark { + /** */ + private IgniteTransactions txs; + + /** */ + private Callable clo; + + /** {@inheritDoc} */ + @Override public void setUp(BenchmarkConfiguration cfg) throws Exception { + super.setUp(cfg); + + if (!IgniteSystemProperties.getBoolean("SKIP_MAP_CHECK")) + ignite().compute().broadcast(new WaitMapExchangeFinishCallable()); + + txs = ignite().transactions(); + + clo = new Callable() { + @Override public Void call() throws Exception { + int key = nextRandom(args.range()); + + cache.getAndPut(key, new SampleValue(key)); + + return null; + } + }; + } + + /** {@inheritDoc} */ + @Override public boolean test(Map ctx) throws Exception { + IgniteBenchmarkUtils.doInTransaction(txs, args.txConcurrency(), args.txIsolation(), clo); + + return true; + } + + /** {@inheritDoc} */ + @Override protected IgniteCache cache() { + return ignite().cache("tx"); + } +} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteInvokeTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteInvokeTxBenchmark.java index 8f05598705f16..64dc6b8bdd363 100644 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteInvokeTxBenchmark.java +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteInvokeTxBenchmark.java @@ -17,12 +17,52 @@ package org.apache.ignite.yardstick.cache; +import java.util.Map; +import java.util.concurrent.Callable; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.IgniteTransactions; +import org.apache.ignite.yardstick.IgniteBenchmarkUtils; +import org.apache.ignite.yardstick.cache.model.SampleValue; +import org.yardstickframework.BenchmarkConfiguration; /** * Ignite benchmark that performs invoke operations. */ public class IgniteInvokeTxBenchmark extends IgniteInvokeBenchmark { + /** */ + private IgniteTransactions txs; + + /** */ + private Callable clo; + + /** {@inheritDoc} */ + @Override public void setUp(BenchmarkConfiguration cfg) throws Exception { + super.setUp(cfg); + + if (!IgniteSystemProperties.getBoolean("SKIP_MAP_CHECK")) + ignite().compute().broadcast(new WaitMapExchangeFinishCallable()); + + txs = ignite().transactions(); + + clo = new Callable() { + @Override public Void call() throws Exception { + int key = nextRandom(args.range()); + + cache.invoke(key, new SetValueEntryProcessor(new SampleValue(key))); + + return null; + } + }; + } + + /** {@inheritDoc} */ + @Override public boolean test(Map ctx) throws Exception { + IgniteBenchmarkUtils.doInTransaction(txs, args.txConcurrency(), args.txIsolation(), clo); + + return true; + } + /** {@inheritDoc} */ @Override protected IgniteCache cache() { return ignite().cache("tx"); From 780bf23d5c89452dd062be4fab9e2e56d50bb9e2 Mon Sep 17 00:00:00 2001 From: sboikov Date: Mon, 19 Sep 2016 18:19:33 +0300 Subject: [PATCH 146/487] ignite-3810 Fixed hang in FileSwapSpaceSpi when too large value is stored --- .../spi/swapspace/file/FileSwapSpaceSpi.java | 38 ++++++-- .../CacheSwapUnswapGetTestSmallQueueSize.java | 35 ++++++++ .../file/GridFileSwapSpaceSpiSelfTest.java | 89 +++++++++++++++++++ .../testsuites/IgniteCacheTestSuite4.java | 2 + 4 files changed, 158 insertions(+), 6 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTestSmallQueueSize.java diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java index 8809f08538725..9be5b93cc6789 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java @@ -639,7 +639,7 @@ private void notifyListener(int evtType, @Nullable String spaceName) { if (space == null && create) { validateName(name); - Space old = spaces.putIfAbsent(masked, space = new Space(masked)); + Space old = spaces.putIfAbsent(masked, space = new Space(masked, log)); if (old != null) space = old; @@ -833,13 +833,21 @@ private static class SwapValuesQueue { /** */ private final int maxSize; + /** */ + private final IgniteLogger log; + + /** */ + private boolean queueSizeWarn; + /** * @param minTakeSize Min size. * @param maxSize Max size. + * @param log logger */ - private SwapValuesQueue(int minTakeSize, int maxSize) { + private SwapValuesQueue(int minTakeSize, int maxSize, IgniteLogger log) { this.minTakeSize = minTakeSize; this.maxSize = maxSize; + this.log = log; } /** @@ -852,8 +860,24 @@ public void add(SwapValue val) throws IgniteSpiException { lock.lock(); try { - while (size + val.len > maxSize) - mayAdd.await(); + boolean largeVal = val.len > maxSize; + + if (largeVal) { + if (!queueSizeWarn) { + U.warn(log, "Trying to save in swap entry which have size more than write queue size. " + + "You may wish to increase 'maxWriteQueueSize' in FileSwapSpaceSpi configuration " + + "[queueMaxSize=" + maxSize + ", valSize=" + val.len + ']'); + + queueSizeWarn = true; + } + + while (size >= minTakeSize) + mayAdd.await(); + } + else { + while (size + val.len > maxSize) + mayAdd.await(); + } size += val.len; @@ -1419,7 +1443,7 @@ private class Space { private SwapFile right; /** */ - private final SwapValuesQueue que = new SwapValuesQueue(writeBufSize, maxWriteQueSize); + private final SwapValuesQueue que; /** Partitions. */ private final ConcurrentMap> parts = @@ -1442,11 +1466,13 @@ private class Space { /** * @param name Space name. + * @param log Logger. */ - private Space(String name) { + private Space(String name, IgniteLogger log) { assert name != null; this.name = name; + this.que = new SwapValuesQueue(writeBufSize, maxWriteQueSize, log); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTestSmallQueueSize.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTestSmallQueueSize.java new file mode 100644 index 0000000000000..8d189feb57ff9 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTestSmallQueueSize.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi; + +/** + * + */ +public class CacheSwapUnswapGetTestSmallQueueSize extends CacheSwapUnswapGetTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((FileSwapSpaceSpi)cfg.getSwapSpaceSpi()).setMaxWriteQueueSize(2); + + return cfg; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java index 64652b19e3220..ab211652273f3 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java @@ -25,11 +25,14 @@ import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.CIX1; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiInClosure; @@ -37,8 +40,10 @@ import org.apache.ignite.spi.swapspace.GridSwapSpaceSpiAbstractSelfTest; import org.apache.ignite.spi.swapspace.SwapKey; import org.apache.ignite.spi.swapspace.SwapSpaceSpi; +import org.apache.ignite.testframework.GridTestUtils; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; +import org.junit.Assert; /** * Test for {@link FileSwapSpaceSpi}. @@ -364,4 +369,88 @@ public void testMultithreadedOperations() throws Exception { assertEquals(hash0, hash1); } + + /** + * @throws IgniteCheckedException If failed. + */ + public void testSaveValueLargeThenQueueSize() throws IgniteCheckedException { + final String spaceName = "mySpace"; + final SwapKey key = new SwapKey("key"); + + final byte[] val = new byte[FileSwapSpaceSpi.DFLT_QUE_SIZE * 2]; + Arrays.fill(val, (byte)1); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public byte[] call() throws Exception { + return saveAndGet(spaceName, key, val); + } + }); + + byte[] bytes = fut.get(10_000); + + Assert.assertArrayEquals(val, bytes); + } + + /** + * @throws IgniteCheckedException If failed. + */ + public void testSaveValueLargeThenQueueSizeMultiThreaded() throws Exception { + final String spaceName = "mySpace"; + + final int threads = 5; + + long DURATION = 30_000; + + final int maxSize = FileSwapSpaceSpi.DFLT_QUE_SIZE * 2; + + final AtomicBoolean done = new AtomicBoolean(); + + try { + IgniteInternalFuture fut = multithreadedAsync(new Callable() { + @Override public Void call() throws Exception { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + while (!done.get()) { + SwapKey key = new SwapKey(rnd.nextInt(1000)); + + spi.store(spaceName, key, new byte[rnd.nextInt(0, maxSize)], context()); + } + + return null; + } + }, threads, " async-put"); + + Thread.sleep(DURATION); + + done.set(true); + + fut.get(); + } + finally { + done.set(true); + } + } + + /** + * @param spaceName Space name. + * @param key Key. + * @param val Value. + * @throws Exception If failed. + * @return Read bytes. + */ + private byte[] saveAndGet(final String spaceName, final SwapKey key, byte[] val) throws Exception { + spi.store(spaceName, key, val, context()); + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return spi.read(spaceName, key, context()) != null; + } + }, 10_000); + + byte[] res = spi.read(spaceName, key, context()); + + assertNotNull(res); + + return res; + } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java index 60d59d70424b9..c494e732b4f66 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java @@ -41,6 +41,7 @@ import org.apache.ignite.internal.processors.cache.CacheStoreUsageMultinodeStaticStartAtomicTest; import org.apache.ignite.internal.processors.cache.CacheStoreUsageMultinodeStaticStartTxTest; import org.apache.ignite.internal.processors.cache.CacheSwapUnswapGetTest; +import org.apache.ignite.internal.processors.cache.CacheSwapUnswapGetTestSmallQueueSize; import org.apache.ignite.internal.processors.cache.CacheTxNotAllowReadFromBackupTest; import org.apache.ignite.internal.processors.cache.CrossCacheLockTest; import org.apache.ignite.internal.processors.cache.GridCacheMarshallingNodeJoinSelfTest; @@ -304,6 +305,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheVersionedEntryReplicatedTransactionalOffHeapSelfTest.class); suite.addTestSuite(CacheSwapUnswapGetTest.class); + suite.addTestSuite(CacheSwapUnswapGetTestSmallQueueSize.class); suite.addTestSuite(GridCacheDhtTxPreloadSelfTest.class); suite.addTestSuite(GridCacheNearTxPreloadSelfTest.class); From c1372ce2f0633968036fcfb079718214605c3350 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 20 Sep 2016 11:39:37 +0300 Subject: [PATCH 147/487] Client discovery: wait during join if receive RES_CONTINUE_JOIN, RES_WAIT. --- .../ignite/spi/discovery/tcp/ClientImpl.java | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index bf7f519b6ca7f..2c85645305da5 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -497,6 +497,8 @@ else if (state == DISCONNECTED) { Iterator it = addrs.iterator(); + boolean wait = false; + while (it.hasNext()) { if (Thread.currentThread().isInterrupted()) throw new InterruptedException(); @@ -515,12 +517,17 @@ else if (state == DISCONNECTED) { Socket sock = sockAndRes.get1().socket(); + if (log.isDebugEnabled()) + log.debug("Received response to join request [addr=" + addr + ", res=" + sockAndRes.get2() + ']'); + switch (sockAndRes.get2()) { case RES_OK: return new T2<>(sockAndRes.get1(), sockAndRes.get3()); case RES_CONTINUE_JOIN: case RES_WAIT: + wait = true; + U.closeQuiet(sock); break; @@ -533,7 +540,16 @@ else if (state == DISCONNECTED) { } } - if (addrs.isEmpty()) { + if (wait) { + if (timeout > 0 && (U.currentTimeMillis() - startTime) > timeout) + return null; + + if (log.isDebugEnabled()) + log.debug("Will wait before retry join."); + + Thread.sleep(2000); + } + else if (addrs.isEmpty()) { if (timeout > 0 && (U.currentTimeMillis() - startTime) > timeout) return null; From 5a35ee9dad194b3009151b79f0ebd3976bb8fd22 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Tue, 20 Sep 2016 14:10:55 +0500 Subject: [PATCH 148/487] IGNITE-3859: IGFS: Support direct PROXY mode invocation in the open method, add proxy mode to IgfsInputStreamImpl This closes #1065. This closes #1083. --- .../internal/processors/igfs/IgfsContext.java | 35 +++++ .../processors/igfs/IgfsDataManager.java | 121 ++++++++---------- .../internal/processors/igfs/IgfsImpl.java | 82 +++++++++--- .../processors/igfs/IgfsInputStreamImpl.java | 103 +++++++++++---- 4 files changed, 226 insertions(+), 115 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java index 3e012469b0022..3405b5332eed2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java @@ -20,7 +20,10 @@ import java.util.LinkedList; import java.util.List; import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.internal.GridKernalContext; @@ -60,6 +63,12 @@ public class IgfsContext { /** Local cluster node. */ private volatile ClusterNode locNode; + /** IGFS executor service. */ + private ExecutorService igfsSvc; + + /** Logger. */ + protected IgniteLogger log; + /** * @param ctx Kernal context. * @param cfg IGFS configuration. @@ -85,6 +94,10 @@ public IgfsContext( this.srvMgr = add(srvMgr); this.fragmentizerMgr = add(fragmentizerMgr); + log = ctx.log(IgfsContext.class); + + igfsSvc = ctx.getIgfsExecutorService(); + igfs = new IgfsImpl(this); } @@ -205,6 +218,28 @@ public ClusterNode localNode() { return locNode; } + /** + * Executes runnable in IGFS executor service. If execution rejected, runnable will be executed + * in caller thread. + * + * @param r Runnable to execute. + */ + public void runInIgfsThreadPool(Runnable r) { + try { + igfsSvc.submit(r); + } + catch (RejectedExecutionException ignored) { + // This exception will happen if network speed is too low and data comes faster + // than we can send it to remote nodes. + try { + r.run(); + } + catch (Exception e) { + log.warning("Failed to execute IGFS runnable: " + r, e); + } + } + } + /** * Adds manager to managers list. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java index d2183f927de30..2f704aea515de 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java @@ -45,7 +45,6 @@ import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; -import org.apache.ignite.internal.util.lang.GridPlainCallable; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.CX1; import org.apache.ignite.internal.util.typedef.F; @@ -74,12 +73,9 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; @@ -123,9 +119,6 @@ public class IgfsDataManager extends IgfsManager { /** Affinity key generator. */ private AtomicLong affKeyGen = new AtomicLong(); - /** IGFS executor service. */ - private ExecutorService igfsSvc; - /** Request ID counter for write messages. */ private AtomicLong reqIdCtr = new AtomicLong(); @@ -183,8 +176,6 @@ else if (msg instanceof IgfsAckMessage) } }, EVT_NODE_LEFT, EVT_NODE_FAILED); - igfsSvc = igfsCtx.kernalContext().getIgfsExecutorService(); - delWorker = new AsyncDeleteWorker(igfsCtx.kernalContext().gridName(), "igfs-" + igfsName + "-delete-worker", log); } @@ -345,45 +336,11 @@ private IgniteDataStreamer dataStreamer() { if (oldRmtReadFut == null) { try { - if (log.isDebugEnabled()) - log.debug("Reading non-local data block in the secondary file system [path=" + - path + ", fileInfo=" + fileInfo + ", blockIdx=" + blockIdx + ']'); - - int blockSize = fileInfo.blockSize(); - - long pos = blockIdx * blockSize; // Calculate position for Hadoop - - res = new byte[blockSize]; - - int read = 0; - - synchronized (secReader) { - try { - // Delegate to the secondary file system. - while (read < blockSize) { - int r = secReader.read(pos + read, res, read, blockSize - read); - - if (r < 0) - break; - - read += r; - } - } - catch (IOException e) { - throw new IgniteCheckedException("Failed to read data due to secondary file system " + - "exception: " + e.getMessage(), e); - } - } - - // If we did not read full block at the end of the file - trim it. - if (read != blockSize) - res = Arrays.copyOf(res, read); + res = secondaryDataBlock(path, blockIdx, secReader, fileInfo.blockSize()); rmtReadFut.onDone(res); putBlock(fileInfo.blockSize(), key, res); - - igfsCtx.metrics().addReadBlocks(1, 1); } catch (IgniteCheckedException e) { rmtReadFut.onDone(e); @@ -416,12 +373,60 @@ private IgniteDataStreamer dataStreamer() { return fut; } + /** + * Get data block for specified block index from secondary reader. + * + * @param path Path reading from. + * @param blockIdx Block index. + * @param secReader Optional secondary file system reader. + * @param blockSize Block size. + * @return Requested data block or {@code null} if nothing found. + * @throws IgniteCheckedException If failed. + */ + @Nullable public byte[] secondaryDataBlock(IgfsPath path, long blockIdx, + IgfsSecondaryFileSystemPositionedReadable secReader, int blockSize) throws IgniteCheckedException { + if (log.isDebugEnabled()) + log.debug("Reading non-local data block in the secondary file system [path=" + + path + ", blockIdx=" + blockIdx + ']'); + + long pos = blockIdx * blockSize; // Calculate position for Hadoop + + byte[] res = new byte[blockSize]; + + int read = 0; + + try { + // Delegate to the secondary file system. + while (read < blockSize) { + int r = secReader.read(pos + read, res, read, blockSize - read); + + if (r < 0) + break; + + read += r; + } + } + catch (IOException e) { + throw new IgniteCheckedException("Failed to read data due to secondary file system " + + "exception: " + e.getMessage(), e); + } + + // If we did not read full block at the end of the file - trim it. + if (read != blockSize) + res = Arrays.copyOf(res, read); + + igfsCtx.metrics().addReadBlocks(1, 1); + + return res; + } + /** * Stores the given block in data cache. * * @param blockSize The size of the block. * @param key The data cache key of the block. * @param data The new value of the block. + * @throws IgniteCheckedException If failed. */ private void putBlock(int blockSize, IgfsBlockKey key, byte[] data) throws IgniteCheckedException { if (data.length < blockSize) @@ -967,8 +972,8 @@ private void processBatch(IgniteUuid fileId, final ClusterNode node, } } else { - callIgfsLocalSafe(new GridPlainCallable() { - @Override @Nullable public Object call() throws Exception { + igfsCtx.runInIgfsThreadPool(new Runnable() { + @Override public void run() { storeBlocksAsync(blocks).listen(new CI1>() { @Override public void apply(IgniteInternalFuture fut) { try { @@ -981,8 +986,6 @@ private void processBatch(IgniteUuid fileId, final ClusterNode node, } } }); - - return null; } }); } @@ -1069,28 +1072,6 @@ private void processPartialBlockWrite(IgniteUuid fileId, IgfsBlockKey colocatedK } } - /** - * Executes callable in IGFS executor service. If execution rejected, callable will be executed - * in caller thread. - * - * @param c Callable to execute. - */ - private void callIgfsLocalSafe(Callable c) { - try { - igfsSvc.submit(c); - } - catch (RejectedExecutionException ignored) { - // This exception will happen if network speed is too low and data comes faster - // than we can send it to remote nodes. - try { - c.call(); - } - catch (Exception e) { - log.warning("Failed to execute IGFS callable: " + c, e); - } - } - } - /** * @param blocks Blocks to write. * @return Future that will be completed after put is done. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 45596a3f4f376..87a4699a4de6b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -47,6 +47,7 @@ import org.apache.ignite.igfs.mapreduce.IgfsRecordResolver; import org.apache.ignite.igfs.mapreduce.IgfsTask; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; +import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager; @@ -949,34 +950,79 @@ private IgfsEntryInfo primaryInfoForListing(IgfsPath path) throws IgniteCheckedE IgfsMode mode = resolveMode(path); - if (mode != PRIMARY) { - assert IgfsUtils.isDualMode(mode); + switch (mode) { + case PRIMARY: { + IgfsEntryInfo info = meta.infoForPath(path); - IgfsSecondaryInputStreamDescriptor desc = meta.openDual(secondaryFs, path, bufSize0); + if (info == null) + throw new IgfsPathNotFoundException("File not found: " + path); - IgfsInputStreamImpl os = new IgfsInputStreamImpl(igfsCtx, path, desc.info(), - cfg.getPrefetchBlocks(), seqReadsBeforePrefetch, desc.reader()); + if (!info.isFile()) + throw new IgfsPathIsDirectoryException("Failed to open file (not a file): " + path); - IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_READ); + // Input stream to read data from grid cache with separate blocks. + IgfsInputStreamImpl os = new IgfsInputStreamImpl(igfsCtx, path, info, + cfg.getPrefetchBlocks(), seqReadsBeforePrefetch, null, + info.length(), info.blockSize(), info.blocksCount(), false); - return os; - } + IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_READ); - IgfsEntryInfo info = meta.infoForPath(path); + return os; + } - if (info == null) - throw new IgfsPathNotFoundException("File not found: " + path); + case DUAL_ASYNC: + case DUAL_SYNC: { + assert IgfsUtils.isDualMode(mode); - if (!info.isFile()) - throw new IgfsPathIsDirectoryException("Failed to open file (not a file): " + path); + IgfsSecondaryInputStreamDescriptor desc = meta.openDual(secondaryFs, path, bufSize0); + + IgfsEntryInfo info = desc.info(); - // Input stream to read data from grid cache with separate blocks. - IgfsInputStreamImpl os = new IgfsInputStreamImpl(igfsCtx, path, info, - cfg.getPrefetchBlocks(), seqReadsBeforePrefetch, null); + IgfsInputStreamImpl os = new IgfsInputStreamImpl(igfsCtx, path, info, + cfg.getPrefetchBlocks(), seqReadsBeforePrefetch, desc.reader(), + info.length(), info.blockSize(), info.blocksCount(), false); + + IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_READ); + + return os; + } - IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_READ); + case PROXY: { + assert secondaryFs != null; - return os; + IgfsFile info = info(path); + + if (info == null) + throw new IgfsPathNotFoundException("File not found: " + path); + + if (!info.isFile()) + throw new IgfsPathIsDirectoryException("Failed to open file (not a file): " + path); + + IgfsSecondaryFileSystemPositionedReadable secReader = + new IgfsLazySecondaryFileSystemPositionedReadable(secondaryFs, path, bufSize); + + long len = info.length(); + + int blockSize = info.blockSize() > 0 ? info.blockSize() : cfg.getBlockSize(); + + long blockCnt = len / blockSize; + + if (len % blockSize != 0) + blockCnt++; + + IgfsInputStream os = new IgfsInputStreamImpl(igfsCtx, path, null, + cfg.getPrefetchBlocks(), seqReadsBeforePrefetch, secReader, + info.length(), blockSize, blockCnt, true); + + IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_READ); + + return os; + } + + default: + assert false : "Unexpected mode " + mode; + return null; + } } }); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java index 2f9f2fcab5e1f..0d9f2cd36fda5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java @@ -28,6 +28,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager; import org.apache.ignite.internal.util.GridConcurrentHashSet; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteInClosure; @@ -109,21 +110,44 @@ public class IgfsInputStreamImpl extends IgfsInputStream implements IgfsSecondar /** Time consumed on reading. */ private long time; + /** File Length. */ + private long len; + + /** Block size to read. */ + private int blockSize; + + /** Block size to read. */ + private long blocksCnt; + + /** Proxy mode. */ + private boolean proxy; + /** * Constructs file output stream. - * - * @param igfsCtx IGFS context. + * @param igfsCtx IGFS context. * @param path Path to stored file. * @param fileInfo File info to write binary data to. * @param prefetchBlocks Number of blocks to prefetch. * @param seqReadsBeforePrefetch Amount of sequential reads before prefetch is triggered. * @param secReader Optional secondary file system reader. + * @param len File length. + * @param blockSize Block size. + * @param blocksCnt Blocks count. + * @param proxy Proxy mode flag. */ - IgfsInputStreamImpl(IgfsContext igfsCtx, IgfsPath path, IgfsEntryInfo fileInfo, int prefetchBlocks, - int seqReadsBeforePrefetch, @Nullable IgfsSecondaryFileSystemPositionedReadable secReader) { + IgfsInputStreamImpl( + IgfsContext igfsCtx, + IgfsPath path, + @Nullable IgfsEntryInfo fileInfo, + int prefetchBlocks, + int seqReadsBeforePrefetch, + @Nullable IgfsSecondaryFileSystemPositionedReadable secReader, + long len, + int blockSize, + long blocksCnt, + boolean proxy) { assert igfsCtx != null; assert path != null; - assert fileInfo != null; this.igfsCtx = igfsCtx; this.path = path; @@ -131,6 +155,10 @@ public class IgfsInputStreamImpl extends IgfsInputStream implements IgfsSecondar this.prefetchBlocks = prefetchBlocks; this.seqReadsBeforePrefetch = seqReadsBeforePrefetch; this.secReader = secReader; + this.len = len; + this.blockSize = blockSize; + this.blocksCnt = blocksCnt; + this.proxy = proxy; log = igfsCtx.kernalContext().log(IgfsInputStream.class); @@ -154,7 +182,7 @@ public synchronized long bytes() { /** {@inheritDoc} */ @Override public long length() { - return fileInfo.length(); + return len; } /** {@inheritDoc} */ @@ -195,7 +223,7 @@ public synchronized long bytes() { /** {@inheritDoc} */ @Override public synchronized int available() throws IOException { - long l = fileInfo.length() - pos; + long l = len - pos; if (l < 0) return 0; @@ -240,7 +268,7 @@ public synchronized long bytes() { @SuppressWarnings("IfMayBeConditional") public synchronized byte[][] readChunks(long pos, int len) throws IOException { // Readable bytes in the file, starting from the specified position. - long readable = fileInfo.length() - pos; + long readable = this.len - pos; if (readable <= 0) return EMPTY_CHUNKS; @@ -254,8 +282,8 @@ public synchronized byte[][] readChunks(long pos, int len) throws IOException { bytes += len; - int start = (int)(pos / fileInfo.blockSize()); - int end = (int)((pos + len - 1) / fileInfo.blockSize()); + int start = (int)(pos / blockSize); + int end = (int)((pos + len - 1) / blockSize); int chunkCnt = end - start + 1; @@ -264,7 +292,7 @@ public synchronized byte[][] readChunks(long pos, int len) throws IOException { for (int i = 0; i < chunkCnt; i++) { byte[] block = blockFragmentizerSafe(start + i); - int blockOff = (int)(pos % fileInfo.blockSize()); + int blockOff = (int)(pos % blockSize); int blockLen = Math.min(len, block.length - blockOff); // If whole block can be used as result, do not do array copy. @@ -366,7 +394,7 @@ private int readFromStore(long pos, byte[] buf, int off, int len) throws IOExcep return 0; // Fully read done: read zero bytes correctly. // Readable bytes in the file, starting from the specified position. - long readable = fileInfo.length() - pos; + long readable = this.len - pos; if (readable <= 0) return -1; // EOF. @@ -378,10 +406,10 @@ private int readFromStore(long pos, byte[] buf, int off, int len) throws IOExcep assert len > 0; - byte[] block = blockFragmentizerSafe(pos / fileInfo.blockSize()); + byte[] block = blockFragmentizerSafe(pos / blockSize); // Skip bytes to expected position. - int blockOff = (int)(pos % fileInfo.blockSize()); + int blockOff = (int)(pos % blockSize); len = Math.min(len, block.length - blockOff); @@ -412,7 +440,7 @@ private byte[] blockFragmentizerSafe(long blockIdx) throws IOException { ", blockIdx=" + blockIdx + ", errMsg=" + e.getMessage() + ']'); // This failure may be caused by file being fragmented. - if (fileInfo.fileMap() != null && !fileInfo.fileMap().ranges().isEmpty()) { + if (fileInfo != null && fileInfo.fileMap() != null && !fileInfo.fileMap().ranges().isEmpty()) { IgfsEntryInfo newInfo = igfsCtx.meta().info(fileInfo.id()); // File was deleted. @@ -459,7 +487,7 @@ private byte[] block(long blockIdx) throws IOException, IgniteCheckedException { prevBlockIdx = blockIdx; - bytesFut = dataBlock(fileInfo, blockIdx); + bytesFut = dataBlock(blockIdx); assert bytesFut != null; @@ -470,10 +498,10 @@ private byte[] block(long blockIdx) throws IOException, IgniteCheckedException { if (prefetchBlocks > 0 && seqReads >= seqReadsBeforePrefetch - 1) { for (int i = 1; i <= prefetchBlocks; i++) { // Ensure that we do not prefetch over file size. - if (fileInfo.blockSize() * (i + blockIdx) >= fileInfo.length()) + if (blockSize * (i + blockIdx) >= len) break; else if (locCache.get(blockIdx + i) == null) - addLocalCacheFuture(blockIdx + i, dataBlock(fileInfo, blockIdx + i)); + addLocalCacheFuture(blockIdx + i, dataBlock(blockIdx + i)); } } @@ -483,17 +511,17 @@ else if (locCache.get(blockIdx + i) == null) throw new IgfsCorruptedFileException("Failed to retrieve file's data block (corrupted file?) " + "[path=" + path + ", blockIdx=" + blockIdx + ']'); - int blockSize = fileInfo.blockSize(); + int blockSize0 = blockSize; - if (blockIdx == fileInfo.blocksCount() - 1) - blockSize = (int)(fileInfo.length() % blockSize); + if (blockIdx == blocksCnt - 1) + blockSize0 = (int)(len % blockSize0); // If part of the file was reserved for writing, but was not actually written. - if (bytes.length < blockSize) + if (bytes.length < blockSize0) throw new IOException("Inconsistent file's data block (incorrectly written?)" + " [path=" + path + ", blockIdx=" + blockIdx + ", blockSize=" + bytes.length + - ", expectedBlockSize=" + blockSize + ", fileBlockSize=" + fileInfo.blockSize() + - ", fileLen=" + fileInfo.length() + ']'); + ", expectedBlockSize=" + blockSize0 + ", fileBlockSize=" + blockSize + + ", fileLen=" + len + ']'); return bytes; } @@ -538,14 +566,35 @@ public void apply(IgniteInternalFuture t) { /** * Get data block for specified block index. * - * @param fileInfo File info. * @param blockIdx Block index. * @return Requested data block or {@code null} if nothing found. * @throws IgniteCheckedException If failed. */ - @Nullable protected IgniteInternalFuture dataBlock(IgfsEntryInfo fileInfo, long blockIdx) + @Nullable protected IgniteInternalFuture dataBlock(final long blockIdx) throws IgniteCheckedException { - return igfsCtx.data().dataBlock(fileInfo, path, blockIdx, secReader); + if (proxy) { + assert secReader != null; + + final GridFutureAdapter fut = new GridFutureAdapter<>(); + + igfsCtx.runInIgfsThreadPool(new Runnable() { + @Override public void run() { + try { + fut.onDone(igfsCtx.data().secondaryDataBlock(path, blockIdx, secReader, blockSize)); + } + catch (Throwable e) { + fut.onDone(null, e); + } + } + }); + + return fut; + } + else { + assert fileInfo != null; + + return igfsCtx.data().dataBlock(fileInfo, path, blockIdx, secReader); + } } /** {@inheritDoc} */ From 135f0a8a39fb6895fada18d210260deebfb9426d Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Wed, 21 Sep 2016 10:33:11 +0300 Subject: [PATCH 149/487] Added missing header to BinaryObjectToStringSelfTest. --- .../binary/BinaryObjectToStringSelfTest.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectToStringSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectToStringSelfTest.java index cc6cf8bd226b7..df6bcde0fdf8e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectToStringSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectToStringSelfTest.java @@ -1,3 +1,20 @@ +/* + * 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.binary; import java.util.Arrays; From 4ee52f0a50d9cf8bc64a277f2d02600a832d6ca6 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Wed, 21 Sep 2016 15:37:52 +0700 Subject: [PATCH 150/487] IGNITE-3936 Added check for already processed key types on load cache. Added info message about started/finished load cache. Improved exceptions messages. --- .../store/jdbc/CacheAbstractJdbcStore.java | 42 ++++++++++++------- .../cache/store/jdbc/CacheJdbcPojoStore.java | 5 ++- 2 files changed, 29 insertions(+), 18 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java index a33a1e6625f86..fe8a50b5460bd 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java @@ -827,17 +827,31 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { throw new CacheLoaderException("Provided key type is not found in store or cache configuration " + "[cache=" + U.maskName(cacheName) + ", key=" + keyType + "]"); - String selQry = args[i + 1].toString(); + String qry = args[i + 1].toString(); EntryMapping em = entryMapping(cacheName, typeIdForTypeName(kindForName(keyType), keyType)); - futs.add(pool.submit(new LoadCacheCustomQueryWorker<>(em, selQry, clo))); + if (log.isInfoEnabled()) + log.info("Started load cache using custom query [cache=" + U.maskName(cacheName) + + ", keyType=" + keyType + ", query=" + qry + "]"); + + futs.add(pool.submit(new LoadCacheCustomQueryWorker<>(em, qry, clo))); } } else { - Collection entryMappings = mappings.values(); + Collection processedKeyTypes = new HashSet<>(); + + for (EntryMapping em : mappings.values()) { + String keyType = em.keyType(); + + if (processedKeyTypes.contains(keyType)) + continue; + + processedKeyTypes.add(keyType); + + if (log.isInfoEnabled()) + log.info("Started load cache [cache=" + U.maskName(cacheName) + ", keyType=" + keyType + "]"); - for (EntryMapping em : entryMappings) { if (parallelLoadCacheMinThreshold > 0) { Connection conn = null; @@ -853,7 +867,7 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { if (rs.next()) { if (log.isDebugEnabled()) log.debug("Multithread loading entries from db [cache=" + U.maskName(cacheName) + - ", keyType=" + em.keyType() + " ]"); + ", keyType=" + keyType + "]"); int keyCnt = em.keyCols.size(); @@ -876,13 +890,13 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { } futs.add(pool.submit(loadCacheRange(em, clo, upperBound, null, 0))); - - continue; } + + continue; } catch (SQLException e) { - log.warning("Failed to load entries from db in multithreaded mode " + - "[cache=" + U.maskName(cacheName) + ", keyType=" + em.keyType() + " ]", e); + log.warning("Failed to load entries from db in multithreaded mode, will try in single thread " + + "[cache=" + U.maskName(cacheName) + ", keyType=" + keyType + " ]", e); } finally { U.closeQuiet(conn); @@ -891,7 +905,7 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { if (log.isDebugEnabled()) log.debug("Single thread loading entries from db [cache=" + U.maskName(cacheName) + - ", keyType=" + em.keyType() + " ]"); + ", keyType=" + keyType + "]"); futs.add(pool.submit(loadCacheFull(em, clo))); } @@ -900,8 +914,8 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { for (Future fut : futs) U.get(fut); - if (log.isDebugEnabled()) - log.debug("Cache loaded from db: " + U.maskName(cacheName)); + if (log.isInfoEnabled()) + log.info("Finished load cache: " + U.maskName(cacheName)); } catch (IgniteCheckedException e) { throw new CacheLoaderException("Failed to load cache: " + U.maskName(cacheName), e.getCause()); @@ -1941,10 +1955,6 @@ private LoadCacheCustomQueryWorker(EntryMapping em, String qry, IgniteBiInClosur /** {@inheritDoc} */ @Override public Void call() throws Exception { - if (log.isDebugEnabled()) - log.debug("Load cache using custom query [cache= " + U.maskName(em.cacheName) + - ", keyType=" + em.keyType() + ", query=" + qry + "]"); - Connection conn = null; PreparedStatement stmt = null; diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java index 798b84a8acba2..dd3e812d7d979 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java @@ -102,7 +102,8 @@ public class CacheJdbcPojoStore extends CacheAbstractJdbcStore { return prop.get(obj); } catch (Exception e) { - throw new CacheException("Failed to read object of class: " + typeName, e); + throw new CacheException("Failed to read object property [cache=" + U.maskName(cacheName) + + ", type=" + typeName + ", prop=" + fldName + "]", e); } } @@ -262,7 +263,7 @@ protected Object buildBinaryObject(String typeName, JdbcTypeField[] fields, return builder.build(); } catch (SQLException e) { - throw new CacheException("Failed to read binary object", e); + throw new CacheException("Failed to read binary object: " + typeName, e); } } From 16b82b77f00dff8e525c8cc68d3387de107c78d1 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Wed, 21 Sep 2016 12:35:07 +0300 Subject: [PATCH 151/487] IGNITE-3635: Additional fix for stack overflow in binary objects. --- .../org/apache/ignite/internal/binary/BinaryObjectExImpl.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java index e6df4074bdb02..063bd837ac2cb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java @@ -30,6 +30,7 @@ import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl; import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory; import org.apache.ignite.internal.util.typedef.internal.SB; +import org.apache.ignite.lang.IgniteUuid; import org.jetbrains.annotations.Nullable; /** @@ -248,6 +249,8 @@ else if (val instanceof boolean[]) buf.a(Arrays.toString((boolean[]) val)); else if (val instanceof BigDecimal[]) buf.a(Arrays.toString((BigDecimal[])val)); + else if (val instanceof IgniteUuid) + buf.a(val); else if (val instanceof BinaryObjectExImpl) { BinaryObjectExImpl po = (BinaryObjectExImpl)val; From e3827a40bcbd54664cc0688e8e6ec57e4e328172 Mon Sep 17 00:00:00 2001 From: Valentin Kulichenko Date: Wed, 21 Sep 2016 16:07:02 -0700 Subject: [PATCH 152/487] IGNITE-3892 - Fixed bug in BinaryWriterExImpl.doWriteClass() method --- .../internal/binary/BinaryWriterExImpl.java | 21 ++++---- .../binary/BinaryMarshallerSelfTest.java | 53 +++++++++++++++---- .../marshaller/MarshallerContextTestImpl.java | 28 ++++++++-- 3 files changed, 77 insertions(+), 25 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java index 21fb2bfe5033d..1a818199b9944 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java @@ -17,16 +17,6 @@ package org.apache.ignite.internal.binary; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.binary.BinaryRawWriter; -import org.apache.ignite.binary.BinaryWriter; -import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream; -import org.apache.ignite.internal.binary.streams.BinaryOutputStream; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.internal.util.typedef.internal.A; -import org.jetbrains.annotations.Nullable; - import java.io.IOException; import java.io.ObjectOutput; import java.lang.reflect.InvocationHandler; @@ -38,6 +28,15 @@ import java.util.Date; import java.util.Map; import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryRawWriter; +import org.apache.ignite.binary.BinaryWriter; +import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream; +import org.apache.ignite.internal.binary.streams.BinaryOutputStream; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.internal.util.typedef.internal.A; +import org.jetbrains.annotations.Nullable; import static java.nio.charset.StandardCharsets.UTF_8; @@ -823,7 +822,7 @@ void doWriteClass(@Nullable Class val) { else { out.unsafeWriteInt(GridBinaryMarshaller.UNREGISTERED_TYPE_ID); - doWriteString(val.getClass().getName()); + doWriteString(val.getName()); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java index b347ec0590030..f4154724bfd9a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java @@ -2648,7 +2648,7 @@ public void testThreadLocalArrayReleased() throws Exception { */ public void testDuplicateNameSimpleNameMapper() throws Exception { BinaryMarshaller marsh = binaryMarshaller(new BinaryBasicNameMapper(true), - new BinaryBasicIdMapper(true), null, null); + new BinaryBasicIdMapper(true), null, null, null); Test1.Job job1 = new Test1().new Job(); Test2.Job job2 = new Test2().new Job(); @@ -2672,7 +2672,7 @@ public void testDuplicateNameSimpleNameMapper() throws Exception { */ public void testDuplicateNameFullNameMapper() throws Exception { BinaryMarshaller marsh = binaryMarshaller(new BinaryBasicNameMapper(false), - new BinaryBasicIdMapper(false), null, null); + new BinaryBasicIdMapper(false), null, null, null); Test1.Job job1 = new Test1().new Job(); Test2.Job job2 = new Test2().new Job(); @@ -2910,6 +2910,17 @@ public void testSingleHandle() throws Exception { assertEquals(SingleHandleB.class, innerBo.deserialize().getClass()); } + /** + * @throws Exception If failed. + */ + public void testUnregisteredClass() throws Exception { + BinaryMarshaller m = binaryMarshaller(null, Collections.singletonList(Value.class.getName())); + + ClassFieldObject res = m.unmarshal(m.marshal(new ClassFieldObject(Value.class)), null); + + assertEquals(Value.class, res.cls); + } + /** * */ @@ -3227,9 +3238,8 @@ protected BinaryContext binaryContext(BinaryMarshaller marsh) { /** * */ - protected BinaryMarshaller binaryMarshaller() - throws IgniteCheckedException { - return binaryMarshaller(null, null, null, null); + protected BinaryMarshaller binaryMarshaller() throws IgniteCheckedException { + return binaryMarshaller(null, null, null, null, null); } /** @@ -3237,7 +3247,15 @@ protected BinaryMarshaller binaryMarshaller() */ protected BinaryMarshaller binaryMarshaller(Collection cfgs) throws IgniteCheckedException { - return binaryMarshaller(null, null, null, cfgs); + return binaryMarshaller(null, null, null, cfgs, null); + } + + /** + * + */ + protected BinaryMarshaller binaryMarshaller(Collection cfgs, + Collection excludedClasses) throws IgniteCheckedException { + return binaryMarshaller(null, null, null, cfgs, excludedClasses); } /** @@ -3246,7 +3264,7 @@ protected BinaryMarshaller binaryMarshaller(Collection protected BinaryMarshaller binaryMarshaller(BinaryNameMapper nameMapper, BinaryIdMapper mapper, Collection cfgs) throws IgniteCheckedException { - return binaryMarshaller(nameMapper, mapper, null, cfgs); + return binaryMarshaller(nameMapper, mapper, null, cfgs, null); } /** @@ -3254,7 +3272,7 @@ protected BinaryMarshaller binaryMarshaller(BinaryNameMapper nameMapper, BinaryI */ protected BinaryMarshaller binaryMarshaller(BinarySerializer serializer, Collection cfgs) throws IgniteCheckedException { - return binaryMarshaller(null, null, serializer, cfgs); + return binaryMarshaller(null, null, serializer, cfgs, null); } /** @@ -3264,7 +3282,8 @@ protected BinaryMarshaller binaryMarshaller( BinaryNameMapper nameMapper, BinaryIdMapper mapper, BinarySerializer serializer, - Collection cfgs + Collection cfgs, + Collection excludedClasses ) throws IgniteCheckedException { IgniteConfiguration iCfg = new IgniteConfiguration(); @@ -3283,7 +3302,7 @@ protected BinaryMarshaller binaryMarshaller( BinaryMarshaller marsh = new BinaryMarshaller(); - marsh.setContext(new MarshallerContextTestImpl(null)); + marsh.setContext(new MarshallerContextTestImpl(null, excludedClasses)); IgniteUtils.invoke(BinaryMarshaller.class, marsh, "setBinaryContext", ctx, iCfg); @@ -4885,4 +4904,18 @@ private static class SingleHandleA { /** */ private static class SingleHandleB {} + + /** + */ + private static class ClassFieldObject { + /** */ + private Class cls; + + /** + * @param cls Class field. + */ + public ClassFieldObject(Class cls) { + this.cls = cls; + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextTestImpl.java b/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextTestImpl.java index c600ca46d98a4..9ff127d245613 100644 --- a/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextTestImpl.java +++ b/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextTestImpl.java @@ -17,11 +17,13 @@ package org.apache.ignite.marshaller; +import java.util.Collection; import java.util.List; import java.util.concurrent.ConcurrentMap; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.MarshallerContextAdapter; import org.apache.ignite.plugin.PluginProvider; +import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; /** @@ -29,26 +31,44 @@ */ public class MarshallerContextTestImpl extends MarshallerContextAdapter { /** */ - private final static ConcurrentMap map = new ConcurrentHashMap8<>(); + private static final ConcurrentMap map = new ConcurrentHashMap8<>(); + + /** */ + private final Collection excluded; /** * Initializes context. * * @param plugins Plugins. + * @param excluded Excluded classes. */ - public MarshallerContextTestImpl(List plugins) { + public MarshallerContextTestImpl(@Nullable List plugins, Collection excluded) { super(plugins); + + this.excluded = excluded; + } + + /** + * Initializes context. + * + * @param plugins Plugins. + */ + public MarshallerContextTestImpl(List plugins) { + this(plugins, null); } /** * Initializes context. */ public MarshallerContextTestImpl() { - super(null); + this(null); } /** {@inheritDoc} */ @Override protected boolean registerClassName(int id, String clsName) throws IgniteCheckedException { + if (excluded != null && excluded.contains(clsName)) + return false; + String oldClsName = map.putIfAbsent(id, clsName); if (oldClsName != null && !oldClsName.equals(clsName)) @@ -69,4 +89,4 @@ public MarshallerContextTestImpl() { public ConcurrentMap internalMap() { return map; } -} \ No newline at end of file +} From 0d5ee7887be03295133704227133d619898b6abf Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Thu, 22 Sep 2016 14:36:50 +0700 Subject: [PATCH 153/487] IGNITE-3937 implemented support for unsigned types in MySQL. --- .../apache/ignite/schema/parser/DbColumn.java | 16 +++++++++++-- .../dialect/DatabaseMetadataDialect.java | 9 +++++++ .../parser/dialect/JdbcMetadataDialect.java | 23 +++++++++++------- .../parser/dialect/MySQLMetadataDialect.java | 24 +++++++++++++++++-- .../parser/dialect/OracleMetadataDialect.java | 5 ++-- .../ignite/schema/model/PojoDescriptor.java | 24 ++++++++++++------- 6 files changed, 77 insertions(+), 24 deletions(-) diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DbColumn.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DbColumn.java index 8b0c813467c80..10fd50f94cc51 100644 --- a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DbColumn.java +++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DbColumn.java @@ -33,17 +33,22 @@ public class DbColumn { /** Is {@code NULL} allowed for column in database. */ private final boolean nullable; + /** Whether column unsigned. */ + private final boolean unsigned; + /** * @param name Column name. * @param type Column JDBC type. * @param key {@code true} if this column belongs to primary key. * @param nullable {@code true} if {@code NULL } allowed for column in database. + * @param unsigned {@code true} if column is unsigned. */ - public DbColumn(String name, int type, boolean key, boolean nullable) { + public DbColumn(String name, int type, boolean key, boolean nullable, boolean unsigned) { this.name = name; this.type = type; this.key = key; this.nullable = nullable; + this.unsigned = unsigned; } /** @@ -68,9 +73,16 @@ public boolean key() { } /** - * @return nullable {@code true} if {@code NULL } allowed for column in database. + * @return {@code true} if {@code NULL } allowed for column in database. */ public boolean nullable() { return nullable; } + + /** + * @return {@code true} if column is unsigned. + */ + public boolean unsigned() { + return unsigned; + } } diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java index 5ef088ad1799c..56ee59b7732df 100644 --- a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java +++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java @@ -18,6 +18,7 @@ package org.apache.ignite.schema.parser.dialect; import java.sql.Connection; +import java.sql.DatabaseMetaData; import java.sql.SQLException; import java.util.Collection; import java.util.Collections; @@ -62,6 +63,14 @@ public Set systemSchemas() { return Collections.singleton("INFORMATION_SCHEMA"); } + /** + * @return Collection of unsigned type names. + * @throws SQLException If failed to get unsigned type names. + */ + public Set unsignedTypes(DatabaseMetaData dbMeta) throws SQLException { + return Collections.emptySet(); + } + /** * Create table descriptor. * diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java index 235c84c3edfd0..f6c27f29fc60b 100644 --- a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java +++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java @@ -30,7 +30,6 @@ import java.util.Set; import org.apache.ignite.cache.QueryIndex; -import org.apache.ignite.cache.QueryIndexType; import org.apache.ignite.schema.parser.DbColumn; import org.apache.ignite.schema.parser.DbTable; @@ -62,6 +61,9 @@ public class JdbcMetadataDialect extends DatabaseMetadataDialect { /** Column data type index. */ private static final int COL_DATA_TYPE_IDX = 5; + /** Column type name index. */ + private static final int COL_TYPE_NAME_IDX = 6; + /** Column nullable index. */ private static final int COL_NULLABLE_IDX = 11; @@ -116,11 +118,11 @@ protected boolean useSchema() { Set sys = systemSchemas(); - Collection tbls = new ArrayList<>(); - if (schemas.isEmpty()) schemas.add(null); + Collection tbls = new ArrayList<>(); + for (String toSchema: schemas) { try (ResultSet tblsRs = dbMeta.getTables(useCatalog() ? toSchema : null, useSchema() ? toSchema : null, "%", tblsOnly ? TABLES_ONLY : TABLES_AND_VIEWS)) { @@ -136,24 +138,27 @@ protected boolean useSchema() { if (sys.contains(schema)) continue; - Set pkCols = new HashSet<>(); + Collection pkCols = new HashSet<>(); try (ResultSet pkRs = dbMeta.getPrimaryKeys(tblCatalog, tblSchema, tblName)) { while (pkRs.next()) pkCols.add(pkRs.getString(PK_COL_NAME_IDX)); } - List cols = new ArrayList<>(); + Collection cols = new ArrayList<>(); + + Collection unsignedTypes = unsignedTypes(dbMeta); try (ResultSet colsRs = dbMeta.getColumns(tblCatalog, tblSchema, tblName, null)) { while (colsRs.next()) { String colName = colsRs.getString(COL_NAME_IDX); cols.add(new DbColumn( - colName, - colsRs.getInt(COL_DATA_TYPE_IDX), - pkCols.contains(colName), - colsRs.getInt(COL_NULLABLE_IDX) == DatabaseMetaData.columnNullable)); + colName, + colsRs.getInt(COL_DATA_TYPE_IDX), + pkCols.contains(colName), + colsRs.getInt(COL_NULLABLE_IDX) == DatabaseMetaData.columnNullable, + unsignedTypes.contains(colsRs.getString(COL_TYPE_NAME_IDX)))); } } diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/MySQLMetadataDialect.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/MySQLMetadataDialect.java index 3332046da5226..7bd6f311c6495 100644 --- a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/MySQLMetadataDialect.java +++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/MySQLMetadataDialect.java @@ -18,20 +18,24 @@ package org.apache.ignite.schema.parser.dialect; import java.sql.Connection; +import java.sql.DatabaseMetaData; import java.sql.ResultSet; import java.sql.SQLException; import java.util.ArrayList; import java.util.Collection; -import java.util.List; +import java.util.HashSet; import java.util.Set; /** * MySQL specific metadata dialect. */ public class MySQLMetadataDialect extends JdbcMetadataDialect { + /** Type name index. */ + private static final int TYPE_NAME_IDX = 1; + /** {@inheritDoc} */ @Override public Collection schemas(Connection conn) throws SQLException { - List schemas = new ArrayList<>(); + Collection schemas = new ArrayList<>(); ResultSet rs = conn.getMetaData().getCatalogs(); @@ -59,4 +63,20 @@ public class MySQLMetadataDialect extends JdbcMetadataDialect { @Override protected boolean useSchema() { return false; } + + /** {@inheritDoc} */ + @Override public Set unsignedTypes(DatabaseMetaData dbMeta) throws SQLException { + Set unsignedTypes = new HashSet<>(); + + try (ResultSet typeRs = dbMeta.getTypeInfo()) { + while (typeRs.next()) { + String typeName = typeRs.getString(TYPE_NAME_IDX); + + if (typeName.contains("UNSIGNED")) + unsignedTypes.add(typeName); + } + } + + return unsignedTypes; + } } diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java index 35356558e91d2..47fb05c1146e3 100644 --- a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java +++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java @@ -258,8 +258,7 @@ private Set primaryKeys(PreparedStatement stmt, String owner, String tbl * @return Indexes. * @throws SQLException If failed to retrieve indexes columns. */ - private Collection indexes(PreparedStatement stmt, String owner, String tbl) - throws SQLException { + private Collection indexes(PreparedStatement stmt, String owner, String tbl) throws SQLException { Map idxs = new LinkedHashMap<>(); stmt.setString(1, owner); @@ -347,7 +346,7 @@ private Collection indexes(PreparedStatement stmt, String owner, Str String colName = colsRs.getString(COL_NAME_IDX); cols.add(new DbColumn(colName, decodeType(colsRs), pkCols.contains(colName), - !"N".equals(colsRs.getString(NULLABLE_IDX)))); + !"N".equals(colsRs.getString(NULLABLE_IDX)), false)); } if (!cols.isEmpty()) diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java index 5053b07a8bdb8..4f696d6c0263b 100644 --- a/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java +++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java @@ -123,7 +123,7 @@ public PojoDescriptor(PojoDescriptor prn, DbTable tbl) { String colName = col.name(); PojoField fld = new PojoField(colName, col.type(), - toJavaFieldName(colName), toJavaType(col.type(), col.nullable()).getName(), + toJavaFieldName(colName), toJavaType(col).getName(), col.key(), col.nullable()); fld.owner(this); @@ -427,24 +427,32 @@ private static String toJavaFieldName(String name) { /** * Convert JDBC data type to java type. * - * @param type JDBC SQL data type. - * @param nullable {@code true} if {@code NULL} is allowed for this field in database. + * @param col Database column descriptor. * @return Java data type. */ - private static Class toJavaType(int type, boolean nullable) { - switch (type) { + private static Class toJavaType(DbColumn col) { + boolean nullable = col.nullable(); + boolean unsigned = col.unsigned(); + + switch (col.type()) { case BIT: case BOOLEAN: return nullable ? Boolean.class : boolean.class; case TINYINT: - return nullable ? Byte.class : byte.class; + return unsigned + ? (nullable ? Short.class : short.class) + : (nullable ? Byte.class : byte.class); case SMALLINT: - return nullable ? Short.class : short.class; + return unsigned + ? (nullable ? Integer.class : int.class) + : (nullable ? Short.class : short.class); case INTEGER: - return nullable ? Integer.class : int.class; + return unsigned + ? (nullable ? Long.class : long.class) + : (nullable ? Integer.class : int.class); case BIGINT: return nullable ? Long.class : long.class; From a97483a4ce2c00bd0cca025c4ef4bfa181897aa9 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Thu, 22 Sep 2016 10:51:05 +0300 Subject: [PATCH 154/487] IGNITE-3858 IGFS: Support direct PROXY mode invocation in methods: create / append. This closes #1070. This closes #1084. --- .../igfs/IgfsAbstractOutputStream.java | 266 +++++++++++++++ .../internal/processors/igfs/IgfsImpl.java | 27 +- .../processors/igfs/IgfsOutputStreamImpl.java | 319 ++++-------------- .../igfs/IgfsOutputStreamProxyImpl.java | 163 +++++++++ .../igfs/IgfsAbstractBaseSelfTest.java | 2 +- 5 files changed, 518 insertions(+), 259 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractOutputStream.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamProxyImpl.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractOutputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractOutputStream.java new file mode 100644 index 0000000000000..c1e751ecf53bc --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractOutputStream.java @@ -0,0 +1,266 @@ +/* + * 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.igfs; + +import org.apache.ignite.events.IgfsEvent; +import org.apache.ignite.igfs.IgfsOutputStream; +import org.apache.ignite.igfs.IgfsPath; +import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager; +import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; + +import java.io.DataInput; +import java.io.IOException; +import java.nio.ByteBuffer; + +import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CLOSED_WRITE; + +/** + * Output stream to store data into grid cache with separate blocks. + */ +abstract class IgfsAbstractOutputStream extends IgfsOutputStream { + /** IGFS context. */ + protected final IgfsContext igfsCtx; + + /** Path to file. */ + protected final IgfsPath path; + + /** Buffer size. */ + protected final int bufSize; + + /** File worker batch. */ + protected final IgfsFileWorkerBatch batch; + + /** Mutex for synchronization. */ + protected final Object mux = new Object(); + + /** Flag for this stream open/closed state. */ + protected boolean closed; + + /** Local buffer to store stream data as consistent block. */ + protected ByteBuffer buf; + + /** Bytes written. */ + protected long bytes; + + /** Time consumed by write operations. */ + protected long time; + + /** + * Constructs file output stream. + * + * @param igfsCtx IGFS context. + * @param path Path to stored file. + * @param bufSize The size of the buffer to be used. + * @param batch Optional secondary file system batch. + */ + IgfsAbstractOutputStream(IgfsContext igfsCtx, IgfsPath path, int bufSize, @Nullable IgfsFileWorkerBatch batch) { + synchronized (mux) { + this.path = path; + this.bufSize = optimizeBufferSize(bufSize); + this.igfsCtx = igfsCtx; + this.batch = batch; + } + + igfsCtx.metrics().incrementFilesOpenedForWrite(); + } + + /** + * Optimize buffer size. + * + * @param bufSize Original byffer size. + * @return Optimized buffer size. + */ + protected abstract int optimizeBufferSize(int bufSize); + + /** {@inheritDoc} */ + @Override public void write(int b) throws IOException { + synchronized (mux) { + checkClosed(null, 0); + + b &= 0xFF; + + long startTime = System.nanoTime(); + + if (buf == null) + buf = allocateNewBuffer(); + + buf.put((byte)b); + + sendBufferIfFull(); + + time += System.nanoTime() - startTime; + } + } + + /** {@inheritDoc} */ + @SuppressWarnings("NullableProblems") + @Override public void write(byte[] b, int off, int len) throws IOException { + A.notNull(b, "b"); + + if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) > b.length) || ((off + len) < 0)) { + throw new IndexOutOfBoundsException("Invalid bounds [data.length=" + b.length + ", offset=" + off + + ", length=" + len + ']'); + } + + synchronized (mux) { + checkClosed(null, 0); + + // Check if there is anything to write. + if (len == 0) + return; + + long startTime = System.nanoTime(); + + if (buf == null) { + if (len >= bufSize) { + // Send data right away. + ByteBuffer tmpBuf = ByteBuffer.wrap(b, off, len); + + send(tmpBuf, tmpBuf.remaining()); + } + else { + buf = allocateNewBuffer(); + + buf.put(b, off, len); + } + } + else { + // Re-allocate buffer if needed. + if (buf.remaining() < len) + buf = ByteBuffer.allocate(buf.position() + len).put((ByteBuffer)buf.flip()); + + buf.put(b, off, len); + + sendBufferIfFull(); + } + + time += System.nanoTime() - startTime; + } + } + + /** {@inheritDoc} */ + @Override public void transferFrom(DataInput in, int len) throws IOException { + synchronized (mux) { + checkClosed(in, len); + + long startTime = System.nanoTime(); + + // Clean-up local buffer before streaming. + sendBufferIfNotEmpty(); + + // Perform transfer. + send(in, len); + + time += System.nanoTime() - startTime; + } + } + + /** + * Validate this stream is open. + * + * @param in Data input. + * @param len Data len in bytes. + * @throws IOException If this stream is closed. + */ + protected void checkClosed(@Nullable DataInput in, int len) throws IOException { + assert Thread.holdsLock(mux); + + if (closed) { + // Must read data from stream before throwing exception. + if (in != null) + in.skipBytes(len); + + throw new IOException("Stream has been closed: " + this); + } + } + + /** + * Send local buffer if it full. + * + * @throws IOException If failed. + */ + private void sendBufferIfFull() throws IOException { + if (buf.position() >= bufSize) + sendBuffer(); + } + + /** + * Send local buffer if at least something is stored there. + * + * @throws IOException If failed. + */ + void sendBufferIfNotEmpty() throws IOException { + if (buf != null && buf.position() > 0) + sendBuffer(); + } + + /** + * Send all local-buffered data to server. + * + * @throws IOException In case of IO exception. + */ + private void sendBuffer() throws IOException { + buf.flip(); + + send(buf, buf.remaining()); + + buf = null; + } + + /** + * Store data block. + * + * @param data Block. + * @param writeLen Write length. + * @throws IOException If failed. + */ + protected abstract void send(Object data, int writeLen) throws IOException; + + /** + * Allocate new buffer. + * + * @return New buffer. + */ + private ByteBuffer allocateNewBuffer() { + return ByteBuffer.allocate(bufSize); + } + + /** + * Updates IGFS metrics when the stream is closed. + */ + protected void updateMetricsOnClose() { + IgfsLocalMetrics metrics = igfsCtx.metrics(); + + metrics.addWrittenBytesTime(bytes, time); + metrics.decrementFilesOpenedForWrite(); + + GridEventStorageManager evts = igfsCtx.kernalContext().event(); + + if (evts.isRecordable(EVT_IGFS_FILE_CLOSED_WRITE)) + evts.record(new IgfsEvent(path, igfsCtx.localNode(), + EVT_IGFS_FILE_CLOSED_WRITE, bytes)); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(IgfsAbstractOutputStream.class, this); + } + +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 87a4699a4de6b..bee9d9a72a53c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -92,7 +92,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Callable; -import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -274,7 +274,7 @@ public final class IgfsImpl implements IgfsEx { } dualPool = secondaryFs != null ? new IgniteThreadPoolExecutor(4, Integer.MAX_VALUE, 5000L, - new LinkedBlockingQueue(), new IgfsThreadFactory(cfg.getName()), null) : null; + new SynchronousQueue(), new IgfsThreadFactory(cfg.getName()), null) : null; } /** {@inheritDoc} */ @@ -1088,6 +1088,17 @@ private IgfsOutputStream create0( else dirProps = fileProps = new HashMap<>(props); + if (mode == PROXY) { + assert secondaryFs != null; + + OutputStream secondaryStream = secondaryFs.create(path, bufSize, overwrite, replication, + groupBlockSize(), props); + + IgfsFileWorkerBatch batch = newBatch(path, secondaryStream); + + return new IgfsOutputStreamProxyImpl(igfsCtx, path, info(path), bufferSize(bufSize), batch); + } + // Prepare context for DUAL mode. IgfsSecondaryFileSystemCreateContext secondaryCtx = null; @@ -1142,7 +1153,15 @@ private IgfsOutputStream create0( final IgfsMode mode = resolveMode(path); - IgfsFileWorkerBatch batch; + if (mode == PROXY) { + assert secondaryFs != null; + + OutputStream secondaryStream = secondaryFs.append(path, bufSize, create, props); + + IgfsFileWorkerBatch batch = newBatch(path, secondaryStream); + + return new IgfsOutputStreamProxyImpl(igfsCtx, path, info(path), bufferSize(bufSize), batch); + } if (mode != PRIMARY) { assert IgfsUtils.isDualMode(mode); @@ -1151,7 +1170,7 @@ private IgfsOutputStream create0( IgfsCreateResult desc = meta.appendDual(secondaryFs, path, bufSize, create); - batch = newBatch(path, desc.secondaryOutputStream()); + IgfsFileWorkerBatch batch = newBatch(path, desc.secondaryOutputStream()); return new IgfsOutputStreamImpl(igfsCtx, path, desc.info(), bufferSize(bufSize), mode, batch); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java index 6dec0c1d9c957..f97624247cfb8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java @@ -18,14 +18,10 @@ package org.apache.ignite.internal.processors.igfs; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.events.IgfsEvent; import org.apache.ignite.igfs.IgfsException; import org.apache.ignite.igfs.IgfsMode; -import org.apache.ignite.igfs.IgfsOutputStream; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager; -import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; @@ -35,7 +31,6 @@ import java.io.IOException; import java.nio.ByteBuffer; -import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CLOSED_WRITE; import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC; import static org.apache.ignite.igfs.IgfsMode.PRIMARY; import static org.apache.ignite.igfs.IgfsMode.PROXY; @@ -43,57 +38,30 @@ /** * Output stream to store data into grid cache with separate blocks. */ -class IgfsOutputStreamImpl extends IgfsOutputStream { +class IgfsOutputStreamImpl extends IgfsAbstractOutputStream { /** Maximum number of blocks in buffer. */ private static final int MAX_BLOCKS_CNT = 16; - /** IGFS context. */ - private final IgfsContext igfsCtx; - - /** Path to file. */ - private final IgfsPath path; - - /** Buffer size. */ - private final int bufSize; - /** IGFS mode. */ private final IgfsMode mode; - /** File worker batch. */ - private final IgfsFileWorkerBatch batch; - - /** Mutex for synchronization. */ - private final Object mux = new Object(); - /** Write completion future. */ private final IgniteInternalFuture writeFut; - /** Flag for this stream open/closed state. */ - private boolean closed; - - /** Local buffer to store stream data as consistent block. */ - private ByteBuffer buf; - - /** Bytes written. */ - private long bytes; - - /** Time consumed by write operations. */ - private long time; - /** File descriptor. */ private IgfsEntryInfo fileInfo; - /** Space in file to write data. */ - private long space; + /** Affinity written by this output stream. */ + private IgfsFileAffinityRange streamRange; + + /** Data length in remainder. */ + protected int remainderDataLen; /** Intermediate remainder to keep data. */ private byte[] remainder; - /** Data length in remainder. */ - private int remainderDataLen; - - /** Affinity written by this output stream. */ - private IgfsFileAffinityRange streamRange; + /** Space in file to write data. */ + protected long space; /** * Constructs file output stream. @@ -107,6 +75,8 @@ class IgfsOutputStreamImpl extends IgfsOutputStream { */ IgfsOutputStreamImpl(IgfsContext igfsCtx, IgfsPath path, IgfsEntryInfo fileInfo, int bufSize, IgfsMode mode, @Nullable IgfsFileWorkerBatch batch) { + super(igfsCtx, path, bufSize, batch); + assert fileInfo != null && fileInfo.isFile() : "Unexpected file info: " + fileInfo; assert mode != null && mode != PROXY && (mode == PRIMARY && batch == null || batch != null); @@ -115,108 +85,55 @@ class IgfsOutputStreamImpl extends IgfsOutputStream { throw new IgfsException("Failed to acquire file lock (concurrently modified?): " + path); synchronized (mux) { - this.path = path; - this.bufSize = optimizeBufferSize(bufSize, fileInfo); - this.igfsCtx = igfsCtx; this.fileInfo = fileInfo; this.mode = mode; - this.batch = batch; streamRange = initialStreamRange(fileInfo); writeFut = igfsCtx.data().writeStart(fileInfo.id()); } - - igfsCtx.metrics().incrementFilesOpenedForWrite(); } - /** {@inheritDoc} */ - @Override public void write(int b) throws IOException { - synchronized (mux) { - checkClosed(null, 0); - - b &= 0xFF; - - long startTime = System.nanoTime(); - - if (buf == null) - buf = allocateNewBuffer(); - - buf.put((byte)b); - - sendBufferIfFull(); - - time += System.nanoTime() - startTime; - } + /** + * @return Length of file. + */ + private long length() { + return fileInfo.length(); } /** {@inheritDoc} */ - @SuppressWarnings("NullableProblems") - @Override public void write(byte[] b, int off, int len) throws IOException { - A.notNull(b, "b"); - - if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) > b.length) || ((off + len) < 0)) { - throw new IndexOutOfBoundsException("Invalid bounds [data.length=" + b.length + ", offset=" + off + - ", length=" + len + ']'); - } - - synchronized (mux) { - checkClosed(null, 0); - - // Check if there is anything to write. - if (len == 0) - return; - - long startTime = System.nanoTime(); - - if (buf == null) { - if (len >= bufSize) { - // Send data right away. - ByteBuffer tmpBuf = ByteBuffer.wrap(b, off, len); - - send(tmpBuf, tmpBuf.remaining()); - } - else { - buf = allocateNewBuffer(); - - buf.put(b, off, len); - } - } - else { - // Re-allocate buffer if needed. - if (buf.remaining() < len) - buf = ByteBuffer.allocate(buf.position() + len).put((ByteBuffer)buf.flip()); + @Override protected int optimizeBufferSize(int bufSize) { + assert bufSize > 0; - buf.put(b, off, len); + if (fileInfo == null) + return bufSize; - sendBufferIfFull(); - } + int blockSize = fileInfo.blockSize(); - time += System.nanoTime() - startTime; - } - } + if (blockSize <= 0) + return bufSize; - /** {@inheritDoc} */ - @Override public void transferFrom(DataInput in, int len) throws IOException { - synchronized (mux) { - checkClosed(in, len); + if (bufSize <= blockSize) + // Optimize minimum buffer size to be equal file's block size. + return blockSize; - long startTime = System.nanoTime(); + int maxBufSize = blockSize * MAX_BLOCKS_CNT; - // Clean-up local buffer before streaming. - sendBufferIfNotEmpty(); + if (bufSize > maxBufSize) + // There is no profit or optimization from larger buffers. + return maxBufSize; - // Perform transfer. - send(in, len); + if (fileInfo.length() == 0) + // Make buffer size multiple of block size (optimized for new files). + return bufSize / blockSize * blockSize; - time += System.nanoTime() - startTime; - } + return bufSize; } /** * Flushes this output stream and forces any buffered output bytes to be written out. * - * @exception IOException if an I/O error occurs. + * @throws IOException if an I/O error occurs. */ @Override public void flush() throws IOException { synchronized (mux) { @@ -250,40 +167,6 @@ class IgfsOutputStreamImpl extends IgfsOutputStream { } } - /** - * Await acknowledgments. - * - * @throws IOException If failed. - */ - private void awaitAcks() throws IOException { - try { - igfsCtx.data().awaitAllAcksReceived(fileInfo.id()); - } - catch (IgniteCheckedException e) { - throw new IOException("Failed to wait for flush acknowledge: " + fileInfo.id, e); - } - } - - /** - * Flush remainder. - * - * @throws IOException If failed. - */ - private void flushRemainder() throws IOException { - try { - if (remainder != null) { - igfsCtx.data().storeDataBlocks(fileInfo, fileInfo.length() + space, null, 0, - ByteBuffer.wrap(remainder, 0, remainderDataLen), true, streamRange, batch); - - remainder = null; - remainderDataLen = 0; - } - } - catch (IgniteCheckedException e) { - throw new IOException("Failed to flush data (remainder) [path=" + path + ", space=" + space + ']', e); - } - } - /** {@inheritDoc} */ @Override public final void close() throws IOException { synchronized (mux) { @@ -355,75 +238,33 @@ private void flushRemainder() throws IOException { if (err != null) throw err; - igfsCtx.metrics().addWrittenBytesTime(bytes, time); - igfsCtx.metrics().decrementFilesOpenedForWrite(); - - GridEventStorageManager evts = igfsCtx.kernalContext().event(); - - if (evts.isRecordable(EVT_IGFS_FILE_CLOSED_WRITE)) - evts.record(new IgfsEvent(path, igfsCtx.kernalContext().discovery().localNode(), - EVT_IGFS_FILE_CLOSED_WRITE, bytes)); - } - } - - /** - * Validate this stream is open. - * - * @throws IOException If this stream is closed. - */ - private void checkClosed(@Nullable DataInput in, int len) throws IOException { - assert Thread.holdsLock(mux); - - if (closed) { - // Must read data from stream before throwing exception. - if (in != null) - in.skipBytes(len); - - throw new IOException("Stream has been closed: " + this); + updateMetricsOnClose(); } } /** - * Send local buffer if it full. - * - * @throws IOException If failed. - */ - private void sendBufferIfFull() throws IOException { - if (buf.position() >= bufSize) - sendBuffer(); - } - - /** - * Send local buffer if at least something is stored there. + * Flush remainder. * * @throws IOException If failed. */ - private void sendBufferIfNotEmpty() throws IOException { - if (buf != null && buf.position() > 0) - sendBuffer(); - } - - /** - * Send all local-buffered data to server. - * - * @throws IOException In case of IO exception. - */ - private void sendBuffer() throws IOException { - buf.flip(); + private void flushRemainder() throws IOException { + try { + if (remainder != null) { - send(buf, buf.remaining()); + remainder = igfsCtx.data().storeDataBlocks(fileInfo, length() + space, null, + 0, ByteBuffer.wrap(remainder, 0, remainderDataLen), true, streamRange, batch); - buf = null; + remainder = null; + remainderDataLen = 0; + } + } + catch (IgniteCheckedException e) { + throw new IOException("Failed to flush data (remainder) [path=" + path + ", space=" + space + ']', e); + } } - /** - * Store data block. - * - * @param data Block. - * @param writeLen Write length. - * @throws IOException If failed. - */ - private void send(Object data, int writeLen) throws IOException { + /** {@inheritDoc} */ + @Override protected void send(Object data, int writeLen) throws IOException { assert Thread.holdsLock(mux); assert data instanceof ByteBuffer || data instanceof DataInput; @@ -449,20 +290,20 @@ else if (remainder.length != blockSize) { } if (data instanceof ByteBuffer) - ((ByteBuffer) data).get(remainder, remainderDataLen, writeLen); + ((ByteBuffer)data).get(remainder, remainderDataLen, writeLen); else - ((DataInput) data).readFully(remainder, remainderDataLen, writeLen); + ((DataInput)data).readFully(remainder, remainderDataLen, writeLen); remainderDataLen += writeLen; } else { if (data instanceof ByteBuffer) { - remainder = igfsCtx.data().storeDataBlocks(fileInfo, fileInfo.length() + space, remainder, - remainderDataLen, (ByteBuffer) data, false, streamRange, batch); + remainder = igfsCtx.data().storeDataBlocks(fileInfo, length() + space, remainder, + remainderDataLen, (ByteBuffer)data, false, streamRange, batch); } else { - remainder = igfsCtx.data().storeDataBlocks(fileInfo, fileInfo.length() + space, remainder, - remainderDataLen, (DataInput) data, writeLen, false, streamRange, batch); + remainder = igfsCtx.data().storeDataBlocks(fileInfo, length() + space, remainder, + remainderDataLen, (DataInput)data, writeLen, false, streamRange, batch); } remainderDataLen = remainder == null ? 0 : remainder.length; @@ -474,12 +315,17 @@ else if (remainder.length != blockSize) { } /** - * Allocate new buffer. + * Await acknowledgments. * - * @return New buffer. + * @throws IOException If failed. */ - private ByteBuffer allocateNewBuffer() { - return ByteBuffer.allocate(bufSize); + private void awaitAcks() throws IOException { + try { + igfsCtx.data().awaitAllAcksReceived(fileInfo.id()); + } + catch (IgniteCheckedException e) { + throw new IOException("Failed to wait for flush acknowledge: " + fileInfo.id, e); + } } /** @@ -516,41 +362,6 @@ private IgfsFileAffinityRange initialStreamRange(IgfsEntryInfo fileInfo) { return affKey == null ? null : new IgfsFileAffinityRange(off, off, affKey); } - /** - * Optimize buffer size. - * - * @param bufSize Requested buffer size. - * @param fileInfo File info. - * @return Optimized buffer size. - */ - private static int optimizeBufferSize(int bufSize, IgfsEntryInfo fileInfo) { - assert bufSize > 0; - - if (fileInfo == null) - return bufSize; - - int blockSize = fileInfo.blockSize(); - - if (blockSize <= 0) - return bufSize; - - if (bufSize <= blockSize) - // Optimize minimum buffer size to be equal file's block size. - return blockSize; - - int maxBufSize = blockSize * MAX_BLOCKS_CNT; - - if (bufSize > maxBufSize) - // There is no profit or optimization from larger buffers. - return maxBufSize; - - if (fileInfo.length() == 0) - // Make buffer size multiple of block size (optimized for new files). - return bufSize / blockSize * blockSize; - - return bufSize; - } - /** {@inheritDoc} */ @Override public String toString() { return S.toString(IgfsOutputStreamImpl.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamProxyImpl.java new file mode 100644 index 0000000000000..7b74a1f00cf64 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamProxyImpl.java @@ -0,0 +1,163 @@ +/* + * 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.igfs; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.igfs.IgfsFile; +import org.apache.ignite.igfs.IgfsPath; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; + +import java.io.DataInput; +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Output stream to store data into grid cache with separate blocks. + */ +class IgfsOutputStreamProxyImpl extends IgfsAbstractOutputStream { + /** File info. */ + private IgfsFile info; + + /** + * Constructs file output stream. + * + * @param igfsCtx IGFS context. + * @param path Path to stored file. + * @param info File info. + * @param bufSize The size of the buffer to be used. + * @param batch Optional secondary file system batch. + */ + IgfsOutputStreamProxyImpl(IgfsContext igfsCtx, IgfsPath path, IgfsFile info, int bufSize, + @Nullable IgfsFileWorkerBatch batch) { + super(igfsCtx, path, bufSize, batch); + + assert batch != null; + + this.info = info; + } + + /** {@inheritDoc} */ + @Override protected int optimizeBufferSize(int bufSize) { + assert bufSize > 0; + + return bufSize; + } + + /** + * Flushes this output stream and forces any buffered output bytes to be written out. + * + * @throws IOException if an I/O error occurs. + */ + @Override public void flush() throws IOException { + synchronized (mux) { + checkClosed(null, 0); + + sendBufferIfNotEmpty(); + } + } + + /** {@inheritDoc} */ + @Override public final void close() throws IOException { + synchronized (mux) { + // Do nothing if stream is already closed. + if (closed) + return; + + // Set closed flag immediately. + closed = true; + + // Flush data. + IOException err = null; + + try { + sendBufferIfNotEmpty(); + } + catch (Exception e) { + err = new IOException("Failed to flush data during stream close [path=" + path + + ", fileInfo=" + info + ']', e); + } + + // Finish batch before file unlocking to support the assertion that unlocked file batch, + // if any, must be in finishing state (e.g. append see more IgfsImpl.newBatch) + batch.finish(); + + // Finally, await secondary file system flush. + try { + batch.await(); + } + catch (IgniteCheckedException e) { + if (err == null) + err = new IOException("Failed to close secondary file system stream [path=" + path + + ", fileInfo=" + info + ']', e); + else + err.addSuppressed(e); + } + + // Throw error, if any. + if (err != null) + throw err; + + updateMetricsOnClose(); + } + } + + /** {@inheritDoc} */ + @Override protected void send(Object data, int writeLen) throws IOException { + assert Thread.holdsLock(mux); + assert data instanceof ByteBuffer || data instanceof DataInput; + + try { + // Increment metrics. + bytes += writeLen; + + byte [] dataBuf = new byte[writeLen]; + + if (data instanceof ByteBuffer) { + ByteBuffer byteBuf = (ByteBuffer)data; + + byteBuf.get(dataBuf); + } + else { + DataInput dataIn = (DataInput)data; + + try { + dataIn.readFully(dataBuf); + } + catch (IOException e) { + throw new IgniteCheckedException(e); + } + } + + if (!batch.write(dataBuf)) + throw new IgniteCheckedException("Cannot write more data to the secondary file system output " + + "stream because it was marked as closed: " + batch.path()); + else + igfsCtx.metrics().addWriteBlocks(1, 1); + + } + catch (IgniteCheckedException e) { + throw new IOException("Failed to store data into file: " + path, e); + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(IgfsOutputStreamProxyImpl.class, this); + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java index 3f62cf55013c4..14a653bf83b68 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java @@ -904,7 +904,7 @@ protected static IgfsPath[] paths(IgfsPath... paths) { protected void clear(IgniteFileSystem igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary) throws Exception { clear(igfs); - if (dual) + if (mode != PRIMARY) clear(igfsSecondary); } From 63888bd99b8a84cc961b226efba9964dcc292f4c Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Thu, 22 Sep 2016 16:22:40 +0700 Subject: [PATCH 155/487] IGNITE-3938 Implemented escaping for SQL identifiers. Added tests for escaped mode. --- .../store/jdbc/CacheAbstractJdbcStore.java | 104 +++++++++++++++--- .../store/jdbc/CacheJdbcPojoStoreFactory.java | 46 ++++++-- .../store/jdbc/dialect/BasicJdbcDialect.java | 8 +- .../cache/store/jdbc/dialect/DB2Dialect.java | 6 +- .../cache/store/jdbc/dialect/H2Dialect.java | 5 +- .../cache/store/jdbc/dialect/JdbcDialect.java | 18 ++- .../store/jdbc/dialect/MySQLDialect.java | 9 +- .../store/jdbc/dialect/OracleDialect.java | 5 +- .../store/jdbc/dialect/SQLServerDialect.java | 14 ++- .../CacheJdbcPojoStoreAbstractSelfTest.java | 48 +++++--- ...JdbcPojoStoreBinaryMarshallerSelfTest.java | 2 +- ...BinaryMarshallerWithSqlEscapeSelfTest.java | 28 +++++ ...cPojoStoreOptimizedMarshallerSelfTest.java | 2 +- ...imizedMarshallerWithSqlEscapeSelfTest.java | 28 +++++ .../testsuites/IgniteCacheTestSuite.java | 4 + .../test/config/jdbc-pojo-store-builtin.xml | 12 +- .../src/test/config/jdbc-pojo-store-obj.xml | 12 +- .../CacheJdbcBlobStoreFactorySelfTest.java | 12 +- .../CacheJdbcPojoStoreFactorySelfTest.java | 14 ++- .../store/jdbc/CachePojoStoreXmlSelfTest.java | 13 ++- ...achePojoStoreXmlWithSqlEscapeSelfTest.java | 28 +++++ .../testsuites/IgniteSpringTestSuite.java | 5 +- 22 files changed, 328 insertions(+), 95 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest.java create mode 100644 modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlWithSqlEscapeSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java index fe8a50b5460bd..a9a8ce1d0b697 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java @@ -187,6 +187,9 @@ public abstract class CacheAbstractJdbcStore implements CacheStore, /** Hash calculator. */ protected JdbcTypeHasher hasher = JdbcTypeDefaultHasher.INSTANCE; + /** Flag indicating that table and field names should be escaped in all SQL queries created by JDBC POJO store. */ + private boolean sqlEscapeAll; + /** * Get field value from object for use as query parameter. * @@ -727,7 +730,7 @@ private Map getOrCreateCacheMappings(@Nullable String cach checkTypeConfiguration(cacheName, valKind, valType, type.getValueFields()); - entryMappings.put(keyTypeId, new EntryMapping(cacheName, dialect, type, keyKind, valKind)); + entryMappings.put(keyTypeId, new EntryMapping(cacheName, dialect, type, keyKind, valKind, sqlEscapeAll)); // Add one more binding to binary typeId for POJOs, // because object could be passed to store in binary format. @@ -736,7 +739,7 @@ private Map getOrCreateCacheMappings(@Nullable String cach valKind = valKind == TypeKind.POJO ? TypeKind.BINARY : valKind; - entryMappings.put(keyTypeId, new EntryMapping(cacheName, dialect, type, TypeKind.BINARY, valKind)); + entryMappings.put(keyTypeId, new EntryMapping(cacheName, dialect, type, TypeKind.BINARY, valKind, sqlEscapeAll)); } } @@ -1674,6 +1677,28 @@ public void setParallelLoadCacheMinimumThreshold(int parallelLoadCacheMinThresho this.parallelLoadCacheMinThreshold = parallelLoadCacheMinThreshold; } + /** + * If {@code true} all the SQL table and field names will be escaped with double quotes like + * ({@code "tableName"."fieldsName"}). This enforces case sensitivity for field names and + * also allows having special characters in table and field names. + * + * @return Flag value. + */ + public boolean isSqlEscapeAll() { + return sqlEscapeAll; + } + + /** + * If {@code true} all the SQL table and field names will be escaped with double quotes like + * ({@code "tableName"."fieldsName"}). This enforces case sensitivity for field names and + * also allows having special characters in table and field names. + * + * @param sqlEscapeAll Flag value. + */ + public void setSqlEscapeAll(boolean sqlEscapeAll) { + this.sqlEscapeAll = sqlEscapeAll; + } + /** * @return Ignite instance. */ @@ -1740,9 +1765,15 @@ protected static class EntryMapping { /** Database key columns. */ private final Collection keyCols; + /** Database key columns prepared for building SQL queries.. */ + private final Collection sqlKeyCols; + /** Database unique value columns. */ private final Collection cols; + /** Database unique value columns prepared for building SQL queries. */ + private final Collection sqlCols; + /** Select query columns index. */ private final Map loadColIdxs; @@ -1761,13 +1792,34 @@ protected static class EntryMapping { /** Full table name. */ private final String fullTblName; + /** Full table name prepared for building SQL queries. */ + private final String sqlFullTblName; + + /** + * Escape collection of column names. + * @param dialect Database dialect. + * @param cols Columns. + * @return Collection of escaped names. + */ + private static Collection escape(JdbcDialect dialect, Collection cols) { + Collection res = new ArrayList<>(cols.size()); + + for (String col : cols) + res.add(dialect.escape(col)); + + return res; + } + /** * @param cacheName Cache name. * @param dialect JDBC dialect. * @param typeMeta Type metadata. + * @param keyKind Type kind. + * @param valKind Value kind. + * @param escape Escape SQL identifiers flag. */ public EntryMapping(@Nullable String cacheName, JdbcDialect dialect, JdbcType typeMeta, - TypeKind keyKind, TypeKind valKind) { + TypeKind keyKind, TypeKind valKind, boolean escape) { this.cacheName = cacheName; this.dialect = dialect; @@ -1794,11 +1846,9 @@ public EntryMapping(@Nullable String cacheName, JdbcDialect dialect, JdbcType ty String tblName = typeMeta.getDatabaseTable(); - fullTblName = F.isEmpty(schema) ? tblName : schema + "." + tblName; - - Collection uniqValCols = databaseColumns(uniqValFlds); + Collection uniqueValCols = databaseColumns(uniqValFlds); - cols = F.concat(false, keyCols, uniqValCols); + cols = F.concat(false, keyCols, uniqueValCols); loadColIdxs = U.newHashMap(cols.size()); @@ -1807,23 +1857,41 @@ public EntryMapping(@Nullable String cacheName, JdbcDialect dialect, JdbcType ty for (String col : cols) loadColIdxs.put(col.toUpperCase(), idx++); - loadCacheQry = dialect.loadCacheQuery(fullTblName, cols); + fullTblName = F.isEmpty(schema) ? tblName : schema + "." + tblName; + + Collection sqlUniqueValCols; + + if (escape) { + sqlFullTblName = F.isEmpty(schema) ? dialect.escape(tblName) : dialect.escape(schema) + "." + dialect.escape(tblName); + + sqlCols = escape(dialect, cols); + sqlKeyCols = escape(dialect, keyCols); + sqlUniqueValCols = escape(dialect, uniqueValCols); + } + else { + sqlFullTblName = fullTblName; + sqlCols = cols; + sqlKeyCols = keyCols; + sqlUniqueValCols = uniqueValCols; + } + + loadCacheQry = dialect.loadCacheQuery(sqlFullTblName, sqlCols); - loadCacheSelRangeQry = dialect.loadCacheSelectRangeQuery(fullTblName, keyCols); + loadCacheSelRangeQry = dialect.loadCacheSelectRangeQuery(sqlFullTblName, sqlKeyCols); - loadQrySingle = dialect.loadQuery(fullTblName, keyCols, cols, 1); + loadQrySingle = dialect.loadQuery(sqlFullTblName, sqlKeyCols, sqlCols, 1); - maxKeysPerStmt = dialect.getMaxParameterCount() / keyCols.size(); + maxKeysPerStmt = dialect.getMaxParameterCount() / sqlKeyCols.size(); - loadQry = dialect.loadQuery(fullTblName, keyCols, cols, maxKeysPerStmt); + loadQry = dialect.loadQuery(sqlFullTblName, sqlKeyCols, sqlCols, maxKeysPerStmt); - insQry = dialect.insertQuery(fullTblName, keyCols, uniqValCols); + insQry = dialect.insertQuery(sqlFullTblName, sqlKeyCols, sqlUniqueValCols); - updQry = dialect.updateQuery(fullTblName, keyCols, uniqValCols); + updQry = dialect.updateQuery(sqlFullTblName, sqlKeyCols, sqlUniqueValCols); - mergeQry = dialect.mergeQuery(fullTblName, keyCols, uniqValCols); + mergeQry = dialect.mergeQuery(sqlFullTblName, sqlKeyCols, sqlUniqueValCols); - remQry = dialect.removeQuery(fullTblName, keyCols); + remQry = dialect.removeQuery(sqlFullTblName, sqlKeyCols); } /** @@ -1884,7 +1952,7 @@ protected String loadQuery(int keyCnt) { if (keyCnt == 1) return loadQrySingle; - return dialect.loadQuery(fullTblName, keyCols, cols, keyCnt); + return dialect.loadQuery(sqlFullTblName, sqlKeyCols, sqlCols, keyCnt); } /** @@ -1895,7 +1963,7 @@ protected String loadQuery(int keyCnt) { * @return Query with range. */ protected String loadCacheRangeQuery(boolean appendLowerBound, boolean appendUpperBound) { - return dialect.loadCacheRangeQuery(fullTblName, keyCols, cols, appendLowerBound, appendUpperBound); + return dialect.loadCacheRangeQuery(sqlFullTblName, sqlKeyCols, sqlCols, appendLowerBound, appendUpperBound); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java index 0fca3abe5ef5a..8bc603148bf06 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java @@ -111,7 +111,7 @@ public class CacheJdbcPojoStoreFactory implements Factory implements Factory implements Factory dataSrcFactory; + /** Flag indicating that table and field names should be escaped in all SQL queries created by JDBC POJO store. */ + private boolean sqlEscapeAll; + /** Application context. */ @SpringApplicationContextResource private transient Object appCtx; @@ -148,12 +151,14 @@ public class CacheJdbcPojoStoreFactory implements Factory create() { CacheJdbcPojoStore store = new CacheJdbcPojoStore<>(); - store.setBatchSize(batchSizw); + store.setBatchSize(batchSize); store.setDialect(dialect); store.setMaximumPoolSize(maxPoolSize); - store.setMaximumWriteAttempts(maxWriteAttempts); + store.setMaximumWriteAttempts(maxWrtAttempts); store.setParallelLoadCacheMinimumThreshold(parallelLoadCacheMinThreshold); store.setTypes(types); + store.setHasher(hasher); + store.setSqlEscapeAll(sqlEscapeAll); if (dataSrc != null) store.setDataSource(dataSrc); @@ -201,7 +206,7 @@ public CacheJdbcPojoStoreFactory setDataSource(DataSource dataSrc) { * @return Maximum batch size. */ public int getBatchSize() { - return batchSizw; + return batchSize; } /** @@ -211,7 +216,7 @@ public int getBatchSize() { * @return {@code This} for chaining. */ public CacheJdbcPojoStoreFactory setBatchSize(int batchSize) { - this.batchSizw = batchSize; + this.batchSize = batchSize; return this; } @@ -285,7 +290,7 @@ public CacheJdbcPojoStoreFactory setMaximumPoolSize(int maxPoolSize) { * @return Maximum number of write attempts. */ public int getMaximumWriteAttempts() { - return maxWriteAttempts; + return maxWrtAttempts; } /** @@ -295,7 +300,7 @@ public int getMaximumWriteAttempts() { * @return {@code This} for chaining. */ public CacheJdbcPojoStoreFactory setMaximumWriteAttempts(int maxWrtAttempts) { - this.maxWriteAttempts = maxWrtAttempts; + this.maxWrtAttempts = maxWrtAttempts; return this; } @@ -386,6 +391,31 @@ public CacheJdbcPojoStoreFactory setDataSourceFactory(Factory return this; } + /** + * If {@code true} all the SQL table and field names will be escaped with double quotes like + * ({@code "tableName"."fieldsName"}). This enforces case sensitivity for field names and + * also allows having special characters in table and field names. + * + * @return Flag value. + */ + public boolean isSqlEscapeAll() { + return sqlEscapeAll; + } + + /** + * If {@code true} all the SQL table and field names will be escaped with double quotes like + * ({@code "tableName"."fieldsName"}). This enforces case sensitivity for field names and + * also allows having special characters in table and field names. + * + * @param sqlEscapeAll Flag value. + * @return {@code this} for chaining. + */ + public CacheJdbcPojoStoreFactory setSqlEscapeAll(boolean sqlEscapeAll) { + this.sqlEscapeAll = sqlEscapeAll; + + return this; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(CacheJdbcPojoStoreFactory.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java index cd9c9868a9495..5fa564b082fa5 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java @@ -148,6 +148,11 @@ private static String where(Collection keyCols, int keyCnt) { return sb.toString(); } + /** {@inheritDoc} */ + @Override public String escape(String ident) { + return '"' + ident + '"'; + } + /** {@inheritDoc} */ @Override public String loadCacheSelectRangeQuery(String fullTblName, Collection keyCols) { String cols = mkString(keyCols, ","); @@ -245,8 +250,7 @@ private static String where(Collection keyCols, int keyCnt) { } /** {@inheritDoc} */ - @Override public String mergeQuery(String fullTblName, Collection keyCols, - Collection uniqCols) { + @Override public String mergeQuery(String fullTblName, Collection keyCols, Collection uniqCols) { return ""; } diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java index 2a92bddf48f26..4bae14a443f5b 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java @@ -34,9 +34,7 @@ public class DB2Dialect extends BasicJdbcDialect { } /** {@inheritDoc} */ - @Override public String mergeQuery(String fullTblName, Collection keyCols, - Collection uniqCols) { - + @Override public String mergeQuery(String fullTblName, Collection keyCols, Collection uniqCols) { Collection cols = F.concat(false, keyCols, uniqCols); String colsLst = mkString(cols, ", "); @@ -68,4 +66,4 @@ public class DB2Dialect extends BasicJdbcDialect { " INSERT (%s) VALUES (%s)", fullTblName, repeat("?", cols.size(), "", ",", ""), colsLst, match, setCols, colsLst, valuesCols); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/H2Dialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/H2Dialect.java index 94d27a9727492..019a4b30fa9aa 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/H2Dialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/H2Dialect.java @@ -33,11 +33,10 @@ public class H2Dialect extends BasicJdbcDialect { } /** {@inheritDoc} */ - @Override public String mergeQuery(String fullTblName, Collection keyCols, - Collection uniqCols) { + @Override public String mergeQuery(String fullTblName, Collection keyCols, Collection uniqCols) { Collection cols = F.concat(false, keyCols, uniqCols); return String.format("MERGE INTO %s (%s) KEY (%s) VALUES(%s)", fullTblName, mkString(cols, ","), mkString(keyCols, ","), repeat("?", cols.size(), "", ", ", "")); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java index 9daa00b36bec5..8c153b2db87af 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java @@ -25,7 +25,13 @@ */ public interface JdbcDialect extends Serializable { /** - * Construct select count query. + * @param ident SQL identifier to escape. + * @return Escaped SQL identifier. + */ + public String escape(String ident); + + /** + * Construct query to get ranges bounds. * * @param fullTblName Full table name. * @param keyCols Database key columns for order. @@ -34,7 +40,7 @@ public interface JdbcDialect extends Serializable { public String loadCacheSelectRangeQuery(String fullTblName, Collection keyCols); /** - * Construct select count query. + * Construct load cache query over specified range. * * @param fullTblName Full table name. * @param keyCols Database key columns for order. @@ -43,8 +49,8 @@ public interface JdbcDialect extends Serializable { * @param appendUpperBound Need add upper bound for range. * @return Query for select count. */ - public String loadCacheRangeQuery(String fullTblName, - Collection keyCols, Iterable uniqCols, boolean appendLowerBound, boolean appendUpperBound); + public String loadCacheRangeQuery(String fullTblName, Collection keyCols, Iterable uniqCols, + boolean appendLowerBound, boolean appendUpperBound); /** * Construct load cache query. @@ -73,6 +79,7 @@ public String loadQuery(String fullTblName, Collection keyCols, Iterable * @param fullTblName Full table name. * @param keyCols Database key columns. * @param valCols Database value columns. + * @return Insert query. */ public String insertQuery(String fullTblName, Collection keyCols, Collection valCols); @@ -82,6 +89,7 @@ public String loadQuery(String fullTblName, Collection keyCols, Iterable * @param fullTblName Full table name. * @param keyCols Database key columns. * @param valCols Database value columns. + * @return Update query. */ public String updateQuery(String fullTblName, Collection keyCols, Iterable valCols); @@ -96,7 +104,7 @@ public String loadQuery(String fullTblName, Collection keyCols, Iterable * @param fullTblName Full table name. * @param keyCols Database key columns. * @param uniqCols Database unique value columns. - * @return Put query. + * @return Merge query. */ public String mergeQuery(String fullTblName, Collection keyCols, Collection uniqCols); diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java index 84e6d05f56f46..1a5730b675170 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java @@ -28,6 +28,11 @@ public class MySQLDialect extends BasicJdbcDialect { /** */ private static final long serialVersionUID = 0L; + /** {@inheritDoc} */ + @Override public String escape(String ident) { + return '`' + ident + '`'; + } + /** {@inheritDoc} */ @Override public String loadCacheSelectRangeQuery(String fullTblName, Collection keyCols) { String cols = mkString(keyCols, ","); @@ -43,9 +48,7 @@ public class MySQLDialect extends BasicJdbcDialect { } /** {@inheritDoc} */ - @Override public String mergeQuery(String fullTblName, Collection keyCols, - Collection uniqCols) { - + @Override public String mergeQuery(String fullTblName, Collection keyCols, Collection uniqCols) { Collection cols = F.concat(false, keyCols, uniqCols); String updPart = mkString(uniqCols, new C1() { diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java index b7a0400c869c2..a9efe0dbd2c28 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java @@ -34,8 +34,7 @@ public class OracleDialect extends BasicJdbcDialect { } /** {@inheritDoc} */ - @Override public String mergeQuery(String fullTblName, Collection keyCols, - Collection uniqCols) { + @Override public String mergeQuery(String fullTblName, Collection keyCols, Collection uniqCols) { Collection cols = F.concat(false, keyCols, uniqCols); String colsLst = mkString(cols, ", "); @@ -72,4 +71,4 @@ public class OracleDialect extends BasicJdbcDialect { " WHEN NOT MATCHED THEN" + " INSERT (%s) VALUES (%s)", fullTblName, selCols, match, setCols, colsLst, valuesCols); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java index 9831aa88b1dba..883918f9803a4 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java @@ -28,14 +28,18 @@ public class SQLServerDialect extends BasicJdbcDialect { /** */ private static final long serialVersionUID = 0L; + /** {@inheritDoc} */ + @Override public String escape(String ident) { + return '[' + ident + ']'; + } + /** {@inheritDoc} */ @Override public boolean hasMerge() { return true; } /** {@inheritDoc} */ - @Override public String mergeQuery(String fullTblName, Collection keyCols, - Collection uniqCols) { + @Override public String mergeQuery(String fullTblName, Collection keyCols, Collection uniqCols) { Collection cols = F.concat(false, keyCols, uniqCols); String colsLst = mkString(cols, ", "); @@ -64,7 +68,7 @@ public class SQLServerDialect extends BasicJdbcDialect { " WHEN MATCHED THEN" + " UPDATE SET %s" + " WHEN NOT MATCHED THEN" + - " INSERT (%s) VALUES (%s);", fullTblName, repeat("?", cols.size(), "", ",", ""), colsLst, - match, setCols, colsLst, valuesCols); + " INSERT (%s) VALUES (%s);", fullTblName, repeat("?", cols.size(), "", ",", ""), + colsLst, match, setCols, colsLst, valuesCols); } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java index e8592d74fc334..83065f1bbb146 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java @@ -46,7 +46,7 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED; /** - * Class for {@code PojoCacheStore} tests. + * Class for {@link CacheJdbcPojoStore} tests. */ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstractTest { /** IP finder. */ @@ -76,6 +76,13 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr /** Flag indicating that classes for values available on class path or not. */ protected static boolean noValClasses; + /** + * @return Flag indicating that all internal SQL queries should use escaped identifiers. + */ + protected boolean sqlEscapeAll(){ + return false; + } + /** * @return Connection to test in-memory H2 database. * @throws SQLException if failed to connect. @@ -164,10 +171,13 @@ protected JdbcType[] storeTypes() { } storeTypes[0].setValueType("org.apache.ignite.cache.store.jdbc.model.Organization" + (noValClasses ? "1" : "")); + + boolean escape = sqlEscapeAll(); + storeTypes[0].setValueFields( - new JdbcTypeField(Types.INTEGER, "Id", Integer.class, "id"), - new JdbcTypeField(Types.VARCHAR, "Name", String.class, "name"), - new JdbcTypeField(Types.VARCHAR, "City", String.class, "city")); + new JdbcTypeField(Types.INTEGER, escape ? "ID" : "Id", Integer.class, "id"), + new JdbcTypeField(Types.VARCHAR, escape ? "NAME" : "Name", String.class, "name"), + new JdbcTypeField(Types.VARCHAR, escape ? "CITY" : "City", String.class, "city")); storeTypes[1] = new JdbcType(); storeTypes[1].setCacheName(CACHE_NAME); @@ -210,6 +220,7 @@ protected CacheConfiguration cacheConfiguration() throws Exception { storeFactory.setDialect(new H2Dialect()); storeFactory.setTypes(storeTypes()); storeFactory.setDataSourceFactory(new H2DataSourceFactory()); // H2 DataSource factory. + storeFactory.setSqlEscapeAll(sqlEscapeAll()); cc.setCacheStoreFactory(storeFactory); cc.setReadThrough(true); @@ -228,8 +239,6 @@ protected CacheConfiguration cacheConfiguration() throws Exception { protected void fillSampleDatabase(Connection conn) throws SQLException { info("Start to fill sample database..."); - Random rnd = new Random(); - PreparedStatement orgStmt = conn.prepareStatement("INSERT INTO Organization(id, name, city) VALUES (?, ?, ?)"); for (int i = 0; i < ORGANIZATION_CNT; i++) { @@ -249,6 +258,8 @@ protected void fillSampleDatabase(Connection conn) throws SQLException { PreparedStatement prnStmt = conn.prepareStatement( "INSERT INTO Person(id, org_id, birthday, name) VALUES (?, ?, ?, ?)"); + Random rnd = new Random(); + for (int i = 0; i < PERSON_CNT; i++) { prnStmt.setInt(1, i); prnStmt.setInt(2, i % 100); @@ -366,7 +377,7 @@ public void testLoadCachePrimitiveKeysTx() throws Exception { * * @throws Exception If failed. */ - private void checkPut() throws Exception { + private void checkPutRemove() throws Exception { IgniteCache c1 = grid().cache(CACHE_NAME); Connection conn = getConnection(); @@ -419,6 +430,13 @@ private void checkPut() throws Exception { assertFalse("Unexpected more data in result set", rs.next()); + // Test remove. + c1.remove(key); + + rs = stmt.executeQuery(); + + assertFalse("Unexpected non-empty result set", rs.next()); + U.closeQuiet(rs); } finally { @@ -429,37 +447,37 @@ private void checkPut() throws Exception { /** * @throws Exception If failed. */ - public void testPutBuiltIn() throws Exception { + public void testPutRemoveBuiltIn() throws Exception { startTestGrid(true, false, false, false); - checkPut(); + checkPutRemove(); } /** * @throws Exception If failed. */ - public void testPut() throws Exception { + public void testPutRemove() throws Exception { startTestGrid(false, false, false, false); - checkPut(); + checkPutRemove(); } /** * @throws Exception If failed. */ - public void testPutTxBuiltIn() throws Exception { + public void testPutRemoveTxBuiltIn() throws Exception { startTestGrid(true, false, false, true); - checkPut(); + checkPutRemove(); } /** * @throws Exception If failed. */ - public void testPutTx() throws Exception { + public void testPutRemoveTx() throws Exception { startTestGrid(false, false, false, true); - checkPut(); + checkPutRemove(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java index f998027cd0fbd..14c743cbb3bd5 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java @@ -21,7 +21,7 @@ import org.apache.ignite.internal.binary.BinaryMarshaller; /** - * Class for {@code PojoCacheStore} tests. + * Test for {@link CacheJdbcPojoStore} with binary marshaller. */ public class CacheJdbcPojoStoreBinaryMarshallerSelfTest extends CacheJdbcPojoStoreAbstractSelfTest { /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest.java new file mode 100644 index 0000000000000..829fffa442f49 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.store.jdbc; + +/** + * Test for {@link CacheJdbcPojoStore} with binary marshaller and enabled SQL escaping. + */ +public class CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest extends CacheJdbcPojoStoreBinaryMarshallerSelfTest { + /** {@inheritDoc} */ + @Override protected boolean sqlEscapeAll(){ + return true; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerSelfTest.java index f40f7d7f4b8d1..3f6c9b4425612 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerSelfTest.java @@ -21,7 +21,7 @@ import org.apache.ignite.marshaller.optimized.OptimizedMarshaller; /** - * Class for {@code PojoCacheStore} tests. + * Test for {@link CacheJdbcPojoStore} with optimized marshaller. */ public class CacheJdbcPojoStoreOptimizedMarshallerSelfTest extends CacheJdbcPojoStoreAbstractSelfTest { /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest.java new file mode 100644 index 0000000000000..d1ce726b53344 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.store.jdbc; + +/** + * Test for {@link CacheJdbcPojoStore} with optimized marshaller and enabled SQL escaping. + */ +public class CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest extends CacheJdbcPojoStoreOptimizedMarshallerSelfTest { + /** {@inheritDoc} */ + @Override protected boolean sqlEscapeAll(){ + return true; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java index 9240ef5959f59..5ad4cb82d70ad 100755 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java @@ -32,8 +32,10 @@ import org.apache.ignite.cache.store.GridCacheLoadOnlyStoreAdapterSelfTest; import org.apache.ignite.cache.store.StoreResourceInjectionSelfTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerSelfTest; +import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreMultitreadedSelfTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreOptimizedMarshallerSelfTest; +import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreTest; import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest; import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest; @@ -243,7 +245,9 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(GridCacheJdbcBlobStoreMultithreadedSelfTest.class); suite.addTestSuite(CacheJdbcPojoStoreTest.class); suite.addTestSuite(CacheJdbcPojoStoreOptimizedMarshallerSelfTest.class); + suite.addTestSuite(CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest.class); suite.addTestSuite(CacheJdbcPojoStoreBinaryMarshallerSelfTest.class); + suite.addTestSuite(CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest.class); suite.addTestSuite(CacheJdbcPojoStoreMultitreadedSelfTest.class); suite.addTestSuite(GridCacheBalancingStoreSelfTest.class); suite.addTestSuite(GridCacheAffinityApiSelfTest.class); diff --git a/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml b/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml index 3480ece45e152..d62e76b9edc21 100644 --- a/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml +++ b/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml @@ -70,7 +70,7 @@ - + @@ -83,7 +83,7 @@ - + @@ -91,7 +91,7 @@ - + @@ -112,7 +112,7 @@ - + @@ -125,7 +125,7 @@ - + @@ -133,7 +133,7 @@ - + diff --git a/modules/spring/src/test/config/jdbc-pojo-store-obj.xml b/modules/spring/src/test/config/jdbc-pojo-store-obj.xml index ee761d0ea9b30..83e05487aad1d 100644 --- a/modules/spring/src/test/config/jdbc-pojo-store-obj.xml +++ b/modules/spring/src/test/config/jdbc-pojo-store-obj.xml @@ -70,7 +70,7 @@ - + @@ -83,7 +83,7 @@ - + @@ -91,7 +91,7 @@ - + @@ -112,7 +112,7 @@ - + @@ -125,7 +125,7 @@ - + @@ -133,7 +133,7 @@ - + diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java index da7004261aaa8..6ecf67fdcd6a6 100644 --- a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java @@ -34,7 +34,7 @@ import org.h2.jdbcx.JdbcDataSource; /** - * Test for Cache jdbc blob store factory. + * Test for Cache JDBC blob store factory. */ public class CacheJdbcBlobStoreFactorySelfTest extends GridCommonAbstractTest { /** Cache name. */ @@ -107,16 +107,16 @@ private CacheConfiguration cacheConfiguration() { /** * @param cache Ignite cache. - * @param dataSrcClass Data source class. + * @param dataSrcCls Data source class. * @throws Exception If store parameters is not the same as in configuration xml. */ - private void checkStore(IgniteCache cache, Class dataSrcClass) throws Exception { + private void checkStore(IgniteCache cache, Class dataSrcCls) throws Exception { CacheJdbcBlobStore store = (CacheJdbcBlobStore) cache.getConfiguration(CacheConfiguration.class). getCacheStoreFactory().create(); assertEquals(USER_NAME, GridTestUtils.getFieldValue(store, CacheJdbcBlobStore.class, "user")); - assertEquals(dataSrcClass, + assertEquals(dataSrcCls, GridTestUtils.getFieldValue(store, CacheJdbcBlobStore.class, "dataSrc").getClass()); } @@ -135,7 +135,7 @@ public DummyDataSource() { } /** {@inheritDoc} */ - @Override public Connection getConnection(String username, String password) throws SQLException { + @Override public Connection getConnection(String username, String pwd) throws SQLException { return null; } @@ -174,4 +174,4 @@ public DummyDataSource() { return false; } } -} \ No newline at end of file +} diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java index dfa14529c4f50..e3549355d6e3b 100644 --- a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java @@ -17,7 +17,6 @@ package org.apache.ignite.cache.store.jdbc; -import java.io.Serializable; import java.util.Collection; import java.util.concurrent.Callable; import org.apache.ignite.Ignite; @@ -32,7 +31,7 @@ import org.h2.jdbcx.JdbcDataSource; /** - * Test for Cache jdbc blob store factory. + * Test for Cache JDBC POJO store factory. */ public class CacheJdbcPojoStoreFactorySelfTest extends GridCommonAbstractTest { /** Cache name. */ @@ -117,14 +116,14 @@ private CacheConfiguration cacheConfigurationH2Dialect() { /** * @param cache Ignite cache. - * @param dataSrcClass Data source class. + * @param dataSrcCls Data source class. * @throws Exception If store parameters is not the same as in configuration xml. */ - private void checkStore(IgniteCache cache, Class dataSrcClass) throws Exception { + private void checkStore(IgniteCache cache, Class dataSrcCls) throws Exception { CacheJdbcPojoStore store = (CacheJdbcPojoStore)cache.getConfiguration(CacheConfiguration.class). getCacheStoreFactory().create(); - assertEquals(dataSrcClass, + assertEquals(dataSrcCls, GridTestUtils.getFieldValue(store, CacheAbstractJdbcStore.class, "dataSrc").getClass()); } @@ -132,6 +131,11 @@ private void checkStore(IgniteCache cache, Class dataSrcClas * Dummy JDBC dialect that does nothing. */ public static class DummyDialect implements JdbcDialect { + /** {@inheritDoc} */ + @Override public String escape(String ident) { + return null; + } + /** {@inheritDoc} */ @Override public String loadCacheSelectRangeQuery(String fullTblName, Collection keyCols) { return null; diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlSelfTest.java index c712f882f8ecc..7de0cfe2d3727 100644 --- a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlSelfTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlSelfTest.java @@ -18,6 +18,7 @@ package org.apache.ignite.cache.store.jdbc; import java.net.URL; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteComponentType; import org.apache.ignite.internal.util.spring.IgniteSpringHelper; @@ -25,13 +26,14 @@ import org.apache.ignite.marshaller.Marshaller; /** - * Tests for {@code PojoCacheStore} created via XML. + * Tests for {@link CacheJdbcPojoStore} created via XML. */ public class CachePojoStoreXmlSelfTest extends CacheJdbcPojoStoreAbstractSelfTest { /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - String path = builtinKeys ? "modules/spring/src/test/config/jdbc-pojo-store-builtin.xml" : - "modules/spring/src/test/config/jdbc-pojo-store-obj.xml"; + String path = builtinKeys + ? "modules/spring/src/test/config/jdbc-pojo-store-builtin.xml" + : "modules/spring/src/test/config/jdbc-pojo-store-obj.xml"; URL url = U.resolveIgniteUrl(path); @@ -39,6 +41,11 @@ public class CachePojoStoreXmlSelfTest extends CacheJdbcPojoStoreAbstractSelfTes IgniteConfiguration cfg = spring.loadConfigurations(url).get1().iterator().next(); + if (sqlEscapeAll()) { + for (CacheConfiguration ccfg : cfg.getCacheConfiguration()) + ((CacheJdbcPojoStoreFactory)ccfg.getCacheStoreFactory()).setSqlEscapeAll(true); + } + cfg.setGridName(gridName); return cfg; diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlWithSqlEscapeSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlWithSqlEscapeSelfTest.java new file mode 100644 index 0000000000000..e801682ce1cda --- /dev/null +++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlWithSqlEscapeSelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.store.jdbc; + +/** + * Tests for {@link CacheJdbcPojoStore} created via XML. + */ +public class CachePojoStoreXmlWithSqlEscapeSelfTest extends CachePojoStoreXmlSelfTest { + /** {@inheritDoc} */ + @Override protected boolean sqlEscapeAll() { + return true; + } +} diff --git a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java index 67b117dbee686..c4a4b75c5171d 100644 --- a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java +++ b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java @@ -22,6 +22,8 @@ import org.apache.ignite.cache.spring.SpringCacheManagerContextInjectionTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactorySelfTest; import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactorySelfTest; +import org.apache.ignite.cache.store.jdbc.CachePojoStoreXmlSelfTest; +import org.apache.ignite.cache.store.jdbc.CachePojoStoreXmlWithSqlEscapeSelfTest; import org.apache.ignite.cache.store.spring.CacheSpringStoreSessionListenerSelfTest; import org.apache.ignite.internal.GridFactorySelfTest; import org.apache.ignite.internal.GridSpringBeanSerializationSelfTest; @@ -64,8 +66,9 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheSpringStoreSessionListenerSelfTest.class); suite.addTestSuite(CacheJdbcBlobStoreFactorySelfTest.class); - suite.addTestSuite(CacheJdbcPojoStoreFactorySelfTest.class); + suite.addTestSuite(CachePojoStoreXmlSelfTest.class); + suite.addTestSuite(CachePojoStoreXmlWithSqlEscapeSelfTest.class); suite.addTest(new TestSuite(GridSpringTransactionManagerSelfTest.class)); From 9666dc49d704c32a4b0b4b7920af62b78af69298 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 23 Sep 2016 10:18:15 +0700 Subject: [PATCH 156/487] IGNITE-3937 implemented support for unsigned types in MySQL. Minor fix. --- .../ignite/schema/parser/dialect/JdbcMetadataDialect.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java index f6c27f29fc60b..07361ba14527d 100644 --- a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java +++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java @@ -118,6 +118,8 @@ protected boolean useSchema() { Set sys = systemSchemas(); + Collection unsignedTypes = unsignedTypes(dbMeta); + if (schemas.isEmpty()) schemas.add(null); @@ -147,8 +149,6 @@ protected boolean useSchema() { Collection cols = new ArrayList<>(); - Collection unsignedTypes = unsignedTypes(dbMeta); - try (ResultSet colsRs = dbMeta.getColumns(tblCatalog, tblSchema, tblName, null)) { while (colsRs.next()) { String colName = colsRs.getString(COL_NAME_IDX); From 29f508b17eb383a1c910c72e7d5056564acc37c2 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Fri, 23 Sep 2016 10:59:55 +0300 Subject: [PATCH 157/487] IGNITE-3925: Output process ID to the log on node start. --- .../main/java/org/apache/ignite/internal/IgniteKernal.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index b85692e19d719..827b54a262e97 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -1781,6 +1781,11 @@ private void ackOsInfo() { if (log.isInfoEnabled()) { log.info("OS: " + U.osString()); log.info("OS user: " + System.getProperty("user.name")); + + int jvmPid = U.jvmPid(); + + if (log.isInfoEnabled()) + log.info("PID: " + (jvmPid == -1 ? "N/A" : jvmPid)); } } From 40f647258a10fa8b42ac6fda67d6fa4db71b8100 Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Thu, 8 Sep 2016 18:58:31 +0300 Subject: [PATCH 158/487] IGNITE-3874 sync putIfAbsent forever blocked on all nodes after one node is stopped --- .../processors/cache/transactions/IgniteTxManager.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java index a1580a58fa413..c72d7f781d938 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java @@ -374,7 +374,7 @@ private boolean salvageTx(IgniteInternalTx tx, boolean warn, IgniteInternalTx.Fi "crashed or left grid: " + CU.txString(tx)); } } - catch (IgniteTxOptimisticCheckedException ignore) { + catch (IgniteCheckedException ignore) { if (log.isDebugEnabled()) log.debug("Optimistic failure while invalidating transaction (will rollback): " + tx.xidVersion()); @@ -386,9 +386,6 @@ private boolean salvageTx(IgniteInternalTx tx, boolean warn, IgniteInternalTx.Fi U.error(log, "Failed to rollback transaction: " + tx.xidVersion(), e); } } - catch (IgniteCheckedException e) { - U.error(log, "Failed to invalidate transaction: " + tx, e); - } } else if (state == MARKED_ROLLBACK) { try { From 062ecd75908131084e27883f3f160f3504e92165 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Fri, 23 Sep 2016 14:05:17 +0300 Subject: [PATCH 159/487] IGNITE-3931: IGFS: Test fixes for PROXY mode. This closes #1102. --- .../local/LocalIgfsSecondaryFileSystem.java | 13 +- .../internal/processors/igfs/IgfsImpl.java | 16 +- .../igfs/IgfsAbstractBaseSelfTest.java | 2 +- .../processors/igfs/IgfsAbstractSelfTest.java | 26 +- ...condaryFileSystemDualAbstractSelfTest.java | 3 +- ...econdaryFileSystemProxyClientSelfTest.java | 28 ++ ...LocalSecondaryFileSystemProxySelfTest.java | 272 ++++++++++++++++++ .../processors/igfs/IgfsProxySelfTest.java | 2 +- .../testsuites/IgniteIgfsTestSuite.java | 8 + 9 files changed, 345 insertions(+), 25 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxyClientSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxySelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java index c84fcf2adefb4..8dd4fdac14728 100644 --- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java +++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java @@ -115,40 +115,41 @@ else if(!srcFile.renameTo(destFile)) if (!recursive) return f.delete(); else - return deleteRecursive(f); + return deleteRecursive(f, false); } /** * Delete directory recursively. * * @param f Directory. + * @param deleteIfExists Ignore delete errors if the file doesn't exist. * @return {@code true} if successful. */ - private boolean deleteRecursive(File f) { + private boolean deleteRecursive(File f, boolean deleteIfExists) { BasicFileAttributes attrs; try { attrs = Files.readAttributes(f.toPath(), BasicFileAttributes.class, LinkOption.NOFOLLOW_LINKS); } catch (IOException ignore) { - return false; + return deleteIfExists && !f.exists(); } if (!attrs.isDirectory() || attrs.isSymbolicLink()) - return f.delete(); + return f.delete() || (deleteIfExists && !f.exists()); File[] entries = f.listFiles(); if (entries != null) { for (File entry : entries) { - boolean res = deleteRecursive(entry); + boolean res = deleteRecursive(entry, true); if (!res) return false; } } - return f.delete(); + return f.delete() || (deleteIfExists && !f.exists()); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index bee9d9a72a53c..6ff1f8fa3eddb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -1232,10 +1232,20 @@ private IgfsOutputStream create0( @Override public Void call() throws Exception { IgfsMode mode = resolveMode(path); - boolean useSecondary = IgfsUtils.isDualMode(mode) && secondaryFs instanceof IgfsSecondaryFileSystemV2; + if (mode == PROXY) { + if (secondaryFs instanceof IgfsSecondaryFileSystemV2) + ((IgfsSecondaryFileSystemV2)secondaryFs).setTimes(path, accessTime, modificationTime); + else + throw new UnsupportedOperationException("setTimes is not supported in PROXY mode for " + + "this secondary file system,"); + } + else { + boolean useSecondary = + IgfsUtils.isDualMode(mode) && secondaryFs instanceof IgfsSecondaryFileSystemV2; - meta.updateTimes(path, accessTime, modificationTime, - useSecondary ? (IgfsSecondaryFileSystemV2)secondaryFs : null); + meta.updateTimes(path, accessTime, modificationTime, + useSecondary ? (IgfsSecondaryFileSystemV2) secondaryFs : null); + } return null; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java index 14a653bf83b68..58c4c50707656 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java @@ -713,7 +713,7 @@ protected void checkNotExist(IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter i throws Exception { checkNotExist(igfs, paths); - if (dual) + if (mode != PRIMARY) checkNotExist(igfsSecondary, paths); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java index a115e03c7f981..e8d65f18e06ab 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java @@ -61,6 +61,9 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.ignite.igfs.IgfsMode.PRIMARY; +import static org.apache.ignite.igfs.IgfsMode.PROXY; + /** * Test fo regular igfs operations. */ @@ -657,6 +660,9 @@ public void testUpdatePathDoesNotExist() throws Exception { */ @SuppressWarnings("ConstantConditions") public void testFormat() throws Exception { + if (mode == PROXY) + return; + final GridCacheAdapter dataCache = getDataCache(igfs); assert dataCache != null; @@ -1010,7 +1016,7 @@ public void testCreateParentRoot() throws Exception { * @throws Exception If failed. */ public void testCreateNoClose() throws Exception { - if (dual) + if (mode != PRIMARY) return; create(igfs, paths(DIR, SUBDIR), null); @@ -1089,7 +1095,7 @@ public void testCreateRenameParentNoClose() throws Exception { * @throws Exception If failed. */ public void testCreateDeleteNoClose() throws Exception { - if (dual) + if (mode != PRIMARY) return; create(igfs, paths(DIR, SUBDIR), null); @@ -1143,7 +1149,7 @@ public void testCreateDeleteNoClose() throws Exception { * @throws Exception If failed. */ public void testCreateDeleteParentNoClose() throws Exception { - if (dual) + if (mode != PRIMARY) return; create(igfs, paths(DIR, SUBDIR), null); @@ -1541,7 +1547,7 @@ public void testAppendParentRoot() throws Exception { * @throws Exception If failed. */ public void testAppendNoClose() throws Exception { - if (dual) + if (mode != PRIMARY) return; if (appendSupported()) { @@ -1631,7 +1637,7 @@ public void testAppendRenameParentNoClose() throws Exception { * @throws Exception If failed. */ public void testAppendDeleteNoClose() throws Exception { - if (dual) + if (mode != PRIMARY) return; if (appendSupported()) { @@ -1686,7 +1692,7 @@ public boolean apply() { * @throws Exception If failed. */ public void testAppendDeleteParentNoClose() throws Exception { - if (dual) + if (mode != PRIMARY) return; if (appendSupported()) { @@ -2154,9 +2160,7 @@ public void testConcurrentDeletes() throws Exception { U.awaitQuiet(barrier); try { - igfs.delete(SUBDIR, true); - - return true; + return igfs.delete(SUBDIR, true); } catch (IgniteException ignored) { return false; @@ -2169,9 +2173,7 @@ public void testConcurrentDeletes() throws Exception { U.awaitQuiet(barrier); try { - igfs.delete(SUBSUBDIR, true); - - return true; + return igfs.delete(SUBSUBDIR, true); } catch (IgniteException ignored) { return false; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java index 8baaf4a4dff8d..0e6fc48c4a8f3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java @@ -66,7 +66,7 @@ public abstract class IgfsLocalSecondaryFileSystemDualAbstractSelfTest extends I * * @param mode IGFS mode. */ - public IgfsLocalSecondaryFileSystemDualAbstractSelfTest(IgfsMode mode) { + protected IgfsLocalSecondaryFileSystemDualAbstractSelfTest(IgfsMode mode) { super(mode); } @@ -82,7 +82,6 @@ public IgfsLocalSecondaryFileSystemDualAbstractSelfTest(IgfsMode mode) { cleanDirectory(extDir); } - /** * Creates secondary filesystems. * @return IgfsSecondaryFileSystem diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxyClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxyClientSelfTest.java new file mode 100644 index 0000000000000..4e5d85f9dee35 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxyClientSelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.igfs; + +/** + * Tests for PROXY mode with local FS is configured as a secondary. + */ +public class IgfsLocalSecondaryFileSystemProxyClientSelfTest extends IgfsLocalSecondaryFileSystemProxySelfTest { + /** {@inheritDoc} */ + @Override protected boolean client() { + return true; + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxySelfTest.java new file mode 100644 index 0000000000000..848abe2b825ae --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxySelfTest.java @@ -0,0 +1,272 @@ +/* + * 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.igfs; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.file.Files; +import java.util.Collection; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.ignite.igfs.IgfsFile; +import org.apache.ignite.igfs.IgfsPath; +import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; +import org.apache.ignite.igfs.secondary.local.LocalIgfsSecondaryFileSystem; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiInClosure; +import org.jetbrains.annotations.Nullable; + +/** + * Tests for PROXY mode with local FS is configured as a secondary. + */ +public class IgfsLocalSecondaryFileSystemProxySelfTest extends IgfsProxySelfTest { + /** Fs work directory. */ + private static final String FS_WORK_DIR = U.getIgniteHome() + File.separatorChar + "work" + + File.separatorChar + "fs"; + + /** */ + private static final String FS_EXT_DIR = U.getIgniteHome() + File.separatorChar + "work" + + File.separatorChar + "ext"; + + /** */ + private final File dirLinkDest = new File(FS_EXT_DIR + File.separatorChar + "extdir"); + + /** */ + private final File fileLinkDest = + new File(FS_EXT_DIR + File.separatorChar + "extdir" + File.separatorChar + "filedest"); + + /** */ + private final File dirLinkSrc = new File(FS_WORK_DIR + File.separatorChar + "dir"); + + /** */ + private final File fileLinkSrc = new File(FS_WORK_DIR + File.separatorChar + "file"); + + /** + * Creates secondary filesystems. + * @return IgfsSecondaryFileSystem + * @throws Exception On failure. + */ + @Override protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception { + final File workDir = new File(FS_WORK_DIR); + + if (!workDir.exists()) + assert workDir.mkdirs(); + + LocalIgfsSecondaryFileSystem second = new LocalIgfsSecondaryFileSystem(); + + second.setWorkDirectory(workDir.getAbsolutePath()); + + igfsSecondary = new IgfsLocalSecondaryFileSystemTestAdapter(workDir); + + return second; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + final File extDir = new File(FS_EXT_DIR); + + if (!extDir.exists()) + assert extDir.mkdirs(); + else + cleanDirectory(extDir); + } + + /** {@inheritDoc} */ + @Override protected boolean permissionsSupported() { + return false; + } + + /** {@inheritDoc} */ + @Override protected boolean propertiesSupported() { + return false; + } + + /** {@inheritDoc} */ + @Override protected boolean timesSupported() { + return false; + } + + /** {@inheritDoc} */ + @Override public void testUpdatePathDoesNotExist() throws Exception { + fail("IGNITE-3645"); + } + + /** + * + * @throws Exception If failed. + */ + @SuppressWarnings("ConstantConditions") + public void testListPathForSymlink() throws Exception { + if (U.isWindows()) + return; + + createSymlinks(); + + assertTrue(igfs.info(DIR).isDirectory()); + + Collection pathes = igfs.listPaths(DIR); + Collection files = igfs.listFiles(DIR); + + assertEquals(1, pathes.size()); + assertEquals(1, files.size()); + + assertEquals("filedest", F.first(pathes).name()); + assertEquals("filedest", F.first(files).path().name()); + } + + /** + * + * @throws Exception If failed. + */ + public void testDeleteSymlinkDir() throws Exception { + if (U.isWindows()) + return; + + createSymlinks(); + + // Only symlink must be deleted. Destination content must be exist. + igfs.delete(DIR, true); + + assertTrue(fileLinkDest.exists()); + } + + /** + * + * @throws Exception If failed. + */ + public void testSymlinkToFile() throws Exception { + if (U.isWindows()) + return; + + createSymlinks(); + + checkFileContent(igfs, new IgfsPath("/file"), chunk); + } + + /** + * + * @throws Exception If failed. + */ + public void testMkdirsInsideSymlink() throws Exception { + if (U.isWindows()) + return; + + createSymlinks(); + + igfs.mkdirs(SUBSUBDIR); + + assertTrue(Files.isDirectory(dirLinkDest.toPath().resolve("subdir/subsubdir"))); + assertTrue(Files.isDirectory(dirLinkSrc.toPath().resolve("subdir/subsubdir"))); + } + + /** + * + * @throws Exception If failed. + */ + public void testUsedSpaceSize() throws Exception { + final int DIRS_COUNT = 5; + final int DIRS_MAX_DEEP = 3; + final int FILES_COUNT = 10; + final AtomicLong totalSize = new AtomicLong(); + + IgniteBiInClosure createHierarchy = new IgniteBiInClosure() { + @Override public void apply(Integer level, IgfsPath levelDir) { + try { + for (int i = 0; i < FILES_COUNT; ++i) { + IgfsPath filePath = new IgfsPath(levelDir, "file" + Integer.toString(i)); + + createFile(igfs, filePath, true, chunk); + + totalSize.getAndAdd(chunk.length); + } + + if (level < DIRS_MAX_DEEP) { + for (int dir = 0; dir < DIRS_COUNT; dir++) { + IgfsPath dirPath = new IgfsPath(levelDir, "dir" + Integer.toString(dir)); + + igfs.mkdirs(dirPath); + + apply(level + 1, dirPath); + } + } + } catch (Exception e) { + fail(e.getMessage()); + } + } + }; + + createHierarchy.apply(1, new IgfsPath("/dir")); + + assertEquals(totalSize.get(), igfs.metrics().secondarySpaceSize()); + } + + /** + * + * @throws Exception If failed. + */ + private void createSymlinks() throws Exception { + assert dirLinkDest.mkdir(); + + createFile(fileLinkDest, true, chunk); + + Files.createSymbolicLink(dirLinkSrc.toPath(), dirLinkDest.toPath()); + Files.createSymbolicLink(fileLinkSrc.toPath(), fileLinkDest.toPath()); + } + + /** + * @param dir Directory to clean. + */ + private static void cleanDirectory(File dir){ + File[] entries = dir.listFiles(); + + if (entries != null) { + for (File entry : entries) { + if (entry.isDirectory()) { + cleanDirectory(entry); + + assert entry.delete(); + } + else + assert entry.delete(); + } + } + } + + /** + * @param f File object. + * @param overwrite Overwrite flag. + * @param chunks File content. + * @throws IOException If failed. + */ + private static void createFile(File f, boolean overwrite, @Nullable byte[]... chunks) throws IOException { + OutputStream os = null; + + try { + os = new FileOutputStream(f, overwrite); + + writeFileChunks(os, chunks); + } + finally { + U.closeQuiet(os); + } + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProxySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProxySelfTest.java index 3b8c606592dac..73835a535fca9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProxySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProxySelfTest.java @@ -20,7 +20,7 @@ import static org.apache.ignite.igfs.IgfsMode.PROXY; /** - * Tests for PRIMARY mode. + * Tests for PROXY mode. */ public class IgfsProxySelfTest extends IgfsAbstractSelfTest { /** diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java index 0241068bcaac4..5a7e3d7456c9e 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsCacheSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsDualAsyncClientSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsDualSyncClientSelfTest; +import org.apache.ignite.internal.processors.igfs.IgfsLocalSecondaryFileSystemProxyClientSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsMaxSizeSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsPrimaryClientSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsDataManagerSelfTest; @@ -52,6 +53,8 @@ import org.apache.ignite.internal.processors.igfs.IgfsPrimarySelfTest; import org.apache.ignite.internal.processors.igfs.IgfsProcessorSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsProcessorValidationSelfTest; +import org.apache.ignite.internal.processors.igfs.IgfsProxySelfTest; +import org.apache.ignite.internal.processors.igfs.IgfsLocalSecondaryFileSystemProxySelfTest; import org.apache.ignite.internal.processors.igfs.IgfsServerManagerIpcEndpointRegistrationOnWindowsSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsSizeSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsStartCacheTest; @@ -151,6 +154,11 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgfsMaxSizeSelfTest.class); + + suite.addTestSuite(IgfsProxySelfTest.class); + suite.addTestSuite(IgfsLocalSecondaryFileSystemProxySelfTest.class); + suite.addTestSuite(IgfsLocalSecondaryFileSystemProxyClientSelfTest.class); + return suite; } } \ No newline at end of file From dfc728817c3cd3512760bc7383b747e8b4f8b13d Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Fri, 23 Sep 2016 19:18:26 +0300 Subject: [PATCH 160/487] IGNITE-3199 .NET: ASP.NET Session-State Store Provider - fix javadoc --- .../platform/websession/PlatformDotNetSessionLockProcessor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionLockProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionLockProcessor.java index 0e51448782120..b1c45a9470f62 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionLockProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/websession/PlatformDotNetSessionLockProcessor.java @@ -77,7 +77,7 @@ public PlatformDotNetSessionLockProcessor(UUID lockNodeId, long lockId, Timestam return new PlatformDotNetSessionLockResult(true, data, null, data.lockId()); } - /** {@inheritDoc */ + /** {@inheritDoc} */ @Override public String toString() { return S.toString(PlatformDotNetSessionLockProcessor.class, this); } From 8e996796279eb31cd167cdd10e32f71a8efaca27 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Fri, 23 Sep 2016 19:28:56 +0300 Subject: [PATCH 161/487] .NET: Fix analysis warnings --- .../Impl/IgniteSessionStateItemCollection.cs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/IgniteSessionStateItemCollection.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/IgniteSessionStateItemCollection.cs index d1ba5da99c6da..34f0124235f8c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/IgniteSessionStateItemCollection.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/IgniteSessionStateItemCollection.cs @@ -80,8 +80,6 @@ internal IgniteSessionStateItemCollection(IBinaryRawReader reader) else AddRemovedKey(key); } - - _isNew = false; } /// @@ -319,6 +317,7 @@ public void Clear() /// /// Applies the changes. /// + [SuppressMessage("Microsoft.Performance", "CA1811:AvoidUncalledPrivateCode")] public void ApplyChanges(IgniteSessionStateItemCollection changes) { var removed = changes._removedKeys; @@ -489,6 +488,7 @@ public object Value /// /// Copies contents to another entry. /// + [SuppressMessage("Microsoft.Performance", "CA1811:AvoidUncalledPrivateCode")] public void CopyTo(Entry entry) { Debug.Assert(entry != null); From 14959f2efea43046a1a2e5c1978fc27a156f5a7a Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 26 Sep 2016 10:12:45 +0300 Subject: [PATCH 162/487] IGNITE-3925: Removed unnecessary "log.isInfoEnabled()" check during PID printout. --- .../src/main/java/org/apache/ignite/internal/IgniteKernal.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 827b54a262e97..c5d274848d7ee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -1784,8 +1784,7 @@ private void ackOsInfo() { int jvmPid = U.jvmPid(); - if (log.isInfoEnabled()) - log.info("PID: " + (jvmPid == -1 ? "N/A" : jvmPid)); + log.info("PID: " + (jvmPid == -1 ? "N/A" : jvmPid)); } } From 3a66596d76eae5b9265a7de74faeae812d95cccc Mon Sep 17 00:00:00 2001 From: ptupitsyn Date: Mon, 26 Sep 2016 11:00:25 +0300 Subject: [PATCH 163/487] IGNITE-3291 .NET: Added IClusterGroup.ForDaemons() method. This closes #1094. --- .../Compute/ComputeApiTest.cs | 26 +++++++++++++++++++ .../Cluster/IClusterGroup.cs | 14 ++++++++++ .../Impl/Cluster/ClusterGroupImpl.cs | 6 +++++ .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs | 10 ++++++- .../Apache.Ignite.Core/Impl/IgniteProxy.cs | 6 +++++ 5 files changed, 61 insertions(+), 1 deletion(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs index bc26e4c14a82e..7b71ad4ad3a23 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs @@ -523,6 +523,32 @@ public void TestForRemotes() Assert.IsTrue(nodes.Contains(prj.GetNodes().ElementAt(1))); } + /// + /// Test for daemon nodes projection. + /// + [Test] + public void TestForDaemons() + { + Assert.AreEqual(0, _grid1.GetCluster().ForDaemons().GetNodes().Count); + + using (var ignite = Ignition.Start(new IgniteConfiguration(TestUtils.GetTestConfiguration()) + { + SpringConfigUrl = "config\\compute\\compute-grid1.xml", + GridName = "daemonGrid", + IsDaemon = true + }) + ) + { + var prj = _grid1.GetCluster().ForDaemons(); + + Assert.AreEqual(1, prj.GetNodes().Count); + Assert.AreEqual(ignite.GetCluster().GetLocalNode().Id, prj.GetNode().Id); + + Assert.IsTrue(prj.GetNode().IsDaemon); + Assert.IsTrue(ignite.GetCluster().GetLocalNode().IsDaemon); + } + } + /// /// Test for host nodes projection. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cluster/IClusterGroup.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cluster/IClusterGroup.cs index bb9adbf485c79..31847c1f5c318 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cluster/IClusterGroup.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cluster/IClusterGroup.cs @@ -141,6 +141,20 @@ public interface IClusterGroup /// Grid projection consisting from the nodes in this projection excluding the local node. IClusterGroup ForRemotes(); + /// + /// Gets a cluster group consisting of the daemon nodes. + /// + /// Daemon nodes are the usual grid nodes that participate in topology but not visible on the main APIs, + /// i.e. they are not part of any cluster group. The only way to see daemon nodes is to use this method. + /// + /// Daemon nodes are used primarily for management and monitoring functionality that + /// is build on Ignite and needs to participate in the topology, but also needs to be + /// excluded from the "normal" topology, so that it won't participate in the task execution + /// or in-memory data grid storage. + /// + /// Cluster group consisting of the daemon nodes. + IClusterGroup ForDaemons(); + /// /// Gets grid projection consisting from the nodes in this projection residing on the /// same host as given node. diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs index 95497f337d920..dbb3cb3a59ac6 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs @@ -280,6 +280,12 @@ public IClusterGroup ForRemotes() return GetClusterGroup(UU.ProjectionForRemotes(Target)); } + /** */ + public IClusterGroup ForDaemons() + { + return GetClusterGroup(UU.ProjectionForDaemons(Target)); + } + /** */ public IClusterGroup ForHost(IClusterNode node) { diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs index d516056042eb2..0fac417a101ad 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs @@ -272,6 +272,12 @@ public IClusterGroup ForRemotes() return _prj.ForRemotes(); } + /** */ + public IClusterGroup ForDaemons() + { + return _prj.ForDaemons(); + } + /** */ public IClusterGroup ForHost(IClusterNode node) { @@ -482,7 +488,9 @@ public void DestroyCache(string name) /** */ public IClusterNode GetLocalNode() { - return _locNode ?? (_locNode = GetNodes().FirstOrDefault(x => x.IsLocal)); + return _locNode ?? (_locNode = + GetNodes().FirstOrDefault(x => x.IsLocal) ?? + ForDaemons().GetNodes().FirstOrDefault(x => x.IsLocal)); } /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs index 949d31c4b8d83..12de8476c3e08 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs @@ -160,6 +160,12 @@ public IClusterGroup ForRemotes() return _ignite.GetCluster().ForRemotes(); } + /** */ + public IClusterGroup ForDaemons() + { + return _ignite.GetCluster().ForDaemons(); + } + /** */ public IClusterGroup ForHost(IClusterNode node) { From 38d1d0491ab9e6858a58b90d0e5d892d9fc0e284 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Mon, 26 Sep 2016 11:04:56 +0300 Subject: [PATCH 164/487] IGNITE-3876: ODBC: Better handling of NULL values for output parameters. This closes #1099. --- modules/platforms/cpp/odbc-test/Makefile.am | 1 + .../cpp/odbc-test/include/test_utils.h | 8 + .../odbc-test/project/vs/odbc-test.vcxproj | 1 + .../project/vs/odbc-test.vcxproj.filters | 3 + .../cpp/odbc-test/src/api_robustness_test.cpp | 1006 +++++++++++++++++ .../odbc/include/ignite/odbc/common_types.h | 8 +- .../platforms/cpp/odbc/src/entry_points.cpp | 19 +- modules/platforms/cpp/odbc/src/odbc.cpp | 199 ++-- modules/platforms/cpp/odbc/src/statement.cpp | 2 +- 9 files changed, 1145 insertions(+), 102 deletions(-) create mode 100644 modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index a22e247c92f72..ccf1192445563 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -72,6 +72,7 @@ ignite_odbc_tests_SOURCES = \ src/sql_types_test.cpp \ src/sql_date_time_functions_test.cpp \ src/sql_outer_join_test.cpp \ + src/api_robustness_test.cpp \ ../odbc/src/cursor.cpp \ ../odbc/src/config/connection_info.cpp \ ../odbc/src/app/application_data_buffer.cpp \ diff --git a/modules/platforms/cpp/odbc-test/include/test_utils.h b/modules/platforms/cpp/odbc-test/include/test_utils.h index e8cd089287af1..e239f45ee12af 100644 --- a/modules/platforms/cpp/odbc-test/include/test_utils.h +++ b/modules/platforms/cpp/odbc-test/include/test_utils.h @@ -27,6 +27,14 @@ #include +#define ODBC_FAIL_ON_ERROR(ret, type, handle) \ + if (!SQL_SUCCEEDED(ret)) \ + { \ + Ignition::StopAll(true); \ + BOOST_FAIL(GetOdbcErrorMessage(type, handle)); \ + } + + namespace ignite { /** Read buffer size. */ diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj index 98a1e587c6553..91603dc5f01f9 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj @@ -161,6 +161,7 @@ + diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters index f348ee7dd5ee6..eef6abbd60bfa 100644 --- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters +++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters @@ -112,6 +112,9 @@ Code + + Code + diff --git a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp new file mode 100644 index 0000000000000..008cf25f26c7e --- /dev/null +++ b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp @@ -0,0 +1,1006 @@ +/* + * 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. + */ + +#ifdef _WIN32 +# include +#endif + +#include +#include + +#include +#include + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include "ignite/ignite.h" +#include "ignite/ignition.h" +#include "ignite/impl/binary/binary_utils.h" + +#include "test_type.h" +#include "test_utils.h" + +using namespace ignite; +using namespace ignite::cache; +using namespace ignite::cache::query; +using namespace ignite::common; + +using namespace boost::unit_test; + +using ignite::impl::binary::BinaryUtils; + +/** + * Test setup fixture. + */ +struct ApiRobustnessTestSuiteFixture +{ + void Prepare() + { + // Allocate an environment handle + SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env); + + BOOST_REQUIRE(env != NULL); + + // We want ODBC 3 support + SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast(SQL_OV_ODBC3), 0); + + // Allocate a connection handle + SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc); + + BOOST_REQUIRE(dbc != NULL); + } + + /** + * Establish connection to node. + * + * @param connectStr Connection string. + */ + void Connect(const std::string& connectStr) + { + Prepare(); + + // Connect string + std::vector connectStr0; + + connectStr0.reserve(connectStr.size() + 1); + std::copy(connectStr.begin(), connectStr.end(), std::back_inserter(connectStr0)); + + SQLCHAR outstr[ODBC_BUFFER_SIZE]; + SQLSMALLINT outstrlen; + + // Connecting to ODBC server. + SQLRETURN ret = SQLDriverConnect(dbc, NULL, &connectStr0[0], static_cast(connectStr0.size()), + outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + // Allocate a statement handle + SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt); + + BOOST_REQUIRE(stmt != NULL); + } + + void Disconnect() + { + // Releasing statement handle. + SQLFreeHandle(SQL_HANDLE_STMT, stmt); + + // Disconneting from the server. + SQLDisconnect(dbc); + + // Releasing allocated handles. + SQLFreeHandle(SQL_HANDLE_DBC, dbc); + SQLFreeHandle(SQL_HANDLE_ENV, env); + } + + static Ignite StartNode(const char* name, const char* config) + { + IgniteConfiguration cfg; + + cfg.jvmOpts.push_back("-Xdebug"); + cfg.jvmOpts.push_back("-Xnoagent"); + cfg.jvmOpts.push_back("-Djava.compiler=NONE"); + cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005"); + cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError"); + cfg.jvmOpts.push_back("-Duser.timezone=GMT"); + +#ifdef IGNITE_TESTS_32 + cfg.jvmInitMem = 256; + cfg.jvmMaxMem = 768; +#else + cfg.jvmInitMem = 1024; + cfg.jvmMaxMem = 4096; +#endif + + char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH"); + + BOOST_REQUIRE(cfgPath != 0); + + cfg.springCfgPath.assign(cfgPath).append("/").append(config); + + IgniteError err; + + return Ignition::Start(cfg, name); + } + + static Ignite StartAdditionalNode(const char* name) + { + return StartNode(name, "queries-test-noodbc.xml"); + } + + /** + * Constructor. + */ + ApiRobustnessTestSuiteFixture() : + testCache(0), + env(NULL), + dbc(NULL), + stmt(NULL) + { + grid = StartNode("NodeMain", "queries-test.xml"); + + testCache = grid.GetCache("cache"); + } + + /** + * Destructor. + */ + ~ApiRobustnessTestSuiteFixture() + { + Disconnect(); + + Ignition::StopAll(true); + } + + /** Node started during the test. */ + Ignite grid; + + /** Test cache instance. */ + Cache testCache; + + /** ODBC Environment. */ + SQLHENV env; + + /** ODBC Connect. */ + SQLHDBC dbc; + + /** ODBC Statement. */ + SQLHSTMT stmt; +}; + +BOOST_FIXTURE_TEST_SUITE(ApiRobustnessTestSuite, ApiRobustnessTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(TestSQLDriverConnect) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Prepare(); + + SQLCHAR connectStr[] = "DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"; + + SQLCHAR outStr[ODBC_BUFFER_SIZE]; + SQLSMALLINT outStrLen; + + // Normal connect. + SQLRETURN ret = SQLDriverConnect(dbc, NULL, connectStr, sizeof(connectStr), + outStr, sizeof(outStr), &outStrLen, SQL_DRIVER_COMPLETE); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLDisconnect(dbc); + + // Null out string resulting length. + SQLDriverConnect(dbc, NULL, connectStr, sizeof(connectStr), outStr, sizeof(outStr), 0, SQL_DRIVER_COMPLETE); + + SQLDisconnect(dbc); + + // Null out string buffer length. + SQLDriverConnect(dbc, NULL, connectStr, sizeof(connectStr), outStr, 0, &outStrLen, SQL_DRIVER_COMPLETE); + + SQLDisconnect(dbc); + + // Null out string. + SQLDriverConnect(dbc, NULL, connectStr, sizeof(connectStr), 0, sizeof(outStr), &outStrLen, SQL_DRIVER_COMPLETE); + + SQLDisconnect(dbc); + + // Null all. + SQLDriverConnect(dbc, NULL, connectStr, sizeof(connectStr), 0, 0, 0, SQL_DRIVER_COMPLETE); + + SQLDisconnect(dbc); +} + +BOOST_AUTO_TEST_CASE(TestSQLConnect) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR buffer[ODBC_BUFFER_SIZE]; + SQLSMALLINT resLen = 0; + + // Everyting is ok. + SQLRETURN ret = SQLGetInfo(dbc, SQL_DRIVER_NAME, buffer, ODBC_BUFFER_SIZE, &resLen); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + // Resulting length is null. + SQLGetInfo(dbc, SQL_DRIVER_NAME, buffer, ODBC_BUFFER_SIZE, 0); + + // Buffer length is null. + SQLGetInfo(dbc, SQL_DRIVER_NAME, buffer, 0, &resLen); + + // Buffer is null. + SQLGetInfo(dbc, SQL_DRIVER_NAME, 0, ODBC_BUFFER_SIZE, &resLen); + + // Unknown info. + SQLGetInfo(dbc, -1, buffer, ODBC_BUFFER_SIZE, &resLen); + + // All nulls. + SQLGetInfo(dbc, SQL_DRIVER_NAME, 0, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLPrepare) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR sql[] = "SELECT strField FROM TestType"; + + // Everyting is ok. + SQLRETURN ret = SQLPrepare(stmt, sql, sizeof(sql)); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLCloseCursor(stmt); + + // Value length is null. + SQLPrepare(stmt, sql, 0); + + SQLCloseCursor(stmt); + + // Value is null. + SQLPrepare(stmt, 0, sizeof(sql)); + + SQLCloseCursor(stmt); + + // All nulls. + SQLPrepare(stmt, 0, 0); + + SQLCloseCursor(stmt); +} + +BOOST_AUTO_TEST_CASE(TestSQLExecDirect) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR sql[] = "SELECT strField FROM TestType"; + + // Everyting is ok. + SQLRETURN ret = SQLExecDirect(stmt, sql, sizeof(sql)); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLCloseCursor(stmt); + + // Value length is null. + SQLExecDirect(stmt, sql, 0); + + SQLCloseCursor(stmt); + + // Value is null. + SQLExecDirect(stmt, 0, sizeof(sql)); + + SQLCloseCursor(stmt); + + // All nulls. + SQLExecDirect(stmt, 0, 0); + + SQLCloseCursor(stmt); +} + +BOOST_AUTO_TEST_CASE(TestSQLExtendedFetch) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + for (int i = 0; i < 100; ++i) + { + TestType obj; + + obj.strField = LexicalCast(i); + + testCache.Put(i, obj); + } + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR sql[] = "SELECT strField FROM TestType"; + + SQLRETURN ret = SQLExecDirect(stmt, sql, sizeof(sql)); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLULEN rowCount; + SQLUSMALLINT rowStatus[16]; + + // Everyting is ok. + ret = SQLExtendedFetch(stmt, SQL_FETCH_NEXT, 0, &rowCount, rowStatus); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + // Row count is null. + SQLExtendedFetch(stmt, SQL_FETCH_NEXT, 0, 0, rowStatus); + + // Row statuses is null. + SQLExtendedFetch(stmt, SQL_FETCH_NEXT, 0, &rowCount, 0); + + // All nulls. + SQLExtendedFetch(stmt, SQL_FETCH_NEXT, 0, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLNumResultCols) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + for (int i = 0; i < 100; ++i) + { + TestType obj; + + obj.strField = LexicalCast(i); + + testCache.Put(i, obj); + } + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR sql[] = "SELECT strField FROM TestType"; + + SQLRETURN ret = SQLExecDirect(stmt, sql, sizeof(sql)); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLSMALLINT columnCount; + + // Everyting is ok. + ret = SQLNumResultCols(stmt, &columnCount); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + // Column count is null. + SQLNumResultCols(stmt, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLTables) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR catalogName[] = ""; + SQLCHAR schemaName[] = ""; + SQLCHAR tableName[] = ""; + SQLCHAR tableType[] = ""; + + // Everithing is ok. + SQLRETURN ret = SQLTables(stmt, catalogName, sizeof(catalogName), schemaName, + sizeof(schemaName), tableName, sizeof(tableName), tableType, sizeof(tableType)); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + // Sizes are nulls. + SQLTables(dbc, catalogName, 0, schemaName, 0, tableName, 0, tableType, 0); + + // Values are nulls. + SQLTables(dbc, 0, sizeof(catalogName), 0, sizeof(schemaName), 0, sizeof(tableName), 0, sizeof(tableType)); + + // All nulls. + SQLTables(dbc, 0, 0, 0, 0, 0, 0, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLColumns) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR catalogName[] = ""; + SQLCHAR schemaName[] = ""; + SQLCHAR tableName[] = ""; + SQLCHAR columnName[] = ""; + + // Everithing is ok. + SQLRETURN ret = SQLColumns(stmt, catalogName, sizeof(catalogName), schemaName, + sizeof(schemaName), tableName, sizeof(tableName), columnName, sizeof(columnName)); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + // Sizes are nulls. + SQLColumns(dbc, catalogName, 0, schemaName, 0, tableName, 0, columnName, 0); + + // Values are nulls. + SQLColumns(dbc, 0, sizeof(catalogName), 0, sizeof(schemaName), 0, sizeof(tableName), 0, sizeof(columnName)); + + // All nulls. + SQLColumns(dbc, 0, 0, 0, 0, 0, 0, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLBindCol) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLINTEGER ind1; + SQLLEN len1 = 0; + + // Everithing is ok. + SQLRETURN ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &ind1, sizeof(ind1), &len1); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + // Size is null. + SQLBindCol(stmt, 1, SQL_C_SLONG, &ind1, 0, &len1); + + // Res size is null. + SQLBindCol(stmt, 2, SQL_C_SLONG, &ind1, sizeof(ind1), 0); + + // Value is null. + SQLBindCol(stmt, 3, SQL_C_SLONG, 0, sizeof(ind1), &len1); + + // All nulls. + SQLBindCol(stmt, 4, SQL_C_SLONG, 0, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLBindParameter) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLINTEGER ind1; + SQLLEN len1 = 0; + + // Everithing is ok. + SQLRETURN ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, + SQL_C_SLONG, SQL_INTEGER, 100, 100, &ind1, sizeof(ind1), &len1); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + // Size is null. + SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_INTEGER, 100, 100, &ind1, 0, &len1); + + // Res size is null. + SQLBindParameter(stmt, 3, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_INTEGER, 100, 100, &ind1, sizeof(ind1), 0); + + // Value is null. + SQLBindParameter(stmt, 4, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_INTEGER, 100, 100, 0, sizeof(ind1), &len1); + + // All nulls. + SQLBindParameter(stmt, 5, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_INTEGER, 100, 100, 0, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLNativeSql) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR sql[] = "SELECT strField FROM TestType"; + SQLCHAR buffer[ODBC_BUFFER_SIZE]; + SQLINTEGER resLen = 0; + + // Everithing is ok. + SQLRETURN ret = SQLNativeSql(dbc, sql, sizeof(sql), buffer, sizeof(buffer), &resLen); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + // Value size is null. + SQLNativeSql(dbc, sql, 0, buffer, sizeof(buffer), &resLen); + + // Buffer size is null. + SQLNativeSql(dbc, sql, sizeof(sql), buffer, 0, &resLen); + + // Res size is null. + SQLNativeSql(dbc, sql, sizeof(sql), buffer, sizeof(buffer), 0); + + // Value is null. + SQLNativeSql(dbc, sql, 0, buffer, sizeof(buffer), &resLen); + + // Buffer is null. + SQLNativeSql(dbc, sql, sizeof(sql), 0, sizeof(buffer), &resLen); + + // All nulls. + SQLNativeSql(dbc, sql, 0, 0, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLColAttribute) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR sql[] = "SELECT strField FROM TestType"; + + SQLRETURN ret = SQLExecDirect(stmt, sql, sizeof(sql)); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLCHAR buffer[ODBC_BUFFER_SIZE]; + SQLSMALLINT resLen = 0; + SQLLEN numericAttr = 0; + + // Everithing is ok. Character attribute. + ret = SQLColAttribute(stmt, 1, SQL_COLUMN_TABLE_NAME, buffer, sizeof(buffer), &resLen, &numericAttr); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + // Everithing is ok. Numeric attribute. + ret = SQLColAttribute(stmt, 1, SQL_DESC_COUNT, buffer, sizeof(buffer), &resLen, &numericAttr); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLColAttribute(stmt, 1, SQL_COLUMN_TABLE_NAME, buffer, sizeof(buffer), &resLen, 0); + SQLColAttribute(stmt, 1, SQL_COLUMN_TABLE_NAME, buffer, sizeof(buffer), 0, &numericAttr); + SQLColAttribute(stmt, 1, SQL_COLUMN_TABLE_NAME, buffer, 0, &resLen, &numericAttr); + SQLColAttribute(stmt, 1, SQL_COLUMN_TABLE_NAME, 0, sizeof(buffer), &resLen, &numericAttr); + SQLColAttribute(stmt, 1, SQL_COLUMN_TABLE_NAME, 0, 0, 0, 0); + + SQLColAttribute(stmt, 1, SQL_DESC_COUNT, buffer, sizeof(buffer), &resLen, 0); + SQLColAttribute(stmt, 1, SQL_DESC_COUNT, buffer, sizeof(buffer), 0, &numericAttr); + SQLColAttribute(stmt, 1, SQL_DESC_COUNT, buffer, 0, &resLen, &numericAttr); + SQLColAttribute(stmt, 1, SQL_DESC_COUNT, 0, sizeof(buffer), &resLen, &numericAttr); + SQLColAttribute(stmt, 1, SQL_DESC_COUNT, 0, 0, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLDescribeCol) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR sql[] = "SELECT strField FROM TestType"; + + SQLRETURN ret = SQLExecDirect(stmt, sql, sizeof(sql)); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLCHAR columnName[ODBC_BUFFER_SIZE]; + SQLSMALLINT columnNameLen = 0; + SQLSMALLINT dataType = 0; + SQLULEN columnSize = 0; + SQLSMALLINT decimalDigits = 0; + SQLSMALLINT nullable = 0; + + // Everithing is ok. + ret = SQLDescribeCol(stmt, 1, columnName, sizeof(columnName), + &columnNameLen, &dataType, &columnSize, &decimalDigits, &nullable); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLDescribeCol(stmt, 1, 0, sizeof(columnName), &columnNameLen, &dataType, &columnSize, &decimalDigits, &nullable); + SQLDescribeCol(stmt, 1, columnName, 0, &columnNameLen, &dataType, &columnSize, &decimalDigits, &nullable); + SQLDescribeCol(stmt, 1, columnName, sizeof(columnName), 0, &dataType, &columnSize, &decimalDigits, &nullable); + SQLDescribeCol(stmt, 1, columnName, sizeof(columnName), &columnNameLen, 0, &columnSize, &decimalDigits, &nullable); + SQLDescribeCol(stmt, 1, columnName, sizeof(columnName), &columnNameLen, &dataType, 0, &decimalDigits, &nullable); + SQLDescribeCol(stmt, 1, columnName, sizeof(columnName), &columnNameLen, &dataType, &columnSize, 0, &nullable); + SQLDescribeCol(stmt, 1, columnName, sizeof(columnName), &columnNameLen, &dataType, &columnSize, &decimalDigits, 0); + SQLDescribeCol(stmt, 1, 0, 0, 0, 0, 0, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLRowCount) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR sql[] = "SELECT strField FROM TestType"; + + SQLRETURN ret = SQLExecDirect(stmt, sql, sizeof(sql)); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLLEN rows = 0; + + // Everithing is ok. + ret = SQLRowCount(stmt, &rows); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLRowCount(stmt, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLForeignKeys) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR catalogName[] = ""; + SQLCHAR schemaName[] = "cache"; + SQLCHAR tableName[] = "TestType"; + + // Everithing is ok. + SQLRETURN ret = SQLForeignKeys(stmt, catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), + tableName, sizeof(tableName), catalogName, sizeof(catalogName), + schemaName, sizeof(schemaName), tableName, sizeof(tableName)); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLCloseCursor(stmt); + + SQLForeignKeys(stmt, 0, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName), + catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName)); + + SQLCloseCursor(stmt); + + SQLForeignKeys(stmt, catalogName, 0, schemaName, sizeof(schemaName), tableName, sizeof(tableName), + catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName)); + + SQLCloseCursor(stmt); + + SQLForeignKeys(stmt, catalogName, sizeof(catalogName), 0, sizeof(schemaName), tableName, sizeof(tableName), + catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName)); + + SQLCloseCursor(stmt); + + SQLForeignKeys(stmt, catalogName, sizeof(catalogName), schemaName, 0, tableName, sizeof(tableName), + catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName)); + + SQLCloseCursor(stmt); + + SQLForeignKeys(stmt, catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), 0, sizeof(tableName), + catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName)); + + SQLCloseCursor(stmt); + + SQLForeignKeys(stmt, catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, 0, catalogName, + sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName)); + + SQLCloseCursor(stmt); + + SQLForeignKeys(stmt, catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName), + 0, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName)); + + SQLCloseCursor(stmt); + + SQLForeignKeys(stmt, catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName), + catalogName, 0, schemaName, sizeof(schemaName), tableName, sizeof(tableName)); + + SQLCloseCursor(stmt); + + SQLForeignKeys(stmt, catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName), + catalogName, sizeof(catalogName), 0, sizeof(schemaName), tableName, sizeof(tableName)); + + SQLCloseCursor(stmt); + + SQLForeignKeys(stmt, catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName), + catalogName, sizeof(catalogName), schemaName, 0, tableName, sizeof(tableName)); + + SQLCloseCursor(stmt); + + SQLForeignKeys(stmt, catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName), + catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), 0, sizeof(tableName)); + + SQLCloseCursor(stmt); + + SQLForeignKeys(stmt, catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName), + catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, 0); + + SQLCloseCursor(stmt); + + SQLForeignKeys(stmt, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0); + + SQLCloseCursor(stmt); +} + +BOOST_AUTO_TEST_CASE(TestSQLGetStmtAttr) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR buffer[ODBC_BUFFER_SIZE]; + SQLINTEGER resLen = 0; + + // Everithing is ok. + SQLRETURN ret = SQLGetStmtAttr(stmt, SQL_ATTR_ROW_ARRAY_SIZE, buffer, sizeof(buffer), &resLen); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLGetStmtAttr(stmt, SQL_ATTR_ROW_ARRAY_SIZE, 0, sizeof(buffer), &resLen); + SQLGetStmtAttr(stmt, SQL_ATTR_ROW_ARRAY_SIZE, buffer, 0, &resLen); + SQLGetStmtAttr(stmt, SQL_ATTR_ROW_ARRAY_SIZE, buffer, sizeof(buffer), 0); + SQLGetStmtAttr(stmt, SQL_ATTR_ROW_ARRAY_SIZE, 0, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLSetStmtAttr) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLULEN val = 1; + + // Everithing is ok. + SQLRETURN ret = SQLSetStmtAttr(stmt, SQL_ATTR_ROW_ARRAY_SIZE, reinterpret_cast(val), sizeof(val)); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLSetStmtAttr(stmt, SQL_ATTR_ROW_ARRAY_SIZE, 0, sizeof(val)); + SQLSetStmtAttr(stmt, SQL_ATTR_ROW_ARRAY_SIZE, reinterpret_cast(val), 0); + SQLSetStmtAttr(stmt, SQL_ATTR_ROW_ARRAY_SIZE, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLPrimaryKeys) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR catalogName[] = ""; + SQLCHAR schemaName[] = "cache"; + SQLCHAR tableName[] = "TestType"; + + // Everithing is ok. + SQLRETURN ret = SQLPrimaryKeys(stmt, catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), + tableName, sizeof(tableName)); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLPrimaryKeys(stmt, 0, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, sizeof(tableName)); + SQLPrimaryKeys(stmt, catalogName, 0, schemaName, sizeof(schemaName), tableName, sizeof(tableName)); + SQLPrimaryKeys(stmt, catalogName, sizeof(catalogName), 0, sizeof(schemaName), tableName, sizeof(tableName)); + SQLPrimaryKeys(stmt, catalogName, sizeof(catalogName), schemaName, 0, tableName, sizeof(tableName)); + SQLPrimaryKeys(stmt, catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), 0, sizeof(tableName)); + SQLPrimaryKeys(stmt, catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, 0); + SQLPrimaryKeys(stmt, 0, 0, 0, 0, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLNumParams) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR sql[] = "SELECT strField FROM TestType"; + + // Everyting is ok. + SQLRETURN ret = SQLPrepare(stmt, sql, sizeof(sql)); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLSMALLINT params; + + // Everithing is ok. + ret = SQLNumParams(stmt, ¶ms); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLNumParams(stmt, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLGetDiagField) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + // Should fail. + SQLRETURN ret = SQLGetTypeInfo(stmt, SQL_INTERVAL_MONTH); + + BOOST_REQUIRE_EQUAL(ret, SQL_ERROR); + + SQLCHAR buffer[ODBC_BUFFER_SIZE]; + SQLSMALLINT resLen = 0; + + // Everithing is ok + ret = SQLGetDiagField(SQL_HANDLE_STMT, stmt, 1, SQL_DIAG_MESSAGE_TEXT, buffer, sizeof(buffer), &resLen); + + BOOST_REQUIRE_EQUAL(ret, SQL_SUCCESS); + + SQLGetDiagField(SQL_HANDLE_STMT, stmt, 1, SQL_DIAG_MESSAGE_TEXT, 0, sizeof(buffer), &resLen); + SQLGetDiagField(SQL_HANDLE_STMT, stmt, 1, SQL_DIAG_MESSAGE_TEXT, buffer, 0, &resLen); + SQLGetDiagField(SQL_HANDLE_STMT, stmt, 1, SQL_DIAG_MESSAGE_TEXT, buffer, sizeof(buffer), 0); + SQLGetDiagField(SQL_HANDLE_STMT, stmt, 1, SQL_DIAG_MESSAGE_TEXT, 0, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLGetDiagRec) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + // Should fail. + SQLRETURN ret = SQLGetTypeInfo(stmt, SQL_INTERVAL_MONTH); + + BOOST_REQUIRE_EQUAL(ret, SQL_ERROR); + + SQLCHAR state[ODBC_BUFFER_SIZE]; + SQLINTEGER nativeError = 0; + SQLCHAR message[ODBC_BUFFER_SIZE]; + SQLSMALLINT messageLen = 0; + + // Everithing is ok + ret = SQLGetDiagRec(SQL_HANDLE_STMT, stmt, 1, state, &nativeError, message, sizeof(message), &messageLen); + + BOOST_REQUIRE_EQUAL(ret, SQL_SUCCESS); + + SQLGetDiagRec(SQL_HANDLE_STMT, stmt, 1, 0, &nativeError, message, sizeof(message), &messageLen); + SQLGetDiagRec(SQL_HANDLE_STMT, stmt, 1, state, 0, message, sizeof(message), &messageLen); + SQLGetDiagRec(SQL_HANDLE_STMT, stmt, 1, state, &nativeError, 0, sizeof(message), &messageLen); + SQLGetDiagRec(SQL_HANDLE_STMT, stmt, 1, state, &nativeError, message, 0, &messageLen); + SQLGetDiagRec(SQL_HANDLE_STMT, stmt, 1, state, &nativeError, message, sizeof(message), 0); + SQLGetDiagRec(SQL_HANDLE_STMT, stmt, 1, 0, 0, 0, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLGetData) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + for (int i = 0; i < 100; ++i) + { + TestType obj; + + obj.strField = LexicalCast(i); + + testCache.Put(i, obj); + } + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR sql[] = "SELECT strField FROM TestType"; + + SQLRETURN ret = SQLExecDirect(stmt, sql, sizeof(sql)); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + ret = SQLFetch(stmt); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLCHAR buffer[ODBC_BUFFER_SIZE]; + SQLLEN resLen = 0; + + // Everything is ok. + ret = SQLGetData(stmt, 1, SQL_C_CHAR, buffer, sizeof(buffer), &resLen); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLFetch(stmt); + + SQLGetData(stmt, 1, SQL_C_CHAR, 0, sizeof(buffer), &resLen); + + SQLFetch(stmt); + + SQLGetData(stmt, 1, SQL_C_CHAR, buffer, 0, &resLen); + + SQLFetch(stmt); + + SQLGetData(stmt, 1, SQL_C_CHAR, buffer, sizeof(buffer), 0); + + SQLFetch(stmt); + + SQLGetData(stmt, 1, SQL_C_CHAR, 0, 0, 0); + + SQLFetch(stmt); +} + +BOOST_AUTO_TEST_CASE(TestSQLGetEnvAttr) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR buffer[ODBC_BUFFER_SIZE]; + SQLINTEGER resLen = 0; + + // Everything is ok. + SQLRETURN ret = SQLGetEnvAttr(env, SQL_ATTR_ODBC_VERSION, buffer, sizeof(buffer), &resLen); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_ENV, env); + + SQLGetEnvAttr(env, SQL_ATTR_ODBC_VERSION, 0, sizeof(buffer), &resLen); + SQLGetEnvAttr(env, SQL_ATTR_ODBC_VERSION, buffer, 0, &resLen); + SQLGetEnvAttr(env, SQL_ATTR_ODBC_VERSION, buffer, sizeof(buffer), 0); + SQLGetEnvAttr(env, SQL_ATTR_ODBC_VERSION, 0, 0, 0); +} + +BOOST_AUTO_TEST_CASE(TestSQLSpecialColumns) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR catalogName[] = ""; + SQLCHAR schemaName[] = "cache"; + SQLCHAR tableName[] = "TestType"; + + // Everything is ok. + SQLRETURN ret = SQLSpecialColumns(stmt, SQL_BEST_ROWID, catalogName, sizeof(catalogName), + schemaName, sizeof(schemaName), tableName, sizeof(tableName), SQL_SCOPE_CURROW, SQL_NO_NULLS); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt); + + SQLCloseCursor(stmt); + + SQLSpecialColumns(stmt, SQL_BEST_ROWID, 0, sizeof(catalogName), schemaName, sizeof(schemaName), tableName, + sizeof(tableName), SQL_SCOPE_CURROW, SQL_NO_NULLS); + + SQLCloseCursor(stmt); + + SQLSpecialColumns(stmt, SQL_BEST_ROWID, catalogName, 0, schemaName, sizeof(schemaName), tableName, + sizeof(tableName), SQL_SCOPE_CURROW, SQL_NO_NULLS); + + SQLCloseCursor(stmt); + + SQLSpecialColumns(stmt, SQL_BEST_ROWID, catalogName, sizeof(catalogName), 0, sizeof(schemaName), tableName, + sizeof(tableName), SQL_SCOPE_CURROW, SQL_NO_NULLS); + + SQLCloseCursor(stmt); + + SQLSpecialColumns(stmt, SQL_BEST_ROWID, catalogName, sizeof(catalogName), schemaName, 0, tableName, + sizeof(tableName), SQL_SCOPE_CURROW, SQL_NO_NULLS); + + SQLCloseCursor(stmt); + + SQLSpecialColumns(stmt, SQL_BEST_ROWID, catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), + 0, sizeof(tableName), SQL_SCOPE_CURROW, SQL_NO_NULLS); + + SQLCloseCursor(stmt); + + SQLSpecialColumns(stmt, SQL_BEST_ROWID, catalogName, sizeof(catalogName), schemaName, sizeof(schemaName), + tableName, 0, SQL_SCOPE_CURROW, SQL_NO_NULLS); + + SQLCloseCursor(stmt); + + SQLSpecialColumns(stmt, SQL_BEST_ROWID, 0, 0, 0, 0, 0, 0, SQL_SCOPE_CURROW, SQL_NO_NULLS); + + SQLCloseCursor(stmt); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h index 250eaf2c16c0c..b01ec76fd1e1a 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h @@ -19,17 +19,13 @@ #define _IGNITE_ODBC_COMMON_TYPES #include +#include "system/odbc_constants.h" namespace ignite { namespace odbc { - -#ifdef _WIN64 - typedef long long SqlLen; -#else - typedef long SqlLen; -#endif + typedef SQLLEN SqlLen; /** * SQL result. diff --git a/modules/platforms/cpp/odbc/src/entry_points.cpp b/modules/platforms/cpp/odbc/src/entry_points.cpp index f6195e1168b44..c3d24bb960c9b 100644 --- a/modules/platforms/cpp/odbc/src/entry_points.cpp +++ b/modules/platforms/cpp/odbc/src/entry_points.cpp @@ -218,6 +218,8 @@ SQLRETURN SQL_API SQLNativeSql(SQLHDBC conn, outQueryBuffer, outQueryBufferLen, outQueryLen); } + +#if defined _WIN64 || !defined _WIN32 SQLRETURN SQL_API SQLColAttribute(SQLHSTMT stmt, SQLUSMALLINT columnNum, SQLUSMALLINT fieldId, @@ -225,9 +227,18 @@ SQLRETURN SQL_API SQLColAttribute(SQLHSTMT stmt, SQLSMALLINT bufferLen, SQLSMALLINT* strAttrLen, SQLLEN* numericAttr) +#else +SQLRETURN SQL_API SQLColAttribute(SQLHSTMT stmt, + SQLUSMALLINT columnNum, + SQLUSMALLINT fieldId, + SQLPOINTER strAttr, + SQLSMALLINT bufferLen, + SQLSMALLINT* strAttrLen, + SQLPOINTER numericAttr) +#endif { return ignite::SQLColAttribute(stmt, columnNum, fieldId, - strAttr, bufferLen, strAttrLen, numericAttr); + strAttr, bufferLen, strAttrLen, (SQLLEN*)numericAttr); } SQLRETURN SQL_API SQLDescribeCol(SQLHSTMT stmt, @@ -373,15 +384,15 @@ SQLRETURN SQL_API SQLGetEnvAttr(SQLHENV env, } SQLRETURN SQL_API SQLSpecialColumns(SQLHSTMT stmt, - SQLSMALLINT idType, + SQLUSMALLINT idType, SQLCHAR* catalogName, SQLSMALLINT catalogNameLen, SQLCHAR* schemaName, SQLSMALLINT schemaNameLen, SQLCHAR* tableName, SQLSMALLINT tableNameLen, - SQLSMALLINT scope, - SQLSMALLINT nullable) + SQLUSMALLINT scope, + SQLUSMALLINT nullable) { return ignite::SQLSpecialColumns(stmt, idType, catalogName, catalogNameLen, schemaName, schemaNameLen, tableName, diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp index fd35cbab6875e..74d0f9d833f4d 100644 --- a/modules/platforms/cpp/odbc/src/odbc.cpp +++ b/modules/platforms/cpp/odbc/src/odbc.cpp @@ -39,8 +39,8 @@ namespace ignite SQLSMALLINT infoValueMax, SQLSMALLINT* length) { - using ignite::odbc::Connection; - using ignite::odbc::config::ConnectionInfo; + using odbc::Connection; + using odbc::config::ConnectionInfo; LOG_MSG("SQLGetInfo called: %d (%s), %p, %d, %p\n", infoType, ConnectionInfo::InfoTypeToString(infoType), @@ -81,7 +81,7 @@ namespace ignite SQLRETURN SQLAllocEnv(SQLHENV* env) { - using ignite::odbc::Environment; + using odbc::Environment; LOG_MSG("SQLAllocEnv called\n"); @@ -92,8 +92,8 @@ namespace ignite SQLRETURN SQLAllocConnect(SQLHENV env, SQLHDBC* conn) { - using ignite::odbc::Environment; - using ignite::odbc::Connection; + using odbc::Environment; + using odbc::Connection; LOG_MSG("SQLAllocConnect called\n"); @@ -116,8 +116,8 @@ namespace ignite SQLRETURN SQLAllocStmt(SQLHDBC conn, SQLHSTMT* stmt) { - using ignite::odbc::Connection; - using ignite::odbc::Statement; + using odbc::Connection; + using odbc::Statement; LOG_MSG("SQLAllocStmt called\n"); @@ -158,7 +158,7 @@ namespace ignite SQLRETURN SQLFreeEnv(SQLHENV env) { - using ignite::odbc::Environment; + using odbc::Environment; LOG_MSG("SQLFreeEnv called\n"); @@ -174,7 +174,7 @@ namespace ignite SQLRETURN SQLFreeConnect(SQLHDBC conn) { - using ignite::odbc::Connection; + using odbc::Connection; LOG_MSG("SQLFreeConnect called\n"); @@ -190,7 +190,7 @@ namespace ignite SQLRETURN SQLFreeStmt(SQLHSTMT stmt, SQLUSMALLINT option) { - using ignite::odbc::Statement; + using odbc::Statement; LOG_MSG("SQLFreeStmt called\n"); @@ -236,7 +236,7 @@ namespace ignite SQLRETURN SQLCloseCursor(SQLHSTMT stmt) { - using ignite::odbc::Statement; + using odbc::Statement; LOG_MSG("SQLCloseCursor called\n"); @@ -311,9 +311,9 @@ namespace ignite SQLCHAR* auth, SQLSMALLINT authLen) { - using ignite::odbc::Connection; - using ignite::odbc::config::Configuration; - using ignite::utility::SqlStringToString; + using odbc::Connection; + using odbc::config::Configuration; + using utility::SqlStringToString; LOG_MSG("SQLConnect called\n"); @@ -335,7 +335,7 @@ namespace ignite SQLRETURN SQLDisconnect(SQLHDBC conn) { - using ignite::odbc::Connection; + using odbc::Connection; LOG_MSG("SQLDisconnect called\n"); @@ -351,8 +351,8 @@ namespace ignite SQLRETURN SQLPrepare(SQLHSTMT stmt, SQLCHAR* query, SQLINTEGER queryLen) { - using ignite::odbc::Statement; - using ignite::utility::SqlStringToString; + using odbc::Statement; + using utility::SqlStringToString; LOG_MSG("SQLPrepare called\n"); @@ -372,7 +372,7 @@ namespace ignite SQLRETURN SQLExecute(SQLHSTMT stmt) { - using ignite::odbc::Statement; + using odbc::Statement; LOG_MSG("SQLExecute called\n"); @@ -388,8 +388,8 @@ namespace ignite SQLRETURN SQLExecDirect(SQLHSTMT stmt, SQLCHAR* query, SQLINTEGER queryLen) { - using ignite::odbc::Statement; - using ignite::utility::SqlStringToString; + using odbc::Statement; + using utility::SqlStringToString; LOG_MSG("SQLExecDirect called\n"); @@ -414,10 +414,10 @@ namespace ignite SQLLEN bufferLength, SQLLEN* strLengthOrIndicator) { - using namespace ignite::odbc::type_traits; + using namespace odbc::type_traits; - using ignite::odbc::Statement; - using ignite::odbc::app::ApplicationDataBuffer; + using odbc::Statement; + using odbc::app::ApplicationDataBuffer; LOG_MSG("SQLBindCol called: index=%d, type=%d\n", colNum, targetType); @@ -448,7 +448,7 @@ namespace ignite SQLRETURN SQLFetch(SQLHSTMT stmt) { - using ignite::odbc::Statement; + using odbc::Statement; LOG_MSG("SQLFetch called\n"); @@ -483,7 +483,7 @@ namespace ignite SQLRETURN res = SQLFetchScroll(stmt, orientation, offset); - if (res == SQL_SUCCESS || res == SQL_NO_DATA) + if (res == SQL_SUCCESS) { if (rowCount) *rowCount = 1; @@ -491,14 +491,16 @@ namespace ignite if (rowStatusArray) rowStatusArray[0] = SQL_ROW_SUCCESS; } + else if (res == SQL_NO_DATA && rowCount) + *rowCount = 0; return res; } SQLRETURN SQLNumResultCols(SQLHSTMT stmt, SQLSMALLINT *columnNum) { - using ignite::odbc::Statement; - using ignite::odbc::meta::ColumnMetaVector; + using odbc::Statement; + using odbc::meta::ColumnMetaVector; LOG_MSG("SQLNumResultCols called\n"); @@ -509,7 +511,8 @@ namespace ignite int32_t res = statement->GetColumnNumber(); - *columnNum = static_cast(res); + if (columnNum) + *columnNum = static_cast(res); LOG_MSG("columnNum: %d\n", *columnNum); @@ -526,8 +529,8 @@ namespace ignite SQLCHAR* tableType, SQLSMALLINT tableTypeLen) { - using ignite::odbc::Statement; - using ignite::utility::SqlStringToString; + using odbc::Statement; + using utility::SqlStringToString; LOG_MSG("SQLTables called\n"); @@ -561,8 +564,8 @@ namespace ignite SQLCHAR* columnName, SQLSMALLINT columnNameLen) { - using ignite::odbc::Statement; - using ignite::utility::SqlStringToString; + using odbc::Statement; + using utility::SqlStringToString; LOG_MSG("SQLColumns called\n"); @@ -588,7 +591,7 @@ namespace ignite SQLRETURN SQLMoreResults(SQLHSTMT stmt) { - using ignite::odbc::Statement; + using odbc::Statement; LOG_MSG("SQLMoreResults called\n"); @@ -611,12 +614,12 @@ namespace ignite SQLLEN bufferLen, SQLLEN* resLen) { - using namespace ignite::odbc::type_traits; + using namespace odbc::type_traits; - using ignite::odbc::Statement; - using ignite::odbc::app::ApplicationDataBuffer; - using ignite::odbc::app::Parameter; - using ignite::odbc::type_traits::IsSqlTypeSupported; + using odbc::Statement; + using odbc::app::ApplicationDataBuffer; + using odbc::app::Parameter; + using odbc::type_traits::IsSqlTypeSupported; LOG_MSG("SQLBindParameter called\n"); @@ -628,7 +631,7 @@ namespace ignite if (ioType != SQL_PARAM_INPUT) return SQL_ERROR; - if (*resLen == SQL_DATA_AT_EXEC || *resLen <= SQL_LEN_DATA_AT_EXEC_OFFSET) + if (resLen && (*resLen == SQL_DATA_AT_EXEC || *resLen <= SQL_LEN_DATA_AT_EXEC_OFFSET)) return SQL_ERROR; if (!IsSqlTypeSupported(paramSqlType)) @@ -660,7 +663,7 @@ namespace ignite SQLINTEGER outQueryBufferLen, SQLINTEGER* outQueryLen) { - using namespace ignite::utility; + using namespace utility; LOG_MSG("SQLNativeSql called\n"); @@ -669,7 +672,8 @@ namespace ignite CopyStringToBuffer(in, reinterpret_cast(outQueryBuffer), static_cast(outQueryBufferLen)); - *outQueryLen = std::min(outQueryBufferLen, static_cast(in.size())); + if (outQueryLen) + *outQueryLen = std::min(outQueryBufferLen, static_cast(in.size())); return SQL_SUCCESS; } @@ -682,9 +686,9 @@ namespace ignite SQLSMALLINT* strAttrLen, SQLLEN* numericAttr) { - using ignite::odbc::Statement; - using ignite::odbc::meta::ColumnMetaVector; - using ignite::odbc::meta::ColumnMeta; + using odbc::Statement; + using odbc::meta::ColumnMetaVector; + using odbc::meta::ColumnMeta; LOG_MSG("SQLColAttribute called: %d (%s)\n", fieldId, ColumnMeta::AttrIdToString(fieldId)); @@ -700,7 +704,7 @@ namespace ignite SQLRETURN res = SQLNumResultCols(stmt, &val); - if (res == SQL_SUCCESS) + if (numericAttr && res == SQL_SUCCESS) *numericAttr = val; return res; @@ -722,8 +726,8 @@ namespace ignite SQLSMALLINT* decimalDigits, SQLSMALLINT* nullable) { - using ignite::odbc::Statement; - using ignite::odbc::SqlLen; + using odbc::Statement; + using odbc::SqlLen; LOG_MSG("SQLDescribeCol called\n"); @@ -750,13 +754,20 @@ namespace ignite LOG_MSG("columnSizeRes: %lld\n", columnSizeRes); LOG_MSG("decimalDigitsRes: %lld\n", decimalDigitsRes); LOG_MSG("nullableRes: %lld\n", nullableRes); - LOG_MSG("columnNameBuf: %s\n", columnNameBuf); - LOG_MSG("columnNameLen: %d\n", *columnNameLen); + LOG_MSG("columnNameBuf: %s\n", columnNameBuf ? columnNameBuf : ""); + LOG_MSG("columnNameLen: %d\n", columnNameLen ? *columnNameLen : -1); + + if (dataType) + *dataType = static_cast(dataTypeRes); + + if (columnSize) + *columnSize = static_cast(columnSizeRes); - *dataType = static_cast(dataTypeRes); - *columnSize = static_cast(columnSizeRes); - *decimalDigits = static_cast(decimalDigitsRes); - *nullable = static_cast(nullableRes); + if (decimalDigits) + *decimalDigits = static_cast(decimalDigitsRes); + + if (nullable) + *nullable = static_cast(nullableRes); return statement->GetDiagnosticRecords().GetReturnCode(); } @@ -764,7 +775,7 @@ namespace ignite SQLRETURN SQLRowCount(SQLHSTMT stmt, SQLLEN* rowCnt) { - using ignite::odbc::Statement; + using odbc::Statement; LOG_MSG("SQLRowCount called\n"); @@ -775,7 +786,8 @@ namespace ignite int64_t res = statement->AffectedRows(); - *rowCnt = static_cast(res); + if (rowCnt) + *rowCnt = static_cast(res); return statement->GetDiagnosticRecords().GetReturnCode(); } @@ -794,8 +806,8 @@ namespace ignite SQLCHAR* foreignTableName, SQLSMALLINT foreignTableNameLen) { - using ignite::odbc::Statement; - using ignite::utility::SqlStringToString; + using odbc::Statement; + using utility::SqlStringToString; LOG_MSG("SQLForeignKeys called\n"); @@ -830,21 +842,24 @@ namespace ignite SQLINTEGER valueBufLen, SQLINTEGER* valueResLen) { - using ignite::odbc::Statement; + using odbc::Statement; LOG_MSG("SQLGetStmtAttr called"); - #ifdef ODBC_DEBUG - using ignite::odbc::type_traits::StatementAttrIdToString; +#ifdef ODBC_DEBUG + using odbc::type_traits::StatementAttrIdToString; LOG_MSG("Attr: %s (%d)\n", StatementAttrIdToString(attr), attr); - #endif //ODBC_DEBUG +#endif //ODBC_DEBUG Statement *statement = reinterpret_cast(stmt); if (!statement) return SQL_INVALID_HANDLE; + if (!valueBuf) + return SQL_ERROR; + switch (attr) { case SQL_ATTR_APP_ROW_DESC: @@ -916,15 +931,15 @@ namespace ignite SQLPOINTER value, SQLINTEGER valueLen) { - using ignite::odbc::Statement; + using odbc::Statement; LOG_MSG("SQLSetStmtAttr called"); - #ifdef ODBC_DEBUG - using ignite::odbc::type_traits::StatementAttrIdToString; +#ifdef ODBC_DEBUG + using odbc::type_traits::StatementAttrIdToString; LOG_MSG("Attr: %s (%d)\n", StatementAttrIdToString(attr), attr); - #endif //ODBC_DEBUG +#endif //ODBC_DEBUG Statement *statement = reinterpret_cast(stmt); @@ -988,8 +1003,8 @@ namespace ignite SQLCHAR* tableName, SQLSMALLINT tableNameLen) { - using ignite::odbc::Statement; - using ignite::utility::SqlStringToString; + using odbc::Statement; + using utility::SqlStringToString; LOG_MSG("SQLPrimaryKeys called\n"); @@ -1013,7 +1028,7 @@ namespace ignite SQLRETURN SQLNumParams(SQLHSTMT stmt, SQLSMALLINT* paramCnt) { - using ignite::odbc::Statement; + using odbc::Statement; LOG_MSG("SQLNumParams called\n"); @@ -1022,7 +1037,8 @@ namespace ignite if (!statement) return SQL_INVALID_HANDLE; - *paramCnt = static_cast(statement->GetParametersNumber()); + if (paramCnt) + *paramCnt = static_cast(statement->GetParametersNumber()); return statement->GetDiagnosticRecords().GetReturnCode(); } @@ -1035,11 +1051,11 @@ namespace ignite SQLSMALLINT bufferLen, SQLSMALLINT* resLen) { - using namespace ignite::odbc; - using namespace ignite::odbc::diagnostic; - using namespace ignite::odbc::type_traits; + using namespace odbc; + using namespace odbc::diagnostic; + using namespace odbc::type_traits; - using ignite::odbc::app::ApplicationDataBuffer; + using odbc::app::ApplicationDataBuffer; LOG_MSG("SQLGetDiagField called: %d\n", recNum); @@ -1070,7 +1086,7 @@ namespace ignite } } - if (result == SQL_RESULT_SUCCESS) + if (resLen && result == SQL_RESULT_SUCCESS) *resLen = static_cast(outResLen); return SqlResultToReturnCode(result); @@ -1085,12 +1101,12 @@ namespace ignite SQLSMALLINT msgBufferLen, SQLSMALLINT* msgLen) { - using namespace ignite::utility; - using namespace ignite::odbc; - using namespace ignite::odbc::diagnostic; - using namespace ignite::odbc::type_traits; + using namespace utility; + using namespace odbc; + using namespace odbc::diagnostic; + using namespace odbc::type_traits; - using ignite::odbc::app::ApplicationDataBuffer; + using odbc::app::ApplicationDataBuffer; LOG_MSG("SQLGetDiagRec called\n"); @@ -1129,14 +1145,15 @@ namespace ignite outBuffer.PutString(record.GetMessageText()); - *msgLen = static_cast(outResLen); + if (msgLen) + *msgLen = static_cast(outResLen); return SQL_SUCCESS; } SQLRETURN SQLGetTypeInfo(SQLHSTMT stmt, SQLSMALLINT type) { - using ignite::odbc::Statement; + using odbc::Statement; LOG_MSG("SQLGetTypeInfo called\n"); @@ -1152,7 +1169,7 @@ namespace ignite SQLRETURN SQLEndTran(SQLSMALLINT handleType, SQLHANDLE handle, SQLSMALLINT completionType) { - using namespace ignite::odbc; + using namespace odbc; LOG_MSG("SQLEndTran called\n"); @@ -1212,10 +1229,10 @@ namespace ignite SQLLEN bufferLength, SQLLEN* strLengthOrIndicator) { - using namespace ignite::odbc::type_traits; + using namespace odbc::type_traits; - using ignite::odbc::Statement; - using ignite::odbc::app::ApplicationDataBuffer; + using odbc::Statement; + using odbc::app::ApplicationDataBuffer; LOG_MSG("SQLGetData called\n"); @@ -1238,7 +1255,7 @@ namespace ignite SQLPOINTER value, SQLINTEGER valueLen) { - using ignite::odbc::Environment; + using odbc::Environment; LOG_MSG("SQLSetEnvAttr called\n"); @@ -1258,10 +1275,10 @@ namespace ignite SQLINTEGER valueBufLen, SQLINTEGER* valueResLen) { - using namespace ignite::odbc; - using namespace ignite::odbc::type_traits; + using namespace odbc; + using namespace odbc::type_traits; - using ignite::odbc::app::ApplicationDataBuffer; + using odbc::app::ApplicationDataBuffer; LOG_MSG("SQLGetEnvAttr called\n"); @@ -1293,9 +1310,9 @@ namespace ignite SQLSMALLINT scope, SQLSMALLINT nullable) { - using namespace ignite::odbc; + using namespace odbc; - using ignite::utility::SqlStringToString; + using utility::SqlStringToString; LOG_MSG("SQLSpecialColumns called\n"); diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp index 96a63276bfd7f..8aae156dc7818 100644 --- a/modules/platforms/cpp/odbc/src/statement.cpp +++ b/modules/platforms/cpp/odbc/src/statement.cpp @@ -463,7 +463,7 @@ namespace ignite if (found && strbuf) outSize = utility::CopyStringToBuffer(out, strbuf, buflen); - if (found && strbuf) + if (found && reslen) *reslen = static_cast(outSize); } From 548fe6a3fe4d38caebe3b45366f3e812b035d006 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Mon, 26 Sep 2016 12:05:07 +0300 Subject: [PATCH 165/487] IGNITE-3333: IGFS: Now it is possible to use ATOMIC cache for data blocks when fragmentizer it disabled. This closes #1108. --- .../FileSystemConfiguration.java | 2 +- .../processors/igfs/IgfsDataManager.java | 2 +- .../processors/igfs/IgfsProcessor.java | 17 +++++++- .../igfs/IgfsAbstractBaseSelfTest.java | 8 ++++ .../IgfsAtomicPrimaryMultiNodeSelfTest.java | 39 +++++++++++++++++++ ...gfsAtomicPrimaryOffheapTieredSelfTest.java | 39 +++++++++++++++++++ ...gfsAtomicPrimaryOffheapValuesSelfTest.java | 39 +++++++++++++++++++ .../igfs/IgfsAtomicPrimarySelfTest.java | 39 +++++++++++++++++++ .../testsuites/IgniteIgfsTestSuite.java | 10 ++++- 9 files changed, 190 insertions(+), 5 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimaryMultiNodeSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimaryOffheapTieredSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimaryOffheapValuesSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimarySelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java index 6e0e5e78a0203..e665e843c0ccf 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java @@ -879,7 +879,7 @@ public void setInitializeDefaultPathModes(boolean initDfltPathModes) { *

          * IGFS stores information about file system structure (metadata) inside a transactional cache configured through * {@link #getMetaCacheName()} property. Metadata updates caused by operations on IGFS usually require several - * intearnal keys to be updated. As IGFS metadata cache usually operates in {@link CacheMode#REPLICATED} mode, + * internal keys to be updated. As IGFS metadata cache usually operates in {@link CacheMode#REPLICATED} mode, * meaning that all nodes have all metadata locally, it makes sense to give a hint to Ignite to co-locate * ownership of all metadata keys on a single node. This will decrease amount of network trips required to update * metadata and hence could improve performance. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java index 2f704aea515de..e534800efd7a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java @@ -609,7 +609,7 @@ public void cleanBlocks(IgfsEntryInfo fileInfo, IgfsFileAffinityRange range, boo } /** - * Moves all colocated blocks in range to non-colocated keys. + * Moves all collocated blocks in range to non-colocated keys. * @param fileInfo File info to move data for. * @param range Range to move. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java index 85dcb1cd1cbc0..5c0e030b1b8c9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java @@ -311,8 +311,9 @@ private void validateLocalIgfsConfigurations(FileSystemConfiguration[] cfgs) thr if (GridQueryProcessor.isEnabled(dataCacheCfg)) throw new IgniteCheckedException("IGFS data cache cannot start with enabled query indexing."); - if (dataCacheCfg.getAtomicityMode() != TRANSACTIONAL) - throw new IgniteCheckedException("Data cache should be transactional: " + cfg.getDataCacheName()); + if (dataCacheCfg.getAtomicityMode() != TRANSACTIONAL && cfg.isFragmentizerEnabled()) + throw new IgniteCheckedException("Data cache should be transactional: " + cfg.getDataCacheName() + + " when fragmentizer is enabled"); if (metaCacheCfg == null) throw new IgniteCheckedException("Metadata cache is not configured locally for IGFS: " + cfg); @@ -442,6 +443,18 @@ private void checkIgfsOnRemoteNode(ClusterNode rmtNode) throws IgniteCheckedExce } } + /** + * Check IGFS property equality on local and remote nodes. + * + * @param name Property human readable name. + * @param propName Property name/ + * @param rmtNodeId Remote node ID. + * @param rmtVal Remote value. + * @param locVal Local value. + * @param igfsName IGFS name. + * + * @throws IgniteCheckedException If failed. + */ private void checkSame(String name, String propName, UUID rmtNodeId, Object rmtVal, Object locVal, String igfsName) throws IgniteCheckedException { if (!F.eq(rmtVal, locVal)) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java index 58c4c50707656..79dc57b630ee9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java @@ -219,6 +219,13 @@ protected boolean relaxedConsistency() { return false; } + /** + * @return FragmentizerEnabled IGFS config flag. + */ + protected boolean fragmentizerEnabled() { + return true; + } + /** * @return Relaxed consistency flag. */ @@ -378,6 +385,7 @@ protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mo igfsCfg.setPrefetchBlocks(PREFETCH_BLOCKS); igfsCfg.setSequentialReadsBeforePrefetch(SEQ_READS_BEFORE_PREFETCH); igfsCfg.setRelaxedConsistency(relaxedConsistency()); + igfsCfg.setFragmentizerEnabled(fragmentizerEnabled()); igfsCfg.setInitializeDefaultPathModes(initializeDefaultPathModes()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimaryMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimaryMultiNodeSelfTest.java new file mode 100644 index 0000000000000..0e342a9fce8fc --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimaryMultiNodeSelfTest.java @@ -0,0 +1,39 @@ +/* + * 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.igfs; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.configuration.CacheConfiguration; + +/** + * Tests for PRIMARY mode. + */ +public class IgfsAtomicPrimaryMultiNodeSelfTest extends IgfsPrimaryMultiNodeSelfTest { + /** {@inheritDoc} */ + @Override protected boolean fragmentizerEnabled() { + return false; + } + + /** {@inheritDoc} */ + @Override protected void prepareCacheConfigurations(CacheConfiguration dataCacheCfg, + CacheConfiguration metaCacheCfg) { + super.prepareCacheConfigurations(dataCacheCfg, metaCacheCfg); + + dataCacheCfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimaryOffheapTieredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimaryOffheapTieredSelfTest.java new file mode 100644 index 0000000000000..09b4f9e184bf6 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimaryOffheapTieredSelfTest.java @@ -0,0 +1,39 @@ +/* + * 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.igfs; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.configuration.CacheConfiguration; + +/** + * Tests for PRIMARY mode. + */ +public class IgfsAtomicPrimaryOffheapTieredSelfTest extends IgfsPrimaryOffheapTieredSelfTest { + /** {@inheritDoc} */ + @Override protected boolean fragmentizerEnabled() { + return false; + } + + /** {@inheritDoc} */ + @Override protected void prepareCacheConfigurations(CacheConfiguration dataCacheCfg, + CacheConfiguration metaCacheCfg) { + super.prepareCacheConfigurations(dataCacheCfg, metaCacheCfg); + + dataCacheCfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimaryOffheapValuesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimaryOffheapValuesSelfTest.java new file mode 100644 index 0000000000000..8e9965df1f179 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimaryOffheapValuesSelfTest.java @@ -0,0 +1,39 @@ +/* + * 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.igfs; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.configuration.CacheConfiguration; + +/** + * Tests for PRIMARY mode. + */ +public class IgfsAtomicPrimaryOffheapValuesSelfTest extends IgfsPrimaryOffheapValuesSelfTest { + /** {@inheritDoc} */ + @Override protected boolean fragmentizerEnabled() { + return false; + } + + /** {@inheritDoc} */ + @Override protected void prepareCacheConfigurations(CacheConfiguration dataCacheCfg, + CacheConfiguration metaCacheCfg) { + super.prepareCacheConfigurations(dataCacheCfg, metaCacheCfg); + + dataCacheCfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimarySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimarySelfTest.java new file mode 100644 index 0000000000000..1c28d6baa7883 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAtomicPrimarySelfTest.java @@ -0,0 +1,39 @@ +/* + * 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.igfs; + +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.configuration.CacheConfiguration; + +/** + * Tests for PRIMARY mode. + */ +public class IgfsAtomicPrimarySelfTest extends IgfsPrimarySelfTest { + /** {@inheritDoc} */ + @Override protected boolean fragmentizerEnabled() { + return false; + } + + /** {@inheritDoc} */ + @Override protected void prepareCacheConfigurations(CacheConfiguration dataCacheCfg, + CacheConfiguration metaCacheCfg) { + super.prepareCacheConfigurations(dataCacheCfg, metaCacheCfg); + + dataCacheCfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java index 5a7e3d7456c9e..775c2ceb11d69 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java @@ -20,6 +20,10 @@ import junit.framework.TestSuite; import org.apache.ignite.igfs.IgfsFragmentizerSelfTest; import org.apache.ignite.igfs.IgfsFragmentizerTopologySelfTest; +import org.apache.ignite.internal.processors.igfs.IgfsAtomicPrimaryMultiNodeSelfTest; +import org.apache.ignite.internal.processors.igfs.IgfsAtomicPrimaryOffheapTieredSelfTest; +import org.apache.ignite.internal.processors.igfs.IgfsAtomicPrimaryOffheapValuesSelfTest; +import org.apache.ignite.internal.processors.igfs.IgfsAtomicPrimarySelfTest; import org.apache.ignite.internal.processors.igfs.IgfsAttributesSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsBackupsDualAsyncSelfTest; import org.apache.ignite.internal.processors.igfs.IgfsBackupsDualSyncSelfTest; @@ -154,11 +158,15 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgfsMaxSizeSelfTest.class); - suite.addTestSuite(IgfsProxySelfTest.class); suite.addTestSuite(IgfsLocalSecondaryFileSystemProxySelfTest.class); suite.addTestSuite(IgfsLocalSecondaryFileSystemProxyClientSelfTest.class); + suite.addTestSuite(IgfsAtomicPrimarySelfTest.class); + suite.addTestSuite(IgfsAtomicPrimaryMultiNodeSelfTest.class); + suite.addTestSuite(IgfsAtomicPrimaryOffheapTieredSelfTest.class); + suite.addTestSuite(IgfsAtomicPrimaryOffheapValuesSelfTest.class); + return suite; } } \ No newline at end of file From 8032fc2c8a7cf9f404eb75c65164bb2900aab79d Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 26 Sep 2016 12:31:30 +0300 Subject: [PATCH 166/487] IGNITE-3912: Hadoop: Implemented new class loading architecture for embedded execution mode. --- bin/include/setenv.bat | 8 - bin/include/setenv.sh | 31 - .../ignite/internal/GridKernalContext.java | 8 + .../internal/GridKernalContextImpl.java | 12 + .../ignite/internal/IgniteComponentType.java | 11 +- .../apache/ignite/internal/IgniteKernal.java | 8 +- .../processors/hadoop/HadoopClassLoader.java | 487 +++++++++ .../hadoop/HadoopClasspathUtils.java | 57 +- .../hadoop/HadoopDefaultJobInfo.java | 6 +- .../processors/hadoop/HadoopHelper.java | 55 + .../processors/hadoop/HadoopJobInfo.java | 4 +- .../processors/hadoop/HadoopLocations.java | 0 .../processors/hadoop/HadoopNoopHelper.java | 66 ++ .../hadoop/HadoopNoopProcessor.java | 4 +- .../internal/processors/igfs/IgfsImpl.java | 5 +- .../igfs/IgfsKernalContextAware.java} | 20 +- .../fs/BasicHadoopFileSystemFactory.java | 125 +-- .../fs/CachingHadoopFileSystemFactory.java | 54 +- .../hadoop/fs/HadoopFileSystemFactory.java | 11 +- .../IgniteHadoopFileSystemCounterWriter.java | 79 +- .../IgniteHadoopIgfsSecondaryFileSystem.java | 402 +------- .../fs/KerberosHadoopFileSystemFactory.java | 77 +- .../hadoop/fs/v1/IgniteHadoopFileSystem.java | 47 +- .../hadoop/fs/v2/IgniteHadoopFileSystem.java | 47 +- .../IgniteHadoopClientProtocolProvider.java | 11 +- .../IgniteHadoopMapReducePlanner.java | 22 +- .../IgniteHadoopWeightedMapReducePlanner.java | 4 +- .../ignite/hadoop/util/UserNameMapper.java | 4 +- .../processors/hadoop/HadoopClassLoader.java | 964 ------------------ .../hadoop/HadoopClasspathMain.java | 44 - .../processors/hadoop/HadoopCommonUtils.java | 154 +++ .../processors/hadoop/HadoopContext.java | 1 - .../hadoop/{v2 => }/HadoopExternalSplit.java | 3 +- .../processors/hadoop/HadoopHelperImpl.java | 120 +++ .../processors/hadoop/HadoopProcessor.java | 32 +- .../hadoop/{v2 => }/HadoopSplitWrapper.java | 4 +- .../hadoop/counter/HadoopCounterAdapter.java | 1 + .../counter/HadoopPerformanceCounter.java | 12 +- .../hadoop/delegate/HadoopDelegateUtils.java | 138 +++ ...adoopFileSystemCounterWriterDelegate.java} | 20 +- .../HadoopFileSystemFactoryDelegate.java} | 28 +- ...adoopIgfsSecondaryFileSystemDelegate.java} | 9 +- .../HadoopMapReduceCounterGroup.java | 9 +- .../{ => impl}/HadoopMapReduceCounters.java | 23 +- .../hadoop/{ => impl}/HadoopUtils.java | 155 +-- .../HadoopBasicFileSystemFactoryDelegate.java | 164 +++ ...adoopCachingFileSystemFactoryDelegate.java | 75 ++ ...adoopDefaultFileSystemFactoryDelegate.java | 62 ++ ...opFileSystemCounterWriterDelegateImpl.java | 108 ++ ...opIgfsSecondaryFileSystemDelegateImpl.java | 472 +++++++++ ...doopKerberosFileSystemFactoryDelegate.java | 112 ++ .../fs/HadoopFileSystemCacheUtils.java | 7 +- .../{ => impl}/fs/HadoopFileSystemsUtils.java | 2 +- .../fs/HadoopLazyConcurrentMap.java | 14 +- .../fs/HadoopLocalFileSystemV1.java | 5 +- .../fs/HadoopLocalFileSystemV2.java | 9 +- .../{ => impl}/fs/HadoopParameters.java | 2 +- .../fs/HadoopRawLocalFileSystem.java | 21 +- .../hadoop/{ => impl}/igfs/HadoopIgfs.java | 9 +- .../HadoopIgfsCommunicationException.java | 2 +- .../hadoop/{ => impl}/igfs/HadoopIgfsEx.java | 5 +- .../{ => impl}/igfs/HadoopIgfsFuture.java | 2 +- .../{ => impl}/igfs/HadoopIgfsInProc.java | 15 +- .../igfs/HadoopIgfsInputStream.java | 9 +- .../hadoop/{ => impl}/igfs/HadoopIgfsIo.java | 2 +- .../{ => impl}/igfs/HadoopIgfsIpcIo.java | 27 +- .../igfs/HadoopIgfsIpcIoListener.java | 2 +- .../{ => impl}/igfs/HadoopIgfsJclLogger.java | 2 +- .../{ => impl}/igfs/HadoopIgfsOutProc.java | 9 +- .../igfs/HadoopIgfsOutputStream.java | 7 +- .../{ => impl}/igfs/HadoopIgfsProperties.java | 5 +- .../igfs/HadoopIgfsProxyInputStream.java | 7 +- .../igfs/HadoopIgfsProxyOutputStream.java | 7 +- ...SecondaryFileSystemPositionedReadable.java | 5 +- .../igfs/HadoopIgfsStreamDelegate.java | 2 +- .../igfs/HadoopIgfsStreamEventListener.java | 2 +- .../{ => impl}/igfs/HadoopIgfsUtils.java | 7 +- .../{ => impl}/igfs/HadoopIgfsWrapper.java | 24 +- .../proto/HadoopClientProtocol.java | 27 +- .../{ => impl}/v1/HadoopV1CleanupTask.java | 7 +- .../hadoop/{ => impl}/v1/HadoopV1Counter.java | 11 +- .../hadoop/{ => impl}/v1/HadoopV1MapTask.java | 4 +- .../v1/HadoopV1OutputCollector.java | 5 +- .../{ => impl}/v1/HadoopV1Partitioner.java | 2 +- .../{ => impl}/v1/HadoopV1ReduceTask.java | 4 +- .../{ => impl}/v1/HadoopV1Reporter.java | 2 +- .../{ => impl}/v1/HadoopV1SetupTask.java | 7 +- .../{ => impl}/v1/HadoopV1Splitter.java | 11 +- .../hadoop/{ => impl}/v1/HadoopV1Task.java | 9 +- .../hadoop/{ => impl}/v2/HadoopDaemon.java | 2 +- .../v2/HadoopSerializationWrapper.java | 13 +- .../v2/HadoopShutdownHookManager.java | 2 +- .../{ => impl}/v2/HadoopV2CleanupTask.java | 5 +- .../hadoop/{ => impl}/v2/HadoopV2Context.java | 7 +- .../hadoop/{ => impl}/v2/HadoopV2Counter.java | 7 +- .../hadoop/{ => impl}/v2/HadoopV2Job.java | 83 +- .../v2/HadoopV2JobResourceManager.java | 33 +- .../hadoop/{ => impl}/v2/HadoopV2MapTask.java | 2 +- .../{ => impl}/v2/HadoopV2Partitioner.java | 2 +- .../{ => impl}/v2/HadoopV2ReduceTask.java | 2 +- .../{ => impl}/v2/HadoopV2SetupTask.java | 5 +- .../{ => impl}/v2/HadoopV2Splitter.java | 15 +- .../hadoop/{ => impl}/v2/HadoopV2Task.java | 5 +- .../{ => impl}/v2/HadoopV2TaskContext.java | 61 +- .../v2/HadoopWritableSerialization.java | 9 +- .../hadoop/jobtracker/HadoopJobTracker.java | 23 +- .../planner/HadoopDefaultMapReducePlan.java | 7 +- .../child/HadoopChildProcessRunner.java | 16 +- .../hadoop/HadoopClassLoaderTest.java | 110 -- .../processors/hadoop/HadoopSharedMap.java | 1 + .../hadoop/HadoopTestClassLoader.java | 106 ++ .../hadoop/deps/CircularWithoutHadoop.java | 27 - .../processors/hadoop/deps/WithCast.java | 41 - .../deps/WithConstructorInvocation.java | 31 - .../processors/hadoop/deps/WithExtends.java | 27 - .../processors/hadoop/deps/WithField.java | 29 - .../hadoop/deps/WithIndirectField.java | 27 - .../hadoop/deps/WithInnerClass.java | 31 - .../hadoop/deps/WithLocalVariable.java | 38 - .../hadoop/deps/WithMethodAnnotation.java | 32 - .../hadoop/deps/WithMethodArgument.java | 31 - .../deps/WithMethodCheckedException.java | 31 - .../hadoop/deps/WithMethodInvocation.java | 31 - .../hadoop/deps/WithMethodReturnType.java | 31 - .../deps/WithMethodRuntimeException.java | 31 - .../hadoop/deps/WithParameterAnnotation.java | 31 - .../hadoop/deps/WithStaticField.java | 29 - .../hadoop/deps/WithStaticInitializer.java | 34 - .../HadoopAbstractMapReduceTest.java | 13 +- .../{ => impl}/HadoopAbstractSelfTest.java | 4 +- .../HadoopAbstractWordCountTest.java | 2 +- .../{ => impl}/HadoopCommandLineTest.java | 6 +- ...HadoopDefaultMapReducePlannerSelfTest.java | 6 +- .../{ => impl}/HadoopErrorSimulator.java | 2 +- .../{ => impl}/HadoopFileSystemsTest.java | 4 +- .../hadoop/{ => impl}/HadoopGroupingTest.java | 41 +- .../{ => impl}/HadoopJobTrackerSelfTest.java | 29 +- .../HadoopMapReduceEmbeddedSelfTest.java | 16 +- .../HadoopMapReduceErrorResilienceTest.java | 4 +- .../{ => impl}/HadoopMapReduceTest.java | 4 +- .../HadoopNoHadoopMapReduceTest.java | 2 +- .../{ => impl}/HadoopPlannerMockJob.java | 11 +- .../{ => impl}/HadoopPopularWordsTest.java | 2 +- .../HadoopSerializationWrapperSelfTest.java | 5 +- .../HadoopSnappyFullMapReduceTest.java | 2 +- .../hadoop/{ => impl}/HadoopSnappyTest.java | 6 +- .../{ => impl}/HadoopSortingExternalTest.java | 2 +- .../hadoop/{ => impl}/HadoopSortingTest.java | 5 +- .../HadoopSplitWrapperSelfTest.java | 6 +- .../hadoop/{ => impl}/HadoopStartup.java | 2 +- .../HadoopTaskExecutionSelfTest.java | 39 +- .../HadoopTasksAllVersionsTest.java | 8 +- .../hadoop/{ => impl}/HadoopTasksV1Test.java | 14 +- .../hadoop/{ => impl}/HadoopTasksV2Test.java | 14 +- .../HadoopTestRoundRobinMrPlanner.java | 6 +- .../{ => impl}/HadoopTestTaskContext.java | 11 +- .../hadoop/{ => impl}/HadoopTestUtils.java | 2 +- .../hadoop/impl}/HadoopTxConfigCacheTest.java | 2 +- .../{ => impl}/HadoopUserLibsSelfTest.java | 3 +- .../{ => impl}/HadoopV2JobSelfTest.java | 22 +- .../{ => impl}/HadoopValidationSelfTest.java | 2 +- .../HadoopWeightedMapReducePlannerTest.java | 5 +- .../HadoopWeightedPlannerMapReduceTest.java | 2 +- .../{ => impl}/books/alice-in-wonderland.txt | 0 .../hadoop/{ => impl}/books/art-of-war.txt | 0 .../{ => impl}/books/huckleberry-finn.txt | 0 .../{ => impl}/books/sherlock-holmes.txt | 0 .../hadoop/{ => impl}/books/tom-sawyer.txt | 0 .../HadoopClientProtocolEmbeddedSelfTest.java | 2 +- .../client}/HadoopClientProtocolSelfTest.java | 8 +- .../{ => impl}/examples/HadoopWordCount1.java | 2 +- .../examples/HadoopWordCount1Map.java | 4 +- .../examples/HadoopWordCount1Reduce.java | 4 +- .../{ => impl}/examples/HadoopWordCount2.java | 2 +- .../examples/HadoopWordCount2Combiner.java | 4 +- .../examples/HadoopWordCount2Mapper.java | 4 +- .../examples/HadoopWordCount2Reducer.java | 4 +- ...rberosHadoopFileSystemFactorySelfTest.java | 9 +- .../impl}/igfs/Hadoop1DualAbstractTest.java | 15 +- .../igfs/Hadoop1OverIgfsDualAsyncTest.java | 4 +- .../igfs/Hadoop1OverIgfsDualSyncTest.java | 4 +- .../igfs/HadoopFIleSystemFactorySelfTest.java | 62 +- ...adoopIgfs20FileSystemAbstractSelfTest.java | 11 +- ...fs20FileSystemLoopbackPrimarySelfTest.java | 5 +- ...pIgfs20FileSystemShmemPrimarySelfTest.java | 5 +- .../igfs/HadoopIgfsDualAbstractSelfTest.java | 11 +- .../igfs/HadoopIgfsDualAsyncSelfTest.java | 2 +- .../igfs/HadoopIgfsDualSyncSelfTest.java | 2 +- ...oopIgfsSecondaryFileSystemTestAdapter.java | 14 +- ...pSecondaryFileSystemConfigurationTest.java | 22 +- .../impl}/igfs/IgfsEventsTestSuite.java | 6 +- .../igfs/IgfsNearOnlyMultiNodeSelfTest.java | 5 +- ...gniteHadoopFileSystemAbstractSelfTest.java | 13 +- .../IgniteHadoopFileSystemClientSelfTest.java | 14 +- ...niteHadoopFileSystemHandshakeSelfTest.java | 12 +- ...gniteHadoopFileSystemIpcCacheSelfTest.java | 7 +- .../IgniteHadoopFileSystemLoggerSelfTest.java | 3 +- ...teHadoopFileSystemLoggerStateSelfTest.java | 9 +- ...oopFileSystemLoopbackAbstractSelfTest.java | 6 +- ...stemLoopbackEmbeddedDualAsyncSelfTest.java | 2 +- ...ystemLoopbackEmbeddedDualSyncSelfTest.java | 2 +- ...SystemLoopbackEmbeddedPrimarySelfTest.java | 2 +- ...stemLoopbackEmbeddedSecondarySelfTest.java | 2 +- ...stemLoopbackExternalDualAsyncSelfTest.java | 2 +- ...ystemLoopbackExternalDualSyncSelfTest.java | 2 +- ...SystemLoopbackExternalPrimarySelfTest.java | 2 +- ...stemLoopbackExternalSecondarySelfTest.java | 2 +- ...ndaryFileSystemInitializationSelfTest.java | 5 +- ...HadoopFileSystemShmemAbstractSelfTest.java | 5 +- ...eSystemShmemEmbeddedDualAsyncSelfTest.java | 2 +- ...leSystemShmemEmbeddedDualSyncSelfTest.java | 2 +- ...ileSystemShmemEmbeddedPrimarySelfTest.java | 2 +- ...eSystemShmemEmbeddedSecondarySelfTest.java | 2 +- ...eSystemShmemExternalDualAsyncSelfTest.java | 2 +- ...leSystemShmemExternalDualSyncSelfTest.java | 2 +- ...ileSystemShmemExternalPrimarySelfTest.java | 2 +- ...eSystemShmemExternalSecondarySelfTest.java | 2 +- .../collections/HadoopAbstractMapTest.java | 7 +- .../HadoopConcurrentHashMultimapSelftest.java | 4 +- .../collections/HadoopHashMapSelfTest.java | 4 +- .../collections/HadoopSkipListSelfTest.java | 8 +- .../streams/HadoopDataStreamSelfTest.java | 5 +- .../HadoopExecutorServiceTest.java | 3 +- .../HadoopExternalTaskExecutionSelfTest.java | 6 +- .../HadoopExternalCommunicationSelfTest.java | 4 +- .../util/BasicUserNameMapperSelfTest.java | 3 +- .../util/ChainedUserNameMapperSelfTest.java | 6 +- .../util/KerberosUserNameMapperSelfTest.java | 3 +- .../HadoopGroupingTestState.java} | 30 +- .../state/HadoopJobTrackerSelfTestState.java | 45 + ...HadoopMapReduceEmbeddedSelfTestState.java} | 21 +- .../HadoopTaskExecutionSelfTestValues.java | 51 + .../testsuites/IgniteHadoopTestSuite.java | 114 +-- .../IgniteIgfsLinuxAndMacOSTestSuite.java | 25 +- 234 files changed, 3377 insertions(+), 3305 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java rename modules/{hadoop => core}/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathUtils.java (89%) rename modules/{hadoop => core}/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java (96%) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelper.java rename modules/{hadoop => core}/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopLocations.java (100%) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopHelper.java rename modules/{hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWIthHadoop.java => core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsKernalContextAware.java} (69%) delete mode 100644 modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java delete mode 100644 modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathMain.java create mode 100644 modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCommonUtils.java rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{v2 => }/HadoopExternalSplit.java (94%) create mode 100644 modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelperImpl.java rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{v2 => }/HadoopSplitWrapper.java (95%) create mode 100644 modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java rename modules/hadoop/src/{test/java/org/apache/ignite/internal/processors/hadoop/deps/WithClassAnnotation.java => main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopFileSystemCounterWriterDelegate.java} (56%) rename modules/hadoop/src/{test/java/org/apache/ignite/internal/processors/hadoop/deps/WithImplements.java => main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopFileSystemFactoryDelegate.java} (62%) rename modules/hadoop/src/{test/java/org/apache/ignite/internal/processors/hadoop/deps/Without.java => main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopIgfsSecondaryFileSystemDelegate.java} (70%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopMapReduceCounterGroup.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopMapReduceCounters.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopUtils.java (73%) create mode 100644 modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopBasicFileSystemFactoryDelegate.java create mode 100644 modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopCachingFileSystemFactoryDelegate.java create mode 100644 modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopDefaultFileSystemFactoryDelegate.java create mode 100644 modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopFileSystemCounterWriterDelegateImpl.java create mode 100644 modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java create mode 100644 modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopKerberosFileSystemFactoryDelegate.java rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/fs/HadoopFileSystemCacheUtils.java (99%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/fs/HadoopFileSystemsUtils.java (97%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/fs/HadoopLazyConcurrentMap.java (97%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/fs/HadoopLocalFileSystemV1.java (95%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/fs/HadoopLocalFileSystemV2.java (97%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/fs/HadoopParameters.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/fs/HadoopRawLocalFileSystem.java (99%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfs.java (99%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsCommunicationException.java (96%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsEx.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsFuture.java (97%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsInProc.java (99%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsInputStream.java (99%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsIo.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsIpcIo.java (99%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsIpcIoListener.java (94%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsJclLogger.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsOutProc.java (99%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsOutputStream.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsProperties.java (97%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsProxyInputStream.java (99%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsProxyOutputStream.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsSecondaryFileSystemPositionedReadable.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsStreamDelegate.java (97%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsStreamEventListener.java (95%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsUtils.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/igfs/HadoopIgfsWrapper.java (97%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/proto/HadoopClientProtocol.java (91%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v1/HadoopV1CleanupTask.java (93%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v1/HadoopV1Counter.java (95%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v1/HadoopV1MapTask.java (96%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v1/HadoopV1OutputCollector.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v1/HadoopV1Partitioner.java (96%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v1/HadoopV1ReduceTask.java (95%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v1/HadoopV1Reporter.java (97%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v1/HadoopV1SetupTask.java (92%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v1/HadoopV1Splitter.java (96%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v1/HadoopV1Task.java (95%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopDaemon.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopSerializationWrapper.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopShutdownHookManager.java (97%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopV2CleanupTask.java (97%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopV2Context.java (99%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopV2Counter.java (97%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopV2Job.java (88%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopV2JobResourceManager.java (96%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopV2MapTask.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopV2Partitioner.java (96%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopV2ReduceTask.java (97%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopV2SetupTask.java (97%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopV2Splitter.java (96%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopV2Task.java (98%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopV2TaskContext.java (89%) rename modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/v2/HadoopWritableSerialization.java (97%) delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java create mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestClassLoader.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWithoutHadoop.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithCast.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithConstructorInvocation.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithExtends.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithField.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithIndirectField.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInnerClass.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithLocalVariable.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodAnnotation.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodArgument.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodCheckedException.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodInvocation.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodReturnType.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodRuntimeException.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithParameterAnnotation.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticField.java delete mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticInitializer.java rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopAbstractMapReduceTest.java (96%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopAbstractSelfTest.java (98%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopAbstractWordCountTest.java (98%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopCommandLineTest.java (98%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopDefaultMapReducePlannerSelfTest.java (98%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopErrorSimulator.java (99%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopFileSystemsTest.java (97%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopGroupingTest.java (93%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopJobTrackerSelfTest.java (92%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopMapReduceEmbeddedSelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopMapReduceErrorResilienceTest.java (97%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopMapReduceTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopNoHadoopMapReduceTest.java (96%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopPlannerMockJob.java (87%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopPopularWordsTest.java (99%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopSerializationWrapperSelfTest.java (93%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopSnappyFullMapReduceTest.java (95%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopSnappyTest.java (93%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopSortingExternalTest.java (96%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopSortingTest.java (97%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopSplitWrapperSelfTest.java (91%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopStartup.java (96%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopTaskExecutionSelfTest.java (93%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopTasksAllVersionsTest.java (95%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopTasksV1Test.java (75%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopTasksV2Test.java (80%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopTestRoundRobinMrPlanner.java (87%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopTestTaskContext.java (92%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopTestUtils.java (98%) rename modules/hadoop/src/test/java/org/apache/ignite/{hadoop/cache => internal/processors/hadoop/impl}/HadoopTxConfigCacheTest.java (96%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopUserLibsSelfTest.java (98%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopV2JobSelfTest.java (79%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopValidationSelfTest.java (96%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopWeightedMapReducePlannerTest.java (98%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/HadoopWeightedPlannerMapReduceTest.java (96%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/books/alice-in-wonderland.txt (100%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/books/art-of-war.txt (100%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/books/huckleberry-finn.txt (100%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/books/sherlock-holmes.txt (100%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/books/tom-sawyer.txt (100%) rename modules/hadoop/src/test/java/org/apache/ignite/{client/hadoop => internal/processors/hadoop/impl/client}/HadoopClientProtocolEmbeddedSelfTest.java (95%) rename modules/hadoop/src/test/java/org/apache/ignite/{client/hadoop => internal/processors/hadoop/impl/client}/HadoopClientProtocolSelfTest.java (98%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/examples/HadoopWordCount1.java (97%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/examples/HadoopWordCount1Map.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/examples/HadoopWordCount1Reduce.java (93%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/examples/HadoopWordCount2.java (98%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/examples/HadoopWordCount2Combiner.java (91%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/examples/HadoopWordCount2Mapper.java (95%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/examples/HadoopWordCount2Reducer.java (95%) rename modules/hadoop/src/test/java/org/apache/ignite/{hadoop => internal/processors/hadoop/impl}/fs/KerberosHadoopFileSystemFactorySelfTest.java (90%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/Hadoop1DualAbstractTest.java (88%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/Hadoop1OverIgfsDualAsyncTest.java (90%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/Hadoop1OverIgfsDualSyncTest.java (90%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/HadoopFIleSystemFactorySelfTest.java (83%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java (99%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/HadoopIgfs20FileSystemLoopbackPrimarySelfTest.java (93%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/HadoopIgfs20FileSystemShmemPrimarySelfTest.java (93%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/HadoopIgfsDualAbstractSelfTest.java (95%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/HadoopIgfsDualAsyncSelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/HadoopIgfsDualSyncSelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java (89%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/HadoopSecondaryFileSystemConfigurationTest.java (95%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgfsEventsTestSuite.java (97%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgfsNearOnlyMultiNodeSelfTest.java (97%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemAbstractSelfTest.java (99%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemClientSelfTest.java (92%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java (95%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java (96%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemLoggerSelfTest.java (99%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemLoggerStateSelfTest.java (95%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemLoopbackAbstractSelfTest.java (88%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest.java (96%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemShmemAbstractSelfTest.java (93%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemShmemExternalDualSyncSelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemShmemExternalPrimarySelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/{ => internal/processors/hadoop/impl}/igfs/IgniteHadoopFileSystemShmemExternalSecondarySelfTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/shuffle/collections/HadoopAbstractMapTest.java (94%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/shuffle/collections/HadoopConcurrentHashMultimapSelftest.java (96%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/shuffle/collections/HadoopHashMapSelfTest.java (93%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/shuffle/collections/HadoopSkipListSelfTest.java (96%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/shuffle/streams/HadoopDataStreamSelfTest.java (95%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/taskexecutor/HadoopExecutorServiceTest.java (95%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java (96%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{ => impl}/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java (95%) rename modules/hadoop/src/test/java/org/apache/ignite/{hadoop => internal/processors/hadoop/impl}/util/BasicUserNameMapperSelfTest.java (97%) rename modules/hadoop/src/test/java/org/apache/ignite/{hadoop => internal/processors/hadoop/impl}/util/ChainedUserNameMapperSelfTest.java (92%) rename modules/hadoop/src/test/java/org/apache/ignite/{hadoop => internal/processors/hadoop/impl}/util/KerberosUserNameMapperSelfTest.java (96%) rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{deps/WithOuterClass.java => state/HadoopGroupingTestState.java} (57%) create mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/state/HadoopJobTrackerSelfTestState.java rename modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/{deps/WithInitializer.java => state/HadoopMapReduceEmbeddedSelfTestState.java} (64%) create mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/state/HadoopTaskExecutionSelfTestValues.java diff --git a/bin/include/setenv.bat b/bin/include/setenv.bat index 9d55521655623..82b8fcec5d6bf 100644 --- a/bin/include/setenv.bat +++ b/bin/include/setenv.bat @@ -43,16 +43,8 @@ set IGNITE_LIBS=%IGNITE_HOME%\libs\* for /D %%F in (%IGNITE_HOME%\libs\*) do if not "%%F" == "%IGNITE_HOME%\libs\optional" call :concat %%F\* -if exist %IGNITE_HOME%\libs\ignite-hadoop set HADOOP_EDITION=1 - if defined USER_LIBS set IGNITE_LIBS=%USER_LIBS%;%IGNITE_LIBS% -if "%HADOOP_EDITION%" == "1" FOR /F "delims=" %%i IN ('%JAVA_HOME%\bin\java.exe -cp %IGNITE_HOME%\libs\ignite-hadoop\* org.apache.ignite.internal.processors.hadoop.HadoopClasspathMain ";"' ) DO set IGNITE_HADOOP_CLASSPATH=%%i - -if "%IGNITE_HADOOP_CLASSPATH%" == "" goto :eof - -set IGNITE_LIBS=%IGNITE_LIBS%;%IGNITE_HADOOP_CLASSPATH% - goto :eof :concat diff --git a/bin/include/setenv.sh b/bin/include/setenv.sh index a85cba3004092..e088c0852cd4d 100755 --- a/bin/include/setenv.sh +++ b/bin/include/setenv.sh @@ -61,39 +61,8 @@ do if [ -d ${file} ] && [ "${file}" != "${IGNITE_HOME}"/libs/optional ]; then IGNITE_LIBS=${IGNITE_LIBS}${SEP}${file}/* fi - - if [ -d ${file} ] && [ "${file}" == "${IGNITE_HOME}"/libs/ignite-hadoop ]; then - HADOOP_EDITION=1 - fi done if [ "${USER_LIBS}" != "" ]; then IGNITE_LIBS=${USER_LIBS}${SEP}${IGNITE_LIBS} fi - -if [ "${HADOOP_EDITION}" == "1" ]; then - # Resolve constants. - HADOOP_DEFAULTS="/etc/default/hadoop" - - # - # Resolve the rest of Hadoop environment variables. - # - if [[ -z "${HADOOP_COMMON_HOME}" || -z "${HADOOP_HDFS_HOME}" || -z "${HADOOP_MAPRED_HOME}" ]]; then - if [ -f "$HADOOP_DEFAULTS" ]; then - source "$HADOOP_DEFAULTS" - fi - fi - - IGNITE_HADOOP_CLASSPATH=$( "$JAVA" -cp "${IGNITE_HOME}"/libs/ignite-hadoop/'*' \ - org.apache.ignite.internal.processors.hadoop.HadoopClasspathMain ":" ) - - statusCode=${?} - - if [ "${statusCode}" -ne 0 ]; then - exit ${statusCode} - fi - - unset statusCode - - IGNITE_LIBS=${IGNITE_LIBS}${SEP}${IGNITE_HADOOP_CLASSPATH} -fi diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java index 3eaef1e705cee..b123a4a49cc95 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java @@ -44,6 +44,7 @@ import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor; import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor; import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter; +import org.apache.ignite.internal.processors.hadoop.HadoopHelper; import org.apache.ignite.internal.processors.igfs.IgfsHelper; import org.apache.ignite.internal.processors.igfs.IgfsProcessorAdapter; import org.apache.ignite.internal.processors.job.GridJobProcessor; @@ -284,6 +285,13 @@ public interface GridKernalContext extends Iterable { */ public HadoopProcessorAdapter hadoop(); + /** + * Gets Hadoop helper. + * + * @return Hadoop helper. + */ + public HadoopHelper hadoopHelper(); + /** * Gets utility cache pool. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java index 1ff4543d1419c..eb214e884b5ff 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java @@ -60,6 +60,7 @@ import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor; import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor; import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter; +import org.apache.ignite.internal.processors.hadoop.HadoopHelper; import org.apache.ignite.internal.processors.igfs.IgfsHelper; import org.apache.ignite.internal.processors.igfs.IgfsProcessorAdapter; import org.apache.ignite.internal.processors.job.GridJobProcessor; @@ -236,6 +237,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable @GridToStringInclude private IgfsHelper igfsHelper; + /** */ + @GridToStringInclude + private HadoopHelper hadoopHelper; + /** */ @GridToStringInclude private GridSegmentationProcessor segProc; @@ -541,6 +546,8 @@ public void addHelper(Object helper) { if (helper instanceof IgfsHelper) igfsHelper = (IgfsHelper)helper; + else if (helper instanceof HadoopHelper) + hadoopHelper = (HadoopHelper)helper; else assert false : "Unknown helper class: " + helper.getClass(); } @@ -732,6 +739,11 @@ public void addHelper(Object helper) { return igfsHelper; } + /** {@inheritDoc} */ + @Override public HadoopHelper hadoopHelper() { + return hadoopHelper; + } + /** {@inheritDoc} */ @Override public GridContinuousProcessor continuous() { return contProc; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java index 76e495f354678..0cd2fc16d1e2a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java @@ -41,6 +41,13 @@ public enum IgniteComponentType { "ignite-hadoop" ), + /** Hadoop Helper component. */ + HADOOP_HELPER( + "org.apache.ignite.internal.processors.hadoop.HadoopNoopHelper", + "org.apache.ignite.internal.processors.hadoop.HadoopHelperImpl", + "ignite-hadoop" + ), + /** IGFS helper component. */ IGFS_HELPER( "org.apache.ignite.internal.processors.igfs.IgfsNoopHelper", @@ -160,7 +167,7 @@ public boolean inClassPath() { * @return Created component. * @throws IgniteCheckedException If failed. */ - public T create(GridKernalContext ctx, boolean noOp) throws IgniteCheckedException { + public T create(GridKernalContext ctx, boolean noOp) throws IgniteCheckedException { return create0(ctx, noOp ? noOpClsName : clsName); } @@ -172,7 +179,7 @@ public T create(GridKernalContext ctx, boolean noOp) t * @return Created component. * @throws IgniteCheckedException If failed. */ - public T createIfInClassPath(GridKernalContext ctx, boolean mandatory) + public T createIfInClassPath(GridKernalContext ctx, boolean mandatory) throws IgniteCheckedException { String cls = clsName; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index c5d274848d7ee..6c5a6282420b9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -115,6 +115,7 @@ import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor; import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor; import org.apache.ignite.internal.processors.hadoop.Hadoop; +import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader; import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter; import org.apache.ignite.internal.processors.job.GridJobProcessor; import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor; @@ -188,6 +189,7 @@ import static org.apache.ignite.internal.GridKernalState.STARTING; import static org.apache.ignite.internal.GridKernalState.STOPPED; import static org.apache.ignite.internal.GridKernalState.STOPPING; +import static org.apache.ignite.internal.IgniteComponentType.HADOOP_HELPER; import static org.apache.ignite.internal.IgniteComponentType.IGFS; import static org.apache.ignite.internal.IgniteComponentType.IGFS_HELPER; import static org.apache.ignite.internal.IgniteComponentType.SCHEDULE; @@ -821,6 +823,8 @@ public void start(final IgniteConfiguration cfg, addHelper(IGFS_HELPER.create(F.isEmpty(cfg.getFileSystemConfiguration()))); + addHelper(HADOOP_HELPER.createIfInClassPath(ctx, false)); + startProcessor(new IgnitePluginProcessor(ctx, cfg, plugins)); // Off-heap processor has no dependencies. @@ -881,7 +885,7 @@ public void start(final IgniteConfiguration cfg, startProcessor(new DataStreamProcessor(ctx)); startProcessor((GridProcessor)IGFS.create(ctx, F.isEmpty(cfg.getFileSystemConfiguration()))); startProcessor(new GridContinuousProcessor(ctx)); - startProcessor((GridProcessor)createHadoopComponent()); + startProcessor(createHadoopComponent()); startProcessor(new DataStructuresProcessor(ctx)); startProcessor(createComponent(PlatformProcessor.class, ctx)); @@ -1148,7 +1152,7 @@ private HadoopProcessorAdapter createHadoopComponent() throws IgniteCheckedExcep else { HadoopProcessorAdapter cmp = null; - if (IgniteComponentType.HADOOP.inClassPath() && cfg.isPeerClassLoadingEnabled()) { + if (!ctx.hadoopHelper().isNoOp() && cfg.isPeerClassLoadingEnabled()) { U.warn(log, "Hadoop module is found in classpath, but will not be started because peer class " + "loading is enabled (set IgniteConfiguration.peerClassLoadingEnabled to \"false\" if you want " + "to use Hadoop module)."); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java new file mode 100644 index 0000000000000..cd94c8978573d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java @@ -0,0 +1,487 @@ +/* + * 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.hadoop; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.util.ClassCache; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; +import org.jsr166.ConcurrentHashMap8; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.Map; +import java.util.Vector; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +/** + * Class loader allowing explicitly load classes without delegation to parent class loader. + * Also supports class parsing for finding dependencies which contain transitive dependencies + * unavailable for parent. + */ +public class HadoopClassLoader extends URLClassLoader implements ClassCache { + /** Hadoop class name: Daemon. */ + public static final String CLS_DAEMON = "org.apache.hadoop.util.Daemon"; + + /** Hadoop class name: ShutdownHookManager. */ + public static final String CLS_SHUTDOWN_HOOK_MANAGER = "org.apache.hadoop.util.ShutdownHookManager"; + + /** Hadoop class name: Daemon replacement. */ + public static final String CLS_DAEMON_REPLACE = "org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopDaemon"; + + /** Hadoop class name: ShutdownHookManager replacement. */ + public static final String CLS_SHUTDOWN_HOOK_MANAGER_REPLACE = + "org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopShutdownHookManager"; + + /** */ + private static final URLClassLoader APP_CLS_LDR = (URLClassLoader)HadoopClassLoader.class.getClassLoader(); + + /** */ + private static final Collection appJars = F.asList(APP_CLS_LDR.getURLs()); + + /** Mutex for native libraries initialization. */ + private static final Object LIBS_MUX = new Object(); + + /** Predefined native libraries to load. */ + private static final Collection PREDEFINED_NATIVE_LIBS; + + /** Native libraries. */ + private static Collection NATIVE_LIBS; + + /** */ + private static volatile Collection hadoopJars; + + /** */ + private static final Map bytesCache = new ConcurrentHashMap8<>(); + + /** Class cache. */ + private final ConcurrentMap cacheMap = new ConcurrentHashMap<>(); + + /** Diagnostic name of this class loader. */ + @SuppressWarnings({"FieldCanBeLocal", "UnusedDeclaration"}) + private final String name; + + /** Igfs Helper. */ + private final HadoopHelper helper; + + static { + // We are very parallel capable. + registerAsParallelCapable(); + + PREDEFINED_NATIVE_LIBS = new HashSet<>(); + + PREDEFINED_NATIVE_LIBS.add("hadoop"); + PREDEFINED_NATIVE_LIBS.add("MapRClient"); + } + + /** + * Gets name for the task class loader. Task class loader + * @param info The task info. + * @param prefix Get only prefix (without task type and number) + * @return The class loader name. + */ + public static String nameForTask(HadoopTaskInfo info, boolean prefix) { + if (prefix) + return "hadoop-task-" + info.jobId() + "-"; + else + return "hadoop-task-" + info.jobId() + "-" + info.type() + "-" + info.taskNumber(); + } + + /** + * Constructor. + * + * @param urls Urls. + * @param name Classloader name. + * @param libNames Optional additional native library names to be linked from parent classloader. + */ + public HadoopClassLoader(URL[] urls, String name, @Nullable String[] libNames, HadoopHelper helper) { + super(addHadoopUrls(urls), APP_CLS_LDR); + + assert !(getParent() instanceof HadoopClassLoader); + + this.name = name; + this.helper = helper; + + initializeNativeLibraries(libNames); + } + + /** + * Workaround to load native Hadoop libraries. Java doesn't allow native libraries to be loaded from different + * classloaders. But we load Hadoop classes many times and one of these classes - {@code NativeCodeLoader} - tries + * to load the same native library over and over again. + *

          + * To fix the problem, we force native library load in parent class loader and then "link" handle to this native + * library to our class loader. As a result, our class loader will think that the library is already loaded and will + * be able to link native methods. + * + * @see + * JNI specification + */ + @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") + private void initializeNativeLibraries(@Nullable String[] usrLibs) { + Collection res; + + synchronized (LIBS_MUX) { + if (NATIVE_LIBS == null) { + LinkedList libs = new LinkedList<>(); + + for (String lib : PREDEFINED_NATIVE_LIBS) + libs.add(new NativeLibrary(lib, true)); + + if (!F.isEmpty(usrLibs)) { + for (String usrLib : usrLibs) + libs.add(new NativeLibrary(usrLib, false)); + } + + NATIVE_LIBS = initializeNativeLibraries0(libs); + } + + res = NATIVE_LIBS; + } + + // Link libraries to class loader. + Vector ldrLibs = nativeLibraries(this); + + synchronized (ldrLibs) { + ldrLibs.addAll(res); + } + } + + /** + * Initialize native libraries. + * + * @param libs Libraries to initialize. + * @return Initialized libraries. + */ + @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") + private static Collection initializeNativeLibraries0(Collection libs) { + assert Thread.holdsLock(LIBS_MUX); + + Collection res = new HashSet<>(); + + for (NativeLibrary lib : libs) { + String libName = lib.name; + + File libFile = new File(libName); + + try { + // Load library. + if (libFile.isAbsolute()) + System.load(libName); + else + System.loadLibrary(libName); + + // Find library in class loader internals. + Object libObj = null; + + ClassLoader ldr = APP_CLS_LDR; + + while (ldr != null) { + Vector ldrLibObjs = nativeLibraries(ldr); + + synchronized (ldrLibObjs) { + for (Object ldrLibObj : ldrLibObjs) { + String name = nativeLibraryName(ldrLibObj); + + if (libFile.isAbsolute()) { + if (F.eq(name, libFile.getCanonicalPath())) { + libObj = ldrLibObj; + + break; + } + } else { + if (name.contains(libName)) { + libObj = ldrLibObj; + + break; + } + } + } + } + + if (libObj != null) + break; + + ldr = ldr.getParent(); + } + + if (libObj == null) + throw new IgniteException("Failed to find loaded library: " + libName); + + res.add(libObj); + } + catch (UnsatisfiedLinkError e) { + if (!lib.optional) + throw e; + } + catch (IOException e) { + throw new IgniteException("Failed to initialize native libraries due to unexpected exception.", e); + } + } + + return res; + } + + /** + * Get native libraries collection for the given class loader. + * + * @param ldr Class loaded. + * @return Native libraries. + */ + private static Vector nativeLibraries(ClassLoader ldr) { + assert ldr != null; + + return U.field(ldr, "nativeLibraries"); + } + + /** + * Get native library name. + * + * @param lib Library. + * @return Name. + */ + private static String nativeLibraryName(Object lib) { + assert lib != null; + + return U.field(lib, "name"); + } + + /** {@inheritDoc} */ + @Override protected Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + try { + // Always load Hadoop classes explicitly, since Hadoop can be available in App classpath. + if (name.equals(CLS_SHUTDOWN_HOOK_MANAGER)) // Dirty hack to get rid of Hadoop shutdown hooks. + return loadReplace(name, CLS_SHUTDOWN_HOOK_MANAGER_REPLACE); + else if (name.equals(CLS_DAEMON)) + // We replace this in order to be able to forcibly stop some daemon threads + // that otherwise never stop (e.g. PeerCache runnables): + return loadReplace(name, CLS_DAEMON_REPLACE); + + // For Ignite Hadoop and IGFS classes we have to check if they depend on Hadoop. + if (loadByCurrentClassloader(name)) + return loadClassExplicitly(name, resolve); + + return super.loadClass(name, resolve); + } + catch (NoClassDefFoundError | ClassNotFoundException e) { + throw new ClassNotFoundException("Failed to load class: " + name, e); + } + } + + /** + * Load a class replacing it with our own implementation. + * + * @param originalName Name. + * @param replaceName Replacement. + * @return Class. + */ + private Class loadReplace(final String originalName, final String replaceName) { + synchronized (getClassLoadingLock(originalName)) { + // First, check if the class has already been loaded + Class c = findLoadedClass(originalName); + + if (c != null) + return c; + + byte[] bytes = bytesCache.get(originalName); + + if (bytes == null) { + InputStream in = helper.loadClassBytes(this, replaceName); + + if (in == null) + throw new IgniteException("Failed to replace class [originalName=" + originalName + + ", replaceName=" + replaceName + ']'); + + bytes = helper.loadReplace(in, originalName, replaceName); + + bytesCache.put(originalName, bytes); + } + + return defineClass(originalName, bytes, 0, bytes.length); + } + } + + /** {@inheritDoc} */ + @Override public Class getFromCache(String clsName) throws ClassNotFoundException { + Class cls = cacheMap.get(clsName); + + if (cls == null) { + Class old = cacheMap.putIfAbsent(clsName, cls = Class.forName(clsName, true, this)); + + if (old != null) + cls = old; + } + + return cls; + } + + /** + * Check whether file must be loaded with current class loader, or normal delegation model should be used. + *

          + * Override is only necessary for Ignite classes which have direct or transitive dependencies on Hadoop classes. + * These are all classes from "org.apache.ignite.internal.processors.hadoop.impl" package, + * and these are several well-know classes from "org.apache.ignite.hadoop" package. + * + * @param clsName Class name. + * @return Whether class must be loaded by current classloader without delegation. + */ + @SuppressWarnings("RedundantIfStatement") + public static boolean loadByCurrentClassloader(String clsName) { + // All impl classes. + if (clsName.startsWith("org.apache.ignite.internal.processors.hadoop.impl")) + return true; + + // Several classes from public API. + if (clsName.startsWith("org.apache.ignite.hadoop")) { + // We use "contains" instead of "equals" to handle subclasses properly. + if (clsName.contains("org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem") || + clsName.contains("org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem") || + clsName.contains("org.apache.ignite.hadoop.mapreduce.IgniteHadoopClientProtocolProvider")) + return true; + } + + return false; + } + + /** + * @param name Class name. + * @param resolve Resolve class. + * @return Class. + * @throws ClassNotFoundException If failed. + */ + private Class loadClassExplicitly(String name, boolean resolve) throws ClassNotFoundException { + synchronized (getClassLoadingLock(name)) { + // First, check if the class has already been loaded + Class c = findLoadedClass(name); + + if (c == null) { + long t1 = System.nanoTime(); + + c = findClass(name); + + // this is the defining class loader; record the stats + sun.misc.PerfCounter.getFindClassTime().addElapsedTimeFrom(t1); + sun.misc.PerfCounter.getFindClasses().increment(); + } + + if (resolve) + resolveClass(c); + + return c; + } + } + + /** + * @param urls URLs. + * @return URLs. + */ + private static URL[] addHadoopUrls(URL[] urls) { + Collection hadoopJars; + + try { + hadoopJars = hadoopUrls(); + } + catch (IgniteCheckedException e) { + throw new RuntimeException(e); + } + + ArrayList list = new ArrayList<>(hadoopJars.size() + appJars.size() + (urls == null ? 0 : urls.length)); + + list.addAll(appJars); + list.addAll(hadoopJars); + + if (!F.isEmpty(urls)) + list.addAll(F.asList(urls)); + + return list.toArray(new URL[list.size()]); + } + + /** + * @return Collection of jar URLs. + * @throws IgniteCheckedException If failed. + */ + public static Collection hadoopUrls() throws IgniteCheckedException { + Collection hadoopUrls = hadoopJars; + + if (hadoopUrls != null) + return hadoopUrls; + + synchronized (HadoopClassLoader.class) { + hadoopUrls = hadoopJars; + + if (hadoopUrls != null) + return hadoopUrls; + + try { + hadoopUrls = HadoopClasspathUtils.classpathForClassLoader(); + } + catch (IOException e) { + throw new IgniteCheckedException("Failed to resolve Hadoop JAR locations: " + e.getMessage(), e); + } + + hadoopJars = hadoopUrls; + + return hadoopUrls; + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(HadoopClassLoader.class, this); + } + + /** + * Getter for name field. + */ + public String name() { + return name; + } + + /** + * Native library abstraction. + */ + private static class NativeLibrary { + /** Library name. */ + private final String name; + + /** Whether library is optional. */ + private final boolean optional; + + /** + * Constructor. + * + * @param name Library name. + * @param optional Optional flag. + */ + public NativeLibrary(String name, boolean optional) { + this.name = name; + this.optional = optional; + } + } +} \ No newline at end of file diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathUtils.java similarity index 89% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathUtils.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathUtils.java index f5c2814d0f3dd..7579ddb1d8e7f 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathUtils.java @@ -55,31 +55,6 @@ public class HadoopClasspathUtils { /** Empty string. */ private static final String EMPTY_STR = ""; - /** - * Gets Hadoop class path as list of classpath elements for process. - * - * @return List of the class path elements. - * @throws IOException If failed. - */ - public static List classpathForProcess() throws IOException { - List res = new ArrayList<>(); - - for (final SearchDirectory dir : classpathDirectories()) { - File[] files = dir.files(); - - if (dir.useWildcard()) { - if (files.length > 0) - res.add(dir.absolutePath() + File.separator + '*'); - } - else { - for (File file : files) - res.add(file.getAbsolutePath()); - } - } - - return res; - } - /** * Gets Hadoop class path as a list of URLs (for in-process class loader usage). * @@ -190,6 +165,7 @@ private static Collection classpathDirectories() throws IOExcep Collection res = new ArrayList<>(); + // Add libraries from Hadoop distribution: res.add(new SearchDirectory(new File(loc.common(), "lib"), AcceptAllDirectoryFilter.INSTANCE)); res.add(new SearchDirectory(new File(loc.hdfs(), "lib"), AcceptAllDirectoryFilter.INSTANCE)); res.add(new SearchDirectory(new File(loc.mapred(), "lib"), AcceptAllDirectoryFilter.INSTANCE)); @@ -204,6 +180,7 @@ private static Collection classpathDirectories() throws IOExcep res.add(new SearchDirectory(new File(loc.mapred()), new PrefixDirectoryFilter("hadoop-mapreduce-client-core"))); + // Add user provided libs: res.addAll(parseUserLibs()); return res; @@ -215,7 +192,7 @@ private static Collection classpathDirectories() throws IOExcep * @return Parsed libs search patterns. * @throws IOException If failed. */ - static Collection parseUserLibs() throws IOException { + public static Collection parseUserLibs() throws IOException { return parseUserLibs(systemOrEnv(HADOOP_USER_LIBS, null)); } @@ -226,7 +203,7 @@ static Collection parseUserLibs() throws IOException { * @return Result. * @throws IOException If failed. */ - static Collection parseUserLibs(String str) throws IOException { + public static Collection parseUserLibs(String str) throws IOException { Collection res = new LinkedList<>(); if (!isEmpty(str)) { @@ -314,7 +291,7 @@ private static String normalize(String str) { /** * Simple pair-like structure to hold directory name and a mask assigned to it. */ - static class SearchDirectory { + public static class SearchDirectory { /** File. */ private final File dir; @@ -352,17 +329,10 @@ private SearchDirectory(File dir, DirectoryFilter filter, boolean strict) throws throw new IOException("Directory cannot be read: " + dir.getAbsolutePath()); } - /** - * @return Absolute path. - */ - String absolutePath() { - return dir.getAbsolutePath(); - } - /** * @return Child files. */ - File[] files() throws IOException { + public File[] files() throws IOException { File[] files = dir.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { return filter.test(name); @@ -378,19 +348,12 @@ File[] files() throws IOException { else return files; } - - /** - * @return {@code True} if wildcard can be used. - */ - boolean useWildcard() { - return filter instanceof AcceptAllDirectoryFilter; - } } /** * Directory filter interface. */ - static interface DirectoryFilter { + public static interface DirectoryFilter { /** * Test if file with this name should be included. * @@ -403,7 +366,7 @@ static interface DirectoryFilter { /** * Filter to accept all files. */ - static class AcceptAllDirectoryFilter implements DirectoryFilter { + public static class AcceptAllDirectoryFilter implements DirectoryFilter { /** Singleton instance. */ public static final AcceptAllDirectoryFilter INSTANCE = new AcceptAllDirectoryFilter(); @@ -416,7 +379,7 @@ static class AcceptAllDirectoryFilter implements DirectoryFilter { /** * Filter which uses prefix to filter files. */ - static class PrefixDirectoryFilter implements DirectoryFilter { + public static class PrefixDirectoryFilter implements DirectoryFilter { /** Prefix. */ private final String prefix; @@ -440,7 +403,7 @@ public PrefixDirectoryFilter(String prefix) { /** * Filter which uses exact comparison. */ - static class ExactDirectoryFilter implements DirectoryFilter { + public static class ExactDirectoryFilter implements DirectoryFilter { /** Name. */ private final String name; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java similarity index 96% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java index 1382c1fff84c7..ae17ac813acff 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java @@ -83,14 +83,14 @@ public HadoopDefaultJobInfo(String jobName, String user, boolean hasCombiner, in /** {@inheritDoc} */ @Override public HadoopJob createJob(Class jobCls, HadoopJobId jobId, IgniteLogger log, - @Nullable String[] libNames) throws IgniteCheckedException { + @Nullable String[] libNames, HadoopHelper helper) throws IgniteCheckedException { assert jobCls != null; try { Constructor constructor = jobCls.getConstructor(HadoopJobId.class, - HadoopDefaultJobInfo.class, IgniteLogger.class, String[].class); + HadoopDefaultJobInfo.class, IgniteLogger.class, String[].class, HadoopHelper.class); - return constructor.newInstance(jobId, this, log, libNames); + return constructor.newInstance(jobId, this, log, libNames, helper); } catch (Throwable t) { if (t instanceof Error) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelper.java new file mode 100644 index 0000000000000..a8fee79d3380e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelper.java @@ -0,0 +1,55 @@ +/* + * 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.hadoop; + +import java.io.InputStream; +import org.jetbrains.annotations.Nullable; + +/** + * IGFS utility processor adapter. + */ +public interface HadoopHelper { + /** + * @return Whether this is no-op implementation. + */ + public boolean isNoOp(); + + /** + * Get common Hadoop class loader. + * + * @return Common Hadoop class loader. + */ + public HadoopClassLoader commonClassLoader(); + + /** + * Load special replacement and impersonate. + * + * @param in Input stream. + * @param originalName Original class name. + * @param replaceName Replacer class name. + * @return Result. + */ + public byte[] loadReplace(InputStream in, final String originalName, final String replaceName); + + /** + * @param ldr Loader. + * @param clsName Class. + * @return Input stream. + */ + @Nullable public InputStream loadClassBytes(ClassLoader ldr, String clsName); +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java index a3b1bb66b3960..853c63dd42509 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java @@ -59,11 +59,13 @@ public interface HadoopJobInfo extends Serializable { * @param jobId Job ID. * @param log Logger. * @param libNames Optional additional native library names. + * @param helper HadoopHelper. * @return Job. * @throws IgniteCheckedException If failed. */ public HadoopJob createJob(Class jobCls, - HadoopJobId jobId, IgniteLogger log, @Nullable String[] libNames) throws IgniteCheckedException; + HadoopJobId jobId, IgniteLogger log, @Nullable String[] libNames, HadoopHelper helper) + throws IgniteCheckedException; /** * @return Number of reducers configured for job. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopLocations.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopLocations.java similarity index 100% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopLocations.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopLocations.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopHelper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopHelper.java new file mode 100644 index 0000000000000..d3348caae33f4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopHelper.java @@ -0,0 +1,66 @@ +/* + * 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.hadoop; + +import org.apache.ignite.internal.GridKernalContext; +import org.jetbrains.annotations.Nullable; + +import java.io.InputStream; + +/** + * Noop Hadoop Helper implementation. + */ +@SuppressWarnings("unused") +public class HadoopNoopHelper implements HadoopHelper { + /** + * Constructor. + * + * @param ctx Kernal context. + */ + @SuppressWarnings("UnusedParameters") + public HadoopNoopHelper(GridKernalContext ctx) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public boolean isNoOp() { + return true; + } + + /** {@inheritDoc} */ + @Override public HadoopClassLoader commonClassLoader() { + throw unsupported(); + } + + /** {@inheritDoc} */ + @Override public byte[] loadReplace(InputStream in, String originalName, String replaceName) { + throw unsupported(); + } + + /** {@inheritDoc} */ + @Nullable @Override public InputStream loadClassBytes(ClassLoader ldr, String clsName) { + throw unsupported(); + } + + /** + * @return Exception. + */ + private static UnsupportedOperationException unsupported() { + throw new UnsupportedOperationException("Operation is unsupported (Hadoop module is not in the classpath)."); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopProcessor.java index 501870a31040e..fa4ab47087ee5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopProcessor.java @@ -79,8 +79,8 @@ public HadoopNoopProcessor(GridKernalContext ctx) { * Creates an exception to be uniformly thrown from all the methods. */ private IllegalStateException createException() { - return new IllegalStateException("Hadoop module is not loaded (please ensure that ignite-hadoop.jar is in " + - "classpath and IgniteConfiguration.peerClassLoadingEnabled is set to false)."); + return new IllegalStateException("Hadoop module is not loaded (please ensure that ignite-hadoop.jar is " + + "in libs and IgniteConfiguration.peerClassLoadingEnabled is set to false)."); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 6ff1f8fa3eddb..1dd12d913a590 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -183,8 +183,11 @@ public final class IgfsImpl implements IgfsEx { data = igfsCtx.data(); secondaryFs = cfg.getSecondaryFileSystem(); + if (secondaryFs instanceof IgfsKernalContextAware) + ((IgfsKernalContextAware)secondaryFs).setKernalContext(igfsCtx.kernalContext()); + if (secondaryFs instanceof LifecycleAware) - ((LifecycleAware) secondaryFs).start(); + ((LifecycleAware)secondaryFs).start(); /* Default IGFS mode. */ IgfsMode dfltMode; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWIthHadoop.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsKernalContextAware.java similarity index 69% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWIthHadoop.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsKernalContextAware.java index c3aa7d9a760b1..7f59db4dc2089 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWIthHadoop.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsKernalContextAware.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.deps; +package org.apache.ignite.internal.processors.igfs; -import org.apache.hadoop.mapreduce.Job; +import org.apache.ignite.internal.GridKernalContext; /** - * Class has a direct Hadoop dependency and a circular dependency on another class. + * Indicates whether particular file system accepts kernal context. */ -@SuppressWarnings("unused") -public class CircularWIthHadoop { - /** */ - private Job[][] jobs = new Job[4][4]; - - /** */ - private CircularWithoutHadoop y; +public interface IgfsKernalContextAware { + /** + * Set kernal context. + * + * @param ctx Kernal context. + */ + public void setKernalContext(GridKernalContext ctx); } diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java index a01bfaf690d40..89b802806515e 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java @@ -17,37 +17,26 @@ package org.apache.ignite.hadoop.fs; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.ignite.IgniteException; -import org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem; import org.apache.ignite.hadoop.util.KerberosUserNameMapper; import org.apache.ignite.hadoop.util.UserNameMapper; -import org.apache.ignite.internal.processors.hadoop.HadoopUtils; -import org.apache.ignite.internal.processors.igfs.IgfsUtils; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lifecycle.LifecycleAware; import org.jetbrains.annotations.Nullable; import java.io.Externalizable; import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; -import java.net.URI; -import java.net.URISyntaxException; -import java.net.URL; -import java.util.Arrays; /** * Simple Hadoop file system factory which delegates to {@code FileSystem.get()} on each call. *

          * If {@code "fs.[prefix].impl.disable.cache"} is set to {@code true}, file system instances will be cached by Hadoop. */ -public class BasicHadoopFileSystemFactory implements HadoopFileSystemFactory, Externalizable, LifecycleAware { +public class BasicHadoopFileSystemFactory implements HadoopFileSystemFactory, Externalizable { /** */ private static final long serialVersionUID = 0L; - /** File system URI. */ + /** File system URI. */ private String uri; /** File system config paths. */ @@ -56,12 +45,6 @@ public class BasicHadoopFileSystemFactory implements HadoopFileSystemFactory, Ex /** User name mapper. */ private UserNameMapper usrNameMapper; - /** Configuration of the secondary filesystem, never null. */ - protected transient Configuration cfg; - - /** Resulting URI. */ - protected transient URI fullUri; - /** * Constructor. */ @@ -70,64 +53,17 @@ public BasicHadoopFileSystemFactory() { } /** {@inheritDoc} */ - @Override public final FileSystem get(String name) throws IOException { - String name0 = IgfsUtils.fixUserName(name); - - if (usrNameMapper != null) - name0 = IgfsUtils.fixUserName(usrNameMapper.map(name0)); - - return getWithMappedName(name0); - } - - /** - * Internal file system create routine. - * - * @param usrName User name. - * @return File system. - * @throws IOException If failed. - */ - protected FileSystem getWithMappedName(String usrName) throws IOException { - assert cfg != null; - - try { - // FileSystem.get() might delegate to ServiceLoader to get the list of file system implementation. - // And ServiceLoader is known to be sensitive to context classloader. Therefore, we change context - // classloader to classloader of current class to avoid strange class-cast-exceptions. - ClassLoader oldLdr = HadoopUtils.setContextClassLoader(getClass().getClassLoader()); - - try { - return create(usrName); - } - finally { - HadoopUtils.restoreContextClassLoader(oldLdr); - } - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - - throw new IOException("Failed to create file system due to interrupt.", e); - } - } - - /** - * Internal file system creation routine, invoked in correct class loader context. - * - * @param usrName User name. - * @return File system. - * @throws IOException If failed. - * @throws InterruptedException if the current thread is interrupted. - */ - protected FileSystem create(String usrName) throws IOException, InterruptedException { - return FileSystem.get(fullUri, cfg, usrName); + @Override public final Object get(String name) throws IOException { + throw new UnsupportedOperationException("Method should not be called directly."); } /** * Gets file system URI. *

          - * This URI will be used as a first argument when calling {@link FileSystem#get(URI, Configuration, String)}. + * This URI will be used as a first argument when calling {@code FileSystem.get(URI, Configuration, String)}. *

          * If not set, default URI will be picked from file system configuration using - * {@link FileSystem#getDefaultUri(Configuration)} method. + * {@code FileSystem.getDefaultUri(Configuration)} method. * * @return File system URI. */ @@ -149,11 +85,8 @@ public void setUri(@Nullable String uri) { *

          * Path could be either absolute or relative to {@code IGNITE_HOME} environment variable. *

          - * All provided paths will be loaded in the order they provided and then applied to {@link Configuration}. It means + * All provided paths will be loaded in the order they provided and then applied to {@code Configuration}. It means * that path order might be important in some cases. - *

          - * NOTE! Factory can be serialized and transferred to other machines where instance of - * {@link IgniteHadoopFileSystem} resides. Corresponding paths must exist on these machines as well. * * @return Paths to file system configuration files. */ @@ -197,50 +130,6 @@ public void setUserNameMapper(@Nullable UserNameMapper usrNameMapper) { this.usrNameMapper = usrNameMapper; } - /** {@inheritDoc} */ - @Override public void start() throws IgniteException { - cfg = HadoopUtils.safeCreateConfiguration(); - - if (cfgPaths != null) { - for (String cfgPath : cfgPaths) { - if (cfgPath == null) - throw new NullPointerException("Configuration path cannot be null: " + Arrays.toString(cfgPaths)); - else { - URL url = U.resolveIgniteUrl(cfgPath); - - if (url == null) { - // If secConfPath is given, it should be resolvable: - throw new IgniteException("Failed to resolve secondary file system configuration path " + - "(ensure that it exists locally and you have read access to it): " + cfgPath); - } - - cfg.addResource(url); - } - } - } - - // If secondary fs URI is not given explicitly, try to get it from the configuration: - if (uri == null) - fullUri = FileSystem.getDefaultUri(cfg); - else { - try { - fullUri = new URI(uri); - } - catch (URISyntaxException use) { - throw new IgniteException("Failed to resolve secondary file system URI: " + uri); - } - } - - if (usrNameMapper != null && usrNameMapper instanceof LifecycleAware) - ((LifecycleAware)usrNameMapper).start(); - } - - /** {@inheritDoc} */ - @Override public void stop() throws IgniteException { - if (usrNameMapper != null && usrNameMapper instanceof LifecycleAware) - ((LifecycleAware)usrNameMapper).stop(); - } - /** {@inheritDoc} */ @Override public void writeExternal(ObjectOutput out) throws IOException { U.writeString(out, uri); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/CachingHadoopFileSystemFactory.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/CachingHadoopFileSystemFactory.java index bcbb08236d6e7..b90777c4d173a 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/CachingHadoopFileSystemFactory.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/CachingHadoopFileSystemFactory.java @@ -17,24 +17,14 @@ package org.apache.ignite.hadoop.fs; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteException; -import org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemsUtils; -import org.apache.ignite.internal.processors.hadoop.fs.HadoopLazyConcurrentMap; - -import java.io.IOException; -import java.net.URI; - /** - * Caching Hadoop file system factory. Caches {@link FileSystem} instances on per-user basis. Doesn't rely on + * Caching Hadoop file system factory. Caches {@code FileSystem} instances on per-user basis. Doesn't rely on * built-in Hadoop {@code FileSystem} caching mechanics. Separate {@code FileSystem} instance is created for each * user instead. *

          - * This makes cache instance resistant to concurrent calls to {@link FileSystem#close()} in other parts of the user + * This makes cache instance resistant to concurrent calls to {@code FileSystem.close()} in other parts of the user * code. On the other hand, this might cause problems on some environments. E.g. if Kerberos is enabled, a call to - * {@link FileSystem#get(URI, Configuration, String)} will refresh Kerberos token. But this factory implementation + * {@code FileSystem.get(URI, Configuration, String)} will refresh Kerberos token. But this factory implementation * calls this method only once per user what may lead to token expiration. In such cases it makes sense to either * use {@link BasicHadoopFileSystemFactory} or implement your own factory. */ @@ -42,44 +32,10 @@ public class CachingHadoopFileSystemFactory extends BasicHadoopFileSystemFactory /** */ private static final long serialVersionUID = 0L; - /** Per-user file system cache. */ - private final transient HadoopLazyConcurrentMap cache = new HadoopLazyConcurrentMap<>( - new HadoopLazyConcurrentMap.ValueFactory() { - @Override public FileSystem createValue(String key) throws IOException { - return CachingHadoopFileSystemFactory.super.getWithMappedName(key); - } - } - ); - /** - * Public non-arg constructor. + * Constructor. */ public CachingHadoopFileSystemFactory() { - // noop - } - - /** {@inheritDoc} */ - @Override public FileSystem getWithMappedName(String name) throws IOException { - return cache.getOrCreate(name); - } - - /** {@inheritDoc} */ - @Override public void start() throws IgniteException { - super.start(); - - // Disable caching. - cfg.setBoolean(HadoopFileSystemsUtils.disableFsCachePropertyName(fullUri.getScheme()), true); - } - - /** {@inheritDoc} */ - @Override public void stop() throws IgniteException { - super.stop(); - - try { - cache.close(); - } - catch (IgniteCheckedException ice) { - throw new IgniteException(ice); - } + // No-op. } } diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/HadoopFileSystemFactory.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/HadoopFileSystemFactory.java index 5ad08abd5ee9b..214328fefdd1b 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/HadoopFileSystemFactory.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/HadoopFileSystemFactory.java @@ -17,16 +17,13 @@ package org.apache.ignite.hadoop.fs; -import org.apache.hadoop.fs.FileSystem; -import org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem; -import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.lifecycle.LifecycleAware; import java.io.IOException; import java.io.Serializable; /** - * Factory for Hadoop {@link FileSystem} used by {@link IgniteHadoopIgfsSecondaryFileSystem}. + * Factory for Hadoop {@code FileSystem} used by {@link IgniteHadoopIgfsSecondaryFileSystem}. *

          * {@link #get(String)} method will be used whenever a call to a target {@code FileSystem} is required. *

          @@ -35,10 +32,6 @@ *

          * Concrete factory may implement {@link LifecycleAware} interface. In this case start and stop callbacks will be * performed by Ignite. You may want to implement some initialization or cleanup there. - *

          - * Note that factory extends {@link Serializable} interface as it might be necessary to transfer factories over the - * wire to {@link IgniteHadoopFileSystem} if {@link IgfsMode#PROXY} is enabled for some file - * system paths. */ public interface HadoopFileSystemFactory extends Serializable { /** @@ -48,5 +41,5 @@ public interface HadoopFileSystemFactory extends Serializable { * @return File system. * @throws IOException In case of error. */ - public FileSystem get(String usrName) throws IOException; + public Object get(String usrName) throws IOException; } diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java index 80858269eac25..f1c1b16524466 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java @@ -17,25 +17,12 @@ package org.apache.ignite.hadoop.fs; -import java.io.IOException; -import java.io.PrintStream; -import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo; import org.apache.ignite.internal.processors.hadoop.HadoopJob; -import org.apache.ignite.internal.processors.hadoop.HadoopJobId; -import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo; -import org.apache.ignite.internal.processors.hadoop.HadoopUtils; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounterWriter; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters; -import org.apache.ignite.internal.processors.hadoop.counter.HadoopPerformanceCounter; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopV2Job; -import org.apache.ignite.internal.processors.igfs.IgfsUtils; -import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemCounterWriterDelegate; /** * Statistic writer implementation that writes info into any Hadoop file system. @@ -47,57 +34,39 @@ public class IgniteHadoopFileSystemCounterWriter implements HadoopCounterWriter /** */ public static final String COUNTER_WRITER_DIR_PROPERTY = "ignite.counters.fswriter.directory"; - /** */ - private static final String USER_MACRO = "${USER}"; + /** Mutex. */ + private final Object mux = new Object(); - /** */ - private static final String DEFAULT_COUNTER_WRITER_DIR = "/user/" + USER_MACRO; + /** Delegate. */ + private volatile HadoopFileSystemCounterWriterDelegate delegate; /** {@inheritDoc} */ @Override public void write(HadoopJob job, HadoopCounters cntrs) throws IgniteCheckedException { + delegate(job).write(job, cntrs); + } - Configuration hadoopCfg = HadoopUtils.safeCreateConfiguration(); - - final HadoopJobInfo jobInfo = job.info(); - - final HadoopJobId jobId = job.id(); - - for (Map.Entry e : ((HadoopDefaultJobInfo)jobInfo).properties().entrySet()) - hadoopCfg.set(e.getKey(), e.getValue()); - - String user = jobInfo.user(); - - user = IgfsUtils.fixUserName(user); - - String dir = jobInfo.property(COUNTER_WRITER_DIR_PROPERTY); - - if (dir == null) - dir = DEFAULT_COUNTER_WRITER_DIR; - - Path jobStatPath = new Path(new Path(dir.replace(USER_MACRO, user)), jobId.toString()); - - HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(cntrs, null); - - try { - hadoopCfg.set(MRJobConfig.USER_NAME, user); + /** + * Get delegate creating it if needed. + * + * @param job Job. + * @return Delegate. + */ + private HadoopFileSystemCounterWriterDelegate delegate(HadoopJob job) { + HadoopFileSystemCounterWriterDelegate delegate0 = delegate; - FileSystem fs = ((HadoopV2Job)job).fileSystem(jobStatPath.toUri(), hadoopCfg); + if (delegate0 == null) { + synchronized (mux) { + delegate0 = delegate; - fs.mkdirs(jobStatPath); + if (delegate0 == null) { + delegate0 = HadoopDelegateUtils.counterWriterDelegate(job.getClass().getClassLoader(), this); - try (PrintStream out = new PrintStream(fs.create(new Path(jobStatPath, PERFORMANCE_COUNTER_FILE_NAME)))) { - for (T2 evt : perfCntr.evts()) { - out.print(evt.get1()); - out.print(':'); - out.println(evt.get2().toString()); + delegate = delegate0; } - - out.flush(); } } - catch (IOException e) { - throw new IgniteCheckedException(e); - } + + return delegate0; } } \ No newline at end of file diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java index 6b5c7762f9ff7..c9d08c5be4a2c 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java @@ -17,62 +17,48 @@ package org.apache.ignite.hadoop.fs; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.ParentNotDirectoryException; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathExistsException; -import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException; -import org.apache.hadoop.fs.permission.FsPermission; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteFileSystem; -import org.apache.ignite.igfs.IgfsDirectoryNotEmptyException; -import org.apache.ignite.igfs.IgfsException; import org.apache.ignite.igfs.IgfsFile; -import org.apache.ignite.igfs.IgfsParentNotDirectoryException; import org.apache.ignite.igfs.IgfsPath; -import org.apache.ignite.igfs.IgfsPathAlreadyExistsException; -import org.apache.ignite.igfs.IgfsPathNotFoundException; import org.apache.ignite.igfs.IgfsUserContext; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader; +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; import org.apache.ignite.internal.processors.hadoop.HadoopPayloadAware; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsProperties; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsSecondaryFileSystemPositionedReadable; -import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo; -import org.apache.ignite.internal.processors.igfs.IgfsFileImpl; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopIgfsSecondaryFileSystemDelegate; +import org.apache.ignite.internal.processors.igfs.IgfsKernalContextAware; import org.apache.ignite.internal.processors.igfs.IgfsSecondaryFileSystemV2; -import org.apache.ignite.internal.processors.igfs.IgfsUtils; -import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.lang.IgniteOutClosure; -import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.lifecycle.LifecycleAware; import org.jetbrains.annotations.Nullable; -import java.io.FileNotFoundException; -import java.io.IOException; import java.io.OutputStream; -import java.net.URI; -import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.concurrent.Callable; /** - * Secondary file system which delegates calls to an instance of Hadoop {@link FileSystem}. + * Secondary file system which delegates calls to Hadoop {@code org.apache.hadoop.fs.FileSystem}. *

          * Target {@code FileSystem}'s are created on per-user basis using passed {@link HadoopFileSystemFactory}. */ -public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSystemV2, LifecycleAware, - HadoopPayloadAware { +public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSystemV2, IgfsKernalContextAware, + LifecycleAware, HadoopPayloadAware { /** The default user name. It is used if no user context is set. */ private String dfltUsrName; /** Factory. */ - private HadoopFileSystemFactory fsFactory; + private HadoopFileSystemFactory factory; + + /** Kernal context. */ + private volatile GridKernalContext ctx; + + /** Target. */ + private volatile HadoopIgfsSecondaryFileSystemDelegate target; /** * Default constructor for Spring. @@ -135,7 +121,7 @@ public IgniteHadoopIgfsSecondaryFileSystem(@Nullable String uri, @Nullable Strin * Gets default user name. *

          * Defines user name which will be used during file system invocation in case no user name is defined explicitly - * through {@link FileSystem#get(URI, Configuration, String)}. + * through {@code FileSystem.get(URI, Configuration, String)}. *

          * Also this name will be used if you manipulate {@link IgniteFileSystem} directly and do not set user name * explicitly using {@link IgfsUserContext#doAs(String, IgniteOutClosure)} or @@ -162,14 +148,14 @@ public void setDefaultUserName(@Nullable String dfltUsrName) { /** * Gets secondary file system factory. *

          - * This factory will be used whenever a call to a target {@link FileSystem} is required. + * This factory will be used whenever a call to a target {@code FileSystem} is required. *

          * If not set, {@link CachingHadoopFileSystemFactory} will be used. * * @return Secondary file system factory. */ public HadoopFileSystemFactory getFileSystemFactory() { - return fsFactory; + return factory; } /** @@ -178,403 +164,115 @@ public HadoopFileSystemFactory getFileSystemFactory() { * @param factory Secondary file system factory. */ public void setFileSystemFactory(HadoopFileSystemFactory factory) { - this.fsFactory = factory; - } - - /** - * Convert IGFS path into Hadoop path. - * - * @param path IGFS path. - * @return Hadoop path. - */ - private Path convert(IgfsPath path) { - URI uri = fileSystemForUser().getUri(); - - return new Path(uri.getScheme(), uri.getAuthority(), path.toString()); - } - - /** - * Heuristically checks if exception was caused by invalid HDFS version and returns appropriate exception. - * - * @param e Exception to check. - * @param detailMsg Detailed error message. - * @return Appropriate exception. - */ - private IgfsException handleSecondaryFsError(IOException e, String detailMsg) { - return cast(detailMsg, e); - } - - /** - * Cast IO exception to IGFS exception. - * - * @param e IO exception. - * @return IGFS exception. - */ - public static IgfsException cast(String msg, IOException e) { - if (e instanceof FileNotFoundException) - return new IgfsPathNotFoundException(e); - else if (e instanceof ParentNotDirectoryException) - return new IgfsParentNotDirectoryException(msg, e); - else if (e instanceof PathIsNotEmptyDirectoryException) - return new IgfsDirectoryNotEmptyException(e); - else if (e instanceof PathExistsException) - return new IgfsPathAlreadyExistsException(msg, e); - else - return new IgfsException(msg, e); - } - - /** - * Convert Hadoop FileStatus properties to map. - * - * @param status File status. - * @return IGFS attributes. - */ - private static Map properties(FileStatus status) { - FsPermission perm = status.getPermission(); - - if (perm == null) - perm = FsPermission.getDefault(); - - HashMap res = new HashMap<>(3); - - res.put(IgfsUtils.PROP_PERMISSION, String.format("%04o", perm.toShort())); - res.put(IgfsUtils.PROP_USER_NAME, status.getOwner()); - res.put(IgfsUtils.PROP_GROUP_NAME, status.getGroup()); - - return res; + this.factory = factory; } /** {@inheritDoc} */ @Override public boolean exists(IgfsPath path) { - try { - return fileSystemForUser().exists(convert(path)); - } - catch (IOException e) { - throw handleSecondaryFsError(e, "Failed to check file existence [path=" + path + "]"); - } + return target.exists(path); } /** {@inheritDoc} */ @Nullable @Override public IgfsFile update(IgfsPath path, Map props) { - HadoopIgfsProperties props0 = new HadoopIgfsProperties(props); - - final FileSystem fileSys = fileSystemForUser(); - - try { - if (props0.userName() != null || props0.groupName() != null) - fileSys.setOwner(convert(path), props0.userName(), props0.groupName()); - - if (props0.permission() != null) - fileSys.setPermission(convert(path), props0.permission()); - } - catch (IOException e) { - throw handleSecondaryFsError(e, "Failed to update file properties [path=" + path + "]"); - } - - //Result is not used in case of secondary FS. - return null; + return target.update(path, props); } /** {@inheritDoc} */ @Override public void rename(IgfsPath src, IgfsPath dest) { - // Delegate to the secondary file system. - try { - if (!fileSystemForUser().rename(convert(src), convert(dest))) - throw new IgfsException("Failed to rename (secondary file system returned false) " + - "[src=" + src + ", dest=" + dest + ']'); - } - catch (IOException e) { - throw handleSecondaryFsError(e, "Failed to rename file [src=" + src + ", dest=" + dest + ']'); - } + target.rename(src, dest); } /** {@inheritDoc} */ @Override public boolean delete(IgfsPath path, boolean recursive) { - try { - return fileSystemForUser().delete(convert(path), recursive); - } - catch (IOException e) { - throw handleSecondaryFsError(e, "Failed to delete file [path=" + path + ", recursive=" + recursive + "]"); - } + return target.delete(path, recursive); } /** {@inheritDoc} */ @Override public void mkdirs(IgfsPath path) { - try { - if (!fileSystemForUser().mkdirs(convert(path))) - throw new IgniteException("Failed to make directories [path=" + path + "]"); - } - catch (IOException e) { - throw handleSecondaryFsError(e, "Failed to make directories [path=" + path + "]"); - } + target.mkdirs(path); } /** {@inheritDoc} */ @Override public void mkdirs(IgfsPath path, @Nullable Map props) { - try { - if (!fileSystemForUser().mkdirs(convert(path), new HadoopIgfsProperties(props).permission())) - throw new IgniteException("Failed to make directories [path=" + path + ", props=" + props + "]"); - } - catch (IOException e) { - throw handleSecondaryFsError(e, "Failed to make directories [path=" + path + ", props=" + props + "]"); - } + target.mkdirs(path, props); } /** {@inheritDoc} */ @Override public Collection listPaths(IgfsPath path) { - try { - FileStatus[] statuses = fileSystemForUser().listStatus(convert(path)); - - if (statuses == null) - throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path); - - Collection res = new ArrayList<>(statuses.length); - - for (FileStatus status : statuses) - res.add(new IgfsPath(path, status.getPath().getName())); - - return res; - } - catch (FileNotFoundException ignored) { - throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path); - } - catch (IOException e) { - throw handleSecondaryFsError(e, "Failed to list statuses due to secondary file system exception: " + path); - } + return target.listPaths(path); } /** {@inheritDoc} */ @Override public Collection listFiles(IgfsPath path) { - try { - FileStatus[] statuses = fileSystemForUser().listStatus(convert(path)); - - if (statuses == null) - throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path); - - Collection res = new ArrayList<>(statuses.length); - - for (FileStatus s : statuses) { - IgfsEntryInfo fsInfo = s.isDirectory() ? - IgfsUtils.createDirectory( - IgniteUuid.randomUuid(), - null, - properties(s), - s.getAccessTime(), - s.getModificationTime() - ) : - IgfsUtils.createFile( - IgniteUuid.randomUuid(), - (int)s.getBlockSize(), - s.getLen(), - null, - null, - false, - properties(s), - s.getAccessTime(), - s.getModificationTime() - ); - - res.add(new IgfsFileImpl(new IgfsPath(path, s.getPath().getName()), fsInfo, 1)); - } - - return res; - } - catch (FileNotFoundException ignored) { - throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path); - } - catch (IOException e) { - throw handleSecondaryFsError(e, "Failed to list statuses due to secondary file system exception: " + path); - } + return target.listFiles(path); } /** {@inheritDoc} */ @Override public IgfsSecondaryFileSystemPositionedReadable open(IgfsPath path, int bufSize) { - return new HadoopIgfsSecondaryFileSystemPositionedReadable(fileSystemForUser(), convert(path), bufSize); + return target.open(path, bufSize); } /** {@inheritDoc} */ @Override public OutputStream create(IgfsPath path, boolean overwrite) { - try { - return fileSystemForUser().create(convert(path), overwrite); - } - catch (IOException e) { - throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", overwrite=" + overwrite + "]"); - } + return target.create(path, overwrite); } /** {@inheritDoc} */ @Override public OutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication, long blockSize, @Nullable Map props) { - HadoopIgfsProperties props0 = - new HadoopIgfsProperties(props != null ? props : Collections.emptyMap()); - - try { - return fileSystemForUser().create(convert(path), props0.permission(), overwrite, bufSize, - (short) replication, blockSize, null); - } - catch (IOException e) { - throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", props=" + props + - ", overwrite=" + overwrite + ", bufSize=" + bufSize + ", replication=" + replication + - ", blockSize=" + blockSize + "]"); - } + return target.create(path, bufSize, overwrite, replication, blockSize, props); } /** {@inheritDoc} */ @Override public OutputStream append(IgfsPath path, int bufSize, boolean create, @Nullable Map props) { - try { - return fileSystemForUser().append(convert(path), bufSize); - } - catch (IOException e) { - throw handleSecondaryFsError(e, "Failed to append file [path=" + path + ", bufSize=" + bufSize + "]"); - } + return target.append(path, bufSize, create, props); } /** {@inheritDoc} */ @Override public IgfsFile info(final IgfsPath path) { - try { - final FileStatus status = fileSystemForUser().getFileStatus(convert(path)); - - if (status == null) - return null; - - final Map props = properties(status); - - return new IgfsFile() { - @Override public IgfsPath path() { - return path; - } - - @Override public boolean isFile() { - return status.isFile(); - } - - @Override public boolean isDirectory() { - return status.isDirectory(); - } - - @Override public int blockSize() { - // By convention directory has blockSize == 0, while file has blockSize > 0: - return isDirectory() ? 0 : (int)status.getBlockSize(); - } - - @Override public long groupBlockSize() { - return status.getBlockSize(); - } - - @Override public long accessTime() { - return status.getAccessTime(); - } - - @Override public long modificationTime() { - return status.getModificationTime(); - } - - @Override public String property(String name) throws IllegalArgumentException { - String val = props.get(name); - - if (val == null) - throw new IllegalArgumentException("File property not found [path=" + path + ", name=" + name + ']'); - - return val; - } - - @Nullable @Override public String property(String name, @Nullable String dfltVal) { - String val = props.get(name); - - return val == null ? dfltVal : val; - } - - @Override public long length() { - return status.getLen(); - } - - /** {@inheritDoc} */ - @Override public Map properties() { - return props; - } - }; - } - catch (FileNotFoundException ignore) { - return null; - } - catch (IOException e) { - throw handleSecondaryFsError(e, "Failed to get file status [path=" + path + "]"); - } + return target.info(path); } /** {@inheritDoc} */ @Override public long usedSpaceSize() { - try { - // We don't use FileSystem#getUsed() since it counts only the files - // in the filesystem root, not all the files recursively. - return fileSystemForUser().getContentSummary(new Path("/")).getSpaceConsumed(); - } - catch (IOException e) { - throw handleSecondaryFsError(e, "Failed to get used space size of file system."); - } + return target.usedSpaceSize(); } /** {@inheritDoc} */ @Override public void setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteException { - try { - // We don't use FileSystem#getUsed() since it counts only the files - // in the filesystem root, not all the files recursively. - fileSystemForUser().setTimes(convert(path), modificationTime, accessTime); - } - catch (IOException e) { - throw handleSecondaryFsError(e, "Failed set times for path: " + path); - } + target.setTimes(path, accessTime, modificationTime); } - /** - * Gets the underlying {@link FileSystem}. - * This method is used solely for testing. - * @return the underlying Hadoop {@link FileSystem}. - */ - public FileSystem fileSystem() { - return fileSystemForUser(); + /** {@inheritDoc} */ + @Override public void setKernalContext(GridKernalContext ctx) { + this.ctx = ctx; } - /** - * Gets the FileSystem for the current context user. - * @return the FileSystem instance, never null. - */ - private FileSystem fileSystemForUser() { - String user = IgfsUserContext.currentUser(); - - if (F.isEmpty(user)) - user = IgfsUtils.fixUserName(dfltUsrName); + /** {@inheritDoc} */ + @Override public void start() throws IgniteException { + HadoopClassLoader ldr = ctx.hadoopHelper().commonClassLoader(); - assert !F.isEmpty(user); + ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(ldr); try { - return fsFactory.get(user); + target = HadoopDelegateUtils.secondaryFileSystemDelegate(ldr, this); + + target.start(); } - catch (IOException ioe) { - throw new IgniteException(ioe); + finally { + HadoopCommonUtils.restoreContextClassLoader(oldLdr); } } - /** {@inheritDoc} */ - @Override public void start() throws IgniteException { - dfltUsrName = IgfsUtils.fixUserName(dfltUsrName); - - if (fsFactory == null) - fsFactory = new CachingHadoopFileSystemFactory(); - - if (fsFactory instanceof LifecycleAware) - ((LifecycleAware) fsFactory).start(); - } - /** {@inheritDoc} */ @Override public void stop() throws IgniteException { - if (fsFactory instanceof LifecycleAware) - ((LifecycleAware)fsFactory).stop(); + if (target != null) + target.stop(); } /** {@inheritDoc} */ @Override public HadoopFileSystemFactory getPayload() { - return fsFactory; + return factory; } } \ No newline at end of file diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactory.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactory.java index bbfbc59100d62..46d626bfb61fe 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactory.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactory.java @@ -17,19 +17,12 @@ package org.apache.ignite.hadoop.fs; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.ignite.IgniteException; -import org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; -import java.security.PrivilegedExceptionAction; /** * Secure Hadoop file system factory that can work with underlying file system protected with Kerberos. @@ -57,9 +50,6 @@ public class KerberosHadoopFileSystemFactory extends BasicHadoopFileSystemFactor /** The re-login interval. See {@link #getReloginInterval()} for more information. */ private long reloginInterval = DFLT_RELOGIN_INTERVAL; - /** Time of last re-login attempt, in system milliseconds. */ - private transient volatile long lastReloginTime; - /** * Constructor. */ @@ -67,25 +57,6 @@ public KerberosHadoopFileSystemFactory() { // No-op. } - /** {@inheritDoc} */ - @Override public FileSystem getWithMappedName(String name) throws IOException { - reloginIfNeeded(); - - return super.getWithMappedName(name); - } - - /** {@inheritDoc} */ - @Override protected FileSystem create(String usrName) throws IOException, InterruptedException { - UserGroupInformation proxyUgi = UserGroupInformation.createProxyUser(usrName, - UserGroupInformation.getLoginUser()); - - return proxyUgi.doAs(new PrivilegedExceptionAction() { - @Override public FileSystem run() throws Exception { - return FileSystem.get(fullUri, cfg); - } - }); - } - /** * Gets the key tab principal short name (e.g. "hdfs"). * @@ -106,9 +77,6 @@ public void setKeyTabPrincipal(@Nullable String keyTabPrincipal) { /** * Gets the key tab full file name (e.g. "/etc/security/keytabs/hdfs.headless.keytab" or "/etc/krb5.keytab"). - *

          - * NOTE! Factory can be serialized and transferred to other machines where instance of - * {@link IgniteHadoopFileSystem} resides. Corresponding path must exist on these machines as well. * * @return The key tab file name. */ @@ -136,10 +104,8 @@ public void setKeyTab(@Nullable String keyTab) { * Negative values are not allowed. * *

          Note, however, that it does not make sense to make this value small, because Hadoop does not allow to - * login if less than {@link org.apache.hadoop.security.UserGroupInformation#MIN_TIME_BEFORE_RELOGIN} milliseconds + * login if less than {@code org.apache.hadoop.security.UserGroupInformation.MIN_TIME_BEFORE_RELOGIN} milliseconds * have passed since the time of the previous login. - * See {@link org.apache.hadoop.security.UserGroupInformation#hasSufficientTimeElapsed(long)} and its usages for - * more detail. * * @return The re-login interval, in milliseconds. */ @@ -156,47 +122,6 @@ public void setReloginInterval(long reloginInterval) { this.reloginInterval = reloginInterval; } - /** {@inheritDoc} */ - @Override public void start() throws IgniteException { - A.ensure(!F.isEmpty(keyTab), "keyTab cannot not be empty."); - A.ensure(!F.isEmpty(keyTabPrincipal), "keyTabPrincipal cannot not be empty."); - A.ensure(reloginInterval >= 0, "reloginInterval cannot not be negative."); - - super.start(); - - try { - UserGroupInformation.setConfiguration(cfg); - UserGroupInformation.loginUserFromKeytab(keyTabPrincipal, keyTab); - } - catch (IOException ioe) { - throw new IgniteException("Failed login from keytab [keyTab=" + keyTab + - ", keyTabPrincipal=" + keyTabPrincipal + ']', ioe); - } - } - - /** - * Re-logins the user if needed. - * First, the re-login interval defined in factory is checked. The re-login attempts will be not more - * frequent than one attempt per {@code reloginInterval}. - * Second, {@link UserGroupInformation#checkTGTAndReloginFromKeytab()} method invoked that gets existing - * TGT and checks its validity. If the TGT is expired or is close to expiry, it performs re-login. - * - *

          This operation expected to be called upon each operation with the file system created with the factory. - * As long as {@link #get(String)} operation is invoked upon each file {@link IgniteHadoopFileSystem}, there - * is no need to invoke it otherwise specially. - * - * @throws IOException If login fails. - */ - private void reloginIfNeeded() throws IOException { - long now = System.currentTimeMillis(); - - if (now >= lastReloginTime + reloginInterval) { - UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab(); - - lastReloginTime = now; - } - } - /** {@inheritDoc} */ @Override public void writeExternal(ObjectOutput out) throws IOException { super.writeExternal(out); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java index a06129e43fc63..7133c087b9dbe 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java @@ -42,12 +42,14 @@ import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.IgfsPathSummary; import org.apache.ignite.internal.igfs.common.IgfsLogger; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsInputStream; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsOutputStream; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsProxyInputStream; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsProxyOutputStream; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsStreamDelegate; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsWrapper; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInputStream; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsOutputStream; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsProxyInputStream; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsProxyOutputStream; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsStreamDelegate; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsWrapper; import org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse; import org.apache.ignite.internal.processors.igfs.IgfsModeResolver; import org.apache.ignite.internal.processors.igfs.IgfsPaths; @@ -58,7 +60,6 @@ import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lifecycle.LifecycleAware; import org.jetbrains.annotations.Nullable; import java.io.BufferedOutputStream; @@ -78,13 +79,13 @@ import static org.apache.ignite.configuration.FileSystemConfiguration.DFLT_IGFS_LOG_BATCH_SIZE; import static org.apache.ignite.configuration.FileSystemConfiguration.DFLT_IGFS_LOG_DIR; import static org.apache.ignite.igfs.IgfsMode.PROXY; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_COLOCATED_WRITES; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_LOG_BATCH_SIZE; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_LOG_DIR; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_LOG_ENABLED; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_PREFER_LOCAL_WRITES; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH; -import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.parameter; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_COLOCATED_WRITES; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_LOG_BATCH_SIZE; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_LOG_DIR; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_LOG_ENABLED; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_PREFER_LOCAL_WRITES; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH; +import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.parameter; import static org.apache.ignite.internal.processors.igfs.IgfsEx.IGFS_SCHEME; /** @@ -165,7 +166,7 @@ public class IgniteHadoopFileSystem extends FileSystem { private IgfsModeResolver modeRslvr; /** The secondary file system factory. */ - private HadoopFileSystemFactory factory; + private HadoopFileSystemFactoryDelegate factory; /** Management connection flag. */ private boolean mgmt; @@ -332,7 +333,10 @@ public void colocateFileWrites(boolean colocateFileWrites) { if (initSecondary) { try { - factory = (HadoopFileSystemFactory) paths.getPayload(getClass().getClassLoader()); + HadoopFileSystemFactory factory0 = + (HadoopFileSystemFactory)paths.getPayload(getClass().getClassLoader()); + + factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory0); } catch (IgniteCheckedException e) { throw new IOException("Failed to get secondary file system factory.", e); @@ -343,11 +347,10 @@ public void colocateFileWrites(boolean colocateFileWrites) { IgniteHadoopIgfsSecondaryFileSystem.class.getName() + " as \"secondaryFIleSystem\" in " + FileSystemConfiguration.class.getName() + "?)"); - if (factory instanceof LifecycleAware) - ((LifecycleAware) factory).start(); + factory.start(); try { - FileSystem secFs = factory.get(user); + FileSystem secFs = (FileSystem)factory.get(user); secondaryUri = secFs.getUri(); @@ -423,8 +426,8 @@ private void close0() throws IOException { if (clientLog.isLogEnabled()) clientLog.close(); - if (factory instanceof LifecycleAware) - ((LifecycleAware) factory).stop(); + if (factory != null) + factory.stop(); // Reset initialized resources. uri = null; @@ -1359,6 +1362,6 @@ public String user() { if (factory == null) return null; - return factory.get(user); + return (FileSystem)factory.get(user); } } \ No newline at end of file diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java index bd8ed2db35c93..18b8bf9fc7e19 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java @@ -46,13 +46,15 @@ import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.igfs.common.IgfsLogger; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate; import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsEndpoint; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsInputStream; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsOutputStream; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsProxyInputStream; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsProxyOutputStream; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsStreamDelegate; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsWrapper; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInputStream; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsOutputStream; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsProxyInputStream; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsProxyOutputStream; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsStreamDelegate; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsWrapper; import org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse; import org.apache.ignite.internal.processors.igfs.IgfsModeResolver; import org.apache.ignite.internal.processors.igfs.IgfsPaths; @@ -63,7 +65,6 @@ import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lifecycle.LifecycleAware; import org.jetbrains.annotations.Nullable; import java.io.BufferedOutputStream; @@ -86,13 +87,13 @@ import static org.apache.ignite.configuration.FileSystemConfiguration.DFLT_IGFS_LOG_DIR; import static org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem.getFsHadoopUser; import static org.apache.ignite.igfs.IgfsMode.PROXY; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_COLOCATED_WRITES; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_LOG_BATCH_SIZE; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_LOG_DIR; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_LOG_ENABLED; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_PREFER_LOCAL_WRITES; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH; -import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.parameter; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_COLOCATED_WRITES; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_LOG_BATCH_SIZE; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_LOG_DIR; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_LOG_ENABLED; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_PREFER_LOCAL_WRITES; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH; +import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.parameter; import static org.apache.ignite.internal.processors.igfs.IgfsEx.IGFS_SCHEME; /** @@ -169,7 +170,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea private IgfsModeResolver modeRslvr; /** The secondary file system factory. */ - private HadoopFileSystemFactory factory; + private HadoopFileSystemFactoryDelegate factory; /** Whether custom sequential reads before prefetch value is provided. */ private boolean seqReadsBeforePrefetchOverride; @@ -341,7 +342,10 @@ private void initialize(URI name, Configuration cfg) throws IOException { if (initSecondary) { try { - factory = (HadoopFileSystemFactory) paths.getPayload(getClass().getClassLoader()); + HadoopFileSystemFactory factory0 = + (HadoopFileSystemFactory) paths.getPayload(getClass().getClassLoader()); + + factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory0); } catch (IgniteCheckedException e) { throw new IOException("Failed to get secondary file system factory.", e); @@ -354,11 +358,10 @@ private void initialize(URI name, Configuration cfg) throws IOException { assert factory != null; - if (factory instanceof LifecycleAware) - ((LifecycleAware) factory).start(); + factory.start(); try { - FileSystem secFs = factory.get(user); + FileSystem secFs = (FileSystem)factory.get(user); secondaryUri = secFs.getUri(); @@ -385,8 +388,8 @@ private void initialize(URI name, Configuration cfg) throws IOException { if (clientLog.isLogEnabled()) clientLog.close(); - if (factory instanceof LifecycleAware) - ((LifecycleAware) factory).stop(); + if (factory != null) + factory.stop(); // Reset initialized resources. rmtClient = null; @@ -1071,6 +1074,6 @@ public String user() { private FileSystem secondaryFileSystem() throws IOException{ assert factory != null; - return factory.get(user); + return (FileSystem)factory.get(user); } } \ No newline at end of file diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java index 583af35e58c2a..343b5edbb4182 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java @@ -17,10 +17,6 @@ package org.apache.ignite.hadoop.mapreduce; -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.Collections; -import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.mapreduce.protocol.ClientProtocol; @@ -32,10 +28,15 @@ import org.apache.ignite.internal.client.GridClientException; import org.apache.ignite.internal.client.GridClientFactory; import org.apache.ignite.internal.client.marshaller.jdk.GridClientJdkMarshaller; -import org.apache.ignite.internal.processors.hadoop.proto.HadoopClientProtocol; +import org.apache.ignite.internal.processors.hadoop.impl.proto.HadoopClientProtocol; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.F; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.Collections; +import java.util.concurrent.ConcurrentHashMap; + import static org.apache.ignite.internal.client.GridClientProtocol.TCP; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopMapReducePlanner.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopMapReducePlanner.java index d4a44fa9e1be4..e1101c5a0e8e4 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopMapReducePlanner.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopMapReducePlanner.java @@ -17,16 +17,6 @@ package org.apache.ignite.hadoop.mapreduce; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.ListIterator; -import java.util.Map; -import java.util.UUID; - import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; @@ -38,13 +28,23 @@ import org.apache.ignite.internal.processors.hadoop.HadoopJob; import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan; import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsEndpoint; -import org.apache.ignite.internal.processors.hadoop.planner.HadoopDefaultMapReducePlan; import org.apache.ignite.internal.processors.hadoop.planner.HadoopAbstractMapReducePlanner; +import org.apache.ignite.internal.processors.hadoop.planner.HadoopDefaultMapReducePlan; import org.apache.ignite.internal.processors.igfs.IgfsEx; import org.apache.ignite.internal.util.typedef.F; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; +import java.util.UUID; + import static org.apache.ignite.IgniteFileSystem.IGFS_SCHEME; /** diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopWeightedMapReducePlanner.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopWeightedMapReducePlanner.java index 27ffc19f9d105..2d1ac0b2ffa8c 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopWeightedMapReducePlanner.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopWeightedMapReducePlanner.java @@ -24,11 +24,11 @@ import org.apache.ignite.igfs.IgfsBlockLocation; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock; import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit; import org.apache.ignite.internal.processors.hadoop.HadoopJob; import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan; -import org.apache.ignite.internal.processors.hadoop.HadoopUtils; import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsEndpoint; import org.apache.ignite.internal.processors.hadoop.planner.HadoopAbstractMapReducePlanner; import org.apache.ignite.internal.processors.hadoop.planner.HadoopDefaultMapReducePlan; @@ -116,7 +116,7 @@ public class IgniteHadoopWeightedMapReducePlanner extends HadoopAbstractMapReduc /** {@inheritDoc} */ @Override public HadoopMapReducePlan preparePlan(HadoopJob job, Collection nodes, @Nullable HadoopMapReducePlan oldPlan) throws IgniteCheckedException { - List splits = HadoopUtils.sortInputSplits(job.input()); + List splits = HadoopCommonUtils.sortInputSplits(job.input()); int reducerCnt = job.info().reducers(); if (reducerCnt < 0) diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/util/UserNameMapper.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/util/UserNameMapper.java index 26dc4b2f86673..12669aa3f2573 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/util/UserNameMapper.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/util/UserNameMapper.java @@ -17,14 +17,12 @@ package org.apache.ignite.hadoop.util; -import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory; import org.jetbrains.annotations.Nullable; import java.io.Serializable; /** - * Hadoop file system name mapper. Used by {@link HadoopFileSystemFactory} implementation to pass proper user names - * to the underlying Hadoop file system. + * Hadoop file system name mapper. Ensures that correct user name is passed to the underlying Hadoop file system. */ public interface UserNameMapper extends Serializable { /** diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java deleted file mode 100644 index 2e0e271b78b5d..0000000000000 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java +++ /dev/null @@ -1,964 +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.hadoop; - -import java.io.IOException; -import java.io.InputStream; -import java.net.URL; -import java.net.URLClassLoader; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import java.util.Vector; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -import org.apache.hadoop.util.NativeCodeLoader; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopDaemon; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopShutdownHookManager; -import org.apache.ignite.internal.util.ClassCache; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.jetbrains.annotations.Nullable; -import org.jsr166.ConcurrentHashMap8; -import org.objectweb.asm.AnnotationVisitor; -import org.objectweb.asm.Attribute; -import org.objectweb.asm.ClassReader; -import org.objectweb.asm.ClassVisitor; -import org.objectweb.asm.ClassWriter; -import org.objectweb.asm.FieldVisitor; -import org.objectweb.asm.Handle; -import org.objectweb.asm.Label; -import org.objectweb.asm.MethodVisitor; -import org.objectweb.asm.Opcodes; -import org.objectweb.asm.Type; -import org.objectweb.asm.commons.Remapper; -import org.objectweb.asm.commons.RemappingClassAdapter; - -/** - * Class loader allowing explicitly load classes without delegation to parent class loader. - * Also supports class parsing for finding dependencies which contain transitive dependencies - * unavailable for parent. - */ -public class HadoopClassLoader extends URLClassLoader implements ClassCache { - static { - // We are very parallel capable. - registerAsParallelCapable(); - } - - /** Name of the Hadoop daemon class. */ - public static final String HADOOP_DAEMON_CLASS_NAME = "org.apache.hadoop.util.Daemon"; - - /** Name of libhadoop library. */ - private static final String LIBHADOOP = "hadoop."; - - /** */ - private static final URLClassLoader APP_CLS_LDR = (URLClassLoader)HadoopClassLoader.class.getClassLoader(); - - /** */ - private static final Collection appJars = F.asList(APP_CLS_LDR.getURLs()); - - /** */ - private static volatile Collection hadoopJars; - - /** */ - private static final Map cache = new ConcurrentHashMap8<>(); - - /** */ - private static final Map bytesCache = new ConcurrentHashMap8<>(); - - /** Class cache. */ - private final ConcurrentMap cacheMap = new ConcurrentHashMap<>(); - - /** Diagnostic name of this class loader. */ - @SuppressWarnings({"FieldCanBeLocal", "UnusedDeclaration"}) - private final String name; - - /** Native library names. */ - private final String[] libNames; - - /** - * Gets name for Job class loader. The name is specific for local node id. - * @param locNodeId The local node id. - * @return The class loader name. - */ - public static String nameForJob(UUID locNodeId) { - return "hadoop-job-node-" + locNodeId.toString(); - } - - /** - * Gets name for the task class loader. Task class loader - * @param info The task info. - * @param prefix Get only prefix (without task type and number) - * @return The class loader name. - */ - public static String nameForTask(HadoopTaskInfo info, boolean prefix) { - if (prefix) - return "hadoop-task-" + info.jobId() + "-"; - else - return "hadoop-task-" + info.jobId() + "-" + info.type() + "-" + info.taskNumber(); - } - - /** - * Constructor. - * - * @param urls Urls. - * @param name Classloader name. - * @param libNames Optional additional native library names to be linked from parent classloader. - */ - public HadoopClassLoader(URL[] urls, String name, @Nullable String[] libNames) { - super(addHadoopUrls(urls), APP_CLS_LDR); - - assert !(getParent() instanceof HadoopClassLoader); - - this.name = name; - this.libNames = libNames; - - initializeNativeLibraries(); - } - - /** - * Workaround to load native Hadoop libraries. Java doesn't allow native libraries to be loaded from different - * classloaders. But we load Hadoop classes many times and one of these classes - {@code NativeCodeLoader} - tries - * to load the same native library over and over again. - *

          - * To fix the problem, we force native library load in parent class loader and then "link" handle to this native - * library to our class loader. As a result, our class loader will think that the library is already loaded and will - * be able to link native methods. - * - * @see - * JNI specification - */ - private void initializeNativeLibraries() { - try { - // This must trigger native library load. - Class.forName(NativeCodeLoader.class.getName(), true, APP_CLS_LDR); - - final Vector curVector = U.field(this, "nativeLibraries"); - - ClassLoader ldr = APP_CLS_LDR; - - while (ldr != null) { - Vector vector = U.field(ldr, "nativeLibraries"); - - for (Object lib : vector) { - String name = U.field(lib, "name"); - - boolean add = name.contains(LIBHADOOP); - - if (!add && libNames != null) { - for (String libName : libNames) { - if (libName != null && name.contains(libName)) { - add = true; - - break; - } - } - } - - if (add) { - curVector.add(lib); - - return; - } - } - - ldr = ldr.getParent(); - } - } - catch (Exception e) { - U.quietAndWarn(null, "Failed to initialize Hadoop native library " + - "(native Hadoop methods might not work properly): " + e); - } - } - - /** - * Need to parse only Ignite Hadoop and IGFS classes. - * - * @param cls Class name. - * @return {@code true} if we need to check this class. - */ - private static boolean isHadoopIgfs(String cls) { - String ignitePkgPrefix = "org.apache.ignite"; - - int len = ignitePkgPrefix.length(); - - return cls.startsWith(ignitePkgPrefix) && ( - cls.indexOf("igfs.", len) != -1 || - cls.indexOf(".fs.", len) != -1 || - cls.indexOf("hadoop.", len) != -1); - } - - /** - * @param cls Class name. - * @return {@code true} If this is Hadoop class. - */ - private static boolean isHadoop(String cls) { - return cls.startsWith("org.apache.hadoop."); - } - - /** {@inheritDoc} */ - @Override protected Class loadClass(String name, boolean resolve) throws ClassNotFoundException { - try { - if (isHadoop(name)) { // Always load Hadoop classes explicitly, since Hadoop can be available in App classpath. - if (name.endsWith(".util.ShutdownHookManager")) // Dirty hack to get rid of Hadoop shutdown hooks. - return loadFromBytes(name, HadoopShutdownHookManager.class.getName()); - else if (name.equals(HADOOP_DAEMON_CLASS_NAME)) - // We replace this in order to be able to forcibly stop some daemon threads - // that otherwise never stop (e.g. PeerCache runnables): - return loadFromBytes(name, HadoopDaemon.class.getName()); - - return loadClassExplicitly(name, resolve); - } - - if (isHadoopIgfs(name)) { // For Ignite Hadoop and IGFS classes we have to check if they depend on Hadoop. - Boolean hasDeps = cache.get(name); - - if (hasDeps == null) { - hasDeps = hasExternalDependencies(name); - - cache.put(name, hasDeps); - } - - if (hasDeps) - return loadClassExplicitly(name, resolve); - } - - return super.loadClass(name, resolve); - } - catch (NoClassDefFoundError | ClassNotFoundException e) { - throw new ClassNotFoundException("Failed to load class: " + name, e); - } - } - - /** - * @param name Name. - * @param replace Replacement. - * @return Class. - */ - private Class loadFromBytes(final String name, final String replace) { - synchronized (getClassLoadingLock(name)) { - // First, check if the class has already been loaded - Class c = findLoadedClass(name); - - if (c != null) - return c; - - byte[] bytes = bytesCache.get(name); - - if (bytes == null) { - InputStream in = loadClassBytes(getParent(), replace); - - ClassReader rdr; - - try { - rdr = new ClassReader(in); - } - catch (IOException e) { - throw new RuntimeException(e); - } - - ClassWriter w = new ClassWriter(Opcodes.ASM4); - - rdr.accept(new RemappingClassAdapter(w, new Remapper() { - /** */ - String replaceType = replace.replace('.', '/'); - - /** */ - String nameType = name.replace('.', '/'); - - @Override public String map(String type) { - if (type.equals(replaceType)) - return nameType; - - return type; - } - }), ClassReader.EXPAND_FRAMES); - - bytes = w.toByteArray(); - - bytesCache.put(name, bytes); - } - - return defineClass(name, bytes, 0, bytes.length); - } - } - - /** {@inheritDoc} */ - @Override public Class getFromCache(String clsName) throws ClassNotFoundException { - Class cls = cacheMap.get(clsName); - - if (cls == null) { - Class old = cacheMap.putIfAbsent(clsName, cls = Class.forName(clsName, true, this)); - - if (old != null) - cls = old; - } - - return cls; - } - - /** - * @param name Class name. - * @param resolve Resolve class. - * @return Class. - * @throws ClassNotFoundException If failed. - */ - private Class loadClassExplicitly(String name, boolean resolve) throws ClassNotFoundException { - synchronized (getClassLoadingLock(name)) { - // First, check if the class has already been loaded - Class c = findLoadedClass(name); - - if (c == null) { - long t1 = System.nanoTime(); - - c = findClass(name); - - // this is the defining class loader; record the stats - sun.misc.PerfCounter.getFindClassTime().addElapsedTimeFrom(t1); - sun.misc.PerfCounter.getFindClasses().increment(); - } - - if (resolve) - resolveClass(c); - - return c; - } - } - - /** - * @param ldr Loader. - * @param clsName Class. - * @return Input stream. - */ - @Nullable private InputStream loadClassBytes(ClassLoader ldr, String clsName) { - return ldr.getResourceAsStream(clsName.replace('.', '/') + ".class"); - } - - /** - * Check whether class has external dependencies on Hadoop. - * - * @param clsName Class name. - * @return {@code True} if class has external dependencies. - */ - boolean hasExternalDependencies(String clsName) { - CollectingContext ctx = new CollectingContext(); - - ctx.annVisitor = new CollectingAnnotationVisitor(ctx); - ctx.mthdVisitor = new CollectingMethodVisitor(ctx, ctx.annVisitor); - ctx.fldVisitor = new CollectingFieldVisitor(ctx, ctx.annVisitor); - ctx.clsVisitor = new CollectingClassVisitor(ctx, ctx.annVisitor, ctx.mthdVisitor, ctx.fldVisitor); - - return hasExternalDependencies(clsName, ctx); - } - - /** - * Check whether class has external dependencies on Hadoop. - * - * @param clsName Class name. - * @param ctx Context. - * @return {@code true} If the class has external dependencies. - */ - boolean hasExternalDependencies(String clsName, CollectingContext ctx) { - if (isHadoop(clsName)) // Hadoop must not be in classpath but Idea sucks, so filtering explicitly as external. - return true; - - // Try to get from parent to check if the type accessible. - InputStream in = loadClassBytes(getParent(), clsName); - - if (in == null) // The class is external itself, it must be loaded from this class loader. - return true; - - if (!isHadoopIgfs(clsName)) // Other classes should not have external dependencies. - return false; - - final ClassReader rdr; - - try { - rdr = new ClassReader(in); - } - catch (IOException e) { - throw new RuntimeException("Failed to read class: " + clsName, e); - } - - ctx.visited.add(clsName); - - rdr.accept(ctx.clsVisitor, 0); - - if (ctx.found) // We already know that we have dependencies, no need to check parent. - return true; - - // Here we are known to not have any dependencies but possibly we have a parent which has them. - int idx = clsName.lastIndexOf('$'); - - if (idx == -1) // No parent class. - return false; - - String parentCls = clsName.substring(0, idx); - - if (ctx.visited.contains(parentCls)) - return false; - - Boolean res = cache.get(parentCls); - - if (res == null) - res = hasExternalDependencies(parentCls, ctx); - - return res; - } - - /** - * @param name Class name. - * @return {@code true} If this is a valid class name. - */ - private static boolean validateClassName(String name) { - int len = name.length(); - - if (len <= 1) - return false; - - if (!Character.isJavaIdentifierStart(name.charAt(0))) - return false; - - boolean hasDot = false; - - for (int i = 1; i < len; i++) { - char c = name.charAt(i); - - if (c == '.') - hasDot = true; - else if (!Character.isJavaIdentifierPart(c)) - return false; - } - - return hasDot; - } - - /** - * @param urls URLs. - * @return URLs. - */ - private static URL[] addHadoopUrls(URL[] urls) { - Collection hadoopJars; - - try { - hadoopJars = hadoopUrls(); - } - catch (IgniteCheckedException e) { - throw new RuntimeException(e); - } - - ArrayList list = new ArrayList<>(hadoopJars.size() + appJars.size() + (urls == null ? 0 : urls.length)); - - list.addAll(appJars); - list.addAll(hadoopJars); - - if (!F.isEmpty(urls)) - list.addAll(F.asList(urls)); - - return list.toArray(new URL[list.size()]); - } - - /** - * @return Collection of jar URLs. - * @throws IgniteCheckedException If failed. - */ - public static Collection hadoopUrls() throws IgniteCheckedException { - Collection hadoopUrls = hadoopJars; - - if (hadoopUrls != null) - return hadoopUrls; - - synchronized (HadoopClassLoader.class) { - hadoopUrls = hadoopJars; - - if (hadoopUrls != null) - return hadoopUrls; - - try { - hadoopUrls = HadoopClasspathUtils.classpathForClassLoader(); - } - catch (IOException e) { - throw new IgniteCheckedException("Failed to resolve Hadoop JAR locations: " + e.getMessage(), e); - } - - hadoopJars = hadoopUrls; - - return hadoopUrls; - } - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(HadoopClassLoader.class, this); - } - - /** - * Getter for name field. - */ - public String name() { - return name; - } - - /** - * Context for dependencies collection. - */ - private class CollectingContext { - /** Visited classes. */ - private final Set visited = new HashSet<>(); - - /** Whether dependency found. */ - private boolean found; - - /** Annotation visitor. */ - private AnnotationVisitor annVisitor; - - /** Method visitor. */ - private MethodVisitor mthdVisitor; - - /** Field visitor. */ - private FieldVisitor fldVisitor; - - /** Class visitor. */ - private ClassVisitor clsVisitor; - - /** - * Processes a method descriptor - * @param methDesc The method desc String. - */ - void onMethodsDesc(final String methDesc) { - // Process method return type: - onType(Type.getReturnType(methDesc)); - - if (found) - return; - - // Process method argument types: - for (Type t: Type.getArgumentTypes(methDesc)) { - onType(t); - - if (found) - return; - } - } - - /** - * Processes dependencies of a class. - * - * @param depCls The class name as dot-notated FQN. - */ - void onClass(final String depCls) { - assert depCls.indexOf('/') == -1 : depCls; // class name should be fully converted to dot notation. - assert depCls.charAt(0) != 'L' : depCls; - assert validateClassName(depCls) : depCls; - - if (depCls.startsWith("java.") || depCls.startsWith("javax.")) // Filter out platform classes. - return; - - if (visited.contains(depCls)) - return; - - Boolean res = cache.get(depCls); - - if (res == Boolean.TRUE || (res == null && hasExternalDependencies(depCls, this))) - found = true; - } - - /** - * Analyses dependencies of given type. - * - * @param t The type to process. - */ - void onType(Type t) { - if (t == null) - return; - - int sort = t.getSort(); - - switch (sort) { - case Type.ARRAY: - onType(t.getElementType()); - - break; - - case Type.OBJECT: - onClass(t.getClassName()); - - break; - } - } - - /** - * Analyses dependencies of given object type. - * - * @param objType The object type to process. - */ - void onInternalTypeName(String objType) { - if (objType == null) - return; - - assert objType.length() > 1 : objType; - - if (objType.charAt(0) == '[') - // handle array. In this case this is a type descriptor notation, like "[Ljava/lang/Object;" - onType(objType); - else { - assert objType.indexOf('.') == -1 : objType; // Must be slash-separated FQN. - - String clsName = objType.replace('/', '.'); // Convert it to dot notation. - - onClass(clsName); // Process. - } - } - - /** - * Type description analyser. - * - * @param desc The description. - */ - void onType(String desc) { - if (!F.isEmpty(desc)) { - if (desc.length() <= 1) - return; // Optimization: filter out primitive types in early stage. - - Type t = Type.getType(desc); - - onType(t); - } - } - } - - /** - * Annotation visitor. - */ - private static class CollectingAnnotationVisitor extends AnnotationVisitor { - /** */ - final CollectingContext ctx; - - /** - * Annotation visitor. - * - * @param ctx The collector. - */ - CollectingAnnotationVisitor(CollectingContext ctx) { - super(Opcodes.ASM4); - - this.ctx = ctx; - } - - /** {@inheritDoc} */ - @Override public AnnotationVisitor visitAnnotation(String name, String desc) { - if (ctx.found) - return null; - - ctx.onType(desc); - - return this; - } - - /** {@inheritDoc} */ - @Override public void visitEnum(String name, String desc, String val) { - if (ctx.found) - return; - - ctx.onType(desc); - } - - /** {@inheritDoc} */ - @Override public AnnotationVisitor visitArray(String name) { - return ctx.found ? null : this; - } - - /** {@inheritDoc} */ - @Override public void visit(String name, Object val) { - if (ctx.found) - return; - - if (val instanceof Type) - ctx.onType((Type)val); - } - - /** {@inheritDoc} */ - @Override public void visitEnd() { - // No-op. - } - } - - /** - * Field visitor. - */ - private static class CollectingFieldVisitor extends FieldVisitor { - /** Collector. */ - private final CollectingContext ctx; - - /** Annotation visitor. */ - private final AnnotationVisitor av; - - /** - * Constructor. - */ - CollectingFieldVisitor(CollectingContext ctx, AnnotationVisitor av) { - super(Opcodes.ASM4); - - this.ctx = ctx; - this.av = av; - } - - /** {@inheritDoc} */ - @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) { - if (ctx.found) - return null; - - ctx.onType(desc); - - return ctx.found ? null : av; - } - - /** {@inheritDoc} */ - @Override public void visitAttribute(Attribute attr) { - // No-op. - } - - /** {@inheritDoc} */ - @Override public void visitEnd() { - // No-op. - } - } - - /** - * Class visitor. - */ - private static class CollectingClassVisitor extends ClassVisitor { - /** Collector. */ - private final CollectingContext ctx; - - /** Annotation visitor. */ - private final AnnotationVisitor av; - - /** Method visitor. */ - private final MethodVisitor mv; - - /** Field visitor. */ - private final FieldVisitor fv; - - /** - * Constructor. - * - * @param ctx Collector. - * @param av Annotation visitor. - * @param mv Method visitor. - * @param fv Field visitor. - */ - CollectingClassVisitor(CollectingContext ctx, AnnotationVisitor av, MethodVisitor mv, FieldVisitor fv) { - super(Opcodes.ASM4); - - this.ctx = ctx; - this.av = av; - this.mv = mv; - this.fv = fv; - } - - /** {@inheritDoc} */ - @Override public void visit(int i, int i2, String name, String signature, String superName, String[] ifaces) { - if (ctx.found) - return; - - ctx.onInternalTypeName(superName); - - if (ctx.found) - return; - - if (ifaces != null) { - for (String iface : ifaces) { - ctx.onInternalTypeName(iface); - - if (ctx.found) - return; - } - } - } - - /** {@inheritDoc} */ - @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) { - if (ctx.found) - return null; - - ctx.onType(desc); - - return ctx.found ? null : av; - } - - /** {@inheritDoc} */ - @Override public void visitInnerClass(String name, String outerName, String innerName, int i) { - if (ctx.found) - return; - - ctx.onInternalTypeName(name); - } - - /** {@inheritDoc} */ - @Override public FieldVisitor visitField(int i, String name, String desc, String signature, Object val) { - if (ctx.found) - return null; - - ctx.onType(desc); - - return ctx.found ? null : fv; - } - - /** {@inheritDoc} */ - @Override public MethodVisitor visitMethod(int i, String name, String desc, String signature, - String[] exceptions) { - if (ctx.found) - return null; - - ctx.onMethodsDesc(desc); - - // Process declared method exceptions: - if (exceptions != null) { - for (String e : exceptions) - ctx.onInternalTypeName(e); - } - - return ctx.found ? null : mv; - } - } - - /** - * Method visitor. - */ - private static class CollectingMethodVisitor extends MethodVisitor { - /** Collector. */ - private final CollectingContext ctx; - - /** Annotation visitor. */ - private final AnnotationVisitor av; - - /** - * Constructor. - * - * @param ctx Collector. - * @param av Annotation visitor. - */ - private CollectingMethodVisitor(CollectingContext ctx, AnnotationVisitor av) { - super(Opcodes.ASM4); - - this.ctx = ctx; - this.av = av; - } - - /** {@inheritDoc} */ - @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) { - if (ctx.found) - return null; - - ctx.onType(desc); - - return ctx.found ? null : av; - } - - /** {@inheritDoc} */ - @Override public AnnotationVisitor visitParameterAnnotation(int i, String desc, boolean b) { - if (ctx.found) - return null; - - ctx.onType(desc); - - return ctx.found ? null : av; - } - - /** {@inheritDoc} */ - @Override public AnnotationVisitor visitAnnotationDefault() { - return ctx.found ? null : av; - } - - /** {@inheritDoc} */ - @Override public void visitFieldInsn(int opcode, String owner, String name, String desc) { - if (ctx.found) - return; - - ctx.onInternalTypeName(owner); - - if (ctx.found) - return; - - ctx.onType(desc); - } - - /** {@inheritDoc} */ - @Override public void visitInvokeDynamicInsn(String name, String desc, Handle bsm, Object... bsmArgs) { - // No-op. - } - - /** {@inheritDoc} */ - @Override public void visitFrame(int type, int nLoc, Object[] locTypes, int nStack, Object[] stackTypes) { - // No-op. - } - - /** {@inheritDoc} */ - @Override public void visitLocalVariable(String name, String desc, String signature, Label lb, - Label lb2, int i) { - if (ctx.found) - return; - - ctx.onType(desc); - } - - /** {@inheritDoc} */ - @Override public void visitMethodInsn(int i, String owner, String name, String desc) { - if (ctx.found) - return; - - ctx.onInternalTypeName(owner); - - if (ctx.found) - return; - - ctx.onMethodsDesc(desc); - } - - /** {@inheritDoc} */ - @Override public void visitMultiANewArrayInsn(String desc, int dim) { - if (ctx.found) - return; - - ctx.onType(desc); - } - - /** {@inheritDoc} */ - @Override public void visitTryCatchBlock(Label start, Label end, Label hndl, String typeStr) { - if (ctx.found) - return; - - ctx.onInternalTypeName(typeStr); - } - - /** {@inheritDoc} */ - @Override public void visitTypeInsn(int opcode, String type) { - if (ctx.found) - return; - - ctx.onInternalTypeName(type); - } - } -} \ No newline at end of file diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathMain.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathMain.java deleted file mode 100644 index 40694967a278a..0000000000000 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClasspathMain.java +++ /dev/null @@ -1,44 +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.hadoop; - -/** - * Main class to compose Hadoop classpath depending on the environment. - * This class is designed to be independent on any Ignite classes as possible. - * Please make sure to pass the path separator character as the 1st parameter to the main method. - */ -public class HadoopClasspathMain { - /** - * Main method to be executed from scripts. It prints the classpath to the standard output. - * - * @param args The 1st argument should be the path separator character (":" on Linux, ";" on Windows). - */ - public static void main(String[] args) throws Exception { - if (args.length < 1) - throw new IllegalArgumentException("Path separator must be passed as the first argument."); - - String separator = args[0]; - - StringBuilder sb = new StringBuilder(); - - for (String path : HadoopClasspathUtils.classpathForProcess()) - sb.append(path).append(separator); - - System.out.println(sb); - } -} diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCommonUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCommonUtils.java new file mode 100644 index 0000000000000..37af14737507f --- /dev/null +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCommonUtils.java @@ -0,0 +1,154 @@ +/* + * 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.hadoop; + +import org.jetbrains.annotations.Nullable; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.TreeSet; + +/** + * Common Hadoop utility methods which do not depend on Hadoop API. + */ +public class HadoopCommonUtils { + /** Job class name. */ + public static final String JOB_CLS_NAME = "org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopV2Job"; + + /** Property to store timestamp of new job id request. */ + public static final String REQ_NEW_JOBID_TS_PROPERTY = "ignite.job.requestNewIdTs"; + + /** Property to store timestamp of response of new job id request. */ + public static final String RESPONSE_NEW_JOBID_TS_PROPERTY = "ignite.job.responseNewIdTs"; + + /** Property to store timestamp of job submission. */ + public static final String JOB_SUBMISSION_START_TS_PROPERTY = "ignite.job.submissionStartTs"; + + /** Property to set custom writer of job statistics. */ + public static final String JOB_COUNTER_WRITER_PROPERTY = "ignite.counters.writer"; + + /** + * Sort input splits by length. + * + * @param splits Splits. + * @return Sorted splits. + */ + public static List sortInputSplits(Collection splits) { + int id = 0; + + TreeSet sortedSplits = new TreeSet<>(); + + for (HadoopInputSplit split : splits) { + long len = split instanceof HadoopFileBlock ? ((HadoopFileBlock)split).length() : 0; + + sortedSplits.add(new SplitSortWrapper(id++, split, len)); + } + + ArrayList res = new ArrayList<>(sortedSplits.size()); + + for (SplitSortWrapper sortedSplit : sortedSplits) + res.add(sortedSplit.split); + + return res; + } + + /** + * Set context class loader. + * + * @param newLdr New class loader. + * @return Old class loader. + */ + @Nullable public static ClassLoader setContextClassLoader(@Nullable ClassLoader newLdr) { + ClassLoader oldLdr = Thread.currentThread().getContextClassLoader(); + + if (newLdr != oldLdr) + Thread.currentThread().setContextClassLoader(newLdr); + + return oldLdr; + } + + /** + * Restore context class loader. + * + * @param oldLdr Original class loader. + */ + public static void restoreContextClassLoader(@Nullable ClassLoader oldLdr) { + ClassLoader newLdr = Thread.currentThread().getContextClassLoader(); + + if (newLdr != oldLdr) + Thread.currentThread().setContextClassLoader(oldLdr); + } + + /** + * Split wrapper for sorting. + */ + private static class SplitSortWrapper implements Comparable { + /** Unique ID. */ + private final int id; + + /** Split. */ + private final HadoopInputSplit split; + + /** Split length. */ + private final long len; + + /** + * Constructor. + * + * @param id Unique ID. + * @param split Split. + * @param len Split length. + */ + public SplitSortWrapper(int id, HadoopInputSplit split, long len) { + this.id = id; + this.split = split; + this.len = len; + } + + /** {@inheritDoc} */ + @SuppressWarnings("NullableProblems") + @Override public int compareTo(SplitSortWrapper other) { + long res = len - other.len; + + if (res > 0) + return -1; + else if (res < 0) + return 1; + else + return id - other.id; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return id; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object obj) { + return obj instanceof SplitSortWrapper && id == ((SplitSortWrapper)obj).id; + } + } + + /** + * Private constructor. + */ + private HadoopCommonUtils() { + // No-op. + } +} diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java index 42a3d726896ce..4326ad282ad17 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java @@ -24,7 +24,6 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.HadoopConfiguration; import org.apache.ignite.internal.GridKernalContext; -import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobMetadata; import org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker; import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffle; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopExternalSplit.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java similarity index 94% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopExternalSplit.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java index c7e8a0a23f9b2..bd767b368dba1 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopExternalSplit.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java @@ -15,13 +15,12 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop; import java.io.Externalizable; import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; -import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit; /** * Split serialized in external file. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelperImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelperImpl.java new file mode 100644 index 0000000000000..71bb8a4cafded --- /dev/null +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelperImpl.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ignite.internal.processors.hadoop; + +import org.apache.ignite.internal.GridKernalContext; +import org.jetbrains.annotations.Nullable; +import org.objectweb.asm.ClassReader; +import org.objectweb.asm.ClassWriter; +import org.objectweb.asm.Opcodes; +import org.objectweb.asm.commons.Remapper; +import org.objectweb.asm.commons.RemappingClassAdapter; + +import java.io.IOException; +import java.io.InputStream; + +/** + * Utility methods for Hadoop classloader required to avoid direct 3rd-party dependencies in class loader. + */ +public class HadoopHelperImpl implements HadoopHelper { + /** Kernal context. */ + private final GridKernalContext ctx; + + /** Common class loader. */ + private volatile HadoopClassLoader ldr; + + /** + * Default constructor. + */ + public HadoopHelperImpl() { + this(null); + } + + /** + * Constructor. + * + * @param ctx Kernal context. + */ + public HadoopHelperImpl(GridKernalContext ctx) { + this.ctx = ctx; + } + + /** {@inheritDoc} */ + @Override public boolean isNoOp() { + return false; + } + + /** {@inheritDoc} */ + @Override public HadoopClassLoader commonClassLoader() { + HadoopClassLoader res = ldr; + + if (res == null) { + synchronized (this) { + res = ldr; + + if (res == null) { + String[] libNames = null; + + if (ctx != null && ctx.config().getHadoopConfiguration() != null) + libNames = ctx.config().getHadoopConfiguration().getNativeLibraryNames(); + + res = new HadoopClassLoader(null, "hadoop-common", libNames, this); + + ldr = res; + } + } + } + + return res; + } + + /** {@inheritDoc} */ + @Override public byte[] loadReplace(InputStream in, final String originalName, final String replaceName) { + ClassReader rdr; + + try { + rdr = new ClassReader(in); + } + catch (IOException e) { + throw new RuntimeException(e); + } + + ClassWriter w = new ClassWriter(Opcodes.ASM4); + + rdr.accept(new RemappingClassAdapter(w, new Remapper() { + /** */ + String replaceType = replaceName.replace('.', '/'); + + /** */ + String nameType = originalName.replace('.', '/'); + + @Override public String map(String type) { + if (type.equals(replaceType)) + return nameType; + + return type; + } + }), ClassReader.EXPAND_FRAMES); + + return w.toByteArray(); + } + + /** {@inheritDoc} */ + @Override @Nullable public InputStream loadClassBytes(ClassLoader ldr, String clsName) { + return ldr.getResourceAsStream(clsName.replace('.', '/') + ".class"); + } +} diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java index b9c20c30a023f..f0df1e90c7f6b 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java @@ -40,6 +40,9 @@ * Hadoop processor. */ public class HadoopProcessor extends HadoopProcessorAdapter { + /** Class to probe for Hadoop libraries in Ignite classpath. */ + private static final String HADOOP_PROBE_CLS = "org.apache.hadoop.conf.Configuration"; + /** Job ID counter. */ private final AtomicInteger idCtr = new AtomicInteger(); @@ -164,7 +167,14 @@ public HadoopContext context() { /** {@inheritDoc} */ @Override public IgniteInternalFuture submit(HadoopJobId jobId, HadoopJobInfo jobInfo) { - return hctx.jobTracker().submit(jobId, jobInfo); + ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(getClass().getClassLoader()); + + try { + return hctx.jobTracker().submit(jobId, jobInfo); + } + finally { + HadoopCommonUtils.restoreContextClassLoader(oldLdr); + } } /** {@inheritDoc} */ @@ -203,6 +213,26 @@ public HadoopContext context() { throw new IgniteCheckedException(ioe.getMessage(), ioe); } + // Check if Hadoop is in parent class loader classpath. + try { + Class cls = Class.forName(HADOOP_PROBE_CLS, false, getClass().getClassLoader()); + + try { + String path = cls.getProtectionDomain().getCodeSource().getLocation().toString(); + + U.warn(log, "Hadoop libraries are found in Ignite classpath, this could lead to class loading " + + "errors (please remove all Hadoop libraries from Ignite classpath) [path=" + path + ']'); + } + catch (Throwable ignore) { + U.warn(log, "Hadoop libraries are found in Ignite classpath, this could lead to class loading " + + "errors (please remove all Hadoop libraries from Ignite classpath)"); + } + } + catch (Throwable ignore) { + // All is fine. + } + + // Try assembling Hadoop URLs. HadoopClassLoader.hadoopUrls(); } diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopSplitWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java similarity index 95% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopSplitWrapper.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java index df77adbdac81c..511aa5a2ad265 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopSplitWrapper.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop; import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; -import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit; + import org.apache.ignite.internal.util.typedef.internal.U; /** diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterAdapter.java index 3f682d37cf5e0..ee61a823a0c2d 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterAdapter.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterAdapter.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; + import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopPerformanceCounter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopPerformanceCounter.java index dedc6b372e149..9baedc2850e93 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopPerformanceCounter.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopPerformanceCounter.java @@ -24,6 +24,8 @@ import java.util.ArrayList; import java.util.Collection; import java.util.UUID; + +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; import org.apache.ignite.internal.processors.hadoop.HadoopTaskType; @@ -32,10 +34,6 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.JOB_SUBMISSION_START_TS_PROPERTY; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.REQ_NEW_JOBID_TS_PROPERTY; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.RESPONSE_NEW_JOBID_TS_PROPERTY; - /** * Counter for the job statistics accumulation. */ @@ -221,9 +219,9 @@ public void onJobStart(long ts) { public void clientSubmissionEvents(HadoopJobInfo info) { assert nodeId != null; - addEventFromProperty("JOB requestId", info, REQ_NEW_JOBID_TS_PROPERTY); - addEventFromProperty("JOB responseId", info, RESPONSE_NEW_JOBID_TS_PROPERTY); - addEventFromProperty("JOB submit", info, JOB_SUBMISSION_START_TS_PROPERTY); + addEventFromProperty("JOB requestId", info, HadoopCommonUtils.REQ_NEW_JOBID_TS_PROPERTY); + addEventFromProperty("JOB responseId", info, HadoopCommonUtils.RESPONSE_NEW_JOBID_TS_PROPERTY); + addEventFromProperty("JOB submit", info, HadoopCommonUtils.JOB_SUBMISSION_START_TS_PROPERTY); } /** diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java new file mode 100644 index 0000000000000..76d9bff5a712d --- /dev/null +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java @@ -0,0 +1,138 @@ +/* + * 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.hadoop.delegate; + +import org.apache.ignite.IgniteException; +import org.apache.ignite.hadoop.fs.BasicHadoopFileSystemFactory; +import org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory; +import org.apache.ignite.hadoop.fs.IgniteHadoopFileSystemCounterWriter; +import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem; +import org.apache.ignite.hadoop.fs.KerberosHadoopFileSystemFactory; +import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader; +import org.jetbrains.annotations.Nullable; + +import java.lang.reflect.Constructor; +import java.util.HashMap; +import java.util.Map; + +/** + * Utility methods for Hadoop delegates. + */ +public class HadoopDelegateUtils { + /** Secondary file system delegate class. */ + private static final String SECONDARY_FILE_SYSTEM_CLS = + "org.apache.ignite.internal.processors.hadoop.impl.delegate.HadoopIgfsSecondaryFileSystemDelegateImpl"; + + /** Default file system factory class. */ + private static final String DFLT_FACTORY_CLS = + "org.apache.ignite.internal.processors.hadoop.impl.delegate.HadoopDefaultFileSystemFactoryDelegate"; + + /** Factory proxy to delegate class name mapping. */ + private static final Map FACTORY_CLS_MAP; + + /** Counter writer delegate implementation. */ + private static final String COUNTER_WRITER_DELEGATE_CLS = + "org.apache.ignite.internal.processors.hadoop.impl.delegate.HadoopFileSystemCounterWriterDelegateImpl"; + + static { + FACTORY_CLS_MAP = new HashMap<>(); + + FACTORY_CLS_MAP.put(BasicHadoopFileSystemFactory.class.getName(), + "org.apache.ignite.internal.processors.hadoop.impl.delegate.HadoopBasicFileSystemFactoryDelegate"); + + FACTORY_CLS_MAP.put(CachingHadoopFileSystemFactory.class.getName(), + "org.apache.ignite.internal.processors.hadoop.impl.delegate.HadoopCachingFileSystemFactoryDelegate"); + + FACTORY_CLS_MAP.put(KerberosHadoopFileSystemFactory.class.getName(), + "org.apache.ignite.internal.processors.hadoop.impl.delegate.HadoopKerberosFileSystemFactoryDelegate"); + } + + /** + * Create delegate for secondary file system. + * + * @param ldr Hadoop class loader. + * @param proxy Proxy. + * @return Delegate. + */ + public static HadoopIgfsSecondaryFileSystemDelegate secondaryFileSystemDelegate(HadoopClassLoader ldr, + IgniteHadoopIgfsSecondaryFileSystem proxy) { + return newInstance(SECONDARY_FILE_SYSTEM_CLS, ldr, proxy); + } + + /** + * Create delegate for certain file system factory. + * + * @param proxy Proxy. + * @return Delegate. + */ + @SuppressWarnings("unchecked") + public static HadoopFileSystemFactoryDelegate fileSystemFactoryDelegate(Object proxy) { + String clsName = FACTORY_CLS_MAP.get(proxy.getClass().getName()); + + if (clsName == null) + clsName = DFLT_FACTORY_CLS; + + return newInstance(clsName, null, proxy); + } + + /** + * Create delegate for Hadoop counter writer. + * + * @param ldr Class loader. + * @param proxy Proxy. + * @return Delegate. + */ + public static HadoopFileSystemCounterWriterDelegate counterWriterDelegate(ClassLoader ldr, + IgniteHadoopFileSystemCounterWriter proxy) { + return newInstance(COUNTER_WRITER_DELEGATE_CLS, ldr, proxy); + } + + /** + * Get new delegate instance. + * + * @param clsName Class name. + * @param ldr Optional class loader. + * @param proxy Proxy. + * @return Instance. + */ + @SuppressWarnings("unchecked") + private static T newInstance(String clsName, @Nullable ClassLoader ldr, Object proxy) { + try { + Class delegateCls = ldr == null ? Class.forName(clsName) : Class.forName(clsName, true, ldr); + + Constructor[] ctors = delegateCls.getConstructors(); + + assert ctors.length == 1; + + Object res = ctors[0].newInstance(proxy); + + return (T)res; + } + catch (ReflectiveOperationException e) { + throw new IgniteException("Failed to instantiate delegate for proxy [proxy=" + proxy + + ", delegateClsName=" + clsName + ']', e); + } + } + + /** + * Private constructor. + */ + private HadoopDelegateUtils() { + // No-op. + } +} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithClassAnnotation.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopFileSystemCounterWriterDelegate.java similarity index 56% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithClassAnnotation.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopFileSystemCounterWriterDelegate.java index a9ecae0bf10e3..541cf808e3efb 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithClassAnnotation.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopFileSystemCounterWriterDelegate.java @@ -15,14 +15,22 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.deps; +package org.apache.ignite.internal.processors.hadoop.delegate; -import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.hadoop.HadoopJob; +import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters; /** - * Class has Hadoop annotation. + * Counter writer delegate interface. */ -@SuppressWarnings("unused") -@InterfaceAudience.Public -public class WithClassAnnotation { +public interface HadoopFileSystemCounterWriterDelegate { + /** + * Writes counters of given job to some statistics storage. + * + * @param job The job. + * @param cntrs Counters. + * @throws IgniteCheckedException If failed. + */ + public void write(HadoopJob job, HadoopCounters cntrs) throws IgniteCheckedException; } diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithImplements.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopFileSystemFactoryDelegate.java similarity index 62% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithImplements.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopFileSystemFactoryDelegate.java index c2d8e5bd039aa..f051d620c3ef7 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithImplements.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopFileSystemFactoryDelegate.java @@ -15,22 +15,22 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.deps; +package org.apache.ignite.internal.processors.hadoop.delegate; -import org.apache.hadoop.conf.Configurable; -import org.apache.hadoop.conf.Configuration; +import org.apache.ignite.lifecycle.LifecycleAware; + +import java.io.IOException; /** - * Implements a Hadoop interface. + * Hadoop file system factory delegate. */ -public class WithImplements implements Configurable { - /** {@inheritDoc} */ - @Override public void setConf(Configuration conf) { - // noop - } - - /** {@inheritDoc} */ - @Override public Configuration getConf() { - return null; - } +public interface HadoopFileSystemFactoryDelegate extends LifecycleAware { + /** + * Gets file system for the given user name. + * + * @param usrName User name + * @return File system. + * @throws IOException In case of error. + */ + public Object get(String usrName) throws IOException; } diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/Without.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopIgfsSecondaryFileSystemDelegate.java similarity index 70% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/Without.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopIgfsSecondaryFileSystemDelegate.java index ab8474026df39..e381272203cfb 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/Without.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopIgfsSecondaryFileSystemDelegate.java @@ -15,11 +15,14 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.deps; +package org.apache.ignite.internal.processors.hadoop.delegate; + +import org.apache.ignite.internal.processors.igfs.IgfsSecondaryFileSystemV2; +import org.apache.ignite.lifecycle.LifecycleAware; /** - * Class that does not anyhow depend on Hadoop. + * Interface to secondary file system implementation. */ -public class Without { +public interface HadoopIgfsSecondaryFileSystemDelegate extends IgfsSecondaryFileSystemV2, LifecycleAware { // No-op. } diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceCounterGroup.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceCounterGroup.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceCounterGroup.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceCounterGroup.java index 4e03e172ec63d..0ab64d96e1344 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceCounterGroup.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceCounterGroup.java @@ -15,15 +15,16 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; + +import org.apache.hadoop.mapreduce.Counter; +import org.apache.hadoop.mapreduce.CounterGroup; +import org.apache.hadoop.mapreduce.counters.CounterGroupBase; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.Iterator; -import org.apache.hadoop.mapreduce.Counter; -import org.apache.hadoop.mapreduce.CounterGroup; -import org.apache.hadoop.mapreduce.counters.CounterGroupBase; /** * Hadoop +counter group adapter. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceCounters.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceCounters.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceCounters.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceCounters.java index 57a853f884483..df5c1ee53d263 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceCounters.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceCounters.java @@ -15,7 +15,18 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; + +import org.apache.hadoop.mapreduce.Counter; +import org.apache.hadoop.mapreduce.CounterGroup; +import org.apache.hadoop.mapreduce.Counters; +import org.apache.hadoop.mapreduce.FileSystemCounter; +import org.apache.hadoop.mapreduce.counters.AbstractCounters; +import org.apache.hadoop.mapreduce.counters.Limits; +import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounter; +import org.apache.ignite.internal.processors.hadoop.counter.HadoopLongCounter; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopV2Counter; +import org.apache.ignite.internal.util.typedef.T2; import java.io.DataInput; import java.io.DataOutput; @@ -27,16 +38,6 @@ import java.util.Iterator; import java.util.Map; import java.util.NoSuchElementException; -import org.apache.hadoop.mapreduce.Counter; -import org.apache.hadoop.mapreduce.CounterGroup; -import org.apache.hadoop.mapreduce.Counters; -import org.apache.hadoop.mapreduce.FileSystemCounter; -import org.apache.hadoop.mapreduce.counters.AbstractCounters; -import org.apache.hadoop.mapreduce.counters.Limits; -import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounter; -import org.apache.ignite.internal.processors.hadoop.counter.HadoopLongCounter; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopV2Counter; -import org.apache.ignite.internal.util.typedef.T2; /** * Hadoop counters adapter. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUtils.java similarity index 73% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUtils.java index 65d9810ee909a..347bfae09451c 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUtils.java @@ -15,7 +15,24 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.JobPriority; +import org.apache.hadoop.mapreduce.JobStatus; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; +import org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; +import org.apache.ignite.internal.processors.hadoop.HadoopJobStatus; +import org.apache.ignite.internal.processors.hadoop.HadoopSplitWrapper; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; +import org.apache.ignite.internal.util.typedef.internal.U; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -25,42 +42,14 @@ import java.io.ObjectOutput; import java.io.ObjectOutputStream; import java.io.PrintStream; -import java.util.ArrayList; -import java.util.Collection; import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.TreeSet; import java.util.UUID; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapreduce.JobID; -import org.apache.hadoop.mapreduce.JobPriority; -import org.apache.hadoop.mapreduce.JobStatus; -import org.apache.hadoop.mapreduce.MRJobConfig; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopSplitWrapper; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.jetbrains.annotations.Nullable; /** * Hadoop utility methods. */ public class HadoopUtils { - /** Property to store timestamp of new job id request. */ - public static final String REQ_NEW_JOBID_TS_PROPERTY = "ignite.job.requestNewIdTs"; - - /** Property to store timestamp of response of new job id request. */ - public static final String RESPONSE_NEW_JOBID_TS_PROPERTY = "ignite.job.responseNewIdTs"; - - /** Property to store timestamp of job submission. */ - public static final String JOB_SUBMISSION_START_TS_PROPERTY = "ignite.job.submissionStartTs"; - - /** Property to set custom writer of job statistics. */ - public static final String JOB_COUNTER_WRITER_PROPERTY = "ignite.counters.writer"; - /** Staging constant. */ private static final String STAGING_CONSTANT = ".staging"; @@ -327,117 +316,13 @@ public static File taskLocalDir(UUID locNodeId, HadoopTaskInfo info) throws Igni * @return New instance of {@link Configuration}. */ public static Configuration safeCreateConfiguration() { - final ClassLoader oldLdr = setContextClassLoader(Configuration.class.getClassLoader()); + final ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(Configuration.class.getClassLoader()); try { return new Configuration(); } finally { - restoreContextClassLoader(oldLdr); - } - } - - /** - * Sort input splits by length. - * - * @param splits Splits. - * @return Sorted splits. - */ - public static List sortInputSplits(Collection splits) { - int id = 0; - - TreeSet sortedSplits = new TreeSet<>(); - - for (HadoopInputSplit split : splits) { - long len = split instanceof HadoopFileBlock ? ((HadoopFileBlock)split).length() : 0; - - sortedSplits.add(new SplitSortWrapper(id++, split, len)); - } - - ArrayList res = new ArrayList<>(sortedSplits.size()); - - for (SplitSortWrapper sortedSplit : sortedSplits) - res.add(sortedSplit.split); - - return res; - } - - /** - * Set context class loader. - * - * @param newLdr New class loader. - * @return Old class loader. - */ - @Nullable public static ClassLoader setContextClassLoader(@Nullable ClassLoader newLdr) { - ClassLoader oldLdr = Thread.currentThread().getContextClassLoader(); - - if (newLdr != oldLdr) - Thread.currentThread().setContextClassLoader(newLdr); - - return oldLdr; - } - - /** - * Restore context class loader. - * - * @param oldLdr Original class loader. - */ - public static void restoreContextClassLoader(@Nullable ClassLoader oldLdr) { - ClassLoader newLdr = Thread.currentThread().getContextClassLoader(); - - if (newLdr != oldLdr) - Thread.currentThread().setContextClassLoader(oldLdr); - } - - /** - * Split wrapper for sorting. - */ - private static class SplitSortWrapper implements Comparable { - /** Unique ID. */ - private final int id; - - /** Split. */ - private final HadoopInputSplit split; - - /** Split length. */ - private final long len; - - /** - * Constructor. - * - * @param id Unique ID. - * @param split Split. - * @param len Split length. - */ - public SplitSortWrapper(int id, HadoopInputSplit split, long len) { - this.id = id; - this.split = split; - this.len = len; - } - - /** {@inheritDoc} */ - @SuppressWarnings("NullableProblems") - @Override public int compareTo(SplitSortWrapper other) { - assert other != null; - - long res = len - other.len; - - if (res > 0) - return -1; - else if (res < 0) - return 1; - else - return id - other.id; - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - return id; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object obj) { - return obj instanceof SplitSortWrapper && id == ((SplitSortWrapper)obj).id; + HadoopCommonUtils.restoreContextClassLoader(oldLdr); } } } \ No newline at end of file diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopBasicFileSystemFactoryDelegate.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopBasicFileSystemFactoryDelegate.java new file mode 100644 index 0000000000000..a190b14699ba4 --- /dev/null +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopBasicFileSystemFactoryDelegate.java @@ -0,0 +1,164 @@ +/* + * 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.hadoop.impl.delegate; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.ignite.IgniteException; +import org.apache.ignite.hadoop.fs.BasicHadoopFileSystemFactory; +import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory; +import org.apache.ignite.hadoop.util.UserNameMapper; +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils; +import org.apache.ignite.internal.processors.igfs.IgfsUtils; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lifecycle.LifecycleAware; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.util.Arrays; + +/** + * Basic Hadoop file system factory delegate. + */ +public class HadoopBasicFileSystemFactoryDelegate implements HadoopFileSystemFactoryDelegate { + /** Proxy. */ + protected final HadoopFileSystemFactory proxy; + + /** Configuration of the secondary filesystem, never null. */ + protected Configuration cfg; + + /** Resulting URI. */ + protected URI fullUri; + + /** User name mapper. */ + private UserNameMapper usrNameMapper; + + /** + * Constructor. + * + * @param proxy Proxy. + */ + public HadoopBasicFileSystemFactoryDelegate(BasicHadoopFileSystemFactory proxy) { + this.proxy = proxy; + } + + /** {@inheritDoc} */ + @Override public FileSystem get(String name) throws IOException { + String name0 = IgfsUtils.fixUserName(name); + + if (usrNameMapper != null) + name0 = IgfsUtils.fixUserName(usrNameMapper.map(name0)); + + return getWithMappedName(name0); + } + + /** + * Internal file system create routine. + * + * @param usrName User name. + * @return File system. + * @throws IOException If failed. + */ + protected FileSystem getWithMappedName(String usrName) throws IOException { + assert cfg != null; + + try { + // FileSystem.get() might delegate to ServiceLoader to get the list of file system implementation. + // And ServiceLoader is known to be sensitive to context classloader. Therefore, we change context + // classloader to classloader of current class to avoid strange class-cast-exceptions. + ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(getClass().getClassLoader()); + + try { + return create(usrName); + } + finally { + HadoopCommonUtils.restoreContextClassLoader(oldLdr); + } + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + + throw new IOException("Failed to create file system due to interrupt.", e); + } + } + + /** + * Internal file system creation routine, invoked in correct class loader context. + * + * @param usrName User name. + * @return File system. + * @throws IOException If failed. + * @throws InterruptedException if the current thread is interrupted. + */ + protected FileSystem create(String usrName) throws IOException, InterruptedException { + return FileSystem.get(fullUri, cfg, usrName); + } + + /** {@inheritDoc} */ + @Override public void start() throws IgniteException { + BasicHadoopFileSystemFactory proxy0 = (BasicHadoopFileSystemFactory)proxy; + + cfg = HadoopUtils.safeCreateConfiguration(); + + if (proxy0.getConfigPaths() != null) { + for (String cfgPath : proxy0.getConfigPaths()) { + if (cfgPath == null) + throw new NullPointerException("Configuration path cannot be null: " + + Arrays.toString(proxy0.getConfigPaths())); + else { + URL url = U.resolveIgniteUrl(cfgPath); + + if (url == null) { + // If secConfPath is given, it should be resolvable: + throw new IgniteException("Failed to resolve secondary file system configuration path " + + "(ensure that it exists locally and you have read access to it): " + cfgPath); + } + + cfg.addResource(url); + } + } + } + + // If secondary fs URI is not given explicitly, try to get it from the configuration: + if (proxy0.getUri() == null) + fullUri = FileSystem.getDefaultUri(cfg); + else { + try { + fullUri = new URI(proxy0.getUri()); + } + catch (URISyntaxException use) { + throw new IgniteException("Failed to resolve secondary file system URI: " + proxy0.getUri()); + } + } + + usrNameMapper = proxy0.getUserNameMapper(); + + if (usrNameMapper != null && usrNameMapper instanceof LifecycleAware) + ((LifecycleAware)usrNameMapper).start(); + } + + /** {@inheritDoc} */ + @Override public void stop() throws IgniteException { + if (usrNameMapper != null && usrNameMapper instanceof LifecycleAware) + ((LifecycleAware)usrNameMapper).stop(); + } +} diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopCachingFileSystemFactoryDelegate.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopCachingFileSystemFactoryDelegate.java new file mode 100644 index 0000000000000..0cec8cac78eec --- /dev/null +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopCachingFileSystemFactoryDelegate.java @@ -0,0 +1,75 @@ +/* + * 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.hadoop.impl.delegate; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory; +import org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSystemsUtils; +import org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopLazyConcurrentMap; + +import java.io.IOException; + +/** + * Caching Hadoop file system factory delegate. + */ +public class HadoopCachingFileSystemFactoryDelegate extends HadoopBasicFileSystemFactoryDelegate { + /** Per-user file system cache. */ + private final HadoopLazyConcurrentMap cache = new HadoopLazyConcurrentMap<>( + new HadoopLazyConcurrentMap.ValueFactory() { + @Override public FileSystem createValue(String key) throws IOException { + return HadoopCachingFileSystemFactoryDelegate.super.getWithMappedName(key); + } + } + ); + + /** + * Constructor. + * + * @param proxy Proxy. + */ + public HadoopCachingFileSystemFactoryDelegate(CachingHadoopFileSystemFactory proxy) { + super(proxy); + } + + /** {@inheritDoc} */ + @Override public FileSystem getWithMappedName(String name) throws IOException { + return cache.getOrCreate(name); + } + + /** {@inheritDoc} */ + @Override public void start() throws IgniteException { + super.start(); + + // Disable caching. + cfg.setBoolean(HadoopFileSystemsUtils.disableFsCachePropertyName(fullUri.getScheme()), true); + } + + /** {@inheritDoc} */ + @Override public void stop() throws IgniteException { + super.stop(); + + try { + cache.close(); + } + catch (IgniteCheckedException ice) { + throw new IgniteException(ice); + } + } +} diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopDefaultFileSystemFactoryDelegate.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopDefaultFileSystemFactoryDelegate.java new file mode 100644 index 0000000000000..20ac88e6cde44 --- /dev/null +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopDefaultFileSystemFactoryDelegate.java @@ -0,0 +1,62 @@ +/* + * 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.hadoop.impl.delegate; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.ignite.IgniteException; +import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate; +import org.apache.ignite.lifecycle.LifecycleAware; + +import java.io.IOException; + +/** + * Hadoop file system factory delegate for non-standard factories. + */ +public class HadoopDefaultFileSystemFactoryDelegate implements HadoopFileSystemFactoryDelegate { + /** Factory. */ + private final HadoopFileSystemFactory factory; + + /** + * Constructor. + * + * @param factory Factory. + */ + public HadoopDefaultFileSystemFactoryDelegate(HadoopFileSystemFactory factory) { + assert factory != null; + + this.factory = factory; + } + + /** {@inheritDoc} */ + @Override public FileSystem get(String usrName) throws IOException { + return (FileSystem)factory.get(usrName); + } + + /** {@inheritDoc} */ + @Override public void start() throws IgniteException { + if (factory instanceof LifecycleAware) + ((LifecycleAware)factory).start(); + } + + /** {@inheritDoc} */ + @Override public void stop() throws IgniteException { + if (factory instanceof LifecycleAware) + ((LifecycleAware)factory).stop(); + } +} diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopFileSystemCounterWriterDelegateImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopFileSystemCounterWriterDelegateImpl.java new file mode 100644 index 0000000000000..d4c10dad4af53 --- /dev/null +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopFileSystemCounterWriterDelegateImpl.java @@ -0,0 +1,108 @@ +/* + * 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.hadoop.impl.delegate; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.hadoop.fs.IgniteHadoopFileSystemCounterWriter; +import org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopJob; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; +import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo; +import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters; +import org.apache.ignite.internal.processors.hadoop.counter.HadoopPerformanceCounter; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemCounterWriterDelegate; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopV2Job; +import org.apache.ignite.internal.processors.igfs.IgfsUtils; +import org.apache.ignite.internal.util.typedef.T2; + +import java.io.IOException; +import java.io.PrintStream; +import java.util.Map; + +/** + * Counter writer delegate implementation. + */ +@SuppressWarnings("unused") +public class HadoopFileSystemCounterWriterDelegateImpl implements HadoopFileSystemCounterWriterDelegate { + /** */ + private static final String USER_MACRO = "${USER}"; + + /** */ + private static final String DEFAULT_COUNTER_WRITER_DIR = "/user/" + USER_MACRO; + + /** + * Constructor. + * + * @param proxy Proxy (not used). + */ + public HadoopFileSystemCounterWriterDelegateImpl(IgniteHadoopFileSystemCounterWriter proxy) { + // No-op. + } + + /** {@inheritDoc} */ + public void write(HadoopJob job, HadoopCounters cntrs) throws IgniteCheckedException { + Configuration hadoopCfg = HadoopUtils.safeCreateConfiguration(); + + final HadoopJobInfo jobInfo = job.info(); + + final HadoopJobId jobId = job.id(); + + for (Map.Entry e : ((HadoopDefaultJobInfo)jobInfo).properties().entrySet()) + hadoopCfg.set(e.getKey(), e.getValue()); + + String user = jobInfo.user(); + + user = IgfsUtils.fixUserName(user); + + String dir = jobInfo.property(IgniteHadoopFileSystemCounterWriter.COUNTER_WRITER_DIR_PROPERTY); + + if (dir == null) + dir = DEFAULT_COUNTER_WRITER_DIR; + + Path jobStatPath = new Path(new Path(dir.replace(USER_MACRO, user)), jobId.toString()); + + HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(cntrs, null); + + try { + hadoopCfg.set(MRJobConfig.USER_NAME, user); + + FileSystem fs = ((HadoopV2Job)job).fileSystem(jobStatPath.toUri(), hadoopCfg); + + fs.mkdirs(jobStatPath); + + try (PrintStream out = new PrintStream(fs.create( + new Path(jobStatPath, IgniteHadoopFileSystemCounterWriter.PERFORMANCE_COUNTER_FILE_NAME)))) { + for (T2 evt : perfCntr.evts()) { + out.print(evt.get1()); + out.print(':'); + out.println(evt.get2().toString()); + } + + out.flush(); + } + } + catch (IOException e) { + throw new IgniteCheckedException(e); + } + } +} diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java new file mode 100644 index 0000000000000..fcad674deb116 --- /dev/null +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java @@ -0,0 +1,472 @@ +/* + * 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.hadoop.impl.delegate; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.ParentNotDirectoryException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathExistsException; +import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.ignite.IgniteException; +import org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory; +import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory; +import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem; +import org.apache.ignite.igfs.IgfsDirectoryNotEmptyException; +import org.apache.ignite.igfs.IgfsException; +import org.apache.ignite.igfs.IgfsFile; +import org.apache.ignite.igfs.IgfsParentNotDirectoryException; +import org.apache.ignite.igfs.IgfsPath; +import org.apache.ignite.igfs.IgfsPathAlreadyExistsException; +import org.apache.ignite.igfs.IgfsPathNotFoundException; +import org.apache.ignite.igfs.IgfsUserContext; +import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopIgfsSecondaryFileSystemDelegate; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsProperties; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsSecondaryFileSystemPositionedReadable; +import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo; +import org.apache.ignite.internal.processors.igfs.IgfsFileImpl; +import org.apache.ignite.internal.processors.igfs.IgfsUtils; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.OutputStream; +import java.net.URI; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** + * Secondary file system implementation. + */ +@SuppressWarnings("unused") +public class HadoopIgfsSecondaryFileSystemDelegateImpl implements HadoopIgfsSecondaryFileSystemDelegate { + /** The default user name. It is used if no user context is set. */ + private final String dfltUsrName; + + /** Factory. */ + private final HadoopFileSystemFactoryDelegate factory; + + /** + * Constructor. + * + * @param proxy Proxy. + */ + public HadoopIgfsSecondaryFileSystemDelegateImpl(IgniteHadoopIgfsSecondaryFileSystem proxy) { + assert proxy.getFileSystemFactory() != null; + + dfltUsrName = IgfsUtils.fixUserName(proxy.getDefaultUserName()); + + HadoopFileSystemFactory factory0 = proxy.getFileSystemFactory(); + + if (factory0 == null) + factory0 = new CachingHadoopFileSystemFactory(); + + factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory0); + } + + /** {@inheritDoc} */ + @Override public boolean exists(IgfsPath path) { + try { + return fileSystemForUser().exists(convert(path)); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to check file existence [path=" + path + "]"); + } + } + + /** {@inheritDoc} */ + @Nullable @Override public IgfsFile update(IgfsPath path, Map props) { + HadoopIgfsProperties props0 = new HadoopIgfsProperties(props); + + final FileSystem fileSys = fileSystemForUser(); + + try { + if (props0.userName() != null || props0.groupName() != null) + fileSys.setOwner(convert(path), props0.userName(), props0.groupName()); + + if (props0.permission() != null) + fileSys.setPermission(convert(path), props0.permission()); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to update file properties [path=" + path + "]"); + } + + //Result is not used in case of secondary FS. + return null; + } + + /** {@inheritDoc} */ + @Override public void rename(IgfsPath src, IgfsPath dest) { + // Delegate to the secondary file system. + try { + if (!fileSystemForUser().rename(convert(src), convert(dest))) + throw new IgfsException("Failed to rename (secondary file system returned false) " + + "[src=" + src + ", dest=" + dest + ']'); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to rename file [src=" + src + ", dest=" + dest + ']'); + } + } + + /** {@inheritDoc} */ + @Override public boolean delete(IgfsPath path, boolean recursive) { + try { + return fileSystemForUser().delete(convert(path), recursive); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to delete file [path=" + path + ", recursive=" + recursive + "]"); + } + } + + /** {@inheritDoc} */ + @Override public void mkdirs(IgfsPath path) { + try { + if (!fileSystemForUser().mkdirs(convert(path))) + throw new IgniteException("Failed to make directories [path=" + path + "]"); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to make directories [path=" + path + "]"); + } + } + + /** {@inheritDoc} */ + @Override public void mkdirs(IgfsPath path, @Nullable Map props) { + try { + if (!fileSystemForUser().mkdirs(convert(path), new HadoopIgfsProperties(props).permission())) + throw new IgniteException("Failed to make directories [path=" + path + ", props=" + props + "]"); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to make directories [path=" + path + ", props=" + props + "]"); + } + } + + /** {@inheritDoc} */ + @Override public Collection listPaths(IgfsPath path) { + try { + FileStatus[] statuses = fileSystemForUser().listStatus(convert(path)); + + if (statuses == null) + throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path); + + Collection res = new ArrayList<>(statuses.length); + + for (FileStatus status : statuses) + res.add(new IgfsPath(path, status.getPath().getName())); + + return res; + } + catch (FileNotFoundException ignored) { + throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to list statuses due to secondary file system exception: " + path); + } + } + + /** {@inheritDoc} */ + @Override public Collection listFiles(IgfsPath path) { + try { + FileStatus[] statuses = fileSystemForUser().listStatus(convert(path)); + + if (statuses == null) + throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path); + + Collection res = new ArrayList<>(statuses.length); + + for (FileStatus s : statuses) { + IgfsEntryInfo fsInfo = s.isDirectory() ? + IgfsUtils.createDirectory( + IgniteUuid.randomUuid(), + null, + properties(s), + s.getAccessTime(), + s.getModificationTime() + ) : + IgfsUtils.createFile( + IgniteUuid.randomUuid(), + (int)s.getBlockSize(), + s.getLen(), + null, + null, + false, + properties(s), + s.getAccessTime(), + s.getModificationTime() + ); + + res.add(new IgfsFileImpl(new IgfsPath(path, s.getPath().getName()), fsInfo, 1)); + } + + return res; + } + catch (FileNotFoundException ignored) { + throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to list statuses due to secondary file system exception: " + path); + } + } + + /** {@inheritDoc} */ + @Override public IgfsSecondaryFileSystemPositionedReadable open(IgfsPath path, int bufSize) { + return new HadoopIgfsSecondaryFileSystemPositionedReadable(fileSystemForUser(), convert(path), bufSize); + } + + /** {@inheritDoc} */ + @Override public OutputStream create(IgfsPath path, boolean overwrite) { + try { + return fileSystemForUser().create(convert(path), overwrite); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", overwrite=" + overwrite + "]"); + } + } + + /** {@inheritDoc} */ + @Override public OutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication, + long blockSize, @Nullable Map props) { + HadoopIgfsProperties props0 = + new HadoopIgfsProperties(props != null ? props : Collections.emptyMap()); + + try { + return fileSystemForUser().create(convert(path), props0.permission(), overwrite, bufSize, + (short) replication, blockSize, null); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", props=" + props + + ", overwrite=" + overwrite + ", bufSize=" + bufSize + ", replication=" + replication + + ", blockSize=" + blockSize + "]"); + } + } + + /** {@inheritDoc} */ + @Override public OutputStream append(IgfsPath path, int bufSize, boolean create, + @Nullable Map props) { + try { + return fileSystemForUser().append(convert(path), bufSize); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to append file [path=" + path + ", bufSize=" + bufSize + "]"); + } + } + + /** {@inheritDoc} */ + @Override public IgfsFile info(final IgfsPath path) { + try { + final FileStatus status = fileSystemForUser().getFileStatus(convert(path)); + + if (status == null) + return null; + + final Map props = properties(status); + + return new IgfsFile() { + @Override public IgfsPath path() { + return path; + } + + @Override public boolean isFile() { + return status.isFile(); + } + + @Override public boolean isDirectory() { + return status.isDirectory(); + } + + @Override public int blockSize() { + // By convention directory has blockSize == 0, while file has blockSize > 0: + return isDirectory() ? 0 : (int)status.getBlockSize(); + } + + @Override public long groupBlockSize() { + return status.getBlockSize(); + } + + @Override public long accessTime() { + return status.getAccessTime(); + } + + @Override public long modificationTime() { + return status.getModificationTime(); + } + + @Override public String property(String name) throws IllegalArgumentException { + String val = props.get(name); + + if (val == null) + throw new IllegalArgumentException("File property not found [path=" + path + ", name=" + name + ']'); + + return val; + } + + @Nullable @Override public String property(String name, @Nullable String dfltVal) { + String val = props.get(name); + + return val == null ? dfltVal : val; + } + + @Override public long length() { + return status.getLen(); + } + + /** {@inheritDoc} */ + @Override public Map properties() { + return props; + } + }; + } + catch (FileNotFoundException ignore) { + return null; + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to get file status [path=" + path + "]"); + } + } + + /** {@inheritDoc} */ + @Override public long usedSpaceSize() { + try { + // We don't use FileSystem#getUsed() since it counts only the files + // in the filesystem root, not all the files recursively. + return fileSystemForUser().getContentSummary(new Path("/")).getSpaceConsumed(); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed to get used space size of file system."); + } + } + + /** {@inheritDoc} */ + @Override public void setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteException { + try { + // We don't use FileSystem#getUsed() since it counts only the files + // in the filesystem root, not all the files recursively. + fileSystemForUser().setTimes(convert(path), modificationTime, accessTime); + } + catch (IOException e) { + throw handleSecondaryFsError(e, "Failed set times for path: " + path); + } + } + + /** {@inheritDoc} */ + public void start() { + factory.start(); + } + + /** {@inheritDoc} */ + public void stop() { + factory.stop(); + } + + /** + * Convert IGFS path into Hadoop path. + * + * @param path IGFS path. + * @return Hadoop path. + */ + private Path convert(IgfsPath path) { + URI uri = fileSystemForUser().getUri(); + + return new Path(uri.getScheme(), uri.getAuthority(), path.toString()); + } + + /** + * Heuristically checks if exception was caused by invalid HDFS version and returns appropriate exception. + * + * @param e Exception to check. + * @param detailMsg Detailed error message. + * @return Appropriate exception. + */ + private IgfsException handleSecondaryFsError(IOException e, String detailMsg) { + return cast(detailMsg, e); + } + + /** + * Cast IO exception to IGFS exception. + * + * @param e IO exception. + * @return IGFS exception. + */ + public static IgfsException cast(String msg, IOException e) { + if (e instanceof FileNotFoundException) + return new IgfsPathNotFoundException(e); + else if (e instanceof ParentNotDirectoryException) + return new IgfsParentNotDirectoryException(msg, e); + else if (e instanceof PathIsNotEmptyDirectoryException) + return new IgfsDirectoryNotEmptyException(e); + else if (e instanceof PathExistsException) + return new IgfsPathAlreadyExistsException(msg, e); + else + return new IgfsException(msg, e); + } + + /** + * Convert Hadoop FileStatus properties to map. + * + * @param status File status. + * @return IGFS attributes. + */ + private static Map properties(FileStatus status) { + FsPermission perm = status.getPermission(); + + if (perm == null) + perm = FsPermission.getDefault(); + + HashMap res = new HashMap<>(3); + + res.put(IgfsUtils.PROP_PERMISSION, String.format("%04o", perm.toShort())); + res.put(IgfsUtils.PROP_USER_NAME, status.getOwner()); + res.put(IgfsUtils.PROP_GROUP_NAME, status.getGroup()); + + return res; + } + + /** + * Gets the FileSystem for the current context user. + * @return the FileSystem instance, never null. + */ + private FileSystem fileSystemForUser() { + String user = IgfsUserContext.currentUser(); + + if (F.isEmpty(user)) + user = IgfsUtils.fixUserName(dfltUsrName); + + assert !F.isEmpty(user); + + try { + return (FileSystem)factory.get(user); + } + catch (IOException ioe) { + throw new IgniteException(ioe); + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(HadoopIgfsSecondaryFileSystemDelegateImpl.class, this); + } +} diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopKerberosFileSystemFactoryDelegate.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopKerberosFileSystemFactoryDelegate.java new file mode 100644 index 0000000000000..c71dedbc79eab --- /dev/null +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopKerberosFileSystemFactoryDelegate.java @@ -0,0 +1,112 @@ +/* + * 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.hadoop.impl.delegate; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.ignite.IgniteException; +import org.apache.ignite.hadoop.fs.KerberosHadoopFileSystemFactory; +import org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.A; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; + +/** + * Kerberos Hadoop file system factory delegate. + */ +public class HadoopKerberosFileSystemFactoryDelegate extends HadoopBasicFileSystemFactoryDelegate { + /** The re-login interval. */ + private long reloginInterval; + + /** Time of last re-login attempt, in system milliseconds. */ + private volatile long lastReloginTime; + + /** + * Constructor. + * + * @param proxy Proxy. + */ + public HadoopKerberosFileSystemFactoryDelegate(KerberosHadoopFileSystemFactory proxy) { + super(proxy); + } + + /** {@inheritDoc} */ + @Override public FileSystem getWithMappedName(String name) throws IOException { + reloginIfNeeded(); + + return super.getWithMappedName(name); + } + + /** {@inheritDoc} */ + @Override protected FileSystem create(String usrName) throws IOException, InterruptedException { + UserGroupInformation proxyUgi = UserGroupInformation.createProxyUser(usrName, + UserGroupInformation.getLoginUser()); + + return proxyUgi.doAs(new PrivilegedExceptionAction() { + @Override public FileSystem run() throws Exception { + return FileSystem.get(fullUri, cfg); + } + }); + } + + @Override public void start() throws IgniteException { + super.start(); + + KerberosHadoopFileSystemFactory proxy0 = (KerberosHadoopFileSystemFactory)proxy; + + A.ensure(!F.isEmpty(proxy0.getKeyTab()), "keyTab cannot not be empty."); + A.ensure(!F.isEmpty(proxy0.getKeyTabPrincipal()), "keyTabPrincipal cannot not be empty."); + A.ensure(proxy0.getReloginInterval() >= 0, "reloginInterval cannot not be negative."); + + reloginInterval = proxy0.getReloginInterval(); + + try { + UserGroupInformation.setConfiguration(cfg); + UserGroupInformation.loginUserFromKeytab(proxy0.getKeyTabPrincipal(), proxy0.getKeyTab()); + } + catch (IOException ioe) { + throw new IgniteException("Failed login from keytab [keyTab=" + proxy0.getKeyTab() + + ", keyTabPrincipal=" + proxy0.getKeyTabPrincipal() + ']', ioe); + } + } + + /** + * Re-logins the user if needed. + * First, the re-login interval defined in factory is checked. The re-login attempts will be not more + * frequent than one attempt per {@code reloginInterval}. + * Second, {@code UserGroupInformation.checkTGTAndReloginFromKeytab()} method invoked that gets existing + * TGT and checks its validity. If the TGT is expired or is close to expiry, it performs re-login. + * + *

          This operation expected to be called upon each operation with the file system created with the factory. + * As long as {@link #get(String)} operation is invoked upon each file {@link IgniteHadoopFileSystem}, there + * is no need to invoke it otherwise specially. + * + * @throws IOException If login fails. + */ + private void reloginIfNeeded() throws IOException { + long now = System.currentTimeMillis(); + + if (now >= lastReloginTime + reloginInterval) { + UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab(); + + lastReloginTime = now; + } + } +} diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemCacheUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopFileSystemCacheUtils.java similarity index 99% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemCacheUtils.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopFileSystemCacheUtils.java index 1ecbee5e3d939..0b673e9ff2a79 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemCacheUtils.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopFileSystemCacheUtils.java @@ -15,10 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.fs; +package org.apache.ignite.internal.processors.hadoop.impl.fs; -import java.io.IOException; -import java.net.URI; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.mapreduce.MRJobConfig; @@ -28,6 +26,9 @@ import org.apache.ignite.internal.util.typedef.F; import org.jetbrains.annotations.Nullable; +import java.io.IOException; +import java.net.URI; + /** * File system cache utility methods used by Map-Reduce tasks and jobs. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopFileSystemsUtils.java similarity index 97% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopFileSystemsUtils.java index 68c0dc4890b56..5115cb47c212f 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopFileSystemsUtils.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.fs; +package org.apache.ignite.internal.processors.hadoop.impl.fs; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FsConstants; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopLazyConcurrentMap.java similarity index 97% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopLazyConcurrentMap.java index 681cddbf50717..ea1f6642b7d5e 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopLazyConcurrentMap.java @@ -15,7 +15,12 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.fs; +package org.apache.ignite.internal.processors.hadoop.impl.fs; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.jsr166.ConcurrentHashMap8; import java.io.Closeable; import java.io.IOException; @@ -23,11 +28,6 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteException; -import org.apache.ignite.internal.util.future.GridFutureAdapter; -import org.jsr166.ConcurrentHashMap8; /** * Maps values by keys. @@ -54,8 +54,6 @@ public class HadoopLazyConcurrentMap { */ public HadoopLazyConcurrentMap(ValueFactory factory) { this.factory = factory; - - assert getClass().getClassLoader() == Ignite.class.getClassLoader(); } /** diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLocalFileSystemV1.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopLocalFileSystemV1.java similarity index 95% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLocalFileSystemV1.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopLocalFileSystemV1.java index cbb007f982b0c..df3ca2b5e8299 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLocalFileSystemV1.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopLocalFileSystemV1.java @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.fs; +package org.apache.ignite.internal.processors.hadoop.impl.fs; -import java.io.File; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; +import java.io.File; + /** * Local file system replacement for Hadoop jobs. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLocalFileSystemV2.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopLocalFileSystemV2.java similarity index 97% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLocalFileSystemV2.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopLocalFileSystemV2.java index 24844920ee9fe..a98b2e387a873 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLocalFileSystemV2.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopLocalFileSystemV2.java @@ -15,17 +15,18 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.fs; +package org.apache.ignite.internal.processors.hadoop.impl.fs; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.ChecksumFs; import org.apache.hadoop.fs.DelegateToFileSystem; import org.apache.hadoop.fs.FsServerDefaults; import org.apache.hadoop.fs.local.LocalConfigKeys; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; + import static org.apache.hadoop.fs.FsConstants.LOCAL_FS_URI; /** diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopParameters.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopParameters.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopParameters.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopParameters.java index 0aac4a308c6e2..b583a1df33aea 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopParameters.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopParameters.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.fs; +package org.apache.ignite.internal.processors.hadoop.impl.fs; /** * This class lists parameters that can be specified in Hadoop configuration. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopRawLocalFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopRawLocalFileSystem.java similarity index 99% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopRawLocalFileSystem.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopRawLocalFileSystem.java index b8fc8e772965d..89c5938332fd4 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopRawLocalFileSystem.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopRawLocalFileSystem.java @@ -15,17 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.fs; +package org.apache.ignite.internal.processors.hadoop.impl.fs; -import java.io.BufferedOutputStream; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.RandomAccessFile; -import java.net.URI; -import java.nio.file.Files; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -40,6 +31,16 @@ import org.apache.hadoop.util.Progressable; import org.apache.ignite.internal.util.typedef.internal.U; +import java.io.BufferedOutputStream; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.RandomAccessFile; +import java.net.URI; +import java.nio.file.Files; + /** * Local file system implementation for Hadoop. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfs.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfs.java similarity index 99% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfs.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfs.java index fe435967b2354..8bb904fc96f73 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfs.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfs.java @@ -15,11 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; -import java.io.IOException; -import java.util.Collection; -import java.util.Map; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.igfs.IgfsBlockLocation; import org.apache.ignite.igfs.IgfsFile; @@ -29,6 +26,10 @@ import org.apache.ignite.internal.processors.igfs.IgfsStatus; import org.jetbrains.annotations.Nullable; +import java.io.IOException; +import java.util.Collection; +import java.util.Map; + /** * Facade for communication with grid. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsCommunicationException.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsCommunicationException.java similarity index 96% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsCommunicationException.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsCommunicationException.java index d610091790cdb..ddfe35b69cfdc 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsCommunicationException.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsCommunicationException.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import org.apache.ignite.IgniteCheckedException; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsEx.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsEx.java index 014e2a1deb09d..2294134b1717c 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsEx.java @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; -import java.io.IOException; import org.apache.ignite.internal.IgniteInternalFuture; import org.jetbrains.annotations.Nullable; +import java.io.IOException; + /** * Extended IGFS server interface. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsFuture.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsFuture.java similarity index 97% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsFuture.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsFuture.java index 5ff1b2eb43590..cfdd7925a1f8d 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsFuture.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsFuture.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.jetbrains.annotations.Nullable; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsInProc.java similarity index 99% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsInProc.java index 322053829653e..8bdcc83876583 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsInProc.java @@ -15,14 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; -import java.io.Closeable; -import java.io.IOException; -import java.util.Collection; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.logging.Log; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -42,6 +36,13 @@ import org.apache.ignite.lang.IgniteOutClosure; import org.jetbrains.annotations.Nullable; +import java.io.Closeable; +import java.io.IOException; +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; + /** * Communication with grid in the same process. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInputStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsInputStream.java similarity index 99% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInputStream.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsInputStream.java index 46b46d7690bc8..efc270b97e4cb 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInputStream.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsInputStream.java @@ -15,11 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStream; import org.apache.commons.logging.Log; import org.apache.hadoop.fs.PositionedReadable; import org.apache.hadoop.fs.Seekable; @@ -30,6 +27,10 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.NotNull; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; + /** * IGFS input stream wrapper for hadoop interfaces. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsIo.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIo.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsIo.java index 70f645f58e399..b8bcad9a59ee0 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIo.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsIo.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsIpcIo.java similarity index 99% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsIpcIo.java index b0a41354af23d..a2ec977bea74d 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsIpcIo.java @@ -15,20 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; -import java.io.BufferedOutputStream; -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStream; -import java.util.Collection; -import java.util.Iterator; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.commons.logging.Log; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.igfs.IgfsException; @@ -52,6 +40,19 @@ import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; +import java.io.BufferedOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.util.Collection; +import java.util.Iterator; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + /** * IO layer implementation based on blocking IPC streams. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIoListener.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsIpcIoListener.java similarity index 94% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIoListener.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsIpcIoListener.java index c26e896035b74..88e59e67a8051 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIoListener.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsIpcIoListener.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; /** * Listens to the events of {@link HadoopIgfsIpcIo}. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsJclLogger.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsJclLogger.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsJclLogger.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsJclLogger.java index 3a7f45b762f72..0ae8a9fe1a421 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsJclLogger.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsJclLogger.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsOutProc.java similarity index 99% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsOutProc.java index 99021426481f7..4dc3c7fa19d1b 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsOutProc.java @@ -15,11 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; -import java.io.IOException; -import java.util.Collection; -import java.util.Map; import org.apache.commons.logging.Log; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.igfs.IgfsBlockLocation; @@ -44,6 +41,10 @@ import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; +import java.io.IOException; +import java.util.Collection; +import java.util.Map; + import static org.apache.ignite.internal.igfs.common.IgfsIpcCommand.AFFINITY; import static org.apache.ignite.internal.igfs.common.IgfsIpcCommand.CLOSE; import static org.apache.ignite.internal.igfs.common.IgfsIpcCommand.DELETE; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutputStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsOutputStream.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutputStream.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsOutputStream.java index 8f7458b442315..7f95a6b166d45 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutputStream.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsOutputStream.java @@ -15,15 +15,16 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; -import java.io.IOException; -import java.io.OutputStream; import org.apache.commons.logging.Log; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.igfs.common.IgfsLogger; import org.jetbrains.annotations.NotNull; +import java.io.IOException; +import java.io.OutputStream; + /** * IGFS Hadoop output stream implementation. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsProperties.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsProperties.java similarity index 97% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsProperties.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsProperties.java index 90f6bcac8b2f8..5427bf15b6033 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsProperties.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsProperties.java @@ -15,13 +15,14 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; -import java.util.Map; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.ignite.IgniteException; import org.apache.ignite.internal.processors.igfs.IgfsUtils; +import java.util.Map; + /** * Hadoop file system properties. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsProxyInputStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsProxyInputStream.java similarity index 99% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsProxyInputStream.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsProxyInputStream.java index 5cee947ac4d17..133e2077fae2e 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsProxyInputStream.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsProxyInputStream.java @@ -15,15 +15,16 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; -import java.io.IOException; -import java.io.InputStream; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.PositionedReadable; import org.apache.hadoop.fs.Seekable; import org.apache.ignite.internal.igfs.common.IgfsLogger; +import java.io.IOException; +import java.io.InputStream; + /** * Secondary Hadoop file system input stream wrapper. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsProxyOutputStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsProxyOutputStream.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsProxyOutputStream.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsProxyOutputStream.java index eade0f013a7f7..8917a95ca01e6 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsProxyOutputStream.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsProxyOutputStream.java @@ -15,13 +15,14 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; -import java.io.IOException; -import java.io.OutputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.ignite.internal.igfs.common.IgfsLogger; +import java.io.IOException; +import java.io.OutputStream; + /** * Secondary Hadoop file system output stream wrapper. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsSecondaryFileSystemPositionedReadable.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemPositionedReadable.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsSecondaryFileSystemPositionedReadable.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemPositionedReadable.java index a0577cea08305..1a4add54891be 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsSecondaryFileSystemPositionedReadable.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemPositionedReadable.java @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; -import java.io.IOException; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -25,6 +24,8 @@ import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable; import org.apache.ignite.internal.util.typedef.internal.U; +import java.io.IOException; + /** * Secondary file system input stream wrapper which actually opens input stream only in case it is explicitly * requested. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsStreamDelegate.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsStreamDelegate.java similarity index 97% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsStreamDelegate.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsStreamDelegate.java index 37b58ab6cf072..e6f8061a65f22 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsStreamDelegate.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsStreamDelegate.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import org.apache.ignite.internal.util.typedef.internal.S; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsStreamEventListener.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsStreamEventListener.java similarity index 95% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsStreamEventListener.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsStreamEventListener.java index d81f765114e0b..ee46ed414423e 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsStreamEventListener.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsStreamEventListener.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import org.apache.ignite.IgniteCheckedException; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsUtils.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsUtils.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsUtils.java index fa5cbc584e6d8..ee7756e082eef 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsUtils.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsUtils.java @@ -15,10 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; -import java.io.FileNotFoundException; -import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.AbstractFileSystem; import org.apache.hadoop.fs.FileStatus; @@ -34,6 +32,9 @@ import org.apache.ignite.igfs.IgfsPathNotFoundException; import org.jetbrains.annotations.Nullable; +import java.io.FileNotFoundException; +import java.io.IOException; + /** * Utility constants and methods for IGFS Hadoop file system. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsWrapper.java similarity index 97% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsWrapper.java index f4ee97fd8e997..1fda1c32ea8c3 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsWrapper.java @@ -15,14 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; -import java.io.IOException; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.logging.Log; import org.apache.hadoop.conf.Configuration; import org.apache.ignite.IgniteCheckedException; @@ -33,6 +27,7 @@ import org.apache.ignite.igfs.IgfsFile; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.IgfsPathSummary; +import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsEndpoint; import org.apache.ignite.internal.processors.igfs.IgfsEx; import org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse; import org.apache.ignite.internal.processors.igfs.IgfsStatus; @@ -42,12 +37,19 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + import static org.apache.ignite.IgniteState.STARTED; import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsEndpoint.LOCALHOST; -import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_EMBED; -import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM; -import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_LOCAL_TCP; -import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.parameter; +import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_EMBED; +import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM; +import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_LOCAL_TCP; +import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.parameter; /** * Wrapper for IGFS server. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopClientProtocol.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/proto/HadoopClientProtocol.java similarity index 91% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopClientProtocol.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/proto/HadoopClientProtocol.java index 5f96e082cd3f8..be2aa093978da 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopClientProtocol.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/proto/HadoopClientProtocol.java @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.proto; +package org.apache.ignite.internal.processors.hadoop.impl.proto; -import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -46,18 +45,24 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.client.GridClient; import org.apache.ignite.internal.client.GridClientException; +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; import org.apache.ignite.internal.processors.hadoop.HadoopJobId; import org.apache.ignite.internal.processors.hadoop.HadoopJobProperty; import org.apache.ignite.internal.processors.hadoop.HadoopJobStatus; -import org.apache.ignite.internal.processors.hadoop.HadoopMapReduceCounters; -import org.apache.ignite.internal.processors.hadoop.HadoopUtils; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopMapReduceCounters; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils; +import org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolJobCountersTask; +import org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolJobStatusTask; +import org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolKillJobTask; +import org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolNextTaskIdTask; +import org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolSubmitJobTask; +import org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolTaskArguments; import org.apache.ignite.internal.util.typedef.internal.U; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.JOB_SUBMISSION_START_TS_PROPERTY; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.REQ_NEW_JOBID_TS_PROPERTY; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.RESPONSE_NEW_JOBID_TS_PROPERTY; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.createJobInfo; +import java.io.IOException; + +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.createJobInfo; /** * Hadoop client protocol. @@ -97,11 +102,11 @@ public HadoopClientProtocol(Configuration conf, GridClient cli) { /** {@inheritDoc} */ @Override public JobID getNewJobID() throws IOException, InterruptedException { try { - conf.setLong(REQ_NEW_JOBID_TS_PROPERTY, U.currentTimeMillis()); + conf.setLong(HadoopCommonUtils.REQ_NEW_JOBID_TS_PROPERTY, U.currentTimeMillis()); HadoopJobId jobID = cli.compute().execute(HadoopProtocolNextTaskIdTask.class.getName(), null); - conf.setLong(RESPONSE_NEW_JOBID_TS_PROPERTY, U.currentTimeMillis()); + conf.setLong(HadoopCommonUtils.RESPONSE_NEW_JOBID_TS_PROPERTY, U.currentTimeMillis()); return new JobID(jobID.globalId().toString(), jobID.localId()); } @@ -114,7 +119,7 @@ public HadoopClientProtocol(Configuration conf, GridClient cli) { @Override public JobStatus submitJob(JobID jobId, String jobSubmitDir, Credentials ts) throws IOException, InterruptedException { try { - conf.setLong(JOB_SUBMISSION_START_TS_PROPERTY, U.currentTimeMillis()); + conf.setLong(HadoopCommonUtils.JOB_SUBMISSION_START_TS_PROPERTY, U.currentTimeMillis()); HadoopJobStatus status = cli.compute().execute(HadoopProtocolSubmitJobTask.class.getName(), new HadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId(), createJobInfo(conf))); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1CleanupTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1CleanupTask.java similarity index 93% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1CleanupTask.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1CleanupTask.java index 750b31451bd09..ddf6c297c1ec2 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1CleanupTask.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1CleanupTask.java @@ -15,16 +15,17 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v1; +package org.apache.ignite.internal.processors.hadoop.impl.v1; -import java.io.IOException; import org.apache.hadoop.mapred.JobContext; import org.apache.hadoop.mapred.JobStatus; import org.apache.hadoop.mapred.OutputCommitter; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopV2TaskContext; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopV2TaskContext; + +import java.io.IOException; /** * Hadoop cleanup task implementation for v1 API. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Counter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Counter.java similarity index 95% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Counter.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Counter.java index c623eab3f7784..d91730fa5c499 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Counter.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Counter.java @@ -15,15 +15,16 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v1; +package org.apache.ignite.internal.processors.hadoop.impl.v1; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; import org.apache.hadoop.mapred.Counters; import org.apache.hadoop.mapreduce.Counter; import org.apache.ignite.internal.processors.hadoop.counter.HadoopLongCounter; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopV2Counter; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopV2Counter; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; import static org.apache.hadoop.mapreduce.util.CountersStrings.toEscapedCompactString; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1MapTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1MapTask.java similarity index 96% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1MapTask.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1MapTask.java index fb2266a2435b9..65ff280df7659 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1MapTask.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1MapTask.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v1; +package org.apache.ignite.internal.processors.hadoop.impl.v1; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.FileSplit; @@ -33,7 +33,7 @@ import org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException; import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopV2TaskContext; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopV2TaskContext; /** * Hadoop map task implementation for v1 API. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1OutputCollector.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1OutputCollector.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1OutputCollector.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1OutputCollector.java index 37f81a66510b4..1a3c4bd169f41 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1OutputCollector.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1OutputCollector.java @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v1; +package org.apache.ignite.internal.processors.hadoop.impl.v1; -import java.io.IOException; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.OutputCommitter; @@ -31,6 +30,8 @@ import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; import org.jetbrains.annotations.Nullable; +import java.io.IOException; + /** * Hadoop output collector. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Partitioner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Partitioner.java similarity index 96% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Partitioner.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Partitioner.java index 0ab1bba8decfe..97634d989575f 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Partitioner.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Partitioner.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v1; +package org.apache.ignite.internal.processors.hadoop.impl.v1; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.Partitioner; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1ReduceTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1ReduceTask.java similarity index 95% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1ReduceTask.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1ReduceTask.java index e656695a5b031..92c024ec06924 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1ReduceTask.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1ReduceTask.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v1; +package org.apache.ignite.internal.processors.hadoop.impl.v1; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.Reducer; @@ -27,7 +27,7 @@ import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopV2TaskContext; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopV2TaskContext; /** * Hadoop reduce task implementation for v1 API. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Reporter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Reporter.java similarity index 97% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Reporter.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Reporter.java index 5a63aab86cab0..f3229e21a6355 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Reporter.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Reporter.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v1; +package org.apache.ignite.internal.processors.hadoop.impl.v1; import org.apache.hadoop.mapred.Counters; import org.apache.hadoop.mapred.InputSplit; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1SetupTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1SetupTask.java similarity index 92% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1SetupTask.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1SetupTask.java index d2f6823a56dc6..2fd7332ba00c4 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1SetupTask.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1SetupTask.java @@ -15,14 +15,15 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v1; +package org.apache.ignite.internal.processors.hadoop.impl.v1; -import java.io.IOException; import org.apache.hadoop.mapred.OutputCommitter; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopV2TaskContext; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopV2TaskContext; + +import java.io.IOException; /** * Hadoop setup task implementation for v1 API. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Splitter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Splitter.java similarity index 96% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Splitter.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Splitter.java index 203def4779ef9..11a35988b81ea 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Splitter.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Splitter.java @@ -15,11 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v1; +package org.apache.ignite.internal.processors.hadoop.impl.v1; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputFormat; @@ -28,10 +25,14 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock; import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit; -import org.apache.ignite.internal.processors.hadoop.HadoopUtils; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils; import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; + /** * Hadoop API v1 splitter. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Task.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Task.java similarity index 95% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Task.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Task.java index a89323cf9359f..4ed5eb3ca83c9 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Task.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Task.java @@ -15,18 +15,19 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v1; +package org.apache.ignite.internal.processors.hadoop.impl.v1; -import java.io.IOException; -import java.text.NumberFormat; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.TaskAttemptID; import org.apache.ignite.internal.processors.hadoop.HadoopTask; import org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopV2TaskContext; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopV2TaskContext; import org.jetbrains.annotations.Nullable; +import java.io.IOException; +import java.text.NumberFormat; + /** * Extended Hadoop v1 task. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopDaemon.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopDaemon.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopDaemon.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopDaemon.java index 9632525772021..ea7128c59bd45 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopDaemon.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopDaemon.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; import java.util.Collection; import java.util.LinkedList; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopSerializationWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopSerializationWrapper.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopSerializationWrapper.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopSerializationWrapper.java index 844e7f8847a02..e045dba0e17bb 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopSerializationWrapper.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopSerializationWrapper.java @@ -15,13 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import org.apache.hadoop.io.serializer.Deserializer; import org.apache.hadoop.io.serializer.Serialization; import org.apache.hadoop.io.serializer.Serializer; @@ -29,6 +24,12 @@ import org.apache.ignite.internal.processors.hadoop.HadoopSerialization; import org.jetbrains.annotations.Nullable; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + /** * The wrapper around external serializer. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopShutdownHookManager.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopShutdownHookManager.java similarity index 97% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopShutdownHookManager.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopShutdownHookManager.java index 8bd71e0913307..6d947e8ef4bef 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopShutdownHookManager.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopShutdownHookManager.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; import java.util.Collections; import java.util.HashSet; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2CleanupTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2CleanupTask.java similarity index 97% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2CleanupTask.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2CleanupTask.java index abb904ce2db4b..15e3d0f910da8 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2CleanupTask.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2CleanupTask.java @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; -import java.io.IOException; import org.apache.hadoop.mapred.JobContextImpl; import org.apache.hadoop.mapreduce.JobStatus; import org.apache.hadoop.mapreduce.OutputCommitter; @@ -26,6 +25,8 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; +import java.io.IOException; + /** * Hadoop cleanup task (commits or aborts job). */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Context.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java similarity index 99% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Context.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java index 2ff294566ffa9..90a1bad10a97c 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Context.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java @@ -15,10 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; -import java.io.IOException; -import java.util.Iterator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.InputSplit; @@ -39,6 +37,9 @@ import org.apache.ignite.internal.processors.hadoop.HadoopTaskOutput; import org.apache.ignite.internal.processors.hadoop.counter.HadoopLongCounter; +import java.io.IOException; +import java.util.Iterator; + /** * Hadoop context implementation for v2 API. It provides IO operations for hadoop tasks. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Counter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Counter.java similarity index 97% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Counter.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Counter.java index cad9e645b82fb..bd304456a6d89 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Counter.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Counter.java @@ -15,13 +15,14 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; + +import org.apache.hadoop.mapreduce.Counter; +import org.apache.ignite.internal.processors.hadoop.counter.HadoopLongCounter; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import org.apache.hadoop.mapreduce.Counter; -import org.apache.ignite.internal.processors.hadoop.counter.HadoopLongCounter; /** * Adapter from own counter implementation into Hadoop API Counter od version 2.0. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java similarity index 88% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java index a0f30eba54416..3731b2c1e1f3e 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java @@ -15,26 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInput; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.File; -import java.io.IOException; -import java.lang.reflect.Constructor; -import java.lang.reflect.Method; -import java.net.URI; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Map; -import java.util.Queue; -import java.util.UUID; -import java.util.concurrent.ConcurrentLinkedDeque; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ConcurrentMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; @@ -50,8 +32,11 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader; +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; import org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopExternalSplit; import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock; +import org.apache.ignite.internal.processors.hadoop.HadoopHelper; import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit; import org.apache.ignite.internal.processors.hadoop.HadoopJob; import org.apache.ignite.internal.processors.hadoop.HadoopJobId; @@ -59,10 +44,9 @@ import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; import org.apache.ignite.internal.processors.hadoop.HadoopTaskType; -import org.apache.ignite.internal.processors.hadoop.HadoopUtils; -import org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemsUtils; -import org.apache.ignite.internal.processors.hadoop.fs.HadoopLazyConcurrentMap; -import org.apache.ignite.internal.processors.hadoop.v1.HadoopV1Splitter; +import org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSystemsUtils; +import org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopLazyConcurrentMap; +import org.apache.ignite.internal.processors.hadoop.impl.v1.HadoopV1Splitter; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; @@ -70,12 +54,31 @@ import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.jobLocalDir; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.taskLocalDir; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.transformException; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemCacheUtils.FsCacheKey; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemCacheUtils.createHadoopLazyConcurrentMap; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemCacheUtils.fileSystemForMrUserWithCaching; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.Method; +import java.net.URI; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Map; +import java.util.Queue; +import java.util.UUID; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentMap; + +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.jobLocalDir; +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.taskLocalDir; +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.transformException; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSystemCacheUtils.FsCacheKey; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSystemCacheUtils.createHadoopLazyConcurrentMap; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSystemCacheUtils.fileSystemForMrUserWithCaching; /** * Hadoop job implementation for v2 API. @@ -87,6 +90,9 @@ public class HadoopV2Job implements HadoopJob { /** */ private final JobContextImpl jobCtx; + /** */ + private final HadoopHelper helper; + /** Hadoop job ID. */ private final HadoopJobId jobId; @@ -130,15 +136,16 @@ public class HadoopV2Job implements HadoopJob { * @param libNames Optional additional native library names. */ public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, IgniteLogger log, - @Nullable String[] libNames) { + @Nullable String[] libNames, HadoopHelper helper) { assert jobId != null; assert jobInfo != null; this.jobId = jobId; this.jobInfo = jobInfo; this.libNames = libNames; + this.helper = helper; - ClassLoader oldLdr = HadoopUtils.setContextClassLoader(getClass().getClassLoader()); + ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(getClass().getClassLoader()); try { hadoopJobID = new JobID(jobId.globalId().toString(), jobId.localId()); @@ -155,7 +162,7 @@ public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, Ignite rsrcMgr = new HadoopV2JobResourceManager(jobId, jobCtx, log, this); } finally { - HadoopUtils.setContextClassLoader(oldLdr); + HadoopCommonUtils.restoreContextClassLoader(oldLdr); } } @@ -171,7 +178,7 @@ public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, Ignite /** {@inheritDoc} */ @Override public Collection input() throws IgniteCheckedException { - ClassLoader oldLdr = HadoopUtils.setContextClassLoader(jobConf.getClassLoader()); + ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(jobConf.getClassLoader()); try { String jobDirPath = jobConf.get(MRJobConfig.MAPREDUCE_JOB_DIR); @@ -228,7 +235,7 @@ public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, Ignite } } finally { - HadoopUtils.restoreContextClassLoader(oldLdr); + HadoopCommonUtils.restoreContextClassLoader(oldLdr); } } @@ -255,7 +262,7 @@ public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, Ignite // Note that the classloader identified by the task it was initially created for, // but later it may be reused for other tasks. HadoopClassLoader ldr = new HadoopClassLoader(rsrcMgr.classPath(), - HadoopClassLoader.nameForTask(info, false), libNames); + HadoopClassLoader.nameForTask(info, false), libNames, helper); cls = (Class)ldr.loadClass(HadoopV2TaskContext.class.getName()); @@ -301,13 +308,13 @@ public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, Ignite this.locNodeId = locNodeId; - ClassLoader oldLdr = HadoopUtils.setContextClassLoader(getClass().getClassLoader()); + ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(getClass().getClassLoader()); try { rsrcMgr.prepareJobEnvironment(!external, jobLocalDir(locNodeId, jobId)); } finally { - HadoopUtils.restoreContextClassLoader(oldLdr); + HadoopCommonUtils.restoreContextClassLoader(oldLdr); } } @@ -382,7 +389,7 @@ public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, Ignite * @throws Exception On error. */ private void stopHadoopFsDaemons(ClassLoader ldr) throws Exception { - Class daemonCls = ldr.loadClass(HadoopClassLoader.HADOOP_DAEMON_CLASS_NAME); + Class daemonCls = ldr.loadClass(HadoopClassLoader.CLS_DAEMON); Method m = daemonCls.getMethod("dequeueAndStopAll"); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2JobResourceManager.java similarity index 96% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2JobResourceManager.java index 33aef60c63039..3984f838aa46f 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2JobResourceManager.java @@ -15,18 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.net.URL; -import java.nio.file.FileSystemException; -import java.nio.file.Files; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; @@ -36,13 +26,24 @@ import org.apache.hadoop.util.RunJar; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; import org.apache.ignite.internal.processors.hadoop.HadoopJobId; -import org.apache.ignite.internal.processors.hadoop.HadoopUtils; -import org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemsUtils; +import org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSystemsUtils; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.FileSystemException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; + /** * Provides all resources are needed to the job execution. Downloads the main jar, the configuration and additional * files are needed to be placed on local files system. @@ -95,7 +96,7 @@ public HadoopV2JobResourceManager(HadoopJobId jobId, JobContextImpl ctx, IgniteL private void setLocalFSWorkingDirectory(File dir) throws IOException { JobConf cfg = ctx.getJobConf(); - ClassLoader oldLdr = HadoopUtils.setContextClassLoader(cfg.getClassLoader()); + ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(cfg.getClassLoader()); try { cfg.set(HadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP, dir.getAbsolutePath()); @@ -104,7 +105,7 @@ private void setLocalFSWorkingDirectory(File dir) throws IOException { FileSystem.getLocal(cfg).setWorkingDirectory(new Path(dir.getAbsolutePath())); } finally { - HadoopUtils.restoreContextClassLoader(oldLdr); + HadoopCommonUtils.restoreContextClassLoader(oldLdr); } } @@ -176,7 +177,7 @@ else if (!jobLocDir.mkdirs()) * Process list of resources. * * @param jobLocDir Job working directory. - * @param files Array of {@link java.net.URI} or {@link org.apache.hadoop.fs.Path} to process resources. + * @param files Array of {@link URI} or {@link org.apache.hadoop.fs.Path} to process resources. * @param download {@code true}, if need to download. Process class path only else. * @param extract {@code true}, if need to extract archive. * @param clsPathUrls Collection to add resource as classpath resource. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2MapTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2MapTask.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2MapTask.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2MapTask.java index fafa79b704920..418df4eb759f4 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2MapTask.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2MapTask.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; import org.apache.hadoop.mapred.JobContextImpl; import org.apache.hadoop.mapreduce.InputFormat; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Partitioner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Partitioner.java similarity index 96% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Partitioner.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Partitioner.java index e199ede44e485..5a82dcfe8981c 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Partitioner.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Partitioner.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.Partitioner; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2ReduceTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2ReduceTask.java similarity index 97% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2ReduceTask.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2ReduceTask.java index e5c2ed2164e74..930ec1d49746e 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2ReduceTask.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2ReduceTask.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; import org.apache.hadoop.mapred.JobContextImpl; import org.apache.hadoop.mapreduce.OutputFormat; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2SetupTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2SetupTask.java similarity index 97% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2SetupTask.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2SetupTask.java index 49b5ee7e23034..b466019f63e96 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2SetupTask.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2SetupTask.java @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; -import java.io.IOException; import org.apache.hadoop.mapred.JobContextImpl; import org.apache.hadoop.mapreduce.OutputCommitter; import org.apache.hadoop.mapreduce.OutputFormat; @@ -25,6 +24,8 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; +import java.io.IOException; + /** * Hadoop setup task (prepares job). */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Splitter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Splitter.java similarity index 96% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Splitter.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Splitter.java index f4ed668eceeb1..667ef1e4a6e8f 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Splitter.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Splitter.java @@ -15,13 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; -import java.io.DataInput; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.JobContext; @@ -31,9 +26,15 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock; import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit; -import org.apache.ignite.internal.processors.hadoop.HadoopUtils; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils; import org.jetbrains.annotations.Nullable; +import java.io.DataInput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + /** * Hadoop API v2 splitter. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Task.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Task.java similarity index 98% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Task.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Task.java index 1383a613f6fef..ee8bd98e305bf 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Task.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Task.java @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; -import java.io.IOException; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.OutputCommitter; import org.apache.hadoop.mapreduce.OutputFormat; @@ -29,6 +28,8 @@ import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; import org.jetbrains.annotations.Nullable; +import java.io.IOException; + /** * Extended Hadoop v2 task. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java similarity index 89% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java index 4b1121c45e8ed..96fa89218250c 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java @@ -15,15 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; -import java.io.DataInput; -import java.io.File; -import java.io.IOException; -import java.security.PrivilegedExceptionAction; -import java.util.Comparator; -import java.util.UUID; -import java.util.concurrent.Callable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; @@ -47,39 +40,49 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader; +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; +import org.apache.ignite.internal.processors.hadoop.HadoopExternalSplit; import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit; import org.apache.ignite.internal.processors.hadoop.HadoopJob; import org.apache.ignite.internal.processors.hadoop.HadoopJobId; import org.apache.ignite.internal.processors.hadoop.HadoopPartitioner; import org.apache.ignite.internal.processors.hadoop.HadoopSerialization; +import org.apache.ignite.internal.processors.hadoop.HadoopSplitWrapper; import org.apache.ignite.internal.processors.hadoop.HadoopTask; import org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException; import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; import org.apache.ignite.internal.processors.hadoop.HadoopTaskType; -import org.apache.ignite.internal.processors.hadoop.HadoopUtils; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounter; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCountersImpl; -import org.apache.ignite.internal.processors.hadoop.fs.HadoopLazyConcurrentMap; -import org.apache.ignite.internal.processors.hadoop.v1.HadoopV1CleanupTask; -import org.apache.ignite.internal.processors.hadoop.v1.HadoopV1MapTask; -import org.apache.ignite.internal.processors.hadoop.v1.HadoopV1Partitioner; -import org.apache.ignite.internal.processors.hadoop.v1.HadoopV1ReduceTask; -import org.apache.ignite.internal.processors.hadoop.v1.HadoopV1SetupTask; +import org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopLazyConcurrentMap; +import org.apache.ignite.internal.processors.hadoop.impl.v1.HadoopV1CleanupTask; +import org.apache.ignite.internal.processors.hadoop.impl.v1.HadoopV1MapTask; +import org.apache.ignite.internal.processors.hadoop.impl.v1.HadoopV1Partitioner; +import org.apache.ignite.internal.processors.hadoop.impl.v1.HadoopV1ReduceTask; +import org.apache.ignite.internal.processors.hadoop.impl.v1.HadoopV1SetupTask; import org.apache.ignite.internal.processors.igfs.IgfsUtils; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; import org.jetbrains.annotations.Nullable; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.jobLocalDir; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.taskLocalDir; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.transformException; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.unwrapSplit; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemCacheUtils.FsCacheKey; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemCacheUtils.createHadoopLazyConcurrentMap; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemCacheUtils.fileSystemForMrUserWithCaching; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_PREFER_LOCAL_WRITES; +import java.io.DataInput; +import java.io.File; +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.util.Comparator; +import java.util.UUID; +import java.util.concurrent.Callable; + +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.jobLocalDir; +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.taskLocalDir; +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.transformException; +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.unwrapSplit; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSystemCacheUtils.FsCacheKey; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSystemCacheUtils.createHadoopLazyConcurrentMap; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSystemCacheUtils.fileSystemForMrUserWithCaching; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_PREFER_LOCAL_WRITES; /** * Context for task execution. @@ -159,7 +162,7 @@ public HadoopV2TaskContext(HadoopTaskInfo taskInfo, HadoopJob job, HadoopJobId j this.locNodeId = locNodeId; // Before create JobConf instance we should set new context class loader. - ClassLoader oldLdr = HadoopUtils.setContextClassLoader(getClass().getClassLoader()); + ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(getClass().getClassLoader()); try { JobConf jobConf = new JobConf(); @@ -181,7 +184,7 @@ public HadoopV2TaskContext(HadoopTaskInfo taskInfo, HadoopJob job, HadoopJobId j useNewCombiner = jobConf.getCombinerClass() == null; } finally { - HadoopUtils.restoreContextClassLoader(oldLdr); + HadoopCommonUtils.restoreContextClassLoader(oldLdr); } } @@ -230,7 +233,7 @@ private HadoopTask createTask() { /** {@inheritDoc} */ @Override public void run() throws IgniteCheckedException { - ClassLoader oldLdr = HadoopUtils.setContextClassLoader(jobConf().getClassLoader()); + ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(jobConf().getClassLoader()); try { try { @@ -259,7 +262,7 @@ private HadoopTask createTask() { finally { task = null; - HadoopUtils.restoreContextClassLoader(oldLdr); + HadoopCommonUtils.restoreContextClassLoader(oldLdr); } } @@ -290,7 +293,7 @@ private HadoopTask createTask() { locDir = jobLocalDir(locNodeId, taskInfo().jobId()); } - ClassLoader oldLdr = HadoopUtils.setContextClassLoader(jobConf().getClassLoader()); + ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(jobConf().getClassLoader()); try { FileSystem.get(jobConf()); @@ -306,7 +309,7 @@ private HadoopTask createTask() { throw transformException(e); } finally { - HadoopUtils.restoreContextClassLoader(oldLdr); + HadoopCommonUtils.restoreContextClassLoader(oldLdr); } } diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopWritableSerialization.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopWritableSerialization.java similarity index 97% rename from modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopWritableSerialization.java rename to modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopWritableSerialization.java index f46f06800689e..e612f1bccfb36 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopWritableSerialization.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopWritableSerialization.java @@ -15,17 +15,18 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.v2; +package org.apache.ignite.internal.processors.hadoop.impl.v2; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; import org.apache.hadoop.io.Writable; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.hadoop.HadoopSerialization; import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + /** * Optimized serialization for Hadoop {@link Writable} types. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java index f3e17f350aae9..bffb82b1e9180 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java @@ -46,6 +46,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader; +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; import org.apache.ignite.internal.processors.hadoop.HadoopComponent; import org.apache.ignite.internal.processors.hadoop.HadoopContext; import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit; @@ -58,14 +59,12 @@ import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlanner; import org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; -import org.apache.ignite.internal.processors.hadoop.HadoopUtils; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounterWriter; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCountersImpl; import org.apache.ignite.internal.processors.hadoop.counter.HadoopPerformanceCounter; import org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskStatus; import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopProcessDescriptor; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopV2Job; import org.apache.ignite.internal.util.GridMutex; import org.apache.ignite.internal.util.GridSpinReadWriteLock; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -153,23 +152,16 @@ public class HadoopJobTracker extends HadoopComponent { evtProcSvc = Executors.newFixedThreadPool(1); - UUID nodeId = ctx.localNodeId(); - assert jobCls == null; - String[] libNames = null; - - if (ctx.configuration() != null) - libNames = ctx.configuration().getNativeLibraryNames(); - - HadoopClassLoader ldr = new HadoopClassLoader(null, HadoopClassLoader.nameForJob(nodeId), libNames); + HadoopClassLoader ldr = ctx.kernalContext().hadoopHelper().commonClassLoader(); try { - jobCls = (Class)ldr.loadClass(HadoopV2Job.class.getName()); + jobCls = (Class)ldr.loadClass(HadoopCommonUtils.JOB_CLS_NAME); } catch (Exception ioe) { - throw new IgniteCheckedException("Failed to load job class [class=" - + HadoopV2Job.class.getName() + ']', ioe); + throw new IgniteCheckedException("Failed to load job class [class=" + + HadoopCommonUtils.JOB_CLS_NAME + ']', ioe); } } @@ -903,7 +895,7 @@ private void processJobMetaUpdate(HadoopJobId jobId, HadoopJobMetadata meta, UUI ClassLoader ldr = job.getClass().getClassLoader(); try { - String statWriterClsName = job.info().property(HadoopUtils.JOB_COUNTER_WRITER_PROPERTY); + String statWriterClsName = job.info().property(HadoopCommonUtils.JOB_COUNTER_WRITER_PROPERTY); if (statWriterClsName != null) { Class cls = ldr.loadClass(statWriterClsName); @@ -1060,7 +1052,8 @@ private JobLocalState initState(HadoopJobId jobId) { jobInfo = meta.jobInfo(); } - job = jobInfo.createJob(jobCls, jobId, log, ctx.configuration().getNativeLibraryNames()); + job = jobInfo.createJob(jobCls, jobId, log, ctx.configuration().getNativeLibraryNames(), + ctx.kernalContext().hadoopHelper()); job.initialize(false, ctx.localNodeId()); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlan.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlan.java index 15c62c8dc842a..7aaf3faece580 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlan.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlan.java @@ -17,13 +17,14 @@ package org.apache.ignite.internal.processors.hadoop.planner; -import java.util.Collection; -import java.util.Map; -import java.util.UUID; import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit; import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan; import org.jetbrains.annotations.Nullable; +import java.util.Collection; +import java.util.Map; +import java.util.UUID; + /** * Map-reduce plan. */ diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java index 4a946e938a9ed..45d9a27c51dec 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java @@ -21,8 +21,11 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; +import org.apache.ignite.internal.processors.hadoop.HadoopHelperImpl; import org.apache.ignite.internal.processors.hadoop.HadoopJob; import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; @@ -44,7 +47,6 @@ import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopTaskFinishedMessage; import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopExternalCommunication; import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopMessageListener; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopV2Job; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.IgniteInClosure2X; import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory; @@ -126,6 +128,7 @@ public void start(HadoopExternalCommunication comm, HadoopProcessDescriptor node * * @param req Initialization request. */ + @SuppressWarnings("unchecked") private void prepareProcess(HadoopPrepareForJobRequest req) { if (initGuard.compareAndSet(false, true)) { try { @@ -134,7 +137,16 @@ private void prepareProcess(HadoopPrepareForJobRequest req) { assert job == null; - job = req.jobInfo().createJob(HadoopV2Job.class, req.jobId(), log, null); + Class jobCls; + + try { + jobCls = Class.forName(HadoopCommonUtils.JOB_CLS_NAME); + } + catch (ClassNotFoundException e) { + throw new IgniteException("Failed to load job class: " + HadoopCommonUtils.JOB_CLS_NAME, e); + } + + job = req.jobInfo().createJob(jobCls, req.jobId(), log, null, new HadoopHelperImpl()); job.initialize(true, nodeDesc.processId()); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java deleted file mode 100644 index 02d98d0a89f34..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java +++ /dev/null @@ -1,110 +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.hadoop; - -import javax.security.auth.AuthPermission; -import junit.framework.TestCase; -import org.apache.hadoop.conf.Configuration; -import org.apache.ignite.internal.processors.hadoop.deps.CircularWIthHadoop; -import org.apache.ignite.internal.processors.hadoop.deps.CircularWithoutHadoop; -import org.apache.ignite.internal.processors.hadoop.deps.WithIndirectField; -import org.apache.ignite.internal.processors.hadoop.deps.WithCast; -import org.apache.ignite.internal.processors.hadoop.deps.WithClassAnnotation; -import org.apache.ignite.internal.processors.hadoop.deps.WithConstructorInvocation; -import org.apache.ignite.internal.processors.hadoop.deps.WithMethodCheckedException; -import org.apache.ignite.internal.processors.hadoop.deps.WithMethodRuntimeException; -import org.apache.ignite.internal.processors.hadoop.deps.WithExtends; -import org.apache.ignite.internal.processors.hadoop.deps.WithField; -import org.apache.ignite.internal.processors.hadoop.deps.WithImplements; -import org.apache.ignite.internal.processors.hadoop.deps.WithInitializer; -import org.apache.ignite.internal.processors.hadoop.deps.WithInnerClass; -import org.apache.ignite.internal.processors.hadoop.deps.WithLocalVariable; -import org.apache.ignite.internal.processors.hadoop.deps.WithMethodAnnotation; -import org.apache.ignite.internal.processors.hadoop.deps.WithMethodInvocation; -import org.apache.ignite.internal.processors.hadoop.deps.WithMethodArgument; -import org.apache.ignite.internal.processors.hadoop.deps.WithMethodReturnType; -import org.apache.ignite.internal.processors.hadoop.deps.WithOuterClass; -import org.apache.ignite.internal.processors.hadoop.deps.WithParameterAnnotation; -import org.apache.ignite.internal.processors.hadoop.deps.WithStaticField; -import org.apache.ignite.internal.processors.hadoop.deps.WithStaticInitializer; -import org.apache.ignite.internal.processors.hadoop.deps.Without; - -/** - * Tests for Hadoop classloader. - */ -public class HadoopClassLoaderTest extends TestCase { - /** */ - final HadoopClassLoader ldr = new HadoopClassLoader(null, "test", null); - - /** - * @throws Exception If failed. - */ - public void testClassLoading() throws Exception { - assertNotSame(CircularWIthHadoop.class, ldr.loadClass(CircularWIthHadoop.class.getName())); - assertNotSame(CircularWithoutHadoop.class, ldr.loadClass(CircularWithoutHadoop.class.getName())); - - assertSame(Without.class, ldr.loadClass(Without.class.getName())); - } - - /** - * Test dependency search. - */ - public void testDependencySearch() { - // Positive cases: - final Class[] positiveClasses = { - Configuration.class, - HadoopUtils.class, - WithStaticField.class, - WithCast.class, - WithClassAnnotation.class, - WithConstructorInvocation.class, - WithMethodCheckedException.class, - WithMethodRuntimeException.class, - WithExtends.class, - WithField.class, - WithImplements.class, - WithInitializer.class, - WithInnerClass.class, - WithOuterClass.InnerNoHadoop.class, - WithLocalVariable.class, - WithMethodAnnotation.class, - WithMethodInvocation.class, - WithMethodArgument.class, - WithMethodReturnType.class, - WithParameterAnnotation.class, - WithStaticField.class, - WithStaticInitializer.class, - WithIndirectField.class, - CircularWIthHadoop.class, - CircularWithoutHadoop.class, - }; - - for (Class c: positiveClasses) - assertTrue(c.getName(), ldr.hasExternalDependencies(c.getName())); - - // Negative cases: - final Class[] negativeClasses = { - Object.class, - AuthPermission.class, - Without.class, - }; - - for (Class c: negativeClasses) - assertFalse(c.getName(), ldr.hasExternalDependencies(c.getName())); - } -} \ No newline at end of file diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSharedMap.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSharedMap.java index 7552028a93d79..fd72821c3f4b7 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSharedMap.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSharedMap.java @@ -43,6 +43,7 @@ private HadoopSharedMap() { * @param key Key. * @param val Value. */ + @SuppressWarnings("unchecked") public T put(String key, T val) { Object old = map.putIfAbsent(key, val); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestClassLoader.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestClassLoader.java new file mode 100644 index 0000000000000..c5302f811c1d1 --- /dev/null +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestClassLoader.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.internal.processors.hadoop; + +import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.X; + +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +/** + * Hadoop test class loader aimed to provide better isolation. + */ +public class HadoopTestClassLoader extends URLClassLoader { + /** Parent class loader. */ + private static final URLClassLoader APP_CLS_LDR = (URLClassLoader)HadoopTestClassLoader.class.getClassLoader(); + + /** */ + private static final Collection APP_JARS = F.asList(APP_CLS_LDR.getURLs()); + + /** All participating URLs. */ + private static final URL[] URLS; + + static { + try { + List res = new ArrayList<>(); + + for (URL url : APP_JARS) { + String urlStr = url.toString(); + + if (urlStr.contains("modules/hadoop/")) + res.add(url); + } + + res.addAll(HadoopClasspathUtils.classpathForClassLoader()); + + X.println(">>> " + HadoopTestClassLoader.class.getSimpleName() + " static paths:"); + + for (URL url : res) + X.println(">>> \t" + url.toString()); + + URLS = res.toArray(new URL[res.size()]); + } + catch (Exception e) { + throw new IgniteException("Failed to initialize class loader JARs.", e); + } + } + + /** + * Constructor. + */ + public HadoopTestClassLoader() { + super(URLS, APP_CLS_LDR); + } + + /** {@inheritDoc} */ + @Override protected Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + if (HadoopClassLoader.loadByCurrentClassloader(name)) { + try { + synchronized (getClassLoadingLock(name)) { + // First, check if the class has already been loaded + Class c = findLoadedClass(name); + + if (c == null) { + long t1 = System.nanoTime(); + + c = findClass(name); + + // this is the defining class loader; record the stats + sun.misc.PerfCounter.getFindClassTime().addElapsedTimeFrom(t1); + sun.misc.PerfCounter.getFindClasses().increment(); + } + + if (resolve) + resolveClass(c); + + return c; + } + } + catch (NoClassDefFoundError | ClassNotFoundException e) { + throw new IgniteException("Failed to load class by test class loader: " + name, e); + } + } + + return super.loadClass(name, resolve); + } +} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWithoutHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWithoutHadoop.java deleted file mode 100644 index 93d659ccd9f09..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWithoutHadoop.java +++ /dev/null @@ -1,27 +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.hadoop.deps; - -/** - * Does not have direct Hadoop dependency, but has a circular - */ -@SuppressWarnings("unused") -public class CircularWithoutHadoop { - /** */ - private CircularWIthHadoop x; -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithCast.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithCast.java deleted file mode 100644 index 5b1e8e0e4abb7..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithCast.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.internal.processors.hadoop.deps; - -import org.apache.hadoop.fs.FileSystem; - -/** - * Class contains casting to a Hadoop type. - */ -@SuppressWarnings("unused") -public abstract class WithCast { - /** */ - public abstract T create(); - - /** */ - public void consume(T t) { - // noop - } - - /** */ - void test(WithCast c) { - FileSystem fs = c.create(); - - c.consume(fs); - } -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithConstructorInvocation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithConstructorInvocation.java deleted file mode 100644 index 98c8991d3cc47..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithConstructorInvocation.java +++ /dev/null @@ -1,31 +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.hadoop.deps; - -import org.apache.hadoop.conf.Configuration; - -/** - * Invokes a Hadoop type constructor. - */ -@SuppressWarnings("unused") -public class WithConstructorInvocation { - /** */ - private void foo() { - Object x = new Configuration(); - } -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithExtends.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithExtends.java deleted file mode 100644 index 80c99e1e2e769..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithExtends.java +++ /dev/null @@ -1,27 +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.hadoop.deps; - -import org.apache.hadoop.fs.LocalFileSystem; - -/** - * Class extends a Hadoop class. - */ -public class WithExtends extends LocalFileSystem { - // noop -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithField.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithField.java deleted file mode 100644 index dd979db104f63..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithField.java +++ /dev/null @@ -1,29 +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.hadoop.deps; - -import org.apache.hadoop.conf.Configuration; - -/** - * Has a Hadoop field. - */ -@SuppressWarnings("unused") -public class WithField { - /** */ - private Configuration conf; -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithIndirectField.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithIndirectField.java deleted file mode 100644 index ce078f19fc1a8..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithIndirectField.java +++ /dev/null @@ -1,27 +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.hadoop.deps; - -/** - * Has a unidirected dependency on Hadoop-dependent class. - */ -@SuppressWarnings("unused") -public class WithIndirectField { - /** */ - WithField x; -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInnerClass.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInnerClass.java deleted file mode 100644 index 4a5a49c70fcaf..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInnerClass.java +++ /dev/null @@ -1,31 +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.hadoop.deps; - -import org.apache.hadoop.conf.Configurable; - -/** - * Has a *static* inner class depending on Hadoop. - */ -@SuppressWarnings("unused") -public class WithInnerClass { - /** */ - private static abstract class Foo implements Configurable { - // No-op. - } -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithLocalVariable.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithLocalVariable.java deleted file mode 100644 index ea4a5de6b9e7a..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithLocalVariable.java +++ /dev/null @@ -1,38 +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.hadoop.deps; - -import org.apache.hadoop.conf.Configuration; - -/** - * Has a local variable of Hadoop type. - */ -@SuppressWarnings({"unused", "ConstantConditions"}) -public class WithLocalVariable { - /** */ - void foo() { - Configuration c = null; - - moo(c); - } - - /** */ - void moo(Object x) { - - } -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodAnnotation.java deleted file mode 100644 index ff9fbe0df8043..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodAnnotation.java +++ /dev/null @@ -1,32 +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.hadoop.deps; - -import org.apache.hadoop.classification.InterfaceStability; - -/** - * Method has a Hadoop annotation. - */ -@SuppressWarnings("unused") -public class WithMethodAnnotation { - /** */ - @InterfaceStability.Unstable - void foo() { - // No-op. - } -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodArgument.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodArgument.java deleted file mode 100644 index 7f639e41df1a9..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodArgument.java +++ /dev/null @@ -1,31 +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.hadoop.deps; - -import org.apache.hadoop.conf.Configuration; - -/** - * Contains a formal parameter of Hadoop type. - */ -@SuppressWarnings("unused") -public class WithMethodArgument { - /** */ - protected void paramaterMethod(Configuration c) { - // No-op. - } -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodCheckedException.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodCheckedException.java deleted file mode 100644 index 8fd12ae5cc86e..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodCheckedException.java +++ /dev/null @@ -1,31 +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.hadoop.deps; - -import org.apache.hadoop.fs.ChecksumException; - -/** - * Method declares a checked Hadoop Exception. - */ -@SuppressWarnings("unused") -public class WithMethodCheckedException { - /** */ - void foo() throws ChecksumException { - // No-op. - } -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodInvocation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodInvocation.java deleted file mode 100644 index de8b306e4ac27..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodInvocation.java +++ /dev/null @@ -1,31 +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.hadoop.deps; - -import org.apache.hadoop.fs.FileSystem; - -/** - * Method contains a Hadoop type method invocation. - */ -@SuppressWarnings("unused") -public class WithMethodInvocation { - /** */ - void foo(FileSystem fs) { - fs.getChildFileSystems(); - } -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodReturnType.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodReturnType.java deleted file mode 100644 index 0e0ea7282463b..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodReturnType.java +++ /dev/null @@ -1,31 +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.hadoop.deps; - -import org.apache.hadoop.fs.FileSystem; - -/** - * Contains a method return value of Hadoop type. - */ -@SuppressWarnings("unused") -public class WithMethodReturnType { - /** */ - FileSystem fsMethod() { - return null; - } -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodRuntimeException.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodRuntimeException.java deleted file mode 100644 index dcd471c95d67f..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodRuntimeException.java +++ /dev/null @@ -1,31 +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.hadoop.deps; - -import org.apache.hadoop.HadoopIllegalArgumentException; - -/** - * Method declares a runtime Hadoop Exception. - */ -@SuppressWarnings("unused") -public class WithMethodRuntimeException { - /** */ - void foo() throws HadoopIllegalArgumentException { - // No-op. - } -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithParameterAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithParameterAnnotation.java deleted file mode 100644 index 9d3414e010857..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithParameterAnnotation.java +++ /dev/null @@ -1,31 +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.hadoop.deps; - -import org.apache.hadoop.classification.InterfaceStability; - -/** - * Has a paramater annotated with a Hadoop annotation. - */ -@SuppressWarnings("unused") -public class WithParameterAnnotation { - /** */ - void foo(@InterfaceStability.Stable Object annotatedParam) { - // No-op. - } -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticField.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticField.java deleted file mode 100644 index 301b9129b4d8a..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticField.java +++ /dev/null @@ -1,29 +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.hadoop.deps; - -import org.apache.hadoop.fs.FileSystem; - -/** - * Has a static field of Hadoop type. - */ -@SuppressWarnings("unused") -public class WithStaticField { - /** */ - static FileSystem fs; -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticInitializer.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticInitializer.java deleted file mode 100644 index e0fc2f3578fb5..0000000000000 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticInitializer.java +++ /dev/null @@ -1,34 +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.hadoop.deps; - -import java.util.List; -import org.apache.hadoop.fs.FileSystem; - -/** - * Uses Hadoop type in a static initializer. - */ -@SuppressWarnings("unused") -public class WithStaticInitializer { - /** */ - static final List x; - - static { - x = FileSystem.getAllStatistics(); - } -} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractMapReduceTest.java similarity index 96% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractMapReduceTest.java index 37312133ea5ac..89005f60b44e4 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractMapReduceTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractMapReduceTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.io.BufferedReader; import java.io.IOException; @@ -49,10 +49,12 @@ import org.apache.ignite.igfs.IgfsUserContext; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters; import org.apache.ignite.internal.processors.hadoop.counter.HadoopPerformanceCounter; -import org.apache.ignite.internal.processors.hadoop.examples.HadoopWordCount1; -import org.apache.ignite.internal.processors.hadoop.examples.HadoopWordCount2; +import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount1; +import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount2; import org.apache.ignite.internal.processors.igfs.IgfsEx; import org.apache.ignite.internal.processors.igfs.IgfsUtils; import org.apache.ignite.internal.util.lang.GridAbsPredicate; @@ -68,8 +70,7 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED; import static org.apache.ignite.cache.CacheMode.REPLICATED; import static org.apache.ignite.igfs.IgfsMode.PRIMARY; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.JOB_COUNTER_WRITER_PROPERTY; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.createJobInfo; +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.createJobInfo; /** * Abstract test of whole cycle of map-reduce processing via Job tracker. @@ -175,7 +176,7 @@ protected final void doTest(IgfsPath inFile, boolean useNewMapper, boolean useNe JobConf jobConf = new JobConf(); - jobConf.set(JOB_COUNTER_WRITER_PROPERTY, IgniteHadoopFileSystemCounterWriter.class.getName()); + jobConf.set(HadoopCommonUtils.JOB_COUNTER_WRITER_PROPERTY, IgniteHadoopFileSystemCounterWriter.class.getName()); jobConf.setUser(USER); jobConf.set(IgniteHadoopFileSystemCounterWriter.COUNTER_WRITER_DIR_PROPERTY, "/xxx/${USER}/zzz"); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractSelfTest.java similarity index 98% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractSelfTest.java index fb1698885ea28..68009dd7d3949 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.io.File; import org.apache.hadoop.conf.Configuration; @@ -28,7 +28,7 @@ import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; import org.apache.ignite.igfs.IgfsIpcEndpointType; -import org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemsUtils; +import org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSystemsUtils; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractWordCountTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractWordCountTest.java similarity index 98% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractWordCountTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractWordCountTest.java index e45c1275905a5..3cb8f914db5f5 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractWordCountTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractWordCountTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import com.google.common.base.Joiner; import java.io.BufferedReader; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopCommandLineTest.java similarity index 98% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopCommandLineTest.java index 7ee318a283f9e..0be8bf9ac663b 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopCommandLineTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import com.google.common.base.Joiner; import java.io.BufferedReader; @@ -37,6 +37,8 @@ import org.apache.ignite.igfs.IgfsInputStream; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.IgnitionEx; +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; +import org.apache.ignite.internal.processors.hadoop.HadoopJob; import org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker; import org.apache.ignite.internal.processors.igfs.IgfsEx; import org.apache.ignite.internal.processors.resource.GridSpringResourceContext; @@ -183,7 +185,7 @@ private void generateHiveTestFiles() throws FileNotFoundException { if (line.startsWith("")) out.println( " \n" + - " " + HadoopUtils.JOB_COUNTER_WRITER_PROPERTY + "\n" + + " " + HadoopCommonUtils.JOB_COUNTER_WRITER_PROPERTY + "\n" + " " + IgniteHadoopFileSystemCounterWriter.class.getName() + "\n" + " \n"); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopDefaultMapReducePlannerSelfTest.java similarity index 98% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopDefaultMapReducePlannerSelfTest.java index a69b72a161567..ee1c88f05ef97 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopDefaultMapReducePlannerSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteFileSystem; @@ -23,6 +23,10 @@ import org.apache.ignite.hadoop.mapreduce.IgniteHadoopMapReducePlanner; import org.apache.ignite.igfs.IgfsBlockLocation; import org.apache.ignite.igfs.IgfsPath; +import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock; +import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit; +import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan; +import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlanner; import org.apache.ignite.internal.processors.hadoop.planner.HadoopAbstractMapReducePlanner; import org.apache.ignite.internal.processors.igfs.IgfsBlockLocationImpl; import org.apache.ignite.internal.processors.igfs.IgfsIgniteMock; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopErrorSimulator.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopErrorSimulator.java similarity index 99% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopErrorSimulator.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopErrorSimulator.java index 843b42ba540d8..b89dcc124eca3 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopErrorSimulator.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopErrorSimulator.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.io.IOException; import java.util.concurrent.atomic.AtomicReference; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopFileSystemsTest.java similarity index 97% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopFileSystemsTest.java index 946ba77d63313..252d6cb4af907 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopFileSystemsTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.io.IOException; import java.net.URI; @@ -25,7 +25,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.MRJobConfig; -import org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemsUtils; +import org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSystemsUtils; import org.apache.ignite.testframework.GridTestUtils; /** diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopGroupingTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopGroupingTest.java similarity index 93% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopGroupingTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopGroupingTest.java index db87e331e4dfc..19c71e84e9778 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopGroupingTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopGroupingTest.java @@ -15,17 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Random; -import java.util.Set; -import java.util.UUID; import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableComparable; @@ -41,24 +32,28 @@ import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.ignite.configuration.HadoopConfiguration; -import org.apache.ignite.internal.util.GridConcurrentHashSet; +import org.apache.ignite.internal.processors.hadoop.state.HadoopGroupingTestState; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; import org.apache.ignite.internal.util.GridRandom; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.S; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.createJobInfo; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.UUID; + +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.createJobInfo; /** * Grouping test. */ public class HadoopGroupingTest extends HadoopAbstractSelfTest { - /** */ - private static final String PATH_OUTPUT = "/test-out"; - - /** */ - private static final GridConcurrentHashSet vals = HadoopSharedMap.map(HadoopGroupingTest.class) - .put("vals", new GridConcurrentHashSet()); - /** {@inheritDoc} */ @Override protected int gridCount() { return 3; @@ -108,7 +103,7 @@ public void testGroupingCombiner() throws Exception { * @throws Exception If failed. */ public void doTestGrouping(boolean combiner) throws Exception { - vals.clear(); + HadoopGroupingTestState.values().clear(); Job job = Job.getInstance(); @@ -134,7 +129,7 @@ public void doTestGrouping(boolean combiner) throws Exception { grid(0).hadoop().submit(new HadoopJobId(UUID.randomUUID(), 2), createJobInfo(job.getConfiguration())).get(30000); - assertTrue(vals.isEmpty()); + assertTrue(HadoopGroupingTestState.values().isEmpty()); } public static class MyReducer extends Reducer { @@ -160,7 +155,7 @@ public static class MyReducer extends Reducer { @Override public Text getCurrentValue() { UUID id = UUID.randomUUID(); - assertTrue(vals.add(id)); + assertTrue(HadoopGroupingTestState.values().add(id)); val.set(id.toString()); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopJobTrackerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopJobTrackerSelfTest.java similarity index 92% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopJobTrackerSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopJobTrackerSelfTest.java index 9e268b7771e21..a3bf49c59c22f 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopJobTrackerSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopJobTrackerSelfTest.java @@ -15,18 +15,15 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.UUID; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; @@ -41,9 +38,16 @@ import org.apache.ignite.configuration.HadoopConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.processors.hadoop.Hadoop; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; +import org.apache.ignite.internal.processors.hadoop.HadoopJobStatus; import org.apache.ignite.internal.util.typedef.internal.U; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.createJobInfo; +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.createJobInfo; +import static org.apache.ignite.internal.processors.hadoop.state.HadoopJobTrackerSelfTestState.combineExecCnt; +import static org.apache.ignite.internal.processors.hadoop.state.HadoopJobTrackerSelfTestState.latch; +import static org.apache.ignite.internal.processors.hadoop.state.HadoopJobTrackerSelfTestState.mapExecCnt; +import static org.apache.ignite.internal.processors.hadoop.state.HadoopJobTrackerSelfTestState.reduceExecCnt; /** * Job tracker self test. @@ -55,21 +59,6 @@ public class HadoopJobTrackerSelfTest extends HadoopAbstractSelfTest { /** Test block count parameter name. */ private static final int BLOCK_CNT = 10; - /** */ - private static HadoopSharedMap m = HadoopSharedMap.map(HadoopJobTrackerSelfTest.class); - - /** Map task execution count. */ - private static final AtomicInteger mapExecCnt = m.put("mapExecCnt", new AtomicInteger()); - - /** Reduce task execution count. */ - private static final AtomicInteger reduceExecCnt = m.put("reduceExecCnt", new AtomicInteger()); - - /** Reduce task execution count. */ - private static final AtomicInteger combineExecCnt = m.put("combineExecCnt", new AtomicInteger()); - - /** */ - private static final Map latch = m.put("latch", new HashMap()); - /** {@inheritDoc} */ @Override protected boolean igfsEnabled() { return true; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceEmbeddedSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceEmbeddedSelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceEmbeddedSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceEmbeddedSelfTest.java index 25ef38298e1ef..b04deeb8e4017 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceEmbeddedSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceEmbeddedSelfTest.java @@ -15,10 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; -import java.util.HashMap; -import java.util.Map; import java.util.UUID; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; @@ -35,19 +33,17 @@ import org.apache.ignite.configuration.HadoopConfiguration; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.processors.hadoop.examples.HadoopWordCount1; -import org.apache.ignite.internal.processors.hadoop.examples.HadoopWordCount2; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; +import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount1; +import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount2; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.createJobInfo; +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.createJobInfo; +import static org.apache.ignite.internal.processors.hadoop.state.HadoopMapReduceEmbeddedSelfTestState.flags; /** * Tests map-reduce execution with embedded mode. */ public class HadoopMapReduceEmbeddedSelfTest extends HadoopMapReduceTest { - /** */ - private static Map flags = HadoopSharedMap.map(HadoopMapReduceEmbeddedSelfTest.class) - .put("flags", new HashMap()); - /** {@inheritDoc} */ @Override public HadoopConfiguration hadoopConfiguration(String gridName) { HadoopConfiguration cfg = super.hadoopConfiguration(gridName); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceErrorResilienceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceErrorResilienceTest.java similarity index 97% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceErrorResilienceTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceErrorResilienceTest.java index dd12935d63bd4..afd6f26d92e69 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceErrorResilienceTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceErrorResilienceTest.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import org.apache.ignite.igfs.IgfsPath; -import org.apache.ignite.internal.processors.hadoop.examples.HadoopWordCount2; +import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount2; /** * Test of error resiliency after an error in a map-reduce job execution. diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceTest.java index b7038965f041c..feccb59193001 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceTest.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import org.apache.ignite.igfs.IgfsPath; -import org.apache.ignite.internal.processors.hadoop.examples.HadoopWordCount2; +import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount2; /** * Test of whole cycle of map-reduce processing via Job tracker. diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopNoHadoopMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopNoHadoopMapReduceTest.java similarity index 96% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopNoHadoopMapReduceTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopNoHadoopMapReduceTest.java index 0c172c37cf3bb..3bb873521e036 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopNoHadoopMapReduceTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopNoHadoopMapReduceTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import org.apache.ignite.configuration.IgniteConfiguration; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopPlannerMockJob.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPlannerMockJob.java similarity index 87% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopPlannerMockJob.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPlannerMockJob.java index 88d0f80f4e440..220614c88244f 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopPlannerMockJob.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPlannerMockJob.java @@ -15,10 +15,17 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.hadoop.HadoopHelper; +import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit; +import org.apache.ignite.internal.processors.hadoop.HadoopJob; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; +import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; import org.jetbrains.annotations.Nullable; import java.util.Collection; @@ -145,7 +152,7 @@ public JobInfo(int reducers) { /** {@inheritDoc} */ @Override public HadoopJob createJob(Class jobCls, HadoopJobId jobId, IgniteLogger log, - @Nullable String[] libNames) throws IgniteCheckedException { + @Nullable String[] libNames, HadoopHelper helper) throws IgniteCheckedException { throwUnsupported(); return null; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopPopularWordsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPopularWordsTest.java similarity index 99% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopPopularWordsTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPopularWordsTest.java index 3f825b0c83a50..5a55430be1b4e 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopPopularWordsTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPopularWordsTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import com.google.common.collect.MinMaxPriorityQueue; import java.io.IOException; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSerializationWrapperSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSerializationWrapperSelfTest.java similarity index 93% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSerializationWrapperSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSerializationWrapperSelfTest.java index 789a6b3556375..5ccc8cea50115 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSerializationWrapperSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSerializationWrapperSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -27,7 +27,8 @@ import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.serializer.JavaSerialization; import org.apache.hadoop.io.serializer.WritableSerialization; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopSerializationWrapper; +import org.apache.ignite.internal.processors.hadoop.HadoopSerialization; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopSerializationWrapper; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSnappyFullMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSnappyFullMapReduceTest.java similarity index 95% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSnappyFullMapReduceTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSnappyFullMapReduceTest.java index 27a5fcdebe7ae..e27c212d1952d 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSnappyFullMapReduceTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSnappyFullMapReduceTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; /** * Same test as HadoopMapReduceTest, but with enabled Snappy output compression. diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSnappyTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSnappyTest.java similarity index 93% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSnappyTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSnappyTest.java index b4e3dc2bc0f26..80ff7547e2988 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSnappyTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSnappyTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -28,6 +28,8 @@ import org.apache.hadoop.io.compress.SnappyCodec; import org.apache.hadoop.io.compress.snappy.SnappyCompressor; import org.apache.hadoop.util.NativeCodeLoader; +import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader; +import org.apache.ignite.internal.processors.hadoop.HadoopHelperImpl; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -49,7 +51,7 @@ public void testSnappy() throws Throwable { // Run the same in several more class loaders simulating jobs and tasks: for (int i = 0; i < 2; i++) { - ClassLoader hadoopClsLdr = new HadoopClassLoader(null, "cl-" + i, null); + ClassLoader hadoopClsLdr = new HadoopClassLoader(null, "cl-" + i, null, new HadoopHelperImpl()); Class cls = (Class)Class.forName(HadoopSnappyTest.class.getName(), true, hadoopClsLdr); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSortingExternalTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSortingExternalTest.java similarity index 96% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSortingExternalTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSortingExternalTest.java index dff5e7008ec2c..eb4a7d425112d 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSortingExternalTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSortingExternalTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import org.apache.ignite.configuration.HadoopConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSortingTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSortingTest.java similarity index 97% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSortingTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSortingTest.java index 20f5eef3f3d26..a4e736804743c 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSortingTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSortingTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.io.DataInput; import java.io.DataOutput; @@ -48,9 +48,10 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.ignite.configuration.HadoopConfiguration; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; import org.apache.ignite.internal.util.typedef.X; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.createJobInfo; +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.createJobInfo; /** * Tests correct sorting. diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapperSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSplitWrapperSelfTest.java similarity index 91% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapperSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSplitWrapperSelfTest.java index 11c3907e03992..be2bfc2413727 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapperSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopSplitWrapperSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -27,11 +27,11 @@ import java.util.concurrent.Callable; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.lib.input.FileSplit; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopSplitWrapper; +import org.apache.ignite.internal.processors.hadoop.HadoopSplitWrapper; import org.apache.ignite.testframework.GridTestUtils; /** - * Self test of {@link org.apache.ignite.internal.processors.hadoop.v2.HadoopSplitWrapper}. + * Self test of {@link HadoopSplitWrapper}. */ public class HadoopSplitWrapperSelfTest extends HadoopAbstractSelfTest { /** diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopStartup.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopStartup.java similarity index 96% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopStartup.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopStartup.java index 820a1f3e745f1..66e341b60f2e4 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopStartup.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopStartup.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import org.apache.hadoop.conf.Configuration; import org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTaskExecutionSelfTest.java similarity index 93% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTaskExecutionSelfTest.java index 431433e3e3444..027f921f0d83b 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTaskExecutionSelfTest.java @@ -15,18 +15,15 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.io.IOException; import java.io.OutputStream; import java.io.OutputStreamWriter; import java.io.PrintWriter; import java.util.List; -import java.util.Map; import java.util.UUID; import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalFileSystem; @@ -48,43 +45,29 @@ import org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.hadoop.Hadoop; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException; import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.createJobInfo; +import static org.apache.ignite.internal.processors.hadoop.state.HadoopTaskExecutionSelfTestValues.cancelledTasks; +import static org.apache.ignite.internal.processors.hadoop.state.HadoopTaskExecutionSelfTestValues.executedTasks; +import static org.apache.ignite.internal.processors.hadoop.state.HadoopTaskExecutionSelfTestValues.failMapperId; +import static org.apache.ignite.internal.processors.hadoop.state.HadoopTaskExecutionSelfTestValues.splitsCount; +import static org.apache.ignite.internal.processors.hadoop.state.HadoopTaskExecutionSelfTestValues.taskWorkDirs; +import static org.apache.ignite.internal.processors.hadoop.state.HadoopTaskExecutionSelfTestValues.totalLineCnt; +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.createJobInfo; /** * Tests map-reduce task execution basics. */ public class HadoopTaskExecutionSelfTest extends HadoopAbstractSelfTest { - /** */ - private static HadoopSharedMap m = HadoopSharedMap.map(HadoopTaskExecutionSelfTest.class); - - /** Line count. */ - private static final AtomicInteger totalLineCnt = m.put("totalLineCnt", new AtomicInteger()); - - /** Executed tasks. */ - private static final AtomicInteger executedTasks = m.put("executedTasks", new AtomicInteger()); - - /** Cancelled tasks. */ - private static final AtomicInteger cancelledTasks = m.put("cancelledTasks", new AtomicInteger()); - - /** Working directory of each task. */ - private static final Map taskWorkDirs = m.put("taskWorkDirs", - new ConcurrentHashMap()); - - /** Mapper id to fail. */ - private static final AtomicInteger failMapperId = m.put("failMapperId", new AtomicInteger()); - - /** Number of splits of the current input. */ - private static final AtomicInteger splitsCount = m.put("splitsCount", new AtomicInteger()); - /** Test param. */ private static final String MAP_WRITE = "test.map.write"; - /** {@inheritDoc} */ @Override public FileSystemConfiguration igfsConfiguration() throws Exception { FileSystemConfiguration cfg = super.igfsConfiguration(); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksAllVersionsTest.java similarity index 95% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksAllVersionsTest.java index 7c6d244819224..8b1b6935301e0 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksAllVersionsTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import com.google.common.base.Joiner; import java.io.IOException; @@ -27,7 +27,11 @@ import org.apache.hadoop.io.Text; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.igfs.IgfsPath; -import org.apache.ignite.internal.processors.hadoop.examples.HadoopWordCount2; +import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock; +import org.apache.ignite.internal.processors.hadoop.HadoopJob; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskType; +import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount2; /** * Tests of Map, Combine and Reduce task executions of any version of hadoop API. diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV1Test.java similarity index 75% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV1Test.java index 27d7fc229fa03..d7cd7388f0d9d 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV1Test.java @@ -15,15 +15,19 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.io.IOException; import java.util.UUID; import org.apache.hadoop.mapred.JobConf; -import org.apache.ignite.internal.processors.hadoop.examples.HadoopWordCount1; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopV2Job; +import org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopJob; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; +import org.apache.ignite.internal.processors.hadoop.HadoopHelperImpl; +import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount1; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopV2Job; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.createJobInfo; +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.createJobInfo; /** * Tests of Map, Combine and Reduce task executions via running of job of hadoop API v1. @@ -48,7 +52,7 @@ public class HadoopTasksV1Test extends HadoopTasksAllVersionsTest { HadoopJobId jobId = new HadoopJobId(uuid, 0); - return jobInfo.createJob(HadoopV2Job.class, jobId, log, null); + return jobInfo.createJob(HadoopV2Job.class, jobId, log, null, new HadoopHelperImpl()); } /** {@inheritDoc} */ diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV2Test.java similarity index 80% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV2Test.java index 30cf50c85301c..c635c41bff85c 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV2Test.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.util.UUID; import org.apache.hadoop.conf.Configuration; @@ -25,10 +25,14 @@ import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; -import org.apache.ignite.internal.processors.hadoop.examples.HadoopWordCount2; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopV2Job; +import org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopJob; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; +import org.apache.ignite.internal.processors.hadoop.HadoopHelperImpl; +import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount2; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopV2Job; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.createJobInfo; +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.createJobInfo; /** * Tests of Map, Combine and Reduce task executions via running of job of hadoop API v2. @@ -67,7 +71,7 @@ public class HadoopTasksV2Test extends HadoopTasksAllVersionsTest { HadoopJobId jobId = new HadoopJobId(uuid, 0); - return jobInfo.createJob(HadoopV2Job.class, jobId, log, null); + return jobInfo.createJob(HadoopV2Job.class, jobId, log, null, new HadoopHelperImpl()); } /** {@inheritDoc} */ diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestRoundRobinMrPlanner.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestRoundRobinMrPlanner.java similarity index 87% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestRoundRobinMrPlanner.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestRoundRobinMrPlanner.java index edafecd4c60d8..81f6f3c866e5e 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestRoundRobinMrPlanner.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestRoundRobinMrPlanner.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.util.ArrayList; import java.util.Collection; @@ -26,6 +26,10 @@ import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit; +import org.apache.ignite.internal.processors.hadoop.HadoopJob; +import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan; +import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlanner; import org.apache.ignite.internal.processors.hadoop.planner.HadoopDefaultMapReducePlan; import org.jetbrains.annotations.Nullable; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestTaskContext.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestTaskContext.java similarity index 92% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestTaskContext.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestTaskContext.java index f542cf2c7c1ba..cfd41cf7b420b 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestTaskContext.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestTaskContext.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -32,7 +32,12 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.JobConf; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopV2TaskContext; +import org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopJob; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskOutput; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopV2TaskContext; /** * Context for test purpose. @@ -168,7 +173,7 @@ public Map mockInput() { public void makeTreeOfWritables(Iterable> flatData) { Text key = new Text(); - for (HadoopTestTaskContext.Pair pair : flatData) { + for (Pair pair : flatData) { key.set(pair.key); ArrayList valList; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestUtils.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestUtils.java similarity index 98% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestUtils.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestUtils.java index da0d922a988f0..e8ec8a9a2d24f 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestUtils.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestUtils.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/cache/HadoopTxConfigCacheTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTxConfigCacheTest.java similarity index 96% rename from modules/hadoop/src/test/java/org/apache/ignite/hadoop/cache/HadoopTxConfigCacheTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTxConfigCacheTest.java index 6f910f1cf10ff..e85baed8404b6 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/cache/HadoopTxConfigCacheTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTxConfigCacheTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.hadoop.cache; +package org.apache.ignite.internal.processors.hadoop.impl; import org.apache.ignite.Ignite; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopUserLibsSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUserLibsSelfTest.java similarity index 98% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopUserLibsSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUserLibsSelfTest.java index 9e3c8f402917a..0e4a0ef1d1a82 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopUserLibsSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUserLibsSelfTest.java @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; +import org.apache.ignite.internal.processors.hadoop.HadoopClasspathUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import java.io.File; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopV2JobSelfTest.java similarity index 79% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopV2JobSelfTest.java index ae2c00d412499..540a7aaea3305 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopV2JobSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import java.io.ByteArrayInputStream; import java.io.DataInput; @@ -30,13 +30,21 @@ import org.apache.hadoop.io.serializer.WritableSerialization; import org.apache.hadoop.mapred.JobConf; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopSerializationWrapper; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopV2Job; - -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.createJobInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopJob; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; +import org.apache.ignite.internal.processors.hadoop.HadoopSerialization; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskType; +import org.apache.ignite.internal.processors.hadoop.HadoopHelperImpl; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopSerializationWrapper; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopV2Job; + +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.createJobInfo; /** - * Self test of {@link org.apache.ignite.internal.processors.hadoop.v2.HadoopV2Job}. + * Self test of {@link org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopV2Job}. */ public class HadoopV2JobSelfTest extends HadoopAbstractSelfTest { /** */ @@ -78,7 +86,7 @@ public void testCustomSerializationApplying() throws IgniteCheckedException { HadoopJobId id = new HadoopJobId(uuid, 1); - HadoopJob job = info.createJob(HadoopV2Job.class, id, log, null); + HadoopJob job = info.createJob(HadoopV2Job.class, id, log, null, new HadoopHelperImpl()); HadoopTaskContext taskCtx = job.getTaskContext(new HadoopTaskInfo(HadoopTaskType.MAP, null, 0, 0, null)); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopValidationSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopValidationSelfTest.java similarity index 96% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopValidationSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopValidationSelfTest.java index 14961509f1790..2d61016290400 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopValidationSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopValidationSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import org.apache.ignite.configuration.IgniteConfiguration; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopWeightedMapReducePlannerTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopWeightedMapReducePlannerTest.java similarity index 98% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopWeightedMapReducePlannerTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopWeightedMapReducePlannerTest.java index 4e7cc50c86b1e..430c6757e2456 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopWeightedMapReducePlannerTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopWeightedMapReducePlannerTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.cluster.ClusterNode; @@ -23,6 +23,9 @@ import org.apache.ignite.igfs.IgfsBlockLocation; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.IgniteNodeAttributes; +import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock; +import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit; +import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan; import org.apache.ignite.internal.processors.hadoop.planner.HadoopAbstractMapReducePlanner; import org.apache.ignite.internal.processors.igfs.IgfsIgniteMock; import org.apache.ignite.internal.processors.igfs.IgfsMock; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopWeightedPlannerMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopWeightedPlannerMapReduceTest.java similarity index 96% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopWeightedPlannerMapReduceTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopWeightedPlannerMapReduceTest.java index e0403c28d3d41..13f00bdbd42b8 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopWeightedPlannerMapReduceTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopWeightedPlannerMapReduceTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl; import org.apache.ignite.configuration.HadoopConfiguration; import org.apache.ignite.hadoop.mapreduce.IgniteHadoopWeightedMapReducePlanner; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/books/alice-in-wonderland.txt b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/books/alice-in-wonderland.txt similarity index 100% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/books/alice-in-wonderland.txt rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/books/alice-in-wonderland.txt diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/books/art-of-war.txt b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/books/art-of-war.txt similarity index 100% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/books/art-of-war.txt rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/books/art-of-war.txt diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/books/huckleberry-finn.txt b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/books/huckleberry-finn.txt similarity index 100% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/books/huckleberry-finn.txt rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/books/huckleberry-finn.txt diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/books/sherlock-holmes.txt b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/books/sherlock-holmes.txt similarity index 100% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/books/sherlock-holmes.txt rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/books/sherlock-holmes.txt diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/books/tom-sawyer.txt b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/books/tom-sawyer.txt similarity index 100% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/books/tom-sawyer.txt rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/books/tom-sawyer.txt diff --git a/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolEmbeddedSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolEmbeddedSelfTest.java similarity index 95% rename from modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolEmbeddedSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolEmbeddedSelfTest.java index 5a20a754fb7ce..a65d691994f0a 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolEmbeddedSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolEmbeddedSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.client.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl.client; import org.apache.ignite.configuration.HadoopConfiguration; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolSelfTest.java similarity index 98% rename from modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolSelfTest.java index 1344e261afe96..1ef7dd0a5cca9 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.client.hadoop; +package org.apache.ignite.internal.processors.hadoop.impl.client; import java.io.BufferedReader; import java.io.BufferedWriter; @@ -50,8 +50,9 @@ import org.apache.ignite.hadoop.mapreduce.IgniteHadoopClientProtocolProvider; import org.apache.ignite.igfs.IgfsFile; import org.apache.ignite.igfs.IgfsPath; -import org.apache.ignite.internal.processors.hadoop.HadoopAbstractSelfTest; -import org.apache.ignite.internal.processors.hadoop.HadoopUtils; +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopAbstractSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils; import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; @@ -114,7 +115,6 @@ public class HadoopClientProtocolSelfTest extends HadoopAbstractSelfTest { stopAllGrids(); super.afterTestsStopped(); - // IgniteHadoopClientProtocolProvider.cliMap.clear(); } diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount1.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount1.java similarity index 97% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount1.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount1.java index a2faf954c16f6..0df9c6a437732 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount1.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount1.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.examples; +package org.apache.ignite.internal.processors.hadoop.impl.examples; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IntWritable; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount1Map.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount1Map.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount1Map.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount1Map.java index d4cd190bcf059..6a98a244bf743 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount1Map.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount1Map.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.examples; +package org.apache.ignite.internal.processors.hadoop.impl.examples; import java.io.IOException; import java.util.StringTokenizer; @@ -27,7 +27,7 @@ import org.apache.hadoop.mapred.Mapper; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; -import org.apache.ignite.internal.processors.hadoop.HadoopErrorSimulator; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopErrorSimulator; /** * Mapper phase of WordCount job. diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount1Reduce.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount1Reduce.java similarity index 93% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount1Reduce.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount1Reduce.java index b400d9b222344..ab91e0c438b5e 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount1Reduce.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount1Reduce.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.examples; +package org.apache.ignite.internal.processors.hadoop.impl.examples; import java.io.IOException; import java.util.Iterator; @@ -26,7 +26,7 @@ import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reducer; import org.apache.hadoop.mapred.Reporter; -import org.apache.ignite.internal.processors.hadoop.HadoopErrorSimulator; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopErrorSimulator; /** * Combiner and Reducer phase of WordCount job. diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount2.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount2.java similarity index 98% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount2.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount2.java index b2cfee334f9c4..3ddc923cdefcf 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount2.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount2.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.examples; +package org.apache.ignite.internal.processors.hadoop.impl.examples; import java.io.IOException; import org.apache.hadoop.fs.Path; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount2Combiner.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount2Combiner.java similarity index 91% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount2Combiner.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount2Combiner.java index 0d25e3c20d349..a643a924918ab 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount2Combiner.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount2Combiner.java @@ -14,10 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.examples; +package org.apache.ignite.internal.processors.hadoop.impl.examples; import java.io.IOException; -import org.apache.ignite.internal.processors.hadoop.HadoopErrorSimulator; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopErrorSimulator; /** * Combiner function with pluggable error simulator. diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount2Mapper.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount2Mapper.java similarity index 95% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount2Mapper.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount2Mapper.java index 76857e6e2ffaf..336db8410b8b2 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount2Mapper.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount2Mapper.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.examples; +package org.apache.ignite.internal.processors.hadoop.impl.examples; import java.io.IOException; import java.util.StringTokenizer; @@ -24,7 +24,7 @@ import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Mapper; -import org.apache.ignite.internal.processors.hadoop.HadoopErrorSimulator; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopErrorSimulator; /** * Mapper phase of WordCount job. diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount2Reducer.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount2Reducer.java similarity index 95% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount2Reducer.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount2Reducer.java index e780170e9caee..f24288ebe1bb0 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/examples/HadoopWordCount2Reducer.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/examples/HadoopWordCount2Reducer.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.examples; +package org.apache.ignite.internal.processors.hadoop.impl.examples; import java.io.IOException; import org.apache.hadoop.conf.Configurable; @@ -23,7 +23,7 @@ import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Reducer; -import org.apache.ignite.internal.processors.hadoop.HadoopErrorSimulator; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopErrorSimulator; /** * Combiner and Reducer phase of WordCount job. diff --git a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactorySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java similarity index 90% rename from modules/hadoop/src/test/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactorySelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java index ea7fa996bc80a..8c95a0ea57d51 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/fs/KerberosHadoopFileSystemFactorySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.hadoop.fs; +package org.apache.ignite.internal.processors.hadoop.impl.fs; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -25,6 +25,9 @@ import java.io.ObjectOutputStream; import java.util.concurrent.Callable; +import org.apache.ignite.hadoop.fs.KerberosHadoopFileSystemFactory; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Assert; @@ -67,7 +70,9 @@ private void checkParameters(String keyTab, String keyTabPrincipal, long relogin GridTestUtils.assertThrows(null, new Callable() { @Override public Object call() throws Exception { - fac.start(); + HadoopFileSystemFactoryDelegate delegate = HadoopDelegateUtils.fileSystemFactoryDelegate(fac); + + delegate.start(); return null; } diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/Hadoop1DualAbstractTest.java similarity index 88% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/Hadoop1DualAbstractTest.java index 2c25a065276ba..a585e544d72d9 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/Hadoop1DualAbstractTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import java.io.IOException; import org.apache.hadoop.conf.Configuration; @@ -25,16 +25,15 @@ import org.apache.ignite.hadoop.util.ChainedUserNameMapper; import org.apache.ignite.hadoop.util.KerberosUserNameMapper; import org.apache.ignite.hadoop.util.UserNameMapper; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; +import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; import org.apache.ignite.internal.processors.igfs.IgfsDualAbstractSelfTest; import org.apache.ignite.lifecycle.LifecycleAware; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.IgniteFileSystem.IGFS_SCHEME; -import static org.apache.ignite.igfs.HadoopSecondaryFileSystemConfigurationTest.SECONDARY_CFG_PATH; -import static org.apache.ignite.igfs.HadoopSecondaryFileSystemConfigurationTest.configuration; -import static org.apache.ignite.igfs.HadoopSecondaryFileSystemConfigurationTest.mkUri; -import static org.apache.ignite.igfs.HadoopSecondaryFileSystemConfigurationTest.writeConfiguration; import static org.apache.ignite.igfs.IgfsMode.PRIMARY; /** @@ -118,13 +117,13 @@ protected void startUnderlying() throws Exception { * @throws IOException On failure. */ protected void prepareConfiguration() throws IOException { - Configuration secondaryConf = configuration(IGFS_SCHEME, SECONDARY_AUTHORITY, true, true); + Configuration secondaryConf = HadoopSecondaryFileSystemConfigurationTest.configuration(IGFS_SCHEME, SECONDARY_AUTHORITY, true, true); secondaryConf.setInt("fs.igfs.block.size", 1024); - secondaryConfFullPath = writeConfiguration(secondaryConf, SECONDARY_CFG_PATH); + secondaryConfFullPath = HadoopSecondaryFileSystemConfigurationTest.writeConfiguration(secondaryConf, HadoopSecondaryFileSystemConfigurationTest.SECONDARY_CFG_PATH); - secondaryUri = mkUri(IGFS_SCHEME, SECONDARY_AUTHORITY); + secondaryUri = HadoopSecondaryFileSystemConfigurationTest.mkUri(IGFS_SCHEME, SECONDARY_AUTHORITY); } /** diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1OverIgfsDualAsyncTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/Hadoop1OverIgfsDualAsyncTest.java similarity index 90% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1OverIgfsDualAsyncTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/Hadoop1OverIgfsDualAsyncTest.java index bbf12232aaa2a..97cc7e95cb033 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1OverIgfsDualAsyncTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/Hadoop1OverIgfsDualAsyncTest.java @@ -15,7 +15,9 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; + +import org.apache.ignite.igfs.IgfsMode; /** * DUAL_ASYNC mode test. diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1OverIgfsDualSyncTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/Hadoop1OverIgfsDualSyncTest.java similarity index 90% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1OverIgfsDualSyncTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/Hadoop1OverIgfsDualSyncTest.java index c57415cd9aced..12036bc6f234f 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1OverIgfsDualSyncTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/Hadoop1OverIgfsDualSyncTest.java @@ -15,7 +15,9 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; + +import org.apache.ignite.igfs.IgfsMode; /** * DUAL_SYNC mode. diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFIleSystemFactorySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java similarity index 83% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFIleSystemFactorySelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java index 5be3a6413c8ee..7cf7f2dcaa1f2 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFIleSystemFactorySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -26,20 +26,29 @@ import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory; +import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory; import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem; import org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem; +import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; +import org.apache.ignite.igfs.IgfsMode; +import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate; import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest; import org.apache.ignite.internal.processors.igfs.IgfsEx; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lifecycle.LifecycleAware; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.jetbrains.annotations.Nullable; -import java.io.Externalizable; import java.io.File; import java.io.FileOutputStream; +import java.io.IOException; import java.net.URI; import java.util.concurrent.atomic.AtomicInteger; @@ -176,19 +185,22 @@ private static IgfsEx startPrimary() throws Exception { writeConfigurationToFile(conf); - // Configure factory. - TestFactory factory = new TestFactory(); + // Get file system instance to be used. + CachingHadoopFileSystemFactory delegate = new CachingHadoopFileSystemFactory(); + + delegate.setUri("igfs://secondary:secondary@127.0.0.1:11500/"); + delegate.setConfigPaths(SECONDARY_CFG_PATH); - factory.setUri("igfs://secondary:secondary@127.0.0.1:11500/"); - factory.setConfigPaths(SECONDARY_CFG_PATH); + // Configure factory. + TestFactory factory = new TestFactory(delegate); // Configure file system. - IgniteHadoopIgfsSecondaryFileSystem fs = new IgniteHadoopIgfsSecondaryFileSystem(); + IgniteHadoopIgfsSecondaryFileSystem secondaryFs = new IgniteHadoopIgfsSecondaryFileSystem(); - fs.setFileSystemFactory(factory); + secondaryFs.setFileSystemFactory(factory); // Start. - return start("primary", 10500, IgfsMode.DUAL_ASYNC, fs); + return start("primary", 10500, IgfsMode.DUAL_ASYNC, secondaryFs); } /** @@ -292,26 +304,42 @@ private static void writeConfigurationToFile(Configuration conf) throws Exceptio /** * Test factory. */ - private static class TestFactory extends CachingHadoopFileSystemFactory { + private static class TestFactory implements HadoopFileSystemFactory, LifecycleAware { + /** */ + private static final long serialVersionUID = 0L; + + /** File system factory. */ + private CachingHadoopFileSystemFactory factory; + + /** File system. */ + private transient HadoopFileSystemFactoryDelegate delegate; + /** - * {@link Externalizable} support. + * Constructor. + * + * @param factory File system factory. */ - public TestFactory() { - // No-op. + public TestFactory(CachingHadoopFileSystemFactory factory) { + this.factory = factory; + } + + /** {@inheritDoc} */ + @Override public Object get(String usrName) throws IOException { + return delegate.get(usrName); } /** {@inheritDoc} */ @Override public void start() throws IgniteException { - START_CNT.incrementAndGet(); + delegate = HadoopDelegateUtils.fileSystemFactoryDelegate(factory); - super.start(); + delegate.start(); + + START_CNT.incrementAndGet(); } /** {@inheritDoc} */ @Override public void stop() throws IgniteException { STOP_CNT.incrementAndGet(); - - super.stop(); } } } diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java similarity index 99% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java index 93a924c72a1ea..d8f6f546fac1c 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import java.io.BufferedOutputStream; import java.io.Closeable; @@ -59,7 +59,14 @@ import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils; +import org.apache.ignite.igfs.IgfsBlockLocation; +import org.apache.ignite.igfs.IgfsFile; +import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsMetrics; +import org.apache.ignite.igfs.IgfsMode; +import org.apache.ignite.igfs.IgfsPath; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils; import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.typedef.F; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemLoopbackPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfs20FileSystemLoopbackPrimarySelfTest.java similarity index 93% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemLoopbackPrimarySelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfs20FileSystemLoopbackPrimarySelfTest.java index ff5cd5b7f3ea4..ee167d1699eba 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemLoopbackPrimarySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfs20FileSystemLoopbackPrimarySelfTest.java @@ -15,7 +15,10 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; + +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; import static org.apache.ignite.igfs.IgfsMode.PRIMARY; import static org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.DFLT_IPC_PORT; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemShmemPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfs20FileSystemShmemPrimarySelfTest.java similarity index 93% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemShmemPrimarySelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfs20FileSystemShmemPrimarySelfTest.java index 2bc9eb814cb77..5f45488facd72 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemShmemPrimarySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfs20FileSystemShmemPrimarySelfTest.java @@ -15,7 +15,10 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; + +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; import static org.apache.ignite.igfs.IgfsMode.PRIMARY; import static org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.DFLT_IPC_PORT; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsDualAbstractSelfTest.java similarity index 95% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsDualAbstractSelfTest.java index bb155b40528c5..881618217ce28 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsDualAbstractSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -28,6 +28,13 @@ import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem; +import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.igfs.IgfsInputStream; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; +import org.apache.ignite.igfs.IgfsMode; +import org.apache.ignite.igfs.IgfsOutputStream; +import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; import org.apache.ignite.internal.processors.igfs.IgfsBlockKey; import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest; @@ -51,7 +58,7 @@ import static org.apache.ignite.igfs.IgfsMode.DUAL_ASYNC; import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC; import static org.apache.ignite.igfs.IgfsMode.PRIMARY; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH; import static org.apache.ignite.internal.processors.igfs.IgfsAbstractSelfTest.awaitFileClose; import static org.apache.ignite.internal.processors.igfs.IgfsAbstractSelfTest.clear; import static org.apache.ignite.internal.processors.igfs.IgfsAbstractSelfTest.create; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsDualAsyncSelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAsyncSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsDualAsyncSelfTest.java index 6c6e7099ab134..eb85700e5fe58 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAsyncSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsDualAsyncSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.DUAL_ASYNC; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualSyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsDualSyncSelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualSyncSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsDualSyncSelfTest.java index 96a63d510e49c..98a2dc5357014 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualSyncSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsDualSyncSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java similarity index 89% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java index f7af6f07483f9..adb1330f2a319 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import java.io.IOException; import java.io.InputStream; @@ -28,7 +28,9 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils; import org.apache.ignite.internal.processors.igfs.IgfsEx; import org.apache.ignite.internal.processors.igfs.IgfsUtils; import org.apache.ignite.internal.processors.igfs.IgfsSecondaryFileSystemTestAdapter; @@ -39,7 +41,7 @@ */ public class HadoopIgfsSecondaryFileSystemTestAdapter implements IgfsSecondaryFileSystemTestAdapter { /** File system factory. */ - private final HadoopFileSystemFactory factory; + private final HadoopFileSystemFactoryDelegate factory; /** * Constructor. @@ -48,7 +50,9 @@ public class HadoopIgfsSecondaryFileSystemTestAdapter implements IgfsSecondaryFi public HadoopIgfsSecondaryFileSystemTestAdapter(HadoopFileSystemFactory factory) { assert factory != null; - this.factory = factory; + this.factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory); + + this.factory.start(); } /** {@inheritDoc} */ @@ -144,6 +148,6 @@ private String permission(FileStatus status) { * @throws IOException If failed. */ protected FileSystem get() throws IOException { - return factory.get(FileSystemConfiguration.DFLT_USER_NAME); + return (FileSystem)factory.get(FileSystemConfiguration.DFLT_USER_NAME); } } \ No newline at end of file diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java similarity index 95% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java index d9b5d66797170..fd7523315c86c 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; @@ -31,7 +31,13 @@ import org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory; import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem; import org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils; +import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; +import org.apache.ignite.igfs.IgfsMode; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils; +import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils; import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.U; @@ -161,7 +167,7 @@ public HadoopSecondaryFileSystemConfigurationTest() { /** * Executes before each test. - * @throws Exception + * @throws Exception If failed. */ private void before() throws Exception { initSecondary(); @@ -179,16 +185,18 @@ private void before() throws Exception { fac.setConfigPaths(primaryConfFullPath); fac.setUri(primaryFsUriStr); - fac.start(); + HadoopFileSystemFactoryDelegate facDelegate = HadoopDelegateUtils.fileSystemFactoryDelegate(fac); + + facDelegate.start(); - primaryFs = fac.get(null); //provider.createFileSystem(null); + primaryFs = (FileSystem)facDelegate.get(null); //provider.createFileSystem(null); primaryFsUri = primaryFs.getUri(); } /** * Executes after each test. - * @throws Exception + * @throws Exception If failed. */ private void after() throws Exception { if (primaryFs != null) { @@ -225,7 +233,7 @@ private static void delete(String file) { /** * Initialize underlying secondary filesystem. * - * @throws Exception + * @throws Exception If failed. */ private void initSecondary() throws Exception { if (passSecondaryConfiguration) { diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgfsEventsTestSuite.java similarity index 97% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgfsEventsTestSuite.java index a9d7bad49ef31..6d7dc995a8690 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgfsEventsTestSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import junit.framework.TestSuite; import org.apache.ignite.Ignite; @@ -24,6 +24,10 @@ import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem; +import org.apache.ignite.igfs.IgfsEventsAbstractSelfTest; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; +import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint; import org.apache.ignite.internal.util.typedef.G; import org.jetbrains.annotations.Nullable; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgfsNearOnlyMultiNodeSelfTest.java similarity index 97% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgfsNearOnlyMultiNodeSelfTest.java index 8e7935630cf8d..51048c6b3fc62 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgfsNearOnlyMultiNodeSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import java.io.OutputStream; import java.net.URI; @@ -31,6 +31,9 @@ import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; import org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemAbstractSelfTest.java similarity index 99% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemAbstractSelfTest.java index f793ec31f0b5a..bd17edc8ab2fd 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemAbstractSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; @@ -39,10 +39,13 @@ import org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory; import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem; import org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsEx; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsIpcIo; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsOutProc; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils; +import org.apache.ignite.igfs.IgfsBlockLocation; +import org.apache.ignite.igfs.IgfsFile; +import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; +import org.apache.ignite.igfs.IgfsMode; +import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.lang.GridAbsPredicate; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemClientSelfTest.java similarity index 92% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemClientSelfTest.java index 8ddb3596908fe..555da1e0422c7 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemClientSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import java.io.IOException; import java.lang.reflect.Field; @@ -27,12 +27,16 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; +import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.igfs.common.IgfsLogger; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfs; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsOutProc; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsOutputStream; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsStreamDelegate; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfs; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsOutProc; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsOutputStream; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsStreamDelegate; import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest; import org.apache.ignite.internal.processors.igfs.IgfsContext; import org.apache.ignite.internal.processors.igfs.IgfsProcessorAdapter; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java similarity index 95% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java index fdb0d7724cfa7..9891c602b90a5 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import java.io.IOException; import java.net.URI; @@ -29,6 +29,10 @@ import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem; +import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; +import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; @@ -42,9 +46,9 @@ import static org.apache.ignite.cache.CacheMode.REPLICATED; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; import static org.apache.ignite.igfs.IgfsMode.PRIMARY; -import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_EMBED; -import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM; -import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_LOCAL_TCP; +import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_EMBED; +import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM; +import static org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_LOCAL_TCP; import static org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.DFLT_IPC_PORT; /** diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java similarity index 96% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java index 4d7a39e3c1142..35f1ceb50ee17 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import java.lang.reflect.Field; import java.net.URI; @@ -27,8 +27,9 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsIpcIo; -import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils; +import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest; import org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint; import org.apache.ignite.internal.util.typedef.G; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoggerSelfTest.java similarity index 99% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoggerSelfTest.java index 3013311618bb6..b61492a6e5be7 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoggerSelfTest.java @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; +import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.igfs.common.IgfsLogger; import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest; import org.apache.ignite.internal.util.typedef.internal.SB; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerStateSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoggerStateSelfTest.java similarity index 95% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerStateSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoggerStateSelfTest.java index 1bd5b41195eaf..e30779f8d08f0 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoggerStateSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoggerStateSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import java.lang.reflect.Field; import java.net.URI; @@ -28,6 +28,9 @@ import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem; +import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; import org.apache.ignite.internal.igfs.common.IgfsLogger; import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest; import org.apache.ignite.internal.processors.igfs.IgfsEx; @@ -40,8 +43,8 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED; import static org.apache.ignite.cache.CacheMode.REPLICATED; import static org.apache.ignite.igfs.IgfsMode.PRIMARY; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_LOG_DIR; -import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.PARAM_IGFS_LOG_ENABLED; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_LOG_DIR; +import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopParameters.PARAM_IGFS_LOG_ENABLED; /** * Ensures that sampling is really turned on/off. diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackAbstractSelfTest.java similarity index 88% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackAbstractSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackAbstractSelfTest.java index 6ed2249187213..4f48078179927 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackAbstractSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackAbstractSelfTest.java @@ -15,7 +15,11 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; + +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; +import org.apache.ignite.igfs.IgfsMode; import static org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.DFLT_IPC_PORT; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java index f1edb28525433..93b64dc5171ce 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.DUAL_ASYNC; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java index 97a6991a11ce5..0c41ed357fc56 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java index f9ecc4b7fe22b..ed7eb3271f0ac 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.PRIMARY; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java index 719df6da5d4d1..b41e5daff6031 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.PROXY; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java index 764624da38a5c..ce8046a3d33ac 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.DUAL_ASYNC; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest.java index 21a248aa3c914..1327d6b8df0d8 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest.java index 092c7a5a31429..2a3c2eecd2744 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.PRIMARY; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest.java index 9f7d21b1b6ac2..8d42336b43ea4 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.PROXY; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest.java similarity index 96% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest.java index 1b488702e8900..4c96244165ac1 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -25,6 +25,9 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem; import org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem; +import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.U; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemAbstractSelfTest.java similarity index 93% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemAbstractSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemAbstractSelfTest.java index d8cf74c2b11aa..67b5007a3201a 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemAbstractSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemAbstractSelfTest.java @@ -15,12 +15,15 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import java.util.Collection; import java.util.LinkedList; import java.util.concurrent.Callable; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; +import org.apache.ignite.igfs.IgfsIpcEndpointType; +import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.internal.util.ipc.IpcEndpoint; import org.apache.ignite.internal.util.ipc.IpcEndpointFactory; import org.apache.ignite.internal.util.typedef.X; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java index d0d570f0a06e5..cca960bdc89e5 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.DUAL_ASYNC; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java index 2e5b015195e45..73db4f8c414f0 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest.java index 214c2a8bc0c53..48a4694702c21 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.PRIMARY; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest.java index d7f34a1902637..ab9c35722cb1c 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.PROXY; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java index 0435eaaa65e06..5154642b2a62b 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.DUAL_ASYNC; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalDualSyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualSyncSelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalDualSyncSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualSyncSelfTest.java index 3af72746e99b0..d88a38b87df00 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalDualSyncSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalDualSyncSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalPrimarySelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalPrimarySelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalPrimarySelfTest.java index ce9dbd9f2dcbf..7b41b224669b4 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalPrimarySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalPrimarySelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.PRIMARY; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalSecondarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalSecondarySelfTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalSecondarySelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalSecondarySelfTest.java index bc8c182f0f64e..e54b0200f4717 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemShmemExternalSecondarySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/IgniteHadoopFileSystemShmemExternalSecondarySelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.igfs; +package org.apache.ignite.internal.processors.hadoop.impl.igfs; import static org.apache.ignite.igfs.IgfsMode.PROXY; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopAbstractMapTest.java similarity index 94% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopAbstractMapTest.java index 5266875504179..9d1fd4f710184 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopAbstractMapTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.shuffle.collections; +package org.apache.ignite.internal.processors.hadoop.impl.shuffle.collections; import java.util.Comparator; import java.util.concurrent.Callable; @@ -23,6 +23,7 @@ import org.apache.hadoop.io.IntWritable; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.hadoop.HadoopHelper; import org.apache.ignite.internal.processors.hadoop.HadoopJob; import org.apache.ignite.internal.processors.hadoop.HadoopJobId; import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo; @@ -31,7 +32,7 @@ import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounter; import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters; -import org.apache.ignite.internal.processors.hadoop.v2.HadoopWritableSerialization; +import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopWritableSerialization; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jetbrains.annotations.Nullable; @@ -144,7 +145,7 @@ protected static class JobInfo implements HadoopJobInfo { /** {@inheritDoc} */ @Override public HadoopJob createJob(Class jobCls, HadoopJobId jobId, IgniteLogger log, - @Nullable String[] libNames) throws IgniteCheckedException { + @Nullable String[] libNames, HadoopHelper helper) throws IgniteCheckedException { assert false; return null; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopConcurrentHashMultimapSelftest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopConcurrentHashMultimapSelftest.java similarity index 96% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopConcurrentHashMultimapSelftest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopConcurrentHashMultimapSelftest.java index a37d74bb636ae..019b17279e4a4 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopConcurrentHashMultimapSelftest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopConcurrentHashMultimapSelftest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.shuffle.collections; +package org.apache.ignite.internal.processors.hadoop.impl.shuffle.collections; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; @@ -35,6 +35,8 @@ import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo; import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput; +import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopConcurrentHashMultimap; +import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopMultimap; import org.apache.ignite.internal.util.GridRandom; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.io.GridDataInput; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopHashMapSelfTest.java similarity index 93% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopHashMapSelfTest.java index 04585eccff2f3..195bcbbfcb752 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMapSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopHashMapSelfTest.java @@ -15,13 +15,15 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.shuffle.collections; +package org.apache.ignite.internal.processors.hadoop.impl.shuffle.collections; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; import org.apache.hadoop.io.IntWritable; import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput; +import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopHashMultimap; +import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopMultimap; import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory; import org.apache.ignite.internal.util.typedef.X; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopSkipListSelfTest.java similarity index 96% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopSkipListSelfTest.java index f70ef2f9829b2..d04becaa607be 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopSkipListSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.shuffle.collections; +package org.apache.ignite.internal.processors.hadoop.impl.shuffle.collections; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; @@ -36,6 +36,8 @@ import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo; import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput; +import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopMultimap; +import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopSkipList; import org.apache.ignite.internal.util.GridRandom; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.io.GridDataInput; @@ -103,7 +105,7 @@ public void testMapSimple() throws Exception { HadoopMultimap m = new HadoopSkipList(job, mem); - HadoopConcurrentHashMultimap.Adder a = m.startAdding(taskCtx); + HadoopMultimap.Adder a = m.startAdding(taskCtx); Multimap mm = ArrayListMultimap.create(); Multimap vis = ArrayListMultimap.create(); @@ -180,7 +182,7 @@ private void check(HadoopMultimap m, Multimap mm, final Multim final GridDataInput dataInput = new GridUnsafeDataInput(); - m.visit(false, new HadoopConcurrentHashMultimap.Visitor() { + m.visit(false, new HadoopMultimap.Visitor() { /** */ IntWritable key = new IntWritable(); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataStreamSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/streams/HadoopDataStreamSelfTest.java similarity index 95% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataStreamSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/streams/HadoopDataStreamSelfTest.java index dd571afb69c99..612e892a96afa 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataStreamSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/streams/HadoopDataStreamSelfTest.java @@ -15,10 +15,13 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.shuffle.streams; +package org.apache.ignite.internal.processors.hadoop.impl.shuffle.streams; import java.io.IOException; import java.util.Arrays; + +import org.apache.ignite.internal.processors.hadoop.shuffle.streams.HadoopDataInStream; +import org.apache.ignite.internal.processors.hadoop.shuffle.streams.HadoopDataOutStream; import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorServiceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/HadoopExecutorServiceTest.java similarity index 95% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorServiceTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/HadoopExecutorServiceTest.java index 7dd045a14ab24..b4e63d1e15f78 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorServiceTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/HadoopExecutorServiceTest.java @@ -15,11 +15,12 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.taskexecutor; +package org.apache.ignite.internal.processors.hadoop.impl.taskexecutor; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopExecutorService; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jsr166.LongAdder8; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java similarity index 96% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java index ec33836d58e4e..7c43500209730 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.taskexecutor.external; +package org.apache.ignite.internal.processors.hadoop.impl.taskexecutor.external; import java.io.IOException; import java.io.OutputStreamWriter; @@ -37,12 +37,12 @@ import org.apache.ignite.igfs.IgfsOutputStream; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.processors.hadoop.HadoopAbstractSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopAbstractSelfTest; import org.apache.ignite.internal.processors.hadoop.HadoopJobId; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.marshaller.jdk.JdkMarshaller; -import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.createJobInfo; +import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.createJobInfo; /** * Job tracker self test. diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java similarity index 95% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java index 851c3af10410d..a40c5313d8442 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication; +package org.apache.ignite.internal.processors.hadoop.impl.taskexecutor.external.communication; import java.io.Externalizable; import java.io.IOException; @@ -30,6 +30,8 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.processors.hadoop.message.HadoopMessage; import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopProcessDescriptor; +import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopExternalCommunication; +import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopMessageListener; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.jdk.JdkMarshaller; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/util/BasicUserNameMapperSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/util/BasicUserNameMapperSelfTest.java similarity index 97% rename from modules/hadoop/src/test/java/org/apache/ignite/hadoop/util/BasicUserNameMapperSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/util/BasicUserNameMapperSelfTest.java index fd8fdef0369eb..43924ed6b29cf 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/util/BasicUserNameMapperSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/util/BasicUserNameMapperSelfTest.java @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.ignite.hadoop.util; +package org.apache.ignite.internal.processors.hadoop.impl.util; +import org.apache.ignite.hadoop.util.BasicUserNameMapper; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jetbrains.annotations.Nullable; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/util/ChainedUserNameMapperSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/util/ChainedUserNameMapperSelfTest.java similarity index 92% rename from modules/hadoop/src/test/java/org/apache/ignite/hadoop/util/ChainedUserNameMapperSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/util/ChainedUserNameMapperSelfTest.java index bfac49c6ed2c0..a9d295f861d6e 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/util/ChainedUserNameMapperSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/util/ChainedUserNameMapperSelfTest.java @@ -15,9 +15,13 @@ * limitations under the License. */ -package org.apache.ignite.hadoop.util; +package org.apache.ignite.internal.processors.hadoop.impl.util; import org.apache.ignite.IgniteException; +import org.apache.ignite.hadoop.util.BasicUserNameMapper; +import org.apache.ignite.hadoop.util.ChainedUserNameMapper; +import org.apache.ignite.hadoop.util.KerberosUserNameMapper; +import org.apache.ignite.hadoop.util.UserNameMapper; import org.apache.ignite.internal.processors.igfs.IgfsUtils; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/util/KerberosUserNameMapperSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/util/KerberosUserNameMapperSelfTest.java similarity index 96% rename from modules/hadoop/src/test/java/org/apache/ignite/hadoop/util/KerberosUserNameMapperSelfTest.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/util/KerberosUserNameMapperSelfTest.java index cc685bbf22016..bd76b51a1685b 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/hadoop/util/KerberosUserNameMapperSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/util/KerberosUserNameMapperSelfTest.java @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.ignite.hadoop.util; +package org.apache.ignite.internal.processors.hadoop.impl.util; +import org.apache.ignite.hadoop.util.KerberosUserNameMapper; import org.apache.ignite.internal.processors.igfs.IgfsUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jetbrains.annotations.Nullable; diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithOuterClass.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/state/HadoopGroupingTestState.java similarity index 57% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithOuterClass.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/state/HadoopGroupingTestState.java index cae1da76f4455..af82d49b4db17 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithOuterClass.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/state/HadoopGroupingTestState.java @@ -15,24 +15,26 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.deps; +package org.apache.ignite.internal.processors.hadoop.state; -import org.apache.hadoop.conf.Configuration; +import org.apache.ignite.internal.processors.hadoop.HadoopSharedMap; +import org.apache.ignite.internal.util.GridConcurrentHashSet; + +import java.util.Collection; +import java.util.UUID; /** - * Outer class depends on Hadoop, but Inner *static* one does not. + * Shared state for HadoopGroupingTest. */ -@SuppressWarnings("unused") -public class WithOuterClass { - /** */ - Configuration c; - - /** */ - public static class InnerNoHadoop { - /** */ - int x; +public class HadoopGroupingTestState { + /** Values. */ + private static final GridConcurrentHashSet vals = HadoopSharedMap.map(HadoopGroupingTestState.class) + .put("vals", new GridConcurrentHashSet()); - /** */ - void foo() {} + /** + * @return Values. + */ + public static Collection values() { + return vals; } } diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/state/HadoopJobTrackerSelfTestState.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/state/HadoopJobTrackerSelfTestState.java new file mode 100644 index 0000000000000..4183f4e220b36 --- /dev/null +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/state/HadoopJobTrackerSelfTestState.java @@ -0,0 +1,45 @@ +/* + * 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.hadoop.state; + +import org.apache.ignite.internal.processors.hadoop.HadoopSharedMap; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Shared state for HadoopJobTrackerSelfTest. + */ +public class HadoopJobTrackerSelfTestState { + /** */ + private static HadoopSharedMap m = HadoopSharedMap.map(HadoopJobTrackerSelfTestState.class); + + /** Map task execution count. */ + public static final AtomicInteger mapExecCnt = m.put("mapExecCnt", new AtomicInteger()); + + /** Reduce task execution count. */ + public static final AtomicInteger reduceExecCnt = m.put("reduceExecCnt", new AtomicInteger()); + + /** Reduce task execution count. */ + public static final AtomicInteger combineExecCnt = m.put("combineExecCnt", new AtomicInteger()); + + /** */ + public static final Map latch = m.put("latch", new HashMap()); +} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInitializer.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/state/HadoopMapReduceEmbeddedSelfTestState.java similarity index 64% rename from modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInitializer.java rename to modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/state/HadoopMapReduceEmbeddedSelfTestState.java index 360986c4f2ede..ae6ce82129e00 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInitializer.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/state/HadoopMapReduceEmbeddedSelfTestState.java @@ -15,19 +15,18 @@ * limitations under the License. */ -package org.apache.ignite.internal.processors.hadoop.deps; +package org.apache.ignite.internal.processors.hadoop.state; -/** - * Has a field initialized with an expression invoking Hadoop method. - */ +import org.apache.ignite.internal.processors.hadoop.HadoopSharedMap; -@SuppressWarnings({"ConstantConditions", "unused"}) -public class WithInitializer { - /** */ - private final Object x = org.apache.hadoop.fs.FileSystem.getDefaultUri(null); +import java.util.HashMap; +import java.util.Map; +/** + * Sharead state for HadoopMapReduceEmbeddedSelfTest. + */ +public class HadoopMapReduceEmbeddedSelfTestState { /** */ - WithInitializer() throws Exception { - // noop - } + public static Map flags = HadoopSharedMap.map(HadoopMapReduceEmbeddedSelfTestState.class) + .put("flags", new HashMap()); } diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/state/HadoopTaskExecutionSelfTestValues.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/state/HadoopTaskExecutionSelfTestValues.java new file mode 100644 index 0000000000000..beefa68f37c43 --- /dev/null +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/state/HadoopTaskExecutionSelfTestValues.java @@ -0,0 +1,51 @@ +/* + * 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.hadoop.state; + +import org.apache.ignite.internal.processors.hadoop.HadoopSharedMap; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Shared state for HadoopTaskExecutionSelfTest. + */ +public class HadoopTaskExecutionSelfTestValues { + /** */ + private static HadoopSharedMap m = HadoopSharedMap.map(HadoopTaskExecutionSelfTestValues.class); + + /** Line count. */ + public static final AtomicInteger totalLineCnt = m.put("totalLineCnt", new AtomicInteger()); + + /** Executed tasks. */ + public static final AtomicInteger executedTasks = m.put("executedTasks", new AtomicInteger()); + + /** Cancelled tasks. */ + public static final AtomicInteger cancelledTasks = m.put("cancelledTasks", new AtomicInteger()); + + /** Working directory of each task. */ + public static final Map taskWorkDirs = m.put("taskWorkDirs", + new ConcurrentHashMap()); + + /** Mapper id to fail. */ + public static final AtomicInteger failMapperId = m.put("failMapperId", new AtomicInteger()); + + /** Number of splits of the current input. */ + public static final AtomicInteger splitsCount = m.put("splitsCount", new AtomicInteger()); +} diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java index 603fd5b3c2ad5..bbd92d131d21f 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java @@ -22,61 +22,61 @@ import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; import org.apache.ignite.IgniteSystemProperties; -import org.apache.ignite.client.hadoop.HadoopClientProtocolEmbeddedSelfTest; -import org.apache.ignite.client.hadoop.HadoopClientProtocolSelfTest; -import org.apache.ignite.hadoop.cache.HadoopTxConfigCacheTest; -import org.apache.ignite.hadoop.fs.KerberosHadoopFileSystemFactorySelfTest; -import org.apache.ignite.hadoop.util.BasicUserNameMapperSelfTest; -import org.apache.ignite.hadoop.util.ChainedUserNameMapperSelfTest; -import org.apache.ignite.hadoop.util.KerberosUserNameMapperSelfTest; -import org.apache.ignite.igfs.Hadoop1OverIgfsDualAsyncTest; -import org.apache.ignite.igfs.Hadoop1OverIgfsDualSyncTest; -import org.apache.ignite.igfs.HadoopFIleSystemFactorySelfTest; -import org.apache.ignite.igfs.HadoopIgfs20FileSystemLoopbackPrimarySelfTest; -import org.apache.ignite.igfs.HadoopIgfsDualAsyncSelfTest; -import org.apache.ignite.igfs.HadoopIgfsDualSyncSelfTest; -import org.apache.ignite.igfs.HadoopSecondaryFileSystemConfigurationTest; -import org.apache.ignite.igfs.IgfsEventsTestSuite; -import org.apache.ignite.igfs.IgniteHadoopFileSystemClientSelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemHandshakeSelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemLoggerSelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemLoggerStateSelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest; -import org.apache.ignite.internal.processors.hadoop.HadoopClassLoaderTest; -import org.apache.ignite.internal.processors.hadoop.HadoopCommandLineTest; -import org.apache.ignite.internal.processors.hadoop.HadoopDefaultMapReducePlannerSelfTest; -import org.apache.ignite.internal.processors.hadoop.HadoopFileSystemsTest; -import org.apache.ignite.internal.processors.hadoop.HadoopGroupingTest; -import org.apache.ignite.internal.processors.hadoop.HadoopJobTrackerSelfTest; -import org.apache.ignite.internal.processors.hadoop.HadoopMapReduceEmbeddedSelfTest; -import org.apache.ignite.internal.processors.hadoop.HadoopMapReduceErrorResilienceTest; -import org.apache.ignite.internal.processors.hadoop.HadoopMapReduceTest; -import org.apache.ignite.internal.processors.hadoop.HadoopNoHadoopMapReduceTest; -import org.apache.ignite.internal.processors.hadoop.HadoopSerializationWrapperSelfTest; -import org.apache.ignite.internal.processors.hadoop.HadoopSnappyFullMapReduceTest; -import org.apache.ignite.internal.processors.hadoop.HadoopSnappyTest; -import org.apache.ignite.internal.processors.hadoop.HadoopSortingTest; -import org.apache.ignite.internal.processors.hadoop.HadoopSplitWrapperSelfTest; -import org.apache.ignite.internal.processors.hadoop.HadoopTaskExecutionSelfTest; -import org.apache.ignite.internal.processors.hadoop.HadoopTasksV1Test; -import org.apache.ignite.internal.processors.hadoop.HadoopTasksV2Test; -import org.apache.ignite.internal.processors.hadoop.HadoopUserLibsSelfTest; -import org.apache.ignite.internal.processors.hadoop.HadoopV2JobSelfTest; -import org.apache.ignite.internal.processors.hadoop.HadoopValidationSelfTest; -import org.apache.ignite.internal.processors.hadoop.HadoopWeightedMapReducePlannerTest; -import org.apache.ignite.internal.processors.hadoop.HadoopWeightedPlannerMapReduceTest; -import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopConcurrentHashMultimapSelftest; -import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopHashMapSelfTest; -import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopSkipListSelfTest; -import org.apache.ignite.internal.processors.hadoop.shuffle.streams.HadoopDataStreamSelfTest; +import org.apache.ignite.internal.processors.hadoop.HadoopTestClassLoader; +import org.apache.ignite.internal.processors.hadoop.impl.client.HadoopClientProtocolEmbeddedSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.client.HadoopClientProtocolSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopTxConfigCacheTest; +import org.apache.ignite.internal.processors.hadoop.impl.fs.KerberosHadoopFileSystemFactorySelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.util.BasicUserNameMapperSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.util.ChainedUserNameMapperSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.util.KerberosUserNameMapperSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.Hadoop1OverIgfsDualAsyncTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.Hadoop1OverIgfsDualSyncTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopFIleSystemFactorySelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfs20FileSystemLoopbackPrimarySelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsDualAsyncSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsDualSyncSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopSecondaryFileSystemConfigurationTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgfsEventsTestSuite; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemClientSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemHandshakeSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemLoggerSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemLoggerStateSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemLoopbackEmbeddedDualAsyncSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemLoopbackEmbeddedDualSyncSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemLoopbackEmbeddedPrimarySelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemLoopbackEmbeddedSecondarySelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemLoopbackExternalDualAsyncSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemLoopbackExternalDualSyncSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemLoopbackExternalPrimarySelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopCommandLineTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopDefaultMapReducePlannerSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopFileSystemsTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopGroupingTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopJobTrackerSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopMapReduceEmbeddedSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopMapReduceErrorResilienceTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopMapReduceTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopNoHadoopMapReduceTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopSerializationWrapperSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopSnappyFullMapReduceTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopSnappyTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopSortingTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopSplitWrapperSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopTaskExecutionSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopTasksV1Test; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopTasksV2Test; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopUserLibsSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopV2JobSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopValidationSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopWeightedMapReducePlannerTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopWeightedPlannerMapReduceTest; +import org.apache.ignite.internal.processors.hadoop.impl.shuffle.collections.HadoopConcurrentHashMultimapSelftest; +import org.apache.ignite.internal.processors.hadoop.impl.shuffle.collections.HadoopHashMapSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.shuffle.collections.HadoopSkipListSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.shuffle.streams.HadoopDataStreamSelfTest; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; @@ -107,7 +107,7 @@ public static TestSuite suite() throws Exception { downloadHadoop(); downloadHive(); - final ClassLoader ldr = TestSuite.class.getClassLoader(); + final ClassLoader ldr = new HadoopTestClassLoader(); TestSuite suite = new TestSuite("Ignite Hadoop MR Test Suite"); @@ -125,8 +125,6 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(ldr.loadClass(HadoopSnappyTest.class.getName()))); suite.addTest(new TestSuite(ldr.loadClass(HadoopSnappyFullMapReduceTest.class.getName()))); - suite.addTest(new TestSuite(ldr.loadClass(HadoopClassLoaderTest.class.getName()))); - suite.addTest(new TestSuite(ldr.loadClass(HadoopIgfs20FileSystemLoopbackPrimarySelfTest.class.getName()))); suite.addTest(new TestSuite(ldr.loadClass(HadoopIgfsDualSyncSelfTest.class.getName()))); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java index 4ed1d6594d056..21d78aba3707e 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java @@ -18,17 +18,18 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; -import org.apache.ignite.igfs.HadoopIgfs20FileSystemShmemPrimarySelfTest; -import org.apache.ignite.igfs.IgfsEventsTestSuite; -import org.apache.ignite.igfs.IgniteHadoopFileSystemIpcCacheSelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemShmemExternalDualSyncSelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemShmemExternalPrimarySelfTest; -import org.apache.ignite.igfs.IgniteHadoopFileSystemShmemExternalSecondarySelfTest; +import org.apache.ignite.internal.processors.hadoop.HadoopTestClassLoader; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfs20FileSystemShmemPrimarySelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgfsEventsTestSuite; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemIpcCacheSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemShmemEmbeddedDualAsyncSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemShmemEmbeddedDualSyncSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemShmemEmbeddedPrimarySelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemShmemEmbeddedSecondarySelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemShmemExternalDualAsyncSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemShmemExternalDualSyncSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemShmemExternalPrimarySelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemShmemExternalSecondarySelfTest; import org.apache.ignite.internal.processors.igfs.IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest; import static org.apache.ignite.testsuites.IgniteHadoopTestSuite.downloadHadoop; @@ -45,7 +46,7 @@ public class IgniteIgfsLinuxAndMacOSTestSuite extends TestSuite { public static TestSuite suite() throws Exception { downloadHadoop(); - ClassLoader ldr = TestSuite.class.getClassLoader(); + final ClassLoader ldr = new HadoopTestClassLoader(); TestSuite suite = new TestSuite("Ignite IGFS Test Suite For Linux And Mac OS"); From 33a6878eccd60f86648067901613b713ef3bb71f Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 26 Sep 2016 14:56:51 +0300 Subject: [PATCH 167/487] IGNITE-3491 .NET: Allow type name without assembly for type properties in app.config --- .../IgniteConfigurationSerializerTest.cs | 4 ++-- .../IgniteConfigurationXmlSerializer.cs | 21 +++++++++++-------- 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs index b6ee5cb86f839..8afef33a0f256 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs @@ -59,7 +59,7 @@ public void TestPredefinedXml() var xml = @" 127.1.1.1 - + Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+FooClass, Apache.Ignite.Core.Tests @@ -70,7 +70,7 @@ public void TestPredefinedXml() -Xms1g-Xmx4g - + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs index 52fbc306670b5..e1df50bcbdbbb 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs @@ -26,6 +26,7 @@ namespace Apache.Ignite.Core.Impl.Common using System.Linq; using System.Reflection; using System.Xml; + using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Events; /// @@ -57,7 +58,7 @@ public static IgniteConfiguration Deserialize(XmlReader reader) var cfg = new IgniteConfiguration(); if (reader.NodeType == XmlNodeType.Element || reader.Read()) - ReadElement(reader, cfg); + ReadElement(reader, cfg, new TypeResolver()); return cfg; } @@ -164,7 +165,7 @@ private static void WriteComplexProperty(object obj, XmlWriter writer, Type valu /// /// Reads the element. /// - private static void ReadElement(XmlReader reader, object target) + private static void ReadElement(XmlReader reader, object target, TypeResolver resolver) { var targetType = target.GetType(); @@ -197,7 +198,7 @@ private static void ReadElement(XmlReader reader, object target) else if (propType.IsGenericType && propType.GetGenericTypeDefinition() == typeof (ICollection<>)) { // Collection - ReadCollectionProperty(reader, prop, target); + ReadCollectionProperty(reader, prop, target, resolver); } else if (propType.IsGenericType && propType.GetGenericTypeDefinition() == typeof (IDictionary<,>)) { @@ -207,7 +208,7 @@ private static void ReadElement(XmlReader reader, object target) else { // Nested object (complex property) - prop.SetValue(target, ReadComplexProperty(reader, propType, prop.Name, targetType), null); + prop.SetValue(target, ReadComplexProperty(reader, propType, prop.Name, targetType, resolver), null); } } } @@ -215,7 +216,8 @@ private static void ReadElement(XmlReader reader, object target) /// /// Reads the complex property (nested object). /// - private static object ReadComplexProperty(XmlReader reader, Type propType, string propName, Type targetType) + private static object ReadComplexProperty(XmlReader reader, Type propType, string propName, Type targetType, + TypeResolver resolver) { if (propType.IsAbstract) { @@ -225,7 +227,7 @@ private static object ReadComplexProperty(XmlReader reader, Type propType, strin propType = typeName == null ? null - : Type.GetType(typeName, false) ?? derivedTypes.FirstOrDefault(x => x.Name == typeName); + : resolver.ResolveType(typeName) ?? derivedTypes.FirstOrDefault(x => x.Name == typeName); if (propType == null) { @@ -249,7 +251,7 @@ private static object ReadComplexProperty(XmlReader reader, Type propType, strin { subReader.Read(); // read first element - ReadElement(subReader, nestedVal); + ReadElement(subReader, nestedVal, resolver); } return nestedVal; @@ -258,7 +260,8 @@ private static object ReadComplexProperty(XmlReader reader, Type propType, strin /// /// Reads the collection. /// - private static void ReadCollectionProperty(XmlReader reader, PropertyInfo prop, object target) + private static void ReadCollectionProperty(XmlReader reader, PropertyInfo prop, object target, + TypeResolver resolver) { var elementType = prop.PropertyType.GetGenericArguments().Single(); @@ -283,7 +286,7 @@ private static void ReadCollectionProperty(XmlReader reader, PropertyInfo prop, list.Add(converter != null ? converter.ConvertFromInvariantString(subReader.ReadString()) - : ReadComplexProperty(subReader, elementType, prop.Name, target.GetType())); + : ReadComplexProperty(subReader, elementType, prop.Name, target.GetType(), resolver)); } } From a4a933eb708dadb6b67f6c062db32ffeef819184 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Mon, 26 Sep 2016 17:41:57 +0300 Subject: [PATCH 168/487] IGNITE-3883: ODBC: Implemented data-at-execution dialog. This closes #1073. --- .../src/application_data_buffer_test.cpp | 129 ++++--- .../cpp/odbc-test/src/column_test.cpp | 10 +- .../cpp/odbc-test/src/queries_test.cpp | 144 +++++++- .../platforms/cpp/odbc-test/src/row_test.cpp | 4 +- modules/platforms/cpp/odbc/Makefile.am | 3 +- .../platforms/cpp/odbc/include/ignite/odbc.h | 5 +- .../ignite/odbc/app/application_data_buffer.h | 58 ++- .../odbc/include/ignite/odbc/app/parameter.h | 27 +- .../odbc/include/ignite/odbc/common_types.h | 18 +- .../ignite/odbc/config/configuration.h | 17 +- .../cpp/odbc/include/ignite/odbc/statement.h | 58 ++- .../platforms/cpp/odbc/project/vs/module.def | 1 - .../odbc/src/app/application_data_buffer.cpp | 343 +++++++++++++----- .../platforms/cpp/odbc/src/app/parameter.cpp | 126 ++++++- .../platforms/cpp/odbc/src/common_types.cpp | 3 + .../cpp/odbc/src/config/configuration.cpp | 27 ++ .../cpp/odbc/src/config/connection_info.cpp | 35 +- modules/platforms/cpp/odbc/src/connection.cpp | 2 +- .../odbc/src/diagnostic/diagnostic_record.cpp | 18 + .../platforms/cpp/odbc/src/entry_points.cpp | 30 +- modules/platforms/cpp/odbc/src/odbc.cpp | 43 ++- modules/platforms/cpp/odbc/src/statement.cpp | 150 +++++++- 22 files changed, 992 insertions(+), 259 deletions(-) diff --git a/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp b/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp index 1900e7542dfbb..f62bcd6fc6caa 100644 --- a/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/application_data_buffer_test.cpp @@ -43,8 +43,8 @@ BOOST_AUTO_TEST_SUITE(ApplicationDataBufferTestSuite) BOOST_AUTO_TEST_CASE(TestPutIntToString) { char buffer[1024]; - SqlLen reslen; - size_t* offset = 0; + SqlLen reslen = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); @@ -76,8 +76,8 @@ BOOST_AUTO_TEST_CASE(TestPutIntToString) BOOST_AUTO_TEST_CASE(TestPutFloatToString) { char buffer[1024]; - SqlLen reslen; - size_t* offset = 0; + SqlLen reslen = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); @@ -101,8 +101,8 @@ BOOST_AUTO_TEST_CASE(TestPutFloatToString) BOOST_AUTO_TEST_CASE(TestPutGuidToString) { char buffer[1024]; - SqlLen reslen; - size_t* offset = 0; + SqlLen reslen = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); @@ -117,7 +117,7 @@ BOOST_AUTO_TEST_CASE(TestPutGuidToString) BOOST_AUTO_TEST_CASE(TestGetGuidFromString) { char buffer[] = "1da1ef8f-39ff-4d62-8b72-e8e9f3371801"; - SqlLen reslen; + SqlLen reslen = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer) - 1, &reslen, 0); @@ -129,8 +129,8 @@ BOOST_AUTO_TEST_CASE(TestGetGuidFromString) BOOST_AUTO_TEST_CASE(TestPutBinaryToString) { char buffer[1024]; - SqlLen reslen; - size_t* offset = 0; + SqlLen reslen = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); @@ -145,8 +145,8 @@ BOOST_AUTO_TEST_CASE(TestPutBinaryToString) BOOST_AUTO_TEST_CASE(TestPutStringToString) { char buffer[1024]; - SqlLen reslen; - size_t* offset = 0; + SqlLen reslen = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, buffer, sizeof(buffer), &reslen, &offset); @@ -161,8 +161,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToString) BOOST_AUTO_TEST_CASE(TestPutStringToWstring) { wchar_t buffer[1024]; - SqlLen reslen; - size_t* offset = 0; + SqlLen reslen = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_WCHAR, buffer, sizeof(buffer), &reslen, &offset); @@ -175,8 +175,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToWstring) BOOST_AUTO_TEST_CASE(TestPutStringToLong) { long numBuf; - SqlLen reslen; - size_t* offset = 0; + SqlLen reslen = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, sizeof(numBuf), &reslen, &offset); @@ -190,8 +190,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToLong) BOOST_AUTO_TEST_CASE(TestPutStringToTiny) { int8_t numBuf; - SqlLen reslen; - size_t* offset = 0; + SqlLen reslen = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_TINYINT, &numBuf, sizeof(numBuf), &reslen, &offset); @@ -205,8 +205,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToTiny) BOOST_AUTO_TEST_CASE(TestPutStringToFloat) { float numBuf; - SqlLen reslen; - size_t* offset = 0; + SqlLen reslen = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &numBuf, sizeof(numBuf), &reslen, &offset); @@ -220,8 +220,8 @@ BOOST_AUTO_TEST_CASE(TestPutStringToFloat) BOOST_AUTO_TEST_CASE(TestPutIntToFloat) { float numBuf; - SqlLen reslen; - size_t* offset = 0; + SqlLen reslen = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &numBuf, sizeof(numBuf), &reslen, &offset); @@ -247,8 +247,8 @@ BOOST_AUTO_TEST_CASE(TestPutIntToFloat) BOOST_AUTO_TEST_CASE(TestPutFloatToShort) { short numBuf; - SqlLen reslen; - size_t* offset = 0; + SqlLen reslen = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_SHORT, &numBuf, sizeof(numBuf), &reslen, &offset); @@ -268,7 +268,7 @@ BOOST_AUTO_TEST_CASE(TestPutFloatToShort) BOOST_AUTO_TEST_CASE(TestPutDecimalToDouble) { double numBuf; - SqlLen reslen; + SqlLen reslen = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &numBuf, sizeof(numBuf), &reslen, 0); @@ -297,7 +297,7 @@ BOOST_AUTO_TEST_CASE(TestPutDecimalToDouble) BOOST_AUTO_TEST_CASE(TestPutDecimalToLong) { long numBuf; - SqlLen reslen; + SqlLen reslen = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, sizeof(numBuf), &reslen, 0); @@ -324,7 +324,7 @@ BOOST_AUTO_TEST_CASE(TestPutDecimalToLong) BOOST_AUTO_TEST_CASE(TestPutDecimalToString) { char strBuf[64]; - SqlLen reslen; + SqlLen reslen = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, 0); @@ -351,7 +351,7 @@ BOOST_AUTO_TEST_CASE(TestPutDecimalToString) BOOST_AUTO_TEST_CASE(TestPutDecimalToNumeric) { SQL_NUMERIC_STRUCT buf; - SqlLen reslen; + SqlLen reslen = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_NUMERIC, &buf, sizeof(buf), &reslen, 0); @@ -432,8 +432,8 @@ BOOST_AUTO_TEST_CASE(TestPutDateToDate) SQL_DATE_STRUCT buf = { 0 }; SqlLen reslen = sizeof(buf); - size_t offset = 0; - size_t* offsetPtr = &offset; + int offset = 0; + int* offsetPtr = &offset; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen, &offsetPtr); @@ -451,8 +451,8 @@ BOOST_AUTO_TEST_CASE(TestPutTimestampToDate) SQL_DATE_STRUCT buf = { 0 }; SqlLen reslen = sizeof(buf); - size_t offset = 0; - size_t* offsetPtr = &offset; + int offset = 0; + int* offsetPtr = &offset; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen, &offsetPtr); @@ -470,8 +470,8 @@ BOOST_AUTO_TEST_CASE(TestPutTimestampToTimestamp) SQL_TIMESTAMP_STRUCT buf = { 0 }; SqlLen reslen = sizeof(buf); - size_t offset = 0; - size_t* offsetPtr = &offset; + int offset = 0; + int* offsetPtr = &offset; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen, &offsetPtr); @@ -494,8 +494,8 @@ BOOST_AUTO_TEST_CASE(TestPutDateToTimestamp) SqlLen reslen = sizeof(buf); - size_t offset = 0; - size_t* offsetPtr = &offset; + int offset = 0; + int* offsetPtr = &offset; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen, &offsetPtr); @@ -516,7 +516,7 @@ BOOST_AUTO_TEST_CASE(TestGetStringFromLong) { long numBuf = 42; SqlLen reslen = sizeof(numBuf); - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_SIGNED_LONG, &numBuf, reslen, &reslen, &offset); @@ -535,7 +535,7 @@ BOOST_AUTO_TEST_CASE(TestGetStringFromDouble) { double numBuf = 43.36; SqlLen reslen = sizeof(numBuf); - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &numBuf, reslen, &reslen, &offset); @@ -554,7 +554,7 @@ BOOST_AUTO_TEST_CASE(TestGetStringFromString) { char buf[] = "Some data 32d2d5hs"; SqlLen reslen = sizeof(buf); - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen, &offset); @@ -567,7 +567,7 @@ BOOST_AUTO_TEST_CASE(TestGetFloatFromUshort) { unsigned short numBuf = 7162; SqlLen reslen = sizeof(numBuf); - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_SHORT, &numBuf, reslen, &reslen, &offset); @@ -584,7 +584,7 @@ BOOST_AUTO_TEST_CASE(TestGetFloatFromString) { char buf[] = "28.562"; SqlLen reslen = sizeof(buf); - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen, &offset); @@ -601,7 +601,7 @@ BOOST_AUTO_TEST_CASE(TestGetFloatFromFloat) { float buf = 207.49f; SqlLen reslen = sizeof(buf); - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &buf, reslen, &reslen, &offset); @@ -618,7 +618,7 @@ BOOST_AUTO_TEST_CASE(TestGetFloatFromDouble) { double buf = 893.162; SqlLen reslen = sizeof(buf); - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &buf, reslen, &reslen, &offset); @@ -635,7 +635,7 @@ BOOST_AUTO_TEST_CASE(TestGetIntFromString) { char buf[] = "39"; SqlLen reslen = sizeof(buf); - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf, reslen, &reslen, &offset); @@ -660,7 +660,7 @@ BOOST_AUTO_TEST_CASE(TestGetIntFromFloat) { float buf = -107.49f; SqlLen reslen = sizeof(buf); - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_FLOAT, &buf, reslen, &reslen, &offset); @@ -685,7 +685,7 @@ BOOST_AUTO_TEST_CASE(TestGetIntFromDouble) { double buf = 42.97f; SqlLen reslen = sizeof(buf); - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_DOUBLE, &buf, reslen, &reslen, &offset); @@ -710,7 +710,7 @@ BOOST_AUTO_TEST_CASE(TestGetIntFromBigint) { uint64_t buf = 19; SqlLen reslen = sizeof(buf); - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_BIGINT, &buf, reslen, &reslen, &offset); @@ -744,8 +744,8 @@ BOOST_AUTO_TEST_CASE(TestGetIntWithOffset) { 42, sizeof(uint64_t) } }; - size_t offset = 0; - size_t* offsetPtr = &offset; + int offset = 0; + int* offsetPtr = &offset; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_UNSIGNED_BIGINT, &buf[0].val, sizeof(buf[0].val), &buf[0].reslen, &offsetPtr); @@ -779,8 +779,8 @@ BOOST_AUTO_TEST_CASE(TestSetStringWithOffset) { "", 0 } }; - size_t offset = 0; - size_t* offsetPtr = &offset; + int offset = 0; + int* offsetPtr = &offset; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0].val, sizeof(buf[0].val), &buf[0].reslen, &offsetPtr); @@ -814,8 +814,8 @@ BOOST_AUTO_TEST_CASE(TestGetDateFromString) char buf[] = "1999-02-22"; SqlLen reslen = sizeof(buf); - size_t offset = 0; - size_t* offsetPtr = &offset; + int offset = 0; + int* offsetPtr = &offset; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0], sizeof(buf), &reslen, &offsetPtr); @@ -837,16 +837,11 @@ BOOST_AUTO_TEST_CASE(TestGetDateFromString) BOOST_AUTO_TEST_CASE(TestGetTimestampFromString) { - LOG_MSG("Test\n"); - LOG_MSG("Test\n"); - LOG_MSG("Test\n"); - LOG_MSG("Test\n"); - char buf[] = "2018-11-01 17:45:59"; SqlLen reslen = sizeof(buf); - size_t offset = 0; - size_t* offsetPtr = &offset; + int offset = 0; + int* offsetPtr = &offset; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_CHAR, &buf[0], sizeof(buf), &reslen, &offsetPtr); @@ -876,8 +871,8 @@ BOOST_AUTO_TEST_CASE(TestGetDateFromDate) SqlLen reslen = sizeof(buf); - size_t offset = 0; - size_t* offsetPtr = &offset; + int offset = 0; + int* offsetPtr = &offset; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen, &offsetPtr); @@ -907,8 +902,8 @@ BOOST_AUTO_TEST_CASE(TestGetTimestampFromDate) SqlLen reslen = sizeof(buf); - size_t offset = 0; - size_t* offsetPtr = &offset; + int offset = 0; + int* offsetPtr = &offset; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TDATE, &buf, sizeof(buf), &reslen, &offsetPtr); @@ -942,8 +937,8 @@ BOOST_AUTO_TEST_CASE(TestGetTimestampFromTimestamp) SqlLen reslen = sizeof(buf); - size_t offset = 0; - size_t* offsetPtr = &offset; + int offset = 0; + int* offsetPtr = &offset; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen, &offsetPtr); @@ -978,8 +973,8 @@ BOOST_AUTO_TEST_CASE(TestGetDateFromTimestamp) SqlLen reslen = sizeof(buf); - size_t offset = 0; - size_t* offsetPtr = &offset; + int offset = 0; + int* offsetPtr = &offset; ApplicationDataBuffer appBuf(IGNITE_ODBC_C_TYPE_TTIMESTAMP, &buf, sizeof(buf), &reslen, &offsetPtr); diff --git a/modules/platforms/cpp/odbc-test/src/column_test.cpp b/modules/platforms/cpp/odbc-test/src/column_test.cpp index 4864a6a1a078b..6cbea8b648f4b 100644 --- a/modules/platforms/cpp/odbc-test/src/column_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/column_test.cpp @@ -66,7 +66,7 @@ BOOST_AUTO_TEST_CASE(TestColumnShort) short shortBuf = 0; SqlLen reslen = 0; - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_SIGNED_SHORT, &shortBuf, sizeof(shortBuf), &reslen, &offset); @@ -114,7 +114,7 @@ BOOST_AUTO_TEST_CASE(TestColumnString) char strBuf[1024] = {}; SqlLen reslen = 0; - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, &offset); @@ -164,7 +164,7 @@ BOOST_AUTO_TEST_CASE(TestColumnStringSeveral) std::string strBuf(data.size() / 3 + 2, 0); SqlLen reslen = 0; - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf[0], strBuf.size(), &reslen, &offset); @@ -246,7 +246,7 @@ BOOST_AUTO_TEST_CASE(TestColumnMultiString) char strBuf[1024] = {}; SqlLen reslen = 0; - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, &offset); @@ -287,4 +287,4 @@ BOOST_AUTO_TEST_CASE(TestColumnMultiString) BOOST_REQUIRE(column2.GetUnreadDataLength() == 0); } -BOOST_AUTO_TEST_SUITE_END() \ No newline at end of file +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 7c10527b4818f..f0168bdf3118e 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -174,8 +174,11 @@ struct QueriesTestSuiteFixture SQLRETURN ret; - TestType in1(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); - TestType in2(8, 7, 6, 5, "4", 3.0f, 2.0, false, Guid(1, 0), BinaryUtils::MakeDateGmt(1976, 1, 12), BinaryUtils::MakeTimestampGmt(1978, 8, 21, 23, 13, 45, 456)); + TestType in1(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), + BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); + + TestType in2(8, 7, 6, 5, "4", 3.0f, 2.0, false, Guid(1, 0), BinaryUtils::MakeDateGmt(1976, 1, 12), + BinaryUtils::MakeTimestampGmt(1978, 8, 21, 23, 13, 45, 456)); testCache.Put(1, in1); testCache.Put(2, in2); @@ -335,8 +338,11 @@ BOOST_AUTO_TEST_CASE(TestTwoRowsString) SQLRETURN ret; - TestType in1(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); - TestType in2(8, 7, 6, 5, "4", 3.0f, 2.0, false, Guid(1, 0), BinaryUtils::MakeDateGmt(1976, 1, 12), BinaryUtils::MakeTimestampGmt(1978, 8, 21, 23, 13, 45, 999999999)); + TestType in1(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), + BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); + + TestType in2(8, 7, 6, 5, "4", 3.0f, 2.0, false, Guid(1, 0), BinaryUtils::MakeDateGmt(1976, 1, 12), + BinaryUtils::MakeTimestampGmt(1978, 8, 21, 23, 13, 45, 999999999)); testCache.Put(1, in1); testCache.Put(2, in2); @@ -430,7 +436,8 @@ BOOST_AUTO_TEST_CASE(TestOneRowString) SQLRETURN ret; - TestType in(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); + TestType in(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), + BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); testCache.Put(1, in); @@ -493,7 +500,8 @@ BOOST_AUTO_TEST_CASE(TestOneRowStringLen) SQLRETURN ret; - TestType in(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); + TestType in(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), + BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); testCache.Put(1, in); @@ -535,4 +543,128 @@ BOOST_AUTO_TEST_CASE(TestOneRowStringLen) BOOST_CHECK(ret == SQL_NO_DATA); } +BOOST_AUTO_TEST_CASE(TestDataAtExecution) +{ + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); + + SQLRETURN ret; + + TestType in1(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), + BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); + + TestType in2(8, 7, 6, 5, "4", 3.0f, 2.0, false, Guid(1, 0), BinaryUtils::MakeDateGmt(1976, 1, 12), + BinaryUtils::MakeTimestampGmt(1978, 8, 21, 23, 13, 45, 999999999)); + + testCache.Put(1, in1); + testCache.Put(2, in2); + + const size_t columnsCnt = 11; + + SQLLEN columnLens[columnsCnt] = { 0 }; + SQLCHAR columns[columnsCnt][ODBC_BUFFER_SIZE] = { 0 }; + + // Binding columns. + for (SQLSMALLINT i = 0; i < columnsCnt; ++i) + { + ret = SQLBindCol(stmt, i + 1, SQL_C_CHAR, &columns[i], ODBC_BUFFER_SIZE, &columnLens[i]); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + } + + SQLCHAR request[] = "SELECT i8Field, i16Field, i32Field, i64Field, strField, " + "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType " + "WHERE i32Field = ? AND strField = ?"; + + ret = SQLPrepare(stmt, request, SQL_NTS); + + SQLLEN ind1 = 1; + SQLLEN ind2 = 2; + + SQLLEN len1 = SQL_DATA_AT_EXEC; + SQLLEN len2 = SQL_LEN_DATA_AT_EXEC(static_cast(in1.strField.size())); + + ret = SQLBindParam(stmt, 1, SQL_C_SLONG, SQL_INTEGER, 100, 100, &ind1, &len1); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindParam(stmt, 2, SQL_C_CHAR, SQL_VARCHAR, 100, 100, &ind2, &len2); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLExecute(stmt); + + BOOST_REQUIRE_EQUAL(ret, SQL_NEED_DATA); + + void* oind; + + ret = SQLParamData(stmt, &oind); + + BOOST_REQUIRE_EQUAL(ret, SQL_NEED_DATA); + + if (oind == &ind1) + ret = SQLPutData(stmt, &in1.i32Field, 0); + else if (oind == &ind2) + ret = SQLPutData(stmt, (SQLPOINTER)in1.strField.c_str(), (SQLLEN)in1.strField.size()); + else + BOOST_FAIL("Unknown indicator value"); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLParamData(stmt, &oind); + + BOOST_REQUIRE_EQUAL(ret, SQL_NEED_DATA); + + if (oind == &ind1) + ret = SQLPutData(stmt, &in1.i32Field, 0); + else if (oind == &ind2) + ret = SQLPutData(stmt, (SQLPOINTER)in1.strField.c_str(), (SQLLEN)in1.strField.size()); + else + BOOST_FAIL("Unknown indicator value"); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLParamData(stmt, &oind); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFetch(stmt); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_EQUAL(std::string(reinterpret_cast(columns[0])), "1"); + BOOST_CHECK_EQUAL(std::string(reinterpret_cast(columns[1])), "2"); + BOOST_CHECK_EQUAL(std::string(reinterpret_cast(columns[2])), "3"); + BOOST_CHECK_EQUAL(std::string(reinterpret_cast(columns[3])), "4"); + BOOST_CHECK_EQUAL(std::string(reinterpret_cast(columns[4])), "5"); + BOOST_CHECK_EQUAL(std::string(reinterpret_cast(columns[5])), "6"); + BOOST_CHECK_EQUAL(std::string(reinterpret_cast(columns[6])), "7"); + BOOST_CHECK_EQUAL(std::string(reinterpret_cast(columns[7])), "1"); + BOOST_CHECK_EQUAL(std::string(reinterpret_cast(columns[8])), "00000000-0000-0008-0000-000000000009"); + // Such format is used because Date returned as Timestamp. + BOOST_CHECK_EQUAL(std::string(reinterpret_cast(columns[9])), "1987-06-05 00:00:00"); + BOOST_CHECK_EQUAL(std::string(reinterpret_cast(columns[10])), "1998-12-27 01:02:03"); + + BOOST_CHECK_EQUAL(columnLens[0], 1); + BOOST_CHECK_EQUAL(columnLens[1], 1); + BOOST_CHECK_EQUAL(columnLens[2], 1); + BOOST_CHECK_EQUAL(columnLens[3], 1); + BOOST_CHECK_EQUAL(columnLens[4], 1); + BOOST_CHECK_EQUAL(columnLens[5], 1); + BOOST_CHECK_EQUAL(columnLens[6], 1); + BOOST_CHECK_EQUAL(columnLens[7], 1); + BOOST_CHECK_EQUAL(columnLens[8], 36); + BOOST_CHECK_EQUAL(columnLens[9], 19); + BOOST_CHECK_EQUAL(columnLens[10], 19); + + ret = SQLFetch(stmt); + BOOST_CHECK(ret == SQL_NO_DATA); +} + + BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc-test/src/row_test.cpp b/modules/platforms/cpp/odbc-test/src/row_test.cpp index 1fcd43e83f56a..f38e9c5b58f1d 100644 --- a/modules/platforms/cpp/odbc-test/src/row_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/row_test.cpp @@ -82,7 +82,7 @@ void CheckRowData(Row& row, size_t rowIdx) char strBuf[1024]; SQLGUID guidBuf; char bitBuf; - size_t* offset = 0; + int* offset = 0; ApplicationDataBuffer appLongBuf(type_traits::IGNITE_ODBC_C_TYPE_SIGNED_LONG, &longBuf, sizeof(longBuf), &reslen, &offset); ApplicationDataBuffer appStrBuf(type_traits::IGNITE_ODBC_C_TYPE_CHAR, &strBuf, sizeof(strBuf), &reslen, &offset); @@ -206,4 +206,4 @@ BOOST_AUTO_TEST_CASE(TestTwoRows) CheckRowData(row, 1); } -BOOST_AUTO_TEST_SUITE_END() \ No newline at end of file +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am index 1781bc09ac4aa..3c8b37a5e97f2 100644 --- a/modules/platforms/cpp/odbc/Makefile.am +++ b/modules/platforms/cpp/odbc/Makefile.am @@ -34,7 +34,8 @@ AM_CXXFLAGS = \ -std=c++0x libignite_odbc_la_LIBADD = \ - @top_srcdir@/binary/libignite-binary.la + @top_srcdir@/binary/libignite-binary.la \ + -lodbcinst libignite_odbc_la_LDFLAGS = \ -no-undefined \ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc.h b/modules/platforms/cpp/odbc/include/ignite/odbc.h index 40158ddf51f08..ec0861c565e19 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc.h @@ -252,6 +252,9 @@ namespace ignite SQLSMALLINT scope, SQLSMALLINT nullable); + SQLRETURN SQLParamData(SQLHSTMT stmt, SQLPOINTER* value); + + SQLRETURN SQLPutData(SQLHSTMT stmt, SQLPOINTER data, SQLLEN strLengthOrIndicator); } // namespace ignite -#endif //_IGNITE_ODBC \ No newline at end of file +#endif //_IGNITE_ODBC diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h b/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h index ed243593d896b..0ce781810be49 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/app/application_data_buffer.h @@ -56,7 +56,8 @@ namespace ignite * @param reslen Resulting data length. * @param offset Pointer to buffer and reslen offset pointer. */ - ApplicationDataBuffer(type_traits::IgniteSqlType type, void* buffer, SqlLen buflen, SqlLen* reslen, size_t** offset = 0); + ApplicationDataBuffer(type_traits::IgniteSqlType type, void* buffer, + SqlLen buflen, SqlLen* reslen, int** offset = 0); /** * Copy constructor. @@ -83,7 +84,7 @@ namespace ignite * * @param offset Pointer to offset pointer. */ - void SetPtrToOffsetPtr(size_t** offset) + void SetPtrToOffsetPtr(int** offset) { this->offset = offset; } @@ -271,6 +272,20 @@ namespace ignite */ const SqlLen* GetResLen() const; + /** + * Get raw data. + * + * @return Buffer data. + */ + void* GetData(); + + /** + * Get result data length. + * + * @return Data length pointer. + */ + SqlLen* GetResLen(); + /** * Get buffer size in bytes. * @@ -281,21 +296,42 @@ namespace ignite return buflen; } - private: /** - * Get raw data. + * Check if the data is going to be provided at execution. * - * @return Buffer data. + * @return True if the data is going to be provided + * at execution. */ - void* GetData(); + bool IsDataAtExec() const; /** - * Get result data length. + * Get size of the data that is going to be provided at + * execution. * - * @return Data length pointer. + * @return Size of the data that is going to be provided + * at execution. */ - SqlLen* GetResLen(); + size_t GetDataAtExecSize() const; + + /** + * Get size of the input buffer. + * + * @return Input buffer size, or zero if the data is going + * to be provided at execution. + */ + size_t GetInputSize() const; + + /** + * Get buffer type. + * + * @return Buffer type. + */ + type_traits::IgniteSqlType GetType() const + { + return type; + } + private: /** * Put value of numeric type in the buffer. * @@ -374,7 +410,7 @@ namespace ignite SqlLen* reslen; /** Pointer to implementation pointer to application offset */ - size_t** offset; + int** offset; }; /** Column binging map type alias. */ @@ -383,4 +419,4 @@ namespace ignite } } -#endif //_IGNITE_ODBC_APP_APPLICATION_DATA_BUFFER \ No newline at end of file +#endif //_IGNITE_ODBC_APP_APPLICATION_DATA_BUFFER diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h b/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h index d1ea6970b0bd7..0bd93952c2717 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/app/parameter.h @@ -90,6 +90,25 @@ namespace ignite */ ApplicationDataBuffer& GetBuffer(); + /** + * Reset stored at-execution data. + */ + void ResetStoredData(); + + /** + * Check if all the at-execution data has been stored. + * @return + */ + bool IsDataReady() const; + + /** + * Put at-execution data. + * + * @param data Data buffer pointer. + * @param len Data length. + */ + void PutData(void* data, SqlLen len); + private: /** Underlying data buffer. */ ApplicationDataBuffer buffer; @@ -102,6 +121,12 @@ namespace ignite /** IPD decimal digits. */ int16_t decDigits; + + /** User provided null data at execution. */ + bool nullData; + + /** Stored at-execution data. */ + std::vector storedData; }; /** Parameter binging map type alias. */ @@ -110,4 +135,4 @@ namespace ignite } } -#endif //_IGNITE_ODBC_APP_PARAMETER \ No newline at end of file +#endif //_IGNITE_ODBC_APP_PARAMETER diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h index b01ec76fd1e1a..354cf0874ec87 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h @@ -21,6 +21,8 @@ #include #include "system/odbc_constants.h" +#include + namespace ignite { namespace odbc @@ -42,7 +44,10 @@ namespace ignite SQL_RESULT_ERROR, /** No more data. */ - SQL_RESULT_NO_DATA + SQL_RESULT_NO_DATA, + + /** No more data. */ + SQL_RESULT_NEED_DATA }; /** @@ -68,9 +73,15 @@ namespace ignite */ SQL_STATE_01S02_OPTION_VALUE_CHANGED, + /** String data, length mismatch. */ + SQL_STATE_22026_DATA_LENGTH_MISMATCH, + /** Invalid cursor state. */ SQL_STATE_24000_INVALID_CURSOR_STATE, + /** Invalid descriptor index. */ + SQL_STATE_07009_INVALID_DESCRIPTOR_INDEX, + /** * The driver was unable to establish a connection with the data * source. @@ -99,6 +110,9 @@ namespace ignite */ SQL_STATE_HY001_MEMORY_ALLOCATION, + /** Invalid use of null pointer. */ + SQL_STATE_HY009_INVALID_USE_OF_NULL_POINTER, + /** Function sequence error. */ SQL_STATE_HY010_SEQUENCE_ERROR, @@ -221,4 +235,4 @@ namespace ignite } } -#endif //_IGNITE_ODBC_COMMON_TYPES \ No newline at end of file +#endif //_IGNITE_ODBC_COMMON_TYPES diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h index b5f385efb279e..4fe4f1b07ed72 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h @@ -167,10 +167,7 @@ namespace ignite * * @param port Server port. */ - void SetTcpPort(uint16_t port) - { - arguments[Key::port] = common::LexicalCast(port); - } + void SetTcpPort(uint16_t port); /** * Get DSN. @@ -217,10 +214,7 @@ namespace ignite * * @param server Server host. */ - void SetHost(const std::string& server) - { - arguments[Key::server] = server; - } + void SetHost(const std::string& server); /** * Get cache. @@ -257,10 +251,7 @@ namespace ignite * * @param address Address. */ - void SetAddress(const std::string& address) - { - arguments[Key::address] = address; - } + void SetAddress(const std::string& address); /** * Get protocol version. @@ -357,4 +348,4 @@ namespace ignite } } -#endif //_IGNITE_ODBC_CONFIG_CONFIGURATION \ No newline at end of file +#endif //_IGNITE_ODBC_CONFIG_CONFIGURATION diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h index 97d586f64bdd4..35f1e98b4f140 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h @@ -81,14 +81,14 @@ namespace ignite * * @param ptr Column binding offset pointer. */ - void SetColumnBindOffsetPtr(size_t* ptr); + void SetColumnBindOffsetPtr(int* ptr); /** * Get column binding offset pointer. * * @return Column binding offset pointer. */ - size_t* GetColumnBindOffsetPtr(); + int* GetColumnBindOffsetPtr(); /** * Get number of columns in the result set. @@ -129,14 +129,14 @@ namespace ignite * * @param ptr Parameter binding offset pointer. */ - void SetParamBindOffsetPtr(size_t* ptr); + void SetParamBindOffsetPtr(int* ptr); /** * Get parameter binding offset pointer. * * @return Parameter binding offset pointer. */ - size_t* GetParamBindOffsetPtr(); + int* GetParamBindOffsetPtr(); /** * Get value of the column in the result set. @@ -328,9 +328,33 @@ namespace ignite */ uint16_t* GetRowStatusesPtr(); + /** + * Select next parameter data for which is required. + * + * @param paramPtr Pointer to param id stored here. + */ + void SelectParam(void** paramPtr); + + /** + * Puts data for previously selected parameter or column. + * + * @param data Data. + * @param len Data length. + */ + void PutData(void* data, SqlLen len); + private: IGNITE_NO_COPY_ASSIGNMENT(Statement); + /** + * Bind parameter. + * + * @param paramIdx Parameter index. + * @param param Parameter. + * @return Operation result. + */ + SqlResult InternalBindParameter(uint16_t paramIdx, const app::Parameter& param); + /** * Get value of the column in the result set. * @@ -487,6 +511,21 @@ namespace ignite */ SqlResult InternalAffectedRows(int64_t& rowCnt); + /** + * Select next parameter data for which is required. + * + * @param paramPtr Pointer to param id stored here. + */ + SqlResult InternalSelectParam(void** paramPtr); + + /** + * Puts data for previously selected parameter or column. + * + * @param data Data. + * @param len Data length. + */ + SqlResult InternalPutData(void* data, SqlLen len); + /** * Constructor. * Called by friend classes. @@ -514,12 +553,15 @@ namespace ignite uint16_t* rowStatuses; /** Offset added to pointers to change binding of parameters. */ - size_t* paramBindOffset; + int* paramBindOffset; - /* Offset added to pointers to change binding of column data. */ - size_t* columnBindOffset; + /** Offset added to pointers to change binding of column data. */ + int* columnBindOffset; + + /** Index of the parameter, which is currently being set. */ + uint16_t currentParamIdx; }; } } -#endif //_IGNITE_ODBC_STATEMENT \ No newline at end of file +#endif //_IGNITE_ODBC_STATEMENT diff --git a/modules/platforms/cpp/odbc/project/vs/module.def b/modules/platforms/cpp/odbc/project/vs/module.def index c94869bed2687..df7bd223b2bfe 100644 --- a/modules/platforms/cpp/odbc/project/vs/module.def +++ b/modules/platforms/cpp/odbc/project/vs/module.def @@ -35,7 +35,6 @@ EXPORTS SQLTables SQLBrowseConnect SQLColumnPrivileges - SQLDescribeParam SQLForeignKeys SQLMoreResults SQLNativeSql diff --git a/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp b/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp index 2190c0036fd1e..1438b0c9e238b 100644 --- a/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp +++ b/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp @@ -36,20 +36,32 @@ namespace ignite using ignite::impl::binary::BinaryUtils; ApplicationDataBuffer::ApplicationDataBuffer() : - type(type_traits::IGNITE_ODBC_C_TYPE_UNSUPPORTED), buffer(0), buflen(0), reslen(0), offset(0) + type(type_traits::IGNITE_ODBC_C_TYPE_UNSUPPORTED), + buffer(0), + buflen(0), + reslen(0), + offset(0) { // No-op. } ApplicationDataBuffer::ApplicationDataBuffer(type_traits::IgniteSqlType type, - void* buffer, SqlLen buflen, SqlLen* reslen, size_t** offset) : - type(type), buffer(buffer), buflen(buflen), reslen(reslen), offset(offset) + void* buffer, SqlLen buflen, SqlLen* reslen, int** offset) : + type(type), + buffer(buffer), + buflen(buflen), + reslen(reslen), + offset(offset) { // No-op. } ApplicationDataBuffer::ApplicationDataBuffer(const ApplicationDataBuffer & other) : - type(other.type), buffer(other.buffer), buflen(other.buflen), reslen(other.reslen), offset(other.offset) + type(other.type), + buffer(other.buffer), + buflen(other.buflen), + reslen(other.reslen), + offset(other.offset) { // No-op. } @@ -74,6 +86,10 @@ namespace ignite void ApplicationDataBuffer::PutNum(T value) { using namespace type_traits; + + SqlLen* resLenPtr = GetResLen(); + void* dataPtr = GetData(); + switch (type) { case IGNITE_ODBC_C_TYPE_SIGNED_TINYINT: @@ -151,10 +167,10 @@ namespace ignite case IGNITE_ODBC_C_TYPE_NUMERIC: { - if (GetData()) + if (dataPtr) { SQL_NUMERIC_STRUCT* out = - reinterpret_cast(GetData()); + reinterpret_cast(dataPtr); uint64_t uval = static_cast(value < 0 ? -value : value); @@ -172,27 +188,26 @@ namespace ignite case IGNITE_ODBC_C_TYPE_BINARY: case IGNITE_ODBC_C_TYPE_DEFAULT: { - if (GetData()) + if (dataPtr) { if (buflen >= sizeof(value)) { - memcpy(GetData(), &value, sizeof(value)); + memcpy(dataPtr, &value, sizeof(value)); - if (GetResLen()) - *GetResLen() = sizeof(value); + if (resLenPtr) + *resLenPtr = sizeof(value); } else { - memcpy(GetData(), &value, static_cast(buflen)); + memcpy(dataPtr, &value, static_cast(buflen)); - if (GetResLen()) - *GetResLen() = SQL_NO_TOTAL; + if (resLenPtr) + *resLenPtr = SQL_NO_TOTAL; } } - else if (GetResLen()) - { - *GetResLen() = sizeof(value); - } + else if (resLenPtr) + *resLenPtr = sizeof(value); + break; } @@ -212,8 +227,8 @@ namespace ignite default: { - if (GetResLen()) - *GetResLen() = SQL_NO_TOTAL; + if (resLenPtr) + *resLenPtr = SQL_NO_TOTAL; } } } @@ -221,9 +236,11 @@ namespace ignite template void ApplicationDataBuffer::PutNumToNumBuffer(Tin value) { - if (GetData()) + void* dataPtr = GetData(); + + if (dataPtr) { - Tbuf* out = reinterpret_cast(GetData()); + Tbuf* out = reinterpret_cast(dataPtr); *out = static_cast(value); } } @@ -257,11 +274,14 @@ namespace ignite { SqlLen charSize = static_cast(sizeof(OutCharT)); - if (GetData()) + SqlLen* resLenPtr = GetResLen(); + void* dataPtr = GetData(); + + if (dataPtr) { if (buflen >= charSize) { - OutCharT* out = reinterpret_cast(GetData()); + OutCharT* out = reinterpret_cast(dataPtr); SqlLen outLen = (buflen / charSize) - 1; @@ -273,38 +293,41 @@ namespace ignite out[toCopy] = 0; } - if (GetResLen()) + if (resLenPtr) { if (buflen >= static_cast((value.size() + 1) * charSize)) - *GetResLen() = static_cast(value.size()); + *resLenPtr = static_cast(value.size()); else - *GetResLen() = SQL_NO_TOTAL; + *resLenPtr = SQL_NO_TOTAL; } } - else if (GetResLen()) - *GetResLen() = value.size(); + else if (resLenPtr) + *resLenPtr = value.size(); } void ApplicationDataBuffer::PutRawDataToBuffer(void *data, size_t len) { SqlLen ilen = static_cast(len); - if (GetData()) + SqlLen* resLenPtr = GetResLen(); + void* dataPtr = GetData(); + + if (dataPtr) { size_t toCopy = static_cast(std::min(buflen, ilen)); - memcpy(GetData(), data, toCopy); + memcpy(dataPtr, data, toCopy); - if (GetResLen()) + if (resLenPtr) { if (buflen >= ilen) - *GetResLen() = ilen; + *resLenPtr = ilen; else - *GetResLen() = SQL_NO_TOTAL; + *resLenPtr = SQL_NO_TOTAL; } } - else if (GetResLen()) - *GetResLen() = ilen; + else if (resLenPtr) + *resLenPtr = ilen; } void ApplicationDataBuffer::PutInt8(int8_t value) @@ -411,8 +434,10 @@ namespace ignite default: { - if (GetResLen()) - *GetResLen() = SQL_NO_TOTAL; + SqlLen* resLenPtr = GetResLen(); + + if (resLenPtr) + *resLenPtr = SQL_NO_TOTAL; } } @@ -456,8 +481,10 @@ namespace ignite default: { - if (GetResLen()) - *GetResLen() = SQL_NO_TOTAL; + SqlLen* resLenPtr = GetResLen(); + + if (resLenPtr) + *resLenPtr = SQL_NO_TOTAL; } } } @@ -524,8 +551,10 @@ namespace ignite default: { - if (GetResLen()) - *GetResLen() = SQL_NO_TOTAL; + SqlLen* resLenPtr = GetResLen(); + + if (resLenPtr) + *resLenPtr = SQL_NO_TOTAL; } } @@ -534,13 +563,16 @@ namespace ignite void ApplicationDataBuffer::PutNull() { - if (GetResLen()) - *GetResLen() = SQL_NULL_DATA; + SqlLen* resLenPtr = GetResLen(); + + if (resLenPtr) + *resLenPtr = SQL_NULL_DATA; } void ApplicationDataBuffer::PutDecimal(const common::Decimal& value) { using namespace type_traits; + switch (type) { case IGNITE_ODBC_C_TYPE_SIGNED_TINYINT: @@ -612,8 +644,10 @@ namespace ignite case IGNITE_ODBC_C_TYPE_BINARY: default: { - if (GetResLen()) - *GetResLen() = SQL_NO_TOTAL; + SqlLen* resLenPtr = GetResLen(); + + if (resLenPtr) + *resLenPtr = SQL_NO_TOTAL; } } } @@ -626,28 +660,31 @@ namespace ignite BinaryUtils::DateToCTm(value, tmTime); + SqlLen* resLenPtr = GetResLen(); + void* dataPtr = GetData(); + switch (type) { case IGNITE_ODBC_C_TYPE_CHAR: { - char* buffer = reinterpret_cast(GetData()); + char* buffer = reinterpret_cast(dataPtr); if (buffer) { strftime(buffer, GetSize(), "%Y-%m-%d", &tmTime); - if (GetResLen()) - *GetResLen() = strlen(buffer); + if (resLenPtr) + *resLenPtr = strlen(buffer); } - else if (GetResLen()) - *GetResLen() = sizeof("HHHH-MM-DD") - 1; + else if (resLenPtr) + *resLenPtr = sizeof("HHHH-MM-DD") - 1; break; } case IGNITE_ODBC_C_TYPE_WCHAR: { - SQLWCHAR* buffer = reinterpret_cast(GetData()); + SQLWCHAR* buffer = reinterpret_cast(dataPtr); if (buffer) { @@ -662,18 +699,18 @@ namespace ignite buffer[toCopy] = 0; - if (GetResLen()) - *GetResLen() = toCopy; + if (resLenPtr) + *resLenPtr = toCopy; } - else if (GetResLen()) - *GetResLen() = sizeof("HHHH-MM-DD") - 1; + else if (resLenPtr) + *resLenPtr = sizeof("HHHH-MM-DD") - 1; break; } case IGNITE_ODBC_C_TYPE_TDATE: { - SQL_DATE_STRUCT* buffer = reinterpret_cast(GetData()); + SQL_DATE_STRUCT* buffer = reinterpret_cast(dataPtr); buffer->year = tmTime.tm_year + 1900; buffer->month = tmTime.tm_mon + 1; @@ -684,7 +721,7 @@ namespace ignite case IGNITE_ODBC_C_TYPE_TTIMESTAMP: { - SQL_TIMESTAMP_STRUCT* buffer = reinterpret_cast(GetData()); + SQL_TIMESTAMP_STRUCT* buffer = reinterpret_cast(dataPtr); buffer->year = tmTime.tm_year + 1900; buffer->month = tmTime.tm_mon + 1; @@ -700,11 +737,11 @@ namespace ignite case IGNITE_ODBC_C_TYPE_BINARY: case IGNITE_ODBC_C_TYPE_DEFAULT: { - if (GetData()) - memcpy(GetData(), &value, std::min(static_cast(buflen), sizeof(value))); + if (dataPtr) + memcpy(dataPtr, &value, std::min(static_cast(buflen), sizeof(value))); - if (GetResLen()) - *GetResLen() = sizeof(value); + if (resLenPtr) + *resLenPtr = sizeof(value); break; } @@ -723,8 +760,8 @@ namespace ignite case IGNITE_ODBC_C_TYPE_NUMERIC: default: { - if (GetResLen()) - *GetResLen() = SQL_NO_TOTAL; + if (resLenPtr) + *resLenPtr = SQL_NO_TOTAL; } } } @@ -737,28 +774,31 @@ namespace ignite BinaryUtils::TimestampToCTm(value, tmTime); + SqlLen* resLenPtr = GetResLen(); + void* dataPtr = GetData(); + switch (type) { case IGNITE_ODBC_C_TYPE_CHAR: { - char* buffer = reinterpret_cast(GetData()); + char* buffer = reinterpret_cast(dataPtr); if (buffer) { strftime(buffer, GetSize(), "%Y-%m-%d %H:%M:%S", &tmTime); - if (GetResLen()) - *GetResLen() = strlen(buffer); + if (resLenPtr) + *resLenPtr = strlen(buffer); } - else if (GetResLen()) - *GetResLen() = sizeof("HHHH-MM-DD HH:MM:SS") - 1; + else if (resLenPtr) + *resLenPtr = sizeof("HHHH-MM-DD HH:MM:SS") - 1; break; } case IGNITE_ODBC_C_TYPE_WCHAR: { - SQLWCHAR* buffer = reinterpret_cast(GetData()); + SQLWCHAR* buffer = reinterpret_cast(dataPtr); if (buffer) { @@ -773,18 +813,18 @@ namespace ignite buffer[toCopy] = 0; - if (GetResLen()) - *GetResLen() = toCopy; + if (resLenPtr) + *resLenPtr = toCopy; } - else if (GetResLen()) - *GetResLen() = sizeof("HHHH-MM-DD HH:MM:SS") - 1; + else if (resLenPtr) + *resLenPtr = sizeof("HHHH-MM-DD HH:MM:SS") - 1; break; } case IGNITE_ODBC_C_TYPE_TDATE: { - SQL_DATE_STRUCT* buffer = reinterpret_cast(GetData()); + SQL_DATE_STRUCT* buffer = reinterpret_cast(dataPtr); buffer->year = tmTime.tm_year + 1900; buffer->month = tmTime.tm_mon + 1; @@ -795,7 +835,7 @@ namespace ignite case IGNITE_ODBC_C_TYPE_TTIMESTAMP: { - SQL_TIMESTAMP_STRUCT* buffer = reinterpret_cast(GetData()); + SQL_TIMESTAMP_STRUCT* buffer = reinterpret_cast(dataPtr); buffer->year = tmTime.tm_year + 1900; buffer->month = tmTime.tm_mon + 1; @@ -811,11 +851,11 @@ namespace ignite case IGNITE_ODBC_C_TYPE_BINARY: case IGNITE_ODBC_C_TYPE_DEFAULT: { - if (GetData()) - memcpy(GetData(), &value, std::min(static_cast(buflen), sizeof(value))); + if (dataPtr) + memcpy(dataPtr, &value, std::min(static_cast(buflen), sizeof(value))); - if (GetResLen()) - *GetResLen() = sizeof(value); + if (resLenPtr) + *resLenPtr = sizeof(value); break; } @@ -834,8 +874,8 @@ namespace ignite case IGNITE_ODBC_C_TYPE_NUMERIC: default: { - if (GetResLen()) - *GetResLen() = SQL_NO_TOTAL; + if (resLenPtr) + *resLenPtr = SQL_NO_TOTAL; } } } @@ -849,8 +889,13 @@ namespace ignite { case IGNITE_ODBC_C_TYPE_CHAR: { + size_t paramLen = GetInputSize(); + + if (!paramLen) + break; + res.assign(reinterpret_cast(GetData()), - std::min(maxLen, static_cast(buflen))); + std::min(maxLen, paramLen)); break; } @@ -953,7 +998,12 @@ namespace ignite { case IGNITE_ODBC_C_TYPE_CHAR: { - std::string str(reinterpret_cast(GetData()), static_cast(buflen)); + size_t paramLen = GetInputSize(); + + if (!paramLen) + break; + + std::string str(reinterpret_cast(GetData()), paramLen); std::stringstream converter; @@ -1014,13 +1064,18 @@ namespace ignite { using namespace type_traits; - T res = 0; + T res = T(); switch (type) { case IGNITE_ODBC_C_TYPE_CHAR: { - std::string str = GetString(static_cast(buflen)); + size_t paramLen = GetInputSize(); + + if (!paramLen) + break; + + std::string str = GetString(paramLen); std::stringstream converter; @@ -1158,9 +1213,13 @@ namespace ignite case IGNITE_ODBC_C_TYPE_CHAR: { + size_t paramLen = GetInputSize(); + + if (!paramLen) + break; + std::string str = utility::SqlStringToString( - reinterpret_cast(GetData()), - static_cast(GetSize())); + reinterpret_cast(GetData()), static_cast(paramLen)); sscanf(str.c_str(), "%d-%d-%d %d:%d:%d", &tmTime.tm_year, &tmTime.tm_mon, &tmTime.tm_mday, &tmTime.tm_hour, &tmTime.tm_min, &tmTime.tm_sec); @@ -1217,9 +1276,13 @@ namespace ignite case IGNITE_ODBC_C_TYPE_CHAR: { + size_t paramLen = GetInputSize(); + + if (!paramLen) + break; + std::string str = utility::SqlStringToString( - reinterpret_cast(GetData()), - static_cast(GetSize())); + reinterpret_cast(GetData()), static_cast(paramLen)); sscanf(str.c_str(), "%d-%d-%d %d:%d:%d", &tmTime.tm_year, &tmTime.tm_mon, &tmTime.tm_mday, &tmTime.tm_hour, &tmTime.tm_min, &tmTime.tm_sec); @@ -1245,7 +1308,12 @@ namespace ignite { case IGNITE_ODBC_C_TYPE_CHAR: { - std::string str = GetString(static_cast(buflen)); + size_t paramLen = GetInputSize(); + + if (!paramLen) + break; + + std::string str = GetString(paramLen); std::stringstream converter; @@ -1315,6 +1383,101 @@ namespace ignite return utility::GetPointerWithOffset(ptr, **offset); } + + bool ApplicationDataBuffer::IsDataAtExec() const + { + const SqlLen* resLenPtr = GetResLen(); + + if (!resLenPtr) + return false; + + int32_t ilen = static_cast(*resLenPtr); + + return ilen <= SQL_LEN_DATA_AT_EXEC_OFFSET || ilen == SQL_DATA_AT_EXEC; + } + + size_t ApplicationDataBuffer::GetDataAtExecSize() const + { + using namespace type_traits; + + switch (type) + { + case IGNITE_ODBC_C_TYPE_WCHAR: + case IGNITE_ODBC_C_TYPE_CHAR: + case IGNITE_ODBC_C_TYPE_BINARY: + { + const SqlLen* resLenPtr = GetResLen(); + + if (!resLenPtr) + return 0; + + int32_t ilen = static_cast(*resLenPtr); + + if (ilen <= SQL_LEN_DATA_AT_EXEC_OFFSET) + ilen = static_cast(SQL_LEN_DATA_AT_EXEC(ilen)); + else + ilen = 0; + + if (type == IGNITE_ODBC_C_TYPE_WCHAR) + ilen *= 2; + + return ilen; + } + + case IGNITE_ODBC_C_TYPE_SIGNED_SHORT: + case IGNITE_ODBC_C_TYPE_UNSIGNED_SHORT: + return sizeof(short); + + case IGNITE_ODBC_C_TYPE_SIGNED_LONG: + case IGNITE_ODBC_C_TYPE_UNSIGNED_LONG: + return sizeof(long); + + case IGNITE_ODBC_C_TYPE_FLOAT: + return sizeof(float); + + case IGNITE_ODBC_C_TYPE_DOUBLE: + return sizeof(double); + + case IGNITE_ODBC_C_TYPE_BIT: + case IGNITE_ODBC_C_TYPE_SIGNED_TINYINT: + case IGNITE_ODBC_C_TYPE_UNSIGNED_TINYINT: + return sizeof(char); + + case IGNITE_ODBC_C_TYPE_SIGNED_BIGINT: + case IGNITE_ODBC_C_TYPE_UNSIGNED_BIGINT: + return sizeof(SQLBIGINT); + + case IGNITE_ODBC_C_TYPE_TDATE: + return sizeof(SQL_DATE_STRUCT); + + case IGNITE_ODBC_C_TYPE_TTIME: + return sizeof(SQL_TIME_STRUCT); + + case IGNITE_ODBC_C_TYPE_TTIMESTAMP: + return sizeof(SQL_TIMESTAMP_STRUCT); + + case IGNITE_ODBC_C_TYPE_NUMERIC: + return sizeof(SQL_NUMERIC_STRUCT); + + case IGNITE_ODBC_C_TYPE_GUID: + return sizeof(SQLGUID); + + case IGNITE_ODBC_C_TYPE_DEFAULT: + case IGNITE_ODBC_C_TYPE_UNSUPPORTED: + default: + break; + } + + return 0; + } + + size_t ApplicationDataBuffer::GetInputSize() const + { + if (!IsDataAtExec()) + return static_cast(GetSize()); + + return GetDataAtExecSize(); + } } } } diff --git a/modules/platforms/cpp/odbc/src/app/parameter.cpp b/modules/platforms/cpp/odbc/src/app/parameter.cpp index 3e1464259609f..d70ef6aab2da8 100644 --- a/modules/platforms/cpp/odbc/src/app/parameter.cpp +++ b/modules/platforms/cpp/odbc/src/app/parameter.cpp @@ -33,7 +33,9 @@ namespace ignite buffer(), sqlType(), columnSize(), - decDigits() + decDigits(), + nullData(false), + storedData() { // No-op. } @@ -43,7 +45,9 @@ namespace ignite buffer(buffer), sqlType(sqlType), columnSize(columnSize), - decDigits(decDigits) + decDigits(decDigits), + nullData(false), + storedData() { // No-op. } @@ -52,7 +56,9 @@ namespace ignite buffer(other.buffer), sqlType(other.sqlType), columnSize(other.columnSize), - decDigits(other.decDigits) + decDigits(other.decDigits), + nullData(other.nullData), + storedData(other.storedData) { // No-op. } @@ -74,67 +80,85 @@ namespace ignite void Parameter::Write(ignite::impl::binary::BinaryWriterImpl& writer) const { + if (buffer.GetInputSize() == SQL_NULL_DATA) + { + writer.WriteNull(); + + return; + } + + // Buffer to use to get data. + ApplicationDataBuffer buf(buffer); + + SqlLen storedDataLen = static_cast(storedData.size()); + + if (buffer.IsDataAtExec()) + { + buf = ApplicationDataBuffer(buffer.GetType(), + const_cast(&storedData[0]), storedDataLen, &storedDataLen); + } + switch (sqlType) { case SQL_CHAR: case SQL_VARCHAR: case SQL_LONGVARCHAR: { - utility::WriteString(writer, buffer.GetString(columnSize)); + utility::WriteString(writer, buf.GetString(columnSize)); break; } case SQL_SMALLINT: { - writer.WriteInt16(buffer.GetInt16()); + writer.WriteObject(buf.GetInt16()); break; } case SQL_INTEGER: { - writer.WriteInt32(buffer.GetInt32()); + writer.WriteObject(buf.GetInt32()); break; } case SQL_FLOAT: { - writer.WriteFloat(buffer.GetFloat()); + writer.WriteObject(buf.GetFloat()); break; } case SQL_DOUBLE: { - writer.WriteDouble(buffer.GetDouble()); + writer.WriteObject(buf.GetDouble()); break; } case SQL_TINYINT: { - writer.WriteInt8(buffer.GetInt8()); + writer.WriteObject(buf.GetInt8()); break; } case SQL_BIT: { - writer.WriteBool(buffer.GetInt8() != 0); + writer.WriteObject(buf.GetInt8() != 0); break; } case SQL_BIGINT: { - writer.WriteInt64(buffer.GetInt64()); + writer.WriteObject(buf.GetInt64()); break; } case SQL_DATE: { - writer.WriteDate(buffer.GetDate()); + writer.WriteDate(buf.GetDate()); break; } case SQL_TIMESTAMP: { - writer.WriteTimestamp(buffer.GetTimestamp()); + writer.WriteTimestamp(buf.GetTimestamp()); break; } @@ -142,14 +166,23 @@ namespace ignite case SQL_VARBINARY: case SQL_LONGVARBINARY: { - writer.WriteInt8Array(reinterpret_cast(buffer.GetData()), - static_cast(buffer.GetSize())); + const ApplicationDataBuffer& constRef = buf; + + const SqlLen* resLenPtr = constRef.GetResLen(); + + if (!resLenPtr) + break; + + int32_t paramLen = static_cast(*resLenPtr); + + writer.WriteInt8Array(reinterpret_cast(constRef.GetData()), paramLen); + break; } case SQL_GUID: { - writer.WriteGuid(buffer.GetGuid()); + writer.WriteGuid(buf.GetGuid()); break; } @@ -157,7 +190,7 @@ namespace ignite case SQL_DECIMAL: { common::Decimal dec; - buffer.GetDecimal(dec); + buf.GetDecimal(dec); utility::WriteDecimal(writer, dec); @@ -169,10 +202,67 @@ namespace ignite } } - ApplicationDataBuffer & Parameter::GetBuffer() + ApplicationDataBuffer& Parameter::GetBuffer() { return buffer; } + + void Parameter::ResetStoredData() + { + storedData.clear(); + + if (buffer.IsDataAtExec()) + storedData.reserve(buffer.GetDataAtExecSize()); + } + + bool Parameter::IsDataReady() const + { + return !buffer.IsDataAtExec() || + storedData.size() == buffer.GetDataAtExecSize(); + } + + void Parameter::PutData(void* data, SqlLen len) + { + if (len == SQL_DEFAULT_PARAM) + return; + + if (len == SQL_NULL_DATA) + { + nullData = true; + + return; + } + + if (buffer.GetType() == type_traits::IGNITE_ODBC_C_TYPE_CHAR || + buffer.GetType() == type_traits::IGNITE_ODBC_C_TYPE_BINARY) + { + SqlLen slen = len; + + if (buffer.GetType() == type_traits::IGNITE_ODBC_C_TYPE_CHAR && slen == SQL_NTSL) + { + const char* str = reinterpret_cast(data); + + slen = strlen(str); + } + + if (slen <= 0) + return; + + size_t beginPos = storedData.size(); + + storedData.resize(storedData.size() + static_cast(slen)); + + memcpy(&storedData[beginPos], data, static_cast(slen)); + + return; + } + + size_t dataSize = buffer.GetDataAtExecSize(); + + storedData.resize(dataSize); + + memcpy(&storedData[0], data, dataSize); + } } } } diff --git a/modules/platforms/cpp/odbc/src/common_types.cpp b/modules/platforms/cpp/odbc/src/common_types.cpp index 276d9fdfafc4d..36d11c2c13602 100644 --- a/modules/platforms/cpp/odbc/src/common_types.cpp +++ b/modules/platforms/cpp/odbc/src/common_types.cpp @@ -37,6 +37,9 @@ namespace ignite case SQL_RESULT_NO_DATA: return SQL_NO_DATA; + case SQL_RESULT_NEED_DATA: + return SQL_NEED_DATA; + case SQL_RESULT_ERROR: default: return SQL_ERROR; diff --git a/modules/platforms/cpp/odbc/src/config/configuration.cpp b/modules/platforms/cpp/odbc/src/config/configuration.cpp index 74ccaaf5c05c0..cc2cc5d897912 100644 --- a/modules/platforms/cpp/odbc/src/config/configuration.cpp +++ b/modules/platforms/cpp/odbc/src/config/configuration.cpp @@ -141,6 +141,33 @@ namespace ignite } } + void Configuration::SetTcpPort(uint16_t port) + { + arguments[Key::port] = common::LexicalCast(port); + + ArgumentMap::const_iterator it = arguments.find(Key::address); + + if (it == arguments.end()) + endPoint.port = port; + } + + void Configuration::SetHost(const std::string& server) + { + arguments[Key::server] = server; + + ArgumentMap::const_iterator it = arguments.find(Key::address); + + if (it == arguments.end()) + endPoint.host = server; + } + + void Configuration::SetAddress(const std::string& address) + { + arguments[Key::address] = address; + + ParseAddress(address, endPoint); + } + ProtocolVersion Configuration::GetProtocolVersion() const { ArgumentMap::const_iterator it = arguments.find(Key::protocolVersion); diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp index ee2c22b668c75..341ab7fea2863 100644 --- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp +++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp @@ -62,7 +62,8 @@ namespace ignite DBG_STR_CASE(SQL_CATALOG_TERM); DBG_STR_CASE(SQL_TABLE_TERM); DBG_STR_CASE(SQL_SCHEMA_TERM); - DBG_STR_CASE(SQL_ASYNC_DBC_FUNCTIONS); + DBG_STR_CASE(SQL_NEED_LONG_DATA_LEN); +// DBG_STR_CASE(SQL_ASYNC_DBC_FUNCTIONS); DBG_STR_CASE(SQL_ASYNC_NOTIFICATION); DBG_STR_CASE(SQL_GETDATA_EXTENSIONS); DBG_STR_CASE(SQL_ODBC_INTERFACE_CONFORMANCE); @@ -92,6 +93,8 @@ namespace ignite DBG_STR_CASE(SQL_SQL92_PREDICATES); DBG_STR_CASE(SQL_SQL92_RELATIONAL_JOIN_OPERATORS); DBG_STR_CASE(SQL_SQL92_VALUE_EXPRESSIONS); + DBG_STR_CASE(SQL_STATIC_CURSOR_ATTRIBUTES1); + DBG_STR_CASE(SQL_STATIC_CURSOR_ATTRIBUTES2); default: break; } @@ -104,7 +107,7 @@ namespace ignite ConnectionInfo::ConnectionInfo() : strParams(), intParams(), shortParams() { - //========================= String Params ========================= + //======================= String Params ======================= // Driver name. strParams[SQL_DRIVER_NAME] = "Apache Ignite"; strParams[SQL_DBMS_NAME] = "Apache Ignite"; @@ -170,8 +173,16 @@ namespace ignite strParams[SQL_SCHEMA_TERM] = "schema"; #endif // SQL_SCHEMA_TERM +#ifdef SQL_NEED_LONG_DATA_LEN + // A character string: "Y" if the data source needs the length + // of a long data value (the data type is SQL_LONGVARCHAR, + // SQL_LONGVARBINARY) before that value is sent to the data + // source, "N" if it does not. + strParams[SQL_NEED_LONG_DATA_LEN ] = "Y"; +#endif // SQL_NEED_LONG_DATA_LEN + #ifdef SQL_ASYNC_DBC_FUNCTIONS - //======================== Integer Params ========================= + //====================== Integer Params ======================= // Indicates if the driver can execute functions asynchronously // on the connection handle. // SQL_ASYNC_DBC_CAPABLE = The driver can execute connection @@ -342,7 +353,23 @@ namespace ignite SQL_SRJO_NATURAL_JOIN | SQL_SRJO_INTERSECT_JOIN | SQL_SRJO_UNION_JOIN; #endif // SQL_SQL92_RELATIONAL_JOIN_OPERATORS - //========================= Short Params ========================== +#ifdef SQL_STATIC_CURSOR_ATTRIBUTES1 + // Bitmask that describes the attributes of a static cursor that + // are supported by the driver. This bitmask contains the first + // subset of attributes; for the second subset, see + // SQL_STATIC_CURSOR_ATTRIBUTES2. + intParams[SQL_STATIC_CURSOR_ATTRIBUTES1] = SQL_CA1_NEXT; +#endif //SQL_STATIC_CURSOR_ATTRIBUTES1 + +#ifdef SQL_STATIC_CURSOR_ATTRIBUTES2 + // Bitmask that describes the attributes of a static cursor that + // are supported by the driver. This bitmask contains the second + // subset of attributes; for the first subset, see + // SQL_STATIC_CURSOR_ATTRIBUTES1. + intParams[SQL_STATIC_CURSOR_ATTRIBUTES2] = 0; +#endif //SQL_STATIC_CURSOR_ATTRIBUTES2 + + //======================= Short Params ======================== #ifdef SQL_MAX_CONCURRENT_ACTIVITIES // The maximum number of active statements that the driver can // support for a connection. Zero mean no limit. diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp index cffecdf5798b3..cbbb2f7e3f550 100644 --- a/modules/platforms/cpp/odbc/src/connection.cpp +++ b/modules/platforms/cpp/odbc/src/connection.cpp @@ -52,7 +52,7 @@ namespace ignite { // No-op. } - + const config::ConnectionInfo& Connection::GetInfo() const { // Connection info is constant and the same for all connections now. diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp index 8553ee47d5188..1b654d26480a3 100644 --- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp +++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp @@ -40,9 +40,15 @@ namespace /** SQL state 01S01 constant. */ const std::string STATE_01S01 = "01S01"; + /** SQL state 22026 constant. */ + const std::string STATE_22026 = "22026"; + /** SQL state 24000 constant. */ const std::string STATE_24000 = "24000"; + /** SQL state 07009 constant. */ + const std::string STATE_07009 = "07009"; + /** SQL state 08001 constant. */ const std::string STATE_08001 = "08001"; @@ -58,6 +64,9 @@ namespace /** SQL state HY001 constant. */ const std::string STATE_HY001 = "HY001"; + /** SQL state HY009 constant. */ + const std::string STATE_HY009 = "HY009"; + /** SQL state HY010 constant. */ const std::string STATE_HY010 = "HY010"; @@ -199,9 +208,15 @@ namespace ignite case SQL_STATE_01S01_ERROR_IN_ROW: return STATE_01S01; + case SQL_STATE_22026_DATA_LENGTH_MISMATCH: + return STATE_22026; + case SQL_STATE_24000_INVALID_CURSOR_STATE: return STATE_24000; + case SQL_STATE_07009_INVALID_DESCRIPTOR_INDEX: + return STATE_07009; + case SQL_STATE_08001_CANNOT_CONNECT: return STATE_08001; @@ -217,6 +232,9 @@ namespace ignite case SQL_STATE_HY001_MEMORY_ALLOCATION: return STATE_HY001; + case SQL_STATE_HY009_INVALID_USE_OF_NULL_POINTER: + return STATE_HY009; + case SQL_STATE_HY010_SEQUENCE_ERROR: return STATE_HY010; diff --git a/modules/platforms/cpp/odbc/src/entry_points.cpp b/modules/platforms/cpp/odbc/src/entry_points.cpp index c3d24bb960c9b..08016cc678a75 100644 --- a/modules/platforms/cpp/odbc/src/entry_points.cpp +++ b/modules/platforms/cpp/odbc/src/entry_points.cpp @@ -399,6 +399,19 @@ SQLRETURN SQL_API SQLSpecialColumns(SQLHSTMT stmt, tableNameLen, scope, nullable); } +SQLRETURN SQL_API SQLParamData(SQLHSTMT stmt, + SQLPOINTER* value) +{ + return ignite::SQLParamData(stmt, value); +} + +SQLRETURN SQL_API SQLPutData(SQLHSTMT stmt, + SQLPOINTER data, + SQLLEN strLengthOrIndicator) +{ + return ignite::SQLPutData(stmt, data, strLengthOrIndicator); +} + // // ==== Not implemented ==== // @@ -467,21 +480,6 @@ SQLRETURN SQL_API SQLGetStmtOption(SQLHSTMT stmt, return SQL_SUCCESS; } -SQLRETURN SQL_API SQLParamData(SQLHSTMT stmt, - SQLPOINTER* value) -{ - LOG_MSG("SQLParamData called\n"); - return SQL_SUCCESS; -} - -SQLRETURN SQL_API SQLPutData(SQLHSTMT stmt, - SQLPOINTER data, - SQLLEN strLengthOrIndicator) -{ - LOG_MSG("SQLPutData called\n"); - return SQL_SUCCESS; -} - SQLRETURN SQL_API SQLSetConnectOption(SQLHDBC conn, SQLUSMALLINT option, SQLULEN value) @@ -665,6 +663,7 @@ SQLRETURN SQL_API SQLColumnPrivileges(SQLHSTMT stmt, return SQL_SUCCESS; } +/* SQLRETURN SQL_API SQLDescribeParam(SQLHSTMT stmt, SQLUSMALLINT paramNum, SQLSMALLINT* dataType, @@ -675,6 +674,7 @@ SQLRETURN SQL_API SQLDescribeParam(SQLHSTMT stmt, LOG_MSG("SQLDescribeParam called\n"); return SQL_SUCCESS; } +*/ SQLRETURN SQL_API SQLParamOptions(SQLHSTMT stmt, SQLULEN paramSetSize, diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp index 74d0f9d833f4d..9df64d3b14133 100644 --- a/modules/platforms/cpp/odbc/src/odbc.cpp +++ b/modules/platforms/cpp/odbc/src/odbc.cpp @@ -326,6 +326,8 @@ namespace ignite std::string dsn = SqlStringToString(serverName, serverNameLen); + LOG_MSG("DSN: %s\n", dsn.c_str()); + odbc::ReadDsnConfiguration(dsn.c_str(), config); connection->Establish(config); @@ -621,7 +623,7 @@ namespace ignite using odbc::app::Parameter; using odbc::type_traits::IsSqlTypeSupported; - LOG_MSG("SQLBindParameter called\n"); + LOG_MSG("SQLBindParameter called: %d, %d, %d\n", paramIdx, bufferType, paramSqlType); Statement *statement = reinterpret_cast(stmt); @@ -631,9 +633,6 @@ namespace ignite if (ioType != SQL_PARAM_INPUT) return SQL_ERROR; - if (resLen && (*resLen == SQL_DATA_AT_EXEC || *resLen <= SQL_LEN_DATA_AT_EXEC_OFFSET)) - return SQL_ERROR; - if (!IsSqlTypeSupported(paramSqlType)) return SQL_ERROR; @@ -976,14 +975,14 @@ namespace ignite case SQL_ATTR_PARAM_BIND_OFFSET_PTR: { - statement->SetParamBindOffsetPtr(reinterpret_cast(value)); + statement->SetParamBindOffsetPtr(reinterpret_cast(value)); break; } case SQL_ATTR_ROW_BIND_OFFSET_PTR: { - statement->SetColumnBindOffsetPtr(reinterpret_cast(value)); + statement->SetColumnBindOffsetPtr(reinterpret_cast(value)); break; } @@ -1334,4 +1333,36 @@ namespace ignite return statement->GetDiagnosticRecords().GetReturnCode(); } + SQLRETURN SQLParamData(SQLHSTMT stmt, SQLPOINTER* value) + { + using namespace ignite::odbc; + + LOG_MSG("SQLParamData called\n"); + + Statement *statement = reinterpret_cast(stmt); + + if (!statement) + return SQL_INVALID_HANDLE; + + statement->SelectParam(value); + + return statement->GetDiagnosticRecords().GetReturnCode(); + } + + SQLRETURN SQLPutData(SQLHSTMT stmt, SQLPOINTER data, SQLLEN strLengthOrIndicator) + { + using namespace ignite::odbc; + + LOG_MSG("SQLPutData called\n"); + + Statement *statement = reinterpret_cast(stmt); + + if (!statement) + return SQL_INVALID_HANDLE; + + statement->PutData(data, strLengthOrIndicator); + + return statement->GetDiagnosticRecords().GetReturnCode(); + } + } // namespace ignite; diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp index 8aae156dc7818..32f7c3f0c33d2 100644 --- a/modules/platforms/cpp/odbc/src/statement.cpp +++ b/modules/platforms/cpp/odbc/src/statement.cpp @@ -33,8 +33,14 @@ namespace ignite namespace odbc { Statement::Statement(Connection& parent) : - connection(parent), columnBindings(), currentQuery(), - rowsFetched(0), rowStatuses(0), paramBindOffset(0), columnBindOffset(0) + connection(parent), + columnBindings(), + currentQuery(), + rowsFetched(0), + rowStatuses(0), + paramBindOffset(0), + columnBindOffset(0), + currentParamIdx(0) { // No-op. } @@ -67,12 +73,12 @@ namespace ignite columnBindings.clear(); } - void Statement::SetColumnBindOffsetPtr(size_t * ptr) + void Statement::SetColumnBindOffsetPtr(int * ptr) { columnBindOffset = ptr; } - size_t * Statement::GetColumnBindOffsetPtr() + int* Statement::GetColumnBindOffsetPtr() { return columnBindOffset; } @@ -104,11 +110,25 @@ namespace ignite void Statement::BindParameter(uint16_t paramIdx, const app::Parameter& param) { - IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS; + IGNITE_ODBC_API_CALL(InternalBindParameter(paramIdx, param)); + } + + + SqlResult Statement::InternalBindParameter(uint16_t paramIdx, const app::Parameter& param) + { + if (paramIdx == 0) + { + AddStatusRecord(SQL_STATE_24000_INVALID_CURSOR_STATE, + "The value specified for the argument ParameterNumber was less than 1."); + + return SQL_RESULT_ERROR; + } paramBindings[paramIdx] = param; paramBindings[paramIdx].GetBuffer().SetPtrToOffsetPtr(¶mBindOffset); + + return SQL_RESULT_SUCCESS; } void Statement::UnbindParameter(uint16_t paramIdx) @@ -132,14 +152,14 @@ namespace ignite return static_cast(paramBindings.size()); } - void Statement::SetParamBindOffsetPtr(size_t* ptr) + void Statement::SetParamBindOffsetPtr(int* ptr) { IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS; paramBindOffset = ptr; } - size_t * Statement::GetParamBindOffsetPtr() + int* Statement::GetParamBindOffsetPtr() { return paramBindOffset; } @@ -219,6 +239,21 @@ namespace ignite return SQL_RESULT_ERROR; } + bool paramDataReady = true; + + app::ParameterBindingMap::iterator it; + for (it = paramBindings.begin(); it != paramBindings.end(); ++it) + { + app::Parameter& param = it->second; + + param.ResetStoredData(); + + paramDataReady &= param.IsDataReady(); + } + + if (!paramDataReady) + return SQL_RESULT_NEED_DATA; + return currentQuery->Execute(); } @@ -519,6 +554,107 @@ namespace ignite { return rowStatuses; } + + void Statement::SelectParam(void** paramPtr) + { + IGNITE_ODBC_API_CALL(InternalSelectParam(paramPtr)); + } + + SqlResult Statement::InternalSelectParam(void** paramPtr) + { + if (!paramPtr) + { + AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, "Invalid parameter: ValuePtrPtr is null."); + + return SQL_RESULT_ERROR; + } + + if (!currentQuery.get()) + { + AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Query is not prepared."); + + return SQL_RESULT_ERROR; + } + + app::ParameterBindingMap::iterator it; + + if (currentParamIdx) + { + it = paramBindings.find(currentParamIdx); + + if (it != paramBindings.end() && !it->second.IsDataReady()) + { + AddStatusRecord(SQL_STATE_22026_DATA_LENGTH_MISMATCH, + "Less data was sent for a parameter than was specified with " + "the StrLen_or_IndPtr argument in SQLBindParameter."); + + return SQL_RESULT_ERROR; + } + } + + for (it = paramBindings.begin(); it != paramBindings.end(); ++it) + { + uint16_t paramIdx = it->first; + app::Parameter& param = it->second; + + if (!param.IsDataReady()) + { + *paramPtr = param.GetBuffer().GetData(); + + currentParamIdx = paramIdx; + + return SQL_RESULT_NEED_DATA; + } + } + + SqlResult res = currentQuery->Execute(); + + if (res != SQL_RESULT_SUCCESS) + res = SQL_RESULT_SUCCESS_WITH_INFO; + + return res; + } + + void Statement::PutData(void* data, SqlLen len) + { + IGNITE_ODBC_API_CALL(InternalPutData(data, len)); + } + + SqlResult Statement::InternalPutData(void* data, SqlLen len) + { + if (!data && len != 0 && len != SQL_DEFAULT_PARAM && len != SQL_NULL_DATA) + { + AddStatusRecord(SQL_STATE_HY009_INVALID_USE_OF_NULL_POINTER, + "Invalid parameter: DataPtr is null StrLen_or_Ind is not 0, " + "SQL_DEFAULT_PARAM, or SQL_NULL_DATA."); + + return SQL_RESULT_ERROR; + } + + if (currentParamIdx == 0) + { + AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, + "Parameter is not selected with the SQLParamData."); + + return SQL_RESULT_ERROR; + } + + app::ParameterBindingMap::iterator it = paramBindings.find(currentParamIdx); + + if (it == paramBindings.end()) + { + AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, + "Selected parameter has been unbound."); + + return SQL_RESULT_ERROR; + } + + app::Parameter& param = it->second; + + param.PutData(data, len); + + return SQL_RESULT_SUCCESS; + } } } From 350e84656686eb81f8e12d569783db9914ca5a37 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 26 Sep 2016 19:55:27 +0300 Subject: [PATCH 169/487] IGNITE-3956 .NET: Fix cache creation in LINQPad examples This closes #1116 --- .../Apache.Ignite.Core/NuGet/LINQPad/PutGetExample.linq | 2 +- .../Apache.Ignite.Core/NuGet/LINQPad/QueryExample.linq | 6 +++--- .../Apache.Ignite.Linq/NuGet/LINQPad/QueryExample.linq | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/PutGetExample.linq b/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/PutGetExample.linq index 97d9bbe7be3d1..7c77d0995e736 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/PutGetExample.linq +++ b/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/PutGetExample.linq @@ -42,7 +42,7 @@ void Main() using (var ignite = Ignition.Start(cfg)) { // Create new cache - var cache = ignite.CreateCache("orgs"); + var cache = ignite.GetOrCreateCache("orgs"); // Put data entry to cache cache.Put(1, new Organization {Name = "Apache", Type="Private"}); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/QueryExample.linq b/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/QueryExample.linq index b7f7981fb8986..2a2454e839315 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/QueryExample.linq +++ b/modules/platforms/dotnet/Apache.Ignite.Core/NuGet/LINQPad/QueryExample.linq @@ -45,14 +45,14 @@ void Main() using (var ignite = Ignition.Start(cfg)) { // Create and populate organization cache - var orgs = ignite.GetOrCreateCache(new CacheConfiguration("orgs", + var orgs = ignite.GetOrCreateCache(new CacheConfiguration("orgs-sql", new QueryEntity(typeof(int), typeof(Organization)))); orgs[1] = new Organization { Name = "Apache", Type = "Private", Size = 5300 }; orgs[2] = new Organization { Name = "Microsoft", Type = "Private", Size = 110000 }; orgs[3] = new Organization { Name = "Red Cross", Type = "Non-Profit", Size = 35000 }; // Create and populate person cache - var persons = ignite.CreateCache(new CacheConfiguration("persons", typeof(Person))); + var persons = ignite.GetOrCreateCache(new CacheConfiguration("persons-sql", typeof(Person))); persons[1] = new Person { OrgId = 1, Name = "James Wilson" }; persons[2] = new Person { OrgId = 1, Name = "Daniel Adams" }; persons[3] = new Person { OrgId = 2, Name = "Christian Moss" }; @@ -64,7 +64,7 @@ void Main() // SQL query with join const string orgName = "Apache"; - persons.Query(new SqlQuery(typeof(Person), "from Person, \"orgs\".Organization where Person.OrgId = \"orgs\".Organization._key and \"orgs\".Organization.Name = ?", orgName)) + persons.Query(new SqlQuery(typeof(Person), "from Person, \"orgs-sql\".Organization where Person.OrgId = \"orgs-sql\".Organization._key and \"orgs-sql\".Organization.Name = ?", orgName)) .Dump("Persons working for " + orgName); // Fields query diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/NuGet/LINQPad/QueryExample.linq b/modules/platforms/dotnet/Apache.Ignite.Linq/NuGet/LINQPad/QueryExample.linq index 9cce4ec9278d6..6a28f1fc86ece 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/NuGet/LINQPad/QueryExample.linq +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/NuGet/LINQPad/QueryExample.linq @@ -46,14 +46,14 @@ void Main() using (var ignite = Ignition.Start(cfg)) { // Create and populate organization cache - var orgs = ignite.GetOrCreateCache(new CacheConfiguration("orgs", + var orgs = ignite.GetOrCreateCache(new CacheConfiguration("orgs-linq", new QueryEntity(typeof(int), typeof(Organization)))); orgs[1] = new Organization { Name = "Apache", Type = "Private", Size = 5300 }; orgs[2] = new Organization { Name = "Microsoft", Type = "Private", Size = 110000 }; orgs[3] = new Organization { Name = "Red Cross", Type = "Non-Profit", Size = 35000 }; // Create and populate person cache - var persons = ignite.CreateCache(new CacheConfiguration("persons", typeof(Person))); + var persons = ignite.GetOrCreateCache(new CacheConfiguration("persons-linq", typeof(Person))); persons[1] = new Person { OrgId = 1, Name = "James Wilson" }; persons[2] = new Person { OrgId = 1, Name = "Daniel Adams" }; persons[3] = new Person { OrgId = 2, Name = "Christian Moss" }; From 6a41dc92da124a921b304392e922886c30597037 Mon Sep 17 00:00:00 2001 From: Saikat Maitra Date: Mon, 26 Sep 2016 15:28:51 -0700 Subject: [PATCH 170/487] IGNITE-3807 IgniteSpiContext registers message listeners incorrectly --- .../internal/managers/GridManagerAdapter.java | 15 ++ .../apache/ignite/spi/IgniteSpiAdapter.java | 11 + .../apache/ignite/spi/IgniteSpiContext.java | 26 +- ...idManagerLocalMessageListenerSelfTest.java | 222 ++++++++++++++++++ .../testframework/GridSpiTestContext.java | 115 ++++++++- .../ignite/testsuites/IgniteSpiTestSuite.java | 6 +- 6 files changed, 391 insertions(+), 4 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerLocalMessageListenerSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java index 4fe8ca87aff42..584cc56e23b3f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java @@ -44,6 +44,7 @@ import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.plugin.extensions.communication.Message; @@ -396,6 +397,20 @@ protected final String stopInfo() { } } + @Override public void addLocalMessageListener(Object topic, IgniteBiPredicate p) { + A.notNull(topic, "topic"); + A.notNull(p, "p"); + + ctx.io().addUserMessageListener(topic, p); + } + + @Override public void removeLocalMessageListener(Object topic, IgniteBiPredicate p) { + A.notNull(topic, "topic"); + A.notNull(topic, "p"); + + ctx.io().removeUserMessageListener(topic, p); + } + @SuppressWarnings("deprecation") @Override public void addMessageListener(GridMessageListener lsnr, String topic) { A.notNull(lsnr, "lsnr"); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java index 65b0420078137..219d07be5f725 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java @@ -47,6 +47,7 @@ import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageFactory; @@ -758,6 +759,11 @@ private class GridDummySpiContext implements IgniteSpiContext { /* No-op. */ } + /** {@inheritDoc} */ + @Override public void addLocalMessageListener(Object topic, IgniteBiPredicate p) { + /* No-op. */ + } + /** {@inheritDoc} */ @Override public void recordEvent(Event evt) { /* No-op. */ @@ -848,6 +854,11 @@ private class GridDummySpiContext implements IgniteSpiContext { return true; } + /** {@inheritDoc} */ + @Override public void removeLocalMessageListener(Object topic, IgniteBiPredicate p) { + /* No-op. */ + } + /** {@inheritDoc} */ @Override public boolean removeMessageListener(GridMessageListener lsnr, String topic) { return false; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java index 899c222d4018f..5eb5227a0a46c 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java @@ -26,6 +26,7 @@ import org.apache.ignite.events.Event; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.plugin.extensions.communication.MessageFormatter; import org.apache.ignite.plugin.security.SecuritySubject; @@ -115,6 +116,23 @@ public interface IgniteSpiContext { */ public void send(ClusterNode node, Serializable msg, String topic) throws IgniteSpiException; + /** + * Register an local message listener to receive messages sent by remote nodes. The underlying + * communication mechanism is defined by {@link org.apache.ignite.spi.communication.CommunicationSpi} implementation used. + * + * @param topic Topic to subscribe to. + * @param p Message predicate. + */ + public void addLocalMessageListener(@Nullable Object topic, IgniteBiPredicate p); + + /** + * Removes a previously registered local message listener. + * + * @param topic Topic to unsubscribe from. + * @param p Message predicate. + */ + public void removeLocalMessageListener(@Nullable Object topic, IgniteBiPredicate p); + /** * Register a message listener to receive messages sent by remote nodes. The underlying * communication mechanism is defined by {@link org.apache.ignite.spi.communication.CommunicationSpi} implementation used. @@ -124,7 +142,10 @@ public interface IgniteSpiContext { * * @param lsnr Message listener to register. * @param topic Topic to register listener for. + * + * @deprecated Use {@link #addLocalMessageListener(Object, IgniteBiPredicate)} instead. */ + @Deprecated public void addMessageListener(GridMessageListener lsnr, String topic); /** @@ -134,7 +155,10 @@ public interface IgniteSpiContext { * @param topic Topic to unregister listener for. * @return {@code true} of message listener was removed, {@code false} if it was not * previously registered. + * + * @deprecated Use {@link #removeLocalMessageListener(Object, IgniteBiPredicate)} instead. */ + @Deprecated public boolean removeMessageListener(GridMessageListener lsnr, String topic); /** @@ -328,4 +352,4 @@ public interface IgniteSpiContext { * @param c Timeout object. */ public void removeTimeoutObject(IgniteSpiTimeoutObject c); -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerLocalMessageListenerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerLocalMessageListenerSelfTest.java new file mode 100644 index 0000000000000..4aadc78b1336f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerLocalMessageListenerSelfTest.java @@ -0,0 +1,222 @@ +/* + * 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.managers; + +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.communication.GridIoUserMessage; +import org.apache.ignite.internal.util.typedef.CO; +import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteSpi; +import org.apache.ignite.spi.IgniteSpiAdapter; +import org.apache.ignite.spi.IgniteSpiContext; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.Nullable; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; + +/** + * Test Managers to add and remove local message listener. + */ +public class GridManagerLocalMessageListenerSelfTest extends GridCommonAbstractTest { + /** IP finder. */ + private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final byte DIRECT_TYPE = (byte)210; + + static { + GridIoMessageFactory.registerCustom(DIRECT_TYPE, new CO() { + @Override public Message apply() { + return new GridIoUserMessage(); + } + }); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration c = super.getConfiguration(gridName); + + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.setIpFinder(ipFinder); + + c.setDiscoverySpi(discoSpi); + + TcpCommunicationSpi commSpi = new TcpCommunicationSpi(); + + c.setCommunicationSpi(commSpi); + + return c; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testSendMessage() throws Exception { + startGridsMultiThreaded(2); + + IgniteSpiContext ctx0 = ((IgniteSpiAdapter)grid(0).context().io().getSpi()).getSpiContext(); + IgniteSpiContext ctx1 = ((IgniteSpiAdapter)grid(1).context().io().getSpi()).getSpiContext(); + + String topic = "test-topic"; + + final CountDownLatch latch = new CountDownLatch(1); + + ctx1.addLocalMessageListener(topic, new IgniteBiPredicate() { + @Override public boolean apply(UUID nodeId, Object msg) { + assertEquals("Message", msg); + + latch.countDown(); + + return true; + } + }); + + long time = System.nanoTime(); + + ctx0.send(grid(1).localNode(), "Message", topic); + + assert latch.await(3, SECONDS); + + time = System.nanoTime() - time; + + info(">>>"); + info(">>> send() time (ms): " + MILLISECONDS.convert(time, NANOSECONDS)); + info(">>>"); + } + + /** + * @throws Exception If failed. + */ + public void testAddLocalMessageListener() throws Exception { + startGrid(); + + Manager mgr = new Manager(grid().context(), new Spi()); + + mgr.start(); + + mgr.onKernalStart(); + + assertTrue(mgr.enabled()); + } + + /** + * @throws Exception If failed. + */ + public void testRemoveLocalMessageListener() throws Exception { + startGrid(); + + Manager mgr = new Manager(grid().context(), new Spi()); + + assertTrue(mgr.enabled()); + + mgr.onKernalStart(); + + mgr.onKernalStop(false); + + mgr.stop(false); + + assertTrue(mgr.enabled()); + } + + /** */ + private static class Manager extends GridManagerAdapter { + /** + * @param ctx Kernal context. + * @param spis Specific SPI instance. + */ + protected Manager(GridKernalContext ctx, IgniteSpi... spis) { + super(ctx, spis); + } + + /** {@inheritDoc} */ + @Override public void start() throws IgniteCheckedException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void stop(boolean cancel) throws IgniteCheckedException { + // No-op. + } + } + + /** + * Test Spi. + */ + private static interface TestSpi extends IgniteSpi { + // No-op. + } + + /** + * Spi + */ + private static class Spi extends IgniteSpiAdapter implements TestSpi { + /** Ignite Spi Context. **/ + private IgniteSpiContext spiCtx; + + /** Test message topic. **/ + private String TEST_TOPIC = "test_topic"; + + /** {@inheritDoc} */ + @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void spiStop() throws IgniteSpiException { + // No-op. + } + + @Override public void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException { + this.spiCtx = spiCtx; + + spiCtx.addLocalMessageListener(TEST_TOPIC, new IgniteBiPredicate() { + @Override public boolean apply(UUID uuid, Object o) { + return true; + } + }); + + } + + @Override public void onContextDestroyed0() { + spiCtx.removeLocalMessageListener(TEST_TOPIC, new IgniteBiPredicate() { + @Override public boolean apply(UUID uuid, Object o) { + return true; + } + }); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java index 0bffe8bfc8710..ac50ef90e29e1 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java @@ -29,21 +29,26 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.Event; import org.apache.ignite.events.TaskEvent; import org.apache.ignite.internal.ClusterMetricsSnapshot; +import org.apache.ignite.internal.GridTopic; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; import org.apache.ignite.internal.managers.communication.GridIoManager; import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.communication.GridIoUserMessage; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.timeout.GridSpiTimeoutObject; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.plugin.extensions.communication.MessageFormatter; @@ -61,6 +66,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_JOINED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.events.EventType.EVT_NODE_METRICS_UPDATED; +import static org.apache.ignite.internal.GridTopic.TOPIC_COMM_USER; /** * Test SPI context. @@ -241,7 +247,7 @@ public void removeNode(ClusterNode node) { * @param nodeId Node ID. */ public void removeNode(UUID nodeId) { - for (Iterator iter = rmtNodes.iterator(); iter.hasNext();) { + for (Iterator iter = rmtNodes.iterator(); iter.hasNext(); ) { ClusterNode node = iter.next(); if (node.id().equals(nodeId)) { @@ -325,6 +331,27 @@ public void triggerMessage(ClusterNode node, Object msg) { lsnr.onMessage(node.id(), msg); } + /** {@inheritDoc} */ + @SuppressWarnings("deprecation") + @Override public void addLocalMessageListener(Object topic, IgniteBiPredicate p) { + try { + addMessageListener(TOPIC_COMM_USER, + new GridLocalMessageListener(topic, (IgniteBiPredicate)p)); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + + /** + * @param topic Listener's topic. + * @param lsnr Listener to add. + */ + @SuppressWarnings({"TypeMayBeWeakened", "deprecation"}) + public void addMessageListener(GridTopic topic, GridMessageListener lsnr) { + addMessageListener(lsnr, ((Object)topic).toString()); + } + /** {@inheritDoc} */ @SuppressWarnings("deprecation") @Override public void addMessageListener(GridMessageListener lsnr, String topic) { @@ -337,6 +364,28 @@ public void triggerMessage(ClusterNode node, Object msg) { return msgLsnrs.remove(lsnr); } + /** {@inheritDoc} */ + @SuppressWarnings("deprecation") + @Override public void removeLocalMessageListener(Object topic, IgniteBiPredicate p) { + try { + removeMessageListener(TOPIC_COMM_USER, + new GridLocalMessageListener(topic, (IgniteBiPredicate)p)); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + + /** + * @param topic Listener's topic. + * @param lsnr Listener to remove. + * @return Whether or not the lsnr was removed. + */ + @SuppressWarnings("deprecation") + public boolean removeMessageListener(GridTopic topic, @Nullable GridMessageListener lsnr) { + return removeMessageListener(lsnr, ((Object)topic).toString()); + } + /** * @param type Event type. * @param taskName Task name. @@ -471,7 +520,7 @@ public void triggerEvent(Event evt) { boolean res = false; try { - res = get(cacheName, key) != null; + res = get(cacheName, key) != null; } catch (IgniteException ignored) { @@ -587,4 +636,66 @@ private CachedObject(long expire, V obj) { this.obj = obj; } } + + /** + * This class represents a message listener wrapper that knows about peer deployment. + */ + private class GridLocalMessageListener implements GridMessageListener { + /** Predicate listeners. */ + private final IgniteBiPredicate predLsnr; + + /** User message topic. */ + private final Object topic; + + /** + * @param topic User topic. + * @param predLsnr Predicate listener. + * @throws IgniteCheckedException If failed to inject resources to predicates. + */ + GridLocalMessageListener(@Nullable Object topic, @Nullable IgniteBiPredicate predLsnr) + throws IgniteCheckedException { + this.topic = topic; + this.predLsnr = predLsnr; + } + + /** {@inheritDoc} */ + @SuppressWarnings({ + "SynchronizationOnLocalVariableOrMethodParameter", "ConstantConditions", + "OverlyStrongTypeCast"}) + @Override public void onMessage(UUID nodeId, Object msg) { + GridIoUserMessage ioMsg = (GridIoUserMessage)msg; + + ClusterNode node = locNode; + Object msgBody = ioMsg.body(); + + assert msgBody != null || ioMsg.bodyBytes() != null; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + GridLocalMessageListener l = (GridLocalMessageListener)o; + + return F.eq(predLsnr, l.predLsnr) && F.eq(topic, l.topic); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int res = predLsnr != null ? predLsnr.hashCode() : 0; + + res = 31 * res + (topic != null ? topic.hashCode() : 0); + + return res; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridLocalMessageListener.class, this); + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiTestSuite.java index fc4023ad6a46e..90f12185e050a 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiTestSuite.java @@ -18,6 +18,7 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.internal.managers.GridManagerLocalMessageListenerSelfTest; import org.apache.ignite.internal.managers.GridNoopManagerSelfTest; /** @@ -64,6 +65,9 @@ public static TestSuite suite() throws Exception { // All other tests. suite.addTestSuite(GridNoopManagerSelfTest.class); + // Local Message Listener tests. + suite.addTestSuite(GridManagerLocalMessageListenerSelfTest.class); + return suite; } -} \ No newline at end of file +} From 5ff9e5e8c95ea0fe37d281a7ff46e148469f952a Mon Sep 17 00:00:00 2001 From: Saikat Maitra Date: Mon, 26 Sep 2016 15:28:51 -0700 Subject: [PATCH 171/487] IGNITE-3807 IgniteSpiContext registers message listeners incorrectly --- .../internal/managers/GridManagerAdapter.java | 15 ++ .../apache/ignite/spi/IgniteSpiAdapter.java | 11 + .../apache/ignite/spi/IgniteSpiContext.java | 26 +- ...idManagerLocalMessageListenerSelfTest.java | 222 ++++++++++++++++++ .../testframework/GridSpiTestContext.java | 115 ++++++++- .../ignite/testsuites/IgniteSpiTestSuite.java | 6 +- 6 files changed, 391 insertions(+), 4 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerLocalMessageListenerSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java index 4fe8ca87aff42..584cc56e23b3f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java @@ -44,6 +44,7 @@ import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.plugin.extensions.communication.Message; @@ -396,6 +397,20 @@ protected final String stopInfo() { } } + @Override public void addLocalMessageListener(Object topic, IgniteBiPredicate p) { + A.notNull(topic, "topic"); + A.notNull(p, "p"); + + ctx.io().addUserMessageListener(topic, p); + } + + @Override public void removeLocalMessageListener(Object topic, IgniteBiPredicate p) { + A.notNull(topic, "topic"); + A.notNull(topic, "p"); + + ctx.io().removeUserMessageListener(topic, p); + } + @SuppressWarnings("deprecation") @Override public void addMessageListener(GridMessageListener lsnr, String topic) { A.notNull(lsnr, "lsnr"); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java index 65b0420078137..219d07be5f725 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java @@ -47,6 +47,7 @@ import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageFactory; @@ -758,6 +759,11 @@ private class GridDummySpiContext implements IgniteSpiContext { /* No-op. */ } + /** {@inheritDoc} */ + @Override public void addLocalMessageListener(Object topic, IgniteBiPredicate p) { + /* No-op. */ + } + /** {@inheritDoc} */ @Override public void recordEvent(Event evt) { /* No-op. */ @@ -848,6 +854,11 @@ private class GridDummySpiContext implements IgniteSpiContext { return true; } + /** {@inheritDoc} */ + @Override public void removeLocalMessageListener(Object topic, IgniteBiPredicate p) { + /* No-op. */ + } + /** {@inheritDoc} */ @Override public boolean removeMessageListener(GridMessageListener lsnr, String topic) { return false; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java index 899c222d4018f..5eb5227a0a46c 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java @@ -26,6 +26,7 @@ import org.apache.ignite.events.Event; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.plugin.extensions.communication.MessageFormatter; import org.apache.ignite.plugin.security.SecuritySubject; @@ -115,6 +116,23 @@ public interface IgniteSpiContext { */ public void send(ClusterNode node, Serializable msg, String topic) throws IgniteSpiException; + /** + * Register an local message listener to receive messages sent by remote nodes. The underlying + * communication mechanism is defined by {@link org.apache.ignite.spi.communication.CommunicationSpi} implementation used. + * + * @param topic Topic to subscribe to. + * @param p Message predicate. + */ + public void addLocalMessageListener(@Nullable Object topic, IgniteBiPredicate p); + + /** + * Removes a previously registered local message listener. + * + * @param topic Topic to unsubscribe from. + * @param p Message predicate. + */ + public void removeLocalMessageListener(@Nullable Object topic, IgniteBiPredicate p); + /** * Register a message listener to receive messages sent by remote nodes. The underlying * communication mechanism is defined by {@link org.apache.ignite.spi.communication.CommunicationSpi} implementation used. @@ -124,7 +142,10 @@ public interface IgniteSpiContext { * * @param lsnr Message listener to register. * @param topic Topic to register listener for. + * + * @deprecated Use {@link #addLocalMessageListener(Object, IgniteBiPredicate)} instead. */ + @Deprecated public void addMessageListener(GridMessageListener lsnr, String topic); /** @@ -134,7 +155,10 @@ public interface IgniteSpiContext { * @param topic Topic to unregister listener for. * @return {@code true} of message listener was removed, {@code false} if it was not * previously registered. + * + * @deprecated Use {@link #removeLocalMessageListener(Object, IgniteBiPredicate)} instead. */ + @Deprecated public boolean removeMessageListener(GridMessageListener lsnr, String topic); /** @@ -328,4 +352,4 @@ public interface IgniteSpiContext { * @param c Timeout object. */ public void removeTimeoutObject(IgniteSpiTimeoutObject c); -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerLocalMessageListenerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerLocalMessageListenerSelfTest.java new file mode 100644 index 0000000000000..4aadc78b1336f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerLocalMessageListenerSelfTest.java @@ -0,0 +1,222 @@ +/* + * 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.managers; + +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.communication.GridIoUserMessage; +import org.apache.ignite.internal.util.typedef.CO; +import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteSpi; +import org.apache.ignite.spi.IgniteSpiAdapter; +import org.apache.ignite.spi.IgniteSpiContext; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.Nullable; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; + +/** + * Test Managers to add and remove local message listener. + */ +public class GridManagerLocalMessageListenerSelfTest extends GridCommonAbstractTest { + /** IP finder. */ + private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final byte DIRECT_TYPE = (byte)210; + + static { + GridIoMessageFactory.registerCustom(DIRECT_TYPE, new CO() { + @Override public Message apply() { + return new GridIoUserMessage(); + } + }); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration c = super.getConfiguration(gridName); + + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.setIpFinder(ipFinder); + + c.setDiscoverySpi(discoSpi); + + TcpCommunicationSpi commSpi = new TcpCommunicationSpi(); + + c.setCommunicationSpi(commSpi); + + return c; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testSendMessage() throws Exception { + startGridsMultiThreaded(2); + + IgniteSpiContext ctx0 = ((IgniteSpiAdapter)grid(0).context().io().getSpi()).getSpiContext(); + IgniteSpiContext ctx1 = ((IgniteSpiAdapter)grid(1).context().io().getSpi()).getSpiContext(); + + String topic = "test-topic"; + + final CountDownLatch latch = new CountDownLatch(1); + + ctx1.addLocalMessageListener(topic, new IgniteBiPredicate() { + @Override public boolean apply(UUID nodeId, Object msg) { + assertEquals("Message", msg); + + latch.countDown(); + + return true; + } + }); + + long time = System.nanoTime(); + + ctx0.send(grid(1).localNode(), "Message", topic); + + assert latch.await(3, SECONDS); + + time = System.nanoTime() - time; + + info(">>>"); + info(">>> send() time (ms): " + MILLISECONDS.convert(time, NANOSECONDS)); + info(">>>"); + } + + /** + * @throws Exception If failed. + */ + public void testAddLocalMessageListener() throws Exception { + startGrid(); + + Manager mgr = new Manager(grid().context(), new Spi()); + + mgr.start(); + + mgr.onKernalStart(); + + assertTrue(mgr.enabled()); + } + + /** + * @throws Exception If failed. + */ + public void testRemoveLocalMessageListener() throws Exception { + startGrid(); + + Manager mgr = new Manager(grid().context(), new Spi()); + + assertTrue(mgr.enabled()); + + mgr.onKernalStart(); + + mgr.onKernalStop(false); + + mgr.stop(false); + + assertTrue(mgr.enabled()); + } + + /** */ + private static class Manager extends GridManagerAdapter { + /** + * @param ctx Kernal context. + * @param spis Specific SPI instance. + */ + protected Manager(GridKernalContext ctx, IgniteSpi... spis) { + super(ctx, spis); + } + + /** {@inheritDoc} */ + @Override public void start() throws IgniteCheckedException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void stop(boolean cancel) throws IgniteCheckedException { + // No-op. + } + } + + /** + * Test Spi. + */ + private static interface TestSpi extends IgniteSpi { + // No-op. + } + + /** + * Spi + */ + private static class Spi extends IgniteSpiAdapter implements TestSpi { + /** Ignite Spi Context. **/ + private IgniteSpiContext spiCtx; + + /** Test message topic. **/ + private String TEST_TOPIC = "test_topic"; + + /** {@inheritDoc} */ + @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void spiStop() throws IgniteSpiException { + // No-op. + } + + @Override public void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException { + this.spiCtx = spiCtx; + + spiCtx.addLocalMessageListener(TEST_TOPIC, new IgniteBiPredicate() { + @Override public boolean apply(UUID uuid, Object o) { + return true; + } + }); + + } + + @Override public void onContextDestroyed0() { + spiCtx.removeLocalMessageListener(TEST_TOPIC, new IgniteBiPredicate() { + @Override public boolean apply(UUID uuid, Object o) { + return true; + } + }); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java index 0bffe8bfc8710..ac50ef90e29e1 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java @@ -29,21 +29,26 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.Event; import org.apache.ignite.events.TaskEvent; import org.apache.ignite.internal.ClusterMetricsSnapshot; +import org.apache.ignite.internal.GridTopic; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; import org.apache.ignite.internal.managers.communication.GridIoManager; import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.communication.GridIoUserMessage; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.timeout.GridSpiTimeoutObject; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.plugin.extensions.communication.MessageFormatter; @@ -61,6 +66,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_JOINED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.events.EventType.EVT_NODE_METRICS_UPDATED; +import static org.apache.ignite.internal.GridTopic.TOPIC_COMM_USER; /** * Test SPI context. @@ -241,7 +247,7 @@ public void removeNode(ClusterNode node) { * @param nodeId Node ID. */ public void removeNode(UUID nodeId) { - for (Iterator iter = rmtNodes.iterator(); iter.hasNext();) { + for (Iterator iter = rmtNodes.iterator(); iter.hasNext(); ) { ClusterNode node = iter.next(); if (node.id().equals(nodeId)) { @@ -325,6 +331,27 @@ public void triggerMessage(ClusterNode node, Object msg) { lsnr.onMessage(node.id(), msg); } + /** {@inheritDoc} */ + @SuppressWarnings("deprecation") + @Override public void addLocalMessageListener(Object topic, IgniteBiPredicate p) { + try { + addMessageListener(TOPIC_COMM_USER, + new GridLocalMessageListener(topic, (IgniteBiPredicate)p)); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + + /** + * @param topic Listener's topic. + * @param lsnr Listener to add. + */ + @SuppressWarnings({"TypeMayBeWeakened", "deprecation"}) + public void addMessageListener(GridTopic topic, GridMessageListener lsnr) { + addMessageListener(lsnr, ((Object)topic).toString()); + } + /** {@inheritDoc} */ @SuppressWarnings("deprecation") @Override public void addMessageListener(GridMessageListener lsnr, String topic) { @@ -337,6 +364,28 @@ public void triggerMessage(ClusterNode node, Object msg) { return msgLsnrs.remove(lsnr); } + /** {@inheritDoc} */ + @SuppressWarnings("deprecation") + @Override public void removeLocalMessageListener(Object topic, IgniteBiPredicate p) { + try { + removeMessageListener(TOPIC_COMM_USER, + new GridLocalMessageListener(topic, (IgniteBiPredicate)p)); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + + /** + * @param topic Listener's topic. + * @param lsnr Listener to remove. + * @return Whether or not the lsnr was removed. + */ + @SuppressWarnings("deprecation") + public boolean removeMessageListener(GridTopic topic, @Nullable GridMessageListener lsnr) { + return removeMessageListener(lsnr, ((Object)topic).toString()); + } + /** * @param type Event type. * @param taskName Task name. @@ -471,7 +520,7 @@ public void triggerEvent(Event evt) { boolean res = false; try { - res = get(cacheName, key) != null; + res = get(cacheName, key) != null; } catch (IgniteException ignored) { @@ -587,4 +636,66 @@ private CachedObject(long expire, V obj) { this.obj = obj; } } + + /** + * This class represents a message listener wrapper that knows about peer deployment. + */ + private class GridLocalMessageListener implements GridMessageListener { + /** Predicate listeners. */ + private final IgniteBiPredicate predLsnr; + + /** User message topic. */ + private final Object topic; + + /** + * @param topic User topic. + * @param predLsnr Predicate listener. + * @throws IgniteCheckedException If failed to inject resources to predicates. + */ + GridLocalMessageListener(@Nullable Object topic, @Nullable IgniteBiPredicate predLsnr) + throws IgniteCheckedException { + this.topic = topic; + this.predLsnr = predLsnr; + } + + /** {@inheritDoc} */ + @SuppressWarnings({ + "SynchronizationOnLocalVariableOrMethodParameter", "ConstantConditions", + "OverlyStrongTypeCast"}) + @Override public void onMessage(UUID nodeId, Object msg) { + GridIoUserMessage ioMsg = (GridIoUserMessage)msg; + + ClusterNode node = locNode; + Object msgBody = ioMsg.body(); + + assert msgBody != null || ioMsg.bodyBytes() != null; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + GridLocalMessageListener l = (GridLocalMessageListener)o; + + return F.eq(predLsnr, l.predLsnr) && F.eq(topic, l.topic); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int res = predLsnr != null ? predLsnr.hashCode() : 0; + + res = 31 * res + (topic != null ? topic.hashCode() : 0); + + return res; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridLocalMessageListener.class, this); + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiTestSuite.java index fc4023ad6a46e..90f12185e050a 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiTestSuite.java @@ -18,6 +18,7 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.internal.managers.GridManagerLocalMessageListenerSelfTest; import org.apache.ignite.internal.managers.GridNoopManagerSelfTest; /** @@ -64,6 +65,9 @@ public static TestSuite suite() throws Exception { // All other tests. suite.addTestSuite(GridNoopManagerSelfTest.class); + // Local Message Listener tests. + suite.addTestSuite(GridManagerLocalMessageListenerSelfTest.class); + return suite; } -} \ No newline at end of file +} From 97bfee4dff807e3049b61fa473472a8395cdcb6a Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Tue, 27 Sep 2016 10:06:48 +0300 Subject: [PATCH 172/487] Fixing RAT. --- parent/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parent/pom.xml b/parent/pom.xml index 3e9a71c23cd7a..6240467594249 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -794,7 +794,7 @@ src/test/java/org/apache/ignite/spi/deployment/uri/META-INF/ignite.incorrefs src/test/java/org/apache/ignite/spi/deployment/uri/META-INF/ignite.empty src/test/java/org/apache/ignite/spi/deployment/uri/META-INF/ignite.brokenxml - src/test/java/org/apache/ignite/internal/processors/hadoop/books/*.txt + **/books/*.txt src/main/java/org/apache/ignite/examples/streaming/wordcount/*.txt examples/src/main/java/org/apache/ignite/examples/streaming/wordcount/*.txt src/main/resources/person.csv From 68b0bcd83c295ce540aa9d9d0910abcf671671df Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 27 Sep 2016 12:08:46 +0300 Subject: [PATCH 173/487] IGNITE-3970 .NET: Fix Cyrillic 'C' letters in code --- .../platforms/dotnet/Apache.Ignite.Core/Compute/ICompute.cs | 6 +++--- .../Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ICompute.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ICompute.cs index a677f39249522..f3bdb7db85d84 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ICompute.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ICompute.cs @@ -577,7 +577,7 @@ public interface ICompute /// /// Job to run. /// Job arguments. - /// Сollection of job results. + /// Collection of job results. /// Type of argument. /// Type of job result. ICollection Apply(IComputeFunc clo, IEnumerable args); @@ -589,7 +589,7 @@ public interface ICompute /// /// Job to run. /// Job arguments. - /// Сollection of job results. + /// Collection of job results. /// Type of argument. /// Type of job result. Task> ApplyAsync(IComputeFunc clo, IEnumerable args); @@ -605,7 +605,7 @@ public interface ICompute /// Job arguments. /// The cancellation token. /// - /// Сollection of job results. + /// Collection of job results. /// Task> ApplyAsync(IComputeFunc clo, IEnumerable args, CancellationToken cancellationToken); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs index b3489a2d9ddd0..5815b4d9b2179 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs @@ -152,7 +152,7 @@ internal unsafe class UnmanagedCallbacks private delegate void ServiceCancelCallbackDelegate(void* target, long svcPtr, long memPtr); private delegate void ServiceInvokeMethodCallbackDelegate(void* target, long svcPtr, long inMemPtr, long outMemPtr); - private delegate int СlusterNodeFilterApplyCallbackDelegate(void* target, long memPtr); + private delegate int ClusterNodeFilterApplyCallbackDelegate(void* target, long memPtr); private delegate void NodeInfoCallbackDelegate(void* target, long memPtr); @@ -245,7 +245,7 @@ public UnmanagedCallbacks() serviceCancel = CreateFunctionPointer((ServiceCancelCallbackDelegate)ServiceCancel), serviceInvokeMethod = CreateFunctionPointer((ServiceInvokeMethodCallbackDelegate)ServiceInvokeMethod), - clusterNodeFilterApply = CreateFunctionPointer((СlusterNodeFilterApplyCallbackDelegate)СlusterNodeFilterApply), + clusterNodeFilterApply = CreateFunctionPointer((ClusterNodeFilterApplyCallbackDelegate)ClusterNodeFilterApply), onStart = CreateFunctionPointer((OnStartCallbackDelegate)OnStart), onStop = CreateFunctionPointer((OnStopCallbackDelegate)OnStop), @@ -993,7 +993,7 @@ private void ServiceInvokeMethod(void* target, long svcPtr, long inMemPtr, long }); } - private int СlusterNodeFilterApply(void* target, long memPtr) + private int ClusterNodeFilterApply(void* target, long memPtr) { return SafeCall(() => { From 48d4a9252536dd82811a10327b2df6ddbd1ff13a Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Tue, 27 Sep 2016 17:03:36 +0700 Subject: [PATCH 174/487] Web console beta-4. --- modules/web-console/DEVNOTES.txt | 25 +-- modules/web-console/backend/.babelrc | 9 - modules/web-console/backend/.eslintrc | 4 +- modules/web-console/backend/.gitignore | 2 - modules/web-console/backend/app/agent.js | 66 +++++-- modules/web-console/backend/app/app.js | 2 + modules/web-console/backend/app/browser.js | 41 ++++- modules/web-console/backend/app/index.js | 116 ------------- modules/web-console/backend/app/mongo.js | 7 +- .../app/mongoose.js} | 21 +-- modules/web-console/backend/app/settings.js | 10 +- .../backend/errors/AuthFailedException.js | 2 +- .../backend/errors/DuplicateKeyException.js | 2 +- .../backend/errors/IllegalAccessError.js | 4 +- .../errors/IllegalArgumentException.js | 2 +- .../errors/MissingResourceException.js | 2 +- modules/web-console/backend/errors/index.js | 14 +- modules/web-console/backend/index.js | 89 +++++++++- modules/web-console/backend/injector.js | 2 +- modules/web-console/backend/package.json | 23 +-- modules/web-console/backend/routes/demo.js | 8 +- modules/web-console/backend/routes/public.js | 69 ++------ modules/web-console/backend/services/auth.js | 79 ++++++++- .../web-console/backend/services/domains.js | 2 + modules/web-console/backend/services/mails.js | 16 +- .../web-console/backend/services/sessions.js | 4 +- modules/web-console/backend/test/app/db.js | 66 +++++++ .../web-console/backend/test/app/httpAgent.js | 50 ++++++ .../web-console/backend/test/app/mockgoose.js | 30 ++++ .../backend/test/data/accounts.json | 5 +- .../web-console/backend/test/data/caches.json | 30 ++-- .../backend/test/data/clusters.json | 10 +- .../backend/test/data/domains.json | 10 ++ .../web-console/backend/test/data/igfss.json | 4 +- .../web-console/backend/test/data/spaces.json | 14 ++ modules/web-console/backend/test/index.js | 35 ++++ modules/web-console/backend/test/injector.js | 43 +++-- .../backend/test/routes/clusters.js | 83 +++++++++ .../web-console/backend/test/routes/public.js | 68 ++++++++ .../backend/test/unit/AuthService.test.js | 107 ++++++++++++ .../backend/test/unit/CacheService.test.js | 20 +-- .../backend/test/unit/ClusterService.test.js | 19 +- .../backend/test/unit/DomainService.test.js | 19 +- .../backend/test/unit/IgfsService.test.js | 19 +- .../web-console/frontend/app/app.config.js | 2 +- modules/web-console/frontend/app/app.js | 27 +-- .../frontend/app/data/jdbc-types.json | 44 +++++ .../frontend/app/data/sql-keywords.json | 41 +++++ .../frontend/app/decorator/tooltip.js | 38 ++-- .../directives/retain-selection.directive.js | 67 +++++++ .../ui-ace-pom/ui-ace-pom.controller.js | 4 +- .../frontend/app/helpers/jade/form.jade | 1 + .../jade/form/form-field-datalist.jade | 8 +- .../jade/form/form-field-dropdown.jade | 8 +- .../helpers/jade/form/form-field-label.jade | 4 +- .../helpers/jade/form/form-field-number.jade | 8 +- .../jade/form/form-field-password.jade | 47 +++++ .../helpers/jade/form/form-field-text.jade | 10 +- .../frontend/app/helpers/jade/mixins.jade | 101 +++++------ .../modules/configuration/Version.service.js | 25 +++ .../configuration/configuration.module.js | 2 + .../configuration/generator/Pom.service.js | 30 ++-- .../validator/java-identifier.directive.js | 4 +- .../form/validator/java-keywords.directive.js | 6 +- .../validator/java-package-name.directive.js | 4 +- .../java-package-specified.directive.js | 7 +- .../modules/form/validator/uuid.directive.js | 12 +- .../frontend/app/modules/sql/Notebook.data.js | 16 +- .../states/configuration/caches/memory.jade | 5 +- .../states/configuration/caches/query.jade | 12 +- .../configuration/clusters/cache-key-cfg.jade | 2 +- .../configuration/clusters/communication.jade | 2 +- .../states/configuration/clusters/odbc.jade | 47 +++++ .../states/configuration/domains/general.jade | 4 +- .../states/configuration/domains/query.jade | 16 +- .../summary/summary.controller.js | 24 ++- .../app/services/ErrorPopover.service.js | 3 + .../app/services/JavaTypes.service.js | 163 ++++++++++-------- .../app/services/LegacyUtils.service.js | 44 ----- .../frontend/app/services/SqlTypes.service.js | 67 +++++++ .../frontend/controllers/admin-controller.js | 7 +- .../controllers/domains-controller.js | 86 +++++---- .../controllers/profile-controller.js | 5 +- .../frontend/generator/generator-common.js | 19 +- .../frontend/generator/generator-java.js | 89 +++++++++- .../generator/generator-properties.js | 2 +- .../frontend/generator/generator-xml.js | 17 +- .../public/stylesheets/form-field.scss | 108 ++++++++++++ .../frontend/public/stylesheets/style.scss | 1 + .../frontend/test/unit/JavaTypes.test.js | 82 ++++++--- .../frontend/test/unit/SqlTypes.test.js | 68 ++++++++ .../views/configuration/clusters.jade | 1 + .../views/configuration/domains-import.jade | 116 +++---------- 93 files changed, 1968 insertions(+), 791 deletions(-) delete mode 100644 modules/web-console/backend/.babelrc delete mode 100644 modules/web-console/backend/app/index.js rename modules/web-console/{frontend/app/modules/version/Version.provider.js => backend/app/mongoose.js} (71%) create mode 100644 modules/web-console/backend/test/app/db.js create mode 100644 modules/web-console/backend/test/app/httpAgent.js create mode 100644 modules/web-console/backend/test/app/mockgoose.js create mode 100644 modules/web-console/backend/test/data/spaces.json create mode 100644 modules/web-console/backend/test/index.js create mode 100644 modules/web-console/backend/test/routes/clusters.js create mode 100644 modules/web-console/backend/test/routes/public.js create mode 100644 modules/web-console/backend/test/unit/AuthService.test.js create mode 100644 modules/web-console/frontend/app/data/jdbc-types.json create mode 100644 modules/web-console/frontend/app/data/sql-keywords.json create mode 100644 modules/web-console/frontend/app/directives/retain-selection.directive.js create mode 100644 modules/web-console/frontend/app/helpers/jade/form/form-field-password.jade create mode 100644 modules/web-console/frontend/app/modules/configuration/Version.service.js create mode 100644 modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade create mode 100644 modules/web-console/frontend/app/services/SqlTypes.service.js create mode 100644 modules/web-console/frontend/public/stylesheets/form-field.scss create mode 100644 modules/web-console/frontend/test/unit/SqlTypes.test.js diff --git a/modules/web-console/DEVNOTES.txt b/modules/web-console/DEVNOTES.txt index 27211aa36792b..cf2b6507cb7d6 100644 --- a/modules/web-console/DEVNOTES.txt +++ b/modules/web-console/DEVNOTES.txt @@ -3,30 +3,21 @@ Ignite Web Console Instructions How to deploy locally: -1. Install locally MongoDB (version >=3.x) follow instructions from site http://docs.mongodb.org/manual/installation. -2. Install locally NodeJS (version >=4.x) using installer from site https://nodejs.org for your OS. -3. Change directory '$IGNITE_HOME/modules/web-console/src/main/js'. -4. Update npm to 3.x: - Linux: - npm install -g npm - Windows: - npm install -g npm-windows-upgrade - npm-windows-upgrade - See: https://github.com/felixrieseberg/npm-windows-upgrade - Check npm version: "npm --version", it should be 3.x. -5. Change directory to '$IGNITE_HOME/modules/web-console/backend' and +1. Install locally MongoDB (version >=3.2.x) follow instructions from site http://docs.mongodb.org/manual/installation. +2. Install locally NodeJS (version >=6.5.x) using installer from site https://nodejs.org/en/download/current for your OS. +3. Change directory to '$IGNITE_HOME/modules/web-console/backend' and run "npm install --no-optional" for download backend dependencies. -6. Change directory to '$IGNITE_HOME/modules/web-console/frontend' and +4. Change directory to '$IGNITE_HOME/modules/web-console/frontend' and run "npm install --no-optional" for download frontend dependencies. -7. Build ignite-web-agent module follow instructions from 'modules/web-agent/README.txt'. -8. Copy ignite-web-agent-.zip from '$IGNITE_HOME/modules/web-console/web-agent/target' +5. Build ignite-web-agent module follow instructions from 'modules/web-agent/README.txt'. +6. Copy ignite-web-agent-.zip from '$IGNITE_HOME/modules/web-console/web-agent/target' to '$IGNITE_HOME/modules/web-console/backend/agent_dists' folder. -Steps 1 - 8 should be executed once. +Steps 1 - 6 should be executed once. How to run console in development mode: -1. Configure MongoDB to run as service or in terminal change dir to $MONGO_INSTALL_DIR/server/3.0/bin +1. Configure MongoDB to run as service or in terminal change dir to $MONGO_INSTALL_DIR/server/3.2/bin and start MongoDB by executing "mongod". 2. In new terminal change directory to '$IGNITE_HOME/modules/web-console/backend'. diff --git a/modules/web-console/backend/.babelrc b/modules/web-console/backend/.babelrc deleted file mode 100644 index 7eb36f493578a..0000000000000 --- a/modules/web-console/backend/.babelrc +++ /dev/null @@ -1,9 +0,0 @@ -{ - "presets": ["es2015", "stage-1"], - "plugins": [[ - "transform-builtin-extend", { - "globals": ["Error", "Array"], - "approximate": true - } - ]] -} diff --git a/modules/web-console/backend/.eslintrc b/modules/web-console/backend/.eslintrc index c0c772b421e0c..7eb04b7964f53 100644 --- a/modules/web-console/backend/.eslintrc +++ b/modules/web-console/backend/.eslintrc @@ -1,5 +1,3 @@ -parser: "babel-eslint" - env: es6: true node: true @@ -124,7 +122,7 @@ rules: no-path-concat: 0 no-plusplus: 0 no-process-env: 0 - no-process-exit: 1 + no-process-exit: 0 no-proto: 2 no-redeclare: 2 no-regex-spaces: 1 diff --git a/modules/web-console/backend/.gitignore b/modules/web-console/backend/.gitignore index f95e2bf0a9aa7..1074affcff0a6 100644 --- a/modules/web-console/backend/.gitignore +++ b/modules/web-console/backend/.gitignore @@ -2,7 +2,5 @@ *.log .npmrc node_modules -serve/config/*.json -serve/agent_dists/*.zip agent_dists/*.zip config/*.json diff --git a/modules/web-console/backend/app/agent.js b/modules/web-console/backend/app/agent.js index a1858fddb1182..aa06c84b77cde 100644 --- a/modules/web-console/backend/app/agent.js +++ b/modules/web-console/backend/app/agent.js @@ -24,13 +24,12 @@ */ module.exports = { implements: 'agent-manager', - inject: ['require(lodash)', 'require(ws)', 'require(fs)', 'require(path)', 'require(jszip)', 'require(socket.io)', 'settings', 'mongo'] + inject: ['require(lodash)', 'require(fs)', 'require(path)', 'require(jszip)', 'require(socket.io)', 'settings', 'mongo'] }; /** * @param _ * @param fs - * @param ws * @param path * @param JSZip * @param socketio @@ -38,7 +37,7 @@ module.exports = { * @param mongo * @returns {AgentManager} */ -module.exports.factory = function(_, ws, fs, path, JSZip, socketio, settings, mongo) { +module.exports.factory = function(_, fs, path, JSZip, socketio, settings, mongo) { /** * */ @@ -487,6 +486,56 @@ module.exports.factory = function(_, ws, fs, path, JSZip, socketio, settings, mo return this.executeRest(cmd); } + + /** + * Collect cache partitions. + * @param {Boolean} demo Is need run command on demo node. + * @param {Array.} nids Cache node IDs. + * @param {String} cacheName Cache name. + * @returns {Promise} + */ + partitions(demo, nids, cacheName) { + const cmd = new Command(demo, 'exe') + .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask') + .addParam('p1', nids) + .addParam('p2', 'org.apache.ignite.internal.visor.cache.VisorCachePartitionsTask') + .addParam('p3', 'java.lang.String') + .addParam('p4', cacheName); + + return this.executeRest(cmd); + } + + /** + * Stops given node IDs. + * @param {Boolean} demo Is need run command on demo node. + * @param {Array.} nids Nodes IDs. + * @returns {Promise} + */ + stopNodes(demo, nids) { + const cmd = new Command(demo, 'exe') + .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask') + .addParam('p1', nids) + .addParam('p2', 'org.apache.ignite.internal.visor.node.VisorNodeStopTask') + .addParam('p3', 'java.lang.Void'); + + return this.executeRest(cmd); + } + + /** + * Restarts given node IDs. + * @param {Boolean} demo Is need run command on demo node. + * @param {Array.} nids Nodes IDs. + * @returns {Promise} + */ + restartNodes(demo, nids) { + const cmd = new Command(demo, 'exe') + .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask') + .addParam('p1', nids) + .addParam('p2', 'org.apache.ignite.internal.visor.node.VisorNodeRestartTask') + .addParam('p3', 'java.lang.Void'); + + return this.executeRest(cmd); + } } /** @@ -573,17 +622,6 @@ module.exports.factory = function(_, ws, fs, path, JSZip, socketio, settings, mo }); } - attachLegacy(server) { - const wsSrv = new ws.Server({server}); - - wsSrv.on('connection', (_wsClient) => { - _wsClient.send(JSON.stringify({ - method: 'authResult', - args: ['You are using an older version of the agent. Please reload agent archive'] - })); - }); - } - /** * @param {http.Server|https.Server} srv Server instance that we want to attach agent handler. */ diff --git a/modules/web-console/backend/app/app.js b/modules/web-console/backend/app/app.js index 1bbfd2c23db36..eb236e7d2d40a 100644 --- a/modules/web-console/backend/app/app.js +++ b/modules/web-console/backend/app/app.js @@ -56,6 +56,8 @@ module.exports.factory = function(Express, configure, routes) { }); srv.addListener('request', app); + + return app; } }; }; diff --git a/modules/web-console/backend/app/browser.js b/modules/web-console/backend/app/browser.js index 3256b6a2c1875..d3a673ba06121 100644 --- a/modules/web-console/backend/app/browser.js +++ b/modules/web-console/backend/app/browser.js @@ -377,7 +377,7 @@ module.exports.factory = (_, socketio, agentMgr, configure) => { .catch((err) => cb(_errorToJson(err))); }); - // GC node and return result to browser. + // Thread dump for node. socket.on('node:thread:dump', (nid, cb) => { agentMgr.findAgent(accountId()) .then((agent) => agent.threadDump(demo, nid)) @@ -390,6 +390,45 @@ module.exports.factory = (_, socketio, agentMgr, configure) => { .catch((err) => cb(_errorToJson(err))); }); + // Collect cache partitions. + socket.on('node:cache:partitions', (nids, cacheName, cb) => { + agentMgr.findAgent(accountId()) + .then((agent) => agent.partitions(demo, nids, cacheName)) + .then((data) => { + if (data.finished) + return cb(null, data.result); + + cb(_errorToJson(data.error)); + }) + .catch((err) => cb(_errorToJson(err))); + }); + + // Stops given node IDs + socket.on('node:stop', (nids, cb) => { + agentMgr.findAgent(accountId()) + .then((agent) => agent.stopNodes(demo, nids)) + .then((data) => { + if (data.finished) + return cb(null, data.result); + + cb(_errorToJson(data.error)); + }) + .catch((err) => cb(_errorToJson(err))); + }); + + // Restarts given node IDs. + socket.on('node:restart', (nids, cb) => { + agentMgr.findAgent(accountId()) + .then((agent) => agent.restartNodes(demo, nids)) + .then((data) => { + if (data.finished) + return cb(null, data.result); + + cb(_errorToJson(data.error)); + }) + .catch((err) => cb(_errorToJson(err))); + }); + const count = agentMgr.addAgentListener(user._id, socket); socket.emit('agent:count', {count}); diff --git a/modules/web-console/backend/app/index.js b/modules/web-console/backend/app/index.js deleted file mode 100644 index 57963186b314b..0000000000000 --- a/modules/web-console/backend/app/index.js +++ /dev/null @@ -1,116 +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. - */ - -'use strict'; - -import fs from 'fs'; -import path from 'path'; -import http from 'http'; -import https from 'https'; - -const igniteModules = process.env.IGNITE_MODULES || './ignite_modules'; - -let injector; - -try { - const igniteModulesInjector = path.resolve(path.join(igniteModules, 'backend', 'injector.js')); - - fs.accessSync(igniteModulesInjector, fs.F_OK); - - injector = require(igniteModulesInjector); -} catch (ignore) { - injector = require(path.join(__dirname, '../injector')); -} - -/** - * Event listener for HTTP server "error" event. - */ -const _onError = (port, error) => { - if (error.syscall !== 'listen') - throw error; - - const bind = typeof port === 'string' ? 'Pipe ' + port : 'Port ' + port; - - // Handle specific listen errors with friendly messages. - switch (error.code) { - case 'EACCES': - console.error(bind + ' requires elevated privileges'); - process.exit(1); - - break; - case 'EADDRINUSE': - console.error(bind + ' is already in use'); - process.exit(1); - - break; - default: - throw error; - } -}; - -/** - * Event listener for HTTP server "listening" event. - */ -const _onListening = (addr) => { - const bind = typeof addr === 'string' ? 'pipe ' + addr : 'port ' + addr.port; - - console.log('Start listening on ' + bind); -}; - -Promise.all([injector('settings'), injector('app'), injector('agent-manager'), injector('browser-manager')]) - .then(([settings, app, agentMgr, browserMgr]) => { - // Start rest server. - const server = settings.server.SSLOptions - ? https.createServer(settings.server.SSLOptions) : http.createServer(); - - server.listen(settings.server.port); - server.on('error', _onError.bind(null, settings.server.port)); - server.on('listening', _onListening.bind(null, server.address())); - - app.listen(server); - browserMgr.attach(server); - - // Start legacy agent server for reject connection with message. - if (settings.agent.legacyPort) { - const agentLegacySrv = settings.agent.SSLOptions - ? https.createServer(settings.agent.SSLOptions) : http.createServer(); - - agentLegacySrv.listen(settings.agent.legacyPort); - agentLegacySrv.on('error', _onError.bind(null, settings.agent.legacyPort)); - agentLegacySrv.on('listening', _onListening.bind(null, agentLegacySrv.address())); - - agentMgr.attachLegacy(agentLegacySrv); - } - - // Start agent server. - const agentServer = settings.agent.SSLOptions - ? https.createServer(settings.agent.SSLOptions) : http.createServer(); - - agentServer.listen(settings.agent.port); - agentServer.on('error', _onError.bind(null, settings.agent.port)); - agentServer.on('listening', _onListening.bind(null, agentServer.address())); - - agentMgr.attach(agentServer); - - // Used for automated test. - if (process.send) - process.send('running'); - }).catch((err) => { - console.error(err); - - process.exit(1); - }); diff --git a/modules/web-console/backend/app/mongo.js b/modules/web-console/backend/app/mongo.js index 7fe39f037bf73..ba7ed09a663ca 100644 --- a/modules/web-console/backend/app/mongo.js +++ b/modules/web-console/backend/app/mongo.js @@ -24,7 +24,7 @@ */ module.exports = { implements: 'mongo', - inject: ['require(passport-local-mongoose)', 'settings', 'ignite_modules/mongo:*', 'require(mongoose)'] + inject: ['require(passport-local-mongoose)', 'settings', 'ignite_modules/mongo:*', 'mongoose'] }; module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose) { @@ -564,6 +564,11 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose trustManagers: [String] }, rebalanceThreadPoolSize: Number, + odbc: { + odbcEnabled: Boolean, + endpointAddress: String, + maxOpenCursors: Number + }, attributes: [{name: String, value: String}], collision: { kind: {type: String, enum: ['Noop', 'PriorityQueue', 'FifoQueue', 'JobStealing', 'Custom']}, diff --git a/modules/web-console/frontend/app/modules/version/Version.provider.js b/modules/web-console/backend/app/mongoose.js similarity index 71% rename from modules/web-console/frontend/app/modules/version/Version.provider.js rename to modules/web-console/backend/app/mongoose.js index 31ff8d066bdb5..7b6e7f31e676a 100644 --- a/modules/web-console/frontend/app/modules/version/Version.provider.js +++ b/modules/web-console/backend/app/mongoose.js @@ -15,18 +15,15 @@ * limitations under the License. */ -import angular from 'angular'; +'use strict'; -angular - .module('ignite-console.version', []) - .provider('IgniteVersion', function() { - const version = { - version: '1.7.0' - }; +// Fire me up! - this.update = (newVersion) => { - version.version = newVersion; - }; +module.exports = { + implements: 'mongoose', + inject: ['require(mongoose)'] +}; - this.$get = [() => version]; - }); +module.exports.factory = (mongoose) => { + return mongoose; +}; diff --git a/modules/web-console/backend/app/settings.js b/modules/web-console/backend/app/settings.js index b3609e86c7e26..a79572e6989f0 100644 --- a/modules/web-console/backend/app/settings.js +++ b/modules/web-console/backend/app/settings.js @@ -45,13 +45,14 @@ module.exports.factory = function(nconf, fs) { return false; }; - const mailConfig = nconf.get('mail') || {}; + const mail = nconf.get('mail') || {}; + + mail.address = (username, email) => username ? '"' + username + '" <' + email + '>' : email; return { agent: { dists: 'agent_dists', port: _normalizePort(nconf.get('agentServer:port') || 3002), - legacyPort: _normalizePort(nconf.get('agentServer:legacyPort')), SSLOptions: nconf.get('agentServer:ssl') && { key: fs.readFileSync(nconf.get('agentServer:key')), cert: fs.readFileSync(nconf.get('agentServer:cert')), @@ -68,10 +69,7 @@ module.exports.factory = function(nconf, fs) { passphrase: nconf.get('server:keyPassphrase') } }, - smtp: { - ...mailConfig, - address: (username, email) => username ? '"' + username + '" <' + email + '>' : email - }, + mail, mongoUrl: nconf.get('mongodb:url') || 'mongodb://localhost/console', cookieTTL: 3600000 * 24 * 30, sessionSecret: nconf.get('server:sessionSecret') || 'keyboard cat', diff --git a/modules/web-console/backend/errors/AuthFailedException.js b/modules/web-console/backend/errors/AuthFailedException.js index 3208e1d9e642e..2772fad136f40 100644 --- a/modules/web-console/backend/errors/AuthFailedException.js +++ b/modules/web-console/backend/errors/AuthFailedException.js @@ -17,7 +17,7 @@ 'use strict'; -import AppErrorException from './AppErrorException'; +const AppErrorException = require('./AppErrorException'); class AuthFailedException extends AppErrorException { constructor(message) { diff --git a/modules/web-console/backend/errors/DuplicateKeyException.js b/modules/web-console/backend/errors/DuplicateKeyException.js index b228d0c77ce4c..536d53d888426 100644 --- a/modules/web-console/backend/errors/DuplicateKeyException.js +++ b/modules/web-console/backend/errors/DuplicateKeyException.js @@ -17,7 +17,7 @@ 'use strict'; -import AppErrorException from './AppErrorException'; +const AppErrorException = require('./AppErrorException'); class DuplicateKeyException extends AppErrorException { constructor(message) { diff --git a/modules/web-console/backend/errors/IllegalAccessError.js b/modules/web-console/backend/errors/IllegalAccessError.js index 4fcd2d40c4e57..bc07ef81f25ac 100644 --- a/modules/web-console/backend/errors/IllegalAccessError.js +++ b/modules/web-console/backend/errors/IllegalAccessError.js @@ -17,12 +17,12 @@ 'use strict'; -import AppErrorException from './AppErrorException'; +const AppErrorException = require('./AppErrorException'); class IllegalAccessError extends AppErrorException { constructor(message) { super(message); - this.httpCode = 401; + this.httpCode = 403; } } diff --git a/modules/web-console/backend/errors/IllegalArgumentException.js b/modules/web-console/backend/errors/IllegalArgumentException.js index 0487d05358984..41ccd9b3f72e2 100644 --- a/modules/web-console/backend/errors/IllegalArgumentException.js +++ b/modules/web-console/backend/errors/IllegalArgumentException.js @@ -17,7 +17,7 @@ 'use strict'; -import AppErrorException from './AppErrorException'; +const AppErrorException = require('./AppErrorException'); class IllegalArgumentException extends AppErrorException { constructor(message) { diff --git a/modules/web-console/backend/errors/MissingResourceException.js b/modules/web-console/backend/errors/MissingResourceException.js index 799775b796561..bcfb4088bc59e 100644 --- a/modules/web-console/backend/errors/MissingResourceException.js +++ b/modules/web-console/backend/errors/MissingResourceException.js @@ -17,7 +17,7 @@ 'use strict'; -import AppErrorException from './AppErrorException'; +const AppErrorException = require('./AppErrorException'); class MissingResourceException extends AppErrorException { constructor(message) { diff --git a/modules/web-console/backend/errors/index.js b/modules/web-console/backend/errors/index.js index 0af5cd5dd6086..2fadc120ab08d 100644 --- a/modules/web-console/backend/errors/index.js +++ b/modules/web-console/backend/errors/index.js @@ -19,17 +19,19 @@ // Fire me up! -import AppErrorException from './AppErrorException'; -import IllegalArgumentException from './IllegalArgumentException'; -import DuplicateKeyException from './DuplicateKeyException'; -import ServerErrorException from './ServerErrorException'; -import MissingResourceException from './MissingResourceException'; -import AuthFailedException from './AuthFailedException'; +const AppErrorException = require('./AppErrorException'); +const IllegalArgumentException = require('./IllegalArgumentException'); +const IllegalAccessError = require('./IllegalAccessError'); +const DuplicateKeyException = require('./DuplicateKeyException'); +const ServerErrorException = require('./ServerErrorException'); +const MissingResourceException = require('./MissingResourceException'); +const AuthFailedException = require('./AuthFailedException'); module.exports = { implements: 'errors', factory: () => ({ AppErrorException, + IllegalAccessError, IllegalArgumentException, DuplicateKeyException, ServerErrorException, diff --git a/modules/web-console/backend/index.js b/modules/web-console/backend/index.js index dcb3f411c3ab4..3a8ada9f2c02a 100644 --- a/modules/web-console/backend/index.js +++ b/modules/web-console/backend/index.js @@ -15,5 +15,90 @@ * limitations under the License. */ -require('babel-core/register'); -require('./app/index.js'); +'use strict'; + +const fs = require('fs'); +const path = require('path'); +const http = require('http'); +const https = require('https'); + +const igniteModules = process.env.IGNITE_MODULES || './ignite_modules'; + +let injector; + +try { + const igniteModulesInjector = path.resolve(path.join(igniteModules, 'backend', 'injector.js')); + + fs.accessSync(igniteModulesInjector, fs.F_OK); + + injector = require(igniteModulesInjector); +} catch (ignore) { + injector = require(path.join(__dirname, './injector')); +} + +/** + * Event listener for HTTP server "error" event. + */ +const _onError = (port, error) => { + if (error.syscall !== 'listen') + throw error; + + const bind = typeof port === 'string' ? 'Pipe ' + port : 'Port ' + port; + + // Handle specific listen errors with friendly messages. + switch (error.code) { + case 'EACCES': + console.error(bind + ' requires elevated privileges'); + process.exit(1); + + break; + case 'EADDRINUSE': + console.error(bind + ' is already in use'); + process.exit(1); + + break; + default: + throw error; + } +}; + +/** + * Event listener for HTTP server "listening" event. + */ +const _onListening = (addr) => { + const bind = typeof addr === 'string' ? 'pipe ' + addr : 'port ' + addr.port; + + console.log('Start listening on ' + bind); +}; + +Promise.all([injector('settings'), injector('app'), injector('agent-manager'), injector('browser-manager')]) + .then(([settings, app, agentMgr, browserMgr]) => { + // Start rest server. + const server = settings.server.SSLOptions + ? https.createServer(settings.server.SSLOptions) : http.createServer(); + + server.listen(settings.server.port); + server.on('error', _onError.bind(null, settings.server.port)); + server.on('listening', _onListening.bind(null, server.address())); + + app.listen(server); + browserMgr.attach(server); + + // Start agent server. + const agentServer = settings.agent.SSLOptions + ? https.createServer(settings.agent.SSLOptions) : http.createServer(); + + agentServer.listen(settings.agent.port); + agentServer.on('error', _onError.bind(null, settings.agent.port)); + agentServer.on('listening', _onListening.bind(null, agentServer.address())); + + agentMgr.attach(agentServer); + + // Used for automated test. + if (process.send) + process.send('running'); + }).catch((err) => { + console.error(err); + + process.exit(1); + }); diff --git a/modules/web-console/backend/injector.js b/modules/web-console/backend/injector.js index 62f89808a427e..a5996b39eb9b1 100644 --- a/modules/web-console/backend/injector.js +++ b/modules/web-console/backend/injector.js @@ -15,7 +15,7 @@ * limitations under the License. */ -import fireUp from 'fire-up'; +const fireUp = require('fire-up'); module.exports = fireUp.newInjector({ basePath: __dirname, diff --git a/modules/web-console/backend/package.json b/modules/web-console/backend/package.json index 598dc0030e10b..d50136fb48b29 100644 --- a/modules/web-console/backend/package.json +++ b/modules/web-console/backend/package.json @@ -4,8 +4,8 @@ "description": "Interactive Web console for configuration, executing SQL queries and monitoring of Apache Ignite Cluster", "private": true, "scripts": { - "ci-test": "cross-env NODE_ENV=test CONFIG_PATH='./test/config/settings.json' mocha -u tdd --require babel-core/register --reporter mocha-teamcity-reporter --recursive ./test/unit", - "test": "cross-env NODE_ENV=test CONFIG_PATH='./test/config/settings.json' mocha -u tdd --require babel-core/register --recursive ./test/unit", + "ci-test": "cross-env NODE_ENV=test MOCHA_REPORTER=mocha-teamcity-reporter node ./test/index.js", + "test": "cross-env NODE_ENV=test CONFIG_PATH='./test/config/settings.json' node ./test/index.js", "eslint": "eslint --env node --format node_modules/eslint-friendly-formatter ./ -- --eff-by-issue", "start": "node ./index.js" }, @@ -21,7 +21,7 @@ "homepage": "https://ignite.apache.org/", "engines": { "npm": "^3.x.x", - "node": "^4.x.x" + "node": "^6.5.x" }, "os": [ "darwin", @@ -32,7 +32,6 @@ "body-parser": "^1.15.0", "connect-mongo": "^1.1.0", "cookie-parser": "~1.4.0", - "es6-promise": "^3.0.2", "express": "^4.14.0", "express-session": "^1.12.0", "fire-up": "^1.0.0", @@ -47,25 +46,17 @@ "passport-local": "^1.0.0", "passport-local-mongoose": "^4.0.0", "passport.socketio": "^3.6.1", - "socket.io": "^1.4.5", - "ws": "^0.8.0" + "socket.io": "^1.4.5" }, "devDependencies": { - "babel-core": "^6.7.6", - "babel-eslint": "^6.0.4", - "babel-plugin-add-module-exports": "^0.2.1", - "babel-plugin-transform-builtin-extend": "^1.1.0", - "babel-plugin-transform-runtime": "^6.7.5", - "babel-polyfill": "^6.7.4", - "babel-preset-es2015": "^6.9.0", - "babel-preset-stage-1": "^6.5.0", - "babel-runtime": "^6.6.1", "chai": "^3.5.0", "cross-env": "^1.0.7", "eslint": "^2.9.0", "eslint-friendly-formatter": "^2.0.5", "jasmine-core": "^2.4.1", "mocha": "~2.5.3", - "mocha-teamcity-reporter": "^1.0.0" + "mocha-teamcity-reporter": "^1.0.0", + "mockgoose": "^6.0.6", + "supertest": "^2.0.0" } } diff --git a/modules/web-console/backend/routes/demo.js b/modules/web-console/backend/routes/demo.js index 724b5c1ecff36..ad4be6e6a4e15 100644 --- a/modules/web-console/backend/routes/demo.js +++ b/modules/web-console/backend/routes/demo.js @@ -19,10 +19,10 @@ // Fire me up! -import clusters from './demo/clusters.json'; -import caches from './demo/caches.json'; -import domains from './demo/domains.json'; -import igfss from './demo/igfss.json'; +const clusters = require('./demo/clusters.json'); +const caches = require('./demo/caches.json'); +const domains = require('./demo/domains.json'); +const igfss = require('./demo/igfss.json'); module.exports = { implements: 'routes/demo', diff --git a/modules/web-console/backend/routes/public.js b/modules/web-console/backend/routes/public.js index 5aad11a406ad0..590d395179abd 100644 --- a/modules/web-console/backend/routes/public.js +++ b/modules/web-console/backend/routes/public.js @@ -21,35 +21,23 @@ module.exports = { implements: 'routes/public', - inject: ['require(express)', 'require(passport)', 'settings', 'mongo', 'services/mails', 'services/users'] + inject: ['require(express)', 'require(passport)', 'mongo', 'services/mails', 'services/users', 'services/auth'] }; /** * * @param express * @param passport - * @param settings * @param mongo * @param mailsService * @param {UsersService} usersService + * @param {AuthService} authService * @returns {Promise} */ -module.exports.factory = function(express, passport, settings, mongo, mailsService, usersService) { +module.exports.factory = function(express, passport, mongo, mailsService, usersService, authService) { return new Promise((factoryResolve) => { const router = new express.Router(); - const _randomString = () => { - const possible = 'ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789'; - const possibleLen = possible.length; - - let res = ''; - - for (let i = 0; i < settings.tokenLength; i++) - res += possible.charAt(Math.floor(Math.random() * possibleLen)); - - return res; - }; - // GET user. router.post('/user', (req, res) => { usersService.get(req.user, req.session.viewedUser) @@ -107,60 +95,33 @@ module.exports.factory = function(express, passport, settings, mongo, mailsServi * Send e-mail to user with reset token. */ router.post('/password/forgot', (req, res) => { - mongo.Account.findOne({email: req.body.email}).exec() - .then((user) => { - if (!user) - throw new Error('Account with that email address does not exists!'); - - user.resetPasswordToken = _randomString(); - - return user.save(); - }) + authService.resetPasswordToken(req.body.email) .then((user) => mailsService.emailUserResetLink(req.origin(), user)) - .then(() => res.status(200).send('An email has been sent with further instructions.')) - .catch((err) => { - // TODO IGNITE-843 Send email to admin - return res.status(401).send(err.message); - }); + .then(() => 'An email has been sent with further instructions.') + .then(res.api.ok) + .catch(res.api.error); }); /** * Change password with given token. */ router.post('/password/reset', (req, res) => { - mongo.Account.findOne({resetPasswordToken: req.body.token}).exec() - .then((user) => { - if (!user) - throw new Error('Failed to find account with this token! Please check link from email.'); - - return new Promise((resolve, reject) => { - user.setPassword(req.body.password, (err, _user) => { - if (err) - return reject(new Error('Failed to reset password: ' + err.message)); + const {token, password} = req.body; - _user.resetPasswordToken = undefined; // eslint-disable-line no-undefined - - resolve(_user.save()); - }); - }); - }) + authService.resetPasswordByToken(token, password) .then((user) => mailsService.emailPasswordChanged(req.origin(), user)) - .then((user) => res.status(200).send(user.email)) - .catch((err) => res.status(401).send(err.message)); + .then((user) => user.email) + .then(res.api.ok) + .then(res.api.error); }); /* GET reset password page. */ router.post('/password/validate/token', (req, res) => { const token = req.body.token; - mongo.Account.findOne({resetPasswordToken: token}).exec() - .then((user) => { - if (!user) - throw new Error('Invalid token for password reset!'); - - return res.json({token, email: user.email}); - }) - .catch((err) => res.status(401).send(err.message)); + authService.validateResetToken(token) + .then(res.api.ok) + .catch(res.api.error); }); factoryResolve(router); diff --git a/modules/web-console/backend/services/auth.js b/modules/web-console/backend/services/auth.js index 9f7d77dd9dc0e..67a3f2f991971 100644 --- a/modules/web-console/backend/services/auth.js +++ b/modules/web-console/backend/services/auth.js @@ -21,25 +21,92 @@ module.exports = { implements: 'services/auth', - inject: ['require(lodash)', 'mongo', 'services/spaces', 'errors'] + inject: ['require(lodash)', 'mongo', 'settings', 'errors'] }; /** * @param _ * @param mongo - * @param {SpacesService} spacesService + * @param settings * @param errors * @returns {AuthService} */ -module.exports.factory = (_, mongo, spacesService, errors) => { + +module.exports.factory = (_, mongo, settings, errors) => { class AuthService { - // TODO IGNITE-3774: move implementation from public router. - static resetPassword() { + /** + * Generate token string. + * @param length - length of string + * @returns {string} - generated token + */ + static generateResetToken(length) { + length = length || settings.tokenLength; + const possible = 'ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789'; + const possibleLen = possible.length; + + let res = ''; + + for (let i = 0; i < length; i++) + res += possible.charAt(Math.floor(Math.random() * possibleLen)); + + return res; + } + + /** + * Reset password reset token for user + * @param email - user email + * @returns {Promise.} - that resolves account found by email with new reset password token. + */ + static resetPasswordToken(email) { + return mongo.Account.findOne({email}).exec() + .then((user) => { + if (!user) + throw new errors.MissingResourceException('Account with that email address does not exists!'); + + user.resetPasswordToken = AuthService.generateResetToken(settings.tokenLength); + + return user.save(); + }); + } + + /** + * Reset password by reset token. + * @param {string} token - reset token + * @param {string} newPassword - new password + * @returns {Promise.} - that resolves account with new password + */ + static resetPasswordByToken(token, newPassword) { + return mongo.Account.findOne({resetPasswordToken: token}).exec() + .then((user) => { + if (!user) + throw new errors.MissingResourceException('Failed to find account with this token! Please check link from email.'); + + return new Promise((resolve, reject) => { + user.setPassword(newPassword, (err, _user) => { + if (err) + return reject(new errors.AppErrorException('Failed to reset password: ' + err.message)); + + _user.resetPasswordToken = undefined; // eslint-disable-line no-undefined + resolve(_user.save()); + }); + }); + }); } - static validateResetToken() { + /** + * Find account by token + * @param {string} token - reset token + * @returns {Promise.<{token, email}>} - that resolves token and user email + */ + static validateResetToken(token) { + return mongo.Account.findOne({resetPasswordToken: token}).exec() + .then((user) => { + if (!user) + throw new errors.IllegalAccessError('Invalid token for password reset!'); + return {token, email: user.email}; + }); } } diff --git a/modules/web-console/backend/services/domains.js b/modules/web-console/backend/services/domains.js index 3e4e12963434a..791e229597f62 100644 --- a/modules/web-console/backend/services/domains.js +++ b/modules/web-console/backend/services/domains.js @@ -117,6 +117,8 @@ module.exports.factory = (_, mongo, spacesService, cachesService, errors) => { .then((cache) => { domain.caches = [cache._id]; + generatedCaches.push(cache); + return _saveDomainModel(domain, savedDomains); }); } diff --git a/modules/web-console/backend/services/mails.js b/modules/web-console/backend/services/mails.js index 0700985fedfb2..3c22a95a5fb3f 100644 --- a/modules/web-console/backend/services/mails.js +++ b/modules/web-console/backend/services/mails.js @@ -43,18 +43,18 @@ module.exports.factory = (_, nodemailer, settings) => { */ const send = (user, subject, html, sendErr) => { return new Promise((resolve, reject) => { - const transportConfig = settings.smtp; + const transportConfig = settings.mail; if (_.isEmpty(transportConfig.service) || _.isEmpty(transportConfig.auth.user) || _.isEmpty(transportConfig.auth.pass)) throw new Error('Failed to send email. SMTP server is not configured. Please ask webmaster to setup SMTP server!'); const mailer = nodemailer.createTransport(transportConfig); - const sign = settings.smtp.sign ? `

          --------------
          ${settings.smtp.sign}
          ` : ''; + const sign = settings.mail.sign ? `

          --------------
          ${settings.mail.sign}
          ` : ''; const mail = { - from: settings.smtp.from, - to: settings.smtp.address(`${user.firstName} ${user.lastName}`, user.email), + from: settings.mail.from, + to: settings.mail.address(`${user.firstName} ${user.lastName}`, user.email), subject, html: html + sign }; @@ -77,9 +77,9 @@ module.exports.factory = (_, nodemailer, settings) => { static emailUserSignUp(host, user) { const resetLink = `${host}/password/reset?token=${user.resetPasswordToken}`; - return send(user, `Thanks for signing up for ${settings.smtp.greeting}.`, + return send(user, `Thanks for signing up for ${settings.mail.greeting}.`, `Hello ${user.firstName} ${user.lastName}!

          ` + - `You are receiving this email because you have signed up to use ${settings.smtp.greeting}.

          ` + + `You are receiving this email because you have signed up to use ${settings.mail.greeting}.

          ` + 'If you have not done the sign up and do not know what this email is about, please ignore it.
          ' + 'You may reset the password by clicking on the following link, or paste this into your browser:

          ' + `${resetLink}`); @@ -110,7 +110,7 @@ module.exports.factory = (_, nodemailer, settings) => { static emailPasswordChanged(host, user) { return send(user, 'Your password has been changed', `Hello ${user.firstName} ${user.lastName}!

          ` + - `This is a confirmation that the password for your account on ${settings.smtp.greeting} has just been changed.

          `, + `This is a confirmation that the password for your account on ${settings.mail.greeting} has just been changed.

          `, 'Password was changed, but failed to send confirmation email!'); } @@ -122,7 +122,7 @@ module.exports.factory = (_, nodemailer, settings) => { static emailUserDeletion(host, user) { return send(user, 'Your account was removed', `Hello ${user.firstName} ${user.lastName}!

          ` + - `You are receiving this email because your account for ${settings.smtp.greeting} was removed.`, + `You are receiving this email because your account for ${settings.mail.greeting} was removed.`, 'Account was removed, but failed to send email notification to user!'); } } diff --git a/modules/web-console/backend/services/sessions.js b/modules/web-console/backend/services/sessions.js index 4fa95a3bb286c..ff0e303068080 100644 --- a/modules/web-console/backend/services/sessions.js +++ b/modules/web-console/backend/services/sessions.js @@ -38,11 +38,13 @@ module.exports.factory = (_, mongo, errors) => { * @param {mongo.ObjectId|String} viewedUserId - id of user to become. */ static become(session, viewedUserId) { - return mongo.Account.findById(viewedUserId).exec() + return mongo.Account.findById(viewedUserId).lean().exec() .then((viewedUser) => { if (!session.req.user.admin) throw new errors.IllegalAccessError('Became this user is not permitted. Only administrators can perform this actions.'); + viewedUser.token = session.req.user.token; + session.viewedUser = viewedUser; }); } diff --git a/modules/web-console/backend/test/app/db.js b/modules/web-console/backend/test/app/db.js new file mode 100644 index 0000000000000..e07f887ab6e96 --- /dev/null +++ b/modules/web-console/backend/test/app/db.js @@ -0,0 +1,66 @@ +/* + * 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. + */ + +'use strict'; + +// Fire me up! + +const testAccounts = require('../data/accounts.json'); +const testClusters = require('../data/clusters.json'); +const testCaches = require('../data/caches.json'); +const testDomains = require('../data/domains.json'); +const testIgfss = require('../data/igfss.json'); +const testSpaces = require('../data/spaces.json'); + +module.exports = { + implements: 'dbHelper', + inject: ['require(lodash)', 'mongo', 'mongoose'] +}; + +module.exports.factory = (_, mongo, mongoose) => { + const prepareUserSpaces = () => Promise.all([mongo.Account.create(testAccounts), mongo.Space.create(testSpaces)]); + const prepareClusters = () => mongo.Cluster.create(testClusters); + const prepareDomains = () => mongo.DomainModel.create(testDomains); + const prepareCaches = () => mongo.Cache.create(testCaches); + const prepareIgfss = () => mongo.Igfs.create(testIgfss); + + const drop = () => { + return Promise.all(_.map(mongoose.connection.collections, (collection) => collection.remove())); + }; + + const init = () => { + return drop() + .then(prepareUserSpaces) + .then(prepareClusters) + .then(prepareDomains) + .then(prepareCaches) + .then(prepareIgfss); + }; + + return { + drop, + init, + mocks: { + accounts: testAccounts, + clusters: testClusters, + caches: testCaches, + domains: testDomains, + igfss: testIgfss, + spaces: testSpaces + } + }; +}; diff --git a/modules/web-console/backend/test/app/httpAgent.js b/modules/web-console/backend/test/app/httpAgent.js new file mode 100644 index 0000000000000..1394dc5c46e8a --- /dev/null +++ b/modules/web-console/backend/test/app/httpAgent.js @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'agentFactory', + inject: ['app', 'require(http)', 'require(supertest)'] +}; + +module.exports.factory = (app, http, request) => { + const express = app.listen(http.createServer()); + let authAgentInstance = null; + + return { + authAgent: ({email, password}) => { + if (authAgentInstance) + return Promise.resolve(authAgentInstance); + + return new Promise((resolve, reject) => { + authAgentInstance = request.agent(express); + authAgentInstance.post('/signin') + .send({email, password}) + .end((err, res) => { + if (res.status === 401 || err) + return reject(err); + + resolve(authAgentInstance); + }); + }); + }, + guestAgent: () => Promise.resolve(request.agent(express)) + }; +}; diff --git a/modules/web-console/backend/test/app/mockgoose.js b/modules/web-console/backend/test/app/mockgoose.js new file mode 100644 index 0000000000000..4944f90e9d8e2 --- /dev/null +++ b/modules/web-console/backend/test/app/mockgoose.js @@ -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. + */ + +'use strict'; + +// Fire me up! + +module.exports = { + implements: 'mongoose:mock', + inject: ['require(mongoose)', 'require(mockgoose)'] +}; + +module.exports.factory = (mongoose, mockgoose) => { + return mockgoose(mongoose) + .then(() => mongoose); +}; diff --git a/modules/web-console/backend/test/data/accounts.json b/modules/web-console/backend/test/data/accounts.json index e5b7f98967a5f..9dcb0eda15a70 100644 --- a/modules/web-console/backend/test/data/accounts.json +++ b/modules/web-console/backend/test/data/accounts.json @@ -1,8 +1,9 @@ [ { - "_id" : "57725443e6d604c05dab9ded", + "_id" : "000000000000000000000001", "salt" : "ca8b49c2eacd498a0973de30c0873c166ed99fa0605981726aedcc85bee17832", "hash" : "c052c87e454cd0875332719e1ce085ccd92bedb73c8f939ba45d387f724da97128280643ad4f841d929d48de802f48f4a27b909d2dc806d957d38a1a4049468ce817490038f00ac1416aaf9f8f5a5c476730b46ea22d678421cd269869d4ba9d194f73906e5d5a4fec5229459e20ebda997fb95298067126f6c15346d886d44b67def03bf3ffe484b2e4fa449985de33a0c12e4e1da4c7d71fe7af5d138433f703d8c7eeebbb3d57f1a89659010a1f1d3cd4fbc524abab07860daabb08f08a28b8bfc64ecde2ea3c103030d0d54fc24d9c02f92ee6b3aa1bcd5c70113ab9a8045faea7dd2dc59ec4f9f69fcf634232721e9fb44012f0e8c8fdf7c6bf642db6867ef8e7877123e1bc78af7604fee2e34ad0191f8b97613ea458e0fca024226b7055e08a4bdb256fabf0a203a1e5b6a6c298fb0c60308569cefba779ce1e41fb971e5d1745959caf524ab0bedafce67157922f9c505cea033f6ed28204791470d9d08d31ce7e8003df8a3a05282d4d60bfe6e2f7de06f4b18377dac0fe764ed683c9b2553e75f8280c748aa166fef6f89190b1c6d369ab86422032171e6f9686de42ac65708e63bf018a043601d85bc5c820c7ad1d51ded32e59cdaa629a3f7ae325bbc931f9f21d90c9204effdbd53721a60c8b180dd8c236133e287a47ccc9e5072eb6593771e435e4d5196d50d6ddb32c226651c6503387895c5ad025f69fd3", + "password": "a", "email" : "a@a", "firstName" : "TestFirstName", "lastName" : "TestLastName", @@ -15,4 +16,4 @@ "__v" : 0, "resetPasswordToken" : "892rnLbEnVp1FP75Jgpi" } -] \ No newline at end of file +] diff --git a/modules/web-console/backend/test/data/caches.json b/modules/web-console/backend/test/data/caches.json index f7a8690ce7fa7..697d4148562ef 100644 --- a/modules/web-console/backend/test/data/caches.json +++ b/modules/web-console/backend/test/data/caches.json @@ -1,5 +1,7 @@ [ { + "_id" : "000000000000000000000001", + "space": "000000000000000000000001", "name": "CarCache", "cacheMode": "PARTITIONED", "atomicityMode": "ATOMIC", @@ -13,10 +15,12 @@ "dialect": "H2" } }, - "domains": [], - "clusters": [] + "domains": ["000000000000000000000001", "000000000000000000000002", "000000000000000000000003", "000000000000000000000004", "000000000000000000000005"], + "clusters": ["000000000000000000000001", "000000000000000000000002"] }, { + "_id" : "000000000000000000000002", + "space": "000000000000000000000001", "name": "ParkingCache", "cacheMode": "PARTITIONED", "atomicityMode": "ATOMIC", @@ -30,10 +34,12 @@ "dialect": "H2" } }, - "domains": [], - "clusters": [] + "domains": ["000000000000000000000001", "000000000000000000000002", "000000000000000000000003", "000000000000000000000004", "000000000000000000000005"], + "clusters": ["000000000000000000000001", "000000000000000000000002"] }, { + "_id" : "000000000000000000000003", + "space": "000000000000000000000001", "name": "CountryCache", "cacheMode": "PARTITIONED", "atomicityMode": "ATOMIC", @@ -47,10 +53,12 @@ "dialect": "H2" } }, - "domains": [], - "clusters": [] + "domains": ["000000000000000000000001", "000000000000000000000002", "000000000000000000000003", "000000000000000000000004", "000000000000000000000005"], + "clusters": ["000000000000000000000002"] }, { + "_id" : "000000000000000000000004", + "space": "000000000000000000000001", "name": "DepartmentCache", "cacheMode": "PARTITIONED", "atomicityMode": "ATOMIC", @@ -64,10 +72,12 @@ "dialect": "H2" } }, - "domains": [], - "clusters": [] + "domains": ["000000000000000000000001", "000000000000000000000002", "000000000000000000000003", "000000000000000000000004", "000000000000000000000005"], + "clusters": ["000000000000000000000002"] }, { + "_id" : "000000000000000000000005", + "space": "000000000000000000000001", "name": "EmployeeCache", "cacheMode": "PARTITIONED", "atomicityMode": "ATOMIC", @@ -81,7 +91,7 @@ "dialect": "H2" } }, - "domains": [], - "clusters": [] + "domains": ["000000000000000000000001", "000000000000000000000002", "000000000000000000000003", "000000000000000000000004", "000000000000000000000005"], + "clusters": ["000000000000000000000002"] } ] diff --git a/modules/web-console/backend/test/data/clusters.json b/modules/web-console/backend/test/data/clusters.json index 014b51938f1dc..8e16e76dfb1b0 100644 --- a/modules/web-console/backend/test/data/clusters.json +++ b/modules/web-console/backend/test/data/clusters.json @@ -1,5 +1,7 @@ [ { + "_id" : "000000000000000000000001", + "space": "000000000000000000000001", "name": "cluster-igfs", "connector": { "noDelay": true @@ -7,8 +9,8 @@ "communication": { "tcpNoDelay": true }, - "igfss": [], - "caches": [], + "igfss": ["000000000000000000000001"], + "caches": ["000000000000000000000001", "000000000000000000000002"], "binaryConfiguration": { "compactFooter": true, "typeConfigurations": [] @@ -24,6 +26,8 @@ } }, { + "_id" : "000000000000000000000002", + "space": "000000000000000000000001", "name": "cluster-caches", "connector": { "noDelay": true @@ -32,7 +36,7 @@ "tcpNoDelay": true }, "igfss": [], - "caches": [], + "caches": ["000000000000000000000001", "000000000000000000000002", "000000000000000000000003", "000000000000000000000004", "000000000000000000000005"], "binaryConfiguration": { "compactFooter": true, "typeConfigurations": [] diff --git a/modules/web-console/backend/test/data/domains.json b/modules/web-console/backend/test/data/domains.json index 980d8d1059a94..e2662db589c1b 100644 --- a/modules/web-console/backend/test/data/domains.json +++ b/modules/web-console/backend/test/data/domains.json @@ -1,5 +1,7 @@ [ { + "_id" : "000000000000000000000001", + "space": "000000000000000000000001", "keyType": "Integer", "valueType": "model.Parking", "queryMetadata": "Configuration", @@ -42,6 +44,8 @@ "caches": [] }, { + "_id" : "000000000000000000000002", + "space": "000000000000000000000001", "keyType": "Integer", "valueType": "model.Department", "queryMetadata": "Configuration", @@ -84,6 +88,8 @@ "caches": [] }, { + "_id" : "000000000000000000000003", + "space": "000000000000000000000001", "keyType": "Integer", "valueType": "model.Employee", "queryMetadata": "Configuration", @@ -221,6 +227,8 @@ "caches": [] }, { + "_id" : "000000000000000000000004", + "space": "000000000000000000000001", "keyType": "Integer", "valueType": "model.Country", "queryMetadata": "Configuration", @@ -263,6 +271,8 @@ "caches": [] }, { + "_id" : "000000000000000000000005", + "space": "000000000000000000000001", "keyType": "Integer", "valueType": "model.Car", "queryMetadata": "Configuration", diff --git a/modules/web-console/backend/test/data/igfss.json b/modules/web-console/backend/test/data/igfss.json index cd128a68bc6fb..c1f06455fb3cf 100644 --- a/modules/web-console/backend/test/data/igfss.json +++ b/modules/web-console/backend/test/data/igfss.json @@ -1,10 +1,12 @@ [ { + "_id" : "000000000000000000000001", + "space": "000000000000000000000001", "ipcEndpointEnabled": true, "fragmentizerEnabled": true, "name": "igfs", "dataCacheName": "igfs-data", "metaCacheName": "igfs-meta", - "clusters": [] + "clusters": ["000000000000000000000001"] } ] diff --git a/modules/web-console/backend/test/data/spaces.json b/modules/web-console/backend/test/data/spaces.json new file mode 100644 index 0000000000000..519f7fee36ced --- /dev/null +++ b/modules/web-console/backend/test/data/spaces.json @@ -0,0 +1,14 @@ +[ + { + "_id": "000000000000000000000001", + "name": "Personal space", + "owner": "000000000000000000000001", + "demo": false + }, + { + "_id": "000000000000000000000002", + "name": "Demo space", + "owner": "000000000000000000000001", + "demo": true + } +] diff --git a/modules/web-console/backend/test/index.js b/modules/web-console/backend/test/index.js new file mode 100644 index 0000000000000..4519219c795c0 --- /dev/null +++ b/modules/web-console/backend/test/index.js @@ -0,0 +1,35 @@ +/* + * 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. + */ + +const Mocha = require('mocha'); +const glob = require('glob'); +const path = require('path'); + +const mocha = new Mocha({ui: 'tdd', reporter: process.env.MOCHA_REPORTER || 'spec'}); +const testPath = ['./test/unit/**/*.js', './test/routes/**/*.js']; + +if (process.env.IGNITE_MODULES) + testPath.push(path.join(process.env.IGNITE_MODULES, 'backend', 'test', 'unit', '**', '*.js')); + +testPath + .map((mask) => glob.sync(mask)) + .reduce((acc, items) => acc.concat(items), []) + .map(mocha.addFile.bind(mocha)); + +const runner = mocha.run(); + +runner.on('end', (failures) => process.exit(failures)); diff --git a/modules/web-console/backend/test/injector.js b/modules/web-console/backend/test/injector.js index 8d44d310e3f03..bdeca91a82dd9 100644 --- a/modules/web-console/backend/test/injector.js +++ b/modules/web-console/backend/test/injector.js @@ -15,17 +15,34 @@ * limitations under the License. */ -import path from 'path'; -import fireUp from 'fire-up'; +const fs = require('fs'); +const path = require('path'); +const fireUp = require('fire-up'); -module.exports = fireUp.newInjector({ - basePath: path.join(__dirname, '../'), - modules: [ - './app/**/*.js', - './config/**/*.js', - './errors/**/*.js', - './middlewares/**/*.js', - './routes/**/*.js', - './services/**/*.js' - ] -}); +const igniteModules = process.env.IGNITE_MODULES || './ignite_modules'; + +let injector; + +try { + const igniteModulesInjector = path.resolve(path.join(igniteModules, 'backend', 'test', 'injector.js')); + + fs.accessSync(igniteModulesInjector, fs.F_OK); + + injector = require(igniteModulesInjector); +} catch (ignore) { + injector = fireUp.newInjector({ + basePath: path.join(__dirname, '../'), + modules: [ + './app/**/*.js', + './config/**/*.js', + './errors/**/*.js', + './middlewares/**/*.js', + './routes/**/*.js', + './services/**/*.js', + './test/app/*.js' + ], + use: ['mongoose:mock'] + }); +} + +module.exports = injector; diff --git a/modules/web-console/backend/test/routes/clusters.js b/modules/web-console/backend/test/routes/clusters.js new file mode 100644 index 0000000000000..5dd7a60bfdbd8 --- /dev/null +++ b/modules/web-console/backend/test/routes/clusters.js @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +const assert = require('chai').assert; +const injector = require('../injector'); +const mongoose = require('mongoose'); + +let agentFactory; +let db; + +suite('routes.clusters', () => { + suiteSetup(() => { + return Promise.all([injector('agentFactory'), injector('dbHelper')]) + .then(([_agent, _db]) => { + agentFactory = _agent; + db = _db; + }); + }); + + setup(() => { + return db.init(); + }); + + test('Save cluster model', (done) => { + const newCluster = Object.assign({}, db.mocks.clusters[0], {name: 'newClusterName'}); + + agentFactory.authAgent(db.mocks.accounts[0]) + .then((agent) => { + agent.post('/configuration/clusters/save') + .send(newCluster) + .expect(200) + .expect((res) => { + assert.isNotNull(res.body); + assert.isTrue(mongoose.Types.ObjectId.isValid(res.body)); + }) + .end(done); + }) + .catch(done); + }); + + test('Remove cluster model', (done) => { + return agentFactory.authAgent(db.mocks.accounts[0]) + .then((agent) => { + agent.post('/configuration/clusters/remove') + .send({_id: db.mocks.clusters[0]._id}) + .expect(200) + .expect((res) => { + assert.isNotNull(res.body); + assert.equal(res.body.rowsAffected, 1); + }) + .end(done); + }) + .catch(done); + }); + + test('Remove all clusters', (done) => { + return agentFactory.authAgent(db.mocks.accounts[0]) + .then((agent) => { + agent.post('/configuration/clusters/remove/all') + .expect(200) + .expect((res) => { + assert.isNotNull(res.body); + assert.equal(res.body.rowsAffected, db.mocks.clusters.length); + }) + .end(done); + }) + .catch(done); + }); +}); diff --git a/modules/web-console/backend/test/routes/public.js b/modules/web-console/backend/test/routes/public.js new file mode 100644 index 0000000000000..3c573c5fa3e32 --- /dev/null +++ b/modules/web-console/backend/test/routes/public.js @@ -0,0 +1,68 @@ +/* + * 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. + */ + +const assert = require('chai').assert; +const injector = require('../injector'); + +const testAccounts = require('../data/accounts.json'); + +let agentFactory; +let db; + +suite('routes.public', () => { + suiteSetup(() => { + return Promise.all([injector('agentFactory'), injector('dbHelper')]) + .then(([_agent, _db]) => { + agentFactory = _agent; + db = _db; + }); + }); + + setup(() => { + return db.init(); + }); + + test('Login success', (done) => { + const user = testAccounts[0]; + + agentFactory.guestAgent() + .then((agent) => { + agent.post('/signin') + .send({email: user.email, password: user.password}) + .expect(200) + .expect((res) => { + assert.isNotNull(res.headers['set-cookie']); + assert.match(res.headers['set-cookie'], /connect\.sid/); + }) + .end(done); + }) + .catch(done); + }); + + test('Login fail', (done) => { + const user = testAccounts[0]; + + agentFactory.guestAgent() + .then((agent) => { + agent.post('/signin') + .send({email: user.email, password: 'notvalidpassword'}) + .expect(401) + .end(done); + }) + .catch(done); + }); +}); diff --git a/modules/web-console/backend/test/unit/AuthService.test.js b/modules/web-console/backend/test/unit/AuthService.test.js new file mode 100644 index 0000000000000..eec60c41ea5d1 --- /dev/null +++ b/modules/web-console/backend/test/unit/AuthService.test.js @@ -0,0 +1,107 @@ +/* + * 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. + */ + +const assert = require('chai').assert; +const injector = require('../injector'); +const testAccounts = require('../data/accounts.json'); + +let authService; +let errors; +let db; + +suite('AuthServiceTestsSuite', () => { + suiteSetup(() => { + return Promise.all([injector('services/auth'), + injector('errors'), + injector('dbHelper')]) + .then(([_authService, _errors, _db]) => { + authService = _authService; + errors = _errors; + db = _db; + }); + }); + + setup(() => { + return db.init(); + }); + + test('Check token generator', () => { + const tokenLength = 16; + const token1 = authService.generateResetToken(tokenLength); + const token2 = authService.generateResetToken(tokenLength); + + assert.equal(token1.length, tokenLength); + assert.equal(token2.length, tokenLength); + assert.notEqual(token1, token2); + }); + + + test('Reset password token for non existing user', (done) => { + authService.resetPasswordToken('non-exisitng@email.ee') + .catch((err) => { + assert.instanceOf(err, errors.MissingResourceException); + done(); + }); + }); + + test('Reset password token for existing user', (done) => { + authService.resetPasswordToken(testAccounts[0].email) + .then((account) => { + assert.notEqual(account.resetPasswordToken.length, 0); + assert.notEqual(account.resetPasswordToken, testAccounts[0].resetPasswordToken); + }) + .then(done) + .catch(done); + }); + + test('Reset password by token for non existing user', (done) => { + authService.resetPasswordByToken('0') + .catch((err) => { + assert.instanceOf(err, errors.MissingResourceException); + done(); + }); + }); + + test('Reset password by token for existing user', (done) => { + authService.resetPasswordByToken(testAccounts[0].resetPasswordToken, 'NewUniquePassword$1') + .then((account) => { + assert.isUndefined(account.resetPasswordToken); + assert.notEqual(account.hash, 0); + assert.notEqual(account.hash, testAccounts[0].hash); + }) + .then(done) + .catch(done); + }); + + test('Validate user for non existing reset token', (done) => { + authService.validateResetToken('Non existing token') + .catch((err) => { + assert.instanceOf(err, errors.IllegalAccessError); + done(); + }); + }); + + test('Validate reset token', (done) => { + authService.validateResetToken(testAccounts[0].resetPasswordToken) + .then(({token, email}) => { + assert.equal(email, testAccounts[0].email); + assert.equal(token, testAccounts[0].resetPasswordToken); + }) + .then(done) + .catch(done); + }); +}); diff --git a/modules/web-console/backend/test/unit/CacheService.test.js b/modules/web-console/backend/test/unit/CacheService.test.js index 14427757d1796..980f47afb0e9b 100644 --- a/modules/web-console/backend/test/unit/CacheService.test.js +++ b/modules/web-console/backend/test/unit/CacheService.test.js @@ -15,11 +15,10 @@ * limitations under the License. */ - -import {assert} from 'chai'; -import injector from '../injector'; -import testCaches from '../data/caches.json'; -import testAccounts from '../data/accounts.json'; +const assert = require('chai').assert; +const injector = require('../injector'); +const testCaches = require('../data/caches.json'); +const testAccounts = require('../data/accounts.json'); let cacheService; let mongo; @@ -79,7 +78,7 @@ suite('CacheServiceTestsSuite', () => { cacheService.merge(testCaches[0]) .then((cache) => { - const cacheBeforeMerge = {...testCaches[0], _id: cache._id, name: newName}; + const cacheBeforeMerge = Object.assign({}, testCaches[0], {_id: cache._id, name: newName}); return cacheService.merge(cacheBeforeMerge); }) @@ -92,11 +91,12 @@ suite('CacheServiceTestsSuite', () => { }); test('Create duplicated cache', (done) => { + const dupleCache = Object.assign({}, testCaches[0], {_id: null}); + cacheService.merge(testCaches[0]) - .then(() => cacheService.merge(testCaches[0])) + .then(() => cacheService.merge(dupleCache)) .catch((err) => { assert.instanceOf(err, errors.DuplicateKeyException); - done(); }); }); @@ -145,7 +145,7 @@ suite('CacheServiceTestsSuite', () => { prepareUserSpaces() .then(([accounts, spaces]) => { const currentUser = accounts[0]; - const userCache = {...testCaches[0], space: spaces[0][0]._id}; + const userCache = Object.assign({}, testCaches[0], {space: spaces[0][0]._id}); return cacheService.merge(userCache) .then(() => cacheService.removeAll(currentUser._id, false)); @@ -160,7 +160,7 @@ suite('CacheServiceTestsSuite', () => { test('Get all caches by space', (done) => { prepareUserSpaces() .then(([accounts, spaces]) => { - const userCache = {...testCaches[0], space: spaces[0][0]._id}; + const userCache = Object.assign({}, testCaches[0], {space: spaces[0][0]._id}); return cacheService.merge(userCache) .then((cache) => { diff --git a/modules/web-console/backend/test/unit/ClusterService.test.js b/modules/web-console/backend/test/unit/ClusterService.test.js index ab0e912b639e1..aff54f7a4ed0b 100644 --- a/modules/web-console/backend/test/unit/ClusterService.test.js +++ b/modules/web-console/backend/test/unit/ClusterService.test.js @@ -15,11 +15,10 @@ * limitations under the License. */ - -import {assert} from 'chai'; -import injector from '../injector'; -import testClusters from '../data/clusters.json'; -import testAccounts from '../data/accounts.json'; +const assert = require('chai').assert; +const injector = require('../injector'); +const testClusters = require('../data/clusters.json'); +const testAccounts = require('../data/accounts.json'); let clusterService; let mongo; @@ -78,7 +77,7 @@ suite('ClusterServiceTestsSuite', () => { clusterService.merge(testClusters[0]) .then((cluster) => { - const clusterBeforeMerge = {...testClusters[0], _id: cluster._id, name: newName}; + const clusterBeforeMerge = Object.assign({}, testClusters[0], {_id: cluster._id, name: newName}); return clusterService.merge(clusterBeforeMerge); }) @@ -91,8 +90,10 @@ suite('ClusterServiceTestsSuite', () => { }); test('Create duplicated cluster', (done) => { + const dupleCluster = Object.assign({}, testClusters[0], {_id: null}); + clusterService.merge(testClusters[0]) - .then(() => clusterService.merge(testClusters[0])) + .then(() => clusterService.merge(dupleCluster)) .catch((err) => { assert.instanceOf(err, errors.DuplicateKeyException); @@ -143,7 +144,7 @@ suite('ClusterServiceTestsSuite', () => { prepareUserSpaces() .then(([accounts, spaces]) => { const currentUser = accounts[0]; - const userCluster = {...testClusters[0], space: spaces[0][0]._id}; + const userCluster = Object.assign({}, testClusters[0], {space: spaces[0][0]._id}); return clusterService.merge(userCluster) .then(() => clusterService.removeAll(currentUser._id, false)); @@ -158,7 +159,7 @@ suite('ClusterServiceTestsSuite', () => { test('Get all clusters by space', (done) => { prepareUserSpaces() .then(([accounts, spaces]) => { - const userCluster = {...testClusters[0], space: spaces[0][0]._id}; + const userCluster = Object.assign({}, testClusters[0], {space: spaces[0][0]._id}); return clusterService.merge(userCluster) .then((existCluster) => { diff --git a/modules/web-console/backend/test/unit/DomainService.test.js b/modules/web-console/backend/test/unit/DomainService.test.js index 477b45418d4bb..8ce3fef3b0bea 100644 --- a/modules/web-console/backend/test/unit/DomainService.test.js +++ b/modules/web-console/backend/test/unit/DomainService.test.js @@ -15,11 +15,10 @@ * limitations under the License. */ - -import {assert} from 'chai'; -import injector from '../injector'; -import testDomains from '../data/domains.json'; -import testAccounts from '../data/accounts.json'; +const assert = require('chai').assert; +const injector = require('../injector'); +const testDomains = require('../data/domains.json'); +const testAccounts = require('../data/accounts.json'); let domainService; let mongo; @@ -82,7 +81,7 @@ suite('DomainsServiceTestsSuite', () => { .then((results) => { const domain = results.savedDomains[0]; - const domainBeforeMerge = {...testDomains[0], _id: domain._id, valueType: newValType}; + const domainBeforeMerge = Object.assign({}, testDomains[0], {_id: domain._id, valueType: newValType}); return domainService.batchMerge([domainBeforeMerge]); }) @@ -95,8 +94,10 @@ suite('DomainsServiceTestsSuite', () => { }); test('Create duplicated domain', (done) => { + const dupleDomain = Object.assign({}, testDomains[0], {_id: null}); + domainService.batchMerge([testDomains[0]]) - .then(() => domainService.batchMerge([testDomains[0]])) + .then(() => domainService.batchMerge([dupleDomain])) .catch((err) => { assert.instanceOf(err, errors.DuplicateKeyException); @@ -149,7 +150,7 @@ suite('DomainsServiceTestsSuite', () => { prepareUserSpaces() .then(([accounts, spaces]) => { const currentUser = accounts[0]; - const userDomain = {...testDomains[0], space: spaces[0][0]._id}; + const userDomain = Object.assign({}, testDomains[0], {space: spaces[0][0]._id}); return domainService.batchMerge([userDomain]) .then(() => domainService.removeAll(currentUser._id, false)); @@ -164,7 +165,7 @@ suite('DomainsServiceTestsSuite', () => { test('Get all domains by space', (done) => { prepareUserSpaces() .then(([accounts, spaces]) => { - const userDomain = {...testDomains[0], space: spaces[0][0]._id}; + const userDomain = Object.assign({}, testDomains[0], {space: spaces[0][0]._id}); return domainService.batchMerge([userDomain]) .then((results) => { diff --git a/modules/web-console/backend/test/unit/IgfsService.test.js b/modules/web-console/backend/test/unit/IgfsService.test.js index 3d78148ce3cbb..4fab27947a27f 100644 --- a/modules/web-console/backend/test/unit/IgfsService.test.js +++ b/modules/web-console/backend/test/unit/IgfsService.test.js @@ -15,11 +15,10 @@ * limitations under the License. */ - -import {assert} from 'chai'; -import injector from '../injector'; -import testIgfss from '../data/igfss.json'; -import testAccounts from '../data/accounts.json'; +const assert = require('chai').assert; +const injector = require('../injector'); +const testIgfss = require('../data/igfss.json'); +const testAccounts = require('../data/accounts.json'); let igfsService; let mongo; @@ -78,7 +77,7 @@ suite('IgfsServiceTestsSuite', () => { igfsService.merge(testIgfss[0]) .then((existIgfs) => { - const igfsBeforeMerge = {...testIgfss[0], _id: existIgfs._id, name: newName}; + const igfsBeforeMerge = Object.assign({}, testIgfss[0], {_id: existIgfs._id, name: newName}); return igfsService.merge(igfsBeforeMerge); }) @@ -91,8 +90,10 @@ suite('IgfsServiceTestsSuite', () => { }); test('Create duplicated igfs', (done) => { + const dupleIfgs = Object.assign({}, testIgfss[0], {_id: null}); + igfsService.merge(testIgfss[0]) - .then(() => igfsService.merge(testIgfss[0])) + .then(() => igfsService.merge(dupleIfgs)) .catch((err) => { assert.instanceOf(err, errors.DuplicateKeyException); @@ -143,7 +144,7 @@ suite('IgfsServiceTestsSuite', () => { prepareUserSpaces() .then(([accounts, spaces]) => { const currentUser = accounts[0]; - const userIgfs = {...testIgfss[0], space: spaces[0][0]._id}; + const userIgfs = Object.assign({}, testIgfss[0], {space: spaces[0][0]._id}); return igfsService.merge(userIgfs) .then(() => igfsService.removeAll(currentUser._id, false)); @@ -158,7 +159,7 @@ suite('IgfsServiceTestsSuite', () => { test('Get all igfss by space', (done) => { prepareUserSpaces() .then(([accounts, spaces]) => { - const userIgfs = {...testIgfss[0], space: spaces[0][0]._id}; + const userIgfs = Object.assign({}, testIgfss[0], {space: spaces[0][0]._id}); return igfsService.merge(userIgfs) .then((existIgfs) => { diff --git a/modules/web-console/frontend/app/app.config.js b/modules/web-console/frontend/app/app.config.js index 25c24b0ee2852..7adfc94fdd6e8 100644 --- a/modules/web-console/frontend/app/app.config.js +++ b/modules/web-console/frontend/app/app.config.js @@ -46,7 +46,7 @@ igniteConsoleCfg.config(['$popoverProvider', ($popoverProvider) => { igniteConsoleCfg.config(['$tooltipProvider', ($tooltipProvider) => { angular.extend($tooltipProvider.defaults, { container: 'body', - delay: 150, + delay: {show: 150, hide: 150}, placement: 'right', html: 'true', trigger: 'click hover' diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js index 45851a25cfe57..830d067be936f 100644 --- a/modules/web-console/frontend/app/app.js +++ b/modules/web-console/frontend/app/app.js @@ -43,7 +43,6 @@ import './modules/navbar/navbar.module'; import './modules/configuration/configuration.module'; import './modules/getting-started/GettingStarted.provider'; import './modules/dialog/dialog.module'; -import './modules/version/Version.provider'; import './modules/ace.module'; import './modules/socket.module'; import './modules/loading/loading.module'; @@ -67,6 +66,7 @@ import igniteUiAcePojos from './directives/ui-ace-pojos/ui-ace-pojos.directive'; import igniteUiAcePom from './directives/ui-ace-pom/ui-ace-pom.directive'; import igniteUiAceTabs from './directives/ui-ace-tabs.directive'; import igniteUiAceXml from './directives/ui-ace-xml/ui-ace-xml.directive'; +import igniteRetainSelection from './directives/retain-selection.directive'; // Services. import ChartColors from './services/ChartColors.service'; @@ -75,19 +75,18 @@ import Confirm from './services/Confirm.service.js'; import ConfirmBatch from './services/ConfirmBatch.service.js'; import CopyToClipboard from './services/CopyToClipboard.service'; import Countries from './services/Countries.service'; +import ErrorPopover from './services/ErrorPopover.service'; import Focus from './services/Focus.service'; +import FormUtils from './services/FormUtils.service'; import InetAddress from './services/InetAddress.service'; import JavaTypes from './services/JavaTypes.service'; -import Messages from './services/Messages.service'; -import ModelNormalizer from './services/ModelNormalizer.service.js'; +import SqlTypes from './services/SqlTypes.service'; import LegacyTable from './services/LegacyTable.service'; -import ErrorPopover from './services/ErrorPopover.service'; -import FormUtils from './services/FormUtils.service'; import LegacyUtils from './services/LegacyUtils.service'; +import Messages from './services/Messages.service'; +import ModelNormalizer from './services/ModelNormalizer.service.js'; import UnsavedChangesGuard from './services/UnsavedChangesGuard.service'; -// Providers. - // Filters. import byName from './filters/byName.filter'; import domainsValidation from './filters/domainsValidation.filter'; @@ -167,7 +166,6 @@ angular 'ignite-console.navbar', 'ignite-console.configuration', 'ignite-console.getting-started', - 'ignite-console.version', 'ignite-console.loading', // Ignite configuration module. 'ignite-console.config', @@ -192,7 +190,11 @@ angular .directive(...igniteUiAcePom) .directive(...igniteUiAceTabs) .directive(...igniteUiAceXml) +.directive(...igniteRetainSelection) // Services. +.service('IgniteErrorPopover', ErrorPopover) +.service('JavaTypes', JavaTypes) +.service('SqlTypes', SqlTypes) .service(...ChartColors) .service(...Clone) .service(...Confirm) @@ -201,11 +203,9 @@ angular .service(...Countries) .service(...Focus) .service(...InetAddress) -.service(...JavaTypes) .service(...Messages) .service(...ModelNormalizer) .service(...LegacyTable) -.service('IgniteErrorPopover', ErrorPopover) .service(...FormUtils) .service(...LegacyUtils) .service(...UnsavedChangesGuard) @@ -254,16 +254,17 @@ angular _.forEach(angular.element('.modal'), (m) => angular.element(m).scope().$hide()); }); }]) -.run(['$rootScope', '$http', '$state', 'IgniteMessages', 'User', - ($root, $http, $state, Messages, User) => { // eslint-disable-line no-shadow +.run(['$rootScope', '$http', '$state', 'IgniteMessages', 'User', 'IgniteNotebookData', + ($root, $http, $state, Messages, User, Notebook) => { // eslint-disable-line no-shadow $root.revertIdentity = () => { $http.get('/api/v1/admin/revert/identity') - .then(User.load) + .then(() => User.load()) .then((user) => { $root.$broadcast('user', user); $state.go('settings.admin'); }) + .then(() => Notebook.load()) .catch(Messages.showError); }; } diff --git a/modules/web-console/frontend/app/data/jdbc-types.json b/modules/web-console/frontend/app/data/jdbc-types.json new file mode 100644 index 0000000000000..07abbaf998016 --- /dev/null +++ b/modules/web-console/frontend/app/data/jdbc-types.json @@ -0,0 +1,44 @@ +[ + {"dbName": "BIT", "dbType": -7, "signed": {"javaType": "Boolean", "primitiveType": "boolean"}}, + {"dbName": "TINYINT", "dbType": -6, + "signed": {"javaType": "Byte", "primitiveType": "byte"}, + "unsigned": {"javaType": "Short", "primitiveType": "short"}}, + {"dbName": "SMALLINT", "dbType": 5, + "signed": {"javaType": "Short", "primitiveType": "short"}, + "unsigned": {"javaType": "Integer", "primitiveType": "int"}}, + {"dbName": "INTEGER", "dbType": 4, + "signed": {"javaType": "Integer", "primitiveType": "int"}, + "unsigned": {"javaType": "Long", "primitiveType": "long"}}, + {"dbName": "BIGINT", "dbType": -5, "signed": {"javaType": "Long", "primitiveType": "long"}}, + {"dbName": "FLOAT", "dbType": 6, "signed": {"javaType": "Float", "primitiveType": "float"}}, + {"dbName": "REAL", "dbType": 7, "signed": {"javaType": "Double", "primitiveType": "double"}}, + {"dbName": "DOUBLE", "dbType": 8, "signed": {"javaType": "Double", "primitiveType": "double"}}, + {"dbName": "NUMERIC", "dbType": 2, "signed": {"javaType": "BigDecimal"}}, + {"dbName": "DECIMAL", "dbType": 3, "signed": {"javaType": "BigDecimal"}}, + {"dbName": "CHAR", "dbType": 1, "signed": {"javaType": "String"}}, + {"dbName": "VARCHAR", "dbType": 12, "signed": {"javaType": "String"}}, + {"dbName": "LONGVARCHAR", "dbType": -1, "signed": {"javaType": "String"}}, + {"dbName": "DATE", "dbType": 91, "signed": {"javaType": "Date"}}, + {"dbName": "TIME", "dbType": 92, "signed": {"javaType": "Time"}}, + {"dbName": "TIMESTAMP", "dbType": 93, "signed": {"javaType": "Timestamp"}}, + {"dbName": "BINARY", "dbType": -2, "signed": {"javaType": "Object"}}, + {"dbName": "VARBINARY", "dbType": -3, "signed": {"javaType": "Object"}}, + {"dbName": "LONGVARBINARY", "dbType": -4, "signed": {"javaType": "Object"}}, + {"dbName": "NULL", "dbType": 0, "signed": {"javaType": "Object"}}, + {"dbName": "OTHER", "dbType": 1111, "signed": {"javaType": "Object"}}, + {"dbName": "JAVA_OBJECT", "dbType": 2000, "signed": {"javaType": "Object"}}, + {"dbName": "DISTINCT", "dbType": 2001, "signed": {"javaType": "Object"}}, + {"dbName": "STRUCT", "dbType": 2002, "signed": {"javaType": "Object"}}, + {"dbName": "ARRAY", "dbType": 2003, "signed": {"javaType": "Object"}}, + {"dbName": "BLOB", "dbType": 2004, "signed": {"javaType": "Object"}}, + {"dbName": "CLOB", "dbType": 2005, "signed": {"javaType": "String"}}, + {"dbName": "REF", "dbType": 2006, "signed": {"javaType": "Object"}}, + {"dbName": "DATALINK", "dbType": 70, "signed": {"javaType": "Object"}}, + {"dbName": "BOOLEAN", "dbType": 16, "signed": {"javaType": "Boolean", "primitiveType": "boolean"}}, + {"dbName": "ROWID", "dbType": -8, "signed": {"javaType": "Object"}}, + {"dbName": "NCHAR", "dbType": -15, "signed": {"javaType": "String"}}, + {"dbName": "NVARCHAR", "dbType": -9, "signed": {"javaType": "String"}}, + {"dbName": "LONGNVARCHAR", "dbType": -16, "signed": {"javaType": "String"}}, + {"dbName": "NCLOB", "dbType": 2011, "signed": {"javaType": "String"}}, + {"dbName": "SQLXML", "dbType": 2009, "signed": {"javaType": "Object"}} +] diff --git a/modules/web-console/frontend/app/data/sql-keywords.json b/modules/web-console/frontend/app/data/sql-keywords.json new file mode 100644 index 0000000000000..00f4eebaa302b --- /dev/null +++ b/modules/web-console/frontend/app/data/sql-keywords.json @@ -0,0 +1,41 @@ +[ + "CROSS", + "CURRENT_DATE", + "CURRENT_TIME", + "CURRENT_TIMESTAMP", + "DISTINCT", + "EXCEPT", + "EXISTS", + "FALSE", + "FETCH", + "FOR", + "FROM", + "FULL", + "GROUP", + "HAVING", + "INNER", + "INTERSECT", + "IS", + "JOIN", + "LIKE", + "LIMIT", + "MINUS", + "NATURAL", + "NOT", + "NULL", + "OFFSET", + "ON", + "ORDER", + "PRIMARY", + "ROWNUM", + "SELECT", + "SYSDATE", + "SYSTIMESTAMP", + "SYSTIME", + "TODAY", + "TRUE", + "UNION", + "UNIQUE", + "WHERE", + "WITH" +] diff --git a/modules/web-console/frontend/app/decorator/tooltip.js b/modules/web-console/frontend/app/decorator/tooltip.js index a47337a8253ef..221cf7c15a565 100644 --- a/modules/web-console/frontend/app/decorator/tooltip.js +++ b/modules/web-console/frontend/app/decorator/tooltip.js @@ -25,31 +25,45 @@ import angular from 'angular'; angular.module('mgcrea.ngStrap.tooltip') .decorator('$tooltip', ['$delegate', ($delegate) => { function TooltipFactoryDecorated(element, config) { - const delegate = $delegate(element, config); + let tipElementEntered = false; - const scope = delegate.$scope; + config.onShow = ($tooltip) => { + $tooltip.$element.on('mouseenter', () => tipElementEntered = true); + $tooltip.$element.on('mouseleave', () => { + tipElementEntered = false; - const options = delegate.$options; + $tooltip.leave(); + }); + }; + + const $tooltip = $delegate(element, config); - const hideWraped = delegate.hide; + const scope = $tooltip.$scope; + const options = $tooltip.$options; - delegate.hide = (blur) => { - if (!delegate.$isShown) + const _hide = $tooltip.hide; + + $tooltip.hide = (blur) => { + if (!$tooltip.$isShown || tipElementEntered) return; - if (delegate.$element !== null) - return hideWraped(blur); + if ($tooltip.$element) { + $tooltip.$element.off('mouseenter'); + $tooltip.$element.off('mouseleave'); + + return _hide(blur); + } - scope.$emit(options.prefixEvent + '.hide.before', delegate); + scope.$emit(options.prefixEvent + '.hide.before', $tooltip); if (angular.isDefined(options.onBeforeHide) && angular.isFunction(options.onBeforeHide)) - options.onBeforeHide(delegate); + options.onBeforeHide($tooltip); - delegate.$isShown = scope.$isShown = false; + $tooltip.$isShown = scope.$isShown = false; scope.$$phase || (scope.$root && scope.$root.$$phase) || scope.$digest(); }; - return delegate; + return $tooltip; } return TooltipFactoryDecorated; diff --git a/modules/web-console/frontend/app/directives/retain-selection.directive.js b/modules/web-console/frontend/app/directives/retain-selection.directive.js new file mode 100644 index 0000000000000..74d6872ef1ba7 --- /dev/null +++ b/modules/web-console/frontend/app/directives/retain-selection.directive.js @@ -0,0 +1,67 @@ +/* + * 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. + */ + +// Directive to workaround known issue with type ahead edit lost cursor position. +export default ['igniteRetainSelection', ['$timeout', ($timeout) => { + let promise; + + return function(scope, elem) { + elem.on('keydown', function(evt) { + const key = evt.which; + const ctrlDown = evt.ctrlKey || evt.metaKey; + const input = this; + let start = input.selectionStart; + + if (promise) + $timeout.cancel(promise); + + promise = $timeout(() => { + let setCursor = false; + + // Handle Backspace[8]. + if (key === 8 && start > 0) { + start -= 1; + + setCursor = true; + } + // Handle Del[46]. + else if (key === 46) + setCursor = true; + // Handle: Caps Lock[20], Tab[9], Shift[16], Ctrl[17], Alt[18], Esc[27], Enter[13], Arrows[37..40], Home[36], End[35], Ins[45], PgUp[33], PgDown[34], F1..F12[111..124], Num Lock[], Scroll Lock[145]. + else if (!(key === 8 || key === 9 || key === 13 || (key > 15 && key < 20) || key === 27 || + (key > 32 && key < 41) || key === 45 || (key > 111 && key < 124) || key === 144 || key === 145)) { + // Handle: Ctrl + [A[65], C[67], V[86]]. + if (!(ctrlDown && (key === 65 || key === 67 || key === 86))) { + start += 1; + + setCursor = true; + } + } + + if (setCursor) + input.setSelectionRange(start, start); + + promise = null; + }); + }); + + // Removes bound events in the element itself when the scope is destroyed + scope.$on('$destroy', function() { + elem.off('keydown'); + }); + }; +}]]; diff --git a/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js index ec880bd2354c2..82afeac245ad9 100644 --- a/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js +++ b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js @@ -15,7 +15,7 @@ * limitations under the License. */ -export default ['$scope', 'GeneratorPom', 'IgniteVersion', function($scope, pom, IgniteVersion) { +export default ['$scope', 'GeneratorPom', 'IgniteVersion', function($scope, pom, Version) { const ctrl = this; // Watchers definition. @@ -25,7 +25,7 @@ export default ['$scope', 'GeneratorPom', 'IgniteVersion', function($scope, pom, if (!value) return; - ctrl.data = pom.generate($scope.cluster, IgniteVersion.version).asString(); + ctrl.data = pom.generate($scope.cluster, Version.ignite).asString(); }; // Setup watchers. diff --git a/modules/web-console/frontend/app/helpers/jade/form.jade b/modules/web-console/frontend/app/helpers/jade/form.jade index 4b017ea46133a..c0c558d9cb925 100644 --- a/modules/web-console/frontend/app/helpers/jade/form.jade +++ b/modules/web-console/frontend/app/helpers/jade/form.jade @@ -17,6 +17,7 @@ include ./form/form-field-feedback.jade include ./form/form-field-label.jade include ./form/form-field-text.jade +include ./form/form-field-password.jade include ./form/form-field-dropdown.jade include ./form/form-field-datalist.jade include ./form/form-field-checkbox.jade diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade index 98a0b9a02d5e7..25e58058bc661 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade @@ -37,10 +37,10 @@ mixin form-field-datalist(label, model, name, disabled, required, placeholder, o data-ignite-form-panel-field='' )&attributes(attributes.attributes) - div - +ignite-form-field-label(label, name, required) - .col-xs-8.col-sm-8.col-md-8 - i.tipField.fa.fa-question-circle(bs-tooltip data-title=tip) + .ignite-form-field + +ignite-form-field__label(label, name, required) + .ignite-form-field__control + i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip) +form-field-feedback(name, 'required', errLbl + ' could not be empty!') diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade index a8496bc7698d8..f5d035db2e7ae 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade @@ -38,10 +38,10 @@ mixin ignite-form-field-dropdown(label, model, name, disabled, required, multipl data-ignite-form-panel-field='' )&attributes(attributes.attributes) - div - +ignite-form-field-label(label, name, required) - .col-xs-8.col-sm-8.col-md-8 - i.tipField.fa.fa-question-circle(bs-tooltip data-title=tip) + .ignite-form-field + +ignite-form-field__label(label, name, required) + .ignite-form-field__control + i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip) if block block diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-label.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-label.jade index cd5f8fca14e0e..d6aef8192cc4b 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-label.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-label.jade @@ -14,8 +14,8 @@ See the License for the specific language governing permissions and limitations under the License. -mixin ignite-form-field-label(label, name, required) - label.col-xs-4.col-sm-4.col-md-4( +mixin ignite-form-field__label(label, name, required) + label.ignite-form-field__label( id='{{ #{name} }}Label' for='{{ #{name} }}Input' class="{{ #{required} ? 'required' : '' }}" diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade index c32d3d9f02cb7..62b3e09b300c2 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade @@ -35,10 +35,10 @@ mixin ignite-form-field-number(label, model, name, disabled, required, placehold data-ignite-form-panel-field='' )&attributes(attributes.attributes) - div - +ignite-form-field-label(label, name, required) - .col-xs-8.col-sm-8.col-md-8 - i.tipField.fa.fa-question-circle(bs-tooltip data-title=tip) + .ignite-form-field + +ignite-form-field__label(label, name, required) + .ignite-form-field__control + i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip) +form-field-feedback(name, 'required', 'This field could not be empty') +form-field-feedback(name, 'min', 'Value is less than allowable minimum: '+ min || 0) diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-password.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-password.jade new file mode 100644 index 0000000000000..51cc1098fa01a --- /dev/null +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-password.jade @@ -0,0 +1,47 @@ +//- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +mixin ignite-form-field-password-input(name, model, disabled, required, placeholder) + input.form-control( + id='{{ #{name} }}Input' + name='{{ #{name} }}' + placeholder=placeholder + type='password' + + data-ng-model=model + + data-ng-required=required && '#{required}' + data-ng-disabled=disabled && '#{disabled}' + data-ng-focus='tableReset()' + + data-ignite-form-panel-field='' + )&attributes(attributes ? attributes.attributes ? attributes.attributes : attributes : {}) + +mixin ignite-form-field-password(label, model, name, disabled, required, placeholder, tip) + -var errLbl = label.substring(0, label.length - 1) + + .ignite-form-field + +ignite-form-field__label(label, name, required) + .ignite-form-field__control + i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip) + + if block + block + + +form-field-feedback(name, 'required', errLbl + ' could not be empty!') + + .input-tip + +ignite-form-field-password-input(name, model, disabled, required, placeholder)(attributes=attributes) diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade index 796e641de1966..55b850dfe1810 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade @@ -28,15 +28,15 @@ mixin ignite-form-field-input(name, model, disabled, required, placeholder) data-ng-focus='tableReset()' data-ignite-form-panel-field='' - )&attributes(attributes ? attributes.attributes ? attributes.attributes : attributes : {}) + )&attributes(attributes ? attributes.attributes ? attributes.attributes : attributes: {}) mixin ignite-form-field-text(label, model, name, disabled, required, placeholder, tip) -var errLbl = label.substring(0, label.length - 1) - div - +ignite-form-field-label(label, name, required) - .col-xs-8.col-sm-8.col-md-8 - i.tipField.fa.fa-question-circle(bs-tooltip data-title=tip) + .ignite-form-field + +ignite-form-field__label(label, name, required) + .ignite-form-field__control + i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip) if block block diff --git a/modules/web-console/frontend/app/helpers/jade/mixins.jade b/modules/web-console/frontend/app/helpers/jade/mixins.jade index c37ab157c28ce..4979c1234bc08 100644 --- a/modules/web-console/frontend/app/helpers/jade/mixins.jade +++ b/modules/web-console/frontend/app/helpers/jade/mixins.jade @@ -101,25 +101,20 @@ mixin uuid-feedback(name) mixin checkbox(lbl, model, name, tip) +form-field-checkbox(lbl, model, name, false, false, tip) +//- Function that convert enabled state to corresponding disabled state. +-var enabledToDisabled = function (enabled) { +- return (enabled === false || enabled === true) ? !enabled : '!(' + enabled + ')'; +-} + //- Mixin for checkbox with enabled condition. mixin checkbox-enabled(lbl, model, name, enabled, tip) - if enabled === false || enabled === true - -var disabled = !enabled - else - -var disabled = '!('+enabled+')' - - +form-field-checkbox(lbl, model, name, disabled, false, tip) + +form-field-checkbox(lbl, model, name, enabledToDisabled(enabled), false, tip) //- Mixin for java name field with enabled condition. mixin java-class(lbl, model, name, enabled, required, tip) -var errLbl = lbl.substring(0, lbl.length - 1) - if enabled === false || enabled === true - -var disabled = !enabled - else - -var disabled = '!('+enabled+')' - - +ignite-form-field-text(lbl, model, name, disabled, required, 'Enter fully qualified class name', tip)( + +ignite-form-field-text(lbl, model, name, enabledToDisabled(enabled), required, 'Enter fully qualified class name', tip)( data-java-identifier='true' data-java-package-specified='true' data-java-keywords='true' @@ -133,13 +128,11 @@ mixin java-class(lbl, model, name, enabled, required, tip) +form-field-feedback(name, 'javaPackageSpecified', errLbl + ' does not have package specified!') +form-field-feedback(name, 'javaIdentifier', errLbl + ' is invalid Java identifier!') - - //- Mixin for text field with enabled condition with options. mixin java-class-typeahead(lbl, model, name, options, enabled, required, placeholder, tip) -var errLbl = lbl.substring(0, lbl.length - 1) - +form-field-datalist(lbl, model, name, '!('+enabled+')', required, placeholder, options, tip)( + +form-field-datalist(lbl, model, name, enabledToDisabled(enabled), required, placeholder, options, tip)( data-java-identifier='true' data-java-package-specified='allow-built-in' data-java-keywords='true' @@ -148,25 +141,30 @@ mixin java-class-typeahead(lbl, model, name, options, enabled, required, placeho +form-field-feedback(name, 'javaPackageSpecified', errLbl + ' does not have package specified!') +form-field-feedback(name, 'javaIdentifier', errLbl + ' is invalid Java identifier!') +//- Mixin for java package field with enabled condition. +mixin java-package(lbl, model, name, enabled, required, tip) + +ignite-form-field-text(lbl, model, name, enabledToDisabled(enabled), required, 'Enter package name', tip)( + data-java-keywords='true' + data-java-package-name='package-only' + ) + +form-field-feedback(name, 'javaPackageName', 'Package name is invalid') + +form-field-feedback(name, 'javaKeywords', 'Package name could not contains reserved java keyword') + //- Mixin for text field with IP address check. mixin text-ip-address(lbl, model, name, enabled, placeholder, tip) - +ignite-form-field-text(lbl, model, name, '!('+enabled+')', false, placeholder, tip)(data-ipaddress='true') + +ignite-form-field-text(lbl, model, name, enabledToDisabled(enabled), false, placeholder, tip)(data-ipaddress='true') +ipaddress-feedback(name) -//- Mixin for text field with IP address and port check. -mixin text-ip-address-with-port(lbl, model, name, enabled, placeholder, tip) - +ignite-form-field-text(lbl, model, name, '!('+enabled+')', false, placeholder, tip)(data-ipaddress='true' data-ipaddress-with-port='true') +//- Mixin for text field with IP address and port range check. +mixin text-ip-address-with-port-range(lbl, model, name, enabled, placeholder, tip) + +ignite-form-field-text(lbl, model, name, enabledToDisabled(enabled), false, placeholder, tip)(data-ipaddress='true' data-ipaddress-with-port='true' data-ipaddress-with-port-range='true') +ipaddress-feedback(name) +ipaddress-port-feedback(name) + +ipaddress-port-range-feedback(name) //- Mixin for text field. mixin text-enabled(lbl, model, name, enabled, required, placeholder, tip) - if enabled === false || enabled === true - -var disabled = !enabled - else - -var disabled = '!('+enabled+')' - - +ignite-form-field-text(lbl, model, name, disabled, required, placeholder, tip) + +ignite-form-field-text(lbl, model, name, enabledToDisabled(enabled), required, placeholder, tip) if block block @@ -176,67 +174,53 @@ mixin text(lbl, model, name, required, placeholder, tip) if block block +//- Mixin for password field. +mixin password(lbl, model, name, required, placeholder, tip) + +ignite-form-field-password(lbl, model, name, false, required, placeholder, tip) + if block + block + //- Mixin for text field with enabled condition with options. mixin text-options(lbl, model, name, options, enabled, required, placeholder, tip) - +form-field-datalist(lbl, model, name, '!('+enabled+')', required, placeholder, options, tip) + +form-field-datalist(lbl, model, name, enabledToDisabled(enabled), required, placeholder, options, tip) //- Mixin for required numeric field. mixin number-required(lbl, model, name, enabled, required, placeholder, min, tip) - +ignite-form-field-number(lbl, model, name, '!('+enabled+')', required, placeholder, min, false, false, tip) + +ignite-form-field-number(lbl, model, name, enabledToDisabled(enabled), required, placeholder, min, false, false, tip) //- Mixin for required numeric field with maximum and minimum limit. mixin number-min-max(lbl, model, name, enabled, placeholder, min, max, tip) - +ignite-form-field-number(lbl, model, name, '!('+enabled+')', false, placeholder, min, max, '1', tip) + +ignite-form-field-number(lbl, model, name, enabledToDisabled(enabled), false, placeholder, min, max, '1', tip) //- Mixin for required numeric field with maximum and minimum limit. mixin number-min-max-step(lbl, model, name, enabled, placeholder, min, max, step, tip) - +ignite-form-field-number(lbl, model, name, '!('+enabled+')', false, placeholder, min, max, step, tip) + +ignite-form-field-number(lbl, model, name, enabledToDisabled(enabled), false, placeholder, min, max, step, tip) //- Mixin for numeric field. mixin number(lbl, model, name, enabled, placeholder, min, tip) - +ignite-form-field-number(lbl, model, name, '!('+enabled+')', false, placeholder, min, false, false, tip) + +ignite-form-field-number(lbl, model, name, enabledToDisabled(enabled), false, placeholder, min, false, false, tip) //- Mixin for required dropdown field. mixin dropdown-required-empty(lbl, model, name, enabled, required, placeholder, placeholderEmpty, options, tip) - if enabled === false || enabled === true - -var disabled = !enabled - else - -var disabled = '!('+enabled+')' - - +ignite-form-field-dropdown(lbl, model, name, disabled, required, false, placeholder, placeholderEmpty, options, tip) + +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), required, false, placeholder, placeholderEmpty, options, tip) if block block - + //- Mixin for required dropdown field. mixin dropdown-required(lbl, model, name, enabled, required, placeholder, options, tip) - if enabled === false || enabled === true - -var disabled = !enabled - else - -var disabled = '!('+enabled+')' - - +ignite-form-field-dropdown(lbl, model, name, disabled, required, false, placeholder, '', options, tip) + +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), required, false, placeholder, '', options, tip) if block block //- Mixin for dropdown field. mixin dropdown(lbl, model, name, enabled, placeholder, options, tip) - if enabled === false || enabled === true - -var disabled = !enabled - else - -var disabled = '!('+enabled+')' - - +ignite-form-field-dropdown(lbl, model, name, disabled, false, false, placeholder, '', options, tip) + +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), false, false, placeholder, '', options, tip) if block block //- Mixin for dropdown-multiple field. mixin dropdown-multiple(lbl, model, name, enabled, placeholder, placeholderEmpty, options, tip) - if enabled === false || enabled === true - -var disabled = !enabled - else - -var disabled = '!('+enabled+')' - - +ignite-form-field-dropdown(lbl, model, name, disabled, false, true, placeholder, placeholderEmpty, options, tip) + +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), false, true, placeholder, placeholderEmpty, options, tip) if block block @@ -319,7 +303,7 @@ mixin table-java-package-field(name, model, items, valid, save, newItem) .input-tip +ignite-form-field-input(name, model, false, 'true', 'Enter package name')( data-java-keywords='true' - data-java-package-name='true' + data-java-package-name='package-only' data-ignite-unique=items data-ignite-form-field-input-autofocus='true' @@ -360,7 +344,7 @@ mixin table-address-field(name, model, items, valid, save, newItem, portRange) ) //- Mixin for table UUID field. -mixin table-uuid-field(name, model, items, valid, save, newItem, portRange) +mixin table-uuid-field(name, model, items, valid, save, newItem) -var resetOnEnter = newItem ? '(stopblur = true) && (group.add = [{}])' : '(field.edit = false)' -var onEnter = valid + ' && (' + save + '); ' + valid + ' && ' + resetOnEnter + ';' @@ -506,7 +490,6 @@ mixin table-pair-edit(tbl, prefix, keyPlaceholder, valPlaceholder, keyJavaBuiltI else input.form-control(id=keyFocusId ignite-on-enter-focus-move=valFocusId type='text' ng-model=keyModel placeholder=keyPlaceholder ignite-on-escape='tableReset()') .col-xs-6.col-sm-6.col-md-6 - -var arg = keyModel + ', ' + valModel -var btnVisible = 'tablePairSaveVisible(' + tbl + ', ' + index + ')' -var btnSave = 'tablePairSave(tablePairValid, backupItem, ' + tbl + ', ' + index + ')' -var btnVisibleAndSave = btnVisible + ' && ' + btnSave @@ -529,7 +512,7 @@ mixin dialect(lbl, model, name, required, tipTitle, genericDialectName, placehol {value: "PostgreSQL", label: "PostgreSQL"},\ {value: "H2", label: "H2 database"}\ ]', - tipTitle + + tipTitle + '
            \
          • ' + genericDialectName + '
          • \
          • Oracle database
          • \ diff --git a/modules/web-console/frontend/app/modules/configuration/Version.service.js b/modules/web-console/frontend/app/modules/configuration/Version.service.js new file mode 100644 index 0000000000000..c36a98b921e5d --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/Version.service.js @@ -0,0 +1,25 @@ +/* + * 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. + */ + +/** + * Utility service for version. + */ +export default () => { + return { + ignite: '1.7.0' + }; +}; diff --git a/modules/web-console/frontend/app/modules/configuration/configuration.module.js b/modules/web-console/frontend/app/modules/configuration/configuration.module.js index 99830b0cabdd6..dc6fa2fa31a95 100644 --- a/modules/web-console/frontend/app/modules/configuration/configuration.module.js +++ b/modules/web-console/frontend/app/modules/configuration/configuration.module.js @@ -19,6 +19,7 @@ import angular from 'angular'; import igniteEventGroups from './EventGroups.provider'; import igniteSidebar from './Sidebar.provider'; +import Version from './Version.service'; import GeneratorXml from './generator/Xml.service'; import GeneratorJava from './generator/Java.service'; @@ -35,6 +36,7 @@ angular .provider(...igniteEventGroups) .provider(...igniteSidebar) .directive(...igniteSidebarDirective) +.service('IgniteVersion', Version) .service(...GeneratorXml) .service(...GeneratorJava) .service(...GeneratorDocker) diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js index 627a1e34d132f..f2d04b08b2719 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js @@ -52,10 +52,10 @@ class GeneratorPom { res.endBlock(''); } - artifact(res, cluster, igniteVersion) { + artifact(res, cluster, version) { this.addProperty(res, 'groupId', 'org.apache.ignite'); this.addProperty(res, 'artifactId', this.escapeId(cluster.name) + '-project'); - this.addProperty(res, 'version', igniteVersion); + this.addProperty(res, 'version', version); res.needEmptyLine = true; } @@ -143,11 +143,11 @@ class GeneratorPom { * Generate pom.xml. * * @param cluster Cluster to take info about dependencies. - * @param igniteVersion Ignite version for Ignite dependencies. + * @param version Ignite version for Ignite dependencies. * @param res Resulting output with generated pom. * @returns {string} Generated content. */ - generate(cluster, igniteVersion, res) { + generate(cluster, version, res) { const caches = cluster.caches; const deps = []; const storeDeps = []; @@ -163,14 +163,14 @@ class GeneratorPom { this.storeFactoryDependency(storeDeps, cache.cacheStoreFactory[cache.cacheStoreFactory.kind]); if (_.get(cache, 'nodeFilter.kind') === 'Exclude') - this.addDependency(deps, 'org.apache.ignite', 'ignite-extdata-p2p', igniteVersion); + this.addDependency(deps, 'org.apache.ignite', 'ignite-extdata-p2p', version); }); res.line(''); res.needEmptyLine = true; - res.line(''); + res.line(''); res.needEmptyLine = true; @@ -180,18 +180,18 @@ class GeneratorPom { res.needEmptyLine = true; - this.artifact(res, cluster, igniteVersion); + this.artifact(res, cluster, version); - this.addDependency(deps, 'org.apache.ignite', 'ignite-core', igniteVersion); + this.addDependency(deps, 'org.apache.ignite', 'ignite-core', version); - this.addDependency(deps, 'org.apache.ignite', 'ignite-spring', igniteVersion); - this.addDependency(deps, 'org.apache.ignite', 'ignite-indexing', igniteVersion); - this.addDependency(deps, 'org.apache.ignite', 'ignite-rest-http', igniteVersion); + this.addDependency(deps, 'org.apache.ignite', 'ignite-spring', version); + this.addDependency(deps, 'org.apache.ignite', 'ignite-indexing', version); + this.addDependency(deps, 'org.apache.ignite', 'ignite-rest-http', version); let dep = POM_DEPENDENCIES[cluster.discovery.kind]; if (dep) - this.addDependency(deps, 'org.apache.ignite', dep.artifactId, igniteVersion); + this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version); if (cluster.discovery.kind === 'Jdbc') { const store = cluster.discovery.Jdbc; @@ -201,16 +201,16 @@ class GeneratorPom { } if (_.find(cluster.igfss, (igfs) => igfs.secondaryFileSystemEnabled)) - this.addDependency(deps, 'org.apache.ignite', 'ignite-hadoop', igniteVersion); + this.addDependency(deps, 'org.apache.ignite', 'ignite-hadoop', version); if (_.find(caches, blobStoreFactory)) - this.addDependency(deps, 'org.apache.ignite', 'ignite-hibernate', igniteVersion); + this.addDependency(deps, 'org.apache.ignite', 'ignite-hibernate', version); if (cluster.logger && cluster.logger.kind) { dep = POM_DEPENDENCIES[cluster.logger.kind]; if (dep) - this.addDependency(deps, 'org.apache.ignite', dep.artifactId, igniteVersion); + this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version); } this.dependencies(res, cluster, deps.concat(storeDeps)); diff --git a/modules/web-console/frontend/app/modules/form/validator/java-identifier.directive.js b/modules/web-console/frontend/app/modules/form/validator/java-identifier.directive.js index 5cbf7fbe6dcc9..2abe1df0db33d 100644 --- a/modules/web-console/frontend/app/modules/form/validator/java-identifier.directive.js +++ b/modules/web-console/frontend/app/modules/form/validator/java-identifier.directive.js @@ -17,10 +17,10 @@ export default ['javaIdentifier', ['JavaTypes', (JavaTypes) => { const link = (scope, el, attrs, [ngModel]) => { - if (_.isUndefined(attrs.javaIdentifier) || !attrs.javaIdentifier) + if (_.isNil(attrs.javaIdentifier) || attrs.javaIdentifier !== 'true') return; - ngModel.$validators.javaIdentifier = (value) => JavaTypes.validIdentifier(value); + ngModel.$validators.javaIdentifier = (value) => _.isEmpty(value) || JavaTypes.validClassName(value); }; return { diff --git a/modules/web-console/frontend/app/modules/form/validator/java-keywords.directive.js b/modules/web-console/frontend/app/modules/form/validator/java-keywords.directive.js index d97e59a62ff42..c9eef414e37f7 100644 --- a/modules/web-console/frontend/app/modules/form/validator/java-keywords.directive.js +++ b/modules/web-console/frontend/app/modules/form/validator/java-keywords.directive.js @@ -17,17 +17,17 @@ export default ['javaKeywords', ['JavaTypes', (JavaTypes) => { const link = (scope, el, attrs, [ngModel]) => { - if (_.isUndefined(attrs.javaKeywords) || !attrs.javaKeywords) + if (_.isNil(attrs.javaKeywords) || attrs.javaKeywords === 'false') return; const packageOnly = attrs.javaPackageName === 'package-only'; ngModel.$validators.javaKeywords = (value) => { if (value) { - if (!JavaTypes.validIdentifier(value) || (!packageOnly && !JavaTypes.packageSpecified(value))) + if (_.isEmpty(value) || !JavaTypes.validClassName(value) || (!packageOnly && !JavaTypes.packageSpecified(value))) return true; - return _.findIndex(value.split('.'), JavaTypes.isKeywords) < 0; + return _.findIndex(value.split('.'), JavaTypes.isKeyword) < 0; } return true; diff --git a/modules/web-console/frontend/app/modules/form/validator/java-package-name.directive.js b/modules/web-console/frontend/app/modules/form/validator/java-package-name.directive.js index ac38179aaddb1..49d4b551cdd14 100644 --- a/modules/web-console/frontend/app/modules/form/validator/java-package-name.directive.js +++ b/modules/web-console/frontend/app/modules/form/validator/java-package-name.directive.js @@ -17,10 +17,10 @@ export default ['javaPackageName', ['JavaTypes', (JavaTypes) => { const link = (scope, el, attrs, [ngModel]) => { - if (_.isUndefined(attrs.javaPackageName) || !attrs.javaPackageName) + if (_.isNil(attrs.javaPackageName) || attrs.javaPackageName === 'false') return; - ngModel.$validators.javaPackageName = (value) => JavaTypes.validPackage(value); + ngModel.$validators.javaPackageName = (value) => _.isEmpty(value) || JavaTypes.validPackage(value); }; return { diff --git a/modules/web-console/frontend/app/modules/form/validator/java-package-specified.directive.js b/modules/web-console/frontend/app/modules/form/validator/java-package-specified.directive.js index 451d7ec6c65ef..5f24eb7eddf7b 100644 --- a/modules/web-console/frontend/app/modules/form/validator/java-package-specified.directive.js +++ b/modules/web-console/frontend/app/modules/form/validator/java-package-specified.directive.js @@ -17,13 +17,14 @@ export default ['javaPackageSpecified', ['JavaTypes', (JavaTypes) => { const link = (scope, el, attrs, [ngModel]) => { - if (_.isUndefined(attrs.javaPackageSpecified)) + if (_.isNil(attrs.javaPackageSpecified) || attrs.javaPackageSpecified === 'false') return; const allowBuiltIn = attrs.javaPackageSpecified === 'allow-built-in'; - ngModel.$validators.javaPackageSpecified = (value) => !value || !JavaTypes.validIdentifier(value) || JavaTypes.packageSpecified(value) || - (allowBuiltIn && !JavaTypes.nonBuiltInClass(value)); + ngModel.$validators.javaPackageSpecified = (value) => _.isEmpty(value) || + !JavaTypes.validClassName(value) || JavaTypes.packageSpecified(value) || + (allowBuiltIn && !JavaTypes.nonBuiltInClass(value)); }; return { diff --git a/modules/web-console/frontend/app/modules/form/validator/uuid.directive.js b/modules/web-console/frontend/app/modules/form/validator/uuid.directive.js index 070417563bc91..94c6e3f7aa38d 100644 --- a/modules/web-console/frontend/app/modules/form/validator/uuid.directive.js +++ b/modules/web-console/frontend/app/modules/form/validator/uuid.directive.js @@ -17,16 +17,10 @@ export default ['uuid', ['JavaTypes', (JavaTypes) => { const link = (scope, el, attrs, [ngModel]) => { - const isEmpty = (modelValue) => { - return ngModel.$isEmpty(modelValue) || _.isUndefined(attrs.uuid) || attrs.uuid !== 'true'; - }; + if (_.isNil(attrs.uuid) || attrs.uuid !== 'true') + return; - ngModel.$validators.uuid = (modelValue) => { - if (isEmpty(modelValue)) - return true; - - return JavaTypes.validUUID(modelValue); - }; + ngModel.$validators.uuid = (modelValue) => _.isEmpty(modelValue) || JavaTypes.validUUID(modelValue); }; return { diff --git a/modules/web-console/frontend/app/modules/sql/Notebook.data.js b/modules/web-console/frontend/app/modules/sql/Notebook.data.js index aef72ebce508f..f66faba07cd8f 100644 --- a/modules/web-console/frontend/app/modules/sql/Notebook.data.js +++ b/modules/web-console/frontend/app/modules/sql/Notebook.data.js @@ -83,18 +83,26 @@ export default class NotebookData { this.$q = $q; } - read() { - if (!_.isNil(this.initLatch)) - return this.initLatch; + load() { + if (this.demo) { + if (this.initLatch) + return this.initLatch; - if (this.demo) return this.initLatch = this.$q.when(this.notebooks = [DEMO_NOTEBOOK]); + } return this.initLatch = this.$http.get('/api/v1/notebooks') .then(({data}) => this.notebooks = data) .catch(({data}) => Promise.reject(data)); } + read() { + if (this.initLatch) + return this.initLatch; + + return this.load(); + } + find(_id) { return this.read() .then(() => { diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade index debbe0d794b22..e9f29fd5007ac 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade @@ -78,7 +78,7 @@ include ../../../../../app/helpers/jade/mixins.jade .settings-row(data-ng-if=model + '.offHeapMode === 1 && ' + model + '.memoryMode !== "OFFHEAP_VALUES"') +number-required('Off-heap memory max size:', model + '.offHeapMaxMemory', '"offHeapMaxMemory"', 'true', model + '.offHeapMode === 1', '', 1, - 'Sets maximum amount of memory available to off-heap storage in bytes') + 'Maximum amount of memory available to off-heap storage in bytes') .settings-row -var onHeapTired = model + '.memoryMode === "ONHEAP_TIERED"' -var swapEnabled = model + '.swapEnabled' @@ -94,8 +94,7 @@ include ../../../../../app/helpers/jade/mixins.jade
          • SORTED - Eviction policy which will select the minimum cache entry for eviction
          • \
          ') .settings-row - +number('Start size:', model + '.startSize', '"startSize"', 'true', '1500000', '0', - 'Initial cache size which will be used to pre-create internal hash table after start') + +number('Start size:', model + '.startSize', '"startSize"', 'true', '1500000', '0', 'Initial cache size in entries number') .settings-row +checkbox('Swap enabled', model + '.swapEnabled', '"swapEnabled"', 'Flag indicating whether swap storage is enabled or not for this cache') .col-sm-6 diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade index c83114b6dbf38..c709135672559 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade @@ -30,7 +30,15 @@ include ../../../../../app/helpers/jade/mixins.jade .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - +text('SQL schema name:', model + '.sqlSchema', '"sqlSchema"', 'false', 'Input schema name', 'Schema name for cache according to SQL ANSI-99') + +text('SQL schema name:', model + '.sqlSchema', '"sqlSchema"', 'false', 'Input schema name', + 'Schema name allow to use existing database queries in your application and according to SQL ANSI-99
          \ + Cache is reffered by schema name in cross-cache queries
          \ + Nonquoted identifiers are not case sensitive. Quoted identifiers are case sensitive
          \ + When SQL schema is not specified, quoted cache name should be used
          \ + Query example without schema name:
          \ + SELECT .... FROM "cache1".Type1 JOIN "cache2".Type2 ...
          \ + The same query using schema name:
          \ + SELECT .... FROM cache1.Type1 JOIN cache2.Type2 ...') .settings-row +number('On-heap cache for off-heap indexes:', model + '.sqlOnheapRowCacheSize', '"sqlOnheapRowCacheSize"', 'true', '10240', '1', 'Number of SQL rows which will be cached onheap to avoid deserialization on each SQL index access') @@ -89,7 +97,7 @@ include ../../../../../app/helpers/jade/mixins.jade 'Flag indicating whether SQL indexes should support snapshots') .settings-row +checkbox('Escape table and filed names', model + '.sqlEscapeAll', '"sqlEscapeAll"', - 'If set then all the SQL table and field names will be escaped with double quotes
          \ + 'If enabled than all the SQL table and field names will be escaped with double quotes like "tableName"."fieldName"
          \ This enforces case sensitivity for field names and also allows having special characters in table and field names') .col-sm-6 +preview-xml-java(model, 'cacheQuery', 'domains') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade index a078bf149bf1a..3531c779a69be 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade @@ -25,7 +25,7 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label Cache key configuration ignite-form-field-tooltip.tipLabel - | Cache key configuration allows to collocate objects in a partitioned cache based on field in cache key without explicit usage of annotations on user classes. + | Cache key configuration allows to collocate objects in a partitioned cache based on field in cache key without explicit usage of annotations on user classes ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade index 0643555c3a1bd..2c60478f45ca1 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade @@ -90,7 +90,7 @@ include ../../../../../app/helpers/jade/mixins.jade .settings-row +checkbox('Direct buffer', communication + '.directBuffer', '"directBuffer"', 'If value is true, then SPI will use ByteBuffer.allocateDirect(int) call
          \ - Otherwise, SPI will use ByteBuffer.allocate(int) call.') + Otherwise, SPI will use ByteBuffer.allocate(int) call') .settings-row +checkbox('Direct send buffer', communication + '.directSendBuffer', '"directSendBuffer"', 'Flag defining whether direct send buffer should be used') .settings-row diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade new file mode 100644 index 0000000000000..c267891c69ad3 --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade @@ -0,0 +1,47 @@ +//- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +include ../../../../../app/helpers/jade/mixins.jade + +-var form = 'odbcConfiguration' +-var model = 'backupItem.odbc' +-var enabled = model + '.odbcEnabled' + +.panel.panel-default(ng-form=form novalidate) + .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') + ignite-form-panel-chevron + label ODBC configuration + ignite-form-field-tooltip.tipLabel + | ODBC server configuration + ignite-form-revert + .panel-collapse(role='tabpanel' bs-collapse-target id=form) + .panel-body(ng-if='ui.isPanelLoaded("#{form}")') + .col-sm-6 + .settings-row + +checkbox('Enabled', enabled, '"odbcEnabled"', 'Flag indicating whether to configure ODBC configuration') + .settings-row + +text-ip-address-with-port-range('ODBC endpoint address:', model + '.endpointAddress', '"endpointAddress"', enabled, '0.0.0.0:10800..10810', + 'ODBC endpoint address.
          \ + The following address formats are permitted:\ +
            \ +
          • hostname - will use provided hostname and default port range
          • \ +
          • hostname:port - will use provided hostname and port
          • \ +
          • hostname:port_from..port_to - will use provided hostname and port range
          • \ +
          ') + .settings-row + +number('Maximum open cursors', model + '.maxOpenCursors', '"maxOpenCursors"', enabled, '128', '1', 'Maximum number of opened cursors per connection') + .col-sm-6 + +preview-xml-java(model, 'clusterODBC') diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade b/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade index 5c55e0cfcac23..74895f5a7c443 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade @@ -16,8 +16,8 @@ include ../../../../../app/helpers/jade/mixins.jade -- var form = 'general' -- var model = 'backupItem' +-var form = 'general' +-var model = 'backupItem' .panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle) diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade b/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade index 33c358a28024e..0cf21f5130beb 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade @@ -16,14 +16,14 @@ include ../../../../../app/helpers/jade/mixins.jade -- var form = 'query' -- var model = 'backupItem' -- var queryFields = model + '.fields' -- var queryAliases = model + '.aliases' -- var queryIndexes = model + '.indexes' -- var queryFieldsForm = 'queryFields' -- var queryAliasesForm = 'queryAliases' -- var queryIndexesForm = 'queryIndexes' +-var form = 'query' +-var model = 'backupItem' +-var queryFields = model + '.fields' +-var queryAliases = model + '.aliases' +-var queryIndexes = model + '.indexes' +-var queryFieldsForm = 'queryFields' +-var queryAliasesForm = 'queryAliases' +-var queryIndexesForm = 'queryIndexes' // LEGACY mixin for LEGACY index fields table. mixin table-index-item-edit(prefix, index, sortAvailable, idAddition) diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js index f0cb842c8120c..4aad7e24f380f 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js +++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js @@ -21,7 +21,7 @@ import saver from 'file-saver'; export default [ '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteLoading', '$filter', 'igniteConfigurationResource', 'JavaTypes', 'IgniteVersion', 'GeneratorDocker', 'GeneratorPom', 'IgniteFormUtils', - function($root, $scope, $http, LegacyUtils, Messages, Loading, $filter, Resource, JavaTypes, IgniteVersion, docker, pom, FormUtils) { + function($root, $scope, $http, LegacyUtils, Messages, Loading, $filter, Resource, JavaTypes, Version, docker, pom, FormUtils) { const ctrl = this; $scope.ui = { ready: false }; @@ -81,6 +81,14 @@ export default [ ] }; + const loadFolder = { + type: 'folder', + name: 'load', + children: [ + { type: 'file', name: 'LoadCaches.java' } + ] + }; + const javaStartupFolder = { type: 'folder', name: 'startup', @@ -224,7 +232,11 @@ export default [ sessionStorage.summarySelectedId = $scope.clusters.indexOf(cluster); mainFolder.children = [javaFolder]; - javaFolder.children = [javaConfigFolder, javaStartupFolder]; + + if (_.find(cluster.caches, (cache) => !_.isNil(cache.cacheStoreFactory))) + javaFolder.children = [javaConfigFolder, loadFolder, javaStartupFolder]; + else + javaFolder.children = [javaConfigFolder, javaStartupFolder]; if ($generatorCommon.secretPropertiesNeeded(cluster)) mainFolder.children.push(resourcesFolder); @@ -308,6 +320,12 @@ export default [ 'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory')); } + // Generate loader for caches with configured store. + const cachesToLoad = _.filter(cluster.caches, (cache) => !_.isNil(cache.cacheStoreFactory)); + + if (!_.isEmpty(cachesToLoad)) + zip.file(srcPath + 'load/LoadCaches.java', $generatorJava.loadCaches(cachesToLoad, 'load', 'LoadCaches', '"' + clientXml + '"')); + zip.file(srcPath + 'startup/ServerNodeSpringStartup.java', $generatorJava.nodeStartup(cluster, 'startup', 'ServerNodeSpringStartup', '"' + serverXml + '"')); zip.file(srcPath + 'startup/ClientNodeSpringStartup.java', $generatorJava.nodeStartup(cluster, 'startup', 'ClientNodeSpringStartup', '"' + clientXml + '"')); @@ -316,7 +334,7 @@ export default [ zip.file(srcPath + 'startup/ClientNodeCodeStartup.java', $generatorJava.nodeStartup(cluster, 'startup', 'ClientNodeCodeStartup', 'ClientConfigurationFactory.createConfiguration()', 'config.ClientConfigurationFactory', clientNearCfg)); - zip.file('pom.xml', pom.generate(cluster, IgniteVersion.version).asString()); + zip.file('pom.xml', pom.generate(cluster, Version.ignite).asString()); zip.file('README.txt', $generatorReadme.readme().asString()); zip.file('jdbc-drivers/README.txt', $generatorReadme.readmeJdbc().asString()); diff --git a/modules/web-console/frontend/app/services/ErrorPopover.service.js b/modules/web-console/frontend/app/services/ErrorPopover.service.js index 85e4fdab3d820..313043137a055 100644 --- a/modules/web-console/frontend/app/services/ErrorPopover.service.js +++ b/modules/web-console/frontend/app/services/ErrorPopover.service.js @@ -15,6 +15,9 @@ * limitations under the License. */ +/** + * Service to show/hide error popover. + */ export default class ErrorPopover { static $inject = ['$popover', '$anchorScroll', '$location', '$timeout', 'IgniteFormUtils']; diff --git a/modules/web-console/frontend/app/services/JavaTypes.service.js b/modules/web-console/frontend/app/services/JavaTypes.service.js index e8d49030f8355..8cb87be6b9f9f 100644 --- a/modules/web-console/frontend/app/services/JavaTypes.service.js +++ b/modules/web-console/frontend/app/services/JavaTypes.service.js @@ -15,79 +15,104 @@ * limitations under the License. */ +import _ from 'lodash'; + // Java built-in class names. import JAVA_CLASSES from '../data/java-classes.json'; -// Java build-in primitive. +// Java build-in primitives. import JAVA_PRIMITIVES from '../data/java-primitives.json'; +// Java keywords. import JAVA_KEYWORDS from '../data/java-keywords.json'; -export default ['JavaTypes', function() { - return { - /** - * @param {String} clsName Class name to check. - * @returns boolean 'true' if given class name non a Java built-in type. - */ - nonBuiltInClass(clsName) { - return _.isNil(_.find(JAVA_CLASSES, (clazz) => clsName === clazz.short || clsName === clazz.full)); - }, - /** - * @param clsName Class name to check. - * @returns Full class name for java build-in types or source class otherwise. - */ - fullClassName(clsName) { - const type = _.find(JAVA_CLASSES, (clazz) => clsName === clazz.short); - - return type ? type.full : clsName; - }, - /** - * @param {String} value text to check. - * @returns boolean 'true' if given text is valid Java identifier. - */ - validIdentifier(value) { - const regexp = /^(([a-zA-Z_$][a-zA-Z0-9_$]*)\.)*([a-zA-Z_$][a-zA-Z0-9_$]*)$/igm; - - return value === '' || regexp.test(value); - }, - /** - * @param {String} value text to check. - * @returns boolean 'true' if given text is valid Java package. - */ - validPackage(value) { - const regexp = /^(([a-zA-Z_$][a-zA-Z0-9_$]*)\.)*([a-zA-Z_$][a-zA-Z0-9_$]*(\.?\*)?)$/igm; - - return value === '' || regexp.test(value); - }, - /** - * @param {String} value text to check. - * @returns boolean 'true' if given text is valid Java UUID value. - */ - validUUID(value) { - const regexp = /^[0-9a-f]{8}-[0-9a-f]{4}-[1-5][0-9a-f]{3}-[89ab][0-9a-f]{3}-[0-9a-f]{12}$/igm; - - return value === '' || regexp.test(value); - }, - /** - * @param {String} value text to check. - * @returns boolean 'true' if given text is a Java type with package. - */ - packageSpecified(value) { - return value.split('.').length >= 2; - }, - /** - * @param {String} value text to check. - * @returns boolean 'true' if given text non Java keyword. - */ - isKeywords(value) { - return _.includes(JAVA_KEYWORDS, value); - }, - /** - * @param {String} clsName Class name to check. - * @returns {boolean} 'true' if givent class name is java primitive. - */ - isJavaPrimitive(clsName) { - return _.includes(JAVA_PRIMITIVES, clsName); - } - }; -}]; +// Regular expression to check Java identifier. +const VALID_IDENTIFIER = /^[a-zA-Z_$][a-zA-Z0-9_$]*$/im; + +// Regular expression to check Java class name. +const VALID_CLASS_NAME = /^(([a-zA-Z_$][a-zA-Z0-9_$]*)\.)*([a-zA-Z_$][a-zA-Z0-9_$]*)$/im; + +// Regular expression to check Java package. +const VALID_PACKAGE = /^(([a-zA-Z_$][a-zA-Z0-9_$]*)\.)*([a-zA-Z_$][a-zA-Z0-9_$]*(\.?\*)?)$/im; + +// Regular expression to check UUID string representation. +const VALID_UUID = /^[0-9a-f]{8}-[0-9a-f]{4}-[1-5][0-9a-f]{3}-[89ab][0-9a-f]{3}-[0-9a-f]{12}$/im; + +/** + * Utility service for various check on java types. + */ +export default class JavaTypes { + /** + * @param clsName {String} Class name to check. + * @returns {boolean} 'true' if provided class name is a not Java built in class. + */ + nonBuiltInClass(clsName) { + return _.isNil(_.find(JAVA_CLASSES, (clazz) => clsName === clazz.short || clsName === clazz.full)); + } + + /** + * @param clsName Class name to check. + * @returns Full class name for java build-in types or source class otherwise. + */ + fullClassName(clsName) { + const type = _.find(JAVA_CLASSES, (clazz) => clsName === clazz.short); + + return type ? type.full : clsName; + } + + /** + * @param value {String} Value text to check. + * @returns {boolean} 'true' if given text is valid Java class name. + */ + validIdentifier(value) { + return !!(value && VALID_IDENTIFIER.test(value)); + } + + /** + * @param value {String} Value text to check. + * @returns {boolean} 'true' if given text is valid Java class name. + */ + validClassName(value) { + return !!(value && VALID_CLASS_NAME.test(value)); + } + + /** + * @param value {String} Value text to check. + * @returns {boolean} 'true' if given text is valid Java package. + */ + validPackage(value) { + return !!(value && VALID_PACKAGE.test(value)); + } + + /** + * @param value {String} Value text to check. + * @returns {boolean} 'true' if given text is valid Java UUID value. + */ + validUUID(value) { + return !!(value && VALID_UUID.test(value)); + } + + /** + * @param value {String} Value text to check. + * @returns {boolean} 'true' if given text is a Java type with package. + */ + packageSpecified(value) { + return value.split('.').length >= 2; + } + + /** + * @param value {String} Value text to check. + * @returns {boolean} 'true' if given value is one of Java reserved keywords. + */ + isKeyword(value) { + return !!(value && _.includes(JAVA_KEYWORDS, value.toLowerCase())); + } + + /** + * @param {String} clsName Class name to check. + * @returns {boolean} 'true' if given class name is java primitive. + */ + isJavaPrimitive(clsName) { + return _.includes(JAVA_PRIMITIVES, clsName); + } +} diff --git a/modules/web-console/frontend/app/services/LegacyUtils.service.js b/modules/web-console/frontend/app/services/LegacyUtils.service.js index ed555a1ff50f0..dcf0bc869e9c8 100644 --- a/modules/web-console/frontend/app/services/LegacyUtils.service.js +++ b/modules/web-console/frontend/app/services/LegacyUtils.service.js @@ -122,45 +122,6 @@ export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { 'VARCHAR' ]; - const ALL_JDBC_TYPES = [ - {dbName: 'BIT', dbType: -7, javaType: 'Boolean', primitiveType: 'boolean'}, - {dbName: 'TINYINT', dbType: -6, javaType: 'Byte', primitiveType: 'byte'}, - {dbName: 'SMALLINT', dbType: 5, javaType: 'Short', primitiveType: 'short'}, - {dbName: 'INTEGER', dbType: 4, javaType: 'Integer', primitiveType: 'int'}, - {dbName: 'BIGINT', dbType: -5, javaType: 'Long', primitiveType: 'long'}, - {dbName: 'FLOAT', dbType: 6, javaType: 'Float', primitiveType: 'float'}, - {dbName: 'REAL', dbType: 7, javaType: 'Double', primitiveType: 'double'}, - {dbName: 'DOUBLE', dbType: 8, javaType: 'Double', primitiveType: 'double'}, - {dbName: 'NUMERIC', dbType: 2, javaType: 'BigDecimal'}, - {dbName: 'DECIMAL', dbType: 3, javaType: 'BigDecimal'}, - {dbName: 'CHAR', dbType: 1, javaType: 'String'}, - {dbName: 'VARCHAR', dbType: 12, javaType: 'String'}, - {dbName: 'LONGVARCHAR', dbType: -1, javaType: 'String'}, - {dbName: 'DATE', dbType: 91, javaType: 'Date'}, - {dbName: 'TIME', dbType: 92, javaType: 'Time'}, - {dbName: 'TIMESTAMP', dbType: 93, javaType: 'Timestamp'}, - {dbName: 'BINARY', dbType: -2, javaType: 'Object'}, - {dbName: 'VARBINARY', dbType: -3, javaType: 'Object'}, - {dbName: 'LONGVARBINARY', dbType: -4, javaType: 'Object'}, - {dbName: 'NULL', dbType: 0, javaType: 'Object'}, - {dbName: 'OTHER', dbType: 1111, javaType: 'Object'}, - {dbName: 'JAVA_OBJECT', dbType: 2000, javaType: 'Object'}, - {dbName: 'DISTINCT', dbType: 2001, javaType: 'Object'}, - {dbName: 'STRUCT', dbType: 2002, javaType: 'Object'}, - {dbName: 'ARRAY', dbType: 2003, javaType: 'Object'}, - {dbName: 'BLOB', dbType: 2004, javaType: 'Object'}, - {dbName: 'CLOB', dbType: 2005, javaType: 'String'}, - {dbName: 'REF', dbType: 2006, javaType: 'Object'}, - {dbName: 'DATALINK', dbType: 70, javaType: 'Object'}, - {dbName: 'BOOLEAN', dbType: 16, javaType: 'Boolean', primitiveType: 'boolean'}, - {dbName: 'ROWID', dbType: -8, javaType: 'Object'}, - {dbName: 'NCHAR', dbType: -15, javaType: 'String'}, - {dbName: 'NVARCHAR', dbType: -9, javaType: 'String'}, - {dbName: 'LONGNVARCHAR', dbType: -16, javaType: 'String'}, - {dbName: 'NCLOB', dbType: 2011, javaType: 'String'}, - {dbName: 'SQLXML', dbType: 2009, javaType: 'Object'} - ]; - /*eslint-disable */ const JAVA_KEYWORDS = [ 'abstract', @@ -361,11 +322,6 @@ export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { }, isEmptyString, SUPPORTED_JDBC_TYPES, - findJdbcType(jdbcType) { - const res = _.find(ALL_JDBC_TYPES, (item) => item.dbType === jdbcType); - - return res ? res : {dbName: 'Unknown', javaType: 'Unknown'}; - }, javaBuiltInClasses, javaBuiltInTypes, isJavaBuiltInClass, diff --git a/modules/web-console/frontend/app/services/SqlTypes.service.js b/modules/web-console/frontend/app/services/SqlTypes.service.js new file mode 100644 index 0000000000000..2a16e9d559f0c --- /dev/null +++ b/modules/web-console/frontend/app/services/SqlTypes.service.js @@ -0,0 +1,67 @@ +/* + * 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. + */ + +import _ from 'lodash'; + +// List of H2 reserved SQL keywords. +import H2_SQL_KEYWORDS from '../data/sql-keywords.json'; + +// List of JDBC type descriptors. +import JDBC_TYPES from '../data/jdbc-types.json'; + +// Regular expression to check H2 SQL identifier. +const VALID_IDENTIFIER = /^[a-zA-Z_][a-zA-Z0-9_$]*$/im; + +// Descriptor for unknown JDBC type. +const UNKNOWN_JDBC_TYPE = { + dbName: 'Unknown', + signed: {javaType: 'Unknown', primitiveType: 'Unknown'}, + unsigned: {javaType: 'Unknown', primitiveType: 'Unknown'} +}; + +/** + * Utility service for various check on SQL types. + */ +export default class SqlTypes { + /** + * @param value {String} Value to check. + * @returns {boolean} 'true' if given text is valid Java class name. + */ + validIdentifier(value) { + return !!(value && VALID_IDENTIFIER.test(value)); + } + + /** + * @param value {String} Value to check. + * @returns {boolean} 'true' if given text is one of H2 reserved keywords. + */ + isKeyword(value) { + return !!(value && _.includes(H2_SQL_KEYWORDS, value.toUpperCase())); + } + + /** + * Find JDBC type descriptor for specified JDBC type and options. + * + * @param dbType {Number} Column db type. + * @return {String} Java type. + */ + findJdbcType(dbType) { + const jdbcType = _.find(JDBC_TYPES, (item) => item.dbType === dbType); + + return jdbcType ? jdbcType : UNKNOWN_JDBC_TYPE; + } +} diff --git a/modules/web-console/frontend/controllers/admin-controller.js b/modules/web-console/frontend/controllers/admin-controller.js index 57a39b2e87b6b..70043016bff1d 100644 --- a/modules/web-console/frontend/controllers/admin-controller.js +++ b/modules/web-console/frontend/controllers/admin-controller.js @@ -17,8 +17,8 @@ // Controller for Admin screen. export default ['adminController', [ - '$rootScope', '$scope', '$http', '$q', '$state', 'IgniteMessages', 'IgniteConfirm', 'User', 'IgniteCountries', - ($rootScope, $scope, $http, $q, $state, Messages, Confirm, User, Countries) => { + '$rootScope', '$scope', '$http', '$q', '$state', 'IgniteMessages', 'IgniteConfirm', 'User', 'IgniteNotebookData', 'IgniteCountries', + ($rootScope, $scope, $http, $q, $state, Messages, Confirm, User, Notebook, Countries) => { $scope.users = null; const _reloadUsers = () => { @@ -41,12 +41,13 @@ export default ['adminController', [ $scope.becomeUser = function(user) { $http.get('/api/v1/admin/become', { params: {viewedUserId: user._id}}) .catch(({data}) => Promise.reject(data)) - .then(User.load) + .then(() => User.load()) .then((becomeUser) => { $rootScope.$broadcast('user', becomeUser); $state.go('base.configuration.clusters'); }) + .then(() => Notebook.load()) .catch(Messages.showError); }; diff --git a/modules/web-console/frontend/controllers/domains-controller.js b/modules/web-console/frontend/controllers/domains-controller.js index 6cb3540c82391..2d450dbe91f25 100644 --- a/modules/web-console/frontend/controllers/domains-controller.js +++ b/modules/web-console/frontend/controllers/domains-controller.js @@ -17,8 +17,8 @@ // Controller for Domain model screen. export default ['domainsController', [ - '$rootScope', '$scope', '$http', '$state', '$filter', '$timeout', '$modal', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteConfirmBatch', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteAgentMonitor', 'IgniteLegacyTable', 'igniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', - function($root, $scope, $http, $state, $filter, $timeout, $modal, LegacyUtils, Messages, Focus, Confirm, ConfirmBatch, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, IgniteAgentMonitor, LegacyTable, Resource, ErrorPopover, FormUtils) { + '$rootScope', '$scope', '$http', '$state', '$filter', '$timeout', '$modal', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteConfirmBatch', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteAgentMonitor', 'IgniteLegacyTable', 'igniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', 'JavaTypes', 'SqlTypes', + function($root, $scope, $http, $state, $filter, $timeout, $modal, LegacyUtils, Messages, Focus, Confirm, ConfirmBatch, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, IgniteAgentMonitor, LegacyTable, Resource, ErrorPopover, FormUtils, JavaTypes, SqlTypes) { UnsavedChangesGuard.install($scope); const emptyDomain = {empty: true}; @@ -533,10 +533,15 @@ export default ['domainsController', [ return 'Associate with ' + cacheName; }; - function toJavaClassName(name) { + function isValidJavaIdentifier(s) { + return JavaTypes.validIdentifier(s) && !JavaTypes.isKeyword(s) && + SqlTypes.validIdentifier(s) && !SqlTypes.isKeyword(s); + } + + function toJavaIdentifier(name) { const len = name.length; - let buf = ''; + let ident = ''; let capitalizeNext = true; @@ -546,21 +551,35 @@ export default ['domainsController', [ if (ch === ' ' || ch === '_') capitalizeNext = true; else if (capitalizeNext) { - buf += ch.toLocaleUpperCase(); + ident += ch.toLocaleUpperCase(); capitalizeNext = false; } else - buf += ch.toLocaleLowerCase(); + ident += ch.toLocaleLowerCase(); } - return buf; + return ident; } - function toJavaName(dbName) { - const javaName = toJavaClassName(dbName); + function toJavaClassName(name) { + const clazzName = toJavaIdentifier(name); + + if (isValidJavaIdentifier(clazzName)) + return clazzName; - return javaName.charAt(0).toLocaleLowerCase() + javaName.slice(1); + return 'Class' + clazzName; + } + + function toJavaFieldName(dbName) { + const javaName = toJavaIdentifier(dbName); + + const fieldName = javaName.charAt(0).toLocaleLowerCase() + javaName.slice(1); + + if (isValidJavaIdentifier(fieldName)) + return fieldName; + + return 'field' + javaName; } function _fillCommonCachesOrTemplates(item) { @@ -588,6 +607,7 @@ export default ['domainsController', [ item.cacheOrTemplate = item.cachesOrTemplates[0].value; }; } + /** * Load list of database tables. */ @@ -752,18 +772,16 @@ export default ['domainsController', [ let containKey = true; let containDup = false; - function queryField(name, jdbcType) { - return {name: toJavaName(name), className: jdbcType.javaType}; - } + function dbField(name, jdbcType, nullable, unsigned) { + const javaTypes = (unsigned && jdbcType.unsigned) ? jdbcType.unsigned : jdbcType.signed; + const javaFieldType = (!nullable && javaTypes.primitiveType && $scope.ui.usePrimitives) ? javaTypes.primitiveType : javaTypes.javaType; - function dbField(name, jdbcType, nullable) { return { - jdbcType, databaseFieldName: name, databaseFieldType: jdbcType.dbName, - javaFieldName: toJavaName(name), - javaFieldType: nullable ? jdbcType.javaType : - ($scope.ui.usePrimitives && jdbcType.primitiveType ? jdbcType.primitiveType : jdbcType.javaType) + javaType: javaTypes.javaType, + javaFieldName: toJavaFieldName(name), + javaFieldType }; } @@ -790,17 +808,17 @@ export default ['domainsController', [ let _containKey = false; _.forEach(table.cols, function(col) { - const colName = col.name; - const jdbcType = LegacyUtils.findJdbcType(col.type); - const nullable = col.nullable; + const fld = dbField(col.name, SqlTypes.findJdbcType(col.type), col.nullable, col.unsigned); - qryFields.push(queryField(colName, jdbcType)); + qryFields.push({name: fld.javaFieldName, className: fld.javaType}); - const fld = dbField(colName, jdbcType, nullable); + const dbName = fld.databaseFieldName; - if ($scope.ui.generateAliases && !_.find(aliases, {field: fld.javaFieldName}) && - fld.javaFieldName.toUpperCase() !== fld.databaseFieldName.toUpperCase()) - aliases.push({field: fld.javaFieldName, alias: fld.databaseFieldName}); + if ($scope.ui.generateAliases && + SqlTypes.validIdentifier(dbName) && !SqlTypes.isKeyword(dbName) && + !_.find(aliases, {field: fld.javaFieldName}) && + fld.javaFieldName.toUpperCase() !== dbName.toUpperCase()) + aliases.push({field: fld.javaFieldName, alias: dbName}); if (col.key) { keyFields.push(fld); @@ -820,7 +838,7 @@ export default ['domainsController', [ indexes.push({ name: idx.name, indexType: 'SORTED', fields: _.map(fields, function(fieldName) { return { - name: toJavaName(fieldName), + name: toJavaFieldName(fieldName), direction: idx.fields[fieldName] }; }) @@ -828,9 +846,7 @@ export default ['domainsController', [ }); } - const domainFound = _.find($scope.domains, function(domain) { - return domain.valueType === valType; - }); + const domainFound = _.find($scope.domains, (domain) => domain.valueType === valType); const newDomain = { confirm: false, @@ -864,17 +880,13 @@ export default ['domainsController', [ if ($scope.ui.builtinKeys && newDomain.keyFields.length === 1) { const keyField = newDomain.keyFields[0]; - newDomain.keyType = keyField.jdbcType.javaType; + newDomain.keyType = keyField.javaType; // Exclude key column from query fields and indexes. - newDomain.fields = _.filter(newDomain.fields, function(field) { - return field.name !== keyField.javaFieldName; - }); + newDomain.fields = _.filter(newDomain.fields, (field) => field.name !== keyField.javaFieldName); _.forEach(newDomain.indexes, function(index) { - index.fields = _.filter(index.fields, function(field) { - return field.name !== keyField.javaFieldName; - }); + index.fields = _.filter(index.fields, (field) => field.name !== keyField.javaFieldName); }); newDomain.indexes = _.filter(newDomain.indexes, (index) => !_.isEmpty(index.fields)); diff --git a/modules/web-console/frontend/controllers/profile-controller.js b/modules/web-console/frontend/controllers/profile-controller.js index 9499db5e96f0a..fd595d9df8c66 100644 --- a/modules/web-console/frontend/controllers/profile-controller.js +++ b/modules/web-console/frontend/controllers/profile-controller.js @@ -19,7 +19,8 @@ export default ['profileController', [ '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteCountries', 'User', function($root, $scope, $http, LegacyUtils, Messages, Focus, Confirm, Countries, User) { - $scope.user = angular.copy($root.user); + User.read() + .then((user) => $scope.user = angular.copy(user)); $scope.countries = Countries.getAll(); @@ -74,7 +75,7 @@ export default ['profileController', [ $scope.saveUser = () => { $http.post('/api/v1/profile/save', $scope.user) .catch(({data}) => Promise.reject(data)) - .then(User.read) + .then(User.load) .then(() => { if ($scope.expandedPassword) $scope.togglePassword(); diff --git a/modules/web-console/frontend/generator/generator-common.js b/modules/web-console/frontend/generator/generator-common.js index 124a1b73c0c47..d502c8a5c4605 100644 --- a/modules/web-console/frontend/generator/generator-common.js +++ b/modules/web-console/frontend/generator/generator-common.js @@ -39,9 +39,14 @@ $generatorCommon.formatDate = function(date) { return mm + '/' + dd + '/' + yyyy + ' ' + $generatorCommon.addLeadingZero(date.getHours(), 2) + ':' + $generatorCommon.addLeadingZero(date.getMinutes(), 2); }; -// Generate comment for generated XML, Java, ... files. -$generatorCommon.mainComment = function mainComment() { - return 'This configuration was generated by Ignite Web Console (' + $generatorCommon.formatDate(new Date()) + ')'; +/** + * Generate title comment for XML, Java, ... files. + * + * @param sbj {string} What is generated. + * @returns {string} Text to add as title comment in generated java class. + */ +$generatorCommon.mainComment = function mainComment(sbj) { + return 'This ' + sbj + ' was generated by Ignite Web Console (' + $generatorCommon.formatDate(new Date()) + ')'; }; // Create result holder with service functions and properties for XML and java code generation. @@ -489,6 +494,14 @@ $generatorCommon.IGFS_IPC_CONFIGURATION = { } }; +$generatorCommon.ODBC_CONFIGURATION = { + className: 'org.apache.ignite.configuration.OdbcConfiguration', + fields: { + endpointAddress: {dflt: '0.0.0.0:10800..10810'}, + maxOpenCursors: {dflt: 128} + } +}; + // Check that cache has datasource. $generatorCommon.cacheHasDatasource = function(cache) { if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) { diff --git a/modules/web-console/frontend/generator/generator-java.js b/modules/web-console/frontend/generator/generator-java.js index 6cc3323b6580a..db54928e4f53e 100644 --- a/modules/web-console/frontend/generator/generator-java.js +++ b/modules/web-console/frontend/generator/generator-java.js @@ -1296,6 +1296,21 @@ $generatorJava.clusterTime = function(cluster, res) { return res; }; +// Generate ODBC configuration group. +$generatorJava.clusterODBC = function(odbc, res) { + if (!res) + res = $generatorCommon.builder(); + + if (odbc && odbc.odbcEnabled) { + $generatorJava.beanProperty(res, 'cfg', odbc, 'odbcConfiguration', 'odbcConfiguration', + $generatorCommon.ODBC_CONFIGURATION.className, $generatorCommon.ODBC_CONFIGURATION.fields, true); + } + + res.needEmptyLine = true; + + return res; +}; + // Generate thread pools group. $generatorJava.clusterPools = function(cluster, res) { if (!res) @@ -2332,7 +2347,7 @@ $generatorJava.javaClassCode = function(domain, key, pkg, useConstructor, includ res.line('/**'); res.line(' * ' + type + ' definition.'); res.line(' *'); - res.line(' * ' + $generatorCommon.mainComment()); + res.line(' * ' + $generatorCommon.mainComment('POJO')); res.line(' */'); res.startBlock('public class ' + type + ' implements ' + res.importClass('java.io.Serializable') + ' {'); @@ -2537,7 +2552,7 @@ $generatorJava.javaClassCode = function(domain, key, pkg, useConstructor, includ res.endBlock('}'); - return 'package ' + pkg + ';' + '\n\n' + res.generateImports() + '\n\n' + res.generateStaticImports() + '\n\n' + res.asString(); + return 'package ' + pkg + ';' + '\n\n' + res.generateImports() + '\n' + res.generateStaticImports() + '\n' + res.asString(); }; /** @@ -2872,6 +2887,8 @@ $generatorJava.clusterConfiguration = function(cluster, clientNearCfg, res) { $generatorJava.clusterLogger(cluster.logger, res); + $generatorJava.clusterODBC(cluster.odbc, res); + $generatorJava.clusterMarshaller(cluster, res); $generatorJava.clusterMetrics(cluster, res); @@ -2938,7 +2955,7 @@ $generatorJava.cluster = function(cluster, pkg, javaClass, clientNearCfg) { res.mergeProps(resCfg); res.line('/**'); - res.line(' * ' + $generatorCommon.mainComment()); + res.line(' * ' + $generatorCommon.mainComment('configuration')); res.line(' */'); res.startBlock('public class ' + javaClass + ' {'); @@ -3002,7 +3019,7 @@ $generatorJava.cluster = function(cluster, pkg, javaClass, clientNearCfg) { res.endBlock('}'); - return 'package ' + pkg + ';\n\n' + res.generateImports() + '\n\n' + res.generateStaticImports() + '\n\n' + res.asString(); + return 'package ' + pkg + ';\n\n' + res.generateImports() + '\n' + res.generateStaticImports() + '\n' + res.asString(); } return res.asString(); @@ -3447,7 +3464,7 @@ $generatorJava.nodeStartup = function(cluster, pkg, cls, cfg, factoryCls, client const res = $generatorCommon.builder(); res.line('/**'); - res.line(' * ' + $generatorCommon.mainComment()); + res.line(' * ' + $generatorCommon.mainComment('node start up')); if (demo) { res.line(' *'); @@ -3528,7 +3545,67 @@ $generatorJava.nodeStartup = function(cluster, pkg, cls, cfg, factoryCls, client res.endBlock('}'); - return 'package ' + pkg + ';\n\n' + res.generateImports() + '\n\n' + res.generateStaticImports() + '\n\n' + res.asString(); + return 'package ' + pkg + ';\n\n' + res.generateImports() + '\n' + res.generateStaticImports() + '\n' + res.asString(); }; +/** + * Function to generate java class for load caches. + * + * @param caches Caches to load. + * @param pkg Class package name. + * @param cls Class name. + * @param cfg Config. + */ +$generatorJava.loadCaches = function(caches, pkg, cls, cfg) { + const res = $generatorCommon.builder(); + + res.line('/**'); + res.line(' * ' + $generatorCommon.mainComment('utility')); + res.line(' */'); + res.startBlock('public class ' + cls + ' {'); + + res.line('/**'); + res.line(' *

          '); + res.line(' * Utility to load caches from database.'); + res.line(' *

          '); + res.line(' * How to use:'); + res.line(' *

            '); + res.line(' *
          • Start cluster.
          • '); + res.line(' *
          • Start this utility and wait while load complete.
          • '); + res.line(' *
          '); + res.line(' *'); + res.line(' * @param args Command line arguments, none required.'); + res.line(' * @throws Exception If failed.'); + res.line(' */'); + + res.startBlock('public static void main(String[] args) throws Exception {'); + + res.startBlock('try (' + res.importClass('org.apache.ignite.Ignite') + ' ignite = ' + + res.importClass('org.apache.ignite.Ignition') + '.start(' + cfg + ')) {'); + + res.line('System.out.println(">>> Loading caches...");'); + + res.needEmptyLine = true; + + _.forEach(caches, (cache) => { + res.line('System.out.println(">>> Loading cache: ' + cache.name + '");'); + res.line('ignite.cache("' + cache.name + '").loadCache(null);'); + + res.needEmptyLine = true; + }); + + res.needEmptyLine = true; + + res.line('System.out.println(">>> All caches loaded!");'); + + res.endBlock('}'); + + res.endBlock('}'); + + res.endBlock('}'); + + return 'package ' + pkg + ';\n\n' + res.generateImports() + '\n' + res.generateStaticImports() + '\n' + res.asString(); +}; + + export default $generatorJava; diff --git a/modules/web-console/frontend/generator/generator-properties.js b/modules/web-console/frontend/generator/generator-properties.js index 9d1e7e32c2285..57427138f43a3 100644 --- a/modules/web-console/frontend/generator/generator-properties.js +++ b/modules/web-console/frontend/generator/generator-properties.js @@ -41,7 +41,7 @@ $generatorProperties.jdbcUrlTemplate = function(dialect) { $generatorProperties.createBuilder = function() { const res = $generatorCommon.builder(); - res.line('# ' + $generatorCommon.mainComment()); + res.line('# ' + $generatorCommon.mainComment('list of properties')); return res; }; diff --git a/modules/web-console/frontend/generator/generator-xml.js b/modules/web-console/frontend/generator/generator-xml.js index b49b052efdcda..ded93af048294 100644 --- a/modules/web-console/frontend/generator/generator-xml.js +++ b/modules/web-console/frontend/generator/generator-xml.js @@ -1009,6 +1009,19 @@ $generatorXml.clusterTime = function(cluster, res) { return res; }; +// Generate OBC configuration group. +$generatorXml.clusterODBC = function(odbc, res) { + if (!res) + res = $generatorCommon.builder(); + + if (odbc && odbc.odbcEnabled) + $generatorXml.beanProperty(res, odbc, 'odbcConfiguration', $generatorCommon.ODBC_CONFIGURATION, true); + + res.needEmptyLine = true; + + return res; +}; + // Generate thread pools group. $generatorXml.clusterPools = function(cluster, res) { if (!res) @@ -2003,6 +2016,8 @@ $generatorXml.clusterConfiguration = function(cluster, clientNearCfg, res) { $generatorXml.clusterLogger(cluster.logger, res); + $generatorXml.clusterODBC(cluster.odbc, res); + $generatorXml.clusterMarshaller(cluster, res); $generatorXml.clusterMetrics(cluster, res); @@ -2058,7 +2073,7 @@ $generatorXml.cluster = function(cluster, clientNearCfg) { // 1. Add header. let xml = '\n\n'; - xml += '\n\n'; + xml += '\n\n'; xml += ' { test('nonBuiltInClass', () => { - assert.equal(nonBuiltInClass('BigDecimal'), false); - assert.equal(nonBuiltInClass('java.math.BigDecimal'), false); + assert.equal(INSTANCE.nonBuiltInClass('BigDecimal'), false); + assert.equal(INSTANCE.nonBuiltInClass('java.math.BigDecimal'), false); - assert.equal(nonBuiltInClass('String'), false); - assert.equal(nonBuiltInClass('java.lang.String'), false); + assert.equal(INSTANCE.nonBuiltInClass('String'), false); + assert.equal(INSTANCE.nonBuiltInClass('java.lang.String'), false); - assert.equal(nonBuiltInClass('Timestamp'), false); - assert.equal(nonBuiltInClass('java.sql.Timestamp'), false); + assert.equal(INSTANCE.nonBuiltInClass('Timestamp'), false); + assert.equal(INSTANCE.nonBuiltInClass('java.sql.Timestamp'), false); - assert.equal(nonBuiltInClass('Date'), false); - assert.equal(nonBuiltInClass('java.sql.Date'), false); + assert.equal(INSTANCE.nonBuiltInClass('Date'), false); + assert.equal(INSTANCE.nonBuiltInClass('java.sql.Date'), false); - assert.equal(nonBuiltInClass('Date'), false); - assert.equal(nonBuiltInClass('java.util.Date'), false); + assert.equal(INSTANCE.nonBuiltInClass('Date'), false); + assert.equal(INSTANCE.nonBuiltInClass('java.util.Date'), false); - assert.equal(nonBuiltInClass('CustomClass'), true); - assert.equal(nonBuiltInClass('java.util.CustomClass'), true); - assert.equal(nonBuiltInClass('my.package.CustomClass'), true); + assert.equal(INSTANCE.nonBuiltInClass('CustomClass'), true); + assert.equal(INSTANCE.nonBuiltInClass('java.util.CustomClass'), true); + assert.equal(INSTANCE.nonBuiltInClass('my.package.CustomClass'), true); }); test('fullClassName', () => { - assert.equal(fullClassName('BigDecimal'), 'java.math.BigDecimal'); + assert.equal(INSTANCE.fullClassName('BigDecimal'), 'java.math.BigDecimal'); }); test('validIdentifier', () => { - assert.equal(validIdentifier('java.math.BigDecimal'), true); + assert.equal(INSTANCE.validIdentifier('myIdent'), true); + assert.equal(INSTANCE.validIdentifier('java.math.BigDecimal'), false); + assert.equal(INSTANCE.validIdentifier('2Demo'), false); + assert.equal(INSTANCE.validIdentifier('abra kadabra'), false); + assert.equal(INSTANCE.validIdentifier(undefined), false); + assert.equal(INSTANCE.validIdentifier(null), false); + assert.equal(INSTANCE.validIdentifier(''), false); + assert.equal(INSTANCE.validIdentifier(' '), false); + }); + + test('validClassName', () => { + assert.equal(INSTANCE.validClassName('java.math.BigDecimal'), true); + assert.equal(INSTANCE.validClassName('2Demo'), false); + assert.equal(INSTANCE.validClassName('abra kadabra'), false); + assert.equal(INSTANCE.validClassName(undefined), false); + assert.equal(INSTANCE.validClassName(null), false); + assert.equal(INSTANCE.validClassName(''), false); + assert.equal(INSTANCE.validClassName(' '), false); }); test('validPackage', () => { - assert.equal(validPackage('java.math.BigDecimal'), true); + assert.equal(INSTANCE.validPackage('java.math.BigDecimal'), true); + assert.equal(INSTANCE.validPackage('my.org.SomeClass'), true); + assert.equal(INSTANCE.validPackage('25'), false); + assert.equal(INSTANCE.validPackage('abra kadabra'), false); + assert.equal(INSTANCE.validPackage(''), false); + assert.equal(INSTANCE.validPackage(' '), false); }); test('packageSpecified', () => { - assert.equal(packageSpecified('java.math.BigDecimal'), true); + assert.equal(INSTANCE.packageSpecified('java.math.BigDecimal'), true); + assert.equal(INSTANCE.packageSpecified('BigDecimal'), false); }); - test('isKeywords', () => { - assert.equal(isKeywords('abstract'), true); + test('isKeyword', () => { + assert.equal(INSTANCE.isKeyword('abstract'), true); + assert.equal(INSTANCE.isKeyword('Abstract'), true); + assert.equal(INSTANCE.isKeyword('abra kadabra'), false); + assert.equal(INSTANCE.isKeyword(undefined), false); + assert.equal(INSTANCE.isKeyword(null), false); + assert.equal(INSTANCE.isKeyword(''), false); + assert.equal(INSTANCE.isKeyword(' '), false); }); test('isJavaPrimitive', () => { - assert.equal(isJavaPrimitive('boolean'), true); + assert.equal(INSTANCE.isJavaPrimitive('boolean'), true); + }); + + test('validUUID', () => { + assert.equal(INSTANCE.validUUID('123e4567-e89b-12d3-a456-426655440000'), true); + assert.equal(INSTANCE.validUUID('12345'), false); + assert.equal(INSTANCE.validUUID(undefined), false); + assert.equal(INSTANCE.validUUID(null), false); + assert.equal(INSTANCE.validUUID(''), false); + assert.equal(INSTANCE.validUUID(' '), false); }); }); diff --git a/modules/web-console/frontend/test/unit/SqlTypes.test.js b/modules/web-console/frontend/test/unit/SqlTypes.test.js new file mode 100644 index 0000000000000..3cfaafc1212f3 --- /dev/null +++ b/modules/web-console/frontend/test/unit/SqlTypes.test.js @@ -0,0 +1,68 @@ +/* + * 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. + */ + +/* + * 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. + */ + +import SqlTypes from '../../app/services/SqlTypes.service.js'; + +const INSTANCE = new SqlTypes(); + +import { assert } from 'chai'; + +suite('SqlTypesTestsSuite', () => { + test('validIdentifier', () => { + assert.equal(INSTANCE.validIdentifier('myIdent'), true); + assert.equal(INSTANCE.validIdentifier('java.math.BigDecimal'), false); + assert.equal(INSTANCE.validIdentifier('2Demo'), false); + assert.equal(INSTANCE.validIdentifier('abra kadabra'), false); + assert.equal(INSTANCE.validIdentifier(undefined), false); + assert.equal(INSTANCE.validIdentifier(null), false); + assert.equal(INSTANCE.validIdentifier(''), false); + assert.equal(INSTANCE.validIdentifier(' '), false); + }); + + test('isKeyword', () => { + assert.equal(INSTANCE.isKeyword('group'), true); + assert.equal(INSTANCE.isKeyword('Group'), true); + assert.equal(INSTANCE.isKeyword('select'), true); + assert.equal(INSTANCE.isKeyword('abra kadabra'), false); + assert.equal(INSTANCE.isKeyword(undefined), false); + assert.equal(INSTANCE.isKeyword(null), false); + assert.equal(INSTANCE.isKeyword(''), false); + assert.equal(INSTANCE.isKeyword(' '), false); + }); + + test('findJdbcType', () => { + assert.equal(INSTANCE.findJdbcType(0).dbName, 'NULL'); + assert.equal(INSTANCE.findJdbcType(5555).dbName, 'Unknown'); + }) +}); diff --git a/modules/web-console/frontend/views/configuration/clusters.jade b/modules/web-console/frontend/views/configuration/clusters.jade index b10a477448ceb..b79b1ea13e86c 100644 --- a/modules/web-console/frontend/views/configuration/clusters.jade +++ b/modules/web-console/frontend/views/configuration/clusters.jade @@ -56,6 +56,7 @@ include ../../app/helpers/jade/mixins.jade include ../../app/modules/states/configuration/clusters/logger.jade include ../../app/modules/states/configuration/clusters/marshaller.jade include ../../app/modules/states/configuration/clusters/metrics.jade + include ../../app/modules/states/configuration/clusters/odbc.jade include ../../app/modules/states/configuration/clusters/ssl.jade include ../../app/modules/states/configuration/clusters/swap.jade include ../../app/modules/states/configuration/clusters/thread.jade diff --git a/modules/web-console/frontend/views/configuration/domains-import.jade b/modules/web-console/frontend/views/configuration/domains-import.jade index d95e98ab626ba..e2eaf97b958c5 100644 --- a/modules/web-console/frontend/views/configuration/domains-import.jade +++ b/modules/web-console/frontend/views/configuration/domains-import.jade @@ -53,64 +53,31 @@ mixin td-ellipses-lbl(w, lbl) li Copy h2-x.x.x.jar into agent 'jdbc-drivers' folder and try again li Refer to agent README.txt for more information .import-domain-model-wizard-page(ng-if='importDomain.action == "connect" && !importDomain.demo') - - var form = 'connectForm' + -var form = 'connectForm' form.form-horizontal(name=form novalidate) + .settings-row.settings-row_small-label + +ignite-form-field-dropdown('Driver JAR:', 'ui.selectedJdbcDriverJar', '"jdbcDriverJar"', false, true, false, + 'Choose JDBC driver', '', 'jdbcDriverJars', + 'Select appropriate JAR with JDBC driver
          To add another driver you need to place it into "/jdbc-drivers" folder of Ignite Web Agent
          Refer to Ignite Web Agent README.txt for for more information' + )(data-container='.modal-domain-import') + .settings-row.settings-row_small-label + +java-class('JDBC driver:', 'selectedPreset.jdbcDriverClass', '"jdbcDriverClass"', true, true, 'Fully qualified class name of JDBC driver that will be used to connect to database') + .settings-row.settings-row_small-label + +text('JDBC URL:', 'selectedPreset.jdbcUrl', '"jdbcUrl"', true, 'JDBC URL', 'JDBC URL for connecting to database
          Refer to your database documentation for details') + .settings-row.settings-row_small-label + +text('User:', 'selectedPreset.user', '"jdbdUser"', false, '', 'User name for connecting to database') + .settings-row.settings-row_small-label + +password('Password:', 'selectedPreset.password', '"jdbcPassword"', false, '', 'Password for connecting to database
          Note, password would not be saved in preferences for security reasons')(ignite-on-enter='importDomainNext()') .settings-row - label.col-xs-4.col-sm-2.col-md-2.required Driver JAR: - .col-xs-8.col-sm-10.col-md-10 - i.tipField.fa.fa-question-circle(bs-tooltip='' data-title='Select appropriate JAR with JDBC driver
          To add another driver you need to place it into "/jdbc-drivers" folder of Ignite Web Agent
          Refer to Ignite Web Agent README.txt for for more information') - .input-tip - button.select-toggle.form-control(id='jdbcDriverJar' bs-select data-container='.modal-domain-import' ng-model='ui.selectedJdbcDriverJar' ng-class='{placeholder: !(jdbcDriverJars && jdbcDriverJars.length > 0)}' placeholder='Choose JDBC driver' bs-options='item.value as item.label for item in jdbcDriverJars') - .settings-row - label.col-xs-4.col-sm-2.col-md-2.required JDBC driver: - .col-xs-8.col-sm-10.col-md-10 - - var name = '"jdbcDriverClass"' - i.tipField.fa.fa-question-circle(bs-tooltip='' data-title='Fully qualified class name of JDBC driver that will be used to connect to database') - - +form-field-feedback(name, 'javaBuiltInClass', 'JDBC Driver should not be the Java built-in class!') - +form-field-feedback(name, 'javaKeywords', 'JDBC Driver could not contains reserved Java keyword!') - +form-field-feedback(name, 'javaPackageSpecified', 'JDBC Driver does not have package specified!') - +form-field-feedback(name, 'javaIdentifier', 'JDBC Driver is invalid Java identifier!') - - .input-tip - +ignite-form-field-input(name, 'selectedPreset.jdbcDriverClass', false, true, 'Enter fully qualified class name')( - data-java-identifier='true' - data-java-package-specified='true' - data-java-keywords='true' - data-java-built-in-class='true' - ) - .settings-row - label.col-xs-4.col-sm-2.col-md-2.required JDBC URL: - .col-xs-8.col-sm-10.col-md-10 - i.tipField.fa.fa-question-circle(bs-tooltip='' data-title='JDBC URL for connecting to database
          Refer to your database documentation for details') - .input-tip - +ignite-form-field-input('jdbcUrl', 'selectedPreset.jdbcUrl', false, true, 'JDBC URL') - .settings-row - label.col-xs-4.col-sm-2.col-md-2 User: - .col-xs-8.col-sm-10.col-md-10 - i.tipField.fa.fa-question-circle(bs-tooltip='' data-title='User name for connecting to database') - .input-tip - input.form-control(id='user' type='text' ng-model='selectedPreset.user') - .settings-row - label.col-xs-4.col-sm-2.col-md-2 Password: - .col-xs-8.col-sm-10.col-md-10 - i.tipField.fa.fa-question-circle(bs-tooltip='' data-title='Password for connecting to database
          Note, password would not be saved in preferences for security reasons') - .input-tip - input.form-control(id='password' type='password' ng-model='selectedPreset.password' ignite-on-enter='importDomainNext()') - .settings-row - .checkbox - label - input(id='tablesOnly' type='checkbox' ng-model='selectedPreset.tablesOnly') - | Tables only - i.tipLabel.fa.fa-question-circle(bs-tooltip='' data-title='If selected, then only tables metadata will be parsed
          Otherwise table and view metadata will be parsed') + +checkbox('Tables only', 'selectedPreset.tablesOnly', '"tablesOnly"', 'If selected, then only tables metadata will be parsed
          Otherwise table and view metadata will be parsed') .import-domain-model-wizard-page(ng-show='importDomain.action == "schemas"') table.table.metadata(st-table='importDomain.displayedSchemas' st-safe-src='importDomain.schemas') thead tr th.header(colspan='2') .col-sm-4.pull-right(style='margin-bottom: 5px') - input.form-control(type='text' st-search='name' placeholder='Filter schemas...' ng-model='importDomain.displayedSchemasFilter' ng-change='selectSchema()') + input.form-control(type='text' st-search='name' placeholder='Filter schemas...' ng-model='importDomain.displayedSchemasFilter' ) tr th(width='30px') +chk('importDomain.allSchemasSelected', 'selectAllSchemas()', 'Select all schemas') @@ -174,49 +141,22 @@ mixin td-ellipses-lbl(w, lbl) .col-sm-1(style='padding-left: 5px') button.btn.btn-primary(ng-click='applyDefaults()') Apply .import-domain-model-wizard-page(ng-show='importDomain.action == "options"') - - var form = 'optionsForm' - form.form-horizontal(name='optionsForm' novalidate) - .settings-row - .col-xs-3.col-sm-2.col-md-2.required - label.required Package: - .col-xs-9.col-sm-10.col-md-10 - - var name = '"domainPackageName"' - i.tipField.fa.fa-question-circle(bs-tooltip='' data-title='Package that will be used for POJOs generation') - - +form-field-feedback(name, 'javaPackageName', 'Package name is invalid') - +form-field-feedback(name, 'javaKeywords', 'Package name could not contains reserved java keyword') + -var form = 'optionsForm' - .input-tip - +ignite-form-field-input(name, 'ui.packageName', false, true, 'Enter package name')( - data-java-keywords='true' - data-java-package-name='package-only' - ng-model-options='{allowInvalid: true}' - ) - .settings-row - .checkbox - label - input(id='domainBuiltinKeys' type='checkbox' ng-model='ui.builtinKeys') - | Use Java built-in types for keys - i.tipLabel.fa.fa-question-circle(bs-tooltip='' data-title='Use Java built-in types like "Integer", "Long", "String" instead of POJO generation in case when table primary key contains only one field') + form.form-horizontal(name=form novalidate) + .settings-row.settings-row_small-label + +java-package('Package:', 'ui.packageName', '"domainPackageName"', true, true, 'Package that will be used for POJOs generation')(data-container='.modal-domain-import') .settings-row - .checkbox - label - input(id='domainUsePrimitives' type='checkbox' ng-model='ui.usePrimitives') - | Use primitive types for NOT NULL table columns - i.tipLabel.fa.fa-question-circle(bs-tooltip='' data-title='Use primitive types like "int", "long", "double" for POJOs fields generation in case of NOT NULL columns') + +checkbox('Use Java built-in types for keys', 'ui.builtinKeys', '"domainBuiltinKeys"', 'Use Java built-in types like "Integer", "Long", "String" instead of POJO generation in case when table primary key contains only one field') .settings-row - .checkbox - label - input(id='domainGenerateAliases' type='checkbox' ng-model='ui.generateAliases') - | Generate aliases for query fields - i.tipLabel.fa.fa-question-circle(bs-tooltip='' data-title='Generate aliases for query fields with database field names when database field name differ from Java field name') + +checkbox('Use primitive types for NOT NULL table columns', 'ui.usePrimitives', '"domainUsePrimitives"', 'Use primitive types like "int", "long", "double" for POJOs fields generation in case of NOT NULL columns') .settings-row - .col-xs-3.col-sm-2.col-md-2.required - label Clusters: - .col-xs-9.col-sm-10.col-md-10 - i.tipField.fa.fa-question-circle(bs-tooltip='' data-title='Choose clusters that will be associated with generated caches') - .input-tip - button.select-toggle.form-control(id='generatedCachesClusters' bs-select ng-model='ui.generatedCachesClusters' ng-class='{placeholder: !(ui.generatedCachesClusters && ui.generatedCachesClusters.length > 0)}' data-container='.modal-domain-import' data-multiple='1' placeholder='Choose clusters for generated caches' bs-options='item.value as item.label for item in clusters') + +checkbox('Generate aliases for query fields', 'ui.generateAliases', '"domainGenerateAliases"', 'Generate aliases for query fields with database field names when database field name differ from Java field name') + .settings-row.settings-row_small-label + +ignite-form-field-dropdown('Clusters:', 'ui.generatedCachesClusters', '"generatedCachesClusters"', false, false, true, + 'Choose clusters for generated caches', '', 'clusters', + 'Choose clusters that will be associated with generated caches' + )(data-container='.modal-domain-import') .modal-footer label(ng-hide='importDomain.action == "drivers" || (importDomain.action == "connect" && importDomain.demo)').labelField {{importDomain.info}} a.btn.btn-primary(ng-hide='importDomain.action == "drivers" || importDomain.action == "connect"' ng-click='importDomainPrev()' bs-tooltip='' data-title='{{prevTooltipText()}}' data-placement='bottom') Prev From 29acb33293c3d3130e16b7ff4d6b7ae260b7b78b Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Tue, 27 Sep 2016 17:15:38 +0700 Subject: [PATCH 175/487] Fixed typos. --- .../main/java/org/apache/ignite/cache/store/jdbc/JdbcType.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcType.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcType.java index 2107240338b85..22f0fac280c4e 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcType.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcType.java @@ -82,7 +82,7 @@ public JdbcType(JdbcType type) { /** * Gets associated cache name. * - * @return Сache name. + * @return Cache name. */ public String getCacheName() { return cacheName; From c2a3f11ca14cf9f9cf5bd2d6e2a87764f7cda5a7 Mon Sep 17 00:00:00 2001 From: Andrey Martianov Date: Tue, 20 Sep 2016 17:41:49 +0300 Subject: [PATCH 176/487] ignite-3621 Use single ttl cleanup worker thread for all caches (cherry picked from commit 1bc6058) --- .../processors/cache/GridCacheProcessor.java | 2 + .../cache/GridCacheSharedContext.java | 24 +++- .../GridCacheSharedTtlCleanupManager.java | 132 ++++++++++++++++++ .../processors/cache/GridCacheTtlManager.java | 115 +++------------ .../GridCacheTtlManagerNotificationTest.java | 107 +++++++++++++- .../IgniteCacheExpiryPolicyTestSuite.java | 2 + ...acheOnlyOneTtlCleanupThreadExistsTest.java | 102 ++++++++++++++ .../hashmap/GridCacheTestContext.java | 2 + 8 files changed, 384 insertions(+), 102 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheOnlyOneTtlCleanupThreadExistsTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 6640db8c7bcef..0a0b40aa01415 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -1870,6 +1870,7 @@ private GridCacheSharedContext createSharedContext(GridKernalContext kernalCtx, GridCachePartitionExchangeManager exchMgr = new GridCachePartitionExchangeManager(); GridCacheIoManager ioMgr = new GridCacheIoManager(); CacheAffinitySharedManager topMgr = new CacheAffinitySharedManager(); + GridCacheSharedTtlCleanupManager ttl = new GridCacheSharedTtlCleanupManager(); CacheJtaManagerAdapter jta = JTA.createOptional(); @@ -1882,6 +1883,7 @@ private GridCacheSharedContext createSharedContext(GridKernalContext kernalCtx, exchMgr, topMgr, ioMgr, + ttl, jta, storeSesLsnrs ); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java index 0cdf0a44ae4b2..8f3923515de5b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java @@ -93,6 +93,9 @@ public class GridCacheSharedContext { /** Affinity manager. */ private CacheAffinitySharedManager affMgr; + /** Ttl cleanup manager. */ + private GridCacheSharedTtlCleanupManager ttlMgr; + /** Cache contexts map. */ private ConcurrentMap> ctxMap; @@ -135,6 +138,7 @@ public class GridCacheSharedContext { * @param exchMgr Exchange manager. * @param affMgr Affinity manager. * @param ioMgr IO manager. + * @param ttlMgr Ttl cleanup manager. * @param jtaMgr JTA manager. * @param storeSesLsnrs Store session listeners. */ @@ -147,12 +151,13 @@ public GridCacheSharedContext( GridCachePartitionExchangeManager exchMgr, CacheAffinitySharedManager affMgr, GridCacheIoManager ioMgr, + GridCacheSharedTtlCleanupManager ttlMgr, CacheJtaManagerAdapter jtaMgr, Collection storeSesLsnrs ) { this.kernalCtx = kernalCtx; - setManagers(mgrs, txMgr, jtaMgr, verMgr, mvccMgr, depMgr, exchMgr, affMgr, ioMgr); + setManagers(mgrs, txMgr, jtaMgr, verMgr, mvccMgr, depMgr, exchMgr, affMgr, ioMgr, ttlMgr); this.storeSesLsnrs = storeSesLsnrs; @@ -248,7 +253,8 @@ void onReconnected() throws IgniteCheckedException { new GridCacheDeploymentManager(), new GridCachePartitionExchangeManager(), affMgr, - ioMgr); + ioMgr, + ttlMgr); this.mgrs = mgrs; @@ -272,13 +278,14 @@ private boolean restartOnDisconnect(GridCacheSharedManager mgr) { /** * @param mgrs Managers list. * @param txMgr Transaction manager. + * @param jtaMgr JTA manager. * @param verMgr Version manager. * @param mvccMgr MVCC manager. * @param depMgr Deployment manager. * @param exchMgr Exchange manager. * @param affMgr Affinity manager. * @param ioMgr IO manager. - * @param jtaMgr JTA manager. + * @param ttlMgr Ttl cleanup manager. */ private void setManagers(List> mgrs, IgniteTxManager txMgr, @@ -288,7 +295,8 @@ private void setManagers(List> mgrs, GridCacheDeploymentManager depMgr, GridCachePartitionExchangeManager exchMgr, CacheAffinitySharedManager affMgr, - GridCacheIoManager ioMgr) { + GridCacheIoManager ioMgr, + GridCacheSharedTtlCleanupManager ttlMgr) { this.mvccMgr = add(mgrs, mvccMgr); this.verMgr = add(mgrs, verMgr); this.txMgr = add(mgrs, txMgr); @@ -297,6 +305,7 @@ private void setManagers(List> mgrs, this.exchMgr = add(mgrs, exchMgr); this.affMgr = add(mgrs, affMgr); this.ioMgr = add(mgrs, ioMgr); + this.ttlMgr = add(mgrs, ttlMgr); } /** @@ -492,6 +501,13 @@ public GridCacheIoManager io() { return ioMgr; } + /** + * @return Ttl cleanup manager. + * */ + public GridCacheSharedTtlCleanupManager ttl() { + return ttlMgr; + } + /** * @return Cache deployment manager. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java new file mode 100644 index 0000000000000..d7d2cad901283 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.util.worker.GridWorker; +import org.apache.ignite.thread.IgniteThread; + +/** + * Periodically removes expired entities from caches with {@link CacheConfiguration#isEagerTtl()} flag set. + */ +public class GridCacheSharedTtlCleanupManager extends GridCacheSharedManagerAdapter { + /** Ttl cleanup worker thread sleep interval, ms. */ + private static final long CLEANUP_WORKER_SLEEP_INTERVAL = 500; + + /** Limit of expired entries processed by worker for certain cache in one pass. */ + private static final int CLEANUP_WORKER_ENTRIES_PROCESS_LIMIT = 1000; + + /** Cleanup worker. */ + private CleanupWorker cleanupWorker; + + /** Mutex on worker thread creation. */ + private final Object mux = new Object(); + + /** List of registered ttl managers. */ + private List mgrs = new CopyOnWriteArrayList<>(); + + /** {@inheritDoc} */ + @Override protected void onKernalStop0(boolean cancel) { + synchronized (mux) { + stopCleanupWorker(); + } + } + + /** + * Register ttl manager of cache for periodical check on expired entries. + * + * @param mgr ttl manager of cache. + * */ + public void register(GridCacheTtlManager mgr) { + synchronized (mux) { + if (cleanupWorker == null) + startCleanupWorker(); + + mgrs.add(mgr); + } + } + + /** + * Unregister ttl manager of cache from periodical check on expired entries. + * + * @param mgr ttl manager of cache. + * */ + public void unregister(GridCacheTtlManager mgr) { + synchronized (mux) { + mgrs.remove(mgr); + + if (mgrs.isEmpty()) + stopCleanupWorker(); + } + } + + /** + * + */ + private void startCleanupWorker() { + cleanupWorker = new CleanupWorker(); + + new IgniteThread(cleanupWorker).start(); + } + + /** + * + */ + private void stopCleanupWorker() { + if (null != cleanupWorker) { + U.cancel(cleanupWorker); + U.join(cleanupWorker, log); + + cleanupWorker = null; + } + } + + /** + * Entry cleanup worker. + */ + private class CleanupWorker extends GridWorker { + /** + * Creates cleanup worker. + */ + CleanupWorker() { + super(cctx.gridName(), "ttl-cleanup-worker", cctx.logger(GridCacheSharedTtlCleanupManager.class)); + } + + /** {@inheritDoc} */ + @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException { + while (!isCancelled()) { + boolean expiredRemains = false; + + for (GridCacheTtlManager mgr : mgrs) { + if (mgr.expire(CLEANUP_WORKER_ENTRIES_PROCESS_LIMIT)) + expiredRemains = true; + + if (isCancelled()) + return; + } + + if (!expiredRemains) + U.sleep(CLEANUP_WORKER_SLEEP_INTERVAL); + } + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java index 8ff035813528e..996544fdb4dd9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java @@ -17,20 +17,15 @@ package org.apache.ignite.internal.processors.cache; -import java.util.concurrent.atomic.AtomicLongFieldUpdater; - import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.GridConcurrentSkipListSet; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.internal.util.worker.GridWorker; -import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.Nullable; import org.jsr166.LongAdder8; @@ -43,19 +38,6 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter { /** Entries pending removal. */ private final GridConcurrentSkipListSetEx pendingEntries = new GridConcurrentSkipListSetEx(); - /** Cleanup worker. */ - private CleanupWorker cleanupWorker; - - /** Mutex. */ - private final Object mux = new Object(); - - /** Next expire time. */ - private volatile long nextExpireTime; - - /** Next expire time updater. */ - private static final AtomicLongFieldUpdater nextExpireTimeUpdater = - AtomicLongFieldUpdater.newUpdater(GridCacheTtlManager.class, "nextExpireTime"); - /** {@inheritDoc} */ @Override protected void start0() throws IgniteCheckedException { boolean cleanupDisabled = cctx.kernalContext().isDaemon() || @@ -68,19 +50,14 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter { if (cleanupDisabled) return; - cleanupWorker = new CleanupWorker(); - } - - /** {@inheritDoc} */ - @Override protected void onKernalStart0() throws IgniteCheckedException { - if (cleanupWorker != null) - new IgniteThread(cleanupWorker).start(); + cctx.shared().ttl().register(this); } /** {@inheritDoc} */ @Override protected void onKernalStop0(boolean cancel) { - U.cancel(cleanupWorker); - U.join(cleanupWorker, log); + pendingEntries.clear(); + + cctx.shared().ttl().unregister(this); } /** @@ -90,27 +67,10 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter { */ public void addTrackedEntry(GridCacheMapEntry entry) { assert Thread.holdsLock(entry); - assert cleanupWorker != null; EntryWrapper e = new EntryWrapper(entry); pendingEntries.add(e); - - while (true) { - long nextExpireTime = this.nextExpireTime; - - if (e.expireTime < nextExpireTime) { - if (nextExpireTimeUpdater.compareAndSet(this, nextExpireTime, e.expireTime)) { - synchronized (mux) { - mux.notifyAll(); - } - - break; - } - } - else - break; - } } /** @@ -118,7 +78,6 @@ public void addTrackedEntry(GridCacheMapEntry entry) { */ public void removeTrackedEntry(GridCacheMapEntry entry) { assert Thread.holdsLock(entry); - assert cleanupWorker != null; pendingEntries.remove(new EntryWrapper(entry)); } @@ -141,15 +100,27 @@ public int pendingSize() { * Expires entries by TTL. */ public void expire() { + expire(-1); + } + + /** + * Processes specified amount of expired entries. + * + * @param amount Limit of processed entries by single call, {@code -1} for no limit. + * @return {@code True} if unprocessed expired entries remains. + */ + public boolean expire(int amount) { long now = U.currentTimeMillis(); GridCacheVersion obsoleteVer = null; - for (int size = pendingEntries.sizex(); size > 0; size--) { + int limit = (-1 != amount) ? amount : pendingEntries.sizex(); + + for (int cnt = limit; cnt > 0; cnt--) { EntryWrapper e = pendingEntries.firstx(); if (e == null || e.expireTime > now) - return; + return false; // All expired entries are processed. if (pendingEntries.remove(e)) { if (obsoleteVer == null) @@ -158,7 +129,6 @@ public void expire() { if (log.isTraceEnabled()) log.trace("Trying to remove expired entry from cache: " + e); - boolean touch = false; GridCacheEntryEx entry = e.ctx.cache().entryEx(e.key); @@ -181,53 +151,14 @@ public void expire() { entry.context().evicts().touch(entry, null); } } - } - - /** - * Entry cleanup worker. - */ - private class CleanupWorker extends GridWorker { - /** - * Creates cleanup worker. - */ - CleanupWorker() { - super(cctx.gridName(), "ttl-cleanup-worker-" + cctx.name(), cctx.logger(GridCacheTtlManager.class)); - } - - /** {@inheritDoc} */ - @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException { - while (!isCancelled()) { - expire(); - - long waitTime; - - while (true) { - long curTime = U.currentTimeMillis(); - GridCacheTtlManager.EntryWrapper first = pendingEntries.firstx(); - - if (first == null) { - waitTime = 500; - nextExpireTime = curTime + 500; - } - else { - long expireTime = first.expireTime; - - waitTime = expireTime - curTime; - nextExpireTime = expireTime; - } - - synchronized (mux) { - if (pendingEntries.firstx() == first) { - if (waitTime > 0) - mux.wait(waitTime); + if (amount != -1) { + EntryWrapper e = pendingEntries.firstx(); - break; - } - } - } - } + return e != null && e.expireTime <= now; } + + return false; } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerNotificationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerNotificationTest.java index 85a491e3af55d..79f8a65a7eca8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerNotificationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerNotificationTest.java @@ -17,6 +17,8 @@ package org.apache.ignite.internal.processors.cache; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.atomic.AtomicInteger; import javax.cache.expiry.CreatedExpiryPolicy; @@ -24,6 +26,7 @@ import javax.cache.expiry.ExpiryPolicy; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; @@ -44,6 +47,12 @@ * */ public class GridCacheTtlManagerNotificationTest extends GridCommonAbstractTest { + /** Count of caches in multi caches test. */ + private static final int CACHES_CNT = 10; + + /** Prefix for cache name fir multi caches test. */ + private static final String CACHE_PREFIX = "cache-"; + /** IP finder. */ private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); @@ -60,14 +69,30 @@ public class GridCacheTtlManagerNotificationTest extends GridCommonAbstractTest cfg.setDiscoverySpi(discoSpi); + CacheConfiguration[] ccfgs = new CacheConfiguration[CACHES_CNT + 1]; + + ccfgs[0] = createCacheConfiguration(null); + + for (int i = 0; i < CACHES_CNT; i++) + ccfgs[i + 1] = createCacheConfiguration(CACHE_PREFIX + i); + + cfg.setCacheConfiguration(ccfgs); + + return cfg; + } + + /** + * @param name Cache name. + * @return Cache configuration. + */ + private CacheConfiguration createCacheConfiguration(String name) { CacheConfiguration ccfg = new CacheConfiguration(); ccfg.setCacheMode(cacheMode); ccfg.setEagerTtl(true); + ccfg.setName(name); - cfg.setCacheConfiguration(ccfg); - - return cfg; + return ccfg; } /** @@ -104,8 +129,10 @@ public void testThatNotificationWorkAsExpected() throws Exception { } /** - * Add in several threads value to cache with different expiration policy. - * Wait for expiration of keys with small expiration duration. + * Adds in several threads value to cache with different expiration policy. + * Waits for expiration of keys with small expiration duration. + * + * @throws Exception If failed. */ public void testThatNotificationWorkAsExpectedInMultithreadedMode() throws Exception { final CyclicBarrier barrier = new CyclicBarrier(21); @@ -152,16 +179,83 @@ public void testThatNotificationWorkAsExpectedInMultithreadedMode() throws Excep } } + /** + * Adds in several threads value to several caches with different expiration policy. + * Waits for expiration of keys with small expiration duration. + * + * @throws Exception If failed. + */ + public void testThatNotificationWorkAsExpectedManyCaches() throws Exception { + final int smallDuration = 4_000; + + final int cnt = 1_000; + final int cacheCnt = CACHES_CNT; + final int threadCnt = 2; + + final CyclicBarrier barrier = new CyclicBarrier(2 * threadCnt * cacheCnt + 1); + final AtomicInteger keysRangeGen = new AtomicInteger(); + final AtomicInteger evtCnt = new AtomicInteger(0); + final List> caches = new ArrayList<>(cacheCnt); + + try (final Ignite g = startGrid(0)) { + for (int i = 0; i < cacheCnt; i++) { + IgniteCache cache = g.cache("cache-" + i); + + caches.add(cache); + } + + g.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + evtCnt.incrementAndGet(); + + return true; + } + }, EventType.EVT_CACHE_OBJECT_EXPIRED); + + for (int i = 0; i < cacheCnt; i++) { + GridTestUtils.runMultiThreadedAsync( + new CacheFiller(caches.get(i), 100_000, barrier, keysRangeGen, cnt), + threadCnt, + "put-large-duration"); + + GridTestUtils.runMultiThreadedAsync( + new CacheFiller(caches.get(i), smallDuration, barrier, keysRangeGen, cnt), + threadCnt, + "put-small-duration"); + } + + barrier.await(); + + Thread.sleep(1_000); + + barrier.await(); + + for (int i = 0; i < cacheCnt; i++) + assertEquals("Unexpected size of " + CACHE_PREFIX + i, 2 * threadCnt * cnt, caches.get(i).size()); + + Thread.sleep(2 * smallDuration); + + for (int i = 0; i < cacheCnt; i++) + assertEquals("Unexpected size of " + CACHE_PREFIX + i, threadCnt * cnt, caches.get(i).size()); + + assertEquals("Unexpected count of expired entries", threadCnt * CACHES_CNT * cnt, evtCnt.get()); + } + } + /** */ private static class CacheFiller implements Runnable { /** Barrier. */ private final CyclicBarrier barrier; + /** Keys range generator. */ private final AtomicInteger keysRangeGenerator; + /** Count. */ private final int cnt; + /** Cache. */ private final IgniteCache cache; + /** Expiration duration. */ private final int expirationDuration; @@ -187,6 +281,7 @@ private static class CacheFiller implements Runnable { barrier.await(); ExpiryPolicy plc1 = new CreatedExpiryPolicy(new Duration(MILLISECONDS, expirationDuration)); + int keyStart = keysRangeGenerator.getAndIncrement() * cnt; for (int i = keyStart; i < keyStart + cnt; i++) @@ -195,7 +290,7 @@ private static class CacheFiller implements Runnable { barrier.await(); } catch (Exception e) { - e.printStackTrace(); + throw new IgniteException(e); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java index 28cb2da94e4f8..e371dc7228abc 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java @@ -72,7 +72,9 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCacheExpireAndUpdateConsistencyTest.class); + // Eager ttl expiration tests. suite.addTestSuite(GridCacheTtlManagerNotificationTest.class); + suite.addTestSuite(IgniteCacheOnlyOneTtlCleanupThreadExistsTest.class); return suite; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheOnlyOneTtlCleanupThreadExistsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheOnlyOneTtlCleanupThreadExistsTest.java new file mode 100644 index 0000000000000..84f5144aef5b1 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheOnlyOneTtlCleanupThreadExistsTest.java @@ -0,0 +1,102 @@ +/* + * 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.expiry; + +import org.apache.ignite.Ignite; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Checks that one and only one Ttl cleanup worker thread must exists, and only + * if at least one cache with set 'eagerTtl' flag exists. + */ +public class IgniteCacheOnlyOneTtlCleanupThreadExistsTest extends GridCommonAbstractTest { + /** */ + private static final String CACHE_NAME1 = "cache-1"; + + /** */ + private static final String CACHE_NAME2 = "cache-2"; + + /** + * @throws Exception If failed. + */ + public void testOnlyOneTtlCleanupThreadExists() throws Exception { + try (final Ignite g = startGrid(0)) { + checkCleanupThreadExists(false); + + g.createCache(createCacheConfiguration(CACHE_NAME1, false)); + + checkCleanupThreadExists(false); + + g.createCache(createCacheConfiguration(CACHE_NAME2, true)); + + checkCleanupThreadExists(true); + + g.destroyCache(CACHE_NAME1); + + checkCleanupThreadExists(true); + + g.createCache(createCacheConfiguration(CACHE_NAME1, true)); + + checkCleanupThreadExists(true); + + g.destroyCache(CACHE_NAME1); + + checkCleanupThreadExists(true); + + g.destroyCache(CACHE_NAME2); + + checkCleanupThreadExists(false); + } + } + + /** + * @param name Cache name. + * @param eagerTtl Eager ttl falg. + * @return Cache configuration. + */ + private CacheConfiguration createCacheConfiguration(String name, boolean eagerTtl) { + CacheConfiguration ccfg = new CacheConfiguration(); + + ccfg.setEagerTtl(eagerTtl); + ccfg.setName(name); + + return ccfg; + } + + /** + * @param exists {@code True} if ttl cleanup worker thread expected. + * @throws Exception If failed. + */ + private void checkCleanupThreadExists(boolean exists) throws Exception { + int cnt = 0; + + for (Thread t : Thread.getAllStackTraces().keySet()) { + if (t.getName().contains("ttl-cleanup-worker")) + cnt++; + } + + if (cnt > 1) + fail("More then one ttl cleanup worker threads exists"); + + if (exists) + assertEquals("Ttl cleanup thread does not exist", cnt, 1); + else + assertEquals("Ttl cleanup thread exists", cnt, 0); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java index fb82e20dadf45..6c2c4c1734328 100644 --- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java +++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java @@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.GridCacheSwapManager; import org.apache.ignite.internal.processors.cache.GridCacheTtlManager; +import org.apache.ignite.internal.processors.cache.GridCacheSharedTtlCleanupManager; import org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager; import org.apache.ignite.internal.processors.cache.dr.GridOsCacheDrManager; import org.apache.ignite.internal.processors.cache.jta.CacheNoopJtaManager; @@ -68,6 +69,7 @@ public GridCacheTestContext(GridTestKernalContext ctx) throws Exception { new GridCachePartitionExchangeManager(), new CacheAffinitySharedManager(), new GridCacheIoManager(), + new GridCacheSharedTtlCleanupManager(), new CacheNoopJtaManager(), null ), From 39fc5477c19cbe2b2116aaf575a2d0a9c9a618b1 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Tue, 27 Sep 2016 14:48:18 +0300 Subject: [PATCH 177/487] IGNITE-3639: IGFS: Removed BufferedOutputStream from LocalIgfsSecondaryFileSystem because it doesn't give any performance benefit. --- .../local/LocalIgfsSecondaryFileSystem.java | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java index 8dd4fdac14728..1775db6e430d9 100644 --- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java +++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java @@ -34,7 +34,6 @@ import org.apache.ignite.lifecycle.LifecycleAware; import org.jetbrains.annotations.Nullable; -import java.io.BufferedOutputStream; import java.io.File; import java.io.FileInputStream; import java.io.FileNotFoundException; @@ -53,9 +52,6 @@ * Secondary file system which delegates to local file system. */ public class LocalIgfsSecondaryFileSystem implements IgfsSecondaryFileSystem, LifecycleAware { - /** Default buffer size. */ - private static final int DFLT_BUF_SIZE = 8 * 1024; - /** Path that will be added to each passed path. */ private String workDir; @@ -256,13 +252,13 @@ private boolean mkdirs0(@Nullable File dir) { /** {@inheritDoc} */ @Override public OutputStream create(IgfsPath path, boolean overwrite) { - return create0(path, overwrite, DFLT_BUF_SIZE); + return create0(path, overwrite); } /** {@inheritDoc} */ @Override public OutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication, long blockSize, @Nullable Map props) { - return create0(path, overwrite, bufSize); + return create0(path, overwrite); } /** {@inheritDoc} */ @@ -274,10 +270,10 @@ private boolean mkdirs0(@Nullable File dir) { boolean exists = file.exists(); if (exists) - return new BufferedOutputStream(new FileOutputStream(file, true), bufSize); + return new FileOutputStream(file, true); else { if (create) - return create0(path, false, bufSize); + return create0(path, false); else throw new IgfsPathNotFoundException("Failed to append to file because it doesn't exist: " + path); } @@ -389,11 +385,10 @@ private IgfsPath igfsPath(File f) throws IgfsException { * Internal create routine. * * @param path Path. - * @param overwrite Overwirte flag. - * @param bufSize Buffer size. + * @param overwrite Overwrite flag. * @return Output stream. */ - private OutputStream create0(IgfsPath path, boolean overwrite, int bufSize) { + private OutputStream create0(IgfsPath path, boolean overwrite) { File file = fileForPath(path); boolean exists = file.exists(); @@ -411,7 +406,7 @@ private OutputStream create0(IgfsPath path, boolean overwrite, int bufSize) { } try { - return new BufferedOutputStream(new FileOutputStream(file), bufSize); + return new FileOutputStream(file); } catch (IOException e) { throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", overwrite=" + overwrite + ']'); From 5cffd3c3d6cb006e3745c314d6f85a066e6a0f06 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Tue, 27 Sep 2016 15:13:21 +0300 Subject: [PATCH 178/487] IGNITE-3661: First attempt to move ignored and flaky tests into a single suite. Applied to web-session module. --- .../ignite/testframework/IgniteTestSuite.java | 122 +++++++++++++----- .../junits/GridAbstractTest.java | 22 +++- .../ignite/testsuites/IgniteIgnore.java | 11 +- modules/ignored-tests/README.txt | 4 + modules/ignored-tests/pom.xml | 93 +++++++++++++ .../testsuites/IgniteIgnoredTestSuite.java | 41 ++---- .../ignite/testsuites/package-info.java | 22 ++++ .../IgniteWebSessionSelfTestSuite.java | 68 +--------- .../WebSessionReplicatedSelfTest.java | 28 ++++ .../WebSessionReplicatedV1SelfTest.java | 28 ++++ .../websession/WebSessionSelfTest.java | 2 + .../WebSessionTransactionalSelfTest.java | 48 +++++++ .../WebSessionTransactionalV1SelfTest.java | 28 ++++ .../websession/WebSessionV1SelfTest.java | 28 ++++ pom.xml | 7 + 15 files changed, 423 insertions(+), 129 deletions(-) create mode 100644 modules/ignored-tests/README.txt create mode 100644 modules/ignored-tests/pom.xml rename modules/{core => ignored-tests}/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java (56%) create mode 100644 modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/package-info.java create mode 100644 modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionReplicatedSelfTest.java create mode 100644 modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionReplicatedV1SelfTest.java create mode 100644 modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionTransactionalSelfTest.java create mode 100644 modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionTransactionalV1SelfTest.java create mode 100644 modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionV1SelfTest.java diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java index 2828065ea68ff..4153600fdb08a 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java @@ -20,6 +20,8 @@ import junit.framework.Test; import junit.framework.TestCase; import junit.framework.TestSuite; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.testframework.junits.GridAbstractTest; import org.apache.ignite.testsuites.IgniteIgnore; import org.jetbrains.annotations.Nullable; import org.junit.internal.MethodSorter; @@ -94,16 +96,6 @@ public IgniteTestSuite(@Nullable Class theClass, @Nullable S /** {@inheritDoc} */ @Override public void addTestSuite(Class testClass) { - addTestSuite(testClass, false); - } - - /** - * Add test class to the suite. - * - * @param testClass Test class. - * @param ignoredOnly Ignore only flag. - */ - public void addTestSuite(Class testClass, boolean ignoredOnly) { addTest(new IgniteTestSuite(testClass, ignoredOnly)); } @@ -133,6 +125,7 @@ private void addTestsFromTestCase(Class theClass) { for(List names = new ArrayList<>(); Test.class.isAssignableFrom(superCls); superCls = superCls.getSuperclass()) { + Method[] methods = MethodSorter.getDeclaredMethods(superCls); for (Method each : methods) { @@ -147,45 +140,110 @@ private void addTestsFromTestCase(Class theClass) { } /** - * @param method test method - * @param names test name list - * @param theClass test class + * Add test method. + * + * @param m Test method. + * @param names Test name list. + * @param theClass Test class. + * @return Whether test method was added. */ - private boolean addTestMethod(Method method, List names, Class theClass) { - String name = method.getName(); + private boolean addTestMethod(Method m, List names, Class theClass) { + String name = m.getName(); + + if (names.contains(name)) + return false; + + if (!isPublicTestMethod(m)) { + if (isTestMethod(m)) + addTest(warning("Test method isn't public: " + m.getName() + "(" + theClass.getCanonicalName() + ")")); + + return false; + } + + names.add(name); + + boolean hasIgnore = m.isAnnotationPresent(IgniteIgnore.class); + + if (ignoredOnly) { + if (hasIgnore) { + IgniteIgnore ignore = m.getAnnotation(IgniteIgnore.class); + + String reason = ignore.value(); + + if (F.isEmpty(reason)) + throw new IllegalArgumentException("Reason is not set for ignored test [class=" + + theClass.getName() + ", method=" + name + ']'); - if(!names.contains(name) && canAddMethod(method)) { - if(!Modifier.isPublic(method.getModifiers())) - addTest(warning("Test method isn\'t public: " + method.getName() + "(" + - theClass.getCanonicalName() + ")")); - else { - names.add(name); + Test test = createTest(theClass, name); + if (ignore.forceFailure()) { + if (test instanceof GridAbstractTest) + ((GridAbstractTest)test).forceFailure(ignore.value()); + else + test = new ForcedFailure(name, ignore.value()); + } + + addTest(test); + + return true; + } + } + else { + if (!hasIgnore) { addTest(createTest(theClass, name)); return true; } } + return false; } /** - * Check whether method should be ignored. + * Check whether this is a test method. * - * @param method Method. - * @return {@code True} if it should be ignored. + * @param m Method. + * @return {@code True} if this is a test method. */ - protected boolean canAddMethod(Method method) { - boolean res = method.getParameterTypes().length == 0 && method.getName().startsWith("test") - && method.getReturnType().equals(Void.TYPE); + private static boolean isTestMethod(Method m) { + return m.getParameterTypes().length == 0 && + m.getName().startsWith("test") && + m.getReturnType().equals(Void.TYPE); + } - if (res) { - // If method signature and name matches check if it is ignored or not. - boolean hasIgnore = method.isAnnotationPresent(IgniteIgnore.class); + /** + * Check whether this is a public test method. + * + * @param m Method. + * @return {@code True} if this is a public test method. + */ + private static boolean isPublicTestMethod(Method m) { + return isTestMethod(m) && Modifier.isPublic(m.getModifiers()); + } - res = hasIgnore == ignoredOnly; + /** + * Test case simulating failure. + */ + private static class ForcedFailure extends TestCase { + /** Message. */ + private final String msg; + + /** + * Constructor. + * + * @param name Name. + * @param msg Message. + */ + private ForcedFailure(String name, String msg) { + super(name); + + this.msg = msg; } - return res; + /** {@inheritDoc} */ + @Override protected void runTest() { + fail("Forced failure: " + msg + " (extend " + GridAbstractTest.class.getSimpleName() + + " for better output)."); + } } } diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java index 3910ce42ac02a..8d6fd07d90e62 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java @@ -155,6 +155,12 @@ public abstract class GridAbstractTest extends TestCase { /** Starting grid name. */ protected static final ThreadLocal startingGrid = new ThreadLocal<>(); + /** Force failure flag. */ + private boolean forceFailure; + + /** Force failure message. */ + private String forceFailureMsg; + /** * */ @@ -1752,12 +1758,26 @@ protected IgniteClosure errorHandler() { return null; } + /** + * Force test failure. + * + * @param msg Message. + */ + public void forceFailure(@Nullable String msg) { + forceFailure = true; + + forceFailureMsg = msg; + } + /** * @throws Throwable If failed. */ @SuppressWarnings({"ProhibitedExceptionDeclared"}) private void runTestInternal() throws Throwable { - super.runTest(); + if (forceFailure) + fail("Forced failure: " + forceFailureMsg); + else + super.runTest(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgnore.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgnore.java index ac9a88532ed12..dbb1d7a01326a 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgnore.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgnore.java @@ -26,10 +26,15 @@ * Annotation which indicates that the test is ignored. */ @Retention(RetentionPolicy.RUNTIME) -@Target({ElementType.METHOD, ElementType.TYPE}) +@Target({ElementType.METHOD}) public @interface IgniteIgnore { /** - * The optional reason why the test is ignored. + * Reason for ignore (usually link to JIRA ticket). */ - String value() default ""; + String value(); + + /** + * Whether test should be failed immediately. Useful when test hangs or consumes a lot of time. + */ + boolean forceFailure() default false; } diff --git a/modules/ignored-tests/README.txt b/modules/ignored-tests/README.txt new file mode 100644 index 0000000000000..70f728dd170b6 --- /dev/null +++ b/modules/ignored-tests/README.txt @@ -0,0 +1,4 @@ +Apache Ignite Ignored Tests +------------------------ + +Special module containing ignored and flaky tests grouped in a single test suite. \ No newline at end of file diff --git a/modules/ignored-tests/pom.xml b/modules/ignored-tests/pom.xml new file mode 100644 index 0000000000000..a82a5bb6db23c --- /dev/null +++ b/modules/ignored-tests/pom.xml @@ -0,0 +1,93 @@ + + + + + + + 4.0.0 + + + org.apache.ignite + ignite-parent + 1 + ../../parent + + + ignite-ignored-tests + 1.7.0-SNAPSHOT + http://ignite.apache.org + + + + org.apache.ignite + ignite-core + ${project.version} + + + + org.apache.ignite + ignite-core + ${project.version} + test-jar + test + + + + org.apache.ignite + ignite-log4j + ${project.version} + + + + org.apache.ignite + ignite-spring + ${project.version} + test + + + + org.apache.ignite + ignite-web + ${project.version} + + + + org.apache.ignite + ignite-web + ${project.version} + test-jar + test + + + + org.eclipse.jetty + jetty-servlets + ${jetty.version} + test + + + + org.eclipse.jetty + jetty-webapp + ${jetty.version} + test + + + diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java similarity index 56% rename from modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java rename to modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java index c3ec5e4c0f235..f6ce3e3f09444 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java +++ b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java @@ -18,8 +18,13 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.internal.websession.WebSessionReplicatedSelfTest; +import org.apache.ignite.internal.websession.WebSessionReplicatedV1SelfTest; +import org.apache.ignite.internal.websession.WebSessionSelfTest; +import org.apache.ignite.internal.websession.WebSessionTransactionalSelfTest; +import org.apache.ignite.internal.websession.WebSessionTransactionalV1SelfTest; +import org.apache.ignite.internal.websession.WebSessionV1SelfTest; import org.apache.ignite.testframework.IgniteTestSuite; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** * Special test suite with ignored tests. @@ -30,34 +35,16 @@ public class IgniteIgnoredTestSuite extends TestSuite { * @throws Exception Thrown in case of the failure. */ public static TestSuite suite() throws Exception { - IgniteTestSuite suite = new IgniteTestSuite("Ignite Ignored Test Suite"); + IgniteTestSuite suite = new IgniteTestSuite(null, "Ignite Ignored Test Suite", true); - suite.addTestSuite(SampleTestClass.class, true); + /* --- WEB SESSIONS --- */ + suite.addTestSuite(WebSessionSelfTest.class); + suite.addTestSuite(WebSessionTransactionalSelfTest.class); + suite.addTestSuite(WebSessionReplicatedSelfTest.class); + suite.addTestSuite(WebSessionV1SelfTest.class); + suite.addTestSuite(WebSessionTransactionalV1SelfTest.class); + suite.addTestSuite(WebSessionReplicatedV1SelfTest.class); return suite; } - - /** - * Sample test class. To be removed once the very first really ignored test class is there. - */ - public static class SampleTestClass extends GridCommonAbstractTest { - /** - * Test 1. - * - * @throws Exception If failed. - */ - public void testMethod1() throws Exception { - System.out.println("Normal test method called."); - } - - /** - * Test 2. - * - * @throws Exception If failed. - */ - @IgniteIgnore - public void testMethod2() throws Exception { - System.out.println("Ignored method called."); - } - } } diff --git a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/package-info.java b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/package-info.java new file mode 100644 index 0000000000000..cb71478f97828 --- /dev/null +++ b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * + * Contains internal tests or test related classes and interfaces. + */ +package org.apache.ignite.testsuites; \ No newline at end of file diff --git a/modules/web/src/test/java/org/apache/ignite/internal/websession/IgniteWebSessionSelfTestSuite.java b/modules/web/src/test/java/org/apache/ignite/internal/websession/IgniteWebSessionSelfTestSuite.java index 1d151277ebd1f..e1d5c3b757be6 100644 --- a/modules/web/src/test/java/org/apache/ignite/internal/websession/IgniteWebSessionSelfTestSuite.java +++ b/modules/web/src/test/java/org/apache/ignite/internal/websession/IgniteWebSessionSelfTestSuite.java @@ -19,6 +19,7 @@ import junit.framework.TestSuite; import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.IgniteTestSuite; import static org.apache.ignite.IgniteSystemProperties.IGNITE_OVERRIDE_MCAST_GRP; @@ -32,7 +33,7 @@ public class IgniteWebSessionSelfTestSuite extends TestSuite { * @throws Exception Thrown in case of the failure. */ public static TestSuite suite() throws Exception { - TestSuite suite = new TestSuite("Ignite Web Sessions Test Suite"); + TestSuite suite = new IgniteTestSuite("Ignite Web Sessions Test Suite"); suite.addTestSuite(WebSessionSelfTest.class); suite.addTestSuite(WebSessionTransactionalSelfTest.class); @@ -48,69 +49,4 @@ public static TestSuite suite() throws Exception { return suite; } - - /** - * Tests web sessions with TRANSACTIONAL cache. - */ - public static class WebSessionTransactionalSelfTest extends WebSessionSelfTest { - /** {@inheritDoc} */ - @Override protected String getCacheName() { - return "partitioned_tx"; - } - - /** {@inheritDoc} */ - @Override public void testRestarts() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-810"); - } - - /** {@inheritDoc} */ - @Override public void testInvalidatedSession() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-810"); - } - - /** {@inheritDoc} */ - @Override public void testClientReconnectRequest() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-810"); - } - } - - /** - * Tests web sessions with REPLICATED cache. - */ - public static class WebSessionReplicatedSelfTest extends WebSessionSelfTest { - /** {@inheritDoc} */ - @Override protected String getCacheName() { - return "replicated"; - } - } - - /** - * Old version test. - */ - public static class WebSessionV1SelfTest extends WebSessionSelfTest { - /** {@inheritDoc} */ - @Override protected boolean keepBinary() { - return false; - } - } - - /** - * Tests web sessions with TRANSACTIONAL cache in compatibility mode. - */ - public static class WebSessionTransactionalV1SelfTest extends WebSessionTransactionalSelfTest { - /** {@inheritDoc} */ - @Override protected boolean keepBinary() { - return false; - } - } - - /** - * Tests web sessions with REPLICATED cache in compatibility mode. - */ - public static class WebSessionReplicatedV1SelfTest extends WebSessionReplicatedSelfTest { - /** {@inheritDoc} */ - @Override protected boolean keepBinary() { - return false; - } - } } \ No newline at end of file diff --git a/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionReplicatedSelfTest.java b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionReplicatedSelfTest.java new file mode 100644 index 0000000000000..638fdccb878d8 --- /dev/null +++ b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionReplicatedSelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.websession; + +/** + * Tests web sessions with REPLICATED cache. + */ +public class WebSessionReplicatedSelfTest extends WebSessionSelfTest { + /** {@inheritDoc} */ + @Override protected String getCacheName() { + return "replicated"; + } +} diff --git a/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionReplicatedV1SelfTest.java b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionReplicatedV1SelfTest.java new file mode 100644 index 0000000000000..ba69d1322f56a --- /dev/null +++ b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionReplicatedV1SelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.websession; + +/** + * Tests web sessions with REPLICATED cache in compatibility mode. + */ +public class WebSessionReplicatedV1SelfTest extends WebSessionReplicatedSelfTest { + /** {@inheritDoc} */ + @Override protected boolean keepBinary() { + return false; + } +} \ No newline at end of file diff --git a/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java index 0ab113077f284..5138e3a5a8ed6 100644 --- a/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java +++ b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java @@ -46,6 +46,7 @@ import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.testsuites.IgniteIgnore; import org.eclipse.jetty.security.HashLoginService; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.servlet.ServletHolder; @@ -88,6 +89,7 @@ public void testSingleRequest() throws Exception { /** * @throws Exception If failed. */ + @IgniteIgnore("https://issues.apache.org/jira/browse/IGNITE-3663") public void testSessionRenewalDuringLogin() throws Exception { testSessionRenewalDuringLogin("/modules/core/src/test/config/websession/example-cache.xml"); } diff --git a/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionTransactionalSelfTest.java b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionTransactionalSelfTest.java new file mode 100644 index 0000000000000..4cc1a637f7aad --- /dev/null +++ b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionTransactionalSelfTest.java @@ -0,0 +1,48 @@ +/* + * 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.websession; + +import org.apache.ignite.testsuites.IgniteIgnore; + +/** + * Tests web sessions with TRANSACTIONAL cache. + */ +public class WebSessionTransactionalSelfTest extends WebSessionSelfTest { + /** {@inheritDoc} */ + @Override protected String getCacheName() { + return "partitioned_tx"; + } + + /** {@inheritDoc} */ + @IgniteIgnore(value = "https://issues.apache.org/jira/browse/IGNITE-810", forceFailure = true) + @Override public void testRestarts() throws Exception { + // No-op. + } + + /** {@inheritDoc} */ + @IgniteIgnore(value = "https://issues.apache.org/jira/browse/IGNITE-810", forceFailure = true) + @Override public void testInvalidatedSession() throws Exception { + // No-op. + } + + /** {@inheritDoc} */ + @IgniteIgnore(value = "https://issues.apache.org/jira/browse/IGNITE-810", forceFailure = true) + @Override public void testClientReconnectRequest() throws Exception { + // No-op. + } +} diff --git a/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionTransactionalV1SelfTest.java b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionTransactionalV1SelfTest.java new file mode 100644 index 0000000000000..6f94471a753c7 --- /dev/null +++ b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionTransactionalV1SelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.websession; + +/** + * Tests web sessions with TRANSACTIONAL cache in compatibility mode. + */ +public class WebSessionTransactionalV1SelfTest extends WebSessionTransactionalSelfTest { + /** {@inheritDoc} */ + @Override protected boolean keepBinary() { + return false; + } +} diff --git a/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionV1SelfTest.java b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionV1SelfTest.java new file mode 100644 index 0000000000000..791bec02285a9 --- /dev/null +++ b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionV1SelfTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.websession; + +/** + * Tests the correctness of web sessions caching functionality in compatibility mode. + */ +public class WebSessionV1SelfTest extends WebSessionSelfTest { + /** {@inheritDoc} */ + @Override protected boolean keepBinary() { + return false; + } +} diff --git a/pom.xml b/pom.xml index 2c7bad19c745c..36051b4aca0b4 100644 --- a/pom.xml +++ b/pom.xml @@ -495,6 +495,13 @@ + + ignored-tests + + modules/ignored-tests + + + examples From c8dc92ecc8a5d76e68d2d75f12158e0a581a0326 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 27 Sep 2016 15:17:05 +0300 Subject: [PATCH 179/487] ignite-3973 In TcpDiscoveryMulticastIpFinder.requestAddresses wait full timeout for remote addresses --- .../ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java | 3 --- .../discovery/tcp/TcpClientDiscoverySpiMulticastTest.java | 2 +- .../ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java | 2 +- .../multicast/TcpDiscoveryMulticastIpFinderSelfTest.java | 6 +++--- 4 files changed, 5 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java index d5b3dae3270a5..5bbe90edbafc1 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java @@ -618,9 +618,6 @@ private T2, Boolean> requestAddresses(InetAddress U.close(sock); } - if (rmtAddrs.size() > locNodeAddrs.size()) - break; - if (i < addrReqAttempts - 1) // Wait some time before re-sending address request. U.sleep(200); } diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMulticastTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMulticastTest.java index 27ce883e8cde4..ba8fe73149fc1 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMulticastTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMulticastTest.java @@ -58,7 +58,7 @@ public class TcpClientDiscoverySpiMulticastTest extends GridCommonAbstractTest { TcpDiscoveryMulticastIpFinder ipFinder = new TcpDiscoveryMulticastIpFinder(); - ipFinder.setAddressRequestAttempts(10); + ipFinder.setAddressRequestAttempts(5); spi.setIpFinder(ipFinder); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java index 035c6fefbe47c..1ce98a538657e 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java @@ -181,7 +181,7 @@ public TcpDiscoverySelfTest() throws Exception { else if (gridName.contains("MulticastIpFinder")) { TcpDiscoveryMulticastIpFinder finder = new TcpDiscoveryMulticastIpFinder(); - finder.setAddressRequestAttempts(10); + finder.setAddressRequestAttempts(5); finder.setMulticastGroup(GridTestUtils.getNextMulticastGroup(getClass())); finder.setMulticastPort(GridTestUtils.getNextMulticastPort(getClass())); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinderSelfTest.java index 90fdb0a76c22e..29ed595f38189 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinderSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinderSelfTest.java @@ -59,19 +59,19 @@ public void testExchange() throws Exception { try { ipFinder1 = ipFinder(); ipFinder1.setResponseWaitTime(1000); - ipFinder1.setAddressRequestAttempts(10); + ipFinder1.setAddressRequestAttempts(5); ipFinder2 = new TcpDiscoveryMulticastIpFinder(); ipFinder2.setResponseWaitTime(1000); - ipFinder2.setAddressRequestAttempts(10); + ipFinder2.setAddressRequestAttempts(5); ipFinder2.setMulticastGroup(ipFinder1.getMulticastGroup()); ipFinder2.setMulticastPort(ipFinder1.getMulticastPort()); ipFinder3 = new TcpDiscoveryMulticastIpFinder(); ipFinder3.setResponseWaitTime(1000); - ipFinder3.setAddressRequestAttempts(10); + ipFinder3.setAddressRequestAttempts(5); ipFinder3.setMulticastGroup(ipFinder1.getMulticastGroup()); ipFinder3.setMulticastPort(ipFinder1.getMulticastPort()); From 8ba2b947895cabdddb8633a39063c8739c18ad1b Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 27 Sep 2016 16:07:52 +0300 Subject: [PATCH 180/487] ignite-3967 Do not use GridBoundedConcurrentOrderedMap.clear --- .../apache/ignite/internal/IgniteKernal.java | 2 +- .../discovery/GridDiscoveryManager.java | 73 ++++++++++--------- .../util/GridBoundedConcurrentOrderedMap.java | 5 ++ .../IgniteClientReconnectCacheTest.java | 35 +++++++++ ...ridDiscoveryManagerAliveCacheSelfTest.java | 4 +- .../GridDiscoveryManagerSelfTest.java | 6 +- 6 files changed, 85 insertions(+), 40 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 6c5a6282420b9..e0a36a7e19a41 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -1126,7 +1126,7 @@ else if (e instanceof IgniteCheckedException) ackStart(rtBean); if (!isDaemon()) - ctx.discovery().ackTopology(); + ctx.discovery().ackTopology(localNode().order()); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java index bbf3ebd2a2fc3..345cfb48b6d7f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java @@ -42,7 +42,6 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; @@ -81,7 +80,7 @@ import org.apache.ignite.internal.processors.service.GridServiceProcessor; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.F0; -import org.apache.ignite.internal.util.GridBoundedConcurrentOrderedMap; +import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -113,15 +112,16 @@ import org.apache.ignite.spi.discovery.DiscoverySpiListener; import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator; import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport; -import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCOVERY_HISTORY_SIZE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE; -import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2; +import static org.apache.ignite.IgniteSystemProperties.getInteger; import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED; import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED; import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; @@ -169,7 +169,7 @@ public class GridDiscoveryManager extends GridManagerAdapter { private static final String PREFIX = "Topology snapshot"; /** Discovery cached history size. */ - protected static final int DISCOVERY_HISTORY_SIZE = 100; + static final int DISCOVERY_HISTORY_SIZE = getInteger(IGNITE_DISCOVERY_HISTORY_SIZE, 500); /** Predicate filtering out daemon nodes. */ private static final IgnitePredicate FILTER_DAEMON = new P1() { @@ -210,8 +210,8 @@ public class GridDiscoveryManager extends GridManagerAdapter { private final AtomicBoolean lastSegChkRes = new AtomicBoolean(true); /** Topology cache history. */ - private final ConcurrentNavigableMap discoCacheHist = - new GridBoundedConcurrentOrderedMap<>(DISCOVERY_HISTORY_SIZE); + private final GridBoundedConcurrentLinkedHashMap discoCacheHist = + new GridBoundedConcurrentLinkedHashMap<>(DISCOVERY_HISTORY_SIZE); /** Topology snapshots history. */ private volatile Map> topHist = new HashMap<>(); @@ -599,7 +599,11 @@ else if (type == EVT_CLIENT_NODE_DISCONNECTED) { registeredCaches.clear(); - discoCacheHist.clear(); + for (AffinityTopologyVersion histVer : discoCacheHist.keySet()) { + Object rmvd = discoCacheHist.remove(histVer); + + assert rmvd != null : histVer; + } topHist.clear(); @@ -1199,9 +1203,11 @@ private static int cpus(Collection nodes) { /** * Prints the latest topology info into log taking into account logging/verbosity settings. + * + * @param topVer Topology version. */ - public void ackTopology() { - ackTopology(topSnap.get().topVer.topologyVersion(), false); + public void ackTopology(long topVer) { + ackTopology(topVer, false); } /** @@ -1213,7 +1219,22 @@ public void ackTopology() { private void ackTopology(long topVer, boolean throttle) { assert !isLocDaemon; - DiscoCache discoCache = discoCache(); + DiscoCache discoCache = discoCacheHist.get(new AffinityTopologyVersion(topVer)); + + if (discoCache == null) { + String msg = "Failed to resolve nodes topology [topVer=" + topVer + + ", hist=" + discoCacheHist.keySet() + ']'; + + if (log.isQuiet()) + U.quiet(false, msg); + + if (log.isDebugEnabled()) + log.debug(msg); + else if (log.isInfoEnabled()) + log.info(msg); + + return; + } Collection rmtNodes = discoCache.remoteNodes(); @@ -1237,7 +1258,7 @@ private void ackTopology(long topVer, boolean throttle) { double heap = U.heapSize(allNodes, 2); if (log.isQuiet()) - U.quiet(false, topologySnapshotMessage(srvNodes.size(), clientNodes.size(), totalCpus, heap)); + U.quiet(false, topologySnapshotMessage(topVer, srvNodes.size(), clientNodes.size(), totalCpus, heap)); if (log.isDebugEnabled()) { String dbg = ""; @@ -1281,19 +1302,20 @@ private void ackTopology(long topVer, boolean throttle) { log.debug(dbg); } else if (log.isInfoEnabled()) - log.info(topologySnapshotMessage(srvNodes.size(), clientNodes.size(), totalCpus, heap)); + log.info(topologySnapshotMessage(topVer, srvNodes.size(), clientNodes.size(), totalCpus, heap)); } /** + * @param topVer Topology version. * @param srvNodesNum Server nodes number. * @param clientNodesNum Client nodes number. * @param totalCpus Total cpu number. * @param heap Heap size. * @return Topology snapshot message. */ - private String topologySnapshotMessage(int srvNodesNum, int clientNodesNum, int totalCpus, double heap) { + private String topologySnapshotMessage(long topVer, int srvNodesNum, int clientNodesNum, int totalCpus, double heap) { return PREFIX + " [" + - (discoOrdered ? "ver=" + topSnap.get().topVer.topologyVersion() + ", " : "") + + (discoOrdered ? "ver=" + topVer + ", " : "") + "servers=" + srvNodesNum + ", clients=" + clientNodesNum + ", CPUs=" + totalCpus + @@ -1506,7 +1528,7 @@ public IgniteInternalFuture topologyFuture(final long awaitVer) { * * @return Discovery collection cache. */ - public DiscoCache discoCache() { + private DiscoCache discoCache() { Snapshot cur = topSnap.get(); assert cur != null; @@ -1533,15 +1555,6 @@ public Collection allNodes() { return discoCache().allNodes(); } - /** - * Gets topology grouped by node versions. - * - * @return Version to collection of nodes map. - */ - public NavigableMap> topologyVersionMap() { - return discoCache().versionsMap(); - } - /** @return Full topology size. */ public int size() { return discoCache().allNodes().size(); @@ -1781,16 +1794,6 @@ private DiscoCache resolveDiscoCache(@Nullable String cacheName, AffinityTopolog DiscoCache cache = AffinityTopologyVersion.NONE.equals(topVer) || topVer.equals(snap.topVer) ? snap.discoCache : discoCacheHist.get(topVer); - if (cache == null) { - // Find the eldest acceptable discovery cache. - Map.Entry eldest = discoCacheHist.firstEntry(); - - if (eldest != null) { - if (topVer.compareTo(eldest.getKey()) < 0) - cache = eldest.getValue(); - } - } - if (cache == null) { throw new IgniteException("Failed to resolve nodes topology [cacheName=" + cacheName + ", topVer=" + topVer + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentOrderedMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentOrderedMap.java index 3f6db30a08d70..33b2e6032bc04 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentOrderedMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedConcurrentOrderedMap.java @@ -236,4 +236,9 @@ private void onPut() { return rmvd; } + + /** {@inheritDoc} */ + @Override public void clear() { + throw new UnsupportedOperationException(); + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java index ad6c46f284b74..0f0165b0f04f9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java @@ -27,6 +27,7 @@ import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import javax.cache.CacheException; import junit.framework.AssertionFailedError; @@ -1084,6 +1085,21 @@ public void testReconnectClusterRestartMultinode() throws Exception { * @throws Exception If failed. */ public void testReconnectMultinode() throws Exception { + reconnectMultinode(false); + } + + /** + * @throws Exception If failed. + */ + public void testReconnectMultinodeLongHistory() throws Exception { + reconnectMultinode(true); + } + + /** + * @param longHist If {@code true} generates many discovery events to overflow events history. + * @throws Exception If failed. + */ + private void reconnectMultinode(boolean longHist) throws Exception { grid(0).createCache(new CacheConfiguration<>()); clientMode = true; @@ -1100,6 +1116,25 @@ public void testReconnectMultinode() throws Exception { clients.add(client); } + if (longHist) { + // Generate many discovery events to overflow discovery events history. + final AtomicInteger nodeIdx = new AtomicInteger(SRV_CNT + CLIENTS); + + GridTestUtils.runMultiThreaded(new Callable() { + @Override public Void call() throws Exception { + int idx = nodeIdx.incrementAndGet(); + + for (int i = 0; i < 25; i++) { + startGrid(idx); + + stopGrid(idx); + } + + return null; + } + }, 4, "restart-thread"); + } + int nodes = SRV_CNT + CLIENTS; int srvNodes = SRV_CNT; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java index 50bcc415b8860..390c83e46bba1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java @@ -219,7 +219,9 @@ private void validateAlives() { long currVer = discoMgr.topologyVersion(); - for (long v = currVer; v > currVer - GridDiscoveryManager.DISCOVERY_HISTORY_SIZE && v > 0; v--) { + long startVer = discoMgr.localNode().order(); + + for (long v = currVer; v > currVer - GridDiscoveryManager.DISCOVERY_HISTORY_SIZE && v >= startVer; v--) { F.forAll(discoMgr.aliveCacheNodes(null, new AffinityTopologyVersion(v)), new IgnitePredicate() { @Override public boolean apply(ClusterNode e) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java index 7912ee1293c86..c9179d44c0060 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java @@ -88,15 +88,15 @@ public abstract class GridDiscoveryManagerSelfTest extends GridCommonAbstractTes public void testHasNearCache() throws Exception { IgniteKernal g0 = (IgniteKernal)startGrid(0); // PARTITIONED_ONLY cache. - AffinityTopologyVersion zero = new AffinityTopologyVersion(0); + AffinityTopologyVersion none = new AffinityTopologyVersion(-1); AffinityTopologyVersion one = new AffinityTopologyVersion(1); AffinityTopologyVersion two = new AffinityTopologyVersion(2, 2); AffinityTopologyVersion three = new AffinityTopologyVersion(3); AffinityTopologyVersion four = new AffinityTopologyVersion(4); AffinityTopologyVersion five = new AffinityTopologyVersion(5); - assertFalse(g0.context().discovery().hasNearCache(CACHE_NAME, zero)); - assertFalse(g0.context().discovery().hasNearCache(null, zero)); + assertFalse(g0.context().discovery().hasNearCache(CACHE_NAME, none)); + assertFalse(g0.context().discovery().hasNearCache(null, none)); assertFalse(g0.context().discovery().hasNearCache(CACHE_NAME, one)); assertFalse(g0.context().discovery().hasNearCache(null, one)); From 7f8281cd191ea576a8d6358b53fb13e4344cb9d5 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Tue, 27 Sep 2016 16:37:40 +0300 Subject: [PATCH 181/487] IGNITE-3978: Applied "IgniteIgnore" annotation to failing S3 tests. This closes #1123. --- .../s3/S3CheckpointManagerSelfTest.java | 3 ++ .../s3/S3CheckpointSpiSelfTest.java | 4 ++ .../s3/S3CheckpointSpiStartStopSelfTest.java | 7 ++++ .../s3/S3SessionCheckpointSelfTest.java | 2 + .../s3/TcpDiscoveryS3IpFinderSelfTest.java | 7 ++++ .../ignite/testsuites/IgniteS3TestSuite.java | 15 +++---- .../ignite/testframework/IgniteTestSuite.java | 41 +++++++++++++++++-- modules/ignored-tests/pom.xml | 14 +++++++ .../testsuites/IgniteIgnoredTestSuite.java | 21 ++++------ 9 files changed, 90 insertions(+), 24 deletions(-) diff --git a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointManagerSelfTest.java b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointManagerSelfTest.java index db09e337c19bc..a90323e7dd4c7 100644 --- a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointManagerSelfTest.java +++ b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointManagerSelfTest.java @@ -21,6 +21,7 @@ import com.amazonaws.auth.BasicAWSCredentials; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManagerAbstractSelfTest; +import org.apache.ignite.testsuites.IgniteIgnore; import org.apache.ignite.testsuites.IgniteS3TestSuite; /** @@ -50,6 +51,7 @@ public class S3CheckpointManagerSelfTest extends GridCheckpointManagerAbstractSe /** * @throws Exception Thrown if any exception occurs. */ + @IgniteIgnore("https://issues.apache.org/jira/browse/IGNITE-2420") public void testS3Based() throws Exception { retries = 6; @@ -59,6 +61,7 @@ public void testS3Based() throws Exception { /** * @throws Exception Thrown if any exception occurs. */ + @IgniteIgnore("https://issues.apache.org/jira/browse/IGNITE-2420") public void testMultiNodeS3Based() throws Exception { retries = 6; diff --git a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java index 11e5bad029a9c..23abe0642136f 100644 --- a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java +++ b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java @@ -34,6 +34,7 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.spi.GridSpiAbstractTest; import org.apache.ignite.testframework.junits.spi.GridSpiTest; +import org.apache.ignite.testsuites.IgniteIgnore; import org.apache.ignite.testsuites.IgniteS3TestSuite; /** @@ -91,6 +92,7 @@ public class S3CheckpointSpiSelfTest extends GridSpiAbstractTest IGNORE_DFLT = new ThreadLocal() { + @Override protected Boolean initialValue() { + return false; + } + }; + /** Whether to execute only ignored tests. */ - private final boolean ignoredOnly; + private boolean ignoredOnly; /** * Constructor. @@ -54,7 +61,7 @@ public IgniteTestSuite(String name) { * @param theClass TestCase class */ public IgniteTestSuite(Class theClass) { - this(theClass, false); + this(theClass, ignoreDefault()); } /** @@ -74,7 +81,7 @@ public IgniteTestSuite(Class theClass, boolean ignoredOnly) * @param name Test suite name. */ public IgniteTestSuite(Class theClass, String name) { - this(theClass, name, false); + this(theClass, name, ignoreDefault()); } /** @@ -94,6 +101,13 @@ public IgniteTestSuite(@Nullable Class theClass, @Nullable S setName(name); } + /** + * Adds a test to the suite. + */ + @Override public void addTest(Test test) { + super.addTest(test); + } + /** {@inheritDoc} */ @Override public void addTestSuite(Class testClass) { addTest(new IgniteTestSuite(testClass, ignoredOnly)); @@ -122,6 +136,7 @@ private void addTestsFromTestCase(Class theClass) { Class superCls = theClass; int testAdded = 0; + int testIgnored = 0; for(List names = new ArrayList<>(); Test.class.isAssignableFrom(superCls); superCls = superCls.getSuperclass()) { @@ -131,10 +146,12 @@ private void addTestsFromTestCase(Class theClass) { for (Method each : methods) { if (addTestMethod(each, names, theClass)) testAdded++; + else + testIgnored++; } } - if(testAdded == 0) + if(testAdded == 0 && testIgnored == 0) addTest(warning("No tests found in " + theClass.getName())); } } @@ -221,6 +238,22 @@ private static boolean isPublicTestMethod(Method m) { return isTestMethod(m) && Modifier.isPublic(m.getModifiers()); } + /** + * @param val Default value of ignore flag. + */ + public static void ignoreDefault(boolean val) { + IGNORE_DFLT.set(val); + } + + /** + * @return Default value of ignore flag. + */ + private static boolean ignoreDefault() { + Boolean res = IGNORE_DFLT.get(); + + return res != null && res; + } + /** * Test case simulating failure. */ diff --git a/modules/ignored-tests/pom.xml b/modules/ignored-tests/pom.xml index a82a5bb6db23c..cfeacb2283e4c 100644 --- a/modules/ignored-tests/pom.xml +++ b/modules/ignored-tests/pom.xml @@ -62,6 +62,20 @@ test + + org.apache.ignite + ignite-aws + ${project.version} + + + + org.apache.ignite + ignite-aws + ${project.version} + test-jar + test + + org.apache.ignite ignite-web diff --git a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java index f6ce3e3f09444..fa60721c5d0ad 100644 --- a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java +++ b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java @@ -18,12 +18,7 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; -import org.apache.ignite.internal.websession.WebSessionReplicatedSelfTest; -import org.apache.ignite.internal.websession.WebSessionReplicatedV1SelfTest; -import org.apache.ignite.internal.websession.WebSessionSelfTest; -import org.apache.ignite.internal.websession.WebSessionTransactionalSelfTest; -import org.apache.ignite.internal.websession.WebSessionTransactionalV1SelfTest; -import org.apache.ignite.internal.websession.WebSessionV1SelfTest; +import org.apache.ignite.internal.websession.IgniteWebSessionSelfTestSuite; import org.apache.ignite.testframework.IgniteTestSuite; /** @@ -35,15 +30,15 @@ public class IgniteIgnoredTestSuite extends TestSuite { * @throws Exception Thrown in case of the failure. */ public static TestSuite suite() throws Exception { - IgniteTestSuite suite = new IgniteTestSuite(null, "Ignite Ignored Test Suite", true); + IgniteTestSuite.ignoreDefault(true); + + IgniteTestSuite suite = new IgniteTestSuite(null, "Ignite Ignored Test Suite"); + + /* --- AWS --- */ + suite.addTest(IgniteS3TestSuite.suite()); /* --- WEB SESSIONS --- */ - suite.addTestSuite(WebSessionSelfTest.class); - suite.addTestSuite(WebSessionTransactionalSelfTest.class); - suite.addTestSuite(WebSessionReplicatedSelfTest.class); - suite.addTestSuite(WebSessionV1SelfTest.class); - suite.addTestSuite(WebSessionTransactionalV1SelfTest.class); - suite.addTestSuite(WebSessionReplicatedV1SelfTest.class); + suite.addTest(IgniteWebSessionSelfTestSuite.suite()); return suite; } From 2bfa06dd75fa55ad01438fa59d14b864ec95834e Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Wed, 28 Sep 2016 08:22:40 +0700 Subject: [PATCH 182/487] Fixed typo. --- .../examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs index 8c689dc04352f..46524a1698ca9 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs @@ -26,7 +26,7 @@ namespace Apache.Ignite.ExamplesDll.Events /// public class LocalListener : IEventListener { - /** Сount of received events. */ + /** Count of received events. */ private int _eventsReceived; /// From c188c3c4a96eacb85ea8e08f0634288332432c1c Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Wed, 28 Sep 2016 08:46:23 +0700 Subject: [PATCH 183/487] IGNITE-3983 Fixed wrong cache load optimization. Test added. --- .../store/jdbc/CacheAbstractJdbcStore.java | 4 +- .../CacheJdbcPojoStoreAbstractSelfTest.java | 55 ++++++++++++------- ...JdbcPojoStoreBinaryMarshallerSelfTest.java | 12 ++-- 3 files changed, 43 insertions(+), 28 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java index a9a8ce1d0b697..f316676cabae5 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java @@ -893,9 +893,9 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { } futs.add(pool.submit(loadCacheRange(em, clo, upperBound, null, 0))); - } - continue; + continue; + } } catch (SQLException e) { log.warning("Failed to load entries from db in multithreaded mode, will try in single thread " + diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java index 83065f1bbb146..368a28e010c3b 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java @@ -50,31 +50,34 @@ */ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstractTest { /** IP finder. */ - protected static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); /** DB connection URL. */ - protected static final String DFLT_CONN_URL = "jdbc:h2:mem:TestDatabase;DB_CLOSE_DELAY=-1"; + private static final String DFLT_CONN_URL = "jdbc:h2:mem:TestDatabase;DB_CLOSE_DELAY=-1"; /** Organization count. */ - protected static final int ORGANIZATION_CNT = 1000; + private static final int ORGANIZATION_CNT = 1000; /** Person count. */ - protected static final int PERSON_CNT = 100000; + private static final int PERSON_CNT = 100000; /** Test cache name. */ - protected static final String CACHE_NAME = "test-cache"; + private static final String CACHE_NAME = "test-cache"; /** Flag indicating that tests should use transactional cache. */ - protected static boolean transactional; + private static boolean transactional; /** Flag indicating that tests should use primitive classes like java.lang.Integer for keys. */ protected static boolean builtinKeys; /** Flag indicating that classes for keys available on class path or not. */ - protected static boolean noKeyClasses; + private static boolean noKeyClasses; /** Flag indicating that classes for values available on class path or not. */ - protected static boolean noValClasses; + private static boolean noValClasses; + + /** Batch size to load in parallel. */ + private static int parallelLoadThreshold; /** * @return Flag indicating that all internal SQL queries should use escaped identifiers. @@ -221,6 +224,7 @@ protected CacheConfiguration cacheConfiguration() throws Exception { storeFactory.setTypes(storeTypes()); storeFactory.setDataSourceFactory(new H2DataSourceFactory()); // H2 DataSource factory. storeFactory.setSqlEscapeAll(sqlEscapeAll()); + storeFactory.setParallelLoadCacheMinimumThreshold(parallelLoadThreshold); cc.setCacheStoreFactory(storeFactory); cc.setReadThrough(true); @@ -285,13 +289,15 @@ protected void fillSampleDatabase(Connection conn) throws SQLException { * @param noKeyCls {@code True} if keys classes are not on class path. * @param noValCls {@code True} if values classes are not on class path. * @param trn {@code True} if cache should be started in transactional mode. + * @param threshold Load batch size. * @throws Exception If failed to start grid. */ - protected void startTestGrid(boolean builtin, boolean noKeyCls, boolean noValCls, boolean trn) throws Exception { + protected void startTestGrid(boolean builtin, boolean noKeyCls, boolean noValCls, boolean trn, int threshold) throws Exception { builtinKeys = builtin; noKeyClasses = noKeyCls; noValClasses = noValCls; transactional = trn; + parallelLoadThreshold = threshold; startGrid(); } @@ -322,7 +328,16 @@ protected void checkCacheLoadWithSql() { * @throws Exception If failed. */ public void testLoadCache() throws Exception { - startTestGrid(false, false, false, false); + startTestGrid(false, false, false, false, 512); + + checkCacheLoad(); + } + + /** + * @throws Exception If failed. + */ + public void testLoadCacheAll() throws Exception { + startTestGrid(false, false, false, false, ORGANIZATION_CNT + PERSON_CNT + 1); checkCacheLoad(); } @@ -331,7 +346,7 @@ public void testLoadCache() throws Exception { * @throws Exception If failed. */ public void testLoadCacheWithSql() throws Exception { - startTestGrid(false, false, false, false); + startTestGrid(false, false, false, false, 512); checkCacheLoadWithSql(); } @@ -340,7 +355,7 @@ public void testLoadCacheWithSql() throws Exception { * @throws Exception If failed. */ public void testLoadCacheTx() throws Exception { - startTestGrid(false, false, false, true); + startTestGrid(false, false, false, true, 512); checkCacheLoad(); } @@ -349,7 +364,7 @@ public void testLoadCacheTx() throws Exception { * @throws Exception If failed. */ public void testLoadCacheWithSqlTx() throws Exception { - startTestGrid(false, false, false, true); + startTestGrid(false, false, false, true, 512); checkCacheLoadWithSql(); } @@ -358,7 +373,7 @@ public void testLoadCacheWithSqlTx() throws Exception { * @throws Exception If failed. */ public void testLoadCachePrimitiveKeys() throws Exception { - startTestGrid(true, false, false, false); + startTestGrid(true, false, false, false, 512); checkCacheLoad(); } @@ -367,7 +382,7 @@ public void testLoadCachePrimitiveKeys() throws Exception { * @throws Exception If failed. */ public void testLoadCachePrimitiveKeysTx() throws Exception { - startTestGrid(true, false, false, true); + startTestGrid(true, false, false, true, 512); checkCacheLoad(); } @@ -448,7 +463,7 @@ private void checkPutRemove() throws Exception { * @throws Exception If failed. */ public void testPutRemoveBuiltIn() throws Exception { - startTestGrid(true, false, false, false); + startTestGrid(true, false, false, false, 512); checkPutRemove(); } @@ -457,7 +472,7 @@ public void testPutRemoveBuiltIn() throws Exception { * @throws Exception If failed. */ public void testPutRemove() throws Exception { - startTestGrid(false, false, false, false); + startTestGrid(false, false, false, false, 512); checkPutRemove(); } @@ -466,7 +481,7 @@ public void testPutRemove() throws Exception { * @throws Exception If failed. */ public void testPutRemoveTxBuiltIn() throws Exception { - startTestGrid(true, false, false, true); + startTestGrid(true, false, false, true, 512); checkPutRemove(); } @@ -475,7 +490,7 @@ public void testPutRemoveTxBuiltIn() throws Exception { * @throws Exception If failed. */ public void testPutRemoveTx() throws Exception { - startTestGrid(false, false, false, true); + startTestGrid(false, false, false, true, 512); checkPutRemove(); } @@ -484,7 +499,7 @@ public void testPutRemoveTx() throws Exception { * @throws Exception If failed. */ public void testLoadNotRegisteredType() throws Exception { - startTestGrid(false, false, false, false); + startTestGrid(false, false, false, false, 512); IgniteCache c1 = grid().cache(CACHE_NAME); diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java index 14c743cbb3bd5..b6d6fe13ac275 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java @@ -33,7 +33,7 @@ public class CacheJdbcPojoStoreBinaryMarshallerSelfTest extends CacheJdbcPojoSto * @throws Exception If failed. */ public void testLoadCacheNoKeyClasses() throws Exception { - startTestGrid(false, true, false, false); + startTestGrid(false, true, false, false, 512); checkCacheLoad(); } @@ -42,7 +42,7 @@ public void testLoadCacheNoKeyClasses() throws Exception { * @throws Exception If failed. */ public void testLoadCacheNoKeyClassesTx() throws Exception { - startTestGrid(false, true, false, true); + startTestGrid(false, true, false, true, 512); checkCacheLoad(); } @@ -51,7 +51,7 @@ public void testLoadCacheNoKeyClassesTx() throws Exception { * @throws Exception If failed. */ public void testLoadCacheNoValueClasses() throws Exception { - startTestGrid(false, false, true, false); + startTestGrid(false, false, true, false, 512); checkCacheLoad(); } @@ -60,7 +60,7 @@ public void testLoadCacheNoValueClasses() throws Exception { * @throws Exception If failed. */ public void testLoadCacheNoValueClassesTx() throws Exception { - startTestGrid(false, false, true, true); + startTestGrid(false, false, true, true, 512); checkCacheLoad(); } @@ -69,7 +69,7 @@ public void testLoadCacheNoValueClassesTx() throws Exception { * @throws Exception If failed. */ public void testLoadCacheNoKeyAndValueClasses() throws Exception { - startTestGrid(false, true, true, false); + startTestGrid(false, true, true, false, 512); checkCacheLoad(); } @@ -78,7 +78,7 @@ public void testLoadCacheNoKeyAndValueClasses() throws Exception { * @throws Exception If failed. */ public void testLoadCacheNoKeyAndValueClassesTx() throws Exception { - startTestGrid(false, true, true, true); + startTestGrid(false, true, true, true, 512); checkCacheLoad(); } From 89c30c8b0be6915d2399be508ddcd9eb439a9aaa Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Wed, 28 Sep 2016 08:57:45 +0700 Subject: [PATCH 184/487] IGNITE-3965 @GridInternal tasks should run via standart LoadBalancingSpi. Added test. --- .../ignite/compute/ComputeTaskSpis.java | 8 +- .../ignite/internal/GridTaskSessionImpl.java | 15 +- .../apache/ignite/internal/IgnitionEx.java | 22 ++ .../loadbalancer/GridLoadBalancerManager.java | 10 +- .../processors/job/GridJobProcessor.java | 3 +- .../session/GridTaskSessionProcessor.java | 9 +- .../processors/task/GridTaskProcessor.java | 3 +- .../processors/task/GridTaskWorker.java | 2 +- ...ridInternalTasksLoadBalancingSelfTest.java | 201 ++++++++++++++++++ .../IgniteSpiLoadBalancingSelfTestSuite.java | 34 +-- 10 files changed, 279 insertions(+), 28 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/internal/GridInternalTasksLoadBalancingSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSpis.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSpis.java index 5edd464f80386..0b6e425639d2f 100644 --- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSpis.java +++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSpis.java @@ -38,22 +38,22 @@ public @interface ComputeTaskSpis { /** * Optional load balancing SPI name. By default, SPI name is equal - * to the name of the SPI class. You can change SPI name by explicitely + * to the name of the SPI class. You can change SPI name by explicitly * supplying {@link org.apache.ignite.spi.IgniteSpi#getName()} parameter in grid configuration. */ public String loadBalancingSpi() default ""; /** * Optional failover SPI name. By default, SPI name is equal - * to the name of the SPI class. You can change SPI name by explicitely + * to the name of the SPI class. You can change SPI name by explicitly * supplying {@link org.apache.ignite.spi.IgniteSpi#getName()} parameter in grid configuration. */ public String failoverSpi() default ""; /** * Optional checkpoint SPI name. By default, SPI name is equal - * to the name of the SPI class. You can change SPI name by explicitely + * to the name of the SPI class. You can change SPI name by explicitly * supplying {@link org.apache.ignite.spi.IgniteSpi#getName()} parameter in grid configuration. */ public String checkpointSpi() default ""; -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionImpl.java index bbe05b8127221..dd1caa15396b3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionImpl.java @@ -102,6 +102,9 @@ public class GridTaskSessionImpl implements GridTaskSessionInternal { /** */ private final boolean fullSup; + /** */ + private final boolean internal; + /** */ private final Collection top; @@ -124,6 +127,7 @@ public class GridTaskSessionImpl implements GridTaskSessionInternal { * @param attrs Session attributes. * @param ctx Grid Kernal Context. * @param fullSup Session full support enabled flag. + * @param internal Internal task flag. * @param subjId Subject ID. */ public GridTaskSessionImpl( @@ -139,6 +143,7 @@ public GridTaskSessionImpl( @Nullable Map attrs, GridKernalContext ctx, boolean fullSup, + boolean internal, UUID subjId) { assert taskNodeId != null; assert taskName != null; @@ -166,6 +171,7 @@ public GridTaskSessionImpl( } this.fullSup = fullSup; + this.internal = internal; this.subjId = subjId; mapFut = new IgniteFutureImpl(new GridFutureAdapter()); @@ -860,8 +866,15 @@ public void onDone() { return mapFut; } + /** + * @return {@code True} if task is internal. + */ + public boolean isInternal() { + return internal; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(GridTaskSessionImpl.class, this); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index b54c17d1794a0..4f64da2f411d5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -93,6 +93,7 @@ import org.apache.ignite.spi.eventstorage.memory.MemoryEventStorageSpi; import org.apache.ignite.spi.failover.always.AlwaysFailoverSpi; import org.apache.ignite.spi.indexing.noop.NoopIndexingSpi; +import org.apache.ignite.spi.loadbalancing.LoadBalancingSpi; import org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi; import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi; import org.apache.ignite.spi.swapspace.noop.NoopSwapSpaceSpi; @@ -2052,6 +2053,27 @@ private void initializeDefaultSpi(IgniteConfiguration cfg) { if (cfg.getLoadBalancingSpi() == null) cfg.setLoadBalancingSpi(new RoundRobinLoadBalancingSpi()); + else { + Collection spis = new ArrayList<>(); + + boolean dfltLoadBalancingSpi = false; + + for (LoadBalancingSpi spi : cfg.getLoadBalancingSpi()) { + spis.add(spi); + + if (spi instanceof RoundRobinLoadBalancingSpi) { + dfltLoadBalancingSpi = true; + + break; + } + } + + // Add default load balancing SPI for internal tasks. + if (!dfltLoadBalancingSpi) + spis.add(new RoundRobinLoadBalancingSpi()); + + cfg.setLoadBalancingSpi(spis.toArray(new LoadBalancingSpi[spis.size()])); + } if (cfg.getIndexingSpi() == null) cfg.setIndexingSpi(new NoopIndexingSpi()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/loadbalancer/GridLoadBalancerManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/loadbalancer/GridLoadBalancerManager.java index 631168b64c7df..15a4d2cd9510b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/loadbalancer/GridLoadBalancerManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/loadbalancer/GridLoadBalancerManager.java @@ -30,6 +30,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.spi.loadbalancing.LoadBalancingSpi; +import org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi; import org.jetbrains.annotations.Nullable; /** @@ -72,7 +73,12 @@ public ClusterNode getBalancedNode(GridTaskSessionImpl ses, List to assert top != null; assert job != null; - return getSpi(ses.getLoadBalancingSpi()).getBalancedNode(ses, top, job); + LoadBalancingSpi spi = getSpi(ses.getLoadBalancingSpi()); + + if (ses.isInternal() && !(spi instanceof RoundRobinLoadBalancingSpi)) + return getSpi(RoundRobinLoadBalancingSpi.class.getSimpleName()).getBalancedNode(ses, top, job); + + return spi.getBalancedNode(ses, top, job); } /** @@ -102,4 +108,4 @@ public ComputeLoadBalancer getLoadBalancer(final GridTaskSessionImpl ses, final } }; } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java index 6a162d377c3f0..563a3d8ddf7c7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java @@ -1057,6 +1057,7 @@ public void processJobExecuteRequest(ClusterNode node, final GridJobExecuteReque siblings, sesAttrs, req.isSessionFullSupport(), + req.isInternal(), req.getSubjectId()); taskSes.setCheckpointSpi(req.getCheckpointSpi()); @@ -2034,4 +2035,4 @@ private JobsMap(int initCap, float loadFactor, int concurLvl) { return sizex(); } } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/session/GridTaskSessionProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/session/GridTaskSessionProcessor.java index 2ac39c909c37d..d660948e29baa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/session/GridTaskSessionProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/session/GridTaskSessionProcessor.java @@ -73,8 +73,8 @@ public GridTaskSessionProcessor(GridKernalContext ctx) { * @param endTime Execution end time. * @param siblings Collection of siblings. * @param attrs Map of attributes. - * @param fullSup {@code True} to enable distributed session attributes - * and checkpoints. + * @param fullSup {@code True} to enable distributed session attributes and checkpoints. + * @param internal {@code True} in case of internal task. * @param subjId Subject ID. * @return New session if one did not exist, or existing one. */ @@ -90,6 +90,7 @@ public GridTaskSessionImpl createTaskSession( Collection siblings, Map attrs, boolean fullSup, + boolean internal, UUID subjId) { if (!fullSup) { return new GridTaskSessionImpl( @@ -105,6 +106,7 @@ public GridTaskSessionImpl createTaskSession( attrs, ctx, false, + internal, subjId); } @@ -127,6 +129,7 @@ public GridTaskSessionImpl createTaskSession( attrs, ctx, true, + internal, subjId)); if (old != null) @@ -177,4 +180,4 @@ public boolean removeSession(IgniteUuid sesId) { X.println(">>> Task session processor memory stats [grid=" + ctx.gridName() + ']'); X.println(">>> sesMapSize: " + sesMap.size()); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java index f54277cbd46d4..6d9722933bcc3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java @@ -621,6 +621,7 @@ else if (task != null) { Collections.emptyList(), Collections.emptyMap(), fullSup, + dep != null && dep.internalTask(task, taskCls), subjId); ComputeTaskInternalFuture fut = new ComputeTaskInternalFuture<>(ses, ctx); @@ -1367,4 +1368,4 @@ private class TaskCancelMessageListener implements GridMessageListener { } } } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java index 79d1a2cf4e35d..452e48cb9ac28 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java @@ -490,7 +490,7 @@ private void initializeSpis() { } } - internal = dep.internalTask(task, taskCls); + internal = ses.isInternal(); recordTaskEvent(EVT_TASK_STARTED, "Task started."); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/internal/GridInternalTasksLoadBalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/internal/GridInternalTasksLoadBalancingSelfTest.java new file mode 100644 index 0000000000000..da1a4e57d91a9 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/internal/GridInternalTasksLoadBalancingSelfTest.java @@ -0,0 +1,201 @@ +/* + * 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.spi.loadbalancing.internal; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.compute.ComputeJob; +import org.apache.ignite.compute.ComputeJobAdapter; +import org.apache.ignite.compute.ComputeJobResult; +import org.apache.ignite.compute.ComputeTaskSession; +import org.apache.ignite.compute.ComputeTaskSplitAdapter; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.lang.GridTuple3; +import org.apache.ignite.internal.visor.VisorTaskArgument; +import org.apache.ignite.internal.visor.node.VisorNodePingTask; +import org.apache.ignite.spi.IgniteSpiAdapter; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.IgniteSpiMultipleInstancesSupport; +import org.apache.ignite.spi.loadbalancing.LoadBalancingSpi; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.Nullable; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.UUID; + +/** + * Test that will start two nodes with custom load balancing SPI and execute {@link GridInternal} task on it. + */ +public class GridInternalTasksLoadBalancingSelfTest extends GridCommonAbstractTest { + /** Grid count. */ + private static final int GRID_CNT = 2; + + /** Expected job result. */ + private static final String JOB_RESULT = "EXPECTED JOB RESULT"; + + /** Expected task result. */ + private static final String TASK_RESULT = JOB_RESULT + JOB_RESULT; + + /** */ + private static Ignite ignite; + + /** If {@code true} then special custom load balancer SPI will be used. */ + private static boolean customLoadBalancer = true; + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + if (customLoadBalancer) + cfg.setLoadBalancingSpi(new CustomLoadBalancerSpi()); + + return cfg; + } + + /** + * This test execute internal tasks over grid with custom balancer. + * + * @throws Exception In case of error. + */ + public void testInternalTaskBalancing() throws Exception { + customLoadBalancer = true; + + ignite = startGrids(GRID_CNT); + + // Task with GridInternal should pass. + assertEquals(TASK_RESULT, ignite.compute().execute(GridInternalTestTask.class.getName(), null)); + + // Visor task should pass. + UUID nid = ignite.cluster().localNode().id(); + + GridTuple3 ping = ignite.compute() + .execute(VisorNodePingTask.class.getName(), new VisorTaskArgument<>(nid, nid, false)); + + assertTrue(ping.get1()); + + // Custom task should fail, because special test load balancer SPI returns null as balanced node. + try { + ignite.compute().execute(CustomTestTask.class.getName(), null); + } + catch (IgniteException e) { + assertTrue(e.getMessage().startsWith("Node can not be null [mappedJob=org.apache.ignite.spi.loadbalancing.internal.GridInternalTasksLoadBalancingSelfTest$CustomTestJob")); + } + } + + /** + * This test execute internal tasks over grid with default balancer. + * + * @throws Exception In case of error. + */ + public void testInternalTaskDefaultBalancing() throws Exception { + customLoadBalancer = false; + + ignite = startGrids(GRID_CNT); + + // Task with GridInternal should pass. + assertEquals(TASK_RESULT, ignite.compute().execute(GridInternalTestTask.class.getName(), null)); + + // Visor task should pass. + UUID nid = ignite.cluster().localNode().id(); + + GridTuple3 ping = ignite.compute() + .execute(VisorNodePingTask.class.getName(), new VisorTaskArgument<>(nid, nid, false)); + + assertTrue(ping.get1()); + + // Custom task should pass. + assertEquals(TASK_RESULT, ignite.compute().execute(CustomTestTask.class.getName(), null)); + } + + /** + * Test task. + */ + private static class CustomTestTask extends ComputeTaskSplitAdapter { + /** {@inheritDoc} */ + @Override protected Collection split(int gridSize, Object arg) { + Collection jobs = new ArrayList<>(gridSize); + + for (int i = 0; i < gridSize; i++) + jobs.add(new CustomTestJob()); + + return jobs; + } + + /** {@inheritDoc} */ + @Override public Serializable reduce(List results) { + assert results != null; + + String res = ""; + + for (ComputeJobResult jobRes : results) + res += jobRes.getData(); + + return res; + } + } + + /** + * Test job. + */ + private static class CustomTestJob extends ComputeJobAdapter { + /** {@inheritDoc} */ + @Override public String execute() { + return JOB_RESULT; + } + } + + + /** + * Test task marked with @GridInternal. + */ + @GridInternal + private static class GridInternalTestTask extends CustomTestTask{ + + } + + /** + * Special test balancer that will do not any balancing. + */ + @IgniteSpiMultipleInstancesSupport(true) + private static class CustomLoadBalancerSpi extends IgniteSpiAdapter implements LoadBalancingSpi { + /** {@inheritDoc} */ + @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void spiStop() throws IgniteSpiException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public ClusterNode getBalancedNode(ComputeTaskSession ses, List top, ComputeJob job) throws IgniteException { + return null; // Intentionally return null. + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiLoadBalancingSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiLoadBalancingSelfTestSuite.java index a20b3c8278682..52d4fddabef24 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiLoadBalancingSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiLoadBalancingSelfTestSuite.java @@ -23,6 +23,7 @@ import org.apache.ignite.spi.loadbalancing.adaptive.GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest; import org.apache.ignite.spi.loadbalancing.adaptive.GridAdaptiveLoadBalancingSpiSelfTest; import org.apache.ignite.spi.loadbalancing.adaptive.GridAdaptiveLoadBalancingSpiStartStopSelfTest; +import org.apache.ignite.spi.loadbalancing.internal.GridInternalTasksLoadBalancingSelfTest; import org.apache.ignite.spi.loadbalancing.roundrobin.GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest; import org.apache.ignite.spi.loadbalancing.roundrobin.GridRoundRobinLoadBalancingSpiLocalNodeSelfTest; import org.apache.ignite.spi.loadbalancing.roundrobin.GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest; @@ -52,25 +53,28 @@ public static Test suite() { TestSuite suite = new TestSuite("Ignite Load Balancing Test Suite"); // Random. - suite.addTest(new TestSuite(GridWeightedRandomLoadBalancingSpiSelfTest.class)); - suite.addTest(new TestSuite(GridWeightedRandomLoadBalancingSpiWeightedSelfTest.class)); - suite.addTest(new TestSuite(GridWeightedRandomLoadBalancingSpiStartStopSelfTest.class)); - suite.addTest(new TestSuite(GridWeightedRandomLoadBalancingSpiConfigSelfTest.class)); + suite.addTestSuite(GridWeightedRandomLoadBalancingSpiSelfTest.class); + suite.addTestSuite(GridWeightedRandomLoadBalancingSpiWeightedSelfTest.class); + suite.addTestSuite(GridWeightedRandomLoadBalancingSpiStartStopSelfTest.class); + suite.addTestSuite(GridWeightedRandomLoadBalancingSpiConfigSelfTest.class); // Round-robin. - suite.addTest(new TestSuite(GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.class)); - suite.addTest(new TestSuite(GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.class)); - suite.addTest(new TestSuite(GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.class)); - suite.addTest(new TestSuite(GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.class)); - suite.addTest(new TestSuite(GridRoundRobinLoadBalancingSpiStartStopSelfTest.class)); - suite.addTest(new TestSuite(GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.class)); + suite.addTestSuite(GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.class); + suite.addTestSuite(GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.class); + suite.addTestSuite(GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.class); + suite.addTestSuite(GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.class); + suite.addTestSuite(GridRoundRobinLoadBalancingSpiStartStopSelfTest.class); + suite.addTestSuite(GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.class); // Adaptive. - suite.addTest(new TestSuite(GridAdaptiveLoadBalancingSpiSelfTest.class)); - suite.addTest(new TestSuite(GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.class)); - suite.addTest(new TestSuite(GridAdaptiveLoadBalancingSpiStartStopSelfTest.class)); - suite.addTest(new TestSuite(GridAdaptiveLoadBalancingSpiConfigSelfTest.class)); + suite.addTestSuite(GridAdaptiveLoadBalancingSpiSelfTest.class); + suite.addTestSuite(GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.class); + suite.addTestSuite(GridAdaptiveLoadBalancingSpiStartStopSelfTest.class); + suite.addTestSuite(GridAdaptiveLoadBalancingSpiConfigSelfTest.class); + + // Load balancing for internal tasks. + suite.addTestSuite(GridInternalTasksLoadBalancingSelfTest.class); return suite; } -} \ No newline at end of file +} From ec9ddcd3d99d19403bf19e1172ede2afdab6c86f Mon Sep 17 00:00:00 2001 From: sboikov Date: Wed, 28 Sep 2016 12:05:28 +0300 Subject: [PATCH 185/487] Code style fixes. --- .../stream/jms11/IgniteJmsStreamerTest.java | 206 +++++++++++------- .../jms11/IgniteJmsStreamerTestSuite.java | 2 +- 2 files changed, 134 insertions(+), 74 deletions(-) diff --git a/modules/jms11/src/test/java/org/apache/ignite/stream/jms11/IgniteJmsStreamerTest.java b/modules/jms11/src/test/java/org/apache/ignite/stream/jms11/IgniteJmsStreamerTest.java index 238d939f357fd..290185eb902e7 100644 --- a/modules/jms11/src/test/java/org/apache/ignite/stream/jms11/IgniteJmsStreamerTest.java +++ b/modules/jms11/src/test/java/org/apache/ignite/stream/jms11/IgniteJmsStreamerTest.java @@ -59,26 +59,39 @@ * @author Raul Kripalani */ public class IgniteJmsStreamerTest extends GridCommonAbstractTest { - + /** */ private static final int CACHE_ENTRY_COUNT = 100; + + /** */ private static final String QUEUE_NAME = "ignite.test.queue"; + + /** */ private static final String TOPIC_NAME = "ignite.test.topic"; + + /** */ private static final Map TEST_DATA = new HashMap<>(); + /** */ + private BrokerService broker; + + /** */ + private ConnectionFactory connFactory; + static { for (int i = 1; i <= CACHE_ENTRY_COUNT; i++) TEST_DATA.put(Integer.toString(i), "v" + i); } - private BrokerService broker; - private ConnectionFactory connectionFactory; - /** Constructor. */ public IgniteJmsStreamerTest() { super(true); } - @Before @SuppressWarnings("unchecked") + /** + * @throws Exception If failed. + */ + @Before + @SuppressWarnings("unchecked") public void beforeTest() throws Exception { grid().getOrCreateCache(defaultCacheConfiguration()); @@ -88,21 +101,24 @@ public void beforeTest() throws Exception { broker.setPersistenceAdapter(null); broker.setPersistenceFactory(null); - PolicyMap policyMap = new PolicyMap(); - PolicyEntry policy = new PolicyEntry(); + PolicyMap plcMap = new PolicyMap(); + PolicyEntry plc = new PolicyEntry(); - policy.setQueuePrefetch(1); + plc.setQueuePrefetch(1); - broker.setDestinationPolicy(policyMap); - broker.getDestinationPolicy().setDefaultEntry(policy); + broker.setDestinationPolicy(plcMap); + broker.getDestinationPolicy().setDefaultEntry(plc); broker.setSchedulerSupport(false); broker.start(true); - connectionFactory = new ActiveMQConnectionFactory(BrokerRegistry.getInstance().findFirst().getVmConnectorURI()); - + connFactory = new ActiveMQConnectionFactory(BrokerRegistry.getInstance().findFirst().getVmConnectorURI()); } + /** + * + * @throws Exception Iff ailed. + */ @After public void afterTest() throws Exception { grid().cache(null).clear(); @@ -111,11 +127,14 @@ public void afterTest() throws Exception { broker.deleteAllMessages(); } + /** + * @throws Exception If failed. + */ public void testQueueFromName() throws Exception { - Destination destination = new ActiveMQQueue(QUEUE_NAME); + Destination dest = new ActiveMQQueue(QUEUE_NAME); // produce messages into the queue - produceObjectMessages(destination, false); + produceObjectMessages(dest, false); try (IgniteDataStreamer dataStreamer = grid().dataStreamer(null)) { JmsStreamer jmsStreamer = newJmsStreamer(ObjectMessage.class, dataStreamer); @@ -137,8 +156,11 @@ public void testQueueFromName() throws Exception { } + /** + * @throws Exception If failed. + */ public void testTopicFromName() throws JMSException, InterruptedException { - Destination destination = new ActiveMQTopic(TOPIC_NAME); + Destination dest = new ActiveMQTopic(TOPIC_NAME); // should not produced messages until subscribed to the topic; otherwise they will be missed because this is not // a durable subscriber (for which a dedicated test exists) @@ -154,7 +176,7 @@ public void testTopicFromName() throws JMSException, InterruptedException { jmsStreamer.start(); // produce messages - produceObjectMessages(destination, false); + produceObjectMessages(dest, false); // all cache PUT events received in 10 seconds latch.await(10, TimeUnit.SECONDS); @@ -166,15 +188,18 @@ public void testTopicFromName() throws JMSException, InterruptedException { } + /** + * @throws Exception If failed. + */ public void testQueueFromExplicitDestination() throws Exception { - Destination destination = new ActiveMQQueue(QUEUE_NAME); + Destination dest = new ActiveMQQueue(QUEUE_NAME); // produce messages into the queue - produceObjectMessages(destination, false); + produceObjectMessages(dest, false); try (IgniteDataStreamer dataStreamer = grid().dataStreamer(null)) { JmsStreamer jmsStreamer = newJmsStreamer(ObjectMessage.class, dataStreamer); - jmsStreamer.setDestination(destination); + jmsStreamer.setDestination(dest); // subscribe to cache PUT events and return a countdown latch starting at CACHE_ENTRY_COUNT CountDownLatch latch = subscribeToPutEvents(CACHE_ENTRY_COUNT); @@ -192,15 +217,18 @@ public void testQueueFromExplicitDestination() throws Exception { } + /** + * @throws Exception If failed. + */ public void testTopicFromExplicitDestination() throws JMSException, InterruptedException { - Destination destination = new ActiveMQTopic(TOPIC_NAME); + Destination dest = new ActiveMQTopic(TOPIC_NAME); // should not produced messages until subscribed to the topic; otherwise they will be missed because this is not // a durable subscriber (for which a dedicated test exists) try (IgniteDataStreamer dataStreamer = grid().dataStreamer(null)) { JmsStreamer jmsStreamer = newJmsStreamer(ObjectMessage.class, dataStreamer); - jmsStreamer.setDestination(destination); + jmsStreamer.setDestination(dest); // subscribe to cache PUT events and return a countdown latch starting at CACHE_ENTRY_COUNT CountDownLatch latch = subscribeToPutEvents(CACHE_ENTRY_COUNT); @@ -208,7 +236,7 @@ public void testTopicFromExplicitDestination() throws JMSException, InterruptedE jmsStreamer.start(); // produce messages - produceObjectMessages(destination, false); + produceObjectMessages(dest, false); // all cache PUT events received in 10 seconds latch.await(10, TimeUnit.SECONDS); @@ -220,15 +248,18 @@ public void testTopicFromExplicitDestination() throws JMSException, InterruptedE } + /** + * @throws Exception If failed. + */ public void testInsertMultipleCacheEntriesFromOneMessage() throws Exception { - Destination destination = new ActiveMQQueue(QUEUE_NAME); + Destination dest = new ActiveMQQueue(QUEUE_NAME); // produce A SINGLE MESSAGE, containing all data, into the queue - produceStringMessages(destination, true); + produceStringMessages(dest, true); try (IgniteDataStreamer dataStreamer = grid().dataStreamer(null)) { JmsStreamer jmsStreamer = newJmsStreamer(TextMessage.class, dataStreamer); - jmsStreamer.setDestination(destination); + jmsStreamer.setDestination(dest); // subscribe to cache PUT events and return a countdown latch starting at CACHE_ENTRY_COUNT CountDownLatch latch = subscribeToPutEvents(CACHE_ENTRY_COUNT); @@ -245,12 +276,15 @@ public void testInsertMultipleCacheEntriesFromOneMessage() throws Exception { } + /** + * @throws Exception If failed. + */ public void testDurableSubscriberStartStopStart() throws Exception { - Destination destination = new ActiveMQTopic(TOPIC_NAME); + Destination dest = new ActiveMQTopic(TOPIC_NAME); try (IgniteDataStreamer dataStreamer = grid().dataStreamer(null)) { JmsStreamer jmsStreamer = newJmsStreamer(TextMessage.class, dataStreamer); - jmsStreamer.setDestination(destination); + jmsStreamer.setDestination(dest); jmsStreamer.setDurableSubscription(true); jmsStreamer.setClientId(Long.toString(System.currentTimeMillis())); jmsStreamer.setDurableSubscriptionName("ignite-test-durable"); @@ -265,7 +299,7 @@ public void testDurableSubscriberStartStopStart() throws Exception { assertEquals(0, broker.getCurrentConnections()); // we send messages while we're still away - produceStringMessages(destination, false); + produceStringMessages(dest, false); // subscribe to cache PUT events and return a countdown latch starting at CACHE_ENTRY_COUNT CountDownLatch latch = subscribeToPutEvents(CACHE_ENTRY_COUNT); @@ -282,15 +316,18 @@ public void testDurableSubscriberStartStopStart() throws Exception { } + /** + * @throws Exception If failed. + */ public void testQueueMessagesConsumedInBatchesCompletionSizeBased() throws Exception { - Destination destination = new ActiveMQQueue(QUEUE_NAME); + Destination dest = new ActiveMQQueue(QUEUE_NAME); // produce multiple messages into the queue - produceStringMessages(destination, false); + produceStringMessages(dest, false); try (IgniteDataStreamer dataStreamer = grid().dataStreamer(null)) { JmsStreamer jmsStreamer = newJmsStreamer(TextMessage.class, dataStreamer); - jmsStreamer.setDestination(destination); + jmsStreamer.setDestination(dest); jmsStreamer.setBatched(true); jmsStreamer.setBatchClosureSize(99); @@ -309,7 +346,7 @@ public void testQueueMessagesConsumedInBatchesCompletionSizeBased() throws Excep // we expect all entries to be loaded, but still one (uncommitted) message should remain in the queue // as observed by the broker - DestinationStatistics qStats = broker.getBroker().getDestinationMap().get(destination).getDestinationStatistics(); + DestinationStatistics qStats = broker.getBroker().getDestinationMap().get(dest).getDestinationStatistics(); assertEquals(1, qStats.getMessages().getCount()); assertEquals(1, qStats.getInflight().getCount()); @@ -318,15 +355,18 @@ public void testQueueMessagesConsumedInBatchesCompletionSizeBased() throws Excep } + /** + * @throws Exception If failed. + */ public void testQueueMessagesConsumedInBatchesCompletionTimeBased() throws Exception { - Destination destination = new ActiveMQQueue(QUEUE_NAME); + Destination dest = new ActiveMQQueue(QUEUE_NAME); // produce multiple messages into the queue - produceStringMessages(destination, false); + produceStringMessages(dest, false); try (IgniteDataStreamer dataStreamer = grid().dataStreamer(null)) { JmsStreamer jmsStreamer = newJmsStreamer(TextMessage.class, dataStreamer); - jmsStreamer.setDestination(destination); + jmsStreamer.setDestination(dest); jmsStreamer.setBatched(true); jmsStreamer.setBatchClosureMillis(2000); // disable size-based session commits @@ -334,7 +374,7 @@ public void testQueueMessagesConsumedInBatchesCompletionTimeBased() throws Excep // subscribe to cache PUT events and return a countdown latch starting at CACHE_ENTRY_COUNT CountDownLatch latch = subscribeToPutEvents(CACHE_ENTRY_COUNT); - DestinationStatistics qStats = broker.getBroker().getDestinationMap().get(destination).getDestinationStatistics(); + DestinationStatistics qStats = broker.getBroker().getDestinationMap().get(dest).getDestinationStatistics(); jmsStreamer.start(); @@ -365,17 +405,20 @@ public void testQueueMessagesConsumedInBatchesCompletionTimeBased() throws Excep } + /** + * @throws Exception If failed. + */ public void testGenerateNoEntries() throws Exception { - Destination destination = new ActiveMQQueue(QUEUE_NAME); + Destination dest = new ActiveMQQueue(QUEUE_NAME); // produce multiple messages into the queue - produceStringMessages(destination, false); + produceStringMessages(dest, false); try (IgniteDataStreamer dataStreamer = grid().dataStreamer(null)) { JmsStreamer jmsStreamer = newJmsStreamer(TextMessage.class, dataStreamer); // override the transformer with one that generates no cache entries jmsStreamer.setTransformer(TestTransformers.generateNoEntries()); - jmsStreamer.setDestination(destination); + jmsStreamer.setDestination(dest); // subscribe to cache PUT events and return a countdown latch starting at CACHE_ENTRY_COUNT CountDownLatch latch = subscribeToPutEvents(1); @@ -390,16 +433,19 @@ public void testGenerateNoEntries() throws Exception { } + /** + * @throws Exception If failed. + */ public void testTransactedSessionNoBatching() throws Exception { - Destination destination = new ActiveMQQueue(QUEUE_NAME); + Destination dest = new ActiveMQQueue(QUEUE_NAME); // produce multiple messages into the queue - produceStringMessages(destination, false); + produceStringMessages(dest, false); try (IgniteDataStreamer dataStreamer = grid().dataStreamer(null)) { JmsStreamer jmsStreamer = newJmsStreamer(TextMessage.class, dataStreamer); jmsStreamer.setTransacted(true); - jmsStreamer.setDestination(destination); + jmsStreamer.setDestination(dest); // subscribe to cache PUT events and return a countdown latch starting at CACHE_ENTRY_COUNT CountDownLatch latch = subscribeToPutEvents(CACHE_ENTRY_COUNT); @@ -416,15 +462,18 @@ public void testTransactedSessionNoBatching() throws Exception { } + /** + * @throws Exception If failed. + */ public void testQueueMultipleThreads() throws Exception { - Destination destination = new ActiveMQQueue(QUEUE_NAME); + Destination dest = new ActiveMQQueue(QUEUE_NAME); // produce messages into the queue - produceObjectMessages(destination, false); + produceObjectMessages(dest, false); try (IgniteDataStreamer dataStreamer = grid().dataStreamer(null)) { JmsStreamer jmsStreamer = newJmsStreamer(ObjectMessage.class, dataStreamer); - jmsStreamer.setDestination(destination); + jmsStreamer.setDestination(dest); jmsStreamer.setThreads(5); // subscribe to cache PUT events and return a countdown latch starting at CACHE_ENTRY_COUNT @@ -433,14 +482,14 @@ public void testQueueMultipleThreads() throws Exception { // start the streamer jmsStreamer.start(); - DestinationStatistics qStats = broker.getBroker().getDestinationMap().get(destination).getDestinationStatistics(); + DestinationStatistics qStats = broker.getBroker().getDestinationMap().get(dest).getDestinationStatistics(); assertEquals(5, qStats.getConsumers().getCount()); // all cache PUT events received in 10 seconds latch.await(10, TimeUnit.SECONDS); // assert that all consumers received messages - given that the prefetch is 1 - for (Subscription subscription : broker.getBroker().getDestinationMap().get(destination).getConsumers()) + for (Subscription subscription : broker.getBroker().getDestinationMap().get(dest).getConsumers()) assertTrue(subscription.getDequeueCounter() > 0); assertAllCacheEntriesLoaded(); @@ -450,6 +499,9 @@ public void testQueueMultipleThreads() throws Exception { } + /** + * + */ private void assertAllCacheEntriesLoaded() { // Get the cache and check that the entries are present IgniteCache cache = grid().cache(null); @@ -464,39 +516,44 @@ private JmsStreamer newJmsStreamer(Class< JmsStreamer jmsStreamer = new JmsStreamer<>(); jmsStreamer.setIgnite(grid()); jmsStreamer.setStreamer(dataStreamer); - jmsStreamer.setConnectionFactory(connectionFactory); + jmsStreamer.setConnectionFactory(connFactory); - if (type == ObjectMessage.class) { + if (type == ObjectMessage.class) jmsStreamer.setTransformer((MessageTransformer) TestTransformers.forObjectMessage()); - } - else { + else jmsStreamer.setTransformer((MessageTransformer) TestTransformers.forTextMessage()); - } dataStreamer.allowOverwrite(true); dataStreamer.autoFlushFrequency(10); return jmsStreamer; } + /** + * @param expect Expected events number. + * @return Event receive latch. + */ private CountDownLatch subscribeToPutEvents(int expect) { Ignite ignite = grid(); // Listen to cache PUT events and expect as many as messages as test data items final CountDownLatch latch = new CountDownLatch(expect); - @SuppressWarnings("serial") IgniteBiPredicate callback = new IgniteBiPredicate() { + + @SuppressWarnings("serial") IgniteBiPredicate cb = new IgniteBiPredicate() { @Override public boolean apply(UUID uuid, CacheEvent evt) { latch.countDown(); return true; } }; - ignite.events(ignite.cluster().forCacheNodes(null)).remoteListen(callback, null, EVT_CACHE_OBJECT_PUT); + ignite.events(ignite.cluster().forCacheNodes(null)).remoteListen(cb, null, EVT_CACHE_OBJECT_PUT); return latch; } - private void produceObjectMessages(Destination destination, boolean singleMessage) throws JMSException { - Session session = connectionFactory.createConnection().createSession(false, Session.AUTO_ACKNOWLEDGE); - MessageProducer mp = session.createProducer(destination); + private void produceObjectMessages(Destination dest, boolean singleMsg) throws JMSException { + Session ses = connFactory.createConnection().createSession(false, Session.AUTO_ACKNOWLEDGE); + + MessageProducer mp = ses.createProducer(dest); + HashSet set = new HashSet<>(); for (String key : TEST_DATA.keySet()) { @@ -505,20 +562,21 @@ private void produceObjectMessages(Destination destination, boolean singleMessag } int messagesSent; - if (singleMessage) { - mp.send(session.createObjectMessage(set)); + + if (singleMsg) { + mp.send(ses.createObjectMessage(set)); messagesSent = 1; } else { for (TestTransformers.TestObject to : set) - mp.send(session.createObjectMessage(to)); + mp.send(ses.createObjectMessage(to)); messagesSent = set.size(); } - if (destination instanceof Queue) { + if (dest instanceof Queue) { try { - assertEquals(messagesSent, broker.getBroker().getDestinationMap().get(destination) + assertEquals(messagesSent, broker.getBroker().getDestinationMap().get(dest) .getDestinationStatistics().getMessages().getCount()); } catch (Exception e) { @@ -528,36 +586,39 @@ private void produceObjectMessages(Destination destination, boolean singleMessag } - private void produceStringMessages(Destination destination, boolean singleMessage) throws JMSException { - Session session = connectionFactory.createConnection().createSession(false, Session.AUTO_ACKNOWLEDGE); - MessageProducer mp = session.createProducer(destination); + private void produceStringMessages(Destination dest, boolean singleMsg) throws JMSException { + Session ses = connFactory.createConnection().createSession(false, Session.AUTO_ACKNOWLEDGE); + + MessageProducer mp = ses.createProducer(dest); + HashSet set = new HashSet<>(); for (String key : TEST_DATA.keySet()) set.add(key + "," + TEST_DATA.get(key)); int messagesSent; - if (singleMessage) { + + if (singleMsg) { StringBuilder sb = new StringBuilder(); for (String s : set) sb.append(s).append("|"); sb.deleteCharAt(sb.length() - 1); - mp.send(session.createTextMessage(sb.toString())); + mp.send(ses.createTextMessage(sb.toString())); messagesSent = 1; } else { - for (String s : set) { - mp.send(session.createTextMessage(s)); - } + for (String s : set) + mp.send(ses.createTextMessage(s)); + messagesSent = set.size(); } - if (destination instanceof Queue) { + if (dest instanceof Queue) { try { - assertEquals(messagesSent, broker.getBroker().getDestinationMap().get(destination) + assertEquals(messagesSent, broker.getBroker().getDestinationMap().get(dest) .getDestinationStatistics().getMessages().getCount()); } catch (Exception e) { @@ -566,5 +627,4 @@ private void produceStringMessages(Destination destination, boolean singleMessag } } - } \ No newline at end of file diff --git a/modules/jms11/src/test/java/org/apache/ignite/stream/jms11/IgniteJmsStreamerTestSuite.java b/modules/jms11/src/test/java/org/apache/ignite/stream/jms11/IgniteJmsStreamerTestSuite.java index e299f041c8a73..071ff9b6ab054 100644 --- a/modules/jms11/src/test/java/org/apache/ignite/stream/jms11/IgniteJmsStreamerTestSuite.java +++ b/modules/jms11/src/test/java/org/apache/ignite/stream/jms11/IgniteJmsStreamerTestSuite.java @@ -30,5 +30,5 @@ IgniteJmsStreamerTest.class }) public class IgniteJmsStreamerTestSuite { - + // No-op. } \ No newline at end of file From 17c2fc0b69abd023b2a1e5da344e67951fd49408 Mon Sep 17 00:00:00 2001 From: sboikov Date: Wed, 28 Sep 2016 12:56:17 +0300 Subject: [PATCH 186/487] ignite-2833 Need call 'touch' for cache entry if it was obtained using 'entryEx'. --- .../processors/cache/GridCacheTtlManager.java | 32 ++++++++++--------- .../GridCacheAbstractFullApiSelfTest.java | 2 +- 2 files changed, 18 insertions(+), 16 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java index 996544fdb4dd9..0f855fe6dca91 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java @@ -129,26 +129,28 @@ public boolean expire(int amount) { if (log.isTraceEnabled()) log.trace("Trying to remove expired entry from cache: " + e); - boolean touch = false; + boolean touch = e.ctx.isSwapOrOffheapEnabled(); - GridCacheEntryEx entry = e.ctx.cache().entryEx(e.key); + GridCacheEntryEx entry = touch ? e.ctx.cache().entryEx(e.key) : e.ctx.cache().peekEx(e.key); - while (true) { - try { - if (entry.onTtlExpired(obsoleteVer)) - touch = false; + if (entry != null) { + while (true) { + try { + if (entry.onTtlExpired(obsoleteVer)) + touch = false; - break; - } - catch (GridCacheEntryRemovedException e0) { - entry = entry.context().cache().entryEx(entry.key()); + break; + } + catch (GridCacheEntryRemovedException e0) { + entry = entry.context().cache().entryEx(entry.key()); - touch = true; + touch = true; + } } - } - if (touch) - entry.context().evicts().touch(entry, null); + if (touch) + entry.context().evicts().touch(entry, null); + } } } @@ -216,7 +218,7 @@ private static class EntryWrapper implements Comparable { private final GridCacheContext ctx; /** Cache Object Key */ - private final CacheObject key; + private final KeyCacheObject key; /** * @param entry Cache entry to create wrapper for. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java index c0318c81a2f59..f2cf4147378b3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java @@ -4059,7 +4059,7 @@ public void testCompactExpired() throws Exception { // Peek will actually remove entry from cache. assertNull(cache.localPeek(key)); - assert cache.localSize() == 0; + assertEquals(0, cache.localSize()); // Clear readers, if any. cache.remove(key); From daf974d261efa525678d5fabc6191642c07f9ad4 Mon Sep 17 00:00:00 2001 From: AKuznetsov Date: Wed, 28 Sep 2016 17:22:10 +0700 Subject: [PATCH 187/487] IGNITE-3965 Fixed issues found on review. --- .../src/main/java/org/apache/ignite/internal/IgnitionEx.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index 4f64da2f411d5..aad96fbc2056f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -2061,11 +2061,8 @@ private void initializeDefaultSpi(IgniteConfiguration cfg) { for (LoadBalancingSpi spi : cfg.getLoadBalancingSpi()) { spis.add(spi); - if (spi instanceof RoundRobinLoadBalancingSpi) { + if (!dfltLoadBalancingSpi && spi instanceof RoundRobinLoadBalancingSpi) dfltLoadBalancingSpi = true; - - break; - } } // Add default load balancing SPI for internal tasks. From 4ff19c20b169e0373eafc8025a838db8bfc61f27 Mon Sep 17 00:00:00 2001 From: sboikov Date: Wed, 28 Sep 2016 13:47:51 +0300 Subject: [PATCH 188/487] ignite-3621 Fixed 'testEvictExpired'. --- .../cache/GridCacheAbstractFullApiSelfTest.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java index f2cf4147378b3..b328960cd6a57 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java @@ -3477,9 +3477,16 @@ public void testEvictExpired() throws Exception { grid(0).cache(null).withExpiryPolicy(expiry).put(key, 1); + final Affinity aff = ignite(0).affinity(null); + boolean wait = waitForCondition(new GridAbsPredicate() { @Override public boolean apply() { - return cache.localPeek(key) == null; + for (int i = 0; i < gridCount(); i++) { + if (peek(jcache(i), key) != null) + return false; + } + + return true; } }, ttl + 1000); @@ -3498,8 +3505,6 @@ public void testEvictExpired() throws Exception { load(cache, key, true); - Affinity aff = ignite(0).affinity(null); - for (int i = 0; i < gridCount(); i++) { if (aff.isPrimary(grid(i).cluster().localNode(), key)) assertEquals((Integer)1, peek(jcache(i), key)); From bfe4458448a59542713830f57713b3ac2af08e2b Mon Sep 17 00:00:00 2001 From: sboikov Date: Wed, 28 Sep 2016 14:31:24 +0300 Subject: [PATCH 189/487] ignite-3621 Fixed 'testEvictExpired'. --- ...tomicClientOnlyMultiNodeFullApiSelfTest.java | 17 +++++++++++++++-- ...eAtomicNearOnlyMultiNodeFullApiSelfTest.java | 17 +++++++++++++++-- 2 files changed, 30 insertions(+), 4 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest.java index 927ee6273cf41..65f780b2c3fb8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest.java @@ -36,6 +36,7 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.events.Event; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgnitePredicate; @@ -45,6 +46,7 @@ import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_SWAPPED; import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_UNSWAPPED; +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; /** * @@ -193,7 +195,7 @@ public class GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest extends GridCache @Override public void testEvictExpired() throws Exception { IgniteCache cache = jcache(); - String key = primaryKeysForCache(cache, 1).get(0); + final String key = primaryKeysForCache(cache, 1).get(0); cache.put(key, 1); @@ -204,7 +206,18 @@ public class GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest extends GridCache grid(0).cache(null). withExpiryPolicy(new TouchedExpiryPolicy(new Duration(MILLISECONDS, ttl))).put(key, 1); - Thread.sleep(ttl + 100); + boolean wait = waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + for (int i = 0; i < gridCount(); i++) { + if (peek(jcache(i), key) != null) + return false; + } + + return true; + } + }, ttl + 1000); + + assertTrue("Failed to wait for entry expiration.", wait); // Expired entry should not be swapped. cache.localEvict(Collections.singleton(key)); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest.java index 3ce38f3a4072f..6542e762779c7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest.java @@ -29,9 +29,11 @@ import org.apache.ignite.cache.affinity.Affinity; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; /** * Tests NEAR_ONLY cache. @@ -124,7 +126,7 @@ public class GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest extends GridCacheNe IgniteCache cache = jcache(); - String key = primaryKeysForCache(cache, 1).get(0); + final String key = primaryKeysForCache(cache, 1).get(0); cache.put(key, 1); @@ -135,7 +137,18 @@ public class GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest extends GridCacheNe grid(0).cache(null). withExpiryPolicy(new TouchedExpiryPolicy(new Duration(MILLISECONDS, ttl))).put(key, 1); - Thread.sleep(ttl + 100); + boolean wait = waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + for (int i = 0; i < gridCount(); i++) { + if (peek(jcache(i), key) != null) + return false; + } + + return true; + } + }, ttl + 1000); + + assertTrue("Failed to wait for entry expiration.", wait); // Expired entry should not be swapped. cache.localEvict(Collections.singleton(key)); From d643dcf2dd2caac4c3ff04cb72f31bbfbf97339a Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Wed, 28 Sep 2016 14:34:23 +0300 Subject: [PATCH 190/487] IGNITE-3970 .NET: Fix Cyrillic 'C' letters in code - add test --- .../ProjectFilesTest.cs | 37 ++++++++++++------- 1 file changed, 24 insertions(+), 13 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs index 75167b54573eb..a030bf29080f0 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs @@ -18,6 +18,7 @@ namespace Apache.Ignite.Core.Tests { using System; + using System.Collections.Generic; using System.IO; using System.Linq; using System.Reflection; @@ -37,11 +38,7 @@ public void TestCsprojToolsVersion() var projFiles = GetDotNetSourceDir().GetFiles("*.csproj", SearchOption.AllDirectories); Assert.GreaterOrEqual(projFiles.Length, 7); - var invalidFiles = - projFiles.Where(x => !File.ReadAllText(x.FullName).Contains("ToolsVersion=\"4.0\"")).ToArray(); - - Assert.AreEqual(0, invalidFiles.Length, - "Invalid csproj files: " + string.Join(", ", invalidFiles.Select(x => x.FullName))); + CheckFiles(projFiles, x => !x.Contains("ToolsVersion=\"4.0\""), "Invalid csproj files: "); } /// @@ -53,17 +50,31 @@ public void TestSlnToolsVersion() var slnFiles = GetDotNetSourceDir().GetFiles("*.sln", SearchOption.AllDirectories); Assert.GreaterOrEqual(slnFiles.Length, 2); - var invalidFiles = - slnFiles.Where(x => - { - var text = File.ReadAllText(x.FullName); + CheckFiles(slnFiles, x => !x.Contains("# Visual Studio 2010") || + !x.Contains("Microsoft Visual Studio Solution File, Format Version 11.00"), + "Invalid sln files: "); + } - return !text.Contains("# Visual Studio 2010") || - !text.Contains("Microsoft Visual Studio Solution File, Format Version 11.00"); - }).ToArray(); + /// + /// Tests that there are no Cyrillic C instead of English C (which are on the same keyboard key). + /// + [Test] + public void TestCyrillicChars() + { + var srcFiles = GetDotNetSourceDir().GetFiles("*.cs", SearchOption.AllDirectories); + + CheckFiles(srcFiles, x => x.Contains('\u0441') || x.Contains('\u0421'), "Files with Cyrillic 'C': "); + } + + /// + /// Checks the files. + /// + private static void CheckFiles(IEnumerable files, Func isInvalid, string errorText) + { + var invalidFiles = files.Where(x => isInvalid(File.ReadAllText(x.FullName))).ToArray(); Assert.AreEqual(0, invalidFiles.Length, - "Invalid sln files: " + string.Join(", ", invalidFiles.Select(x => x.FullName))); + errorText + string.Join(", ", invalidFiles.Select(x => x.FullName))); } /// From 474ade276c4ae3e8f93cce37473d37270b4e7ad9 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Wed, 28 Sep 2016 14:38:04 +0300 Subject: [PATCH 191/487] IGNITE-3988: Moved failing cloud tests to ignore module. --- .../cloud/TcpDiscoveryCloudIpFinderSelfTest.java | 6 ++++-- .../ignite/testsuites/IgniteCloudTestSuite.java | 5 +++-- modules/ignored-tests/pom.xml | 14 ++++++++++++++ .../ignite/testsuites/IgniteIgnoredTestSuite.java | 3 +++ 4 files changed, 24 insertions(+), 4 deletions(-) diff --git a/modules/cloud/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java b/modules/cloud/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java index bf0cbd577dc3c..4bddb18941762 100644 --- a/modules/cloud/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java +++ b/modules/cloud/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java @@ -20,9 +20,11 @@ import com.google.common.collect.ImmutableList; import java.net.InetSocketAddress; import java.util.Collection; + import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinderAbstractSelfTest; import org.apache.ignite.testsuites.IgniteCloudTestSuite; +import org.apache.ignite.testsuites.IgniteIgnore; /** * TcpDiscoveryCloudIpFinder test. @@ -58,6 +60,7 @@ public TcpDiscoveryCloudIpFinderSelfTest() throws Exception { * * @throws Exception If any error occurs. */ + @IgniteIgnore("https://issues.apache.org/jira/browse/IGNITE-845") public void testAmazonWebServices() throws Exception { testCloudProvider("aws-ec2"); } @@ -67,9 +70,8 @@ public void testAmazonWebServices() throws Exception { * * @throws Exception If any error occurs. */ + @IgniteIgnore("https://issues.apache.org/jira/browse/IGNITE-1585") public void testGoogleComputeEngine() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-1585"); - testCloudProvider("google-compute-engine"); } diff --git a/modules/cloud/src/test/java/org/apache/ignite/testsuites/IgniteCloudTestSuite.java b/modules/cloud/src/test/java/org/apache/ignite/testsuites/IgniteCloudTestSuite.java index 12a67e74f26c0..632cddc6c7bbc 100644 --- a/modules/cloud/src/test/java/org/apache/ignite/testsuites/IgniteCloudTestSuite.java +++ b/modules/cloud/src/test/java/org/apache/ignite/testsuites/IgniteCloudTestSuite.java @@ -21,6 +21,7 @@ import java.util.LinkedList; import junit.framework.TestSuite; import org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinderSelfTest; +import org.apache.ignite.testframework.IgniteTestSuite; /** * Ignite Cloud integration test. @@ -31,10 +32,10 @@ public class IgniteCloudTestSuite extends TestSuite { * @throws Exception Thrown in case of the failure. */ public static TestSuite suite() throws Exception { - TestSuite suite = new TestSuite("Cloud Integration Test Suite"); + TestSuite suite = new IgniteTestSuite("Cloud Integration Test Suite"); // Cloud Nodes IP finder. - suite.addTest(new TestSuite(TcpDiscoveryCloudIpFinderSelfTest.class)); + suite.addTestSuite(TcpDiscoveryCloudIpFinderSelfTest.class); return suite; } diff --git a/modules/ignored-tests/pom.xml b/modules/ignored-tests/pom.xml index cfeacb2283e4c..9fdff2cf8fd1c 100644 --- a/modules/ignored-tests/pom.xml +++ b/modules/ignored-tests/pom.xml @@ -76,6 +76,20 @@ test + + org.apache.ignite + ignite-cloud + ${project.version} + + + + org.apache.ignite + ignite-cloud + ${project.version} + test-jar + test + + org.apache.ignite ignite-web diff --git a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java index fa60721c5d0ad..3ab1d7f4850cc 100644 --- a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java +++ b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java @@ -37,6 +37,9 @@ public static TestSuite suite() throws Exception { /* --- AWS --- */ suite.addTest(IgniteS3TestSuite.suite()); + /* --- CLOUDS --- */ + suite.addTest(IgniteCloudTestSuite.suite()); + /* --- WEB SESSIONS --- */ suite.addTest(IgniteWebSessionSelfTestSuite.suite()); From b9105df77cc70b532f94899c754fba47e3e05e9a Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Wed, 28 Sep 2016 14:53:20 +0300 Subject: [PATCH 192/487] IGNITE-3989: Moved failing JTA tests to ignore module. --- modules/ignored-tests/pom.xml | 75 +++++++++++++++++-- .../testsuites/IgniteIgnoredTestSuite.java | 3 + ...tionedCacheJtaLookupClassNameSelfTest.java | 4 +- .../ignite/testsuites/IgniteJtaTestSuite.java | 3 +- 4 files changed, 74 insertions(+), 11 deletions(-) diff --git a/modules/ignored-tests/pom.xml b/modules/ignored-tests/pom.xml index 9fdff2cf8fd1c..661e237921ccf 100644 --- a/modules/ignored-tests/pom.xml +++ b/modules/ignored-tests/pom.xml @@ -35,15 +35,16 @@ http://ignite.apache.org + org.apache.ignite - ignite-core + ignite-aws ${project.version} org.apache.ignite - ignite-core + ignite-aws ${project.version} test-jar test @@ -51,26 +52,27 @@ org.apache.ignite - ignite-log4j + ignite-cloud ${project.version} org.apache.ignite - ignite-spring + ignite-cloud ${project.version} + test-jar test org.apache.ignite - ignite-aws + ignite-core ${project.version} org.apache.ignite - ignite-aws + ignite-core ${project.version} test-jar test @@ -78,18 +80,31 @@ org.apache.ignite - ignite-cloud + ignite-jta ${project.version} org.apache.ignite - ignite-cloud + ignite-jta ${project.version} test-jar test + + org.apache.ignite + ignite-log4j + ${project.version} + + + + org.apache.ignite + ignite-spring + ${project.version} + test + + org.apache.ignite ignite-web @@ -104,6 +119,21 @@ test + + + javax.resource + connector-api + 1.5 + test + + + + org.apache.tomcat + catalina + 6.0.43 + test + + org.eclipse.jetty jetty-servlets @@ -117,5 +147,34 @@ ${jetty.version} test + + + org.springframework + spring-beans + ${spring.version} + test + + + + org.springframework + spring-core + ${spring.version} + test + + + + org.springframework + spring-context + ${spring.version} + test + + + + org.ow2.jotm + jotm-core + 2.1.9 + test + + diff --git a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java index 3ab1d7f4850cc..97f65c5c84b90 100644 --- a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java +++ b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java @@ -40,6 +40,9 @@ public static TestSuite suite() throws Exception { /* --- CLOUDS --- */ suite.addTest(IgniteCloudTestSuite.suite()); + /* --- JTA --- */ + suite.addTest(IgniteJtaTestSuite.suite()); + /* --- WEB SESSIONS --- */ suite.addTest(IgniteWebSessionSelfTestSuite.suite()); diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaLookupClassNameSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaLookupClassNameSelfTest.java index 2e322f86fb5e9..1efbda66934bc 100644 --- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaLookupClassNameSelfTest.java +++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridPartitionedCacheJtaLookupClassNameSelfTest.java @@ -26,6 +26,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testsuites.IgniteIgnore; /** * Lookup class name based JTA integration test using PARTITIONED cache. @@ -39,9 +40,8 @@ public class GridPartitionedCacheJtaLookupClassNameSelfTest extends AbstractCach /** * */ + @IgniteIgnore(value = "https://issues.apache.org/jira/browse/IGNITE-1094", forceFailure = true) public void testUncompatibleTmLookup() { - fail("https://issues.apache.org/jira/browse/IGNITE-1094"); - final IgniteEx ignite = grid(0); final CacheConfiguration cacheCfg = new CacheConfiguration(); diff --git a/modules/jta/src/test/java/org/apache/ignite/testsuites/IgniteJtaTestSuite.java b/modules/jta/src/test/java/org/apache/ignite/testsuites/IgniteJtaTestSuite.java index 60c20ddafbaf7..4ae5df06d54ec 100644 --- a/modules/jta/src/test/java/org/apache/ignite/testsuites/IgniteJtaTestSuite.java +++ b/modules/jta/src/test/java/org/apache/ignite/testsuites/IgniteJtaTestSuite.java @@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.GridReplicatedCacheJtaFactoryUseSyncSelfTest; import org.apache.ignite.internal.processors.cache.GridReplicatedCacheJtaLookupClassNameSelfTest; import org.apache.ignite.internal.processors.cache.GridJtaLifecycleAwareSelfTest; +import org.apache.ignite.testframework.IgniteTestSuite; /** * JTA integration tests. @@ -38,7 +39,7 @@ public class IgniteJtaTestSuite extends TestSuite { * @throws Exception Thrown in case of the failure. */ public static TestSuite suite() throws Exception { - TestSuite suite = new TestSuite("JTA Integration Test Suite"); + TestSuite suite = new IgniteTestSuite("JTA Integration Test Suite"); suite.addTestSuite(GridPartitionedCacheJtaFactorySelfTest.class); suite.addTestSuite(GridReplicatedCacheJtaFactorySelfTest.class); From d595345765db2151ff432beecd478ce056393593 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Wed, 28 Sep 2016 15:08:38 +0300 Subject: [PATCH 193/487] IGNITE-3990: Moved failing Spring tests to "ignore" module. --- .../ignite/testframework/IgniteTestSuite.java | 12 ++++-------- modules/ignored-tests/pom.xml | 7 +++++++ .../ignite/testsuites/IgniteIgnoredTestSuite.java | 3 +++ .../p2p/GridP2PUserVersionChangeSelfTest.java | 7 ++----- .../testsuites/IgniteResourceSelfTestSuite.java | 11 ++++++----- .../ignite/testsuites/IgniteSpringTestSuite.java | 15 ++++++++------- 6 files changed, 30 insertions(+), 25 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java index 6fc7917733214..6dad032b1c0ba 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java @@ -101,16 +101,12 @@ public IgniteTestSuite(@Nullable Class theClass, @Nullable S setName(name); } - /** - * Adds a test to the suite. - */ - @Override public void addTest(Test test) { - super.addTest(test); - } - /** {@inheritDoc} */ @Override public void addTestSuite(Class testClass) { - addTest(new IgniteTestSuite(testClass, ignoredOnly)); + IgniteTestSuite suite = new IgniteTestSuite(testClass, ignoredOnly); + + if (suite.testCount() > 0) + addTest(suite); } /** diff --git a/modules/ignored-tests/pom.xml b/modules/ignored-tests/pom.xml index 661e237921ccf..336485c30043c 100644 --- a/modules/ignored-tests/pom.xml +++ b/modules/ignored-tests/pom.xml @@ -102,6 +102,13 @@ org.apache.ignite ignite-spring ${project.version} + + + + org.apache.ignite + ignite-spring + ${project.version} + test-jar test diff --git a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java index 97f65c5c84b90..2188cd67677bb 100644 --- a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java +++ b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java @@ -43,6 +43,9 @@ public static TestSuite suite() throws Exception { /* --- JTA --- */ suite.addTest(IgniteJtaTestSuite.suite()); + /* --- Spring --- */ + suite.addTest(IgniteSpringTestSuite.suite()); + /* --- WEB SESSIONS --- */ suite.addTest(IgniteWebSessionSelfTestSuite.suite()); diff --git a/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java index 8efa440c013dc..0b32b045f583e 100644 --- a/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java @@ -41,6 +41,7 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.config.GridTestProperties; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.testsuites.IgniteIgnore; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; @@ -282,14 +283,10 @@ public void testRedeployOnNodeRestartSharedMode() throws Exception { } /** - * TODO: IGNITE-604. - * * @throws Exception If failed. */ + @IgniteIgnore(value = "https://issues.apache.org/jira/browse/IGNITE-604", forceFailure = true) public void testCacheRedeployVersionChangeContinuousMode() throws Exception { - // Build execution timeout if try to run test on TC. - fail("https://issues.apache.org/jira/browse/IGNITE-604"); - depMode = DeploymentMode.CONTINUOUS; try { diff --git a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteResourceSelfTestSuite.java b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteResourceSelfTestSuite.java index 742190b82325e..e120aaa41cd8f 100644 --- a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteResourceSelfTestSuite.java +++ b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteResourceSelfTestSuite.java @@ -22,6 +22,7 @@ import org.apache.ignite.internal.processors.resource.GridResourceProcessorSelfTest; import org.apache.ignite.internal.processors.resource.GridServiceInjectionSelfTest; import org.apache.ignite.internal.processors.resource.GridSpringResourceInjectionSelfTest; +import org.apache.ignite.testframework.IgniteTestSuite; /** * Ignite resource injection test Suite. @@ -33,12 +34,12 @@ public class IgniteResourceSelfTestSuite extends TestSuite { * @throws Exception If failed. */ public static TestSuite suite() throws Exception { - TestSuite suite = new TestSuite("Ignite Resource Injection Test Suite"); + TestSuite suite = new IgniteTestSuite("Ignite Resource Injection Test Suite"); - suite.addTest(new TestSuite(GridResourceProcessorSelfTest.class)); - suite.addTest(new TestSuite(GridLoggerInjectionSelfTest.class)); - suite.addTest(new TestSuite(GridServiceInjectionSelfTest.class)); - suite.addTest(new TestSuite(GridSpringResourceInjectionSelfTest.class)); + suite.addTestSuite(GridResourceProcessorSelfTest.class); + suite.addTestSuite(GridLoggerInjectionSelfTest.class); + suite.addTestSuite(GridServiceInjectionSelfTest.class); + suite.addTestSuite(GridSpringResourceInjectionSelfTest.class); return suite; } diff --git a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java index c4a4b75c5171d..fff81089c2f11 100644 --- a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java +++ b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java @@ -33,6 +33,7 @@ import org.apache.ignite.spring.IgniteExcludeInConfigurationTest; import org.apache.ignite.spring.IgniteStartFromStreamConfigurationTest; import org.apache.ignite.spring.injection.GridServiceInjectionSpringResourceTest; +import org.apache.ignite.testframework.IgniteTestSuite; import org.apache.ignite.transactions.spring.GridSpringTransactionManagerSelfTest; import org.apache.ignite.transactions.spring.SpringTransactionManagerContextInjectionTest; @@ -45,23 +46,23 @@ public class IgniteSpringTestSuite extends TestSuite { * @throws Exception Thrown in case of the failure. */ public static TestSuite suite() throws Exception { - TestSuite suite = new TestSuite("Spring Test Suite"); + TestSuite suite = new IgniteTestSuite("Spring Test Suite"); suite.addTestSuite(GridSpringBeanSerializationSelfTest.class); suite.addTestSuite(GridFactorySelfTest.class); suite.addTest(IgniteResourceSelfTestSuite.suite()); - suite.addTest(new TestSuite(IgniteExcludeInConfigurationTest.class)); + suite.addTestSuite(IgniteExcludeInConfigurationTest.class); // Tests moved to this suite since they require Spring functionality. - suite.addTest(new TestSuite(GridP2PUserVersionChangeSelfTest.class)); + suite.addTestSuite(GridP2PUserVersionChangeSelfTest.class); - suite.addTest(new TestSuite(GridSpringCacheManagerSelfTest.class)); + suite.addTestSuite(GridSpringCacheManagerSelfTest.class); - suite.addTest(new TestSuite(IgniteDynamicCacheConfigTest.class)); + suite.addTestSuite(IgniteDynamicCacheConfigTest.class); - suite.addTest(new TestSuite(IgniteStartFromStreamConfigurationTest.class)); + suite.addTestSuite(IgniteStartFromStreamConfigurationTest.class); suite.addTestSuite(CacheSpringStoreSessionListenerSelfTest.class); @@ -70,7 +71,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CachePojoStoreXmlSelfTest.class); suite.addTestSuite(CachePojoStoreXmlWithSqlEscapeSelfTest.class); - suite.addTest(new TestSuite(GridSpringTransactionManagerSelfTest.class)); + suite.addTestSuite(GridSpringTransactionManagerSelfTest.class); suite.addTestSuite(GridServiceInjectionSpringResourceTest.class); From e3f13455d4273e615727d0410783e3719db98f76 Mon Sep 17 00:00:00 2001 From: sboikov Date: Wed, 28 Sep 2016 12:56:17 +0300 Subject: [PATCH 194/487] ignite-2833 Need call 'touch' for cache entry if it was obtained using 'entryEx'. (cherry picked from commit 17c2fc0) --- .../processors/cache/GridCacheTtlManager.java | 32 ++++++++++--------- .../GridCacheAbstractFullApiSelfTest.java | 2 +- 2 files changed, 18 insertions(+), 16 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java index 996544fdb4dd9..0f855fe6dca91 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java @@ -129,26 +129,28 @@ public boolean expire(int amount) { if (log.isTraceEnabled()) log.trace("Trying to remove expired entry from cache: " + e); - boolean touch = false; + boolean touch = e.ctx.isSwapOrOffheapEnabled(); - GridCacheEntryEx entry = e.ctx.cache().entryEx(e.key); + GridCacheEntryEx entry = touch ? e.ctx.cache().entryEx(e.key) : e.ctx.cache().peekEx(e.key); - while (true) { - try { - if (entry.onTtlExpired(obsoleteVer)) - touch = false; + if (entry != null) { + while (true) { + try { + if (entry.onTtlExpired(obsoleteVer)) + touch = false; - break; - } - catch (GridCacheEntryRemovedException e0) { - entry = entry.context().cache().entryEx(entry.key()); + break; + } + catch (GridCacheEntryRemovedException e0) { + entry = entry.context().cache().entryEx(entry.key()); - touch = true; + touch = true; + } } - } - if (touch) - entry.context().evicts().touch(entry, null); + if (touch) + entry.context().evicts().touch(entry, null); + } } } @@ -216,7 +218,7 @@ private static class EntryWrapper implements Comparable { private final GridCacheContext ctx; /** Cache Object Key */ - private final CacheObject key; + private final KeyCacheObject key; /** * @param entry Cache entry to create wrapper for. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java index a31c82ef72bca..e7daf2b6fef2d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java @@ -4057,7 +4057,7 @@ public void testCompactExpired() throws Exception { // Peek will actually remove entry from cache. assertNull(cache.localPeek(key)); - assert cache.localSize() == 0; + assertEquals(0, cache.localSize()); // Clear readers, if any. cache.remove(key); From b2faa339acb2eea24e6dd5e0c21fc3d3d0592ff6 Mon Sep 17 00:00:00 2001 From: sboikov Date: Wed, 28 Sep 2016 13:47:51 +0300 Subject: [PATCH 195/487] ignite-3621 Fixed 'testEvictExpired'. (cherry picked from commit 4ff19c2) --- .../cache/GridCacheAbstractFullApiSelfTest.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java index e7daf2b6fef2d..52d9a5e78e3f8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java @@ -3475,9 +3475,16 @@ public void testEvictExpired() throws Exception { grid(0).cache(null).withExpiryPolicy(expiry).put(key, 1); + final Affinity aff = ignite(0).affinity(null); + boolean wait = waitForCondition(new GridAbsPredicate() { @Override public boolean apply() { - return cache.localPeek(key) == null; + for (int i = 0; i < gridCount(); i++) { + if (peek(jcache(i), key) != null) + return false; + } + + return true; } }, ttl + 1000); @@ -3496,8 +3503,6 @@ public void testEvictExpired() throws Exception { load(cache, key, true); - Affinity aff = ignite(0).affinity(null); - for (int i = 0; i < gridCount(); i++) { if (aff.isPrimary(grid(i).cluster().localNode(), key)) assertEquals((Integer)1, peek(jcache(i), key)); From 74d2fc2416b8e6bc0598152552021f984a013061 Mon Sep 17 00:00:00 2001 From: sboikov Date: Wed, 28 Sep 2016 14:31:24 +0300 Subject: [PATCH 196/487] ignite-3621 Fixed 'testEvictExpired'. (cherry picked from commit bfe4458) --- ...tomicClientOnlyMultiNodeFullApiSelfTest.java | 17 +++++++++++++++-- ...eAtomicNearOnlyMultiNodeFullApiSelfTest.java | 17 +++++++++++++++-- 2 files changed, 30 insertions(+), 4 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest.java index 927ee6273cf41..65f780b2c3fb8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest.java @@ -36,6 +36,7 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.events.Event; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgnitePredicate; @@ -45,6 +46,7 @@ import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_SWAPPED; import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_UNSWAPPED; +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; /** * @@ -193,7 +195,7 @@ public class GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest extends GridCache @Override public void testEvictExpired() throws Exception { IgniteCache cache = jcache(); - String key = primaryKeysForCache(cache, 1).get(0); + final String key = primaryKeysForCache(cache, 1).get(0); cache.put(key, 1); @@ -204,7 +206,18 @@ public class GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest extends GridCache grid(0).cache(null). withExpiryPolicy(new TouchedExpiryPolicy(new Duration(MILLISECONDS, ttl))).put(key, 1); - Thread.sleep(ttl + 100); + boolean wait = waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + for (int i = 0; i < gridCount(); i++) { + if (peek(jcache(i), key) != null) + return false; + } + + return true; + } + }, ttl + 1000); + + assertTrue("Failed to wait for entry expiration.", wait); // Expired entry should not be swapped. cache.localEvict(Collections.singleton(key)); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest.java index 3ce38f3a4072f..6542e762779c7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest.java @@ -29,9 +29,11 @@ import org.apache.ignite.cache.affinity.Affinity; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; /** * Tests NEAR_ONLY cache. @@ -124,7 +126,7 @@ public class GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest extends GridCacheNe IgniteCache cache = jcache(); - String key = primaryKeysForCache(cache, 1).get(0); + final String key = primaryKeysForCache(cache, 1).get(0); cache.put(key, 1); @@ -135,7 +137,18 @@ public class GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest extends GridCacheNe grid(0).cache(null). withExpiryPolicy(new TouchedExpiryPolicy(new Duration(MILLISECONDS, ttl))).put(key, 1); - Thread.sleep(ttl + 100); + boolean wait = waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + for (int i = 0; i < gridCount(); i++) { + if (peek(jcache(i), key) != null) + return false; + } + + return true; + } + }, ttl + 1000); + + assertTrue("Failed to wait for entry expiration.", wait); // Expired entry should not be swapped. cache.localEvict(Collections.singleton(key)); From d2563dacceea61b19bb6e083e29ebacc28fdd323 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Wed, 28 Sep 2016 15:51:55 +0300 Subject: [PATCH 197/487] IGNITE-3993: Added failing client tests to "ignored" test suite. --- .../client/ClientReconnectionSelfTest.java | 4 +- .../client/router/TcpSslRouterSelfTest.java | 7 +- .../client/suite/IgniteClientTestSuite.java | 71 +++++------ .../ignite/testframework/IgniteTestSuite.java | 116 +++++++++++++++--- .../ignite/testsuites/IgniteIgnore.java | 2 +- modules/ignored-tests/pom.xml | 46 +++++++ .../testsuites/IgniteIgnoredTestSuite.java | 5 +- 7 files changed, 192 insertions(+), 59 deletions(-) diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientReconnectionSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientReconnectionSelfTest.java index 92c8e76ec02bf..f1085b3ec361a 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientReconnectionSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientReconnectionSelfTest.java @@ -24,6 +24,7 @@ import org.apache.ignite.internal.client.impl.connection.GridClientConnectionResetException; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.testsuites.IgniteIgnore; /** * @@ -182,9 +183,8 @@ public void testFailedInit() throws Exception { * * @throws Exception If failed. */ + @IgniteIgnore(value = "https://issues.apache.org/jira/browse/IGNITE-590", forceFailure = true) public void testIdleConnection() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-590"); - int srvsCnt = 4; // TODO: IGNITE-590 it may be wrong value. Need to investigate after IGNITE-590 will be fixed. for (int i = 0; i < srvsCnt; i++) diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/router/TcpSslRouterSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/router/TcpSslRouterSelfTest.java index 1e2a2bdf5425e..3b47ae5ca9d87 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/client/router/TcpSslRouterSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/router/TcpSslRouterSelfTest.java @@ -20,16 +20,13 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.client.ssl.GridSslContextFactory; import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testsuites.IgniteIgnore; /** * */ +@IgniteIgnore(value = "https://issues.apache.org/jira/browse/IGNITE-433", forceFailure = true) public class TcpSslRouterSelfTest extends TcpRouterAbstractSelfTest { - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-433"); - } - /** {@inheritDoc} */ @Override protected boolean useSsl() { return true; diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java index fdb5456113ffb..cc8f9fa976651 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java @@ -57,6 +57,7 @@ import org.apache.ignite.internal.processors.rest.RestProcessorStartSelfTest; import org.apache.ignite.internal.processors.rest.TaskCommandHandlerSelfTest; import org.apache.ignite.internal.processors.rest.protocols.tcp.TcpRestParserSelfTest; +import org.apache.ignite.testframework.IgniteTestSuite; /** * Test suite includes all test that concern REST processors. @@ -66,65 +67,65 @@ public class IgniteClientTestSuite extends TestSuite { * @return Suite that contains all tests for REST. */ public static TestSuite suite() { - TestSuite suite = new TestSuite("Ignite Clients Test Suite"); + TestSuite suite = new IgniteTestSuite("Ignite Clients Test Suite"); - suite.addTest(new TestSuite(RouterFactorySelfTest.class)); + suite.addTestSuite(RouterFactorySelfTest.class); // Parser standalone test. - suite.addTest(new TestSuite(TcpRestParserSelfTest.class)); + suite.addTestSuite(TcpRestParserSelfTest.class); // Test memcache protocol with custom test client. - suite.addTest(new TestSuite(RestMemcacheProtocolSelfTest.class)); + suite.addTestSuite(RestMemcacheProtocolSelfTest.class); // Test custom binary protocol with test client. - suite.addTest(new TestSuite(RestBinaryProtocolSelfTest.class)); + suite.addTestSuite(RestBinaryProtocolSelfTest.class); // Test jetty rest processor - suite.addTest(new TestSuite(JettyRestProcessorSignedSelfTest.class)); - suite.addTest(new TestSuite(JettyRestProcessorUnsignedSelfTest.class)); + suite.addTestSuite(JettyRestProcessorSignedSelfTest.class); + suite.addTestSuite(JettyRestProcessorUnsignedSelfTest.class); // Test TCP rest processor with original memcache client. - suite.addTest(new TestSuite(ClientMemcachedProtocolSelfTest.class)); + suite.addTestSuite(ClientMemcachedProtocolSelfTest.class); - suite.addTest(new TestSuite(RestProcessorStartSelfTest.class)); + suite.addTestSuite(RestProcessorStartSelfTest.class); // Test cache flag conversion. - suite.addTest(new TestSuite(ClientCacheFlagsCodecTest.class)); + suite.addTestSuite(ClientCacheFlagsCodecTest.class); // Test multi-start. - suite.addTest(new TestSuite(RestProcessorMultiStartSelfTest.class)); + suite.addTestSuite(RestProcessorMultiStartSelfTest.class); // Test clients. - suite.addTest(new TestSuite(ClientDataImplSelfTest.class)); - suite.addTest(new TestSuite(ClientComputeImplSelfTest.class)); - suite.addTest(new TestSuite(ClientTcpSelfTest.class)); - suite.addTest(new TestSuite(ClientTcpDirectSelfTest.class)); - suite.addTest(new TestSuite(ClientTcpSslSelfTest.class)); - suite.addTest(new TestSuite(ClientTcpSslDirectSelfTest.class)); + suite.addTestSuite(ClientDataImplSelfTest.class); + suite.addTestSuite(ClientComputeImplSelfTest.class); + suite.addTestSuite(ClientTcpSelfTest.class); + suite.addTestSuite(ClientTcpDirectSelfTest.class); + suite.addTestSuite(ClientTcpSslSelfTest.class); + suite.addTestSuite(ClientTcpSslDirectSelfTest.class); // Test client with many nodes. - suite.addTest(new TestSuite(ClientTcpMultiNodeSelfTest.class)); - suite.addTest(new TestSuite(ClientTcpDirectMultiNodeSelfTest.class)); - suite.addTest(new TestSuite(ClientTcpSslMultiNodeSelfTest.class)); - suite.addTest(new TestSuite(ClientTcpSslDirectMultiNodeSelfTest.class)); - suite.addTest(new TestSuite(ClientTcpUnreachableMultiNodeSelfTest.class)); - suite.addTest(new TestSuite(ClientPreferDirectSelfTest.class)); + suite.addTestSuite(ClientTcpMultiNodeSelfTest.class); + suite.addTestSuite(ClientTcpDirectMultiNodeSelfTest.class); + suite.addTestSuite(ClientTcpSslMultiNodeSelfTest.class); + suite.addTestSuite(ClientTcpSslDirectMultiNodeSelfTest.class); + suite.addTestSuite(ClientTcpUnreachableMultiNodeSelfTest.class); + suite.addTestSuite(ClientPreferDirectSelfTest.class); // Test client with many nodes and in multithreaded scenarios - suite.addTest(new TestSuite(ClientTcpMultiThreadedSelfTest.class)); - suite.addTest(new TestSuite(ClientTcpSslMultiThreadedSelfTest.class)); + suite.addTestSuite(ClientTcpMultiThreadedSelfTest.class); + suite.addTestSuite(ClientTcpSslMultiThreadedSelfTest.class); // Test client authentication. - suite.addTest(new TestSuite(ClientTcpSslAuthenticationSelfTest.class)); + suite.addTestSuite(ClientTcpSslAuthenticationSelfTest.class); - suite.addTest(new TestSuite(ClientTcpConnectivitySelfTest.class)); - suite.addTest(new TestSuite(ClientReconnectionSelfTest.class)); + suite.addTestSuite(ClientTcpConnectivitySelfTest.class); + suite.addTestSuite(ClientReconnectionSelfTest.class); // Rest task command handler test. - suite.addTest(new TestSuite(TaskCommandHandlerSelfTest.class)); + suite.addTestSuite(TaskCommandHandlerSelfTest.class); // Default cache only test. - suite.addTest(new TestSuite(ClientDefaultCacheSelfTest.class)); + suite.addTestSuite(ClientDefaultCacheSelfTest.class); suite.addTestSuite(ClientFutureAdapterSelfTest.class); suite.addTestSuite(ClientPropertiesConfigurationSelfTest.class); @@ -134,13 +135,13 @@ public static TestSuite suite() { suite.addTestSuite(ClientByteUtilsTest.class); // Router tests. - suite.addTest(new TestSuite(TcpRouterSelfTest.class)); - suite.addTest(new TestSuite(TcpSslRouterSelfTest.class)); - suite.addTest(new TestSuite(TcpRouterMultiNodeSelfTest.class)); + suite.addTestSuite(TcpRouterSelfTest.class); + suite.addTestSuite(TcpSslRouterSelfTest.class); + suite.addTestSuite(TcpRouterMultiNodeSelfTest.class); - suite.addTest(new TestSuite(ClientFailedInitSelfTest.class)); + suite.addTestSuite(ClientFailedInitSelfTest.class); - suite.addTest(new TestSuite(ClientTcpTaskExecutionAfterTopologyRestartSelfTest.class)); + suite.addTestSuite(ClientTcpTaskExecutionAfterTopologyRestartSelfTest.class); return suite; } diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java index 6dad032b1c0ba..7db9664a7ec9b 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java @@ -129,6 +129,8 @@ private void addTestsFromTestCase(Class theClass) { if(!Modifier.isPublic(theClass.getModifiers())) addTest(warning("Class " + theClass.getName() + " is not public")); else { + IgnoreDescriptor clsIgnore = IgnoreDescriptor.forClass(theClass); + Class superCls = theClass; int testAdded = 0; @@ -140,7 +142,7 @@ private void addTestsFromTestCase(Class theClass) { Method[] methods = MethodSorter.getDeclaredMethods(superCls); for (Method each : methods) { - if (addTestMethod(each, names, theClass)) + if (addTestMethod(each, names, theClass, clsIgnore)) testAdded++; else testIgnored++; @@ -158,9 +160,11 @@ private void addTestsFromTestCase(Class theClass) { * @param m Test method. * @param names Test name list. * @param theClass Test class. + * @param clsIgnore Class ignore descriptor (if any). * @return Whether test method was added. */ - private boolean addTestMethod(Method m, List names, Class theClass) { + private boolean addTestMethod(Method m, List names, Class theClass, + @Nullable IgnoreDescriptor clsIgnore) { String name = m.getName(); if (names.contains(name)) @@ -175,25 +179,20 @@ private boolean addTestMethod(Method m, List names, Class theClass) { names.add(name); - boolean hasIgnore = m.isAnnotationPresent(IgniteIgnore.class); - - if (ignoredOnly) { - if (hasIgnore) { - IgniteIgnore ignore = m.getAnnotation(IgniteIgnore.class); + IgnoreDescriptor ignore = IgnoreDescriptor.forMethod(theClass, m); - String reason = ignore.value(); - - if (F.isEmpty(reason)) - throw new IllegalArgumentException("Reason is not set for ignored test [class=" + - theClass.getName() + ", method=" + name + ']'); + if (ignore == null) + ignore = clsIgnore; + if (ignoredOnly) { + if (ignore != null) { Test test = createTest(theClass, name); if (ignore.forceFailure()) { if (test instanceof GridAbstractTest) - ((GridAbstractTest)test).forceFailure(ignore.value()); + ((GridAbstractTest)test).forceFailure(ignore.reason()); else - test = new ForcedFailure(name, ignore.value()); + test = new ForcedFailure(name, ignore.reason()); } addTest(test); @@ -202,7 +201,7 @@ private boolean addTestMethod(Method m, List names, Class theClass) { } } else { - if (!hasIgnore) { + if (ignore == null) { addTest(createTest(theClass, name)); return true; @@ -250,6 +249,93 @@ private static boolean ignoreDefault() { return res != null && res; } + /** + * Ignore descriptor. + */ + private static class IgnoreDescriptor { + /** Reason. */ + private final String reason; + + /** Force failure. */ + private final boolean forceFailure; + + /** + * Get descriptor for class (if any). + * + * @param cls Class. + * @return Descriptor or {@code null}. + */ + @Nullable public static IgnoreDescriptor forClass(Class cls) { + Class cls0 = cls; + + while (Test.class.isAssignableFrom(cls0)) { + if (cls0.isAnnotationPresent(IgniteIgnore.class)) { + IgniteIgnore ignore = (IgniteIgnore)cls0.getAnnotation(IgniteIgnore.class); + + String reason = ignore.value(); + + if (F.isEmpty(reason)) + throw new IllegalArgumentException("Reason is not set for ignored test [class=" + + cls0.getName() + ']'); + + return new IgnoreDescriptor(reason, ignore.forceFailure()); + } + + cls0 = cls0.getSuperclass(); + } + + return null; + } + + /** + * Get descriptor for method (if any). + * + * @param cls Class. + * @param mthd Method. + * @return Descriptor or {@code null}. + */ + @Nullable public static IgnoreDescriptor forMethod(Class cls, Method mthd) { + if (mthd.isAnnotationPresent(IgniteIgnore.class)) { + IgniteIgnore ignore = mthd.getAnnotation(IgniteIgnore.class); + + String reason = ignore.value(); + + if (F.isEmpty(reason)) + throw new IllegalArgumentException("Reason is not set for ignored test [class=" + + cls.getName() + ", method=" + mthd.getName() + ']'); + + return new IgnoreDescriptor(reason, ignore.forceFailure()); + } + else + return null; + } + + /** + * Constructor. + * + * @param reason Reason. + * @param forceFailure Force failure. + */ + private IgnoreDescriptor(String reason, boolean forceFailure) { + this.reason = reason; + this.forceFailure = forceFailure; + } + + /** + * @return Reason. + */ + public String reason() { + return reason; + } + + /** + * @return Force failure. + */ + public boolean forceFailure() { + return forceFailure; + } + } + /** * Test case simulating failure. */ diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgnore.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgnore.java index dbb1d7a01326a..05af2a27e6a91 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgnore.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgnore.java @@ -26,7 +26,7 @@ * Annotation which indicates that the test is ignored. */ @Retention(RetentionPolicy.RUNTIME) -@Target({ElementType.METHOD}) +@Target({ElementType.METHOD, ElementType.TYPE}) public @interface IgniteIgnore { /** * Reason for ignore (usually link to JIRA ticket). diff --git a/modules/ignored-tests/pom.xml b/modules/ignored-tests/pom.xml index 336485c30043c..142754e10fa8f 100644 --- a/modules/ignored-tests/pom.xml +++ b/modules/ignored-tests/pom.xml @@ -50,6 +50,20 @@ test + + org.apache.ignite + ignite-clients + ${project.version} + + + + org.apache.ignite + ignite-clients + ${project.version} + test-jar + test + + org.apache.ignite ignite-cloud @@ -78,6 +92,12 @@ test + + org.apache.ignite + ignite-indexing + ${project.version} + + org.apache.ignite ignite-jta @@ -98,6 +118,12 @@ ${project.version} + + org.apache.ignite + ignite-rest-http + ${project.version} + + org.apache.ignite ignite-spring @@ -127,6 +153,26 @@ + + com.google.code.simple-spring-memcached + spymemcached + 2.7.3 + test + + + commons-codec + commons-codec + + + + + + commons-io + commons-io + 2.4 + test + + javax.resource connector-api diff --git a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java index 2188cd67677bb..fd92a35aeae0b 100644 --- a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java +++ b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java @@ -37,13 +37,16 @@ public static TestSuite suite() throws Exception { /* --- AWS --- */ suite.addTest(IgniteS3TestSuite.suite()); + /* --- CLIENTS --- */ + suite.addTest(org.apache.ignite.internal.client.suite.IgniteClientTestSuite.suite()); + /* --- CLOUDS --- */ suite.addTest(IgniteCloudTestSuite.suite()); /* --- JTA --- */ suite.addTest(IgniteJtaTestSuite.suite()); - /* --- Spring --- */ + /* --- SPRING --- */ suite.addTest(IgniteSpringTestSuite.suite()); /* --- WEB SESSIONS --- */ From 78144c4c9d6200ceef8b666a186039685f053381 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Wed, 28 Sep 2016 16:52:13 +0300 Subject: [PATCH 198/487] Fixed incorrect test count calculation leading to afterTestsStopped() not being called. --- .../ignite/testframework/IgniteTestSuite.java | 79 ++++++++++++++++--- .../junits/GridAbstractTest.java | 31 +++++++- 2 files changed, 94 insertions(+), 16 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java index 7db9664a7ec9b..1cf69ae16e502 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java @@ -29,6 +29,7 @@ import java.lang.reflect.Method; import java.lang.reflect.Modifier; import java.util.ArrayList; +import java.util.LinkedList; import java.util.List; /** @@ -134,7 +135,9 @@ private void addTestsFromTestCase(Class theClass) { Class superCls = theClass; int testAdded = 0; - int testIgnored = 0; + int testSkipped = 0; + + LinkedList addedTests = new LinkedList<>(); for(List names = new ArrayList<>(); Test.class.isAssignableFrom(superCls); superCls = superCls.getSuperclass()) { @@ -142,15 +145,29 @@ private void addTestsFromTestCase(Class theClass) { Method[] methods = MethodSorter.getDeclaredMethods(superCls); for (Method each : methods) { - if (addTestMethod(each, names, theClass, clsIgnore)) + AddResult res = addTestMethod(each, names, theClass, clsIgnore); + + if (res.added()) { testAdded++; + + addedTests.add(res.test()); + } else - testIgnored++; + testSkipped++; } } - if(testAdded == 0 && testIgnored == 0) + if(testAdded == 0 && testSkipped == 0) addTest(warning("No tests found in " + theClass.getName())); + + // Populate tests count. + for (Test test : addedTests) { + if (test instanceof GridAbstractTest) { + GridAbstractTest test0 = (GridAbstractTest)test; + + test0.forceTestCount(addedTests.size()); + } + } } } @@ -161,20 +178,20 @@ private void addTestsFromTestCase(Class theClass) { * @param names Test name list. * @param theClass Test class. * @param clsIgnore Class ignore descriptor (if any). - * @return Whether test method was added. + * @return Result. */ - private boolean addTestMethod(Method m, List names, Class theClass, + private AddResult addTestMethod(Method m, List names, Class theClass, @Nullable IgnoreDescriptor clsIgnore) { String name = m.getName(); if (names.contains(name)) - return false; + return new AddResult(false, null); if (!isPublicTestMethod(m)) { if (isTestMethod(m)) addTest(warning("Test method isn't public: " + m.getName() + "(" + theClass.getCanonicalName() + ")")); - return false; + return new AddResult(false, null); } names.add(name); @@ -197,18 +214,20 @@ private boolean addTestMethod(Method m, List names, Class theClass, addTest(test); - return true; + return new AddResult(true, test); } } else { if (ignore == null) { - addTest(createTest(theClass, name)); + Test test = createTest(theClass, name); + + addTest(test); - return true; + return new AddResult(true, test); } } - return false; + return new AddResult(false, null); } /** @@ -336,6 +355,42 @@ public boolean forceFailure() { } } + /** + * Test add result. + */ + private static class AddResult { + /** Result. */ + private final boolean added; + + /** Test */ + private final Test test; + + /** + * Constructor. + * + * @param added Result. + * @param test Test. + */ + public AddResult(boolean added, Test test) { + this.added = added; + this.test = test; + } + + /** + * @return Result. + */ + public boolean added() { + return added; + } + + /** + * @return Test. + */ + public Test test() { + return test; + } + } + /** * Test case simulating failure. */ diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java index 8d6fd07d90e62..aa90af0b54026 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java @@ -161,6 +161,12 @@ public abstract class GridAbstractTest extends TestCase { /** Force failure message. */ private String forceFailureMsg; + /** Whether test count is known is advance. */ + private boolean forceTestCnt; + + /** Number of tests. */ + private int testCnt; + /** * */ @@ -1769,6 +1775,15 @@ public void forceFailure(@Nullable String msg) { forceFailureMsg = msg; } + /** + * Set test count. + */ + public void forceTestCount(int cnt) { + testCnt = cnt; + + forceTestCnt = true; + } + /** * @throws Throwable If failed. */ @@ -2076,11 +2091,19 @@ public void incrementStopped() { */ public int getNumberOfTests() { if (numOfTests == -1) { - int cnt = 0; + GridAbstractTest this0 = GridAbstractTest.this; + + int cnt; - for (Method m : GridAbstractTest.this.getClass().getMethods()) - if (m.getName().startsWith("test") && Modifier.isPublic(m.getModifiers())) - cnt++; + if (this0.forceTestCnt) + cnt = this0.testCnt; + else { + cnt = 0; + + for (Method m : this0.getClass().getMethods()) + if (m.getName().startsWith("test") && Modifier.isPublic(m.getModifiers())) + cnt++; + } numOfTests = cnt; } From e3dfdecc3607b5f3183bfcb1ce36c57543a8965f Mon Sep 17 00:00:00 2001 From: Alexander Paschenko Date: Wed, 28 Sep 2016 16:46:46 +0300 Subject: [PATCH 199/487] Validate hash code presence in BinaryObject. Fixes #928 --- .../binary/BinaryClassDescriptor.java | 16 ++++- .../internal/binary/BinaryEnumObjectImpl.java | 5 ++ .../internal/binary/BinaryObjectEx.java | 8 +++ .../internal/binary/BinaryObjectImpl.java | 7 ++ .../binary/BinaryObjectOffheapImpl.java | 7 ++ .../ignite/internal/binary/BinaryUtils.java | 5 +- .../internal/binary/BinaryWriterExImpl.java | 6 +- .../builder/BinaryObjectBuilderImpl.java | 11 +++- .../processors/cache/GridCacheUtils.java | 5 ++ .../ignite/internal/util/IgniteUtils.java | 23 ++++++- ...dCacheStoreManagerDeserializationTest.java | 1 + .../cache/GridCacheUtilsSelfTest.java | 64 ++++++++++++++++++- ...lCacheStoreManagerDeserializationTest.java | 2 +- ...ridCacheBinaryObjectsAbstractSelfTest.java | 31 +++++++++ 14 files changed, 182 insertions(+), 9 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java index 083057ddd5eea..4c824d4409fb1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java @@ -41,6 +41,7 @@ import org.apache.ignite.binary.Binarylizable; import org.apache.ignite.internal.processors.cache.CacheObjectImpl; import org.apache.ignite.internal.util.GridUnsafe; +import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -113,6 +114,9 @@ public class BinaryClassDescriptor { /** */ private final boolean excluded; + /** */ + private final boolean overridesHashCode; + /** */ private final Class[] intfs; @@ -164,6 +168,8 @@ public class BinaryClassDescriptor { this.mapper = mapper; this.registered = registered; + overridesHashCode = IgniteUtils.overridesEqualsAndHashCode(cls); + schemaReg = ctx.schemaRegistry(typeId); excluded = MarshallerExclusions.isExcluded(cls); @@ -845,7 +851,15 @@ private boolean preWrite(BinaryWriterExImpl writer, Object obj) { * @param obj Object. */ private void postWrite(BinaryWriterExImpl writer, Object obj) { - writer.postWrite(userType, registered, obj instanceof CacheObjectImpl ? 0 : obj.hashCode()); + if (obj instanceof CacheObjectImpl) + writer.postWrite(userType, registered, 0, false); + else if (obj instanceof BinaryObjectEx) { + boolean flagSet = ((BinaryObjectEx)obj).isFlagSet(BinaryUtils.FLAG_EMPTY_HASH_CODE); + + writer.postWrite(userType, registered, obj.hashCode(), !flagSet); + } + else + writer.postWrite(userType, registered, obj.hashCode(), overridesHashCode); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java index c9874ed1cbc7c..dcfcc9d39f40f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java @@ -138,6 +138,11 @@ public BinaryEnumObjectImpl(BinaryContext ctx, byte[] arr) { return BinaryUtils.type(ctx, this); } + /** {@inheritDoc} */ + @Override public boolean isFlagSet(short flag) { + return false; + } + /** {@inheritDoc} */ @Override public F field(String fieldName) throws BinaryObjectException { return null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectEx.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectEx.java index e3566bc2e7fba..4e137b7c816ce 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectEx.java @@ -38,4 +38,12 @@ public interface BinaryObjectEx extends BinaryObject { * @throws BinaryObjectException If failed. */ @Nullable public BinaryType rawType() throws BinaryObjectException; + + /** + * Check if flag set. + * + * @param flag flag to check. + * @return {@code true} if flag is set, {@code false} otherwise. + */ + public boolean isFlagSet(short flag); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java index 7b42c03c7d94a..f37d7c2051e36 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java @@ -245,6 +245,13 @@ public void context(BinaryContext ctx) { return true; } + /** {@inheritDoc} */ + @Override public boolean isFlagSet(short flag) { + short flags = BinaryPrimitives.readShort(arr, start + GridBinaryMarshaller.FLAGS_POS); + + return BinaryUtils.isFlagSet(flags, flag); + } + /** {@inheritDoc} */ @Override public int typeId() { int off = start + GridBinaryMarshaller.TYPE_ID_POS; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java index 2225b7a09ae9a..9cbbaa2767092 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java @@ -144,6 +144,13 @@ public BinaryObject heapCopy() { return false; } + /** {@inheritDoc} */ + @Override public boolean isFlagSet(short flag) { + short flags = BinaryPrimitives.readShort(ptr, start + GridBinaryMarshaller.FLAGS_POS); + + return BinaryUtils.isFlagSet(flags, flag); + } + /** {@inheritDoc} */ @Nullable @Override public BinaryType type() throws BinaryObjectException { return BinaryUtils.typeProxy(ctx, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java index b5834a5254425..25d87ffe802be 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java @@ -109,6 +109,9 @@ public class BinaryUtils { /** Flag: compact footer, no field IDs. */ public static final short FLAG_COMPACT_FOOTER = 0x0020; + /** Flag: no hash code has been set. */ + public static final short FLAG_EMPTY_HASH_CODE = 0x0040; + /** Offset which fits into 1 byte. */ public static final int OFFSET_1 = 1; @@ -305,7 +308,7 @@ public static boolean isCompactFooter(short flags) { * @param flag Flag. * @return {@code True} if flag is set in flags. */ - private static boolean isFlagSet(short flags, short flag) { + static boolean isFlagSet(short flags, short flag) { return (flags & flag) == flag; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java index 1a818199b9944..22b4d1f481f5d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java @@ -245,8 +245,9 @@ public void preWrite(@Nullable String clsName) { * @param userType User type flag. * @param registered Whether type is registered. * @param hashCode Hash code. + * @param isHashCodeSet Hash code presence flag. */ - public void postWrite(boolean userType, boolean registered, int hashCode) { + public void postWrite(boolean userType, boolean registered, int hashCode, boolean isHashCodeSet) { short flags; boolean useCompactFooter; @@ -303,6 +304,9 @@ else if (offsetByteCnt == BinaryUtils.OFFSET_2) } } + if (!isHashCodeSet) + flags |= BinaryUtils.FLAG_EMPTY_HASH_CODE; + // Actual write. int retPos = out.position(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java index 086da5c689057..2c761925e68af 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java @@ -86,6 +86,9 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder { /** */ private int hashCode; + /** */ + private boolean isHashCodeSet; + /** * @param clsName Class name. * @param ctx Binary context. @@ -117,7 +120,7 @@ public BinaryObjectBuilderImpl(BinaryContext ctx, int typeId, String typeName) { */ public BinaryObjectBuilderImpl(BinaryObjectImpl obj) { this(new BinaryBuilderReader(obj), obj.start()); - + isHashCodeSet = !obj.isFlagSet(BinaryUtils.FLAG_EMPTY_HASH_CODE); reader.registerObject(this); } @@ -329,7 +332,8 @@ else if (readCache == null) { reader.position(start + BinaryUtils.length(reader, start)); } - writer.postWrite(true, registeredType, hashCode); + //noinspection NumberEquality + writer.postWrite(true, registeredType, hashCode, isHashCodeSet); // Update metadata if needed. int schemaId = writer.schemaId(); @@ -408,9 +412,12 @@ else if (!nullFieldVal) { } /** {@inheritDoc} */ + @SuppressWarnings("UnnecessaryBoxing") @Override public BinaryObjectBuilderImpl hashCode(int hashCode) { this.hashCode = hashCode; + isHashCodeSet = true; + return this; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 1a4ffd502178c..0f4e89bc89963 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -1170,6 +1170,7 @@ private static String capitalize(String str) { /** * Validates that cache key object has overridden equals and hashCode methods. + * Will also check that a BinaryObject has a hash code set. * * @param key Key. * @throws IllegalArgumentException If equals or hashCode is not implemented. @@ -1181,6 +1182,10 @@ public static void validateCacheKey(@Nullable Object key) { if (!U.overridesEqualsAndHashCode(key)) throw new IllegalArgumentException("Cache key must override hashCode() and equals() methods: " + key.getClass().getName()); + + if (U.isHashCodeEmpty(key)) + throw new IllegalArgumentException("Cache key created with BinaryBuilder is missing hash code - " + + "please set it explicitly during building by using BinaryBuilder.hashCode(int)"); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index cdaeab1abe552..501cdb2eaba49 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -149,6 +149,7 @@ import org.apache.ignite.IgniteInterruptedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryRawReader; import org.apache.ignite.binary.BinaryRawWriter; import org.apache.ignite.cluster.ClusterGroupEmptyException; @@ -170,6 +171,8 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteNodeAttributes; +import org.apache.ignite.internal.binary.BinaryObjectEx; +import org.apache.ignite.internal.binary.BinaryUtils; import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.compute.ComputeTaskCancelledCheckedException; @@ -8479,9 +8482,15 @@ public static String consistentId(Collection addrs, int port) { * @return {@code True} if given object has overridden equals and hashCode method. */ public static boolean overridesEqualsAndHashCode(Object obj) { - try { - Class cls = obj.getClass(); + return overridesEqualsAndHashCode(obj.getClass()); + } + /** + * @param cls Class. + * @return {@code True} if given class has overridden equals and hashCode method. + */ + public static boolean overridesEqualsAndHashCode(Class cls) { + try { return !Object.class.equals(cls.getMethod("equals", Object.class).getDeclaringClass()) && !Object.class.equals(cls.getMethod("hashCode").getDeclaringClass()); } @@ -8490,6 +8499,16 @@ public static boolean overridesEqualsAndHashCode(Object obj) { } } + /** + * @param obj Object. + * @return {@code True} if given object is a {@link BinaryObjectEx} and + * has {@link BinaryUtils#FLAG_EMPTY_HASH_CODE} set + */ + public static boolean isHashCodeEmpty(Object obj) { + return obj != null && obj instanceof BinaryObjectEx && + ((BinaryObjectEx)obj).isFlagSet(BinaryUtils.FLAG_EMPTY_HASH_CODE); + } + /** * Checks if error is MAC invalid argument error which ususally requires special handling. * diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java index 4a069a95b9122..39ce33d4bb22a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManagerDeserializationTest.java @@ -279,6 +279,7 @@ private BinaryObject streamBinaryData(final Ignite grid) { for (int i = 0; i < 1; i++) { builder.setField("id", i); + builder.hashCode(i); entity = builder.build(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheUtilsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheUtilsSelfTest.java index d5888e725c8e0..5f2c0046daae9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheUtilsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheUtilsSelfTest.java @@ -18,7 +18,21 @@ package org.apache.ignite.internal.processors.cache; import java.util.concurrent.Callable; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.configuration.BinaryConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.binary.BinaryCachingMetadataHandler; +import org.apache.ignite.internal.binary.BinaryContext; +import org.apache.ignite.internal.binary.BinaryMarshaller; +import org.apache.ignite.internal.binary.BinaryObjectImpl; +import org.apache.ignite.internal.binary.GridBinaryMarshaller; +import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl; +import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.logger.NullLogger; +import org.apache.ignite.marshaller.MarshallerContextTestImpl; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -104,7 +118,8 @@ private static class ExtendsClassWithEqualsAndHashCode2 extends EqualsAndHashCod /** */ - public void testCacheKeyValidation() { + @SuppressWarnings("ResultOfMethodCallIgnored") + public void testCacheKeyValidation() throws IgniteCheckedException { CU.validateCacheKey("key"); CU.validateCacheKey(1); @@ -124,6 +139,53 @@ public void testCacheKeyValidation() { assertThrowsForInvalidKey(new NoHashCode()); assertThrowsForInvalidKey(new WrongEquals()); + + BinaryObjectBuilderImpl binBuilder = new BinaryObjectBuilderImpl(binaryContext(), + EqualsAndHashCode.class.getName()); + + assertThrowsForInvalidKey(binBuilder.build()); + + binBuilder.hashCode(0xFE12); + + BinaryObject binObj = binBuilder.build(); + + CU.validateCacheKey(binObj); + + BinaryObjectBuilderImpl binBuilder2 = new BinaryObjectBuilderImpl((BinaryObjectImpl) binObj); + + CU.validateCacheKey(binBuilder2.build()); + } + + /** + * @return Binary marshaller. + * @throws IgniteCheckedException if failed. + */ + private BinaryMarshaller binaryMarshaller() throws IgniteCheckedException { + IgniteConfiguration iCfg = new IgniteConfiguration(); + + BinaryConfiguration bCfg = new BinaryConfiguration(); + + iCfg.setBinaryConfiguration(bCfg); + + BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), iCfg, new NullLogger()); + + BinaryMarshaller marsh = new BinaryMarshaller(); + + marsh.setContext(new MarshallerContextTestImpl(null)); + + IgniteUtils.invoke(BinaryMarshaller.class, marsh, "setBinaryContext", ctx, iCfg); + + return marsh; + } + + /** + * @return Binary context. + * @throws IgniteCheckedException if failed. + */ + private BinaryContext binaryContext() throws IgniteCheckedException { + GridBinaryMarshaller impl = U.field(binaryMarshaller(), "impl"); + + return impl.context(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalCacheStoreManagerDeserializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalCacheStoreManagerDeserializationTest.java index 827b3cf3a1e54..b86fe53102e9e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalCacheStoreManagerDeserializationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridLocalCacheStoreManagerDeserializationTest.java @@ -86,7 +86,7 @@ public void testBinaryUpdate() throws Exception { final BinaryObjectBuilder builder = grid.binary().builder("custom_type"); - final BinaryObject entity = builder.setField("id", 0).build(); + final BinaryObject entity = builder.setField("id", 0).hashCode(0).build(); cache.put(entity, entity); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java index 3a510c354d02e..7936ea49b7303 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.Callable; import javax.cache.Cache; import javax.cache.processor.EntryProcessor; import javax.cache.processor.EntryProcessorException; @@ -886,6 +887,36 @@ private void checkGetAllAsyncTx(TransactionConcurrency concurrency, TransactionI } } + /** + * @throws Exception If failed. + */ + @SuppressWarnings({ "ThrowableResultOfMethodCallIgnored", "unchecked" }) + public void testPutWithoutHashCode() throws Exception { + final IgniteCache c = jcache(0); + + GridTestUtils.assertThrows(log, new Callable() { + /** {@inheritDoc} */ + @Override public Object call() throws Exception { + c.put(new TestObject(5), 5); + return null; + } + }, IllegalArgumentException.class, "Cache key must override hashCode() and equals() methods: "); + + BinaryObjectBuilder bldr = grid(0).binary().builder(TestObject.class.getName()); + bldr.setField("val", 5); + + final BinaryObject binKey = bldr.build(); + + GridTestUtils.assertThrows(log, new Callable() { + /** {@inheritDoc} */ + @Override public Object call() throws Exception { + c.put(binKey, 5); + return null; + } + }, IllegalArgumentException.class, "Cache key created with BinaryBuilder is missing hash code - " + + "please set it explicitly during building by using BinaryBuilder.hashCode(int)"); + } + /** * @throws Exception if failed. */ From 53229e290f7d6aab9b504693bd2b93155ecd2bad Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Wed, 28 Sep 2016 17:07:47 +0300 Subject: [PATCH 200/487] IGNITE-3995 .NET: Introduced default non-null ASP.NET Session-State Store Provider cache name. This closes #1128. --- .../IgniteSessionStateStoreProviderTest.cs | 13 +++++++++++++ .../IgniteOutputCacheProvider.cs | 2 +- .../IgniteSessionStateStoreProvider.cs | 8 +++++++- .../dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs | 13 +++++++------ 4 files changed, 28 insertions(+), 8 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs index fc239add7034e..2c73359c0d850 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs @@ -133,6 +133,19 @@ public void TestInitialization() stateProvider = GetProvider(); CheckProvider(stateProvider); + + // Omitted cache name results in default cache name (not null). + stateProvider = new IgniteSessionStateStoreProvider(); + + stateProvider.Initialize("testName", new NameValueCollection + { + {GridNameAttr, GridName} + }); + + var cacheNames = Ignition.GetIgnite(GridName).GetCacheNames(); + + Assert.IsFalse(cacheNames.Contains(null)); + Assert.IsTrue(cacheNames.Contains(IgniteSessionStateStoreProvider.DefaultCacheName)); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteOutputCacheProvider.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteOutputCacheProvider.cs index d232726dd790c..27904936b9f76 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteOutputCacheProvider.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteOutputCacheProvider.cs @@ -99,7 +99,7 @@ public override void Initialize(string name, NameValueCollection config) { base.Initialize(name, config); - var cache = ConfigUtil.InitializeCache(config, GetType()); + var cache = ConfigUtil.InitializeCache(config, GetType(), null); _expiryCacheHolder = new ExpiryCacheHolder(cache); } diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs index 1ee6d925b6515..86035dd2037bc 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs @@ -47,6 +47,11 @@ namespace Apache.Ignite.AspNet /// public class IgniteSessionStateStoreProvider : SessionStateStoreProviderBase { + /// + /// The default cache name to be used when cacheName attribute is not specified. + /// + public const string DefaultCacheName = "ASPNET_SESSION_STATE"; + /** Extension id */ private const int ExtensionId = 0; @@ -94,7 +99,8 @@ public override void Initialize(string name, NameValueCollection config) { base.Initialize(name, config); - var cache = ConfigUtil.InitializeCache(config, GetType()); + var cache = ConfigUtil.InitializeCache(config, GetType(), + DefaultCacheName); _expiryCacheHolder = new ExpiryCacheHolder(cache); diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs index 3eb3d90c71677..a162d81677ae2 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs @@ -22,6 +22,7 @@ namespace Apache.Ignite.AspNet.Impl using System.Configuration; using System.Diagnostics; using System.Globalization; + using System.Linq; using Apache.Ignite.Core; using Apache.Ignite.Core.Cache; using Apache.Ignite.Core.Cache.Configuration; @@ -33,24 +34,25 @@ namespace Apache.Ignite.AspNet.Impl internal static class ConfigUtil { /** */ - public const string GridName = "gridName"; + private const string GridName = "gridName"; /** */ - public const string CacheName = "cacheName"; + private const string CacheName = "cacheName"; /** */ - public const string IgniteConfigurationSectionName = "igniteConfigurationSectionName"; + private const string IgniteConfigurationSectionName = "igniteConfigurationSectionName"; /// /// Initializes the cache from configuration. /// - public static ICache InitializeCache(NameValueCollection config, Type callerType) + public static ICache InitializeCache(NameValueCollection config, Type callerType, + string defaultCacheName) { Debug.Assert(config != null); Debug.Assert(callerType != null); var gridName = config[GridName]; - var cacheName = config[CacheName]; + var cacheName = config.AllKeys.Contains(CacheName) ? config[CacheName] : defaultCacheName; var cfgSection = config[IgniteConfigurationSectionName]; try @@ -66,7 +68,6 @@ internal static class ConfigUtil throw new IgniteException(string.Format(CultureInfo.InvariantCulture, "Failed to initialize {0}: {1}", callerType, ex), ex); } - } /// From d1e3a78ae569fa5d5692816db44f2c677e1b8283 Mon Sep 17 00:00:00 2001 From: Valentin Kulichenko Date: Wed, 28 Sep 2016 17:53:42 -0700 Subject: [PATCH 201/487] IGNITE-3191 - Fixed ordering of fields in binary objects --- .../binary/BinaryClassDescriptor.java | 16 +-- .../builder/BinaryObjectBuilderImpl.java | 24 ++--- .../binary/BinaryFieldOrderSelfTest.java | 98 +++++++++++++++++++ .../IgniteBinaryObjectsTestSuite.java | 2 + 4 files changed, 121 insertions(+), 19 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java index 4c824d4409fb1..276dfe5134252 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java @@ -25,7 +25,6 @@ import java.lang.reflect.Proxy; import java.math.BigDecimal; import java.sql.Timestamp; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Date; @@ -33,6 +32,7 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.TreeMap; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.binary.BinaryObjectException; @@ -269,10 +269,9 @@ else if (useOptMarshaller) case OBJECT: // Must not use constructor to honor transient fields semantics. ctor = null; - ArrayList fields0 = new ArrayList<>(); stableFieldsMeta = metaDataEnabled ? new HashMap() : null; - BinarySchema.Builder schemaBuilder = BinarySchema.Builder.newBuilder(); + Map fields0 = new TreeMap<>(); Set duplicates = duplicateFields(cls); @@ -300,9 +299,7 @@ else if (useOptMarshaller) BinaryFieldAccessor fieldInfo = BinaryFieldAccessor.create(f, fieldId); - fields0.add(fieldInfo); - - schemaBuilder.addField(fieldId); + fields0.put(name, fieldInfo); if (metaDataEnabled) stableFieldsMeta.put(name, fieldInfo.mode().typeId()); @@ -310,7 +307,12 @@ else if (useOptMarshaller) } } - fields = fields0.toArray(new BinaryFieldAccessor[fields0.size()]); + fields = fields0.values().toArray(new BinaryFieldAccessor[fields0.size()]); + + BinarySchema.Builder schemaBuilder = BinarySchema.Builder.newBuilder(); + + for (BinaryFieldAccessor field : fields) + schemaBuilder.addField(field.id); stableSchema = schemaBuilder.build(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java index 2c761925e68af..d166051d18f35 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java @@ -17,32 +17,32 @@ package org.apache.ignite.internal.binary.builder; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; import org.apache.ignite.binary.BinaryInvalidTypeException; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.binary.BinaryType; +import org.apache.ignite.internal.binary.BinaryContext; import org.apache.ignite.internal.binary.BinaryMetadata; import org.apache.ignite.internal.binary.BinaryObjectImpl; -import org.apache.ignite.internal.binary.BinaryWriterExImpl; -import org.apache.ignite.internal.binary.GridBinaryMarshaller; -import org.apache.ignite.internal.binary.BinaryContext; +import org.apache.ignite.internal.binary.BinaryObjectOffheapImpl; import org.apache.ignite.internal.binary.BinarySchema; import org.apache.ignite.internal.binary.BinarySchemaRegistry; -import org.apache.ignite.internal.binary.BinaryObjectOffheapImpl; import org.apache.ignite.internal.binary.BinaryUtils; +import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.binary.GridBinaryMarshaller; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.Nullable; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.Map; -import java.util.Set; - /** * */ @@ -522,7 +522,7 @@ private void ensureReadCacheInit() { Object val = val0 == null ? new BinaryValueWithType(BinaryUtils.typeByClass(Object.class), null) : val0; if (assignedVals == null) - assignedVals = new LinkedHashMap<>(); + assignedVals = new TreeMap<>(); Object oldVal = assignedVals.put(name, val); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java new file mode 100644 index 0000000000000..6bb1e1380da8a --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java @@ -0,0 +1,98 @@ +package org.apache.ignite.internal.binary; + +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Test that field ordering doesn't change the schema. + */ +public class BinaryFieldOrderSelfTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setMarshaller(new BinaryMarshaller()); + cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(new TcpDiscoveryVmIpFinder(true))); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + startGrid(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testEquals() throws Exception { + IgniteEx ignite = grid(); + + BinaryObject bo0 = ignite.binary().toBinary(new MyType(222, 333, 111)); + + BinaryObject bo1 = ignite.binary().builder(bo0.type().typeName()). + setField("b", 222). + setField("c", 333). + setField("a", 111). + hashCode(12345). + build(); + + BinaryObject bo2 = ignite.binary().builder(bo0.type().typeName()). + setField("a", 111). + setField("b", 222). + setField("c", 333). + hashCode(12345). + build(); + + assertEquals(12345, bo0.hashCode()); + assertEquals(12345, bo1.hashCode()); + assertEquals(12345, bo2.hashCode()); + + assertTrue(bo0.equals(bo1)); + assertTrue(bo0.equals(bo2)); + assertTrue(bo1.equals(bo2)); + } + + /** + */ + private static class MyType { + /** B. */ + private int b; + + /** C. */ + private int c; + + /** A. */ + private int a; + + /** + * @param b B. + * @param c C. + * @param a A. + */ + MyType(int b, int c, int a) { + this.b = b; + this.c = c; + this.a = a; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object obj) { + return super.equals(obj); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return 12345; + } + } +} 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 c1d9974837fb3..50c6f0b95a95d 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 @@ -22,6 +22,7 @@ import org.apache.ignite.internal.binary.BinaryBasicNameMapperSelfTest; import org.apache.ignite.internal.binary.BinaryConfigurationConsistencySelfTest; import org.apache.ignite.internal.binary.BinaryEnumsSelfTest; +import org.apache.ignite.internal.binary.BinaryFieldOrderSelfTest; import org.apache.ignite.internal.binary.BinaryFieldsHeapSelfTest; import org.apache.ignite.internal.binary.BinaryFieldsOffheapSelfTest; import org.apache.ignite.internal.binary.BinaryFooterOffsetsHeapSelfTest; @@ -104,6 +105,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridBinaryAffinityKeySelfTest.class); suite.addTestSuite(GridBinaryWildcardsSelfTest.class); suite.addTestSuite(BinaryObjectToStringSelfTest.class); + suite.addTestSuite(BinaryFieldOrderSelfTest.class); // Tests for objects with non-compact footers. suite.addTestSuite(BinaryMarshallerNonCompactSelfTest.class); From b280c3efa1eb84c6bc8abbe31ba669b0c24323d8 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Thu, 29 Sep 2016 10:23:54 +0300 Subject: [PATCH 202/487] Fixed missing Apache header. --- .../binary/BinaryFieldOrderSelfTest.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java index 6bb1e1380da8a..e4709480f49a4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java @@ -1,3 +1,20 @@ +/* + * 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.binary; import org.apache.ignite.binary.BinaryObject; From 3e8a1c6b045c231dbb3c972463000f824386aee9 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Thu, 29 Sep 2016 11:08:23 +0300 Subject: [PATCH 203/487] Added node stop to IgniteCacheExpireAndUpdateConsistencyTest. --- .../cache/IgniteCacheExpireAndUpdateConsistencyTest.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheExpireAndUpdateConsistencyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheExpireAndUpdateConsistencyTest.java index 7f54a832becf0..882ed2200b15e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheExpireAndUpdateConsistencyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheExpireAndUpdateConsistencyTest.java @@ -97,6 +97,13 @@ public class IgniteCacheExpireAndUpdateConsistencyTest extends GridCommonAbstrac assertTrue(client.configuration().isClientMode()); } + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + /** * @throws Exception If failed. */ From 062b9b69aa851d5253dfb7f44066b7d749f1ca12 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Thu, 29 Sep 2016 11:26:50 +0300 Subject: [PATCH 204/487] IGNITE-3633 - Enforce key validation for tests. --- .../ignite/internal/processors/cache/GridCacheAdapter.java | 7 +++++++ .../binary/GridCacheBinaryObjectsAbstractSelfTest.java | 6 ++++++ 2 files changed, 13 insertions(+) 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 fe6bb1be27c89..55400abb72021 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 @@ -4890,6 +4890,13 @@ public void onReconnected() { // No-op. } + /** + * For tests only. + */ + public void forceKeyCheck() { + keyCheck = true; + } + /** * Validates that given cache key has overridden equals and hashCode methods and * implements {@link Externalizable}. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java index 7936ea49b7303..150c2454762b5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java @@ -894,6 +894,10 @@ private void checkGetAllAsyncTx(TransactionConcurrency concurrency, TransactionI public void testPutWithoutHashCode() throws Exception { final IgniteCache c = jcache(0); + GridCacheAdapter cache0 = grid(0).context().cache().internalCache(null); + + cache0.forceKeyCheck(); + GridTestUtils.assertThrows(log, new Callable() { /** {@inheritDoc} */ @Override public Object call() throws Exception { @@ -907,6 +911,8 @@ public void testPutWithoutHashCode() throws Exception { final BinaryObject binKey = bldr.build(); + cache0.forceKeyCheck(); + GridTestUtils.assertThrows(log, new Callable() { /** {@inheritDoc} */ @Override public Object call() throws Exception { From a4d7aa343df23a5e00df11c5080cc1762b187161 Mon Sep 17 00:00:00 2001 From: sboikov Date: Thu, 29 Sep 2016 12:04:44 +0300 Subject: [PATCH 205/487] ignite-3621 Fixed 'testEvictExpired'. --- .../cache/IgniteCacheConfigVariationsFullApiTest.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java index 2ca09c836ce7d..6b0e1932e1f2c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java @@ -3336,7 +3336,12 @@ public void testEvictExpired() throws Exception { boolean wait = waitForCondition(new GridAbsPredicate() { @Override public boolean apply() { - return cache.localPeek(key) == null; + for (int i = 0; i < gridCount(); i++) { + if (peek(jcache(i), key) != null) + return false; + } + + return true; } }, ttl + 1000); From ad613afd7165ed2d8bf0a62805fc2506dfe6c59a Mon Sep 17 00:00:00 2001 From: sboikov Date: Thu, 29 Sep 2016 12:04:44 +0300 Subject: [PATCH 206/487] ignite-3621 Fixed 'testEvictExpired'. (cherry picked from commit a4d7aa3) --- .../cache/IgniteCacheConfigVariationsFullApiTest.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java index 2ca09c836ce7d..6b0e1932e1f2c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java @@ -3336,7 +3336,12 @@ public void testEvictExpired() throws Exception { boolean wait = waitForCondition(new GridAbsPredicate() { @Override public boolean apply() { - return cache.localPeek(key) == null; + for (int i = 0; i < gridCount(); i++) { + if (peek(jcache(i), key) != null) + return false; + } + + return true; } }, ttl + 1000); From 22dc2c9e05e5183f838865023c9cb7a8291ac67f Mon Sep 17 00:00:00 2001 From: Alexander Paschenko Date: Thu, 29 Sep 2016 15:43:20 +0300 Subject: [PATCH 207/487] GridNearCacheAdapter, GridDhtCacheAdapter - test fix --- .../cache/distributed/dht/GridDhtCacheAdapter.java | 12 ++++++++++++ .../cache/distributed/near/GridNearCacheAdapter.java | 12 ++++++++++++ 2 files changed, 24 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java index 35e62670ea2a9..8ced02f27fd9e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java @@ -261,6 +261,18 @@ protected GridCacheMapEntryFactory entryFactory() { */ public abstract GridNearCacheAdapter near(); + /** {@inheritDoc} */ + @Override public void forceKeyCheck() { + if (!keyCheck) { + super.forceKeyCheck(); + + GridNearCacheAdapter near = near(); + + if (near != null) + near.forceKeyCheck(); + } + } + /** * @return Partition topology. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java index dd66a334c8ed2..6acf48ea0fa55 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java @@ -114,6 +114,18 @@ protected GridNearCacheAdapter(GridCacheContext ctx) { */ public abstract GridDhtCacheAdapter dht(); + /** {@inheritDoc} */ + @Override public void forceKeyCheck() { + if (!keyCheck) { + super.forceKeyCheck(); + + GridDhtCacheAdapter dht = dht(); + + if (dht != null) + dht.forceKeyCheck(); + } + } + /** {@inheritDoc} */ @Override public void onReconnected() { map = new GridCacheConcurrentMapImpl( From f447559ecba55149452e4e48e2bf44ef1fa1b4d7 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Thu, 29 Sep 2016 17:10:14 +0300 Subject: [PATCH 208/487] ignite-1.6.9 - Fixing tests --- .../cache/distributed/dht/GridDhtCacheAdapter.java | 12 ------------ .../cache/distributed/near/GridNearCacheAdapter.java | 9 ++------- 2 files changed, 2 insertions(+), 19 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java index 8ced02f27fd9e..35e62670ea2a9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java @@ -261,18 +261,6 @@ protected GridCacheMapEntryFactory entryFactory() { */ public abstract GridNearCacheAdapter near(); - /** {@inheritDoc} */ - @Override public void forceKeyCheck() { - if (!keyCheck) { - super.forceKeyCheck(); - - GridNearCacheAdapter near = near(); - - if (near != null) - near.forceKeyCheck(); - } - } - /** * @return Partition topology. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java index 6acf48ea0fa55..4ddad74157987 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java @@ -116,14 +116,9 @@ protected GridNearCacheAdapter(GridCacheContext ctx) { /** {@inheritDoc} */ @Override public void forceKeyCheck() { - if (!keyCheck) { - super.forceKeyCheck(); + super.forceKeyCheck(); - GridDhtCacheAdapter dht = dht(); - - if (dht != null) - dht.forceKeyCheck(); - } + dht().forceKeyCheck(); } /** {@inheritDoc} */ From a4c63ae113931615e940ada2411ebce32dba5db8 Mon Sep 17 00:00:00 2001 From: iveselovskiy Date: Fri, 30 Sep 2016 10:45:13 +0300 Subject: [PATCH 209/487] IGNITE-3998: IGFS: Enabled IgfsAbstractSelfTest.testCreateConsistencyMultithreaded. This closes #1129. --- .../processors/igfs/IgfsAbstractSelfTest.java | 141 +++++++++--------- 1 file changed, 70 insertions(+), 71 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java index e8d65f18e06ab..7058954f01770 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java @@ -1310,77 +1310,76 @@ public void testCreateConsistency() throws Exception { * @throws Exception If failed. */ public void testCreateConsistencyMultithreaded() throws Exception { - // TODO: Enable -// final AtomicBoolean stop = new AtomicBoolean(); -// -// final AtomicInteger createCtr = new AtomicInteger(); // How many times the file was re-created. -// final AtomicReference err = new AtomicReference<>(); -// -// igfs.create(FILE, false).close(); -// -// int threadCnt = 50; -// -// IgniteInternalFuture fut = multithreadedAsync(new Runnable() { -// @SuppressWarnings("ThrowFromFinallyBlock") -// @Override public void run() { -// while (!stop.get() && err.get() == null) { -// IgfsOutputStream os = null; -// -// try { -// os = igfs.create(FILE, true); -// -// os.write(chunk); -// -// os.close(); -// -// createCtr.incrementAndGet(); -// } -// catch (IgniteException e) { -// // No-op. -// } -// catch (IOException e) { -// err.compareAndSet(null, e); -// -// Throwable[] chain = X.getThrowables(e); -// -// Throwable cause = chain[chain.length - 1]; -// -// System.out.println("Failed due to IOException exception. Cause:"); -// cause.printStackTrace(System.out); -// } -// finally { -// if (os != null) -// try { -// os.close(); -// } -// catch (IOException ioe) { -// throw new IgniteException(ioe); -// } -// } -// } -// } -// }, threadCnt); -// -// long startTime = U.currentTimeMillis(); -// -// while (err.get() == null -// && createCtr.get() < 500 -// && U.currentTimeMillis() - startTime < 60 * 1000) -// U.sleep(100); -// -// stop.set(true); -// -// fut.get(); -// -// awaitFileClose(igfs.asSecondary(), FILE); -// -// if (err.get() != null) { -// X.println("Test failed: rethrowing first error: " + err.get()); -// -// throw err.get(); -// } -// -// checkFileContent(igfs, FILE, chunk); + final AtomicBoolean stop = new AtomicBoolean(); + + final AtomicInteger createCtr = new AtomicInteger(); // How many times the file was re-created. + final AtomicReference err = new AtomicReference<>(); + + igfs.create(FILE, false).close(); + + int threadCnt = 50; + + IgniteInternalFuture fut = multithreadedAsync(new Runnable() { + @SuppressWarnings("ThrowFromFinallyBlock") + @Override public void run() { + while (!stop.get() && err.get() == null) { + IgfsOutputStream os = null; + + try { + os = igfs.create(FILE, true); + + os.write(chunk); + + os.close(); + + createCtr.incrementAndGet(); + } + catch (IgniteException e) { + // No-op. + } + catch (IOException e) { + err.compareAndSet(null, e); + + Throwable[] chain = X.getThrowables(e); + + Throwable cause = chain[chain.length - 1]; + + System.out.println("Failed due to IOException exception. Cause:"); + cause.printStackTrace(System.out); + } + finally { + if (os != null) + try { + os.close(); + } + catch (IOException ioe) { + throw new IgniteException(ioe); + } + } + } + } + }, threadCnt); + + long startTime = U.currentTimeMillis(); + + while (err.get() == null + && createCtr.get() < 500 + && U.currentTimeMillis() - startTime < 60 * 1000) + U.sleep(100); + + stop.set(true); + + fut.get(); + + awaitFileClose(igfs.asSecondary(), FILE); + + if (err.get() != null) { + X.println("Test failed: rethrowing first error: " + err.get()); + + throw err.get(); + } + + checkFileContent(igfs, FILE, chunk); } /** From a12298c0baf19407f3110ba3c4f0aab88b67946d Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Fri, 30 Sep 2016 11:00:51 +0300 Subject: [PATCH 210/487] IGNITE-3868: ODBC: Fixed connection string parsing. This closes #1051. --- modules/platforms/cpp/odbc/src/odbc.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp index 9df64d3b14133..7416ad2b5b663 100644 --- a/modules/platforms/cpp/odbc/src/odbc.cpp +++ b/modules/platforms/cpp/odbc/src/odbc.cpp @@ -289,9 +289,7 @@ namespace ignite if (!diag.IsSuccessful()) return diag.GetReturnCode(); - std::string outConnectStr = connection->GetConfiguration().ToConnectString(); - - size_t reslen = CopyStringToBuffer(outConnectStr, + size_t reslen = CopyStringToBuffer(connectStr, reinterpret_cast(outConnectionString), static_cast(outConnectionStringBufferLen)); @@ -753,7 +751,7 @@ namespace ignite LOG_MSG("columnSizeRes: %lld\n", columnSizeRes); LOG_MSG("decimalDigitsRes: %lld\n", decimalDigitsRes); LOG_MSG("nullableRes: %lld\n", nullableRes); - LOG_MSG("columnNameBuf: %s\n", columnNameBuf ? columnNameBuf : ""); + LOG_MSG("columnNameBuf: %s\n", columnNameBuf ? (const char*)columnNameBuf : ""); LOG_MSG("columnNameLen: %d\n", columnNameLen ? *columnNameLen : -1); if (dataType) From c32082fe8b1e02758179c1b7bb61a75be53534fe Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 30 Sep 2016 15:20:11 +0700 Subject: [PATCH 211/487] IGNITE-4007 Fixed update of QueryMetrics.minimumTime() metric. Tests added. Added Visor reset metrics task. --- .../query/GridCacheQueryMetricsAdapter.java | 12 ++-- .../VisorCacheResetQueryMetricsTask.java | 69 +++++++++++++++++++ .../CacheAbstractQueryMetricsSelfTest.java | 6 +- 3 files changed, 80 insertions(+), 7 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetQueryMetricsTask.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java index 1928ea56d6e0d..e70ea9ff46222 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java @@ -34,7 +34,7 @@ public class GridCacheQueryMetricsAdapter implements QueryMetrics, Externalizabl private static final long serialVersionUID = 0L; /** Minimum time of execution. */ - private final GridAtomicLong minTime = new GridAtomicLong(); + private final GridAtomicLong minTime = new GridAtomicLong(Long.MAX_VALUE); /** Maximum time of execution. */ private final GridAtomicLong maxTime = new GridAtomicLong(); @@ -58,7 +58,9 @@ public class GridCacheQueryMetricsAdapter implements QueryMetrics, Externalizabl /** {@inheritDoc} */ @Override public long minimumTime() { - return minTime.get(); + long min = minTime.get(); + + return min == Long.MAX_VALUE ? 0 : min; } /** {@inheritDoc} */ @@ -71,9 +73,9 @@ public class GridCacheQueryMetricsAdapter implements QueryMetrics, Externalizabl if (avgTime > 0) return avgTime; else { - long val = completed.sum(); + double val = completed.sum(); - return val > 0 ? sumTime.sum() / val : 0; + return val > 0 ? sumTime.sum() / val : 0.0; } } @@ -170,4 +172,4 @@ public GridCacheQueryMetricsAdapter copy() { @Override public String toString() { return S.toString(GridCacheQueryMetricsAdapter.class, this); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetQueryMetricsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetQueryMetricsTask.java new file mode 100644 index 0000000000000..96d98573e9f8d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetQueryMetricsTask.java @@ -0,0 +1,69 @@ +/* + * 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.visor.cache; + +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorOneNodeTask; + +/** + * Reset compute grid query metrics. + */ +@GridInternal +public class VisorCacheResetQueryMetricsTask extends VisorOneNodeTask { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorCacheResetQueryMetricsJob job(String arg) { + return new VisorCacheResetQueryMetricsJob(arg, debug); + } + + /** + * Job that reset cache query metrics. + */ + private static class VisorCacheResetQueryMetricsJob extends VisorJob { + /** */ + private static final long serialVersionUID = 0L; + + /** + * @param arg Cache name to reset query metrics for. + * @param debug Debug flag. + */ + private VisorCacheResetQueryMetricsJob(String arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected Void run(String cacheName) { + IgniteInternalCache cache = ignite.cachex(cacheName); + + if (cache != null) + cache.context().queries().resetMetrics(); + + return null; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorCacheResetQueryMetricsJob.class, this); + } + } +} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java index d2d8c4d75ee0c..10f761237cdb1 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java @@ -236,6 +236,7 @@ private void testQueryMetrics(IgniteCache cache, Query qry) { assertTrue(m.averageTime() >= 0); assertTrue(m.maximumTime() >= 0); assertTrue(m.minimumTime() >= 0); + assertTrue("On first execution minTime == maxTime", m.minimumTime() == m.maximumTime()); // Execute again with the same parameters. cache.query(qry).getAll(); @@ -274,6 +275,7 @@ private void testQueryNotFullyFetchedMetrics(IgniteCache cache, assertTrue(m.averageTime() >= 0); assertTrue(m.maximumTime() >= 0); assertTrue(m.minimumTime() >= 0); + assertTrue("On first execution minTime == maxTime", m.minimumTime() == m.maximumTime()); // Execute again with the same parameters. cache.query(qry).iterator().next(); @@ -301,7 +303,7 @@ private void testQueryFailedMetrics(IgniteCache cache, Query qr try { cache.query(qry).getAll(); } - catch (Exception e) { + catch (Exception ignored) { // No-op. } @@ -320,7 +322,7 @@ private void testQueryFailedMetrics(IgniteCache cache, Query qr try { cache.query(qry).getAll(); } - catch (Exception e) { + catch (Exception ignored) { // No-op. } From f745371af76baa676385c22257b4577d2e697d16 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 3 Oct 2016 10:07:26 +0300 Subject: [PATCH 212/487] Revert "Fixed missing Apache header." This reverts commit b280c3efa1eb84c6bc8abbe31ba669b0c24323d8. --- .../binary/BinaryFieldOrderSelfTest.java | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java index e4709480f49a4..6bb1e1380da8a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java @@ -1,20 +1,3 @@ -/* - * 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.binary; import org.apache.ignite.binary.BinaryObject; From de50287d493acc73186926d34431733bc76c549b Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 3 Oct 2016 10:07:33 +0300 Subject: [PATCH 213/487] Revert "IGNITE-3191 - Fixed ordering of fields in binary objects" This reverts commit d1e3a78ae569fa5d5692816db44f2c677e1b8283. --- .../binary/BinaryClassDescriptor.java | 16 ++- .../builder/BinaryObjectBuilderImpl.java | 24 ++--- .../binary/BinaryFieldOrderSelfTest.java | 98 ------------------- .../IgniteBinaryObjectsTestSuite.java | 2 - 4 files changed, 19 insertions(+), 121 deletions(-) delete mode 100644 modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java index 276dfe5134252..4c824d4409fb1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java @@ -25,6 +25,7 @@ import java.lang.reflect.Proxy; import java.math.BigDecimal; import java.sql.Timestamp; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Date; @@ -32,7 +33,6 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; -import java.util.TreeMap; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.binary.BinaryObjectException; @@ -269,9 +269,10 @@ else if (useOptMarshaller) case OBJECT: // Must not use constructor to honor transient fields semantics. ctor = null; + ArrayList fields0 = new ArrayList<>(); stableFieldsMeta = metaDataEnabled ? new HashMap() : null; - Map fields0 = new TreeMap<>(); + BinarySchema.Builder schemaBuilder = BinarySchema.Builder.newBuilder(); Set duplicates = duplicateFields(cls); @@ -299,7 +300,9 @@ else if (useOptMarshaller) BinaryFieldAccessor fieldInfo = BinaryFieldAccessor.create(f, fieldId); - fields0.put(name, fieldInfo); + fields0.add(fieldInfo); + + schemaBuilder.addField(fieldId); if (metaDataEnabled) stableFieldsMeta.put(name, fieldInfo.mode().typeId()); @@ -307,12 +310,7 @@ else if (useOptMarshaller) } } - fields = fields0.values().toArray(new BinaryFieldAccessor[fields0.size()]); - - BinarySchema.Builder schemaBuilder = BinarySchema.Builder.newBuilder(); - - for (BinaryFieldAccessor field : fields) - schemaBuilder.addField(field.id); + fields = fields0.toArray(new BinaryFieldAccessor[fields0.size()]); stableSchema = schemaBuilder.build(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java index d166051d18f35..2c761925e68af 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java @@ -17,32 +17,32 @@ package org.apache.ignite.internal.binary.builder; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.Map; -import java.util.Set; -import java.util.TreeMap; import org.apache.ignite.binary.BinaryInvalidTypeException; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.binary.BinaryType; -import org.apache.ignite.internal.binary.BinaryContext; import org.apache.ignite.internal.binary.BinaryMetadata; import org.apache.ignite.internal.binary.BinaryObjectImpl; -import org.apache.ignite.internal.binary.BinaryObjectOffheapImpl; +import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.binary.GridBinaryMarshaller; +import org.apache.ignite.internal.binary.BinaryContext; import org.apache.ignite.internal.binary.BinarySchema; import org.apache.ignite.internal.binary.BinarySchemaRegistry; +import org.apache.ignite.internal.binary.BinaryObjectOffheapImpl; import org.apache.ignite.internal.binary.BinaryUtils; -import org.apache.ignite.internal.binary.BinaryWriterExImpl; -import org.apache.ignite.internal.binary.GridBinaryMarshaller; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.Nullable; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; + /** * */ @@ -522,7 +522,7 @@ private void ensureReadCacheInit() { Object val = val0 == null ? new BinaryValueWithType(BinaryUtils.typeByClass(Object.class), null) : val0; if (assignedVals == null) - assignedVals = new TreeMap<>(); + assignedVals = new LinkedHashMap<>(); Object oldVal = assignedVals.put(name, val); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java deleted file mode 100644 index 6bb1e1380da8a..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldOrderSelfTest.java +++ /dev/null @@ -1,98 +0,0 @@ -package org.apache.ignite.internal.binary; - -import org.apache.ignite.binary.BinaryObject; -import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; -import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -/** - * Test that field ordering doesn't change the schema. - */ -public class BinaryFieldOrderSelfTest extends GridCommonAbstractTest { - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setMarshaller(new BinaryMarshaller()); - cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(new TcpDiscoveryVmIpFinder(true))); - - return cfg; - } - - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - startGrid(); - } - - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - stopAllGrids(); - } - - /** - * @throws Exception If failed. - */ - public void testEquals() throws Exception { - IgniteEx ignite = grid(); - - BinaryObject bo0 = ignite.binary().toBinary(new MyType(222, 333, 111)); - - BinaryObject bo1 = ignite.binary().builder(bo0.type().typeName()). - setField("b", 222). - setField("c", 333). - setField("a", 111). - hashCode(12345). - build(); - - BinaryObject bo2 = ignite.binary().builder(bo0.type().typeName()). - setField("a", 111). - setField("b", 222). - setField("c", 333). - hashCode(12345). - build(); - - assertEquals(12345, bo0.hashCode()); - assertEquals(12345, bo1.hashCode()); - assertEquals(12345, bo2.hashCode()); - - assertTrue(bo0.equals(bo1)); - assertTrue(bo0.equals(bo2)); - assertTrue(bo1.equals(bo2)); - } - - /** - */ - private static class MyType { - /** B. */ - private int b; - - /** C. */ - private int c; - - /** A. */ - private int a; - - /** - * @param b B. - * @param c C. - * @param a A. - */ - MyType(int b, int c, int a) { - this.b = b; - this.c = c; - this.a = a; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object obj) { - return super.equals(obj); - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - return 12345; - } - } -} 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 50c6f0b95a95d..c1d9974837fb3 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 @@ -22,7 +22,6 @@ import org.apache.ignite.internal.binary.BinaryBasicNameMapperSelfTest; import org.apache.ignite.internal.binary.BinaryConfigurationConsistencySelfTest; import org.apache.ignite.internal.binary.BinaryEnumsSelfTest; -import org.apache.ignite.internal.binary.BinaryFieldOrderSelfTest; import org.apache.ignite.internal.binary.BinaryFieldsHeapSelfTest; import org.apache.ignite.internal.binary.BinaryFieldsOffheapSelfTest; import org.apache.ignite.internal.binary.BinaryFooterOffsetsHeapSelfTest; @@ -105,7 +104,6 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridBinaryAffinityKeySelfTest.class); suite.addTestSuite(GridBinaryWildcardsSelfTest.class); suite.addTestSuite(BinaryObjectToStringSelfTest.class); - suite.addTestSuite(BinaryFieldOrderSelfTest.class); // Tests for objects with non-compact footers. suite.addTestSuite(BinaryMarshallerNonCompactSelfTest.class); From 59b46d3e6ea073d054ca2262d676b055a74bbb1f Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 3 Oct 2016 10:40:04 +0300 Subject: [PATCH 214/487] IGNITE-2881: Fixed SPI queries. --- .../apache/ignite/cache/query/SpiQuery.java | 8 +- .../processors/cache/IgniteCacheProxy.java | 34 ++- .../cache/query/GridCacheQueryManager.java | 2 +- .../cache/query/IndexingSpiQuerySelfTest.java | 218 ++++++++++++++++++ .../query/IndexingSpiQueryTxSelfTest.java | 162 +++++++++++++ .../IgniteBinaryCacheQueryTestSuite.java | 1 - .../IgniteCacheQuerySelfTestSuite.java | 5 + 7 files changed, 416 insertions(+), 14 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/SpiQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/SpiQuery.java index 0c1143715897b..c3aa472080f24 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/query/SpiQuery.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/SpiQuery.java @@ -58,13 +58,13 @@ public SpiQuery setArgs(Object... args) { } /** {@inheritDoc} */ - @Override public SqlQuery setPageSize(int pageSize) { - return (SqlQuery)super.setPageSize(pageSize); + @Override public SpiQuery setPageSize(int pageSize) { + return (SpiQuery)super.setPageSize(pageSize); } /** {@inheritDoc} */ - @Override public SqlQuery setLocal(boolean loc) { - return (SqlQuery)super.setLocal(loc); + @Override public SpiQuery setLocal(boolean loc) { + return (SpiQuery)super.setLocal(loc); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java index 81d4b49f882e6..58c7c9ce83817 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java @@ -533,11 +533,11 @@ private QueryCursor query( @SuppressWarnings("unchecked") private QueryCursor> query(final Query filter, @Nullable ClusterGroup grp) throws IgniteCheckedException { - final CacheQuery> qry; + final CacheQuery qry; boolean isKeepBinary = opCtx != null && opCtx.isKeepBinary(); - final CacheQueryFuture> fut; + final CacheQueryFuture fut; if (filter instanceof TextQuery) { TextQuery p = (TextQuery)filter; @@ -561,8 +561,8 @@ else if (filter instanceof SpiQuery) { qry.projection(grp); fut = ctx.kernalContext().query().executeQuery(ctx, - new IgniteOutClosureX>>() { - @Override public CacheQueryFuture> applyx() throws IgniteCheckedException { + new IgniteOutClosureX>>() { + @Override public CacheQueryFuture> applyx() throws IgniteCheckedException { return qry.execute(((SpiQuery)filter).getArgs()); } }, false); @@ -577,21 +577,39 @@ else if (filter instanceof SpiQuery) { return new QueryCursorImpl<>(new GridCloseableIteratorAdapter>() { /** */ - private Map.Entry cur; + private Cache.Entry cur; @Override protected Entry onNext() throws IgniteCheckedException { if (!onHasNext()) throw new NoSuchElementException(); - Map.Entry e = cur; + Cache.Entry e = cur; cur = null; - return new CacheEntryImpl<>(e.getKey(), e.getValue()); + return e; } @Override protected boolean onHasNext() throws IgniteCheckedException { - return cur != null || (cur = fut.next()) != null; + if (cur != null) + return true; + + Object next = fut.next(); + + // Workaround a bug: if IndexingSpi is configured future represents Iterator + // instead of Iterator due to IndexingSpi interface. + if (next == null) + return false; + + if (next instanceof Cache.Entry) + cur = (Cache.Entry)next; + else { + Map.Entry e = (Map.Entry)next; + + cur = new CacheEntryImpl(e.getKey(), e.getValue()); + } + + return true; } @Override protected void onClose() throws IgniteCheckedException { 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 9699f09eacfec..7bd1a51b314d3 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 @@ -690,7 +690,7 @@ private FieldsResult executeFieldsQuery(GridCacheQueryAdapter qry, @Nullable T2> resKey = null; - if (qry.clause() == null) { + if (qry.clause() == null && qry.type() != SPI) { assert !loc; throw new IgniteCheckedException("Received next page request after iterator was removed. " + diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java new file mode 100644 index 0000000000000..94b0c8aedbb2b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java @@ -0,0 +1,218 @@ +/* + * 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.query; + +import junit.framework.TestCase; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteTransactions; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.SpiQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.processors.cache.CacheEntryImpl; +import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; +import org.apache.ignite.spi.IgniteSpiAdapter; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.spi.indexing.IndexingQueryFilter; +import org.apache.ignite.spi.indexing.IndexingSpi; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.apache.ignite.transactions.TransactionState; +import org.jetbrains.annotations.Nullable; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.Callable; +import javax.cache.Cache; + +/** + * Indexing Spi query test + */ +public class IndexingSpiQuerySelfTest extends TestCase { + /** {@inheritDoc} */ + @Override public void tearDown() throws Exception { + Ignition.stopAll(true); + } + + /** + * @throws Exception If failed. + */ + public void testSimpleIndexingSpi() throws Exception { + IgniteConfiguration cfg = configuration(); + + cfg.setIndexingSpi(new MyIndexingSpi()); + + Ignite ignite = Ignition.start(cfg); + + CacheConfiguration ccfg = new CacheConfiguration<>("test-cache"); + + ccfg.setIndexedTypes(Integer.class, Integer.class); + + IgniteCache cache = ignite.createCache(ccfg); + + for (int i = 0; i < 10; i++) + cache.put(i, i); + + QueryCursor> cursor = cache.query(new SpiQuery().setArgs(2, 5)); + + for (Cache.Entry entry : cursor) + System.out.println(entry); + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testIndexingSpiFailure() throws Exception { + IgniteConfiguration cfg = configuration(); + + cfg.setIndexingSpi(new MyBrokenIndexingSpi()); + + Ignite ignite = Ignition.start(cfg); + + CacheConfiguration ccfg = new CacheConfiguration<>("test-cache"); + + ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + ccfg.setIndexedTypes(Integer.class, Integer.class); + + final IgniteCache cache = ignite.createCache(ccfg); + + final IgniteTransactions txs = ignite.transactions(); + + for (final TransactionConcurrency concurrency : TransactionConcurrency.values()) { + for (final TransactionIsolation isolation : TransactionIsolation.values()) { + System.out.println("Run in transaction: " + concurrency + " " + isolation); + + GridTestUtils.assertThrowsWithCause(new Callable() { + @Override public Void call() throws Exception { + Transaction tx; + + try (Transaction tx0 = tx = txs.txStart(concurrency, isolation)) { + cache.put(1, 1); + + tx0.commit(); + } + + assertEquals(TransactionState.ROLLED_BACK, tx.state()); + return null; + } + }, IgniteTxHeuristicCheckedException.class); + } + } + } + + /** + * @return Configuration. + */ + private IgniteConfiguration configuration() { + IgniteConfiguration cfg = new IgniteConfiguration(); + + TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setMaxMissedHeartbeats(Integer.MAX_VALUE); + + disco.setIpFinder(ipFinder); + + cfg.setDiscoverySpi(disco); + + return cfg; + } + + /** + * Indexing Spi implementation for test + */ + private static class MyIndexingSpi extends IgniteSpiAdapter implements IndexingSpi { + /** Index. */ + private final SortedMap idx = new TreeMap<>(); + + /** {@inheritDoc} */ + @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void spiStop() throws IgniteSpiException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public Iterator> query(@Nullable String spaceName, Collection params, + @Nullable IndexingQueryFilter filters) throws IgniteSpiException { + if (params.size() < 2) + throw new IgniteSpiException("Range parameters required."); + + Iterator paramsIt = params.iterator(); + + Object from = paramsIt.next(); + Object to = paramsIt.next(); + + SortedMap map = idx.subMap(from, to); + + Collection> res = new ArrayList<>(map.size()); + + for (Map.Entry entry : map.entrySet()) + res.add(new CacheEntryImpl<>(entry.getKey(), entry.getValue())); + + return res.iterator(); + } + + /** {@inheritDoc} */ + @Override public void store(@Nullable String spaceName, Object key, Object val, long expirationTime) + throws IgniteSpiException { + idx.put(key, val); + } + + /** {@inheritDoc} */ + @Override public void remove(@Nullable String spaceName, Object key) throws IgniteSpiException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void onSwap(@Nullable String spaceName, Object key) throws IgniteSpiException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void onUnswap(@Nullable String spaceName, Object key, Object val) throws IgniteSpiException { + // No-op. + } + } + + /** + * Broken Indexing Spi implementation for test + */ + private class MyBrokenIndexingSpi extends MyIndexingSpi { + /** {@inheritDoc} */ + @Override public void store(@Nullable String spaceName, Object key, Object val, + long expirationTime) throws IgniteSpiException { + throw new IgniteSpiException("Test exception"); + } + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java new file mode 100644 index 0000000000000..78ed1fda1dbc0 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java @@ -0,0 +1,162 @@ +/* + * 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.query; + +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteTransactions; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest; +import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; +import org.apache.ignite.spi.IgniteSpiAdapter; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.indexing.IndexingQueryFilter; +import org.apache.ignite.spi.indexing.IndexingSpi; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.apache.ignite.transactions.TransactionState; +import org.jetbrains.annotations.Nullable; + +import java.util.Collection; +import java.util.Iterator; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.Cache; + +/** + * Indexing Spi transactional query test + */ +public class IndexingSpiQueryTxSelfTest extends GridCacheAbstractSelfTest { + /** */ + private static AtomicInteger cnt; + + /** {@inheritDoc} */ + @Override protected int gridCount() { + return 4; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + cnt = new AtomicInteger(); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true); + + if (cnt.getAndIncrement() == 0) + cfg.setClientMode(true); + else { + cfg.setIndexingSpi(new MyBrokenIndexingSpi()); + + CacheConfiguration ccfg = cacheConfiguration(gridName); + ccfg.setName("test-cache"); + ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + + ccfg.setIndexedTypes(Integer.class, Integer.class); + + cfg.setCacheConfiguration(ccfg); + } + return cfg; + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testIndexingSpiWithTx() throws Exception { + IgniteEx ignite = grid(0); + + final IgniteCache cache = ignite.cache("test-cache"); + + final IgniteTransactions txs = ignite.transactions(); + + for (final TransactionConcurrency concurrency : TransactionConcurrency.values()) { + for (final TransactionIsolation isolation : TransactionIsolation.values()) { + System.out.println("Run in transaction: " + concurrency + " " + isolation); + + GridTestUtils.assertThrowsWithCause(new Callable() { + @Override public Void call() throws Exception { + Transaction tx; + + try (Transaction tx0 = tx = txs.txStart(concurrency, isolation)) { + cache.put(1, 1); + + tx0.commit(); + } + + assertEquals(TransactionState.ROLLED_BACK, tx.state()); + + return null; + } + }, IgniteTxHeuristicCheckedException.class); + } + } + } + + /** + * Indexing SPI implementation for test + */ + private static class MyBrokenIndexingSpi extends IgniteSpiAdapter implements IndexingSpi { + /** {@inheritDoc} */ + @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void spiStop() throws IgniteSpiException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public Iterator> query(@Nullable String spaceName, Collection params, + @Nullable IndexingQueryFilter filters) throws IgniteSpiException { + return null; + } + + /** {@inheritDoc} */ + @Override public void store(@Nullable String spaceName, Object key, Object val, long expirationTime) + throws IgniteSpiException { + throw new IgniteSpiException("Test exception"); + } + + /** {@inheritDoc} */ + @Override public void remove(@Nullable String spaceName, Object key) throws IgniteSpiException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void onSwap(@Nullable String spaceName, Object key) throws IgniteSpiException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void onUnswap(@Nullable String spaceName, Object key, Object val) throws IgniteSpiException { + // No-op. + } + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java index 761d4bd4cc7dc..3cb603c6d0ae7 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java @@ -24,7 +24,6 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheBinaryObjectsScanSelfTest; import org.apache.ignite.internal.processors.cache.binary.distributed.dht.GridCacheBinaryDuplicateIndexObjectPartitionedAtomicSelfTest; import org.apache.ignite.internal.processors.cache.binary.distributed.dht.GridCacheBinaryDuplicateIndexObjectPartitionedTransactionalSelfTest; -import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryLostPartitionTest; import org.apache.ignite.testframework.config.GridTestProperties; /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java index e7f55a1faae2c..21741478f945e 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java @@ -60,6 +60,8 @@ import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest; import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQuerySelfTest; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryTransformerSelfTest; +import org.apache.ignite.internal.processors.cache.query.IndexingSpiQuerySelfTest; +import org.apache.ignite.internal.processors.cache.query.IndexingSpiQueryTxSelfTest; import org.apache.ignite.internal.processors.query.IgniteSqlSchemaIndexingTest; import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest; import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest; @@ -127,6 +129,9 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteBinaryObjectQueryArgumentsOffheapLocalTest.class); suite.addTestSuite(IgniteBinaryObjectLocalQueryArgumentsTest.class); + suite.addTestSuite(IndexingSpiQuerySelfTest.class); + suite.addTestSuite(IndexingSpiQueryTxSelfTest.class); + return suite; } } From 0b66d2d7b695cf370a4b8a717844ad67742c6090 Mon Sep 17 00:00:00 2001 From: sboikov Date: Mon, 3 Oct 2016 12:04:03 +0300 Subject: [PATCH 215/487] ignite-3601 Do not check version on commit for read-only serializable transactions. --- .../colocated/GridDhtDetachedCacheEntry.java | 5 + .../distributed/near/GridNearTxLocal.java | 2 +- .../transactions/IgniteTxLocalAdapter.java | 62 ++++--- .../cache/CacheGetEntryAbstractTest.java | 38 +++++ .../cache/CachePutIfAbsentTest.java | 161 ++++++++++++++++++ .../CacheSerializableTransactionsTest.java | 70 +++----- .../cache/CacheTxFastFinishTest.java | 2 +- .../testsuites/IgniteCacheTestSuite4.java | 2 + 8 files changed, 276 insertions(+), 66 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CachePutIfAbsentTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java index db9113446fb8f..2e055604b2687 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java @@ -97,4 +97,9 @@ public void resetFromPrimary(CacheObject val, GridCacheVersion ver) { // No-op for detached cache entry. return true; } + + /** {@inheritDoc} */ + @Override public int partition() { + return cctx.affinity().partition(key); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index 410baf837f34c..ed37059b20cb8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -953,7 +953,7 @@ void readyNearLock(IgniteTxEntry txEntry, * @return {@code True} if 'fast finish' path can be used for transaction completion. */ private boolean fastFinish() { - return writeMap().isEmpty() && ((optimistic() && !serializable()) || readMap().isEmpty()); + return writeMap().isEmpty() && (optimistic() || readMap().isEmpty()); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index fe695368117c3..6d21dcfb716fd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -2269,7 +2269,16 @@ private IgniteInternalFuture loadMissing( addInvokeResult(e, cacheVal, ret, ver); } else { - boolean success = !hasFilters || isAll(e.context(), key, cacheVal, filter); + boolean success; + + if (hasFilters) { + success = isAll(e.context(), key, cacheVal, filter); + + if (!success) + e.value(cacheVal, false, false); + } + else + success = true; ret.set(cacheCtx, cacheVal, success, keepBinary); } @@ -2411,25 +2420,43 @@ private boolean enlistWriteEntry(GridCacheContext cacheCtx, else old = retval ? entry.rawGetOrUnmarshal(false) : entry.rawGet(); + final GridCacheOperation op = lockOnly ? NOOP : rmv ? DELETE : + entryProcessor != null ? TRANSFORM : old != null ? UPDATE : CREATE; + if (old != null && hasFilters && !filter(entry.context(), cacheKey, old, filter)) { ret.set(cacheCtx, old, false, keepBinary); if (!readCommitted()) { - // Enlist failed filters as reads for non-read-committed mode, - // so future ops will get the same values. - txEntry = addEntry(READ, - old, - null, - null, - entry, - null, - CU.empty0(), - false, - -1L, - -1L, - null, - skipStore, - keepBinary); + if (optimistic() && serializable()) { + txEntry = addEntry(op, + old, + entryProcessor, + invokeArgs, + entry, + expiryPlc, + filter, + true, + drTtl, + drExpireTime, + drVer, + skipStore, + keepBinary); + } + else { + txEntry = addEntry(READ, + old, + null, + null, + entry, + null, + CU.empty0(), + false, + -1L, + -1L, + null, + skipStore, + keepBinary); + } txEntry.markValid(); @@ -2446,9 +2473,6 @@ private boolean enlistWriteEntry(GridCacheContext cacheCtx, break; // While. } - final GridCacheOperation op = lockOnly ? NOOP : rmv ? DELETE : - entryProcessor != null ? TRANSFORM : old != null ? UPDATE : CREATE; - txEntry = addEntry(op, cacheCtx.toCacheObject(val), entryProcessor, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java index 34480a24e5b30..2eab6d9f0393e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java @@ -40,6 +40,7 @@ import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; import org.apache.ignite.transactions.TransactionIsolation; +import org.apache.ignite.transactions.TransactionOptimisticException; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; @@ -52,6 +53,7 @@ import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; +import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE; /** * Test getEntry and getEntries methods. @@ -247,6 +249,10 @@ private void test(CacheConfiguration cfg, final boolean oneEntry) throws Excepti testConcurrentTx(cache, PESSIMISTIC, REPEATABLE_READ, oneEntry); testConcurrentTx(cache, PESSIMISTIC, READ_COMMITTED, oneEntry); + + testConcurrentOptimisticTxGet(cache, REPEATABLE_READ); + testConcurrentOptimisticTxGet(cache, READ_COMMITTED); + testConcurrentOptimisticTxGet(cache, SERIALIZABLE); } } finally { @@ -254,6 +260,38 @@ private void test(CacheConfiguration cfg, final boolean oneEntry) throws Excepti } } + /** + * @param cache Cache. + * @param txIsolation Transaction isolation. + * @throws Exception If failed. + */ + private void testConcurrentOptimisticTxGet(final IgniteCache cache, + final TransactionIsolation txIsolation) throws Exception { + GridTestUtils.runMultiThreaded(new Runnable() { + @Override public void run() { + final int key = 42; + + IgniteTransactions txs = grid(0).transactions(); + + cache.put(key, new TestValue(key)); + + long stopTime = System.currentTimeMillis() + 3000; + + while (System.currentTimeMillis() < stopTime) { + try (Transaction tx = txs.txStart(OPTIMISTIC, txIsolation)) { + cache.get(key); + + tx.commit(); + } + catch (TransactionOptimisticException e) { + assertTrue("Should not throw optimistic exception in only read TX. Tx isolation: " + + txIsolation, false); + } + } + } + }, 10, "tx-thread"); + } + /** * @param cache Cache. * @param txConcurrency Transaction concurrency. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CachePutIfAbsentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CachePutIfAbsentTest.java new file mode 100644 index 0000000000000..6ed90494d822f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CachePutIfAbsentTest.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteTransactions; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheMode.REPLICATED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; + +/** + * + */ +public class CachePutIfAbsentTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int SRVS = 4; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGridsMultiThreaded(SRVS); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 5 * 60_000; + } + + /** + * @return Cache configurations. + */ + private List> cacheConfigurations() { + List> ccfgs = new ArrayList<>(); + + ccfgs.add(cacheConfiguration(REPLICATED, FULL_SYNC, 0)); + + ccfgs.add(cacheConfiguration(PARTITIONED, FULL_SYNC, 0)); + ccfgs.add(cacheConfiguration(PARTITIONED, FULL_SYNC, 1)); + ccfgs.add(cacheConfiguration(PARTITIONED, FULL_SYNC, 2)); + + return ccfgs; + } + + /** + * @param cacheMode Cache mode. + * @param syncMode Write synchronization mode. + * @param backups Number of backups. + * @return Cache configuration. + */ + private CacheConfiguration cacheConfiguration( + CacheMode cacheMode, + CacheWriteSynchronizationMode syncMode, + int backups) { + CacheConfiguration ccfg = new CacheConfiguration<>(); + + ccfg.setCacheMode(cacheMode); + ccfg.setAtomicityMode(TRANSACTIONAL); + ccfg.setWriteSynchronizationMode(syncMode); + + if (cacheMode == PARTITIONED) + ccfg.setBackups(backups); + + return ccfg; + } + + /** + * @throws Exception If failed. + */ + public void testTxConflictGetAndPutIfAbsent() throws Exception { + Ignite ignite0 = ignite(0); + + final IgniteTransactions txs = ignite0.transactions(); + + for (CacheConfiguration ccfg : cacheConfigurations()) { + try { + IgniteCache cache = ignite0.createCache(ccfg); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + for (int i = 0; i < 10; i++) { + Integer key = rnd.nextInt(10_000); + + cache.put(key, 2); + + for (TransactionConcurrency concurrency : TransactionConcurrency.values()) { + for (TransactionIsolation isolation : TransactionIsolation.values()) { + try (Transaction tx = txs.txStart(concurrency, isolation)) { + Object old = cache.getAndPutIfAbsent(key, 3); + + assertEquals(2, old); + + Object val = cache.get(key); + + assertEquals(2, val); + + tx.commit(); + } + + assertEquals((Integer)2, cache.get(key)); + } + } + } + } + finally { + ignite0.destroyCache(ccfg.getName()); + } + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java index 3d4f8502d20f1..6a73f79dfc303 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java @@ -709,6 +709,7 @@ public void testTxConflictReadEntry2() throws Exception { /** * @param noVal If {@code true} there is no cache value when read in tx. + * @param needVer If {@code true} then gets entry, otherwise just value. * @throws Exception If failed. */ private void txConflictRead(boolean noVal, boolean needVer) throws Exception { @@ -735,28 +736,21 @@ private void txConflictRead(boolean noVal, boolean needVer) throws Exception { cache.put(key, expVal); } - try { - try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) { - if (needVer) { - CacheEntry val = cache.getEntry(key); - - assertEquals(expVal, val == null ? null : val.getValue()); - } - else { - Integer val = cache.get(key); - - assertEquals(expVal, val); - } + try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) { + if (needVer) { + CacheEntry val = cache.getEntry(key); - updateKey(cache, key, 1); + assertEquals(expVal, val == null ? null : val.getValue()); + } + else { + Integer val = cache.get(key); - tx.commit(); + assertEquals(expVal, val); } - fail(); - } - catch (TransactionOptimisticException e) { - log.info("Expected exception: " + e); + updateKey(cache, key, 1); + + tx.commit(); } checkValue(key, 1, cache.getName()); @@ -2625,21 +2619,14 @@ private void rollbackNearCacheRead(boolean near) throws Exception { cache0.put(key2, -1); cache0.put(key3, -1); - try { - try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) { - cache.get(key1); - cache.get(key2); - cache.get(key3); - - updateKey(near ? cache : cache0, key2, -2); + try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) { + cache.get(key1); + cache.get(key2); + cache.get(key3); - tx.commit(); - } + updateKey(near ? cache : cache0, key2, -2); - fail(); - } - catch (TransactionOptimisticException e) { - log.info("Expected exception: " + e); + tx.commit(); } checkValue(key1, -1, cacheName); @@ -2890,23 +2877,16 @@ public void testCrossCacheTx() throws Exception { checkValue(key1, newVal, CACHE1); checkValue(key2, newVal, CACHE2); - try { - try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) { - Object val1 = cache1.get(key1); - Object val2 = cache2.get(key2); - - assertEquals(newVal, val1); - assertEquals(newVal, val2); + try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) { + Object val1 = cache1.get(key1); + Object val2 = cache2.get(key2); - updateKey(cache2, key2, newVal); + assertEquals(newVal, val1); + assertEquals(newVal, val2); - tx.commit(); - } + updateKey(cache2, key2, newVal); - fail(); - } - catch (TransactionOptimisticException e) { - log.info("Expected exception: " + e); + tx.commit(); } try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheTxFastFinishTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheTxFastFinishTest.java index 35b14055066d4..f9c66834d088e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheTxFastFinishTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheTxFastFinishTest.java @@ -173,7 +173,7 @@ private void fastFinishTx(Ignite ignite) { try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) { cache.get(i); - checkNormalTxFinish(tx, commit); + checkFastTxFinish(tx, commit); } try (Transaction tx = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) { diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java index c494e732b4f66..2b446bbea8e6d 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java @@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.CacheGetEntryPessimisticRepeatableReadSeltTest; import org.apache.ignite.internal.processors.cache.CacheGetEntryPessimisticSerializableSeltTest; import org.apache.ignite.internal.processors.cache.CacheOffheapMapEntrySelfTest; +import org.apache.ignite.internal.processors.cache.CachePutIfAbsentTest; import org.apache.ignite.internal.processors.cache.CacheReadThroughAtomicRestartSelfTest; import org.apache.ignite.internal.processors.cache.CacheReadThroughLocalAtomicRestartSelfTest; import org.apache.ignite.internal.processors.cache.CacheReadThroughLocalRestartSelfTest; @@ -326,6 +327,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCachePrimarySyncTest.class); suite.addTestSuite(IgniteTxCachePrimarySyncTest.class); suite.addTestSuite(IgniteTxCacheWriteSynchronizationModesMultithreadedTest.class); + suite.addTestSuite(CachePutIfAbsentTest.class); suite.addTestSuite(MarshallerCacheJobRunNodeRestartTest.class); From 3c9e254ee762dfef0b27b5b95a63502069904d70 Mon Sep 17 00:00:00 2001 From: ptupitsyn Date: Mon, 15 Aug 2016 16:40:24 +0300 Subject: [PATCH 216/487] IGNITE-1629 .NET: Introduced native logging facility --- .../logger/platform/PlatformLogger.java | 223 +++++++++ .../platform/PlatformAbstractBootstrap.java | 18 +- .../platform/PlatformConfigurationEx.java | 6 + .../platform/PlatformNoopProcessor.java | 10 + .../platform/PlatformProcessor.java | 16 + .../platform/PlatformProcessorImpl.java | 56 +++ .../callback/PlatformCallbackGateway.java | 45 ++ .../callback/PlatformCallbackUtils.java | 20 + .../cpp/PlatformCppConfigurationEx.java | 6 + .../dotnet/PlatformDotNetBootstrap.java | 9 + .../PlatformDotNetConfigurationClosure.java | 11 +- .../dotnet/PlatformDotNetConfigurationEx.java | 12 +- .../cpp/jni/include/ignite/jni/exports.h | 2 + .../cpp/jni/include/ignite/jni/java.h | 13 + .../platforms/cpp/jni/project/vs/module.def | 3 +- modules/platforms/cpp/jni/src/exports.cpp | 12 +- modules/platforms/cpp/jni/src/java.cpp | 72 ++- .../Apache.Ignite.Core.Tests.csproj | 8 + .../Binary/BinaryStringTest.cs | 26 +- .../Config/Log/custom-log.xml | 50 ++ .../Config/Log/dotnet-log4j.xml | 143 ++++++ .../IgniteConfigurationSerializerTest.cs | 30 +- .../Log/CustomLoggerTest.cs | 430 ++++++++++++++++++ .../Log/DefaultLoggerTest.cs | 114 +++++ .../Apache.Ignite.Core.Tests/TestUtils.cs | 26 ++ .../Apache.Ignite.Core.csproj | 6 +- .../Cache/Configuration/CacheConfiguration.cs | 19 +- .../Cache/Configuration/QueryEntity.cs | 25 +- .../Cache/Configuration/QueryField.cs | 19 +- .../dotnet/Apache.Ignite.Core/IIgnite.cs | 8 + .../Apache.Ignite.Core/IgniteConfiguration.cs | 88 ++++ .../IgniteConfigurationSection.xsd | 12 + .../dotnet/Apache.Ignite.Core/Ignition.cs | 39 +- .../Impl/Binary/JavaTypes.cs | 29 +- .../Apache.Ignite.Core/Impl/Common/Logger.cs | 37 -- .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs | 9 + .../Apache.Ignite.Core/Impl/IgniteManager.cs | 6 +- .../Apache.Ignite.Core/Impl/IgniteProxy.cs | 7 + .../Apache.Ignite.Core/Impl/IgniteUtils.cs | 26 +- .../Apache.Ignite.Core/Impl/Log/JavaLogger.cs | 110 +++++ .../Impl/Unmanaged/IgniteJniNativeMethods.cs | 7 + .../Unmanaged/UnmanagedCallbackHandlers.cs | 3 + .../Impl/Unmanaged/UnmanagedCallbacks.cs | 76 +++- .../Impl/Unmanaged/UnmanagedUtils.cs | 27 +- .../Apache.Ignite.Core/Log/CategoryLogger.cs | 82 ++++ .../dotnet/Apache.Ignite.Core/Log/ILogger.cs | 51 +++ .../dotnet/Apache.Ignite.Core/Log/LogLevel.cs | 53 +++ .../Log/LoggerExtensions.cs | 320 +++++++++++++ 48 files changed, 2308 insertions(+), 112 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/logger/platform/PlatformLogger.java create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/custom-log.xml create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/dotnet-log4j.xml create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/CustomLoggerTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/DefaultLoggerTest.cs delete mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Logger.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Impl/Log/JavaLogger.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Log/CategoryLogger.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Log/ILogger.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Log/LogLevel.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Log/LoggerExtensions.cs diff --git a/modules/core/src/main/java/org/apache/ignite/internal/logger/platform/PlatformLogger.java b/modules/core/src/main/java/org/apache/ignite/internal/logger/platform/PlatformLogger.java new file mode 100644 index 0000000000000..0a0437e9eed89 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/logger/platform/PlatformLogger.java @@ -0,0 +1,223 @@ +/* + * 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.logger.platform; + +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.binary.BinaryRawWriterEx; +import org.apache.ignite.internal.processors.platform.PlatformContext; +import org.apache.ignite.internal.processors.platform.PlatformNativeException; +import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway; +import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; +import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream; +import org.apache.ignite.internal.util.typedef.X; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_QUIET; + +/** + * Logger that delegates to platform. + */ +public class PlatformLogger implements IgniteLogger { + /** */ + public static final int LVL_TRACE = 0; + + /** */ + public static final int LVL_DEBUG = 1; + + /** */ + public static final int LVL_INFO = 2; + + /** */ + public static final int LVL_WARN = 3; + + /** */ + public static final int LVL_ERROR = 4; + + /** Callbacks. */ + private volatile PlatformCallbackGateway gate; + + /** Context. */ + private volatile PlatformContext ctx; + + /** Category. */ + private final String category; + + /** Trace flag. */ + private volatile boolean traceEnabled; + + /** Debug flag. */ + private volatile boolean debugEnabled; + + /** Info flag. */ + private volatile boolean infoEnabled; + + /** Quiet flag. */ + private static final boolean isQuiet = Boolean.valueOf(System.getProperty(IGNITE_QUIET, "true")); + + /** + * Ctor. + * + */ + public PlatformLogger() { + category = null; + } + + /** + * Ctor. + */ + private PlatformLogger(PlatformCallbackGateway gate, PlatformContext ctx, String category, + boolean traceEnabled, boolean debugEnabled, boolean infoEnabled) { + this.gate = gate; + this.ctx = ctx; + this.category = category; + this.traceEnabled = traceEnabled; + this.debugEnabled = debugEnabled; + this.infoEnabled = infoEnabled; + } + + /** {@inheritDoc} */ + @Override public IgniteLogger getLogger(Object ctgr) { + return new PlatformLogger(gate, ctx, getCategoryString(ctgr), traceEnabled, debugEnabled, infoEnabled); + } + + /** {@inheritDoc} */ + @Override public void trace(String msg) { + log(LVL_TRACE, msg, null); + } + + /** {@inheritDoc} */ + @Override public void debug(String msg) { + log(LVL_DEBUG, msg, null); + } + + /** {@inheritDoc} */ + @Override public void info(String msg) { + log(LVL_INFO, msg, null); + } + + /** {@inheritDoc} */ + @Override public void warning(String msg) { + log(LVL_WARN, msg, null); + } + + /** {@inheritDoc} */ + @Override public void warning(String msg, @Nullable Throwable e) { + log(LVL_WARN, msg, e); + } + + /** {@inheritDoc} */ + @Override public void error(String msg) { + log(LVL_ERROR, msg, null); + } + + /** {@inheritDoc} */ + @Override public void error(String msg, @Nullable Throwable e) { + log(LVL_ERROR, msg, e); + } + + /** {@inheritDoc} */ + @Override public boolean isTraceEnabled() { + return traceEnabled; + } + + /** {@inheritDoc} */ + @Override public boolean isDebugEnabled() { + return debugEnabled; + } + + /** {@inheritDoc} */ + @Override public boolean isInfoEnabled() { + return infoEnabled; + } + + /** {@inheritDoc} */ + @Override public boolean isQuiet() { + return isQuiet; + } + + /** {@inheritDoc} */ + @Override public String fileName() { + return null; + } + + /** + * Sets the gateway. + * + * @param gate Callback gateway. + */ + public void setGateway(PlatformCallbackGateway gate) { + assert gate != null; + this.gate = gate; + + // Pre-calculate enabled levels (JNI calls are expensive) + traceEnabled = gate.loggerIsLevelEnabled(LVL_TRACE); + debugEnabled = gate.loggerIsLevelEnabled(LVL_DEBUG); + infoEnabled = gate.loggerIsLevelEnabled(LVL_INFO); + } + + /** + * Sets the context. + * + * @param ctx Platform context. + */ + public void setContext(PlatformContext ctx) { + assert ctx != null; + this.ctx = ctx; + } + + /** + * Logs the message. + * + * @param level Log level. + * @param msg Message. + * @param e Exception. + */ + private void log(int level, String msg, @Nullable Throwable e) { + String errorInfo = null; + + if (e != null) + errorInfo = X.getFullStackTrace(e); + + PlatformNativeException e0 = X.cause(e, PlatformNativeException.class); + if (ctx != null && e0 != null) { + try (PlatformMemory mem = ctx.memory().allocate()) { + PlatformOutputStream out = mem.output(); + BinaryRawWriterEx writer = ctx.writer(out); + writer.writeObject(e0.cause()); + out.synchronize(); + + gate.loggerLog(level, msg, category, errorInfo, mem.pointer()); + } + } + else { + gate.loggerLog(level, msg, category, errorInfo, 0); + } + } + + /** + * Gets the category string. + * + * @param ctgr Category object. + * @return Category string. + */ + private static String getCategoryString(Object ctgr) { + return ctgr instanceof Class + ? ((Class)ctgr).getName() + : (ctgr == null ? null : String.valueOf(ctgr)); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractBootstrap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractBootstrap.java index a28677fb710a5..8b4eb75992fa4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractBootstrap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractBootstrap.java @@ -22,7 +22,9 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgnitionEx; +import org.apache.ignite.internal.logger.platform.PlatformLogger; import org.apache.ignite.internal.processors.platform.memory.PlatformExternalMemory; +import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream; import org.apache.ignite.internal.processors.resource.GridSpringResourceContext; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; @@ -35,7 +37,11 @@ public abstract class PlatformAbstractBootstrap implements PlatformBootstrap { /** {@inheritDoc} */ @Override public PlatformProcessor start(IgniteConfiguration cfg, @Nullable GridSpringResourceContext springCtx, long envPtr, long dataPtr) { - Ignition.setClientMode(new PlatformExternalMemory(null, dataPtr).input().readBoolean()); + final PlatformInputStream input = new PlatformExternalMemory(null, dataPtr).input(); + + Ignition.setClientMode(input.readBoolean()); + + processInput(input, cfg); IgniteConfiguration cfg0 = closure(envPtr).apply(cfg); @@ -61,4 +67,14 @@ public abstract class PlatformAbstractBootstrap implements PlatformBootstrap { * @return Closure. */ protected abstract IgniteClosure closure(long envPtr); + + /** + * Processes any additional input data. + * + * @param input Input stream. + * @param cfg Config. + */ + protected void processInput(PlatformInputStream input, IgniteConfiguration cfg) { + // No-op. + } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformConfigurationEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformConfigurationEx.java index b7c8895860a00..97f08661d3a09 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformConfigurationEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformConfigurationEx.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.platform; import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension; +import org.apache.ignite.internal.logger.platform.PlatformLogger; import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway; import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManagerImpl; import org.jetbrains.annotations.Nullable; @@ -52,4 +53,9 @@ public interface PlatformConfigurationEx { * @return Available cache extensions. */ @Nullable public Collection cacheExtensions(); + + /** + * @return Platform logger. + */ + public PlatformLogger logger(); } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java index cdf418f9b733a..a7b7a8d9b028c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java @@ -175,4 +175,14 @@ public PlatformNoopProcessor(GridKernalContext ctx) { @Override public PlatformTarget getOrCreateNearCache(@Nullable String cacheName, long memPtr) { return null; } + + /** {@inheritDoc} */ + @Override public boolean loggerIsLevelEnabled(int level) { + return false; + } + + /** {@inheritDoc} */ + @Override public void loggerLog(int level, String message, String category, String errorInfo) { + // No-op. + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java index f0201eff67d80..1d9d3cdc1eee9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java @@ -255,4 +255,20 @@ public interface PlatformProcessor extends GridProcessor { */ public PlatformTarget getOrCreateNearCache(@Nullable String cacheName, long memPtr); + /** + * Gets a value indicating whether Ignite logger has specified level enabled. + * + * @param level Log level. + */ + public boolean loggerIsLevelEnabled(int level); + + /** + * Logs to the Ignite logger. + * + * @param level Log level. + * @param message Message. + * @param category Category. + * @param errorInfo Error info. + */ + public void loggerLog(int level, String message, String category, String errorInfo); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java index b364c4a11ca99..548145e56639a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java @@ -30,6 +30,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.binary.BinaryRawReaderEx; import org.apache.ignite.internal.binary.BinaryRawWriterEx; +import org.apache.ignite.internal.logger.platform.PlatformLogger; import org.apache.ignite.internal.processors.GridProcessorAdapter; import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl; @@ -128,6 +129,9 @@ public PlatformProcessorImpl(GridKernalContext ctx) { // Initialize cache extensions (if any). cacheExts = prepareCacheExtensions(interopCfg.cacheExtensions()); + + if (interopCfg.logger() != null) + interopCfg.logger().setContext(platformCtx); } /** {@inheritDoc} */ @@ -433,6 +437,58 @@ private PlatformTarget createPlatformCache(IgniteCacheProxy cache) { return new PlatformCache(platformCtx, cache, false, cacheExts); } + /** {@inheritDoc} */ + @Override public boolean loggerIsLevelEnabled(int level) { + IgniteLogger log = ctx.grid().log(); + + switch (level) { + case PlatformLogger.LVL_TRACE: + return log.isTraceEnabled(); + case PlatformLogger.LVL_DEBUG: + return log.isDebugEnabled(); + case PlatformLogger.LVL_INFO: + return log.isInfoEnabled(); + case PlatformLogger.LVL_WARN: + return true; + case PlatformLogger.LVL_ERROR: + return true; + default: + assert false; + } + + return false; + } + + /** {@inheritDoc} */ + @Override public void loggerLog(int level, String message, String category, String errorInfo) { + IgniteLogger log = ctx.grid().log(); + + if (category != null) + log = log.getLogger(category); + + Throwable err = errorInfo == null ? null : new IgniteException("Platform error:" + errorInfo); + + switch (level) { + case PlatformLogger.LVL_TRACE: + log.trace(message); + break; + case PlatformLogger.LVL_DEBUG: + log.debug(message); + break; + case PlatformLogger.LVL_INFO: + log.info(message); + break; + case PlatformLogger.LVL_WARN: + log.warning(message, err); + break; + case PlatformLogger.LVL_ERROR: + log.error(message, err); + break; + default: + assert false; + } + } + /** * Gets the near cache config. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java index 41d3802fecc94..de23242f45fa6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java @@ -942,6 +942,44 @@ public void onClientReconnected(boolean clusterRestarted) { } } + /** + * Logs to the platform. + * + * @param level Log level. + * @param message Message. + * @param category Category. + * @param errorInfo Error info. + * @param memPtr Pointer to optional payload (serialized exception). + */ + public void loggerLog(int level, String message, String category, String errorInfo, long memPtr) { + if (!tryEnter()) + return; // Do not lock for logger: this should work during shutdown + + try { + PlatformCallbackUtils.loggerLog(envPtr, level, message, category, errorInfo, memPtr); + } + finally { + leave(); + } + } + + /** + * Gets a value indicating whether native logger has specified level enabled. + * + * @param level Log level. + */ + public boolean loggerIsLevelEnabled(int level) { + if (!tryEnter()) + return false; // Do not lock for logger: this should work during shutdown + + try { + return PlatformCallbackUtils.loggerIsLevelEnabled(envPtr, level); + } + finally { + leave(); + } + } + /** * Kernal stop callback. */ @@ -1057,6 +1095,13 @@ protected void enter() { throw new IgniteException("Failed to execute native callback because grid is stopping."); } + /** + * Enter gateway. + */ + protected boolean tryEnter() { + return lock.enterBusy(); + } + /** * Leave gateway. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java index 63c6682377b03..09a7f1bbfc791 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java @@ -552,6 +552,26 @@ static native void dataStreamerStreamReceiverInvoke(long envPtr, long ptr, Objec */ static native void consoleWrite(String str, boolean isErr); + /** + * Logs to the native logger. + * + * @param envPtr Environment pointer. + * @param level Log level. + * @param message Message. + * @param category Category. + * @param errorInfo Error info. + * @param memPtr Pointer to optional payload (serialized exception). + */ + static native void loggerLog(long envPtr, int level, String message, String category, String errorInfo, long memPtr); + + /** + * Gets a value indicating whether native logger has specified level enabled. + * + * @param envPtr Environment pointer. + * @param level Log level. + */ + static native boolean loggerIsLevelEnabled(long envPtr, int level); + /** * Private constructor. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationEx.java index 931a18eb812cb..785c9bd06c262 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationEx.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.platform.cpp; +import org.apache.ignite.internal.logger.platform.PlatformLogger; import org.apache.ignite.internal.processors.platform.PlatformConfigurationEx; import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension; import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway; @@ -80,6 +81,11 @@ public PlatformCppConfigurationEx(PlatformCppConfiguration cfg, PlatformCallback return null; } + /** {@inheritDoc} */ + @Override public PlatformLogger logger() { + return null; + } + /** * @param warnings Warnings. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetBootstrap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetBootstrap.java index 9278246fc9ad1..84a45778e35b7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetBootstrap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetBootstrap.java @@ -17,8 +17,11 @@ package org.apache.ignite.internal.processors.platform.dotnet; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.logger.platform.PlatformLogger; import org.apache.ignite.internal.processors.platform.PlatformAbstractBootstrap; import org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure; +import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream; import java.io.PrintStream; @@ -40,4 +43,10 @@ public class PlatformDotNetBootstrap extends PlatformAbstractBootstrap { @Override protected PlatformAbstractConfigurationClosure closure(long envPtr) { return new PlatformDotNetConfigurationClosure(envPtr); } + + /** {@inheritDoc} */ + @Override protected void processInput(PlatformInputStream input, IgniteConfiguration cfg) { + if (input.readBoolean()) + cfg.setGridLogger(new PlatformLogger()); + } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java index cb9696ce609c4..0c691af45c68b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java @@ -31,6 +31,7 @@ import org.apache.ignite.internal.binary.BinaryRawWriterEx; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.binary.GridBinaryMarshaller; +import org.apache.ignite.internal.logger.platform.PlatformLogger; import org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure; import org.apache.ignite.internal.processors.platform.lifecycle.PlatformLifecycleBean; import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; @@ -89,7 +90,15 @@ public PlatformDotNetConfigurationClosure(long envPtr) { memMgr = new PlatformMemoryManagerImpl(gate, 1024); - PlatformDotNetConfigurationEx dotNetCfg0 = new PlatformDotNetConfigurationEx(dotNetCfg, gate, memMgr); + PlatformLogger userLogger = null; + + if (igniteCfg.getGridLogger() instanceof PlatformLogger) { + userLogger = (PlatformLogger)igniteCfg.getGridLogger(); + userLogger.setGateway(gate); + } + + PlatformDotNetConfigurationEx dotNetCfg0 = new PlatformDotNetConfigurationEx(dotNetCfg, gate, memMgr, + userLogger); igniteCfg.setPlatformConfiguration(dotNetCfg0); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java index 78fb755decc00..eded0e720656c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.platform.dotnet; +import org.apache.ignite.internal.logger.platform.PlatformLogger; import org.apache.ignite.internal.processors.platform.PlatformConfigurationEx; import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension; import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway; @@ -39,6 +40,9 @@ public class PlatformDotNetConfigurationEx extends PlatformDotNetConfiguration i /** Memory manager. */ private final PlatformMemoryManagerImpl memMgr; + /** Logger. */ + private final PlatformLogger logger; + /** Warnings */ private Collection warnings; @@ -50,11 +54,12 @@ public class PlatformDotNetConfigurationEx extends PlatformDotNetConfiguration i * @param memMgr Memory manager. */ public PlatformDotNetConfigurationEx(PlatformDotNetConfiguration cfg, PlatformCallbackGateway gate, - PlatformMemoryManagerImpl memMgr) { + PlatformMemoryManagerImpl memMgr, PlatformLogger logger) { super(cfg); this.gate = gate; this.memMgr = memMgr; + this.logger = logger; } /** {@inheritDoc} */ @@ -82,6 +87,11 @@ public PlatformDotNetConfigurationEx(PlatformDotNetConfiguration cfg, PlatformCa return Collections.singleton(new PlatformDotNetSessionCacheExtension()); } + /** {@inheritDoc} */ + @Override public PlatformLogger logger() { + return logger; + } + /** * @param warnings Warnings. */ diff --git a/modules/platforms/cpp/jni/include/ignite/jni/exports.h b/modules/platforms/cpp/jni/include/ignite/jni/exports.h index 3f400fb92437b..7fae0d22d3b9e 100644 --- a/modules/platforms/cpp/jni/include/ignite/jni/exports.h +++ b/modules/platforms/cpp/jni/include/ignite/jni/exports.h @@ -54,6 +54,8 @@ extern "C" { void* IGNITE_CALL IgniteProcessorAtomicReference(gcj::JniContext* ctx, void* obj, char* name, long long memPtr, bool create); void IGNITE_CALL IgniteProcessorGetIgniteConfiguration(gcj::JniContext* ctx, void* obj, long long memPtr); void IGNITE_CALL IgniteProcessorGetCacheNames(gcj::JniContext* ctx, void* obj, long long memPtr); + bool IGNITE_CALL IgniteProcessorLoggerIsLevelEnabled(gcj::JniContext* ctx, void* obj, int level); + void IGNITE_CALL IgniteProcessorLoggerLog(gcj::JniContext* ctx, void* obj, int level, char* message, char* category, char* errorInfo); long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr); void IGNITE_CALL IgniteTargetInStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, long long inMemPtr, long long outMemPtr); diff --git a/modules/platforms/cpp/jni/include/ignite/jni/java.h b/modules/platforms/cpp/jni/include/ignite/jni/java.h index 8d79a7d8daa79..4cf00a3c24c70 100644 --- a/modules/platforms/cpp/jni/include/ignite/jni/java.h +++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h @@ -108,6 +108,9 @@ namespace ignite typedef void(JNICALL *ConsoleWriteHandler)(const char* chars, int charsLen, unsigned char isErr); + typedef void(JNICALL *LoggerLogHandler)(void* target, int level, const char* messageChars, int messageCharsLen, const char* categoryChars, int categoryCharsLen, const char* errorInfoChars, int errorInfoCharsLen, long long memPtr); + typedef bool(JNICALL *LoggerIsLevelEnabledHandler)(void* target, int level); + /** * JNI handlers holder. */ @@ -191,6 +194,9 @@ namespace ignite AffinityFunctionAssignPartitionsHandler affinityFunctionAssignPartitions; AffinityFunctionRemoveNodeHandler affinityFunctionRemoveNode; AffinityFunctionDestroyHandler affinityFunctionDestroy; + + LoggerLogHandler loggerLog; + LoggerIsLevelEnabledHandler loggerIsLevelEnabled; }; /** @@ -335,6 +341,8 @@ namespace ignite jmethodID m_PlatformProcessor_getCacheNames; jmethodID m_PlatformProcessor_atomicSequence; jmethodID m_PlatformProcessor_atomicReference; + jmethodID m_PlatformProcessor_loggerIsLevelEnabled; + jmethodID m_PlatformProcessor_loggerLog; jclass c_PlatformTarget; jmethodID m_PlatformTarget_inStreamOutLong; @@ -554,6 +562,8 @@ namespace ignite jobject ProcessorAtomicReference(jobject obj, char* name, long long memPtr, bool create); void ProcessorGetIgniteConfiguration(jobject obj, long long memPtr); void ProcessorGetCacheNames(jobject obj, long long memPtr); + bool ProcessorLoggerIsLevelEnabled(jobject obj, int level); + void ProcessorLoggerLog(jobject obj, int level, char* message, char* category, char* errorInfo); long long TargetInStreamOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); void TargetInStreamOutStream(jobject obj, int opType, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL); @@ -767,6 +777,9 @@ namespace ignite JNIEXPORT void JNICALL JniAffinityFunctionDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong ptr); JNIEXPORT void JNICALL JniConsoleWrite(JNIEnv *env, jclass cls, jstring str, jboolean isErr); + + JNIEXPORT void JNICALL JniLoggerLog(JNIEnv *env, jclass cls, jlong envPtr, jint level, jstring message, jstring category, jstring errorInfo, jlong memPtr); + JNIEXPORT jboolean JNICALL JniLoggerIsLevelEnabled(JNIEnv *env, jclass cls, jlong envPtr, jint level); } } } diff --git a/modules/platforms/cpp/jni/project/vs/module.def b/modules/platforms/cpp/jni/project/vs/module.def index ddddace20496e..c1582e6b1be09 100644 --- a/modules/platforms/cpp/jni/project/vs/module.def +++ b/modules/platforms/cpp/jni/project/vs/module.def @@ -136,4 +136,5 @@ IgniteProcessorGetCacheNames @133 IgniteProjectionForServers @134 IgniteSetConsoleHandler @135 IgniteRemoveConsoleHandler @136 - \ No newline at end of file +IgniteProcessorLoggerIsLevelEnabled @137 +IgniteProcessorLoggerLog @138 \ No newline at end of file diff --git a/modules/platforms/cpp/jni/src/exports.cpp b/modules/platforms/cpp/jni/src/exports.cpp index 2950d15d0ea52..8ef81885cd7f0 100644 --- a/modules/platforms/cpp/jni/src/exports.cpp +++ b/modules/platforms/cpp/jni/src/exports.cpp @@ -93,11 +93,11 @@ extern "C" { void*IGNITE_CALL IgniteProcessorDataStreamer(gcj::JniContext* ctx, void* obj, char* name, bool keepPortable) { return ctx->ProcessorDataStreamer(static_cast(obj), name, keepPortable); } - + void* IGNITE_CALL IgniteProcessorTransactions(gcj::JniContext* ctx, void* obj) { return ctx->ProcessorTransactions(static_cast(obj)); } - + void* IGNITE_CALL IgniteProcessorCompute(gcj::JniContext* ctx, void* obj, void* prj) { return ctx->ProcessorCompute(static_cast(obj), static_cast(prj)); } @@ -138,6 +138,14 @@ extern "C" { return ctx->ProcessorGetCacheNames(static_cast(obj), memPtr); } + bool IGNITE_CALL IgniteProcessorLoggerIsLevelEnabled(gcj::JniContext* ctx, void* obj, int level) { + return ctx->ProcessorLoggerIsLevelEnabled(static_cast(obj), level); + } + + void IGNITE_CALL IgniteProcessorLoggerLog(gcj::JniContext* ctx, void* obj, int level, char* message, char* category, char* errorInfo) { + ctx->ProcessorLoggerLog(static_cast(obj), level, message, category, errorInfo); + } + long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr) { return ctx->TargetInStreamOutLong(static_cast(obj), opType, memPtr); } diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp index 7bd628784fdd5..eb0d89a7ccee2 100644 --- a/modules/platforms/cpp/jni/src/java.cpp +++ b/modules/platforms/cpp/jni/src/java.cpp @@ -216,6 +216,8 @@ namespace ignite JniMethod M_PLATFORM_PROCESSOR_ATOMIC_REFERENCE = JniMethod("atomicReference", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false); JniMethod M_PLATFORM_PROCESSOR_GET_IGNITE_CONFIGURATION = JniMethod("getIgniteConfiguration", "(J)V", false); JniMethod M_PLATFORM_PROCESSOR_GET_CACHE_NAMES = JniMethod("getCacheNames", "(J)V", false); + JniMethod M_PLATFORM_PROCESSOR_LOGGER_IS_LEVEL_ENABLED = JniMethod("loggerIsLevelEnabled", "(I)Z", false); + JniMethod M_PLATFORM_PROCESSOR_LOGGER_LOG = JniMethod("loggerLog", "(ILjava/lang/String;Ljava/lang/String;Ljava/lang/String;)V", false); const char* C_PLATFORM_TARGET = "org/apache/ignite/internal/processors/platform/PlatformTarget"; JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_LONG = JniMethod("inStreamOutLong", "(IJ)J", false); @@ -366,6 +368,9 @@ namespace ignite JniMethod M_PLATFORM_CALLBACK_UTILS_ON_CLIENT_DISCONNECTED = JniMethod("onClientDisconnected", "(J)V", true); JniMethod M_PLATFORM_CALLBACK_UTILS_ON_CLIENT_RECONNECTED = JniMethod("onClientReconnected", "(JZ)V", true); + JniMethod M_PLATFORM_CALLBACK_UTILS_LOGGER_LOG = JniMethod("loggerLog", "(JILjava/lang/String;Ljava/lang/String;Ljava/lang/String;J)V", true); + JniMethod M_PLATFORM_CALLBACK_UTILS_LOGGER_IS_LEVEL_ENABLED = JniMethod("loggerIsLevelEnabled", "(JI)Z", true); + JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_INIT = JniMethod("affinityFunctionInit", "(JJLorg/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunctionTarget;)J", true); JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_PARTITION = JniMethod("affinityFunctionPartition", "(JJJ)I", true); JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_ASSIGN_PARTITIONS = JniMethod("affinityFunctionAssignPartitions", "(JJJJ)V", true); @@ -716,6 +721,8 @@ namespace ignite m_PlatformProcessor_atomicReference = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_ATOMIC_REFERENCE); m_PlatformProcessor_getIgniteConfiguration = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_GET_IGNITE_CONFIGURATION); m_PlatformProcessor_getCacheNames = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_GET_CACHE_NAMES); + m_PlatformProcessor_loggerIsLevelEnabled = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_LOGGER_IS_LEVEL_ENABLED); + m_PlatformProcessor_loggerLog = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_LOGGER_LOG); c_PlatformTarget = FindClass(env, C_PLATFORM_TARGET); m_PlatformTarget_inStreamOutLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_LONG); @@ -851,7 +858,7 @@ namespace ignite void RegisterNatives(JNIEnv* env) { { - JNINativeMethod methods[60]; + JNINativeMethod methods[62]; int idx = 0; @@ -935,6 +942,9 @@ namespace ignite AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_DESTROY, reinterpret_cast(JniAffinityFunctionDestroy)); AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CONSOLE_WRITE, reinterpret_cast(JniConsoleWrite)); + AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_LOGGER_LOG, reinterpret_cast(JniLoggerLog)); + AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_LOGGER_IS_LEVEL_ENABLED, reinterpret_cast(JniLoggerIsLevelEnabled)); + jint res = env->RegisterNatives(FindClass(env, C_PLATFORM_CALLBACK_UTILS), methods, idx); if (res != JNI_OK) @@ -1526,6 +1536,40 @@ namespace ignite ExceptionCheck(env); } + bool JniContext::ProcessorLoggerIsLevelEnabled(jobject obj, int level) + { + JNIEnv* env = Attach(); + + jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformProcessor_loggerIsLevelEnabled, level); + + ExceptionCheck(env); + + return res != 0; + } + + void JniContext::ProcessorLoggerLog(jobject obj, int level, char* message, char* category, char* errorInfo) + { + JNIEnv* env = Attach(); + + jstring message0 = message != NULL ? env->NewStringUTF(message) : NULL; + jstring category0 = category != NULL ? env->NewStringUTF(category) : NULL; + jstring errorInfo0 = errorInfo != NULL ? env->NewStringUTF(errorInfo) : NULL; + + + env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformProcessor_loggerLog, level, message0, category0, errorInfo0); + + if (message0) + env->DeleteLocalRef(message0); + + if (category0) + env->DeleteLocalRef(category0); + + if (errorInfo0) + env->DeleteLocalRef(errorInfo0); + + ExceptionCheck(env); + } + long long JniContext::TargetInStreamOutLong(jobject obj, int opType, long long memPtr, JniErrorInfo* err) { JNIEnv* env = Attach(); @@ -2952,6 +2996,32 @@ namespace ignite CONSOLE_LOCK.Leave(); } + + JNIEXPORT void JNICALL JniLoggerLog(JNIEnv *env, jclass cls, jlong envPtr, jint level, jstring message, jstring category, jstring errorInfo, jlong memPtr) { + int messageLen; + char* messageChars = StringToChars(env, message, &messageLen); + + int categoryLen; + char* categoryChars = StringToChars(env, category, &categoryLen); + + int errorInfoLen; + char* errorInfoChars = StringToChars(env, errorInfo, &errorInfoLen); + + IGNITE_SAFE_PROC(env, envPtr, LoggerLogHandler, loggerLog, level, messageChars, messageLen, categoryChars, categoryLen, errorInfoChars, errorInfoLen, memPtr); + + if (messageChars) + delete[] messageChars; + + if (categoryChars) + delete[] categoryChars; + + if (errorInfoChars) + delete[] errorInfoChars; + } + + JNIEXPORT jboolean JNICALL JniLoggerIsLevelEnabled(JNIEnv *env, jclass cls, jlong envPtr, jint level) { + IGNITE_SAFE_FUNC(env, envPtr, LoggerIsLevelEnabledHandler, loggerIsLevelEnabled, level); + } } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj index b1e0dbe3b2d8d..2c6150c150dda 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj @@ -53,6 +53,7 @@ + @@ -135,6 +136,7 @@ + @@ -241,6 +243,12 @@ PreserveNewest + + PreserveNewest + + + PreserveNewest + PreserveNewest diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryStringTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryStringTest.cs index 0c1f0f3991311..0593af77093bf 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryStringTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryStringTest.cs @@ -18,7 +18,6 @@ namespace Apache.Ignite.Core.Tests.Binary { using System; - using System.Diagnostics; using Apache.Ignite.Core.Impl.Binary; using NUnit.Framework; @@ -83,29 +82,10 @@ private static void CheckString(IIgnite ignite, string test, bool fail = false) [Test] public void TestNewMode() { - // Run "TestNewMode" in a separate process - var envVar = BinaryUtils.IgniteBinaryMarshallerUseStringSerializationVer2; + // Run "TestOldMode" in a separate process with changed setting. + Environment.SetEnvironmentVariable(BinaryUtils.IgniteBinaryMarshallerUseStringSerializationVer2, "true"); - Environment.SetEnvironmentVariable(envVar, "true"); - - var procStart = new ProcessStartInfo - { - FileName = GetType().Assembly.Location, - Arguments = GetType().FullName + " TestOldMode", - CreateNoWindow = true, - UseShellExecute = false, - RedirectStandardOutput = true, - RedirectStandardError = true - }; - - var proc = Process.Start(procStart); - - Assert.IsNotNull(proc); - - Console.WriteLine(proc.StandardOutput.ReadToEnd()); - Console.WriteLine(proc.StandardError.ReadToEnd()); - Assert.IsTrue(proc.WaitForExit(15000)); - Assert.AreEqual(0, proc.ExitCode); + TestUtils.RunTestInNewProcess(GetType().FullName, "TestOldMode"); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/custom-log.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/custom-log.xml new file mode 100644 index 0000000000000..5d7dc658006a1 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/custom-log.xml @@ -0,0 +1,50 @@ + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500 + + + + + + + + + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/dotnet-log4j.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/dotnet-log4j.xml new file mode 100644 index 0000000000000..c8c62d755953b --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Log/dotnet-log4j.xml @@ -0,0 +1,143 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs index e8e8f5d862a02..beb1e8d0dacc0 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs @@ -43,6 +43,7 @@ namespace Apache.Ignite.Core.Tests using Apache.Ignite.Core.Events; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Lifecycle; + using Apache.Ignite.Core.Log; using Apache.Ignite.Core.Tests.Binary; using Apache.Ignite.Core.Transactions; using NUnit.Framework; @@ -109,6 +110,7 @@ public void TestPredefinedXml() + "; var reader = XmlReader.Create(new StringReader(xml)); @@ -193,6 +195,8 @@ public void TestPredefinedXml() Assert.IsNotNull(comm); Assert.AreEqual(33, comm.AckSendThreshold); Assert.AreEqual(new TimeSpan(0, 1, 2), comm.IdleConnectionTimeout); + + Assert.IsInstanceOf(cfg.Logger); } /// @@ -253,7 +257,7 @@ private static void CheckSchemaValidation(string xml) { var document = new XmlDocument(); - document.Schemas.Add("http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection", + document.Schemas.Add("http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection", XmlReader.Create("IgniteConfigurationSection.xsd")); document.Load(new StringReader(xml)); @@ -542,7 +546,10 @@ private static IgniteConfiguration GetTestConfig() SlowClientQueueLimit = 98, SocketSendBufferSize = 2045, UnacknowledgedMessagesBufferSize = 3450 - } + }, + IsLateAffinityAssignment = false, + SpringConfigUrl = "test", + Logger = new TestLogger() }; } @@ -688,5 +695,24 @@ public ICacheStore CreateInstance() return null; } } + + /// + /// Test logger. + /// + public class TestLogger : ILogger + { + /** */ + public void Log(LogLevel level, string message, object[] args, IFormatProvider formatProvider, string category, + string nativeErrorInfo, Exception ex) + { + throw new NotImplementedException(); + } + + /** */ + public bool IsEnabled(LogLevel level) + { + throw new NotImplementedException(); + } + } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/CustomLoggerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/CustomLoggerTest.cs new file mode 100644 index 0000000000000..73134febd4203 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/CustomLoggerTest.cs @@ -0,0 +1,430 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Tests.Log +{ + using System; + using System.Collections.Generic; + using System.Globalization; + using System.Linq; + using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Common; + using Apache.Ignite.Core.Communication.Tcp; + using Apache.Ignite.Core.Compute; + using Apache.Ignite.Core.Impl; + using Apache.Ignite.Core.Lifecycle; + using Apache.Ignite.Core.Log; + using Apache.Ignite.Core.Resource; + using NUnit.Framework; + + /// + /// Tests that user-defined logger receives Ignite events. + /// + public class CustomLoggerTest + { + /** */ + private static readonly LogLevel[] AllLevels = Enum.GetValues(typeof (LogLevel)).OfType().ToArray(); + + /// + /// Test setup. + /// + [SetUp] + public void TestSetUp() + { + TestLogger.Entries.Clear(); + } + + /// + /// Tests the startup output. + /// + [Test] + public void TestStartupOutput() + { + var cfg = GetConfigWithLogger(true); + using (var ignite = Ignition.Start(cfg)) + { + // Check injection + Assert.AreEqual(((Ignite) ignite).Proxy, ((TestLogger) cfg.Logger).Ignite); + + // Check initial message + Assert.IsTrue(TestLogger.Entries[0].Message.StartsWith("Starting Ignite.NET")); + + // Check topology message + Assert.IsTrue( + TestUtils.WaitForCondition(() => + { + lock (TestLogger.Entries) + { + return TestLogger.Entries.Any(x => x.Message.Contains("Topology snapshot")); + } + }, 9000), "No topology snapshot"); + } + + // Test that all levels are present + foreach (var level in AllLevels.Where(x => x != LogLevel.Error)) + Assert.IsTrue(TestLogger.Entries.Any(x => x.Level == level), "No messages with level " + level); + } + + + /// + /// Tests startup error in Java. + /// + [Test] + public void TestStartupJavaError() + { + // Invalid config + Assert.Throws(() => + Ignition.Start(new IgniteConfiguration(GetConfigWithLogger()) + { + CommunicationSpi = new TcpCommunicationSpi + { + IdleConnectionTimeout = TimeSpan.MinValue + } + })); + + var err = TestLogger.Entries.First(x => x.Level == LogLevel.Error); + Assert.IsTrue(err.NativeErrorInfo.Contains("SPI parameter failed condition check: idleConnTimeout > 0")); + Assert.AreEqual("org.apache.ignite.internal.IgniteKernal", err.Category); + Assert.IsNull(err.Exception); + } + + /// + /// Tests startup error in .NET. + /// + [Test] + public void TestStartupDotNetError() + { + // Invalid bean + Assert.Throws(() => + Ignition.Start(new IgniteConfiguration(GetConfigWithLogger()) + { + LifecycleBeans = new[] {new FailBean()} + })); + + var err = TestLogger.Entries.First(x => x.Level == LogLevel.Error); + Assert.IsInstanceOf(err.Exception); + } + + /// + /// Tests that .NET exception propagates through Java to the log. + /// + [Test] + public void TestDotNetErrorPropagation() + { + // Start 2 nodes: PlatformNativeException does not occur in local scenario + using (var ignite = Ignition.Start(GetConfigWithLogger())) + using (Ignition.Start(new IgniteConfiguration(TestUtils.GetTestConfiguration()) {GridName = "1"})) + { + var compute = ignite.GetCluster().ForRemotes().GetCompute(); + + Assert.Throws(() => compute.Call(new FailFunc())); + + // Log updates may not arrive immediately + TestUtils.WaitForCondition(() => TestLogger.Entries.Any(x => x.Exception != null), 3000); + + var errFromJava = TestLogger.Entries.Single(x => x.Exception != null); + Assert.AreEqual("Error in func.", ((ArithmeticException) errFromJava.Exception.InnerException).Message); + } + } + + /// + /// Tests the validation. + /// + [Test] + public void TestQueryEntityValidation() + { + var cacheCfg = new CacheConfiguration("cache1", new QueryEntity(typeof(uint), typeof(ulong)) + { + Fields = new[] + { + new QueryField("myField", typeof(ushort)) + } + }); + + var cfg = new IgniteConfiguration(GetConfigWithLogger()) + { + CacheConfiguration = new[] + { + cacheCfg + } + }; + + using (var ignite = Ignition.Start(cfg)) + { + // Check static and dynamic cache start + cacheCfg.Name = "cache2"; + ignite.CreateCache(cacheCfg); + + var warns = TestLogger.Entries.Where(x => x.Level == LogLevel.Warn && x.Args != null) + .Select(x => string.Format(x.Message, x.Args)).ToList(); + + Assert.AreEqual(6, warns.Count); + + Assert.AreEqual("Validating cache configuration 'cache1', QueryEntity 'java.lang.Integer:java.lang." + + "Long': Type 'System.UInt32' maps to Java type 'java.lang.Integer' using unchecked " + + "conversion. This may cause issues in SQL queries. You can use 'System.Int32' " + + "instead to achieve direct mapping.", warns[0]); + + Assert.AreEqual("Validating cache configuration 'cache1', QueryEntity 'java.lang.Integer:java.lang." + + "Long': Type 'System.UInt64' maps to Java type 'java.lang.Long' using unchecked " + + "conversion. This may cause issues in SQL queries. You can use 'System.Int64' " + + "instead to achieve direct mapping.", warns[1]); + + Assert.AreEqual("Validating cache configuration 'cache1', QueryEntity 'java.lang.Integer:java.lang." + + "Long', QueryField 'myField': Type 'System.UInt16' maps to Java type 'java.lang." + + "Short' using unchecked conversion. This may cause issues in SQL queries. You " + + "can use 'System.Int16' instead to achieve direct mapping.", warns[2]); + + Assert.AreEqual("Validating cache configuration 'cache2', QueryEntity 'java.lang.Integer:java.lang." + + "Long': Type 'System.UInt32' maps to Java type 'java.lang.Integer' using unchecked " + + "conversion. This may cause issues in SQL queries. You can use 'System.Int32' " + + "instead to achieve direct mapping.", warns[3]); + + Assert.AreEqual("Validating cache configuration 'cache2', QueryEntity 'java.lang.Integer:java.lang." + + "Long': Type 'System.UInt64' maps to Java type 'java.lang.Long' using unchecked " + + "conversion. This may cause issues in SQL queries. You can use 'System.Int64' " + + "instead to achieve direct mapping.", warns[4]); + + Assert.AreEqual("Validating cache configuration 'cache2', QueryEntity 'java.lang.Integer:java.lang." + + "Long', QueryField 'myField': Type 'System.UInt16' maps to Java type 'java.lang." + + "Short' using unchecked conversion. This may cause issues in SQL queries. You " + + "can use 'System.Int16' instead to achieve direct mapping.", warns[5]); + } + } + + /// + /// Tests the methods. + /// + [Test] + public void TestExtensions() + { + var log = new TestLogger(LogLevel.Trace); + var ex = new FieldAccessException("abc"); + + // Log + log.Log(LogLevel.Trace, "trace"); + CheckLastMessage(LogLevel.Trace, "trace"); + + log.Log(LogLevel.Debug, "msg {0} {1}", 1, "2"); + CheckLastMessage(LogLevel.Debug, "msg {0} {1}", new object[] { 1, "2" }, CultureInfo.InvariantCulture); + + log.Log(LogLevel.Info, ex, "msg"); + CheckLastMessage(LogLevel.Info, "msg", e: ex); + + log.Log(LogLevel.Warn, ex, "msg {0}", 1); + CheckLastMessage(LogLevel.Warn, "msg {0}", new object[] {1}, CultureInfo.InvariantCulture, e: ex); + + // Trace + log.Trace("trace"); + CheckLastMessage(LogLevel.Trace, "trace"); + + log.Trace("msg {0} {1}", 1, "2"); + CheckLastMessage(LogLevel.Trace, "msg {0} {1}", new object[] { 1, "2" }, CultureInfo.InvariantCulture); + + log.Trace(ex, "msg"); + CheckLastMessage(LogLevel.Trace, "msg", e: ex); + + log.Trace(ex, "msg {0}", 1); + CheckLastMessage(LogLevel.Trace, "msg {0}", new object[] { 1 }, CultureInfo.InvariantCulture, e: ex); + + // Debug + log.Debug("test"); + CheckLastMessage(LogLevel.Debug, "test"); + + log.Debug("msg {0} {1}", 1, "2"); + CheckLastMessage(LogLevel.Debug, "msg {0} {1}", new object[] { 1, "2" }, CultureInfo.InvariantCulture); + + log.Debug(ex, "msg"); + CheckLastMessage(LogLevel.Debug, "msg", e: ex); + + log.Debug(ex, "msg {0}", 1); + CheckLastMessage(LogLevel.Debug, "msg {0}", new object[] { 1 }, CultureInfo.InvariantCulture, e: ex); + + // Info + log.Info("test"); + CheckLastMessage(LogLevel.Info, "test"); + + log.Info("msg {0} {1}", 1, "2"); + CheckLastMessage(LogLevel.Info, "msg {0} {1}", new object[] { 1, "2" }, CultureInfo.InvariantCulture); + + log.Info(ex, "msg"); + CheckLastMessage(LogLevel.Info, "msg", e: ex); + + log.Info(ex, "msg {0}", 1); + CheckLastMessage(LogLevel.Info, "msg {0}", new object[] { 1 }, CultureInfo.InvariantCulture, e: ex); + + // Warn + log.Warn("test"); + CheckLastMessage(LogLevel.Warn, "test"); + + log.Warn("msg {0} {1}", 1, "2"); + CheckLastMessage(LogLevel.Warn, "msg {0} {1}", new object[] { 1, "2" }, CultureInfo.InvariantCulture); + + log.Warn(ex, "msg"); + CheckLastMessage(LogLevel.Warn, "msg", e: ex); + + log.Warn(ex, "msg {0}", 1); + CheckLastMessage(LogLevel.Warn, "msg {0}", new object[] { 1 }, CultureInfo.InvariantCulture, e: ex); + + // Error + log.Error("test"); + CheckLastMessage(LogLevel.Error, "test"); + + log.Error("msg {0} {1}", 1, "2"); + CheckLastMessage(LogLevel.Error, "msg {0} {1}", new object[] { 1, "2" }, CultureInfo.InvariantCulture); + + log.Error(ex, "msg"); + CheckLastMessage(LogLevel.Error, "msg", e: ex); + + log.Error(ex, "msg {0}", 1); + CheckLastMessage(LogLevel.Error, "msg {0}", new object[] { 1 }, CultureInfo.InvariantCulture, e: ex); + + // GetLogger + var catLog = log.GetLogger("myCategory"); + catLog.Info("info"); + CheckLastMessage(LogLevel.Info, "info", category: "myCategory"); + + catLog.Log(LogLevel.Info, "info", null, null, "explicitCat", null, null); + CheckLastMessage(LogLevel.Info, "info", category: "explicitCat"); + + catLog = catLog.GetLogger("newCat"); + catLog.Info("info"); + CheckLastMessage(LogLevel.Info, "info", category: "newCat"); + + catLog.Log(LogLevel.Info, "info", null, null, "explicitCat", null, null); + CheckLastMessage(LogLevel.Info, "info", category: "explicitCat"); + } + + /// + /// Checks the last message. + /// + private static void CheckLastMessage(LogLevel level, string message, object[] args = null, + IFormatProvider formatProvider = null, string category = null, string nativeErr = null, Exception e = null) + { + var msg = TestLogger.Entries.Last(); + + Assert.AreEqual(msg.Level, level); + Assert.AreEqual(msg.Message, message); + Assert.AreEqual(msg.Args, args); + Assert.AreEqual(msg.FormatProvider, formatProvider); + Assert.AreEqual(msg.Category, category); + Assert.AreEqual(msg.NativeErrorInfo, nativeErr); + Assert.AreEqual(msg.Exception, e); + } + + /// + /// Gets the configuration with logger. + /// + private static IgniteConfiguration GetConfigWithLogger(bool verbose = false) + { + return new IgniteConfiguration(TestUtils.GetTestConfiguration()) + { + Logger = new TestLogger(verbose ? LogLevel.Trace : LogLevel.Info) + }; + } + + /// + /// Test log entry. + /// + private class LogEntry + { + public LogLevel Level; + public string Message; + public object[] Args; + public IFormatProvider FormatProvider; + public string Category; + public string NativeErrorInfo; + public Exception Exception; + + public override string ToString() + { + return string.Format("Level: {0}, Message: {1}, Args: {2}, FormatProvider: {3}, Category: {4}, " + + "NativeErrorInfo: {5}, Exception: {6}", Level, Message, Args, FormatProvider, + Category, NativeErrorInfo, Exception); + } + } + + /// + /// Test logger. + /// + private class TestLogger : ILogger + { + public static readonly List Entries = new List(5000); + + private readonly LogLevel _minLevel; + + public TestLogger(LogLevel minLevel) + { + _minLevel = minLevel; + } + + public void Log(LogLevel level, string message, object[] args, IFormatProvider formatProvider, + string category, string nativeErrorInfo, Exception ex) + { + if (!IsEnabled(level)) + return; + + lock (Entries) + { + Entries.Add(new LogEntry + { + Level = level, + Message = message, + Args = args, + FormatProvider = formatProvider, + Category = category, + NativeErrorInfo = nativeErrorInfo, + Exception = ex + }); + } + } + + public bool IsEnabled(LogLevel level) + { + return level >= _minLevel; + } + + [InstanceResource] + // ReSharper disable once UnusedAutoPropertyAccessor.Local + public IIgnite Ignite { get; set; } + } + + + /// + /// Failing lifecycle bean. + /// + private class FailBean : ILifecycleBean + { + public void OnLifecycleEvent(LifecycleEventType evt) + { + throw new ArithmeticException("Failure in bean"); + } + } + + /// + /// Failing computation. + /// + [Serializable] + private class FailFunc : IComputeFunc + { + public string Invoke() + { + throw new ArithmeticException("Error in func."); + } + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/DefaultLoggerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/DefaultLoggerTest.cs new file mode 100644 index 0000000000000..66234167e06e5 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/DefaultLoggerTest.cs @@ -0,0 +1,114 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Tests.Log +{ + using System; + using System.IO; + using System.Linq; + using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Impl.Common; + using NUnit.Framework; + using LogLevel = Apache.Ignite.Core.Log.LogLevel; + + /// + /// Tests the default logger. + /// + public class DefaultLoggerTest + { + /// + /// Tests that default Java mechanism is used when there is no custom logger. + /// + [Test] + public void TestJavaLogger() + { + // Run the test in a separate process because log4jlogger has some static state, + // and after Ignite has been started once, it is not possible to start a new node + // with a different logger config. + const string envVar = "DefaultLoggerTest.TestJavaLogger"; + + if (Environment.GetEnvironmentVariable(envVar) == "true") + { + // Delete all log files from the work dir + Func getLogs = () => + Directory.GetFiles(IgniteHome.Resolve(null), "dotnet-logger-test.log", SearchOption.AllDirectories); + + getLogs().ToList().ForEach(File.Delete); + + var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration(false)) + { + SpringConfigUrl = @"config\log\custom-log.xml", + CacheConfiguration = new[] + { + new CacheConfiguration("cache1", new QueryEntity(typeof(uint), typeof(ulong))) + } + }; + + // Start Ignite and verify file log + using (var ignite = Ignition.Start(cfg)) + { + // Log with all levels + var log = ignite.Logger; + var levels = new[] {LogLevel.Trace, LogLevel.Info, LogLevel.Debug, LogLevel.Warn, LogLevel.Error}; + + foreach (var level in levels) + { + var ex = new Exception("EXCEPTION_TEST_" + level); + + log.Log(level, "DOTNET-" + level, null, null, "=DOTNET=", null, ex); + } + } + + using (var fs = File.Open(getLogs().Single(), FileMode.Open, FileAccess.Read, FileShare.ReadWrite)) + { + var log = new StreamReader(fs).ReadToEnd(); + + // Check output from Java: + Assert.IsTrue(log.Contains(">>> Topology snapshot.")); + + // Check output from .NET: + Assert.IsTrue(log.Contains("Starting Ignite.NET " + typeof(Ignition).Assembly.GetName().Version)); + + Assert.IsTrue(log.Contains( + "Validating cache configuration 'cache1', QueryEntity 'java.lang.Integer:java.lang." + + "Long': Type 'System.UInt32' maps to Java type 'java.lang.Integer' using unchecked " + + "conversion. This may cause issues in SQL queries. You can use 'System.Int32' " + + "instead to achieve direct mapping.")); + + + // Check custom log output (trace is disabled, errors are logged from Warn and up): + Assert.IsTrue(log.Contains("[INFO ][main][=DOTNET=] DOTNET-Info")); + + Assert.IsTrue(log.Contains("[DEBUG][main][=DOTNET=] DOTNET-Debug")); + + Assert.IsTrue(log.Contains("[WARN ][main][=DOTNET=] DOTNET-Warn")); + Assert.IsTrue(log.Contains("class org.apache.ignite.IgniteException: " + + "Platform error:System.Exception: EXCEPTION_TEST_Warn")); + + Assert.IsTrue(log.Contains("[ERROR][main][=DOTNET=] DOTNET-Error")); + Assert.IsTrue(log.Contains("class org.apache.ignite.IgniteException: " + + "Platform error:System.Exception: EXCEPTION_TEST_Error")); + } + } + else + { + Environment.SetEnvironmentVariable(envVar, "true"); + TestUtils.RunTestInNewProcess(GetType().FullName, "TestJavaLogger"); + } + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs index 6ca2f9df62e7a..d3851dbca33e1 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs @@ -21,6 +21,7 @@ namespace Apache.Ignite.Core.Tests using System; using System.Collections.Concurrent; using System.Collections.Generic; + using System.Diagnostics; using System.Linq; using System.Threading; using Apache.Ignite.Core.Discovery.Tcp; @@ -336,5 +337,30 @@ public static IgniteConfiguration GetTestConfiguration(bool? jvmDebug = null) JvmClasspath = CreateTestClasspath() }; } + + /// + /// Runs the test in new process. + /// + public static void RunTestInNewProcess(string fixtureName, string testName) + { + var procStart = new ProcessStartInfo + { + FileName = typeof(TestUtils).Assembly.Location, + Arguments = fixtureName + " " + testName, + CreateNoWindow = true, + UseShellExecute = false, + RedirectStandardOutput = true, + RedirectStandardError = true + }; + + var proc = System.Diagnostics.Process.Start(procStart); + + Assert.IsNotNull(proc); + + Console.WriteLine(proc.StandardOutput.ReadToEnd()); + Console.WriteLine(proc.StandardError.ReadToEnd()); + Assert.IsTrue(proc.WaitForExit(15000)); + Assert.AreEqual(0, proc.ExitCode); + } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj index 59bf090094ad8..db2a96b202bdb 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj @@ -308,7 +308,6 @@ - @@ -343,6 +342,7 @@ + @@ -435,6 +435,10 @@ + + + + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs index e5e79cdf33d9b..cb15564dc4750 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs @@ -23,6 +23,7 @@ namespace Apache.Ignite.Core.Cache.Configuration using System; using System.Collections.Generic; using System.ComponentModel; + using System.Diagnostics; using System.Diagnostics.CodeAnalysis; using System.Linq; using Apache.Ignite.Core.Binary; @@ -35,6 +36,7 @@ namespace Apache.Ignite.Core.Cache.Configuration using Apache.Ignite.Core.Common; using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Cache.Affinity; + using Apache.Ignite.Core.Log; /// /// Defines grid cache configuration. @@ -352,6 +354,21 @@ internal void Write(IBinaryRawWriter writer) AffinityFunctionSerializer.Write(writer, AffinityFunction); } + /// + /// Validates this instance and outputs information to the log, if necessary. + /// + internal void Validate(ILogger log) + { + Debug.Assert(log != null); + + var entities = QueryEntities; + if (entities != null) + { + foreach (var entity in entities) + entity.Validate(log, string.Format("Validating cache configuration '{0}'", Name ?? "")); + } + } + /// /// Gets or sets write synchronization mode. This mode controls whether the main /// caller should wait for update on other nodes to complete or not. @@ -669,7 +686,7 @@ internal void Write(IBinaryRawWriter writer) /// /// Gets or sets the affinity function to provide mapping from keys to nodes. /// - /// Predefined implementations: + /// Predefined implementations: /// , . /// public IAffinityFunction AffinityFunction { get; set; } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs index e6eceb8725044..adfe9e16b03b2 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs @@ -27,6 +27,7 @@ namespace Apache.Ignite.Core.Cache.Configuration using System.Reflection; using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Impl.Binary; + using Apache.Ignite.Core.Log; /// /// Query entity is a description of cache entry (composed of key and value) @@ -104,7 +105,7 @@ public Type KeyType KeyTypeName = value == null ? null - : (JavaTypes.GetJavaTypeNameAndLogWarning(value) ?? BinaryUtils.GetTypeName(value)); + : (JavaTypes.GetJavaTypeName(value) ?? BinaryUtils.GetTypeName(value)); _keyType = value; } @@ -140,7 +141,7 @@ public Type ValueType ValueTypeName = value == null ? null - : (JavaTypes.GetJavaTypeNameAndLogWarning(value) ?? BinaryUtils.GetTypeName(value)); + : (JavaTypes.GetJavaTypeName(value) ?? BinaryUtils.GetTypeName(value)); _valueType = value; } @@ -239,6 +240,26 @@ internal void Write(IBinaryRawWriter writer) writer.WriteInt(0); } + /// + /// Validates this instance and outputs information to the log, if necessary. + /// + internal void Validate(ILogger log, string logInfo) + { + Debug.Assert(log != null); + Debug.Assert(logInfo != null); + + logInfo += string.Format(", QueryEntity '{0}:{1}'", _keyTypeName ?? "", _valueTypeName ?? ""); + + JavaTypes.LogIndirectMappingWarning(_keyType, log, logInfo); + JavaTypes.LogIndirectMappingWarning(_valueType, log, logInfo); + + var fields = Fields; + if (fields != null) + { + foreach (var field in fields) + field.Validate(log, logInfo); + } + } /// /// Rescans the attributes in and . diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs index b11e53d1717f3..12028e2b06d78 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs @@ -20,8 +20,10 @@ namespace Apache.Ignite.Core.Cache.Configuration { using System; + using System.Diagnostics; using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Common; + using Apache.Ignite.Core.Log; /// /// Represents a queryable field. @@ -71,7 +73,7 @@ public QueryField(string name, Type fieldType) } /// - /// Gets the field name. + /// Gets or sets the field name. /// public string Name { get; set; } @@ -87,7 +89,7 @@ public Type FieldType { FieldTypeName = value == null ? null - : (JavaTypes.GetJavaTypeNameAndLogWarning(value) ?? BinaryUtils.GetTypeName(value)); + : (JavaTypes.GetJavaTypeName(value) ?? BinaryUtils.GetTypeName(value)); _type = value; } @@ -105,5 +107,18 @@ public string FieldTypeName _type = null; } } + + /// + /// Validates this instance and outputs information to the log, if necessary. + /// + internal void Validate(ILogger log, string logInfo) + { + Debug.Assert(log != null); + Debug.Assert(logInfo != null); + + logInfo += string.Format(", QueryField '{0}'", Name); + + JavaTypes.LogIndirectMappingWarning(_type, log, logInfo); + } } } \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs index a16ae3ab75f7a..91aa1dd71fa2e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs @@ -29,6 +29,7 @@ namespace Apache.Ignite.Core using Apache.Ignite.Core.Datastream; using Apache.Ignite.Core.DataStructures; using Apache.Ignite.Core.Events; + using Apache.Ignite.Core.Log; using Apache.Ignite.Core.Lifecycle; using Apache.Ignite.Core.Messaging; using Apache.Ignite.Core.Services; @@ -276,6 +277,13 @@ public interface IIgnite : IDisposable /// Collection of names of currently available caches. ICollection GetCacheNames(); + /// + /// Gets the logger. + /// + /// See for customization. + /// + ILogger Logger { get; } + /// /// Occurs when node begins to stop. Node is fully functional at this point. /// See also: . diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs index 6bdf1ab835f88..8e16fb523f73d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs @@ -38,6 +38,7 @@ namespace Apache.Ignite.Core using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Lifecycle; + using Apache.Ignite.Core.Log; using Apache.Ignite.Core.Transactions; using BinaryReader = Apache.Ignite.Core.Impl.Binary.BinaryReader; using BinaryWriter = Apache.Ignite.Core.Impl.Binary.BinaryWriter; @@ -291,6 +292,21 @@ internal void Write(BinaryWriter writer) writer.WriteBoolean(false); } + /// + /// Validates this instance and outputs information to the log, if necessary. + /// + internal void Validate(ILogger log) + { + Debug.Assert(log != null); + + var ccfg = CacheConfiguration; + if (ccfg != null) + { + foreach (var cfg in ccfg) + cfg.Validate(log); + } + } + /// /// Reads data from specified reader into current instance. /// @@ -398,6 +414,7 @@ private void CopyLocalProperties(IgniteConfiguration cfg) Assemblies = cfg.Assemblies; SuppressWarnings = cfg.SuppressWarnings; LifecycleBeans = cfg.LifecycleBeans; + Logger = cfg.Logger; JvmInitialMemoryMb = cfg.JvmInitialMemoryMb; JvmMaxMemoryMb = cfg.JvmMaxMemoryMb; } @@ -678,5 +695,76 @@ public bool IsLateAffinityAssignment get { return _isLateAffinityAssignment ?? DefaultIsLateAffinityAssignment; } set { _isLateAffinityAssignment = value; } } + + /// + /// Serializes this instance to the specified XML writer. + /// + /// The writer. + /// Name of the root element. + public void ToXml(XmlWriter writer, string rootElementName) + { + IgniteArgumentCheck.NotNull(writer, "writer"); + IgniteArgumentCheck.NotNullOrEmpty(rootElementName, "rootElementName"); + + IgniteConfigurationXmlSerializer.Serialize(this, writer, rootElementName); + } + + /// + /// Serializes this instance to an XML string. + /// + public string ToXml() + { + var sb = new StringBuilder(); + + var settings = new XmlWriterSettings + { + Indent = true + }; + + using (var xmlWriter = XmlWriter.Create(sb, settings)) + { + ToXml(xmlWriter, "igniteConfiguration"); + } + + return sb.ToString(); + } + + /// + /// Deserializes IgniteConfiguration from the XML reader. + /// + /// The reader. + /// Deserialized instance. + public static IgniteConfiguration FromXml(XmlReader reader) + { + IgniteArgumentCheck.NotNull(reader, "reader"); + + return IgniteConfigurationXmlSerializer.Deserialize(reader); + } + + /// + /// Deserializes IgniteConfiguration from the XML string. + /// + /// Xml string. + /// Deserialized instance. + public static IgniteConfiguration FromXml(string xml) + { + IgniteArgumentCheck.NotNullOrEmpty(xml, "xml"); + + using (var xmlReader = XmlReader.Create(new StringReader(xml))) + { + // Skip XML header. + xmlReader.MoveToContent(); + + return FromXml(xmlReader); + } + } + + /// + /// Gets or sets the logger. + /// + /// If no logger is set, logging is delegated to Java, which uses the logger defined in Spring XML (if present) + /// or logs to console otherwise. + /// + public ILogger Logger { get; set; } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd index 24eaa307d10df..916fd2058ef93 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd @@ -357,6 +357,18 @@ + + + The logger. If no logger is set, logging is delegated to Java, which uses the logger defined in Spring XML (if present) or logs to console otherwise. + + + + + Assembly-qualified type name. + + + + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs index 552a7f2d1c0cd..7b023f3e44220 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs @@ -35,9 +35,11 @@ namespace Apache.Ignite.Core using Apache.Ignite.Core.Impl.Cache.Affinity; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Impl.Handle; + using Apache.Ignite.Core.Impl.Log; using Apache.Ignite.Core.Impl.Memory; using Apache.Ignite.Core.Impl.Unmanaged; using Apache.Ignite.Core.Lifecycle; + using Apache.Ignite.Core.Log; using Apache.Ignite.Core.Resource; using BinaryReader = Apache.Ignite.Core.Impl.Binary.BinaryReader; using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils; @@ -201,15 +203,21 @@ public static unsafe IIgnite Start(IgniteConfiguration cfg) lock (SyncRoot) { + // 0. Init logger + var log = cfg.Logger ?? new JavaLogger(); + + log.Debug("Starting Ignite.NET " + Assembly.GetExecutingAssembly().GetName().Version); + // 1. Check GC settings. - CheckServerGc(cfg); + CheckServerGc(cfg, log); // 2. Create context. - IgniteUtils.LoadDlls(cfg.JvmDllPath); + IgniteUtils.LoadDlls(cfg.JvmDllPath, log); - var cbs = new UnmanagedCallbacks(); + var cbs = new UnmanagedCallbacks(log); - IgniteManager.CreateJvmContext(cfg, cbs); + IgniteManager.CreateJvmContext(cfg, cbs, log); + log.Debug("JVM started."); var gridName = cfg.GridName; @@ -221,12 +229,17 @@ public static unsafe IIgnite Start(IgniteConfiguration cfg) try { // 4. Initiate Ignite start. - UU.IgnitionStart(cbs.Context, cfg.SpringConfigUrl, gridName, ClientMode); + UU.IgnitionStart(cbs.Context, cfg.SpringConfigUrl, gridName, ClientMode, cfg.Logger != null); + // 5. At this point start routine is finished. We expect STARTUP object to have all necessary data. var node = _startup.Ignite; interopProc = node.InteropProcessor; + var javaLogger = log as JavaLogger; + if (javaLogger != null) + javaLogger.SetProcessor(interopProc); + // 6. On-start callback (notify lifecycle components). node.OnStart(); @@ -271,10 +284,11 @@ public static unsafe IIgnite Start(IgniteConfiguration cfg) /// Check whether GC is set to server mode. /// /// Configuration. - private static void CheckServerGc(IgniteConfiguration cfg) + /// Log. + private static void CheckServerGc(IgniteConfiguration cfg, ILogger log) { if (!cfg.SuppressWarnings && !GCSettings.IsServerGC && Interlocked.CompareExchange(ref _gcWarn, 1, 0) == 0) - Logger.LogWarning("GC server mode is not enabled, this could lead to less " + + log.Warn("GC server mode is not enabled, this could lead to less " + "than optimal performance on multi-core machines (to enable see " + "http://msdn.microsoft.com/en-us/library/ms229357(v=vs.110).aspx)."); } @@ -285,14 +299,15 @@ private static void CheckServerGc(IgniteConfiguration cfg) /// Input stream with data. /// Output stream. /// Handle registry. - internal static void OnPrepare(PlatformMemoryStream inStream, PlatformMemoryStream outStream, - HandleRegistry handleRegistry) + /// Log. + internal static void OnPrepare(PlatformMemoryStream inStream, PlatformMemoryStream outStream, + HandleRegistry handleRegistry, ILogger log) { try { BinaryReader reader = BinaryUtils.Marshaller.StartUnmarshal(inStream); - PrepareConfiguration(reader, outStream); + PrepareConfiguration(reader, outStream, log); PrepareLifecycleBeans(reader, outStream, handleRegistry); @@ -313,7 +328,8 @@ private static void CheckServerGc(IgniteConfiguration cfg) /// /// Reader. /// Response stream. - private static void PrepareConfiguration(BinaryReader reader, PlatformMemoryStream outStream) + /// Log. + private static void PrepareConfiguration(BinaryReader reader, PlatformMemoryStream outStream, ILogger log) { // 1. Load assemblies. IgniteConfiguration cfg = _startup.Configuration; @@ -334,6 +350,7 @@ private static void PrepareConfiguration(BinaryReader reader, PlatformMemoryStre _startup.Marshaller = new Marshaller(cfg.BinaryConfiguration); // 3. Send configuration details to Java + cfg.Validate(log); cfg.Write(_startup.Marshaller.StartMarshal(outStream)); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs index f30264db320f3..a8d94f279df12 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs @@ -20,7 +20,7 @@ namespace Apache.Ignite.Core.Impl.Binary using System; using System.Collections.Generic; using System.Linq; - using Apache.Ignite.Core.Impl.Common; + using Apache.Ignite.Core.Log; /// /// Provides mapping between Java and .NET basic types. @@ -67,26 +67,33 @@ internal static class JavaTypes /// /// Gets the corresponding Java type name. - /// Logs a warning for indirectly mapped types. /// - public static string GetJavaTypeNameAndLogWarning(Type type) + public static string GetJavaTypeName(Type type) { if (type == null) return null; string res; - if (!NetToJava.TryGetValue(type, out res)) - return null; + return NetToJava.TryGetValue(type, out res) ? res : null; + } - Type directType; + /// + /// Logs a warning for indirectly mapped types. + /// + public static void LogIndirectMappingWarning(Type type, ILogger log, string logInfo) + { + if (type == null) + return; - if (IndirectMappingTypes.TryGetValue(type, out directType)) - Logger.LogWarning("Type '{0}' maps to Java type '{1}' using unchecked conversion. " + - "This may cause issues in SQL queries. " + - "You can use '{2}' instead to achieve direct mapping.", type, res, directType); + Type directType; + if (!IndirectMappingTypes.TryGetValue(type, out directType)) + return; - return res; + log.Warn("{0}: Type '{1}' maps to Java type '{2}' using unchecked conversion. " + + "This may cause issues in SQL queries. " + + "You can use '{3}' instead to achieve direct mapping.", + logInfo, type, NetToJava[type], directType); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Logger.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Logger.cs deleted file mode 100644 index cab5afc9d26d6..0000000000000 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Logger.cs +++ /dev/null @@ -1,37 +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. -*/ - -namespace Apache.Ignite.Core.Impl.Common -{ - using System; - - /// - /// Console logger. - /// - internal static class Logger - { - /// - /// Logs the warning. - /// - /// The warning. - /// The arguments. - public static void LogWarning(string warning, params object[] args) - { - Console.WriteLine("WARNING: " + string.Format(warning, args)); - } - } -} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs index 0fac417a101ad..2a3a0149696af 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs @@ -43,6 +43,7 @@ namespace Apache.Ignite.Core.Impl using Apache.Ignite.Core.Impl.Transactions; using Apache.Ignite.Core.Impl.Unmanaged; using Apache.Ignite.Core.Lifecycle; + using Apache.Ignite.Core.Log; using Apache.Ignite.Core.Messaging; using Apache.Ignite.Core.Services; using Apache.Ignite.Core.Transactions; @@ -407,6 +408,7 @@ internal void AfterNodeStop() NearCacheConfiguration nearConfiguration) { IgniteArgumentCheck.NotNull(configuration, "configuration"); + configuration.Validate(Logger); using (var stream = IgniteManager.Memory.Allocate().GetStream()) { @@ -445,6 +447,7 @@ internal void AfterNodeStop() NearCacheConfiguration nearConfiguration) { IgniteArgumentCheck.NotNull(configuration, "configuration"); + configuration.Validate(Logger); using (var stream = IgniteManager.Memory.Allocate().GetStream()) { @@ -673,6 +676,12 @@ public ICollection GetCacheNames() } } + /** */ + public ILogger Logger + { + get { return _cbs.Log; } + } + /** */ public event EventHandler Stopping; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs index 05bc7866c65b2..ee5421825c3cc 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs @@ -27,6 +27,7 @@ namespace Apache.Ignite.Core.Impl using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Impl.Memory; using Apache.Ignite.Core.Impl.Unmanaged; + using Apache.Ignite.Core.Log; using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils; /// @@ -57,8 +58,9 @@ public static unsafe class IgniteManager /// /// Configuration. /// Callbacks. + /// /// Context. - internal static void CreateJvmContext(IgniteConfiguration cfg, UnmanagedCallbacks cbs) + internal static void CreateJvmContext(IgniteConfiguration cfg, UnmanagedCallbacks cbs, ILogger log) { lock (SyncRoot) { @@ -69,7 +71,7 @@ internal static void CreateJvmContext(IgniteConfiguration cfg, UnmanagedCallback { if (!_jvmCfg.Equals(jvmCfg)) { - Logger.LogWarning("Attempting to start Ignite node with different Java " + + log.Warn("Attempting to start Ignite node with different Java " + "configuration; current Java configuration will be ignored (consider " + "starting node in separate process) [oldConfig=" + _jvmCfg + ", newConfig=" + jvmCfg + ']'); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs index 12de8476c3e08..914a87dedaa49 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs @@ -31,6 +31,7 @@ namespace Apache.Ignite.Core.Impl using Apache.Ignite.Core.Events; using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Cluster; + using Apache.Ignite.Core.Log; using Apache.Ignite.Core.Lifecycle; using Apache.Ignite.Core.Messaging; using Apache.Ignite.Core.Services; @@ -391,6 +392,12 @@ public ICollection GetCacheNames() return _ignite.GetCacheNames(); } + /** */ + public ILogger Logger + { + get { return _ignite.Logger; } + } + /** */ public event EventHandler Stopping { diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs index 048c8ac5c5844..70d483da0e21a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs @@ -35,6 +35,7 @@ namespace Apache.Ignite.Core.Impl using Apache.Ignite.Core.Impl.Cluster; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Impl.Unmanaged; + using Apache.Ignite.Core.Log; using Microsoft.Win32; using BinaryReader = Apache.Ignite.Core.Impl.Binary.BinaryReader; @@ -124,12 +125,17 @@ public static IList Shuffle(IList list) /// Load JVM DLL if needed. /// /// JVM DLL path from config. - public static void LoadDlls(string configJvmDllPath) + /// Log. + public static void LoadDlls(string configJvmDllPath, ILogger log) { - if (_loaded) return; + if (_loaded) + { + log.Debug("JNI dll is already loaded."); + return; + } // 1. Load JNI dll. - LoadJvmDll(configJvmDllPath); + LoadJvmDll(configJvmDllPath, log); // 2. Load GG JNI dll. UnmanagedUtils.Initialize(); @@ -190,17 +196,25 @@ private static void SetProperties(object target, IEnumerable /// Loads the JVM DLL. /// - private static void LoadJvmDll(string configJvmDllPath) + private static void LoadJvmDll(string configJvmDllPath, ILogger log) { var messages = new List(); foreach (var dllPath in GetJvmDllPaths(configJvmDllPath)) { + log.Debug("Trying to load JVM dll from [option={0}, path={1}]...", dllPath.Key, dllPath.Value); + var errCode = LoadDll(dllPath.Value, FileJvmDll); if (errCode == 0) + { + log.Debug("jvm.dll successfully loaded from [option={0}, path={1}]", dllPath.Key, dllPath.Value); return; + } + + var message = string.Format(CultureInfo.InvariantCulture, "[option={0}, path={1}, error={2}]", + dllPath.Key, dllPath.Value, FormatWin32Error(errCode)); + messages.Add(message); - messages.Add(string.Format(CultureInfo.InvariantCulture, "[option={0}, path={1}, error={2}]", - dllPath.Key, dllPath.Value, FormatWin32Error(errCode))); + log.Debug("Failed to load jvm.dll: " + message); if (dllPath.Value == configJvmDllPath) break; // if configJvmDllPath is specified and is invalid - do not try other options diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Log/JavaLogger.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Log/JavaLogger.cs new file mode 100644 index 0000000000000..23e7a3733ca6c --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Log/JavaLogger.cs @@ -0,0 +1,110 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Impl.Log +{ + using System; + using System.Collections.Generic; + using System.Diagnostics; + using System.Linq; + using Apache.Ignite.Core.Impl.Unmanaged; + using Apache.Ignite.Core.Log; + + /// + /// Logger that delegates to Java. + /// + internal class JavaLogger : ILogger + { + /** */ + private IUnmanagedTarget _proc; + + /** */ + private readonly List _enabledLevels = new List(5); + + /** */ + private readonly List> _pendingLogs + = new List>(); + + /** */ + private readonly object _syncRoot = new object(); + + /// + /// Sets the processor. + /// + /// The proc. + public void SetProcessor(IUnmanagedTarget proc) + { + Debug.Assert(proc != null); + + lock (_syncRoot) + { + _proc = proc; + + // Preload enabled levels. + _enabledLevels.AddRange( + new[] { LogLevel.Trace, LogLevel.Debug, LogLevel.Info, LogLevel.Warn, LogLevel.Error } + .Where(x => UnmanagedUtils.ProcessorLoggerIsLevelEnabled(proc, (int)x))); + + foreach (var log in _pendingLogs) + { + Log(log.Item1, log.Item2, log.Item3, log.Item4); + } + } + } + + /** */ + public void Log(LogLevel level, string message, object[] args, IFormatProvider formatProvider, string category, + string nativeErrorInfo, Exception ex) + { + // Java error info should not go back to Java. + // Either we log in .NET, and Java sends us logs, or we log in Java, and .NET sends logs, not both. + Debug.Assert(nativeErrorInfo == null); + + lock (_syncRoot) + { + if (!IsEnabled(level)) + return; + + var msg = args == null ? message : string.Format(formatProvider, message, args); + var err = ex != null ? ex.ToString() : null; + + if (_proc != null) + Log(level, msg, category, err); + else + _pendingLogs.Add(Tuple.Create(level, msg, category, err)); + } + } + + /** */ + public bool IsEnabled(LogLevel level) + { + lock (_syncRoot) + { + return _proc == null || _enabledLevels.Contains(level); + } + } + + /// + /// Logs the message. + /// + private void Log(LogLevel level, string msg, string category, string err) + { + if (IsEnabled(level)) + UnmanagedUtils.ProcessorLoggerLog(_proc, (int)level, msg, category, err); + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs index 2da41928a5469..8756dec84d7e8 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs @@ -113,6 +113,13 @@ internal static unsafe class IgniteJniNativeMethods [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorGetCacheNames")] public static extern void ProcessorGetCacheNames(void* ctx, void* obj, long memPtr); + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorLoggerIsLevelEnabled")] + [return: MarshalAs(UnmanagedType.U1)] + public static extern bool ProcessorLoggerIsLevelEnabled(void* ctx, void* obj, int level); + + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorLoggerLog")] + public static extern void ProcessorLoggerLog(void* ctx, void* obj, int level, sbyte* messsage, sbyte* category, sbyte* errorInfo); + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInStreamOutLong")] public static extern long TargetInStreamOutLong(void* ctx, void* target, int opType, long memPtr); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs index 51d9c74555e7f..6367e1eef45c0 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackHandlers.cs @@ -104,5 +104,8 @@ internal unsafe struct UnmanagedCallbackHandlers internal void* affinityFunctionAssignPartitions; internal void* affinityFunctionRemoveNode; internal void* affinityFunctionDestroy; + + internal void* loggerLog; + internal void* loggerIsLevelEnabled; } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs index 223eb5b287012..493e0615b562e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs @@ -22,6 +22,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged using System.Diagnostics; using System.Diagnostics.CodeAnalysis; using System.IO; + using System.Globalization; using System.Runtime.InteropServices; using System.Threading; using Apache.Ignite.Core.Cache.Affinity; @@ -38,11 +39,13 @@ namespace Apache.Ignite.Core.Impl.Unmanaged using Apache.Ignite.Core.Impl.Datastream; using Apache.Ignite.Core.Impl.Events; using Apache.Ignite.Core.Impl.Handle; + using Apache.Ignite.Core.Impl.Log; using Apache.Ignite.Core.Impl.Memory; using Apache.Ignite.Core.Impl.Messaging; using Apache.Ignite.Core.Impl.Resource; using Apache.Ignite.Core.Impl.Services; using Apache.Ignite.Core.Lifecycle; + using Apache.Ignite.Core.Log; using Apache.Ignite.Core.Services; using UU = UnmanagedUtils; @@ -89,6 +92,9 @@ internal unsafe class UnmanagedCallbacks [SuppressMessage("Microsoft.Reliability", "CA2006:UseSafeHandleToEncapsulateNativeResources")] private readonly IntPtr _cbsPtr; + /** Log. */ + private readonly ILogger _log; + /** Error type: generic. */ private const int ErrGeneric = 1; @@ -98,7 +104,7 @@ internal unsafe class UnmanagedCallbacks /** Error type: attach. */ private const int ErrJvmAttach = 3; - /** Opeartion: prepare .Net. */ + /** Operation: prepare .Net. */ private const int OpPrepareDotNet = 1; private delegate long CacheStoreCreateCallbackDelegate(void* target, long memPtr); @@ -174,6 +180,9 @@ internal unsafe class UnmanagedCallbacks private delegate void OnClientDisconnectedDelegate(void* target); private delegate void OnClientReconnectedDelegate(void* target, bool clusterRestarted); + private delegate void LoggerLogDelegate(void* target, int level, sbyte* messageChars, int messageCharsLen, sbyte* categoryChars, int categoryCharsLen, sbyte* errorInfoChars, int errorInfoCharsLen, long memPtr); + private delegate bool LoggerIsLevelEnabledDelegate(void* target, int level); + private delegate long AffinityFunctionInitDelegate(void* target, long memPtr, void* baseFunc); private delegate int AffinityFunctionPartitionDelegate(void* target, long ptr, long memPtr); private delegate void AffinityFunctionAssignPartitionsDelegate(void* target, long ptr, long inMemPtr, long outMemPtr); @@ -183,10 +192,14 @@ internal unsafe class UnmanagedCallbacks private delegate void ConsoleWriteDelegate(sbyte* chars, int charsLen, bool isErr); /// - /// constructor. + /// Constructor. /// - public UnmanagedCallbacks() + /// Logger. + public UnmanagedCallbacks(ILogger log) { + Debug.Assert(log != null); + _log = log; + var cbs = new UnmanagedCallbackHandlers { target = IntPtr.Zero.ToPointer(), // Target is not used in .Net as we rely on dynamic FP creation. @@ -270,7 +283,10 @@ public UnmanagedCallbacks() affinityFunctionPartition = CreateFunctionPointer((AffinityFunctionPartitionDelegate)AffinityFunctionPartition), affinityFunctionAssignPartitions = CreateFunctionPointer((AffinityFunctionAssignPartitionsDelegate)AffinityFunctionAssignPartitions), affinityFunctionRemoveNode = CreateFunctionPointer((AffinityFunctionRemoveNodeDelegate)AffinityFunctionRemoveNode), - affinityFunctionDestroy = CreateFunctionPointer((AffinityFunctionDestroyDelegate)AffinityFunctionDestroy) + affinityFunctionDestroy = CreateFunctionPointer((AffinityFunctionDestroyDelegate)AffinityFunctionDestroy), + + loggerLog = CreateFunctionPointer((LoggerLogDelegate)LoggerLog), + loggerIsLevelEnabled = CreateFunctionPointer((LoggerIsLevelEnabledDelegate)LoggerIsLevelEnabled) }; _cbsPtr = Marshal.AllocHGlobal(UU.HandlersSize()); @@ -550,7 +566,7 @@ private IComputeTaskHolder Task(long taskPtr) } /// - /// Get comptue job using it's GC handle pointer. + /// Get compute job using it's GC handle pointer. /// /// Job pointer. /// Compute job. @@ -883,7 +899,7 @@ private long ExtensionCallbackInLongLongOutLong(void* target, int op, long arg1, using (var inStream = IgniteManager.Memory.Get(arg1).GetStream()) using (var outStream = IgniteManager.Memory.Get(arg2).GetStream()) { - Ignition.OnPrepare(inStream, outStream, _handleRegistry); + Ignition.OnPrepare(inStream, outStream, _handleRegistry, _log); return 0; } @@ -1131,6 +1147,40 @@ private void OnClientReconnected(void* target, bool clusterRestarted) }); } + private void LoggerLog(void* target, int level, sbyte* messageChars, int messageCharsLen, sbyte* categoryChars, + int categoryCharsLen, sbyte* errorInfoChars, int errorInfoCharsLen, long memPtr) + { + // When custom logger in .NET is not defined, Java should not call us. + Debug.Assert(!(_log is JavaLogger)); + + SafeCall(() => + { + var message = IgniteUtils.Utf8UnmanagedToString(messageChars, messageCharsLen); + var category = IgniteUtils.Utf8UnmanagedToString(categoryChars, categoryCharsLen); + var nativeError = IgniteUtils.Utf8UnmanagedToString(errorInfoChars, errorInfoCharsLen); + + Exception ex = null; + + if (memPtr != 0 && _ignite != null) + { + using (var stream = IgniteManager.Memory.Get(memPtr).GetStream()) + { + ex = _ignite.Marshaller.Unmarshal(stream); + } + } + + _log.Log((LogLevel) level, message, null, CultureInfo.InvariantCulture, category, nativeError, ex); + }, true); + } + + private bool LoggerIsLevelEnabled(void* target, int level) + { + // When custom logger in .NET is not defined, Java should not call us. + Debug.Assert(!(_log is JavaLogger)); + + return SafeCall(() => _log.IsEnabled((LogLevel) level), true); + } + private static void ConsoleWrite(sbyte* chars, int charsLen, bool isErr) { try @@ -1246,6 +1296,8 @@ private void SafeCall(Action func, bool allowUnitialized = false) } catch (Exception e) { + _log.Error(e, "Failure in Java callback"); + UU.ThrowToJava(_ctx.NativeContext, e); } } @@ -1262,6 +1314,8 @@ private T SafeCall(Func func, bool allowUnitialized = false) } catch (Exception e) { + _log.Error(e, "Failure in Java callback"); + UU.ThrowToJava(_ctx.NativeContext, e); return default(T); @@ -1286,6 +1340,14 @@ public UnmanagedContext Context get { return _ctx; } } + /// + /// Gets the log. + /// + public ILogger Log + { + get { return _log; } + } + /// /// Create function pointer for the given function. /// @@ -1323,6 +1385,8 @@ public void Initialize(Ignite grid) } _initEvent.Set(); + + ResourceProcessor.Inject(_log, grid); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs index 0e9556d0911bb..f89caa87ff9bd 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs @@ -77,11 +77,12 @@ internal static void Initialize() #region NATIVE METHODS: PROCESSOR internal static void IgnitionStart(UnmanagedContext ctx, string cfgPath, string gridName, - bool clientMode) + bool clientMode, bool userLogger) { using (var mem = IgniteManager.Memory.Allocate().GetStream()) { mem.WriteBool(clientMode); + mem.WriteBool(userLogger); sbyte* cfgPath0 = IgniteUtils.StringToUtf8Unmanaged(cfgPath); sbyte* gridName0 = IgniteUtils.StringToUtf8Unmanaged(gridName); @@ -377,6 +378,30 @@ internal static void ProcessorGetCacheNames(IUnmanagedTarget target, long memPtr JNI.ProcessorGetCacheNames(target.Context, target.Target, memPtr); } + internal static bool ProcessorLoggerIsLevelEnabled(IUnmanagedTarget target, int level) + { + return JNI.ProcessorLoggerIsLevelEnabled(target.Context, target.Target, level); + } + + internal static void ProcessorLoggerLog(IUnmanagedTarget target, int level, string message, string category, + string errorInfo) + { + var message0 = IgniteUtils.StringToUtf8Unmanaged(message); + var category0 = IgniteUtils.StringToUtf8Unmanaged(category); + var errorInfo0 = IgniteUtils.StringToUtf8Unmanaged(errorInfo); + + try + { + JNI.ProcessorLoggerLog(target.Context, target.Target, level, message0, category0, errorInfo0); + } + finally + { + Marshal.FreeHGlobal(new IntPtr(message0)); + Marshal.FreeHGlobal(new IntPtr(category0)); + Marshal.FreeHGlobal(new IntPtr(errorInfo0)); + } + } + #endregion #region NATIVE METHODS: TARGET diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Log/CategoryLogger.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Log/CategoryLogger.cs new file mode 100644 index 0000000000000..2d7f8768f1691 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Log/CategoryLogger.cs @@ -0,0 +1,82 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Log +{ + using System; + using Apache.Ignite.Core.Impl.Common; + + /// + /// Wrapping logger with a predefined category. + /// + /// When method is called, and category parameter is null, predefined category + /// will be used. + /// + public class CategoryLogger : ILogger + { + /** Wrapped logger. */ + private readonly ILogger _logger; + + /** Category to use. */ + private readonly string _category; + + /// + /// Initializes a new instance of the class. + /// + /// The logger to wrap. + /// The category. + public CategoryLogger(ILogger logger, string category) + { + IgniteArgumentCheck.NotNull(logger, "log"); + + // If logger is already a CategoryLogger, get underlying logger instead to avoid unnecessary nesting. + var catLogger = logger as CategoryLogger; + _logger = catLogger != null ? catLogger._logger : logger; + + _category = category; + } + + /// + /// Logs the specified message. + /// + /// The level. + /// The message. + /// The arguments to format . + /// Can be null (formatting will not occur). + /// The format provider. Can be null if is null. + /// The logging category name. + /// The native error information. + /// The exception. Can be null. + public void Log(LogLevel level, string message, object[] args, IFormatProvider formatProvider, string category, + string nativeErrorInfo, Exception ex) + { + _logger.Log(level, message, args, formatProvider, category ?? _category, nativeErrorInfo, ex); + } + + /// + /// Determines whether the specified log level is enabled. + /// + /// The level. + /// + /// Value indicating whether the specified log level is enabled + /// + public bool IsEnabled(LogLevel level) + { + return _logger.IsEnabled(level); + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Log/ILogger.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Log/ILogger.cs new file mode 100644 index 0000000000000..a2f2f20113f22 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Log/ILogger.cs @@ -0,0 +1,51 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Log +{ + using System; + + /// + /// Defines Ignite logging interface. + /// + /// This interface only provides essential log methods. + /// All convenience overloads are in . + /// + public interface ILogger + { + /// + /// Logs the specified message. + /// + /// The level. + /// The message. + /// The arguments to format . + /// Can be null (formatting will not occur). + /// The format provider. Can be null if is null. + /// The logging category name. + /// The native error information. + /// The exception. Can be null. + void Log(LogLevel level, string message, object[] args, IFormatProvider formatProvider, string category, + string nativeErrorInfo, Exception ex); + + /// + /// Determines whether the specified log level is enabled. + /// + /// The level. + /// Value indicating whether the specified log level is enabled + bool IsEnabled(LogLevel level); + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Log/LogLevel.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Log/LogLevel.cs new file mode 100644 index 0000000000000..75694ab1a3af6 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Log/LogLevel.cs @@ -0,0 +1,53 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Log +{ + using System; + + /// + /// Defines log levels. + /// + [Serializable] + public enum LogLevel + { + /// + /// Trace log level. + /// + Trace = 0, + + /// + /// Debug log level. + /// + Debug = 1, + + /// + /// Info log level. + /// + Info = 2, + + /// + /// Warning log level. + /// + Warn = 3, + + /// + /// Error log level. + /// + Error = 4 + } +} \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Log/LoggerExtensions.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Log/LoggerExtensions.cs new file mode 100644 index 0000000000000..93748e3549492 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Log/LoggerExtensions.cs @@ -0,0 +1,320 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Log +{ + using System; + using System.Globalization; + using Apache.Ignite.Core.Impl.Common; + + /// + /// Extension methods for + /// + public static class LoggerExtensions + { + // 4 overloads per level (message, message+args, ex+message, ex+message+args) + + /// + /// Logs the message with level. + /// + /// The logger. + /// The message. + public static void Trace(this ILogger logger, string message) + { + Log(logger, LogLevel.Trace, message); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The message. + /// The arguments. + public static void Trace(this ILogger logger, string message, params object[] args) + { + Log(logger, LogLevel.Trace, message, args); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The exception. + /// The message. + public static void Trace(this ILogger logger, Exception ex, string message) + { + Log(logger, LogLevel.Trace, ex, message); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The exception. + /// The message. + /// The arguments. + public static void Trace(this ILogger logger, Exception ex, string message, params object[] args) + { + Log(logger, LogLevel.Trace, ex, message, args); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The message. + public static void Debug(this ILogger logger, string message) + { + Log(logger, LogLevel.Debug, message); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The message. + /// The arguments. + public static void Debug(this ILogger logger, string message, params object[] args) + { + Log(logger, LogLevel.Debug, message, args); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The exception. + /// The message. + public static void Debug(this ILogger logger, Exception ex, string message) + { + Log(logger, LogLevel.Debug, ex, message); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The exception. + /// The message. + /// The arguments. + public static void Debug(this ILogger logger, Exception ex, string message, params object[] args) + { + Log(logger, LogLevel.Debug, ex, message, args); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The message. + public static void Info(this ILogger logger, string message) + { + Log(logger, LogLevel.Info, message); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The message. + /// The arguments. + public static void Info(this ILogger logger, string message, params object[] args) + { + Log(logger, LogLevel.Info, message, args); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The exception. + /// The message. + public static void Info(this ILogger logger, Exception ex, string message) + { + Log(logger, LogLevel.Info, ex, message); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The exception. + /// The message. + /// The arguments. + public static void Info(this ILogger logger, Exception ex, string message, params object[] args) + { + Log(logger, LogLevel.Info, ex, message, args); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The message. + public static void Warn(this ILogger logger, string message) + { + Log(logger, LogLevel.Warn, message); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The message. + /// The arguments. + public static void Warn(this ILogger logger, string message, params object[] args) + { + Log(logger, LogLevel.Warn, message, args); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The exception. + /// The message. + public static void Warn(this ILogger logger, Exception ex, string message) + { + Log(logger, LogLevel.Warn, ex, message); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The exception. + /// The message. + /// The arguments. + public static void Warn(this ILogger logger, Exception ex, string message, params object[] args) + { + Log(logger, LogLevel.Warn, ex, message, args); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The message. + public static void Error(this ILogger logger, string message) + { + Log(logger, LogLevel.Error, message); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The message. + /// The arguments. + public static void Error(this ILogger logger, string message, params object[] args) + { + Log(logger, LogLevel.Error, message, args); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The exception. + /// The message. + public static void Error(this ILogger logger, Exception ex, string message) + { + Log(logger, LogLevel.Error, ex, message); + } + + /// + /// Logs the message with level. + /// + /// The logger. + /// The exception. + /// The message. + /// The arguments. + public static void Error(this ILogger logger, Exception ex, string message, params object[] args) + { + Log(logger, LogLevel.Error, ex, message, args); + } + + /// + /// Logs the message. + /// + /// The logger. + /// The level. + /// The message. + public static void Log(this ILogger logger, LogLevel level, string message) + { + IgniteArgumentCheck.NotNull(logger, "logger"); + + logger.Log(level, message, null, null, null, null, null); + } + + /// + /// Logs the message. + /// + /// The logger. + /// The level. + /// The message. + /// The arguments. + public static void Log(this ILogger logger, LogLevel level, string message, params object[] args) + { + IgniteArgumentCheck.NotNull(logger, "logger"); + + logger.Log(level, message, args, CultureInfo.InvariantCulture, null, null, null); + } + + /// + /// Logs the message. + /// + /// The logger. + /// The level. + /// The exception. + /// The message. + public static void Log(this ILogger logger, LogLevel level, Exception ex, string message) + { + IgniteArgumentCheck.NotNull(logger, "logger"); + + logger.Log(level, message, null, null, null, null, ex); + } + + /// + /// Logs the message. + /// + /// The logger. + /// The level. + /// The exception. + /// The message. + /// The arguments. + public static void Log(this ILogger logger, LogLevel level, Exception ex, string message, params object[] args) + { + IgniteArgumentCheck.NotNull(logger, "logger"); + + logger.Log(level, message, args, CultureInfo.InvariantCulture, null, null, ex); + } + + /// + /// Gets the with a specified category that wraps provided logger. + /// + /// The logger. + /// The category. + /// Logger that always uses specified category. + public static ILogger GetLogger(this ILogger logger, string category) + { + IgniteArgumentCheck.NotNull(logger, "logger"); + + return new CategoryLogger(logger, category); + } + } +} \ No newline at end of file From ae4ae71f0481662a5fb13eb8ee63a17e6c4c3802 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 3 Oct 2016 12:16:34 +0300 Subject: [PATCH 217/487] IGNITE-1629 .NET: Introduced native logging facility - fix merge --- .../platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs index 8e16fb523f73d..765209230deee 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs @@ -24,6 +24,8 @@ namespace Apache.Ignite.Core using System.Diagnostics.CodeAnalysis; using System.IO; using System.Linq; + using System.Text; + using System.Xml; using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cache; using Apache.Ignite.Core.Cache.Configuration; From 9f211e413332931f0fb1190744ddd7c7f38fd213 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Mon, 3 Oct 2016 12:26:12 +0300 Subject: [PATCH 218/487] IGNITE-3645: IGFS: Local secondary: Implemented update() operation. This closes #1003. --- .../local/LocalIgfsSecondaryFileSystem.java | 90 ++- .../internal/processors/igfs/IgfsImpl.java | 11 +- .../processors/igfs/IgfsMetaManager.java | 1 - .../local/LocalFileSystemIgfsFile.java | 9 +- .../secondary/local/LocalFileSystemUtils.java | 142 +++++ .../igfs/IgfsAbstractBaseSelfTest.java | 19 + .../processors/igfs/IgfsAbstractSelfTest.java | 5 +- .../igfs/IgfsDualAbstractSelfTest.java | 42 +- ...condaryFileSystemDualAbstractSelfTest.java | 26 +- ...LocalSecondaryFileSystemProxySelfTest.java | 5 - ...fsLocalSecondaryFileSystemTestAdapter.java | 27 +- .../igfs/benchmark/IgfsBenchmark.java | 561 ++++++++++++++++++ ...opIgfsSecondaryFileSystemDelegateImpl.java | 3 +- .../impl/igfs/HadoopIgfsProperties.java | 3 + 14 files changed, 900 insertions(+), 44 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemUtils.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/benchmark/IgfsBenchmark.java diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java index 1775db6e430d9..ef00beada796c 100644 --- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java +++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java @@ -26,8 +26,10 @@ import org.apache.ignite.igfs.IgfsPathNotFoundException; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable; +import org.apache.ignite.internal.processors.igfs.IgfsUtils; import org.apache.ignite.internal.processors.igfs.secondary.local.LocalFileSystemIgfsFile; import org.apache.ignite.internal.processors.igfs.secondary.local.LocalFileSystemSizeVisitor; +import org.apache.ignite.internal.processors.igfs.secondary.local.LocalFileSystemUtils; import org.apache.ignite.internal.processors.igfs.secondary.local.LocalIgfsSecondaryFileSystemPositionedReadable; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; @@ -44,6 +46,7 @@ import java.nio.file.LinkOption; import java.nio.file.Path; import java.nio.file.attribute.BasicFileAttributes; +import java.nio.file.attribute.PosixFileAttributes; import java.util.Collection; import java.util.Collections; import java.util.Map; @@ -76,7 +79,14 @@ private IgfsException handleSecondaryFsError(IOException e, String msg) { /** {@inheritDoc} */ @Nullable @Override public IgfsFile update(IgfsPath path, Map props) { - throw new UnsupportedOperationException("Update operation is not yet supported."); + File f = fileForPath(path); + + if (!f.exists()) + return null; + + updatePropertiesIfNeeded(path, props); + + return info(path); } /** {@inheritDoc} */ @@ -157,6 +167,8 @@ private boolean deleteRecursive(File f, boolean deleteIfExists) { /** {@inheritDoc} */ @Override public void mkdirs(IgfsPath path, @Nullable Map props) { mkdirs(path); + + updatePropertiesIfNeeded(path, props); } /** @@ -258,7 +270,23 @@ private boolean mkdirs0(@Nullable File dir) { /** {@inheritDoc} */ @Override public OutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication, long blockSize, @Nullable Map props) { - return create0(path, overwrite); + OutputStream os = create0(path, overwrite); + + try { + updatePropertiesIfNeeded(path, props); + + return os; + } + catch (Exception err) { + try { + os.close(); + } + catch (IOException closeErr) { + err.addSuppressed(closeErr); + } + + throw err; + } } /** {@inheritDoc} */ @@ -269,11 +297,30 @@ private boolean mkdirs0(@Nullable File dir) { boolean exists = file.exists(); - if (exists) - return new FileOutputStream(file, true); + if (exists) { + OutputStream os = new FileOutputStream(file, true); + + try { + updatePropertiesIfNeeded(path, props); + + return os; + } + catch (Exception err) { + try { + os.close(); + + throw err; + } + catch (IOException closeErr) { + err.addSuppressed(closeErr); + + throw err; + } + } + } else { if (create) - return create0(path, false); + return create(path, bufSize, false, 0, 0, props); else throw new IgfsPathNotFoundException("Failed to append to file because it doesn't exist: " + path); } @@ -285,17 +332,21 @@ private boolean mkdirs0(@Nullable File dir) { /** {@inheritDoc} */ @Override public IgfsFile info(final IgfsPath path) { - File f = fileForPath(path); + File file = fileForPath(path); - if (!f.exists()) + if (!file.exists()) return null; - boolean isDir = f.isDirectory(); + boolean isDir = file.isDirectory(); + + PosixFileAttributes attrs = LocalFileSystemUtils.posixAttributes(file); + + Map props = LocalFileSystemUtils.posixAttributesToMap(attrs); if (isDir) - return new LocalFileSystemIgfsFile(path, false, true, 0, f.lastModified(), 0, null); + return new LocalFileSystemIgfsFile(path, false, true, 0, file.lastModified(), 0, props); else - return new LocalFileSystemIgfsFile(path, f.isFile(), false, 0, f.lastModified(), f.length(), null); + return new LocalFileSystemIgfsFile(path, file.isFile(), false, 0, file.lastModified(), file.length(), props); } /** {@inheritDoc} */ @@ -412,4 +463,23 @@ private OutputStream create0(IgfsPath path, boolean overwrite) { throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", overwrite=" + overwrite + ']'); } } + + /** + * Update path properties if needed. + * + * @param path IGFS path + * @param props Properties map. + */ + private void updatePropertiesIfNeeded(IgfsPath path, Map props) { + if (props == null || props.isEmpty()) + return; + + File file = fileForPath(path); + + if (!file.exists()) + throw new IgfsPathNotFoundException("Failed to update properties for path: " + path); + + LocalFileSystemUtils.updateProperties(file, props.get(IgfsUtils.PROP_GROUP_NAME), + props.get(IgfsUtils.PROP_PERMISSION)); + } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 1dd12d913a590..1c985c0d6ed5c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -152,9 +152,6 @@ public final class IgfsImpl implements IgfsEx { /** Writers map. */ private final ConcurrentHashMap8 workerMap = new ConcurrentHashMap8<>(); - /** Local metrics holder. */ - private final IgfsLocalMetrics metrics = new IgfsLocalMetrics(); - /** Client log directory. */ private volatile String logDir; @@ -765,13 +762,11 @@ else if (val) if (log.isDebugEnabled()) log.debug("Make directories: " + path); - final Map props0 = props == null ? DFLT_DIR_META : new HashMap<>(props); - IgfsMode mode = resolveMode(path); switch (mode) { case PRIMARY: - meta.mkdirs(path, props0); + meta.mkdirs(path, props == null ? DFLT_DIR_META : new HashMap<>(props)); break; @@ -779,12 +774,12 @@ else if (val) case DUAL_SYNC: await(path); - meta.mkdirsDual(secondaryFs, path, props0); + meta.mkdirsDual(secondaryFs, path, props); break; case PROXY: - secondaryFs.mkdirs(path, props0); + secondaryFs.mkdirs(path, props); break; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java index 89cadced7ed08..ffa502b0b75ea 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java @@ -2226,7 +2226,6 @@ public boolean mkdirsDual(final IgfsSecondaryFileSystem fs, final IgfsPath path, try { assert fs != null; assert path != null; - assert props != null; if (path.parent() == null) return true; // No additional handling for root directory is needed. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemIgfsFile.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemIgfsFile.java index 5abe4eb49b154..400ac34e393ae 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemIgfsFile.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemIgfsFile.java @@ -17,14 +17,13 @@ package org.apache.ignite.internal.processors.igfs.secondary.local; +import java.util.Collections; +import java.util.Map; import org.apache.ignite.igfs.IgfsFile; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.processors.igfs.IgfsUtils; import org.jetbrains.annotations.Nullable; -import java.util.Collections; -import java.util.Map; - /** * Implementation of the IgfsFile interface for the local filesystem. */ @@ -45,7 +44,7 @@ public class LocalFileSystemIgfsFile implements IgfsFile { private final long len; /** Properties. */ - private final Map props; + private Map props; /** * @param path IGFS path. @@ -63,7 +62,7 @@ public LocalFileSystemIgfsFile(IgfsPath path, boolean isFile, boolean isDir, int assert !isDir || len == 0 : "length must be 0 for dirs. [length=" + len + ']'; this.path = path; - this.flags = IgfsUtils.flags(isDir, isFile); + flags = IgfsUtils.flags(isDir, isFile); this.blockSize = blockSize; this.modTime = modTime; this.len = len; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemUtils.java new file mode 100644 index 0000000000000..59383c5a42576 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemUtils.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.igfs.secondary.local; + +import org.apache.ignite.igfs.IgfsException; +import org.apache.ignite.internal.processors.igfs.IgfsUtils; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import java.io.File; +import java.io.IOException; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.nio.file.attribute.GroupPrincipal; +import java.nio.file.attribute.PosixFileAttributeView; +import java.nio.file.attribute.PosixFileAttributes; +import java.nio.file.attribute.PosixFilePermission; +import java.nio.file.attribute.UserPrincipalLookupService; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** + * Various utility methods for local file system. + */ +public class LocalFileSystemUtils { + /** Posix file permissions. */ + public static final PosixFilePermission[] POSIX_PERMISSIONS = PosixFilePermission.values(); + + /** + * Update file properties. + * + * @param file File. + * @param grp Group. + * @param perm Permissions. + */ + public static void updateProperties(File file, String grp, String perm) { + PosixFileAttributeView attrs = Files.getFileAttributeView(file.toPath(), PosixFileAttributeView.class); + + if (attrs == null) + throw new UnsupportedOperationException("Posix file attributes not available"); + + if (grp != null) { + try { + UserPrincipalLookupService lookupService = FileSystems.getDefault().getUserPrincipalLookupService(); + + GroupPrincipal grp0 = lookupService.lookupPrincipalByGroupName(grp); + + attrs.setGroup(grp0); + } + catch (IOException e) { + throw new IgfsException("Update the '" + IgfsUtils.PROP_GROUP_NAME + "' property is failed.", e); + } + } + + if (perm != null) { + int perm0 = Integer.parseInt(perm, 8); + + Set permSet = new HashSet<>(9); + + for (int i = 0; i < LocalFileSystemUtils.POSIX_PERMISSIONS.length; ++i) { + if ((perm0 & (1 << i)) != 0) + permSet.add(LocalFileSystemUtils.POSIX_PERMISSIONS[i]); + } + + try { + attrs.setPermissions(permSet); + } + catch (IOException e) { + throw new IgfsException("Update the '" + IgfsUtils.PROP_PERMISSION + "' property is failed.", e); + } + } + } + + /** + * Get POSIX attributes for file. + * + * @param file File. + */ + @Nullable public static PosixFileAttributes posixAttributes(File file) { + PosixFileAttributes attrs = null; + + try { + PosixFileAttributeView view = Files.getFileAttributeView(file.toPath(), PosixFileAttributeView.class); + + if (view != null) + attrs = view.readAttributes(); + } + catch (IOException e) { + throw new IgfsException("Failed to read POSIX attributes: " + file.getAbsolutePath(), e); + } + + return attrs; + } + + /** + * Convert POSIX attributes to property map. + * + * @param attrs Attributes view. + * @return IGFS properties map. + */ + public static Map posixAttributesToMap(PosixFileAttributes attrs) { + if (attrs == null) + return null; + + Map props = U.newHashMap(3); + + props.put(IgfsUtils.PROP_USER_NAME, attrs.owner().getName()); + props.put(IgfsUtils.PROP_GROUP_NAME, attrs.group().getName()); + + int perm = 0; + + for(PosixFilePermission p : attrs.permissions()) + perm |= (1 << 8 - p.ordinal()); + + props.put(IgfsUtils.PROP_PERMISSION, '0' + Integer.toOctalString(perm)); + + return props; + } + + /** + * Private constructor. + */ + private LocalFileSystemUtils() { + // No-op. + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java index 79dc57b630ee9..374d3d39f2f98 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java @@ -868,6 +868,25 @@ protected void checkFileContent(IgfsSecondaryFileSystemTestAdapter uni, String p } } + /** + * Create map with properties. + * + * @param grpName Group name. + * @param perm Permission. + * @return Map with properties. + */ + protected Map properties(@Nullable String grpName, @Nullable String perm) { + Map props = new HashMap<>(); + + if (grpName != null) + props.put(IgfsUtils.PROP_GROUP_NAME, grpName); + + if (perm != null) + props.put(IgfsUtils.PROP_PERMISSION, perm); + + return props; + } + /** * Create map with properties. * diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java index 7058954f01770..128239dacc793 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java @@ -532,7 +532,10 @@ public void testMkdirs() throws Exception { */ @SuppressWarnings("ConstantConditions") public void testMkdirsParentRoot() throws Exception { - Map props = properties(null, null, "0555"); // mkdirs command doesn't propagate user info. + Map props = null; + + if (permissionsSupported()) + props = properties(null, null, "0555"); // mkdirs command doesn't propagate user info. igfs.mkdirs(DIR, props); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java index 742d20c76ec7d..1d6010dfd83d1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java @@ -970,7 +970,10 @@ public void testMoveRenameDirectorySourceParentRootDestinationMissing() throws E * @throws Exception If failed. */ public void testMkdirsParentPathMissingPartially() throws Exception { - Map props = properties(null, null, "0555"); // mkdirs command doesn't propagate user info. + Map props = null; + + if (permissionsSupported()) + props = properties(null, null, "0555"); // mkdirs command doesn't propagate user info. create(igfsSecondary, paths(DIR, SUBDIR), null); create(igfs, paths(DIR), null); @@ -997,7 +1000,10 @@ public void testMkdirsParentPathMissingPartially() throws Exception { * @throws Exception If failed. */ public void testMkdrisParentPathMissing() throws Exception { - Map props = properties(null, null, "0555"); // mkdirs command doesn't propagate user info. + Map props = null; + + if (permissionsSupported()) + props = properties(null, null, "0555"); // mkdirs command doesn't propagate user info. create(igfsSecondary, paths(DIR, SUBDIR), null); create(igfs, null, null); @@ -1131,11 +1137,19 @@ public void testUpdatePathMissing() throws Exception { * @throws Exception If failed. */ public void testUpdateParentRootPathMissing() throws Exception { + doUpdateParentRootPathMissing(properties("owner", "group", "0555")); + } + + /** + * Test update when parent is the root and the path being updated is missing locally. + * + * @param props Properties. + * @throws Exception If failed. + */ + protected void doUpdateParentRootPathMissing(Map props) throws Exception { if (!propertiesSupported()) return; - Map props = properties("owner", "group", "0555"); - create(igfsSecondary, paths(DIR), null); create(igfs, null, null); @@ -1143,8 +1157,8 @@ public void testUpdateParentRootPathMissing() throws Exception { checkExist(igfs, DIR); - assertEquals(props, igfsSecondary.properties(DIR.toString())); - assertEquals(props, igfs.info(DIR).properties()); + assertTrue(propertiesContains(igfsSecondary.properties(DIR.toString()), props)); + assertTrue(propertiesContains(igfs.info(DIR).properties(), props)); } /** @@ -1613,4 +1627,20 @@ public void testSecondarySize() throws Exception { assertEquals(chunk.length, igfs.size(FILE)); assertEquals(chunk.length * 2, igfs.size(SUBDIR)); } + + /** + * @param allProps All properties. + * @param checkedProps Checked properies + * @return {@code true} If allchecked properties are contained in the #propsAll. + */ + public static boolean propertiesContains(Map allProps, Map checkedProps) { + for (String name : checkedProps.keySet()) + if (!checkedProps.get(name).equals(allProps.get(name))) { + System.err.println("All properties: " + allProps); + System.err.println("Checked properties: " + checkedProps); + return false; + } + + return true; + } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java index 0e6fc48c4a8f3..8a23954859a90 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.igfs; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.igfs.IgfsFile; import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsPath; @@ -60,6 +61,13 @@ public abstract class IgfsLocalSecondaryFileSystemDualAbstractSelfTest extends I /** */ private final File fileLinkSrc = new File(FS_WORK_DIR + File.separatorChar + "file"); + /** */ + private final String TEST_GROUP = System.getProperty("IGFS_LOCAL_FS_TEST_GROUP", "igfs_grp_0"); + + /** */ + private final Boolean PROPERTIES_SUPPORT = + IgniteSystemProperties.getBoolean("IGFS_LOCAL_FS_PROPERTIES_SUPPORT", false); + /** * Constructor. @@ -103,13 +111,13 @@ protected IgfsLocalSecondaryFileSystemDualAbstractSelfTest(IgfsMode mode) { } /** {@inheritDoc} */ - @Override protected boolean permissionsSupported() { - return false; + @Override protected boolean propertiesSupported() { + return !U.isWindows() && PROPERTIES_SUPPORT; } /** {@inheritDoc} */ - @Override protected boolean propertiesSupported() { - return false; + @Override protected boolean permissionsSupported() { + return !U.isWindows(); } /** {@inheritDoc} */ @@ -169,6 +177,16 @@ public void testSymlinkToFile() throws Exception { checkFileContent(igfs, new IgfsPath("/file"), chunk); } + /** + * Test update when parent is the root and the path being updated is missing locally. + * + * @throws Exception If failed. + */ + public void testUpdateParentRootPathMissing() throws Exception { + doUpdateParentRootPathMissing(properties(TEST_GROUP, "0555")); + } + + /** * * @throws Exception If failed. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxySelfTest.java index 848abe2b825ae..e7f9bbb99a3a1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemProxySelfTest.java @@ -105,11 +105,6 @@ public class IgfsLocalSecondaryFileSystemProxySelfTest extends IgfsProxySelfTest return false; } - /** {@inheritDoc} */ - @Override public void testUpdatePathDoesNotExist() throws Exception { - fail("IGNITE-3645"); - } - /** * * @throws Exception If failed. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemTestAdapter.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemTestAdapter.java index 12714c461b892..8f6af832fb811 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemTestAdapter.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemTestAdapter.java @@ -17,6 +17,10 @@ package org.apache.ignite.internal.processors.igfs; +import java.nio.file.attribute.PosixFileAttributeView; +import java.nio.file.attribute.PosixFileAttributes; +import java.nio.file.attribute.PosixFilePermission; +import java.util.HashMap; import org.apache.ignite.internal.util.typedef.T2; import java.io.File; @@ -77,12 +81,30 @@ public IgfsLocalSecondaryFileSystemTestAdapter(final File workDir) { /** {@inheritDoc} */ @Override public Map properties(final String path) throws IOException { - throw new UnsupportedOperationException("properties"); + Path p = path(path); + PosixFileAttributes attrs = Files.getFileAttributeView(p, PosixFileAttributeView.class).readAttributes(); + + Map props = new HashMap<>(); + props.put(IgfsUtils.PROP_USER_NAME, attrs.owner().getName()); + props.put(IgfsUtils.PROP_GROUP_NAME, attrs.group().getName()); + props.put(IgfsUtils.PROP_PERMISSION, permissions(path)); + + return props; } /** {@inheritDoc} */ @Override public String permissions(String path) throws IOException { - throw new UnsupportedOperationException("permissions"); + Path p = path(path); + PosixFileAttributeView attrView = Files.getFileAttributeView(p, PosixFileAttributeView.class); + + if (attrView == null) + throw new UnsupportedOperationException("Posix file attributes not available"); + + int perm = 0; + for(PosixFilePermission pfp : attrView.readAttributes().permissions()) + perm |= (1 << 8 - pfp.ordinal()); + + return '0' + Integer.toOctalString(perm); } /** {@inheritDoc} */ @@ -123,6 +145,7 @@ private Path path(String path) { * * @param path Path. * @throws IOException If failed. + * @return {@code true} if the file is deleted successfully. {@code false} otherwise. */ private boolean deleteRecursively(Path path) throws IOException { if (Files.isDirectory(path)) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/benchmark/IgfsBenchmark.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/benchmark/IgfsBenchmark.java new file mode 100644 index 0000000000000..9cf6e8faf4982 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/benchmark/IgfsBenchmark.java @@ -0,0 +1,561 @@ +/* + * 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.igfs.benchmark; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteFileSystem; +import org.apache.ignite.Ignition; +import org.apache.ignite.igfs.IgfsFile; +import org.apache.ignite.igfs.IgfsInputStream; +import org.apache.ignite.igfs.IgfsOutputStream; +import org.apache.ignite.igfs.IgfsPath; +import org.apache.ignite.igfs.IgfsPathNotFoundException; + +/** + * + */ +class FileOperation { + /** Buff size. */ + public static final int BUFF_SIZE = 8192; + + /** Data bufer. */ + ByteBuffer dataBufer = ByteBuffer.allocate(BUFF_SIZE); + + /** Filesystem. */ + protected final IgniteFileSystem fs; + + /** + * @param fs Ignite filesystem to benchmark. + */ + public FileOperation(IgniteFileSystem fs) { + this.fs = fs; + } + + /** + * @param path Path to do operation. + * @throws Exception If failed. + */ + public void handleFile(String path) throws Exception { + // No-op. + } + + /** + * @param path Path to do operation. + * @throws Exception If failed. + */ + public void preHandleDir(String path) throws Exception { + // No-op. + } + + /** + * @param path Path to do operation. + * @throws Exception If failed. + */ + public void postHandleDir(String path) throws Exception { + // No-op. + } +} + +/** + * + */ +class WriteFileOperation extends FileOperation { + /** Size. */ + private int size; + + /** + * @param fs Filesystem/ + * @param size Size to write. + */ + public WriteFileOperation(IgniteFileSystem fs, int size) { + super(fs); + this.size = size; + } + + /** {@inheritDoc} */ + @Override public void handleFile(String strPath) throws Exception { + IgfsPath path = new IgfsPath(strPath); + IgfsOutputStream out = null; + + try { + out = fs.create(path, false); + } + catch (IgniteException ex) { + System.out.println("create file " + path.toString() + " failed: " + ex); + throw ex; + } + + try { + for (int i = 0; i < size / dataBufer.capacity(); i++) + out.write(dataBufer.array()); + } + catch (IOException ex) { + System.out.println("write file " + path.toString() + " failed: " + ex); + throw ex; + } + finally { + out.close(); + } + } + + /** {@inheritDoc} */ + @Override public void preHandleDir(String strPath) throws Exception { + IgfsPath path = new IgfsPath(strPath); + + if (fs.exists(path)) + throw new IgniteException("path " + path.toString() + " already exists"); + + try { + fs.mkdirs(path); + } + catch (IgniteException ex) { + throw ex; + } + } +} + +/** + * + */ +class ReadFileOperation extends FileOperation { + /** Size. */ + private int size; + + /** + * @param fs Filesystem + * @param size Size to read. + */ + public ReadFileOperation(IgniteFileSystem fs, int size) { + super(fs); + this.size = size; + } + + /** {@inheritDoc} */ + @Override public void handleFile(String strPath) throws Exception { + IgfsPath path = new IgfsPath(strPath); + IgfsInputStream in = null; + + try { + in = fs.open(path); + } + catch (IgfsPathNotFoundException ex) { + System.out.println("file " + path.toString() + " not exist: " + ex); + throw ex; + } + catch (IgniteException ex) { + System.out.println("open file " + path.toString() + " failed: " + ex); + throw ex; + } + + try { + for (int i = 0; i < size / dataBufer.capacity(); i++) + in.read(dataBufer.array()); + } + catch (IOException ex) { + System.out.println("read file " + path.toString() + " failed: " + ex); + throw ex; + } + finally { + in.close(); + } + } + + /** {@inheritDoc} */ + @Override public void preHandleDir(String strPath) throws Exception { + IgfsPath path = new IgfsPath(strPath); + + if (!fs.exists(path)) { + System.out.println("path " + path.toString() + " not exist"); + throw new IgniteException("path " + path.toString() + " not exist"); + } + } +} + +/** + * + */ +class DeleteFileOperation extends FileOperation { + /** Size. */ + private int size; + + /** + * @param fs Filesystem. + * @param size Size. + */ + public DeleteFileOperation(IgniteFileSystem fs, int size) { + super(fs); + this.size = size; + } + + /** {@inheritDoc} */ + @Override public void handleFile(String strPath) throws Exception { + IgfsPath path = new IgfsPath(strPath); + fs.delete(path, false); + } + + /** {@inheritDoc} */ + @Override public void postHandleDir(String strPath) throws Exception { + IgfsPath path = new IgfsPath(strPath); + fs.delete(path, false); + } +} + +/** + * + */ +class InfoFileOperation extends FileOperation { + /** + * @param fs Filesystem. + */ + public InfoFileOperation(IgniteFileSystem fs) { + super(fs); + } + + /** {@inheritDoc} */ + @Override public void handleFile(String strPath) throws Exception { + IgfsPath path = new IgfsPath(strPath); + IgfsFile info = fs.info(path); + + assert info != null : "Info must be not null for exists file. All files must be exists for benchmark"; + } + + /** {@inheritDoc} */ + @Override public void postHandleDir(String strPath) throws Exception { + IgfsPath path = new IgfsPath(strPath); + IgfsFile info = fs.info(path); + + assert info != null : "Info must be not null for exists dir. All dirs must be exists for benchmark"; + } +} + +/** + * + */ +class ListPathFileOperation extends FileOperation { + /** + * @param fs Filesystem. + */ + public ListPathFileOperation(IgniteFileSystem fs) { + super(fs); + } + + /** {@inheritDoc} */ + @Override public void postHandleDir(String strPath) throws Exception { + IgfsPath path = new IgfsPath(strPath); + + Collection lst = fs.listPaths(path); + + assert lst != null : "List of paths must not be null"; + } +} + +/** + * + */ +public class IgfsBenchmark { + /** Path. */ + private final String path; + + /** Depth. */ + private final int depth; + + /** Width. */ + private final int subDirsCount; + + /** Count. */ + private final int filesCount; + + /** Size. */ + private final int size; + + /** + * @param path Root test path. + * @param depth Directory depth. + * @param subDirsCount Count of subdirectories. + * @param filesCount Count of files. + * @param size Size of file. + */ + public IgfsBenchmark(String path, + int depth, + int subDirsCount, + int filesCount, + int size) { + this.path = path; + this.depth = depth; + this.subDirsCount = subDirsCount; + this.filesCount = filesCount; + this.size = (size > FileOperation.BUFF_SIZE) ? size : FileOperation.BUFF_SIZE; + } + + /** + * @param lst List of measurement results. + * @return Average value. + */ + public static long avg(List lst) { + if (lst.isEmpty()) + throw new IllegalArgumentException("List must be not empty"); + + long sum = 0; + for (long l : lst) + sum += l; + + return sum / lst.size(); + } + + /** + * @param lst List of measurement results. + * @param avg Average value. + * @return THe value of the standard derivation. + */ + public static long stdDev(List lst, long avg) { + if (lst.isEmpty()) + throw new IllegalArgumentException("List must be not empty"); + + long sum = 0; + for (long l : lst) + sum += (l - avg) * (l - avg); + + return (long)Math.sqrt((double)sum / (double)lst.size()); + } + + /** + * @param args Commandline arguments + */ + public static void main(String[] args) { + Ignition.setClientMode(Boolean.getBoolean("clientMode")); + + Ignite ignite = Ignition.start(System.getProperty("cfg", "default-config.xml")); + + int wormUpCount = Integer.getInteger("wormup", 2); + int cycles = Integer.getInteger("cycles", 10); + + final IgfsBenchmark fsTest = new IgfsBenchmark( + System.getProperty("testDir", "/test"), + Integer.getInteger("depth", 3), + Integer.getInteger("subDirs", 10), + Integer.getInteger("files", 10), + Integer.getInteger("fileSize", 8) * 1024); + + final IgniteFileSystem fs = ignite.fileSystem("igfs"); + + try { + for (int i = 0; i < wormUpCount; ++i) { + System.out.println("Wormup #" + i + " / " + wormUpCount); + fsTest.testWriteFile(fs); + fsTest.testReadFile(fs); + fsTest.testDeleteFile(fs); + } + } + catch (Exception ex) { + System.err.println("Wormup error"); + ex.printStackTrace(System.err); + Ignition.stop(false); + return; + } + + List writeRes = new ArrayList<>(cycles); + List readRes = new ArrayList<>(cycles); + List infoRes = new ArrayList<>(cycles); + List listRes = new ArrayList<>(cycles); + List delRes = new ArrayList<>(cycles); + + try { + for (int i = 0; i < cycles; ++i) { + System.out.println("Benchmark cycle #" + i + " / " + cycles); + + writeRes.add(bench(new Runnable() { + @Override public void run() { + fsTest.testWriteFile(fs); + } + })); + + readRes.add(bench(new Runnable() { + @Override public void run() { + fsTest.testReadFile(fs); + } + })); + + infoRes.add(bench(new Runnable() { + @Override public void run() { + fsTest.testInfoFile(fs); + } + })); + + listRes.add(bench(new Runnable() { + @Override public void run() { + fsTest.testListPathFile(fs); + } + })); + + delRes.add(bench(new Runnable() { + @Override public void run() { + fsTest.testDeleteFile(fs); + } + })); + } + + System.out.println("\n"); + System.out.println("Write " + avg(writeRes) + " +/- " + stdDev(writeRes, avg(writeRes))); + System.out.println("Read " + avg(readRes) + " +/- " + stdDev(readRes, avg(readRes))); + System.out.println("Info " + avg(infoRes) + " +/- " + stdDev(infoRes, avg(infoRes))); + System.out.println("List " + avg(listRes) + " +/- " + stdDev(listRes, avg(listRes))); + System.out.println("Delete " + avg(delRes) + " +/- " + stdDev(delRes, avg(delRes))); + } + catch (Exception ex) { + System.err.println("Benchmark error"); + ex.printStackTrace(System.err); + } + finally { + Ignition.stop(false); + } + } + + /** + * @param parentPath Begin path. + * @param depth Current deep. + * @return List of subdirs. + */ + private String[] buildPath(String parentPath, int depth) { + String curPath[] = new String[subDirsCount]; + + for (int i = 1; i <= curPath.length; i++) + curPath[i - 1] = parentPath + "/vdb." + depth + "_" + i + ".dir"; + + return curPath; + } + + /** + * @param parentPath Begin path. + * @param operation Test operation to do. + * @throws Exception If failed. + */ + private void recurseFile(String parentPath, FileOperation operation) throws Exception { + for (int i = 1; i <= filesCount; i++) { + String filePath = parentPath + "/vdb_f" + String.format("%0" + String.valueOf(this.filesCount).length() + "d", i) + ".file"; + operation.handleFile(filePath); + } + } + + /** + * @param parentPath Begin path. + * @param depth depth of recurse. + * @param operation Test operation to do. + * @throws Exception If failed. + */ + private void recursePath(String parentPath, int depth, FileOperation operation) throws Exception { + if (depth == this.depth + 1) + recurseFile(parentPath, operation); + else { + String curPath[] = buildPath(parentPath, depth); + + for (String path : curPath) { + operation.preHandleDir(path); + recursePath(path, depth + 1, operation); + operation.postHandleDir(path); + } + } + } + + /** + * Do read file operations. Files must be exist. + * + * @param fs Filesystem. + */ + public void testReadFile(IgniteFileSystem fs) { + try { + recursePath(path, 1, new ReadFileOperation(fs, size)); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Do write file operations. + * + * @param fs Filesystem. + */ + public void testWriteFile(IgniteFileSystem fs) { + try { + recursePath(path, 1, new WriteFileOperation(fs, size)); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Do delete file operations. Files must be exist. + * + * @param fs Filesystem. + */ + public void testDeleteFile(IgniteFileSystem fs) { + try { + recursePath(path, 1, new DeleteFileOperation(fs, 0)); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Do info file operations. Files must be exist. + * + * @param fs Filesystem. + */ + public void testInfoFile(IgniteFileSystem fs) { + try { + recursePath(path, 1, new InfoFileOperation(fs)); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Do info file operations. Files must be exist. + * + * @param fs Filesystem. + */ + public void testListPathFile(IgniteFileSystem fs) { + try { + recursePath(path, 1, new ListPathFileOperation(fs)); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * @param r Runnable. + * @return Time of execution in millis. + */ + public static long bench(Runnable r) { + long t0 = System.currentTimeMillis(); + + r.run(); + + return System.currentTimeMillis() - t0; + } +} diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java index fcad674deb116..203965cebc137 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java @@ -250,8 +250,7 @@ public HadoopIgfsSecondaryFileSystemDelegateImpl(IgniteHadoopIgfsSecondaryFileSy /** {@inheritDoc} */ @Override public OutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication, long blockSize, @Nullable Map props) { - HadoopIgfsProperties props0 = - new HadoopIgfsProperties(props != null ? props : Collections.emptyMap()); + HadoopIgfsProperties props0 = new HadoopIgfsProperties(props); try { return fileSystemForUser().create(convert(path), props0.permission(), overwrite, bufSize, diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsProperties.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsProperties.java index 5427bf15b6033..a322a9939fd25 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsProperties.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsProperties.java @@ -43,6 +43,9 @@ public class HadoopIgfsProperties { * @throws IgniteException In case of error. */ public HadoopIgfsProperties(Map props) throws IgniteException { + if (props == null) + return; + usrName = props.get(IgfsUtils.PROP_USER_NAME); grpName = props.get(IgfsUtils.PROP_GROUP_NAME); From fa698d6fcfde0006423ef009337dbef58ac62515 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 3 Oct 2016 13:33:12 +0300 Subject: [PATCH 219/487] Fixed TcpDiscoverySnapshotHistoryTest intermittent failures caused by multicast IP finder. --- .../tcp/TcpDiscoverySnapshotHistoryTest.java | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySnapshotHistoryTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySnapshotHistoryTest.java index 9667fb8cc5d7d..47e911204edb2 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySnapshotHistoryTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySnapshotHistoryTest.java @@ -22,24 +22,26 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.discovery.DiscoverySpi; import org.apache.ignite.spi.discovery.DiscoverySpiHistorySupport; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import java.util.Collections; + import static org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi.DFLT_TOP_HISTORY_SIZE; /** * Tests for topology snapshots history. */ public class TcpDiscoverySnapshotHistoryTest extends GridCommonAbstractTest { - /** */ - public TcpDiscoverySnapshotHistoryTest() { - super(false); - } - /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); - cfg.setDiscoverySpi(new TcpDiscoverySpi()); + TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(); + + ipFinder.setAddresses(Collections.singleton("127.0.0.1:47500")); + + cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder)); cfg.setCacheConfiguration(); cfg.setLocalHost("127.0.0.1"); cfg.setConnectorConfiguration(null); From 1c82cd04fadc61ab5da02e1d395aedfb28a039da Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 23 Aug 2016 13:28:40 +0300 Subject: [PATCH 220/487] IGNITE-3279 .NET: NLog logger --- .../Apache.Ignite.Core.Tests.NuGet.csproj | 18 +- .../NLogTest.cs | 82 +++++++++ .../packages.config | 13 +- .../Apache.Ignite.Core.Tests.csproj | 13 +- .../IgniteConfigurationSerializerTest.cs | 3 +- .../Log/NLogLoggerTest.cs | 166 ++++++++++++++++++ .../Apache.Ignite.NLog.csproj | 74 ++++++++ .../Apache.Ignite.NLog.nuspec | 50 ++++++ .../Apache.Ignite.NLog/Apache.Ignite.NLog.snk | Bin 0 -> 596 bytes .../Apache.Ignite.NLog/IgniteNLogLogger.cs | 125 +++++++++++++ .../Properties/AssemblyInfo.cs | 40 +++++ .../dotnet/Apache.Ignite.NLog/packages.config | 4 + modules/platforms/dotnet/Apache.Ignite.sln | 14 ++ 13 files changed, 589 insertions(+), 13 deletions(-) create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/NLogTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/NLogLoggerTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj create mode 100644 modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.nuspec create mode 100644 modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.snk create mode 100644 modules/platforms/dotnet/Apache.Ignite.NLog/IgniteNLogLogger.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.NLog/packages.config diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Apache.Ignite.Core.Tests.NuGet.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Apache.Ignite.Core.Tests.NuGet.csproj index 8c4c46b5e3440..f3f917e9957a9 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Apache.Ignite.Core.Tests.NuGet.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Apache.Ignite.Core.Tests.NuGet.csproj @@ -72,17 +72,26 @@ False - packages\Apache.Ignite.1.6.0\lib\net40\Apache.Ignite.Core.dll + packages\Apache.Ignite.1.8.0\lib\net40\Apache.Ignite.Core.dll True False - packages\Apache.Ignite.Linq.1.6.0\lib\net40\Apache.Ignite.Linq.dll + packages\Apache.Ignite.Linq.1.8.0\lib\net40\Apache.Ignite.Linq.dll True False - packages\Apache.Ignite.AspNet.1.6.0\lib\net40\Apache.Ignite.AspNet.dll + packages\Apache.Ignite.AspNet.1.8.0\lib\net40\Apache.Ignite.AspNet.dll + True + + + False + packages\Apache.Ignite.NLog.1.8.0\lib\net40\Apache.Ignite.NLog.dll + True + + + packages\NLog.4.3.7\lib\net40\NLog.dll True @@ -103,6 +112,7 @@ + @@ -132,7 +142,7 @@ if not exist "$(TargetDir)Libs" md "$(TargetDir)Libs" -xcopy /s /y "$(SolutionDir)packages\Apache.Ignite.1.6.0\Libs\*.*" "$(TargetDir)Libs" +xcopy /s /y "$(SolutionDir)packages\Apache.Ignite.1.8.0\Libs\*.*" "$(TargetDir)Libs" diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/NLogTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/NLogTest.cs new file mode 100644 index 0000000000000..d3c58e18e2a10 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/NLogTest.cs @@ -0,0 +1,82 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Tests.NuGet +{ + using System.Linq; + using Apache.Ignite.NLog; + using global::NLog; + using global::NLog.Config; + using global::NLog.Layouts; + using global::NLog.Targets; + using NUnit.Framework; + + /// + /// NLog test. + /// + public class NLogTest + { + /// + /// The log target. + /// + private MemoryTarget _logTarget; + + /// + /// Test set up. + /// + [SetUp] + public void SetUp() + { + var cfg = new LoggingConfiguration(); + + _logTarget = new MemoryTarget("mem") + { + Layout = new SimpleLayout("${Logger}|${Level}|${Message}|${exception}|${all-event-properties}") + }; + + cfg.AddTarget(_logTarget); + + cfg.AddRule(LogLevel.Trace, LogLevel.Error, _logTarget); + + LogManager.Configuration = cfg; + } + + /// + /// Tests the logger with Ignite. + /// + [Test] + public void TestIgniteStartup() + { + var cfg = new IgniteConfiguration + { + DiscoverySpi = TestUtil.GetLocalDiscoverySpi(), + Logger = new IgniteNLogLogger(LogManager.GetCurrentClassLogger()) + }; + + using (Ignition.Start(cfg)) + { + Assert.IsTrue(_logTarget.Logs.Contains( + string.Format("|Debug|Starting Ignite.NET {0}||", typeof(Ignition).Assembly.GetName().Version))); + + Assert.IsTrue(_logTarget.Logs.Any(x => x.Contains(">>> Topology snapshot."))); + } + + Assert.IsTrue(_logTarget.Logs.Contains( + "org.apache.ignite.internal.IgniteKernal|Debug|Grid is stopping.||")); + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/packages.config b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/packages.config index 88d9cc21d42f3..30e71a1dfe081 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/packages.config +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/packages.config @@ -1,5 +1,4 @@  - - - - - - + + + + + + + \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj index 2c6150c150dda..ef62498f89c01 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj @@ -38,8 +38,12 @@ - - ..\libs\nunit-console-runner.dll + + ..\packages\NLog.4.3.7\lib\net40\NLog.dll + + + False + ..\packages\NUnit.Runners.2.6.3\tools\lib\nunit-console-runner.dll False @@ -54,6 +58,7 @@ + @@ -175,6 +180,10 @@ {5b571661-17f4-4f29-8c7d-0edb38ca9b55} Apache.Ignite.Linq + + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7} + Apache.Ignite.NLog + {27F7F3C6-BDDE-43A9-B565-856F8395A04B} Apache.Ignite diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs index beb1e8d0dacc0..bb703f52b05f4 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs @@ -46,6 +46,7 @@ namespace Apache.Ignite.Core.Tests using Apache.Ignite.Core.Log; using Apache.Ignite.Core.Tests.Binary; using Apache.Ignite.Core.Transactions; + using Apache.Ignite.NLog; using NUnit.Framework; /// @@ -549,7 +550,7 @@ private static IgniteConfiguration GetTestConfig() }, IsLateAffinityAssignment = false, SpringConfigUrl = "test", - Logger = new TestLogger() + Logger = new IgniteNLogLogger() }; } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/NLogLoggerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/NLogLoggerTest.cs new file mode 100644 index 0000000000000..7806ecd24dcb2 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/NLogLoggerTest.cs @@ -0,0 +1,166 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Tests.Log +{ + using System; + using System.Globalization; + using System.Linq; + using Apache.Ignite.Core.Log; + using Apache.Ignite.NLog; + using global::NLog; + using global::NLog.Config; + using global::NLog.Layouts; + using global::NLog.Targets; + using NUnit.Framework; + using LogLevel = Apache.Ignite.Core.Log.LogLevel; + + /// + /// Tests the NLog integration. + /// + public class NLogLoggerTest + { + /** */ + private MemoryTarget _logTarget; + + /// + /// Test set up. + /// + [SetUp] + public void SetUp() + { + var cfg = new LoggingConfiguration(); + + _logTarget = new MemoryTarget("mem") + { + Layout = new SimpleLayout("${Logger}|${Level}|${Message}|${exception}|${all-event-properties}") + }; + + cfg.AddTarget(_logTarget); + + cfg.AddRule(global::NLog.LogLevel.Trace, global::NLog.LogLevel.Error, _logTarget); + + LogManager.Configuration = cfg; + } + + /// + /// Tests the log level conversion. + /// + [Test] + public void TestLogLevelConversion() + { + var levels = new[] { LogLevel.Trace, LogLevel.Info, LogLevel.Debug, LogLevel.Warn, LogLevel.Error }; + + var nLogger = new IgniteNLogLogger(LogManager.GetCurrentClassLogger()); + + foreach (var igniteLevel in levels) + { + var nlogLevel = IgniteNLogLogger.ConvertLogLevel(igniteLevel); + + Assert.AreEqual(igniteLevel.ToString(), nlogLevel.ToString()); + + + Assert.IsTrue(nLogger.IsEnabled(igniteLevel)); + } + } + + /// + /// Tests the logger in isolated environment. + /// + [Test] + public void TestLogging() + { + var nLogger = new IgniteNLogLogger(); + + // All parameters. + nLogger.Log(LogLevel.Trace, "msg{0}", new object[] {1}, CultureInfo.InvariantCulture, "category", + "java-err", new Exception("myException")); + + Assert.AreEqual("category|Trace|msg1|myException|nativeErrorInfo=java-err", GetLastLog()); + + // No Java error. + nLogger.Log(LogLevel.Info, "msg{0}", new object[] { 1 }, CultureInfo.InvariantCulture, "category", + null, new Exception("myException")); + + Assert.AreEqual("category|Info|msg1|myException|", GetLastLog()); + + // No exception. + nLogger.Log(LogLevel.Debug, "msg{0}", new object[] { 1 }, CultureInfo.InvariantCulture, "category", + null, null); + + Assert.AreEqual("category|Debug|msg1||", GetLastLog()); + + // No params. + nLogger.Log(LogLevel.Warn, "msg{0}", null, CultureInfo.InvariantCulture, "category", null, null); + + Assert.AreEqual("category|Warn|msg{0}||", GetLastLog()); + + // No formatter. + nLogger.Log(LogLevel.Error, "msg{0}", null, null, "category", null, null); + + Assert.AreEqual("category|Error|msg{0}||", GetLastLog()); + + // No category. + nLogger.Log(LogLevel.Error, "msg{0}", null, null, null, null, null); + + Assert.AreEqual("|Error|msg{0}||", GetLastLog()); + + // No message. + nLogger.Log(LogLevel.Error, null, null, null, null, null, null); + + Assert.AreEqual("|Error|||", GetLastLog()); + } + + /// + /// Tests the logger with Ignite. + /// + [Test] + public void TestIgniteStartup() + { + var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration()) + { + Logger = new IgniteNLogLogger(LogManager.GetLogger("foo")) + }; + + using (var ignite = Ignition.Start(cfg)) + { + Assert.IsTrue(_logTarget.Logs.Contains( + string.Format("|Debug|Starting Ignite.NET {0}||", typeof(Ignition).Assembly.GetName().Version))); + + Assert.IsTrue(_logTarget.Logs.Any(x => x.Contains(">>> Topology snapshot."))); + + Assert.IsInstanceOf(ignite.Logger); + + ignite.Logger.Info("Log from user code."); + + Assert.IsTrue(_logTarget.Logs.Contains("|Info|Log from user code.||")); + } + + Assert.IsTrue(_logTarget.Logs.Contains( + "org.apache.ignite.internal.IgniteKernal|Debug|Grid is stopping.||")); + } + + /// + /// Gets the last log. + /// + private string GetLastLog() + { + return _logTarget.Logs.Last(); + } + + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj b/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj new file mode 100644 index 0000000000000..c8d87057ad00a --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj @@ -0,0 +1,74 @@ + + + + + Debug + AnyCPU + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7} + Library + Properties + Apache.Ignite.NLog + Apache.Ignite.NLog + v4.0 + 512 + + + true + full + false + bin\Debug\ + DEBUG;TRACE + prompt + 4 + + + pdbonly + true + bin\Release\ + TRACE + prompt + 4 + + + true + + + Apache.Ignite.NLog.snk + + + + ..\packages\NLog.4.3.7\lib\net40\NLog.dll + True + + + + + + + + + + + + + + + + {4CD2F726-7E2B-46C4-A5BA-057BB82EECB6} + Apache.Ignite.Core + + + + + + + + + + \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.nuspec b/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.nuspec new file mode 100644 index 0000000000000..765e26f390cfb --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.nuspec @@ -0,0 +1,50 @@ + + + + + + + + + Apache.Ignite.NLog + Apache Ignite NLog Logger + + $version$ + Apache Ignite + Apache Software Foundation + http://www.apache.org/licenses/LICENSE-2.0 + https://ignite.apache.org/ + https://ignite.apache.org/images/logo_ignite_32_32.png + false + NLog Logger for Apache Ignite + + Copyright 2016 + Apache Ignite In-Memory Distributed Computing SQL NoSQL LINQ Grid Map Reduce Cache NLog logger + + + + + + \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.snk b/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.snk new file mode 100644 index 0000000000000000000000000000000000000000..799e7426cc22c75b1d9227ef15b9be16ca3b5d69 GIT binary patch literal 596 zcmV-a0;~N80ssI2Bme+XQ$aES1ONa50098WI1J!kw@cT>G50Q>rWM7Ve$KF)*?C;E z-|_tRIQ@B?Dp;w$LjUnr@8QVJ>$g$jcVqU+r^pQ49l^WwLW5wA@r=z-7yVbVuqIG8 zH^`hNH}heDe>>etCSOfNM-~;Gn3FbH=E~b3Xl|Cz5&3CEDkOJ^w`c>vm0sy;rbFDm)+jiE6CRxxzp2(vm{nRko zyu37iYmu#%&mGZDWup?f4QWHBdodC-*Y734QiR>Qyb!}(u!cVctHhaZjq)FSt$8)W8n>~=n z6{a3iqa@Mc3aidg&2}il<)?CS`n$hwf`Ii z#4eft)WS|@Mu1+5Si4L&6xMNZgtQX{pUmY}Iiz=8Pw{K(rqvYzYsKD=)JAI1*8Vic ihIz#0$MA9g*TUC{Xy9bze(k}}o64o!sUK4b(_ejv91 literal 0 HcmV?d00001 diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/IgniteNLogLogger.cs b/modules/platforms/dotnet/Apache.Ignite.NLog/IgniteNLogLogger.cs new file mode 100644 index 0000000000000..2730e372d1285 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.NLog/IgniteNLogLogger.cs @@ -0,0 +1,125 @@ +/* + * 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. + */ + +namespace Apache.Ignite.NLog +{ + using System; + using Apache.Ignite.Core.Impl.Common; + using global::NLog; + using ILogger = Apache.Ignite.Core.Log.ILogger; + using IgniteLogLevel = Apache.Ignite.Core.Log.LogLevel; + using NLogLogLevel = global::NLog.LogLevel; + + /// + /// Ignite NLog integration. + /// + public class IgniteNLogLogger : ILogger + { + /// + /// The NLog logger. + /// + private readonly Logger _logger; + + /// + /// Initializes a new instance of the class using the + /// to retrieve the NLog logger. + /// + public IgniteNLogLogger() : this(LogManager.GetCurrentClassLogger()) + { + // No-op. + } + + /// + /// Initializes a new instance of the class. + /// + /// The NLog logger instance. + public IgniteNLogLogger(Logger logger) + { + IgniteArgumentCheck.NotNull(logger, "logger"); + + _logger = logger; + } + + /// + /// Logs the specified message. + /// + /// The level. + /// The message. + /// The arguments to format . + /// Can be null (formatting will not occur). + /// The format provider. Can be null if is null. + /// The logging category name. + /// The native error information. + /// The exception. Can be null. + /// + public void Log(IgniteLogLevel level, string message, object[] args, IFormatProvider formatProvider, + string category, string nativeErrorInfo, Exception ex) + { + var logEvent = new LogEventInfo + { + Level = ConvertLogLevel(level), + Message = message, + FormatProvider = formatProvider, + Parameters = args, + Exception = ex, + LoggerName = category + }; + + if (nativeErrorInfo != null) + logEvent.Properties.Add("nativeErrorInfo", nativeErrorInfo); + + _logger.Log(logEvent); + } + + /// + /// Determines whether the specified log level is enabled. + /// + /// The level. + /// + /// Value indicating whether the specified log level is enabled + /// + /// + public bool IsEnabled(IgniteLogLevel level) + { + return _logger.IsEnabled(ConvertLogLevel(level)); + } + + /// + /// Converts the Ignite LogLevel to the NLog log level. + /// + /// The Ignite log level. + /// Corresponding NLog log level. + public static NLogLogLevel ConvertLogLevel(IgniteLogLevel level) + { + switch (level) + { + case IgniteLogLevel.Trace: + return NLogLogLevel.Trace; + case IgniteLogLevel.Debug: + return NLogLogLevel.Debug; + case IgniteLogLevel.Info: + return NLogLogLevel.Info; + case IgniteLogLevel.Warn: + return NLogLogLevel.Warn; + case IgniteLogLevel.Error: + return NLogLogLevel.Error; + default: + throw new ArgumentOutOfRangeException("level", level, "Invalid Ignite LogLevel."); + } + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs new file mode 100644 index 0000000000000..66038361f3792 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs @@ -0,0 +1,40 @@ +/* +* 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. +*/ + +using System; +using System.Reflection; +using System.Runtime.InteropServices; + +[assembly: AssemblyTitle("Apache.Ignite.NLog")] +[assembly: AssemblyDescription("Apache Ignite.NET NLog integration.")] +[assembly: AssemblyConfiguration("")] +[assembly: AssemblyCompany("Apache Software Foundation")] +[assembly: AssemblyProduct("Apache Ignite.NET")] +[assembly: AssemblyCopyright("Copyright © 2015")] +[assembly: AssemblyTrademark("")] +[assembly: AssemblyCulture("")] + +[assembly: ComVisible(false)] + +// The following GUID is for the ID of the typelib if this project is exposed to COM +[assembly: Guid("c6b58e4a-a2e9-4554-ad02-68ce6da5cfb7")] + +[assembly: AssemblyVersion("1.8.0.13244")] +[assembly: AssemblyFileVersion("1.8.0.13244")] +[assembly: AssemblyInformationalVersion("1.8.0")] + +[assembly: CLSCompliant(true)] diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/packages.config b/modules/platforms/dotnet/Apache.Ignite.NLog/packages.config new file mode 100644 index 0000000000000..50aeb52c2b59a --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.NLog/packages.config @@ -0,0 +1,4 @@ + + + + \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.sln b/modules/platforms/dotnet/Apache.Ignite.sln index 8a3bf041b221e..2978780877bd4 100644 --- a/modules/platforms/dotnet/Apache.Ignite.sln +++ b/modules/platforms/dotnet/Apache.Ignite.sln @@ -38,6 +38,8 @@ Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.AspNet", "Apa EndProject Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.AspNet.Tests", "Apache.Ignite.AspNet.Tests\Apache.Ignite.AspNet.Tests.csproj", "{18EA4C71-A11D-4AB1-8042-418F7559D84F}" EndProject +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.NLog", "Apache.Ignite.NLog\Apache.Ignite.NLog.csproj", "{C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}" +EndProject Global GlobalSection(SolutionConfigurationPlatforms) = preSolution Debug|Any CPU = Debug|Any CPU @@ -188,6 +190,18 @@ Global {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Release|x64.Build.0 = Release|Any CPU {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Release|x86.ActiveCfg = Release|Any CPU {18EA4C71-A11D-4AB1-8042-418F7559D84F}.Release|x86.Build.0 = Release|Any CPU + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Debug|Any CPU.Build.0 = Debug|Any CPU + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Debug|x64.ActiveCfg = Debug|Any CPU + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Debug|x64.Build.0 = Debug|Any CPU + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Debug|x86.ActiveCfg = Debug|Any CPU + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Debug|x86.Build.0 = Debug|Any CPU + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Release|Any CPU.ActiveCfg = Release|Any CPU + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Release|Any CPU.Build.0 = Release|Any CPU + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Release|x64.ActiveCfg = Release|Any CPU + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Release|x64.Build.0 = Release|Any CPU + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Release|x86.ActiveCfg = Release|Any CPU + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Release|x86.Build.0 = Release|Any CPU EndGlobalSection GlobalSection(SolutionProperties) = preSolution HideSolutionNode = FALSE From 48b293db4d443b4d8739f709ff12f19aad008b84 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 3 Oct 2016 14:08:07 +0300 Subject: [PATCH 221/487] IGNITE-3279 .NET: NLog logger --- .../dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs index 66038361f3792..baaa60d94c8e7 100644 --- a/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs @@ -33,8 +33,8 @@ // The following GUID is for the ID of the typelib if this project is exposed to COM [assembly: Guid("c6b58e4a-a2e9-4554-ad02-68ce6da5cfb7")] -[assembly: AssemblyVersion("1.8.0.13244")] -[assembly: AssemblyFileVersion("1.8.0.13244")] +[assembly: AssemblyVersion("1.8.0.14218")] +[assembly: AssemblyFileVersion("1.8.0.14218")] [assembly: AssemblyInformationalVersion("1.8.0")] [assembly: CLSCompliant(true)] From bba019fd5076412ca43c10a32fd300b6031ccd0b Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 3 Oct 2016 17:25:20 +0300 Subject: [PATCH 222/487] IGNITE-3980: Processed failing tests in query suites. This closes #1137. --- .../ignite/testframework/IgniteTestSuite.java | 18 ++++++-- modules/ignored-tests/pom.xml | 8 ++++ ...iteIgnoredBinarySimpleMapperTestSuite.java | 41 ++++++++++++++++++ .../IgniteIgnoredBinaryTestSuite.java | 42 +++++++++++++++++++ .../testsuites/IgniteIgnoredTestSuite.java | 4 ++ .../IgniteCacheReplicatedQuerySelfTest.java | 4 +- .../query/IgniteSqlSplitterSelfTest.java | 4 +- .../query/h2/sql/BaseH2CompareQueryTest.java | 3 +- ...rySimpleNameMapperCacheQueryTestSuite.java | 2 - .../IgniteCacheQuerySelfTestSuite.java | 3 +- .../IgniteCacheQuerySelfTestSuite2.java | 3 +- 11 files changed, 119 insertions(+), 13 deletions(-) create mode 100644 modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredBinarySimpleMapperTestSuite.java create mode 100644 modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredBinaryTestSuite.java diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java index 1cf69ae16e502..157e1f5af4277 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/IgniteTestSuite.java @@ -103,11 +103,21 @@ public IgniteTestSuite(@Nullable Class theClass, @Nullable S } /** {@inheritDoc} */ - @Override public void addTestSuite(Class testClass) { - IgniteTestSuite suite = new IgniteTestSuite(testClass, ignoredOnly); + @Override public void addTest(Test test) { + // Ignore empty test suites. + if (test instanceof IgniteTestSuite) { + IgniteTestSuite suite = (IgniteTestSuite)test; + + if (suite.testCount() == 0) + return; + } - if (suite.testCount() > 0) - addTest(suite); + super.addTest(test); + } + + /** {@inheritDoc} */ + @Override public void addTestSuite(Class testClass) { + addTest(new IgniteTestSuite(testClass, ignoredOnly)); } /** diff --git a/modules/ignored-tests/pom.xml b/modules/ignored-tests/pom.xml index 142754e10fa8f..00c7d554d7650 100644 --- a/modules/ignored-tests/pom.xml +++ b/modules/ignored-tests/pom.xml @@ -98,6 +98,14 @@ ${project.version} + + org.apache.ignite + ignite-indexing + ${project.version} + test-jar + test + + org.apache.ignite ignite-jta diff --git a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredBinarySimpleMapperTestSuite.java b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredBinarySimpleMapperTestSuite.java new file mode 100644 index 0000000000000..aa2e219e27e67 --- /dev/null +++ b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredBinarySimpleMapperTestSuite.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.testsuites; + +import junit.framework.TestSuite; +import org.apache.ignite.testframework.IgniteTestSuite; + +/** + * Special test suite with ignored tests for Binary mode. + */ +public class IgniteIgnoredBinarySimpleMapperTestSuite extends TestSuite { + /** + * @return IgniteCache test suite. + * @throws Exception Thrown in case of the failure. + */ + public static TestSuite suite() throws Exception { + IgniteTestSuite.ignoreDefault(true); + + IgniteTestSuite suite = new IgniteTestSuite(null, "Ignite Ignored Binary Simple Mapper Test Suite"); + + /* --- QUERY --- */ + suite.addTest(IgniteBinarySimpleNameMapperCacheQueryTestSuite.suite()); + + return suite; + } +} diff --git a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredBinaryTestSuite.java b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredBinaryTestSuite.java new file mode 100644 index 0000000000000..5116bd74d3346 --- /dev/null +++ b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredBinaryTestSuite.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.testsuites; + +import junit.framework.TestSuite; +import org.apache.ignite.testframework.IgniteTestSuite; + +/** + * Special test suite with ignored tests for Binary mode. + */ +public class IgniteIgnoredBinaryTestSuite extends TestSuite { + /** + * @return IgniteCache test suite. + * @throws Exception Thrown in case of the failure. + */ + public static TestSuite suite() throws Exception { + IgniteTestSuite.ignoreDefault(true); + + IgniteTestSuite suite = new IgniteTestSuite(null, "Ignite Ignored Binary Test Suite"); + + /* --- QUERY --- */ + suite.addTest(IgniteBinaryCacheQueryTestSuite.suite()); + suite.addTest(IgniteBinaryCacheQueryTestSuite2.suite()); + + return suite; + } +} diff --git a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java index fd92a35aeae0b..414d166062c1d 100644 --- a/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java +++ b/modules/ignored-tests/src/test/java/org/apache/ignite/testsuites/IgniteIgnoredTestSuite.java @@ -46,6 +46,10 @@ public static TestSuite suite() throws Exception { /* --- JTA --- */ suite.addTest(IgniteJtaTestSuite.suite()); + /* --- QUERIES --- */ + suite.addTest(IgniteCacheQuerySelfTestSuite.suite()); + suite.addTest(IgniteCacheQuerySelfTestSuite2.suite()); + /* --- SPRING --- */ suite.addTest(IgniteSpringTestSuite.suite()); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java index 06adb688637d9..a673a73be77e8 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java @@ -53,6 +53,7 @@ import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testsuites.IgniteIgnore; import org.apache.ignite.transactions.Transaction; import org.springframework.util.ReflectionUtils; @@ -354,9 +355,8 @@ public void testLostIterator() throws Exception { /** * @throws Exception If failed. */ + @IgniteIgnore(value = "https://issues.apache.org/jira/browse/IGNITE-613", forceFailure = true) public void testNodeLeft() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-613"); - Ignite g = startGrid("client"); try { diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java index fd52469ac8fc4..64be936559d15 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java @@ -39,6 +39,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.testsuites.IgniteIgnore; /** * Tests for correct distributed partitioned queries. @@ -281,9 +282,8 @@ private static List column(int idx, List> rows) { /** * */ + @IgniteIgnore(value = "https://issues.apache.org/jira/browse/IGNITE-1886", forceFailure = true) public void testFunctionNpe() { - assert false : "https://issues.apache.org/jira/browse/IGNITE-1886"; - IgniteCache userCache = ignite(0).createCache( cacheConfig("UserCache", true, Integer.class, User.class)); IgniteCache userOrderCache = ignite(0).createCache( diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/BaseH2CompareQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/BaseH2CompareQueryTest.java index 46076a9c6d685..7cae5854a37f7 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/BaseH2CompareQueryTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/BaseH2CompareQueryTest.java @@ -37,6 +37,7 @@ import org.apache.ignite.cache.query.annotations.QuerySqlFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testsuites.IgniteIgnore; /** * Base set of queries to compare query results from h2 database instance and mixed ignite caches (replicated and partitioned) @@ -204,8 +205,8 @@ public void testInvalidQuery() throws Exception { /** * @throws Exception */ + @IgniteIgnore(value = "https://issues.apache.org/jira/browse/IGNITE-705", forceFailure = true) public void testAllExamples() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-705"); // compareQueryRes0("select ? limit ? offset ?"); // compareQueryRes0("select cool1()"); diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperCacheQueryTestSuite.java index 3eed9800ff6ae..109e244e0b5b7 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperCacheQueryTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinarySimpleNameMapperCacheQueryTestSuite.java @@ -18,7 +18,6 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; -import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.testframework.config.GridTestProperties; /** @@ -30,7 +29,6 @@ public class IgniteBinarySimpleNameMapperCacheQueryTestSuite extends TestSuite { * @throws Exception In case of error. */ public static TestSuite suite() throws Exception { - GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName()); GridTestProperties.setProperty(GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER, "true"); return IgniteBinaryCacheQueryTestSuite.suite(); diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java index 21741478f945e..d1f4c4ac6ebd3 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java @@ -65,6 +65,7 @@ import org.apache.ignite.internal.processors.query.IgniteSqlSchemaIndexingTest; import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest; import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest; +import org.apache.ignite.testframework.IgniteTestSuite; /** * Test suite for cache queries. @@ -75,7 +76,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite { * @throws Exception If failed. */ public static TestSuite suite() throws Exception { - TestSuite suite = new TestSuite("Ignite Cache Queries Test Suite"); + IgniteTestSuite suite = new IgniteTestSuite("Ignite Cache Queries Test Suite"); // Parsing suite.addTestSuite(GridQueryParsingTest.class); diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java index e5f4027a274ec..40fc15759d3f3 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java @@ -47,6 +47,7 @@ import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest; import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest; import org.apache.ignite.spi.communication.tcp.GridOrderedMessageCancelSelfTest; +import org.apache.ignite.testframework.IgniteTestSuite; /** * Test suite for cache queries. @@ -57,7 +58,7 @@ public class IgniteCacheQuerySelfTestSuite2 extends TestSuite { * @throws Exception If failed. */ public static TestSuite suite() throws Exception { - TestSuite suite = new TestSuite("Ignite Cache Queries Test Suite 2"); + TestSuite suite = new IgniteTestSuite("Ignite Cache Queries Test Suite 2"); // Scan queries. suite.addTestSuite(CacheScanPartitionQueryFallbackSelfTest.class); From 02f48d72364fb0c52e95aef8ed383a14ee531bf6 Mon Sep 17 00:00:00 2001 From: ptupitsyn Date: Tue, 4 Oct 2016 11:17:54 +0300 Subject: [PATCH 223/487] IGNITE-3820: .NET: Added log4net integration. This closes #1138. --- .../Apache.Ignite.Core.Tests.NuGet.csproj | 10 + .../Log4NetTest.cs | 87 ++++++++ .../packages.config | 2 + .../Apache.Ignite.Core.Tests.csproj | 9 + .../Log/Log4NetLoggerTest.cs | 188 ++++++++++++++++++ .../Apache.Ignite.Core.Tests/packages.config | 24 +++ .../Apache.Ignite.Log4Net.csproj | 76 +++++++ .../Apache.Ignite.Log4Net.nuspec | 50 +++++ .../Apache.Ignite.Log4Net.snk | Bin 0 -> 596 bytes .../IgniteLog4NetLogger.cs | 123 ++++++++++++ .../Properties/AssemblyInfo.cs | 40 ++++ .../Apache.Ignite.Log4Net/packages.config | 20 ++ .../Properties/AssemblyInfo.cs | 2 +- modules/platforms/dotnet/Apache.Ignite.sln | 16 +- 14 files changed, 645 insertions(+), 2 deletions(-) create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Log4NetTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/Log4NetLoggerTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/packages.config create mode 100644 modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj create mode 100644 modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.nuspec create mode 100644 modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.snk create mode 100644 modules/platforms/dotnet/Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Log4Net/packages.config diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Apache.Ignite.Core.Tests.NuGet.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Apache.Ignite.Core.Tests.NuGet.csproj index f3f917e9957a9..335d711743797 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Apache.Ignite.Core.Tests.NuGet.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Apache.Ignite.Core.Tests.NuGet.csproj @@ -90,6 +90,11 @@ packages\Apache.Ignite.NLog.1.8.0\lib\net40\Apache.Ignite.NLog.dll True + + False + packages\Apache.Ignite.Log4Net.1.8.0\lib\net40\Apache.Ignite.Log4Net.dll + True + packages\NLog.4.3.7\lib\net40\NLog.dll True @@ -105,6 +110,10 @@ packages\Remotion.Linq.2.0.1\lib\net40\Remotion.Linq.dll True + + packages\log4net.2.0.5\lib\net40-full\log4net.dll + True + @@ -112,6 +121,7 @@ + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Log4NetTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Log4NetTest.cs new file mode 100644 index 0000000000000..c53ea289ce120 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Log4NetTest.cs @@ -0,0 +1,87 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Tests.NuGet +{ + using System; + using System.Collections.Generic; + using System.Linq; + using Apache.Ignite.Core.Log; + using Apache.Ignite.Log4Net; + using global::log4net; + using global::log4net.Appender; + using global::log4net.Core; + using global::log4net.Repository.Hierarchy; + using NUnit.Framework; + + /// + /// log4net tests. + /// + public class Log4NetTest + { + /// + /// Tests the logger with Ignite. + /// + [Test] + public void TestIgniteStartup() + { + var memoryLog = CreateMemoryLogger(); + var logger = new IgniteLog4NetLogger(); + + var cfg = new IgniteConfiguration + { + DiscoverySpi = TestUtil.GetLocalDiscoverySpi(), + Logger = logger + }; + + Func> getLogs = () => memoryLog.GetEvents().Select(x => x.MessageObject.ToString()); + + using (var ignite = Ignition.Start(cfg)) + { + Assert.IsTrue(getLogs().Contains( + string.Format("Starting Ignite.NET {0}", typeof(Ignition).Assembly.GetName().Version))); + + Assert.IsTrue(getLogs().Any(x => x.Contains(">>> Topology snapshot."))); + + Assert.IsInstanceOf(ignite.Logger); + + ignite.Logger.Info("Log from user code."); + + Assert.IsTrue(getLogs().Contains("Log from user code.")); + } + + Assert.IsTrue(getLogs().Contains("Grid is stopping.")); + } + + /// + /// Creates the memory logger. + /// + private static MemoryAppender CreateMemoryLogger() + { + var hierarchy = (Hierarchy) LogManager.GetRepository(); + + var memory = new MemoryAppender(); + memory.ActivateOptions(); + hierarchy.Root.AddAppender(memory); + + hierarchy.Root.Level = Level.All; + hierarchy.Configured = true; + + return memory; + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/packages.config b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/packages.config index 30e71a1dfe081..80454e0e6e3f1 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/packages.config +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/packages.config @@ -20,8 +20,10 @@ + + \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj index ef62498f89c01..008229a8d197a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj @@ -37,6 +37,10 @@ MinimumRecommendedRules.ruleset + + ..\packages\log4net.2.0.5\lib\net40-full\log4net.dll + True + ..\packages\NLog.4.3.7\lib\net40\NLog.dll @@ -58,6 +62,7 @@ + @@ -180,6 +185,10 @@ {5b571661-17f4-4f29-8c7d-0edb38ca9b55} Apache.Ignite.Linq + + {6F82D669-382E-4435-8092-68C4440146D8} + Apache.Ignite.Log4Net + {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7} Apache.Ignite.NLog diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/Log4NetLoggerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/Log4NetLoggerTest.cs new file mode 100644 index 0000000000000..2b284397a21ed --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/Log4NetLoggerTest.cs @@ -0,0 +1,188 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Tests.Log +{ + using System; + using System.Collections.Generic; + using System.Globalization; + using System.Linq; + using Apache.Ignite.Core.Log; + using Apache.Ignite.Log4Net; + using global::log4net; + using global::log4net.Appender; + using global::log4net.Core; + using global::log4net.Repository.Hierarchy; + using NUnit.Framework; + + /// + /// Tests for . + /// + public class Log4NetLoggerTest + { + /// + /// Tests the log level conversion. + /// + [Test] + public void TestLogLevelConversion() + { + var levels = new[] { LogLevel.Trace, LogLevel.Info, LogLevel.Debug, LogLevel.Warn, LogLevel.Error }; + + foreach (var igniteLevel in levels) + { + var log4NetLevel = IgniteLog4NetLogger.ConvertLogLevel(igniteLevel); + + Assert.AreEqual(igniteLevel.ToString().ToUpperInvariant(), log4NetLevel.ToString()); + } + } + + /// + /// Tests the logger in isolated environment. + /// + [Test] + public void TestLogging() + { + var memoryLog = CreateMemoryLogger(); + var logger = new IgniteLog4NetLogger(); + + Func getLastLog = () => memoryLog.PopAllEvents().Single(); + + // All parameters. + logger.Log(LogLevel.Trace, "msg{0}", new object[] { 1 }, CultureInfo.InvariantCulture, "category", + "java-err", new Exception("myException")); + + var log = getLastLog(); + Assert.AreEqual("msg1", log.MessageObject.ToString()); + Assert.AreEqual("category", log.LoggerName); + Assert.AreEqual("java-err", log.Properties["nativeErrorInfo"]); + Assert.AreEqual("myException", log.ExceptionObject.Message); + Assert.AreEqual(Level.Trace, log.Level); + + // No Java error. + logger.Log(LogLevel.Info, "msg{0}", new object[] { 1 }, CultureInfo.InvariantCulture, "category", + null, new Exception("myException")); + + log = getLastLog(); + Assert.AreEqual("msg1", log.MessageObject.ToString()); + Assert.AreEqual("category", log.LoggerName); + Assert.AreEqual(null, log.Properties["nativeErrorInfo"]); + Assert.AreEqual("myException", log.ExceptionObject.Message); + Assert.AreEqual(Level.Info, log.Level); + + // No exception. + logger.Log(LogLevel.Debug, "msg{0}", new object[] { 1 }, CultureInfo.InvariantCulture, "category", + null, null); + + log = getLastLog(); + Assert.AreEqual("msg1", log.MessageObject.ToString()); + Assert.AreEqual("category", log.LoggerName); + Assert.AreEqual(null, log.Properties["nativeErrorInfo"]); + Assert.AreEqual(null, log.ExceptionObject); + Assert.AreEqual(Level.Debug, log.Level); + + // No params. + logger.Log(LogLevel.Warn, "msg{0}", null, CultureInfo.InvariantCulture, "category", null, null); + + log = getLastLog(); + Assert.AreEqual("msg{0}", log.MessageObject.ToString()); + Assert.AreEqual("category", log.LoggerName); + Assert.AreEqual(null, log.Properties["nativeErrorInfo"]); + Assert.AreEqual(null, log.ExceptionObject); + Assert.AreEqual(Level.Warn, log.Level); + + // No formatter. + logger.Log(LogLevel.Error, "msg{0}", null, null, "category", null, null); + + log = getLastLog(); + Assert.AreEqual("msg{0}", log.MessageObject.ToString()); + Assert.AreEqual("category", log.LoggerName); + Assert.AreEqual(null, log.Properties["nativeErrorInfo"]); + Assert.AreEqual(null, log.ExceptionObject); + Assert.AreEqual(Level.Error, log.Level); + + // No category. + logger.Log(LogLevel.Error, "msg{0}", null, null, null, null, null); + + log = getLastLog(); + Assert.AreEqual("msg{0}", log.MessageObject.ToString()); + Assert.AreEqual(null, log.LoggerName); + Assert.AreEqual(null, log.Properties["nativeErrorInfo"]); + Assert.AreEqual(null, log.ExceptionObject); + Assert.AreEqual(Level.Error, log.Level); + + // No message. + logger.Log(LogLevel.Error, null, null, null, null, null, null); + + log = getLastLog(); + Assert.AreEqual(null, log.MessageObject); + Assert.AreEqual(null, log.LoggerName); + Assert.AreEqual(null, log.Properties["nativeErrorInfo"]); + Assert.AreEqual(null, log.ExceptionObject); + Assert.AreEqual(Level.Error, log.Level); + } + + /// + /// Tests the logger with Ignite. + /// + [Test] + public void TestIgniteStartup() + { + var memoryLog = CreateMemoryLogger(); + var logger = new IgniteLog4NetLogger(); + + var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration()) + { + Logger = logger + }; + + Func> getLogs = () => memoryLog.GetEvents().Select(x => x.MessageObject.ToString()); + + using (var ignite = Ignition.Start(cfg)) + { + Assert.IsTrue(getLogs().Contains( + string.Format("Starting Ignite.NET {0}", typeof(Ignition).Assembly.GetName().Version))); + + Assert.IsTrue(getLogs().Any(x => x.Contains(">>> Topology snapshot."))); + + Assert.IsInstanceOf(ignite.Logger); + + ignite.Logger.Info("Log from user code."); + + Assert.IsTrue(getLogs().Contains("Log from user code.")); + } + + Assert.IsTrue(getLogs().Contains("Grid is stopping.")); + } + + /// + /// Creates the memory logger. + /// + private static MemoryAppender CreateMemoryLogger() + { + var hierarchy = (Hierarchy) LogManager.GetRepository(); + + var memory = new MemoryAppender(); + memory.ActivateOptions(); + hierarchy.Root.AddAppender(memory); + + hierarchy.Root.Level = Level.All; + hierarchy.Configured = true; + + return memory; + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/packages.config b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/packages.config new file mode 100644 index 0000000000000..d369a350c0a47 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/packages.config @@ -0,0 +1,24 @@ + + + + + + + + + \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj new file mode 100644 index 0000000000000..0fdd6116bf3d9 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj @@ -0,0 +1,76 @@ + + + + + Debug + AnyCPU + {6F82D669-382E-4435-8092-68C4440146D8} + Library + Properties + Apache.Ignite.Log4Net + Apache.Ignite.Log4Net + v4.0 + 512 + + + true + full + false + bin\Debug\ + DEBUG;TRACE + prompt + 4 + true + AllRules.ruleset + + + pdbonly + true + bin\Release\ + TRACE + prompt + 4 + + + true + + + Apache.Ignite.Log4Net.snk + + + + ..\packages\log4net.2.0.5\lib\net40-full\log4net.dll + True + + + + + + + + + + + + + + + + {4CD2F726-7E2B-46C4-A5BA-057BB82EECB6} + Apache.Ignite.Core + + + + + + + + + + \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.nuspec b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.nuspec new file mode 100644 index 0000000000000..fa5c39a49098c --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.nuspec @@ -0,0 +1,50 @@ + + + + + + + + + Apache.Ignite.Log4Net + Apache Ignite log4net Logger + + $version$ + Apache Ignite + Apache Software Foundation + http://www.apache.org/licenses/LICENSE-2.0 + https://ignite.apache.org/ + https://ignite.apache.org/images/logo_ignite_32_32.png + false + log4net Logger for Apache Ignite + + Copyright 2016 + Apache Ignite In-Memory Distributed Computing SQL NoSQL LINQ Grid Map Reduce Cache log4net logger + + + + + + \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.snk b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.snk new file mode 100644 index 0000000000000000000000000000000000000000..a4d1622c4b925bc6255c54b5c448316460b24af6 GIT binary patch literal 596 zcmV-a0;~N80ssI2Bme+XQ$aES1ONa50096QZ{{%G=LsspSRHGy#KM_TM&pSrW~ury zcLc6ogiFe90_1ur2MlaHs|u4nN{1(fCqg<_v#Du@vtih*y`yQ1#uut?avG`gbm-t6 zP1V@Lq6v~iwy#PPI3;+PQN+I6I6|MBobx!rMRj{E?)^Sj&!7rZXkG_v7#m7RwYZkW za!{3C14!yWTa%%rHbfOH;W3WAQ#@F=@K+-^^fM}{x>(T@uRIiPqGOuc&(_D{^%k7b zOxU&4`I0zPj8?iufYDCs2Sa5SLXdsB@OR}flw26t@ zCP-YcZnHrgRc13sSi#%^EL2rj;q-2ds&1v7go9Wv1+$|0)?`H^7N(>IWl^0ebnZrh zFH-Zh-C!$#twbOA(OU~L;M4oV>l^<@Na5b1p62$t<_^oS7cDb45^AO7d$^SDc+}iI zO9wzoxhq#D;aeY*6Rv6z)GWIF-Q&-r@D?7UosIq%5*Z^S-ruo_Pd5MzCg`1(g(R(F zw&<|FfQnR&1OXM%AZle07}iUo@}@+yZ#Zd8K(Q7p`z%V7Rx6dGRu literal 0 HcmV?d00001 diff --git a/modules/platforms/dotnet/Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs b/modules/platforms/dotnet/Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs new file mode 100644 index 0000000000000..058176ff60134 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs @@ -0,0 +1,123 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Log4Net +{ + using System; + using Apache.Ignite.Core.Impl.Common; + using Apache.Ignite.Core.Log; + using global::log4net; + using global::log4net.Core; + using global::log4net.Util; + using ILogger = Apache.Ignite.Core.Log.ILogger; + + /// + /// Ignite log4net integration. + /// + public class IgniteLog4NetLogger : ILogger + { + /** Wrapped log4net log. */ + private readonly ILog _log; + + /// + /// Initializes a new instance of the class. + /// + public IgniteLog4NetLogger() : this (LogManager.GetLogger(typeof(IgniteLog4NetLogger))) + { + // No-op. + } + + /// + /// Initializes a new instance of the class. + /// + /// The log. + public IgniteLog4NetLogger(ILog log) + { + IgniteArgumentCheck.NotNull(log, "log"); + + _log = log; + } + + /// + /// Logs the specified message. + /// + /// The level. + /// The message. + /// The arguments to format . + /// Can be null (formatting will not occur). + /// The format provider. Can be null if is null. + /// The logging category name. + /// The native error information. + /// The exception. Can be null. + public void Log(LogLevel logLevel, string message, object[] args, IFormatProvider formatProvider, + string category, string nativeErrorInfo, Exception ex) + { + var level = ConvertLogLevel(logLevel); + + var repo = _log.Logger.Repository; + + var messageObject = args == null + ? (object) message + : new SystemStringFormat(formatProvider, message, args); + + var evt = new LoggingEvent(GetType(), repo, category, level, messageObject, ex); + + if (nativeErrorInfo != null) + evt.Properties["nativeErrorInfo"] = nativeErrorInfo; + + _log.Logger.Log(evt); + } + + /// + /// Determines whether the specified log level is enabled. + /// + /// The level. + /// + /// Value indicating whether the specified log level is enabled + /// + public bool IsEnabled(LogLevel logLevel) + { + var level = ConvertLogLevel(logLevel); + + return _log.Logger.IsEnabledFor(level); + } + + /// + /// Converts the Ignite LogLevel to the log4net log level. + /// + /// The Ignite log level. + /// Corresponding log4net log level. + public static Level ConvertLogLevel(LogLevel level) + { + switch (level) + { + case LogLevel.Trace: + return Level.Trace; + case LogLevel.Debug: + return Level.Debug; + case LogLevel.Info: + return Level.Info; + case LogLevel.Warn: + return Level.Warn; + case LogLevel.Error: + return Level.Error; + default: + throw new ArgumentOutOfRangeException("level", level, null); + } + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs new file mode 100644 index 0000000000000..bb8e830dbf864 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs @@ -0,0 +1,40 @@ +/* +* 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. +*/ + +using System; +using System.Reflection; +using System.Runtime.InteropServices; + +[assembly: AssemblyTitle("Apache.Ignite.Log4Net")] +[assembly: AssemblyDescription("Apache Ignite.NET log4net integration.")] +[assembly: AssemblyConfiguration("")] +[assembly: AssemblyCompany("Apache Software Foundation")] +[assembly: AssemblyProduct("Apache Ignite.NET")] +[assembly: AssemblyCopyright("Copyright © 2016")] +[assembly: AssemblyTrademark("")] +[assembly: AssemblyCulture("")] + +[assembly: ComVisible(false)] + +// The following GUID is for the ID of the typelib if this project is exposed to COM +[assembly: Guid("6f82d669-382e-4435-8092-68c4440146d8")] + +[assembly: AssemblyVersion("1.8.0.14218")] +[assembly: AssemblyFileVersion("1.8.0.14218")] +[assembly: AssemblyInformationalVersion("1.8.0")] + +[assembly: CLSCompliant(true)] \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Log4Net/packages.config b/modules/platforms/dotnet/Apache.Ignite.Log4Net/packages.config new file mode 100644 index 0000000000000..e1e44b8bf6856 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Log4Net/packages.config @@ -0,0 +1,20 @@ + + + + + \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs index baaa60d94c8e7..50220d205321d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs @@ -24,7 +24,7 @@ [assembly: AssemblyConfiguration("")] [assembly: AssemblyCompany("Apache Software Foundation")] [assembly: AssemblyProduct("Apache Ignite.NET")] -[assembly: AssemblyCopyright("Copyright © 2015")] +[assembly: AssemblyCopyright("Copyright © 2016")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] diff --git a/modules/platforms/dotnet/Apache.Ignite.sln b/modules/platforms/dotnet/Apache.Ignite.sln index 2978780877bd4..de7cf19f7d408 100644 --- a/modules/platforms/dotnet/Apache.Ignite.sln +++ b/modules/platforms/dotnet/Apache.Ignite.sln @@ -36,9 +36,11 @@ Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.Linq", "Apach EndProject Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.AspNet", "Apache.Ignite.AspNet\Apache.Ignite.AspNet.csproj", "{13EA96FC-CC83-4164-A7C0-4F30ED797460}" EndProject +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.NLog", "Apache.Ignite.NLog\Apache.Ignite.NLog.csproj", "{C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}" +EndProject Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.AspNet.Tests", "Apache.Ignite.AspNet.Tests\Apache.Ignite.AspNet.Tests.csproj", "{18EA4C71-A11D-4AB1-8042-418F7559D84F}" EndProject -Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.NLog", "Apache.Ignite.NLog\Apache.Ignite.NLog.csproj", "{C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}" +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.Log4Net", "Apache.Ignite.log4net\Apache.Ignite.Log4Net.csproj", "{6F82D669-382E-4435-8092-68C4440146D8}" EndProject Global GlobalSection(SolutionConfigurationPlatforms) = preSolution @@ -202,6 +204,18 @@ Global {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Release|x64.Build.0 = Release|Any CPU {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Release|x86.ActiveCfg = Release|Any CPU {C6B58E4A-A2E9-4554-AD02-68CE6DA5CFB7}.Release|x86.Build.0 = Release|Any CPU + {6F82D669-382E-4435-8092-68C4440146D8}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {6F82D669-382E-4435-8092-68C4440146D8}.Debug|Any CPU.Build.0 = Debug|Any CPU + {6F82D669-382E-4435-8092-68C4440146D8}.Debug|x64.ActiveCfg = Debug|Any CPU + {6F82D669-382E-4435-8092-68C4440146D8}.Debug|x64.Build.0 = Debug|Any CPU + {6F82D669-382E-4435-8092-68C4440146D8}.Debug|x86.ActiveCfg = Debug|Any CPU + {6F82D669-382E-4435-8092-68C4440146D8}.Debug|x86.Build.0 = Debug|Any CPU + {6F82D669-382E-4435-8092-68C4440146D8}.Release|Any CPU.ActiveCfg = Release|Any CPU + {6F82D669-382E-4435-8092-68C4440146D8}.Release|Any CPU.Build.0 = Release|Any CPU + {6F82D669-382E-4435-8092-68C4440146D8}.Release|x64.ActiveCfg = Release|Any CPU + {6F82D669-382E-4435-8092-68C4440146D8}.Release|x64.Build.0 = Release|Any CPU + {6F82D669-382E-4435-8092-68C4440146D8}.Release|x86.ActiveCfg = Release|Any CPU + {6F82D669-382E-4435-8092-68C4440146D8}.Release|x86.Build.0 = Release|Any CPU EndGlobalSection GlobalSection(SolutionProperties) = preSolution HideSolutionNode = FALSE From bfdb5c3b374fd3512481cf16779d227d7f96e569 Mon Sep 17 00:00:00 2001 From: Saikat Maitra Date: Tue, 4 Oct 2016 16:40:35 +0700 Subject: [PATCH 224/487] IGNITE-3841 Web console added check for eviction policy max mem and max size consistency. Fixes #1136. --- .../frontend/controllers/caches-controller.js | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/modules/web-console/frontend/controllers/caches-controller.js b/modules/web-console/frontend/controllers/caches-controller.js index 98730515c3444..8c3290694ee88 100644 --- a/modules/web-console/frontend/controllers/caches-controller.js +++ b/modules/web-console/frontend/controllers/caches-controller.js @@ -292,6 +292,17 @@ export default ['cachesController', [ return true; } + function checkEvictionPolicy(evictionPlc) { + if (evictionPlc && evictionPlc.kind) { + const plc = evictionPlc[evictionPlc.kind]; + + if (plc.maxMemorySize === 0 && plc.maxSize === 0) + return ErrorPopover.show('evictionPolicymaxMemorySizeInput', 'Either maximum memory size or maximum size should be great than 0!', $scope.ui, 'memory'); + } + + return true; + } + function checkSQLSchemas() { const clusters = cacheClusters(); @@ -367,6 +378,9 @@ export default ['cachesController', [ if (item.memoryMode === 'OFFHEAP_TIERED' && item.offHeapMaxMemory === -1) return ErrorPopover.show('offHeapModeInput', 'Invalid value!', $scope.ui, 'memory'); + if (!checkEvictionPolicy(item.evictionPolicy)) + return false; + if (!checkSQLSchemas()) return false; From a92f20b5cc75e6b80b2731da0192723526b0c1dc Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Tue, 4 Oct 2016 14:10:26 +0300 Subject: [PATCH 225/487] IGNITE-3597: Removed static work directory. --- .../internal/GridKernalContextImpl.java | 4 +- .../apache/ignite/internal/IgnitionEx.java | 15 +- .../internal/MarshallerContextImpl.java | 5 +- .../processors/hadoop/HadoopHelper.java | 7 + .../internal/processors/hadoop/HadoopJob.java | 5 + .../processors/hadoop/HadoopNoopHelper.java | 5 + .../internal/processors/igfs/IgfsServer.java | 5 +- .../cpp/PlatformCppConfigurationClosure.java | 13 +- .../PlatformDotNetConfigurationClosure.java | 13 +- .../platform/utils/PlatformUtils.java | 8 +- .../ignite/internal/util/IgniteUtils.java | 92 +++++----- .../shmem/IpcSharedMemoryServerEndpoint.java | 15 +- .../apache/ignite/logger/java/JavaLogger.java | 14 +- .../logger/java/JavaLoggerFileHandler.java | 10 +- .../sharedfs/SharedFsCheckpointSpi.java | 3 +- .../tcp/TcpCommunicationSpi.java | 4 +- .../TcpDiscoverySharedFsIpFinder.java | 2 +- .../spi/swapspace/file/FileSwapSpaceSpi.java | 2 +- .../MarshallerContextLockingSelfTest.java | 2 +- .../MarshallerCacheJobRunNodeRestartTest.java | 2 +- .../log/GridLogCommandHandlerTest.java | 5 +- ...tupWithSpecifiedWorkDirectorySelfTest.java | 166 ------------------ .../ipc/shmem/IgfsSharedMemoryTestServer.java | 4 +- ...IpcSharedMemoryCrashDetectionSelfTest.java | 9 +- .../IpcSharedMemoryBenchmarkReader.java | 2 +- .../ignite/logger/java/JavaLoggerTest.java | 3 +- .../OptimizedMarshallerNodeFailoverTest.java | 4 - ...ckpointSpiMultipleDirectoriesSelfTest.java | 6 +- .../GridAbstractCommunicationSelfTest.java | 2 - ...CommunicationSpiMultithreadedSelfTest.java | 2 - .../GridSwapSpaceSpiAbstractSelfTest.java | 2 - .../junits/GridAbstractTest.java | 2 +- .../testframework/junits/IgniteMock.java | 10 ++ .../junits/IgniteTestResources.java | 2 - .../junits/spi/GridSpiAbstractTest.java | 2 - .../IgniteBinaryBasicTestSuite.java | 2 - .../testsuites/IgniteKernalSelfTestSuite.java | 2 - .../CacheNoValueClassOnServerTestClient.java | 2 - .../CacheConfigurationP2PTestClient.java | 2 - .../processors/hadoop/HadoopHelperImpl.java | 13 ++ .../processors/hadoop/impl/HadoopUtils.java | 11 +- .../hadoop/impl/v2/HadoopV2Job.java | 13 +- .../hadoop/impl/v2/HadoopV2TaskContext.java | 4 +- .../external/HadoopExternalTaskExecutor.java | 13 +- .../child/HadoopExternalProcessStarter.java | 10 +- .../HadoopExternalCommunication.java | 10 +- .../hadoop/impl/HadoopPlannerMockJob.java | 7 + .../HadoopExternalCommunicationSelfTest.java | 2 +- .../CacheConfigurationP2PTestServer.java | 2 - .../IgniteConfigurationTest.cs | 2 +- .../spi/deployment/uri/UriDeploymentSpi.java | 2 +- .../scala/org/apache/ignite/visor/visor.scala | 2 +- 52 files changed, 209 insertions(+), 337 deletions(-) delete mode 100644 modules/core/src/test/java/org/apache/ignite/internal/util/GridStartupWithSpecifiedWorkDirectorySelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java index eb214e884b5ff..c7e26e9d70619 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java @@ -408,7 +408,9 @@ protected GridKernalContextImpl( this.restExecSvc = restExecSvc; this.callbackExecSvc = callbackExecSvc; - marshCtx = new MarshallerContextImpl(plugins); + String workDir = U.workDirectory(cfg.getWorkDirectory(), cfg.getIgniteHome()); + + marshCtx = new MarshallerContextImpl(workDir, plugins); try { spring = SPRING.create(false); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index aad96fbc2056f..2914c7cf867f7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -1823,7 +1823,10 @@ private IgniteConfiguration initializeConfiguration(IgniteConfiguration cfg) // If user provided IGNITE_HOME - set it as a system property. U.setIgniteHome(ggHome); - U.setWorkDirectory(cfg.getWorkDirectory(), ggHome); + // Correctly resolve work directory and set it back to configuration. + String workDir = U.workDirectory(cfg.getWorkDirectory(), ggHome); + + myCfg.setWorkDirectory(workDir); // Ensure invariant. // It's a bit dirty - but this is a result of late refactoring @@ -1834,7 +1837,7 @@ private IgniteConfiguration initializeConfiguration(IgniteConfiguration cfg) myCfg.setNodeId(nodeId); - IgniteLogger cfgLog = initLogger(cfg.getGridLogger(), nodeId); + IgniteLogger cfgLog = initLogger(cfg.getGridLogger(), nodeId, workDir); assert cfgLog != null; @@ -2095,11 +2098,13 @@ private void initializeDefaultSpi(IgniteConfiguration cfg) { /** * @param cfgLog Configured logger. * @param nodeId Local node ID. + * @param workDir Work directory. * @return Initialized logger. * @throws IgniteCheckedException If failed. */ @SuppressWarnings("ErrorNotRethrown") - private IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId) throws IgniteCheckedException { + private IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId, String workDir) + throws IgniteCheckedException { try { Exception log4jInitErr = null; @@ -2157,6 +2162,10 @@ private IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId) thro cfgLog = new JavaLogger(); } + // Special handling for Java logger which requires work directory. + if (cfgLog instanceof JavaLogger) + ((JavaLogger)cfgLog).setWorkDirectory(workDir); + // Set node IDs for all file appenders. if (cfgLog instanceof LoggerNodeIdAware) ((LoggerNodeIdAware)cfgLog).setNodeId(nodeId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java index 0420e184f6b25..e501d2782de39 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java @@ -72,13 +72,14 @@ public class MarshallerContextImpl extends MarshallerContextAdapter { private ContinuousQueryListener lsnr; /** + * @param igniteWorkDir Ignite work directory. * @param plugins Plugins. * @throws IgniteCheckedException In case of error. */ - public MarshallerContextImpl(List plugins) throws IgniteCheckedException { + public MarshallerContextImpl(String igniteWorkDir, List plugins) throws IgniteCheckedException { super(plugins); - workDir = U.resolveWorkDirectory("marshaller", false); + workDir = U.resolveWorkDirectory(igniteWorkDir, "marshaller", false); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelper.java index a8fee79d3380e..7936fef92ea6f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelper.java @@ -52,4 +52,11 @@ public interface HadoopHelper { * @return Input stream. */ @Nullable public InputStream loadClassBytes(ClassLoader ldr, String clsName); + + /** + * Get work directory. + * + * @return Work directory. + */ + public String workDirectory(); } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java index 2f6f6a7fd5b79..a77c7448dd5a7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java @@ -99,4 +99,9 @@ public interface HadoopJob { * Cleans up the job staging directory. */ public void cleanupStagingDirectory(); + + /** + * @return Ignite work directory. + */ + public String igniteWorkDirectory(); } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopHelper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopHelper.java index d3348caae33f4..f8f870fabf858 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopHelper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopNoopHelper.java @@ -57,6 +57,11 @@ public HadoopNoopHelper(GridKernalContext ctx) { throw unsupported(); } + /** {@inheritDoc} */ + @Override public String workDirectory() { + throw unsupported(); + } + /** * @return Exception. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServer.java index aa4b115e32c56..b2c3c83c96510 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServer.java @@ -153,7 +153,7 @@ public void start() throws IgniteCheckedException { * @return Server endpoint. * @throws IgniteCheckedException If failed. */ - private static IpcServerEndpoint createEndpoint(IgfsIpcEndpointConfiguration endpointCfg, boolean mgmt) + private IpcServerEndpoint createEndpoint(IgfsIpcEndpointConfiguration endpointCfg, boolean mgmt) throws IgniteCheckedException { A.notNull(endpointCfg, "endpointCfg"); @@ -164,7 +164,8 @@ private static IpcServerEndpoint createEndpoint(IgfsIpcEndpointConfiguration end switch (typ) { case SHMEM: { - IpcSharedMemoryServerEndpoint endpoint = new IpcSharedMemoryServerEndpoint(); + IpcSharedMemoryServerEndpoint endpoint = + new IpcSharedMemoryServerEndpoint(igfsCtx.kernalContext().config().getWorkDirectory()); endpoint.setPort(endpointCfg.getPort()); endpoint.setSize(endpointCfg.getMemorySize()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java index 2e60025e378e9..f6711d272e5cf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal.processors.platform.cpp; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.binary.BinaryIdMapper; import org.apache.ignite.binary.BinaryBasicIdMapper; @@ -130,17 +129,7 @@ else if (!(marsh instanceof BinaryMarshaller)) // Set Ignite home so that marshaller context works. String ggHome = igniteCfg.getIgniteHome(); - if (ggHome == null) - ggHome = U.getIgniteHome(); - else - // If user provided IGNITE_HOME - set it as a system property. + if (ggHome != null) U.setIgniteHome(ggHome); - - try { - U.setWorkDirectory(igniteCfg.getWorkDirectory(), ggHome); - } - catch (IgniteCheckedException e) { - throw U.convertException(e); - } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java index 5a7d15d1d6ea6..ed40be51cdd04 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal.processors.platform.dotnet; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.binary.BinaryBasicIdMapper; import org.apache.ignite.binary.BinaryBasicNameMapper; @@ -151,19 +150,9 @@ else if (!(marsh instanceof BinaryMarshaller)) // Set Ignite home so that marshaller context works. String ggHome = igniteCfg.getIgniteHome(); - if (ggHome == null) - ggHome = U.getIgniteHome(); - else - // If user provided IGNITE_HOME - set it as a system property. + if (ggHome != null) U.setIgniteHome(ggHome); - try { - U.setWorkDirectory(igniteCfg.getWorkDirectory(), ggHome); - } - catch (IgniteCheckedException e) { - throw U.convertException(e); - } - // 4. Callback to .Net. prepare(igniteCfg, dotNetCfg0); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java index b31bbd365f425..ec025c00fb0b5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java @@ -786,12 +786,16 @@ public static Object readInvocationResult(PlatformContext ctx, BinaryRawReaderEx @SuppressWarnings("deprecation") public static GridBinaryMarshaller marshaller() { try { + IgniteConfiguration cfg = new IgniteConfiguration(); + BinaryContext ctx = - new BinaryContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration(), new NullLogger()); + new BinaryContext(BinaryNoopMetadataHandler.instance(), cfg, new NullLogger()); BinaryMarshaller marsh = new BinaryMarshaller(); - marsh.setContext(new MarshallerContextImpl(null)); + String workDir = U.workDirectory(cfg.getWorkDirectory(), cfg.getIgniteHome()); + + marsh.setContext(new MarshallerContextImpl(workDir, null)); ctx.configure(marsh, new IgniteConfiguration()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 501cdb2eaba49..f07266bdef7ed 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -305,9 +305,6 @@ public abstract class IgniteUtils { /** Project home directory. */ private static volatile GridTuple ggHome; - /** Project work directory. */ - private static volatile String igniteWork; - /** OS JDK string. */ private static String osJdkStr; @@ -8826,54 +8823,56 @@ public static String addressesAsString(Collection addrs, Collection= 0xffff) throw new IpcEndpointBindException("Port value is illegal: " + port); diff --git a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java index 4214b5b98f52e..d5ff5e388d130 100644 --- a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java +++ b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java @@ -113,6 +113,9 @@ public class JavaLogger implements IgniteLogger, LoggerNodeIdAware { /** Quiet flag. */ private final boolean quiet; + /** Work directory. */ + private volatile String workDir; + /** Node ID. */ private volatile UUID nodeId; @@ -340,6 +343,15 @@ private void configure(@Nullable Logger initImpl) { } } + /** + * Set work directory. + * + * @param workDir Work directory. + */ + public void setWorkDirectory(String workDir) { + this.workDir = workDir; + } + /** {@inheritDoc} */ @Override public void setNodeId(UUID nodeId) { A.notNull(nodeId, "nodeId"); @@ -361,7 +373,7 @@ private void configure(@Nullable Logger initImpl) { return; try { - fileHnd.nodeId(nodeId); + fileHnd.nodeId(nodeId, workDir); } catch (IgniteCheckedException | IOException e) { throw new RuntimeException("Failed to enable file handler.", e); diff --git a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java index 5acdc0891c78a..7d1fa08d3c62c 100644 --- a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLoggerFileHandler.java @@ -80,7 +80,7 @@ public final class JavaLoggerFileHandler extends StreamHandler { * * @param nodeId Node id. */ - public void nodeId(UUID nodeId) throws IgniteCheckedException, IOException { + public void nodeId(UUID nodeId, String workDir) throws IgniteCheckedException, IOException { if (delegate != null) return; @@ -91,7 +91,7 @@ public void nodeId(UUID nodeId) throws IgniteCheckedException, IOException { if (ptrn == null) ptrn = "ignite-%{id8}.%g.log"; - ptrn = new File(logDirectory(), ptrn.replace("%{id8}", U.id8(nodeId))).getAbsolutePath(); + ptrn = new File(logDirectory(workDir), ptrn.replace("%{id8}", U.id8(nodeId))).getAbsolutePath(); int limit = getIntProperty(clsName + ".limit", 0); @@ -133,10 +133,12 @@ public void nodeId(UUID nodeId) throws IgniteCheckedException, IOException { /** * Resolves logging directory. * + * @param workDir Work directory. * @return Logging directory. */ - private static File logDirectory() throws IgniteCheckedException { - return !F.isEmpty(U.IGNITE_LOG_DIR) ? new File(U.IGNITE_LOG_DIR) : U.resolveWorkDirectory("log", false); + private static File logDirectory(String workDir) throws IgniteCheckedException { + return !F.isEmpty(U.IGNITE_LOG_DIR) ? new File(U.IGNITE_LOG_DIR) : + U.resolveWorkDirectory(workDir, "log", false); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java index 092c864947177..21011a7cb4788 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java @@ -36,7 +36,6 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.MarshallerUtils; -import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.LoggerResource; import org.apache.ignite.spi.IgniteSpiAdapter; @@ -291,7 +290,7 @@ public void setDirectoryPaths(Collection dirPaths) { folder = new File(curDirPath); else { try { - folder = U.resolveWorkDirectory(curDirPath, false); + folder = U.resolveWorkDirectory(ignite.configuration().getWorkDirectory(), curDirPath, false); } catch (IgniteCheckedException e) { if (log.isDebugEnabled()) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index d81b9f3397aa2..d61cee6fdaa8e 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -1779,8 +1779,10 @@ private GridNioServer resetNioServer() throws IgniteCheckedException { // If configured TCP port is busy, find first available in range. for (int port = shmemPort; port < shmemPort + locPortRange; port++) { try { + IgniteConfiguration cfg = ignite.configuration(); + IpcSharedMemoryServerEndpoint srv = - new IpcSharedMemoryServerEndpoint(log, ignite.configuration().getNodeId(), gridName); + new IpcSharedMemoryServerEndpoint(log, cfg.getNodeId(), gridName, cfg.getWorkDirectory()); srv.setPort(port); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/sharedfs/TcpDiscoverySharedFsIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/sharedfs/TcpDiscoverySharedFsIpFinder.java index bcd9513e8de93..d4e93d2de948f 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/sharedfs/TcpDiscoverySharedFsIpFinder.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/sharedfs/TcpDiscoverySharedFsIpFinder.java @@ -140,7 +140,7 @@ private File initFolder() throws IgniteSpiException { tmp = new File(path); else { try { - tmp = U.resolveWorkDirectory(path, false); + tmp = U.resolveWorkDirectory(ignite.configuration().getWorkDirectory(), path, false); } catch (IgniteCheckedException e) { throw new IgniteSpiException("Failed to resolve directory [path=" + path + diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java index 9be5b93cc6789..222f463c82588 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java @@ -299,7 +299,7 @@ public void setReadStripesNumber(int readStripesNum) { String path = baseDir + File.separator + gridName + File.separator + ignite.configuration().getNodeId(); try { - dir = U.resolveWorkDirectory(path, true); + dir = U.resolveWorkDirectory(ignite.configuration().getWorkDirectory(), path, true); } catch (IgniteCheckedException e) { throw new IgniteSpiException(e); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/MarshallerContextLockingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/MarshallerContextLockingSelfTest.java index e549336d49b08..bf0adf8d8f0cf 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/MarshallerContextLockingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/MarshallerContextLockingSelfTest.java @@ -90,7 +90,7 @@ public static class InternalExecutor { public void executeTest(GridTestLog4jLogger log) throws Exception { counter.incrementAndGet(); - File workDir = U.resolveWorkDirectory("marshaller", false); + File workDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "marshaller", false); final MarshallerContextImpl.ContinuousQueryListener queryListener = new MarshallerContextImpl.ContinuousQueryListener(log, workDir); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MarshallerCacheJobRunNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MarshallerCacheJobRunNodeRestartTest.java index c7aecb40117ae..2b7922ea4fd5f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MarshallerCacheJobRunNodeRestartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MarshallerCacheJobRunNodeRestartTest.java @@ -57,7 +57,7 @@ public class MarshallerCacheJobRunNodeRestartTest extends GridCommonAbstractTest */ public void testJobRun() throws Exception { for (int i = 0; i < 5; i++) { - U.resolveWorkDirectory("marshaller", true); + U.resolveWorkDirectory(U.defaultWorkDirectory(), "marshaller", true); log.info("Iteration: " + i); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/log/GridLogCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/log/GridLogCommandHandlerTest.java index 7b0e950589329..1fb093a1b4076 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/log/GridLogCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/log/GridLogCommandHandlerTest.java @@ -61,12 +61,9 @@ public class GridLogCommandHandlerTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void afterTestsStopped() throws Exception { - Path file = Paths.get(igniteHome + "/work/log/" + "test.log"); - Files.delete(file); - + Files.delete(Paths.get(igniteHome + "/work/log/" + "test.log")); Files.delete(Paths.get(igniteHome + "/work/log/" + "ignite.log")); Files.delete(Paths.get(igniteHome + "/work/log/")); - Files.delete(Paths.get(igniteHome + "/work/")); super.afterTestsStopped(); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/GridStartupWithSpecifiedWorkDirectorySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/GridStartupWithSpecifiedWorkDirectorySelfTest.java deleted file mode 100644 index 3eb0ebad24542..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/GridStartupWithSpecifiedWorkDirectorySelfTest.java +++ /dev/null @@ -1,166 +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.util; - -import java.io.File; -import java.util.UUID; -import junit.framework.TestCase; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteLogger; -import org.apache.ignite.IgniteSystemProperties; -import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.util.typedef.G; -import org.apache.ignite.internal.util.typedef.X; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.logger.java.JavaLogger; -import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; -import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; -import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; - -import static org.apache.ignite.IgniteSystemProperties.IGNITE_HOME; -import static org.apache.ignite.internal.util.IgniteUtils.nullifyHomeDirectory; -import static org.apache.ignite.internal.util.IgniteUtils.nullifyWorkDirectory; - -/** - * Checks creation of work folder. - */ -public class GridStartupWithSpecifiedWorkDirectorySelfTest extends TestCase { - /** */ - private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); - - /** */ - private static final int GRID_COUNT = 2; - - /** System temp directory. */ - private static final String TMP_DIR = System.getProperty("java.io.tmpdir"); - - /** {@inheritDoc} */ - @Override protected void setUp() throws Exception { - // Protection against previously cached values. - nullifyHomeDirectory(); - nullifyWorkDirectory(); - } - - /** {@inheritDoc} */ - @Override protected void tearDown() throws Exception { - // Next grid in the same VM shouldn't use cached values produced by these tests. - nullifyHomeDirectory(); - nullifyWorkDirectory(); - - U.setWorkDirectory(null, U.getIgniteHome()); - } - - /** - * @param log Grid logger. - * @return Grid configuration. - */ - private IgniteConfiguration getConfiguration(IgniteLogger log) { - // We can't use U.getIgniteHome() here because - // it will initialize cached value which is forbidden to override. - String ggHome = IgniteSystemProperties.getString(IGNITE_HOME); - - assert ggHome != null; - - U.setIgniteHome(null); - - String ggHome0 = U.getIgniteHome(); - - assert ggHome0 == null; - - TcpDiscoverySpi disc = new TcpDiscoverySpi(); - - disc.setIpFinder(IP_FINDER); - - IgniteConfiguration cfg = new IgniteConfiguration(); - - cfg.setGridLogger(log); - cfg.setDiscoverySpi(disc); - - return cfg; - } - - /** - * @throws Exception If failed. - */ - public void testStartStopWithUndefinedHomeAndWorkDirs() throws Exception { - IgniteLogger log = new JavaLogger(); - - log.info(">>> Test started: " + getName()); - log.info("Grid start-stop test count: " + GRID_COUNT); - - File testWorkDir = null; - - try { - for (int i = 0; i < GRID_COUNT; i++) { - try (Ignite g = G.start(getConfiguration(log))) { - assert g != null; - - testWorkDir = U.resolveWorkDirectory(getName(), true); - - assertTrue("Work directory wasn't created", testWorkDir.exists()); - - assertTrue("Work directory must be located in OS temp directory", - testWorkDir.getAbsolutePath().startsWith(TMP_DIR)); - - System.out.println(testWorkDir); - - X.println("Stopping grid " + g.cluster().localNode().id()); - } - } - } - finally { - if (testWorkDir != null && testWorkDir.getAbsolutePath().startsWith(TMP_DIR)) - U.delete(testWorkDir); - } - } - - /** - * @throws Exception If failed. - */ - public void testStartStopWithUndefinedHomeAndConfiguredWorkDirs() throws Exception { - IgniteLogger log = new JavaLogger(); - - log.info(">>> Test started: " + getName()); - log.info("Grid start-stop test count: " + GRID_COUNT); - - String tmpWorkDir = new File(TMP_DIR, getName() + "_" + UUID.randomUUID()).getAbsolutePath(); - - try { - for (int i = 0; i < GRID_COUNT; i++) { - IgniteConfiguration cfg = getConfiguration(log); - - cfg.setWorkDirectory(tmpWorkDir); - - try (Ignite g = G.start(cfg)) { - assert g != null; - - File testWorkDir = U.resolveWorkDirectory(getName(), true); - - assertTrue("Work directory wasn't created", testWorkDir.exists()); - - assertTrue("Work directory must be located in configured directory", - testWorkDir.getAbsolutePath().startsWith(tmpWorkDir)); - - X.println("Stopping grid " + g.cluster().localNode().id()); - } - } - } finally { - U.delete(new File(tmpWorkDir)); - } - } -} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IgfsSharedMemoryTestServer.java b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IgfsSharedMemoryTestServer.java index 28737081908b3..a983f826c20f2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IgfsSharedMemoryTestServer.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IgfsSharedMemoryTestServer.java @@ -35,15 +35,13 @@ public class IgfsSharedMemoryTestServer { public static void main(String[] args) throws IgniteCheckedException { System.out.println("Starting server ..."); - U.setWorkDirectory(null, U.getIgniteHome()); - // Tell our process PID to the wrapper. X.println(GridJavaProcess.PID_MSG_PREFIX + U.jvmPid()); InputStream is = null; try { - IpcServerEndpoint srv = new IpcSharedMemoryServerEndpoint(); + IpcServerEndpoint srv = new IpcSharedMemoryServerEndpoint(U.defaultWorkDirectory()); new IgniteTestResources().inject(srv); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryCrashDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryCrashDetectionSelfTest.java index 1aae999b37f85..0c5f5644e89b5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryCrashDetectionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryCrashDetectionSelfTest.java @@ -55,10 +55,7 @@ public class IpcSharedMemoryCrashDetectionSelfTest extends GridCommonAbstractTes @Override protected void afterTestsStopped() throws Exception { // Start and stop server endpoint to let GC worker // make a run and cleanup resources. - - U.setWorkDirectory(null, U.getIgniteHome()); - - IpcSharedMemoryServerEndpoint srv = new IpcSharedMemoryServerEndpoint(); + IpcSharedMemoryServerEndpoint srv = new IpcSharedMemoryServerEndpoint(U.defaultWorkDirectory()); new IgniteTestResources().inject(srv); @@ -74,10 +71,8 @@ public class IpcSharedMemoryCrashDetectionSelfTest extends GridCommonAbstractTes * @throws Exception If failed. */ public void testIgfsServerClientInteractionsUponClientKilling() throws Exception { - U.setWorkDirectory(null, U.getIgniteHome()); - // Run server endpoint. - IpcSharedMemoryServerEndpoint srv = new IpcSharedMemoryServerEndpoint(); + IpcSharedMemoryServerEndpoint srv = new IpcSharedMemoryServerEndpoint(U.defaultWorkDirectory()); new IgniteTestResources().inject(srv); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/benchmark/IpcSharedMemoryBenchmarkReader.java b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/benchmark/IpcSharedMemoryBenchmarkReader.java index 90c74f9f5c91e..d86f12786bcee 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/benchmark/IpcSharedMemoryBenchmarkReader.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/benchmark/IpcSharedMemoryBenchmarkReader.java @@ -76,7 +76,7 @@ public static void main(String[] args) throws IgniteCheckedException { } }); - try (IpcSharedMemoryServerEndpoint srv = new IpcSharedMemoryServerEndpoint()) { + try (IpcSharedMemoryServerEndpoint srv = new IpcSharedMemoryServerEndpoint(U.defaultWorkDirectory())) { new IgniteTestResources().inject(srv); srv.start(); diff --git a/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java b/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java index 5b0251d3d6845..5fd5b5e7ce2c3 100644 --- a/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/logger/java/JavaLoggerTest.java @@ -37,10 +37,9 @@ public class JavaLoggerTest extends TestCase { * @throws Exception If failed. */ public void testLogInitialize() throws Exception { - U.setWorkDirectory(null, U.getIgniteHome()); - log = new JavaLogger(); + ((JavaLogger)log).setWorkDirectory(U.defaultWorkDirectory()); ((LoggerNodeIdAware)log).setNodeId(UUID.fromString("00000000-1111-2222-3333-444444444444")); if (log.isDebugEnabled()) diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java index eddc4deda3ae9..ebbd8acf1775f 100644 --- a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java @@ -160,8 +160,6 @@ public void testRestartAllNodes() throws Exception { for (int i = 0; i < workDirs.length; i++) { workDir = workDirs[i]; - U.nullifyWorkDirectory(); - startGrid(i); } @@ -178,8 +176,6 @@ public void testRestartAllNodes() throws Exception { for (int i = 0; i < workDirs.length; i++) { workDir = workDirs[i]; - U.nullifyWorkDirectory(); - startGrid(i); } diff --git a/modules/core/src/test/java/org/apache/ignite/spi/checkpoint/sharedfs/GridSharedFsCheckpointSpiMultipleDirectoriesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/checkpoint/sharedfs/GridSharedFsCheckpointSpiMultipleDirectoriesSelfTest.java index ce893029ccd34..2eea7c16d5dd5 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/checkpoint/sharedfs/GridSharedFsCheckpointSpiMultipleDirectoriesSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/checkpoint/sharedfs/GridSharedFsCheckpointSpiMultipleDirectoriesSelfTest.java @@ -70,7 +70,7 @@ public void testMultipleSharedDirectories() throws Exception { String curSpiPath1 = getSpi().getCurrentDirectoryPath(); - File folder1 = U.resolveWorkDirectory(curSpiPath1, false); + File folder1 = U.resolveWorkDirectory(U.defaultWorkDirectory(), curSpiPath1, false); assert folder1.exists() : "Checkpoint folder doesn't exist."; @@ -83,7 +83,7 @@ public void testMultipleSharedDirectories() throws Exception { String curSpiPath2 = getSpi().getCurrentDirectoryPath(); - File folder2 = U.resolveWorkDirectory(curSpiPath2, false); + File folder2 = U.resolveWorkDirectory(U.defaultWorkDirectory(), curSpiPath2, false); assert folder2.exists() : "Check point folder doesn't exist."; @@ -97,7 +97,7 @@ public void testMultipleSharedDirectories() throws Exception { String newCurSpiPath = getSpi().getCurrentDirectoryPath(); - File changedFolder = U.resolveWorkDirectory(newCurSpiPath, false); + File changedFolder = U.resolveWorkDirectory(U.defaultWorkDirectory(), newCurSpiPath, false); assert changedFolder.exists() : "Check point folder doesn't exist."; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java index fc9cf0fc23bbc..93339ed86263f 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java @@ -283,8 +283,6 @@ protected int getMaxTransmitMessagesTime() { * @throws Exception If failed. */ private void startSpis() throws Exception { - U.setWorkDirectory(null, U.getIgniteHome()); - spis.clear(); nodes.clear(); spiRsrcs.clear(); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java index 7bbf53156f769..a74a07a884536 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java @@ -468,8 +468,6 @@ private int getSpiCount() { /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - U.setWorkDirectory(null, U.getIgniteHome()); - spis.clear(); nodes.clear(); spiRsrcs.clear(); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java index da1bc26f741de..9e927ddbb24f1 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java @@ -65,8 +65,6 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { - U.setWorkDirectory(null, U.getIgniteHome()); - spi = spi(); getTestResources().inject(spi); diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java index aa90af0b54026..9f507e64ca688 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java @@ -497,7 +497,7 @@ protected void afterTest() throws Exception { */ protected void beforeTestsStarted() throws Exception { // Will clean and re-create marshaller directory from scratch. - U.resolveWorkDirectory("marshaller", true); + U.resolveWorkDirectory(U.defaultWorkDirectory(), "marshaller", true); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java index c9859fc6e5b00..b559897705c5b 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java @@ -27,9 +27,11 @@ import org.apache.ignite.IgniteAtomicSequence; import org.apache.ignite.IgniteAtomicStamped; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCluster; import org.apache.ignite.IgniteCompute; import org.apache.ignite.IgniteCountDownLatch; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLock; import org.apache.ignite.IgniteSemaphore; import org.apache.ignite.IgniteDataStreamer; @@ -55,6 +57,7 @@ import org.apache.ignite.internal.binary.BinaryContext; import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl; import org.apache.ignite.internal.processors.cacheobject.NoOpBinary; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.logger.NullLogger; import org.apache.ignite.marshaller.Marshaller; @@ -136,6 +139,13 @@ public IgniteMock( cfg.setIgniteHome(home); cfg.setLocalHost(locHost); + try { + cfg.setWorkDirectory(U.defaultWorkDirectory()); + } + catch (IgniteCheckedException e) { + throw new IgniteException("Failed to get default work directory.", e); + } + return cfg; } diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java index 13d5fbd9e8bd1..4d861d09a43a1 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java @@ -83,8 +83,6 @@ public IgniteTestResources() throws IgniteCheckedException { home = U.getIgniteHome(); locHost = localHost(); - U.setWorkDirectory(null, null); - GridTestKernalContext ctx = new GridTestKernalContext(log); rsrcProc = new GridResourceProcessor(ctx); diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java index 591753bb4c8f8..20b3cf2fdb354 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java @@ -429,8 +429,6 @@ private void configure(IgniteSpi spi) throws Exception { * @throws Exception If failed. */ protected void spiStart(IgniteSpi spi) throws Exception { - U.setWorkDirectory(null, U.getIgniteHome()); - // Start SPI with unique grid name. spi.spiStart(getTestGridName()); diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java index 734e1992e49e0..2e3d8b5447c0e 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java @@ -31,7 +31,6 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingTxErrorTest; import org.apache.ignite.internal.processors.cache.IgniteDaemonNodeMarshallerCacheTest; import org.apache.ignite.internal.util.GridHandleTableSelfTest; -import org.apache.ignite.internal.util.GridStartupWithSpecifiedWorkDirectorySelfTest; import org.apache.ignite.internal.util.IgniteUtilsSelfTest; import org.apache.ignite.internal.util.io.GridUnsafeDataOutputArraySizingSelfTest; import org.apache.ignite.internal.util.nio.GridNioSelfTest; @@ -80,7 +79,6 @@ public static TestSuite suite() throws Exception { ignoredTests.add(IgniteCacheP2pUnmarshallingNearErrorTest.class); ignoredTests.add(IgniteCacheP2pUnmarshallingRebalanceErrorTest.class); ignoredTests.add(GridReleaseTypeSelfTest.class); - ignoredTests.add(GridStartupWithSpecifiedWorkDirectorySelfTest.class); ignoredTests.add(IgniteUtilsSelfTest.class); ignoredTests.add(ClusterGroupSelfTest.class); ignoredTests.add(GridMessagingNoPeerClassLoadingSelfTest.class); diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java index b91fff42e40f0..d9cc8c0473522 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java @@ -68,7 +68,6 @@ import org.apache.ignite.internal.processors.service.IgniteServiceDeploymentClassLoadingOptimizedMarshallerTest; import org.apache.ignite.internal.processors.service.IgniteServiceReassignmentTest; import org.apache.ignite.internal.processors.service.ServicePredicateAccessCacheTest; -import org.apache.ignite.internal.util.GridStartupWithSpecifiedWorkDirectorySelfTest; import org.apache.ignite.internal.util.GridStartupWithUndefinedIgniteHomeSelfTest; import org.apache.ignite.spi.communication.GridCacheMessageSelfTest; import org.apache.ignite.testframework.GridTestUtils; @@ -115,7 +114,6 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(GridDiscoveryEventSelfTest.class); suite.addTestSuite(GridPortProcessorSelfTest.class); suite.addTestSuite(GridHomePathSelfTest.class); - GridTestUtils.addTestIfNeeded(suite, GridStartupWithSpecifiedWorkDirectorySelfTest.class, ignoredTests); suite.addTestSuite(GridStartupWithUndefinedIgniteHomeSelfTest.class); GridTestUtils.addTestIfNeeded(suite, GridVersionSelfTest.class, ignoredTests); suite.addTestSuite(GridListenActorSelfTest.class); diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/CacheNoValueClassOnServerTestClient.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/CacheNoValueClassOnServerTestClient.java index c8455930cb56e..5c8246992fd00 100644 --- a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/CacheNoValueClassOnServerTestClient.java +++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/CacheNoValueClassOnServerTestClient.java @@ -55,8 +55,6 @@ public static void main(String[] args) throws Exception { cfg.setDiscoverySpi(disco); - U.setWorkDirectory(null, U.getIgniteHome()); - try (Ignite ignite = Ignition.start(cfg)) { System.out.println("Test external node started"); diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/startcache/CacheConfigurationP2PTestClient.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/startcache/CacheConfigurationP2PTestClient.java index 1134ab38b802e..e0406c2cc0608 100644 --- a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/startcache/CacheConfigurationP2PTestClient.java +++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/startcache/CacheConfigurationP2PTestClient.java @@ -55,8 +55,6 @@ public static void main(String[] args) throws Exception { cfg.setDiscoverySpi(disco); - U.setWorkDirectory(null, U.getIgniteHome()); - try (Ignite ignite = Ignition.start(cfg)) { System.out.println("Test external node started"); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelperImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelperImpl.java index 71bb8a4cafded..0e86529eed796 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelperImpl.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopHelperImpl.java @@ -16,7 +16,10 @@ */ package org.apache.ignite.internal.processors.hadoop; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; import org.objectweb.asm.ClassReader; import org.objectweb.asm.ClassWriter; @@ -117,4 +120,14 @@ public HadoopHelperImpl(GridKernalContext ctx) { @Override @Nullable public InputStream loadClassBytes(ClassLoader ldr, String clsName) { return ldr.getResourceAsStream(clsName.replace('.', '/') + ".class"); } + + /** {@inheritDoc} */ + @Override public String workDirectory() { + try { + return ctx != null ? ctx.config().getWorkDirectory() : U.defaultWorkDirectory(); + } + catch (IgniteCheckedException e) { + throw new IgniteException("Failed to resolve Ignite work directory.", e); + } + } } diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUtils.java index 347bfae09451c..a34388dba79df 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUtils.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopUtils.java @@ -287,25 +287,28 @@ public static IgniteCheckedException transformException(Throwable e) { /** * Returns work directory for job execution. * + * @param workDir Work directory. * @param locNodeId Local node ID. * @param jobId Job ID. * @return Working directory for job. * @throws IgniteCheckedException If Failed. */ - public static File jobLocalDir(UUID locNodeId, HadoopJobId jobId) throws IgniteCheckedException { - return new File(new File(U.resolveWorkDirectory("hadoop", false), "node-" + locNodeId), "job_" + jobId); + public static File jobLocalDir(String workDir, UUID locNodeId, HadoopJobId jobId) throws IgniteCheckedException { + return new File(new File(U.resolveWorkDirectory(workDir, "hadoop", false), + "node-" + locNodeId), "job_" + jobId); } /** * Returns subdirectory of job working directory for task execution. * + * @param workDir Work directory. * @param locNodeId Local node ID. * @param info Task info. * @return Working directory for task. * @throws IgniteCheckedException If Failed. */ - public static File taskLocalDir(UUID locNodeId, HadoopTaskInfo info) throws IgniteCheckedException { - File jobLocDir = jobLocalDir(locNodeId, info.jobId()); + public static File taskLocalDir(String workDir, UUID locNodeId, HadoopTaskInfo info) throws IgniteCheckedException { + File jobLocDir = jobLocalDir(workDir, locNodeId, info.jobId()); return new File(jobLocDir, info.type() + "_" + info.taskNumber() + "_" + info.attempt()); } diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java index 3731b2c1e1f3e..36da41028678b 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java @@ -311,7 +311,7 @@ public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, Ignite ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(getClass().getClassLoader()); try { - rsrcMgr.prepareJobEnvironment(!external, jobLocalDir(locNodeId, jobId)); + rsrcMgr.prepareJobEnvironment(!external, jobLocalDir(igniteWorkDirectory(), locNodeId, jobId)); } finally { HadoopCommonUtils.restoreContextClassLoader(oldLdr); @@ -323,7 +323,7 @@ public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, Ignite @Override public void dispose(boolean external) throws IgniteCheckedException { try { if (rsrcMgr != null && !external) { - File jobLocDir = jobLocalDir(locNodeId, jobId); + File jobLocDir = jobLocalDir(igniteWorkDirectory(), locNodeId, jobId); if (jobLocDir.exists()) U.delete(jobLocDir); @@ -411,7 +411,7 @@ private void closeCachedTaskFileSystems(ClassLoader ldr) throws Exception { /** {@inheritDoc} */ @Override public void prepareTaskEnvironment(HadoopTaskInfo info) throws IgniteCheckedException { - rsrcMgr.prepareTaskWorkDir(taskLocalDir(locNodeId, info)); + rsrcMgr.prepareTaskWorkDir(taskLocalDir(igniteWorkDirectory(), locNodeId, info)); } /** {@inheritDoc} */ @@ -420,7 +420,7 @@ private void closeCachedTaskFileSystems(ClassLoader ldr) throws Exception { taskCtxClsPool.add(ctx.getClass()); - File locDir = taskLocalDir(locNodeId, info); + File locDir = taskLocalDir(igniteWorkDirectory(), locNodeId, info); if (locDir.exists()) U.delete(locDir); @@ -431,6 +431,11 @@ private void closeCachedTaskFileSystems(ClassLoader ldr) throws Exception { rsrcMgr.cleanupStagingDirectory(); } + /** {@inheritDoc} */ + @Override public String igniteWorkDirectory() { + return helper.workDirectory(); + } + /** * Getter for job configuration. * @return The job configuration. diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java index 96fa89218250c..d444f2b6be4b8 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java @@ -285,12 +285,12 @@ private HadoopTask createTask() { case REDUCE: job().prepareTaskEnvironment(taskInfo()); - locDir = taskLocalDir(locNodeId, taskInfo()); + locDir = taskLocalDir(job.igniteWorkDirectory(), locNodeId, taskInfo()); break; default: - locDir = jobLocalDir(locNodeId, taskInfo().jobId()); + locDir = jobLocalDir(job.igniteWorkDirectory(), locNodeId, taskInfo().jobId()); } ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(jobConf().getClassLoader()); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java index dc5874d905214..2c560bc07a88e 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java @@ -106,7 +106,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter { log = ctx.kernalContext().log(HadoopExternalTaskExecutor.class); - outputBase = U.resolveWorkDirectory("hadoop", false); + outputBase = U.resolveWorkDirectory(ctx.kernalContext().config().getWorkDirectory(), "hadoop", false); pathSep = System.getProperty("path.separator", U.isWindows() ? ";" : ":"); @@ -118,7 +118,8 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter { ctx.kernalContext().config().getMarshaller(), log, ctx.kernalContext().getSystemExecutorService(), - ctx.kernalContext().gridName()); + ctx.kernalContext().gridName(), + ctx.kernalContext().config().getWorkDirectory()); comm.setListener(new MessageListener()); @@ -380,7 +381,8 @@ private HadoopProcess startProcess(final HadoopJob job, final HadoopMapReducePla log.debug("Created hadoop child process metadata for job [job=" + job + ", childProcId=" + childProcId + ", taskMeta=" + startMeta + ']'); - Process proc = startJavaProcess(childProcId, startMeta, job); + Process proc = startJavaProcess(childProcId, startMeta, job, + ctx.kernalContext().config().getWorkDirectory()); BufferedReader rdr = new BufferedReader(new InputStreamReader(proc.getInputStream())); @@ -517,10 +519,11 @@ private Collection readProcessOutput(Process proc) throws IOException { * @param childProcId Child process ID. * @param startMeta Metadata. * @param job Job. + * @param igniteWorkDir Work directory. * @return Started process. */ private Process startJavaProcess(UUID childProcId, HadoopExternalTaskMetadata startMeta, - HadoopJob job) throws Exception { + HadoopJob job, String igniteWorkDir) throws Exception { String outFldr = jobWorkFolder(job.id()) + File.separator + childProcId; if (log.isDebugEnabled()) @@ -528,7 +531,7 @@ private Process startJavaProcess(UUID childProcId, HadoopExternalTaskMetadata st List cmd = new ArrayList<>(); - File workDir = U.resolveWorkDirectory("", false); + File workDir = U.resolveWorkDirectory(igniteWorkDir, "", false); cmd.add(javaCmd); cmd.addAll(startMeta.jvmOptions()); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopExternalProcessStarter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopExternalProcessStarter.java index 32880e4481ae3..90012fba0bfb3 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopExternalProcessStarter.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopExternalProcessStarter.java @@ -77,12 +77,11 @@ public static void main(String[] cmdArgs) { } /** + * Run the process. * - * @throws Exception + * @throws Exception If failed. */ public void run() throws Exception { - U.setWorkDirectory(args.workDir, U.getIgniteHome()); - File outputDir = outputDirectory(); initializeStreams(outputDir); @@ -98,7 +97,8 @@ public void run() throws Exception { new JdkMarshaller(), log, msgExecSvc, - "external" + "external", + args.workDir ); comm.start(); @@ -121,7 +121,7 @@ public void run() throws Exception { /** * @param outputDir Directory for process output. - * @throws Exception + * @throws Exception If failed. */ private void initializeStreams(File outputDir) throws Exception { out = new FileOutputStream(new File(outputDir, args.childProcId + ".out")); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java index 1d59a9569f76b..6d903d8b821a2 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java @@ -177,6 +177,9 @@ public class HadoopExternalCommunication { /** Grid name. */ private String gridName; + /** Work directory. */ + private String workDir; + /** Complex variable that represents this node IP address. */ private volatile InetAddress locHost; @@ -254,6 +257,7 @@ public class HadoopExternalCommunication { * @param log Logger. * @param execSvc Executor service for message notification. * @param gridName Grid name. + * @param workDir Work directory. */ public HadoopExternalCommunication( UUID parentNodeId, @@ -261,7 +265,8 @@ public HadoopExternalCommunication( Marshaller marsh, IgniteLogger log, ExecutorService execSvc, - String gridName + String gridName, + String workDir ) { locProcDesc = new HadoopProcessDescriptor(parentNodeId, procId); @@ -269,6 +274,7 @@ public HadoopExternalCommunication( this.log = log.getLogger(HadoopExternalCommunication.class); this.execSvc = execSvc; this.gridName = gridName; + this.workDir = workDir; } /** @@ -685,7 +691,7 @@ private GridNioServer resetNioServer() throws IgniteCheckedExcept try { IpcSharedMemoryServerEndpoint srv = new IpcSharedMemoryServerEndpoint( log.getLogger(IpcSharedMemoryServerEndpoint.class), - locProcDesc.processId(), gridName); + locProcDesc.processId(), gridName, workDir); srv.setPort(port); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPlannerMockJob.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPlannerMockJob.java index 220614c88244f..7e6fa9c3921de 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPlannerMockJob.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPlannerMockJob.java @@ -101,6 +101,13 @@ public HadoopPlannerMockJob(Collection splits, int reducers) { throwUnsupported(); } + /** {@inheritDoc} */ + @Override public String igniteWorkDirectory() { + throwUnsupported(); + + return null; + } + /** * Throw {@link UnsupportedOperationException}. */ diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java index a40c5313d8442..38ff31898c8f0 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java @@ -81,7 +81,7 @@ private void checkSimpleMessageSending(boolean useShmem) throws Exception { for (int i = 0; i < comms.length; i++) { comms[i] = new HadoopExternalCommunication(parentNodeId, UUID.randomUUID(), marsh, log, - Executors.newFixedThreadPool(1), name + i); + Executors.newFixedThreadPool(1), name + i, U.defaultWorkDirectory()); if (useShmem) comms[i].setSharedMemoryPort(14000); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationP2PTestServer.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationP2PTestServer.java index 05a314ba2bfcc..500811f20f235 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationP2PTestServer.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheConfigurationP2PTestServer.java @@ -35,8 +35,6 @@ public static void main(String[] args) throws Exception { IgniteConfiguration cfg = CacheConfigurationP2PTest.createConfiguration(); - U.setWorkDirectory(null, U.getIgniteHome()); - try (Ignite ignite = Ignition.start(cfg)) { System.out.println(CacheConfigurationP2PTest.NODE_START_MSG); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs index 9d11ad38250d1..47575e1efaed5 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs @@ -116,7 +116,7 @@ public void TestAllConfigurationProperties() Assert.AreEqual(cfg.NetworkSendRetryCount, resCfg.NetworkSendRetryCount); Assert.AreEqual(cfg.NetworkTimeout, resCfg.NetworkTimeout); Assert.AreEqual(cfg.NetworkSendRetryDelay, resCfg.NetworkSendRetryDelay); - Assert.AreEqual(cfg.WorkDirectory, resCfg.WorkDirectory); + Assert.AreEqual(cfg.WorkDirectory.Trim('\\'), resCfg.WorkDirectory.Trim('\\')); Assert.AreEqual(cfg.JvmClasspath, resCfg.JvmClasspath); Assert.AreEqual(cfg.JvmOptions, resCfg.JvmOptions); Assert.IsTrue(File.Exists(resCfg.JvmDllPath)); diff --git a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java index 5f65731afc856..22d42db4ac04f 100644 --- a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java +++ b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java @@ -1029,7 +1029,7 @@ private void addDefaultUri() throws IgniteSpiException { URI uri; try { - uri = U.resolveWorkDirectory(DFLT_DEPLOY_DIR, false).toURI(); + uri = U.resolveWorkDirectory(ignite.configuration().getWorkDirectory(), DFLT_DEPLOY_DIR, false).toURI(); } catch (IgniteCheckedException e) { throw new IgniteSpiException("Failed to initialize default file scanner", e); diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala index 1e8b634998e0b..8673b9cc637f9 100644 --- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala +++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala @@ -2357,7 +2357,7 @@ object visor extends VisorTag { val folder = Option(f.getParent).getOrElse("") val fileName = f.getName - logFile = new File(U.resolveWorkDirectory(folder, false), fileName) + logFile = new File(U.resolveWorkDirectory(U.defaultWorkDirectory(), folder, false), fileName) logFile.createNewFile() From d45383b69cc68c0ec967ebd673b197e437720214 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 4 Oct 2016 18:48:25 +0300 Subject: [PATCH 226/487] .NET: Fix code analysis warnings --- .../dotnet/Apache.Ignite.Core/Common/JavaException.cs | 1 + .../dotnet/Apache.Ignite.Core/IgniteConfiguration.cs | 1 + .../Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs | 5 ++++- .../Apache.Ignite.Core/Impl/Compute/ComputeJob.cs | 10 ++++++++-- .../Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs | 5 ++++- .../Impl/Unmanaged/UnmanagedCallbacks.cs | 1 + 6 files changed, 19 insertions(+), 4 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Common/JavaException.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Common/JavaException.cs index a6340103a3179..5f7ba66b8a333 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Common/JavaException.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Common/JavaException.cs @@ -23,6 +23,7 @@ namespace Apache.Ignite.Core.Common /// /// Indicates an error on Java side and contains full Java stack trace. /// + [Serializable] public class JavaException : IgniteException { /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs index 765209230deee..b045c5a24b16a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs @@ -748,6 +748,7 @@ public static IgniteConfiguration FromXml(XmlReader reader) /// /// Xml string. /// Deserialized instance. + [SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope")] public static IgniteConfiguration FromXml(string xml) { IgniteArgumentCheck.NotNullOrEmpty(xml, "xml"); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs index 454f7bb5a4210..1f5523c989659 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs @@ -66,7 +66,10 @@ public object Invoke(object arg) } catch (TargetInvocationException ex) { - throw ex.InnerException; + if (ex.InnerException != null) + throw ex.InnerException; + + throw; } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs index 526c44520b922..9fa137767b8b4 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs @@ -82,7 +82,10 @@ public object Execute() } catch (TargetInvocationException ex) { - throw ex.InnerException; + if (ex.InnerException != null) + throw ex.InnerException; + + throw; } } @@ -95,7 +98,10 @@ public void Cancel() } catch (TargetInvocationException ex) { - throw ex.InnerException; + if (ex.InnerException != null) + throw ex.InnerException; + + throw; } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs index 1867f8c3cb057..974ada26fa161 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs @@ -70,7 +70,10 @@ public object Invoke() } catch (TargetInvocationException ex) { - throw ex.InnerException; + if (ex.InnerException != null) + throw ex.InnerException; + + throw; } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs index 493e0615b562e..a9de66e769473 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs @@ -1181,6 +1181,7 @@ private bool LoggerIsLevelEnabled(void* target, int level) return SafeCall(() => _log.IsEnabled((LogLevel) level), true); } + [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes")] private static void ConsoleWrite(sbyte* chars, int charsLen, bool isErr) { try From 23461b8d33922772ef8e7217e9e87b3f3b0b37b1 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Thu, 6 Oct 2016 10:14:59 +0300 Subject: [PATCH 227/487] IGNITE-4001: Timeouts for threads in Ignite pools. This closes #1130. --- .../configuration/IgniteConfiguration.java | 18 +++-- .../apache/ignite/internal/IgnitionEx.java | 71 ++++++++++--------- .../processors/igfs/IgfsThreadFactory.java | 61 ++++++++++++++++ 3 files changed, 112 insertions(+), 38 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsThreadFactory.java diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java index 19b9a4d2f5998..73de470719b06 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java @@ -149,29 +149,38 @@ public class IgniteConfiguration { public static final int DFLT_PUBLIC_THREAD_CNT = Math.max(8, AVAILABLE_PROC_CNT) * 2; /** Default keep alive time for public thread pool. */ + @Deprecated public static final long DFLT_PUBLIC_KEEP_ALIVE_TIME = 0; /** Default limit of threads used for rebalance. */ public static final int DFLT_REBALANCE_THREAD_POOL_SIZE = 1; /** Default max queue capacity of public thread pool. */ + @Deprecated public static final int DFLT_PUBLIC_THREADPOOL_QUEUE_CAP = Integer.MAX_VALUE; /** Default size of system thread pool. */ public static final int DFLT_SYSTEM_CORE_THREAD_CNT = DFLT_PUBLIC_THREAD_CNT; /** Default max size of system thread pool. */ + @Deprecated public static final int DFLT_SYSTEM_MAX_THREAD_CNT = DFLT_PUBLIC_THREAD_CNT; /** Default keep alive time for system thread pool. */ + @Deprecated public static final long DFLT_SYSTEM_KEEP_ALIVE_TIME = 0; /** Default keep alive time for utility thread pool. */ + @Deprecated public static final long DFLT_UTILITY_KEEP_ALIVE_TIME = 10_000; /** Default max queue capacity of system thread pool. */ + @Deprecated public static final int DFLT_SYSTEM_THREADPOOL_QUEUE_CAP = Integer.MAX_VALUE; + /** Default Ignite thread keep alive time. */ + public static final long DFLT_THREAD_KEEP_ALIVE_TIME = 60_000L; + /** Default size of peer class loading thread pool. */ public static final int DFLT_P2P_THREAD_CNT = 2; @@ -240,13 +249,13 @@ public class IgniteConfiguration { private int utilityCachePoolSize = DFLT_SYSTEM_CORE_THREAD_CNT; /** Utility cache pool keep alive time. */ - private long utilityCacheKeepAliveTime = DFLT_UTILITY_KEEP_ALIVE_TIME; + private long utilityCacheKeepAliveTime = DFLT_THREAD_KEEP_ALIVE_TIME; /** Marshaller pool size. */ private int marshCachePoolSize = DFLT_SYSTEM_CORE_THREAD_CNT; /** Marshaller pool keep alive time. */ - private long marshCacheKeepAliveTime = DFLT_UTILITY_KEEP_ALIVE_TIME; + private long marshCacheKeepAliveTime = DFLT_THREAD_KEEP_ALIVE_TIME; /** P2P pool size. */ private int p2pPoolSize = DFLT_P2P_THREAD_CNT; @@ -492,6 +501,7 @@ public IgniteConfiguration(IgniteConfiguration cfg) { cacheCfg = cfg.getCacheConfiguration(); cacheKeyCfg = cfg.getCacheKeyConfiguration(); cacheSanityCheckEnabled = cfg.isCacheSanityCheckEnabled(); + callbackPoolSize = cfg.getAsyncCallbackPoolSize(); connectorCfg = cfg.getConnectorConfiguration(); classLdr = cfg.getClassLoader(); clientMode = cfg.isClientMode(); @@ -792,7 +802,7 @@ public int getUtilityCacheThreadPoolSize() { /** * Keep alive time of thread pool that is in charge of processing utility cache messages. *

          - * If not provided, executor service will have keep alive time {@link #DFLT_UTILITY_KEEP_ALIVE_TIME}. + * If not provided, executor service will have keep alive time {@link #DFLT_THREAD_KEEP_ALIVE_TIME}. * * @return Thread pool keep alive time (in milliseconds) to be used in grid for utility cache messages. */ @@ -814,7 +824,7 @@ public int getMarshallerCacheThreadPoolSize() { /** * Keep alive time of thread pool that is in charge of processing marshaller messages. *

          - * If not provided, executor service will have keep alive time {@link #DFLT_UTILITY_KEEP_ALIVE_TIME}. + * If not provided, executor service will have keep alive time {@link #DFLT_THREAD_KEEP_ALIVE_TIME}. * * @return Thread pool keep alive time (in milliseconds) to be used in grid for marshaller messages. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index 2914c7cf867f7..001f599cd0f96 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -36,7 +36,6 @@ import java.util.UUID; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.atomic.AtomicBoolean; @@ -62,6 +61,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.TransactionConfiguration; import org.apache.ignite.internal.binary.BinaryMarshaller; +import org.apache.ignite.internal.processors.igfs.IgfsThreadFactory; import org.apache.ignite.internal.processors.igfs.IgfsUtils; import org.apache.ignite.internal.processors.resource.GridSpringResourceContext; import org.apache.ignite.internal.util.GridConcurrentHashSet; @@ -119,10 +119,7 @@ import static org.apache.ignite.cache.CacheMode.REPLICATED; import static org.apache.ignite.cache.CacheRebalanceMode.SYNC; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; -import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_PUBLIC_KEEP_ALIVE_TIME; -import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_PUBLIC_THREADPOOL_QUEUE_CAP; -import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_SYSTEM_KEEP_ALIVE_TIME; -import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_SYSTEM_THREADPOOL_QUEUE_CAP; +import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME; import static org.apache.ignite.internal.IgniteComponentType.SPRING; import static org.apache.ignite.plugin.segmentation.SegmentationPolicy.RESTART_JVM; @@ -1457,28 +1454,28 @@ private static final class IgniteNamedInstance { private volatile IgniteKernal grid; /** Executor service. */ - private ExecutorService execSvc; + private ThreadPoolExecutor execSvc; /** System executor service. */ - private ExecutorService sysExecSvc; + private ThreadPoolExecutor sysExecSvc; /** Management executor service. */ - private ExecutorService mgmtExecSvc; + private ThreadPoolExecutor mgmtExecSvc; /** P2P executor service. */ - private ExecutorService p2pExecSvc; + private ThreadPoolExecutor p2pExecSvc; /** IGFS executor service. */ - private ExecutorService igfsExecSvc; + private ThreadPoolExecutor igfsExecSvc; /** REST requests executor service. */ - private ExecutorService restExecSvc; + private ThreadPoolExecutor restExecSvc; /** Utility cache executor service. */ - private ExecutorService utilityCacheExecSvc; + private ThreadPoolExecutor utilityCacheExecSvc; /** Marshaller cache executor service. */ - private ExecutorService marshCacheExecSvc; + private ThreadPoolExecutor marshCacheExecSvc; /** Continuous query executor service. */ private IgniteStripedThreadPoolExecutor callbackExecSvc; @@ -1642,12 +1639,10 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { cfg.getGridName(), cfg.getPublicThreadPoolSize(), cfg.getPublicThreadPoolSize(), - DFLT_PUBLIC_KEEP_ALIVE_TIME, - new LinkedBlockingQueue(DFLT_PUBLIC_THREADPOOL_QUEUE_CAP)); + DFLT_THREAD_KEEP_ALIVE_TIME, + new LinkedBlockingQueue()); - if (!myCfg.isClientMode()) - // Pre-start all threads as they are guaranteed to be needed. - ((ThreadPoolExecutor)execSvc).prestartAllCoreThreads(); + execSvc.allowCoreThreadTimeOut(true); // Note that since we use 'LinkedBlockingQueue', number of // maximum threads has no effect. @@ -1656,11 +1651,10 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { cfg.getGridName(), cfg.getSystemThreadPoolSize(), cfg.getSystemThreadPoolSize(), - DFLT_SYSTEM_KEEP_ALIVE_TIME, - new LinkedBlockingQueue(DFLT_SYSTEM_THREADPOOL_QUEUE_CAP)); + DFLT_THREAD_KEEP_ALIVE_TIME, + new LinkedBlockingQueue()); - // Pre-start all threads as they are guaranteed to be needed. - ((ThreadPoolExecutor)sysExecSvc).prestartAllCoreThreads(); + sysExecSvc.allowCoreThreadTimeOut(true); // Note that since we use 'LinkedBlockingQueue', number of // maximum threads has no effect. @@ -1671,9 +1665,11 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { cfg.getGridName(), cfg.getManagementThreadPoolSize(), cfg.getManagementThreadPoolSize(), - 0, + DFLT_THREAD_KEEP_ALIVE_TIME, new LinkedBlockingQueue()); + mgmtExecSvc.allowCoreThreadTimeOut(true); + // Note that since we use 'LinkedBlockingQueue', number of // maximum threads has no effect. // Note, that we do not pre-start threads here as class loading pool may @@ -1683,20 +1679,21 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { cfg.getGridName(), cfg.getPeerClassLoadingThreadPoolSize(), cfg.getPeerClassLoadingThreadPoolSize(), - 0, + DFLT_THREAD_KEEP_ALIVE_TIME, new LinkedBlockingQueue()); + p2pExecSvc.allowCoreThreadTimeOut(true); + // Note that we do not pre-start threads here as igfs pool may not be needed. igfsExecSvc = new IgniteThreadPoolExecutor( - "igfs", - cfg.getGridName(), cfg.getIgfsThreadPoolSize(), cfg.getIgfsThreadPoolSize(), - 0, - new LinkedBlockingQueue()); + DFLT_THREAD_KEEP_ALIVE_TIME, + new LinkedBlockingQueue(), + new IgfsThreadFactory(cfg.getGridName(), "igfs"), + null /* Abort policy will be used. */); - // Pre-start all threads to avoid HadoopClassLoader leaks. - ((ThreadPoolExecutor)igfsExecSvc).prestartAllCoreThreads(); + igfsExecSvc.allowCoreThreadTimeOut(true); // Note that we do not pre-start threads here as this pool may not be needed. callbackExecSvc = new IgniteStripedThreadPoolExecutor( @@ -1710,9 +1707,11 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { myCfg.getGridName(), myCfg.getConnectorConfiguration().getThreadPoolSize(), myCfg.getConnectorConfiguration().getThreadPoolSize(), - ConnectorConfiguration.DFLT_KEEP_ALIVE_TIME, - new LinkedBlockingQueue(ConnectorConfiguration.DFLT_THREADPOOL_QUEUE_CAP) + DFLT_THREAD_KEEP_ALIVE_TIME, + new LinkedBlockingQueue() ); + + restExecSvc.allowCoreThreadTimeOut(true); } utilityCacheExecSvc = new IgniteThreadPoolExecutor( @@ -1721,7 +1720,9 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { myCfg.getUtilityCacheThreadPoolSize(), myCfg.getUtilityCacheThreadPoolSize(), myCfg.getUtilityCacheKeepAliveTime(), - new LinkedBlockingQueue(DFLT_SYSTEM_THREADPOOL_QUEUE_CAP)); + new LinkedBlockingQueue()); + + utilityCacheExecSvc.allowCoreThreadTimeOut(true); marshCacheExecSvc = new IgniteThreadPoolExecutor( "marshaller-cache", @@ -1729,7 +1730,9 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { myCfg.getMarshallerCacheThreadPoolSize(), myCfg.getMarshallerCacheThreadPoolSize(), myCfg.getMarshallerCacheKeepAliveTime(), - new LinkedBlockingQueue(DFLT_SYSTEM_THREADPOOL_QUEUE_CAP)); + new LinkedBlockingQueue()); + + marshCacheExecSvc.allowCoreThreadTimeOut(true); // Register Ignite MBean for current grid instance. registerFactoryMbean(myCfg.getMBeanServer()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsThreadFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsThreadFactory.java new file mode 100644 index 0000000000000..32cab0feea048 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsThreadFactory.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.igfs; + +import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.thread.IgniteThreadFactory; +import org.jetbrains.annotations.NotNull; + +/** + * Special thread factory used only for IGFS pool which prevents {@link HadoopClassLoader} leak into + * {@code Thread.contextClassLoader} field. To achieve this we switch context class loader back and forth when + * creating threads. + */ +public class IgfsThreadFactory extends IgniteThreadFactory { + /** + * Constructor. + * + * @param gridName Grid name. + * @param threadName Thread name. + */ + public IgfsThreadFactory(String gridName, String threadName) { + super(gridName, threadName); + } + + /** {@inheritDoc} */ + @Override public Thread newThread(@NotNull Runnable r) { + Thread curThread = Thread.currentThread(); + + ClassLoader oldLdr = curThread.getContextClassLoader(); + + curThread.setContextClassLoader(IgfsThreadFactory.class.getClassLoader()); + + try { + return super.newThread(r); + } + finally { + curThread.setContextClassLoader(oldLdr); + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(IgfsThreadFactory.class, this); + } +} From b94b0aeae4c42b1d35128c6b1de97e3fa318d497 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Thu, 6 Oct 2016 10:22:50 +0300 Subject: [PATCH 228/487] IGNITE-3163 IGFS: Added working directory support to IgniteHadoopIgfsSecondaryFileSystem. This closes #1030. This closes #1058. This closes #1132. --- .../HadoopBasicFileSystemFactoryDelegate.java | 16 +++++++++++++++- .../HadoopKerberosFileSystemFactoryDelegate.java | 7 ++++++- .../impl/igfs/Hadoop1DualAbstractTest.java | 12 +++++++++--- ...doopSecondaryFileSystemConfigurationTest.java | 12 ++++++++++++ 4 files changed, 42 insertions(+), 5 deletions(-) diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopBasicFileSystemFactoryDelegate.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopBasicFileSystemFactoryDelegate.java index a190b14699ba4..59488bef79171 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopBasicFileSystemFactoryDelegate.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopBasicFileSystemFactoryDelegate.java @@ -19,6 +19,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.ignite.IgniteException; import org.apache.ignite.hadoop.fs.BasicHadoopFileSystemFactory; import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory; @@ -52,6 +53,9 @@ public class HadoopBasicFileSystemFactoryDelegate implements HadoopFileSystemFac /** User name mapper. */ private UserNameMapper usrNameMapper; + /** Work directory. */ + protected Path workDir; + /** * Constructor. * @@ -110,7 +114,12 @@ protected FileSystem getWithMappedName(String usrName) throws IOException { * @throws InterruptedException if the current thread is interrupted. */ protected FileSystem create(String usrName) throws IOException, InterruptedException { - return FileSystem.get(fullUri, cfg, usrName); + FileSystem fs = FileSystem.get(fullUri, cfg, usrName); + + if (workDir != null) + fs.setWorkingDirectory(workDir); + + return fs; } /** {@inheritDoc} */ @@ -150,6 +159,11 @@ protected FileSystem create(String usrName) throws IOException, InterruptedExcep } } + String strWorkDir = fullUri.getPath(); + + if (!"/".equals(strWorkDir)) + workDir = new Path(strWorkDir); + usrNameMapper = proxy0.getUserNameMapper(); if (usrNameMapper != null && usrNameMapper instanceof LifecycleAware) diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopKerberosFileSystemFactoryDelegate.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopKerberosFileSystemFactoryDelegate.java index c71dedbc79eab..75093bb11625f 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopKerberosFileSystemFactoryDelegate.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopKerberosFileSystemFactoryDelegate.java @@ -61,7 +61,12 @@ public HadoopKerberosFileSystemFactoryDelegate(KerberosHadoopFileSystemFactory p return proxyUgi.doAs(new PrivilegedExceptionAction() { @Override public FileSystem run() throws Exception { - return FileSystem.get(fullUri, cfg); + FileSystem fs = FileSystem.get(fullUri, cfg); + + if (workDir != null) + fs.setWorkingDirectory(workDir); + + return fs; } }); } diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/Hadoop1DualAbstractTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/Hadoop1DualAbstractTest.java index a585e544d72d9..367a40637d18c 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/Hadoop1DualAbstractTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/Hadoop1DualAbstractTest.java @@ -58,6 +58,9 @@ public abstract class Hadoop1DualAbstractTest extends IgfsDualAbstractSelfTest { /** Secondary file system authority. */ private static final String SECONDARY_AUTHORITY = IGFS_NAME + ":" + GRID_NAME + "@127.0.0.1:" + PORT; + /** Secondary file system authority. */ + private static final String SECONDARY_WORKDIR = "workdir/test"; + /** Secondary Fs configuration full path. */ protected String secondaryConfFullPath; @@ -117,13 +120,16 @@ protected void startUnderlying() throws Exception { * @throws IOException On failure. */ protected void prepareConfiguration() throws IOException { - Configuration secondaryConf = HadoopSecondaryFileSystemConfigurationTest.configuration(IGFS_SCHEME, SECONDARY_AUTHORITY, true, true); + Configuration secondaryConf = HadoopSecondaryFileSystemConfigurationTest.configuration(IGFS_SCHEME, + SECONDARY_AUTHORITY, true, true); secondaryConf.setInt("fs.igfs.block.size", 1024); - secondaryConfFullPath = HadoopSecondaryFileSystemConfigurationTest.writeConfiguration(secondaryConf, HadoopSecondaryFileSystemConfigurationTest.SECONDARY_CFG_PATH); + secondaryConfFullPath = HadoopSecondaryFileSystemConfigurationTest.writeConfiguration(secondaryConf, + HadoopSecondaryFileSystemConfigurationTest.SECONDARY_CFG_PATH); - secondaryUri = HadoopSecondaryFileSystemConfigurationTest.mkUri(IGFS_SCHEME, SECONDARY_AUTHORITY); + secondaryUri = HadoopSecondaryFileSystemConfigurationTest.mkUri(IGFS_SCHEME, SECONDARY_AUTHORITY, + SECONDARY_WORKDIR); } /** diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java index fd7523315c86c..ae03f145dbd05 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java @@ -580,4 +580,16 @@ static String writeConfiguration(Configuration cfg, String pathFromIgniteHome) t static String mkUri(String scheme, String authority) { return scheme + "://" + authority + "/"; } + + /** + * Makes URI. + * + * @param scheme the scheme + * @param authority the authority + * @param path Path part of URI. + * @return URI String + */ + static String mkUri(String scheme, String authority, String path) { + return scheme + "://" + authority + "/" + path; + } } \ No newline at end of file From e6317e01fa8a0de03e15dcdd84a575c6b06ce701 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Thu, 6 Oct 2016 12:03:48 +0300 Subject: [PATCH 229/487] IGNITE-3593 .NET: IgniteConfiguration.WorkDirectory has no effect. This closes #903. This closes #1145. --- .../IgniteConfigurationTest.cs | 22 +++++++++++++++++++ .../Apache.Ignite.Core/Impl/IgniteUtils.cs | 2 +- 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs index 47575e1efaed5..1773900bda5d9 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs @@ -30,6 +30,7 @@ namespace Apache.Ignite.Core.Tests using Apache.Ignite.Core.Discovery.Tcp.Multicast; using Apache.Ignite.Core.Discovery.Tcp.Static; using Apache.Ignite.Core.Events; + using Apache.Ignite.Core.Impl; using Apache.Ignite.Core.Transactions; using NUnit.Framework; @@ -300,6 +301,27 @@ public void TestMulticastIpFinder() new TcpDiscoveryMulticastIpFinder {MulticastGroup = "228.111.111.223", MulticastPort = 54522}); } + ///

          + /// Tests the work directory. + /// + [Test] + public void TestWorkDirectory() + { + var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration()) + { + WorkDirectory = IgniteUtils.GetTempDirectoryName() + }; + + using (Ignition.Start(cfg)) + { + var marshDir = Path.Combine(cfg.WorkDirectory, "marshaller"); + + Assert.IsTrue(Directory.Exists(marshDir)); + } + + Directory.Delete(cfg.WorkDirectory, true); + } + /// /// Tests the ip finders. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs index e43e33200fb81..57fb0174323db 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs @@ -361,7 +361,7 @@ private static void TryCleanTempDirectories() /// Creates a uniquely named, empty temporary directory on disk and returns the full path of that directory. ///
          /// The full path of the temporary directory. - private static string GetTempDirectoryName() + internal static string GetTempDirectoryName() { while (true) { From 952be8b995050b34379006dd6e739da3fe3b49e3 Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Fri, 7 Oct 2016 15:00:09 +0300 Subject: [PATCH 230/487] Squashed commit of the following: commit 566881b695b8bc00e618fe9a9b4c86a8fd563cc1 Author: sboikov Date: Fri Oct 7 13:08:38 2016 +0300 minor commit 7fe88a1cb21f794ee55a176ab36d895cbf916528 Author: Dmitriy Govorukhin Date: Thu Oct 6 11:11:24 2016 +0300 ignite-update-notifier fix after review (cherry picked from commit a10d2ff) commit f2de749f958a3b18dc479f8a5517d7bf9362b933 Author: Dmitriy Govorukhin Date: Tue Oct 4 12:12:08 2016 +0300 ignite-2079-2 optimize import and change url path (cherry picked from commit 830a3cf) commit 0d1be85ad55b0aa91224690d6c112ae92e8bc0a9 Author: Dmitriy Govorukhin Date: Thu Sep 29 19:54:54 2016 +0300 update-notifier remove parse xml, now parse like properties (cherry picked from commit 9ecaa29) commit e43bca6fb4528a7fc0dcb804a74fca1c59d7468b Author: Dmitriy Govorukhin Date: Tue Sep 27 17:07:21 2016 +0300 remove dom parser (cherry picked from commit d1653b2) --- .../cluster/GridUpdateNotifier.java | 108 ++++-------------- 1 file changed, 21 insertions(+), 87 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifier.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifier.java index b41bf2857592b..592fdd10bde73 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifier.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifier.java @@ -18,13 +18,11 @@ package org.apache.ignite.internal.processors.cluster; import java.io.BufferedReader; -import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; import java.io.OutputStream; import java.io.PrintWriter; -import java.io.StringReader; import java.io.StringWriter; import java.io.UnsupportedEncodingException; import java.net.URL; @@ -32,9 +30,6 @@ import java.util.Collection; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.atomic.AtomicReference; -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; -import javax.xml.parsers.ParserConfigurationException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; @@ -47,12 +42,6 @@ import org.apache.ignite.internal.util.worker.GridWorker; import org.apache.ignite.plugin.PluginProvider; import org.jetbrains.annotations.Nullable; -import org.w3c.dom.Document; -import org.w3c.dom.Element; -import org.w3c.dom.Node; -import org.w3c.dom.NodeList; -import org.xml.sax.EntityResolver; -import org.xml.sax.InputSource; import static java.net.URLEncoder.encode; @@ -73,23 +62,20 @@ class GridUpdateNotifier { private static final long THROTTLE_PERIOD = 24 * 60 * 60 * 1000; // 1 day. /** Sleep milliseconds time for worker thread. */ - public static final int WORKER_THREAD_SLEEP_TIME = 5000; + private static final int WORKER_THREAD_SLEEP_TIME = 5000; + + /** Url for request version. */ + private final static String UPDATE_NOTIFIER_URL = "https://ignite.run/update_status_ignite-plain-text.php"; /** Grid version. */ private final String ver; - /** Site. */ - private final String url; - /** Latest version. */ private volatile String latestVer; /** Download url for latest version. */ private volatile String downloadUrl; - /** HTML parsing helper. */ - private final DocumentBuilder documentBuilder; - /** Grid name. */ private final String gridName; @@ -130,23 +116,7 @@ class GridUpdateNotifier { GridUpdateNotifier(String gridName, String ver, GridKernalGateway gw, Collection pluginProviders, boolean reportOnlyNew) throws IgniteCheckedException { try { - DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance(); - - documentBuilder = factory.newDocumentBuilder(); - - documentBuilder.setEntityResolver(new EntityResolver() { - @Override public InputSource resolveEntity(String publicId, String sysId) { - if (sysId.endsWith(".dtd")) - return new InputSource(new StringReader("")); - - return null; - } - }); - this.ver = ver; - - url = "http://ignite.run/update_status_ignite.php"; - this.gridName = gridName == null ? "null" : gridName; this.gw = gw; @@ -184,9 +154,6 @@ class GridUpdateNotifier { workerThread.start(); } - catch (ParserConfigurationException e) { - throw new IgniteCheckedException("Failed to create xml parser.", e); - } catch (UnsupportedEncodingException e) { throw new IgniteCheckedException("Failed to encode.", e); } @@ -346,7 +313,7 @@ private class UpdateChecker extends GridWorker { (!F.isEmpty(vmProps) ? "&vmProps=" + encode(vmProps, CHARSET) : "") + pluginsVers; - URLConnection conn = new URL(url).openConnection(); + URLConnection conn = new URL(UPDATE_NOTIFIER_URL).openConnection(); if (!isCancelled()) { conn.setDoOutput(true); @@ -356,8 +323,6 @@ private class UpdateChecker extends GridWorker { conn.setConnectTimeout(3000); conn.setReadTimeout(3000); - Document dom = null; - try { try (OutputStream os = conn.getOutputStream()) { os.write(postParams.getBytes(CHARSET)); @@ -369,30 +334,19 @@ private class UpdateChecker extends GridWorker { BufferedReader reader = new BufferedReader(new InputStreamReader(in, CHARSET)); - StringBuilder xml = new StringBuilder(); - - String line; - - while ((line = reader.readLine()) != null) { - if (line.contains("")) - line = line.replace(">", "/>"); - - xml.append(line).append('\n'); + for (String line; (line = reader.readLine()) != null; ) { + if (line.contains("version")) + latestVer = obtainVersionFrom(line); + else if (line.contains("downloadUrl")) + downloadUrl = obtainDownloadUrlFrom(line); } - dom = documentBuilder.parse(new ByteArrayInputStream(xml.toString().getBytes(CHARSET))); } } catch (IOException e) { if (log.isDebugEnabled()) log.debug("Failed to connect to Ignite update server. " + e.getMessage()); } - - if (dom != null) { - latestVer = obtainVersionFrom(dom); - - downloadUrl = obtainDownloadUrlFrom(dom); - } } } catch (Exception e) { @@ -404,55 +358,35 @@ private class UpdateChecker extends GridWorker { /** * Gets the version from the current {@code node}, if one exists. * - * @param node W3C DOM node. + * @param line Line which contains value for extract. + * @param metaName Name for extract. * @return Version or {@code null} if one's not found. */ - @Nullable private String obtainMeta(String metaName, Node node) { - assert node != null; - - if (node instanceof Element && "meta".equals(node.getNodeName().toLowerCase())) { - Element meta = (Element)node; - - String name = meta.getAttribute("name"); - - if (metaName.equals(name)) { - String content = meta.getAttribute("content"); + @Nullable private String obtainMeta(String metaName, String line) { + assert line.contains(metaName); - if (content != null && !content.isEmpty()) - return content; - } - } - - NodeList childNodes = node.getChildNodes(); - - for (int i = 0; i < childNodes.getLength(); i++) { - String ver = obtainMeta(metaName, childNodes.item(i)); + return line.substring(line.indexOf(metaName) + metaName.length()).trim(); - if (ver != null) - return ver; - } - - return null; } /** * Gets the version from the current {@code node}, if one exists. * - * @param node W3C DOM node. + * @param line Line which contains value for extract. * @return Version or {@code null} if one's not found. */ - @Nullable private String obtainVersionFrom(Node node) { - return obtainMeta("version", node); + @Nullable private String obtainVersionFrom(String line) { + return obtainMeta("version=", line); } /** * Gets the download url from the current {@code node}, if one exists. * - * @param node W3C DOM node. + * @param line Which contains value for extract. * @return download url or {@code null} if one's not found. */ - @Nullable private String obtainDownloadUrlFrom(Node node) { - return obtainMeta("downloadUrl", node); + @Nullable private String obtainDownloadUrlFrom(String line) { + return obtainMeta("downloadUrl=", line); } } } From b9c776a8423471706ecb1dc6176b38f23e799077 Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Mon, 10 Oct 2016 11:52:57 +0300 Subject: [PATCH 231/487] IGNITE-3235 Failed to initialize primitive boolean cache property of superclass --- .../internal/jdbc2/JdbcResultSetSelfTest.java | 91 ++++++++++++++++++- .../configuration/CacheConfiguration.java | 13 ++- .../processors/query/GridQueryProcessor.java | 71 +++++++++------ 3 files changed, 142 insertions(+), 33 deletions(-) diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcResultSetSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcResultSetSelfTest.java index 3607f53e20d60..cc67a65d85f65 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcResultSetSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcResultSetSelfTest.java @@ -60,7 +60,8 @@ public class JdbcResultSetSelfTest extends GridCommonAbstractTest { /** SQL query. */ private static final String SQL = "select id, boolVal, byteVal, shortVal, intVal, longVal, floatVal, " + - "doubleVal, bigVal, strVal, arrVal, dateVal, timeVal, tsVal, urlVal, f1, f2, f3, _val " + + "doubleVal, bigVal, strVal, arrVal, dateVal, timeVal, tsVal, urlVal, f1, f2, f3, _val, " + + "boolVal2, boolVal3, boolVal4 " + "from TestObject where id = 1"; /** Statement. */ @@ -142,6 +143,9 @@ private TestObject createObjectWithData(int id) throws MalformedURLException { TestObject o = new TestObject(id); o.boolVal = true; + o.boolVal2 = true; + o.boolVal3 = true; + o.boolVal4 = true; o.byteVal = 1; o.shortVal = 1; o.intVal = 1; @@ -179,6 +183,66 @@ public void testBoolean() throws Exception { assert cnt == 1; } + /** + * @throws Exception If failed. + */ + public void testBoolean2() throws Exception { + ResultSet rs = stmt.executeQuery(SQL); + + int cnt = 0; + + while (rs.next()) { + if (cnt == 0) { + assert rs.getBoolean("boolVal2"); + assert rs.getBoolean(20); + } + + cnt++; + } + + assert cnt == 1; + } + + /** + * @throws Exception If failed. + */ + public void testBoolean3() throws Exception { + ResultSet rs = stmt.executeQuery(SQL); + + int cnt = 0; + + while (rs.next()) { + if (cnt == 0) { + assert rs.getBoolean("boolVal3"); + assert rs.getBoolean(21); + } + + cnt++; + } + + assert cnt == 1; + } + + /** + * @throws Exception If failed. + */ + public void testBoolean4() throws Exception { + ResultSet rs = stmt.executeQuery(SQL); + + int cnt = 0; + + while (rs.next()) { + if (cnt == 0) { + assert rs.getBoolean("boolVal4"); + assert rs.getBoolean(22); + } + + cnt++; + } + + assert cnt == 1; + } + /** * @throws Exception If failed. */ @@ -562,7 +626,30 @@ public void testFindColumn() throws Exception { * Test object. */ @SuppressWarnings("UnusedDeclaration") - private static class TestObject implements Serializable { + private static class BaseTestObject implements Serializable { + /** */ + @QuerySqlField(index = false) + protected Boolean boolVal2; + + /** */ + @QuerySqlField(index = false) + protected boolean boolVal3; + + /** */ + protected boolean boolVal4; + + /** */ + @QuerySqlField(index = false) + public boolean isBoolVal4() { + return boolVal4; + } + } + + /** + * Test object. + */ + @SuppressWarnings("UnusedDeclaration") + private static class TestObject extends BaseTestObject { /** */ @QuerySqlField private final int id; diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java index e28aad5919bd1..f65bf52104dd7 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java @@ -2668,10 +2668,17 @@ private static class ClassProperty { ClassProperty(Member member) { this.member = member; - name = member instanceof Method && member.getName().startsWith("get") && member.getName().length() > 3 ? - member.getName().substring(3) : member.getName(); + name = member.getName(); - ((AccessibleObject) member).setAccessible(true); + if (member instanceof Method) { + if (member.getName().startsWith("get") && member.getName().length() > 3) + name = member.getName().substring(3); + + if (member.getName().startsWith("is") && member.getName().length() > 2) + name = member.getName().substring(2); + } + + ((AccessibleObject)member).setAccessible(true); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index 8469a7ca81511..1605188e12144 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -17,6 +17,30 @@ package org.apache.ignite.internal.processors.query; +import java.lang.reflect.AccessibleObject; +import java.lang.reflect.Field; +import java.lang.reflect.Member; +import java.lang.reflect.Method; +import java.math.BigDecimal; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; +import java.util.UUID; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import javax.cache.Cache; +import javax.cache.CacheException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; @@ -68,31 +92,6 @@ import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; -import javax.cache.Cache; -import javax.cache.CacheException; -import java.lang.reflect.AccessibleObject; -import java.lang.reflect.Field; -import java.lang.reflect.Member; -import java.lang.reflect.Method; -import java.math.BigDecimal; -import java.sql.Time; -import java.sql.Timestamp; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.TreeSet; -import java.util.UUID; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutorService; - import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED; import static org.apache.ignite.internal.IgniteComponentType.INDEXING; import static org.apache.ignite.internal.processors.query.GridQueryIndexType.FULLTEXT; @@ -1686,15 +1685,31 @@ static ClassProperty buildClassProperty(boolean key, Class cls, String pathSt // No-op. } - if (tmp == null) { + if (tmp == null) { // Boolean getter can be defined as is###(). + bld = new StringBuilder("is"); + + bld.append(prop); + + bld.setCharAt(2, Character.toUpperCase(bld.charAt(2))); + try { - tmp = new ClassProperty(cls.getDeclaredField(prop), key, alias, coCtx); + tmp = new ClassProperty(cls.getMethod(bld.toString()), key, alias, coCtx); } - catch (NoSuchFieldException ignored) { + catch (NoSuchMethodException ignore) { // No-op. } } + Class cls0 = cls; + + while (tmp == null && cls0 != null) + try { + tmp = new ClassProperty(cls0.getDeclaredField(prop), key, alias, coCtx); + } + catch (NoSuchFieldException ignored) { + cls0 = cls0.getSuperclass(); + } + if (tmp == null) { try { tmp = new ClassProperty(cls.getMethod(prop), key, alias, coCtx); From f9a0676fad7fd6c23e3c91c10d7e0412ccb27c06 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Tue, 11 Oct 2016 10:23:01 +0300 Subject: [PATCH 232/487] IGNITE-4041: Created separate processor for thread pools and refactored IO manager. This closes #1150. --- .../ignite/internal/GridKernalContext.java | 15 ++ .../internal/GridKernalContextImpl.java | 24 +++ .../apache/ignite/internal/IgniteKernal.java | 6 + .../apache/ignite/internal/IgnitionEx.java | 19 ++- .../managers/communication/GridIoManager.java | 161 ++---------------- .../processors/closure/GridClosurePolicy.java | 51 ------ .../closure/GridClosureProcessor.java | 88 +++------- .../processors/pool/PoolProcessor.java | 149 ++++++++++++++++ .../resources/META-INF/classnames.properties | 1 - .../managers/GridManagerStopSelfTest.java | 2 + .../future/GridFutureAdapterSelfTest.java | 3 + .../junits/GridTestKernalContext.java | 1 + 12 files changed, 250 insertions(+), 270 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosurePolicy.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java index b123a4a49cc95..e608af2a09767 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java @@ -53,6 +53,7 @@ import org.apache.ignite.internal.processors.offheap.GridOffHeapProcessor; import org.apache.ignite.internal.processors.platform.PlatformProcessor; import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor; +import org.apache.ignite.internal.processors.pool.PoolProcessor; import org.apache.ignite.internal.processors.port.GridPortProcessor; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.processors.resource.GridResourceProcessor; @@ -285,6 +286,13 @@ public interface GridKernalContext extends Iterable { */ public HadoopProcessorAdapter hadoop(); + /** + * Gets pool processor. + * + * @return Pool processor. + */ + public PoolProcessor pools(); + /** * Gets Hadoop helper. * @@ -532,6 +540,13 @@ public interface GridKernalContext extends Iterable { */ public ExecutorService getRestExecutorService(); + /** + * Get affinity executor service. + * + * @return Affinity executor service. + */ + public ExecutorService getAffinityExecutorService(); + /** * Gets exception registry. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java index c7e26e9d70619..ddef345ffd151 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java @@ -70,6 +70,7 @@ import org.apache.ignite.internal.processors.offheap.GridOffHeapProcessor; import org.apache.ignite.internal.processors.platform.PlatformProcessor; import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor; +import org.apache.ignite.internal.processors.pool.PoolProcessor; import org.apache.ignite.internal.processors.port.GridPortProcessor; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.processors.resource.GridResourceProcessor; @@ -257,6 +258,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable @GridToStringExclude private HadoopProcessorAdapter hadoopProc; + /** */ + @GridToStringExclude + private PoolProcessor poolProc; + /** */ @GridToStringExclude private IgnitePluginProcessor pluginProc; @@ -309,6 +314,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable @GridToStringExclude protected ExecutorService restExecSvc; + /** */ + @GridToStringExclude + protected ExecutorService affExecSvc; + /** */ @GridToStringExclude protected IgniteStripedThreadPoolExecutor callbackExecSvc; @@ -372,6 +381,7 @@ public GridKernalContextImpl() { * @param mgmtExecSvc Management executor service. * @param igfsExecSvc IGFS executor service. * @param restExecSvc REST executor service. + * @param affExecSvc Affinity executor service. * @param plugins Plugin providers. * @throws IgniteCheckedException In case of error. */ @@ -389,6 +399,7 @@ protected GridKernalContextImpl( ExecutorService mgmtExecSvc, ExecutorService igfsExecSvc, ExecutorService restExecSvc, + ExecutorService affExecSvc, IgniteStripedThreadPoolExecutor callbackExecSvc, List plugins) throws IgniteCheckedException { assert grid != null; @@ -406,6 +417,7 @@ protected GridKernalContextImpl( this.mgmtExecSvc = mgmtExecSvc; this.igfsExecSvc = igfsExecSvc; this.restExecSvc = restExecSvc; + this.affExecSvc = affExecSvc; this.callbackExecSvc = callbackExecSvc; String workDir = U.workDirectory(cfg.getWorkDirectory(), cfg.getIgniteHome()); @@ -533,6 +545,8 @@ else if (comp instanceof ClusterProcessor) cluster = (ClusterProcessor)comp; else if (comp instanceof PlatformProcessor) platformProc = (PlatformProcessor)comp; + else if (comp instanceof PoolProcessor) + poolProc = (PoolProcessor) comp; else if (!(comp instanceof DiscoveryNodeValidationProcessor)) assert (comp instanceof GridPluginComponent) : "Unknown manager class: " + comp.getClass(); @@ -756,6 +770,11 @@ else if (helper instanceof HadoopHelper) return hadoopProc; } + /** {@inheritDoc} */ + @Override public PoolProcessor pools() { + return poolProc; + } + /** {@inheritDoc} */ @Override public ExecutorService utilityCachePool() { return utilityCachePool; @@ -941,6 +960,11 @@ protected Object readResolve() throws ObjectStreamException { return restExecSvc; } + /** {@inheritDoc} */ + @Override public ExecutorService getAffinityExecutorService() { + return affExecSvc; + } + /** {@inheritDoc} */ @Override public IgniteExceptionRegistry exceptionRegistry() { return IgniteExceptionRegistry.get(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index e0a36a7e19a41..02f16af35a8f4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -126,6 +126,7 @@ import org.apache.ignite.internal.processors.platform.PlatformNoopProcessor; import org.apache.ignite.internal.processors.platform.PlatformProcessor; import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor; +import org.apache.ignite.internal.processors.pool.PoolProcessor; import org.apache.ignite.internal.processors.port.GridPortProcessor; import org.apache.ignite.internal.processors.port.GridPortRecord; import org.apache.ignite.internal.processors.query.GridQueryProcessor; @@ -666,6 +667,7 @@ private void notifyLifecycleBeansEx(LifecycleEventType evt) { * @param mgmtExecSvc Management executor service. * @param igfsExecSvc IGFS executor service. * @param restExecSvc Reset executor service. + * @param affExecSvc Affinity executor service. * @param errHnd Error handler to use for notification about startup problems. * @throws IgniteCheckedException Thrown in case of any errors. */ @@ -679,6 +681,7 @@ public void start(final IgniteConfiguration cfg, ExecutorService mgmtExecSvc, ExecutorService igfsExecSvc, ExecutorService restExecSvc, + ExecutorService affExecSvc, IgniteStripedThreadPoolExecutor callbackExecSvc, GridAbsClosure errHnd) throws IgniteCheckedException @@ -784,6 +787,7 @@ public void start(final IgniteConfiguration cfg, mgmtExecSvc, igfsExecSvc, restExecSvc, + affExecSvc, callbackExecSvc, plugins); @@ -827,6 +831,8 @@ public void start(final IgniteConfiguration cfg, startProcessor(new IgnitePluginProcessor(ctx, cfg, plugins)); + startProcessor(new PoolProcessor(ctx)); + // Off-heap processor has no dependencies. startProcessor(new GridOffHeapProcessor(ctx)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index 001f599cd0f96..a6860b3a4ccac 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -1477,6 +1477,9 @@ private static final class IgniteNamedInstance { /** Marshaller cache executor service. */ private ThreadPoolExecutor marshCacheExecSvc; + /** Affinity executor service. */ + private ThreadPoolExecutor affExecSvc; + /** Continuous query executor service. */ private IgniteStripedThreadPoolExecutor callbackExecSvc; @@ -1734,6 +1737,16 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { marshCacheExecSvc.allowCoreThreadTimeOut(true); + affExecSvc = new IgniteThreadPoolExecutor( + "aff", + cfg.getGridName(), + 1, + 1, + DFLT_THREAD_KEEP_ALIVE_TIME, + new LinkedBlockingQueue()); + + affExecSvc.allowCoreThreadTimeOut(true); + // Register Ignite MBean for current grid instance. registerFactoryMbean(myCfg.getMBeanServer()); @@ -1746,7 +1759,7 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { grid = grid0; grid0.start(myCfg, utilityCacheExecSvc, marshCacheExecSvc, execSvc, sysExecSvc, p2pExecSvc, mgmtExecSvc, - igfsExecSvc, restExecSvc, callbackExecSvc, + igfsExecSvc, restExecSvc, affExecSvc, callbackExecSvc, new CA() { @Override public void apply() { startLatch.countDown(); @@ -2381,6 +2394,10 @@ private void stopExecutors0(IgniteLogger log) { marshCacheExecSvc = null; + U.shutdownNow(getClass(), affExecSvc, log); + + affExecSvc = null; + U.shutdownNow(getClass(), callbackExecSvc, log); callbackExecSvc = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 4bc2eea40cfdd..bd285b2e3d828 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -29,9 +29,6 @@ import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -54,6 +51,7 @@ import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter; +import org.apache.ignite.internal.processors.pool.PoolProcessor; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashSet; import org.apache.ignite.internal.util.GridSpinReadWriteLock; @@ -68,7 +66,6 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; -import org.apache.ignite.plugin.extensions.communication.IoPool; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.plugin.extensions.communication.MessageFormatter; @@ -78,7 +75,6 @@ import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; -import org.apache.ignite.thread.IgniteThreadPoolExecutor; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; import org.jsr166.ConcurrentLinkedDeque8; @@ -130,32 +126,8 @@ public class GridIoManager extends GridManagerAdapter disconnectLsnrs = new ConcurrentLinkedQueue<>(); - /** Map of {@link IoPool}-s injected by Ignite plugins. */ - private final IoPool[] ioPools = new IoPool[128]; - - /** Public pool. */ - private ExecutorService pubPool; - - /** Internal P2P pool. */ - private ExecutorService p2pPool; - - /** Internal system pool. */ - private ExecutorService sysPool; - - /** Internal management pool. */ - private ExecutorService mgmtPool; - - /** Affinity assignment executor service. */ - private ExecutorService affPool; - - /** Utility cache pool. */ - private ExecutorService utilityCachePool; - - /** Marshaller cache pool. */ - private ExecutorService marshCachePool; - - /** IGFS pool. */ - private ExecutorService igfsPool; + /** Pool processor. */ + private PoolProcessor pools; /** Discovery listener. */ private GridLocalEventListener discoLsnr; @@ -210,6 +182,10 @@ public class GridIoManager extends GridManagerAdapter()); - getSpi().setListener(commLsnr = new CommunicationListener() { @Override public void onMessage(UUID nodeId, Serializable msg, IgniteRunnable msgC) { try { @@ -335,41 +296,6 @@ public void resetMetrics() { if (log.isDebugEnabled()) log.debug(startInfo()); - - registerIoPoolExtensions(); - } - - /** - * Processes IO messaging pool extensions. - * @throws IgniteCheckedException On error. - */ - private void registerIoPoolExtensions() throws IgniteCheckedException { - // Process custom IO messaging pool extensions: - final IoPool[] executorExtensions = ctx.plugins().extensions(IoPool.class); - - if (executorExtensions != null) { - // Store it into the map and check for duplicates: - for (IoPool ex : executorExtensions) { - final byte id = ex.id(); - - // 1. Check the pool id is non-negative: - if (id < 0) - throw new IgniteCheckedException("Failed to register IO executor pool because its Id is negative " + - "[id=" + id + ']'); - - // 2. Check the pool id is in allowed range: - if (isReservedGridIoPolicy(id)) - throw new IgniteCheckedException("Failed to register IO executor pool because its Id in in the " + - "reserved range (0-31) [id=" + id + ']'); - - // 3. Check the pool for duplicates: - if (ioPools[id] != null) - throw new IgniteCheckedException("Failed to register IO executor pool because its " + - "Id as already used [id=" + id + ']'); - - ioPools[id] = ex; - } - } } /** {@inheritDoc} */ @@ -546,8 +472,6 @@ private void registerIoPoolExtensions() throws IgniteCheckedException { if (interrupted) Thread.currentThread().interrupt(); - U.shutdownNow(getClass(), affPool, log); - GridEventStorageManager evtMgr = ctx.event(); if (evtMgr != null && discoLsnr != null) @@ -566,8 +490,6 @@ private void registerIoPoolExtensions() throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug(stopInfo()); - - Arrays.fill(ioPools, null); } /** @@ -682,67 +604,6 @@ private void onMessage0(UUID nodeId, GridIoMessage msg, IgniteRunnable msgC) { } } - /** - * Gets execution pool for policy. - * - * @param plc Policy. - * @return Execution pool. - * @throws IgniteCheckedException If failed. - */ - private Executor pool(byte plc) throws IgniteCheckedException { - switch (plc) { - case P2P_POOL: - return p2pPool; - case SYSTEM_POOL: - return sysPool; - case PUBLIC_POOL: - return pubPool; - case MANAGEMENT_POOL: - return mgmtPool; - case AFFINITY_POOL: - return affPool; - - case UTILITY_CACHE_POOL: - assert utilityCachePool != null : "Utility cache pool is not configured."; - - return utilityCachePool; - - case MARSH_CACHE_POOL: - assert marshCachePool != null : "Marshaller cache pool is not configured."; - - return marshCachePool; - - case IGFS_POOL: - assert igfsPool != null : "IGFS pool is not configured."; - - return igfsPool; - - default: { - assert plc >= 0 : "Negative policy: " + plc; - - if (isReservedGridIoPolicy(plc)) - throw new IgniteCheckedException("Failed to process message with policy of reserved" + - " range (0-31), [policy=" + plc + ']'); - - IoPool pool = ioPools[plc]; - - if (pool == null) - throw new IgniteCheckedException("Failed to process message because no pool is registered " + - "for policy. [policy=" + plc + ']'); - - assert plc == pool.id(); - - Executor ex = pool.executor(); - - if (ex == null) - throw new IgniteCheckedException("Failed to process message because corresponding executor " + - "is null. [id=" + plc + ']'); - - return ex; - } - } - } - /** * @param nodeId Node ID. * @param msg Message. @@ -778,7 +639,7 @@ private void processP2PMessage( }; try { - p2pPool.execute(c); + pools.p2pPool().execute(c); } catch (RejectedExecutionException e) { U.error(log, "Failed to process P2P message due to execution rejection. Increase the upper bound " + @@ -818,7 +679,7 @@ private void processRegularMessage( }; try { - pool(plc).execute(c); + pools.poolForPolicy(plc).execute(c); } catch (RejectedExecutionException e) { U.error(log, "Failed to process regular message due to execution rejection. Increase the upper bound " + @@ -1154,7 +1015,7 @@ else if (log.isDebugEnabled()) { }; try { - pool(plc).execute(c); + pools.poolForPolicy(plc).execute(c); } catch (RejectedExecutionException e) { U.error(log, "Failed to process ordered message due to execution rejection. " + @@ -1781,7 +1642,7 @@ public void addMessageListener(Object topic, final GridMessageListener lsnr) { try { for (final GridCommunicationMessageSet msgSet : msgSets) { - pool(msgSet.policy()).execute( + pools.poolForPolicy(msgSet.policy()).execute( new Runnable() { @Override public void run() { unwindMessageSet(msgSet, lsnrs0); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosurePolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosurePolicy.java deleted file mode 100644 index c17cedd7e10c7..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosurePolicy.java +++ /dev/null @@ -1,51 +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.closure; - -import org.jetbrains.annotations.Nullable; - -/** - * This enumeration defines different types of closure - * processing by the closure processor. - */ -public enum GridClosurePolicy { - /** Public execution pool. */ - PUBLIC_POOL, - - /** P2P execution pool. */ - P2P_POOL, - - /** System execution pool. */ - SYSTEM_POOL, - - /** IGFS pool. */ - IGFS_POOL; - - /** Enum values. */ - private static final GridClosurePolicy[] VALS = values(); - - /** - * Efficiently gets enumerated value from its ordinal. - * - * @param ord Ordinal value. - * @return Enumerated value. - */ - @Nullable public static GridClosurePolicy fromOrdinal(int ord) { - return ord >= 0 && ord < VALS.length ? VALS[ord] : null; - } -} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java index 6f878ce5d9e7e..a96d6eb1daabe 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java @@ -50,8 +50,10 @@ import org.apache.ignite.internal.GridInternalWrapper; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.processors.GridProcessorAdapter; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.pool.PoolProcessor; import org.apache.ignite.internal.processors.resource.GridNoImplicitInjection; import org.apache.ignite.internal.util.GridSpinReadWriteLock; import org.apache.ignite.internal.util.IgniteUtils; @@ -88,14 +90,8 @@ public class GridClosureProcessor extends GridProcessorAdapter { /** Ignite version in which binarylizable versions of closures were introduced. */ public static final IgniteProductVersion BINARYLIZABLE_CLOSURES_SINCE = IgniteProductVersion.fromString("1.6.0"); - /** */ - private final Executor sysPool; - - /** */ - private final Executor pubPool; - - /** */ - private final Executor igfsPool; + /** Pool processor. */ + private final PoolProcessor pools; /** Lock to control execution after stop. */ private final GridSpinReadWriteLock busyLock = new GridSpinReadWriteLock(); @@ -109,9 +105,9 @@ public class GridClosureProcessor extends GridProcessorAdapter { public GridClosureProcessor(GridKernalContext ctx) { super(ctx); - sysPool = ctx.getSystemExecutorService(); - pubPool = ctx.getExecutorService(); - igfsPool = ctx.getIgfsExecutorService(); + pools = ctx.pools(); + + assert pools != null; } /** {@inheritDoc} */ @@ -730,50 +726,6 @@ public ComputeTaskInternalFuture callAsync(IgniteClosure } } - /** - * Gets pool by execution policy. - * - * @param plc Whether to get system or public pool. - * @return Requested worker pool. - */ - private Executor pool(GridClosurePolicy plc) { - switch (plc) { - case PUBLIC_POOL: - return pubPool; - - case SYSTEM_POOL: - return sysPool; - - case IGFS_POOL: - return igfsPool; - - default: - throw new IllegalArgumentException("Invalid closure execution policy: " + plc); - } - } - - /** - * Gets pool name by execution policy. - * - * @param plc Policy to choose executor pool. - * @return Pool name. - */ - private String poolName(GridClosurePolicy plc) { - switch (plc) { - case PUBLIC_POOL: - return "public"; - - case SYSTEM_POOL: - return "system"; - - case IGFS_POOL: - return "igfs"; - - default: - throw new IllegalArgumentException("Invalid closure execution policy: " + plc); - } - } - /** * @param c Closure to execute. * @param sys If {@code true}, then system pool will be used, otherwise public pool will be used. @@ -781,7 +733,7 @@ private String poolName(GridClosurePolicy plc) { * @throws IgniteCheckedException Thrown in case of any errors. */ private IgniteInternalFuture runLocal(@Nullable final Runnable c, boolean sys) throws IgniteCheckedException { - return runLocal(c, sys ? GridClosurePolicy.SYSTEM_POOL : GridClosurePolicy.PUBLIC_POOL); + return runLocal(c, sys ? GridIoPolicy.SYSTEM_POOL : GridIoPolicy.PUBLIC_POOL); } /** @@ -790,7 +742,8 @@ private IgniteInternalFuture runLocal(@Nullable final Runnable c, boolean sys * @return Future. * @throws IgniteCheckedException Thrown in case of any errors. */ - private IgniteInternalFuture runLocal(@Nullable final Runnable c, GridClosurePolicy plc) throws IgniteCheckedException { + private IgniteInternalFuture runLocal(@Nullable final Runnable c, byte plc) + throws IgniteCheckedException { if (c == null) return new GridFinishedFuture(); @@ -830,11 +783,11 @@ private IgniteInternalFuture runLocal(@Nullable final Runnable c, GridClosure fut.setWorker(w); try { - pool(plc).execute(w); + pools.poolForPolicy(plc).execute(w); } catch (RejectedExecutionException e) { U.error(log, "Failed to execute worker due to execution rejection " + - "(increase upper bound on " + poolName(plc) + " executor service).", e); + "(increase upper bound on executor service) [policy=" + plc + ']', e); w.run(); } @@ -866,7 +819,7 @@ public IgniteInternalFuture runLocalSafe(Runnable c) { * @return Future. */ public IgniteInternalFuture runLocalSafe(Runnable c, boolean sys) { - return runLocalSafe(c, sys ? GridClosurePolicy.SYSTEM_POOL : GridClosurePolicy.PUBLIC_POOL); + return runLocalSafe(c, sys ? GridIoPolicy.SYSTEM_POOL : GridIoPolicy.PUBLIC_POOL); } /** @@ -877,7 +830,7 @@ public IgniteInternalFuture runLocalSafe(Runnable c, boolean sys) { * @param plc Policy to choose executor pool. * @return Future. */ - public IgniteInternalFuture runLocalSafe(Runnable c, GridClosurePolicy plc) { + public IgniteInternalFuture runLocalSafe(Runnable c, byte plc) { try { return runLocal(c, plc); } @@ -921,7 +874,7 @@ public IgniteInternalFuture runLocalSafe(Runnable c, GridClosurePolicy plc) { * @throws IgniteCheckedException Thrown in case of any errors. */ private IgniteInternalFuture callLocal(@Nullable final Callable c, boolean sys) throws IgniteCheckedException { - return callLocal(c, sys ? GridClosurePolicy.SYSTEM_POOL : GridClosurePolicy.PUBLIC_POOL); + return callLocal(c, sys ? GridIoPolicy.SYSTEM_POOL : GridIoPolicy.PUBLIC_POOL); } /** @@ -931,7 +884,8 @@ private IgniteInternalFuture callLocal(@Nullable final Callable c, boo * @return Future. * @throws IgniteCheckedException Thrown in case of any errors. */ - private IgniteInternalFuture callLocal(@Nullable final Callable c, GridClosurePolicy plc) throws IgniteCheckedException { + private IgniteInternalFuture callLocal(@Nullable final Callable c, byte plc) + throws IgniteCheckedException { if (c == null) return new GridFinishedFuture<>(); @@ -969,11 +923,11 @@ private IgniteInternalFuture callLocal(@Nullable final Callable c, Gri fut.setWorker(w); try { - pool(plc).execute(w); + pools.poolForPolicy(plc).execute(w); } catch (RejectedExecutionException e) { U.error(log, "Failed to execute worker due to execution rejection " + - "(increase upper bound on " + poolName(plc) + " executor service).", e); + "(increase upper bound on executor service) [policy=" + plc + ']', e); w.run(); } @@ -1005,7 +959,7 @@ public IgniteInternalFuture callLocalSafe(Callable c) { * @return Future. */ public IgniteInternalFuture callLocalSafe(Callable c, boolean sys) { - return callLocalSafe(c, sys ? GridClosurePolicy.SYSTEM_POOL : GridClosurePolicy.PUBLIC_POOL); + return callLocalSafe(c, sys ? GridIoPolicy.SYSTEM_POOL : GridIoPolicy.PUBLIC_POOL); } /** @@ -1016,7 +970,7 @@ public IgniteInternalFuture callLocalSafe(Callable c, boolean sys) { * @param plc Policy to choose executor pool. * @return Future. */ - public IgniteInternalFuture callLocalSafe(Callable c, GridClosurePolicy plc) { + private IgniteInternalFuture callLocalSafe(Callable c, byte plc) { try { return callLocal(c, plc); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java new file mode 100644 index 0000000000000..41e805ef9c515 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java @@ -0,0 +1,149 @@ +/* + * 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.pool; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.managers.communication.GridIoPolicy; +import org.apache.ignite.internal.processors.GridProcessorAdapter; +import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor; +import org.apache.ignite.plugin.extensions.communication.IoPool; + +import java.util.Arrays; +import java.util.concurrent.Executor; + +/** + * Processor which abstracts out thread pool management. + */ +public class PoolProcessor extends GridProcessorAdapter { + /** Map of {@link IoPool}-s injected by Ignite plugins. */ + private final IoPool[] extPools = new IoPool[128]; + + /** + * Constructor. + * + * @param ctx Kernal context. + */ + public PoolProcessor(GridKernalContext ctx) { + super(ctx); + + IgnitePluginProcessor plugins = ctx.plugins(); + + if (plugins != null) { + // Process custom IO messaging pool extensions: + final IoPool[] executorExtensions = ctx.plugins().extensions(IoPool.class); + + if (executorExtensions != null) { + // Store it into the map and check for duplicates: + for (IoPool ex : executorExtensions) { + final byte id = ex.id(); + + // 1. Check the pool id is non-negative: + if (id < 0) + throw new IgniteException("Failed to register IO executor pool because its ID is " + + "negative: " + id); + + // 2. Check the pool id is in allowed range: + if (GridIoPolicy.isReservedGridIoPolicy(id)) + throw new IgniteException("Failed to register IO executor pool because its ID in in the " + + "reserved range: " + id); + + // 3. Check the pool for duplicates: + if (extPools[id] != null) + throw new IgniteException("Failed to register IO executor pool because its ID as " + + "already used: " + id); + + extPools[id] = ex; + } + } + } + } + + /** {@inheritDoc} */ + @Override public void stop(boolean cancel) throws IgniteCheckedException { + // Avoid external thread pools GC retention. + Arrays.fill(extPools, null); + } + + /** + * @return P2P pool. + */ + public Executor p2pPool() { + return ctx.getPeerClassLoadingExecutorService(); + } + + /** + * Get executor service for policy. + * + * @param plc Policy. + * @return Executor service. + * @throws IgniteCheckedException If failed. + */ + public Executor poolForPolicy(byte plc) throws IgniteCheckedException { + switch (plc) { + case GridIoPolicy.P2P_POOL: + return ctx.getPeerClassLoadingExecutorService(); + case GridIoPolicy.SYSTEM_POOL: + return ctx.getSystemExecutorService(); + case GridIoPolicy.PUBLIC_POOL: + return ctx.getExecutorService(); + case GridIoPolicy.MANAGEMENT_POOL: + return ctx.getManagementExecutorService(); + case GridIoPolicy.AFFINITY_POOL: + return ctx.getAffinityExecutorService(); + + case GridIoPolicy.UTILITY_CACHE_POOL: + assert ctx.utilityCachePool() != null : "Utility cache pool is not configured."; + + return ctx.utilityCachePool(); + + case GridIoPolicy.MARSH_CACHE_POOL: + assert ctx.marshallerCachePool() != null : "Marshaller cache pool is not configured."; + + return ctx.marshallerCachePool(); + + case GridIoPolicy.IGFS_POOL: + assert ctx.getIgfsExecutorService() != null : "IGFS pool is not configured."; + + return ctx.getIgfsExecutorService(); + + default: { + if (plc < 0) + throw new IgniteCheckedException("Policy cannot be negative: " + plc); + + if (GridIoPolicy.isReservedGridIoPolicy(plc)) + throw new IgniteCheckedException("Policy is reserved for internal usage (range 0-31): " + plc); + + IoPool pool = extPools[plc]; + + if (pool == null) + throw new IgniteCheckedException("No pool is registered for policy: " + plc); + + assert plc == pool.id(); + + Executor res = pool.executor(); + + if (res == null) + throw new IgniteCheckedException("Thread pool for policy is null: " + plc); + + return res; + } + } + } +} diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties index 57762c7dda131..2d8d2457c263b 100644 --- a/modules/core/src/main/resources/META-INF/classnames.properties +++ b/modules/core/src/main/resources/META-INF/classnames.properties @@ -948,7 +948,6 @@ org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessorImpl org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessorImpl$UserKeyCacheObjectImpl org.apache.ignite.internal.processors.clock.GridClockDeltaSnapshotMessage org.apache.ignite.internal.processors.clock.GridClockDeltaVersion -org.apache.ignite.internal.processors.closure.GridClosurePolicy org.apache.ignite.internal.processors.closure.GridClosureProcessor$C1 org.apache.ignite.internal.processors.closure.GridClosureProcessor$C1MLA org.apache.ignite.internal.processors.closure.GridClosureProcessor$C1MLAV2 diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerStopSelfTest.java index 04eb8efa942de..328b775bf74bb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerStopSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/GridManagerStopSelfTest.java @@ -27,6 +27,7 @@ import org.apache.ignite.internal.managers.failover.GridFailoverManager; import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager; import org.apache.ignite.internal.managers.swapspace.GridSwapSpaceManager; +import org.apache.ignite.internal.processors.pool.PoolProcessor; import org.apache.ignite.internal.processors.resource.GridResourceProcessor; import org.apache.ignite.marshaller.optimized.OptimizedMarshaller; import org.apache.ignite.resources.LoggerResource; @@ -68,6 +69,7 @@ public GridManagerStopSelfTest() { ctx.config().setPeerClassLoadingEnabled(true); + ctx.add(new PoolProcessor(ctx)); ctx.add(new GridResourceProcessor(ctx)); ctx.start(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java index f44d282215cae..adcd144966318 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java @@ -28,6 +28,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException; import org.apache.ignite.internal.processors.closure.GridClosureProcessor; +import org.apache.ignite.internal.processors.pool.PoolProcessor; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.CX1; import org.apache.ignite.testframework.GridTestUtils; @@ -174,6 +175,7 @@ public void testListenNotify() throws Exception { ctx.setExecutorService(Executors.newFixedThreadPool(1)); ctx.setSystemExecutorService(Executors.newFixedThreadPool(1)); + ctx.add(new PoolProcessor(ctx)); ctx.add(new GridClosureProcessor(ctx)); ctx.start(); @@ -238,6 +240,7 @@ public void testChaining() throws Exception { ctx.setExecutorService(Executors.newFixedThreadPool(1)); ctx.setSystemExecutorService(Executors.newFixedThreadPool(1)); + ctx.add(new PoolProcessor(ctx)); ctx.add(new GridClosureProcessor(ctx)); ctx.start(); diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java index e4afe73f22f74..cba67e0d06005 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java @@ -61,6 +61,7 @@ public GridTestKernalContext(IgniteLogger log, IgniteConfiguration cfg) throws I null, null, null, + null, U.allPluginProviders()); GridTestUtils.setFieldValue(grid(), "cfg", config()); From 9a6cfce659df40b0a4624f19fd91c217b74bafea Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Tue, 11 Oct 2016 13:59:57 +0300 Subject: [PATCH 233/487] IGNITE-4014 Fixed "Transaction hangs if entry processor failed during serialization". This closes #1148. --- .../RendezvousAffinityFunction.java | 2 +- .../cache/store/jdbc/CacheJdbcBlobStore.java | 4 +- .../internal/GridEventConsumeHandler.java | 8 +- .../internal/GridJobSiblingsResponse.java | 3 +- .../internal/GridMessageListenHandler.java | 8 +- .../internal/binary/BinaryEnumObjectImpl.java | 3 +- .../internal/binary/BinaryMarshaller.java | 2 +- .../internal/binary/BinaryWriterExImpl.java | 3 +- .../GridClientOptimizedMarshaller.java | 5 +- .../GridClientZipOptimizedMarshaller.java | 5 +- .../ignite/internal/jdbc/JdbcUtils.java | 4 +- .../checkpoint/GridCheckpointManager.java | 6 +- .../managers/communication/GridIoManager.java | 12 +- .../GridDeploymentCommunication.java | 4 +- .../eventstorage/GridEventStorageManager.java | 16 +- .../swapspace/GridSwapSpaceManager.java | 13 +- .../affinity/GridAffinityUtils.java | 4 +- .../CacheEntrySerializablePredicate.java | 4 +- .../cache/CacheInvokeDirectResult.java | 4 +- .../processors/cache/GridCacheMessage.java | 4 +- .../processors/cache/GridCacheProcessor.java | 13 +- .../processors/cache/GridCacheUtils.java | 2 +- .../GridDistributedLockResponse.java | 4 +- .../GridDistributedTxPrepareRequest.java | 4 +- .../GridDistributedTxPrepareResponse.java | 4 +- .../GridDhtAffinityAssignmentResponse.java | 12 +- .../dht/GridDhtTxFinishResponse.java | 4 +- .../atomic/GridDhtAtomicUpdateResponse.java | 4 +- .../atomic/GridNearAtomicUpdateRequest.java | 2 +- .../atomic/GridNearAtomicUpdateResponse.java | 4 +- .../preloader/GridDhtForceKeysResponse.java | 4 +- .../GridDhtPartitionDemandMessage.java | 4 +- .../GridDhtPartitionsFullMessage.java | 8 +- .../GridDhtPartitionsSingleMessage.java | 8 +- .../distributed/near/GridNearGetResponse.java | 4 +- .../near/GridNearSingleGetResponse.java | 4 +- .../near/GridNearTxFinishResponse.java | 4 +- .../query/GridCacheLocalQueryFuture.java | 4 +- .../cache/query/GridCacheQueryRequest.java | 12 +- .../cache/query/GridCacheQueryResponse.java | 4 +- .../cache/query/GridCacheSqlQuery.java | 4 +- .../CacheContinuousQueryHandler.java | 4 +- .../jdbc/GridCacheQueryJdbcMetadataTask.java | 4 +- .../query/jdbc/GridCacheQueryJdbcTask.java | 6 +- .../cache/transactions/IgniteTxEntry.java | 5 +- .../version/GridCacheRawVersionedEntry.java | 8 +- .../IgniteCacheObjectProcessorImpl.java | 3 +- .../closure/GridClosureProcessor.java | 8 +- .../continuous/GridContinuousProcessor.java | 6 +- .../continuous/StartRequestData.java | 4 +- .../datastreamer/DataStreamProcessor.java | 8 +- .../datastreamer/DataStreamerImpl.java | 6 +- .../processors/igfs/IgfsAckMessage.java | 5 +- .../processors/igfs/IgfsDeleteMessage.java | 5 +- .../internal/processors/igfs/IgfsPaths.java | 4 +- .../processors/job/GridJobProcessor.java | 18 +- .../processors/job/GridJobWorker.java | 8 +- .../offheap/GridOffHeapProcessor.java | 4 +- .../handlers/task/GridTaskCommandHandler.java | 10 +- .../rest/protocols/tcp/GridTcpRestParser.java | 4 +- .../service/GridServiceProcessor.java | 9 +- .../processors/task/GridTaskProcessor.java | 8 +- .../processors/task/GridTaskWorker.java | 14 +- .../ignite/internal/util/IgniteUtils.java | 199 ++++++ .../ignite/marshaller/jdk/JdkMarshaller.java | 8 +- .../optimized/OptimizedMarshaller.java | 14 +- .../checkpoint/sharedfs/SharedFsUtils.java | 4 +- .../ignite/spi/discovery/tcp/ClientImpl.java | 8 +- .../ignite/spi/discovery/tcp/ServerImpl.java | 24 +- .../spi/discovery/tcp/TcpDiscoverySpi.java | 8 +- .../TcpDiscoveryMulticastIpFinder.java | 4 +- .../TcpDiscoveryCustomEventMessage.java | 3 +- .../spi/swapspace/file/FileSwapSpaceSpi.java | 2 +- .../ignite/stream/socket/SocketStreamer.java | 3 +- ...ntryProcessorExternalizableFailedTest.java | 588 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite.java | 2 + .../communication/HadoopMarshallerFilter.java | 13 +- .../hibernate/CacheHibernateBlobStore.java | 4 +- .../processors/query/h2/IgniteH2Indexing.java | 4 +- .../serialization/CacheEventDeserializer.java | 3 +- .../serialization/CacheEventSerializer.java | 3 +- .../ignite/cache/websession/WebSessionV2.java | 20 +- 82 files changed, 1040 insertions(+), 252 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorExternalizableFailedTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java index aa8680c3f886b..ec12973ceefd7 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java @@ -370,7 +370,7 @@ public List assignPartition(int part, List nodes, int try { ByteArrayOutputStream out = new ByteArrayOutputStream(); - byte[] nodeHashBytes = ignite.configuration().getMarshaller().marshal(nodeHash); + byte[] nodeHashBytes = U.marshal(ignite.configuration().getMarshaller(), nodeHash); out.write(U.intToBytes(part), 0, 4); // Avoid IOException. out.write(nodeHashBytes, 0, nodeHashBytes.length); // Avoid IOException. diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStore.java index 25f0f37a2ccea..273986544005f 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStore.java @@ -560,7 +560,7 @@ public void setDataSource(DataSource dataSrc) { * @throws IgniteCheckedException If failed to convert. */ protected byte[] toBytes(Object obj) throws IgniteCheckedException { - return marsh.marshal(obj); + return U.marshal(marsh, obj); } /** @@ -575,7 +575,7 @@ protected X fromBytes(byte[] bytes) throws IgniteCheckedException { if (bytes == null || bytes.length == 0) return null; - return marsh.unmarshal(bytes, getClass().getClassLoader()); + return U.unmarshal(marsh, bytes, getClass().getClassLoader()); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java index ed6998d5b8ab1..715f8a5e48581 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java @@ -365,7 +365,7 @@ public GridEventConsumeHandler() { depInfo = new GridDeploymentInfoBean(dep); - filterBytes = ctx.config().getMarshaller().marshal(filter); + filterBytes = U.marshal(ctx.config().getMarshaller(), filter); } } @@ -382,7 +382,7 @@ public GridEventConsumeHandler() { if (dep == null) throw new IgniteDeploymentCheckedException("Failed to obtain deployment for class: " + clsName); - filter = ctx.config().getMarshaller().unmarshal(filterBytes, U.resolveClassLoader(dep.classLoader(), ctx.config())); + filter = U.unmarshal(ctx, filterBytes, U.resolveClassLoader(dep.classLoader(), ctx.config())); } } @@ -490,7 +490,7 @@ public EventWrapper() { void p2pMarshal(Marshaller marsh) throws IgniteCheckedException { assert marsh != null; - bytes = marsh.marshal(evt); + bytes = U.marshal(marsh, evt); } /** @@ -504,7 +504,7 @@ void p2pUnmarshal(Marshaller marsh, @Nullable ClassLoader ldr) throws IgniteChec assert evt == null; assert bytes != null; - evt = marsh.unmarshal(bytes, ldr); + evt = U.unmarshal(marsh, bytes, ldr); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java index aa81cc5853789..727e79bac00aa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java @@ -23,6 +23,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.compute.ComputeJobSibling; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageReader; @@ -74,7 +75,7 @@ public void unmarshalSiblings(Marshaller marsh) throws IgniteCheckedException { assert marsh != null; if (siblingsBytes != null) - siblings = marsh.unmarshal(siblingsBytes, null); + siblings = U.unmarshal(marsh, siblingsBytes, null); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java index 1bca85cacfd7d..0eeaa8aa2f400 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java @@ -154,9 +154,9 @@ public GridMessageListenHandler(GridMessageListenHandler orig) { assert ctx.config().isPeerClassLoadingEnabled(); if (topic != null) - topicBytes = ctx.config().getMarshaller().marshal(topic); + topicBytes = U.marshal(ctx.config().getMarshaller(), topic); - predBytes = ctx.config().getMarshaller().marshal(pred); + predBytes = U.marshal(ctx.config().getMarshaller(), pred); // Deploy only listener, as it is very likely to be of some user class. GridPeerDeployAware pda = U.peerDeployAware(pred); @@ -188,9 +188,9 @@ public GridMessageListenHandler(GridMessageListenHandler orig) { ClassLoader ldr = dep.classLoader(); if (topicBytes != null) - topic = ctx.config().getMarshaller().unmarshal(topicBytes, U.resolveClassLoader(ldr, ctx.config())); + topic = U.unmarshal(ctx, topicBytes, U.resolveClassLoader(ldr, ctx.config())); - pred = ctx.config().getMarshaller().unmarshal(predBytes, U.resolveClassLoader(ldr, ctx.config())); + pred = U.unmarshal(ctx, predBytes, U.resolveClassLoader(ldr, ctx.config())); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java index dcfcc9d39f40f..69de3f2cdc8f1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java @@ -32,6 +32,7 @@ import org.apache.ignite.internal.processors.cache.CacheObjectContext; import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; import org.apache.ignite.internal.util.typedef.internal.SB; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; import org.jetbrains.annotations.Nullable; @@ -247,7 +248,7 @@ public BinaryEnumObjectImpl(BinaryContext ctx, byte[] arr) { /** {@inheritDoc} */ @Override public byte[] valueBytes(CacheObjectContext cacheCtx) throws IgniteCheckedException { - return ctx.marshaller().marshal(this); + return U.marshal(ctx.marshaller(), this); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java index 168c61abee373..bd9510345201a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java @@ -90,7 +90,7 @@ private void setBinaryContext(BinaryContext ctx, IgniteConfiguration cfg) { try { out.write(arr); } - catch (IOException e) { + catch (Exception e) { throw new BinaryObjectException("Failed to marshal the object: " + obj, e); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java index 22b4d1f481f5d..1de0a6533e600 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java @@ -36,6 +36,7 @@ import org.apache.ignite.internal.binary.streams.BinaryOutputStream; import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; import static java.nio.charset.StandardCharsets.UTF_8; @@ -174,7 +175,7 @@ private void marshal0(Object obj, boolean enableReplace) throws BinaryObjectExce out.writeByte(GridBinaryMarshaller.OPTM_MARSH); try { - byte[] arr = ctx.optimizedMarsh().marshal(obj); + byte[] arr = U.marshal(ctx.optimizedMarsh(), obj); writeInt(arr.length); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java index a112736079a9b..3c65db6b73ab4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java @@ -25,6 +25,7 @@ import org.apache.ignite.internal.MarshallerContextAdapter; import org.apache.ignite.internal.client.marshaller.GridClientMarshaller; import org.apache.ignite.internal.processors.rest.client.message.GridClientMessage; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.optimized.OptimizedMarshaller; import org.apache.ignite.plugin.PluginProvider; import org.jetbrains.annotations.Nullable; @@ -84,7 +85,7 @@ public GridClientOptimizedMarshaller(boolean requireSer, int poolSize) throws IO throw new IOException("Message serialization of given type is not supported: " + obj.getClass().getName()); - byte[] bytes = opMarsh.marshal(obj); + byte[] bytes = U.marshal(opMarsh, obj); ByteBuffer buf = ByteBuffer.allocate(off + bytes.length); @@ -104,7 +105,7 @@ public GridClientOptimizedMarshaller(boolean requireSer, int poolSize) throws IO /** {@inheritDoc} */ @Override public T unmarshal(byte[] bytes) throws IOException { try { - return opMarsh.unmarshal(bytes, null); + return U.unmarshal(opMarsh, bytes, null); } catch (IgniteCheckedException e) { throw new IOException(e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientZipOptimizedMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientZipOptimizedMarshaller.java index d9ce60e291258..b7096340ca857 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientZipOptimizedMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientZipOptimizedMarshaller.java @@ -30,6 +30,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.client.marshaller.GridClientMarshaller; import org.apache.ignite.internal.processors.rest.client.message.GridClientMessage; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.optimized.OptimizedMarshaller; import org.apache.ignite.plugin.PluginProvider; import org.jetbrains.annotations.Nullable; @@ -131,7 +132,7 @@ private static byte[] unzipBytes(byte[] input) throws IOException { throw new IOException("Message serialization of given type is not supported: " + obj.getClass().getName()); - byte[] marshBytes = opMarsh.marshal(obj); + byte[] marshBytes = U.marshal(opMarsh, obj); boolean zip = marshBytes.length > 512; @@ -158,7 +159,7 @@ private static byte[] unzipBytes(byte[] input) throws IOException { byte[] marshBytes = Arrays.copyOfRange(bytes, 1, bytes.length); - return opMarsh.unmarshal(unzip ? unzipBytes(marshBytes) : marshBytes, null); + return U.unmarshal(opMarsh, unzip ? unzipBytes(marshBytes) : marshBytes, null); } catch (IgniteCheckedException e) { throw new IOException(e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/JdbcUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/JdbcUtils.java index ecea21f9b50a1..04e4926fcf705 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/JdbcUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/JdbcUtils.java @@ -67,7 +67,7 @@ public static byte[] marshalArgument(Map args) throws SQLExcepti assert args != null; try { - return MARSHALLER.marshal(args); + return U.marshal(MARSHALLER, args); } catch (IgniteCheckedException e) { throw new SQLException("Failed to unmarshal result.", e); @@ -96,7 +96,7 @@ public static T unmarshal(byte[] bytes) throws SQLException { assert bytes != null; try { - return MARSHALLER.unmarshal(bytes, null); + return U.unmarshal(MARSHALLER, bytes, null); } catch (IgniteCheckedException e) { throw new SQLException("Failed to unmarshal result.", e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java index 6b95dadb504b0..9124cafa79a6b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java @@ -173,7 +173,7 @@ public boolean storeCheckpoint(GridTaskSessionInternal ses, try { switch (scope) { case GLOBAL_SCOPE: { - byte[] data = state == null ? null : marsh.marshal(state); + byte[] data = state == null ? null : U.marshal(marsh, state); saved = getSpi(ses.getCheckpointSpi()).saveCheckpoint(key, data, timeout, override); @@ -204,7 +204,7 @@ public boolean storeCheckpoint(GridTaskSessionInternal ses, timeout = ses.getEndTime() - now; // Save it first to avoid getting null value on another node. - byte[] data = state == null ? null : marsh.marshal(state); + byte[] data = state == null ? null : U.marshal(marsh, state); Set keys = keyMap.get(ses.getId()); @@ -338,7 +338,7 @@ else if (log.isDebugEnabled()) // Always deserialize with task/session class loader. if (data != null) - state = marsh.unmarshal(data, U.resolveClassLoader(ses.getClassLoader(), ctx.config())); + state = U.unmarshal(marsh, data, U.resolveClassLoader(ses.getClassLoader(), ctx.config())); record(EVT_CHECKPOINT_LOADED, key); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index bd285b2e3d828..cd43318c455cd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -527,7 +527,7 @@ private void onMessage0(UUID nodeId, GridIoMessage msg, IgniteRunnable msgC) { int topicOrd = msg.topicOrdinal(); msg.topic(topicOrd >= 0 ? GridTopic.fromOrdinal(topicOrd) : - marsh.unmarshal(msg.topicBytes(), U.resolveClassLoader(ctx.config()))); + U.unmarshal(marsh, msg.topicBytes(), U.resolveClassLoader(ctx.config()))); } if (!started) { @@ -1139,7 +1139,7 @@ private void send( } else { if (topicOrd < 0) - ioMsg.topicBytes(marsh.marshal(topic)); + ioMsg.topicBytes(U.marshal(marsh, topic)); try { if ((CommunicationSpi)getSpi() instanceof TcpCommunicationSpi) @@ -1399,10 +1399,10 @@ public void sendUserMessage(Collection nodes, Object msg, byte[] serTopic = null; if (!loc) { - serMsg = marsh.marshal(msg); + serMsg = U.marshal(marsh, msg); if (topic != null) - serTopic = marsh.marshal(topic); + serTopic = U.marshal(marsh, topic); } GridDeployment dep = null; @@ -2059,7 +2059,7 @@ private class GridUserMessageListener implements GridMessageListener { // Unmarshall message topic if needed. if (msgTopic == null && msgTopicBytes != null) { - msgTopic = marsh.unmarshal(msgTopicBytes, + msgTopic = U.unmarshal(marsh, msgTopicBytes, U.resolveClassLoader(dep != null ? dep.classLoader() : null, ctx.config())); ioMsg.topic(msgTopic); // Save topic to avoid future unmarshallings. @@ -2069,7 +2069,7 @@ private class GridUserMessageListener implements GridMessageListener { return; if (msgBody == null) { - msgBody = marsh.unmarshal(ioMsg.bodyBytes(), + msgBody = U.unmarshal(marsh, ioMsg.bodyBytes(), U.resolveClassLoader(dep != null ? dep.classLoader() : null, ctx.config())); ioMsg.body(msgBody); // Save body to avoid future unmarshallings. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java index d4b35688fc314..a571ae445438f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java @@ -182,7 +182,7 @@ private void processResourceRequest(UUID nodeId, GridDeploymentRequest req) { if (req.responseTopic() == null) { try { - req.responseTopic(marsh.unmarshal(req.responseTopicBytes(), U.resolveClassLoader(ctx.config()))); + req.responseTopic(U.unmarshal(marsh, req.responseTopicBytes(), U.resolveClassLoader(ctx.config()))); } catch (IgniteCheckedException e) { U.error(log, "Failed to process deployment request (will ignore): " + req, e); @@ -443,7 +443,7 @@ GridDeploymentResponse sendResourceRequest(final String rsrcName, IgniteUuid cls long start = U.currentTimeMillis(); if (req.responseTopic() != null && !ctx.localNodeId().equals(dstNode.id())) - req.responseTopicBytes(marsh.marshal(req.responseTopic())); + req.responseTopicBytes(U.marshal(marsh, req.responseTopic())); ctx.io().send(dstNode, TOPIC_CLASSLOAD, req, GridIoPolicy.P2P_POOL); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java index 0095707fe42a1..5b451a17a0c3f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java @@ -891,11 +891,11 @@ private List query(IgnitePredicate p, Collection>unmarshal(res.eventsBytes(), + res.events(U.>unmarshal(marsh, res.eventsBytes(), U.resolveClassLoader(ctx.config()))); if (res.exceptionBytes() != null) - res.exception(marsh.unmarshal(res.exceptionBytes(), + res.exception(U.unmarshal(marsh, res.exceptionBytes(), U.resolveClassLoader(ctx.config()))); } catch (IgniteCheckedException e) { @@ -932,7 +932,7 @@ private List query(IgnitePredicate p, Collection nodes, GridTopic topi ctx.io().send(locNode, topic, msg, plc); if (!rmtNodes.isEmpty()) { - msg.responseTopicBytes(marsh.marshal(msg.responseTopic())); + msg.responseTopicBytes(U.marshal(marsh, msg.responseTopic())); ctx.io().send(rmtNodes, topic, msg, plc); } @@ -1089,7 +1089,7 @@ private class RequestListener implements GridMessageListener { try { if (req.responseTopicBytes() != null) - req.responseTopic(marsh.unmarshal(req.responseTopicBytes(), U.resolveClassLoader(ctx.config()))); + req.responseTopic(U.unmarshal(marsh, req.responseTopicBytes(), U.resolveClassLoader(ctx.config()))); GridDeployment dep = ctx.deploy().getGlobalDeployment( req.deploymentMode(), @@ -1105,7 +1105,7 @@ private class RequestListener implements GridMessageListener { throw new IgniteDeploymentCheckedException("Failed to obtain deployment for event filter " + "(is peer class loading turned on?): " + req); - filter = marsh.unmarshal(req.filter(), U.resolveClassLoader(dep.classLoader(), ctx.config())); + filter = U.unmarshal(marsh, req.filter(), U.resolveClassLoader(dep.classLoader(), ctx.config())); // Resource injection. ctx.resource().inject(dep, dep.deployedClass(req.filterClassName()), filter); @@ -1140,8 +1140,8 @@ private class RequestListener implements GridMessageListener { log.debug("Sending event query response to node [nodeId=" + nodeId + "res=" + res + ']'); if (!ctx.localNodeId().equals(nodeId)) { - res.eventsBytes(marsh.marshal(res.events())); - res.exceptionBytes(marsh.marshal(res.exception())); + res.eventsBytes(U.marshal(marsh, res.events())); + res.exceptionBytes(U.marshal(marsh, res.exception())); } ctx.io().send(node, req.responseTopic(), res, PUBLIC_POOL); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/swapspace/GridSwapSpaceManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/swapspace/GridSwapSpaceManager.java index 437603a120d32..1d36b4e0cdece 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/swapspace/GridSwapSpaceManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/swapspace/GridSwapSpaceManager.java @@ -400,18 +400,7 @@ private T unmarshal(byte[] swapBytes, @Nullable ClassLoader ldr) throws Igni if (swapBytes == null) return null; - return marsh.unmarshal(swapBytes, ldr != null ? ldr : U.gridClassLoader()); - } - - /** - * Marshals object. - * - * @param obj Object to marshal. - * @return Marshalled array. - * @throws IgniteCheckedException If failed. - */ - private byte[] marshal(Object obj) throws IgniteCheckedException { - return ctx.config().getMarshaller().marshal(obj); + return U.unmarshal(marsh, swapBytes, ldr != null ? ldr : U.gridClassLoader()); } /** 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 2952ebccf75d2..c24dd2de7d2ee 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 @@ -77,7 +77,7 @@ private static GridAffinityMessage affinityMessage(GridKernalContext ctx, Object throw new IgniteDeploymentCheckedException("Failed to deploy affinity object with class: " + cls.getName()); return new GridAffinityMessage( - ctx.config().getMarshaller().marshal(o), + U.marshal(ctx, o), cls.getName(), dep.classLoaderId(), dep.deployMode(), @@ -110,7 +110,7 @@ static Object unmarshall(GridKernalContext ctx, UUID sndNodeId, GridAffinityMess throw new IgniteDeploymentCheckedException("Failed to obtain affinity object (is peer class loading turned on?): " + msg); - Object src = ctx.config().getMarshaller().unmarshal(msg.source(), + Object src = U.unmarshal(ctx, msg.source(), U.resolveClassLoader(dep.classLoader(), ctx.config())); // Resource injection. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntrySerializablePredicate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntrySerializablePredicate.java index 5f52cf78b7ebb..623fc4bdf8454 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntrySerializablePredicate.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntrySerializablePredicate.java @@ -80,7 +80,7 @@ public CacheEntryPredicate predicate() { assert p != null || bytes != null; if (p == null) { - p = ctx.marshaller().unmarshal(bytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + p = U.unmarshal(ctx.marshaller(), bytes, U.resolveClassLoader(ldr, ctx.gridConfig())); p.finishUnmarshal(ctx, ldr); } @@ -93,7 +93,7 @@ public CacheEntryPredicate predicate() { p.prepareMarshal(ctx); if (bytes == null) - bytes = ctx.marshaller().marshal(p); + bytes = U.marshal(ctx.marshaller(), p); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java index 5dea51b93f62e..0d519f712d34b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeDirectResult.java @@ -106,7 +106,7 @@ public void prepareMarshal(GridCacheContext ctx) throws IgniteCheckedException { key.prepareMarshal(ctx.cacheObjectContext()); if (err != null && errBytes == null) - errBytes = ctx.marshaller().marshal(err); + errBytes = U.marshal(ctx.marshaller(), err); if (res != null) res.prepareMarshal(ctx.cacheObjectContext()); @@ -121,7 +121,7 @@ public void finishUnmarshal(GridCacheContext ctx, ClassLoader ldr) throws Ignite key.finishUnmarshal(ctx.cacheObjectContext(), ldr); if (errBytes != null && err == null) - err = ctx.marshaller().unmarshal(errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + err = U.unmarshal(ctx.marshaller(), errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); if (res != null) res.finishUnmarshal(ctx.cacheObjectContext(), ldr); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java index c5407b8ba30ce..71f99d3c47da0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java @@ -458,7 +458,7 @@ protected final void unmarshalTx(Iterable txEntries, Marshaller marsh = ctx.marshaller(); for (int i = 0; i < byteCol.length; i++) - args[i] = byteCol[i] == null ? null : marsh.unmarshal(byteCol[i], U.resolveClassLoader(ldr, ctx.gridConfig())); + args[i] = byteCol[i] == null ? null : U.unmarshal(marsh, byteCol[i], U.resolveClassLoader(ldr, ctx.gridConfig())); return args; } @@ -609,7 +609,7 @@ protected final void finishUnmarshalCacheObjects(@Nullable Collectionunmarshal(bytes, U.resolveClassLoader(ldr, ctx.gridConfig()))); + col.add(bytes == null ? null : U.unmarshal(marsh, bytes, U.resolveClassLoader(ldr, ctx.gridConfig()))); return col; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 0a0b40aa01415..fd6abbd104872 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -117,7 +117,6 @@ import org.apache.ignite.lifecycle.LifecycleAware; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.MarshallerUtils; -import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.spi.IgniteNodeValidationResult; import org.jetbrains.annotations.Nullable; @@ -185,8 +184,8 @@ public class GridCacheProcessor extends GridProcessorAdapter { /** */ private IdentityHashMap sesHolders = new IdentityHashMap<>(); - /** Must use JDK marshaller since it is used by discovery to fire custom events. */ - private final Marshaller marshaller; + /** Must use JDK marsh since it is used by discovery to fire custom events. */ + private final Marshaller marsh; /** Count down latch for caches. */ private final CountDownLatch cacheStartedLatch = new CountDownLatch(1); @@ -207,7 +206,7 @@ public GridCacheProcessor(GridKernalContext ctx) { jCacheProxies = new ConcurrentHashMap<>(); stopSeq = new LinkedList<>(); - marshaller = MarshallerUtils.jdkMarshaller(ctx.gridName()); + marsh = MarshallerUtils.jdkMarshaller(ctx.gridName()); } /** @@ -3563,7 +3562,7 @@ private CacheConfiguration cloneCheckSerializable(final CacheConfiguration val) if (ldr == null) ldr = val.getCacheStoreFactory().getClass().getClassLoader(); - marshaller.unmarshal(marshaller.marshal(val.getCacheStoreFactory()), + U.unmarshal(marsh, U.marshal(marsh, val.getCacheStoreFactory()), U.resolveClassLoader(ldr, ctx.config())); } catch (IgniteCheckedException e) { @@ -3573,7 +3572,7 @@ private CacheConfiguration cloneCheckSerializable(final CacheConfiguration val) } try { - return marshaller.unmarshal(marshaller.marshal(val), U.resolveClassLoader(ctx.config())); + return U.unmarshal(marsh, U.marshal(marsh, val), U.resolveClassLoader(ctx.config())); } catch (IgniteCheckedException e) { throw new IgniteCheckedException("Failed to validate cache configuration " + @@ -3615,7 +3614,7 @@ private T withBinaryContext(IgniteOutClosureX c) throws IgniteCheckedExce public T clone(final T obj) throws IgniteCheckedException { return withBinaryContext(new IgniteOutClosureX() { @Override public T applyx() throws IgniteCheckedException { - return marshaller.unmarshal(marshaller.marshal(obj), U.resolveClassLoader(ctx.config())); + return U.unmarshal(marsh, U.marshal(marsh, obj), U.resolveClassLoader(ctx.config())); } }); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 0f4e89bc89963..4c18f213bb131 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -895,7 +895,7 @@ else if (obj.getClass().isArray()) { } } - return ctx.marshaller().marshal(obj); + return U.marshal(ctx, obj); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockResponse.java index 1763ff93d9e24..61a23882cd722 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockResponse.java @@ -194,7 +194,7 @@ protected int valuesSize() { prepareMarshalCacheObjects(vals, ctx.cacheContext(cacheId)); if (err != null && errBytes == null) - errBytes = ctx.marshaller().marshal(err); + errBytes = U.marshal(ctx.marshaller(), err); } /** {@inheritDoc} */ @@ -204,7 +204,7 @@ protected int valuesSize() { finishUnmarshalCacheObjects(vals, ctx.cacheContext(cacheId), ldr); if (errBytes != null) - err = ctx.marshaller().unmarshal(errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java index c6913742399cb..7b1dcc5589151 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java @@ -334,7 +334,7 @@ public boolean onePhaseCommit() { // Marshal txNodes only if there is a node in topology with an older version. if (ctx.exchange().minimumNodeVersion(topologyVersion()).compareTo(TX_NODES_DIRECT_MARSHALLABLE_SINCE) < 0) { if (txNodes != null && txNodesBytes == null) - txNodesBytes = ctx.marshaller().marshal(txNodes); + txNodesBytes = U.marshal(ctx, txNodes); } else { if (txNodesMsg == null) @@ -374,7 +374,7 @@ public boolean onePhaseCommit() { txNodes = F.viewReadOnly(txNodesMsg, MSG_TO_COL); if (txNodesBytes != null && txNodes == null) - txNodes = ctx.marshaller().unmarshal(txNodesBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + txNodes = U.unmarshal(ctx, txNodesBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareResponse.java index 850c095b216e4..6c127c385b588 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareResponse.java @@ -100,7 +100,7 @@ public boolean isRollback() { super.prepareMarshal(ctx); if (err != null && errBytes == null) - errBytes = ctx.marshaller().marshal(err); + errBytes = U.marshal(ctx, err); } /** {@inheritDoc} */ @@ -108,7 +108,7 @@ public boolean isRollback() { super.finishUnmarshal(ctx, ldr); if (errBytes != null && err == null) - err = ctx.marshaller().unmarshal(errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java index e883614c49806..cad1617cdc9e5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java @@ -207,13 +207,13 @@ private List> ids(List> assignments) { assert affAssignment != null ^ affAssignmentIds != null; if (affAssignment != null && affAssignmentBytes == null) - affAssignmentBytes = ctx.marshaller().marshal(affAssignment); + affAssignmentBytes = U.marshal(ctx, affAssignment); if (affAssignmentIds != null && affAssignmentIdsBytes == null) - affAssignmentIdsBytes = ctx.marshaller().marshal(affAssignmentIds); + affAssignmentIdsBytes = U.marshal(ctx, affAssignmentIds); if (idealAffAssignment != null && idealAffAssignmentBytes == null) - idealAffAssignmentBytes = ctx.marshaller().marshal(idealAffAssignment); + idealAffAssignmentBytes = U.marshal(ctx, idealAffAssignment); } /** {@inheritDoc} */ @@ -228,10 +228,10 @@ private List> ids(List> assignments) { affAssignment = unmarshalNodes(affAssignmentBytes, ctx, ldr); if (affAssignmentIdsBytes != null && affAssignmentIds == null) - affAssignmentIds = ctx.marshaller().unmarshal(affAssignmentIdsBytes, ldr); + affAssignmentIds = U.unmarshal(ctx, affAssignmentIdsBytes, ldr); if (idealAffAssignmentBytes != null && idealAffAssignment == null) - idealAffAssignment = ctx.marshaller().unmarshal(idealAffAssignmentBytes, ldr); + idealAffAssignment = U.unmarshal(ctx, idealAffAssignmentBytes, ldr); } /** @@ -247,7 +247,7 @@ private List> unmarshalNodes(byte[] bytes, ClassLoader ldr) throws IgniteCheckedException { - List> affAssignment = ctx.marshaller().unmarshal(bytes, + List> affAssignment = U.unmarshal(ctx, bytes, U.resolveClassLoader(ldr, ctx.gridConfig())); // TODO IGNITE-2110: setting 'local' for nodes not needed when IGNITE-2110 is implemented. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java index 06181729a6c1d..8fb1f4e5df80c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java @@ -115,7 +115,7 @@ public void checkCommitted(boolean checkCommitted) { super.prepareMarshal(ctx); if (checkCommittedErr != null && checkCommittedErrBytes == null) - checkCommittedErrBytes = ctx.marshaller().marshal(checkCommittedErr); + checkCommittedErrBytes = U.marshal(ctx, checkCommittedErr); if (retVal != null && retVal.cacheId() != 0) { GridCacheContext cctx = ctx.cacheContext(retVal.cacheId()); @@ -132,7 +132,7 @@ public void checkCommitted(boolean checkCommitted) { super.finishUnmarshal(ctx, ldr); if (checkCommittedErrBytes != null && checkCommittedErr == null) - checkCommittedErr = ctx.marshaller().unmarshal(checkCommittedErrBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + checkCommittedErr = U.unmarshal(ctx, checkCommittedErrBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); if (retVal != null && retVal.cacheId() != 0) { GridCacheContext cctx = ctx.cacheContext(retVal.cacheId()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java index 133481907388f..ff12af0dd02e4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java @@ -168,7 +168,7 @@ public void addNearEvicted(KeyCacheObject key) { prepareMarshalCacheObjects(nearEvicted, cctx); if (errBytes == null) - errBytes = ctx.marshaller().marshal(err); + errBytes = U.marshal(ctx, err); } /** {@inheritDoc} */ @@ -182,7 +182,7 @@ public void addNearEvicted(KeyCacheObject key) { finishUnmarshalCacheObjects(nearEvicted, cctx, ldr); if (errBytes != null && err == null) - err = ctx.marshaller().unmarshal(errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java index eb9be4de06e11..5057fbbb68371 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java @@ -659,7 +659,7 @@ public boolean onResponse(GridNearAtomicUpdateResponse res) { } if (expiryPlcBytes != null && expiryPlc == null) - expiryPlc = ctx.marshaller().unmarshal(expiryPlcBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + expiryPlc = U.unmarshal(ctx, expiryPlcBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); if (partIds != null && !partIds.isEmpty()) { assert partIds.size() == keys.size(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java index ff4008e200fc1..2e38733e8b900 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java @@ -397,7 +397,7 @@ public synchronized void addFailedKeys(Collection keys, Throwabl super.prepareMarshal(ctx); if (err != null && errBytes == null) - errBytes = ctx.marshaller().marshal(err); + errBytes = U.marshal(ctx, err); GridCacheContext cctx = ctx.cacheContext(cacheId); @@ -416,7 +416,7 @@ public synchronized void addFailedKeys(Collection keys, Throwabl super.finishUnmarshal(ctx, ldr); if (errBytes != null && err == null) - err = ctx.marshaller().unmarshal(errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); GridCacheContext cctx = ctx.cacheContext(cacheId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java index fcdf40241f748..8d90158700075 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java @@ -172,7 +172,7 @@ public void addInfo(GridCacheEntryInfo info) { } if (err != null && errBytes == null) - errBytes = ctx.marshaller().marshal(err); + errBytes = U.marshal(ctx, err); } /** {@inheritDoc} */ @@ -190,7 +190,7 @@ public void addInfo(GridCacheEntryInfo info) { } if (errBytes != null && err == null) - err = ctx.marshaller().unmarshal(errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java index d90ada8b119d1..bb952d4359e15 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java @@ -185,7 +185,7 @@ void workerId(int workerId) { super.prepareMarshal(ctx); if (topic != null && topicBytes == null) - topicBytes = ctx.marshaller().marshal(topic); + topicBytes = U.marshal(ctx, topic); } /** {@inheritDoc} */ @@ -193,7 +193,7 @@ void workerId(int workerId) { super.finishUnmarshal(ctx, ldr); if (topicBytes != null && topic == null) - topic = ctx.marshaller().unmarshal(topicBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + topic = U.unmarshal(ctx, topicBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java index 49942a122fcfc..a4ff04b4ea282 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java @@ -133,10 +133,10 @@ public Map partitionUpdateCounters(int cacheId) { super.prepareMarshal(ctx); if (parts != null && partsBytes == null) - partsBytes = ctx.marshaller().marshal(parts); + partsBytes = U.marshal(ctx, parts); if (partCntrs != null && partCntrsBytes == null) - partCntrsBytes = ctx.marshaller().marshal(partCntrs); + partCntrsBytes = U.marshal(ctx, partCntrs); } /** @@ -158,13 +158,13 @@ public void topologyVersion(AffinityTopologyVersion topVer) { super.finishUnmarshal(ctx, ldr); if (partsBytes != null && parts == null) - parts = ctx.marshaller().unmarshal(partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + parts = U.unmarshal(ctx, partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); if (parts == null) parts = new HashMap<>(); if (partCntrsBytes != null && partCntrs == null) - partCntrs = ctx.marshaller().unmarshal(partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + partCntrs = U.unmarshal(ctx, partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); if (partCntrs == null) partCntrs = new HashMap<>(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java index a2be0b0cede16..e4356b17fbbdc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java @@ -137,10 +137,10 @@ public Map partitions() { super.prepareMarshal(ctx); if (partsBytes == null && parts != null) - partsBytes = ctx.marshaller().marshal(parts); + partsBytes = U.marshal(ctx, parts); if (partCntrsBytes == null && partCntrs != null) - partCntrsBytes = ctx.marshaller().marshal(partCntrs); + partCntrsBytes = U.marshal(ctx, partCntrs); } /** {@inheritDoc} */ @@ -148,10 +148,10 @@ public Map partitions() { super.finishUnmarshal(ctx, ldr); if (partsBytes != null && parts == null) - parts = ctx.marshaller().unmarshal(partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + parts = U.unmarshal(ctx, partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); if (partCntrsBytes != null && partCntrs == null) - partCntrs = ctx.marshaller().unmarshal(partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + partCntrs = U.unmarshal(ctx, partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java index f298515c805b3..049431e5cddcc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java @@ -190,7 +190,7 @@ public void error(IgniteCheckedException err) { } if (err != null && errBytes == null) - errBytes = ctx.marshaller().marshal(err); + errBytes = U.marshal(ctx, err); } /** {@inheritDoc} */ @@ -205,7 +205,7 @@ public void error(IgniteCheckedException err) { } if (errBytes != null && err == null) - err = ctx.marshaller().unmarshal(errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetResponse.java index 95799b2793fb2..bbc968db803ed 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetResponse.java @@ -169,7 +169,7 @@ else if (res instanceof GridCacheEntryInfo) } if (err != null && errBytes == null) - errBytes = ctx.marshaller().marshal(err); + errBytes = U.marshal(ctx, err); } /** {@inheritDoc} */ @@ -188,7 +188,7 @@ else if (res instanceof GridCacheEntryInfo) } if (errBytes != null && err == null) - err = ctx.marshaller().unmarshal(errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishResponse.java index 0bdb09b7749ab..37fbb36485e22 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishResponse.java @@ -101,7 +101,7 @@ public long threadId() { super.prepareMarshal(ctx); if (err != null && errBytes == null) - errBytes = ctx.marshaller().marshal(err); + errBytes = U.marshal(ctx, err); } /** {@inheritDoc} */ @@ -109,7 +109,7 @@ public long threadId() { super.finishUnmarshal(ctx, ldr); if (errBytes != null && err == null) - err = ctx.marshaller().unmarshal(errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java index 91ae12cda5efd..6eaca29083026 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java @@ -113,11 +113,11 @@ private GridCacheQueryInfo localQueryInfo() throws IgniteCheckedException { Marshaller marsh = cctx.marshaller(); IgniteReducer rdc = qry.reducer() != null ? - marsh.>unmarshal(marsh.marshal(qry.reducer()), + U.>unmarshal(marsh, U.marshal(marsh, qry.reducer()), U.resolveClassLoader(cctx.gridConfig())) : null; IgniteClosure trans = qry.transform() != null ? - marsh.>unmarshal(marsh.marshal(qry.transform()), + U.>unmarshal(marsh, U.marshal(marsh, qry.transform()), U.resolveClassLoader(cctx.gridConfig())) : null; return new GridCacheQueryInfo( 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 5610befe1d0f4..60c466226fcf7 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 @@ -319,16 +319,16 @@ public GridCacheQueryRequest( Marshaller mrsh = ctx.marshaller(); if (keyValFilterBytes != null && keyValFilter == null) - keyValFilter = mrsh.unmarshal(keyValFilterBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + keyValFilter = U.unmarshal(mrsh, keyValFilterBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); if (rdcBytes != null && rdc == null) - rdc = mrsh.unmarshal(rdcBytes, ldr); + rdc = U.unmarshal(mrsh, rdcBytes, ldr); if (transBytes != null && trans == null) - trans = mrsh.unmarshal(transBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + trans = U.unmarshal(mrsh, transBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); if (argsBytes != null && args == null) - args = mrsh.unmarshal(argsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + args = U.unmarshal(mrsh, argsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } /** {@inheritDoc} */ @@ -343,9 +343,9 @@ public GridCacheQueryRequest( void beforeLocalExecution(GridCacheContext ctx) throws IgniteCheckedException { Marshaller marsh = ctx.marshaller(); - rdc = rdc != null ? marsh.>unmarshal(marsh.marshal(rdc), + rdc = rdc != null ? U.>unmarshal(marsh, U.marshal(marsh, rdc), U.resolveClassLoader(ctx.gridConfig())) : null; - trans = trans != null ? marsh.>unmarshal(marsh.marshal(trans), + trans = trans != null ? U.>unmarshal(marsh, U.marshal(marsh, trans), U.resolveClassLoader(ctx.gridConfig())) : null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java index 9a03bdcc3fdae..8492c380b1a75 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java @@ -124,7 +124,7 @@ public GridCacheQueryResponse(int cacheId, long reqId, Throwable err, boolean ad GridCacheContext cctx = ctx.cacheContext(cacheId); if (err != null && errBytes == null) - errBytes = ctx.marshaller().marshal(err); + errBytes = U.marshal(ctx, err); if (metaDataBytes == null) metaDataBytes = marshalCollection(metadata, cctx); @@ -149,7 +149,7 @@ public GridCacheQueryResponse(int cacheId, long reqId, Throwable err, boolean ad super.finishUnmarshal(ctx, ldr); if (errBytes != null && err == null) - err = ctx.marshaller().unmarshal(errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); if (metadata == null) metadata = unmarshalCollection(metaDataBytes, ctx, ldr); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java index bcb37c5ec8cc7..4dd3636b39971 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java @@ -141,7 +141,7 @@ public void marshallParams(Marshaller m) throws IgniteCheckedException { assert params != null; - paramsBytes = m.marshal(params); + paramsBytes = U.marshal(m, params); } /** @@ -160,7 +160,7 @@ public void unmarshallParams(Marshaller m, GridKernalContext ctx) throws IgniteC // To avoid deserializing of enum types. params = ((BinaryMarshaller)m).binaryMarshaller().unmarshal(paramsBytes, ldr); else - params = m.unmarshal(paramsBytes, ldr); + params = U.unmarshal(m, paramsBytes, ldr); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index a5752ed47eb92..4b5074c82431b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -1562,7 +1562,7 @@ protected DeployableObject(Object obj, GridKernalContext ctx) throws IgniteCheck depInfo = new GridDeploymentInfoBean(dep); - bytes = ctx.config().getMarshaller().marshal(obj); + bytes = U.marshal(ctx, obj); } /** @@ -1580,7 +1580,7 @@ T unmarshal(UUID nodeId, GridKernalContext ctx) throws IgniteCheckedExceptio if (dep == null) throw new IgniteDeploymentCheckedException("Failed to obtain deployment for class: " + clsName); - return ctx.config().getMarshaller().unmarshal(bytes, U.resolveClassLoader(dep.classLoader(), ctx.config())); + return U.unmarshal(ctx, bytes, U.resolveClassLoader(dep.classLoader(), ctx.config())); } /** {@inheritDoc} */ 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 a8b2da40161b1..e95622c602430 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 @@ -153,7 +153,7 @@ private JdbcDriverMetadataJob(@Nullable String cacheName) { status = 0; - data = MARSHALLER.marshal(F.asList(schemasMap, indexesInfo)); + data = U.marshal(MARSHALLER, F.asList(schemasMap, indexesInfo)); } catch (Throwable t) { U.error(log, "Failed to get metadata for JDBC.", t); @@ -163,7 +163,7 @@ private JdbcDriverMetadataJob(@Nullable String cacheName) { status = 1; try { - data = MARSHALLER.marshal(err); + data = U.marshal(MARSHALLER, err); } catch (IgniteCheckedException e) { throw new IgniteException(e); 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 26d4b823c4cd1..ca08ead4f11eb 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 @@ -84,7 +84,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter { try { assert arg != null; - Map args = MARSHALLER.unmarshal(arg, null); + Map args = U.unmarshal(MARSHALLER, arg, null); boolean first = true; @@ -165,12 +165,12 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter { if (res.getException() == null) { status = 0; - bytes = MARSHALLER.marshal(res.getData()); + bytes = U.marshal(MARSHALLER, res.getData()); } else { status = 1; - bytes = MARSHALLER.marshal(new SQLException(res.getException().getMessage())); + bytes = U.marshal(MARSHALLER, new SQLException(res.getException().getMessage())); } byte[] packet = new byte[bytes.length + 1]; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java index 194208e7dfca9..fde1f52fbc4fd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java @@ -922,8 +922,7 @@ else if (!cacheCtx.isNear() && near) // Unmarshal transform closure anyway if it exists. if (transformClosBytes != null && entryProcessorsCol == null) - entryProcessorsCol = ctx.marshaller().unmarshal(transformClosBytes, - U.resolveClassLoader(clsLdr, ctx.gridConfig())); + entryProcessorsCol = U.unmarshal(ctx, transformClosBytes, U.resolveClassLoader(clsLdr, ctx.gridConfig())); if (filters == null) filters = CU.empty0(); @@ -941,7 +940,7 @@ else if (!cacheCtx.isNear() && near) val.unmarshal(this.ctx, clsLdr); if (expiryPlcBytes != null && expiryPlc == null) - expiryPlc = ctx.marshaller().unmarshal(expiryPlcBytes, U.resolveClassLoader(clsLdr, ctx.gridConfig())); + expiryPlc = U.unmarshal(ctx, expiryPlcBytes, U.resolveClassLoader(clsLdr, ctx.gridConfig())); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheRawVersionedEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheRawVersionedEntry.java index 4c5a70430cf83..b7c96b4f83a7d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheRawVersionedEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheRawVersionedEntry.java @@ -191,7 +191,7 @@ public void unmarshal(CacheObjectContext ctx, Marshaller marsh) throws IgniteChe unmarshalKey(ctx, marsh); if (val == null && valBytes != null) { - val = marsh.unmarshal(valBytes, U.resolveClassLoader(ctx.kernalContext().config())); + val = U.unmarshal(marsh, valBytes, U.resolveClassLoader(ctx.kernalContext().config())); val.finishUnmarshal(ctx, null); } @@ -222,7 +222,7 @@ public void unmarshalKey(CacheObjectContext ctx, Marshaller marsh) throws Ignite if (key == null) { assert keyBytes != null; - key = marsh.unmarshal(keyBytes, U.resolveClassLoader(ctx.kernalContext().config())); + key = U.unmarshal(marsh, keyBytes, U.resolveClassLoader(ctx.kernalContext().config())); key.finishUnmarshal(ctx, null); } @@ -239,13 +239,13 @@ public void marshal(CacheObjectContext ctx, Marshaller marsh) throws IgniteCheck if (keyBytes == null) { key.prepareMarshal(ctx); - keyBytes = marsh.marshal(key); + keyBytes = U.marshal(marsh, key); } if (valBytes == null && val != null) { val.prepareMarshal(ctx); - valBytes = marsh.marshal(val); + valBytes = U.marshal(marsh, val); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java index 9fd4c1f35a01a..0eac610ec64e4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java @@ -103,8 +103,7 @@ public IgniteCacheObjectProcessorImpl(GridKernalContext ctx) { /** {@inheritDoc} */ @Override public Object unmarshal(CacheObjectContext ctx, byte[] bytes, ClassLoader clsLdr) throws IgniteCheckedException { - return ctx.kernalContext().cache().context().marshaller().unmarshal(bytes, U.resolveClassLoader(clsLdr, - ctx.kernalContext().config())); + return U.unmarshal(ctx.kernalContext(), bytes, U.resolveClassLoader(clsLdr, ctx.kernalContext().config())); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java index a96d6eb1daabe..d388584846125 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java @@ -1127,16 +1127,16 @@ public void map(@NotNull ComputeJob job, @NotNull ClusterNode node) throws Ignit if (closureBytes == null) { closure = c.job; - closureBytes = marsh.marshal(c.job); + closureBytes = U.marshal(marsh, c.job); } if (c.job == closure) - c.job = marsh.unmarshal(closureBytes, U.resolveClassLoader(ctx.config())); + c.job = U.unmarshal(marsh, closureBytes, U.resolveClassLoader(ctx.config())); else - c.job = marsh.unmarshal(marsh.marshal(c.job), U.resolveClassLoader(ctx.config())); + c.job = U.unmarshal(marsh, U.marshal(marsh, c.job), U.resolveClassLoader(ctx.config())); } else - job = marsh.unmarshal(marsh.marshal(job), U.resolveClassLoader(ctx.config())); + job = U.unmarshal(marsh, U.marshal(marsh, job), U.resolveClassLoader(ctx.config())); } else hadLocNode = true; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java index ad7ad4fd70db8..f078b1bd01d23 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java @@ -293,7 +293,7 @@ public GridContinuousProcessor(GridKernalContext ctx) { if (msg.data() == null && msg.dataBytes() != null) { try { - msg.data(marsh.unmarshal(msg.dataBytes(), U.resolveClassLoader(ctx.config()))); + msg.data(U.unmarshal(marsh, msg.dataBytes(), U.resolveClassLoader(ctx.config()))); } catch (IgniteCheckedException e) { U.error(log, "Failed to process message (ignoring): " + msg, e); @@ -729,7 +729,7 @@ private void registerMessageListener(GridContinuousHandler hnd) { if (msg.data() == null && msg.dataBytes() != null) { try { - msg.data(marsh.unmarshal(msg.dataBytes(), U.resolveClassLoader(ctx.config()))); + msg.data(U.unmarshal(marsh, msg.dataBytes(), U.resolveClassLoader(ctx.config()))); } catch (IgniteCheckedException e) { U.error(log, "Failed to process message (ignoring): " + msg, e); @@ -1316,7 +1316,7 @@ private void sendWithRetries(Collection nodes, GridContin if (!msg.messages() && msg.data() != null && (nodes.size() > 1 || !ctx.localNodeId().equals(F.first(nodes).id()))) - msg.dataBytes(marsh.marshal(msg.data())); + msg.dataBytes(U.marshal(marsh, msg.data())); for (ClusterNode node : nodes) { int cnt = 0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestData.java index cdfe0e17a44a0..2314f2946ee1e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestData.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestData.java @@ -95,7 +95,7 @@ public StartRequestData() { void p2pMarshal(Marshaller marsh) throws IgniteCheckedException { assert marsh != null; - prjPredBytes = marsh.marshal(prjPred); + prjPredBytes = U.marshal(marsh, prjPred); } /** @@ -109,7 +109,7 @@ void p2pUnmarshal(Marshaller marsh, @Nullable ClassLoader ldr) throws IgniteChec assert prjPred == null; assert prjPredBytes != null; - prjPred = marsh.unmarshal(prjPredBytes, ldr); + prjPred = U.unmarshal(marsh, prjPredBytes, ldr); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java index c7c1f5e1886a5..7663735fc26c6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java @@ -90,7 +90,7 @@ public DataStreamProcessor(GridKernalContext ctx) { if (ctx.config().isDaemon()) return; - marshErrBytes = marsh.marshal(new IgniteCheckedException("Failed to marshal response error, " + + marshErrBytes = U.marshal(marsh, new IgniteCheckedException("Failed to marshal response error, " + "see node log for details.")); flusher = new IgniteThread(new GridWorker(ctx.gridName(), "grid-data-loader-flusher", log) { @@ -235,7 +235,7 @@ private void processRequest(final UUID nodeId, final DataStreamerRequest req) { Object topic; try { - topic = marsh.unmarshal(req.responseTopicBytes(), U.resolveClassLoader(null, ctx.config())); + topic = U.unmarshal(marsh, req.responseTopicBytes(), U.resolveClassLoader(null, ctx.config())); } catch (IgniteCheckedException e) { U.error(log, "Failed to unmarshal topic from request: " + req, e); @@ -275,7 +275,7 @@ private void processRequest(final UUID nodeId, final DataStreamerRequest req) { StreamReceiver updater; try { - updater = marsh.unmarshal(req.updaterBytes(), U.resolveClassLoader(clsLdr, ctx.config())); + updater = U.unmarshal(marsh, req.updaterBytes(), U.resolveClassLoader(clsLdr, ctx.config())); if (updater != null) ctx.resource().injectGeneric(updater); @@ -329,7 +329,7 @@ private void sendResponse(UUID nodeId, Object resTopic, long reqId, @Nullable Th byte[] errBytes; try { - errBytes = err != null ? marsh.marshal(err) : null; + errBytes = err != null ? U.marshal(marsh, err) : null; } catch (Exception e) { U.error(log, "Failed to marshal error [err=" + err + ", marshErr=" + e + ']', e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java index 05e6488cc21f3..c2f226c537025 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java @@ -1353,11 +1353,11 @@ private void submit(final Collection entries, if (updaterBytes == null) { assert rcvr != null; - updaterBytes = ctx.config().getMarshaller().marshal(rcvr); + updaterBytes = U.marshal(ctx, rcvr); } if (topicBytes == null) - topicBytes = ctx.config().getMarshaller().marshal(topic); + topicBytes = U.marshal(ctx, topic); } catch (IgniteCheckedException e) { U.error(log, "Failed to marshal (request will not be sent).", e); @@ -1488,7 +1488,7 @@ void onResponse(DataStreamerResponse res) { try { GridPeerDeployAware jobPda0 = jobPda; - err = ctx.config().getMarshaller().unmarshal( + err = U.unmarshal(ctx, errBytes, U.resolveClassLoader(jobPda0 != null ? jobPda0.classLoader() : null, ctx.config())); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAckMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAckMessage.java index bca459258bcfc..275e7eb7de9d0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAckMessage.java @@ -21,6 +21,7 @@ import java.nio.ByteBuffer; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.GridDirectTransient; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MessageReader; @@ -96,7 +97,7 @@ public IgniteCheckedException error() { super.prepareMarshal(marsh); if (err != null && errBytes == null) - errBytes = marsh.marshal(err); + errBytes = U.marshal(marsh, err); } /** {@inheritDoc} */ @@ -104,7 +105,7 @@ public IgniteCheckedException error() { super.finishUnmarshal(marsh, ldr); if (errBytes != null && err == null) - err = marsh.unmarshal(errBytes, ldr); + err = U.unmarshal(marsh, errBytes, ldr); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteMessage.java index e59b25764d145..3224c201c5df3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteMessage.java @@ -22,6 +22,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.GridDirectTransient; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MessageReader; @@ -95,7 +96,7 @@ public IgniteCheckedException error() { super.prepareMarshal(marsh); if (err != null) - errBytes = marsh.marshal(err); + errBytes = U.marshal(marsh, err); } /** {@inheritDoc} */ @@ -103,7 +104,7 @@ public IgniteCheckedException error() { super.finishUnmarshal(marsh, ldr); if (errBytes != null) - err = marsh.unmarshal(errBytes, ldr); + err = U.unmarshal(marsh, errBytes, ldr); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java index 72b5a19d8244d..975241199f2da 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPaths.java @@ -74,7 +74,7 @@ public IgfsPaths(Object payload, IgfsMode dfltMode, @Nullable ArrayList attrs) throws Ignite boolean loc = ctx.localNodeId().equals(taskNode.id()) && !ctx.config().isMarshalLocalJobs(); GridTaskSessionRequest req = new GridTaskSessionRequest(ses.getId(), ses.getJobId(), - loc ? null : marsh.marshal(attrs), attrs); + loc ? null : U.marshal(marsh, attrs), attrs); Object topic = TOPIC_TASK.topic(ses.getJobId(), ctx.discovery().localNode().id()); @@ -521,7 +521,7 @@ else if (!nodeId.equals(taskNodeId)) ctx.io().send(taskNode, TOPIC_JOB_SIBLINGS, new GridJobSiblingsRequest(ses.getId(), loc ? topic : null, - loc ? null : marsh.marshal(topic)), + loc ? null : U.marshal(marsh, topic)), SYSTEM_POOL); // 4. Listen to discovery events. @@ -1028,7 +1028,7 @@ public void processJobExecuteRequest(ClusterNode node, final GridJobExecuteReque if (siblings0 == null) { assert req.getSiblingsBytes() != null; - siblings0 = marsh.unmarshal(req.getSiblingsBytes(), U.resolveClassLoader(ctx.config())); + siblings0 = U.unmarshal(marsh, req.getSiblingsBytes(), U.resolveClassLoader(ctx.config())); } siblings = new ArrayList<>(siblings0); @@ -1040,7 +1040,7 @@ public void processJobExecuteRequest(ClusterNode node, final GridJobExecuteReque sesAttrs = req.getSessionAttributes(); if (sesAttrs == null) - sesAttrs = marsh.unmarshal(req.getSessionAttributesBytes(), + sesAttrs = U.unmarshal(marsh, req.getSessionAttributesBytes(), U.resolveClassLoader(dep.classLoader(), ctx.config())); } @@ -1068,7 +1068,7 @@ public void processJobExecuteRequest(ClusterNode node, final GridJobExecuteReque Map jobAttrs = req.getJobAttributes(); if (jobAttrs == null) - jobAttrs = marsh.unmarshal(req.getJobAttributesBytes(), + jobAttrs = U.unmarshal(marsh, req.getJobAttributesBytes(), U.resolveClassLoader(dep.classLoader(), ctx.config())); jobCtx = new GridJobContextImpl(ctx, req.getJobId(), jobAttrs); @@ -1343,11 +1343,11 @@ private void handleException(ClusterNode node, GridJobExecuteRequest req, Ignite locNodeId, req.getSessionId(), req.getJobId(), - loc ? null : marsh.marshal(ex), + loc ? null : U.marshal(marsh, ex), ex, - loc ? null : marsh.marshal(null), + loc ? null : U.marshal(marsh, null), null, - loc ? null : marsh.marshal(null), + loc ? null : U.marshal(marsh, null), null, false, null); @@ -1439,7 +1439,7 @@ private void processTaskSessionRequest(UUID nodeId, GridTaskSessionRequest req) boolean loc = ctx.localNodeId().equals(nodeId) && !ctx.config().isMarshalLocalJobs(); Map attrs = loc ? req.getAttributes() : - (Map)marsh.unmarshal(req.getAttributesBytes(), + (Map)U.unmarshal(marsh, req.getAttributesBytes(), U.resolveClassLoader(ses.getClassLoader(), ctx.config())); if (ctx.event().isRecordable(EVT_TASK_SESSION_ATTR_SET)) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java index 16fadafabe3c4..8169eb1101f4f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java @@ -421,7 +421,7 @@ boolean initialize(GridDeployment dep, Class taskCls) { try { if (job == null) { - job = marsh.unmarshal(jobBytes, U.resolveClassLoader(dep.classLoader(), ctx.config())); + job = U.unmarshal(marsh, jobBytes, U.resolveClassLoader(dep.classLoader(), ctx.config())); // No need to hold reference any more. jobBytes = null; @@ -804,11 +804,11 @@ else if (!internal && ctx.event().isRecordable(EVT_JOB_FINISHED)) ctx.localNodeId(), ses.getId(), ses.getJobId(), - loc ? null : marsh.marshal(ex), + loc ? null : U.marshal(marsh, ex), loc ? ex : null, - loc ? null: marsh.marshal(res), + loc ? null: U.marshal(marsh, res), loc ? res : null, - loc ? null : marsh.marshal(attrs), + loc ? null : U.marshal(marsh, attrs), loc ? attrs : null, isCancelled(), retry ? ctx.cache().context().exchange().readyAffinityVersion() : null); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java index d9d4421a16a1f..b91e9abbe9888 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java @@ -108,7 +108,7 @@ public void create(@Nullable String spaceName, int parts, long init, long max, private byte[] keyBytes(KeyCacheObject key, @Nullable byte[] keyBytes) throws IgniteCheckedException { assert key != null; - return keyBytes != null ? keyBytes : marsh.marshal(key); + return keyBytes != null ? keyBytes : U.marshal(marsh, key); } /** @@ -212,7 +212,7 @@ public void enableEviction(@Nullable String spaceName, int part, KeyCacheObject if (valBytes == null) return null; - return marsh.unmarshal(valBytes, U.resolveClassLoader(ldr, ctx.config())); + return U.unmarshal(marsh, valBytes, U.resolveClassLoader(ldr, ctx.config())); } /** 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 2749e96e07455..947435cb58d83 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 @@ -135,14 +135,13 @@ public GridTaskCommandHandler(final GridKernalContext ctx) { res.error(err.getMessage()); else { res.result(desc.result()); - res.resultBytes(ctx.config().getMarshaller().marshal(desc.result())); + res.resultBytes(U.marshal(ctx, desc.result())); } } else res.found(false); - Object topic = ctx.config().getMarshaller().unmarshal(req.topicBytes(), - U.resolveClassLoader(ctx.config())); + Object topic = U.unmarshal(ctx, req.topicBytes(), U.resolveClassLoader(ctx.config())); ctx.io().send(nodeId, topic, res, SYSTEM_POOL); } @@ -440,8 +439,7 @@ else if (!nodeId.equals(resHolderId)) res = (GridTaskResultResponse)msg; try { - res.result(ctx.config().getMarshaller().unmarshal(res.resultBytes(), - U.resolveClassLoader(ctx.config()))); + res.result(U.unmarshal(ctx, res.resultBytes(), U.resolveClassLoader(ctx.config()))); } catch (IgniteCheckedException e) { U.error(log, "Failed to unmarshal task result: " + res, e); @@ -494,7 +492,7 @@ else if (!nodeId.equals(resHolderId)) // 2. Send message. try { - byte[] topicBytes = ctx.config().getMarshaller().marshal(topic); + byte[] topicBytes = U.marshal(ctx, topic); ctx.io().send(taskNode, TOPIC_REST, new GridTaskResultRequest(taskId, topic, topicBytes), SYSTEM_POOL); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestParser.java index 5beff753d5088..de9851c9d52cc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestParser.java @@ -718,7 +718,7 @@ private Object decodeObj(short flags, byte[] bytes) throws IgniteCheckedExceptio assert bytes != null; if ((flags & SERIALIZED_FLAG) != 0) - return jdkMarshaller.unmarshal(bytes, null); + return U.unmarshal(jdkMarshaller, bytes, null); int masked = flags & 0xff00; @@ -800,7 +800,7 @@ else if (obj instanceof byte[]) { flags |= BYTE_ARR_FLAG; } else { - jdkMarshaller.marshal(obj, out); + U.marshal(jdkMarshaller, obj, out); flags |= SERIALIZED_FLAG; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 7b76c4839be34..527d360e9578f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -464,7 +464,7 @@ public IgniteInternalFuture deploy(ServiceConfiguration cfg) { LazyServiceConfiguration cfg0; try { - byte[] srvcBytes = marsh.marshal(cfg.getService()); + byte[] srvcBytes = U.marshal(marsh, cfg.getService()); cfg0 = new LazyServiceConfiguration(cfg, srvcBytes); } @@ -1144,7 +1144,7 @@ private Service copyAndInject(ServiceConfiguration cfg) throws IgniteCheckedExce if (cfg instanceof LazyServiceConfiguration) { byte[] bytes = ((LazyServiceConfiguration)cfg).serviceBytes(); - Service srvc = m.unmarshal(bytes, U.resolveClassLoader(null, ctx.config())); + Service srvc = U.unmarshal(m, bytes, U.resolveClassLoader(null, ctx.config())); ctx.resource().inject(srvc); @@ -1154,10 +1154,9 @@ private Service copyAndInject(ServiceConfiguration cfg) throws IgniteCheckedExce Service svc = cfg.getService(); try { - byte[] bytes = m.marshal(svc); + byte[] bytes = U.marshal(m, svc); - Service cp = m.unmarshal(bytes, - U.resolveClassLoader(svc.getClass().getClassLoader(), ctx.config())); + Service cp = U.unmarshal(m, bytes, U.resolveClassLoader(svc.getClass().getClassLoader(), ctx.config())); ctx.resource().inject(cp); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java index 6d9722933bcc3..d32b51c0867ca 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java @@ -914,7 +914,7 @@ private void sendSessionAttributes(Map attrs, GridTaskSessionImpl ses) GridTaskSessionRequest req = new GridTaskSessionRequest( ses.getId(), null, - loc ? null : marsh.marshal(attrs), + loc ? null : U.marshal(marsh, attrs), attrs); // Make sure to go through IO manager always, since order @@ -1030,7 +1030,7 @@ private void processTaskSessionRequest(UUID nodeId, GridTaskSessionRequest msg) boolean loc = ctx.localNodeId().equals(nodeId) && !ctx.config().isMarshalLocalJobs(); Map attrs = loc ? msg.getAttributes() : - marsh.>unmarshal(msg.getAttributesBytes(), + U.>unmarshal(marsh, msg.getAttributesBytes(), U.resolveClassLoader(task.getTask().getClass().getClassLoader(), ctx.config())); GridTaskSessionImpl ses = task.getSession(); @@ -1306,7 +1306,7 @@ private class JobSiblingsMessageListener implements GridMessageListener { if (topic == null) { assert req.topicBytes() != null; - topic = marsh.unmarshal(req.topicBytes(), U.resolveClassLoader(ctx.config())); + topic = U.unmarshal(marsh, req.topicBytes(), U.resolveClassLoader(ctx.config())); } boolean loc = ctx.localNodeId().equals(nodeId); @@ -1314,7 +1314,7 @@ private class JobSiblingsMessageListener implements GridMessageListener { ctx.io().send(nodeId, topic, new GridJobSiblingsResponse( loc ? siblings : null, - loc ? null : marsh.marshal(siblings)), + loc ? null : U.marshal(marsh, siblings)), SYSTEM_POOL); } catch (IgniteCheckedException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java index 452e48cb9ac28..0be69d126bb36 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java @@ -805,15 +805,15 @@ void onResponse(GridJobExecuteResponse msg) { try { boolean loc = ctx.localNodeId().equals(res.getNodeId()) && !ctx.config().isMarshalLocalJobs(); - Object res0 = loc ? res.getJobResult() : marsh.unmarshal(res.getJobResultBytes(), + Object res0 = loc ? res.getJobResult() : U.unmarshal(marsh, res.getJobResultBytes(), U.resolveClassLoader(clsLdr, ctx.config())); IgniteException ex = loc ? res.getException() : - marsh.unmarshal(res.getExceptionBytes(), + U.unmarshal(marsh, res.getExceptionBytes(), U.resolveClassLoader(clsLdr, ctx.config())); Map attrs = loc ? res.getJobAttributes() : - marsh.>unmarshal(res.getJobAttributesBytes(), + U.>unmarshal(marsh, res.getJobAttributesBytes(), U.resolveClassLoader(clsLdr, ctx.config())); jobRes.onResponse(res0, ex, attrs, res.isCancelled()); @@ -1347,16 +1347,16 @@ private void sendRequest(ComputeJobResult res) { ses.getTaskName(), ses.getUserVersion(), ses.getTaskClassName(), - loc ? null : marsh.marshal(res.getJob()), + loc ? null : U.marshal(marsh, res.getJob()), loc ? res.getJob() : null, ses.getStartTime(), timeout, ses.getTopology(), - loc ? null : marsh.marshal(ses.getJobSiblings()), + loc ? null : U.marshal(marsh, ses.getJobSiblings()), loc ? ses.getJobSiblings() : null, - loc ? null : marsh.marshal(sesAttrs), + loc ? null : U.marshal(marsh, sesAttrs), loc ? sesAttrs : null, - loc ? null : marsh.marshal(jobAttrs), + loc ? null : U.marshal(marsh, jobAttrs), loc ? jobAttrs : null, ses.getCheckpointSpi(), dep.classLoaderId(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index f07266bdef7ed..1e8d648e0e27e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -182,6 +182,7 @@ import org.apache.ignite.internal.managers.deployment.GridDeploymentInfo; import org.apache.ignite.internal.mxbean.IgniteStandardMXBean; import org.apache.ignite.internal.processors.cache.GridCacheAttributes; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx; import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; @@ -213,6 +214,7 @@ import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.lifecycle.LifecycleAware; +import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.PluginProvider; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageWriter; @@ -9634,6 +9636,203 @@ public static > T max(T t0, T t1) { return t0.compareTo(t1) > 0 ? t0 : t1; } + + /** + * Unmarshals object from the input stream using given class loader. + * This method should not close given input stream. + *

          + * This method wraps marshaller invocations and guaranty throws {@link IgniteCheckedException} in fail case. + * + * @param Type of unmarshalled object. + * @param in Input stream. + * @param clsLdr Class loader to use. + * @return Unmarshalled object. + * @throws IgniteCheckedException If unmarshalling failed. + */ + public static T unmarshal(Marshaller marsh, InputStream in, @Nullable ClassLoader clsLdr) + throws IgniteCheckedException { + assert marsh != null; + assert in != null; + + try { + return marsh.unmarshal(in, clsLdr); + } + catch (IgniteCheckedException e) { + throw e; + } + catch (Exception e) { + throw new IgniteCheckedException(e); + } + } + + /** + * Unmarshals object from the input stream using given class loader. + * This method should not close given input stream. + *

          + * This method wraps marshaller invocations and guaranty throws {@link IgniteCheckedException} in fail case. + * + * @param Type of unmarshalled object. + * @param marsh Marshaller. + * @param arr Byte array. + * @param clsLdr Class loader to use. + * @return Unmarshalled object. + * @throws IgniteCheckedException If unmarshalling failed. + */ + public static T unmarshal(Marshaller marsh, byte[] arr, @Nullable ClassLoader clsLdr) + throws IgniteCheckedException { + assert marsh != null; + assert arr != null; + + try { + return marsh.unmarshal(arr, clsLdr); + } + catch (IgniteCheckedException e) { + throw e; + } + catch (Exception e) { + throw new IgniteCheckedException(e); + } + } + + /** + * Unmarshals object from the input stream using given class loader. + * This method should not close given input stream. + *

          + * This method wraps marshaller invocations and guaranty throws {@link IgniteCheckedException} in fail case. + * + * @param Type of unmarshalled object. + * @param ctx Kernal contex. + * @param arr Byte array. + * @param clsLdr Class loader to use. + * @return Unmarshalled object. + * @throws IgniteCheckedException If unmarshalling failed. + */ + public static T unmarshal(GridKernalContext ctx, byte[] arr, @Nullable ClassLoader clsLdr) + throws IgniteCheckedException { + assert ctx != null; + assert arr != null; + + try { + return U.unmarshal(ctx.config().getMarshaller(), arr, clsLdr); + } + catch (IgniteCheckedException e) { + throw e; + } + catch (Exception e) { + throw new IgniteCheckedException(e); + } + } + + /** + * Unmarshals object from the input stream using given class loader. + * This method should not close given input stream. + *

          + * This method wraps marshaller invocations and guaranty throws {@link IgniteCheckedException} in fail case. + * + * @param Type of unmarshalled object. + * @param ctx Kernal contex. + * @param arr Byte array. + * @param clsLdr Class loader to use. + * @return Unmarshalled object. + * @throws IgniteCheckedException If unmarshalling failed. + */ + public static T unmarshal(GridCacheSharedContext ctx, byte[] arr, @Nullable ClassLoader clsLdr) + throws IgniteCheckedException { + assert ctx != null; + assert arr != null; + + try { + return U.unmarshal(ctx.marshaller(), arr, clsLdr); + } + catch (IgniteCheckedException e) { + throw e; + } + catch (Exception e) { + throw new IgniteCheckedException(e); + } + } + + /** + * Marshals object to byte array. + *

          + * This method wraps marshaller invocations and guaranty throws {@link IgniteCheckedException} in fail case. + * + * @param marsh Marshaller. + * @param obj Object to marshal. + * @return Byte array. + * @throws IgniteCheckedException If marshalling failed. + */ + public static byte[] marshal(Marshaller marsh, Object obj) throws IgniteCheckedException { + assert marsh != null; + + try { + return marsh.marshal(obj); + } + catch (IgniteCheckedException e) { + throw e; + } + catch (Exception e) { + throw new IgniteCheckedException(e); + } + } + + /** + * Marshals object to byte array. + *

          + * This method wraps marshaller invocations and guaranty throws {@link IgniteCheckedException} in fail case. + * + * @param marsh Marshaller. + * @param obj Object to marshal. + * @param out Output stream. + * @throws IgniteCheckedException If marshalling failed. + */ + public static void marshal(Marshaller marsh, @Nullable Object obj, OutputStream out) + throws IgniteCheckedException { + assert marsh != null; + + try { + marsh.marshal(obj, out); + } + catch (IgniteCheckedException e) { + throw e; + } + catch (Exception e) { + throw new IgniteCheckedException(e); + } + } + + /** + * Marshals object to byte array. Wrap marshaller + *

          + * This method wraps marshaller invocations and guaranty throws {@link IgniteCheckedException} in fail case. + * + * @param ctx Kernal context. + * @param obj Object to marshal. + * @return Byte array. + * @throws IgniteCheckedException If marshalling failed. + */ + public static byte[] marshal(GridKernalContext ctx, Object obj) throws IgniteCheckedException { + assert ctx != null; + + return marshal(ctx.config().getMarshaller(), obj); + } + + /** + * Marshals object to byte array. Wrap marshaller + *

          + * This method wraps marshaller invocations and guaranty throws {@link IgniteCheckedException} in fail case. + * + * @param ctx Cache context. + * @param obj Object to marshal. + * @return Byte array. + * @throws IgniteCheckedException If marshalling failed. + */ + public static byte[] marshal(GridCacheSharedContext ctx, Object obj) throws IgniteCheckedException { + assert ctx != null; + + return marshal(ctx.marshaller(), obj); + } + /** * Get current Ignite name. * diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java index 7a130d3686931..54172dcc5b052 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java @@ -80,7 +80,7 @@ public class JdkMarshaller extends AbstractNodeNameAwareMarshaller { objOut.flush(); } - catch (IOException e) { + catch (Exception e) { throw new IgniteCheckedException("Failed to serialize object: " + obj, e); } finally{ @@ -119,14 +119,14 @@ public class JdkMarshaller extends AbstractNodeNameAwareMarshaller { return (T)objIn.readObject(); } - catch (IOException e) { - throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e); - } catch (ClassNotFoundException e) { throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " + "(make sure same versions of all classes are available on all nodes or enable peer-class-loading): " + clsLdr, e); } + catch (Exception e) { + throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e); + } finally{ U.closeQuiet(objIn); } diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java index 37f7acb30d318..467dddf1abd39 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java @@ -193,7 +193,7 @@ public void setPoolSize(int poolSize) { return objOut.out().array(); } - catch (IOException e) { + catch (Exception e) { throw new IgniteCheckedException("Failed to serialize object: " + obj, e); } finally { @@ -217,14 +217,14 @@ public void setPoolSize(int poolSize) { return (T)objIn.readObject(); } - catch (IOException e) { - throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e); - } catch (ClassNotFoundException e) { throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " + "(make sure same versions of all classes are available on all nodes or enable peer-class-loading): " + clsLdr, e); } + catch (Exception e) { + throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e); + } finally { OptimizedObjectStreamRegistry.closeIn(objIn); } @@ -246,14 +246,14 @@ public void setPoolSize(int poolSize) { return (T)objIn.readObject(); } - catch (IOException e) { - throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e); - } catch (ClassNotFoundException e) { throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " + "(make sure same version of all classes are available on all nodes or enable peer-class-loading): " + clsLdr, e); } + catch (Exception e) { + throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e); + } finally { OptimizedObjectStreamRegistry.closeIn(objIn); } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsUtils.java b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsUtils.java index b1698faf555a5..f8c41309a3c0d 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsUtils.java @@ -61,7 +61,7 @@ static SharedFsCheckpointData read(File file, Marshaller m, IgniteLogger log) InputStream in = new FileInputStream(file); try { - return (SharedFsCheckpointData)m.unmarshal(in, U.gridClassLoader()); + return U.unmarshal(m, in, U.gridClassLoader()); } finally { U.close(in, log); @@ -91,7 +91,7 @@ static void write(File file, SharedFsCheckpointData data, Marshaller m, IgniteLo try { out = new FileOutputStream(file); - m.marshal(data, out); + U.marshal(m, data, out); } finally { U.close(out, log); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 2c85645305da5..2d948da3801a8 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -431,7 +431,7 @@ else if (state == DISCONNECTED) { try { sockWriter.sendMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt, - spi.marshaller().marshal(evt))); + U.marshal(spi.marshaller(), evt))); } catch (IgniteCheckedException e) { throw new IgniteSpiException("Failed to marshal custom event: " + evt, e); @@ -701,7 +701,7 @@ private void marshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException Map attrs = new HashMap<>(node.getAttributes()); attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS, - spi.marshaller().marshal(attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS))); + U.marshal(spi.marshaller(), attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS))); node.setAttributes(attrs); } @@ -902,7 +902,7 @@ public void setSocket(SocketStream sockStream, UUID rmtNodeId) { TcpDiscoveryAbstractMessage msg; try { - msg = spi.marshaller().unmarshal(in, U.resolveClassLoader(spi.ignite().configuration())); + msg = U.unmarshal(spi.marshaller(), in, U.resolveClassLoader(spi.ignite().configuration())); } catch (IgniteCheckedException e) { if (log.isDebugEnabled()) @@ -1232,7 +1232,7 @@ public void cancel() { List msgs = null; while (!isInterrupted()) { - TcpDiscoveryAbstractMessage msg = spi.marshaller().unmarshal(in, + TcpDiscoveryAbstractMessage msg = U.unmarshal(spi.marshaller(), in, U.resolveClassLoader(spi.ignite().configuration())); if (msg instanceof TcpDiscoveryClientReconnectMessage) { diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 135a737587bc6..78a5f39100c6a 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -744,7 +744,7 @@ private void interruptPing(TcpDiscoveryNode node) { @Override public void sendCustomEvent(DiscoverySpiCustomMessage evt) { try { msgWorker.addMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt, - spi.marshaller().marshal(evt))); + U.marshal(spi.marshaller(), evt))); } catch (IgniteCheckedException e) { throw new IgniteSpiException("Failed to marshal custom event: " + evt, e); @@ -827,7 +827,7 @@ private void joinTopology() throws IgniteSpiException { Map attrs = new HashMap<>(locNode.attributes()); - attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, spi.marshaller().marshal(subj)); + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, U.marshal(spi.marshaller(), subj)); attrs.remove(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS); locNode.setAttributes(attrs); @@ -1243,7 +1243,7 @@ private void marshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException Map attrs = new HashMap<>(node.getAttributes()); attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS, - spi.marshaller().marshal(attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS))); + U.marshal(spi.marshaller(), attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS))); node.setAttributes(attrs); } @@ -1266,7 +1266,7 @@ private SecurityCredentials unmarshalCredentials(TcpDiscoveryNode node) throws I if (credBytes == null) return null; - return spi.marshaller().unmarshal(credBytes, null); + return U.unmarshal(spi.marshaller(), credBytes, null); } catch (IgniteCheckedException e) { throw new IgniteSpiException("Failed to unmarshal node security credentials: " + node.id(), e); @@ -2380,7 +2380,7 @@ private void sendMessageToClients(TcpDiscoveryAbstractMessage msg) { for (ClientMessageWorker clientMsgWorker : clientMsgWorkers.values()) { if (msgBytes == null) { try { - msgBytes = spi.marshaller().marshal(msg); + msgBytes = U.marshal(spi.marshaller(), msg); } catch (IgniteCheckedException e) { U.error(log, "Failed to marshal message: " + msg, e); @@ -2399,7 +2399,7 @@ private void sendMessageToClients(TcpDiscoveryAbstractMessage msg) { if (clientMsgWorker.clientNodeId.equals(node.id())) { try { - msg0 = spi.marshaller().unmarshal(msgBytes, + msg0 = U.unmarshal(spi.marshaller(), msgBytes, U.resolveClassLoader(spi.ignite().configuration())); prepareNodeAddedMessage(msg0, clientMsgWorker.clientNodeId, null, null, null); @@ -3157,7 +3157,7 @@ else if (log.isDebugEnabled()) // Stick in authentication subject to node (use security-safe attributes for copy). Map attrs = new HashMap<>(node.getAttributes()); - attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, spi.marshaller().marshal(subj)); + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, U.marshal(spi.marshaller(), subj)); node.setAttributes(attrs); } @@ -3805,7 +3805,7 @@ else if (!locNodeId.equals(node.id()) && ring.node(node.id()) != null) { else { SecurityContext subj = spi.nodeAuth.authenticateNode(node, cred); - SecurityContext coordSubj = spi.marshaller().unmarshal( + SecurityContext coordSubj = U.unmarshal(spi.marshaller(), node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT), U.resolveClassLoader(spi.ignite().configuration())); @@ -4872,7 +4872,7 @@ private void processCustomMessage(TcpDiscoveryCustomEventMessage msg) { if (nextMsg != null) { try { TcpDiscoveryCustomEventMessage ackMsg = new TcpDiscoveryCustomEventMessage( - getLocalNodeId(), nextMsg, spi.marshaller().marshal(nextMsg)); + getLocalNodeId(), nextMsg, U.marshal(spi.marshaller(), nextMsg)); ackMsg.topologyVersion(msg.topologyVersion()); @@ -5017,7 +5017,7 @@ private void notifyDiscoveryListener(TcpDiscoveryCustomEventMessage msg) { msgObj); if (msgObj.isMutable()) - msg.message(msgObj, spi.marshaller().marshal(msgObj)); + msg.message(msgObj, U.marshal(spi.marshaller(), msgObj)); } catch (Throwable e) { U.error(log, "Failed to unmarshal discovery custom message.", e); @@ -5455,7 +5455,7 @@ else if (e.hasCause(ObjectStreamException.class) || while (!isInterrupted()) { try { - TcpDiscoveryAbstractMessage msg = spi.marshaller().unmarshal(in, + TcpDiscoveryAbstractMessage msg = U.unmarshal(spi.marshaller(), in, U.resolveClassLoader(spi.ignite().configuration())); msg.senderNodeId(nodeId); @@ -5946,7 +5946,7 @@ public void addMessage(TcpDiscoveryAbstractMessage msg, @Nullable byte[] msgByte byte[] msgBytes = msgT.get2(); if (msgBytes == null) - msgBytes = spi.marshaller().marshal(msg); + msgBytes = U.marshal(spi.marshaller(), msg); if (msg instanceof TcpDiscoveryClientAckResponse) { if (clientVer == null) { diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java index 426eb8ef039e5..a8704e7ab66f5 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java @@ -1380,7 +1380,7 @@ protected void writeToSocket(Socket sock, IgniteCheckedException err = null; try { - marshaller().marshal(msg, out); + U.marshal(marshaller(), msg, out); } catch (IgniteCheckedException e) { err = e; @@ -1464,7 +1464,7 @@ protected T readMessage(Socket sock, @Nullable InputStream in, long timeout) try { sock.setSoTimeout((int)timeout); - T res = marshaller().unmarshal(in == null ? sock.getInputStream() : in, + T res = U.unmarshal(marshaller(), in == null ? sock.getInputStream() : in, U.resolveClassLoader(ignite.configuration())); return res; @@ -1682,7 +1682,7 @@ protected Map collectExchangeData(UUID nodeId) { for (Map.Entry entry : data.entrySet()) { try { - byte[] bytes = marshaller().marshal(entry.getValue()); + byte[] bytes = U.marshal(marshaller(), entry.getValue()); data0.put(entry.getKey(), bytes); } @@ -1713,7 +1713,7 @@ protected void onExchange(UUID joiningNodeID, for (Map.Entry entry : data.entrySet()) { try { - Serializable compData = marshaller().unmarshal(entry.getValue(), clsLdr); + Serializable compData = U.unmarshal(marshaller(), entry.getValue(), clsLdr); data0.put(entry.getKey(), compData); } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java index 5bbe90edbafc1..e96abe96550f6 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java @@ -690,7 +690,7 @@ private static class AddressResponse { private AddressResponse(Collection addrs) throws IgniteCheckedException { this.addrs = addrs; - byte[] addrsData = marsh.marshal(addrs); + byte[] addrsData = U.marshal(marsh, addrs); data = new byte[U.IGNITE_HEADER.length + addrsData.length]; if (data.length > MAX_DATA_LENGTH) @@ -709,7 +709,7 @@ private AddressResponse(byte[] data) throws IgniteCheckedException { this.data = data; - addrs = marsh.unmarshal(Arrays.copyOfRange(data, U.IGNITE_HEADER.length, data.length), null); + addrs = U.unmarshal(marsh, Arrays.copyOfRange(data, U.IGNITE_HEADER.length, data.length), null); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java index c0e39d3baeff1..219c0ada93d22 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java @@ -20,6 +20,7 @@ import java.util.UUID; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; import org.jetbrains.annotations.NotNull; @@ -77,7 +78,7 @@ public void message(@Nullable DiscoverySpiCustomMessage msg, @NotNull byte[] msg */ @Nullable public DiscoverySpiCustomMessage message(@NotNull Marshaller marsh, ClassLoader ldr) throws Throwable { if (msg == null) { - msg = marsh.unmarshal(msgBytes, ldr); + msg = U.unmarshal(marsh, msgBytes, ldr); assert msg != null; } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java index 222f463c82588..f0ac7bc8d8302 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java @@ -596,7 +596,7 @@ private byte[] keyBytes(SwapKey key) throws IgniteSpiException { if (keyBytes == null) { try { - keyBytes = ignite.configuration().getMarshaller().marshal(key.key()); + keyBytes = U.marshal(ignite.configuration().getMarshaller(), key.key()); } catch (IgniteCheckedException e) { throw new IgniteSpiException("Failed to marshal key: " + key.key(), e); diff --git a/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java b/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java index a5b89e47b7ae3..d1c8d19f203ed 100644 --- a/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java +++ b/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java @@ -35,6 +35,7 @@ import org.apache.ignite.internal.util.nio.GridNioSession; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.stream.StreamAdapter; @@ -231,7 +232,7 @@ private DefaultConverter(@Nullable String gridName) { /** {@inheritDoc} */ @Override public T convert(byte[] msg) { try { - return marsh.unmarshal(msg, null); + return U.unmarshal(marsh, msg, null); } catch (IgniteCheckedException e) { throw new IgniteException(e); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorExternalizableFailedTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorExternalizableFailedTest.java new file mode 100644 index 0000000000000..031774ee24aaf --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorExternalizableFailedTest.java @@ -0,0 +1,588 @@ +/* + * 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.query.continuous; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.concurrent.Callable; +import java.util.concurrent.ThreadLocalRandom; +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.jetbrains.annotations.NotNull; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC; +import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; +import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE; + +/** + * + */ +public class CacheEntryProcessorExternalizableFailedTest extends GridCommonAbstractTest { + /** */ + private static final int EXPECTED_VALUE = 42; + + /** */ + private static final int WRONG_VALUE = -1; + + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int NODES = 3; + + /** */ + public static final int ITERATION_CNT = 1; + + /** */ + public static final int KEYS = 10; + + /** */ + private boolean client; + + /** */ + private boolean failOnWrite = false; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1); + + cfg.setClientMode(client); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGridsMultiThreaded(getServerNodeCount()); + + client = true; + + startGrid(getServerNodeCount()); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + failOnWrite = false; + } + + /** + * @return Server nodes. + */ + private int getServerNodeCount() { + return NODES; + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticFullSync() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(FULL_SYNC, 2); + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + + failOnWrite = true; + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticOnePhaseCommit() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PRIMARY_SYNC, 1); + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + + failOnWrite = true; + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticOnePhaseCommitWithNearCache() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PRIMARY_SYNC, 1) + .setNearConfiguration(new NearCacheConfiguration()); + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + + failOnWrite = true; + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticOnePhaseCommitFullSync() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(FULL_SYNC, 1); + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + + failOnWrite = true; + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticOnePhaseCommitFullSyncWithNearCache() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(FULL_SYNC, 1) + .setNearConfiguration(new NearCacheConfiguration()); + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + + failOnWrite = true; + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testPessimistic() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PRIMARY_SYNC, 2); + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + + failOnWrite = true; + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticWithNearCache() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PRIMARY_SYNC, 2) + .setNearConfiguration(new NearCacheConfiguration()); + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + + failOnWrite = true; + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testPessimisticFullSync() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(FULL_SYNC, 2); + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + + failOnWrite = true; + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticOnePhaseCommit() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PRIMARY_SYNC, 1); + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + + failOnWrite = true; + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticOnePhaseCommitFullSync() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(FULL_SYNC, 1); + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + + failOnWrite = true; + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticOnePhaseCommitFullSyncWithNearCache() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(FULL_SYNC, 1) + .setNearConfiguration(new NearCacheConfiguration()); + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + + failOnWrite = true; + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testOptimistic() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(PRIMARY_SYNC, 2); + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + + failOnWrite = true; + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + } + + /** + * @throws Exception If failed. + */ + public void testOptimisticFullSyncWithNearCache() throws Exception { + CacheConfiguration ccfg = cacheConfiguration(FULL_SYNC, 2); + + doTestInvokeTest(ccfg, OPTIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, OPTIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, OPTIMISTIC, REPEATABLE_READ); + + failOnWrite = true; + + doTestInvokeTest(ccfg, PESSIMISTIC, SERIALIZABLE); + + doTestInvokeTest(ccfg, PESSIMISTIC, READ_COMMITTED); + + doTestInvokeTest(ccfg, PESSIMISTIC, REPEATABLE_READ); + } + + /** + * @param ccfg Cache configuration. + * @throws Exception If failed. + */ + private void doTestInvokeTest(CacheConfiguration ccfg, TransactionConcurrency txConcurrency, + TransactionIsolation txIsolation) throws Exception { + IgniteEx cln = grid(getServerNodeCount()); + + grid(0).createCache(ccfg); + + IgniteCache clnCache; + + if (ccfg.getNearConfiguration() != null) + clnCache = cln.createNearCache(ccfg.getName(), ccfg.getNearConfiguration()); + else + clnCache = cln.cache(ccfg.getName()); + + putKeys(clnCache, EXPECTED_VALUE); + + try { + // Explicit tx. + for (int i = 0; i < ITERATION_CNT; i++) { + try (final Transaction tx = cln.transactions().txStart(txConcurrency, txIsolation)) { + putKeys(clnCache, WRONG_VALUE); + + clnCache.invoke(KEYS, createEntryProcessor()); + + GridTestUtils.assertThrowsWithCause(new Callable() { + @Override public Object call() throws Exception { + tx.commit(); + + return null; + } + }, UnsupportedOperationException.class); + } + + assertNull(cln.transactions().tx()); + + checkKeys(clnCache, EXPECTED_VALUE); + } + + // From affinity node. + Ignite grid = grid(ThreadLocalRandom.current().nextInt(NODES)); + + final IgniteCache cache = grid.cache(ccfg.getName()); + + // Explicit tx. + for (int i = 0; i < ITERATION_CNT; i++) { + try (final Transaction tx = grid.transactions().txStart(txConcurrency, txIsolation)) { + putKeys(cache, WRONG_VALUE); + + cache.invoke(KEYS, createEntryProcessor()); + + GridTestUtils.assertThrowsWithCause(new Callable() { + @Override public Object call() throws Exception { + tx.commit(); + + return null; + } + }, UnsupportedOperationException.class); + } + + assertNull(cln.transactions().tx()); + + checkKeys(cache, EXPECTED_VALUE); + } + + final IgniteCache clnCache0 = clnCache; + + // Implicit tx. + for (int i = 0; i < ITERATION_CNT; i++) { + GridTestUtils.assertThrowsWithCause(new Callable() { + @Override public Object call() throws Exception { + clnCache0.invoke(KEYS, createEntryProcessor()); + + return null; + } + }, UnsupportedOperationException.class); + + assertNull(cln.transactions().tx()); + } + + checkKeys(clnCache, EXPECTED_VALUE); + } + catch (Exception e) { + e.printStackTrace(); + } + finally { + grid(0).destroyCache(ccfg.getName()); + } + } + + /** + * @return Entry processor. + */ + @NotNull private EntryProcessor createEntryProcessor() { + return failOnWrite ? new ExternalizableFailedWriteEntryProcessor() : + new ExternalizableFailedReadEntryProcessor(); + } + + /** + * @param cache Cache. + * @param val Value. + */ + private void putKeys(IgniteCache cache, int val) { + cache.put(KEYS, val); + } + + /** + * @param cache Cache. + * @param expVal Expected value. + */ + private void checkKeys(IgniteCache cache, int expVal) { + assertEquals(expVal, cache.get(KEYS)); + } + + /** + * @return Cache configuration. + */ + private CacheConfiguration cacheConfiguration(CacheWriteSynchronizationMode wrMode, int backup) { + return new CacheConfiguration("test-cache-" + wrMode + "-" + backup) + .setAtomicityMode(TRANSACTIONAL) + .setWriteSynchronizationMode(FULL_SYNC) + .setBackups(backup); + } + + /** + * + */ + private static class ExternalizableFailedWriteEntryProcessor implements EntryProcessor, + Externalizable{ + /** */ + public ExternalizableFailedWriteEntryProcessor() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public Integer process(MutableEntry entry, Object... arguments) + throws EntryProcessorException { + entry.setValue(42); + + return null; + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + // No-op. + } + } + + /** + * + */ + private static class ExternalizableFailedReadEntryProcessor implements EntryProcessor, + Externalizable { + /** */ + public ExternalizableFailedReadEntryProcessor() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public Integer process(MutableEntry entry, Object... arguments) + throws EntryProcessorException { + entry.setValue(42); + + return null; + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + throw new UnsupportedOperationException(); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java index 5ad4cb82d70ad..388c3a3decd9a 100755 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java @@ -140,6 +140,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearTxExceptionSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedTxExceptionSelfTest; import org.apache.ignite.internal.processors.cache.local.GridCacheLocalTxExceptionSelfTest; +import org.apache.ignite.internal.processors.cache.query.continuous.CacheEntryProcessorExternalizableFailedTest; import org.apache.ignite.internal.processors.cache.query.continuous.CacheEntryProcessorNonSerializableTest; import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorSelfTest; import org.apache.ignite.internal.processors.datastreamer.DataStreamerImplSelfTest; @@ -191,6 +192,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(IgniteCacheAtomicLocalWithStoreInvokeTest.class); suite.addTestSuite(IgniteCacheTxInvokeTest.class); suite.addTestSuite(CacheEntryProcessorNonSerializableTest.class); + suite.addTestSuite(CacheEntryProcessorExternalizableFailedTest.class); suite.addTestSuite(IgniteCacheEntryProcessorCallTest.class); GridTestUtils.addTestIfNeeded(suite, CacheEntryProcessorCopySelfTest.class, ignoredTests); suite.addTestSuite(IgniteCacheTxNearEnabledInvokeTest.class); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java index 3f794698ef648..eeca564e9f107 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java @@ -22,6 +22,7 @@ import org.apache.ignite.internal.util.nio.GridNioFilterAdapter; import org.apache.ignite.internal.util.nio.GridNioFuture; import org.apache.ignite.internal.util.nio.GridNioSession; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; /** @@ -29,15 +30,15 @@ */ public class HadoopMarshallerFilter extends GridNioFilterAdapter { /** Marshaller. */ - private Marshaller marshaller; + private Marshaller marsh; /** - * @param marshaller Marshaller to use. + * @param marsh Marshaller to use. */ - public HadoopMarshallerFilter(Marshaller marshaller) { + public HadoopMarshallerFilter(Marshaller marsh) { super("HadoopMarshallerFilter"); - this.marshaller = marshaller; + this.marsh = marsh; } /** {@inheritDoc} */ @@ -59,14 +60,14 @@ public HadoopMarshallerFilter(Marshaller marshaller) { @Override public GridNioFuture onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException { assert msg instanceof HadoopMessage : "Invalid message type: " + msg; - return proceedSessionWrite(ses, marshaller.marshal(msg)); + return proceedSessionWrite(ses, U.marshal(marsh, msg)); } @Override public void onMessageReceived(GridNioSession ses, Object msg) throws IgniteCheckedException { assert msg instanceof byte[]; // Always unmarshal with system classloader. - proceedMessageReceived(ses, marshaller.unmarshal((byte[])msg, null)); + proceedMessageReceived(ses, U.unmarshal(marsh, (byte[])msg, null)); } /** {@inheritDoc} */ diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStore.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStore.java index b1c269b09adea..c87f08f3fb716 100644 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStore.java +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStore.java @@ -506,7 +506,7 @@ private boolean resourceAvailable(String name) { * @throws IgniteCheckedException If failed to convert. */ protected byte[] toBytes(Object obj) throws IgniteCheckedException { - return marsh.marshal(obj); + return U.marshal(marsh, obj); } /** @@ -521,7 +521,7 @@ protected X fromBytes(byte[] bytes) throws IgniteCheckedException { if (bytes == null || bytes.length == 0) return null; - return marsh.unmarshal(bytes, getClass().getClassLoader()); + return U.unmarshal(marsh, bytes, getClass().getClassLoader()); } /** diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java index 0b0fe851867d6..a5f283e50e9e2 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java @@ -1553,13 +1553,13 @@ public GridReduceQueryExecutor reduceQueryExecutor() { protected JavaObjectSerializer h2Serializer() { return new JavaObjectSerializer() { @Override public byte[] serialize(Object obj) throws Exception { - return marshaller.marshal(obj); + return U.marshal(marshaller, obj); } @Override public Object deserialize(byte[] bytes) throws Exception { ClassLoader clsLdr = ctx != null ? U.resolveClassLoader(ctx.config()) : null; - return marshaller.unmarshal(bytes, clsLdr); + return U.unmarshal(marshaller, bytes, clsLdr); } }; } diff --git a/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/connect/serialization/CacheEventDeserializer.java b/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/connect/serialization/CacheEventDeserializer.java index 47ce1ca125364..3bcf5fb88051b 100644 --- a/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/connect/serialization/CacheEventDeserializer.java +++ b/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/connect/serialization/CacheEventDeserializer.java @@ -20,6 +20,7 @@ import java.util.Map; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.events.CacheEvent; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.kafka.common.errors.SerializationException; @@ -40,7 +41,7 @@ public class CacheEventDeserializer implements Deserializer { /** {@inheritDoc} */ @Override public CacheEvent deserialize(String topic, byte[] bytes) { try { - return marsh.unmarshal(bytes, getClass().getClassLoader()); + return U.unmarshal(marsh, bytes, getClass().getClassLoader()); } catch (IgniteCheckedException e) { throw new SerializationException("Failed to deserialize cache event!", e); diff --git a/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/connect/serialization/CacheEventSerializer.java b/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/connect/serialization/CacheEventSerializer.java index 2f2d66872fe5d..bc0925623f884 100644 --- a/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/connect/serialization/CacheEventSerializer.java +++ b/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/connect/serialization/CacheEventSerializer.java @@ -20,6 +20,7 @@ import java.util.Map; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.events.CacheEvent; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.kafka.common.errors.SerializationException; @@ -40,7 +41,7 @@ public class CacheEventSerializer implements Serializer { /** {@inheritDoc} */ @Override public byte[] serialize(String topic, CacheEvent event) { try { - return marsh.marshal(event); + return U.marshal(marsh, event); } catch (IgniteCheckedException e) { throw new SerializationException("Failed to serialize cache event!", e); diff --git a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionV2.java b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionV2.java index b6540b238671f..83ad1864d214d 100644 --- a/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionV2.java +++ b/modules/web/src/main/java/org/apache/ignite/cache/websession/WebSessionV2.java @@ -21,6 +21,7 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.websession.WebSessionEntity; import org.apache.ignite.marshaller.Marshaller; import org.jetbrains.annotations.Nullable; @@ -82,7 +83,7 @@ class WebSessionV2 implements HttpSession { private boolean invalidated; /** Grid marshaller. */ - private final Marshaller marshaller; + private final Marshaller marsh; /** Original session to delegate invalidation. */ private final HttpSession genuineSes; @@ -91,15 +92,18 @@ class WebSessionV2 implements HttpSession { * @param id Session ID. * @param ses Session. * @param isNew Is new flag. + * @param ctx Servlet context. + * @param entity Entity. + * @param marsh Marshaller. */ WebSessionV2(final String id, final @Nullable HttpSession ses, final boolean isNew, final ServletContext ctx, - @Nullable WebSessionEntity entity, final Marshaller marshaller) { + @Nullable WebSessionEntity entity, final Marshaller marsh) { assert id != null; - assert marshaller != null; + assert marsh != null; assert ctx != null; assert ses != null || entity != null; - this.marshaller = marshaller; + this.marsh = marsh; this.ctx = ctx; this.isNew = isNew; this.genuineSes = ses; @@ -331,9 +335,9 @@ public Map binaryUpdatesMap() throws IOException { * @throws IOException If unarshaling failed. */ @Nullable private T unmarshal(final byte[] bytes) throws IOException { - if (marshaller != null) { + if (marsh != null) { try { - return marshaller.unmarshal(bytes, getClass().getClassLoader()); + return U.unmarshal(marsh, bytes, getClass().getClassLoader()); } catch (IgniteCheckedException e) { throw new IOException(e); @@ -351,9 +355,9 @@ public Map binaryUpdatesMap() throws IOException { * @throws IOException If marshaling failed. */ @Nullable private byte[] marshal(final Object obj) throws IOException { - if (marshaller != null) { + if (marsh != null) { try { - return marshaller.marshal(obj); + return U.marshal(marsh, obj); } catch (IgniteCheckedException e) { throw new IOException(e); From 1938a17b01fac1e08c30011180bbcc3ed7374d83 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 11 Oct 2016 14:50:18 +0300 Subject: [PATCH 234/487] IGNITE-3948: Fixed a bug causing TTL manager to continue tracking of evicted entries. This closes #1101. --- .../processors/cache/GridCacheMapEntry.java | 13 +- .../GridCacheTtlManagerEvictionSelfTest.java | 160 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite.java | 2 + 3 files changed, 173 insertions(+), 2 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerEvictionSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index a9ac1e7191c4f..59966728eb8d7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -29,6 +29,7 @@ import javax.cache.expiry.ExpiryPolicy; import javax.cache.processor.EntryProcessor; import javax.cache.processor.EntryProcessorResult; + import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -4196,8 +4197,12 @@ private synchronized CacheEntryImplEx wrapVersionedWithValue() { } } } - else + else { + if (this.expireTimeUnlocked() > 0) + cctx.ttl().removeTrackedEntry(this); + clearIndex(prev); + } // Nullify value after swap. value(null); @@ -4250,8 +4255,12 @@ private synchronized CacheEntryImplEx wrapVersionedWithValue() { } } } - else + else { + if (this.expireTimeUnlocked() > 0) + cctx.ttl().removeTrackedEntry(this); + clearIndex(prevVal); + } // Nullify value after swap. value(null); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerEvictionSelfTest.java new file mode 100644 index 0000000000000..4c7c1c09524ab --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerEvictionSelfTest.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import java.util.concurrent.TimeUnit; +import javax.cache.expiry.CreatedExpiryPolicy; +import javax.cache.expiry.Duration; + +/** + * TTL manager eviction self test. + */ +public class GridCacheTtlManagerEvictionSelfTest extends GridCommonAbstractTest { + /** */ + private static final int ENTRIES_TO_PUT = 10_100; + + /** */ + private static final int ENTRIES_LIMIT = 1_000; + + /** IP finder. */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** Cache mode. */ + private volatile CacheMode cacheMode; + + /** Cache memory mode. */ + private volatile CacheMemoryMode cacheMemoryMode; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.setIpFinder(IP_FINDER); + + cfg.setDiscoverySpi(discoSpi); + + CacheConfiguration ccfg = new CacheConfiguration(); + + ccfg.setCacheMode(cacheMode); + ccfg.setMemoryMode(cacheMemoryMode); + ccfg.setEagerTtl(true); + ccfg.setSwapEnabled(false); + ccfg.setEvictionPolicy(new FifoEvictionPolicy(ENTRIES_LIMIT, 100)); + ccfg.setExpiryPolicyFactory(CreatedExpiryPolicy.factoryOf(new Duration(TimeUnit.HOURS, 12))); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testLocalEviction() throws Exception { + checkEviction(CacheMode.LOCAL, CacheMemoryMode.ONHEAP_TIERED); + checkEviction(CacheMode.LOCAL, CacheMemoryMode.OFFHEAP_TIERED); + } + + /** + * @throws Exception If failed. + */ + public void testPartitionedEviction() throws Exception { + checkEviction(CacheMode.PARTITIONED, CacheMemoryMode.ONHEAP_TIERED); + checkEviction(CacheMode.PARTITIONED, CacheMemoryMode.OFFHEAP_TIERED); + } + + /** + * @throws Exception If failed. + */ + public void testReplicatedEviction() throws Exception { + checkEviction(CacheMode.REPLICATED, CacheMemoryMode.ONHEAP_TIERED); + checkEviction(CacheMode.REPLICATED, CacheMemoryMode.OFFHEAP_TIERED); + } + + /** + * @param mode Cache mode. + * @throws Exception If failed. + */ + @SuppressWarnings("ConstantConditions") + private void checkEviction(CacheMode mode, CacheMemoryMode memoryMode) throws Exception { + cacheMode = mode; + cacheMemoryMode = memoryMode; + + final IgniteKernal g = (IgniteKernal)startGrid(0); + + try { + final IgniteCache cache = g.cache(null); + + final GridCacheContext cctx = g.cachex(null).context(); + + for (int i = 1; i <= ENTRIES_TO_PUT; i++) { + String key = "Some test entry key#" + i; + String value = "Some test entry value#" + i; + + cache.put(key, value); + } + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return (cctx.isSwapOrOffheapEnabled()) ? + ENTRIES_TO_PUT == cctx.ttl().pendingSize() : + ENTRIES_LIMIT == cctx.ttl().pendingSize(); + } + }, 3_000); + + if (log.isTraceEnabled()) + cctx.ttl().printMemoryStats(); + + final String firstKey = "Some test entry key#0"; + final String lastKey = "Some test entry key#" + ENTRIES_TO_PUT; + + if (cctx.isSwapOrOffheapEnabled()) { + assertTrue("last key should NOT be evicted", cache.containsKey(lastKey)); + + assertEquals(ENTRIES_TO_PUT, cctx.ttl().pendingSize()); + } + else { + assertFalse("first key should be evicted", cache.containsKey(firstKey)); + + assertTrue("last key should NOT be evicted", cache.containsKey(lastKey)); + + assertEquals("Ttl Manager should NOT track evicted entries", ENTRIES_LIMIT, cctx.ttl().pendingSize()); + } + } + finally { + Ignition.stopAll(true); + } + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java index 5ad4cb82d70ad..d8231fc6b7a0e 100755 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java @@ -83,6 +83,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheStoreValueBytesSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheSwapPreloadSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheSwapReloadSelfTest; +import org.apache.ignite.internal.processors.cache.GridCacheTtlManagerEvictionSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheTtlManagerSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheTxPartitionedLocalStoreSelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicInvokeTest; @@ -268,6 +269,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(GridCacheAsyncOperationsLimitSelfTest.class); suite.addTestSuite(IgniteCacheManyAsyncOperationsTest.class); suite.addTestSuite(GridCacheTtlManagerSelfTest.class); + suite.addTestSuite(GridCacheTtlManagerEvictionSelfTest.class); suite.addTestSuite(GridCacheLifecycleAwareSelfTest.class); suite.addTestSuite(IgniteCacheAtomicStopBusySelfTest.class); suite.addTestSuite(IgniteCacheTransactionalStopBusySelfTest.class); From 359a392f1c53217c691c4c40762c51fd330666e2 Mon Sep 17 00:00:00 2001 From: Valentin Kulichenko Date: Fri, 15 Jan 2016 09:58:41 +0300 Subject: [PATCH 235/487] Update notifier fixes (cherry picked from commit a5c85ca) --- .../main/java/org/apache/ignite/internal/IgniteKernal.java | 4 ++-- .../ignite/internal/processors/cluster/ClusterProcessor.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index e0a36a7e19a41..d9854bd8d4466 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -793,8 +793,6 @@ public void start(final IgniteConfiguration cfg, startProcessor(clusterProc); - fillNodeAttributes(clusterProc.updateNotifierEnabled()); - U.onGridStart(); // Start and configure resource processor first as it contains resources used @@ -896,6 +894,8 @@ public void start(final IgniteConfiguration cfg, provider.start(ctx.plugins().pluginContextForProvider(provider)); } + fillNodeAttributes(clusterProc.updateNotifierEnabled()); + gw.writeLock(); try { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java index 5ccd6801972ac..6500cf3e8faee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java @@ -245,4 +245,4 @@ private UpdateNotifierTimerTask(IgniteKernal kernal, GridUpdateNotifier verCheck } } } -} +} \ No newline at end of file From 0659bebe04dc9c0b0a163bc95061519c553e678c Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 12 Oct 2016 14:49:36 +0300 Subject: [PATCH 236/487] IGNITE-3972: Fixed a bug causing continuous queries to be lost for ATOMIC cache when key's primary node leaves topology. This closes #1133. --- .../CacheContinuousQueryHandler.java | 8 +- .../continuous/GridContinuousProcessor.java | 6 + ...ontinuousQueryAsyncFailoverTxSelfTest.java | 5 + ...ntinuousQueryFailoverAbstractSelfTest.java | 225 +++++++++++++++++- ...acheContinuousQueryFailoverTxSelfTest.java | 5 + 5 files changed, 237 insertions(+), 12 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 4b5074c82431b..304d031d81fc0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -595,12 +595,12 @@ private CacheContinuousQueryManager manager(GridKernalContext ctx) { assert objs != null; assert ctx != null; - final List entries = (List)objs; - - if (entries.isEmpty()) + if (objs.isEmpty()) return; if (asyncCallback) { + final List entries = objs instanceof List ? (List)objs : new ArrayList(objs); + IgniteStripedThreadPoolExecutor asyncPool = ctx.asyncCallbackPool(); int threadId = asyncPool.threadId(entries.get(0).partition()); @@ -639,7 +639,7 @@ private CacheContinuousQueryManager manager(GridKernalContext ctx) { }, threadId); } else - notifyCallback0(nodeId, ctx, entries); + notifyCallback0(nodeId, ctx, (Collection)objs); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java index f078b1bd01d23..3a559e7598754 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java @@ -820,6 +820,12 @@ public void addBackupNotification(UUID nodeId, if (!toSnd.isEmpty()) sendNotification(nodeId, routineId, null, toSnd, orderedTopic, true, null); } + else { + LocalRoutineInfo localRoutineInfo = locInfos.get(routineId); + + if (localRoutineInfo != null) + localRoutineInfo.handler().notifyCallback(nodeId, routineId, objs, ctx); + } } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFailoverTxSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFailoverTxSelfTest.java index 8f0bd0e0d4633..900abc8aa9470 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFailoverTxSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFailoverTxSelfTest.java @@ -41,4 +41,9 @@ public class CacheContinuousQueryAsyncFailoverTxSelfTest extends CacheContinuous @Override protected boolean asyncCallback() { return true; } + + /** {@inheritDoc} */ + public void testNoEventLossOnTopologyChange() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-4015"); + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java index 1376be14e726e..1b7fe2babdc12 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java @@ -35,6 +35,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import javax.cache.Cache; import javax.cache.CacheException; @@ -51,6 +52,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.Ignition; import org.apache.ignite.cache.CacheAtomicWriteOrderMode; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheEntryEventSerializableFilter; @@ -69,6 +71,7 @@ import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.managers.communication.GridIoMessage; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -84,9 +87,11 @@ import org.apache.ignite.internal.util.typedef.PAX; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.T3; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteAsyncCallback; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteOutClosure; +import org.apache.ignite.logger.NullLogger; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.resources.LoggerResource; import org.apache.ignite.spi.IgniteSpiException; @@ -141,10 +146,10 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC cfg.setCommunicationSpi(commSpi); - MemoryEventStorageSpi eventSpi = new MemoryEventStorageSpi(); - eventSpi.setExpireCount(50); + MemoryEventStorageSpi evtSpi = new MemoryEventStorageSpi(); + evtSpi.setExpireCount(50); - cfg.setEventStorageSpi(eventSpi); + cfg.setEventStorageSpi(evtSpi); CacheConfiguration ccfg = new CacheConfiguration(); @@ -1180,7 +1185,7 @@ public boolean apply() { CacheEntryEvent e = iter.next(); if ((exp.get2() != null && e.getValue() != null && exp.get2().equals(e.getValue())) - && equalOldValue(e, exp)) { + && equalOldValue(e, exp)) { found = true; iter.remove(); @@ -1254,12 +1259,12 @@ && equalOldValue(e, exp)) { * @param e Event * @param expVals expected value * @return {@code True} if entries has the same key, value and oldValue. If cache start without backups - * than oldValue ignoring in comparison. + * than oldValue ignoring in comparison. */ private boolean equalOldValue(CacheEntryEvent e, T3 expVals) { return (e.getOldValue() == null && expVals.get3() == null) // Both null || (e.getOldValue() != null && expVals.get3() != null // Equals - && e.getOldValue().equals(expVals.get3())) + && e.getOldValue().equals(expVals.get3())) || (backups == 0); // If we start without backup than oldValue might be lose. } @@ -2040,7 +2045,7 @@ public void testMultiThreadedFailover() throws Exception { bar.await(1, MINUTES); } } - catch (Exception e){ + catch (Exception e) { log.error("Failed.", e); err = true; @@ -2250,6 +2255,164 @@ private boolean checkEvents(boolean logAll, return pass; } + /** + * This is failover test detecting CQ event loss while topology changing. + * + * @throws Exception If failed. + */ + public void testNoEventLossOnTopologyChange() throws Exception { + final int stableNodeCnt = 1; + + final int batchLoadSize = 2000; + + final int restartCycles = 5; + + Ignite qryClient = startGridsMultiThreaded(stableNodeCnt); + + final CacheEventListener4 lsnr = new CacheEventListener4(atomicityMode() == CacheAtomicityMode.ATOMIC); + + ContinuousQuery qry = new ContinuousQuery<>(); + + qry.setLocalListener(lsnr); + + IgniteCache cache = qryClient.cache(null); + + QueryCursor cur = cache.query(qry); + + int iteration = 0; + + int putCnt = 0; + + int ignoredDupEvts = 0; + + Thread nodeRestartThread = nodeRestartThread(restartCycles, 2_000, 1_000); + + try { + nodeRestartThread.start(); + + while (!Thread.interrupted() && nodeRestartThread.isAlive()) { + iteration++; + + for (int i = 0; i < batchLoadSize; i++) + cache.put(i, iteration); + + putCnt += batchLoadSize; + + log.info("Batch loaded. Iteration: " + iteration); + + final long cnt = lsnr.count(); + + final long expCnt = putCnt * stableNodeCnt + ignoredDupEvts; + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return cnt == expCnt; + } + }, 6_000); + + if (cnt != expCnt) { + StringBuilder sb = new StringBuilder(); + + for (int i = 0; i < batchLoadSize; i++) { + Integer key = i; + Integer val = cache.get(key); + + if (!F.eq(val, iteration)) + sb.append("\n\t").append(">>> WRONG CACHE VALUE (lost data?) [key=").append(key) + .append(", val=").append(val).append(']'); + } + + for (Map.Entry entry : lsnr.eventMap().entrySet()) { + Integer key = entry.getKey(); + Integer val = entry.getValue(); + + if (!F.eq(val, iteration)) + sb.append("\n\t").append(">>> WRONG LISTENER VALUE (lost event?) [key=").append(key) + .append(", val=").append(val).append(']'); + } + + String msg = sb.toString(); + + // In atomic mode CQ can receive duplicate update events if update retried after fails. + // E.g. topology change + if (atomicityMode() == CacheAtomicityMode.ATOMIC && msg.isEmpty() && cnt > expCnt) + ignoredDupEvts += cnt - expCnt; + else + fail("Unexpected event updates count: EXPECTED=" + expCnt + ", ACTUAL=" + cnt + ", " + + "ITERATION=" + iteration + msg); + } + + sleep(500); + } + } + finally { + nodeRestartThread.interrupt(); + + cur.close(); + + nodeRestartThread.join(3_000); + } + } + + /** + * Starts thread which restarts a node over and over again. + */ + private Thread nodeRestartThread(final int restartCycles, final long initDelay, final long restartDelay) { + Thread t = new Thread(new Runnable() { + public void run() { + sleep(initDelay); + + try { + for (int i = 1; i <= restartCycles && !Thread.interrupted(); i++) { + + IgniteConfiguration cfg = optimize(getConfiguration("restartNode")). + setGridLogger(new NullLogger()); + + log.info("Node restart cycle started: " + i); + + try (Ignite ignored = Ignition.start(cfg)) { + awaitPartitionMapExchange(); + + sleep(restartDelay); + } + + log.info("Node restart cycle finished: " + i); + + awaitPartitionMapExchange(); + + sleep(restartDelay); + } + } + catch (Exception e) { + log.error("Unexpected error.", e); + } + } + }); + + t.setName("flapping-node-thread"); + + t.setDaemon(true); + + return t; + } + + /** + * Sleep quietly + * + * @param sleepTime Sleep time. + */ + private void sleep(long sleepTime) { + try { + if (Thread.currentThread().isInterrupted()) + return; + + U.sleep(sleepTime); + } + catch (IgniteInterruptedCheckedException e) { + Thread.interrupted(); + } + } + /** * */ @@ -2363,7 +2526,7 @@ public synchronized int size() { /** {@inheritDoc} */ @Override public synchronized void onUpdated(Iterable> evts) - throws CacheEntryListenerException { + throws CacheEntryListenerException { try { for (CacheEntryEvent evt : evts) { Integer key = (Integer)evt.getKey(); @@ -2438,6 +2601,52 @@ public static class CacheEventListener3 implements CacheEntryUpdatedListener { + /** Listener count. */ + private final AtomicLong cntr = new AtomicLong(); + + /** Listener map. */ + private final Map evtMap = new ConcurrentHashMap<>(); + + /** Atomicity mode flag. */ + private final boolean atomicModeFlag; + + /** Constructor */ + public CacheEventListener4(boolean atomicModeFlag) { + this.atomicModeFlag = atomicModeFlag; + } + + /** {@inheritDoc} */ + @SuppressWarnings("EqualsBetweenInconvertibleTypes") + @Override public void onUpdated(Iterable> evts) + throws CacheEntryListenerException { + for (CacheEntryEvent evt : evts) { + Integer prev = evtMap.put(evt.getKey(), evt.getValue()); + + //Atomic cache allows duplicate events if cache update operation fails, e.g. due to topology change. + if (!atomicModeFlag || prev == null || !prev.equals(evt)) + cntr.incrementAndGet(); + } + } + + /** + * @return Events count. + */ + public long count() { + return cntr.get(); + } + + /** + * @return Event map. + */ + Map eventMap() { + return evtMap; + } + } + /** * */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverTxSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverTxSelfTest.java index 789a105e02b05..c5240da3f524a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverTxSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverTxSelfTest.java @@ -36,4 +36,9 @@ public class CacheContinuousQueryFailoverTxSelfTest extends CacheContinuousQuery @Override protected CacheAtomicityMode atomicityMode() { return TRANSACTIONAL; } + + /** {@inheritDoc} */ + public void testNoEventLossOnTopologyChange() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-4015"); + } } From f597aff1bdf65d3d430cf85c9932391a72c2d7dc Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 12 Oct 2016 15:44:08 +0300 Subject: [PATCH 237/487] IGNITE-3875: Added separate thread pool for data streamer. This closes #1067. --- .../configuration/IgniteConfiguration.java | 31 ++++++ .../ignite/internal/GridKernalContext.java | 7 ++ .../internal/GridKernalContextImpl.java | 12 +++ .../apache/ignite/internal/IgniteKernal.java | 3 + .../apache/ignite/internal/IgnitionEx.java | 20 +++- .../managers/communication/GridIoManager.java | 2 + .../managers/communication/GridIoPolicy.java | 3 + .../closure/GridClosureProcessor.java | 3 +- .../datastreamer/DataStreamProcessor.java | 82 ++++++++++++++-- .../datastreamer/DataStreamerImpl.java | 31 +----- .../processors/pool/PoolProcessor.java | 3 + .../DataStreamProcessorSelfTest.java | 97 +++++++++++++++++++ .../junits/GridTestKernalContext.java | 12 +-- 13 files changed, 262 insertions(+), 44 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java index 73de470719b06..412ecbf4742c3 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java @@ -148,6 +148,9 @@ public class IgniteConfiguration { /** Default core size of public thread pool. */ public static final int DFLT_PUBLIC_THREAD_CNT = Math.max(8, AVAILABLE_PROC_CNT) * 2; + /** Default size of data streamer thread pool. */ + public static final int DFLT_DATA_STREAMER_POOL_SIZE = DFLT_PUBLIC_THREAD_CNT; + /** Default keep alive time for public thread pool. */ @Deprecated public static final long DFLT_PUBLIC_KEEP_ALIVE_TIME = 0; @@ -245,6 +248,9 @@ public class IgniteConfiguration { /** IGFS pool size. */ private int igfsPoolSize = AVAILABLE_PROC_CNT; + /** Data stream pool size. */ + private int dataStreamerPoolSize = DFLT_DATA_STREAMER_POOL_SIZE; + /** Utility cache pool size. */ private int utilityCachePoolSize = DFLT_SYSTEM_CORE_THREAD_CNT; @@ -508,6 +514,7 @@ public IgniteConfiguration(IgniteConfiguration cfg) { clockSyncFreq = cfg.getClockSyncFrequency(); clockSyncSamples = cfg.getClockSyncSamples(); consistentId = cfg.getConsistentId(); + dataStreamerPoolSize = cfg.getDataStreamerThreadPoolSize(); deployMode = cfg.getDeploymentMode(); discoStartupDelay = cfg.getDiscoveryStartupDelay(); failureDetectionTimeout = cfg.getFailureDetectionTimeout(); @@ -788,6 +795,17 @@ public int getIgfsThreadPoolSize() { return igfsPoolSize; } + /** + * Size of thread pool that is in charge of processing data stream messages. + *

          + * If not provided, executor service will have size {@link #DFLT_DATA_STREAMER_POOL_SIZE}. + * + * @return Thread pool size to be used for data stream messages. + */ + public int getDataStreamerThreadPoolSize() { + return dataStreamerPoolSize; + } + /** * Default size of thread pool that is in charge of processing utility cache messages. *

          @@ -911,6 +929,19 @@ public IgniteConfiguration setIgfsThreadPoolSize(int poolSize) { return this; } + /** + * Set thread pool size that will be used to process data stream messages. + * + * @param poolSize Executor service to use for data stream messages. + * @see IgniteConfiguration#getDataStreamerThreadPoolSize() + * @return {@code this} for chaining. + */ + public IgniteConfiguration setDataStreamerThreadPoolSize(int poolSize) { + dataStreamerPoolSize = poolSize; + + return this; + } + /** * Sets default thread pool size that will be used to process utility cache messages. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java index e608af2a09767..178239bd2db33 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java @@ -531,6 +531,13 @@ public interface GridKernalContext extends Iterable { */ public ExecutorService getIgfsExecutorService(); + /** + * Executor service that is in charge of processing data stream messages. + * + * @return Thread pool implementation to be used for data stream messages. + */ + public ExecutorService getDataStreamerExecutorService(); + /** * Should return an instance of fully configured thread pool to be used for * processing of client messages (REST requests). diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java index ddef345ffd151..1a9663b5e10bf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java @@ -310,6 +310,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable @GridToStringExclude private ExecutorService igfsExecSvc; + /** */ + @GridToStringExclude + private ExecutorService dataStreamExecSvc; + /** */ @GridToStringExclude protected ExecutorService restExecSvc; @@ -380,6 +384,7 @@ public GridKernalContextImpl() { * @param p2pExecSvc P2P executor service. * @param mgmtExecSvc Management executor service. * @param igfsExecSvc IGFS executor service. + * @param dataStreamExecSvc data stream executor service. * @param restExecSvc REST executor service. * @param affExecSvc Affinity executor service. * @param plugins Plugin providers. @@ -398,6 +403,7 @@ protected GridKernalContextImpl( ExecutorService p2pExecSvc, ExecutorService mgmtExecSvc, ExecutorService igfsExecSvc, + ExecutorService dataStreamExecSvc, ExecutorService restExecSvc, ExecutorService affExecSvc, IgniteStripedThreadPoolExecutor callbackExecSvc, @@ -416,6 +422,7 @@ protected GridKernalContextImpl( this.p2pExecSvc = p2pExecSvc; this.mgmtExecSvc = mgmtExecSvc; this.igfsExecSvc = igfsExecSvc; + this.dataStreamExecSvc = dataStreamExecSvc; this.restExecSvc = restExecSvc; this.affExecSvc = affExecSvc; this.callbackExecSvc = callbackExecSvc; @@ -955,6 +962,11 @@ protected Object readResolve() throws ObjectStreamException { return igfsExecSvc; } + /** {@inheritDoc} */ + @Override public ExecutorService getDataStreamerExecutorService() { + return dataStreamExecSvc; + } + /** {@inheritDoc} */ @Override public ExecutorService getRestExecutorService() { return restExecSvc; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 586215aa94ac1..e838dd26abb94 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -666,6 +666,7 @@ private void notifyLifecycleBeansEx(LifecycleEventType evt) { * @param p2pExecSvc P2P executor service. * @param mgmtExecSvc Management executor service. * @param igfsExecSvc IGFS executor service. + * @param dataStreamExecSvc data stream executor service. * @param restExecSvc Reset executor service. * @param affExecSvc Affinity executor service. * @param errHnd Error handler to use for notification about startup problems. @@ -680,6 +681,7 @@ public void start(final IgniteConfiguration cfg, ExecutorService p2pExecSvc, ExecutorService mgmtExecSvc, ExecutorService igfsExecSvc, + ExecutorService dataStreamExecSvc, ExecutorService restExecSvc, ExecutorService affExecSvc, IgniteStripedThreadPoolExecutor callbackExecSvc, @@ -786,6 +788,7 @@ public void start(final IgniteConfiguration cfg, p2pExecSvc, mgmtExecSvc, igfsExecSvc, + dataStreamExecSvc, restExecSvc, affExecSvc, callbackExecSvc, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index a6860b3a4ccac..104599c0f6ecb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -1468,6 +1468,9 @@ private static final class IgniteNamedInstance { /** IGFS executor service. */ private ThreadPoolExecutor igfsExecSvc; + /** Data streamer executor service. */ + private ThreadPoolExecutor dataStreamerExecSvc; + /** REST requests executor service. */ private ThreadPoolExecutor restExecSvc; @@ -1687,6 +1690,17 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { p2pExecSvc.allowCoreThreadTimeOut(true); + // Note that we do not pre-start threads here as this pool may not be needed. + dataStreamerExecSvc = new IgniteThreadPoolExecutor( + "data-streamer", + cfg.getGridName(), + cfg.getDataStreamerThreadPoolSize(), + cfg.getDataStreamerThreadPoolSize(), + DFLT_THREAD_KEEP_ALIVE_TIME, + new LinkedBlockingQueue()); + + dataStreamerExecSvc.allowCoreThreadTimeOut(true); + // Note that we do not pre-start threads here as igfs pool may not be needed. igfsExecSvc = new IgniteThreadPoolExecutor( cfg.getIgfsThreadPoolSize(), @@ -1759,7 +1773,7 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { grid = grid0; grid0.start(myCfg, utilityCacheExecSvc, marshCacheExecSvc, execSvc, sysExecSvc, p2pExecSvc, mgmtExecSvc, - igfsExecSvc, restExecSvc, affExecSvc, callbackExecSvc, + igfsExecSvc, dataStreamerExecSvc, restExecSvc, affExecSvc, callbackExecSvc, new CA() { @Override public void apply() { startLatch.countDown(); @@ -2377,6 +2391,10 @@ private void stopExecutors0(IgniteLogger log) { p2pExecSvc = null; + U.shutdownNow(getClass(), dataStreamerExecSvc, log); + + dataStreamerExecSvc = null; + U.shutdownNow(getClass(), igfsExecSvc, log); igfsExecSvc = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index cd43318c455cd..ba41dab54f03d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -84,6 +84,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.internal.GridTopic.TOPIC_COMM_USER; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.AFFINITY_POOL; +import static org.apache.ignite.internal.managers.communication.GridIoPolicy.DATA_STREAMER_POOL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IGFS_POOL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MANAGEMENT_POOL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MARSH_CACHE_POOL; @@ -574,6 +575,7 @@ private void onMessage0(UUID nodeId, GridIoMessage msg, IgniteRunnable msgC) { case UTILITY_CACHE_POOL: case MARSH_CACHE_POOL: case IGFS_POOL: + case DATA_STREAMER_POOL: { if (msg.isOrdered()) processOrderedMessage(nodeId, msg, plc, msgC); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java index 00590ba50a92e..71279d9613ddc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java @@ -46,6 +46,9 @@ public class GridIoPolicy { /** Marshaller cache execution pool. */ public static final byte IGFS_POOL = 7; + /** Data streamer execution pool. */ + public static final byte DATA_STREAMER_POOL = 8; + /** * Defines the range of reserved pools that are not available for plugins. * @param key The key. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java index d388584846125..4bd22c586b25e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java @@ -27,7 +27,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Callable; -import java.util.concurrent.Executor; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import org.apache.ignite.IgniteCheckedException; @@ -970,7 +969,7 @@ public IgniteInternalFuture callLocalSafe(Callable c, boolean sys) { * @param plc Policy to choose executor pool. * @return Future. */ - private IgniteInternalFuture callLocalSafe(Callable c, byte plc) { + public IgniteInternalFuture callLocalSafe(Callable c, byte plc) { try { return callLocal(c, plc); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java index 7663735fc26c6..bd33f62cbfc7e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java @@ -21,10 +21,12 @@ import java.util.UUID; import java.util.concurrent.DelayQueue; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.managers.communication.GridIoManager; +import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.deployment.GridDeployment; import org.apache.ignite.internal.processors.GridProcessorAdapter; @@ -35,6 +37,7 @@ import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.GridWorker; +import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.stream.StreamReceiver; @@ -42,12 +45,23 @@ import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.GridTopic.TOPIC_DATASTREAM; -import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL; /** - * + * Data stream processor. */ public class DataStreamProcessor extends GridProcessorAdapter { + /** Data streamer separate pool feature major version. */ + private static final int DATA_STREAMER_POOL_MAJOR_VER = 1; + + /** Data streamer separate pool feature minor version. */ + private static final int DATA_STREAMER_POOL_MINOR_VER = 6; + + /** Data streamer separate pool feature maintenance version. */ + private static final int DATA_STREAMER_POOL_MAINTENANCE_VER = 10; + + /** Default pool for data streamer messages processing. */ + public static final byte DFLT_POLICY = GridIoPolicy.PUBLIC_POOL; + /** Loaders map (access is not supposed to be highly concurrent). */ private Collection ldrs = new GridConcurrentHashSet<>(); @@ -218,13 +232,15 @@ private void processRequest(final UUID nodeId, final DataStreamerRequest req) { IgniteInternalFuture fut = ctx.cache().context().exchange().affinityReadyFuture(rmtAffVer); if (fut != null && !fut.isDone()) { + final byte plc = threadIoPolicy(); + fut.listen(new CI1>() { @Override public void apply(IgniteInternalFuture t) { ctx.closure().runLocalSafe(new Runnable() { @Override public void run() { processRequest(nodeId, req); } - }, false); + }, plc); } }); @@ -340,12 +356,7 @@ private void sendResponse(UUID nodeId, Object resTopic, long reqId, @Nullable Th DataStreamerResponse res = new DataStreamerResponse(reqId, errBytes, forceLocDep); try { - Byte plc = GridIoManager.currentPolicy(); - - if (plc == null) - plc = PUBLIC_POOL; - - ctx.io().send(nodeId, resTopic, res, plc); + ctx.io().send(nodeId, resTopic, res, threadIoPolicy()); } catch (IgniteCheckedException e) { if (ctx.discovery().alive(nodeId)) @@ -355,6 +366,59 @@ else if (log.isDebugEnabled()) } } + /** + * Get IO policy. + * + * @return IO policy. + */ + private static byte threadIoPolicy() { + Byte plc = GridIoManager.currentPolicy(); + + if (plc == null) + plc = DFLT_POLICY; + + return plc; + } + + /** + * Get IO policy for particular node. + * + * @param node Node. + * @return Policy. + */ + public static byte ioPolicy(ClusterNode node) { + assert node != null; + + if (node.isLocal() || node.version().greaterThanEqual( + DATA_STREAMER_POOL_MAJOR_VER, + DATA_STREAMER_POOL_MINOR_VER, + DATA_STREAMER_POOL_MAINTENANCE_VER)) + return GridIoPolicy.DATA_STREAMER_POOL; + else + return DFLT_POLICY; + } + + /** + * Get IO policy for particular node with provided resolver. + * + * @param rslvr Resolver. + * @param node Node. + * @return IO policy. + */ + public static byte ioPolicy(@Nullable IgniteClosure rslvr, ClusterNode node) { + assert node != null; + + Byte res = null; + + if (rslvr != null) + res = rslvr.apply(node); + + if (res == null) + res = ioPolicy(node); + + return res; + } + /** {@inheritDoc} */ @Override public void printMemoryStats() { X.println(">>>"); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java index c2f226c537025..46f6380dc22d7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java @@ -39,7 +39,6 @@ import java.util.concurrent.atomic.AtomicReference; import javax.cache.CacheException; import javax.cache.expiry.ExpiryPolicy; - import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; @@ -60,7 +59,6 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; -import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.deployment.GridDeployment; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; @@ -106,16 +104,12 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.internal.GridTopic.TOPIC_DATASTREAM; -import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL; /** * Data streamer implementation. */ @SuppressWarnings("unchecked") public class DataStreamerImpl implements IgniteDataStreamer, Delayed { - /** Default policy reoslver. */ - private static final DefaultIoPolicyResolver DFLT_IO_PLC_RSLVR = new DefaultIoPolicyResolver(); - /** Isolated receiver. */ private static final StreamReceiver ISOLATED_UPDATER = new IsolatedUpdater(); @@ -126,7 +120,7 @@ public class DataStreamerImpl implements IgniteDataStreamer, Delayed private byte[] updaterBytes; /** IO policy resovler for data load request. */ - private IgniteClosure ioPlcRslvr = DFLT_IO_PLC_RSLVR; + private IgniteClosure ioPlcRslvr; /** Max remap count before issuing an error. */ private static final int DFLT_MAX_REMAP_CNT = 32; @@ -1313,14 +1307,12 @@ private void submit(final Collection entries, IgniteInternalFuture fut; - Byte plc = ioPlcRslvr.apply(node); - - if (plc == null) - plc = PUBLIC_POOL; + byte plc = DataStreamProcessor.ioPolicy(ioPlcRslvr, node); - if (isLocNode && plc == GridIoPolicy.PUBLIC_POOL) { + if (isLocNode) { fut = ctx.closure().callLocalSafe( - new DataStreamerUpdateJob(ctx, log, cacheName, entries, false, skipStore, keepBinary, rcvr), false); + new DataStreamerUpdateJob(ctx, log, cacheName, entries, false, skipStore, keepBinary, rcvr), + plc); locFuts.add(fut); @@ -1683,19 +1675,6 @@ else if (ttl == CU.TTL_NOT_CHANGED) } } - /** - * Default IO policy resolver. - */ - private static class DefaultIoPolicyResolver implements IgniteClosure { - /** */ - private static final long serialVersionUID = 0L; - - /** {@inheritDoc} */ - @Override public Byte apply(ClusterNode gridNode) { - return PUBLIC_POOL; - } - } - /** * Key object wrapper. Using identity equals prevents slow down in case of hash code collision. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java index 41e805ef9c515..5d38e1645e8cf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java @@ -123,6 +123,9 @@ public Executor poolForPolicy(byte plc) throws IgniteCheckedException { return ctx.getIgfsExecutorService(); + case GridIoPolicy.DATA_STREAMER_POOL: + return ctx.getDataStreamerExecutorService(); + default: { if (plc < 0) throw new IgniteCheckedException("Policy cannot be negative: " + plc); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java index 9fedc35ae4c8e..401b09c51b20c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java @@ -33,6 +33,7 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.IgniteException; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.affinity.Affinity; @@ -49,6 +50,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; @@ -59,6 +61,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.stream.StreamReceiver; +import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jetbrains.annotations.Nullable; @@ -948,6 +951,100 @@ public void testCustomUserUpdater() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testLocalDataStreamerDedicatedThreadPool() throws Exception { + try { + useCache = true; + + Ignite ignite = startGrid(1); + + final IgniteCache cache = ignite.cache(null); + + IgniteDataStreamer ldr = ignite.dataStreamer(null); + try { + ldr.receiver(new StreamReceiver() { + @Override public void receive(IgniteCache cache, + Collection> entries) throws IgniteException { + String threadName = Thread.currentThread().getName(); + + cache.put("key", threadName); + } + }); + ldr.addData("key", "value"); + + ldr.tryFlush(); + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return cache.get("key") != null; + } + }, 3_000); + } + finally { + ldr.close(true); + } + + assertNotNull(cache.get("key")); + + assertTrue(cache.get("key").startsWith("data-streamer")); + + } + finally { + stopAllGrids(); + } + } + + /** + * @throws Exception If failed. + */ + public void testRemoteDataStreamerDedicatedThreadPool() throws Exception { + try { + useCache = true; + + Ignite ignite = startGrid(1); + + useCache = false; + + Ignite client = startGrid(0); + + final IgniteCache cache = ignite.cache(null); + + IgniteDataStreamer ldr = client.dataStreamer(null); + try { + ldr.receiver(new StreamReceiver() { + @Override public void receive(IgniteCache cache, + Collection> entries) throws IgniteException { + String threadName = Thread.currentThread().getName(); + + cache.put("key", threadName); + } + }); + + ldr.addData("key", "value"); + + ldr.tryFlush(); + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return cache.get("key") != null; + } + }, 3_000); + } + finally { + ldr.close(true); + } + + assertNotNull(cache.get("key")); + + assertTrue(cache.get("key").startsWith("data-streamer")); + } + finally { + stopAllGrids(); + } + } + /** * */ diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java index cba67e0d06005..1d72d4640247e 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java @@ -62,6 +62,7 @@ public GridTestKernalContext(IgniteLogger log, IgniteConfiguration cfg) throws I null, null, null, + null, U.allPluginProviders()); GridTestUtils.setFieldValue(grid(), "cfg", config()); @@ -95,11 +96,6 @@ public void stop(boolean cancel) throws IgniteCheckedException { } } - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(GridTestKernalContext.class, this, super.toString()); - } - /** * Sets system executor service. * @@ -109,7 +105,6 @@ public void setSystemExecutorService(ExecutorService sysExecSvc) { this.sysExecSvc = sysExecSvc; } - /** * Sets executor service. * @@ -118,4 +113,9 @@ public void setSystemExecutorService(ExecutorService sysExecSvc) { public void setExecutorService(ExecutorService execSvc){ this.execSvc = execSvc; } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridTestKernalContext.class, this, super.toString()); + } } From 2ab094e08373dc6667af44d48a38b4f044953a79 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Wed, 12 Oct 2016 16:48:51 +0300 Subject: [PATCH 238/487] IGNITE-2355: Hadoop: added ability to configure multiple job tracker addresses. This closes #1153. --- .../ignite/testframework/GridTestUtils.java | 37 +++ .../IgniteHadoopClientProtocolProvider.java | 53 +-- .../hadoop/impl/HadoopAbstractSelfTest.java | 4 +- ...ClientProtocolMultipleServersSelfTest.java | 307 ++++++++++++++++++ .../testsuites/IgniteHadoopTestSuite.java | 2 + 5 files changed, 381 insertions(+), 22 deletions(-) create mode 100644 modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java index 524c643989841..b3ce46b9ed1d4 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java @@ -263,6 +263,43 @@ public static Throwable assertThrows(@Nullable IgniteLogger log, Callable cal throw new AssertionError("Exception has not been thrown."); } + /** + * Checks whether callable throws an exception with specified cause. + * + * @param log Logger (optional). + * @param call Callable. + * @param cls Exception class. + * @param msg Exception message (optional). If provided exception message + * and this message should be equal. + * @return Thrown throwable. + */ + public static Throwable assertThrowsAnyCause(@Nullable IgniteLogger log, Callable call, + Class cls, @Nullable String msg) { + assert call != null; + assert cls != null; + + try { + call.call(); + } + catch (Throwable e) { + Throwable t = e; + + while (t != null) { + if (cls == t.getClass() && (msg == null || (t.getMessage() != null || t.getMessage().contains(msg)))) { + log.info("Caught expected exception: " + t.getMessage()); + + return t; + } + + t = t.getCause(); + } + + fail("Unexpected exception", e); + } + + throw new AssertionError("Exception has not been thrown."); + } + /** * Checks whether callable throws expected exception or its child or not. * diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java index 343b5edbb4182..1efe6251d640d 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java @@ -17,11 +17,18 @@ package org.apache.ignite.hadoop.mapreduce; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.mapreduce.protocol.ClientProtocol; import org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.ConnectorConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.client.GridClient; import org.apache.ignite.internal.client.GridClientConfiguration; @@ -32,11 +39,6 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.F; -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.Collections; -import java.util.concurrent.ConcurrentHashMap; - import static org.apache.ignite.internal.client.GridClientProtocol.TCP; @@ -53,17 +55,27 @@ public class IgniteHadoopClientProtocolProvider extends ClientProtocolProvider { /** {@inheritDoc} */ @Override public ClientProtocol create(Configuration conf) throws IOException { if (FRAMEWORK_NAME.equals(conf.get(MRConfig.FRAMEWORK_NAME))) { - String addr = conf.get(MRConfig.MASTER_ADDRESS); + Collection addrs = conf.getTrimmedStringCollection(MRConfig.MASTER_ADDRESS); - if (F.isEmpty(addr)) - throw new IOException("Failed to create client protocol because server address is not specified (is " + - MRConfig.MASTER_ADDRESS + " property set?)."); + if (F.isEmpty(addrs)) + throw new IOException("Failed to create client protocol because Ignite node addresses are not " + + "specified (did you set " + MRConfig.MASTER_ADDRESS + " property?)."); - if (F.eq(addr, "local")) + if (F.contains(addrs, "local")) throw new IOException("Local execution mode is not supported, please point " + - MRConfig.MASTER_ADDRESS + " to real Ignite node."); + MRConfig.MASTER_ADDRESS + " to real Ignite nodes."); + + Collection addrs0 = new ArrayList<>(addrs.size()); + + // Set up port by default if need + for (String addr : addrs) { + if (!addr.contains(":")) + addrs0.add(addr + ':' + ConnectorConfiguration.DFLT_TCP_PORT); + else + addrs0.add(addr); + } - return createProtocol(addr, conf); + return new HadoopClientProtocol(conf, client(conf.get(MRConfig.MASTER_ADDRESS), addrs0)); } return null; @@ -91,24 +103,25 @@ public class IgniteHadoopClientProtocolProvider extends ClientProtocolProvider { * @throws IOException If failed. */ private static ClientProtocol createProtocol(String addr, Configuration conf) throws IOException { - return new HadoopClientProtocol(conf, client(addr)); + return new HadoopClientProtocol(conf, client(addr, Collections.singletonList(addr))); } /** * Create client. * - * @param addr Endpoint address. + * @param clusterName Ignite cluster logical name. + * @param addrs Endpoint addresses. * @return Client. * @throws IOException If failed. */ - private static GridClient client(String addr) throws IOException { + private static GridClient client(String clusterName, Collection addrs) throws IOException { try { - IgniteInternalFuture fut = cliMap.get(addr); + IgniteInternalFuture fut = cliMap.get(clusterName); if (fut == null) { GridFutureAdapter fut0 = new GridFutureAdapter<>(); - IgniteInternalFuture oldFut = cliMap.putIfAbsent(addr, fut0); + IgniteInternalFuture oldFut = cliMap.putIfAbsent(clusterName, fut0); if (oldFut != null) return oldFut.get(); @@ -116,7 +129,7 @@ private static GridClient client(String addr) throws IOException { GridClientConfiguration cliCfg = new GridClientConfiguration(); cliCfg.setProtocol(TCP); - cliCfg.setServers(Collections.singletonList(addr)); + cliCfg.setServers(addrs); cliCfg.setMarshaller(new GridClientJdkMarshaller()); cliCfg.setMaxConnectionIdleTime(24 * 60 * 60 * 1000L); // 1 day. cliCfg.setDaemon(true); @@ -131,7 +144,7 @@ private static GridClient client(String addr) throws IOException { catch (GridClientException e) { fut0.onDone(e); - throw new IOException("Failed to establish connection with Ignite node: " + addr, e); + throw new IOException("Failed to establish connection with Ignite: " + addrs, e); } } } @@ -139,7 +152,7 @@ private static GridClient client(String addr) throws IOException { return fut.get(); } catch (IgniteCheckedException e) { - throw new IOException("Failed to establish connection with Ignite node: " + addr, e); + throw new IOException("Failed to establish connection with Ignite сдгые: " + addrs, e); } } } \ No newline at end of file diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractSelfTest.java index 68009dd7d3949..12351c676424b 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractSelfTest.java @@ -48,7 +48,7 @@ public abstract class HadoopAbstractSelfTest extends GridCommonAbstractTest { private static TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); /** REST port. */ - protected static final int REST_PORT = 11212; + protected static final int REST_PORT = ConnectorConfiguration.DFLT_TCP_PORT; /** IGFS name. */ protected static final String igfsName = null; @@ -185,7 +185,7 @@ public CacheConfiguration metaCacheConfiguration() { /** * @return IGFS data cache configuration. */ - private CacheConfiguration dataCacheConfiguration() { + protected CacheConfiguration dataCacheConfiguration() { CacheConfiguration cfg = new CacheConfiguration(); cfg.setName(igfsDataCacheName); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java new file mode 100644 index 0000000000000..04747d0b678f7 --- /dev/null +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java @@ -0,0 +1,307 @@ +/* + * 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.hadoop.impl.client; + +import java.io.BufferedWriter; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.Callable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobStatus; +import org.apache.hadoop.mapreduce.MRConfig; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.OutputCommitter; +import org.apache.hadoop.mapreduce.OutputFormat; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.Reducer; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; +import org.apache.ignite.IgniteFileSystem; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.hadoop.mapreduce.IgniteHadoopClientProtocolProvider; +import org.apache.ignite.igfs.IgfsPath; +import org.apache.ignite.internal.client.GridServerUnreachableException; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopAbstractSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridTestUtils; + +/** + * Hadoop client protocol configured with multiple ignite servers tests. + */ +@SuppressWarnings("ResultOfMethodCallIgnored") +public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractSelfTest { + /** Input path. */ + private static final String PATH_INPUT = "/input"; + + /** Job name. */ + private static final String JOB_NAME = "myJob"; + + /** {@inheritDoc} */ + @Override protected boolean igfsEnabled() { + return true; + } + + /** {@inheritDoc} */ + @Override protected boolean restEnabled() { + return true; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + startGrids(gridCount()); + + awaitPartitionMapExchange(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + + /** {@inheritDoc} */ + @Override protected CacheConfiguration dataCacheConfiguration() { + CacheConfiguration cfg = super.dataCacheConfiguration(); + + cfg.setBackups(1); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + private void beforeJob() throws Exception { + IgniteFileSystem igfs = grid(0).fileSystem(HadoopAbstractSelfTest.igfsName); + + igfs.format(); + + igfs.mkdirs(new IgfsPath(PATH_INPUT)); + + try (BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(igfs.create( + new IgfsPath(PATH_INPUT + "/test.file"), true)))) { + + bw.write("word"); + } + } + + /** + * Test job submission. + * + * @param conf Hadoop configuration. + * @throws Exception If failed. + */ + public void checkJobSubmit(Configuration conf) throws Exception { + final Job job = Job.getInstance(conf); + + job.setJobName(JOB_NAME); + + job.setOutputKeyClass(Text.class); + job.setOutputValueClass(IntWritable.class); + + job.setInputFormatClass(TextInputFormat.class); + job.setOutputFormatClass(OutFormat.class); + + job.setMapperClass(TestMapper.class); + job.setReducerClass(TestReducer.class); + + job.setNumReduceTasks(0); + + FileInputFormat.setInputPaths(job, new Path(PATH_INPUT)); + + job.submit(); + + job.waitForCompletion(false); + + assert job.getStatus().getState() == JobStatus.State.SUCCEEDED : job.getStatus().getState(); + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings("ConstantConditions") + public void testMultipleAddresses() throws Exception { + beforeJob(); + + stopGrid(0); + + U.sleep(5000); + + checkJobSubmit(configMultipleAddrs(gridCount())); + + startGrid(0); + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings({"ConstantConditions", "ThrowableResultOfMethodCallIgnored"}) + public void testSingleAddress() throws Exception { + stopGrid(0); + + U.sleep(5000); + + GridTestUtils.assertThrowsAnyCause(log, new Callable() { + @Override public Object call() throws Exception { + checkJobSubmit(configSingleAddress()); + return null; + } + }, + GridServerUnreachableException.class, "Failed to connect to any of the servers in list"); + + startGrid(0); + + awaitPartitionMapExchange(); + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings("ConstantConditions") + public void testMixedAddrs() throws Exception { + beforeJob(); + + stopGrid(1); + + U.sleep(5000); + + checkJobSubmit(configMixed()); + + startGrid(1); + + awaitPartitionMapExchange(); + } + + /** + * @return Configuration. + */ + private Configuration configSingleAddress() { + Configuration conf = HadoopUtils.safeCreateConfiguration(); + + setupFileSystems(conf); + + conf.set(MRConfig.FRAMEWORK_NAME, IgniteHadoopClientProtocolProvider.FRAMEWORK_NAME); + conf.set(MRConfig.MASTER_ADDRESS, "127.0.0.1:" + REST_PORT); + + conf.set("fs.defaultFS", "igfs:///"); + + return conf; + } + + /** + * @param serversCnt Count ov servers. + * @return Configuration. + */ + private Configuration configMultipleAddrs(int serversCnt) { + Configuration conf = HadoopUtils.safeCreateConfiguration(); + + setupFileSystems(conf); + + conf.set(MRConfig.FRAMEWORK_NAME, IgniteHadoopClientProtocolProvider.FRAMEWORK_NAME); + + Collection addrs = new ArrayList<>(serversCnt); + + for (int i = 0; i < serversCnt; ++i) + addrs.add("127.0.0.1:" + Integer.toString(REST_PORT + i)); + + conf.set(MRConfig.MASTER_ADDRESS, F.concat(addrs, ",")); + + conf.set("fs.defaultFS", "igfs:///"); + + return conf; + } + + /** + * @return Configuration. + */ + private Configuration configMixed() { + Configuration conf = HadoopUtils.safeCreateConfiguration(); + + setupFileSystems(conf); + + conf.set(MRConfig.FRAMEWORK_NAME, IgniteHadoopClientProtocolProvider.FRAMEWORK_NAME); + + Collection addrs = new ArrayList<>(); + + addrs.add("localhost"); + addrs.add("127.0.0.1:" + Integer.toString(REST_PORT + 1)); + + conf.set(MRConfig.MASTER_ADDRESS, F.concat(addrs, ",")); + + conf.set("fs.defaultFS", "igfs:///"); + + return conf; + } + + /** + * Test mapper. + */ + public static class TestMapper extends Mapper { + /** {@inheritDoc} */ + @Override public void map(Object key, Text val, Context ctx) throws IOException, InterruptedException { + // No-op. + } + } + + /** + * Test reducer. + */ + public static class TestReducer extends Reducer { + + /** {@inheritDoc} */ + @Override public void reduce(Text key, Iterable values, Context ctx) throws IOException, + InterruptedException { + // No-op. + } + } + + /** + * Test output formatter. + */ + public static class OutFormat extends OutputFormat { + /** {@inheritDoc} */ + @Override public RecordWriter getRecordWriter(TaskAttemptContext context) throws IOException, + InterruptedException { + return null; + } + + /** {@inheritDoc} */ + @Override public void checkOutputSpecs(JobContext context) throws IOException, InterruptedException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, + InterruptedException { + return null; + } + } +} \ No newline at end of file diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java index bbd92d131d21f..959bc59f819be 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java @@ -24,6 +24,7 @@ import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.processors.hadoop.HadoopTestClassLoader; import org.apache.ignite.internal.processors.hadoop.impl.client.HadoopClientProtocolEmbeddedSelfTest; +import org.apache.ignite.internal.processors.hadoop.impl.client.HadoopClientProtocolMultipleServersSelfTest; import org.apache.ignite.internal.processors.hadoop.impl.client.HadoopClientProtocolSelfTest; import org.apache.ignite.internal.processors.hadoop.impl.HadoopTxConfigCacheTest; import org.apache.ignite.internal.processors.hadoop.impl.fs.KerberosHadoopFileSystemFactorySelfTest; @@ -194,6 +195,7 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(ldr.loadClass(HadoopClientProtocolSelfTest.class.getName()))); suite.addTest(new TestSuite(ldr.loadClass(HadoopClientProtocolEmbeddedSelfTest.class.getName()))); + suite.addTest(new TestSuite(ldr.loadClass(HadoopClientProtocolMultipleServersSelfTest.class.getName()))); suite.addTest(new TestSuite(ldr.loadClass(HadoopCommandLineTest.class.getName()))); From eaf8ae246cc799c1353332fcac05cb3a8efab02c Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Wed, 12 Oct 2016 19:57:09 +0300 Subject: [PATCH 239/487] IGNITE-4034 Get rid of specialized methods in platform targets --- .../platform/PlatformAbstractTarget.java | 90 +- .../processors/platform/PlatformTarget.java | 23 + .../platform/cache/PlatformCache.java | 373 ++--- .../cache/affinity/PlatformAffinity.java | 14 +- .../query/PlatformAbstractQueryCursor.java | 42 +- .../query/PlatformContinuousQueryProxy.java | 53 + .../store/PlatformCacheStoreCallback.java | 61 - .../callback/PlatformCallbackGateway.java | 5 +- .../callback/PlatformCallbackUtils.java | 3 +- .../cluster/PlatformClusterGroup.java | 110 +- .../platform/compute/PlatformCompute.java | 77 +- .../datastreamer/PlatformDataStreamer.java | 112 +- .../datastructures/PlatformAtomicLong.java | 179 ++- .../PlatformAtomicReference.java | 38 +- .../PlatformAtomicSequence.java | 131 +- .../dotnet/PlatformDotNetCacheStore.java | 122 +- .../platform/events/PlatformEvents.java | 91 +- .../platform/messaging/PlatformMessaging.java | 30 +- .../platform/services/PlatformServices.java | 133 +- .../transactions/PlatformTransactions.java | 227 +-- .../cpp/core/include/ignite/cache/cache.h | 15 +- .../include/ignite/impl/cache/cache_impl.h | 41 +- .../ignite/impl/interop/interop_target.h | 18 + .../cpp/core/src/impl/cache/cache_impl.cpp | 48 +- .../core/src/impl/cache/query/query_impl.cpp | 15 +- .../core/src/impl/interop/interop_target.cpp | 25 + .../impl/transactions/transactions_impl.cpp | 130 +- .../cpp/jni/include/ignite/jni/exports.h | 101 +- .../cpp/jni/include/ignite/jni/java.h | 217 +-- .../platforms/cpp/jni/project/vs/module.def | 85 +- modules/platforms/cpp/jni/src/exports.cpp | 345 +---- modules/platforms/cpp/jni/src/java.cpp | 1315 ++--------------- .../Cache/CacheAbstractTest.cs | 44 + .../Services/ServicesTest.cs | 1 + .../Apache.Ignite.Core.csproj | 1 + .../Impl/Cache/CacheAffinityImpl.cs | 5 +- .../Impl/Cache/CacheImpl.cs | 149 +- .../Impl/Cache/CacheLock.cs | 24 +- .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs | 18 +- .../Impl/Cache/ICacheLockInternal.cs | 47 + .../Impl/Cache/Query/AbstractQueryCursor.cs | 10 +- .../Continuous/ContinuousQueryHandleImpl.cs | 117 +- .../Impl/Cache/Store/CacheStore.cs | 108 +- .../Impl/Cluster/ClusterGroupImpl.cs | 54 +- .../Apache.Ignite.Core/Impl/Common/Future.cs | 30 +- .../Impl/Compute/ComputeImpl.cs | 19 +- .../Impl/DataStructures/AtomicLong.cs | 34 +- .../Impl/DataStructures/AtomicReference.cs | 8 +- .../Impl/DataStructures/AtomicSequence.cs | 26 +- .../Impl/Datastream/DataStreamerImpl.cs | 49 +- .../Apache.Ignite.Core/Impl/Events/Events.cs | 18 +- .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs | 2 +- .../Impl/Messaging/Messaging.cs | 6 +- .../Apache.Ignite.Core/Impl/PlatformTarget.cs | 94 ++ .../Impl/Services/Services.cs | 31 +- .../Impl/Transactions/TransactionsImpl.cs | 73 +- .../Impl/Unmanaged/IgniteJniNativeMethods.cs | 261 +--- .../Impl/Unmanaged/UnmanagedCallbacks.cs | 11 +- .../Impl/Unmanaged/UnmanagedUtils.cs | 519 +------ 59 files changed, 2101 insertions(+), 3927 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/ICacheLockInternal.cs diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java index 0cd683d8da87e..22adef88182ca 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java @@ -58,6 +58,16 @@ protected PlatformAbstractTarget(PlatformContext platformCtx) { log = platformCtx.kernalContext().log(PlatformAbstractTarget.class); } + /** {@inheritDoc} */ + @Override public long inLongOutLong(int type, long val) throws Exception { + try { + return processInLongOutLong(type, val); + } + catch (Exception e) { + throw convertException(e); + } + } + /** {@inheritDoc} */ @Override public long inStreamOutLong(int type, long memPtr) throws Exception { try (PlatformMemory mem = platformCtx.memory().get(memPtr)) { @@ -78,8 +88,8 @@ protected PlatformAbstractTarget(PlatformContext platformCtx) { /** {@inheritDoc} */ @Override public Object inStreamOutObject(int type, long memPtr) throws Exception { - try (PlatformMemory mem = platformCtx.memory().get(memPtr)) { - BinaryRawReaderEx reader = platformCtx.reader(mem); + try (PlatformMemory mem = memPtr != 0 ? platformCtx.memory().get(memPtr) : null) { + BinaryRawReaderEx reader = mem != null ? platformCtx.reader(mem) : null; return processInStreamOutObject(type, reader); } @@ -164,6 +174,54 @@ protected PlatformAbstractTarget(PlatformContext platformCtx) { } } + /** {@inheritDoc} */ + @Override public Object inObjectStreamOutObjectStream(int type, Object arg, long inMemPtr, long outMemPtr) + throws Exception { + PlatformMemory inMem = null; + PlatformMemory outMem = null; + + try { + BinaryRawReaderEx reader = null; + + if (inMemPtr != 0) { + inMem = platformCtx.memory().get(inMemPtr); + + reader = platformCtx.reader(inMem); + } + + PlatformOutputStream out = null; + BinaryRawWriterEx writer = null; + + if (outMemPtr != 0) { + outMem = platformCtx.memory().get(outMemPtr); + + out = outMem.output(); + + writer = platformCtx.writer(out); + } + + Object res = processInObjectStreamOutObjectStream(type, arg, reader, writer); + + if (out != null) + out.synchronize(); + + return res; + } + catch (Exception e) { + throw convertException(e); + } + finally { + try { + if (inMem != null) + inMem.close(); + } + finally { + if (outMem != null) + outMem.close(); + } + } + } + /** * Convert caught exception. * @@ -206,7 +264,7 @@ public PlatformContext platformContext() { * When overridden in a derived class, gets future for the current operation. * * @return current future. - * @throws IgniteCheckedException + * @throws IgniteCheckedException If failed. */ protected IgniteInternalFuture currentFuture() throws IgniteCheckedException { throw new IgniteCheckedException("Future listening is not supported in " + getClass()); @@ -222,6 +280,18 @@ protected IgniteInternalFuture currentFuture() throws IgniteCheckedException { return null; } + /** + * Process IN operation. + * + * @param type Type. + * @param val Value. + * @return Result. + * @throws IgniteCheckedException In case of exception. + */ + protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { + return throwUnsupported(type); + } + /** * Process IN operation. * @@ -273,6 +343,20 @@ protected void processInObjectStreamOutStream(int type, @Nullable Object arg, Bi throwUnsupported(type); } + /** + * Process IN-OUT operation. + * + * @param type Type. + * @param arg Argument. + * @param reader Binary reader. + * @param writer Binary writer. + * @throws IgniteCheckedException In case of exception. + */ + protected Object processInObjectStreamOutObjectStream(int type, @Nullable Object arg, BinaryRawReaderEx reader, + BinaryRawWriterEx writer) throws IgniteCheckedException { + return throwUnsupported(type); + } + /** * Process OUT operation. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java index 1ebf7002b602b..40773d0b09bcd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java @@ -26,6 +26,16 @@ */ @SuppressWarnings("UnusedDeclaration") public interface PlatformTarget { + /** + * Operation accepting long value and returning long value. + * + * @param type Operation type. + * @param val Value. + * @return Result. + * @throws Exception If case of failure. + */ + public long inLongOutLong(int type, long val) throws Exception; + /** * Operation accepting memory stream and returning long value. * @@ -67,6 +77,19 @@ public interface PlatformTarget { */ public void inObjectStreamOutStream(int type, @Nullable Object arg, long inMemPtr, long outMemPtr) throws Exception; + /** + * Operation accepting an object and a memory stream and returning result to another memory stream and an object. + * + * @param type Operation type. + * @param arg Argument (optional). + * @param inMemPtr Input memory pointer. + * @param outMemPtr Output memory pointer. + * @return Result. + * @throws Exception In case of failure. + */ + public Object inObjectStreamOutObjectStream(int type, @Nullable Object arg, long inMemPtr, long outMemPtr) + throws Exception; + /** * Operation returning long result. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index 75683a8d0c31f..05945e0f93af8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -28,7 +28,7 @@ import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.SqlQuery; import org.apache.ignite.cache.query.TextQuery; -import org.apache.ignite.configuration.*; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.binary.BinaryRawReaderEx; import org.apache.ignite.internal.binary.BinaryRawWriterEx; @@ -40,6 +40,7 @@ import org.apache.ignite.internal.processors.platform.PlatformContext; import org.apache.ignite.internal.processors.platform.PlatformNativeException; import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery; +import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryProxy; import org.apache.ignite.internal.processors.platform.cache.query.PlatformFieldsQueryCursor; import org.apache.ignite.internal.processors.platform.cache.query.PlatformQueryCursor; import org.apache.ignite.internal.processors.platform.utils.PlatformConfigurationUtils; @@ -70,7 +71,7 @@ /** * Native cache wrapper implementation. */ -@SuppressWarnings({"unchecked", "UnusedDeclaration", "TryFinallyCanBeTryWithResources"}) +@SuppressWarnings({"unchecked", "UnusedDeclaration", "TryFinallyCanBeTryWithResources", "TypeMayBeWeakened"}) public class PlatformCache extends PlatformAbstractTarget { /** */ public static final int OP_CLEAR = 1; @@ -189,6 +190,54 @@ public class PlatformCache extends PlatformAbstractTarget { /** */ public static final int OP_LOAD_ALL = 40; + /** */ + public static final int OP_CLEAR_CACHE = 41; + + /** */ + public static final int OP_WITH_ASYNC = 42; + + /** */ + public static final int OP_REMOVE_ALL2 = 43; + + /** */ + public static final int OP_WITH_KEEP_BINARY = 44; + + /** */ + public static final int OP_WITH_EXPIRY_POLICY = 45; + + /** */ + public static final int OP_WITH_NO_RETRIES = 46; + + /** */ + public static final int OP_WITH_SKIP_STORE = 47; + + /** */ + public static final int OP_SIZE = 48; + + /** */ + public static final int OP_ITERATOR = 49; + + /** */ + public static final int OP_LOC_ITERATOR = 50; + + /** */ + public static final int OP_ENTER_LOCK = 51; + + /** */ + public static final int OP_EXIT_LOCK = 52; + + /** */ + public static final int OP_TRY_ENTER_LOCK = 53; + + /** */ + public static final int OP_CLOSE_LOCK = 54; + + /** */ + public static final int OP_REBALANCE = 55; + + /** */ + public static final int OP_SIZE_LOC = 56; + /** Underlying JCache. */ private final IgniteCacheProxy cache; @@ -224,68 +273,21 @@ public PlatformCache(PlatformContext platformCtx, IgniteCache cache, boolean kee this.keepBinary = keepBinary; } - /** - * Gets cache with "skip-store" flag set. - * - * @return Cache with "skip-store" flag set. - */ - public PlatformCache withSkipStore() { - if (cache.delegate().skipStore()) - return this; - - return new PlatformCache(platformCtx, cache.withSkipStore(), keepBinary); - } - - /** - * Gets cache with "keep binary" flag. - * - * @return Cache with "keep binary" flag set. - */ - public PlatformCache withKeepBinary() { - if (keepBinary) - return this; - - return new PlatformCache(platformCtx, cache.withKeepBinary(), true); - } - - /** - * Gets cache with provided expiry policy. - * - * @param create Create. - * @param update Update. - * @param access Access. - * @return Cache. - */ - public PlatformCache withExpiryPolicy(final long create, final long update, final long access) { - IgniteCache cache0 = cache.withExpiryPolicy(new InteropExpiryPolicy(create, update, access)); - - return new PlatformCache(platformCtx, cache0, keepBinary); - } - - /** - * Gets cache with asynchronous mode enabled. - * - * @return Cache with asynchronous mode enabled. - */ - public PlatformCache withAsync() { - if (cache.isAsync()) - return this; + /** {@inheritDoc} */ + @Override protected long processOutLong(int type) throws IgniteCheckedException { + switch (type) { + case OP_CLEAR_CACHE: + cache.clear(); - return new PlatformCache(platformCtx, (IgniteCache)cache.withAsync(), keepBinary); - } + return TRUE; - /** - * Gets cache with no-retries mode enabled. - * - * @return Cache with no-retries mode enabled. - */ - public PlatformCache withNoRetries() { - CacheOperationContext opCtx = cache.operationContext(); + case OP_REMOVE_ALL2: + cache.removeAll(); - if (opCtx != null && opCtx.noRetries()) - return this; + return TRUE; + } - return new PlatformCache(platformCtx, cache.withNoRetries(), keepBinary); + return super.processOutLong(type); } /** {@inheritDoc} */ @@ -388,6 +390,22 @@ public PlatformCache withNoRetries() { return TRUE; } + case OP_TRY_ENTER_LOCK: { + try { + long id = reader.readLong(); + long timeout = reader.readLong(); + + boolean res = timeout == -1 + ? lock(id).tryLock() + : lock(id).tryLock(timeout, TimeUnit.MILLISECONDS); + + return res ? TRUE : FALSE; + } + catch (InterruptedException e) { + throw new IgniteCheckedException(e); + } + } + default: return super.processInStreamOutLong(type, reader); } @@ -395,6 +413,8 @@ public PlatformCache withNoRetries() { return TRUE; } + + /** * Loads cache via localLoadCache or loadCache. */ @@ -444,7 +464,27 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc) { qry.start(cache, loc, bufSize, timeInterval, autoUnsubscribe, initQry); - return qry; + return new PlatformContinuousQueryProxy(platformCtx, qry); + } + + case OP_WITH_EXPIRY_POLICY: { + long create = reader.readLong(); + long update = reader.readLong(); + long access = reader.readLong(); + + IgniteCache cache0 = cache.withExpiryPolicy(new InteropExpiryPolicy(create, update, access)); + + return new PlatformCache(platformCtx, cache0, keepBinary); + } + + case OP_LOC_ITERATOR: { + int peekModes = reader.readInt(); + + CachePeekMode[] peekModes0 = PlatformUtils.decodeCachePeekModes(peekModes); + + Iterator iter = cache.localEntries(peekModes0).iterator(); + + return new PlatformCacheIterator(platformCtx, iter); } default: @@ -653,6 +693,102 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc) { } } + /** {@inheritDoc} */ + @Override protected Object processOutObject(int type) throws IgniteCheckedException { + switch (type) { + case OP_WITH_ASYNC: { + if (cache.isAsync()) + return this; + + return new PlatformCache(platformCtx, (IgniteCache)cache.withAsync(), keepBinary); + } + + case OP_WITH_KEEP_BINARY: { + if (keepBinary) + return this; + + return new PlatformCache(platformCtx, cache.withKeepBinary(), true); + } + + case OP_WITH_NO_RETRIES: { + CacheOperationContext opCtx = cache.operationContext(); + + if (opCtx != null && opCtx.noRetries()) + return this; + + return new PlatformCache(platformCtx, cache.withNoRetries(), keepBinary); + } + + case OP_WITH_SKIP_STORE: { + if (cache.delegate().skipStore()) + return this; + + return new PlatformCache(platformCtx, cache.withSkipStore(), keepBinary); + } + + case OP_ITERATOR: { + Iterator iter = cache.iterator(); + + return new PlatformCacheIterator(platformCtx, iter); + } + } + + return super.processOutObject(type); + } + + /** {@inheritDoc} */ + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { + switch (type) { + case OP_SIZE: { + CachePeekMode[] modes = PlatformUtils.decodeCachePeekModes((int)val); + + return cache.size(modes); + } + + case OP_SIZE_LOC: { + CachePeekMode[] modes = PlatformUtils.decodeCachePeekModes((int)val); + + return cache.localSize(modes); + } + + case OP_ENTER_LOCK: { + try { + lock(val).lockInterruptibly(); + + return TRUE; + } + catch (InterruptedException e) { + throw new IgniteCheckedException("Failed to enter cache lock.", e); + } + } + + case OP_EXIT_LOCK: { + lock(val).unlock(); + + return TRUE; + } + + case OP_CLOSE_LOCK: { + Lock lock = lockMap.remove(val); + + assert lock != null : "Failed to unregister lock: " + val; + + return TRUE; + } + + case OP_REBALANCE: { + PlatformFutureUtils.listen(platformCtx, cache.rebalance().chain(new C1() { + @Override public Object apply(IgniteFuture fut) { + return null; + } + }), val, PlatformFutureUtils.TYP_OBJ, this); + + return TRUE; + } + } + return super.processInLongOutLong(type, val); + } + /** {@inheritDoc} */ @Override public Exception convertException(Exception e) { if (e instanceof CachePartialUpdateException) @@ -663,7 +799,7 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc) { return new PlatformCachePartialUpdateException((CachePartialUpdateCheckedException)e, platformCtx, keepBinary); if (e.getCause() instanceof EntryProcessorException) - return (EntryProcessorException) e.getCause(); + return (Exception)e.getCause(); return super.convertException(e); } @@ -736,117 +872,6 @@ private static void writeError(BinaryRawWriterEx writer, Exception ex) { return null; } - /** - * Clears the contents of the cache, without notifying listeners or CacheWriters. - * - * @throws IllegalStateException if the cache is closed. - * @throws javax.cache.CacheException if there is a problem during the clear - */ - public void clear() throws IgniteCheckedException { - cache.clear(); - } - - /** - * Removes all entries. - * - * @throws org.apache.ignite.IgniteCheckedException In case of error. - */ - public void removeAll() throws IgniteCheckedException { - cache.removeAll(); - } - - /** - * Read cache size. - * - * @param peekModes Encoded peek modes. - * @param loc Local mode flag. - * @return Size. - */ - public int size(int peekModes, boolean loc) { - CachePeekMode[] modes = PlatformUtils.decodeCachePeekModes(peekModes); - - return loc ? cache.localSize(modes) : cache.size(modes); - } - - /** - * Create cache iterator. - * - * @return Cache iterator. - */ - public PlatformCacheIterator iterator() { - Iterator iter = cache.iterator(); - - return new PlatformCacheIterator(platformCtx, iter); - } - - /** - * Create cache iterator over local entries. - * - * @param peekModes Peke modes. - * @return Cache iterator. - */ - public PlatformCacheIterator localIterator(int peekModes) { - CachePeekMode[] peekModes0 = PlatformUtils.decodeCachePeekModes(peekModes); - - Iterator iter = cache.localEntries(peekModes0).iterator(); - - return new PlatformCacheIterator(platformCtx, iter); - } - - /** - * Enters a lock. - * - * @param id Lock id. - */ - public void enterLock(long id) throws InterruptedException { - lock(id).lockInterruptibly(); - } - - /** - * Exits a lock. - * - * @param id Lock id. - */ - public void exitLock(long id) { - lock(id).unlock(); - } - - /** - * Attempts to enter a lock. - * - * @param id Lock id. - * @param timeout Timeout, in milliseconds. -1 for infinite timeout. - */ - public boolean tryEnterLock(long id, long timeout) throws InterruptedException { - return timeout == -1 - ? lock(id).tryLock() - : lock(id).tryLock(timeout, TimeUnit.MILLISECONDS); - } - - /** - * Rebalances the cache. - * - * @param futId Future id. - */ - public void rebalance(long futId) { - PlatformFutureUtils.listen(platformCtx, cache.rebalance().chain(new C1() { - @Override public Object apply(IgniteFuture fut) { - return null; - } - }), futId, PlatformFutureUtils.TYP_OBJ, this); - } - - /** - * Unregister lock. - * - * @param id Lock id. - */ - public void closeLock(long id){ - Lock lock = lockMap.remove(id); - - assert lock != null : "Failed to unregister lock: " + id; - } - /** * Get lock by id. * @@ -1079,7 +1104,7 @@ private static class InteropExpiryPolicy implements ExpiryPolicy { * @param update Expiry for update. * @param access Expiry for access. */ - public InteropExpiryPolicy(long create, long update, long access) { + private InteropExpiryPolicy(long create, long update, long access) { this.create = convert(create); this.update = convert(update); this.access = convert(access); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java index acd27071ce300..41b58aaaf2c19 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java @@ -81,6 +81,9 @@ public class PlatformAffinity extends PlatformAbstractTarget { /** */ public static final int OP_PRIMARY_PARTITIONS = 14; + /** */ + public static final int OP_PARTITIONS = 15; + /** */ private static final C1 TO_NODE_ID = new C1() { @Nullable @Override public UUID apply(ClusterNode node) { @@ -288,10 +291,11 @@ public PlatformAffinity(PlatformContext platformCtx, GridKernalContext igniteCtx } } - /** - * @return Gets number of partitions in cache. - */ - public int partitions() { - return aff.partitions(); + /** {@inheritDoc} */ + @Override public long outLong(int type) throws Exception { + if (type == OP_PARTITIONS) + return aff.partitions(); + + return super.outLong(type); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java index ab52b52d1de4b..ff28b818066b9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java @@ -38,6 +38,15 @@ public abstract class PlatformAbstractQueryCursor extends PlatformAbstractTar /** Get single entry. */ private static final int OP_GET_SINGLE = 3; + /** Start iterating. */ + private static final int OP_ITERATOR = 4; + + /** Close iterator. */ + private static final int OP_ITERATOR_CLOSE = 5; + + /** Close iterator. */ + private static final int OP_ITERATOR_HAS_NEXT = 6; + /** Underlying cursor. */ private final QueryCursorEx cursor; @@ -126,23 +135,26 @@ public PlatformAbstractQueryCursor(PlatformContext platformCtx, QueryCursorEx } } - /** - * Get cursor iterator. - */ - public void iterator() { - iter = cursor.iterator(); - } + /** {@inheritDoc} */ + @Override protected long processOutLong(int type) throws IgniteCheckedException { + switch (type) { + case OP_ITERATOR: + iter = cursor.iterator(); - /** - * Check whether next iterator entry exists. - * - * @return {@code True} if exists. - */ - @SuppressWarnings("UnusedDeclaration") - public boolean iteratorHasNext() { - assert iter != null : "iterator() has not been called"; + return TRUE; + + case OP_ITERATOR_CLOSE: + cursor.close(); + + return TRUE; + + case OP_ITERATOR_HAS_NEXT: + assert iter != null : "iterator() has not been called"; + + return iter.hasNext() ? TRUE : FALSE; + } - return iter.hasNext(); + return super.processOutLong(type); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java new file mode 100644 index 0000000000000..a4d7cad5abb39 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java @@ -0,0 +1,53 @@ +/* + * 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.platform.cache.query; + +import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; +import org.apache.ignite.internal.processors.platform.PlatformContext; + +/** + * Proxy that implements PlatformTarget. + */ +public class PlatformContinuousQueryProxy extends PlatformAbstractTarget { + private final PlatformContinuousQuery qry; + + /** + * Constructor. + * + * @param platformCtx Context. + */ + public PlatformContinuousQueryProxy(PlatformContext platformCtx, PlatformContinuousQuery qry) { + super(platformCtx); + + assert qry != null; + + this.qry = qry; + } + + /** {@inheritDoc} */ + @Override public Object outObject(int type) throws Exception { + return qry.getInitialQueryCursor(); + } + + /** {@inheritDoc} */ + @Override public long outLong(int type) throws Exception { + qry.close(); + + return 0; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java deleted file mode 100644 index ad0d08112d92b..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java +++ /dev/null @@ -1,61 +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.platform.cache.store; - -import org.apache.ignite.internal.binary.BinaryRawReaderEx; -import org.apache.ignite.internal.processors.platform.PlatformContext; -import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; - -/** - * Platform cache store callback. - */ -public abstract class PlatformCacheStoreCallback { - /** Context. */ - protected final PlatformContext ctx; - - /** - * Constructor. - * - * @param ctx Context. - */ - protected PlatformCacheStoreCallback(PlatformContext ctx) { - this.ctx = ctx; - } - - /** - * Invoke the callback. - * - * @param memPtr Memory pointer. - */ - public void invoke(long memPtr) { - if (memPtr > 0) { - try (PlatformMemory mem = ctx.memory().get(memPtr)) { - BinaryRawReaderEx reader = ctx.reader(mem); - - invoke0(reader); - } - } - } - - /** - * Internal invoke routine. - * - * @param reader Reader. - */ - protected abstract void invoke0(BinaryRawReaderEx reader); -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java index ac1416c10f47f..0017b2d12aa6e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java @@ -71,14 +71,13 @@ public long cacheStoreCreate(long memPtr) { /** * @param objPtr Object pointer. * @param memPtr Memory pointer. - * @param cb Callback. * @return Result. */ - public int cacheStoreInvoke(long objPtr, long memPtr, Object cb) { + public int cacheStoreInvoke(long objPtr, long memPtr) { enter(); try { - return PlatformCallbackUtils.cacheStoreInvoke(envPtr, objPtr, memPtr, cb); + return PlatformCallbackUtils.cacheStoreInvoke(envPtr, objPtr, memPtr); } finally { leave(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java index 7b36e5e271db8..174b0144cf056 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java @@ -37,10 +37,9 @@ public class PlatformCallbackUtils { * @param envPtr Environment pointer. * @param objPtr Object pointer. * @param memPtr Memory pointer. - * @param cb Callback. * @return Result. */ - static native int cacheStoreInvoke(long envPtr, long objPtr, long memPtr, Object cb); + static native int cacheStoreInvoke(long envPtr, long objPtr, long memPtr); /** * @param envPtr Environment pointer. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java index d80079c0527e2..a94e04542516e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; +import org.jetbrains.annotations.Nullable; /** * Interop projection. @@ -80,6 +81,27 @@ public class PlatformClusterGroup extends PlatformAbstractTarget { /** */ private static final int OP_SCHEMA = 15; + /** */ + private static final int OP_FOR_OTHERS = 16; + + /** */ + private static final int OP_FOR_REMOTES = 17; + + /** */ + private static final int OP_FOR_DAEMONS = 18; + + /** */ + private static final int OP_FOR_RANDOM = 19; + + /** */ + private static final int OP_FOR_OLDEST = 20; + + /** */ + private static final int OP_FOR_YOUNGEST = 21; + + /** */ + private static final int OP_RESET_METRICS = 22; + /** Projection. */ private final ClusterGroupEx prj; @@ -260,47 +282,58 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { } } - /** - * @param exclude Projection to exclude. - * @return New projection. - */ - public PlatformClusterGroup forOthers(PlatformClusterGroup exclude) { - return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forOthers(exclude.prj)); - } + /** {@inheritDoc} */ + @Override protected Object processInObjectStreamOutObjectStream( + int type, @Nullable Object arg, BinaryRawReaderEx reader, BinaryRawWriterEx writer) + throws IgniteCheckedException { + switch (type) { + case OP_FOR_OTHERS: { + PlatformClusterGroup exclude = (PlatformClusterGroup) arg; - /** - * @return New projection. - */ - public PlatformClusterGroup forRemotes() { - return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forRemotes()); - } + assert exclude != null; - /** - * @return New projection. - */ - public PlatformClusterGroup forDaemons() { - return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forDaemons()); - } + return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forOthers(exclude.prj)); + } + } - /** - * @return New projection. - */ - public PlatformClusterGroup forRandom() { - return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forRandom()); + return super.processInObjectStreamOutObjectStream(type, arg, reader, writer); } - /** - * @return New projection. - */ - public PlatformClusterGroup forOldest() { - return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forOldest()); + /** {@inheritDoc} */ + @Override protected Object processOutObject(int type) throws IgniteCheckedException { + switch (type) { + case OP_FOR_REMOTES: + return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forRemotes()); + + case OP_FOR_DAEMONS: + return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forDaemons()); + + case OP_FOR_RANDOM: + return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forRandom()); + + case OP_FOR_OLDEST: + return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forOldest()); + + case OP_FOR_YOUNGEST: + return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forYoungest()); + } + + return super.processOutObject(type); } - /** - * @return New projection. - */ - public PlatformClusterGroup forYoungest() { - return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forYoungest()); + /** {@inheritDoc} */ + @Override protected long processOutLong(int type) throws IgniteCheckedException { + switch (type) { + case OP_RESET_METRICS: { + assert prj instanceof IgniteCluster; // Can only be invoked on top-level cluster group. + + ((IgniteCluster)prj).resetMetrics(); + + return TRUE; + } + } + + return super.processOutLong(type); } /** @@ -310,15 +343,6 @@ public ClusterGroupEx projection() { return prj; } - /** - * Resets local I/O, job, and task execution metrics. - */ - public void resetMetrics() { - assert prj instanceof IgniteCluster; // Can only be invoked on top-level cluster group. - - ((IgniteCluster)prj).resetMetrics(); - } - /** * Pings a remote node. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java index 9eb746c4cfa7d..36d709a3f1356 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java @@ -31,7 +31,6 @@ import org.apache.ignite.internal.processors.platform.PlatformContext; import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils; import org.apache.ignite.internal.processors.platform.utils.PlatformListenable; -import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; import org.apache.ignite.internal.util.future.IgniteFutureImpl; import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteInClosure; @@ -64,6 +63,15 @@ public class PlatformCompute extends PlatformAbstractTarget { /** */ private static final int OP_UNICAST = 5; + /** */ + private static final int OP_WITH_NO_FAILOVER = 6; + + /** */ + private static final int OP_WITH_TIMEOUT = 7; + + /** */ + private static final int OP_EXEC_NATIVE = 8; + /** Compute instance. */ private final IgniteComputeImpl compute; @@ -104,11 +112,48 @@ public PlatformCompute(PlatformContext platformCtx, ClusterGroup grp, String pla case OP_AFFINITY: return processClosures(reader.readLong(), reader, false, true); + case OP_EXEC_NATIVE: { + long taskPtr = reader.readLong(); + long topVer = reader.readLong(); + + final PlatformFullTask task = new PlatformFullTask(platformCtx, computeForPlatform, taskPtr, topVer); + + return executeNative0(task); + } + default: return super.processInStreamOutObject(type, reader); } } + /** {@inheritDoc} */ + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { + switch (type) { + case OP_WITH_TIMEOUT: { + compute.withTimeout(val); + computeForPlatform.withTimeout(val); + + return TRUE; + } + } + + return super.processInLongOutLong(type, val); + } + + /** {@inheritDoc} */ + @Override protected long processOutLong(int type) throws IgniteCheckedException { + switch (type) { + case OP_WITH_NO_FAILOVER: { + compute.withNoFailover(); + computeForPlatform.withNoFailover(); + + return TRUE; + } + } + + return super.processOutLong(type); + } + /** * Process closure execution request. * @param taskPtr Task pointer. @@ -200,36 +245,6 @@ private PlatformJob nextClosureJob(PlatformAbstractTask task, BinaryRawReaderEx } } - /** - * Execute native full-fledged task. - * - * @param taskPtr Pointer to the task. - * @param topVer Topology version. - */ - public PlatformListenable executeNative(long taskPtr, long topVer) { - final PlatformFullTask task = new PlatformFullTask(platformCtx, computeForPlatform, taskPtr, topVer); - - return executeNative0(task); - } - - /** - * Set "withTimeout" state. - * - * @param timeout Timeout (milliseconds). - */ - public void withTimeout(long timeout) { - compute.withTimeout(timeout); - computeForPlatform.withTimeout(timeout); - } - - /** - * Set "withNoFailover" state. - */ - public void withNoFailover() { - compute.withNoFailover(); - computeForPlatform.withNoFailover(); - } - /** */ @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException { IgniteInternalFuture fut = curFut.get(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java index 78d5d86047782..2822b7f9125b6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java @@ -59,6 +59,33 @@ public class PlatformDataStreamer extends PlatformAbstractTarget { /** */ private static final int OP_RECEIVER = 2; + /** */ + private static final int OP_ALLOW_OVERWRITE = 3; + + /** */ + private static final int OP_SET_ALLOW_OVERWRITE = 4; + + /** */ + private static final int OP_SKIP_STORE = 5; + + /** */ + private static final int OP_SET_SKIP_STORE = 6; + + /** */ + private static final int OP_PER_NODE_BUFFER_SIZE = 7; + + /** */ + private static final int OP_SET_PER_NODE_BUFFER_SIZE = 8; + + /** */ + private static final int OP_PER_NODE_PARALLEL_OPS = 9; + + /** */ + private static final int OP_SET_PER_NODE_PARALLEL_OPS = 10; + + /** */ + private static final int OP_LISTEN_TOPOLOGY = 11; + /** Cache name. */ private final String cacheName; @@ -126,7 +153,7 @@ else if (plc == PLC_FLUSH) return TRUE; - case OP_RECEIVER: + case OP_RECEIVER: { long ptr = reader.readLong(); Object rec = reader.readObjectDetached(); @@ -134,39 +161,84 @@ else if (plc == PLC_FLUSH) ldr.receiver(platformCtx.createStreamReceiver(rec, ptr, keepBinary)); return TRUE; + } default: return super.processInStreamOutLong(type, reader); } } - /** - * Listen topology changes. - * - * @param ptr Pointer. - */ - public void listenTopology(final long ptr) { - lsnr = new GridLocalEventListener() { - @Override public void onEvent(Event evt) { - DiscoveryEvent discoEvt = (DiscoveryEvent)evt; + /** {@inheritDoc} */ + @Override protected long processInLongOutLong(int type, final long val) throws IgniteCheckedException { + switch (type) { + case OP_SET_ALLOW_OVERWRITE: + ldr.allowOverwrite(val == TRUE); + + return TRUE; + + case OP_SET_PER_NODE_BUFFER_SIZE: + ldr.perNodeBufferSize((int) val); + + return TRUE; + + case OP_SET_SKIP_STORE: + ldr.skipStore(val == TRUE); + + return TRUE; - long topVer = discoEvt.topologyVersion(); - int topSize = platformCtx.kernalContext().discovery().cacheNodes( - cacheName, new AffinityTopologyVersion(topVer)).size(); + case OP_SET_PER_NODE_PARALLEL_OPS: + ldr.perNodeParallelOperations((int) val); + + return TRUE; + + case OP_LISTEN_TOPOLOGY: { + lsnr = new GridLocalEventListener() { + @Override public void onEvent(Event evt) { + DiscoveryEvent discoEvt = (DiscoveryEvent)evt; + + long topVer = discoEvt.topologyVersion(); + int topSize = platformCtx.kernalContext().discovery().cacheNodes( + cacheName, new AffinityTopologyVersion(topVer)).size(); + + platformCtx.gateway().dataStreamerTopologyUpdate(val, topVer, topSize); + } + }; - platformCtx.gateway().dataStreamerTopologyUpdate(ptr, topVer, topSize); + platformCtx.kernalContext().event().addLocalEventListener(lsnr, + EVT_NODE_JOINED, EVT_NODE_FAILED, EVT_NODE_LEFT); + + GridDiscoveryManager discoMgr = platformCtx.kernalContext().discovery(); + + AffinityTopologyVersion topVer = discoMgr.topologyVersionEx(); + + int topSize = discoMgr.cacheNodes(cacheName, topVer).size(); + + platformCtx.gateway().dataStreamerTopologyUpdate(val, topVer.topologyVersion(), topSize); + + return TRUE; } - }; + } + + return super.processInLongOutLong(type, val); + } - platformCtx.kernalContext().event().addLocalEventListener(lsnr, EVT_NODE_JOINED, EVT_NODE_FAILED, EVT_NODE_LEFT); + /** {@inheritDoc} */ + @Override public long processOutLong(int type) throws IgniteCheckedException { + switch (type) { + case OP_ALLOW_OVERWRITE: + return ldr.allowOverwrite() ? TRUE : FALSE; - GridDiscoveryManager discoMgr = platformCtx.kernalContext().discovery(); + case OP_PER_NODE_BUFFER_SIZE: + return ldr.perNodeBufferSize(); - AffinityTopologyVersion topVer = discoMgr.topologyVersionEx(); + case OP_SKIP_STORE: + return ldr.skipStore() ? TRUE : FALSE; - int topSize = discoMgr.cacheNodes(cacheName, topVer).size(); + case OP_PER_NODE_PARALLEL_OPS: + return ldr.perNodeParallelOperations(); + } - platformCtx.gateway().dataStreamerTopologyUpdate(ptr, topVer.topologyVersion(), topSize); + return super.processOutLong(type); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java index 4a5b2e5b6c096..53319560f95c1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java @@ -17,6 +17,8 @@ package org.apache.ignite.internal.processors.platform.datastructures; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.binary.BinaryRawReaderEx; import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicLongImpl; import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; @@ -28,6 +30,42 @@ public class PlatformAtomicLong extends PlatformAbstractTarget { /** */ private final GridCacheAtomicLongImpl atomicLong; + /** */ + private static final int OP_ADD_AND_GET = 1; + + /** */ + private static final int OP_CLOSE = 2; + + /** */ + private static final int OP_COMPARE_AND_SET = 3; + + /** */ + private static final int OP_COMPARE_AND_SET_AND_GET = 4; + + /** */ + private static final int OP_DECREMENT_AND_GET = 5; + + /** */ + private static final int OP_GET = 6; + + /** */ + private static final int OP_GET_AND_ADD = 7; + + /** */ + private static final int OP_GET_AND_DECREMENT = 8; + + /** */ + private static final int OP_GET_AND_INCREMENT = 9; + + /** */ + private static final int OP_GET_AND_SET = 10; + + /** */ + private static final int OP_INCREMENT_AND_GET = 11; + + /** */ + private static final int OP_IS_CLOSED = 12; + /** * Ctor. * @param ctx Context. @@ -41,109 +79,68 @@ public PlatformAtomicLong(PlatformContext ctx, GridCacheAtomicLongImpl atomicLon this.atomicLong = atomicLong; } - /** - * Reads the value. - * - * @return Current atomic long value. - */ - public long get() { - return atomicLong.get(); - } + /** {@inheritDoc} */ + @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException { + switch (type) { + case OP_COMPARE_AND_SET: + long cmp = reader.readLong(); + long val = reader.readLong(); - /** - * Increments the value. - * - * @return Current atomic long value. - */ - public long incrementAndGet() { - return atomicLong.incrementAndGet(); - } + return atomicLong.compareAndSet(cmp, val) ? TRUE : FALSE; - /** - * Increments the value. - * - * @return Original atomic long value. - */ - public long getAndIncrement() { - return atomicLong.getAndIncrement(); - } + case OP_COMPARE_AND_SET_AND_GET: + long expVal = reader.readLong(); + long newVal = reader.readLong(); - /** - * Adds a value. - * - * @return Current atomic long value. - */ - public long addAndGet(long val) { - return atomicLong.addAndGet(val); - } + return atomicLong.compareAndSetAndGet(expVal, newVal); + } - /** - * Adds a value. - * - * @return Original atomic long value. - */ - public long getAndAdd(long val) { - return atomicLong.getAndAdd(val); + return super.processInStreamOutLong(type, reader); } - /** - * Decrements the value. - * - * @return Current atomic long value. - */ - public long decrementAndGet() { - return atomicLong.decrementAndGet(); - } + /** {@inheritDoc} */ + @Override protected long processOutLong(int type) throws IgniteCheckedException { + switch (type) { + case OP_CLOSE: + atomicLong.close(); - /** - * Decrements the value. - * - * @return Original atomic long value. - */ - public long getAndDecrement() { - return atomicLong.getAndDecrement(); - } + return TRUE; - /** - * Gets current value of atomic long and sets new value - * - * @return Original atomic long value. - */ - public long getAndSet(long val) { - return atomicLong.getAndSet(val); - } + case OP_DECREMENT_AND_GET: + return atomicLong.decrementAndGet(); - /** - * Compares two values for equality and, if they are equal, replaces the first value. - * - * @return Original atomic long value. - */ - public long compareAndSetAndGet(long expVal, long newVal) { - return atomicLong.compareAndSetAndGet(expVal, newVal); - } + case OP_GET: + return atomicLong.get(); - /** - * Compares two values for equality and, if they are equal, replaces the first value. - * - * @return Original atomic long value. - */ - public boolean compareAndSet(long cmp, long val) { - return atomicLong.compareAndSet(cmp, val); - } + case OP_GET_AND_DECREMENT: + return atomicLong.getAndDecrement(); - /** - * Gets status of atomic. - * - * @return {@code true} if atomic was removed from cache, {@code false} in other case. - */ - public boolean isClosed() { - return atomicLong.removed(); + case OP_GET_AND_INCREMENT: + return atomicLong.getAndIncrement(); + + case OP_INCREMENT_AND_GET: + return atomicLong.incrementAndGet(); + + case OP_IS_CLOSED: + return atomicLong.removed() ? TRUE : FALSE; + } + + return super.processOutLong(type); } - /** - * Removes this atomic long. - */ - public void close() { - atomicLong.close(); + /** {@inheritDoc} */ + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { + switch (type) { + case OP_ADD_AND_GET: + return atomicLong.addAndGet(val); + + case OP_GET_AND_ADD: + return atomicLong.getAndAdd(val); + + case OP_GET_AND_SET: + return atomicLong.getAndSet(val); + } + + return super.processInLongOutLong(type, val); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java index 50801398dc7fd..e5fc08d8b69e4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java @@ -24,6 +24,7 @@ import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; +import org.omg.CORBA.TRANSACTION_REQUIRED; /** * Platform atomic reference wrapper. @@ -39,6 +40,12 @@ public class PlatformAtomicReference extends PlatformAbstractTarget { /** */ private static final int OP_COMPARE_AND_SET_AND_GET = 3; + /** */ + private static final int OP_CLOSE = 4; + + /** */ + private static final int OP_IS_CLOSED = 5; + /** */ private final GridCacheAtomicReferenceImpl atomicRef; @@ -87,22 +94,6 @@ private PlatformAtomicReference(PlatformContext ctx, GridCacheAtomicReferenceImp atomicRef = ref; } - /** - * Returns a value indicating whether this instance has been closed. - * - * @return Value indicating whether this instance has been closed. - */ - public boolean isClosed() { - return atomicRef.removed(); - } - - /** - * Closes this instance. - */ - public void close() { - atomicRef.close(); - } - /** {@inheritDoc} */ @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException { if (type == OP_GET) @@ -142,5 +133,20 @@ public void close() { else super.processInStreamOutStream(type, reader, writer); } + + /** {@inheritDoc} */ + @Override protected long processOutLong(int type) throws IgniteCheckedException { + switch (type) { + case OP_CLOSE: + atomicRef.close(); + + return TRUE; + + case OP_IS_CLOSED: + return atomicRef.removed() ? TRUE : FALSE; + } + + return super.processOutLong(type); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java index ce7e364550b94..ec946cac303e1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.platform.datastructures; import org.apache.ignite.IgniteAtomicSequence; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; @@ -28,6 +29,33 @@ public class PlatformAtomicSequence extends PlatformAbstractTarget { /** */ private final IgniteAtomicSequence atomicSeq; + /** */ + private static final int OP_ADD_AND_GET = 1; + + /** */ + private static final int OP_CLOSE = 2; + + /** */ + private static final int OP_GET = 3; + + /** */ + private static final int OP_GET_AND_ADD = 4; + + /** */ + private static final int OP_GET_AND_INCREMENT = 5; + + /** */ + private static final int OP_GET_BATCH_SIZE = 6; + + /** */ + private static final int OP_INCREMENT_AND_GET = 7; + + /** */ + private static final int OP_IS_CLOSED = 8; + + /** */ + private static final int OP_SET_BATCH_SIZE = 9; + /** * Ctor. * @param ctx Context. @@ -41,82 +69,49 @@ public PlatformAtomicSequence(PlatformContext ctx, IgniteAtomicSequence atomicSe this.atomicSeq = atomicSeq; } - /** - * Reads the value. - * - * @return Current atomic sequence value. - */ - public long get() { - return atomicSeq.get(); - } - /** - * Increments and reads the value. - * - * @return Current atomic sequence value. - */ - public long incrementAndGet() { - return atomicSeq.incrementAndGet(); - } + /** {@inheritDoc} */ + @Override protected long processOutLong(int type) throws IgniteCheckedException { + switch (type) { + case OP_CLOSE: + atomicSeq.close(); - /** - * Reads and increments the value. - * - * @return Original atomic sequence value. - */ - public long getAndIncrement() { - return atomicSeq.getAndIncrement(); - } + return TRUE; - /** - * Adds a value. - * - * @return Current atomic sequence value. - */ - public long addAndGet(long l) { - return atomicSeq.addAndGet(l); - } + case OP_GET: + return atomicSeq.get(); - /** - * Adds a value. - * - * @return Original atomic sequence value. - */ - public long getAndAdd(long l) { - return atomicSeq.getAndAdd(l); - } + case OP_GET_AND_INCREMENT: + return atomicSeq.getAndIncrement(); - /** - * Gets the batch size. - * - * @return Batch size. - */ - public int getBatchSize() { - return atomicSeq.batchSize(); - } + case OP_INCREMENT_AND_GET: + return atomicSeq.incrementAndGet(); - /** - * Sets the batch size. - * - * @param size Batch size. - */ - public void setBatchSize(int size) { - atomicSeq.batchSize(size); - } + case OP_IS_CLOSED: + return atomicSeq.removed() ? TRUE : FALSE; - /** - * Gets status of atomic. - * - * @return {@code true} if atomic was removed from cache, {@code false} in other case. - */ - public boolean isClosed() { - return atomicSeq.removed(); + case OP_GET_BATCH_SIZE: + return atomicSeq.batchSize(); + } + + return super.processOutLong(type); } - /** - * Removes this atomic. - */ - public void close() { - atomicSeq.close(); + /** {@inheritDoc} */ + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { + switch (type) { + case OP_ADD_AND_GET: + return atomicSeq.addAndGet(val); + + case OP_GET_AND_ADD: + return atomicSeq.getAndAdd(val); + + case OP_SET_BATCH_SIZE: + atomicSeq.batchSize((int)val); + + return TRUE; + } + + return super.processInLongOutLong(type, val); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java index 45d92081c4fe2..68b1d22b50776 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java @@ -25,7 +25,6 @@ import org.apache.ignite.internal.binary.BinaryRawWriterEx; import org.apache.ignite.internal.processors.platform.PlatformContext; import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheStore; -import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheStoreCallback; import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream; import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; @@ -34,6 +33,7 @@ import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiInClosure; +import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.resources.CacheStoreSessionResource; import org.jetbrains.annotations.Nullable; @@ -168,7 +168,11 @@ public void setProperties(Map props) { writer.writeString(ses.cacheName()); writer.writeObject(key); } - }, new LoadCallback<>(platformCtx, val)); + }, new IgniteInClosureX() { + @Override public void applyx(BinaryRawReaderEx reader) { + val.set((V)reader.readObjectDetached()); + } + }); return val.get(); } @@ -182,14 +186,23 @@ public void setProperties(Map props) { try { final Map loaded = new HashMap<>(); + final Collection keys0 = (Collection)keys; + doInvoke(new IgniteInClosureX() { @Override public void applyx(BinaryRawWriterEx writer) throws IgniteCheckedException { writer.writeByte(OP_LOAD_ALL); writer.writeLong(session()); writer.writeString(ses.cacheName()); - writer.writeCollection((Collection)keys); + writer.writeCollection(keys0); + } + }, new IgniteInClosureX() { + @Override public void applyx(BinaryRawReaderEx reader) { + int cnt = reader.readInt(); + + for (int i = 0; i < cnt; i++) + loaded.put((K) reader.readObjectDetached(), (V) reader.readObjectDetached()); } - }, new LoadAllCallback<>(platformCtx, loaded)); + }); return loaded; } @@ -208,7 +221,14 @@ public void setProperties(Map props) { writer.writeString(ses.cacheName()); writer.writeObjectArray(args); } - }, new LoadCacheCallback<>(platformCtx, clo)); + }, new IgniteInClosureX() { + @Override public void applyx(BinaryRawReaderEx reader) { + int cnt = reader.readInt(); + + for (int i = 0; i < cnt; i++) + clo.apply((K) reader.readObjectDetached(), (V) reader.readObjectDetached()); + } + }); } catch (IgniteCheckedException e) { throw new CacheLoaderException(e); @@ -374,11 +394,11 @@ private long session() throws IgniteCheckedException { * Perform actual invoke. * * @param task Task. - * @param cb Optional callback. + * @param readClo Reader. * @return Result. * @throws org.apache.ignite.IgniteCheckedException If failed. */ - protected int doInvoke(IgniteInClosureX task, @Nullable PlatformCacheStoreCallback cb) + protected int doInvoke(IgniteInClosure task, IgniteInClosure readClo) throws IgniteCheckedException{ try (PlatformMemory mem = platformCtx.memory().allocate()) { PlatformOutputStream out = mem.output(); @@ -389,7 +409,15 @@ protected int doInvoke(IgniteInClosureX task, @Nullable Platf out.synchronize(); - return platformCtx.gateway().cacheStoreInvoke(ptr, mem.pointer(), cb); + int res = platformCtx.gateway().cacheStoreInvoke(ptr, mem.pointer()); + + if (readClo != null) { + BinaryRawReaderEx reader = platformCtx.reader(mem); + + readClo.apply(reader); + } + + return res; } } @@ -403,82 +431,4 @@ public void destroy(GridKernalContext ctx) { platformCtx.gateway().cacheStoreDestroy(ptr); } - - /** - * Load callback. - */ - private static class LoadCallback extends PlatformCacheStoreCallback { - /** Value. */ - private final GridTuple val; - - /** - * Constructor. - * - * @param ctx Context. - * @param val Value. - */ - public LoadCallback(PlatformContext ctx, GridTuple val) { - super(ctx); - - this.val = val; - } - - /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override protected void invoke0(BinaryRawReaderEx reader) { - val.set((V)reader.readObjectDetached()); - } - } - - /** - * Load callback. - */ - private static class LoadAllCallback extends PlatformCacheStoreCallback { - /** Value. */ - private final Map loaded; - - /** - * Constructor. - * - * @param ctx Context. - * @param loaded Map with loaded values. - */ - public LoadAllCallback(PlatformContext ctx, Map loaded) { - super(ctx); - - this.loaded = loaded; - } - - /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override protected void invoke0(BinaryRawReaderEx reader) { - loaded.put((K) reader.readObjectDetached(), (V) reader.readObjectDetached()); - } - } - - /** - * Load callback. - */ - private static class LoadCacheCallback extends PlatformCacheStoreCallback { - /** Value. */ - private final IgniteBiInClosure clo; - - /** - * Constructor. - * - * @param ctx Context. - * @param clo Closure. - */ - public LoadCacheCallback(PlatformContext ctx, IgniteBiInClosure clo) { - super(ctx); - - this.clo = clo; - } - - /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - @Override protected void invoke0(BinaryRawReaderEx reader) { - clo.apply((K) reader.readObjectDetached(), (V) reader.readObjectDetached()); - } - } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java index 71708af559c1b..f1335247999cf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java @@ -68,6 +68,18 @@ public class PlatformEvents extends PlatformAbstractTarget { /** */ private static final int OP_GET_ENABLED_EVENTS = 10; + /** */ + private static final int OP_WITH_ASYNC = 11; + + /** */ + private static final int OP_IS_ENABLED = 12; + + /** */ + private static final int OP_LOCAL_LISTEN = 13; + + /** */ + private static final int OP_STOP_LOCAL_LISTEN = 14; + /** */ private final IgniteEvents events; @@ -94,50 +106,6 @@ public PlatformEvents(PlatformContext platformCtx, IgniteEvents events) { eventColResWriter = new EventCollectionResultWriter(platformCtx); } - /** - * Gets events with asynchronous mode enabled. - * - * @return Events with asynchronous mode enabled. - */ - public PlatformEvents withAsync() { - if (events.isAsync()) - return this; - - return new PlatformEvents(platformCtx, events.withAsync()); - } - - /** - * Adds an event listener for local events. - * - * @param hnd Interop listener handle. - * @param type Event type. - */ - @SuppressWarnings({"unchecked"}) - public void localListen(long hnd, int type) { - events.localListen(localFilter(hnd), type); - } - - /** - * Removes an event listener for local events. - * - * @param hnd Interop listener handle. - */ - @SuppressWarnings({"UnusedDeclaration", "unchecked"}) - public boolean stopLocalListen(long hnd) { - return events.stopLocalListen(localFilter(hnd)); - } - - /** - * Check if event is enabled. - * - * @param type Event type. - * @return {@code True} if event of passed in type is enabled. - */ - @SuppressWarnings("UnusedDeclaration") - public boolean isEnabled(int type) { - return events.isEnabled(type); - } - /** {@inheritDoc} */ @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException { @@ -163,6 +131,11 @@ public boolean isEnabled(int type) { return TRUE; + case OP_LOCAL_LISTEN: + events.localListen(localFilter(reader.readLong()), reader.readInt()); + + return TRUE; + default: return super.processInStreamOutLong(type, reader); } @@ -270,12 +243,38 @@ public boolean isEnabled(int type) { } } - /** */ + /** {@inheritDoc} */ + @Override protected Object processOutObject(int type) throws IgniteCheckedException { + switch (type) { + case OP_WITH_ASYNC: + if (events.isAsync()) + return this; + + return new PlatformEvents(platformCtx, events.withAsync()); + } + + return super.processOutObject(type); + } + + /** {@inheritDoc} */ + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { + switch (type) { + case OP_IS_ENABLED: + return events.isEnabled((int)val) ? TRUE : FALSE; + + case OP_STOP_LOCAL_LISTEN: + return events.stopLocalListen(localFilter(val)) ? TRUE : FALSE; + } + + return super.processInLongOutLong(type, val); + } + + /** {@inheritDoc} */ @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException { return ((IgniteFutureImpl)events.future()).internalFuture(); } - /** */ + /** {@inheritDoc} */ @Nullable @Override protected PlatformFutureUtils.Writer futureWriter(int opId) { switch (opId) { case OP_WAIT_FOR_LOCAL: diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java index 619fea771fed3..1b05ecaa68ca6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java @@ -55,6 +55,9 @@ public class PlatformMessaging extends PlatformAbstractTarget { /** */ public static final int OP_STOP_REMOTE_LISTEN = 7; + /** */ + public static final int OP_WITH_ASYNC = 8; + /** */ private final IgniteMessaging messaging; @@ -72,18 +75,6 @@ public PlatformMessaging(PlatformContext platformCtx, IgniteMessaging messaging) this.messaging = messaging; } - /** - * Gets messaging with asynchronous mode enabled. - * - * @return Messaging with asynchronous mode enabled. - */ - public PlatformMessaging withAsync() { - if (messaging.isAsync()) - return this; - - return new PlatformMessaging (platformCtx, messaging.withAsync()); - } - /** {@inheritDoc} */ @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException { @@ -160,8 +151,21 @@ public PlatformMessaging withAsync() { } } - /** */ + /** {@inheritDoc} */ @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException { return ((IgniteFutureImpl)messaging.future()).internalFuture(); } + + /** {@inheritDoc} */ + @Override protected Object processOutObject(int type) throws IgniteCheckedException { + switch (type) { + case OP_WITH_ASYNC: + if (messaging.isAsync()) + return this; + + return new PlatformMessaging (platformCtx, messaging.withAsync()); + } + + return super.processOutObject(type); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java index f35574126a190..7aaf5970a9118 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java @@ -66,6 +66,21 @@ public class PlatformServices extends PlatformAbstractTarget { /** */ private static final int OP_DESCRIPTORS = 5; + /** */ + private static final int OP_WITH_ASYNC = 6; + + /** */ + private static final int OP_WITH_SERVER_KEEP_BINARY = 7; + + /** */ + private static final int OP_SERVICE_PROXY = 8; + + /** */ + private static final int OP_CANCEL = 9; + + /** */ + private static final int OP_CANCEL_ALL = 10; + /** */ private static final byte PLATFORM_JAVA = 0; @@ -98,63 +113,6 @@ public PlatformServices(PlatformContext platformCtx, IgniteServices services, bo this.srvKeepBinary = srvKeepBinary; } - /** - * Gets services with asynchronous mode enabled. - * - * @return Services with asynchronous mode enabled. - */ - public PlatformServices withAsync() { - if (services.isAsync()) - return this; - - return new PlatformServices(platformCtx, services.withAsync(), srvKeepBinary); - } - - /** - * Gets services with server "keep binary" mode enabled. - * - * @return Services with server "keep binary" mode enabled. - */ - public PlatformServices withServerKeepBinary() { - return srvKeepBinary ? this : new PlatformServices(platformCtx, services, true); - } - - /** - * Cancels service deployment. - * - * @param name Name of service to cancel. - */ - public void cancel(String name) { - services.cancel(name); - } - - /** - * Cancels all deployed services. - */ - public void cancelAll() { - services.cancelAll(); - } - - /** - * Gets a remote handle on the service. - * - * @param name Service name. - * @param sticky Whether or not Ignite should always contact the same remote service. - * @return Either proxy over remote service or local service if it is deployed locally. - */ - public Object serviceProxy(String name, boolean sticky) { - ServiceDescriptor d = findDescriptor(name); - - if (d == null) - throw new IgniteException("Failed to find deployed service: " + name); - - Object proxy = PlatformService.class.isAssignableFrom(d.serviceClass()) - ? services.serviceProxy(name, PlatformService.class, sticky) - : new GridServiceProxy<>(services.clusterGroup(), name, Service.class, sticky, platformCtx.kernalContext()); - - return new ServiceProxyHolder(proxy, d.serviceClass()); - } - /** * Finds a service descriptor by name. * @@ -205,6 +163,14 @@ private ServiceDescriptor findDescriptor(String name) { return TRUE; } + case OP_CANCEL: { + String name = reader.readString(); + + services.cancel(name); + + return TRUE; + } + default: return super.processInStreamOutLong(type, reader); } @@ -315,7 +281,58 @@ private ServiceDescriptor findDescriptor(String name) { } } - /** */ + /** {@inheritDoc} */ + @Override protected Object processOutObject(int type) throws IgniteCheckedException { + switch (type) { + case OP_WITH_ASYNC: + if (services.isAsync()) + return this; + + return new PlatformServices(platformCtx, services.withAsync(), srvKeepBinary); + + case OP_WITH_SERVER_KEEP_BINARY: + return srvKeepBinary ? this : new PlatformServices(platformCtx, services, true); + } + + return super.processOutObject(type); + } + + /** {@inheritDoc} */ + @Override protected long processOutLong(int type) throws IgniteCheckedException { + switch (type) { + case OP_CANCEL_ALL: + services.cancelAll(); + + return TRUE; + } + + return super.processOutLong(type); + } + + /** {@inheritDoc} */ + @Override protected Object processInStreamOutObject(int type, BinaryRawReaderEx reader) throws IgniteCheckedException { + switch (type) { + case OP_SERVICE_PROXY: { + String name = reader.readString(); + boolean sticky = reader.readBoolean(); + + ServiceDescriptor d = findDescriptor(name); + + if (d == null) + throw new IgniteException("Failed to find deployed service: " + name); + + Object proxy = PlatformService.class.isAssignableFrom(d.serviceClass()) + ? services.serviceProxy(name, PlatformService.class, sticky) + : new GridServiceProxy<>(services.clusterGroup(), name, Service.class, sticky, + platformCtx.kernalContext()); + + return new ServiceProxyHolder(proxy, d.serviceClass()); + } + } + return super.processInStreamOutObject(type, reader); + } + + /** {@inheritDoc} */ @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException { return ((IgniteFutureImpl)services.future()).internalFuture(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java index 8afabac1bc425..339937c98fc67 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java @@ -23,6 +23,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteTransactions; import org.apache.ignite.configuration.TransactionConfiguration; +import org.apache.ignite.internal.binary.BinaryRawReaderEx; import org.apache.ignite.internal.binary.BinaryRawWriterEx; import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; @@ -46,6 +47,33 @@ public class PlatformTransactions extends PlatformAbstractTarget { /** */ public static final int OP_METRICS = 2; + /** */ + public static final int OP_START = 3; + + /** */ + public static final int OP_COMMIT = 4; + + /** */ + public static final int OP_ROLLBACK = 5; + + /** */ + public static final int OP_CLOSE = 6; + + /** */ + public static final int OP_STATE = 7; + + /** */ + public static final int OP_SET_ROLLBACK_ONLY = 8; + + /** */ + public static final int OP_COMMIT_ASYNC = 9; + + /** */ + public static final int OP_ROLLBACK_ASYNC = 10; + + /** */ + public static final int OP_RESET_METRICS = 11; + /** */ private final IgniteTransactions txs; @@ -67,44 +95,45 @@ public PlatformTransactions(PlatformContext platformCtx) { } /** - * @param concurrency Concurrency. - * @param isolation Isolation. - * @param timeout Timeout - * @param txSize Number of entries participating in transaction. - * @return Transaction thread ID. + * Listens to the transaction future and notifies .NET int future. */ - public long txStart(int concurrency, int isolation, long timeout, int txSize) { - TransactionConcurrency txConcurrency = TransactionConcurrency.fromOrdinal(concurrency); - - assert txConcurrency != null; - - TransactionIsolation txIsolation = TransactionIsolation.fromOrdinal(isolation); - - assert txIsolation != null; + private void listenAndNotifyIntFuture(final long futId, final Transaction asyncTx) { + IgniteFuture fut = asyncTx.future().chain(new C1() { + private static final long serialVersionUID = 0L; - Transaction tx = txs.txStart(txConcurrency, txIsolation); + @Override public Object apply(IgniteFuture fut) { + return null; + } + }); - return registerTx(tx); + PlatformFutureUtils.listen(platformCtx, fut, futId, PlatformFutureUtils.TYP_OBJ, this); } /** - * @param id Transaction ID. - * @throws org.apache.ignite.IgniteCheckedException In case of error. + * Register transaction. + * + * @param tx Transaction. + * @return Transaction ID. */ - public int txCommit(long id) throws IgniteCheckedException { - tx(id).commit(); + private long registerTx(Transaction tx) { + long id = TX_ID_GEN.incrementAndGet(); - return txClose(id); + Transaction old = txMap.put(id, tx); + + assert old == null : "Duplicate TX ids: " + old; + + return id; } /** + * Unregister transaction. + * * @param id Transaction ID. - * @throws org.apache.ignite.IgniteCheckedException In case of error. */ - public int txRollback(long id) throws IgniteCheckedException { - tx(id).rollback(); + private void unregisterTx(long id) { + Transaction tx = txMap.remove(id); - return txClose(id); + assert tx != null : "Failed to unregister transaction: " + id; } /** @@ -112,7 +141,7 @@ public int txRollback(long id) throws IgniteCheckedException { * @throws org.apache.ignite.IgniteCheckedException In case of error. * @return Transaction state. */ - public int txClose(long id) throws IgniteCheckedException { + private int txClose(long id) throws IgniteCheckedException { Transaction tx = tx(id); try { @@ -126,108 +155,108 @@ public int txClose(long id) throws IgniteCheckedException { } /** - * @param id Transaction ID. - * @return Transaction state. + * Get transaction by ID. + * + * @param id ID. + * @return Transaction. */ - public int txState(long id) { - Transaction tx = tx(id); + private Transaction tx(long id) { + Transaction tx = txMap.get(id); - return tx.state().ordinal(); + assert tx != null : "Transaction not found for ID: " + id; + + return tx; } - /** - * @param id Transaction ID. - * @return {@code True} if rollback only flag was set. - */ - public boolean txSetRollbackOnly(long id) { - Transaction tx = tx(id); + /** {@inheritDoc} */ + @Override protected long processOutLong(int type) throws IgniteCheckedException { + switch (type) { + case OP_RESET_METRICS: + txs.resetMetrics(); - return tx.setRollbackOnly(); + return TRUE; + } + + return super.processOutLong(type); } - /** - * Commits tx in async mode. - */ - public void txCommitAsync(final long txId, final long futId) { - final Transaction asyncTx = (Transaction)tx(txId).withAsync(); + /** {@inheritDoc} */ + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { + switch (type) { + case OP_COMMIT: + tx(val).commit(); - asyncTx.commit(); + return txClose(val); - listenAndNotifyIntFuture(futId, asyncTx); + case OP_ROLLBACK: + tx(val).rollback(); + + return txClose(val); + + case OP_CLOSE: + return txClose(val); + + case OP_SET_ROLLBACK_ONLY: + return tx(val).setRollbackOnly() ? TRUE : FALSE; + + case OP_STATE: + return tx(val).state().ordinal(); + } + + return super.processInLongOutLong(type, val); } - /** - * Rolls back tx in async mode. - */ - public void txRollbackAsync(final long txId, final long futId) { + /** {@inheritDoc} */ + @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException { + long txId = reader.readLong(); + long futId = reader.readLong(); + final Transaction asyncTx = (Transaction)tx(txId).withAsync(); - asyncTx.rollback(); + switch (type) { + case OP_COMMIT_ASYNC: + asyncTx.commit(); - listenAndNotifyIntFuture(futId, asyncTx); - } + break; - /** - * Listens to the transaction future and notifies .NET int future. - */ - private void listenAndNotifyIntFuture(final long futId, final Transaction asyncTx) { - IgniteFuture fut = asyncTx.future().chain(new C1() { - private static final long serialVersionUID = 0L; - @Override public Object apply(IgniteFuture fut) { - return null; - } - }); + case OP_ROLLBACK_ASYNC: + asyncTx.rollback(); - PlatformFutureUtils.listen(platformCtx, fut, futId, PlatformFutureUtils.TYP_OBJ, this); - } + break; - /** - * Resets transaction metrics. - */ - public void resetMetrics() { - txs.resetMetrics(); + default: + return super.processInStreamOutLong(type, reader); + } + + listenAndNotifyIntFuture(futId, asyncTx); + + return TRUE; } - /** - * Register transaction. - * - * @param tx Transaction. - * @return Transaction ID. - */ - private long registerTx(Transaction tx) { - long id = TX_ID_GEN.incrementAndGet(); + /** {@inheritDoc} */ + @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer) throws IgniteCheckedException { + switch (type) { + case OP_START: { + TransactionConcurrency txConcurrency = TransactionConcurrency.fromOrdinal(reader.readInt()); - Transaction old = txMap.put(id, tx); + assert txConcurrency != null; - assert old == null : "Duplicate TX ids: " + old; + TransactionIsolation txIsolation = TransactionIsolation.fromOrdinal(reader.readInt()); - return id; - } + assert txIsolation != null; - /** - * Unregister transaction. - * - * @param id Transaction ID. - */ - private void unregisterTx(long id) { - Transaction tx = txMap.remove(id); + Transaction tx = txs.txStart(txConcurrency, txIsolation, reader.readLong(), reader.readInt()); - assert tx != null : "Failed to unregister transaction: " + id; - } + long id = registerTx(tx); - /** - * Get transaction by ID. - * - * @param id ID. - * @return Transaction. - */ - private Transaction tx(long id) { - Transaction tx = txMap.get(id); + writer.writeLong(id); - assert tx != null : "Transaction not found for ID: " + id; + return; + } + } - return tx; + super.processInStreamOutStream(type, reader, writer); } /** {@inheritDoc} */ diff --git a/modules/platforms/cpp/core/include/ignite/cache/cache.h b/modules/platforms/cpp/core/include/ignite/cache/cache.h index 59b7a6abffdad..a975be3589212 100644 --- a/modules/platforms/cpp/core/include/ignite/cache/cache.h +++ b/modules/platforms/cpp/core/include/ignite/cache/cache.h @@ -115,7 +115,7 @@ namespace ignite */ bool IsEmpty(IgniteError& err) { - return impl.Get()->IsEmpty(&err); + return Size(err) == 0; } /** @@ -1054,7 +1054,6 @@ namespace ignite * * This method should only be used on the valid instance. * - * @param err Error. */ void RemoveAll() { @@ -1109,7 +1108,7 @@ namespace ignite * * This method should only be used on the valid instance. * - * @param Peek modes. + * @param peekModes Peek modes. * @return Cache size on this node. */ int32_t LocalSize(int32_t peekModes) @@ -1128,13 +1127,13 @@ namespace ignite * * This method should only be used on the valid instance. * - * @param Peek modes. + * @param peekModes Peek modes. * @param err Error. * @return Cache size on this node. */ int32_t LocalSize(int32_t peekModes, IgniteError& err) { - return impl.Get()->LocalSize(peekModes, &err); + return impl.Get()->Size(peekModes, true, &err); } /** @@ -1170,7 +1169,7 @@ namespace ignite * * This method should only be used on the valid instance. * - * @param Peek modes. + * @param peekModes Peek modes. * @return Cache size across all nodes. */ int32_t Size(int32_t peekModes) @@ -1190,13 +1189,13 @@ namespace ignite * * This method should only be used on the valid instance. * - * @param Peek modes. + * @param peekModes Peek modes. * @param err Error. * @return Cache size across all nodes. */ int32_t Size(int32_t peekModes, IgniteError& err) { - return impl.Get()->Size(peekModes, &err); + return impl.Get()->Size(peekModes, false, &err); } /** diff --git a/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h b/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h index e5f8ab8919b37..3e0f177b2c897 100644 --- a/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h +++ b/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h @@ -59,14 +59,6 @@ namespace ignite */ const char* GetName() const; - /** - * Perform IsEmpty. - * - * @param err Error. - * @return Result. - */ - bool IsEmpty(IgniteError* err); - /** * Perform ContainsKey. * @@ -274,22 +266,13 @@ namespace ignite void RemoveAll(IgniteError* err); /** - * Perform Size. - * - * @param peekModes Peek modes. - * @param err Error. - * @return Result. - */ - int32_t Size(const int32_t peekModes, IgniteError* err); - - /** - * Perform LocalSize. - * - * @param peekModes Peek modes. - * @param err Error. - * @return Result. - */ - int32_t LocalSize(const int32_t peekModes, IgniteError* err); + * Perform Size. + * + * @param peekModes Peek modes. + * @param local Local flag. + * @param err Error. + */ + int32_t Size(int32_t peekModes, bool local, IgniteError* err); /** * Invoke query. @@ -333,16 +316,6 @@ namespace ignite IGNITE_NO_COPY_ASSIGNMENT(CacheImpl) - /** - * Internal cache size routine. - * - * @param peekModes Peek modes. - * @param loc Local flag. - * @param err Error. - * @return Size. - */ - int SizeInternal(const int32_t peekModes, const bool loc, IgniteError* err); - /** * Internal query execution routine. * diff --git a/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h b/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h index 8b6ebb9c47de1..1c91a354c8679 100644 --- a/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h +++ b/modules/platforms/cpp/core/include/ignite/impl/interop/interop_target.h @@ -56,6 +56,15 @@ namespace ignite */ bool OutOp(int32_t opType, InputOperation& inOp, IgniteError* err); + /** + * Internal out operation. + * + * @param opType Operation type. + * @param err Error. + * @return Result. + */ + bool OutOp(int32_t opType, IgniteError* err); + /** * Internal out operation. * @@ -77,6 +86,15 @@ namespace ignite void OutInOp(int32_t opType, InputOperation& inOp, OutputOperation& outOp, IgniteError* err); + /** + * Internal out-in operation. + * + * @param opType Operation type. + * @param val Value. + * @param err Error. + */ + int64_t OutInOpLong(int32_t opType, int64_t val, IgniteError* err); + /** * Get environment shared pointer. * diff --git a/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp b/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp index e728f55a3ae35..d02759d9e4443 100644 --- a/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp +++ b/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp @@ -17,12 +17,8 @@ #include -#include "ignite/cache/cache_peek_mode.h" #include "ignite/impl/cache/cache_impl.h" -#include "ignite/impl/interop/interop.h" -#include "ignite/impl/binary/binary_reader_impl.h" #include "ignite/impl/binary/binary_type_updater_impl.h" -#include "ignite/binary/binary.h" using namespace ignite::common::concurrent; using namespace ignite::jni::java; @@ -120,6 +116,18 @@ namespace ignite /** Operation: Replace(K, V, V). */ const int32_t OP_REPLACE_3 = 38; + /** Operation: Clear(). */ + const int32_t OP_CLEAR_CACHE = 41; + + /** Operation: RemoveAll(). */ + const int32_t OP_REMOVE_ALL2 = 43; + + /** Operation: Size(peekModes). */ + const int32_t OP_SIZE = 48; + + /** Operation: SizeLoc(peekModes). */ + const int32_t OP_SIZE_LOC = 48; + CacheImpl::CacheImpl(char* name, SharedPointer env, jobject javaRef) : InteropTarget(env, javaRef), name(name) @@ -139,11 +147,6 @@ namespace ignite return name; } - bool CacheImpl::IsEmpty(IgniteError* err) - { - return Size(IGNITE_PEEK_MODE_ALL, err) == 0; - } - bool CacheImpl::ContainsKey(InputOperation& inOp, IgniteError* err) { return OutOp(OP_CONTAINS_KEY, inOp, err); @@ -223,7 +226,7 @@ namespace ignite { JniErrorInfo jniErr; - GetEnvironment().Context()->CacheClear(GetTarget(), &jniErr); + OutOp(OP_CLEAR_CACHE, err); IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); } @@ -267,19 +270,16 @@ namespace ignite { JniErrorInfo jniErr; - GetEnvironment().Context()->CacheRemoveAll(GetTarget(), &jniErr); + OutOp(OP_REMOVE_ALL2, err); IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); } - int32_t CacheImpl::Size(const int32_t peekModes, IgniteError* err) + int32_t CacheImpl::Size(int32_t peekModes, bool local, IgniteError* err) { - return SizeInternal(peekModes, false, err); - } + int32_t op = local ? OP_SIZE_LOC : OP_SIZE; - int32_t CacheImpl::LocalSize(const int32_t peekModes, IgniteError* err) - { - return SizeInternal(peekModes, true, err); + return static_cast(OutInOpLong(op, peekModes, err)); } QueryCursorImpl* CacheImpl::QuerySql(const SqlQuery& qry, IgniteError* err) @@ -301,20 +301,6 @@ namespace ignite { return QueryInternal(qry, OP_QRY_SQL_FIELDS, err); } - - int CacheImpl::SizeInternal(const int32_t peekModes, const bool loc, IgniteError* err) - { - JniErrorInfo jniErr; - - int res = GetEnvironment().Context()->CacheSize(GetTarget(), peekModes, loc, &jniErr); - - IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); - - if (jniErr.code == IGNITE_JNI_ERR_SUCCESS) - return res; - else - return -1; - } } } } \ No newline at end of file diff --git a/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp b/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp index 73d99248f0d54..c65e1e8ee5958 100644 --- a/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp +++ b/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp @@ -41,6 +41,15 @@ namespace ignite /** Operation: get single entry. */ const int32_t OP_GET_SINGLE = 3; + /** Operation: start iterator. */ + const int32_t OP_ITERATOR = 4; + + /** Operation: close iterator. */ + const int32_t OP_ITERATOR_CLOSE = 5; + + /** Operation: close iterator. */ + const int32_t OP_ITERATOR_HAS_NEXT = 6; + QueryCursorImpl::QueryCursorImpl(SharedPointer env, jobject javaRef) : env(env), javaRef(javaRef), @@ -58,7 +67,7 @@ namespace ignite delete batch; // 2. Close the cursor. - env.Get()->Context()->QueryCursorClose(javaRef); + env.Get()->Context()->TargetOutLong(javaRef, OP_ITERATOR_CLOSE); // 3. Release Java reference. JniContext::Release(javaRef); @@ -190,7 +199,7 @@ namespace ignite JniErrorInfo jniErr; - env.Get()->Context()->QueryCursorIterator(javaRef, &jniErr); + env.Get()->Context()->TargetOutLong(javaRef, OP_ITERATOR, &jniErr); IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); @@ -240,7 +249,7 @@ namespace ignite { JniErrorInfo jniErr; - bool res = env.Get()->Context()->QueryCursorIteratorHasNext(javaRef, &jniErr); + bool res = env.Get()->Context()->TargetOutLong(javaRef, OP_ITERATOR_HAS_NEXT, &jniErr) == 1; IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); diff --git a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp index 05764c70d9447..592c1ba2eb93a 100644 --- a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp +++ b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp @@ -96,6 +96,20 @@ namespace ignite return false; } + bool InteropTarget::OutOp(int32_t opType, IgniteError* err) + { + JniErrorInfo jniErr; + + long long res = env.Get()->Context()->TargetOutLong(javaRef, opType, &jniErr); + + IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); + + if (jniErr.code == IGNITE_JNI_ERR_SUCCESS) + return res == 1; + + return false; + } + bool InteropTarget::InOp(int32_t opType, OutputOperation& outOp, IgniteError* err) { JniErrorInfo jniErr; @@ -137,6 +151,17 @@ namespace ignite ReadFrom(inMem.Get(), outOp); } } + + int64_t InteropTarget::OutInOpLong(int32_t opType, int64_t val, IgniteError* err) + { + JniErrorInfo jniErr; + + long long res = env.Get()->Context()->TargetInLongOutLong(javaRef, opType, val, &jniErr); + + IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); + + return res; + } } } } \ No newline at end of file diff --git a/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp b/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp index 6c01332c666a6..58f30189804b4 100644 --- a/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp +++ b/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp @@ -32,8 +32,22 @@ namespace ignite */ enum Operation { - /** Get metrics operation. */ - OP_METRICS = 2 + /** Get metrics. */ + OP_METRICS = 2, + /** Start tx. */ + OP_START = 3, + /** Commit. */ + OP_COMMIT = 4, + /** Rollback. */ + OP_ROLLBACK = 5, + /** Close tx. */ + OP_CLOSE = 6, + /** Get tx state. */ + OP_STATE = 7, + /** Set rollback-only mode. */ + OP_SET_ROLLBACK_ONLY = 8, + /** Reset metrics. */ + OP_RESET_METRICS = 11, }; TransactionsImpl::TransactionsImpl(SP_IgniteEnvironment env, jobject javaRef) : @@ -47,28 +61,98 @@ namespace ignite // No-op. } + /* + * Input operation for starting a transaction. + */ + class InTransactionStartOperation : public InputOperation + { + public: + /** + * Constructor. + * + * @param concurrency Concurrency. + * @param isolation Isolation. + * @param timeout Timeout in milliseconds. Zero if for infinite timeout. + * @param txSize Number of entries participating in transaction (may be approximate). + */ + InTransactionStartOperation(int concurrency, int isolation, int64_t timeout, int32_t txSize) : + concurrency(concurrency), isolation(isolation), timeout(timeout), txSize(txSize) + { + // No-op. + } + + virtual void ProcessInput(binary::BinaryWriterImpl& writer) + { + writer.WriteInt32(concurrency); + writer.WriteInt32(isolation); + writer.WriteInt64(timeout); + writer.WriteInt32(txSize); + } + private: + int concurrency; + + int isolation; + + int64_t timeout; + + int32_t txSize; + + IGNITE_NO_COPY_ASSIGNMENT(InTransactionStartOperation) + }; + + /** + * Output operation for starting a transaction. + */ + class OutTransactionStartOperation : public OutputOperation + { + public: + /** + * Constructor. + */ + OutTransactionStartOperation(): val(0) + { + // No-op. + } + + virtual void ProcessOutput(binary::BinaryReaderImpl& reader) + { + val = reader.ReadInt64(); + } + + /** + * Get value. + * + * @return Value. + */ + int64_t Get() + { + return val; + } + + private: + /** Value */ + int64_t val; + + IGNITE_NO_COPY_ASSIGNMENT(OutTransactionStartOperation) + }; + + int64_t TransactionsImpl::TxStart(int concurrency, int isolation, int64_t timeout, int32_t txSize, IgniteError& err) { - JniErrorInfo jniErr; - - int64_t id = GetEnvironment().Context()->TransactionsStart(GetTarget(), - concurrency, isolation, timeout, txSize, &jniErr); + InTransactionStartOperation inOp(concurrency, isolation, timeout, txSize); + OutTransactionStartOperation outOp; - if (jniErr.code != IGNITE_JNI_ERR_SUCCESS) - IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); + OutInOp(OP_START, inOp, outOp, &err); - return id; + return outOp.Get(); } TransactionsImpl::TransactionState TransactionsImpl::TxCommit(int64_t id, IgniteError& err) { JniErrorInfo jniErr; - int state = GetEnvironment().Context()->TransactionsCommit(GetTarget(), id, &jniErr); - - if (jniErr.code != IGNITE_JNI_ERR_SUCCESS) - IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); + int state = static_cast(OutInOpLong(OP_COMMIT, id, &err)); return ToTransactionState(state); } @@ -77,10 +161,7 @@ namespace ignite { JniErrorInfo jniErr; - int state = GetEnvironment().Context()->TransactionsRollback(GetTarget(), id, &jniErr); - - if (jniErr.code != IGNITE_JNI_ERR_SUCCESS) - IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); + int state = static_cast(OutInOpLong(OP_ROLLBACK, id, &err)); return ToTransactionState(state); } @@ -89,10 +170,7 @@ namespace ignite { JniErrorInfo jniErr; - int state = GetEnvironment().Context()->TransactionsClose(GetTarget(), id, &jniErr); - - if (jniErr.code != IGNITE_JNI_ERR_SUCCESS) - IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); + int state = static_cast(OutInOpLong(OP_CLOSE, id, &err)); return ToTransactionState(state); } @@ -101,10 +179,7 @@ namespace ignite { JniErrorInfo jniErr; - bool rollbackOnly = GetEnvironment().Context()->TransactionsSetRollbackOnly(GetTarget(), id, &jniErr); - - if (jniErr.code != IGNITE_JNI_ERR_SUCCESS) - IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); + bool rollbackOnly = OutInOpLong(OP_SET_ROLLBACK_ONLY, id, &err) == 1; return rollbackOnly; } @@ -113,10 +188,7 @@ namespace ignite { JniErrorInfo jniErr; - int state = GetEnvironment().Context()->TransactionsState(GetTarget(), id, &jniErr); - - if (jniErr.code != IGNITE_JNI_ERR_SUCCESS) - IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); + int state = static_cast(OutInOpLong(OP_STATE, id, &err)); return ToTransactionState(state); } diff --git a/modules/platforms/cpp/jni/include/ignite/jni/exports.h b/modules/platforms/cpp/jni/include/ignite/jni/exports.h index bcfc213a30950..84f5a29f0c29f 100644 --- a/modules/platforms/cpp/jni/include/ignite/jni/exports.h +++ b/modules/platforms/cpp/jni/include/ignite/jni/exports.h @@ -55,10 +55,12 @@ extern "C" { void IGNITE_CALL IgniteProcessorGetIgniteConfiguration(gcj::JniContext* ctx, void* obj, long long memPtr); void IGNITE_CALL IgniteProcessorGetCacheNames(gcj::JniContext* ctx, void* obj, long long memPtr); + long long IGNITE_CALL IgniteTargetInLongOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr); long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr); void IGNITE_CALL IgniteTargetInStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, long long inMemPtr, long long outMemPtr); void* IGNITE_CALL IgniteTargetInStreamOutObject(gcj::JniContext* ctx, void* obj, int opType, long long memPtr); void IGNITE_CALL IgniteTargetInObjectStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, void* arg, long long inMemPtr, long long outMemPtr); + void* IGNITE_CALL IgniteTargetInObjectStreamOutObjectStream(gcj::JniContext* ctx, void* obj, int opType, void* arg, long long inMemPtr, long long outMemPtr); long long IGNITE_CALL IgniteTargetOutLong(gcj::JniContext* ctx, void* obj, int opType); void IGNITE_CALL IgniteTargetOutStream(gcj::JniContext* ctx, void* obj, int opType, long long memPtr); void* IGNITE_CALL IgniteTargetOutObject(gcj::JniContext* ctx, void* obj, int opType); @@ -67,69 +69,6 @@ extern "C" { void* IGNITE_CALL IgniteTargetListenFutureAndGet(gcj::JniContext* ctx, void* obj, long long futId, int typ); void* IGNITE_CALL IgniteTargetListenFutureForOperationAndGet(gcj::JniContext* ctx, void* obj, long long futId, int typ, int opId); - int IGNITE_CALL IgniteAffinityPartitions(gcj::JniContext* ctx, void* obj); - - void* IGNITE_CALL IgniteCacheWithSkipStore(gcj::JniContext* ctx, void* obj); - void* IGNITE_CALL IgniteCacheWithNoRetries(gcj::JniContext* ctx, void* obj); - void* IGNITE_CALL IgniteCacheWithExpiryPolicy(gcj::JniContext* ctx, void* obj, long long create, long long update, long long access); - void* IGNITE_CALL IgniteCacheWithAsync(gcj::JniContext* ctx, void* obj); - void* IGNITE_CALL IgniteCacheWithKeepPortable(gcj::JniContext* ctx, void* obj); - void IGNITE_CALL IgniteCacheClear(gcj::JniContext* ctx, void* obj); - void IGNITE_CALL IgniteCacheRemoveAll(gcj::JniContext* ctx, void* obj); - void* IGNITE_CALL IgniteCacheOutOpQueryCursor(gcj::JniContext* ctx, void* obj, int type, long long memPtr); - void* IGNITE_CALL IgniteCacheOutOpContinuousQuery(gcj::JniContext* ctx, void* obj, int type, long long memPtr); - void* IGNITE_CALL IgniteCacheIterator(gcj::JniContext* ctx, void* obj); - void* IGNITE_CALL IgniteCacheLocalIterator(gcj::JniContext* ctx, void* obj, int peekModes); - void IGNITE_CALL IgniteCacheEnterLock(gcj::JniContext* ctx, void* obj, long long id); - void IGNITE_CALL IgniteCacheExitLock(gcj::JniContext* ctx, void* obj, long long id); - bool IGNITE_CALL IgniteCacheTryEnterLock(gcj::JniContext* ctx, void* obj, long long id, long long timeout); - void IGNITE_CALL IgniteCacheCloseLock(gcj::JniContext* ctx, void* obj, long long id); - void IGNITE_CALL IgniteCacheRebalance(gcj::JniContext* ctx, void* obj, long long futId); - int IGNITE_CALL IgniteCacheSize(gcj::JniContext* ctx, void* obj, int peekModes, bool loc); - - void IGNITE_CALL IgniteCacheStoreCallbackInvoke(gcj::JniContext* ctx, void* obj, long long memPtr); - - void IGNITE_CALL IgniteComputeWithNoFailover(gcj::JniContext* ctx, void* obj); - void IGNITE_CALL IgniteComputeWithTimeout(gcj::JniContext* ctx, void* obj, long long timeout); - void* IGNITE_CALL IgniteComputeExecuteNative(gcj::JniContext* ctx, void* obj, long long taskPtr, long long topVer); - - void IGNITE_CALL IgniteContinuousQueryClose(gcj::JniContext* ctx, void* obj); - void* IGNITE_CALL IgniteContinuousQueryGetInitialQueryCursor(gcj::JniContext* ctx, void* obj); - - void IGNITE_CALL IgniteDataStreamerListenTopology(gcj::JniContext* ctx, void* obj, long long ptr); - bool IGNITE_CALL IgniteDataStreamerAllowOverwriteGet(gcj::JniContext* ctx, void* obj); - void IGNITE_CALL IgniteDataStreamerAllowOverwriteSet(gcj::JniContext* ctx, void* obj, bool val); - bool IGNITE_CALL IgniteDataStreamerSkipStoreGet(gcj::JniContext* ctx, void* obj); - void IGNITE_CALL IgniteDataStreamerSkipStoreSet(gcj::JniContext* ctx, void* obj, bool val); - int IGNITE_CALL IgniteDataStreamerPerNodeBufferSizeGet(gcj::JniContext* ctx, void* obj); - void IGNITE_CALL IgniteDataStreamerPerNodeBufferSizeSet(gcj::JniContext* ctx, void* obj, int val); - int IGNITE_CALL IgniteDataStreamerPerNodeParallelOperationsGet(gcj::JniContext* ctx, void* obj); - void IGNITE_CALL IgniteDataStreamerPerNodeParallelOperationsSet(gcj::JniContext* ctx, void* obj, int val); - - void* IGNITE_CALL IgniteMessagingWithAsync(gcj::JniContext* ctx, void* obj); - - void* IGNITE_CALL IgniteProjectionForOthers(gcj::JniContext* ctx, void* obj, void* prj); - void* IGNITE_CALL IgniteProjectionForRemotes(gcj::JniContext* ctx, void* obj); - void* IGNITE_CALL IgniteProjectionForDaemons(gcj::JniContext* ctx, void* obj); - void* IGNITE_CALL IgniteProjectionForRandom(gcj::JniContext* ctx, void* obj); - void* IGNITE_CALL IgniteProjectionForOldest(gcj::JniContext* ctx, void* obj); - void* IGNITE_CALL IgniteProjectionForYoungest(gcj::JniContext* ctx, void* obj); - void IGNITE_CALL IgniteProjectionResetMetrics(gcj::JniContext* ctx, void* obj); - void* IGNITE_CALL IgniteProjectionOutOpRet(gcj::JniContext* ctx, void* obj, int type, long long memPtr); - - void IGNITE_CALL IgniteQueryCursorIterator(gcj::JniContext* ctx, void* obj); - void IGNITE_CALL IgniteQueryCursorClose(gcj::JniContext* ctx, void* obj); - - long long IGNITE_CALL IgniteTransactionsStart(gcj::JniContext* ctx, void* obj, int concurrency, int isolation, long long timeout, int txSize); - int IGNITE_CALL IgniteTransactionsCommit(gcj::JniContext* ctx, void* obj, long long id); - void IGNITE_CALL IgniteTransactionsCommitAsync(gcj::JniContext* ctx, void* obj, long long id, long long futId); - int IGNITE_CALL IgniteTransactionsRollback(gcj::JniContext* ctx, void* obj, long long id); - void IGNITE_CALL IgniteTransactionsRollbackAsync(gcj::JniContext* ctx, void* obj, long long id, long long futId); - int IGNITE_CALL IgniteTransactionsClose(gcj::JniContext* ctx, void* obj, long long id); - int IGNITE_CALL IgniteTransactionsState(gcj::JniContext* ctx, void* obj, long long id); - bool IGNITE_CALL IgniteTransactionsSetRollbackOnly(gcj::JniContext* ctx, void* obj, long long id); - void IGNITE_CALL IgniteTransactionsResetMetrics(gcj::JniContext* ctx, void* obj); - void* IGNITE_CALL IgniteAcquire(gcj::JniContext* ctx, void* obj); void IGNITE_CALL IgniteRelease(void* obj); @@ -142,42 +81,6 @@ extern "C" { void IGNITE_CALL IgniteDestroyJvm(gcj::JniContext* ctx); - void* IGNITE_CALL IgniteEventsWithAsync(gcj::JniContext* ctx, void* obj); - bool IGNITE_CALL IgniteEventsStopLocalListen(gcj::JniContext* ctx, void* obj, long long hnd); - void IGNITE_CALL IgniteEventsLocalListen(gcj::JniContext* ctx, void* obj, long long hnd, int type); - bool IGNITE_CALL IgniteEventsIsEnabled(gcj::JniContext* ctx, void* obj, int type); - - void* IGNITE_CALL IgniteServicesWithAsync(gcj::JniContext* ctx, void* obj); - void* IGNITE_CALL IgniteServicesWithServerKeepPortable(gcj::JniContext* ctx, void* obj); - void IGNITE_CALL IgniteServicesCancel(gcj::JniContext* ctx, void* obj, char* name); - void IGNITE_CALL IgniteServicesCancelAll(gcj::JniContext* ctx, void* obj); - void* IGNITE_CALL IgniteServicesGetServiceProxy(gcj::JniContext* ctx, void* obj, char* name, bool sticky); - - long long IGNITE_CALL IgniteAtomicLongGet(gcj::JniContext* ctx, void* obj); - long long IGNITE_CALL IgniteAtomicLongIncrementAndGet(gcj::JniContext* ctx, void* obj); - long long IGNITE_CALL IgniteAtomicLongGetAndIncrement(gcj::JniContext* ctx, void* obj); - long long IGNITE_CALL IgniteAtomicLongAddAndGet(gcj::JniContext* ctx, void* obj, long long value); - long long IGNITE_CALL IgniteAtomicLongGetAndAdd(gcj::JniContext* ctx, void* obj, long long value); - long long IGNITE_CALL IgniteAtomicLongDecrementAndGet(gcj::JniContext* ctx, void* obj); - long long IGNITE_CALL IgniteAtomicLongGetAndDecrement(gcj::JniContext* ctx, void* obj); - long long IGNITE_CALL IgniteAtomicLongGetAndSet(gcj::JniContext* ctx, void* obj, long long value); - long long IGNITE_CALL IgniteAtomicLongCompareAndSetAndGet(gcj::JniContext* ctx, void* obj, long long expVal, long long newVal); - bool IGNITE_CALL IgniteAtomicLongIsClosed(gcj::JniContext* ctx, void* obj); - void IGNITE_CALL IgniteAtomicLongClose(gcj::JniContext* ctx, void* obj); - - long long IGNITE_CALL IgniteAtomicSequenceGet(gcj::JniContext* ctx, void* obj); - long long IGNITE_CALL IgniteAtomicSequenceIncrementAndGet(gcj::JniContext* ctx, void* obj); - long long IGNITE_CALL IgniteAtomicSequenceGetAndIncrement(gcj::JniContext* ctx, void* obj); - long long IGNITE_CALL IgniteAtomicSequenceAddAndGet(gcj::JniContext* ctx, void* obj, long long l); - long long IGNITE_CALL IgniteAtomicSequenceGetAndAdd(gcj::JniContext* ctx, void* obj, long long l); - int IGNITE_CALL IgniteAtomicSequenceGetBatchSize(gcj::JniContext* ctx, void* obj); - void IGNITE_CALL IgniteAtomicSequenceSetBatchSize(gcj::JniContext* ctx, void* obj, int size); - bool IGNITE_CALL IgniteAtomicSequenceIsClosed(gcj::JniContext* ctx, void* obj); - void IGNITE_CALL IgniteAtomicSequenceClose(gcj::JniContext* ctx, void* obj); - - bool IGNITE_CALL IgniteAtomicReferenceIsClosed(gcj::JniContext* ctx, void* obj); - void IGNITE_CALL IgniteAtomicReferenceClose(gcj::JniContext* ctx, void* obj); - bool IGNITE_CALL IgniteListenableCancel(gcj::JniContext* ctx, void* obj); bool IGNITE_CALL IgniteListenableIsCancelled(gcj::JniContext* ctx, void* obj); } diff --git a/modules/platforms/cpp/jni/include/ignite/jni/java.h b/modules/platforms/cpp/jni/include/ignite/jni/java.h index 18a15e2d60da5..96daba8270f65 100644 --- a/modules/platforms/cpp/jni/include/ignite/jni/java.h +++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h @@ -31,7 +31,7 @@ namespace ignite { /* Handlers for callbacks from Java. */ typedef long long(JNICALL *CacheStoreCreateHandler)(void* target, long long memPtr); - typedef int(JNICALL *CacheStoreInvokeHandler)(void* target, long long objPtr, long long memPtr, void* cb); + typedef int(JNICALL *CacheStoreInvokeHandler)(void* target, long long objPtr, long long memPtr); typedef void(JNICALL *CacheStoreDestroyHandler)(void* target, long long objPtr); typedef long long(JNICALL *CacheStoreSessionCreateHandler)(void* target, long long storePtr); @@ -223,78 +223,8 @@ namespace ignite * JNI members. */ struct JniMembers { - jclass c_PlatformAbstractQryCursor; - jmethodID m_PlatformAbstractQryCursor_iter; - jmethodID m_PlatformAbstractQryCursor_iterHasNext; - jmethodID m_PlatformAbstractQryCursor_close; - - jclass c_PlatformAffinity; - jmethodID m_PlatformAffinity_partitions; - - jclass c_PlatformCache; - jmethodID m_PlatformCache_withSkipStore; - jmethodID m_PlatformCache_withNoRetries; - jmethodID m_PlatformCache_withExpiryPolicy; - jmethodID m_PlatformCache_withAsync; - jmethodID m_PlatformCache_withKeepPortable; - jmethodID m_PlatformCache_clear; - jmethodID m_PlatformCache_removeAll; - jmethodID m_PlatformCache_iterator; - jmethodID m_PlatformCache_localIterator; - jmethodID m_PlatformCache_enterLock; - jmethodID m_PlatformCache_exitLock; - jmethodID m_PlatformCache_tryEnterLock; - jmethodID m_PlatformCache_closeLock; - jmethodID m_PlatformCache_rebalance; - jmethodID m_PlatformCache_size; - - jclass c_PlatformCacheStoreCallback; - jmethodID m_PlatformCacheStoreCallback_invoke; - jclass c_IgniteException; - jclass c_PlatformClusterGroup; - jmethodID m_PlatformClusterGroup_forOthers; - jmethodID m_PlatformClusterGroup_forRemotes; - jmethodID m_PlatformClusterGroup_forDaemons; - jmethodID m_PlatformClusterGroup_forRandom; - jmethodID m_PlatformClusterGroup_forOldest; - jmethodID m_PlatformClusterGroup_forYoungest; - jmethodID m_PlatformClusterGroup_resetMetrics; - - jclass c_PlatformCompute; - jmethodID m_PlatformCompute_withNoFailover; - jmethodID m_PlatformCompute_withTimeout; - jmethodID m_PlatformCompute_executeNative; - - jclass c_PlatformContinuousQuery; - jmethodID m_PlatformContinuousQuery_close; - jmethodID m_PlatformContinuousQuery_getInitialQueryCursor; - - jclass c_PlatformDataStreamer; - jmethodID m_PlatformDataStreamer_listenTopology; - jmethodID m_PlatformDataStreamer_getAllowOverwrite; - jmethodID m_PlatformDataStreamer_setAllowOverwrite; - jmethodID m_PlatformDataStreamer_getSkipStore; - jmethodID m_PlatformDataStreamer_setSkipStore; - jmethodID m_PlatformDataStreamer_getPerNodeBufSize; - jmethodID m_PlatformDataStreamer_setPerNodeBufSize; - jmethodID m_PlatformDataStreamer_getPerNodeParallelOps; - jmethodID m_PlatformDataStreamer_setPerNodeParallelOps; - - jclass c_PlatformEvents; - jmethodID m_PlatformEvents_withAsync; - jmethodID m_PlatformEvents_stopLocalListen; - jmethodID m_PlatformEvents_localListen; - jmethodID m_PlatformEvents_isEnabled; - - jclass c_PlatformServices; - jmethodID m_PlatformServices_withAsync; - jmethodID m_PlatformServices_withServerKeepPortable; - jmethodID m_PlatformServices_cancel; - jmethodID m_PlatformServices_cancelAll; - jmethodID m_PlatformServices_serviceProxy; - jclass c_PlatformIgnition; jmethodID m_PlatformIgnition_start; jmethodID m_PlatformIgnition_instance; @@ -302,9 +232,6 @@ namespace ignite jmethodID m_PlatformIgnition_stop; jmethodID m_PlatformIgnition_stopAll; - jclass c_PlatformMessaging; - jmethodID m_PlatformMessaging_withAsync; - jclass c_PlatformProcessor; jmethodID m_PlatformProcessor_releaseStart; jmethodID m_PlatformProcessor_cache; @@ -331,6 +258,7 @@ namespace ignite jmethodID m_PlatformProcessor_atomicReference; jclass c_PlatformTarget; + jmethodID m_PlatformTarget_inLongOutLong; jmethodID m_PlatformTarget_inStreamOutLong; jmethodID m_PlatformTarget_inStreamOutObject; jmethodID m_PlatformTarget_outLong; @@ -338,54 +266,16 @@ namespace ignite jmethodID m_PlatformTarget_outObject; jmethodID m_PlatformTarget_inStreamOutStream; jmethodID m_PlatformTarget_inObjectStreamOutStream; + jmethodID m_PlatformTarget_inObjectStreamOutObjectStream; jmethodID m_PlatformTarget_listenFuture; jmethodID m_PlatformTarget_listenFutureForOperation; jmethodID m_PlatformTarget_listenFutureAndGet; jmethodID m_PlatformTarget_listenFutureForOperationAndGet; - jclass c_PlatformTransactions; - jmethodID m_PlatformTransactions_txStart; - jmethodID m_PlatformTransactions_txCommit; - jmethodID m_PlatformTransactions_txCommitAsync; - jmethodID m_PlatformTransactions_txRollback; - jmethodID m_PlatformTransactions_txRollbackAsync; - jmethodID m_PlatformTransactions_txState; - jmethodID m_PlatformTransactions_txSetRollbackOnly; - jmethodID m_PlatformTransactions_txClose; - jmethodID m_PlatformTransactions_resetMetrics; - jclass c_PlatformUtils; jmethodID m_PlatformUtils_reallocate; jmethodID m_PlatformUtils_errData; - jclass c_PlatformAtomicLong; - jmethodID m_PlatformAtomicLong_get; - jmethodID m_PlatformAtomicLong_incrementAndGet; - jmethodID m_PlatformAtomicLong_getAndIncrement; - jmethodID m_PlatformAtomicLong_addAndGet; - jmethodID m_PlatformAtomicLong_getAndAdd; - jmethodID m_PlatformAtomicLong_decrementAndGet; - jmethodID m_PlatformAtomicLong_getAndDecrement; - jmethodID m_PlatformAtomicLong_getAndSet; - jmethodID m_PlatformAtomicLong_compareAndSetAndGet; - jmethodID m_PlatformAtomicLong_isClosed; - jmethodID m_PlatformAtomicLong_close; - - jclass c_PlatformAtomicSequence; - jmethodID m_PlatformAtomicSequence_get; - jmethodID m_PlatformAtomicSequence_incrementAndGet; - jmethodID m_PlatformAtomicSequence_getAndIncrement; - jmethodID m_PlatformAtomicSequence_addAndGet; - jmethodID m_PlatformAtomicSequence_getAndAdd; - jmethodID m_PlatformAtomicSequence_getBatchSize; - jmethodID m_PlatformAtomicSequence_setBatchSize; - jmethodID m_PlatformAtomicSequence_isClosed; - jmethodID m_PlatformAtomicSequence_close; - - jclass c_PlatformAtomicReference; - jmethodID m_PlatformAtomicReference_isClosed; - jmethodID m_PlatformAtomicReference_close; - jclass c_PlatformListenable; jmethodID m_PlatformListenable_cancel; jmethodID m_PlatformListenable_isCancelled; @@ -547,10 +437,12 @@ namespace ignite void ProcessorGetIgniteConfiguration(jobject obj, long long memPtr); void ProcessorGetCacheNames(jobject obj, long long memPtr); + long long TargetInLongOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); long long TargetInStreamOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); void TargetInStreamOutStream(jobject obj, int opType, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL); jobject TargetInStreamOutObject(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); void TargetInObjectStreamOutStream(jobject obj, int opType, void* arg, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL); + jobject TargetInObjectStreamOutObjectStream(jobject obj, int opType, void* arg, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL); long long TargetOutLong(jobject obj, int opType, JniErrorInfo* errInfo = NULL); void TargetOutStream(jobject obj, int opType, long long memPtr, JniErrorInfo* errInfo = NULL); jobject TargetOutObject(jobject obj, int opType, JniErrorInfo* errInfo = NULL); @@ -559,105 +451,8 @@ namespace ignite void* TargetListenFutureAndGet(jobject obj, long long futId, int typ); void* TargetListenFutureForOperationAndGet(jobject obj, long long futId, int typ, int opId); - int AffinityPartitions(jobject obj); - - jobject CacheWithSkipStore(jobject obj); - jobject CacheWithNoRetries(jobject obj); - jobject CacheWithExpiryPolicy(jobject obj, long long create, long long update, long long access); - jobject CacheWithAsync(jobject obj); - jobject CacheWithKeepPortable(jobject obj); - void CacheClear(jobject obj, JniErrorInfo* errInfo = NULL); - void CacheRemoveAll(jobject obj, JniErrorInfo* errInfo = NULL); jobject CacheOutOpQueryCursor(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); jobject CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr); - jobject CacheIterator(jobject obj); - jobject CacheLocalIterator(jobject obj, int peekModes); - void CacheEnterLock(jobject obj, long long id); - void CacheExitLock(jobject obj, long long id); - bool CacheTryEnterLock(jobject obj, long long id, long long timeout); - void CacheCloseLock(jobject obj, long long id); - void CacheRebalance(jobject obj, long long futId); - int CacheSize(jobject obj, int peekModes, bool loc, JniErrorInfo* errInfo = NULL); - - void CacheStoreCallbackInvoke(jobject obj, long long memPtr); - - void ComputeWithNoFailover(jobject obj); - void ComputeWithTimeout(jobject obj, long long timeout); - void* ComputeExecuteNative(jobject obj, long long taskPtr, long long topVer); - - void ContinuousQueryClose(jobject obj); - jobject ContinuousQueryGetInitialQueryCursor(jobject obj); - - void DataStreamerListenTopology(jobject obj, long long ptr); - bool DataStreamerAllowOverwriteGet(jobject obj); - void DataStreamerAllowOverwriteSet(jobject obj, bool val); - bool DataStreamerSkipStoreGet(jobject obj); - void DataStreamerSkipStoreSet(jobject obj, bool val); - int DataStreamerPerNodeBufferSizeGet(jobject obj); - void DataStreamerPerNodeBufferSizeSet(jobject obj, int val); - int DataStreamerPerNodeParallelOperationsGet(jobject obj); - void DataStreamerPerNodeParallelOperationsSet(jobject obj, int val); - - jobject MessagingWithAsync(jobject obj); - - jobject ProjectionForOthers(jobject obj, jobject prj); - jobject ProjectionForRemotes(jobject obj); - jobject ProjectionForDaemons(jobject obj); - jobject ProjectionForRandom(jobject obj); - jobject ProjectionForOldest(jobject obj); - jobject ProjectionForYoungest(jobject obj); - void ProjectionResetMetrics(jobject obj); - jobject ProjectionOutOpRet(jobject obj, int type, long long memPtr); - - void QueryCursorIterator(jobject obj, JniErrorInfo* errInfo = NULL); - bool QueryCursorIteratorHasNext(jobject obj, JniErrorInfo* errInfo = NULL); - void QueryCursorClose(jobject obj, JniErrorInfo* errInfo = NULL); - - long long TransactionsStart(jobject obj, int concurrency, int isolation, long long timeout, int txSize, JniErrorInfo* errInfo = NULL); - int TransactionsCommit(jobject obj, long long id, JniErrorInfo* errInfo = NULL); - void TransactionsCommitAsync(jobject obj, long long id, long long futId); - int TransactionsRollback(jobject obj, long long id, JniErrorInfo* errInfo = NULL); - void TransactionsRollbackAsync(jobject obj, long long id, long long futId); - int TransactionsClose(jobject obj, long long id, JniErrorInfo* errInfo = NULL); - int TransactionsState(jobject obj, long long id, JniErrorInfo* errInfo = NULL); - bool TransactionsSetRollbackOnly(jobject obj, long long id, JniErrorInfo* errInfo = NULL); - void TransactionsResetMetrics(jobject obj); - - jobject EventsWithAsync(jobject obj); - bool EventsStopLocalListen(jobject obj, long long hnd); - void EventsLocalListen(jobject obj, long long hnd, int type); - bool EventsIsEnabled(jobject obj, int type); - - jobject ServicesWithAsync(jobject obj); - jobject ServicesWithServerKeepPortable(jobject obj); - void ServicesCancel(jobject obj, char* name); - void ServicesCancelAll(jobject obj); - void* ServicesGetServiceProxy(jobject obj, char* name, bool sticky); - - long long AtomicLongGet(jobject obj); - long long AtomicLongIncrementAndGet(jobject obj); - long long AtomicLongGetAndIncrement(jobject obj); - long long AtomicLongAddAndGet(jobject obj, long long value); - long long AtomicLongGetAndAdd(jobject obj, long long value); - long long AtomicLongDecrementAndGet(jobject obj); - long long AtomicLongGetAndDecrement(jobject obj); - long long AtomicLongGetAndSet(jobject obj, long long value); - long long AtomicLongCompareAndSetAndGet(jobject obj, long long expVal, long long newVal); - bool AtomicLongIsClosed(jobject obj); - void AtomicLongClose(jobject obj); - - long long AtomicSequenceGet(jobject obj); - long long AtomicSequenceIncrementAndGet(jobject obj); - long long AtomicSequenceGetAndIncrement(jobject obj); - long long AtomicSequenceAddAndGet(jobject obj, long long l); - long long AtomicSequenceGetAndAdd(jobject obj, long long l); - int AtomicSequenceGetBatchSize(jobject obj); - void AtomicSequenceSetBatchSize(jobject obj, int size); - bool AtomicSequenceIsClosed(jobject obj); - void AtomicSequenceClose(jobject obj); - - bool AtomicReferenceIsClosed(jobject obj); - void AtomicReferenceClose(jobject obj); bool ListenableCancel(jobject obj); bool ListenableIsCancelled(jobject obj); @@ -682,7 +477,7 @@ namespace ignite }; JNIEXPORT jlong JNICALL JniCacheStoreCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr); - JNIEXPORT jint JNICALL JniCacheStoreInvoke(JNIEnv *env, jclass cls, jlong envPtr, jlong objPtr, jlong memPtr, jobject cb); + JNIEXPORT jint JNICALL JniCacheStoreInvoke(JNIEnv *env, jclass cls, jlong envPtr, jlong objPtr, jlong memPtr); JNIEXPORT void JNICALL JniCacheStoreDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong objPtr); JNIEXPORT jlong JNICALL JniCacheStoreSessionCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong storePtr); diff --git a/modules/platforms/cpp/jni/project/vs/module.def b/modules/platforms/cpp/jni/project/vs/module.def index dc4af3dc92be4..c2069b6a91e19 100644 --- a/modules/platforms/cpp/jni/project/vs/module.def +++ b/modules/platforms/cpp/jni/project/vs/module.def @@ -23,62 +23,10 @@ IgniteTargetInStreamOutStream @20 IgniteTargetInObjectStreamOutStream @21 IgniteTargetListenFuture @22 IgniteTargetListenFutureForOperation @23 -IgniteAffinityPartitions @24 -IgniteCacheWithSkipStore @25 -IgniteCacheWithNoRetries @26 -IgniteCacheWithExpiryPolicy @27 -IgniteCacheWithAsync @28 -IgniteCacheWithKeepPortable @29 -IgniteCacheClear @30 -IgniteCacheRemoveAll @31 -IgniteCacheOutOpQueryCursor @32 -IgniteCacheOutOpContinuousQuery @33 -IgniteCacheIterator @34 -IgniteCacheLocalIterator @35 -IgniteCacheEnterLock @36 -IgniteCacheExitLock @37 -IgniteCacheTryEnterLock @38 -IgniteCacheCloseLock @39 -IgniteCacheRebalance @40 -IgniteCacheSize @41 -IgniteCacheStoreCallbackInvoke @42 -IgniteComputeWithNoFailover @43 -IgniteComputeWithTimeout @44 -IgniteComputeExecuteNative @45 -IgniteContinuousQueryClose @46 -IgniteContinuousQueryGetInitialQueryCursor @47 -IgniteDataStreamerListenTopology @48 -IgniteDataStreamerAllowOverwriteGet @49 -IgniteDataStreamerAllowOverwriteSet @50 -IgniteDataStreamerSkipStoreGet @51 -IgniteDataStreamerSkipStoreSet @52 -IgniteDataStreamerPerNodeBufferSizeGet @53 -IgniteDataStreamerPerNodeBufferSizeSet @54 -IgniteDataStreamerPerNodeParallelOperationsGet @55 -IgniteDataStreamerPerNodeParallelOperationsSet @56 -IgniteMessagingWithAsync @57 -IgniteProjectionForOthers @58 -IgniteProjectionForRemotes @59 -IgniteProjectionForDaemons @60 -IgniteProjectionForRandom @61 -IgniteProjectionForOldest @62 -IgniteProjectionForYoungest @63 +IgniteTargetInLongOutLong @24 IgniteProcessorCompute @64 IgniteProcessorMessage @65 IgniteProcessorEvents @66 -IgniteProjectionResetMetrics @67 -IgniteProjectionOutOpRet @68 -IgniteQueryCursorIterator @69 -IgniteQueryCursorClose @70 -IgniteTransactionsStart @71 -IgniteTransactionsCommit @72 -IgniteTransactionsCommitAsync @73 -IgniteTransactionsRollback @74 -IgniteTransactionsRollbackAsync @75 -IgniteTransactionsClose @76 -IgniteTransactionsState @77 -IgniteTransactionsSetRollbackOnly @78 -IgniteTransactionsResetMetrics @79 IgniteAcquire @80 IgniteRelease @81 IgniteThrowToJava @82 @@ -86,29 +34,9 @@ IgniteHandlersSize @83 IgniteCreateContext @84 IgniteDeleteContext @85 IgniteDestroyJvm @86 -IgniteEventsWithAsync @87 -IgniteEventsStopLocalListen @88 -IgniteEventsLocalListen @89 -IgniteEventsIsEnabled @90 IgniteTargetOutObject @91 -IgniteServicesWithAsync @92 -IgniteServicesWithServerKeepPortable @93 -IgniteServicesCancel @94 -IgniteServicesCancelAll @95 -IgniteServicesGetServiceProxy @96 IgniteProcessorExtensions @97 IgniteProcessorAtomicLong @98 -IgniteAtomicLongGet @99 -IgniteAtomicLongIncrementAndGet @100 -IgniteAtomicLongGetAndIncrement @101 -IgniteAtomicLongAddAndGet @102 -IgniteAtomicLongGetAndAdd @103 -IgniteAtomicLongDecrementAndGet @104 -IgniteAtomicLongGetAndDecrement @105 -IgniteAtomicLongGetAndSet @106 -IgniteAtomicLongCompareAndSetAndGet @107 -IgniteAtomicLongIsClosed @108 -IgniteAtomicLongClose @109 IgniteListenableCancel @110 IgniteListenableIsCancelled @111 IgniteTargetListenFutureAndGet @112 @@ -118,18 +46,7 @@ IgniteProcessorGetOrCreateCacheFromConfig @115 IgniteProcessorGetIgniteConfiguration @116 IgniteProcessorDestroyCache @117 IgniteProcessorAtomicSequence @118 -IgniteAtomicSequenceGet @119 -IgniteAtomicSequenceIncrementAndGet @120 -IgniteAtomicSequenceGetAndIncrement @121 -IgniteAtomicSequenceAddAndGet @122 -IgniteAtomicSequenceGetAndAdd @123 -IgniteAtomicSequenceGetBatchSize @124 -IgniteAtomicSequenceSetBatchSize @125 -IgniteAtomicSequenceIsClosed @126 -IgniteAtomicSequenceClose @127 IgniteProcessorAtomicReference @128 -IgniteAtomicReferenceIsClosed @129 -IgniteAtomicReferenceClose @130 IgniteProcessorCreateNearCache @131 IgniteProcessorGetOrCreateNearCache @132 IgniteProcessorGetCacheNames @133 diff --git a/modules/platforms/cpp/jni/src/exports.cpp b/modules/platforms/cpp/jni/src/exports.cpp index 89878d41724b8..ca332be619084 100644 --- a/modules/platforms/cpp/jni/src/exports.cpp +++ b/modules/platforms/cpp/jni/src/exports.cpp @@ -138,6 +138,10 @@ extern "C" { return ctx->ProcessorGetCacheNames(static_cast(obj), memPtr); } + long long IGNITE_CALL IgniteTargetInLongOutLong(gcj::JniContext* ctx, void* obj, int opType, long long val) { + return ctx->TargetInLongOutLong(static_cast(obj), opType, val); + } + long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr) { return ctx->TargetInStreamOutLong(static_cast(obj), opType, memPtr); } @@ -154,6 +158,10 @@ extern "C" { ctx->TargetInObjectStreamOutStream(static_cast(obj), opType, arg, inMemPtr, outMemPtr); } + void* IGNITE_CALL IgniteTargetInObjectStreamOutObjectStream(gcj::JniContext* ctx, void* obj, int opType, void* arg, long long inMemPtr, long long outMemPtr) { + return ctx->TargetInObjectStreamOutObjectStream(static_cast(obj), opType, arg, inMemPtr, outMemPtr); + } + long long IGNITE_CALL IgniteTargetOutLong(gcj::JniContext* ctx, void* obj, int opType) { return ctx->TargetOutLong(static_cast(obj), opType); } @@ -182,219 +190,6 @@ extern "C" { return ctx->TargetListenFutureForOperationAndGet(static_cast(obj), futId, typ, opId); } - int IGNITE_CALL IgniteAffinityPartitions(gcj::JniContext* ctx, void* obj) { - return ctx->AffinityPartitions(static_cast(obj)); - } - - void* IGNITE_CALL IgniteCacheWithSkipStore(gcj::JniContext* ctx, void* obj) { - return ctx->CacheWithSkipStore(static_cast(obj)); - } - - void* IGNITE_CALL IgniteCacheWithNoRetries(gcj::JniContext* ctx, void* obj) { - return ctx->CacheWithNoRetries(static_cast(obj)); - } - - void* IGNITE_CALL IgniteCacheWithExpiryPolicy(gcj::JniContext* ctx, void* obj, long long create, long long update, long long access) { - return ctx->CacheWithExpiryPolicy(static_cast(obj), create, update, access); - } - - void* IGNITE_CALL IgniteCacheWithAsync(gcj::JniContext* ctx, void* obj) { - return ctx->CacheWithAsync(static_cast(obj)); - } - - void* IGNITE_CALL IgniteCacheWithKeepPortable(gcj::JniContext* ctx, void* obj) - { - return ctx->CacheWithKeepPortable(static_cast(obj)); - } - - void IGNITE_CALL IgniteCacheClear(gcj::JniContext* ctx, void* obj) { - ctx->CacheClear(static_cast(obj)); - } - - void IGNITE_CALL IgniteCacheRemoveAll(gcj::JniContext* ctx, void* obj) { - ctx->CacheRemoveAll(static_cast(obj)); - } - - void* IGNITE_CALL IgniteCacheOutOpQueryCursor(gcj::JniContext* ctx, void* obj, int type, long long memPtr) { - return ctx->CacheOutOpQueryCursor(static_cast(obj), type, memPtr); - } - - void* IGNITE_CALL IgniteCacheOutOpContinuousQuery(gcj::JniContext* ctx, void* obj, int type, long long memPtr) { - return ctx->CacheOutOpContinuousQuery(static_cast(obj), type, memPtr); - } - - void* IGNITE_CALL IgniteCacheIterator(gcj::JniContext* ctx, void* obj) { - return ctx->CacheIterator(static_cast(obj)); - } - - void* IGNITE_CALL IgniteCacheLocalIterator(gcj::JniContext* ctx, void* obj, int peekModes) { - return ctx->CacheLocalIterator(static_cast(obj), peekModes); - } - - void IGNITE_CALL IgniteCacheEnterLock(gcj::JniContext* ctx, void* obj, long long id) { - ctx->CacheEnterLock(static_cast(obj), id); - } - - void IGNITE_CALL IgniteCacheExitLock(gcj::JniContext* ctx, void* obj, long long id) { - ctx->CacheExitLock(static_cast(obj), id); - } - - bool IGNITE_CALL IgniteCacheTryEnterLock(gcj::JniContext* ctx, void* obj, long long id, long long timeout) { - return ctx->CacheTryEnterLock(static_cast(obj), id, timeout); - } - - void IGNITE_CALL IgniteCacheCloseLock(gcj::JniContext* ctx, void* obj, long long id) { - ctx->CacheCloseLock(static_cast(obj), id); - } - - void IGNITE_CALL IgniteCacheRebalance(gcj::JniContext* ctx, void* obj, long long futId) { - ctx->CacheRebalance(static_cast(obj), futId); - } - - int IGNITE_CALL IgniteCacheSize(gcj::JniContext* ctx, void* obj, int peekModes, bool loc) { - return ctx->CacheSize(static_cast(obj), peekModes, loc); - } - - void IGNITE_CALL IgniteComputeWithNoFailover(gcj::JniContext* ctx, void* obj) { - ctx->ComputeWithNoFailover(static_cast(obj)); - } - - void IGNITE_CALL IgniteComputeWithTimeout(gcj::JniContext* ctx, void* obj, long long timeout) { - ctx->ComputeWithTimeout(static_cast(obj), timeout); - } - - void* IGNITE_CALL IgniteComputeExecuteNative(gcj::JniContext* ctx, void* obj, long long taskPtr, long long topVer) { - return ctx->ComputeExecuteNative(static_cast(obj), taskPtr, topVer); - } - - void IGNITE_CALL IgniteContinuousQueryClose(gcj::JniContext* ctx, void* obj) { - ctx->ContinuousQueryClose(static_cast(obj)); - } - - void* IGNITE_CALL IgniteContinuousQueryGetInitialQueryCursor(gcj::JniContext* ctx, void* obj) { - return ctx->ContinuousQueryGetInitialQueryCursor(static_cast(obj)); - } - - void IGNITE_CALL IgniteCacheStoreCallbackInvoke(gcj::JniContext* ctx, void* obj, long long memPtr) { - ctx->CacheStoreCallbackInvoke(static_cast(obj), memPtr); - } - - void IGNITE_CALL IgniteDataStreamerListenTopology(gcj::JniContext* ctx, void* obj, long long ptr) { - ctx->DataStreamerListenTopology(static_cast(obj), ptr); - } - - bool IGNITE_CALL IgniteDataStreamerAllowOverwriteGet(gcj::JniContext* ctx, void* obj) { - return ctx->DataStreamerAllowOverwriteGet(static_cast(obj)); - } - - void IGNITE_CALL IgniteDataStreamerAllowOverwriteSet(gcj::JniContext* ctx, void* obj, bool val) { - ctx->DataStreamerAllowOverwriteSet(static_cast(obj), val); - } - - bool IGNITE_CALL IgniteDataStreamerSkipStoreGet(gcj::JniContext* ctx, void* obj) { - return ctx->DataStreamerSkipStoreGet(static_cast(obj)); - } - - void IGNITE_CALL IgniteDataStreamerSkipStoreSet(gcj::JniContext* ctx, void* obj, bool val) { - ctx->DataStreamerSkipStoreSet(static_cast(obj), val); - } - - int IGNITE_CALL IgniteDataStreamerPerNodeBufferSizeGet(gcj::JniContext* ctx, void* obj) { - return ctx->DataStreamerPerNodeBufferSizeGet(static_cast(obj)); - } - - void IGNITE_CALL IgniteDataStreamerPerNodeBufferSizeSet(gcj::JniContext* ctx, void* obj, int val) { - ctx->DataStreamerPerNodeBufferSizeSet(static_cast(obj), val); - } - - int IGNITE_CALL IgniteDataStreamerPerNodeParallelOperationsGet(gcj::JniContext* ctx, void* obj) { - return ctx->DataStreamerPerNodeParallelOperationsGet(static_cast(obj)); - } - - void IGNITE_CALL IgniteDataStreamerPerNodeParallelOperationsSet(gcj::JniContext* ctx, void* obj, int val) { - ctx->DataStreamerPerNodeParallelOperationsSet(static_cast(obj), val); - } - - void* IGNITE_CALL IgniteMessagingWithAsync(gcj::JniContext* ctx, void* obj) { - return ctx->MessagingWithAsync(static_cast(obj)); - } - - void* IGNITE_CALL IgniteProjectionForOthers(gcj::JniContext* ctx, void* obj, void* prj) { - return ctx->ProjectionForOthers(static_cast(obj), static_cast(prj)); - } - - void* IGNITE_CALL IgniteProjectionForRemotes(gcj::JniContext* ctx, void* obj) { - return ctx->ProjectionForRemotes(static_cast(obj)); - } - - void* IGNITE_CALL IgniteProjectionForDaemons(gcj::JniContext* ctx, void* obj) { - return ctx->ProjectionForDaemons(static_cast(obj)); - } - - void* IGNITE_CALL IgniteProjectionForRandom(gcj::JniContext* ctx, void* obj) { - return ctx->ProjectionForRandom(static_cast(obj)); - } - - void* IGNITE_CALL IgniteProjectionForOldest(gcj::JniContext* ctx, void* obj) { - return ctx->ProjectionForOldest(static_cast(obj)); - } - - void* IGNITE_CALL IgniteProjectionForYoungest(gcj::JniContext* ctx, void* obj) { - return ctx->ProjectionForYoungest(static_cast(obj)); - } - - void IGNITE_CALL IgniteProjectionResetMetrics(gcj::JniContext* ctx, void* obj) { - ctx->ProjectionResetMetrics(static_cast(obj)); - } - - void* IGNITE_CALL IgniteProjectionOutOpRet(gcj::JniContext* ctx, void* obj, int type, long long memPtr) { - return ctx->ProjectionOutOpRet(static_cast(obj), type, memPtr); - } - - void IGNITE_CALL IgniteQueryCursorIterator(gcj::JniContext* ctx, void* obj) { - ctx->QueryCursorIterator(static_cast(obj)); - } - - void IGNITE_CALL IgniteQueryCursorClose(gcj::JniContext* ctx, void* obj) { - ctx->QueryCursorClose(static_cast(obj)); - } - - long long IGNITE_CALL IgniteTransactionsStart(gcj::JniContext* ctx, void* obj, int concurrency, int isolation, long long timeout, int txSize) { - return ctx->TransactionsStart(static_cast(obj), concurrency, isolation, timeout, txSize); - } - - int IGNITE_CALL IgniteTransactionsCommit(gcj::JniContext* ctx, void* obj, long long id) { - return ctx->TransactionsCommit(static_cast(obj), id); - } - - void IGNITE_CALL IgniteTransactionsCommitAsync(gcj::JniContext* ctx, void* obj, long long id, long long futId) { - return ctx->TransactionsCommitAsync(static_cast(obj), id, futId); - } - - int IGNITE_CALL IgniteTransactionsRollback(gcj::JniContext* ctx, void* obj, long long id) { - return ctx->TransactionsRollback(static_cast(obj), id); - } - - void IGNITE_CALL IgniteTransactionsRollbackAsync(gcj::JniContext* ctx, void* obj, long long id, long long futId) { - return ctx->TransactionsRollbackAsync(static_cast(obj), id, futId); - } - - int IGNITE_CALL IgniteTransactionsClose(gcj::JniContext* ctx, void* obj, long long id) { - return ctx->TransactionsClose(static_cast(obj), id); - } - - int IGNITE_CALL IgniteTransactionsState(gcj::JniContext* ctx, void* obj, long long id) { - return ctx->TransactionsState(static_cast(obj), id); - } - - bool IGNITE_CALL IgniteTransactionsSetRollbackOnly(gcj::JniContext* ctx, void* obj, long long id) { - return ctx->TransactionsSetRollbackOnly(static_cast(obj), id); - } - - void IGNITE_CALL IgniteTransactionsResetMetrics(gcj::JniContext* ctx, void* obj) { - ctx->TransactionsResetMetrics(static_cast(obj)); - } - void* IGNITE_CALL IgniteAcquire(gcj::JniContext* ctx, void* obj) { return ctx->Acquire(static_cast(obj)); } @@ -423,130 +218,6 @@ extern "C" { ctx->DestroyJvm(); } - void* IGNITE_CALL IgniteEventsWithAsync(gcj::JniContext* ctx, void* obj) { - return ctx->EventsWithAsync(static_cast(obj)); - } - - bool IGNITE_CALL IgniteEventsStopLocalListen(gcj::JniContext* ctx, void* obj, long long hnd) { - return ctx->EventsStopLocalListen(static_cast(obj), hnd); - } - - void IGNITE_CALL IgniteEventsLocalListen(gcj::JniContext* ctx, void* obj, long long hnd, int type) { - ctx->EventsLocalListen(static_cast(obj), hnd, type); - } - - bool IGNITE_CALL IgniteEventsIsEnabled(gcj::JniContext* ctx, void* obj, int type) { - return ctx->EventsIsEnabled(static_cast(obj), type); - } - - void* IGNITE_CALL IgniteServicesWithAsync(gcj::JniContext* ctx, void* obj) { - return ctx->ServicesWithAsync(static_cast(obj)); - } - - void* IGNITE_CALL IgniteServicesWithServerKeepPortable(gcj::JniContext* ctx, void* obj) { - return ctx->ServicesWithServerKeepPortable(static_cast(obj)); - } - - void IGNITE_CALL IgniteServicesCancel(gcj::JniContext* ctx, void* obj, char* name) { - ctx->ServicesCancel(static_cast(obj), name); - } - - void IGNITE_CALL IgniteServicesCancelAll(gcj::JniContext* ctx, void* obj) { - ctx->ServicesCancelAll(static_cast(obj)); - } - - void* IGNITE_CALL IgniteServicesGetServiceProxy(gcj::JniContext* ctx, void* obj, char* name, bool sticky) { - return ctx->ServicesGetServiceProxy(static_cast(obj), name, sticky); - } - - long long IGNITE_CALL IgniteAtomicLongGet(gcj::JniContext* ctx, void* obj) { - return ctx->AtomicLongGet(static_cast(obj)); - } - - long long IGNITE_CALL IgniteAtomicLongIncrementAndGet(gcj::JniContext* ctx, void* obj) { - return ctx->AtomicLongIncrementAndGet(static_cast(obj)); - } - - long long IGNITE_CALL IgniteAtomicLongGetAndIncrement(gcj::JniContext* ctx, void* obj) { - return ctx->AtomicLongGetAndIncrement(static_cast(obj)); - } - - long long IGNITE_CALL IgniteAtomicLongAddAndGet(gcj::JniContext* ctx, void* obj, long long value) { - return ctx->AtomicLongAddAndGet(static_cast(obj), value); - } - - long long IGNITE_CALL IgniteAtomicLongGetAndAdd(gcj::JniContext* ctx, void* obj, long long value) { - return ctx->AtomicLongGetAndAdd(static_cast(obj), value); - } - - long long IGNITE_CALL IgniteAtomicLongDecrementAndGet(gcj::JniContext* ctx, void* obj) { - return ctx->AtomicLongDecrementAndGet(static_cast(obj)); - } - - long long IGNITE_CALL IgniteAtomicLongGetAndDecrement(gcj::JniContext* ctx, void* obj) { - return ctx->AtomicLongGetAndDecrement(static_cast(obj)); - } - - long long IGNITE_CALL IgniteAtomicLongGetAndSet(gcj::JniContext* ctx, void* obj, long long value) { - return ctx->AtomicLongGetAndSet(static_cast(obj), value); - } - - long long IGNITE_CALL IgniteAtomicLongCompareAndSetAndGet(gcj::JniContext* ctx, void* obj, long long expVal, long long newVal) { - return ctx->AtomicLongCompareAndSetAndGet(static_cast(obj), expVal, newVal); - } - - bool IGNITE_CALL IgniteAtomicLongIsClosed(gcj::JniContext* ctx, void* obj) { - return ctx->AtomicLongIsClosed(static_cast(obj)); - } - - void IGNITE_CALL IgniteAtomicLongClose(gcj::JniContext* ctx, void* obj) { - return ctx->AtomicLongClose(static_cast(obj)); - } - - long long IGNITE_CALL IgniteAtomicSequenceGet(gcj::JniContext* ctx, void* obj) { - return ctx->AtomicSequenceGet(static_cast(obj)); - } - - long long IGNITE_CALL IgniteAtomicSequenceIncrementAndGet(gcj::JniContext* ctx, void* obj) { - return ctx->AtomicSequenceIncrementAndGet(static_cast(obj)); - } - - long long IGNITE_CALL IgniteAtomicSequenceGetAndIncrement(gcj::JniContext* ctx, void* obj) { - return ctx->AtomicSequenceGetAndIncrement(static_cast(obj)); - } - - long long IGNITE_CALL IgniteAtomicSequenceAddAndGet(gcj::JniContext* ctx, void* obj, long long l) { - return ctx->AtomicSequenceAddAndGet(static_cast(obj), l); - } - - long long IGNITE_CALL IgniteAtomicSequenceGetAndAdd(gcj::JniContext* ctx, void* obj, long long l) { - return ctx->AtomicSequenceGetAndAdd(static_cast(obj), l); - } - - int IGNITE_CALL IgniteAtomicSequenceGetBatchSize(gcj::JniContext* ctx, void* obj) { - return ctx->AtomicSequenceGetBatchSize(static_cast(obj)); - } - - void IGNITE_CALL IgniteAtomicSequenceSetBatchSize(gcj::JniContext* ctx, void* obj, int size) { - return ctx->AtomicSequenceSetBatchSize(static_cast(obj), size); - } - - bool IGNITE_CALL IgniteAtomicSequenceIsClosed(gcj::JniContext* ctx, void* obj) { - return ctx->AtomicSequenceIsClosed(static_cast(obj)); - } - - void IGNITE_CALL IgniteAtomicSequenceClose(gcj::JniContext* ctx, void* obj) { - return ctx->AtomicSequenceClose(static_cast(obj)); - } - - bool IGNITE_CALL IgniteAtomicReferenceIsClosed(gcj::JniContext* ctx, void* obj) { - return ctx->AtomicReferenceIsClosed(static_cast(obj)); - } - - void IGNITE_CALL IgniteAtomicReferenceClose(gcj::JniContext* ctx, void* obj) { - ctx->AtomicReferenceClose(static_cast(obj)); - } - bool IGNITE_CALL IgniteListenableCancel(gcj::JniContext* ctx, void* obj) { return ctx->ListenableCancel(static_cast(obj)); } diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp index 56e042c2677c9..56fc4b13a1f23 100644 --- a/modules/platforms/cpp/jni/src/java.cpp +++ b/modules/platforms/cpp/jni/src/java.cpp @@ -214,10 +214,12 @@ namespace ignite JniMethod M_PLATFORM_PROCESSOR_GET_CACHE_NAMES = JniMethod("getCacheNames", "(J)V", false); const char* C_PLATFORM_TARGET = "org/apache/ignite/internal/processors/platform/PlatformTarget"; + JniMethod M_PLATFORM_TARGET_IN_LONG_OUT_LONG = JniMethod("inLongOutLong", "(IJ)J", false); JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_LONG = JniMethod("inStreamOutLong", "(IJ)J", false); JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_OBJECT = JniMethod("inStreamOutObject", "(IJ)Ljava/lang/Object;", false); JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_STREAM = JniMethod("inStreamOutStream", "(IJJ)V", false); JniMethod M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_STREAM = JniMethod("inObjectStreamOutStream", "(ILjava/lang/Object;JJ)V", false); + JniMethod M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_OBJECT_STREAM = JniMethod("inObjectStreamOutObjectStream", "(ILjava/lang/Object;JJ)Ljava/lang/Object;", false); JniMethod M_PLATFORM_TARGET_OUT_LONG = JniMethod("outLong", "(I)J", false); JniMethod M_PLATFORM_TARGET_OUT_STREAM = JniMethod("outStream", "(IJ)V", false); JniMethod M_PLATFORM_TARGET_OUT_OBJECT = JniMethod("outObject", "(I)Ljava/lang/Object;", false); @@ -226,72 +228,10 @@ namespace ignite JniMethod M_PLATFORM_TARGET_LISTEN_FUTURE_AND_GET = JniMethod("listenFutureAndGet", "(JI)Lorg/apache/ignite/internal/processors/platform/utils/PlatformListenable;", false); JniMethod M_PLATFORM_TARGET_LISTEN_FOR_OPERATION_AND_GET = JniMethod("listenFutureForOperationAndGet", "(JII)Lorg/apache/ignite/internal/processors/platform/utils/PlatformListenable;", false); - const char* C_PLATFORM_CLUSTER_GRP = "org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup"; - JniMethod M_PLATFORM_CLUSTER_GRP_FOR_OTHERS = JniMethod("forOthers", "(Lorg/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup;)Lorg/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup;", false); - JniMethod M_PLATFORM_CLUSTER_GRP_FOR_REMOTES = JniMethod("forRemotes", "()Lorg/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup;", false); - JniMethod M_PLATFORM_CLUSTER_GRP_FOR_DAEMONS = JniMethod("forDaemons", "()Lorg/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup;", false); - JniMethod M_PLATFORM_CLUSTER_GRP_FOR_RANDOM = JniMethod("forRandom", "()Lorg/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup;", false); - JniMethod M_PLATFORM_CLUSTER_GRP_FOR_OLDEST = JniMethod("forOldest", "()Lorg/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup;", false); - JniMethod M_PLATFORM_CLUSTER_GRP_FOR_YOUNGEST = JniMethod("forYoungest", "()Lorg/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup;", false); - JniMethod M_PLATFORM_CLUSTER_GRP_RESET_METRICS = JniMethod("resetMetrics", "()V", false); - - const char* C_PLATFORM_MESSAGING = "org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging"; - JniMethod M_PLATFORM_MESSAGING_WITH_ASYNC = JniMethod("withAsync", "()Lorg/apache/ignite/internal/processors/platform/messaging/PlatformMessaging;", false); - - const char* C_PLATFORM_COMPUTE = "org/apache/ignite/internal/processors/platform/compute/PlatformCompute"; - JniMethod M_PLATFORM_COMPUTE_WITH_NO_FAILOVER = JniMethod("withNoFailover", "()V", false); - JniMethod M_PLATFORM_COMPUTE_WITH_TIMEOUT = JniMethod("withTimeout", "(J)V", false); - JniMethod M_PLATFORM_COMPUTE_EXECUTE_NATIVE = JniMethod("executeNative", "(JJ)Lorg/apache/ignite/internal/processors/platform/utils/PlatformListenable;", false); - - const char* C_PLATFORM_CACHE = "org/apache/ignite/internal/processors/platform/cache/PlatformCache"; - JniMethod M_PLATFORM_CACHE_WITH_SKIP_STORE = JniMethod("withSkipStore", "()Lorg/apache/ignite/internal/processors/platform/cache/PlatformCache;", false); - JniMethod M_PLATFORM_CACHE_WITH_NO_RETRIES = JniMethod("withNoRetries", "()Lorg/apache/ignite/internal/processors/platform/cache/PlatformCache;", false); - JniMethod M_PLATFORM_CACHE_WITH_EXPIRY_PLC = JniMethod("withExpiryPolicy", "(JJJ)Lorg/apache/ignite/internal/processors/platform/cache/PlatformCache;", false); - JniMethod M_PLATFORM_CACHE_WITH_ASYNC = JniMethod("withAsync", "()Lorg/apache/ignite/internal/processors/platform/cache/PlatformCache;", false); - JniMethod M_PLATFORM_CACHE_WITH_KEEP_PORTABLE = JniMethod("withKeepBinary", "()Lorg/apache/ignite/internal/processors/platform/cache/PlatformCache;", false); - JniMethod M_PLATFORM_CACHE_CLEAR = JniMethod("clear", "()V", false); - JniMethod M_PLATFORM_CACHE_REMOVE_ALL = JniMethod("removeAll", "()V", false); - JniMethod M_PLATFORM_CACHE_ITERATOR = JniMethod("iterator", "()Lorg/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator;", false); - JniMethod M_PLATFORM_CACHE_LOCAL_ITERATOR = JniMethod("localIterator", "(I)Lorg/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator;", false); - JniMethod M_PLATFORM_CACHE_ENTER_LOCK = JniMethod("enterLock", "(J)V", false); - JniMethod M_PLATFORM_CACHE_EXIT_LOCK = JniMethod("exitLock", "(J)V", false); - JniMethod M_PLATFORM_CACHE_TRY_ENTER_LOCK = JniMethod("tryEnterLock", "(JJ)Z", false); - JniMethod M_PLATFORM_CACHE_CLOSE_LOCK = JniMethod("closeLock", "(J)V", false); - JniMethod M_PLATFORM_CACHE_REBALANCE = JniMethod("rebalance", "(J)V", false); - JniMethod M_PLATFORM_CACHE_SIZE = JniMethod("size", "(IZ)I", false); - - const char* C_PLATFORM_AFFINITY = "org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity"; - JniMethod C_PLATFORM_AFFINITY_PARTITIONS = JniMethod("partitions", "()I", false); - - const char* C_PLATFORM_DATA_STREAMER = "org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer"; - JniMethod M_PLATFORM_DATA_STREAMER_LISTEN_TOPOLOGY = JniMethod("listenTopology", "(J)V", false); - JniMethod M_PLATFORM_DATA_STREAMER_GET_ALLOW_OVERWRITE = JniMethod("allowOverwrite", "()Z", false); - JniMethod M_PLATFORM_DATA_STREAMER_SET_ALLOW_OVERWRITE = JniMethod("allowOverwrite", "(Z)V", false); - JniMethod M_PLATFORM_DATA_STREAMER_GET_SKIP_STORE = JniMethod("skipStore", "()Z", false); - JniMethod M_PLATFORM_DATA_STREAMER_SET_SKIP_STORE = JniMethod("skipStore", "(Z)V", false); - JniMethod M_PLATFORM_DATA_STREAMER_GET_PER_NODE_BUFFER_SIZE = JniMethod("perNodeBufferSize", "()I", false); - JniMethod M_PLATFORM_DATA_STREAMER_SET_PER_NODE_BUFFER_SIZE = JniMethod("perNodeBufferSize", "(I)V", false); - JniMethod M_PLATFORM_DATA_STREAMER_GET_PER_NODE_PARALLEL_OPS = JniMethod("perNodeParallelOperations", "()I", false); - JniMethod M_PLATFORM_DATA_STREAMER_SET_PER_NODE_PARALLEL_OPS = JniMethod("perNodeParallelOperations", "(I)V", false); - - const char* C_PLATFORM_TRANSACTIONS = "org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions"; - JniMethod M_PLATFORM_TRANSACTIONS_TX_START = JniMethod("txStart", "(IIJI)J", false); - JniMethod M_PLATFORM_TRANSACTIONS_TX_COMMIT = JniMethod("txCommit", "(J)I", false); - JniMethod M_PLATFORM_TRANSACTIONS_TX_ROLLBACK = JniMethod("txRollback", "(J)I", false); - JniMethod M_PLATFORM_TRANSACTIONS_TX_COMMIT_ASYNC = JniMethod("txCommitAsync", "(JJ)V", false); - JniMethod M_PLATFORM_TRANSACTIONS_TX_ROLLBACK_ASYNC = JniMethod("txRollbackAsync", "(JJ)V", false); - JniMethod M_PLATFORM_TRANSACTIONS_TX_STATE = JniMethod("txState", "(J)I", false); - JniMethod M_PLATFORM_TRANSACTIONS_TX_SET_ROLLBACK_ONLY = JniMethod("txSetRollbackOnly", "(J)Z", false); - JniMethod M_PLATFORM_TRANSACTIONS_TX_CLOSE = JniMethod("txClose", "(J)I", false); - JniMethod M_PLATFORM_TRANSACTIONS_RESET_METRICS = JniMethod("resetMetrics", "()V", false); - - const char* C_PLATFORM_CACHE_STORE_CALLBACK = "org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback"; - JniMethod M_PLATFORM_CACHE_STORE_CALLBACK_INVOKE = JniMethod("invoke", "(J)V", false); - const char* C_PLATFORM_CALLBACK_UTILS = "org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils"; JniMethod M_PLATFORM_CALLBACK_UTILS_CACHE_STORE_CREATE = JniMethod("cacheStoreCreate", "(JJ)J", true); - JniMethod M_PLATFORM_CALLBACK_UTILS_CACHE_STORE_INVOKE = JniMethod("cacheStoreInvoke", "(JJJLjava/lang/Object;)I", true); + JniMethod M_PLATFORM_CALLBACK_UTILS_CACHE_STORE_INVOKE = JniMethod("cacheStoreInvoke", "(JJJ)I", true); JniMethod M_PLATFORM_CALLBACK_UTILS_CACHE_STORE_DESTROY = JniMethod("cacheStoreDestroy", "(JJ)V", true); JniMethod M_PLATFORM_CALLBACK_UTILS_CACHE_STORE_SESSION_CREATE = JniMethod("cacheStoreSessionCreate", "(JJ)J", true); @@ -378,56 +318,6 @@ namespace ignite JniMethod M_PLATFORM_IGNITION_STOP = JniMethod("stop", "(Ljava/lang/String;Z)Z", true); JniMethod M_PLATFORM_IGNITION_STOP_ALL = JniMethod("stopAll", "(Z)V", true); - const char* C_PLATFORM_ABSTRACT_QRY_CURSOR = "org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor"; - JniMethod M_PLATFORM_ABSTRACT_QRY_CURSOR_ITER = JniMethod("iterator", "()V", false); - JniMethod M_PLATFORM_ABSTRACT_QRY_CURSOR_ITER_HAS_NEXT = JniMethod("iteratorHasNext", "()Z", false); - JniMethod M_PLATFORM_ABSTRACT_QRY_CURSOR_CLOSE = JniMethod("close", "()V", false); - - const char* C_PLATFORM_CONT_QRY = "org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQuery"; - JniMethod M_PLATFORM_CONT_QRY_CLOSE = JniMethod("close", "()V", false); - JniMethod M_PLATFORM_CONT_QRY_GET_INITIAL_QUERY_CURSOR = JniMethod("getInitialQueryCursor", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false); - - const char* C_PLATFORM_EVENTS = "org/apache/ignite/internal/processors/platform/events/PlatformEvents"; - JniMethod M_PLATFORM_EVENTS_WITH_ASYNC = JniMethod("withAsync", "()Lorg/apache/ignite/internal/processors/platform/events/PlatformEvents;", false); - JniMethod M_PLATFORM_EVENTS_STOP_LOCAL_LISTEN = JniMethod("stopLocalListen", "(J)Z", false); - JniMethod M_PLATFORM_EVENTS_LOCAL_LISTEN = JniMethod("localListen", "(JI)V", false); - JniMethod M_PLATFORM_EVENTS_IS_ENABLED = JniMethod("isEnabled", "(I)Z", false); - - const char* C_PLATFORM_SERVICES = "org/apache/ignite/internal/processors/platform/services/PlatformServices"; - JniMethod M_PLATFORM_SERVICES_WITH_ASYNC = JniMethod("withAsync", "()Lorg/apache/ignite/internal/processors/platform/services/PlatformServices;", false); - JniMethod M_PLATFORM_SERVICES_WITH_SERVER_KEEP_PORTABLE = JniMethod("withServerKeepBinary", "()Lorg/apache/ignite/internal/processors/platform/services/PlatformServices;", false); - JniMethod M_PLATFORM_SERVICES_CANCEL = JniMethod("cancel", "(Ljava/lang/String;)V", false); - JniMethod M_PLATFORM_SERVICES_CANCEL_ALL = JniMethod("cancelAll", "()V", false); - JniMethod M_PLATFORM_SERVICES_SERVICE_PROXY = JniMethod("serviceProxy", "(Ljava/lang/String;Z)Ljava/lang/Object;", false); - - const char* C_PLATFORM_ATOMIC_LONG = "org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong"; - JniMethod M_PLATFORM_ATOMIC_LONG_GET = JniMethod("get", "()J", false); - JniMethod M_PLATFORM_ATOMIC_LONG_INCREMENT_AND_GET = JniMethod("incrementAndGet", "()J", false); - JniMethod M_PLATFORM_ATOMIC_LONG_GET_AND_INCREMENT = JniMethod("getAndIncrement", "()J", false); - JniMethod M_PLATFORM_ATOMIC_LONG_ADD_AND_GET = JniMethod("addAndGet", "(J)J", false); - JniMethod M_PLATFORM_ATOMIC_LONG_GET_AND_ADD = JniMethod("getAndAdd", "(J)J", false); - JniMethod M_PLATFORM_ATOMIC_LONG_DECREMENT_AND_GET = JniMethod("decrementAndGet", "()J", false); - JniMethod M_PLATFORM_ATOMIC_LONG_GET_AND_DECREMENT = JniMethod("getAndDecrement", "()J", false); - JniMethod M_PLATFORM_ATOMIC_LONG_GET_AND_SET = JniMethod("getAndSet", "(J)J", false); - JniMethod M_PLATFORM_ATOMIC_LONG_COMPARE_AND_SET_AND_GET = JniMethod("compareAndSetAndGet", "(JJ)J", false); - JniMethod M_PLATFORM_ATOMIC_LONG_IS_CLOSED = JniMethod("isClosed", "()Z", false); - JniMethod M_PLATFORM_ATOMIC_LONG_CLOSE = JniMethod("close", "()V", false); - - const char* C_PLATFORM_ATOMIC_SEQUENCE = "org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence"; - JniMethod M_PLATFORM_ATOMIC_SEQUENCE_GET = JniMethod("get", "()J", false); - JniMethod M_PLATFORM_ATOMIC_SEQUENCE_INCREMENT_AND_GET = JniMethod("incrementAndGet", "()J", false); - JniMethod M_PLATFORM_ATOMIC_SEQUENCE_GET_AND_INCREMENT = JniMethod("getAndIncrement", "()J", false); - JniMethod M_PLATFORM_ATOMIC_SEQUENCE_ADD_AND_GET = JniMethod("addAndGet", "(J)J", false); - JniMethod M_PLATFORM_ATOMIC_SEQUENCE_GET_AND_ADD = JniMethod("getAndAdd", "(J)J", false); - JniMethod M_PLATFORM_ATOMIC_SEQUENCE_GET_BATCH_SIZE = JniMethod("getBatchSize", "()I", false); - JniMethod M_PLATFORM_ATOMIC_SEQUENCE_SET_BATCH_SIZE = JniMethod("setBatchSize", "(I)V", false); - JniMethod M_PLATFORM_ATOMIC_SEQUENCE_IS_CLOSED = JniMethod("isClosed", "()Z", false); - JniMethod M_PLATFORM_ATOMIC_SEQUENCE_CLOSE = JniMethod("close", "()V", false); - - const char* C_PLATFORM_ATOMIC_REFERENCE = "org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference"; - JniMethod M_PLATFORM_ATOMIC_REFERENCE_IS_CLOSED = JniMethod("isClosed", "()Z", false); - JniMethod M_PLATFORM_ATOMIC_REFERENCE_CLOSE = JniMethod("close", "()V", false); - const char* C_PLATFORM_LISTENABLE = "org/apache/ignite/internal/processors/platform/utils/PlatformListenable"; JniMethod M_PLATFORM_LISTENABLE_CANCEL = JniMethod("cancel", "()Z", false); JniMethod M_PLATFORM_LISTENABLE_IS_CANCELED = JniMethod("isCancelled", "()Z", false); @@ -584,78 +474,8 @@ namespace ignite } void JniMembers::Initialize(JNIEnv* env) { - c_PlatformAbstractQryCursor = FindClass(env, C_PLATFORM_ABSTRACT_QRY_CURSOR); - m_PlatformAbstractQryCursor_iter = FindMethod(env, c_PlatformAbstractQryCursor, M_PLATFORM_ABSTRACT_QRY_CURSOR_ITER); - m_PlatformAbstractQryCursor_iterHasNext = FindMethod(env, c_PlatformAbstractQryCursor, M_PLATFORM_ABSTRACT_QRY_CURSOR_ITER_HAS_NEXT); - m_PlatformAbstractQryCursor_close = FindMethod(env, c_PlatformAbstractQryCursor, M_PLATFORM_ABSTRACT_QRY_CURSOR_CLOSE); - - c_PlatformAffinity = FindClass(env, C_PLATFORM_AFFINITY); - m_PlatformAffinity_partitions = FindMethod(env, c_PlatformAffinity, C_PLATFORM_AFFINITY_PARTITIONS); - - c_PlatformCache = FindClass(env, C_PLATFORM_CACHE); - m_PlatformCache_withSkipStore = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_WITH_SKIP_STORE); - m_PlatformCache_withNoRetries = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_WITH_NO_RETRIES); - m_PlatformCache_withExpiryPolicy = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_WITH_EXPIRY_PLC); - m_PlatformCache_withAsync = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_WITH_ASYNC); - m_PlatformCache_withKeepPortable = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_WITH_KEEP_PORTABLE); - m_PlatformCache_clear = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_CLEAR); - m_PlatformCache_removeAll = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_REMOVE_ALL); - m_PlatformCache_iterator = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_ITERATOR); - m_PlatformCache_localIterator = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_LOCAL_ITERATOR); - m_PlatformCache_enterLock = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_ENTER_LOCK); - m_PlatformCache_exitLock = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_EXIT_LOCK); - m_PlatformCache_tryEnterLock = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_TRY_ENTER_LOCK); - m_PlatformCache_closeLock = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_CLOSE_LOCK); - m_PlatformCache_rebalance = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_REBALANCE); - m_PlatformCache_size = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_SIZE); - - c_PlatformCacheStoreCallback = FindClass(env, C_PLATFORM_CACHE_STORE_CALLBACK); - m_PlatformCacheStoreCallback_invoke = FindMethod(env, c_PlatformCacheStoreCallback, M_PLATFORM_CACHE_STORE_CALLBACK_INVOKE); - c_IgniteException = FindClass(env, C_IGNITE_EXCEPTION); - c_PlatformClusterGroup = FindClass(env, C_PLATFORM_CLUSTER_GRP); - m_PlatformClusterGroup_forOthers = FindMethod(env, c_PlatformClusterGroup, M_PLATFORM_CLUSTER_GRP_FOR_OTHERS); - m_PlatformClusterGroup_forRemotes = FindMethod(env, c_PlatformClusterGroup, M_PLATFORM_CLUSTER_GRP_FOR_REMOTES); - m_PlatformClusterGroup_forDaemons = FindMethod(env, c_PlatformClusterGroup, M_PLATFORM_CLUSTER_GRP_FOR_DAEMONS); - m_PlatformClusterGroup_forRandom = FindMethod(env, c_PlatformClusterGroup, M_PLATFORM_CLUSTER_GRP_FOR_RANDOM); - m_PlatformClusterGroup_forOldest = FindMethod(env, c_PlatformClusterGroup, M_PLATFORM_CLUSTER_GRP_FOR_OLDEST); - m_PlatformClusterGroup_forYoungest = FindMethod(env, c_PlatformClusterGroup, M_PLATFORM_CLUSTER_GRP_FOR_YOUNGEST); - m_PlatformClusterGroup_resetMetrics = FindMethod(env, c_PlatformClusterGroup, M_PLATFORM_CLUSTER_GRP_RESET_METRICS); - - c_PlatformCompute = FindClass(env, C_PLATFORM_COMPUTE); - m_PlatformCompute_withNoFailover = FindMethod(env, c_PlatformCompute, M_PLATFORM_COMPUTE_WITH_NO_FAILOVER); - m_PlatformCompute_withTimeout = FindMethod(env, c_PlatformCompute, M_PLATFORM_COMPUTE_WITH_TIMEOUT); - m_PlatformCompute_executeNative = FindMethod(env, c_PlatformCompute, M_PLATFORM_COMPUTE_EXECUTE_NATIVE); - - c_PlatformContinuousQuery = FindClass(env, C_PLATFORM_CONT_QRY); - m_PlatformContinuousQuery_close = FindMethod(env, c_PlatformContinuousQuery, M_PLATFORM_CONT_QRY_CLOSE); - m_PlatformContinuousQuery_getInitialQueryCursor = FindMethod(env, c_PlatformContinuousQuery, M_PLATFORM_CONT_QRY_GET_INITIAL_QUERY_CURSOR); - - c_PlatformDataStreamer = FindClass(env, C_PLATFORM_DATA_STREAMER); - m_PlatformDataStreamer_listenTopology = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_LISTEN_TOPOLOGY); - m_PlatformDataStreamer_getAllowOverwrite = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_GET_ALLOW_OVERWRITE); - m_PlatformDataStreamer_setAllowOverwrite = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_SET_ALLOW_OVERWRITE); - m_PlatformDataStreamer_getSkipStore = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_GET_SKIP_STORE); - m_PlatformDataStreamer_setSkipStore = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_SET_SKIP_STORE); - m_PlatformDataStreamer_getPerNodeBufSize = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_GET_PER_NODE_BUFFER_SIZE); - m_PlatformDataStreamer_setPerNodeBufSize = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_SET_PER_NODE_BUFFER_SIZE); - m_PlatformDataStreamer_getPerNodeParallelOps = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_GET_PER_NODE_PARALLEL_OPS); - m_PlatformDataStreamer_setPerNodeParallelOps = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_SET_PER_NODE_PARALLEL_OPS); - - c_PlatformEvents = FindClass(env, C_PLATFORM_EVENTS); - m_PlatformEvents_withAsync = FindMethod(env, c_PlatformEvents, M_PLATFORM_EVENTS_WITH_ASYNC); - m_PlatformEvents_stopLocalListen = FindMethod(env, c_PlatformEvents, M_PLATFORM_EVENTS_STOP_LOCAL_LISTEN); - m_PlatformEvents_localListen = FindMethod(env, c_PlatformEvents, M_PLATFORM_EVENTS_LOCAL_LISTEN); - m_PlatformEvents_isEnabled = FindMethod(env, c_PlatformEvents, M_PLATFORM_EVENTS_IS_ENABLED); - - c_PlatformServices = FindClass(env, C_PLATFORM_SERVICES); - m_PlatformServices_withAsync = FindMethod(env, c_PlatformServices, M_PLATFORM_SERVICES_WITH_ASYNC); - m_PlatformServices_withServerKeepPortable = FindMethod(env, c_PlatformServices, M_PLATFORM_SERVICES_WITH_SERVER_KEEP_PORTABLE); - m_PlatformServices_cancel = FindMethod(env, c_PlatformServices, M_PLATFORM_SERVICES_CANCEL); - m_PlatformServices_cancelAll = FindMethod(env, c_PlatformServices, M_PLATFORM_SERVICES_CANCEL_ALL); - m_PlatformServices_serviceProxy = FindMethod(env, c_PlatformServices, M_PLATFORM_SERVICES_SERVICE_PROXY); - c_PlatformIgnition = FindClass(env, C_PLATFORM_IGNITION); m_PlatformIgnition_start = FindMethod(env, c_PlatformIgnition, M_PLATFORM_IGNITION_START); m_PlatformIgnition_instance = FindMethod(env, c_PlatformIgnition, M_PLATFORM_IGNITION_INSTANCE); @@ -663,9 +483,6 @@ namespace ignite m_PlatformIgnition_stop = FindMethod(env, c_PlatformIgnition, M_PLATFORM_IGNITION_STOP); m_PlatformIgnition_stopAll = FindMethod(env, c_PlatformIgnition, M_PLATFORM_IGNITION_STOP_ALL); - c_PlatformMessaging = FindClass(env, C_PLATFORM_MESSAGING); - m_PlatformMessaging_withAsync = FindMethod(env, c_PlatformMessaging, M_PLATFORM_MESSAGING_WITH_ASYNC); - c_PlatformProcessor = FindClass(env, C_PLATFORM_PROCESSOR); m_PlatformProcessor_releaseStart = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_RELEASE_START); m_PlatformProcessor_cache = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_CACHE); @@ -692,6 +509,7 @@ namespace ignite m_PlatformProcessor_getCacheNames = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_GET_CACHE_NAMES); c_PlatformTarget = FindClass(env, C_PLATFORM_TARGET); + m_PlatformTarget_inLongOutLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_LONG_OUT_LONG); m_PlatformTarget_inStreamOutLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_LONG); m_PlatformTarget_inStreamOutObject = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_OBJECT); m_PlatformTarget_outLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_OUT_LONG); @@ -699,78 +517,29 @@ namespace ignite m_PlatformTarget_outObject = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_OUT_OBJECT); m_PlatformTarget_inStreamOutStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_STREAM); m_PlatformTarget_inObjectStreamOutStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_STREAM); + m_PlatformTarget_inObjectStreamOutObjectStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_OBJECT_STREAM); m_PlatformTarget_listenFuture = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_LISTEN_FUTURE); m_PlatformTarget_listenFutureForOperation = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_LISTEN_FOR_OPERATION); m_PlatformTarget_listenFutureAndGet = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_LISTEN_FUTURE_AND_GET); m_PlatformTarget_listenFutureForOperationAndGet = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_LISTEN_FOR_OPERATION_AND_GET); - c_PlatformTransactions = FindClass(env, C_PLATFORM_TRANSACTIONS); - m_PlatformTransactions_txStart = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_START); - m_PlatformTransactions_txCommit = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_COMMIT); - m_PlatformTransactions_txRollback = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_ROLLBACK); - m_PlatformTransactions_txCommitAsync = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_COMMIT_ASYNC); - m_PlatformTransactions_txRollbackAsync = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_ROLLBACK_ASYNC); - m_PlatformTransactions_txState = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_STATE); - m_PlatformTransactions_txSetRollbackOnly = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_SET_ROLLBACK_ONLY); - m_PlatformTransactions_txClose = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_CLOSE); - m_PlatformTransactions_resetMetrics = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_RESET_METRICS); - c_PlatformUtils = FindClass(env, C_PLATFORM_UTILS); m_PlatformUtils_reallocate = FindMethod(env, c_PlatformUtils, M_PLATFORM_UTILS_REALLOC); m_PlatformUtils_errData = FindMethod(env, c_PlatformUtils, M_PLATFORM_UTILS_ERR_DATA); - c_PlatformAtomicLong = FindClass(env, C_PLATFORM_ATOMIC_LONG); - m_PlatformAtomicLong_get = FindMethod(env, c_PlatformAtomicLong, M_PLATFORM_ATOMIC_LONG_GET); - m_PlatformAtomicLong_incrementAndGet = FindMethod(env, c_PlatformAtomicLong, M_PLATFORM_ATOMIC_LONG_INCREMENT_AND_GET); - m_PlatformAtomicLong_getAndIncrement = FindMethod(env, c_PlatformAtomicLong, M_PLATFORM_ATOMIC_LONG_GET_AND_INCREMENT); - m_PlatformAtomicLong_addAndGet = FindMethod(env, c_PlatformAtomicLong, M_PLATFORM_ATOMIC_LONG_ADD_AND_GET); - m_PlatformAtomicLong_getAndAdd = FindMethod(env, c_PlatformAtomicLong, M_PLATFORM_ATOMIC_LONG_GET_AND_ADD); - m_PlatformAtomicLong_decrementAndGet = FindMethod(env, c_PlatformAtomicLong, M_PLATFORM_ATOMIC_LONG_DECREMENT_AND_GET); - m_PlatformAtomicLong_getAndDecrement = FindMethod(env, c_PlatformAtomicLong, M_PLATFORM_ATOMIC_LONG_GET_AND_DECREMENT); - m_PlatformAtomicLong_getAndSet = FindMethod(env, c_PlatformAtomicLong, M_PLATFORM_ATOMIC_LONG_GET_AND_SET); - m_PlatformAtomicLong_compareAndSetAndGet = FindMethod(env, c_PlatformAtomicLong, M_PLATFORM_ATOMIC_LONG_COMPARE_AND_SET_AND_GET); - m_PlatformAtomicLong_isClosed = FindMethod(env, c_PlatformAtomicLong, M_PLATFORM_ATOMIC_LONG_IS_CLOSED); - m_PlatformAtomicLong_close = FindMethod(env, c_PlatformAtomicLong, M_PLATFORM_ATOMIC_LONG_CLOSE); - - jclass c_PlatformAtomicSequence = FindClass(env, C_PLATFORM_ATOMIC_SEQUENCE); - m_PlatformAtomicSequence_get = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_GET); - m_PlatformAtomicSequence_incrementAndGet = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_INCREMENT_AND_GET); - m_PlatformAtomicSequence_getAndIncrement = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_GET_AND_INCREMENT); - m_PlatformAtomicSequence_addAndGet = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_ADD_AND_GET); - m_PlatformAtomicSequence_getAndAdd = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_GET_AND_ADD); - m_PlatformAtomicSequence_getBatchSize = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_GET_BATCH_SIZE); - m_PlatformAtomicSequence_setBatchSize = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_SET_BATCH_SIZE); - m_PlatformAtomicSequence_isClosed = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_IS_CLOSED); - m_PlatformAtomicSequence_close = FindMethod(env, c_PlatformAtomicSequence, M_PLATFORM_ATOMIC_SEQUENCE_CLOSE); - - jclass c_PlatformAtomicReference = FindClass(env, C_PLATFORM_ATOMIC_REFERENCE); - m_PlatformAtomicReference_isClosed = FindMethod(env, c_PlatformAtomicReference, M_PLATFORM_ATOMIC_REFERENCE_IS_CLOSED); - m_PlatformAtomicReference_close = FindMethod(env, c_PlatformAtomicReference, M_PLATFORM_ATOMIC_REFERENCE_CLOSE); - c_PlatformListenable = FindClass(env, C_PLATFORM_LISTENABLE); m_PlatformListenable_cancel = FindMethod(env, c_PlatformListenable, M_PLATFORM_LISTENABLE_CANCEL); - m_PlatformListenable_isCancelled = FindMethod(env, c_PlatformListenable, M_PLATFORM_LISTENABLE_IS_CANCELED); + m_PlatformListenable_isCancelled = FindMethod(env, c_PlatformListenable, M_PLATFORM_LISTENABLE_IS_CANCELED); // Find utility classes which are not used from context, but are still required in other places. CheckClass(env, C_PLATFORM_NO_CALLBACK_EXCEPTION); } void JniMembers::Destroy(JNIEnv* env) { - DeleteClass(env, c_PlatformAbstractQryCursor); - DeleteClass(env, c_PlatformAffinity); - DeleteClass(env, c_PlatformCache); - DeleteClass(env, c_PlatformCacheStoreCallback); DeleteClass(env, c_IgniteException); - DeleteClass(env, c_PlatformClusterGroup); - DeleteClass(env, c_PlatformCompute); - DeleteClass(env, c_PlatformContinuousQuery); - DeleteClass(env, c_PlatformDataStreamer); - DeleteClass(env, c_PlatformEvents); DeleteClass(env, c_PlatformIgnition); - DeleteClass(env, c_PlatformMessaging); DeleteClass(env, c_PlatformProcessor); DeleteClass(env, c_PlatformTarget); - DeleteClass(env, c_PlatformTransactions); DeleteClass(env, c_PlatformUtils); } @@ -1488,6 +1257,16 @@ namespace ignite ExceptionCheck(env); } + long long JniContext::TargetInLongOutLong(jobject obj, int opType, long long val, JniErrorInfo* err) { + JNIEnv* env = Attach(); + + long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformTarget_inLongOutLong, opType, val); + + ExceptionCheck(env, err); + + return res; + } + long long JniContext::TargetInStreamOutLong(jobject obj, int opType, long long memPtr, JniErrorInfo* err) { JNIEnv* env = Attach(); @@ -1524,6 +1303,16 @@ namespace ignite ExceptionCheck(env, err); } + jobject JniContext::TargetInObjectStreamOutObjectStream(jobject obj, int opType, void* arg, long long inMemPtr, long long outMemPtr, JniErrorInfo* err) { + JNIEnv* env = Attach(); + + jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformTarget_inObjectStreamOutObjectStream, opType, arg, inMemPtr, outMemPtr); + + ExceptionCheck(env, err); + + return LocalToGlobal(env, res); + } + long long JniContext::TargetOutLong(jobject obj, int opType, JniErrorInfo* err) { JNIEnv* env = Attach(); @@ -1592,83 +1381,6 @@ namespace ignite return LocalToGlobal(env, res); } - int JniContext::AffinityPartitions(jobject obj) { - JNIEnv* env = Attach(); - - jint parts = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformAffinity_partitions); - - ExceptionCheck(env); - - return parts; - } - - jobject JniContext::CacheWithSkipStore(jobject obj) { - JNIEnv* env = Attach(); - - jobject cache = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformCache_withSkipStore); - - ExceptionCheck(env); - - return LocalToGlobal(env, cache); - } - - jobject JniContext::CacheWithNoRetries(jobject obj) { - JNIEnv* env = Attach(); - - jobject cache = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformCache_withNoRetries); - - ExceptionCheck(env); - - return LocalToGlobal(env, cache); - } - - jobject JniContext::CacheWithExpiryPolicy(jobject obj, long long create, long long update, long long access) { - JNIEnv* env = Attach(); - - jobject cache = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformCache_withExpiryPolicy, - create, update, access); - - ExceptionCheck(env); - - return LocalToGlobal(env, cache); - } - - jobject JniContext::CacheWithAsync(jobject obj) { - JNIEnv* env = Attach(); - - jobject cache = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformCache_withAsync); - - ExceptionCheck(env); - - return LocalToGlobal(env, cache); - } - - jobject JniContext::CacheWithKeepPortable(jobject obj) { - JNIEnv* env = Attach(); - - jobject cache = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformCache_withKeepPortable); - - ExceptionCheck(env); - - return LocalToGlobal(env, cache); - } - - void JniContext::CacheClear(jobject obj, JniErrorInfo* err) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCache_clear); - - ExceptionCheck(env, err); - } - - void JniContext::CacheRemoveAll(jobject obj, JniErrorInfo* err) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCache_removeAll); - - ExceptionCheck(env, err); - } - jobject JniContext::CacheOutOpQueryCursor(jobject obj, int type, long long memPtr, JniErrorInfo* err) { JNIEnv* env = Attach(); @@ -1691,932 +1403,179 @@ namespace ignite return LocalToGlobal(env, res); } - jobject JniContext::CacheIterator(jobject obj) { - JNIEnv* env = Attach(); - - jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformCache_iterator); - - ExceptionCheck(env); - - return LocalToGlobal(env, res); - } - - jobject JniContext::CacheLocalIterator(jobject obj, int peekModes) { - JNIEnv*env = Attach(); - - jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformCache_localIterator, peekModes); - - ExceptionCheck(env); - - return LocalToGlobal(env, res); - } - - void JniContext::CacheEnterLock(jobject obj, long long id) { + bool JniContext::ListenableCancel(jobject obj) + { JNIEnv* env = Attach(); - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCache_enterLock, id); + jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformListenable_cancel); ExceptionCheck(env); - } - - void JniContext::CacheExitLock(jobject obj, long long id) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCache_exitLock, id); - ExceptionCheck(env); + return res != 0;; } - bool JniContext::CacheTryEnterLock(jobject obj, long long id, long long timeout) { + bool JniContext::ListenableIsCancelled(jobject obj) + { JNIEnv* env = Attach(); - jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformCache_tryEnterLock, id, timeout); + jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformListenable_isCancelled); ExceptionCheck(env); - return res != 0; + return res != 0;; } - void JniContext::CacheCloseLock(jobject obj, long long id) { - JNIEnv* env = Attach(); + jobject JniContext::Acquire(jobject obj) + { + if (obj) { - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCache_closeLock, id); + JNIEnv* env = Attach(); - ExceptionCheck(env); - } + jobject obj0 = env->NewGlobalRef(obj); - void JniContext::CacheRebalance(jobject obj, long long futId) { - JNIEnv* env = Attach(); + ExceptionCheck(env); - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCache_rebalance, futId); + return obj0; + } - ExceptionCheck(env); + return NULL; } - int JniContext::CacheSize(jobject obj, int peekModes, bool loc, JniErrorInfo* err) { - JNIEnv* env = Attach(); - - jint res = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformCache_size, peekModes, loc); - - ExceptionCheck(env, err); + void JniContext::Release(jobject obj) { + if (obj) + { + JavaVM* jvm = JVM.GetJvm(); - return res; - } + if (jvm) + { + JNIEnv* env; - void JniContext::CacheStoreCallbackInvoke(jobject obj, long long memPtr) { - JNIEnv* env = Attach(); + jint attachRes = jvm->AttachCurrentThread(reinterpret_cast(&env), NULL); - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCacheStoreCallback_invoke, memPtr); + if (attachRes == JNI_OK) + { + AttachHelper::OnThreadAttach(); - ExceptionCheck(env); + env->DeleteGlobalRef(obj); + } + } + } } - void JniContext::ComputeWithNoFailover(jobject obj) { + void JniContext::ThrowToJava(char* msg) { JNIEnv* env = Attach(); - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCompute_withNoFailover); - - ExceptionCheck(env); + env->ThrowNew(jvm->GetMembers().c_IgniteException, msg); } - void JniContext::ComputeWithTimeout(jobject obj, long long timeout) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCompute_withTimeout, timeout); - - ExceptionCheck(env); + void JniContext::DestroyJvm() { + jvm->GetJvm()->DestroyJavaVM(); } - void* JniContext::ComputeExecuteNative(jobject obj, long long taskPtr, long long topVer) { - JNIEnv* env = Attach(); + /** + * Attach thread to JVM. + */ + JNIEnv* JniContext::Attach() { + JNIEnv* env; - jobject res = env->CallObjectMethod(obj, - jvm->GetMembers().m_PlatformCompute_executeNative, taskPtr, topVer); + jint attachRes = jvm->GetJvm()->AttachCurrentThread(reinterpret_cast(&env), NULL); - ExceptionCheck(env); + if (attachRes == JNI_OK) + AttachHelper::OnThreadAttach(); + else { + if (hnds.error) + hnds.error(hnds.target, IGNITE_JNI_ERR_JVM_ATTACH, NULL, 0, NULL, 0, NULL, 0); + } - return LocalToGlobal(env, res); + return env; } - void JniContext::ContinuousQueryClose(jobject obj) { - JNIEnv* env = Attach(); + void JniContext::ExceptionCheck(JNIEnv* env) { + ExceptionCheck(env, NULL); + } - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformContinuousQuery_close); + void JniContext::ExceptionCheck(JNIEnv* env, JniErrorInfo* errInfo) + { + if (env->ExceptionCheck()) { + jthrowable err = env->ExceptionOccurred(); - ExceptionCheck(env); - } + if (PRINT_EXCEPTION) + env->CallVoidMethod(err, jvm->GetJavaMembers().m_Throwable_printStackTrace); - jobject JniContext::ContinuousQueryGetInitialQueryCursor(jobject obj) { - JNIEnv* env = Attach(); + env->ExceptionClear(); - jobject res = env->CallObjectMethod(obj, - jvm->GetMembers().m_PlatformContinuousQuery_getInitialQueryCursor); + // Get error class name and message. + jclass cls = env->GetObjectClass(err); - ExceptionCheck(env); + jstring clsName = static_cast(env->CallObjectMethod(cls, jvm->GetJavaMembers().m_Class_getName)); + jstring msg = static_cast(env->CallObjectMethod(err, jvm->GetJavaMembers().m_Throwable_getMessage)); - return LocalToGlobal(env, res); - } + env->DeleteLocalRef(cls); - void JniContext::DataStreamerListenTopology(jobject obj, long long ptr) { - JNIEnv* env = Attach(); + int clsNameLen; + std::string clsName0 = JavaStringToCString(env, clsName, &clsNameLen); - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_listenTopology, ptr); + int msgLen; + std::string msg0 = JavaStringToCString(env, msg, &msgLen); - ExceptionCheck(env); - } + if (errInfo) + { + JniErrorInfo errInfo0(IGNITE_JNI_ERR_GENERIC, clsName0.c_str(), msg0.c_str()); - bool JniContext::DataStreamerAllowOverwriteGet(jobject obj) { - JNIEnv* env = Attach(); + *errInfo = errInfo0; + } - jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_getAllowOverwrite); + // Get error additional data (if any). + jbyteArray errData = static_cast(env->CallStaticObjectMethod( + jvm->GetMembers().c_PlatformUtils, jvm->GetMembers().m_PlatformUtils_errData, err)); - ExceptionCheck(env); + if (errData) + { + jbyte* errBytesNative = env->GetByteArrayElements(errData, NULL); - return res != 0; - } + int errBytesLen = env->GetArrayLength(errData); - void JniContext::DataStreamerAllowOverwriteSet(jobject obj, bool val) { - JNIEnv* env = Attach(); + if (hnds.error) + hnds.error(hnds.target, IGNITE_JNI_ERR_GENERIC, clsName0.c_str(), clsNameLen, msg0.c_str(), msgLen, + errBytesNative, errBytesLen); - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_setAllowOverwrite, val); + env->ReleaseByteArrayElements(errData, errBytesNative, JNI_ABORT); + } + else + { + if (hnds.error) + hnds.error(hnds.target, IGNITE_JNI_ERR_GENERIC, clsName0.c_str(), clsNameLen, msg0.c_str(), msgLen, + NULL, 0); + } - ExceptionCheck(env); + env->DeleteLocalRef(err); + } } - bool JniContext::DataStreamerSkipStoreGet(jobject obj) { - JNIEnv* env = Attach(); + /** + * Convert local reference to global. + */ + jobject JniContext::LocalToGlobal(JNIEnv* env, jobject localRef) { + if (localRef) { + jobject globalRef = env->NewGlobalRef(localRef); - jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_getSkipStore); + env->DeleteLocalRef(localRef); // Clear local ref irrespective of result. - ExceptionCheck(env); + if (!globalRef) + ExceptionCheck(env); - return res != 0; + return globalRef; + } + else + return NULL; } - void JniContext::DataStreamerSkipStoreSet(jobject obj, bool val) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_setSkipStore, val); + JNIEXPORT jlong JNICALL JniCacheStoreCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr) { + IGNITE_SAFE_FUNC(env, envPtr, CacheStoreCreateHandler, cacheStoreCreate, memPtr); + } - ExceptionCheck(env); - } - - int JniContext::DataStreamerPerNodeBufferSizeGet(jobject obj) { - JNIEnv* env = Attach(); - - jint res = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_getPerNodeBufSize); - - ExceptionCheck(env); - - return res; - } - - void JniContext::DataStreamerPerNodeBufferSizeSet(jobject obj, int val) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_setPerNodeBufSize, val); - - ExceptionCheck(env); - } - - int JniContext::DataStreamerPerNodeParallelOperationsGet(jobject obj) { - JNIEnv* env = Attach(); - - jint res = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_getPerNodeParallelOps); - - ExceptionCheck(env); - - return res; - } - - void JniContext::DataStreamerPerNodeParallelOperationsSet(jobject obj, int val) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_setPerNodeParallelOps, val); - - ExceptionCheck(env); - } - - jobject JniContext::MessagingWithAsync(jobject obj) { - JNIEnv* env = Attach(); - - jobject msg = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformMessaging_withAsync); - - ExceptionCheck(env); - - return LocalToGlobal(env, msg); - } - - jobject JniContext::ProjectionForOthers(jobject obj, jobject prj) { - JNIEnv* env = Attach(); - - jobject newPrj = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformClusterGroup_forOthers, prj); - - ExceptionCheck(env); - - return LocalToGlobal(env, newPrj); - } - - jobject JniContext::ProjectionForRemotes(jobject obj) { - JNIEnv* env = Attach(); - - jobject newPrj = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformClusterGroup_forRemotes); - - ExceptionCheck(env); - - return LocalToGlobal(env, newPrj); - } - - jobject JniContext::ProjectionForDaemons(jobject obj) { - JNIEnv* env = Attach(); - - jobject newPrj = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformClusterGroup_forDaemons); - - ExceptionCheck(env); - - return LocalToGlobal(env, newPrj); - } - - jobject JniContext::ProjectionForRandom(jobject obj) { - JNIEnv* env = Attach(); - - jobject newPrj = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformClusterGroup_forRandom); - - ExceptionCheck(env); - - return LocalToGlobal(env, newPrj); - } - - jobject JniContext::ProjectionForOldest(jobject obj) { - JNIEnv* env = Attach(); - - jobject newPrj = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformClusterGroup_forOldest); - - ExceptionCheck(env); - - return LocalToGlobal(env, newPrj); - } - - jobject JniContext::ProjectionForYoungest(jobject obj) { - JNIEnv* env = Attach(); - - jobject newPrj = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformClusterGroup_forYoungest); - - ExceptionCheck(env); - - return LocalToGlobal(env, newPrj); - } - - void JniContext::ProjectionResetMetrics(jobject obj) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformClusterGroup_resetMetrics); - - ExceptionCheck(env); - } - - jobject JniContext::ProjectionOutOpRet(jobject obj, int type, long long memPtr) { - JNIEnv* env = Attach(); - - jobject res = env->CallObjectMethod( - obj, jvm->GetMembers().m_PlatformTarget_inStreamOutObject, type, memPtr); - - ExceptionCheck(env); - - return LocalToGlobal(env, res); - } - - - void JniContext::QueryCursorIterator(jobject obj, JniErrorInfo* errInfo) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformAbstractQryCursor_iter); - - ExceptionCheck(env, errInfo); - } - - bool JniContext::QueryCursorIteratorHasNext(jobject obj, JniErrorInfo* errInfo) - { - JNIEnv* env = Attach(); - - jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformAbstractQryCursor_iterHasNext); - - ExceptionCheck(env, errInfo); - - return res != 0; - } - - void JniContext::QueryCursorClose(jobject obj, JniErrorInfo* errInfo) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformAbstractQryCursor_close); - - ExceptionCheck(env, errInfo); - } - - long long JniContext::TransactionsStart(jobject obj, int concurrency, - int isolation, long long timeout, int txSize, JniErrorInfo* errInfo) { - JNIEnv* env = Attach(); - - long long id = env->CallLongMethod(obj, - jvm->GetMembers().m_PlatformTransactions_txStart, - concurrency, isolation, timeout, txSize); - - ExceptionCheck(env, errInfo); - - return id; - } - - int JniContext::TransactionsCommit(jobject obj, long long id, JniErrorInfo* errInfo) { - JNIEnv* env = Attach(); - - int res = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformTransactions_txCommit, id); - - ExceptionCheck(env, errInfo); - - return res; - } - - void JniContext::TransactionsCommitAsync(jobject obj, long long id, long long futId) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformTransactions_txCommitAsync, id, futId); - - ExceptionCheck(env); - } - - int JniContext::TransactionsRollback(jobject obj, long long id, JniErrorInfo* errInfo) { - JNIEnv* env = Attach(); - - int res = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformTransactions_txRollback, id); - - ExceptionCheck(env, errInfo); - - return res; - } - - void JniContext::TransactionsRollbackAsync(jobject obj, long long id, long long futId) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformTransactions_txRollbackAsync, id, futId); - - ExceptionCheck(env); - } - - int JniContext::TransactionsClose(jobject obj, long long id, JniErrorInfo* errInfo) { - JNIEnv* env = Attach(); - - jint state = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformTransactions_txClose, id); - - ExceptionCheck(env, errInfo); - - return state; - } - - int JniContext::TransactionsState(jobject obj, long long id, JniErrorInfo* errInfo) { - JNIEnv* env = Attach(); - - jint state = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformTransactions_txState, id); - - ExceptionCheck(env, errInfo); - - return state; - } - - bool JniContext::TransactionsSetRollbackOnly(jobject obj, long long id, JniErrorInfo* errInfo) { - JNIEnv* env = Attach(); - - jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformTransactions_txSetRollbackOnly, id); - - ExceptionCheck(env, errInfo); - - return res != 0; - } - - void JniContext::TransactionsResetMetrics(jobject obj) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformTransactions_resetMetrics); - - ExceptionCheck(env); - } - - jobject JniContext::EventsWithAsync(jobject obj) { - JNIEnv * env = Attach(); - - jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformEvents_withAsync); - - ExceptionCheck(env); - - return LocalToGlobal(env, res); - } - - bool JniContext::EventsStopLocalListen(jobject obj, long long hnd) { - JNIEnv * env = Attach(); - - jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformEvents_stopLocalListen, hnd); - - ExceptionCheck(env); - - return res != 0; - } - - void JniContext::EventsLocalListen(jobject obj, long long hnd, int type) { - JNIEnv * env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformEvents_localListen, hnd, type); - - ExceptionCheck(env); - } - - bool JniContext::EventsIsEnabled(jobject obj, int type) { - JNIEnv * env = Attach(); - - jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformEvents_isEnabled, type); - - ExceptionCheck(env); - - return res != 0; - } - - jobject JniContext::ServicesWithAsync(jobject obj) { - JNIEnv* env = Attach(); - - jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformServices_withAsync); - - ExceptionCheck(env); - - return LocalToGlobal(env, res); - } - - jobject JniContext::ServicesWithServerKeepPortable(jobject obj) { - JNIEnv* env = Attach(); - - jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformServices_withServerKeepPortable); - - ExceptionCheck(env); - - return LocalToGlobal(env, res); - } - - void JniContext::ServicesCancel(jobject obj, char* name) { - JNIEnv* env = Attach(); - - jstring name0 = name != NULL ? env->NewStringUTF(name) : NULL; - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformServices_cancel, name0); - - if (name0) - env->DeleteLocalRef(name0); - - ExceptionCheck(env); - } - - void JniContext::ServicesCancelAll(jobject obj) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformServices_cancelAll); - - ExceptionCheck(env); - } - - void* JniContext::ServicesGetServiceProxy(jobject obj, char* name, bool sticky) { - JNIEnv* env = Attach(); - - jstring name0 = name != NULL ? env->NewStringUTF(name) : NULL; - - jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformServices_serviceProxy, name0, sticky); - - if (name0) - env->DeleteLocalRef(name0); - - ExceptionCheck(env); - - return LocalToGlobal(env, res);; - } - - long long JniContext::AtomicLongGet(jobject obj) - { - JNIEnv* env = Attach(); - - long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicLong_get); - - ExceptionCheck(env); - - return res; - } - - long long JniContext::AtomicLongIncrementAndGet(jobject obj) - { - JNIEnv* env = Attach(); - - long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicLong_incrementAndGet); - - ExceptionCheck(env); - - return res; - } - - long long JniContext::AtomicLongGetAndIncrement(jobject obj) - { - JNIEnv* env = Attach(); - - long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicLong_getAndIncrement); - - ExceptionCheck(env); - - return res; - } - - long long JniContext::AtomicLongAddAndGet(jobject obj, long long value) - { - JNIEnv* env = Attach(); - - long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicLong_addAndGet, value); - - ExceptionCheck(env); - - return res; - } - - long long JniContext::AtomicLongGetAndAdd(jobject obj, long long value) - { - JNIEnv* env = Attach(); - - long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicLong_getAndAdd, value); - - ExceptionCheck(env); - - return res; - } - - long long JniContext::AtomicLongDecrementAndGet(jobject obj) - { - JNIEnv* env = Attach(); - - long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicLong_decrementAndGet); - - ExceptionCheck(env); - - return res; - } - - long long JniContext::AtomicLongGetAndDecrement(jobject obj) - { - JNIEnv* env = Attach(); - - long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicLong_getAndDecrement); - - ExceptionCheck(env); - - return res; - } - - long long JniContext::AtomicLongGetAndSet(jobject obj, long long value) - { - JNIEnv* env = Attach(); - - long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicLong_getAndSet, value); - - ExceptionCheck(env); - - return res; - } - - long long JniContext::AtomicLongCompareAndSetAndGet(jobject obj, long long expVal, long long newVal) - { - JNIEnv* env = Attach(); - - long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicLong_compareAndSetAndGet, expVal, newVal); - - ExceptionCheck(env); - - return res; - } - - bool JniContext::AtomicLongIsClosed(jobject obj) - { - JNIEnv* env = Attach(); - - jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformAtomicLong_isClosed); - - ExceptionCheck(env); - - return res != 0;; - } - - void JniContext::AtomicLongClose(jobject obj) - { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformAtomicLong_close); - - ExceptionCheck(env); - } - - long long JniContext::AtomicSequenceGet(jobject obj) - { - JNIEnv* env = Attach(); - - long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_get); - - ExceptionCheck(env); - - return res; - } - - long long JniContext::AtomicSequenceIncrementAndGet(jobject obj) - { - JNIEnv* env = Attach(); - - long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_incrementAndGet); - - ExceptionCheck(env); - - return res; - } - - long long JniContext::AtomicSequenceGetAndIncrement(jobject obj) - { - JNIEnv* env = Attach(); - - long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_getAndIncrement); - - ExceptionCheck(env); - - return res; - } - - long long JniContext::AtomicSequenceAddAndGet(jobject obj, long long l) - { - JNIEnv* env = Attach(); - - long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_addAndGet, l); - - ExceptionCheck(env); - - return res; - } - - long long JniContext::AtomicSequenceGetAndAdd(jobject obj, long long l) - { - JNIEnv* env = Attach(); - - long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_getAndAdd, l); - - ExceptionCheck(env); - - return res; - } - - int JniContext::AtomicSequenceGetBatchSize(jobject obj) - { - JNIEnv* env = Attach(); - - int res = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_getBatchSize); - - ExceptionCheck(env); - - return res; - } - - void JniContext::AtomicSequenceSetBatchSize(jobject obj, int size) - { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_setBatchSize, size); - - ExceptionCheck(env); - } - - bool JniContext::AtomicSequenceIsClosed(jobject obj) - { - JNIEnv* env = Attach(); - - jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_isClosed); - - ExceptionCheck(env); - - return res != 0; - } - - void JniContext::AtomicSequenceClose(jobject obj) - { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformAtomicSequence_close); - - ExceptionCheck(env); - } - - bool JniContext::AtomicReferenceIsClosed(jobject obj) - { - JNIEnv* env = Attach(); - - jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformAtomicReference_isClosed); - - ExceptionCheck(env); - - return res != 0; - } - - void JniContext::AtomicReferenceClose(jobject obj) - { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformAtomicReference_close); - - ExceptionCheck(env); - } - - bool JniContext::ListenableCancel(jobject obj) - { - JNIEnv* env = Attach(); - - jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformListenable_cancel); - - ExceptionCheck(env); - - return res != 0;; - } - - bool JniContext::ListenableIsCancelled(jobject obj) - { - JNIEnv* env = Attach(); - - jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformListenable_isCancelled); - - ExceptionCheck(env); - - return res != 0;; - } - - jobject JniContext::Acquire(jobject obj) - { - if (obj) { - - JNIEnv* env = Attach(); - - jobject obj0 = env->NewGlobalRef(obj); - - ExceptionCheck(env); - - return obj0; - } - - return NULL; - } - - void JniContext::Release(jobject obj) { - if (obj) - { - JavaVM* jvm = JVM.GetJvm(); - - if (jvm) - { - JNIEnv* env; - - jint attachRes = jvm->AttachCurrentThread(reinterpret_cast(&env), NULL); - - if (attachRes == JNI_OK) - { - AttachHelper::OnThreadAttach(); - - env->DeleteGlobalRef(obj); - } - } - } - } - - void JniContext::ThrowToJava(char* msg) { - JNIEnv* env = Attach(); - - env->ThrowNew(jvm->GetMembers().c_IgniteException, msg); - } - - void JniContext::DestroyJvm() { - jvm->GetJvm()->DestroyJavaVM(); - } - - /** - * Attach thread to JVM. - */ - JNIEnv* JniContext::Attach() { - JNIEnv* env; - - jint attachRes = jvm->GetJvm()->AttachCurrentThread(reinterpret_cast(&env), NULL); - - if (attachRes == JNI_OK) - AttachHelper::OnThreadAttach(); - else { - if (hnds.error) - hnds.error(hnds.target, IGNITE_JNI_ERR_JVM_ATTACH, NULL, 0, NULL, 0, NULL, 0); - } - - return env; - } - - void JniContext::ExceptionCheck(JNIEnv* env) { - ExceptionCheck(env, NULL); - } - - void JniContext::ExceptionCheck(JNIEnv* env, JniErrorInfo* errInfo) - { - if (env->ExceptionCheck()) { - jthrowable err = env->ExceptionOccurred(); - - if (PRINT_EXCEPTION) - env->CallVoidMethod(err, jvm->GetJavaMembers().m_Throwable_printStackTrace); - - env->ExceptionClear(); - - // Get error class name and message. - jclass cls = env->GetObjectClass(err); - - jstring clsName = static_cast(env->CallObjectMethod(cls, jvm->GetJavaMembers().m_Class_getName)); - jstring msg = static_cast(env->CallObjectMethod(err, jvm->GetJavaMembers().m_Throwable_getMessage)); - - env->DeleteLocalRef(cls); - - int clsNameLen; - std::string clsName0 = JavaStringToCString(env, clsName, &clsNameLen); - - int msgLen; - std::string msg0 = JavaStringToCString(env, msg, &msgLen); - - if (errInfo) - { - JniErrorInfo errInfo0(IGNITE_JNI_ERR_GENERIC, clsName0.c_str(), msg0.c_str()); - - *errInfo = errInfo0; - } - - // Get error additional data (if any). - jbyteArray errData = static_cast(env->CallStaticObjectMethod( - jvm->GetMembers().c_PlatformUtils, jvm->GetMembers().m_PlatformUtils_errData, err)); - - if (errData) - { - jbyte* errBytesNative = env->GetByteArrayElements(errData, NULL); - - int errBytesLen = env->GetArrayLength(errData); - - if (hnds.error) - hnds.error(hnds.target, IGNITE_JNI_ERR_GENERIC, clsName0.c_str(), clsNameLen, msg0.c_str(), msgLen, - errBytesNative, errBytesLen); - - env->ReleaseByteArrayElements(errData, errBytesNative, JNI_ABORT); - } - else - { - if (hnds.error) - hnds.error(hnds.target, IGNITE_JNI_ERR_GENERIC, clsName0.c_str(), clsNameLen, msg0.c_str(), msgLen, - NULL, 0); - } - - env->DeleteLocalRef(err); - } - } - - /** - * Convert local reference to global. - */ - jobject JniContext::LocalToGlobal(JNIEnv* env, jobject localRef) { - if (localRef) { - jobject globalRef = env->NewGlobalRef(localRef); - - env->DeleteLocalRef(localRef); // Clear local ref irrespective of result. - - if (!globalRef) - ExceptionCheck(env); - - return globalRef; - } - else - return NULL; - } - - JNIEXPORT jlong JNICALL JniCacheStoreCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr) { - IGNITE_SAFE_FUNC(env, envPtr, CacheStoreCreateHandler, cacheStoreCreate, memPtr); - } - - JNIEXPORT jint JNICALL JniCacheStoreInvoke(JNIEnv *env, jclass cls, jlong envPtr, jlong objPtr, jlong memPtr, jobject cb) { - if (cb) - { - jobject cb0 = env->NewGlobalRef(cb); - - if (cb0) - { - JniGlobalRefGuard guard(env, cb0); - - IGNITE_SAFE_FUNC(env, envPtr, CacheStoreInvokeHandler, cacheStoreInvoke, objPtr, memPtr, cb0); - } - else - return 0; - } - else { - IGNITE_SAFE_FUNC(env, envPtr, CacheStoreInvokeHandler, cacheStoreInvoke, objPtr, memPtr, NULL); - } + JNIEXPORT jint JNICALL JniCacheStoreInvoke(JNIEnv *env, jclass cls, jlong envPtr, jlong objPtr, jlong memPtr) { + IGNITE_SAFE_FUNC(env, envPtr, CacheStoreInvokeHandler, cacheStoreInvoke, objPtr, memPtr); } JNIEXPORT void JNICALL JniCacheStoreDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong objPtr) { diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs index 62dc2df5f78c0..e8448eba231f6 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs @@ -1889,6 +1889,50 @@ public void TestAsyncMultithreadedKeepBinary() }, threads); } + /// + /// Simple cache lock test (while is ignored). + /// + [Test] + public void TestLockSimple() + { + if (!LockingEnabled()) + return; + + var cache = Cache(); + + const int key = 7; + + Action checkLock = lck => + { + using (lck) + { + Assert.Throws(lck.Exit); // can't exit if not entered + + lck.Enter(); + + Assert.IsTrue(cache.IsLocalLocked(key, true)); + Assert.IsTrue(cache.IsLocalLocked(key, false)); + + lck.Exit(); + + Assert.IsFalse(cache.IsLocalLocked(key, true)); + Assert.IsFalse(cache.IsLocalLocked(key, false)); + + Assert.IsTrue(lck.TryEnter()); + + Assert.IsTrue(cache.IsLocalLocked(key, true)); + Assert.IsTrue(cache.IsLocalLocked(key, false)); + + lck.Exit(); + } + + Assert.Throws(lck.Enter); // Can't enter disposed lock + }; + + checkLock(cache.Lock(key)); + checkLock(cache.LockAll(new[] { key, 1, 2, 3 })); + } + [Test] [Ignore("IGNITE-835")] public void TestLock() diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs index 32696514849e7..2d38c723af634 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs @@ -322,6 +322,7 @@ public void TestDuckTyping([Values(true, false)] bool local) // .. but setter does not var ex = Assert.Throws(() => { prx.TestProperty = new object(); }); + Assert.IsNotNull(ex.InnerException); Assert.AreEqual("Object of type 'System.Object' cannot be converted to type 'System.Int32'.", ex.InnerException.Message); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj index 823f9f2dd3acc..667f807be0f83 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj @@ -166,6 +166,7 @@ + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs index 00e13c5cffd65..4ff1f2e5281c1 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs @@ -75,6 +75,9 @@ internal class CacheAffinityImpl : PlatformTarget, ICacheAffinity /** */ private const int OpPrimaryPartitions = 14; + /** */ + private const int OpPartitions = 15; + /** */ private readonly bool _keepBinary; @@ -101,7 +104,7 @@ internal class CacheAffinityImpl : PlatformTarget, ICacheAffinity /** */ public int Partitions { - get { return UU.AffinityPartitions(Target); } + get { return (int) DoOutOp(OpPartitions); } } /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs index 8a25a2cfc4f64..3a82932bf9f54 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs @@ -41,7 +41,7 @@ namespace Apache.Ignite.Core.Impl.Cache /// Native cache wrapper. /// [SuppressMessage("Microsoft.Design", "CA1001:TypesThatOwnDisposableFieldsShouldBeDisposable")] - internal class CacheImpl : PlatformTarget, ICache, ICacheInternal + internal class CacheImpl : PlatformTarget, ICache, ICacheInternal, ICacheLockInternal { /** Duration: unchanged. */ private const long DurUnchanged = -2; @@ -89,20 +89,18 @@ internal class CacheImpl : PlatformTarget, ICache, ICacheInterna _flagAsync = flagAsync; _flagNoRetries = flagNoRetries; - _asyncInstance = new Lazy>(() => new CacheImpl(this)); + _asyncInstance = new Lazy>(WithAsync); } /// - /// Initializes a new async instance. + /// Returns an instance with async mode enabled. /// - /// The cache. - private CacheImpl(CacheImpl cache) : base(UU.CacheWithAsync(cache.Target), cache.Marshaller) + private CacheImpl WithAsync() { - _ignite = cache._ignite; - _flagSkipStore = cache._flagSkipStore; - _flagKeepBinary = cache._flagKeepBinary; - _flagAsync = true; - _flagNoRetries = cache._flagNoRetries; + var target = DoOutOpObject((int) CacheOp.WithAsync); + + return new CacheImpl(_ignite, target, Marshaller, _flagSkipStore, _flagKeepBinary, + true, _flagNoRetries); } /** */ @@ -170,7 +168,7 @@ public bool IsEmpty() if (_flagSkipStore) return this; - return new CacheImpl(_ignite, UU.CacheWithSkipStore(Target), Marshaller, + return new CacheImpl(_ignite, DoOutOpObject((int) CacheOp.WithSkipStore), Marshaller, true, _flagKeepBinary, _flagAsync, true); } @@ -194,7 +192,7 @@ public bool IsEmpty() return result; } - return new CacheImpl(_ignite, UU.CacheWithKeepBinary(Target), Marshaller, + return new CacheImpl(_ignite, DoOutOpObject((int) CacheOp.WithKeepBinary), Marshaller, _flagSkipStore, true, _flagAsync, _flagNoRetries); } @@ -207,7 +205,12 @@ public bool IsEmpty() long update = ConvertDuration(plc.GetExpiryForUpdate()); long access = ConvertDuration(plc.GetExpiryForAccess()); - IUnmanagedTarget cache0 = UU.CacheWithExpiryPolicy(Target, create, update, access); + IUnmanagedTarget cache0 = DoOutOpObject((int)CacheOp.WithExpiryPolicy, w => + { + w.WriteLong(create); + w.WriteLong(update); + w.WriteLong(access); + }); return new CacheImpl(_ignite, cache0, Marshaller, _flagSkipStore, _flagKeepBinary, _flagAsync, _flagNoRetries); } @@ -637,7 +640,7 @@ public void LocalEvict(IEnumerable keys) /** */ public void Clear() { - UU.CacheClear(Target); + DoOutOp((int) CacheOp.ClearCache); } /** */ @@ -749,7 +752,7 @@ public Task RemoveAllAsync(IEnumerable keys) /** */ public void RemoveAll() { - UU.CacheRemoveAll(Target); + DoOutOp((int) CacheOp.RemoveAll2); } /** */ @@ -788,9 +791,11 @@ public Task GetSizeAsync(params CachePeekMode[] modes) /// Size. private int Size0(bool loc, params CachePeekMode[] modes) { - int modes0 = EncodePeekModes(modes); + var modes0 = EncodePeekModes(modes); - return UU.CacheSize(Target, modes0, loc); + var op = loc ? CacheOp.SizeLoc : CacheOp.Size; + + return (int) DoOutInOpLong((int) op, modes0); } /** */ @@ -871,10 +876,10 @@ public ICacheLock Lock(TK key) { IgniteArgumentCheck.NotNull(key, "key"); - return DoOutInOp((int)CacheOp.Lock, writer => + return DoOutInOp((int) CacheOp.Lock, writer => { writer.Write(key); - }, input => new CacheLock(input.ReadInt(), Target)); + }, input => new CacheLock(input.ReadInt(), this)); } /** */ @@ -882,10 +887,10 @@ public ICacheLock LockAll(IEnumerable keys) { IgniteArgumentCheck.NotNull(keys, "keys"); - return DoOutInOp((int)CacheOp.LockAll, writer => + return DoOutInOp((int) CacheOp.LockAll, writer => { WriteEnumerable(writer, keys); - }, input => new CacheLock(input.ReadInt(), Target)); + }, input => new CacheLock(input.ReadInt(), this)); } /** */ @@ -914,7 +919,7 @@ public ICacheMetrics GetMetrics() /** */ public Task Rebalance() { - return GetFuture((futId, futTyp) => UU.CacheRebalance(Target, futId)).Task; + return GetFuture((futId, futTyp) => DoOutInOpLong((int) CacheOp.Rebalance, futId)).Task; } /** */ @@ -923,7 +928,7 @@ public Task Rebalance() if (_flagNoRetries) return this; - return new CacheImpl(_ignite, UU.CacheWithNoRetries(Target), Marshaller, + return new CacheImpl(_ignite, DoOutOpObject((int) CacheOp.WithNoRetries), Marshaller, _flagSkipStore, _flagKeepBinary, _flagAsync, true); } @@ -965,22 +970,14 @@ public IQueryCursor QueryFields(SqlFieldsQuery qry, Func { - var writer = Marshaller.StartMarshal(stream); - writer.WriteBoolean(qry.Local); writer.WriteString(qry.Sql); writer.WriteInt(qry.PageSize); WriteQueryArgs(writer, qry.Arguments); - - FinishMarshal(writer); - - cursor = UU.CacheOutOpQueryCursor(Target, (int) CacheOp.QrySqlFields, stream.SynchronizeOutput()); - } + }); return new FieldsQueryCursor(cursor, Marshaller, _flagKeepBinary, readerFunc); } @@ -990,18 +987,7 @@ public IQueryCursor QueryFields(SqlFieldsQuery qry, Func qry.Write(writer, IsKeepBinary)); return new QueryCursor(cursor, Marshaller, _flagKeepBinary); } @@ -1058,41 +1044,8 @@ public IContinuousQueryHandle QueryContinuous(ContinuousQuery qry) { qry.Validate(); - var hnd = new ContinuousQueryHandleImpl(qry, Marshaller, _flagKeepBinary); - - try - { - using (var stream = IgniteManager.Memory.Allocate().GetStream()) - { - var writer = Marshaller.StartMarshal(stream); - - hnd.Start(_ignite, writer, () => - { - if (initialQry != null) - { - writer.WriteInt((int) initialQry.OpId); - - initialQry.Write(writer, IsKeepBinary); - } - else - writer.WriteInt(-1); // no initial query - - FinishMarshal(writer); - - // ReSharper disable once AccessToDisposedClosure - return UU.CacheOutOpContinuousQuery(Target, (int) CacheOp.QryContinuous, - stream.SynchronizeOutput()); - }, qry); - } - - return hnd; - } - catch (Exception) - { - hnd.Dispose(); - - throw; - } + return new ContinuousQueryHandleImpl(qry, Marshaller, _flagKeepBinary, + writeAction => DoOutOpObject((int) CacheOp.QryContinuous, writeAction), initialQry); } #endregion @@ -1126,9 +1079,13 @@ IEnumerator IEnumerable.GetEnumerator() internal CacheEnumerator CreateEnumerator(bool loc, int peekModes) { if (loc) - return new CacheEnumerator(UU.CacheLocalIterator(Target, peekModes), Marshaller, _flagKeepBinary); + { + var target = DoOutOpObject((int) CacheOp.LocIterator, w => w.WriteInt(peekModes)); - return new CacheEnumerator(UU.CacheIterator(Target), Marshaller, _flagKeepBinary); + return new CacheEnumerator(target, Marshaller, _flagKeepBinary); + } + + return new CacheEnumerator(DoOutOpObject((int) CacheOp.Iterator), Marshaller, _flagKeepBinary); } #endregion @@ -1312,5 +1269,33 @@ private CacheResult DoOutInOpNullable(int type, Action out ? new CacheResult() : new CacheResult((TR)res); } + + /** */ + public void Enter(long id) + { + DoOutInOpLong((int) CacheOp.EnterLock, id); + } + + /** */ + public bool TryEnter(long id, TimeSpan timeout) + { + return DoOutOp((int) CacheOp.TryEnterLock, (IBinaryStream s) => + { + s.WriteLong(id); + s.WriteLong((long) timeout.TotalMilliseconds); + }) == True; + } + + /** */ + public void Exit(long id) + { + DoOutInOpLong((int) CacheOp.ExitLock, id); + } + + /** */ + public void Close(long id) + { + DoOutInOpLong((int) CacheOp.CloseLock, id); + } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheLock.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheLock.cs index ceb3b056d0b77..960c9307cebea 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheLock.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheLock.cs @@ -21,8 +21,6 @@ namespace Apache.Ignite.Core.Impl.Cache using System.Diagnostics; using System.Threading; using Apache.Ignite.Core.Cache; - using Apache.Ignite.Core.Impl.Unmanaged; - using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils; /// /// Cache lock implementation. @@ -32,8 +30,8 @@ internal class CacheLock : ICacheLock /** Unique lock ID.*/ private readonly long _id; - /** Cache. */ - private readonly IUnmanagedTarget _cache; + /** Cache lock. */ + private readonly ICacheLockInternal _lock; /** State (-1 for disposed, >=0 for number of currently executing methods). */ private int _state; @@ -45,13 +43,13 @@ internal class CacheLock : ICacheLock /// Initializes a new instance of the class. /// /// Lock id. - /// Cache. - public CacheLock(long id, IUnmanagedTarget cache) + /// Cache lock. + public CacheLock(long id, ICacheLockInternal cacheLock) { - Debug.Assert(cache != null); + Debug.Assert(cacheLock != null); _id = id; - _cache = cache; + _lock = cacheLock; } /** */ @@ -68,7 +66,7 @@ public void Enter() try { - UU.CacheEnterLock(_cache, _id); + _lock.Enter(_id); res = true; } @@ -104,7 +102,7 @@ public bool TryEnter(TimeSpan timeout) try { - return res = UU.CacheTryEnterLock(_cache, _id, (long)timeout.TotalMilliseconds); + return res = _lock.TryEnter(_id, timeout); } finally { @@ -125,7 +123,7 @@ public void Exit() { ThrowIfDisposed(); - UU.CacheExitLock(_cache, _id); + _lock.Exit(_id); _counter--; } @@ -143,7 +141,7 @@ public void Dispose() "The lock is being disposed while still being used. " + "It either is being held by a thread and/or has active waiters waiting to acquire the lock."); - UU.CacheCloseLock(_cache, _id); + _lock.Close(_id); _state = -1; @@ -156,7 +154,7 @@ public void Dispose() /// ~CacheLock() { - UU.CacheCloseLock(_cache, _id); + _lock.Close(_id); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs index 4c42bf35003a7..713ab42b394a0 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs @@ -61,6 +61,22 @@ internal enum CacheOp Replace2 = 37, Replace3 = 38, GetConfig = 39, - LoadAll = 40 + LoadAll = 40, + ClearCache = 41, + WithAsync = 42, + RemoveAll2 = 43, + WithKeepBinary = 44, + WithExpiryPolicy = 45, + WithNoRetries = 46, + WithSkipStore = 47, + Size = 48, + Iterator = 49, + LocIterator = 50, + EnterLock = 51, + ExitLock = 52, + TryEnterLock = 53, + CloseLock = 54, + Rebalance = 55, + SizeLoc = 56 } } \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/ICacheLockInternal.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/ICacheLockInternal.cs new file mode 100644 index 0000000000000..4329ed0e9eeaa --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/ICacheLockInternal.cs @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +namespace Apache.Ignite.Core.Impl.Cache +{ + using System; + + /// + /// Internal cache locking interface. + /// + internal interface ICacheLockInternal + { + /// + /// Enters the lock. + /// + void Enter(long id); + + /// + /// Tries to enter the lock. + /// + bool TryEnter(long id, TimeSpan timeout); + + /// + /// Exits the lock. + /// + void Exit(long id); + + /// + /// Closes the lock. + /// + void Close(long id); + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs index a22b247fc8b3e..e6092d7ddb115 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs @@ -37,6 +37,12 @@ internal abstract class AbstractQueryCursor : PlatformDisposableTarget, IQuer /** */ private const int OpGetBatch = 2; + /** */ + private const int OpIterator = 4; + + /** */ + private const int OpIteratorClose = 5; + /** Position before head. */ private const int BatchPosBeforeHead = -1; @@ -94,7 +100,7 @@ protected override void Dispose(bool disposing) { try { - UU.QueryCursorClose(Target); + DoOutOp(OpIteratorClose); } finally { @@ -119,7 +125,7 @@ public IEnumerator GetEnumerator() throw new InvalidOperationException("Failed to get enumerator entries because " + "GetAll() method has already been called."); - UU.QueryCursorIterator(Target); + DoOutOp(OpIterator); _iterCalled = true; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs index 6cfbe9237bb20..b66dc48b332c2 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs @@ -48,7 +48,7 @@ internal interface IContinuousQueryHandleImpl : IDisposable /// /// Continuous query handle. /// - internal class ContinuousQueryHandleImpl : IContinuousQueryHandleImpl, IContinuousQueryFilter, + internal class ContinuousQueryHandleImpl : IContinuousQueryHandleImpl, IContinuousQueryFilter, IContinuousQueryHandle> { /** Marshaller. */ @@ -64,11 +64,11 @@ internal class ContinuousQueryHandleImpl : IContinuousQueryHandleImpl, I private readonly ICacheEntryEventFilter _filter; /** GC handle. */ - private long _hnd; + private readonly long _hnd; /** Native query. */ - private volatile IUnmanagedTarget _nativeQry; - + private readonly IUnmanagedTarget _nativeQry; + /** Initial query cursor. */ private volatile IQueryCursor> _initialQueryCursor; @@ -81,64 +81,81 @@ internal class ContinuousQueryHandleImpl : IContinuousQueryHandleImpl, I /// Query. /// Marshaller. /// Keep binary flag. - public ContinuousQueryHandleImpl(ContinuousQuery qry, Marshaller marsh, bool keepBinary) + /// The initialization callback. + /// The initial query. + public ContinuousQueryHandleImpl(ContinuousQuery qry, Marshaller marsh, bool keepBinary, + Func, IUnmanagedTarget> createTargetCb, QueryBase initialQry) { _marsh = marsh; _keepBinary = keepBinary; _lsnr = qry.Listener; _filter = qry.Filter; - } - /// - /// Start execution. - /// - /// Ignite instance. - /// Writer. - /// Callback invoked when all necessary data is written to stream. - /// Query. - public void Start(Ignite grid, BinaryWriter writer, Func cb, - ContinuousQuery qry) - { // 1. Inject resources. - ResourceProcessor.Inject(_lsnr, grid); - ResourceProcessor.Inject(_filter, grid); - - // 2. Allocate handle. - _hnd = grid.HandleRegistry.Allocate(this); - - // 3. Write data to stream. - writer.WriteLong(_hnd); - writer.WriteBoolean(qry.Local); - writer.WriteBoolean(_filter != null); - - var javaFilter = _filter as PlatformJavaObjectFactoryProxy; + ResourceProcessor.Inject(_lsnr, _marsh.Ignite); + ResourceProcessor.Inject(_filter, _marsh.Ignite); - if (javaFilter != null) + try { - writer.WriteObject(javaFilter.GetRawProxy()); - } - else - { - var filterHolder = _filter == null || qry.Local - ? null - : new ContinuousQueryFilterHolder(_filter, _keepBinary); + // 2. Allocate handle. + _hnd = _marsh.Ignite.HandleRegistry.Allocate(this); - writer.WriteObject(filterHolder); + // 3. Call Java. + _nativeQry = createTargetCb(writer => + { + writer.WriteLong(_hnd); + writer.WriteBoolean(qry.Local); + writer.WriteBoolean(_filter != null); + + var javaFilter = _filter as PlatformJavaObjectFactoryProxy; + + if (javaFilter != null) + { + writer.WriteObject(javaFilter.GetRawProxy()); + } + else + { + var filterHolder = _filter == null || qry.Local + ? null + : new ContinuousQueryFilterHolder(_filter, _keepBinary); + + writer.WriteObject(filterHolder); + } + + writer.WriteInt(qry.BufferSize); + writer.WriteLong((long)qry.TimeInterval.TotalMilliseconds); + writer.WriteBoolean(qry.AutoUnsubscribe); + + if (initialQry != null) + { + writer.WriteInt((int)initialQry.OpId); + + initialQry.Write(writer, _keepBinary); + } + else + writer.WriteInt(-1); // no initial query + }); + + // 4. Initial query. + var nativeInitialQryCur = UU.TargetOutObject(_nativeQry, 0); + _initialQueryCursor = nativeInitialQryCur == null + ? null + : new QueryCursor(nativeInitialQryCur, _marsh, _keepBinary); } + catch (Exception) + { + if (_hnd > 0) + _marsh.Ignite.HandleRegistry.Release(_hnd); - writer.WriteInt(qry.BufferSize); - writer.WriteLong((long)qry.TimeInterval.TotalMilliseconds); - writer.WriteBoolean(qry.AutoUnsubscribe); - - // 4. Call Java. - _nativeQry = cb(); + if (_nativeQry != null) + _nativeQry.Dispose(); - // 5. Initial query. - var nativeInitialQryCur = UU.ContinuousQueryGetInitialQueryCursor(_nativeQry); - _initialQueryCursor = nativeInitialQryCur == null - ? null - : new QueryCursor(nativeInitialQryCur, _marsh, _keepBinary); + if (_initialQueryCursor != null) + _initialQueryCursor.Dispose(); + + throw; + } } /** */ @@ -208,7 +225,7 @@ public void Dispose() try { - UU.ContinuousQueryClose(_nativeQry); + UU.TargetOutLong(_nativeQry, 0); } finally { @@ -219,4 +236,4 @@ public void Dispose() } } } -} +} \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStore.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStore.cs index 7785280e92912..b14b2a3803553 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStore.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStore.cs @@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Impl.Cache.Store { using System.Collections; using System.Diagnostics; + using System.IO; using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cache.Store; using Apache.Ignite.Core.Common; @@ -26,7 +27,6 @@ namespace Apache.Ignite.Core.Impl.Cache.Store using Apache.Ignite.Core.Impl.Binary.IO; using Apache.Ignite.Core.Impl.Handle; using Apache.Ignite.Core.Impl.Resource; - using Apache.Ignite.Core.Impl.Unmanaged; /// /// Interop cache store. @@ -145,21 +145,20 @@ public void Init(Ignite grid) /// /// Invokes a store operation. /// - /// Input stream. - /// Callback. + /// Input stream. /// Grid. /// Invocation result. /// Invalid operation type: + opType - public int Invoke(IBinaryStream input, IUnmanagedTarget cb, Ignite grid) + public int Invoke(IBinaryStream stream, Ignite grid) { - IBinaryReader reader = grid.Marshaller.StartUnmarshal(input, + IBinaryReader reader = grid.Marshaller.StartUnmarshal(stream, _convertBinary ? BinaryMode.Deserialize : BinaryMode.ForceBinary); IBinaryRawReader rawReader = reader.GetRawReader(); int opType = rawReader.ReadByte(); - // Setup cache sessoin for this invocation. + // Setup cache session for this invocation. long sesId = rawReader.ReadLong(); CacheStoreSession ses = grid.HandleRegistry.Get(sesId, true); @@ -174,27 +173,79 @@ public int Invoke(IBinaryStream input, IUnmanagedTarget cb, Ignite grid) switch (opType) { case OpLoadCache: - _store.LoadCache((k, v) => WriteObjects(cb, grid, k, v), rawReader.ReadArray()); + { + var args = rawReader.ReadArray(); + + stream.Seek(0, SeekOrigin.Begin); + + int cnt = 0; + stream.WriteInt(cnt); // Reserve space for count. + + var writer = grid.Marshaller.StartMarshal(stream); + + _store.LoadCache((k, v) => + { + lock (writer) // User-defined store can be multithreaded. + { + writer.WithDetach(w => + { + w.WriteObject(k); + w.WriteObject(v); + }); + + cnt++; + } + }, args); + + stream.WriteInt(0, cnt); + + grid.Marshaller.FinishMarshal(writer); break; + } case OpLoad: - object val = _store.Load(rawReader.ReadObject()); + { + var val = _store.Load(rawReader.ReadObject()); + + stream.Seek(0, SeekOrigin.Begin); - if (val != null) - WriteObjects(cb, grid, val); + var writer = grid.Marshaller.StartMarshal(stream); + + writer.WriteObject(val); + + grid.Marshaller.FinishMarshal(writer); break; + } case OpLoadAll: + { var keys = rawReader.ReadCollection(); var result = _store.LoadAll(keys); + stream.Seek(0, SeekOrigin.Begin); + + stream.WriteInt(result.Count); + + var writer = grid.Marshaller.StartMarshal(stream); + foreach (DictionaryEntry entry in result) - WriteObjects(cb, grid, entry.Key, entry.Value); + { + var entry0 = entry; // Copy modified closure. + + writer.WithDetach(w => + { + w.WriteObject(entry0.Key); + w.WriteObject(entry0.Value); + }); + } + + grid.Marshaller.FinishMarshal(writer); break; + } case OpPut: _store.Write(rawReader.ReadObject(), rawReader.ReadObject()); @@ -241,40 +292,5 @@ public int Invoke(IBinaryStream input, IUnmanagedTarget cb, Ignite grid) _sesProxy.ClearSession(); } } - - /// - /// Writes objects to the marshaller. - /// - /// Optional callback. - /// Grid. - /// Objects. - private static void WriteObjects(IUnmanagedTarget cb, Ignite grid, params object[] objects) - { - using (var stream = IgniteManager.Memory.Allocate().GetStream()) - { - BinaryWriter writer = grid.Marshaller.StartMarshal(stream); - - try - { - foreach (var obj in objects) - { - var obj0 = obj; - - writer.WithDetach(w => w.WriteObject(obj0)); - } - } - finally - { - grid.Marshaller.FinishMarshal(writer); - } - - if (cb != null) - { - stream.SynchronizeOutput(); - - UnmanagedUtils.CacheStoreCallbackInvoke(cb, stream.MemoryPointer); - } - } - } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs index e6c000583ef6a..508bdc9d854bf 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs @@ -99,6 +99,21 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx /** */ private const int OpSchema = 15; + /** */ + private const int OpForRemotes = 17; + + /** */ + private const int OpForRandom = 19; + + /** */ + private const int OpForOldest = 20; + + /** */ + private const int OpForYoungest = 21; + + /** */ + public const int OpResetMetrics = 22; + /** Initial Ignite instance. */ private readonly Ignite _ignite; @@ -208,7 +223,7 @@ private IClusterGroup ForNodeIds0(IEnumerable items, Func func) { Debug.Assert(items != null); - IUnmanagedTarget prj = DoProjetionOutOp(OpForNodeIds, writer => + IUnmanagedTarget prj = DoOutOpObject(OpForNodeIds, writer => { WriteEnumerable(writer, items, func); }); @@ -229,11 +244,12 @@ public IClusterGroup ForAttribute(string name, string val) { IgniteArgumentCheck.NotNull(name, "name"); - IUnmanagedTarget prj = DoProjetionOutOp(OpForAttribute, writer => + Action action = writer => { writer.WriteString(name); writer.WriteString(val); - }); + }; + IUnmanagedTarget prj = DoOutOpObject(OpForAttribute, action); return GetClusterGroup(prj); } @@ -248,7 +264,7 @@ public IClusterGroup ForAttribute(string name, string val) /// private IClusterGroup ForCacheNodes(string name, int op) { - IUnmanagedTarget prj = DoProjetionOutOp(op, writer => + IUnmanagedTarget prj = DoOutOpObject(op, writer => { writer.WriteString(name); }); @@ -277,7 +293,7 @@ public IClusterGroup ForClientNodes(string name) /** */ public IClusterGroup ForRemotes() { - return GetClusterGroup(UU.ProjectionForRemotes(Target)); + return GetClusterGroup(DoOutOpObject(OpForRemotes)); } /** */ @@ -285,7 +301,7 @@ public IClusterGroup ForHost(IClusterNode node) { IgniteArgumentCheck.NotNull(node, "node"); - IUnmanagedTarget prj = DoProjetionOutOp(OpForHost, writer => + IUnmanagedTarget prj = DoOutOpObject(OpForHost, writer => { writer.WriteGuid(node.Id); }); @@ -296,19 +312,19 @@ public IClusterGroup ForHost(IClusterNode node) /** */ public IClusterGroup ForRandom() { - return GetClusterGroup(UU.ProjectionForRandom(Target)); + return GetClusterGroup(DoOutOpObject(OpForRandom)); } /** */ public IClusterGroup ForOldest() { - return GetClusterGroup(UU.ProjectionForOldest(Target)); + return GetClusterGroup(DoOutOpObject(OpForOldest)); } /** */ public IClusterGroup ForYoungest() { - return GetClusterGroup(UU.ProjectionForYoungest(Target)); + return GetClusterGroup(DoOutOpObject(OpForYoungest)); } /** */ @@ -519,27 +535,7 @@ private IList RefreshNodes() return _nodes; } - - /// - /// Perform synchronous out operation returning value. - /// - /// Operation type. - /// Action. - /// Native projection. - private IUnmanagedTarget DoProjetionOutOp(int type, Action action) - { - using (var stream = IgniteManager.Memory.Allocate().GetStream()) - { - var writer = Marshaller.StartMarshal(stream); - action(writer); - - FinishMarshal(writer); - - return UU.ProjectionOutOpRet(Target, type, stream.SynchronizeOutput()); - } - } - /** */ public IBinaryType GetBinaryType(int typeId) { diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs index 746577afa8931..68bd9d423acc6 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs @@ -62,7 +62,10 @@ public T Get() } catch (AggregateException ex) { - throw ex.InnerException; + if (ex.InnerException != null) + throw ex.InnerException; + + throw; } } @@ -173,34 +176,13 @@ internal void SetTarget(IUnmanagedTarget target) _unmanagedTarget = target; } - /// - /// Cancels this instance. - /// - internal bool Cancel() - { - if (_unmanagedTarget == null) - return false; - - return UnmanagedUtils.ListenableCancel(_unmanagedTarget); - } - - /// - /// Determines whether this instance is cancelled. - /// - internal bool IsCancelled() - { - if (_unmanagedTarget == null) - return false; - - return UnmanagedUtils.ListenableIsCancelled(_unmanagedTarget); - } - /// /// Called when token cancellation occurs. /// private void OnTokenCancel() { - Cancel(); + if (_unmanagedTarget != null) + UnmanagedUtils.ListenableCancel(_unmanagedTarget); } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs index 86dee30843dff..1b2e2aa914ef8 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs @@ -57,6 +57,15 @@ internal class ComputeImpl : PlatformTarget /** */ private const int OpUnicast = 5; + /** */ + private const int OpWithNoFailover = 6; + + /** */ + private const int OpWithTimeout = 7; + + /** */ + private const int OpExecNative = 8; + /** Underlying projection. */ private readonly ClusterGroupImpl _prj; @@ -97,7 +106,7 @@ public IClusterGroup ClusterGroup /// public void WithNoFailover() { - UU.ComputeWithNoFailover(Target); + DoOutOp(OpWithNoFailover); } /// @@ -107,7 +116,7 @@ public void WithNoFailover() /// Computation timeout in milliseconds. public void WithTimeout(long timeout) { - UU.ComputeWithTimeout(Target, timeout); + DoOutInOpLong(OpWithTimeout, timeout); } /// @@ -193,7 +202,11 @@ public Future ExecuteJavaTaskAsync(string taskName, obje long ptr = Marshaller.Ignite.HandleRegistry.Allocate(holder); - var futTarget = UU.ComputeExecuteNative(Target, ptr, _prj.TopologyVersion); + var futTarget = DoOutOpObject(OpExecNative, w => + { + w.WriteLong(ptr); + w.WriteLong(_prj.TopologyVersion); + }); var future = holder.Future; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicLong.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicLong.cs index 6fefe3be3cb5c..571e6fd2707c0 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicLong.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicLong.cs @@ -20,6 +20,7 @@ namespace Apache.Ignite.Core.Impl.DataStructures using System.Diagnostics; using Apache.Ignite.Core.DataStructures; using Apache.Ignite.Core.Impl.Binary; + using Apache.Ignite.Core.Impl.Binary.IO; using Apache.Ignite.Core.Impl.Unmanaged; using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils; @@ -32,6 +33,19 @@ internal sealed class AtomicLong : PlatformTarget, IAtomicLong /** */ private readonly string _name; + /** Operation codes. */ + private enum Op + { + AddAndGet = 1, + Close = 2, + CompareAndSetAndGet = 4, + DecrementAndGet = 5, + Get = 6, + GetAndSet = 10, + IncrementAndGet = 11, + IsClosed = 12 + } + /// /// Initializes a new instance of the class. /// @@ -54,49 +68,53 @@ public string Name /** */ public long Read() { - return UU.AtomicLongGet(Target); + return DoOutOp((int) Op.Get); } /** */ public long Increment() { - return UU.AtomicLongIncrementAndGet(Target); + return DoOutOp((int) Op.IncrementAndGet); } /** */ public long Add(long value) { - return UU.AtomicLongAddAndGet(Target, value); + return DoOutInOpLong((int) Op.AddAndGet, value); } /** */ public long Decrement() { - return UU.AtomicLongDecrementAndGet(Target); + return DoOutOp((int) Op.DecrementAndGet); } /** */ public long Exchange(long value) { - return UU.AtomicLongGetAndSet(Target, value); + return DoOutInOpLong((int) Op.GetAndSet, value); } /** */ public long CompareExchange(long value, long comparand) { - return UU.AtomicLongCompareAndSetAndGet(Target, comparand, value); + return DoOutOp((int) Op.CompareAndSetAndGet, (IBinaryStream s) => + { + s.WriteLong(comparand); + s.WriteLong(value); + }); } /** */ public void Close() { - UU.AtomicLongClose(Target); + DoOutOp((int) Op.Close); } /** */ public bool IsClosed() { - return UU.AtomicLongIsClosed(Target); + return DoOutOp((int) Op.IsClosed) == True; } } } \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs index d22afebfe4eea..75e36d1c73b91 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs @@ -32,7 +32,9 @@ private enum Op { Get = 1, Set = 2, - CompareAndSetAndGet = 3 + CompareAndSetAndGet = 3, + Close = 4, + IsClosed = 5 } /** */ @@ -80,13 +82,13 @@ public T CompareExchange(T value, T comparand) /** */ public bool IsClosed { - get { return UnmanagedUtils.AtomicReferenceIsClosed(Target); } + get { return DoOutOp((int) Op.IsClosed) == True; } } /** */ public void Close() { - UnmanagedUtils.AtomicReferenceClose(Target); + DoOutOp((int) Op.Close); } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs index 0835b9a797a85..b7b924ed246a0 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs @@ -30,6 +30,18 @@ internal sealed class AtomicSequence: PlatformTarget, IAtomicSequence /** */ private readonly string _name; + /** */ + private enum Op + { + AddAndGet = 1, + Close = 2, + Get = 3, + GetBatchSize = 6, + IncrementAndGet = 7, + IsClosed = 8, + SetBatchSize = 9 + } + /// /// Initializes a new instance of the class. /// @@ -53,38 +65,38 @@ public string Name /** */ public long Read() { - return UnmanagedUtils.AtomicSequenceGet(Target); + return DoOutOp((int) Op.Get); } /** */ public long Increment() { - return UnmanagedUtils.AtomicSequenceIncrementAndGet(Target); + return DoOutOp((int) Op.IncrementAndGet); } /** */ public long Add(long value) { - return UnmanagedUtils.AtomicSequenceAddAndGet(Target, value); + return DoOutInOpLong((int) Op.AddAndGet, value); } /** */ public int BatchSize { - get { return UnmanagedUtils.AtomicSequenceGetBatchSize(Target); } - set { UnmanagedUtils.AtomicSequenceSetBatchSize(Target, value); } + get { return (int) DoOutOp((int) Op.GetBatchSize); } + set { DoOutInOpLong((int) Op.SetBatchSize, value); } } /** */ public bool IsClosed { - get { return UnmanagedUtils.AtomicSequenceIsClosed(Target); } + get { return DoOutOp((int) Op.IsClosed) == True; } } /** */ public void Close() { - UnmanagedUtils.AtomicSequenceClose(Target); + DoOutOp((int) Op.Close); } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs index 74261d3950fa0..8893fc5558695 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs @@ -67,7 +67,34 @@ internal class DataStreamerImpl : PlatformDisposableTarget, IDataStreame /** Operation: set receiver. */ private const int OpReceiver = 2; - + + /** */ + private const int OpAllowOverwrite = 3; + + /** */ + private const int OpSetAllowOverwrite = 4; + + /** */ + private const int OpSkipStore = 5; + + /** */ + private const int OpSetSkipStore = 6; + + /** */ + private const int OpPerNodeBufferSize = 7; + + /** */ + private const int OpSetPerNodeBufferSize = 8; + + /** */ + private const int OpPerNodeParallelOps = 9; + + /** */ + private const int OpSetPerNodeParallelOps = 10; + + /** */ + private const int OpListenTopology = 11; + /** Cache name. */ private readonly string _cacheName; @@ -129,7 +156,7 @@ public DataStreamerImpl(IUnmanagedTarget target, Marshaller marsh, string cacheN _hnd = marsh.Ignite.HandleRegistry.Allocate(thisRef); // Start topology listening. This call will ensure that buffer size member is updated. - UU.DataStreamerListenTopology(target, _hnd); + DoOutInOpLong(OpListenTopology, _hnd); // Membar to ensure fields initialization before leaving constructor. Thread.MemoryBarrier(); @@ -157,7 +184,7 @@ public bool AllowOverwrite { ThrowIfDisposed(); - return UU.DataStreamerAllowOverwriteGet(Target); + return DoOutOp(OpAllowOverwrite) == True; } finally { @@ -172,7 +199,7 @@ public bool AllowOverwrite { ThrowIfDisposed(); - UU.DataStreamerAllowOverwriteSet(Target, value); + DoOutInOpLong(OpSetAllowOverwrite, value ? True : False); } finally { @@ -192,7 +219,7 @@ public bool SkipStore { ThrowIfDisposed(); - return UU.DataStreamerSkipStoreGet(Target); + return DoOutOp(OpSkipStore) == True; } finally { @@ -207,7 +234,7 @@ public bool SkipStore { ThrowIfDisposed(); - UU.DataStreamerSkipStoreSet(Target, value); + DoOutInOpLong(OpSetSkipStore, value ? True : False); } finally { @@ -227,7 +254,7 @@ public int PerNodeBufferSize { ThrowIfDisposed(); - return UU.DataStreamerPerNodeBufferSizeGet(Target); + return (int) DoOutOp(OpPerNodeBufferSize); } finally { @@ -242,7 +269,7 @@ public int PerNodeBufferSize { ThrowIfDisposed(); - UU.DataStreamerPerNodeBufferSizeSet(Target, value); + DoOutInOpLong(OpSetPerNodeBufferSize, value); _bufSndSize = _topSize * value; } @@ -264,7 +291,7 @@ public int PerNodeParallelOperations { ThrowIfDisposed(); - return UU.DataStreamerPerNodeParallelOperationsGet(Target); + return (int) DoOutOp(OpPerNodeParallelOps); } finally { @@ -280,7 +307,7 @@ public int PerNodeParallelOperations { ThrowIfDisposed(); - UU.DataStreamerPerNodeParallelOperationsSet(Target, value); + DoOutInOpLong(OpSetPerNodeParallelOps, value); } finally { @@ -571,7 +598,7 @@ public void TopologyChange(long topVer, int topSize) _topVer = topVer; _topSize = topSize > 0 ? topSize : 1; // Do not set to 0 to avoid 0 buffer size. - _bufSndSize = _topSize * UU.DataStreamerPerNodeBufferSizeGet(Target); + _bufSndSize = (int) (_topSize * DoOutOp(OpPerNodeBufferSize)); } } finally diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs index 783ba94cac005..5d1add640466c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs @@ -52,7 +52,11 @@ private enum Op RecordLocal = 6, EnableLocal = 8, DisableLocal = 9, - GetEnabledEvents = 10 + GetEnabledEvents = 10, + WithAsync = 11, + IsEnabled = 12, + LocalListen = 13, + StopLocalListen = 14 } /** Map from user func to local wrapper, needed for invoke/unsubscribe. */ @@ -85,7 +89,7 @@ public Events(IUnmanagedTarget target, Marshaller marsh, IClusterGroup clusterGr /// Initializes a new async instance. /// /// The events. - private Events(Events events) : base(UU.EventsWithAsync(events.Target), events.Marshaller) + private Events(Events events) : base(UU.TargetOutObject(events.Target, (int) Op.WithAsync), events.Marshaller) { _clusterGroup = events.ClusterGroup; } @@ -335,7 +339,7 @@ public void RecordLocal(IEvent evt) // Should do this inside lock to avoid race with subscription // ToArray is required because we are going to modify underlying dictionary during enumeration foreach (var filter in GetLocalFilters(listener, types).ToArray()) - success |= UU.EventsStopLocalListen(Target, filter.Handle); + success |= (DoOutInOpLong((int) Op.StopLocalListen, filter.Handle) == True); return success; } @@ -384,7 +388,7 @@ public ICollection GetEnabledEvents() /** */ public bool IsEnabled(int type) { - return UU.EventsIsEnabled(Target, type); + return DoOutInOpLong((int) Op.IsEnabled, type) == True; } /// @@ -508,7 +512,11 @@ private IEnumerable GetLocalFilters(object listener, in filters[type] = localFilter; } - UU.EventsLocalListen(Target, localFilter.Handle, type); + DoOutOp((int) Op.LocalListen, (IBinaryStream s) => + { + s.WriteLong(localFilter.Handle); + s.WriteInt(type); + }); } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs index 07069665eb697..675af5e63931b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs @@ -486,7 +486,7 @@ public ICollection GetTopology(long ver) /** */ public void ResetMetrics() { - UU.ProjectionResetMetrics(_prj.Target); + UU.TargetOutLong(_prj.Target, ClusterGroupImpl.OpResetMetrics); } /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs index 588249526a4b3..2216d1a88f379 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs @@ -48,7 +48,8 @@ private enum Op SendMulti = 4, SendOrdered = 5, StopLocalListen = 6, - StopRemoteListen = 7 + StopRemoteListen = 7, + WithAsync = 8 } /** Map from user (func+topic) -> id, needed for unsubscription. */ @@ -89,7 +90,8 @@ public Messaging(IUnmanagedTarget target, Marshaller marsh, IClusterGroup prj) /// Initializes a new async instance. /// /// The messaging. - private Messaging(Messaging messaging) : base(UU.MessagingWithASync(messaging.Target), messaging.Marshaller) + private Messaging(Messaging messaging) : base( + UU.TargetOutObject(messaging.Target, (int) Op.WithAsync), messaging.Marshaller) { _isAsync = true; _ignite = messaging._ignite; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs index 26b6033b5b7e9..8c065bcc7249a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs @@ -39,6 +39,9 @@ namespace Apache.Ignite.Core.Impl [SuppressMessage("ReSharper", "LocalVariableHidesMember")] internal abstract class PlatformTarget { + /** */ + protected const int False = 0; + /** */ protected const int True = 1; @@ -252,6 +255,16 @@ protected static BinaryWriter WriteEnumerable(BinaryWriter writer, IEnumerabl #region OUT operations + /// + /// Perform out operation. + /// + /// Operation type. + /// Long result. + protected long DoOutOp(int type) + { + return UU.TargetOutLong(_target, type); + } + /// /// Perform out operation. /// @@ -308,6 +321,16 @@ protected IUnmanagedTarget DoOutOpObject(int type, Action action) } } + /// + /// Perform out operation. + /// + /// Operation type. + /// Resulting object. + protected IUnmanagedTarget DoOutOpObject(int type) + { + return UU.TargetOutObject(_target, type); + } + /// /// Perform simple output operation accepting single argument. /// @@ -500,6 +523,66 @@ protected unsafe TR DoOutInOp(int type, Action outAction, Func } } + /// + /// Perform out-in operation. + /// + /// Operation type. + /// Out action. + /// In action. + /// Argument. + /// Result. + protected unsafe TR DoOutInOp(int type, Action outAction, + Func inAction, void* arg) + { + PlatformMemoryStream outStream = null; + long outPtr = 0; + + PlatformMemoryStream inStream = null; + long inPtr = 0; + + try + { + if (outAction != null) + { + outStream = IgniteManager.Memory.Allocate().GetStream(); + var writer = _marsh.StartMarshal(outStream); + outAction(writer); + FinishMarshal(writer); + outPtr = outStream.SynchronizeOutput(); + } + + if (inAction != null) + { + inStream = IgniteManager.Memory.Allocate().GetStream(); + inPtr = inStream.MemoryPointer; + } + + var res = UU.TargetInObjectStreamOutObjectStream(_target, type, arg, outPtr, inPtr); + + if (inAction == null) + return default(TR); + + inStream.SynchronizeInput(); + + return inAction(inStream, res); + + } + finally + { + try + { + if (inStream != null) + inStream.Dispose(); + + } + finally + { + if (outStream != null) + outStream.Dispose(); + } + } + } + /// /// Perform out-in operation. /// @@ -583,6 +666,17 @@ protected TR DoOutInOp(int type, Action outAction) } } + /// + /// Perform simple out-in operation accepting two arguments. + /// + /// Operation type. + /// Value. + /// Result. + protected long DoOutInOpLong(int type, long val) + { + return UU.TargetInLongOutLong(_target, type, val); + } + #endregion #region Miscelanneous diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs index 3d55f0632038c..d6b1d054b74b8 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs @@ -50,6 +50,21 @@ internal sealed class Services : PlatformTarget, IServices /** */ private const int OpDescriptors = 5; + /** */ + private const int OpWithAsync = 6; + + /** */ + private const int OpWithServerKeepBinary = 7; + + /** */ + private const int OpServiceProxy = 8; + + /** */ + private const int OpCancel = 9; + + /** */ + private const int OpCancelAll = 10; + /** */ private readonly IClusterGroup _clusterGroup; @@ -87,7 +102,8 @@ internal sealed class Services : PlatformTarget, IServices /// Initializes a new async instance. /// /// The services. - private Services(Services services) : base(UU.ServicesWithAsync(services.Target), services.Marshaller) + private Services(Services services) : base(UU.TargetOutObject(services.Target, OpWithAsync), + services.Marshaller) { _clusterGroup = services.ClusterGroup; _keepBinary = services._keepBinary; @@ -109,7 +125,7 @@ public IServices WithServerKeepBinary() if (_srvKeepBinary) return this; - return new Services(UU.ServicesWithServerKeepBinary(Target), Marshaller, _clusterGroup, _keepBinary, true); + return new Services(DoOutOpObject(OpWithServerKeepBinary), Marshaller, _clusterGroup, _keepBinary, true); } /** */ @@ -243,7 +259,7 @@ public void Cancel(string name) { IgniteArgumentCheck.NotNullOrEmpty(name, "name"); - UU.ServicesCancel(Target, name); + DoOutOp(OpCancel, w => w.WriteString(name)); } /** */ @@ -257,7 +273,7 @@ public Task CancelAsync(string name) /** */ public void CancelAll() { - UU.ServicesCancelAll(Target); + DoOutOp(OpCancelAll); } /** */ @@ -347,7 +363,12 @@ public ICollection GetServices(string name) if (locInst != null) return locInst; - var javaProxy = UU.ServicesGetServiceProxy(Target, name, sticky); + var javaProxy = DoOutOpObject(OpServiceProxy, w => + { + w.WriteString(name); + w.WriteBoolean(sticky); + }); + var platform = GetServiceDescriptors().Cast().Single(x => x.Name == name).Platform; return new ServiceProxy((method, args) => diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs index 51a49d002467c..796044d8d902f 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs @@ -22,9 +22,9 @@ namespace Apache.Ignite.Core.Impl.Transactions using System.Threading.Tasks; using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Impl.Binary; + using Apache.Ignite.Core.Impl.Binary.IO; using Apache.Ignite.Core.Impl.Unmanaged; using Apache.Ignite.Core.Transactions; - using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils; /// /// Transactions facade. @@ -36,7 +36,35 @@ internal class TransactionsImpl : PlatformTarget, ITransactions /** */ private const int OpMetrics = 2; - + + /** */ + private const int OpStart = 3; + + /** */ + private const int OpCommit = 4; + + /** */ + private const int OpRollback = 5; + + /** */ + private const int OpClose = 6; + + /** */ + private const int OpState = 7; + + /** */ + private const int OpSetRollbackOnly = 8; + + /** */ + private const int OpCommitAsync = 9; + + /** */ + private const int OpRollbackAsync = 10; + + /** */ + private const int OpResetMetrics = 11; + + /** */ private readonly TransactionConcurrency _dfltConcurrency; @@ -95,8 +123,13 @@ public ITransaction TxStart(TransactionConcurrency concurrency, TransactionIsola public ITransaction TxStart(TransactionConcurrency concurrency, TransactionIsolation isolation, TimeSpan timeout, int txSize) { - var id = UU.TransactionsStart(Target, (int)concurrency, (int)isolation, (long)timeout.TotalMilliseconds, - txSize); + var id = DoOutInOp(OpStart, w => + { + w.WriteInt((int) concurrency); + w.WriteInt((int) isolation); + w.WriteTimeSpanAsLong(timeout); + w.WriteInt(txSize); + }, s => s.ReadLong()); var innerTx = new TransactionImpl(id, this, concurrency, isolation, timeout, _localNodeId); @@ -123,7 +156,7 @@ public ITransactionMetrics GetMetrics() /** */ public void ResetMetrics() { - UU.TransactionsResetMetrics(Target); + DoOutOp(OpResetMetrics); } /// @@ -133,7 +166,7 @@ public void ResetMetrics() /// Final transaction state. internal TransactionState TxCommit(TransactionImpl tx) { - return (TransactionState) UU.TransactionsCommit(Target, tx.Id); + return (TransactionState) DoOutInOpLong(OpCommit, tx.Id); } /// @@ -143,7 +176,7 @@ internal TransactionState TxCommit(TransactionImpl tx) /// Final transaction state. internal TransactionState TxRollback(TransactionImpl tx) { - return (TransactionState)UU.TransactionsRollback(Target, tx.Id); + return (TransactionState) DoOutInOpLong(OpRollback, tx.Id); } /// @@ -153,7 +186,7 @@ internal TransactionState TxRollback(TransactionImpl tx) /// Final transaction state. internal int TxClose(TransactionImpl tx) { - return UU.TransactionsClose(Target, tx.Id); + return (int) DoOutInOpLong(OpClose, tx.Id); } /// @@ -163,7 +196,7 @@ internal int TxClose(TransactionImpl tx) /// Transaction current state. internal TransactionState TxState(TransactionImpl tx) { - return GetTransactionState(UU.TransactionsState(Target, tx.Id)); + return (TransactionState) DoOutInOpLong(OpState, tx.Id); } /// @@ -173,7 +206,7 @@ internal TransactionState TxState(TransactionImpl tx) /// true if the flag was set. internal bool TxSetRollbackOnly(TransactionImpl tx) { - return UU.TransactionsSetRollbackOnly(Target, tx.Id); + return DoOutInOpLong(OpSetRollbackOnly, tx.Id) == True; } /// @@ -181,7 +214,11 @@ internal bool TxSetRollbackOnly(TransactionImpl tx) /// internal Task CommitAsync(TransactionImpl tx) { - return GetFuture((futId, futTyp) => UU.TransactionsCommitAsync(Target, tx.Id, futId)).Task; + return GetFuture((futId, futTyp) => DoOutOp(OpCommitAsync, (IBinaryStream s) => + { + s.WriteLong(tx.Id); + s.WriteLong(futId); + })).Task; } /// @@ -189,15 +226,11 @@ internal Task CommitAsync(TransactionImpl tx) /// internal Task RollbackAsync(TransactionImpl tx) { - return GetFuture((futId, futTyp) => UU.TransactionsRollbackAsync(Target, tx.Id, futId)).Task; - } - - /// - /// Gets the state of the transaction from int. - /// - private static TransactionState GetTransactionState(int state) - { - return (TransactionState)state; + return GetFuture((futId, futTyp) => DoOutOp(OpRollbackAsync, (IBinaryStream s) => + { + s.WriteLong(tx.Id); + s.WriteLong(futId); + })).Task; } } } \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs index bb26382da8f5f..8de82ee04a09c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs @@ -113,6 +113,9 @@ internal static unsafe class IgniteJniNativeMethods [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorGetCacheNames")] public static extern void ProcessorGetCacheNames(void* ctx, void* obj, long memPtr); + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInLongOutLong")] + public static extern long TargetInLongOutLong(void* ctx, void* target, int opType, long val); + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInStreamOutLong")] public static extern long TargetInStreamOutLong(void* ctx, void* target, int opType, long memPtr); @@ -127,6 +130,10 @@ internal static unsafe class IgniteJniNativeMethods public static extern void TargetInObjectStreamOutStream(void* ctx, void* target, int opType, void* arg, long inMemPtr, long outMemPtr); + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInObjectStreamOutObjectStream")] + public static extern void* TargetInObjectStreamOutObjectStream(void* ctx, void* target, int opType, + void* arg, long inMemPtr, long outMemPtr); + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetOutLong")] public static extern long TargetOutLong(void* ctx, void* target, int opType); @@ -148,178 +155,12 @@ internal static unsafe class IgniteJniNativeMethods [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetListenFutureForOperationAndGet")] public static extern void* TargetListenFutForOpAndGet(void* ctx, void* target, long futId, int typ, int opId); - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAffinityPartitions")] - public static extern int AffinityParts(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheWithSkipStore")] - public static extern void* CacheWithSkipStore(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheWithNoRetries")] - public static extern void* CacheWithNoRetries(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheWithExpiryPolicy")] - public static extern void* CacheWithExpiryPolicy(void* ctx, void* obj, long create, long update, long access); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheWithAsync")] - public static extern void* CacheWithAsync(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheWithKeepPortable")] - public static extern void* CacheWithKeepBinary(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheClear")] - public static extern void CacheClear(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheRemoveAll")] - public static extern void CacheRemoveAll(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheOutOpQueryCursor")] - public static extern void* CacheOutOpQueryCursor(void* ctx, void* obj, int type, long memPtr); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheOutOpContinuousQuery")] - public static extern void* CacheOutOpContinuousQuery(void* ctx, void* obj, int type, long memPtr); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheIterator")] - public static extern void* CacheIterator(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheLocalIterator")] - public static extern void* CacheLocalIterator(void* ctx, void* obj, int peekModes); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheEnterLock")] - public static extern void CacheEnterLock(void* ctx, void* obj, long id); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheExitLock")] - public static extern void CacheExitLock(void* ctx, void* obj, long id); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheTryEnterLock")] - [return: MarshalAs(UnmanagedType.U1)] - public static extern bool CacheTryEnterLock(void* ctx, void* obj, long id, long timeout); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheCloseLock")] - public static extern void CacheCloseLock(void* ctx, void* obj, long id); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheRebalance")] - public static extern void CacheRebalance(void* ctx, void* obj, long futId); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheSize")] - public static extern int CacheSize(void* ctx, void* obj, int peekModes, [MarshalAs(UnmanagedType.U1)] bool loc); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteCacheStoreCallbackInvoke")] - public static extern void CacheStoreCallbackInvoke(void* ctx, void* obj, long memPtr); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteComputeWithNoFailover")] - public static extern void ComputeWithNoFailover(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteComputeWithTimeout")] - public static extern void ComputeWithTimeout(void* ctx, void* target, long timeout); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteComputeExecuteNative")] - public static extern void* ComputeExecuteNative(void* ctx, void* target, long taskPtr, long topVer); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteContinuousQueryClose")] - public static extern void ContinuousQryClose(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteContinuousQueryGetInitialQueryCursor")] - public static extern void* ContinuousQryGetInitialQueryCursor(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerListenTopology")] - public static extern void DataStreamerListenTop(void* ctx, void* obj, long ptr); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerAllowOverwriteGet")] - [return: MarshalAs(UnmanagedType.U1)] - public static extern bool DataStreamerAllowOverwriteGet(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerAllowOverwriteSet")] - public static extern void DataStreamerAllowOverwriteSet(void* ctx, void* obj, - [MarshalAs(UnmanagedType.U1)] bool val); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerSkipStoreGet")] - [return: MarshalAs(UnmanagedType.U1)] - public static extern bool DataStreamerSkipStoreGet(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerSkipStoreSet")] - public static extern void DataStreamerSkipStoreSet(void* ctx, void* obj, - [MarshalAs(UnmanagedType.U1)] bool val); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerPerNodeBufferSizeGet")] - public static extern int DataStreamerPerNodeBufferSizeGet(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerPerNodeBufferSizeSet")] - public static extern void DataStreamerPerNodeBufferSizeSet(void* ctx, void* obj, int val); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerPerNodeParallelOperationsGet")] - public static extern int DataStreamerPerNodeParallelOpsGet(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDataStreamerPerNodeParallelOperationsSet")] - public static extern void DataStreamerPerNodeParallelOpsSet(void* ctx, void* obj, int val); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteMessagingWithAsync")] - public static extern void* MessagingWithAsync(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionForOthers")] - public static extern void* ProjectionForOthers(void* ctx, void* obj, void* prj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionForRemotes")] - public static extern void* ProjectionForRemotes(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionForDaemons")] - public static extern void* ProjectionForDaemons(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionForRandom")] - public static extern void* ProjectionForRandom(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionForOldest")] - public static extern void* ProjectionForOldest(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionForYoungest")] - public static extern void* ProjectionForYoungest(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionResetMetrics")] - public static extern void ProjectionResetMetrics(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProjectionOutOpRet")] - public static extern void* ProjectionOutOpRet(void* ctx, void* obj, int type, long memPtr); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteQueryCursorIterator")] - public static extern void QryCursorIterator(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteQueryCursorClose")] - public static extern void QryCursorClose(void* ctx, void* target); - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAcquire")] public static extern void* Acquire(void* ctx, void* target); [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteRelease")] public static extern void Release(void* target); - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsStart")] - public static extern long TxStart(void* ctx, void* target, int concurrency, int isolation, long timeout, - int txSize); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsCommit")] - public static extern int TxCommit(void* ctx, void* target, long id); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsCommitAsync")] - public static extern void TxCommitAsync(void* ctx, void* target, long id, long futId); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsRollback")] - public static extern int TxRollback(void* ctx, void* target, long id); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsRollbackAsync")] - public static extern void TxRollbackAsync(void* ctx, void* target, long id, long futId); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsClose")] - public static extern int TxClose(void* ctx, void* target, long id); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsState")] - public static extern int TxState(void* ctx, void* target, long id); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsSetRollbackOnly")] - [return: MarshalAs(UnmanagedType.U1)] - public static extern bool TxSetRollbackOnly(void* ctx, void* target, long id); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTransactionsResetMetrics")] - public static extern void TxResetMetrics(void* ctx, void* target); - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteThrowToJava")] public static extern void ThrowToJava(void* ctx, char* msg); @@ -335,96 +176,8 @@ internal static unsafe class IgniteJniNativeMethods [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDestroyJvm")] public static extern void DestroyJvm(void* ctx); - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteEventsWithAsync")] - public static extern void* EventsWithAsync(void* ctx, void* obj); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteEventsStopLocalListen")] - [return: MarshalAs(UnmanagedType.U1)] - public static extern bool EventsStopLocalListen(void* ctx, void* obj, long hnd); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteEventsLocalListen")] - public static extern void EventsLocalListen(void* ctx, void* obj, long hnd, int type); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteEventsIsEnabled")] - [return: MarshalAs(UnmanagedType.U1)] - public static extern bool EventsIsEnabled(void* ctx, void* obj, int type); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteServicesWithAsync")] - public static extern void* ServicesWithAsync(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteServicesWithServerKeepPortable")] - public static extern void* ServicesWithServerKeepBinary(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteServicesCancel")] - public static extern long ServicesCancel(void* ctx, void* target, char* name); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteServicesCancelAll")] - public static extern long ServicesCancelAll(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteServicesGetServiceProxy")] - public static extern void* ServicesGetServiceProxy(void* ctx, void* target, char* name, - [MarshalAs(UnmanagedType.U1)] bool sticky); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongGet")] - public static extern long AtomicLongGet(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongIncrementAndGet")] - public static extern long AtomicLongIncrementAndGet(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongAddAndGet")] - public static extern long AtomicLongAddAndGet(void* ctx, void* target, long value); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongDecrementAndGet")] - public static extern long AtomicLongDecrementAndGet(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongGetAndSet")] - public static extern long AtomicLongGetAndSet(void* ctx, void* target, long value); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongCompareAndSetAndGet")] - public static extern long AtomicLongCompareAndSetAndGet(void* ctx, void* target, long expVal, long newVal); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongIsClosed")] - [return: MarshalAs(UnmanagedType.U1)] - public static extern bool AtomicLongIsClosed(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicLongClose")] - public static extern void AtomicLongClose(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicSequenceGet")] - public static extern long AtomicSequenceGet(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicSequenceIncrementAndGet")] - public static extern long AtomicSequenceIncrementAndGet(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicSequenceAddAndGet")] - public static extern long AtomicSequenceAddAndGet(void* ctx, void* target, long value); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicSequenceGetBatchSize")] - public static extern int AtomicSequenceGetBatchSize(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicSequenceSetBatchSize")] - public static extern void AtomicSequenceSetBatchSize(void* ctx, void* target, int size); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicSequenceIsClosed")] - [return: MarshalAs(UnmanagedType.U1)] - public static extern bool AtomicSequenceIsClosed(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicSequenceClose")] - public static extern void AtomicSequenceClose(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicReferenceIsClosed")] - [return: MarshalAs(UnmanagedType.U1)] - public static extern bool AtomicReferenceIsClosed(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAtomicReferenceClose")] - public static extern void AtomicReferenceClose(void* ctx, void* target); - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteListenableCancel")] [return: MarshalAs(UnmanagedType.U1)] public static extern bool ListenableCancel(void* ctx, void* target); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteListenableIsCancelled")] - [return: MarshalAs(UnmanagedType.U1)] - public static extern bool ListenableIsCancelled(void* ctx, void* target); } } \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs index 5815b4d9b2179..be55ab150e815 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs @@ -95,7 +95,7 @@ internal unsafe class UnmanagedCallbacks private const int OpPrepareDotNet = 1; private delegate long CacheStoreCreateCallbackDelegate(void* target, long memPtr); - private delegate int CacheStoreInvokeCallbackDelegate(void* target, long objPtr, long memPtr, void* cb); + private delegate int CacheStoreInvokeCallbackDelegate(void* target, long objPtr, long memPtr); private delegate void CacheStoreDestroyCallbackDelegate(void* target, long objPtr); private delegate long CacheStoreSessionCreateCallbackDelegate(void* target, long storePtr); @@ -305,20 +305,15 @@ private long CacheStoreCreate(void* target, long memPtr) } [SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope")] - private int CacheStoreInvoke(void* target, long objPtr, long memPtr, void* cb) + private int CacheStoreInvoke(void* target, long objPtr, long memPtr) { return SafeCall(() => { var t = _handleRegistry.Get(objPtr, true); - IUnmanagedTarget cb0 = null; - - if ((long) cb != 0) - cb0 = new UnmanagedNonReleaseableTarget(_ctx, cb); - using (PlatformMemoryStream stream = IgniteManager.Memory.Get(memPtr).GetStream()) { - return t.Invoke(stream, cb0, _ignite); + return t.Invoke(stream, _ignite); } }); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs index a0abfaac54dac..4722d0b17b92f 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs @@ -365,6 +365,11 @@ internal static void ProcessorGetCacheNames(IUnmanagedTarget target, long memPtr #region NATIVE METHODS: TARGET + internal static long TargetInLongOutLong(IUnmanagedTarget target, int opType, long memPtr) + { + return JNI.TargetInLongOutLong(target.Context, target.Target, opType, memPtr); + } + internal static long TargetInStreamOutLong(IUnmanagedTarget target, int opType, long memPtr) { return JNI.TargetInStreamOutLong(target.Context, target.Target, opType, memPtr); @@ -387,6 +392,16 @@ internal static void TargetInObjectStreamOutStream(IUnmanagedTarget target, int JNI.TargetInObjectStreamOutStream(target.Context, target.Target, opType, arg, inMemPtr, outMemPtr); } + internal static IUnmanagedTarget TargetInObjectStreamOutObjectStream(IUnmanagedTarget target, int opType, void* arg, long inMemPtr, long outMemPtr) + { + void* res = JNI.TargetInObjectStreamOutObjectStream(target.Context, target.Target, opType, arg, inMemPtr, outMemPtr); + + if (res == null) + return null; + + return target.ChangeTarget(res); + } + internal static long TargetOutLong(IUnmanagedTarget target, int opType) { return JNI.TargetOutLong(target.Context, target.Target, opType); @@ -431,345 +446,6 @@ internal static IUnmanagedTarget TargetListenFutureAndGet(IUnmanagedTarget targe #endregion - #region NATIVE METHODS: AFFINITY - - internal static int AffinityPartitions(IUnmanagedTarget target) - { - return JNI.AffinityParts(target.Context, target.Target); - } - - #endregion - - #region NATIVE METHODS: CACHE - - internal static IUnmanagedTarget CacheWithSkipStore(IUnmanagedTarget target) - { - void* res = JNI.CacheWithSkipStore(target.Context, target.Target); - - return target.ChangeTarget(res); - } - - internal static IUnmanagedTarget CacheWithNoRetries(IUnmanagedTarget target) - { - void* res = JNI.CacheWithNoRetries(target.Context, target.Target); - - return target.ChangeTarget(res); - } - - internal static IUnmanagedTarget CacheWithExpiryPolicy(IUnmanagedTarget target, long create, long update, long access) - { - void* res = JNI.CacheWithExpiryPolicy(target.Context, target.Target, create, update, access); - - return target.ChangeTarget(res); - } - - internal static IUnmanagedTarget CacheWithAsync(IUnmanagedTarget target) - { - void* res = JNI.CacheWithAsync(target.Context, target.Target); - - return target.ChangeTarget(res); - } - - internal static IUnmanagedTarget CacheWithKeepBinary(IUnmanagedTarget target) - { - void* res = JNI.CacheWithKeepBinary(target.Context, target.Target); - - return target.ChangeTarget(res); - } - - internal static void CacheClear(IUnmanagedTarget target) - { - JNI.CacheClear(target.Context, target.Target); - } - - internal static void CacheRemoveAll(IUnmanagedTarget target) - { - JNI.CacheRemoveAll(target.Context, target.Target); - } - - internal static IUnmanagedTarget CacheOutOpQueryCursor(IUnmanagedTarget target, int type, long memPtr) - { - void* res = JNI.CacheOutOpQueryCursor(target.Context, target.Target, type, memPtr); - - return target.ChangeTarget(res); - } - - internal static IUnmanagedTarget CacheOutOpContinuousQuery(IUnmanagedTarget target, int type, long memPtr) - { - void* res = JNI.CacheOutOpContinuousQuery(target.Context, target.Target, type, memPtr); - - return target.ChangeTarget(res); - } - - internal static IUnmanagedTarget CacheIterator(IUnmanagedTarget target) - { - void* res = JNI.CacheIterator(target.Context, target.Target); - - return target.ChangeTarget(res); - } - - internal static IUnmanagedTarget CacheLocalIterator(IUnmanagedTarget target, int peekModes) - { - void* res = JNI.CacheLocalIterator(target.Context, target.Target, peekModes); - - return target.ChangeTarget(res); - } - - internal static void CacheEnterLock(IUnmanagedTarget target, long id) - { - JNI.CacheEnterLock(target.Context, target.Target, id); - } - - internal static void CacheExitLock(IUnmanagedTarget target, long id) - { - JNI.CacheExitLock(target.Context, target.Target, id); - } - - internal static bool CacheTryEnterLock(IUnmanagedTarget target, long id, long timeout) - { - return JNI.CacheTryEnterLock(target.Context, target.Target, id, timeout); - } - - internal static void CacheCloseLock(IUnmanagedTarget target, long id) - { - JNI.CacheCloseLock(target.Context, target.Target, id); - } - - internal static void CacheRebalance(IUnmanagedTarget target, long futId) - { - JNI.CacheRebalance(target.Context, target.Target, futId); - } - - internal static void CacheStoreCallbackInvoke(IUnmanagedTarget target, long memPtr) - { - JNI.CacheStoreCallbackInvoke(target.Context, target.Target, memPtr); - } - - internal static int CacheSize(IUnmanagedTarget target, int modes, bool loc) - { - return JNI.CacheSize(target.Context, target.Target, modes, loc); - } - - #endregion - - #region NATIVE METHODS: COMPUTE - - internal static void ComputeWithNoFailover(IUnmanagedTarget target) - { - JNI.ComputeWithNoFailover(target.Context, target.Target); - } - - internal static void ComputeWithTimeout(IUnmanagedTarget target, long timeout) - { - JNI.ComputeWithTimeout(target.Context, target.Target, timeout); - } - - internal static IUnmanagedTarget ComputeExecuteNative(IUnmanagedTarget target, long taskPtr, long topVer) - { - void* res = JNI.ComputeExecuteNative(target.Context, target.Target, taskPtr, topVer); - - return target.ChangeTarget(res); - } - - #endregion - - #region NATIVE METHODS: CONTINUOUS QUERY - - internal static void ContinuousQueryClose(IUnmanagedTarget target) - { - JNI.ContinuousQryClose(target.Context, target.Target); - } - - internal static IUnmanagedTarget ContinuousQueryGetInitialQueryCursor(IUnmanagedTarget target) - { - void* res = JNI.ContinuousQryGetInitialQueryCursor(target.Context, target.Target); - - return res == null ? null : target.ChangeTarget(res); - } - - #endregion - - #region NATIVE METHODS: DATA STREAMER - - internal static void DataStreamerListenTopology(IUnmanagedTarget target, long ptr) - { - JNI.DataStreamerListenTop(target.Context, target.Target, ptr); - } - - internal static bool DataStreamerAllowOverwriteGet(IUnmanagedTarget target) - { - return JNI.DataStreamerAllowOverwriteGet(target.Context, target.Target); - } - - internal static void DataStreamerAllowOverwriteSet(IUnmanagedTarget target, bool val) - { - JNI.DataStreamerAllowOverwriteSet(target.Context, target.Target, val); - } - - internal static bool DataStreamerSkipStoreGet(IUnmanagedTarget target) - { - return JNI.DataStreamerSkipStoreGet(target.Context, target.Target); - } - - internal static void DataStreamerSkipStoreSet(IUnmanagedTarget target, bool val) - { - JNI.DataStreamerSkipStoreSet(target.Context, target.Target, val); - } - - internal static int DataStreamerPerNodeBufferSizeGet(IUnmanagedTarget target) - { - return JNI.DataStreamerPerNodeBufferSizeGet(target.Context, target.Target); - } - - internal static void DataStreamerPerNodeBufferSizeSet(IUnmanagedTarget target, int val) - { - JNI.DataStreamerPerNodeBufferSizeSet(target.Context, target.Target, val); - } - - internal static int DataStreamerPerNodeParallelOperationsGet(IUnmanagedTarget target) - { - return JNI.DataStreamerPerNodeParallelOpsGet(target.Context, target.Target); - } - - internal static void DataStreamerPerNodeParallelOperationsSet(IUnmanagedTarget target, int val) - { - JNI.DataStreamerPerNodeParallelOpsSet(target.Context, target.Target, val); - } - - #endregion - - #region NATIVE METHODS: MESSAGING - - internal static IUnmanagedTarget MessagingWithASync(IUnmanagedTarget target) - { - void* res = JNI.MessagingWithAsync(target.Context, target.Target); - - return target.ChangeTarget(res); - } - - #endregion - - #region NATIVE METHODS: PROJECTION - - internal static IUnmanagedTarget ProjectionForOthers(IUnmanagedTarget target, IUnmanagedTarget prj) - { - void* res = JNI.ProjectionForOthers(target.Context, target.Target, prj.Target); - - return target.ChangeTarget(res); - } - - internal static IUnmanagedTarget ProjectionForRemotes(IUnmanagedTarget target) - { - void* res = JNI.ProjectionForRemotes(target.Context, target.Target); - - return target.ChangeTarget(res); - } - - internal static IUnmanagedTarget ProjectionForDaemons(IUnmanagedTarget target) - { - void* res = JNI.ProjectionForDaemons(target.Context, target.Target); - - return target.ChangeTarget(res); - } - - internal static IUnmanagedTarget ProjectionForRandom(IUnmanagedTarget target) - { - void* res = JNI.ProjectionForRandom(target.Context, target.Target); - - return target.ChangeTarget(res); - } - - internal static IUnmanagedTarget ProjectionForOldest(IUnmanagedTarget target) - { - void* res = JNI.ProjectionForOldest(target.Context, target.Target); - - return target.ChangeTarget(res); - } - - internal static IUnmanagedTarget ProjectionForYoungest(IUnmanagedTarget target) - { - void* res = JNI.ProjectionForYoungest(target.Context, target.Target); - - return target.ChangeTarget(res); - } - - internal static void ProjectionResetMetrics(IUnmanagedTarget target) - { - JNI.ProjectionResetMetrics(target.Context, target.Target); - } - - internal static IUnmanagedTarget ProjectionOutOpRet(IUnmanagedTarget target, int type, long memPtr) - { - void* res = JNI.ProjectionOutOpRet(target.Context, target.Target, type, memPtr); - - return target.ChangeTarget(res); - } - - #endregion - - #region NATIVE METHODS: QUERY CURSOR - - internal static void QueryCursorIterator(IUnmanagedTarget target) - { - JNI.QryCursorIterator(target.Context, target.Target); - } - - internal static void QueryCursorClose(IUnmanagedTarget target) - { - JNI.QryCursorClose(target.Context, target.Target); - } - - #endregion - - #region NATIVE METHODS: TRANSACTIONS - - internal static long TransactionsStart(IUnmanagedTarget target, int concurrency, int isolation, long timeout, int txSize) - { - return JNI.TxStart(target.Context, target.Target, concurrency, isolation, timeout, txSize); - } - - internal static int TransactionsCommit(IUnmanagedTarget target, long id) - { - return JNI.TxCommit(target.Context, target.Target, id); - } - - internal static void TransactionsCommitAsync(IUnmanagedTarget target, long id, long futId) - { - JNI.TxCommitAsync(target.Context, target.Target, id, futId); - } - - internal static int TransactionsRollback(IUnmanagedTarget target, long id) - { - return JNI.TxRollback(target.Context, target.Target, id); - } - - internal static void TransactionsRollbackAsync(IUnmanagedTarget target, long id, long futId) - { - JNI.TxRollbackAsync(target.Context, target.Target, id, futId); - } - - internal static int TransactionsClose(IUnmanagedTarget target, long id) - { - return JNI.TxClose(target.Context, target.Target, id); - } - - internal static int TransactionsState(IUnmanagedTarget target, long id) - { - return JNI.TxState(target.Context, target.Target, id); - } - - internal static bool TransactionsSetRollbackOnly(IUnmanagedTarget target, long id) - { - return JNI.TxSetRollbackOnly(target.Context, target.Target, id); - } - - internal static void TransactionsResetMetrics(IUnmanagedTarget target) - { - JNI.TxResetMetrics(target.Context, target.Target); - } - - #endregion - #region NATIVE METHODS: MISCELANNEOUS internal static void Reallocate(long memPtr, int cap) @@ -827,176 +503,11 @@ internal static void DestroyJvm(void* ctx) JNI.DestroyJvm(ctx); } - #endregion - - #region NATIVE METHODS: EVENTS - - internal static IUnmanagedTarget EventsWithAsync(IUnmanagedTarget target) - { - return target.ChangeTarget(JNI.EventsWithAsync(target.Context, target.Target)); - } - - internal static bool EventsStopLocalListen(IUnmanagedTarget target, long handle) - { - return JNI.EventsStopLocalListen(target.Context, target.Target, handle); - } - - internal static bool EventsIsEnabled(IUnmanagedTarget target, int type) - { - return JNI.EventsIsEnabled(target.Context, target.Target, type); - } - - internal static void EventsLocalListen(IUnmanagedTarget target, long handle, int type) - { - JNI.EventsLocalListen(target.Context, target.Target, handle, type); - } - - #endregion - - #region NATIVE METHODS: SERVICES - - internal static IUnmanagedTarget ServicesWithAsync(IUnmanagedTarget target) - { - return target.ChangeTarget(JNI.ServicesWithAsync(target.Context, target.Target)); - } - - internal static IUnmanagedTarget ServicesWithServerKeepBinary(IUnmanagedTarget target) - { - return target.ChangeTarget(JNI.ServicesWithServerKeepBinary(target.Context, target.Target)); - } - - internal static void ServicesCancel(IUnmanagedTarget target, string name) - { - var nameChars = (char*)IgniteUtils.StringToUtf8Unmanaged(name); - - try - { - JNI.ServicesCancel(target.Context, target.Target, nameChars); - } - finally - { - Marshal.FreeHGlobal(new IntPtr(nameChars)); - } - } - - internal static void ServicesCancelAll(IUnmanagedTarget target) - { - JNI.ServicesCancelAll(target.Context, target.Target); - } - - internal static IUnmanagedTarget ServicesGetServiceProxy(IUnmanagedTarget target, string name, bool sticky) - { - var nameChars = (char*)IgniteUtils.StringToUtf8Unmanaged(name); - - try - { - return target.ChangeTarget(JNI.ServicesGetServiceProxy(target.Context, target.Target, nameChars, sticky)); - } - finally - { - Marshal.FreeHGlobal(new IntPtr(nameChars)); - } - } - - #endregion - - #region NATIVE METHODS: DATA STRUCTURES - - internal static long AtomicLongGet(IUnmanagedTarget target) - { - return JNI.AtomicLongGet(target.Context, target.Target); - } - - internal static long AtomicLongIncrementAndGet(IUnmanagedTarget target) - { - return JNI.AtomicLongIncrementAndGet(target.Context, target.Target); - } - - internal static long AtomicLongAddAndGet(IUnmanagedTarget target, long value) - { - return JNI.AtomicLongAddAndGet(target.Context, target.Target, value); - } - - internal static long AtomicLongDecrementAndGet(IUnmanagedTarget target) - { - return JNI.AtomicLongDecrementAndGet(target.Context, target.Target); - } - - internal static long AtomicLongGetAndSet(IUnmanagedTarget target, long value) - { - return JNI.AtomicLongGetAndSet(target.Context, target.Target, value); - } - - internal static long AtomicLongCompareAndSetAndGet(IUnmanagedTarget target, long expVal, long newVal) - { - return JNI.AtomicLongCompareAndSetAndGet(target.Context, target.Target, expVal, newVal); - } - - internal static bool AtomicLongIsClosed(IUnmanagedTarget target) - { - return JNI.AtomicLongIsClosed(target.Context, target.Target); - } - - internal static void AtomicLongClose(IUnmanagedTarget target) - { - JNI.AtomicLongClose(target.Context, target.Target); - } - - internal static long AtomicSequenceGet(IUnmanagedTarget target) - { - return JNI.AtomicSequenceGet(target.Context, target.Target); - } - - internal static long AtomicSequenceIncrementAndGet(IUnmanagedTarget target) - { - return JNI.AtomicSequenceIncrementAndGet(target.Context, target.Target); - } - - internal static long AtomicSequenceAddAndGet(IUnmanagedTarget target, long value) - { - return JNI.AtomicSequenceAddAndGet(target.Context, target.Target, value); - } - - internal static int AtomicSequenceGetBatchSize(IUnmanagedTarget target) - { - return JNI.AtomicSequenceGetBatchSize(target.Context, target.Target); - } - - internal static void AtomicSequenceSetBatchSize(IUnmanagedTarget target, int size) - { - JNI.AtomicSequenceSetBatchSize(target.Context, target.Target, size); - } - - internal static bool AtomicSequenceIsClosed(IUnmanagedTarget target) - { - return JNI.AtomicSequenceIsClosed(target.Context, target.Target); - } - - internal static void AtomicSequenceClose(IUnmanagedTarget target) - { - JNI.AtomicSequenceClose(target.Context, target.Target); - } - - internal static bool AtomicReferenceIsClosed(IUnmanagedTarget target) - { - return JNI.AtomicReferenceIsClosed(target.Context, target.Target); - } - - internal static void AtomicReferenceClose(IUnmanagedTarget target) - { - JNI.AtomicReferenceClose(target.Context, target.Target); - } - internal static bool ListenableCancel(IUnmanagedTarget target) { return JNI.ListenableCancel(target.Context, target.Target); } - internal static bool ListenableIsCancelled(IUnmanagedTarget target) - { - return JNI.ListenableIsCancelled(target.Context, target.Target); - } - #endregion } } From b1ec58f716ece3a5866dd654ebc707bef67caf57 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Thu, 13 Oct 2016 12:58:22 +0700 Subject: [PATCH 240/487] IGNITE-4066 Fixed NPE. --- .../visor/cache/VisorCacheTypeMetadata.java | 70 ++++++++++--------- 1 file changed, 37 insertions(+), 33 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java index 870e0333eb0c5..f17e5889663fc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java @@ -111,51 +111,55 @@ public static Collection list(Collection qr if (factory != null && factory instanceof CacheJdbcPojoStoreFactory) { CacheJdbcPojoStoreFactory jdbcFactory = (CacheJdbcPojoStoreFactory) factory; - for (JdbcType jdbcType : jdbcFactory.getTypes()) { - VisorCacheTypeMetadata meta = metaMap.get(jdbcType.getKeyType()); + JdbcType[] jdbcTypes = jdbcFactory.getTypes(); - boolean notFound = meta == null; + if (jdbcTypes != null && jdbcTypes.length > 0) { + for (JdbcType jdbcType : jdbcTypes) { + VisorCacheTypeMetadata meta = metaMap.get(jdbcType.getKeyType()); - if (notFound) { - meta = new VisorCacheTypeMetadata(); + boolean notFound = meta == null; - meta.keyType = jdbcType.getKeyType(); - meta.valType = jdbcType.getValueType(); + if (notFound) { + meta = new VisorCacheTypeMetadata(); - meta.qryFlds = Collections.emptyMap(); - meta.ascFlds = Collections.emptyMap(); - meta.descFlds = Collections.emptyMap(); - meta.txtFlds = Collections.emptyList(); - meta.grps = Collections.emptyMap(); - } + meta.keyType = jdbcType.getKeyType(); + meta.valType = jdbcType.getValueType(); - meta.dbSchema = jdbcType.getDatabaseSchema(); - meta.dbTbl = jdbcType.getDatabaseTable(); + meta.qryFlds = Collections.emptyMap(); + meta.ascFlds = Collections.emptyMap(); + meta.descFlds = Collections.emptyMap(); + meta.txtFlds = Collections.emptyList(); + meta.grps = Collections.emptyMap(); + } - JdbcTypeField[] keyFields = jdbcType.getKeyFields(); + meta.dbSchema = jdbcType.getDatabaseSchema(); + meta.dbTbl = jdbcType.getDatabaseTable(); - if (keyFields != null) { - meta.keyFields = new ArrayList<>(keyFields.length); + JdbcTypeField[] keyFields = jdbcType.getKeyFields(); - for (JdbcTypeField fld : keyFields) - meta.keyFields.add(new VisorCacheTypeFieldMetadata( - fld.getDatabaseFieldName(), fld.getDatabaseFieldType(), - fld.getDatabaseFieldName(), U.compact(fld.getJavaFieldType().getName()))); - } + if (keyFields != null) { + meta.keyFields = new ArrayList<>(keyFields.length); - JdbcTypeField[] valFields = jdbcType.getValueFields(); + for (JdbcTypeField fld : keyFields) + meta.keyFields.add(new VisorCacheTypeFieldMetadata( + fld.getDatabaseFieldName(), fld.getDatabaseFieldType(), + fld.getDatabaseFieldName(), U.compact(fld.getJavaFieldType().getName()))); + } - if (valFields != null) { - meta.valFields = new ArrayList<>(valFields.length); + JdbcTypeField[] valFields = jdbcType.getValueFields(); - for (JdbcTypeField fld : valFields) - meta.valFields.add(new VisorCacheTypeFieldMetadata( - fld.getDatabaseFieldName(), fld.getDatabaseFieldType(), - fld.getDatabaseFieldName(), U.compact(fld.getJavaFieldType().getName()))); - } + if (valFields != null) { + meta.valFields = new ArrayList<>(valFields.length); - if (notFound) - metas.add(meta); + for (JdbcTypeField fld : valFields) + meta.valFields.add(new VisorCacheTypeFieldMetadata( + fld.getDatabaseFieldName(), fld.getDatabaseFieldType(), + fld.getDatabaseFieldName(), U.compact(fld.getJavaFieldType().getName()))); + } + + if (notFound) + metas.add(meta); + } } } From 447e07c0bb5af75bce6a14612606904e4e3d9705 Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Fri, 14 Oct 2016 11:40:41 +0300 Subject: [PATCH 241/487] IGNITE-1924 Incomplete marshaller cache rebalancing causes Grid hangs under SSL --- .../dht/GridDhtAffinityAssignmentRequest.java | 2 +- .../util/nio/GridNioSessionMetaKey.java | 7 +- .../util/nio/ssl/BlockingSslHandler.java | 9 +- .../util/nio/ssl/GridNioSslFilter.java | 63 ++++++++++--- .../util/nio/ssl/GridNioSslHandler.java | 17 +++- .../internal/util/nio/ssl/GridSslMeta.java | 94 +++++++++++++++++++ .../tcp/TcpCommunicationSpi.java | 49 ++++++---- ...unicationSpiConcurrentConnectSelfTest.java | 14 +++ ...cationSpiConcurrentConnectSslSelfTest.java | 35 +++++++ ...idTcpCommunicationSpiRecoverySelfTest.java | 14 +++ ...cpCommunicationSpiRecoverySslSelfTest.java | 35 +++++++ .../tcp/IgniteCacheSslStartStopSelfTest.java | 1 - .../IgniteSpiCommunicationSelfTestSuite.java | 4 + 13 files changed, 301 insertions(+), 43 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridSslMeta.java create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSslSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySslSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java index 0ef76063a58ca..6959906ab591f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java @@ -130,6 +130,6 @@ public GridDhtAffinityAssignmentRequest(int cacheId, @NotNull AffinityTopologyVe /** {@inheritDoc} */ @Override public String toString() { - return S.toString(GridDhtAffinityAssignmentRequest.class, this); + return S.toString(GridDhtAffinityAssignmentRequest.class, this, super.toString()); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionMetaKey.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionMetaKey.java index b5387d9cb9d7e..bdb3a2955fe25 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionMetaKey.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionMetaKey.java @@ -26,8 +26,8 @@ public enum GridNioSessionMetaKey { /** NIO parser state. */ PARSER_STATE, - /** SSL handler. */ - SSL_HANDLER, + /** SSL Meta. */ + SSL_META, /** NIO operation (request type). */ NIO_OPERATION, @@ -44,9 +44,6 @@ public enum GridNioSessionMetaKey { /** Message writer. */ MSG_WRITER, - /** SSL engine. */ - SSL_ENGINE, - /** Ack closure. */ ACK_CLOSURE; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java index bb566cd376729..273df5d9d69f7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java @@ -112,6 +112,13 @@ public BlockingSslHandler(SSLEngine sslEngine, log.debug("Started SSL session [netBufSize=" + netBufSize + ", appBufSize=" + appBuf.capacity() + ']'); } + /** + * + */ + public ByteBuffer inputBuffer(){ + return inNetBuf; + } + /** * Performs handshake procedure with remote peer. * @@ -255,7 +262,7 @@ public ByteBuffer encrypt(ByteBuffer src) throws SSLException { * @throws SSLException If failed to process SSL data. */ public ByteBuffer decode(ByteBuffer buf) throws IgniteCheckedException, SSLException { - inNetBuf.clear(); + appBuf.clear(); if (buf.limit() > inNetBuf.remaining()) { inNetBuf = expandBuffer(inNetBuf, inNetBuf.capacity() + buf.limit() * 2); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java index 63cdd83d3dc9f..d6f9d106264f1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java @@ -35,7 +35,7 @@ import org.apache.ignite.internal.util.nio.GridNioSessionMetaKey; import org.apache.ignite.internal.util.typedef.internal.U; -import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_HANDLER; +import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META; /** * Implementation of SSL filter using {@link SSLEngine} @@ -154,28 +154,61 @@ public void enabledProtocols(String... enabledProtos) { if (log.isDebugEnabled()) log.debug("Remote client connected, creating SSL handler and performing initial handshake: " + ses); - SSLEngine engine = sslCtx.createSSLEngine(); + SSLEngine engine; - engine.setUseClientMode(clientMode); + boolean handshake; - if (!clientMode) { - engine.setWantClientAuth(wantClientAuth); + GridSslMeta sslMeta = ses.meta(SSL_META.ordinal()); - engine.setNeedClientAuth(needClientAuth); + if (sslMeta == null) { + engine = sslCtx.createSSLEngine(); + + engine.setUseClientMode(clientMode); + + if (!clientMode) { + engine.setWantClientAuth(wantClientAuth); + + engine.setNeedClientAuth(needClientAuth); + } + + if (enabledCipherSuites != null) + engine.setEnabledCipherSuites(enabledCipherSuites); + + if (enabledProtos != null) + engine.setEnabledProtocols(enabledProtos); + + sslMeta = new GridSslMeta(); + + ses.addMeta(SSL_META.ordinal(), sslMeta); + + handshake = true; } + else { + engine = sslMeta.sslEngine(); - if (enabledCipherSuites != null) - engine.setEnabledCipherSuites(enabledCipherSuites); + assert engine != null; - if (enabledProtos != null) - engine.setEnabledProtocols(enabledProtos); + handshake = false; + } try { - GridNioSslHandler hnd = new GridNioSslHandler(this, ses, engine, directBuf, order, log); + GridNioSslHandler hnd = new GridNioSslHandler(this, + ses, + engine, + directBuf, + order, + log, + handshake, + sslMeta.encodedBuffer()); - ses.addMeta(SSL_HANDLER.ordinal(), hnd); + sslMeta.handler(hnd); hnd.handshake(); + + ByteBuffer alreadyDecoded = sslMeta.decodedBuffer(); + + if (alreadyDecoded != null) + proceedMessageReceived(ses, alreadyDecoded); } catch (SSLException e) { U.error(log, "Failed to start SSL handshake (will close inbound connection): " + ses, e); @@ -379,7 +412,11 @@ private GridNioFuture shutdownSession(GridNioSession ses, GridNioSslHan * @return SSL handler. */ private GridNioSslHandler sslHandler(GridNioSession ses) { - GridNioSslHandler hnd = ses.meta(SSL_HANDLER.ordinal()); + GridSslMeta sslMeta = ses.meta(SSL_META.ordinal()); + + assert sslMeta != null; + + GridNioSslHandler hnd = sslMeta.handler(); if (hnd == null) throw new IgniteException("Failed to process incoming message (received message before SSL handler " + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java index 3272b8ec5473a..eb8dad48a3134 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java @@ -43,7 +43,6 @@ import static javax.net.ssl.SSLEngineResult.Status; import static javax.net.ssl.SSLEngineResult.Status.BUFFER_UNDERFLOW; import static javax.net.ssl.SSLEngineResult.Status.CLOSED; -import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_ENGINE; import static org.apache.ignite.internal.util.nio.ssl.GridNioSslFilter.HANDSHAKE_FUT_META_KEY; /** @@ -104,6 +103,8 @@ class GridNioSslHandler extends ReentrantLock { * @param log Logger to use. * @param directBuf Direct buffer flag. * @param order Byte order. + * @param handshake is handshake required. + * @param encBuf encoded buffer to be used. * @throws SSLException If exception occurred when starting SSL handshake. */ GridNioSslHandler(GridNioSslFilter parent, @@ -111,7 +112,9 @@ class GridNioSslHandler extends ReentrantLock { SSLEngine engine, boolean directBuf, ByteOrder order, - IgniteLogger log) throws SSLException { + IgniteLogger log, + boolean handshake, + ByteBuffer encBuf) throws SSLException { assert parent != null; assert ses != null; assert engine != null; @@ -125,11 +128,9 @@ class GridNioSslHandler extends ReentrantLock { sslEngine = engine; - if (ses.meta(SSL_ENGINE.ordinal()) == null) + if (handshake) sslEngine.beginHandshake(); else { - sslEngine = ses.meta(SSL_ENGINE.ordinal()); - handshakeFinished = true; initHandshakeComplete = true; } @@ -147,6 +148,12 @@ class GridNioSslHandler extends ReentrantLock { inNetBuf.order(order); + if (encBuf != null) { + encBuf.flip(); + + inNetBuf.put(encBuf); // Buffer contains bytes read but not handled by sslEngine at BlockingSslHandler. + } + // Initially buffer is empty. outNetBuf.position(0); outNetBuf.limit(0); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridSslMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridSslMeta.java new file mode 100644 index 0000000000000..5ca68b7557405 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridSslMeta.java @@ -0,0 +1,94 @@ +/* + * 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.util.nio.ssl; + +import java.nio.ByteBuffer; +import javax.net.ssl.SSLEngine; + +/** + * + */ +public class GridSslMeta { + /** GridNioSslHandler. */ + private GridNioSslHandler hnd; + + /** Data already decoded by blocking ssl handler. */ + private ByteBuffer decodedBuf; + + /** Data read but not decoded by blocking ssl handler. */ + private ByteBuffer encodedBuf; + + /** Ssl engine. */ + private SSLEngine sslEngine; + + /** + * + */ + public SSLEngine sslEngine() { + return sslEngine; + } + + /** + * @param sslEngine Ssl engine. + */ + public void sslEngine(SSLEngine sslEngine) { + this.sslEngine = sslEngine; + } + + /** + * + */ + public GridNioSslHandler handler() { + return hnd; + } + + /** + * @param hnd Handler. + */ + public void handler(GridNioSslHandler hnd) { + this.hnd = hnd; + } + + /** + * + */ + public ByteBuffer decodedBuffer() { + return decodedBuf; + } + + /** + * @param decodedBuf Decoded buffer. + */ + public void decodedBuffer(ByteBuffer decodedBuf) { + this.decodedBuf = decodedBuf; + } + + /** + * + */ + public ByteBuffer encodedBuffer() { + return encodedBuf; + } + + /** + * @param encodedBuf Encoded buffer. + */ + public void encodedBuffer(ByteBuffer encodedBuf) { + this.encodedBuf = encodedBuf; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index d61cee6fdaa8e..0bbb4d3ef243e 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -90,6 +90,7 @@ import org.apache.ignite.internal.util.nio.GridTcpNioCommunicationClient; import org.apache.ignite.internal.util.nio.ssl.BlockingSslHandler; import org.apache.ignite.internal.util.nio.ssl.GridNioSslFilter; +import org.apache.ignite.internal.util.nio.ssl.GridSslMeta; import org.apache.ignite.internal.util.typedef.CI2; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; @@ -131,6 +132,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; +import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META; /** * TcpCommunicationSpi is default communication SPI which uses @@ -2376,19 +2378,28 @@ protected GridCommunicationClient createTcpClient(ClusterNode node) throws Ignit long rcvCnt = -1; - SSLEngine sslEngine = null; + Map meta = new HashMap<>(); + + GridSslMeta sslMeta = null; try { ch.socket().connect(addr, (int)timeoutHelper.nextTimeoutChunk(connTimeout)); if (isSslEnabled()) { - sslEngine = ignite.configuration().getSslContextFactory().create().createSSLEngine(); + meta.put(SSL_META.ordinal(), sslMeta = new GridSslMeta()); + + SSLEngine sslEngine = ignite.configuration().getSslContextFactory().create().createSSLEngine(); sslEngine.setUseClientMode(true); + + sslMeta.sslEngine(sslEngine); } - rcvCnt = safeHandshake(ch, recoveryDesc, node.id(), - timeoutHelper.nextTimeoutChunk(connTimeout0), sslEngine); + rcvCnt = safeHandshake(ch, + recoveryDesc, + node.id(), + timeoutHelper.nextTimeoutChunk(connTimeout0), + sslMeta); if (rcvCnt == -1) return null; @@ -2399,16 +2410,8 @@ protected GridCommunicationClient createTcpClient(ClusterNode node) throws Ignit } try { - Map meta = new HashMap<>(); - meta.put(NODE_ID_META, node.id()); - if (isSslEnabled()) { - assert sslEngine != null; - - meta.put(GridNioSessionMetaKey.SSL_ENGINE.ordinal(), sslEngine); - } - if (recoveryDesc != null) { recoveryDesc.onHandshake(rcvCnt); @@ -2560,7 +2563,7 @@ else if (X.hasCause(e, SocketTimeoutException.class)) * @param recovery Recovery descriptor if use recovery handshake, otherwise {@code null}. * @param rmtNodeId Remote node. * @param timeout Timeout for handshake. - * @param ssl SSL engine if used cryptography, otherwise {@code null}. + * @param sslMeta Session meta. * @throws IgniteCheckedException If handshake failed or wasn't completed withing timeout. * @return Handshake response. */ @@ -2570,7 +2573,7 @@ private long safeHandshake( @Nullable GridNioRecoveryDescriptor recovery, UUID rmtNodeId, long timeout, - @Nullable SSLEngine ssl + GridSslMeta sslMeta ) throws IgniteCheckedException { HandshakeTimeoutObject obj = new HandshakeTimeoutObject<>(client, U.currentTimeMillis() + timeout); @@ -2592,7 +2595,9 @@ private long safeHandshake( ByteBuffer buf; if (isSslEnabled()) { - sslHnd = new BlockingSslHandler(ssl, ch, directBuf, ByteOrder.nativeOrder(), log); + assert sslMeta != null; + + sslHnd = new BlockingSslHandler(sslMeta.sslEngine(), ch, directBuf, ByteOrder.nativeOrder(), log); if (!sslHnd.handshake()) throw new IgniteCheckedException("SSL handshake is not completed."); @@ -2710,11 +2715,21 @@ else if (log.isDebugEnabled()) i += decode.remaining(); - buf.flip(); - buf.compact(); + buf.clear(); } rcvCnt = decode.getLong(1); + + if (decode.limit() > 9) { + decode.position(9); + + sslMeta.decodedBuffer(decode); + } + + ByteBuffer inBuf = sslHnd.inputBuffer(); + + if (inBuf.position() > 0) + sslMeta.encodedBuffer(inBuf); } else { buf = ByteBuffer.allocate(9); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java index 97eb34caaa9ae..8635d94e2bc8b 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java @@ -33,6 +33,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; import org.apache.ignite.internal.util.lang.GridAbsPredicate; @@ -49,6 +50,7 @@ import org.apache.ignite.testframework.GridSpiTestContext; import org.apache.ignite.testframework.GridTestNode; import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.IgniteMock; import org.apache.ignite.testframework.junits.IgniteTestResources; import org.apache.ignite.testframework.junits.spi.GridSpiAbstractTest; import org.apache.ignite.testframework.junits.spi.GridSpiTest; @@ -78,6 +80,9 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest /** */ protected static int port = 30_000; + /** Use ssl. */ + protected boolean useSsl; + /** * */ @@ -670,6 +675,15 @@ private void startSpis() throws Exception { rsrcs.inject(spi); + if (useSsl) { + IgniteMock ignite = GridTestUtils.getFieldValue(spi, IgniteSpiAdapter.class, "ignite"); + + IgniteConfiguration cfg = ignite.configuration() + .setSslContextFactory(GridTestUtils.sslFactory()); + + ignite.setStaticCfg(cfg); + } + spi.setListener(new TestListener()); node.setAttributes(spi.getNodeAttributes()); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySslSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySslSelfTest.java new file mode 100644 index 0000000000000..4182ea101b80e --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySslSelfTest.java @@ -0,0 +1,35 @@ +/* + * 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.spi.communication.tcp; + +/** + * + */ +public class GridTcpCommunicationSpiRecoverySslSelfTest extends GridTcpCommunicationSpiRecoverySelfTest { + /** + * Default constructor. + */ + public GridTcpCommunicationSpiRecoverySslSelfTest() { + useSsl = true; + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return super.getTestTimeout() * 2; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteCacheSslStartStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteCacheSslStartStopSelfTest.java index 12bd020a4674a..3324fcc5d2884 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteCacheSslStartStopSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteCacheSslStartStopSelfTest.java @@ -32,7 +32,6 @@ public class IgniteCacheSslStartStopSelfTest extends IgniteCachePutRetryAbstract @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); - fail("https://issues.apache.org/jira/browse/IGNITE-1924"); cfg.setSslContextFactory(GridTestUtils.sslFactory()); return cfg; diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java index d48c2c4ae4499..eb88524cddbda 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java @@ -19,12 +19,14 @@ import junit.framework.TestSuite; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiConcurrentConnectSelfTest; +import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiConcurrentConnectSslSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiConfigSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiMultithreadedSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiMultithreadedShmemTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoveryAckSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoverySelfTest; +import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoverySslSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiShmemSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiStartStopSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiTcpFailureDetectionSelfTest; @@ -46,8 +48,10 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoveryAckSelfTest.class)); suite.addTest(new TestSuite(IgniteTcpCommunicationRecoveryAckClosureSelfTest.class)); suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoverySelfTest.class)); + suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoverySslSelfTest.class)); suite.addTest(new TestSuite(GridTcpCommunicationSpiConcurrentConnectSelfTest.class)); + suite.addTest(new TestSuite(GridTcpCommunicationSpiConcurrentConnectSslSelfTest.class)); suite.addTest(new TestSuite(GridTcpCommunicationSpiTcpSelfTest.class)); suite.addTest(new TestSuite(GridTcpCommunicationSpiTcpNoDelayOffSelfTest.class)); From 7adfbcf1ee7bbe0beb95fa82749a2e04449de8fa Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Fri, 14 Oct 2016 17:48:14 +0300 Subject: [PATCH 242/487] IGNITE-4060: Fixed a bug in RoundRobinLoadBalancing API causing exception when running closures after client node reconnect. This closes #1157. --- .../RoundRobinGlobalLoadBalancer.java | 16 ++- ...undRobinErrorAfterClientReconnectTest.java | 99 +++++++++++++++++++ .../IgniteComputeGridTestSuite.java | 2 + 3 files changed, 114 insertions(+), 3 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/IgniteRoundRobinErrorAfterClientReconnectTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java index c48b8199021fa..9ce854fa90f40 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java @@ -26,7 +26,6 @@ import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; @@ -38,6 +37,7 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.IgniteSpiContext; +import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED; import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_JOINED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; @@ -101,6 +101,16 @@ void onContextInitialized(final IgniteSpiContext ctx) { nodeList = new GridNodeList(0, newNodes); } } + else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { + Collection nodes = ((DiscoveryEvent)evt).topologyNodes(); + + List newNodes = new ArrayList<>(nodes.size()); + + for (ClusterNode node : nodes) + newNodes.add(node.id()); + + nodeList = new GridNodeList(0, newNodes); + } else { assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED; @@ -119,7 +129,7 @@ void onContextInitialized(final IgniteSpiContext ctx) { } } }, - EVT_NODE_FAILED, EVT_NODE_JOINED, EVT_NODE_LEFT + EVT_NODE_FAILED, EVT_NODE_JOINED, EVT_NODE_LEFT, EVT_CLIENT_NODE_RECONNECTED ); synchronized (mux) { @@ -147,7 +157,7 @@ void onContextDestroyed() { * * @param top Topology to pick from. * @return Best balanced node. - * @throws IgniteCheckedException Thrown in case of any error. + * @throws IgniteException Thrown in case of any error. */ ClusterNode getBalancedNode(Collection top) throws IgniteException { assert !F.isEmpty(top); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteRoundRobinErrorAfterClientReconnectTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteRoundRobinErrorAfterClientReconnectTest.java new file mode 100644 index 0000000000000..0ec3b6b5374f3 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteRoundRobinErrorAfterClientReconnectTest.java @@ -0,0 +1,99 @@ +/* + * 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.Ignite; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.Event; +import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Test to reproduce IGNITE-4060. + */ +public class IgniteRoundRobinErrorAfterClientReconnectTest extends GridCommonAbstractTest { + /** Server index. */ + private static final int SRV_IDX = 0; + + /** Client index. */ + private static final int CLI_IDX = 1; + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + startGrid(SRV_IDX); + startGrid(CLI_IDX); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + if (gridName.endsWith("1")) + cfg.setClientMode(true); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 60000; + } + + /** + * @throws Exception If failed. + */ + public void testClientReconnect() throws Exception { + final Ignite cli = grid(CLI_IDX); + + final GridFutureAdapter fut = new GridFutureAdapter<>(); + + cli.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event event) { + try { + cli.compute().apply(new IgniteClosure() { + @Override public Void apply(String arg) { + return null; + } + }, "Hello!"); + + fut.onDone(true); + + return true; + } + catch (Exception e) { + fut.onDone(e); + + return false; + } + } + }, EventType.EVT_CLIENT_NODE_RECONNECTED); + + stopGrid(SRV_IDX); + startGrid(SRV_IDX); + + assert fut.get(); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java index 699ad0c920f2d..a1a75f8ada2a0 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java @@ -66,6 +66,7 @@ import org.apache.ignite.internal.IgniteComputeTopologyExceptionTest; import org.apache.ignite.internal.IgniteExecutorServiceTest; import org.apache.ignite.internal.IgniteExplicitImplicitDeploymentSelfTest; +import org.apache.ignite.internal.IgniteRoundRobinErrorAfterClientReconnectTest; import org.apache.ignite.internal.TaskNodeRestartTest; import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManagerSelfTest; import org.apache.ignite.internal.managers.checkpoint.GridCheckpointTaskSelfTest; @@ -148,6 +149,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteComputeTopologyExceptionTest.class); suite.addTestSuite(GridTaskFailoverAffinityRunTest.class); suite.addTestSuite(TaskNodeRestartTest.class); + suite.addTestSuite(IgniteRoundRobinErrorAfterClientReconnectTest.class); return suite; } From 80abd1b72e4fc7b0b95212e7f53c700c0fe21156 Mon Sep 17 00:00:00 2001 From: Aleksei Scherbakov Date: Fri, 14 Oct 2016 19:33:07 +0300 Subject: [PATCH 243/487] GG-11360 - Implement SQL queries cancellation (#18) * GG-11360 Merged IGNITE-2680 to ignite-1.6.3. * GG-11360 Test cleanup. * GG-11360 Fixing broken tests. * GG-11360 Fixing test. * GG-11360 Fixing test. * GG-11360 Fixing broken tests. * GG-11360 Added test for forever-running query cancellation on node restart. * GG-11360 Fixing race. * GG-11360 Added test for forever-running query cancellation on node stop. * GG-11360 Cleanup. * GG-11360 Support for local query cancellation/timeout. * GG-11360 Increase test duration. * GG-11360 Remove redundant catch block. * GG-11360 Fix formatting. * GG-11360 Fix formatting. * GG-11360 Fix formatting. * GG-11360 Fix formatting. * GG-11360 Fix formatting. * GG-11360 Fix formatting. * GG-11360 Fix formatting. * GG-11360 Fix formatting. * GG-11360 Simplify test. * GG-11360 Simplify test. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * GG-11360 Fixing issues. * Merge remote-tracking branch 'remotes/gg/ignite-1.6.10' into ignite-gg-11360 Conflicts: modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java * GG-11360 Review fixes. * GG-11360 Review fixes. * GG-11360 Review fixes. * GG-11360 Review fixes. * GG-11360 Review fixes. * GG-11360 Review fixes. --- .../org/apache/ignite/IgniteScheduler.java | 13 + .../cache/query/QueryCancelledException.java | 35 +++ .../ignite/cache/query/QueryCursor.java | 8 +- .../ignite/cache/query/SqlFieldsQuery.java | 26 ++ .../apache/ignite/cache/query/SqlQuery.java | 25 ++ .../ignite/internal/IgniteSchedulerImpl.java | 18 ++ .../processors/cache/QueryCursorImpl.java | 92 +++++-- .../processors/query/GridQueryCancel.java | 84 ++++++ .../query/GridQueryFieldsResult.java | 3 +- .../query/GridQueryFieldsResultAdapter.java | 3 +- .../processors/query/GridQueryIndexing.java | 26 +- .../processors/query/GridQueryProcessor.java | 115 ++++---- .../messages/GridQueryFailResponse.java | 34 ++- .../h2/twostep/messages/GridQueryRequest.java | 31 ++- .../processors/query/h2/IgniteH2Indexing.java | 177 ++++++++++--- .../h2/twostep/GridMapQueryExecutor.java | 42 ++- .../h2/twostep/GridReduceQueryExecutor.java | 99 +++++-- ...teCacheDistributedQueryCancelSelfTest.java | 176 +++++++++++++ ...tedQueryStopOnCancelOrTimeoutSelfTest.java | 248 ++++++++++++++++++ .../IgniteCacheQueryNodeRestartSelfTest2.java | 125 +++++---- ...acheLocalQueryCancelOrTimeoutSelfTest.java | 158 +++++++++++ .../h2/GridIndexingSpiAbstractSelfTest.java | 9 +- .../IgniteCacheQuerySelfTestSuite2.java | 8 + 23 files changed, 1341 insertions(+), 214 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQueryCancelOrTimeoutSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteScheduler.java b/modules/core/src/main/java/org/apache/ignite/IgniteScheduler.java index b0ea960d73cac..2e2553b95213d 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteScheduler.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteScheduler.java @@ -17,7 +17,9 @@ package org.apache.ignite; +import java.io.Closeable; import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgniteOutClosure; import org.apache.ignite.lang.IgniteRunnable; @@ -60,6 +62,17 @@ public interface IgniteScheduler { */ public IgniteFuture runLocal(@Nullable Runnable r); + /** + * Executes given closure after the delay. + *

          + * Note that class {@link IgniteRunnable} implements {@link Runnable} + * @param r Runnable to execute. + * @param delay Initial delay. + * @param timeUnit Time granularity. + * @return java.io.Closeable which can be used to cancel execution. + */ + public Closeable runLocal(@Nullable Runnable r, long delay, TimeUnit timeUnit); + /** * Executes given callable on internal system thread pool asynchronously. *

          diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java new file mode 100644 index 0000000000000..5f5ffdce163fe --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCancelledException.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.query; + +import org.apache.ignite.IgniteCheckedException; + +/** + * The exception is thrown if a query was cancelled or timed out while executing. + */ +public class QueryCancelledException extends IgniteCheckedException { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Default constructor. + */ + public QueryCancelledException() { + super("The query was cancelled while executing."); + } +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCursor.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCursor.java index 84b6862b889d0..1af47ac6de33a 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCursor.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryCursor.java @@ -22,8 +22,6 @@ /** * Query result cursor. Implements {@link Iterable} only for convenience, e.g. {@link #iterator()} * can be obtained only once. Also if iteration is started then {@link #getAll()} method calls are prohibited. - *

          - * Not thread safe and must be used from single thread only. */ public interface QueryCursor extends Iterable, AutoCloseable { /** @@ -39,7 +37,11 @@ public interface QueryCursor extends Iterable, AutoCloseable { public List getAll(); /** - * Closes all resources related to this cursor. + * Closes all resources related to this cursor. If the query execution is in progress + * (which is possible in case of invoking from another thread), a cancel will be attempted. + * Sequential calls to this method have no effect. + *

          + * Note: don't forget to close query cursors. Not doing so may lead to various resource leaks. */ @Override public void close(); } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java index b2dd181ca5786..d1a5117288005 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java @@ -18,7 +18,9 @@ package org.apache.ignite.cache.query; import java.util.List; +import java.util.concurrent.TimeUnit; import org.apache.ignite.IgniteCache; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; @@ -56,6 +58,9 @@ public final class SqlFieldsQuery extends Query> { /** Collocation flag. */ private boolean collocated; + /** Query timeout in millis. */ + private int timeout; + /** * Constructs SQL fields query. * @@ -120,6 +125,27 @@ public SqlFieldsQuery setArgs(Object... args) { return this; } + /** + * Gets the query execution timeout in milliseconds. + * + * @return Timeout value. + */ + public int getTimeout() { + return timeout; + } + + /** + * Sets the query execution timeout. Query will be automatically cancelled if the execution timeout is exceeded. + * @param timeout Timeout value. Zero value disables timeout. + * @param timeUnit Time unit. + * @return {@code this} For chaining. + */ + public SqlFieldsQuery setTimeout(int timeout, TimeUnit timeUnit) { + this.timeout = GridQueryProcessor.validateTimeout(timeout, timeUnit); + + return this; + } + /** * Checks if this query is collocated. * diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java index be3b390c30383..51c6cb508ebcf 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java @@ -17,6 +17,7 @@ package org.apache.ignite.cache.query; +import java.util.concurrent.TimeUnit; import javax.cache.Cache; import org.apache.ignite.IgniteCache; import org.apache.ignite.internal.processors.query.GridQueryProcessor; @@ -43,6 +44,9 @@ public final class SqlQuery extends Query> { @GridToStringInclude private Object[] args; + /** Timeout in millis. */ + private int timeout; + /** * Constructs query for the given type name and SQL query. * @@ -130,6 +134,27 @@ public SqlQuery setType(String type) { return this; } + /** + * Gets the query execution timeout in milliseconds. + * + * @return Timeout value. + */ + public int getTimeout() { + return timeout; + } + + /** + * Sets the query execution timeout. Query will be automatically cancelled if the execution timeout is exceeded. + * @param timeout Timeout value. Zero value disables timeout. + * @param timeUnit Time granularity. + * @return {@code this} For chaining. + */ + public SqlQuery setTimeout(int timeout, TimeUnit timeUnit) { + this.timeout = GridQueryProcessor.validateTimeout(timeout, timeUnit); + + return this; + } + /** {@inheritDoc} */ @Override public SqlQuery setPageSize(int pageSize) { return (SqlQuery)super.setPageSize(pageSize); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteSchedulerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteSchedulerImpl.java index ed8d1a4b17cb9..623b1f2426e7f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteSchedulerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteSchedulerImpl.java @@ -17,17 +17,20 @@ package org.apache.ignite.internal; +import java.io.Closeable; import java.io.Externalizable; import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; import java.io.ObjectStreamException; import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; import org.apache.ignite.IgniteScheduler; import org.apache.ignite.internal.util.future.IgniteFutureImpl; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.scheduler.SchedulerFuture; +import org.jetbrains.annotations.Nullable; /** * {@link IgniteScheduler} implementation. @@ -67,6 +70,21 @@ public IgniteSchedulerImpl(GridKernalContext ctx) { } } + /** {@inheritDoc} */ + @Override public Closeable runLocal(@Nullable Runnable r, long delay, TimeUnit timeUnit) { + A.notNull(r, "r"); + A.ensure(delay > 0, "Illegal delay"); + + guard(); + + try { + return ctx.timeout().schedule(r, timeUnit.toMillis(delay), -1); + } + finally { + unguard(); + } + } + /** {@inheritDoc} */ @Override public IgniteFuture callLocal(Callable c) { A.notNull(c, "c"); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java index 672a37d9e8050..f68426ee3c93a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java @@ -20,15 +20,28 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import javax.cache.CacheException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.query.QueryCancelledException; import org.apache.ignite.internal.processors.cache.query.QueryCursorEx; +import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; +import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.CLOSED; +import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.EXECUTION; +import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.IDLE; +import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.RESULT_READY; + /** * Query cursor implementation. */ public class QueryCursorImpl implements QueryCursorEx { + /** */ + private final static AtomicReferenceFieldUpdater STATE_UPDATER = + AtomicReferenceFieldUpdater.newUpdater(QueryCursorImpl.class, State.class, "state"); + /** Query executor. */ private Iterable iterExec; @@ -36,29 +49,43 @@ public class QueryCursorImpl implements QueryCursorEx { private Iterator iter; /** */ - private boolean iterTaken; + private volatile State state = IDLE; /** */ private List fieldsMeta; + /** */ + private final GridQueryCancel cancel; + /** * @param iterExec Query executor. + * @param cancel Cancellation closure. */ - public QueryCursorImpl(Iterable iterExec) { + public QueryCursorImpl(Iterable iterExec, GridQueryCancel cancel) { this.iterExec = iterExec; + this.cancel = cancel; + } + + /** + * @param iterExec Query executor. + */ + public QueryCursorImpl(Iterable iterExec) { + this(iterExec, null); } /** {@inheritDoc} */ @Override public Iterator iterator() { - if (iter == null && iterTaken) - throw new IgniteException("Cursor is closed."); + if (!STATE_UPDATER.compareAndSet(this, IDLE, EXECUTION)) + throw new IgniteException("Iterator is already fetched or query was cancelled."); - if (iterTaken) - throw new IgniteException("Iterator is already taken from this cursor."); + iter = iterExec.iterator(); - iterTaken = true; + if (!STATE_UPDATER.compareAndSet(this, EXECUTION, RESULT_READY)) { + // Handle race with cancel and make sure the iterator resources are freed correctly. + closeIter(); - iter = iterExec.iterator(); + throw new CacheException(new QueryCancelledException()); + } assert iter != null; @@ -93,18 +120,35 @@ public QueryCursorImpl(Iterable iterExec) { /** {@inheritDoc} */ @Override public void close() { - Iterator i; - - if ((i = iter) != null) { - iter = null; - - if (i instanceof AutoCloseable) { - try { - ((AutoCloseable)i).close(); - } - catch (Exception e) { - throw new IgniteException(e); - } + while(state != CLOSED) { + if (STATE_UPDATER.compareAndSet(this, RESULT_READY, CLOSED)) { + closeIter(); + + return; + } + + if (STATE_UPDATER.compareAndSet(this, EXECUTION, CLOSED)) { + if (cancel != null) + cancel.cancel(); + + return; + } + + if (STATE_UPDATER.compareAndSet(this, IDLE, CLOSED)) + return; + } + } + + /** + * Closes iterator. + */ + private void closeIter() { + if (iter instanceof AutoCloseable) { + try { + ((AutoCloseable)iter).close(); + } + catch (Exception e) { + throw new IgniteException(e); } } } @@ -122,4 +166,12 @@ public void fieldsMeta(List fieldsMeta) { @Override public List fieldsMeta() { return fieldsMeta; } + + /** Query cursor state */ + protected enum State { + /** Idle. */IDLE, + /** Executing. */EXECUTION, + /** Result ready. */RESULT_READY, + /** Closed. */CLOSED, + } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java new file mode 100644 index 0000000000000..47f1208645055 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.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.query; + +import org.apache.ignite.cache.query.QueryCancelledException; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * Holds query cancel state. + */ +public class GridQueryCancel { + /** */ + private volatile boolean cancelled; + + /** */ + private volatile boolean completed; + + /** */ + private volatile Runnable clo; + + /** + * Sets a cancel closure. The closure must be idempotent to multiple invocations. + * + * @param clo Clo. + */ + public void set(Runnable clo) throws QueryCancelledException{ + checkCancelled(); + + this.clo = clo; + } + + /** + * Spins until a query is completed. + * Only one thread can enter this method. + * This is guaranteed by {@link org.apache.ignite.internal.processors.cache.QueryCursorImpl} + */ + public void cancel() { + cancelled = true; + + int attempt = 0; + + while (!completed) { + if (clo != null) clo.run(); + + try { + U.sleep(++attempt * 10); + } catch (IgniteInterruptedCheckedException ignored) { + return; + } + } + } + + /** + * Stops query execution if a user requested cancel. + */ + public void checkCancelled() throws QueryCancelledException{ + if (cancelled) + throw new QueryCancelledException(); + } + + /** + * Sets completed state. + * The method must be called then a query is completed by any reason, typically in final block. + */ + public void setCompleted() { + completed = true; + } +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryFieldsResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryFieldsResult.java index 25c8caba5c15c..e32a6872c4b8a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryFieldsResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryFieldsResult.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.query; import java.util.List; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.spi.IgniteSpiCloseableIterator; /** @@ -37,5 +38,5 @@ public interface GridQueryFieldsResult { * * @return Iterator over queried fields. */ - IgniteSpiCloseableIterator> iterator(); + IgniteSpiCloseableIterator> iterator() throws IgniteCheckedException; } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryFieldsResultAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryFieldsResultAdapter.java index fc01d2dc1109e..7f1d175505a93 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryFieldsResultAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryFieldsResultAdapter.java @@ -19,6 +19,7 @@ import java.util.Collections; import java.util.List; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.util.lang.GridCloseableIterator; import org.jetbrains.annotations.Nullable; @@ -51,7 +52,7 @@ public GridQueryFieldsResultAdapter(@Nullable List metaD } /** {@inheritDoc} */ - @Override public GridCloseableIterator> iterator() { + @Override public GridCloseableIterator> iterator() throws IgniteCheckedException{ return it; } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java index 7697a1272637f..b1b3c68ad803c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java @@ -29,7 +29,6 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.lang.GridCloseableIterator; import org.apache.ignite.lang.IgniteBiTuple; @@ -57,17 +56,6 @@ public interface GridQueryIndexing { */ public void stop() throws IgniteCheckedException; - /** - * Runs two step query. - * - * @param cctx Cache context. - * @param qry Query. - * @param keepCacheObjects If {@code true}, cache objects representation will be preserved. - * @return Cursor. - */ - public Iterable> queryTwoStep(GridCacheContext cctx, GridCacheTwoStepQuery qry, - boolean keepCacheObjects); - /** * Parses SQL query into two step query and executes it. * @@ -93,11 +81,14 @@ public Iterable> queryTwoStep(GridCacheContext cctx, GridCacheTwoSt * @param qry Query. * @param params Query parameters. * @param filters Space name and key filters. + * @param timeout Query timeout in milliseconds. + * @param cancel Query cancel. * @return Query result. * @throws IgniteCheckedException If failed. */ - public GridQueryFieldsResult queryFields(@Nullable String spaceName, String qry, - Collection params, IndexingQueryFilter filters) throws IgniteCheckedException; + public GridQueryFieldsResult execute(@Nullable String spaceName, String qry, + Collection params, IndexingQueryFilter filters, int timeout, GridQueryCancel cancel) + throws IgniteCheckedException; /** * Executes regular query. @@ -241,4 +232,9 @@ public void store(@Nullable String spaceName, GridQueryTypeDescriptor type, Cach * @param reconnectFut Reconnect future. */ public void onDisconnected(IgniteFuture reconnectFut); -} + + /** + * Cancels all executing queries. + */ + public void cancelAllQueries(); +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index 1605188e12144..3d185c6be38a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -39,6 +39,7 @@ import java.util.UUID; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import javax.cache.Cache; import javax.cache.CacheException; import org.apache.ignite.IgniteCheckedException; @@ -71,7 +72,6 @@ import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture; import org.apache.ignite.internal.processors.cache.query.CacheQueryType; -import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -82,6 +82,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.GridWorker; @@ -434,6 +435,19 @@ private void addTypeByName(CacheConfiguration ccfg, TypeDescriptor desc) th @Override public void onKernalStop(boolean cancel) { super.onKernalStop(cancel); + if (cancel && idx != null) + try { + while (!busyLock.tryBlock(500)) + idx.cancelAllQueries(); + + return; + } + catch (InterruptedException e) { + U.warn(log, "Interrupted while waiting for active queries cancellation."); + + Thread.currentThread().interrupt(); + } + busyLock.block(); } @@ -762,37 +776,6 @@ public GridCloseableIterator> query(final String spac } } - /** - * @param space Space name. - * @param qry Query. - * @return Cursor. - */ - public Iterable> queryTwoStep(String space, final GridCacheTwoStepQuery qry) { - checkxEnabled(); - - if (!busyLock.enterBusy()) - throw new IllegalStateException("Failed to execute query (grid is stopping)."); - - try { - final GridCacheContext cctx = ctx.cache().internalCache(space).context(); - - return executeQuery(cctx, new IgniteOutClosureX>>() { - @Override public Iterable> applyx() throws IgniteCheckedException { - return idx.queryTwoStep( - cctx, - qry, - cctx.keepBinary()); - } - }, false); - } - catch (IgniteCheckedException e) { - throw new IgniteException(e); - } - finally { - busyLock.leaveBusy(); - } - } - /** * @param cctx Cache context. * @param qry Query. @@ -806,7 +789,7 @@ public QueryCursor> queryTwoStep(final GridCacheContext cctx, final try { return executeQuery(cctx, new IgniteOutClosureX>>() { - @Override public QueryCursor> applyx() throws IgniteCheckedException { + @Override public QueryCursor> applyx() { return idx.queryTwoStep(cctx, qry); } }, true); @@ -941,6 +924,24 @@ private void sendQueryExecutedEvent(String sqlQry, Object[] params) { } } + /** + * @param timeout Timeout. + * @param timeUnit Time unit. + * @return Converted time. + */ + public static int validateTimeout(int timeout, TimeUnit timeUnit) { + A.ensure(timeUnit != TimeUnit.MICROSECONDS && timeUnit != TimeUnit.NANOSECONDS, + "timeUnit minimal resolution is millisecond."); + + A.ensure(timeout >= 0, "timeout value should be non-negative."); + + long tmp = TimeUnit.MILLISECONDS.convert(timeout, timeUnit); + + A.ensure(timeout <= Integer.MAX_VALUE, "timeout value too large."); + + return (int) tmp; + } + /** * Closeable iterator. */ @@ -962,20 +963,26 @@ public QueryCursor> queryLocalFields(final GridCacheContext cctx, f return executeQuery(cctx, new IgniteOutClosureX>>() { @Override public QueryCursor> applyx() throws IgniteCheckedException { - String space = cctx.name(); - String sql = qry.getSql(); - Object[] args = qry.getArgs(); - - final GridQueryFieldsResult res = idx.queryFields(space, sql, F.asList(args), - idx.backupFilter(null, requestTopVer.get(), null)); + final String space = cctx.name(); + final String sql = qry.getSql(); + final Object[] args = qry.getArgs(); + final GridQueryCancel cancel = new GridQueryCancel(); - sendQueryExecutedEvent(sql, args); + final GridQueryFieldsResult res = idx.execute(space, sql, F.asList(args), + idx.backupFilter(null, requestTopVer.get(), null), qry.getTimeout(), cancel); QueryCursorImpl> cursor = new QueryCursorImpl<>(new Iterable>() { @Override public Iterator> iterator() { - return new GridQueryCacheObjectsIterator(res.iterator(), cctx, keepBinary); + try { + sendQueryExecutedEvent(sql, args); + + return new GridQueryCacheObjectsIterator(res.iterator(), cctx, keepBinary); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } } - }); + }, cancel); cursor.fieldsMeta(res.metaData()); @@ -1076,14 +1083,13 @@ public static String typeName(Class cls) { * @return Type name. */ public static String typeName(String clsName) { - int packageEnd = clsName.lastIndexOf('.'); + int pkgEnd = clsName.lastIndexOf('.'); - if (packageEnd >= 0 && packageEnd < clsName.length() - 1) - clsName = clsName.substring(packageEnd + 1); + if (pkgEnd >= 0 && pkgEnd < clsName.length() - 1) + clsName = clsName.substring(pkgEnd + 1); - if (clsName.endsWith("[]")) { + if (clsName.endsWith("[]")) clsName = clsName.substring(0, clsName.length() - 2) + "_array"; - } int parentEnd = clsName.lastIndexOf('$'); @@ -1154,7 +1160,7 @@ public GridQueryFieldsResult queryFields(@Nullable final String space, final Str return executeQuery(cctx, new IgniteOutClosureX() { @Override public GridQueryFieldsResult applyx() throws IgniteCheckedException { - return idx.queryFields(space, clause, params, filters); + return idx.execute(space, clause, params, filters, 0, null); } }, false); } @@ -1519,13 +1525,12 @@ private void processClassMeta( for (Map.Entry entry : qryEntity.getFields().entrySet()) { ClassProperty prop = buildClassProperty( - d.keyClass(), - d.valueClass(), - entry.getKey(), - U.classForName(entry.getValue(), Object.class), - aliases, - coCtx); - + d.keyClass(), + d.valueClass(), + entry.getKey(), + U.classForName(entry.getValue(), Object.class), + aliases, + coCtx); d.addProperty(prop, false); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java index 499438d17bec8..0baf6ea88deea 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.query.h2.twostep.messages; import java.nio.ByteBuffer; +import org.apache.ignite.cache.query.QueryCancelledException; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageReader; @@ -27,6 +28,12 @@ * Error message. */ public class GridQueryFailResponse implements Message { + /** General error failure type. */ + public static final byte GENERAL_ERROR = 0; + + /** Cancelled by originator failure type. */ + public static final byte CANCELLED_BY_ORIGINATOR = 1; + /** */ private static final long serialVersionUID = 0L; @@ -36,6 +43,9 @@ public class GridQueryFailResponse implements Message { /** */ private String errMsg; + /** */ + private byte failCode; + /** * Default constructor. */ @@ -50,6 +60,7 @@ public GridQueryFailResponse() { public GridQueryFailResponse(long qryReqId, Throwable err) { this.qryReqId = qryReqId; this.errMsg = err.getClass() + ":" + err.getMessage(); + this.failCode = err instanceof QueryCancelledException ? CANCELLED_BY_ORIGINATOR : GENERAL_ERROR; } /** @@ -66,6 +77,13 @@ public String error() { return errMsg; } + /** + * @return Fail code. + */ + public byte failCode() { + return failCode; + } + /** {@inheritDoc} */ @Override public void onAckReceived() { // No-op. @@ -100,6 +118,12 @@ public String error() { writer.incrementState(); + case 2: + if (!writer.writeByte("failCode", failCode)) + return false; + + writer.incrementState(); + } return true; @@ -129,6 +153,14 @@ public String error() { reader.incrementState(); + case 2: + failCode = reader.readByte("failCode"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } return reader.afterMessageRead(GridQueryFailResponse.class); @@ -141,6 +173,6 @@ public String error() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 2; + return 3; } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java index 60d348bc1ef13..550cf9b5084c6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java @@ -66,6 +66,9 @@ public class GridQueryRequest implements Message { @GridToStringInclude private int[] parts; + /** */ + private int timeout; + /** * Default constructor. */ @@ -81,6 +84,7 @@ public GridQueryRequest() { * @param topVer Topology version. * @param extraSpaces All space names participating in query other than {@code space}. * @param parts Optional partitions for unstable topology. + * @param timeout Timeout in millis. */ public GridQueryRequest( long reqId, @@ -89,7 +93,8 @@ public GridQueryRequest( Collection qrys, AffinityTopologyVersion topVer, List extraSpaces, - int[] parts) { + int[] parts, + int timeout) { this.reqId = reqId; this.pageSize = pageSize; this.space = space; @@ -98,6 +103,7 @@ public GridQueryRequest( this.topVer = topVer; this.extraSpaces = extraSpaces; this.parts = parts; + this.timeout = timeout; } /** @@ -162,6 +168,13 @@ public String space() { return space; } + /** + * @return Timeout. + */ + public int timeout() { + return this.timeout; + } + /** * @return Queries. */ @@ -233,6 +246,12 @@ public Collection queries() throws IgniteCheckedException { writer.incrementState(); + case 7: + if (!writer.writeInt("timeout", timeout)) + return false; + + writer.incrementState(); + } return true; @@ -302,6 +321,14 @@ public Collection queries() throws IgniteCheckedException { reader.incrementState(); + case 7: + timeout = reader.readInt("timeout"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } return reader.afterMessageRead(GridQueryRequest.class); @@ -314,6 +341,6 @@ public Collection queries() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 7; + return 8; } } \ No newline at end of file diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java index a5f283e50e9e2..ab332c123b87b 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java @@ -51,12 +51,14 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; import javax.cache.Cache; import javax.cache.CacheException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.cache.query.QueryCancelledException; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.SqlQuery; @@ -75,6 +77,7 @@ import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.internal.processors.cache.QueryCursorImpl; import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery; +import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; import org.apache.ignite.internal.processors.query.GridQueryFieldsResult; import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter; @@ -120,9 +123,12 @@ import org.h2.command.CommandInterface; import org.h2.constant.ErrorCode; import org.h2.constant.SysProperties; +import org.h2.engine.Session; import org.h2.index.Index; import org.h2.index.SpatialIndex; +import org.h2.jdbc.JdbcConnection; import org.h2.jdbc.JdbcPreparedStatement; +import org.h2.jdbc.JdbcStatement; import org.h2.message.DbException; import org.h2.mvstore.cache.CacheLongKeyLIRS; import org.h2.server.web.WebServer; @@ -350,7 +356,7 @@ private PreparedStatement prepareStatement(Connection c, String sql, boolean use PreparedStatement stmt = cache.get(sql); - if (stmt != null && !stmt.isClosed()) { + if (stmt != null && !((JdbcStatement)stmt).wasCancelled()) { assert stmt.getConnection() == c; return stmt; @@ -727,28 +733,33 @@ private void removeTable(TableDescriptor tbl) throws IgniteCheckedException { /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public GridQueryFieldsResult queryFields(@Nullable final String spaceName, final String qry, - @Nullable final Collection params, final IndexingQueryFilter filters) + @Override public GridQueryFieldsResult execute(@Nullable final String spaceName, final String qry, + @Nullable final Collection params, final IndexingQueryFilter filters, + final int timeout, final GridQueryCancel cancel) throws IgniteCheckedException { setFilters(filters); try { - Connection conn = connectionForThread(schema(spaceName)); + final Connection conn = connectionForThread(schema(spaceName)); - ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, qry, params, true); + final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true); - List meta = null; + List meta; - if (rs != null) { - try { - meta = meta(rs.getMetaData()); - } - catch (SQLException e) { - throw new IgniteCheckedException("Failed to get meta data.", e); - } + try { + meta = meta(stmt.getMetaData()); + } + catch (SQLException e) { + throw new IgniteCheckedException("Cannot prepare query metadata", e); } - return new GridQueryFieldsResultAdapter(meta, new FieldsIterator(rs)); + return new GridQueryFieldsResultAdapter(meta, null) { + @Override public GridCloseableIterator> iterator() throws IgniteCheckedException{ + ResultSet rs = executeSqlQueryWithTimer(spaceName, stmt, conn, qry, params, timeout, cancel); + + return new FieldsIterator(rs); + } + }; } finally { setFilters(null); @@ -808,18 +819,18 @@ private static String schemaNameFromCacheConf(CacheConfiguration ccfg) { } /** - * Executes sql query. + * Prepares sql statement. * - * @param conn Connection,. - * @param sql Sql query. - * @param params Parameters. - * @param useStmtCache If {@code true} uses statement cache. - * @return Result. + * @param conn Connection. + * @param sql Sql. + * @param params Params. + * @param useStmtCache If {@code true} use stmt cache. + * @return Prepared statement with set parameters. * @throws IgniteCheckedException If failed. */ - private ResultSet executeSqlQuery(Connection conn, String sql, Collection params, boolean useStmtCache) - throws IgniteCheckedException { - PreparedStatement stmt; + private PreparedStatement preparedStatementWithParams(Connection conn, String sql, Collection params, + boolean useStmtCache) throws IgniteCheckedException { + final PreparedStatement stmt; try { stmt = prepareStatement(conn, sql, useStmtCache); @@ -840,12 +851,54 @@ private ResultSet executeSqlQuery(Connection conn, String sql, Collection 0) + ((Session)((JdbcConnection)conn).getSession()).setQueryTimeout(timeoutMillis); + + if (cancel != null) { + cancel.set(new Runnable() { + @Override public void run() { + try { + stmt.cancel(); + } catch (SQLException ignored) { + // No-op. + } + } + }); + } + try { return stmt.executeQuery(); } catch (SQLException e) { + // Throw special exception. + if (e.getErrorCode() == ErrorCode.STATEMENT_WAS_CANCELED) + throw new QueryCancelledException(); + throw new IgniteCheckedException("Failed to execute SQL query.", e); } + finally { + if(cancel != null) + cancel.setCompleted(); + + if (timeoutMillis > 0) + ((Session)((JdbcConnection)conn).getSession()).setQueryTimeout(0); + } } /** @@ -855,7 +908,8 @@ private ResultSet executeSqlQuery(Connection conn, String sql, Collection params, - boolean useStmtCache) throws IgniteCheckedException { + boolean useStmtCache, + int timeoutMillis, + @Nullable GridQueryCancel cancel) throws IgniteCheckedException { + return executeSqlQueryWithTimer(space, preparedStatementWithParams(conn, sql, params, useStmtCache), + conn, sql, params, timeoutMillis, cancel); + } + + /** + * Executes sql query and prints warning if query is too slow. + * + * @param space Space name. + * @param stmt Prepared statement for query. + * @param conn Connection. + * @param sql Sql query. + * @param params Parameters. + * @param cancel Query cancel. + * @return Result. + * @throws IgniteCheckedException If failed. + */ + private ResultSet executeSqlQueryWithTimer(String space, PreparedStatement stmt, + Connection conn, + String sql, + @Nullable Collection params, + int timeoutMillis, + @Nullable GridQueryCancel cancel) throws IgniteCheckedException { long start = U.currentTimeMillis(); try { - ResultSet rs = executeSqlQuery(conn, sql, params, useStmtCache); + ResultSet rs = executeSqlQuery(conn, stmt, timeoutMillis, cancel); long time = U.currentTimeMillis() - start; @@ -876,7 +954,8 @@ public ResultSet executeSqlQueryWithTimer(String space, if (time > longQryExecTimeout) { String msg = "Query execution is too long (" + time + " ms): " + sql; - ResultSet plan = executeSqlQuery(conn, "EXPLAIN " + sql, params, false); + ResultSet plan = executeSqlQuery(conn, preparedStatementWithParams(conn, "EXPLAIN " + sql, + params, false), 0, null); plan.next(); @@ -903,16 +982,20 @@ public ResultSet executeSqlQueryWithTimer(String space, * @param qry Query. * @param params Query parameters. * @param tbl Target table of query to generate select. + * @param cancel Query cancel. * @return Result set. * @throws IgniteCheckedException If failed. */ - private ResultSet executeQuery(String space, String qry, @Nullable Collection params, TableDescriptor tbl) + private ResultSet executeQuery(String space, String qry, @Nullable Collection params, TableDescriptor tbl, + @Nullable GridQueryCancel cancel) throws IgniteCheckedException { Connection conn = connectionForThread(tbl.schemaName()); String sql = generateQuery(qry, tbl); - return executeSqlQueryWithTimer(space, conn, sql, params, true); + return executeSqlQueryWithTimer(space, + preparedStatementWithParams(conn, sql, params, true), + conn, sql, params, 0, cancel); } /** @@ -954,7 +1037,7 @@ public void bindParameters(PreparedStatement stmt, @Nullable Collection setFilters(filters); try { - ResultSet rs = executeQuery(spaceName, qry, params, tbl); + ResultSet rs = executeQuery(spaceName, qry, params, tbl, null); return new KeyValIterator(rs); } @@ -964,11 +1047,19 @@ public void bindParameters(PreparedStatement stmt, @Nullable Collection } /** {@inheritDoc} */ - @Override public Iterable> queryTwoStep(final GridCacheContext cctx, final GridCacheTwoStepQuery qry, - final boolean keepCacheObj) { + private Iterable> doQueryTwoStep(final GridCacheContext cctx, final GridCacheTwoStepQuery qry, + final boolean keepCacheObj, + final int timeoutMillis, + final GridQueryCancel cancel) { return new Iterable>() { @Override public Iterator> iterator() { - return rdcQryExec.query(cctx, qry, keepCacheObj); + try { + return rdcQryExec.query(cctx, qry, keepCacheObj, timeoutMillis, cancel); + } + finally { + if (cancel != null) + cancel.setCompleted(); + } } }; } @@ -998,6 +1089,9 @@ public void bindParameters(PreparedStatement stmt, @Nullable Collection fqry.setArgs(qry.getArgs()); fqry.setPageSize(qry.getPageSize()); + if(qry.getTimeout() > 0) + fqry.setTimeout(qry.getTimeout(), TimeUnit.MILLISECONDS); + final QueryCursor> res = queryTwoStep(cctx, fqry); final Iterable> converted = new Iterable>() { @@ -1064,7 +1158,7 @@ public void bindParameters(PreparedStatement stmt, @Nullable Collection try { ctx.cache().createMissingCaches(); } - catch (IgniteCheckedException e1) { + catch (IgniteCheckedException ignored) { throw new CacheException("Failed to create missing caches.", e); } @@ -1104,7 +1198,10 @@ public void bindParameters(PreparedStatement stmt, @Nullable Collection twoStepQry.pageSize(qry.getPageSize()); - QueryCursorImpl> cursor = new QueryCursorImpl<>(queryTwoStep(cctx, twoStepQry, cctx.keepBinary())); + GridQueryCancel cancel = new GridQueryCancel(); + + QueryCursorImpl> cursor = new QueryCursorImpl<>( + doQueryTwoStep(cctx, twoStepQry, cctx.keepBinary(), qry.getTimeout(), cancel), cancel); cursor.fieldsMeta(meta); @@ -1455,8 +1552,8 @@ public String space(String schemaName) { if (tbl == null) return -1; - IgniteSpiCloseableIterator> iter = queryFields(spaceName, - "SELECT COUNT(*) FROM " + tbl.fullTableName(), null, null).iterator(); + IgniteSpiCloseableIterator> iter = execute(spaceName, + "SELECT COUNT(*) FROM " + tbl.fullTableName(), null, null, 0, null).iterator(); return ((Number)iter.next().get(0)).longValue(); } @@ -2618,4 +2715,10 @@ private void updateLastUsage() { lastUsage = U.currentTimeMillis(); } } + + /** {@inheritDoc} */ + @Override public void cancelAllQueries() { + for (Connection conn : conns) + U.close(conn, log); + } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java index 580058cd5734e..1f05bf705ad52 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java @@ -19,8 +19,6 @@ import java.lang.reflect.Field; import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; import java.util.AbstractCollection; import java.util.ArrayList; import java.util.Collection; @@ -52,6 +50,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridReservable; import org.apache.ignite.internal.processors.cache.query.CacheQueryType; import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery; +import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest; import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse; @@ -452,10 +451,12 @@ private void onQueryRequest(ClusterNode node, GridQueryRequest req) { for (GridCacheSqlQuery qry : qrys) { ResultSet rs = h2.executeSqlQueryWithTimer(req.space(), - h2.connectionForSpace(req.space()), - qry.query(), - F.asList(qry.parameters()), - true); + h2.connectionForSpace(req.space()), + qry.query(), + F.asList(qry.parameters()), + true, + req.timeout(), + qr.cancels[i]); if (ctx.event().isRecordable(EVT_CACHE_QUERY_EXECUTED)) { ctx.event().record(new CacheQueryExecutedEvent<>( @@ -509,6 +510,15 @@ private void onQueryRequest(ClusterNode node, GridQueryRequest req) { // Release reserved partitions. for (GridReservable r : reserved) r.release(); + + // Ensure all cancels state is correct. + if (qr != null) + for (int i = 0; i < qr.cancels.length; i++) { + GridQueryCancel cancel = qr.cancels[i]; + + if (cancel != null) + cancel.setCompleted(); + } } } @@ -636,6 +646,9 @@ private class QueryResults { /** */ private final AtomicReferenceArray results; + /** */ + private final GridQueryCancel[] cancels; + /** */ private final GridCacheContext cctx; @@ -652,6 +665,10 @@ private QueryResults(long qryReqId, int qrys, GridCacheContext cctx) { this.cctx = cctx; results = new AtomicReferenceArray<>(qrys); + cancels = new GridQueryCancel[qrys]; + + for (int i = 0; i < cancels.length; i++) + cancels[i] = new GridQueryCancel(); } /** @@ -687,6 +704,9 @@ boolean isAllClosed() { return true; } + /** + * Cancels the query. + */ void cancel() { if (canceled) return; @@ -696,8 +716,16 @@ void cancel() { for (int i = 0; i < results.length(); i++) { QueryResult res = results.get(i); - if (res != null) + if (res != null) { res.close(); + + continue; + } + + GridQueryCancel cancel = cancels[i]; + + if (cancel != null) + cancel.cancel(); } } } 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 6a079f0d13386..3fdbf42bf6a39 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 @@ -62,8 +62,10 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery; import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery; import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator; +import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.h2.GridH2ResultSetIterator; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; +import org.apache.ignite.cache.query.QueryCancelledException; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlType; import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest; @@ -244,7 +246,7 @@ else if (msg instanceof GridQueryFailResponse) private void onFail(ClusterNode node, GridQueryFailResponse msg) { QueryRun r = runs.get(msg.queryRequestId()); - fail(r, node.id(), msg.error()); + fail(r, node.id(), msg.error(), msg.failCode()); } /** @@ -252,9 +254,15 @@ private void onFail(ClusterNode node, GridQueryFailResponse msg) { * @param nodeId Failed node ID. * @param msg Error message. */ - private void fail(QueryRun r, UUID nodeId, String msg) { - if (r != null) - r.state(new CacheException("Failed to execute map query on the node: " + nodeId + ", " + msg), nodeId); + private void fail(QueryRun r, UUID nodeId, String msg, byte failCode) { + if (r != null) { + CacheException e = new CacheException("Failed to execute map query on the node: " + nodeId + ", " + msg); + + if (failCode == GridQueryFailResponse.CANCELLED_BY_ORIGINATOR) + e.addSuppressed(new QueryCancelledException()); + + r.state(e, nodeId); + } } /** @@ -312,7 +320,7 @@ private void onNextPage(final ClusterNode node, GridQueryNextPageResponse msg) { catch (Exception e) { U.error(log, "Error in message.", e); - fail(r, node.id(), "Error in message."); + fail(r, node.id(), "Error in message.", GridQueryFailResponse.GENERAL_ERROR); return; } @@ -449,9 +457,12 @@ else if (!cctx.isReplicated() && !extraCctx.isReplicated()) { * @param cctx Cache context. * @param qry Query. * @param keepBinary Keep binary. - * @return Cursor. + * @param timeoutMillis Timeout in milliseconds. + * @param cancel Query cancel. + * @return Rows iterator. */ - public Iterator> query(GridCacheContext cctx, GridCacheTwoStepQuery qry, boolean keepBinary) { + public Iterator> query(GridCacheContext cctx, GridCacheTwoStepQuery qry, boolean keepBinary, + int timeoutMillis, GridQueryCancel cancel) { for (int attempt = 0;; attempt++) { if (attempt != 0) { try { @@ -464,7 +475,7 @@ public Iterator> query(GridCacheContext cctx, GridCacheTwoStepQuery } } - long qryReqId = reqIdGen.incrementAndGet(); + final long qryReqId = reqIdGen.incrementAndGet(); QueryRun r = new QueryRun(); @@ -510,6 +521,8 @@ public Iterator> query(GridCacheContext cctx, GridCacheTwoStepQuery nodes = Collections.singleton(F.rand(nodes)); } + final Collection finalNodes = nodes; + int tblIdx = 0; final boolean skipMergeTbl = !qry.explain() && qry.skipMergeTable(); @@ -545,6 +558,8 @@ public Iterator> query(GridCacheContext cctx, GridCacheTwoStepQuery runs.put(qryReqId, r); try { + cancel.checkCancelled(); + if (ctx.clientDisconnected()) { throw new CacheException("Query was cancelled, client node disconnected.", new IgniteClientDisconnectedException(ctx.cluster().clientReconnectFuture(), @@ -567,12 +582,20 @@ public Iterator> query(GridCacheContext cctx, GridCacheTwoStepQuery mapQry.marshallParams(m); } + cancel.set(new Runnable() { + @Override public void run() { + send(finalNodes, new GridQueryCancelRequest(qryReqId), null); + } + }); + boolean retry = false; if (send(nodes, - new GridQueryRequest(qryReqId, r.pageSize, space, mapQrys, topVer, extraSpaces, null), partsMap)) { + new GridQueryRequest(qryReqId, r.pageSize, space, mapQrys, topVer, extraSpaces, null, timeoutMillis), partsMap)) { awaitAllReplies(r, nodes); + cancel.checkCancelled(); + Object state = r.state.get(); if (state != null) { @@ -582,6 +605,9 @@ public Iterator> query(GridCacheContext cctx, GridCacheTwoStepQuery if (err.getCause() instanceof IgniteClientDisconnectedException) throw err; + if (wasCancelled(err)) + throw new QueryCancelledException(); // Throw correct exception. + throw new CacheException("Failed to run map query remotely.", err); } @@ -627,6 +653,8 @@ public Iterator> query(GridCacheContext cctx, GridCacheTwoStepQuery resIter = res.iterator(); } else { + cancel.checkCancelled(); + GridCacheSqlQuery rdc = qry.reduceQuery(); // Statement caching is prohibited here because we can't guarantee correct merge index reuse. @@ -634,17 +662,14 @@ public Iterator> query(GridCacheContext cctx, GridCacheTwoStepQuery r.conn, rdc.query(), F.asList(rdc.parameters()), - false); + false, + timeoutMillis, + cancel); resIter = new Iter(res); } } - for (GridMergeIndex idx : r.idxs) { - if (!idx.fetchedAll()) // We have to explicitly cancel queries on remote nodes. - send(nodes, new GridQueryCancelRequest(qryReqId), null); - } - if (retry) { if (Thread.currentThread().isInterrupted()) throw new IgniteInterruptedCheckedException("Query was interrupted."); @@ -657,8 +682,12 @@ public Iterator> query(GridCacheContext cctx, GridCacheTwoStepQuery catch (IgniteCheckedException | RuntimeException e) { U.closeQuiet(r.conn); - if (e instanceof CacheException) + if (e instanceof CacheException) { + if (wasCancelled((CacheException)e)) + throw new CacheException("Failed to run reduce query locally.", new QueryCancelledException()); + throw (CacheException)e; + } Throwable cause = e; @@ -673,6 +702,9 @@ public Iterator> query(GridCacheContext cctx, GridCacheTwoStepQuery throw new CacheException("Failed to run reduce query locally.", cause); } finally { + // Make sure any activity related to current attempt is cancelled. + cancelRemoteQueriesIfNeeded(nodes, r, qryReqId); + if (!runs.remove(qryReqId, r)) U.warn(log, "Query run was already removed: " + qryReqId); @@ -684,6 +716,33 @@ public Iterator> query(GridCacheContext cctx, GridCacheTwoStepQuery } } + /** + * Returns true if the exception is triggered by query cancel. + * + * @param e Exception. + * @return {@code true} if exception is caused by cancel. + */ + private boolean wasCancelled(CacheException e) { + return e.getSuppressed() != null && e.getSuppressed().length > 0 && + e.getSuppressed()[0] instanceof QueryCancelledException; + } + + /** + * Explicitly cancels remote queries. + * @param nodes Nodes. + * @param r Query run. + * @param qryReqId Query id. + */ + private void cancelRemoteQueriesIfNeeded(Collection nodes, QueryRun r, long qryReqId) { + for (GridMergeIndex idx : r.idxs) { + if (!idx.fetchedAll()) { + send(nodes, new GridQueryCancelRequest(qryReqId), null); + + break; + } + } + } + /** * @param r Query run. * @param nodes Nodes to check periodically if they alive. @@ -986,7 +1045,7 @@ private Iterator> explainPlan(JdbcConnection c, String space, GridCacheT List> lists = new ArrayList<>(); for (int i = 0, mapQrys = qry.mapQueries().size(); i < mapQrys; i++) { - ResultSet rs = h2.executeSqlQueryWithTimer(space, c, "SELECT PLAN FROM " + table(i), null, false); + ResultSet rs = h2.executeSqlQueryWithTimer(space, c, "SELECT PLAN FROM " + table(i), null, false, 0, null); lists.add(F.asList(getPlan(rs))); } @@ -1005,7 +1064,9 @@ private Iterator> explainPlan(JdbcConnection c, String space, GridCacheT c, "EXPLAIN " + rdc.query(), F.asList(rdc.parameters()), - false); + false, + 0, + null); lists.add(F.asList(getPlan(rs))); @@ -1240,4 +1301,4 @@ protected Iter(ResultSet data) throws IgniteCheckedException { return res; } } -} +} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java new file mode 100644 index 0000000000000..3fec966e2e982 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.near; + +import java.util.Arrays; +import java.util.List; +import javax.cache.CacheException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.query.QueryCancelledException; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests distributed SQL query cancel related scenarios. + */ +public class IgniteCacheDistributedQueryCancelSelfTest extends GridCommonAbstractTest { + /** Grids count. */ + private static final int GRIDS_COUNT = 3; + + /** IP finder. */ + private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** Cache size. */ + public static final int CACHE_SIZE = 10_000; + + /** Value size. */ + public static final int VAL_SIZE = 16; + + /** */ + private static final String QUERY = "select a._val, b._val from String a, String b"; + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + startGridsMultiThreaded(GRIDS_COUNT); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + TcpDiscoverySpi spi = (TcpDiscoverySpi)cfg.getDiscoverySpi(); + spi.setIpFinder(IP_FINDER); + + CacheConfiguration ccfg = new CacheConfiguration<>(); + ccfg.setIndexedTypes(Integer.class, String.class); + + cfg.setCacheConfiguration(ccfg); + + if ("client".equals(gridName)) + cfg.setClientMode(true); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + + /** */ + public void testQueryCancelsOnGridShutdown() throws Exception { + try (Ignite client = startGrid("client")) { + + IgniteCache cache = client.cache(null); + + assertEquals(0, cache.localSize()); + + int p = 1; + for (int i = 1; i <= CACHE_SIZE; i++) { + char[] tmp = new char[VAL_SIZE]; + Arrays.fill(tmp, ' '); + cache.put(i, new String(tmp)); + + if (i / (float)CACHE_SIZE >= p / 10f) { + log().info("Loaded " + i + " of " + CACHE_SIZE); + + p++; + } + } + + SqlFieldsQuery qry = new SqlFieldsQuery(QUERY); + + IgniteInternalFuture fut = multithreadedAsync(new Runnable() { + @Override public void run() { + try { + Thread.sleep(1_500); + } + catch (InterruptedException e) { + throw new IgniteException(e); + } + + for (Ignite g : G.allGrids()) + if (!g.configuration().getDiscoverySpi().isClientMode()) + stopGrid(g.name(), true); + } + }, 1); + + try { + final QueryCursor> cursor = cache.query(qry); + + cursor.iterator(); + } + catch (CacheException ignored) { + // No-op. + } + + fut.get(); + + // Test must exit gracefully. + } + } + + /** */ + public void testQueryResponseFailCode() throws Exception { + try (Ignite client = startGrid("client")) { + + CacheConfiguration cfg = new CacheConfiguration<>(); + cfg.setSqlFunctionClasses(Functions.class); + cfg.setIndexedTypes(Integer.class, Integer.class); + cfg.setName("test"); + + IgniteCache cache = client.getOrCreateCache(cfg); + + cache.put(1, 1); + + QueryCursor> qry = cache.query(new SqlFieldsQuery("select fail() from Integer")); + + try { + qry.getAll(); + + fail(); + } + catch (Exception e) { + assertTrue(e.getCause() instanceof CacheException); + } + } + } + + /** */ + public static class Functions { + /** */ + @QuerySqlFunction + public static int fail() { + throw new IllegalArgumentException(); + } + } +} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java new file mode 100644 index 0000000000000..f75c9074fb6fe --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java @@ -0,0 +1,248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.near; + +import java.util.Arrays; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import javax.cache.CacheException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.cache.query.QueryCancelledException; +import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests distributed SQL queries cancel by user or timeout. + */ +public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends GridCommonAbstractTest { + /** Grids count. */ + private static final int GRIDS_COUNT = 3; + + /** IP finder. */ + private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** Cache size. */ + public static final int CACHE_SIZE = 10_000; + + /** Value size. */ + public static final int VAL_SIZE = 16; + + /** */ + private static final String QUERY_1 = "select a._val, b._val from String a, String b"; + + /** */ + private static final String QUERY_2 = "select a._key, count(*) from String a group by a._key"; + + /** */ + private static final String QUERY_3 = "select a._val from String a"; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGridsMultiThreaded(GRIDS_COUNT); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + TcpDiscoverySpi spi = (TcpDiscoverySpi)cfg.getDiscoverySpi(); + spi.setIpFinder(IP_FINDER); + + CacheConfiguration ccfg = new CacheConfiguration<>(); + ccfg.setIndexedTypes(Integer.class, String.class); + + cfg.setCacheConfiguration(ccfg); + + if ("client".equals(gridName)) + cfg.setClientMode(true); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + for (Ignite g : G.allGrids()) + g.cache(null).removeAll(); + } + + /** */ + public void testRemoteQueryExecutionTimeout() throws Exception { + testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 500, TimeUnit.MILLISECONDS, true); + } + + /** */ + public void testRemoteQueryWithMergeTableTimeout() throws Exception { + testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 500, TimeUnit.MILLISECONDS, true); + } + + /** */ + public void testRemoteQueryExecutionCancel0() throws Exception { + testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 1, TimeUnit.MILLISECONDS, false); + } + + /** */ + public void testRemoteQueryExecutionCancel1() throws Exception { + testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 500, TimeUnit.MILLISECONDS, false); + } + + /** */ + public void testRemoteQueryExecutionCancel2() throws Exception { + testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 1, TimeUnit.SECONDS, false); + } + + /** */ + public void testRemoteQueryExecutionCancel3() throws Exception { + testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 3, TimeUnit.SECONDS, false); + } + + /** */ + public void testRemoteQueryWithMergeTableCancel0() throws Exception { + testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 1, TimeUnit.MILLISECONDS, false); + } + + /** */ + public void testRemoteQueryWithMergeTableCancel1() throws Exception { + testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 500, TimeUnit.MILLISECONDS, false); + } + + /** */ + public void testRemoteQueryWithMergeTableCancel2() throws Exception { + testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 1_500, TimeUnit.MILLISECONDS, false); + } + + /** */ + public void testRemoteQueryWithMergeTableCancel3() throws Exception { + testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 3, TimeUnit.SECONDS, false); + } + + /** */ + public void testRemoteQueryWithoutMergeTableCancel0() throws Exception { + testQuery(CACHE_SIZE, VAL_SIZE, QUERY_3, 1, TimeUnit.MILLISECONDS, false); + } + + /** */ + public void testRemoteQueryWithoutMergeTableCancel1() throws Exception { + testQuery(CACHE_SIZE, VAL_SIZE, QUERY_3, 500, TimeUnit.MILLISECONDS, false); + } + + /** */ + public void testRemoteQueryWithoutMergeTableCancel2() throws Exception { + testQuery(CACHE_SIZE, VAL_SIZE, QUERY_3, 1_000, TimeUnit.MILLISECONDS, false); + } + + /** */ + public void testRemoteQueryWithoutMergeTableCancel3() throws Exception { + testQuery(CACHE_SIZE, VAL_SIZE, QUERY_3, 3, TimeUnit.SECONDS, false); + } + + /** */ + public void testRemoteQueryAlreadyFinishedStop() throws Exception { + testQuery(100, VAL_SIZE, QUERY_3, 3, TimeUnit.SECONDS, false); + } + + /** */ + private void testQuery(int keyCnt, int valSize, String sql, int timeoutUnits, TimeUnit timeUnit, + boolean timeout) throws Exception { + try (Ignite client = startGrid("client")) { + + IgniteCache cache = client.cache(null); + + assertEquals(0, cache.localSize()); + + int p = 1; + for (int i = 1; i <= keyCnt; i++) { + char[] tmp = new char[valSize]; + Arrays.fill(tmp, ' '); + cache.put(i, new String(tmp)); + + if (i/(float)keyCnt >= p/10f) { + log().info("Loaded " + i + " of " + keyCnt); + + p++; + } + } + + assertEquals(0, cache.localSize()); + + SqlFieldsQuery qry = new SqlFieldsQuery(sql); + + final QueryCursor> cursor; + if (timeout) { + qry.setTimeout(timeoutUnits, timeUnit); + + cursor = cache.query(qry); + } else { + cursor = cache.query(qry); + + client.scheduler().runLocal(new Runnable() { + @Override public void run() { + cursor.close(); + } + }, timeoutUnits, timeUnit); + } + + try(QueryCursor> ignored = cursor) { + cursor.iterator(); + } + catch (CacheException ex) { + log().error("Got expected exception", ex); + + assertTrue("Must throw correct exception", ex.getCause() instanceof QueryCancelledException); + } + + // Give some time to clean up. + Thread.sleep(TimeUnit.MILLISECONDS.convert(timeoutUnits, timeUnit) + 3_000); + + checkCleanState(); + } + } + + /** + * Validates clean state on all participating nodes after query cancellation. + */ + private void checkCleanState() { + for (int i = 0; i < GRIDS_COUNT; i++) { + IgniteEx grid = grid(i); + + // Validate everything was cleaned up. + ConcurrentMap> map = U.field(((IgniteH2Indexing)U.field(U.field( + grid.context(), "qryProc"), "idx")).mapQueryExecutor(), "qryRess"); + + String msg = "Map executor state is not cleared"; + + // TODO FIXME Current implementation leaves map entry for each node that's ever executed a query. + for (ConcurrentMap results : map.values()) + assertEquals(msg, 0, results.size()); + } + } +} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java index 2fa47507995fc..8b33a46b836da 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java @@ -33,7 +33,10 @@ import org.apache.ignite.cache.query.annotations.QuerySqlField; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.cache.query.QueryCancelledException; import org.apache.ignite.internal.util.GridRandom; import org.apache.ignite.internal.util.typedef.CAX; import org.apache.ignite.internal.util.typedef.F; @@ -224,58 +227,77 @@ public void testRestarts() throws Exception { } while (!locks.compareAndSet(g, 0, 1)); - if (rnd.nextBoolean()) { // Partitioned query. - IgniteCache cache = grid(g).cache("pu"); + try { + if (rnd.nextBoolean()) { // Partitioned query. + IgniteCache cache = grid(g).cache("pu"); - SqlFieldsQuery qry = new SqlFieldsQuery(PARTITIONED_QRY); + SqlFieldsQuery qry = new SqlFieldsQuery(PARTITIONED_QRY); - boolean smallPageSize = rnd.nextBoolean(); + boolean smallPageSize = rnd.nextBoolean(); - if (smallPageSize) - qry.setPageSize(3); + if (smallPageSize) + qry.setPageSize(3); - try { - assertEquals(pRes, cache.query(qry).getAll()); - } - catch (CacheException e) { - if (!smallPageSize) - e.printStackTrace(); + try { + assertEquals(pRes, cache.query(qry).getAll()); + } catch (CacheException e) { + // Interruptions are expected here. + if (e.getCause() instanceof IgniteInterruptedCheckedException) + continue; - assertTrue("On large page size must retry.", smallPageSize); + if (e.getCause() instanceof QueryCancelledException) + fail("Retry is expected"); - boolean failedOnRemoteFetch = false; + if (!smallPageSize) + e.printStackTrace(); - for (Throwable th = e; th != null; th = th.getCause()) { - if (!(th instanceof CacheException)) - continue; + assertTrue("On large page size must retry.", smallPageSize); + + boolean failedOnRemoteFetch = false; + boolean failedOnInterruption = false; + + for (Throwable th = e; th != null; th = th.getCause()) { + if (th instanceof InterruptedException) { + failedOnInterruption = true; - if (th.getMessage() != null && - th.getMessage().startsWith("Failed to fetch data from node:")) { - failedOnRemoteFetch = true; + break; + } - break; + if (!(th instanceof CacheException)) + continue; + + if (th.getMessage() != null && + th.getMessage().startsWith("Failed to fetch data from node:")) { + failedOnRemoteFetch = true; + + break; + } } - } - if (!failedOnRemoteFetch) { - e.printStackTrace(); + // Interruptions are expected here. + if (failedOnInterruption) + continue; - fail("Must fail inside of GridResultPage.fetchNextPage or subclass."); - } - } - } - else { // Replicated query. - IgniteCache cache = grid(g).cache("co"); + if (!failedOnRemoteFetch) { + e.printStackTrace(); - assertEquals(rRes, cache.query(new SqlFieldsQuery(REPLICATED_QRY)).getAll()); - } + fail("Must fail inside of GridResultPage.fetchNextPage or subclass."); + } + } + } else { // Replicated query. + IgniteCache cache = grid(g).cache("co"); - locks.set(g, 0); + assertEquals(rRes, cache.query(new SqlFieldsQuery(REPLICATED_QRY)).getAll()); + } + } finally { + // Clearing lock in final handler to avoid endless loop if exception is thrown. + locks.set(g, 0); - int c = qryCnt.incrementAndGet(); + int c = qryCnt.incrementAndGet(); - if (c % logFreq == 0) - info("Executed queries: " + c); + if (c % logFreq == 0) + info("Executed queries: " + c); + } } } }, qryThreadNum, "query-thread"); @@ -297,24 +319,26 @@ public void testRestarts() throws Exception { } while (!locks.compareAndSet(g, 0, -1)); - log.info("Stop node: " + g); - - stopGrid(g); + try { + log.info("Stop node: " + g); - Thread.sleep(rnd.nextInt(nodeLifeTime)); + stopGrid(g); - log.info("Start node: " + g); + Thread.sleep(rnd.nextInt(nodeLifeTime)); - startGrid(g); + log.info("Start node: " + g); - Thread.sleep(rnd.nextInt(nodeLifeTime)); + startGrid(g); - locks.set(g, 0); + Thread.sleep(rnd.nextInt(nodeLifeTime)); + } finally { + locks.set(g, 0); - int c = restartCnt.incrementAndGet(); + int c = restartCnt.incrementAndGet(); - if (c % logFreq == 0) - info("Node restarts: " + c); + if (c % logFreq == 0) + info("Node restarts: " + c); + } } return true; @@ -333,7 +357,12 @@ public void testRestarts() throws Exception { qrysDone.set(true); - fut1.get(); + // Query thread can stuck in next page waiting loop because all nodes are left. + try { + fut1.get(5_000); + } catch (IgniteFutureTimeoutCheckedException e) { + fut1.cancel(); + } info("Queries stopped."); } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQueryCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQueryCancelOrTimeoutSelfTest.java new file mode 100644 index 0000000000000..68ecdff618bc9 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalQueryCancelOrTimeoutSelfTest.java @@ -0,0 +1,158 @@ +/* + * 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.local; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.cache.query.QueryCancelledException; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.cache.CacheMode.LOCAL; + +/** + * Tests local query cancellations and timeouts. + */ +public class IgniteCacheLocalQueryCancelOrTimeoutSelfTest extends GridCommonAbstractTest { + /** Cache size. */ + private static final int CACHE_SIZE = 10_000; + + /** */ + private static final String QUERY = "select a._val, b._val from String a, String b"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + CacheConfiguration ccfg = new CacheConfiguration<>(); + ccfg.setIndexedTypes(Integer.class, String.class); + ccfg.setCacheMode(LOCAL); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(0); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + for (Ignite g : G.allGrids()) + g.cache(null).removeAll(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** + * @param cache Cache. + */ + private void loadCache(IgniteCache cache) { + int p = 1; + + for (int i = 1; i <= CACHE_SIZE; i++) { + char[] tmp = new char[256]; + Arrays.fill(tmp, ' '); + cache.put(i, new String(tmp)); + + if (i / (float)CACHE_SIZE >= p / 10f) { + log().info("Loaded " + i + " of " + CACHE_SIZE); + + p++; + } + } + } + + /** + * Tests cancellation. + */ + public void testQueryCancel() { + testQuery(false, 1, TimeUnit.SECONDS); + } + + /** + * Tests cancellation with zero timeout. + */ + public void testQueryCancelZeroTimeout() { + testQuery(false, 1, TimeUnit.MILLISECONDS); + } + + /** + * Tests timeout. + */ + public void testQueryTimeout() { + testQuery(true, 1, TimeUnit.SECONDS); + } + + /** + * Tests cancellation. + */ + private void testQuery(boolean timeout, int timeoutUnits, TimeUnit timeUnit) { + Ignite ignite = grid(0); + + IgniteCache cache = ignite.cache(null); + + loadCache(cache); + + SqlFieldsQuery qry = new SqlFieldsQuery(QUERY); + + final QueryCursor> cursor; + if (timeout) { + qry.setTimeout(timeoutUnits, timeUnit); + + cursor = cache.query(qry); + } else { + cursor = cache.query(qry); + + ignite.scheduler().runLocal(new Runnable() { + @Override public void run() { + cursor.close(); + } + }, timeoutUnits, timeUnit); + } + + try(QueryCursor> ignored = cursor) { + cursor.iterator(); + + fail("Expecting timeout"); + } + catch (Exception e) { + assertTrue("Must throw correct exception", e.getCause() instanceof QueryCancelledException); + } + + // Test must exit gracefully. + } +} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java index 0da71c8aa29a1..289c7d729a318 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java @@ -347,8 +347,8 @@ public void testSpi() throws Exception { // Fields query GridQueryFieldsResult fieldsRes = - spi.queryFields("A", "select a.a.name n1, a.a.age a1, b.a.name n2, " + - "b.a.age a2 from a.a, b.a where a.a.id = b.a.id ", Collections.emptySet(), null); + spi.execute("A", "select a.a.name n1, a.a.age a1, b.a.name n2, " + + "b.a.age a2 from a.a, b.a where a.a.id = b.a.id ", Collections.emptySet(), null, 0, null); String[] aliases = {"N1", "A1", "N2", "A2"}; Object[] vals = { "Valera", 19, "Kolya", 25}; @@ -447,7 +447,7 @@ public void testLongQueries() throws Exception { time = now; range *= 3; - GridQueryFieldsResult res = spi.queryFields("A", sql, Arrays.asList(1, range), null); + GridQueryFieldsResult res = spi.execute("A", sql, Arrays.asList(1, range), null, 0, null); assert res.iterator().hasNext(); @@ -552,8 +552,7 @@ public String space() { return name; } - @Override - public Class type() { + @Override public Class type() { return Object.class; } }; diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java index 40fc15759d3f3..9128f76e45b5b 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java @@ -34,11 +34,14 @@ import org.apache.ignite.internal.processors.cache.SqlFieldsQuerySelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicFieldsQuerySelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledFieldsQuerySelfTest; +import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheDistributedQueryCancelSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQuerySelfTest; +import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQuerySelfTest; import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest; +import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQueryCancelOrTimeoutSelfTest; import org.apache.ignite.internal.processors.cache.query.GridCacheSwapScanQuerySelfTest; import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryAtomicSelfTest; import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryLocalSelfTest; @@ -103,6 +106,11 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class); suite.addTestSuite(IgniteCacheNoClassQuerySelfTest.class); + // Cancellation. + suite.addTestSuite(IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.class); + suite.addTestSuite(IgniteCacheDistributedQueryCancelSelfTest.class); + suite.addTestSuite(IgniteCacheLocalQueryCancelOrTimeoutSelfTest.class); + // Other. suite.addTestSuite(CacheQueryNewClientSelfTest.class); suite.addTestSuite(CacheOffheapBatchIndexingSingleTypeTest.class); From 43ac3f5d5e8ab664e07f26d99be34f284f7941dd Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 17 Oct 2016 11:26:12 +0300 Subject: [PATCH 244/487] IGNITE-4054: Hadoop: added map-reduce plan debug output. --- .../hadoop/HadoopMapReducePlanner.java | 1 + .../hadoop/HadoopExternalSplit.java | 8 +++ .../processors/hadoop/HadoopSplitWrapper.java | 9 +++ .../hadoop/jobtracker/HadoopJobTracker.java | 61 +++++++++++++++++++ 4 files changed, 79 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlanner.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlanner.java index 185994f4758fb..0009c4a80f46b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlanner.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlanner.java @@ -33,6 +33,7 @@ public interface HadoopMapReducePlanner { * @param top Topology. * @param oldPlan Old plan in case of partial failure. * @return Map reduce plan. + * @throws IgniteCheckedException If an error occurs. */ public HadoopMapReducePlan preparePlan(HadoopJob job, Collection top, @Nullable HadoopMapReducePlan oldPlan) throws IgniteCheckedException; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java index bd767b368dba1..a9b453287c6c2 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java @@ -17,10 +17,13 @@ package org.apache.ignite.internal.processors.hadoop; +import org.apache.ignite.internal.util.typedef.internal.S; + import java.io.Externalizable; import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; +import java.util.Arrays; /** * Split serialized in external file. @@ -85,4 +88,9 @@ public long offset() { @Override public int hashCode() { return (int)(off ^ (off >>> 32)); } + + /** {@inheritDoc} */ + public String toString() { + return S.toString(HadoopExternalSplit.class, this, "hosts", Arrays.toString(hosts)); + } } \ No newline at end of file diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java index 511aa5a2ad265..fb6d0f3008d4a 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java @@ -20,7 +20,10 @@ import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; +import java.util.Arrays; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; /** @@ -33,6 +36,7 @@ public class HadoopSplitWrapper extends HadoopInputSplit { private static final long serialVersionUID = 0L; /** Native hadoop input split. */ + @GridToStringExclude private byte[] bytes; /** */ @@ -116,4 +120,9 @@ public byte[] bytes() { @Override public int hashCode() { return id; } + + /** {@inheritDoc} */ + public String toString() { + return S.toString(HadoopSplitWrapper.class, this, "hosts", Arrays.toString(hosts)); + } } \ No newline at end of file diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java index bffb82b1e9180..36782bf2febe0 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java @@ -75,6 +75,7 @@ import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteInClosure; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -313,6 +314,8 @@ public IgniteInternalFuture submit(HadoopJobId jobId, HadoopJobInfo HadoopMapReducePlan mrPlan = mrPlanner.preparePlan(job, ctx.nodes(), null); + logPlan(info, mrPlan); + HadoopJobMetadata meta = new HadoopJobMetadata(ctx.localNodeId(), jobId, info); meta.mapReducePlan(mrPlan); @@ -353,6 +356,64 @@ public IgniteInternalFuture submit(HadoopJobId jobId, HadoopJobInfo } } + /** + * Log map-reduce plan if needed. + * + * @param info Job info. + * @param plan Plan. + */ + @SuppressWarnings("StringConcatenationInsideStringBufferAppend") + private void logPlan(HadoopJobInfo info, HadoopMapReducePlan plan) { + if (log.isDebugEnabled()) { + Map, int[]>> map = new HashMap<>(); + + for (UUID nodeId : plan.mapperNodeIds()) + map.put(nodeId, new IgniteBiTuple, int[]>(plan.mappers(nodeId), null)); + + for (UUID nodeId : plan.reducerNodeIds()) { + int[] reducers = plan.reducers(nodeId); + + IgniteBiTuple, int[]> entry = map.get(nodeId); + + if (entry == null) + map.put(nodeId, new IgniteBiTuple, int[]>(null, reducers)); + else + entry.set2(reducers); + } + + StringBuilder details = new StringBuilder("["); + + boolean first = true; + + for (Map.Entry, int[]>> entry : map.entrySet()) { + if (first) + first = false; + else + details.append(", "); + + UUID nodeId = entry.getKey(); + + Collection mappers = entry.getValue().get1(); + + if (mappers == null) + mappers = Collections.emptyList(); + + int[] reducers = entry.getValue().get2(); + + if (reducers == null) + reducers = new int[0]; + + details.append("[nodeId=" + nodeId + ", mappers=" + mappers.size() + ", reducers=" + reducers.length + + ", mapperDetails=" + mappers + ", reducerDetails=" + Arrays.toString(reducers) + ']'); + } + + details.append(']'); + + log.debug("Prepared map-reduce plan [jobName=" + info.jobName() + ", mappers=" + plan.mappers() + + ", reducers=" + plan.reducers() + ", details=" + details + ']'); + } + } + /** * Convert Hadoop job metadata to job status. * From d3f042b9ba6c605500ab7155c40a41850babefdb Mon Sep 17 00:00:00 2001 From: sboikov Date: Mon, 17 Oct 2016 12:28:31 +0300 Subject: [PATCH 245/487] Fixed indexing test in according to changes from #80abd1b. --- .../query/h2/GridIndexingSpiAbstractSelfTest.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java index 289c7d729a318..6e493eabf8da8 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java @@ -40,6 +40,7 @@ import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.apache.ignite.spi.IgniteSpiCloseableIterator; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.testframework.GridStringLogger; import org.apache.ignite.testframework.GridTestUtils; @@ -353,9 +354,11 @@ public void testSpi() throws Exception { String[] aliases = {"N1", "A1", "N2", "A2"}; Object[] vals = { "Valera", 19, "Kolya", 25}; - assertTrue(fieldsRes.iterator().hasNext()); + IgniteSpiCloseableIterator> it = fieldsRes.iterator(); - List fields = fieldsRes.iterator().next(); + assertTrue(it.hasNext()); + + List fields = it.next(); assertEquals(4, fields.size()); @@ -366,7 +369,7 @@ public void testSpi() throws Exception { assertEquals(vals[i++], f); } - assertFalse(fieldsRes.iterator().hasNext()); + assertFalse(it.hasNext()); // Remove spi.remove(typeAA.space(), key(2), aa(2, "Valera", 19)); From 59de231c0d0dce56b0cf5c386b19a2880d9c0603 Mon Sep 17 00:00:00 2001 From: sboikov Date: Mon, 17 Oct 2016 16:31:35 +0300 Subject: [PATCH 246/487] Re-generated GridQueryFailResponse. --- .../h2/twostep/messages/GridQueryFailResponse.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java index 0baf6ea88deea..261241e58543e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java @@ -113,13 +113,13 @@ public byte failCode() { writer.incrementState(); case 1: - if (!writer.writeLong("qryReqId", qryReqId)) + if (!writer.writeByte("failCode", failCode)) return false; writer.incrementState(); case 2: - if (!writer.writeByte("failCode", failCode)) + if (!writer.writeLong("qryReqId", qryReqId)) return false; writer.incrementState(); @@ -146,7 +146,7 @@ public byte failCode() { reader.incrementState(); case 1: - qryReqId = reader.readLong("qryReqId"); + failCode = reader.readByte("failCode"); if (!reader.isLastRead()) return false; @@ -154,7 +154,7 @@ public byte failCode() { reader.incrementState(); case 2: - failCode = reader.readByte("failCode"); + qryReqId = reader.readLong("qryReqId"); if (!reader.isLastRead()) return false; @@ -175,4 +175,4 @@ public byte failCode() { @Override public byte fieldsCount() { return 3; } -} \ No newline at end of file +} From 717e91fed944082df35ecdc2e647de2ad3fca2c7 Mon Sep 17 00:00:00 2001 From: ascherbakoff Date: Mon, 17 Oct 2016 22:30:38 +0300 Subject: [PATCH 247/487] GG-11360 Fix java8 compilation. --- ...gniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java index f75c9074fb6fe..0f60db25834fd 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java @@ -235,7 +235,7 @@ private void checkCleanState() { IgniteEx grid = grid(i); // Validate everything was cleaned up. - ConcurrentMap> map = U.field(((IgniteH2Indexing)U.field(U.field( + ConcurrentMap> map = U.field(((IgniteH2Indexing)U.field((Object)U.field( grid.context(), "qryProc"), "idx")).mapQueryExecutor(), "qryRess"); String msg = "Map executor state is not cleared"; From 44eff48f72d2a4c5c6a9a2ba6ed903834bcb63f9 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Tue, 18 Oct 2016 11:53:46 +0300 Subject: [PATCH 248/487] IGNITE-4090: CPP: Fixed compilation for GCC 4.4.7. This closes #1165. --- modules/platforms/cpp/common/Makefile.am | 4 +++- .../cpp/common/include/ignite/common/concurrent.h | 5 +++-- .../platforms/cpp/common/include/ignite/ignite_error.h | 4 ++-- modules/platforms/cpp/common/src/ignite_error.cpp | 4 ++-- modules/platforms/cpp/core-test/Makefile.am | 10 ++++++---- modules/platforms/cpp/core-test/src/decimal_test.cpp | 2 +- .../cpp/core-test/src/interop_memory_test.cpp | 5 +++-- modules/platforms/cpp/jni/src/java.cpp | 2 +- modules/platforms/cpp/odbc-test/Makefile.am | 4 +++- modules/platforms/cpp/odbc-test/src/queries_test.cpp | 4 ++-- .../platforms/cpp/odbc/src/config/configuration.cpp | 2 +- 11 files changed, 27 insertions(+), 19 deletions(-) diff --git a/modules/platforms/cpp/common/Makefile.am b/modules/platforms/cpp/common/Makefile.am index c230384820aa0..2d16c2d3f6e9c 100644 --- a/modules/platforms/cpp/common/Makefile.am +++ b/modules/platforms/cpp/common/Makefile.am @@ -26,7 +26,9 @@ SUBDIRS = \ AM_CPPFLAGS = \ -I$(srcdir)/include \ -I$(srcdir)/os/linux/include \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/common/include/ignite/common/concurrent.h b/modules/platforms/cpp/common/include/ignite/common/concurrent.h index 465e02e9ea369..35c2209e39b7b 100644 --- a/modules/platforms/cpp/common/include/ignite/common/concurrent.h +++ b/modules/platforms/cpp/common/include/ignite/common/concurrent.h @@ -138,7 +138,8 @@ namespace ignite { if (ptr) { - impl = new SharedPointerImpl(ptr, reinterpret_cast(&SharedPointerDefaultDeleter)); + void(*deleter)(T*) = (void(*)(T*)) &SharedPointerDefaultDeleter; + impl = new SharedPointerImpl(ptr, reinterpret_cast(deleter)); ImplEnableShared(ptr, impl); } else @@ -360,4 +361,4 @@ namespace ignite } } -#endif //_IGNITE_COMMON_CONCURRENT \ No newline at end of file +#endif //_IGNITE_COMMON_CONCURRENT diff --git a/modules/platforms/cpp/common/include/ignite/ignite_error.h b/modules/platforms/cpp/common/include/ignite/ignite_error.h index 17a24dc2535ff..4c5c06e3dd53d 100644 --- a/modules/platforms/cpp/common/include/ignite/ignite_error.h +++ b/modules/platforms/cpp/common/include/ignite/ignite_error.h @@ -245,7 +245,7 @@ namespace ignite /** * Destructor. */ - ~IgniteError(); + ~IgniteError() IGNITE_NO_THROW; /** * Get error code. @@ -291,4 +291,4 @@ namespace ignite # pragma warning(pop) #endif //_MSC_VER -#endif //_IGNITE_IGNITE_ERROR \ No newline at end of file +#endif //_IGNITE_IGNITE_ERROR diff --git a/modules/platforms/cpp/common/src/ignite_error.cpp b/modules/platforms/cpp/common/src/ignite_error.cpp index 8179184be8a8a..3076d7a4f4a0e 100644 --- a/modules/platforms/cpp/common/src/ignite_error.cpp +++ b/modules/platforms/cpp/common/src/ignite_error.cpp @@ -70,7 +70,7 @@ namespace ignite return *this; } - IgniteError::~IgniteError() + IgniteError::~IgniteError() IGNITE_NO_THROW { ReleaseChars(msg); } @@ -223,4 +223,4 @@ namespace ignite else if (jniCode == IGNITE_JNI_ERR_JVM_ATTACH) *err = IgniteError(IGNITE_ERR_JVM_ATTACH, "Failed to attach to JVM."); } -} \ No newline at end of file +} diff --git a/modules/platforms/cpp/core-test/Makefile.am b/modules/platforms/cpp/core-test/Makefile.am index 7a62e8ed84d29..b1186399425a5 100644 --- a/modules/platforms/cpp/core-test/Makefile.am +++ b/modules/platforms/cpp/core-test/Makefile.am @@ -33,7 +33,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ @@ -54,13 +56,13 @@ ignite_tests_SOURCES = \ src/ignition_test.cpp \ src/interop_memory_test.cpp \ src/handle_registry_test.cpp \ - src/ignite_error_test.cpp \ + src/ignite_error_test.cpp \ src/binary_test_defs.cpp \ src/binary_reader_writer_raw_test.cpp \ src/binary_reader_writer_test.cpp \ src/binary_session_test.cpp \ - src/decimal_test.cpp \ - src/dynamic_size_array_test.cpp \ + src/decimal_test.cpp \ + src/dynamic_size_array_test.cpp \ src/fixed_size_array_test.cpp \ src/transactions_test.cpp \ src/teamcity_messages.cpp \ diff --git a/modules/platforms/cpp/core-test/src/decimal_test.cpp b/modules/platforms/cpp/core-test/src/decimal_test.cpp index 47fe8fc67c4e8..9d0bacfb40ef0 100644 --- a/modules/platforms/cpp/core-test/src/decimal_test.cpp +++ b/modules/platforms/cpp/core-test/src/decimal_test.cpp @@ -1098,4 +1098,4 @@ BOOST_AUTO_TEST_CASE(TestDoubleCast) CheckDoubleCast(-0.000000000000001); } -BOOST_AUTO_TEST_SUITE_END() \ No newline at end of file +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/core-test/src/interop_memory_test.cpp b/modules/platforms/cpp/core-test/src/interop_memory_test.cpp index 07e928cb4e27e..1c782b5ab2327 100644 --- a/modules/platforms/cpp/core-test/src/interop_memory_test.cpp +++ b/modules/platforms/cpp/core-test/src/interop_memory_test.cpp @@ -39,7 +39,8 @@ BOOST_AUTO_TEST_CASE(MemoryReallocationTest) SharedPointer mem = env.AllocateMemory(); - BOOST_CHECK_EQUAL(mem.Get()->Capacity(), IgniteEnvironment::DEFAULT_ALLOCATION_SIZE); + BOOST_CHECK_EQUAL(mem.Get()->Capacity(), + static_cast(IgniteEnvironment::DEFAULT_ALLOCATION_SIZE)); BOOST_CHECK(mem.Get()->Data() != NULL); @@ -92,4 +93,4 @@ BOOST_AUTO_TEST_CASE(MemoryReallocationTest) memset(mem.Get()->Data(), 0xF0F0F0F0, mem.Get()->Capacity()); } -BOOST_AUTO_TEST_SUITE_END() \ No newline at end of file +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp index 56fc4b13a1f23..5bbaac402ba50 100644 --- a/modules/platforms/cpp/jni/src/java.cpp +++ b/modules/platforms/cpp/jni/src/java.cpp @@ -1794,7 +1794,7 @@ namespace ignite } JNIEXPORT jlong JNICALL JniAffinityFunctionInit(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr, jobject baseFunc) { - void* baseFuncRef = baseFunc ? env->NewGlobalRef(baseFunc) : nullptr; + void* baseFuncRef = baseFunc ? env->NewGlobalRef(baseFunc) : 0; IGNITE_SAFE_FUNC(env, envPtr, AffinityFunctionInitHandler, affinityFunctionInit, memPtr, baseFuncRef); } diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index ccf1192445563..e10ef7a922770 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -34,7 +34,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/core/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index f0168bdf3118e..b7ea967d4d692 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -196,10 +196,10 @@ struct QueriesTestSuiteFixture BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); } - SQLCHAR request[] = "SELECT i8Field, i16Field, i32Field, i64Field, strField, " + char request[] = "SELECT i8Field, i16Field, i32Field, i64Field, strField, " "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType"; - ret = SQLExecDirect(stmt, request, SQL_NTS); + ret = SQLExecDirect(stmt, reinterpret_cast(request), SQL_NTS); if (!SQL_SUCCEEDED(ret)) BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); diff --git a/modules/platforms/cpp/odbc/src/config/configuration.cpp b/modules/platforms/cpp/odbc/src/config/configuration.cpp index cc2cc5d897912..f68769591fdd4 100644 --- a/modules/platforms/cpp/odbc/src/config/configuration.cpp +++ b/modules/platforms/cpp/odbc/src/config/configuration.cpp @@ -237,7 +237,7 @@ namespace ignite utility::IntoLower(key); - if (value.front() == '{' && value.back() == '}') + if (value[0] == '{' && value[value.size() - 1] == '}') value = value.substr(1, value.size() - 2); args[key] = value; From 4edcf126baf75840e3816a3e0876200c7e9c0d45 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Tue, 18 Oct 2016 12:25:56 +0300 Subject: [PATCH 249/487] Revert "IGNITE-3875: Added separate thread pool for data streamer. This closes #1067." This reverts commit f597aff1bdf65d3d430cf85c9932391a72c2d7dc. --- .../configuration/IgniteConfiguration.java | 31 ------ .../ignite/internal/GridKernalContext.java | 7 -- .../internal/GridKernalContextImpl.java | 12 --- .../apache/ignite/internal/IgniteKernal.java | 3 - .../apache/ignite/internal/IgnitionEx.java | 20 +--- .../managers/communication/GridIoManager.java | 2 - .../managers/communication/GridIoPolicy.java | 3 - .../closure/GridClosureProcessor.java | 3 +- .../datastreamer/DataStreamProcessor.java | 82 ++-------------- .../datastreamer/DataStreamerImpl.java | 31 +++++- .../processors/pool/PoolProcessor.java | 3 - .../DataStreamProcessorSelfTest.java | 97 ------------------- .../junits/GridTestKernalContext.java | 12 +-- 13 files changed, 44 insertions(+), 262 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java index 412ecbf4742c3..73de470719b06 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java @@ -148,9 +148,6 @@ public class IgniteConfiguration { /** Default core size of public thread pool. */ public static final int DFLT_PUBLIC_THREAD_CNT = Math.max(8, AVAILABLE_PROC_CNT) * 2; - /** Default size of data streamer thread pool. */ - public static final int DFLT_DATA_STREAMER_POOL_SIZE = DFLT_PUBLIC_THREAD_CNT; - /** Default keep alive time for public thread pool. */ @Deprecated public static final long DFLT_PUBLIC_KEEP_ALIVE_TIME = 0; @@ -248,9 +245,6 @@ public class IgniteConfiguration { /** IGFS pool size. */ private int igfsPoolSize = AVAILABLE_PROC_CNT; - /** Data stream pool size. */ - private int dataStreamerPoolSize = DFLT_DATA_STREAMER_POOL_SIZE; - /** Utility cache pool size. */ private int utilityCachePoolSize = DFLT_SYSTEM_CORE_THREAD_CNT; @@ -514,7 +508,6 @@ public IgniteConfiguration(IgniteConfiguration cfg) { clockSyncFreq = cfg.getClockSyncFrequency(); clockSyncSamples = cfg.getClockSyncSamples(); consistentId = cfg.getConsistentId(); - dataStreamerPoolSize = cfg.getDataStreamerThreadPoolSize(); deployMode = cfg.getDeploymentMode(); discoStartupDelay = cfg.getDiscoveryStartupDelay(); failureDetectionTimeout = cfg.getFailureDetectionTimeout(); @@ -795,17 +788,6 @@ public int getIgfsThreadPoolSize() { return igfsPoolSize; } - /** - * Size of thread pool that is in charge of processing data stream messages. - *

          - * If not provided, executor service will have size {@link #DFLT_DATA_STREAMER_POOL_SIZE}. - * - * @return Thread pool size to be used for data stream messages. - */ - public int getDataStreamerThreadPoolSize() { - return dataStreamerPoolSize; - } - /** * Default size of thread pool that is in charge of processing utility cache messages. *

          @@ -929,19 +911,6 @@ public IgniteConfiguration setIgfsThreadPoolSize(int poolSize) { return this; } - /** - * Set thread pool size that will be used to process data stream messages. - * - * @param poolSize Executor service to use for data stream messages. - * @see IgniteConfiguration#getDataStreamerThreadPoolSize() - * @return {@code this} for chaining. - */ - public IgniteConfiguration setDataStreamerThreadPoolSize(int poolSize) { - dataStreamerPoolSize = poolSize; - - return this; - } - /** * Sets default thread pool size that will be used to process utility cache messages. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java index 178239bd2db33..e608af2a09767 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java @@ -531,13 +531,6 @@ public interface GridKernalContext extends Iterable { */ public ExecutorService getIgfsExecutorService(); - /** - * Executor service that is in charge of processing data stream messages. - * - * @return Thread pool implementation to be used for data stream messages. - */ - public ExecutorService getDataStreamerExecutorService(); - /** * Should return an instance of fully configured thread pool to be used for * processing of client messages (REST requests). diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java index 1a9663b5e10bf..ddef345ffd151 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java @@ -310,10 +310,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable @GridToStringExclude private ExecutorService igfsExecSvc; - /** */ - @GridToStringExclude - private ExecutorService dataStreamExecSvc; - /** */ @GridToStringExclude protected ExecutorService restExecSvc; @@ -384,7 +380,6 @@ public GridKernalContextImpl() { * @param p2pExecSvc P2P executor service. * @param mgmtExecSvc Management executor service. * @param igfsExecSvc IGFS executor service. - * @param dataStreamExecSvc data stream executor service. * @param restExecSvc REST executor service. * @param affExecSvc Affinity executor service. * @param plugins Plugin providers. @@ -403,7 +398,6 @@ protected GridKernalContextImpl( ExecutorService p2pExecSvc, ExecutorService mgmtExecSvc, ExecutorService igfsExecSvc, - ExecutorService dataStreamExecSvc, ExecutorService restExecSvc, ExecutorService affExecSvc, IgniteStripedThreadPoolExecutor callbackExecSvc, @@ -422,7 +416,6 @@ protected GridKernalContextImpl( this.p2pExecSvc = p2pExecSvc; this.mgmtExecSvc = mgmtExecSvc; this.igfsExecSvc = igfsExecSvc; - this.dataStreamExecSvc = dataStreamExecSvc; this.restExecSvc = restExecSvc; this.affExecSvc = affExecSvc; this.callbackExecSvc = callbackExecSvc; @@ -962,11 +955,6 @@ protected Object readResolve() throws ObjectStreamException { return igfsExecSvc; } - /** {@inheritDoc} */ - @Override public ExecutorService getDataStreamerExecutorService() { - return dataStreamExecSvc; - } - /** {@inheritDoc} */ @Override public ExecutorService getRestExecutorService() { return restExecSvc; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index e838dd26abb94..586215aa94ac1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -666,7 +666,6 @@ private void notifyLifecycleBeansEx(LifecycleEventType evt) { * @param p2pExecSvc P2P executor service. * @param mgmtExecSvc Management executor service. * @param igfsExecSvc IGFS executor service. - * @param dataStreamExecSvc data stream executor service. * @param restExecSvc Reset executor service. * @param affExecSvc Affinity executor service. * @param errHnd Error handler to use for notification about startup problems. @@ -681,7 +680,6 @@ public void start(final IgniteConfiguration cfg, ExecutorService p2pExecSvc, ExecutorService mgmtExecSvc, ExecutorService igfsExecSvc, - ExecutorService dataStreamExecSvc, ExecutorService restExecSvc, ExecutorService affExecSvc, IgniteStripedThreadPoolExecutor callbackExecSvc, @@ -788,7 +786,6 @@ public void start(final IgniteConfiguration cfg, p2pExecSvc, mgmtExecSvc, igfsExecSvc, - dataStreamExecSvc, restExecSvc, affExecSvc, callbackExecSvc, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index 104599c0f6ecb..a6860b3a4ccac 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -1468,9 +1468,6 @@ private static final class IgniteNamedInstance { /** IGFS executor service. */ private ThreadPoolExecutor igfsExecSvc; - /** Data streamer executor service. */ - private ThreadPoolExecutor dataStreamerExecSvc; - /** REST requests executor service. */ private ThreadPoolExecutor restExecSvc; @@ -1690,17 +1687,6 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { p2pExecSvc.allowCoreThreadTimeOut(true); - // Note that we do not pre-start threads here as this pool may not be needed. - dataStreamerExecSvc = new IgniteThreadPoolExecutor( - "data-streamer", - cfg.getGridName(), - cfg.getDataStreamerThreadPoolSize(), - cfg.getDataStreamerThreadPoolSize(), - DFLT_THREAD_KEEP_ALIVE_TIME, - new LinkedBlockingQueue()); - - dataStreamerExecSvc.allowCoreThreadTimeOut(true); - // Note that we do not pre-start threads here as igfs pool may not be needed. igfsExecSvc = new IgniteThreadPoolExecutor( cfg.getIgfsThreadPoolSize(), @@ -1773,7 +1759,7 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { grid = grid0; grid0.start(myCfg, utilityCacheExecSvc, marshCacheExecSvc, execSvc, sysExecSvc, p2pExecSvc, mgmtExecSvc, - igfsExecSvc, dataStreamerExecSvc, restExecSvc, affExecSvc, callbackExecSvc, + igfsExecSvc, restExecSvc, affExecSvc, callbackExecSvc, new CA() { @Override public void apply() { startLatch.countDown(); @@ -2391,10 +2377,6 @@ private void stopExecutors0(IgniteLogger log) { p2pExecSvc = null; - U.shutdownNow(getClass(), dataStreamerExecSvc, log); - - dataStreamerExecSvc = null; - U.shutdownNow(getClass(), igfsExecSvc, log); igfsExecSvc = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index ba41dab54f03d..cd43318c455cd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -84,7 +84,6 @@ import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.internal.GridTopic.TOPIC_COMM_USER; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.AFFINITY_POOL; -import static org.apache.ignite.internal.managers.communication.GridIoPolicy.DATA_STREAMER_POOL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IGFS_POOL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MANAGEMENT_POOL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MARSH_CACHE_POOL; @@ -575,7 +574,6 @@ private void onMessage0(UUID nodeId, GridIoMessage msg, IgniteRunnable msgC) { case UTILITY_CACHE_POOL: case MARSH_CACHE_POOL: case IGFS_POOL: - case DATA_STREAMER_POOL: { if (msg.isOrdered()) processOrderedMessage(nodeId, msg, plc, msgC); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java index 71279d9613ddc..00590ba50a92e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java @@ -46,9 +46,6 @@ public class GridIoPolicy { /** Marshaller cache execution pool. */ public static final byte IGFS_POOL = 7; - /** Data streamer execution pool. */ - public static final byte DATA_STREAMER_POOL = 8; - /** * Defines the range of reserved pools that are not available for plugins. * @param key The key. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java index 4bd22c586b25e..d388584846125 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Callable; +import java.util.concurrent.Executor; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import org.apache.ignite.IgniteCheckedException; @@ -969,7 +970,7 @@ public IgniteInternalFuture callLocalSafe(Callable c, boolean sys) { * @param plc Policy to choose executor pool. * @return Future. */ - public IgniteInternalFuture callLocalSafe(Callable c, byte plc) { + private IgniteInternalFuture callLocalSafe(Callable c, byte plc) { try { return callLocal(c, plc); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java index bd33f62cbfc7e..7663735fc26c6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java @@ -21,12 +21,10 @@ import java.util.UUID; import java.util.concurrent.DelayQueue; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.managers.communication.GridIoManager; -import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.deployment.GridDeployment; import org.apache.ignite.internal.processors.GridProcessorAdapter; @@ -37,7 +35,6 @@ import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.GridWorker; -import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.stream.StreamReceiver; @@ -45,23 +42,12 @@ import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.GridTopic.TOPIC_DATASTREAM; +import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL; /** - * Data stream processor. + * */ public class DataStreamProcessor extends GridProcessorAdapter { - /** Data streamer separate pool feature major version. */ - private static final int DATA_STREAMER_POOL_MAJOR_VER = 1; - - /** Data streamer separate pool feature minor version. */ - private static final int DATA_STREAMER_POOL_MINOR_VER = 6; - - /** Data streamer separate pool feature maintenance version. */ - private static final int DATA_STREAMER_POOL_MAINTENANCE_VER = 10; - - /** Default pool for data streamer messages processing. */ - public static final byte DFLT_POLICY = GridIoPolicy.PUBLIC_POOL; - /** Loaders map (access is not supposed to be highly concurrent). */ private Collection ldrs = new GridConcurrentHashSet<>(); @@ -232,15 +218,13 @@ private void processRequest(final UUID nodeId, final DataStreamerRequest req) { IgniteInternalFuture fut = ctx.cache().context().exchange().affinityReadyFuture(rmtAffVer); if (fut != null && !fut.isDone()) { - final byte plc = threadIoPolicy(); - fut.listen(new CI1>() { @Override public void apply(IgniteInternalFuture t) { ctx.closure().runLocalSafe(new Runnable() { @Override public void run() { processRequest(nodeId, req); } - }, plc); + }, false); } }); @@ -356,7 +340,12 @@ private void sendResponse(UUID nodeId, Object resTopic, long reqId, @Nullable Th DataStreamerResponse res = new DataStreamerResponse(reqId, errBytes, forceLocDep); try { - ctx.io().send(nodeId, resTopic, res, threadIoPolicy()); + Byte plc = GridIoManager.currentPolicy(); + + if (plc == null) + plc = PUBLIC_POOL; + + ctx.io().send(nodeId, resTopic, res, plc); } catch (IgniteCheckedException e) { if (ctx.discovery().alive(nodeId)) @@ -366,59 +355,6 @@ else if (log.isDebugEnabled()) } } - /** - * Get IO policy. - * - * @return IO policy. - */ - private static byte threadIoPolicy() { - Byte plc = GridIoManager.currentPolicy(); - - if (plc == null) - plc = DFLT_POLICY; - - return plc; - } - - /** - * Get IO policy for particular node. - * - * @param node Node. - * @return Policy. - */ - public static byte ioPolicy(ClusterNode node) { - assert node != null; - - if (node.isLocal() || node.version().greaterThanEqual( - DATA_STREAMER_POOL_MAJOR_VER, - DATA_STREAMER_POOL_MINOR_VER, - DATA_STREAMER_POOL_MAINTENANCE_VER)) - return GridIoPolicy.DATA_STREAMER_POOL; - else - return DFLT_POLICY; - } - - /** - * Get IO policy for particular node with provided resolver. - * - * @param rslvr Resolver. - * @param node Node. - * @return IO policy. - */ - public static byte ioPolicy(@Nullable IgniteClosure rslvr, ClusterNode node) { - assert node != null; - - Byte res = null; - - if (rslvr != null) - res = rslvr.apply(node); - - if (res == null) - res = ioPolicy(node); - - return res; - } - /** {@inheritDoc} */ @Override public void printMemoryStats() { X.println(">>>"); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java index 46f6380dc22d7..c2f226c537025 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java @@ -39,6 +39,7 @@ import java.util.concurrent.atomic.AtomicReference; import javax.cache.CacheException; import javax.cache.expiry.ExpiryPolicy; + import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; @@ -59,6 +60,7 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; +import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.deployment.GridDeployment; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; @@ -104,12 +106,16 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.internal.GridTopic.TOPIC_DATASTREAM; +import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL; /** * Data streamer implementation. */ @SuppressWarnings("unchecked") public class DataStreamerImpl implements IgniteDataStreamer, Delayed { + /** Default policy reoslver. */ + private static final DefaultIoPolicyResolver DFLT_IO_PLC_RSLVR = new DefaultIoPolicyResolver(); + /** Isolated receiver. */ private static final StreamReceiver ISOLATED_UPDATER = new IsolatedUpdater(); @@ -120,7 +126,7 @@ public class DataStreamerImpl implements IgniteDataStreamer, Delayed private byte[] updaterBytes; /** IO policy resovler for data load request. */ - private IgniteClosure ioPlcRslvr; + private IgniteClosure ioPlcRslvr = DFLT_IO_PLC_RSLVR; /** Max remap count before issuing an error. */ private static final int DFLT_MAX_REMAP_CNT = 32; @@ -1307,12 +1313,14 @@ private void submit(final Collection entries, IgniteInternalFuture fut; - byte plc = DataStreamProcessor.ioPolicy(ioPlcRslvr, node); + Byte plc = ioPlcRslvr.apply(node); + + if (plc == null) + plc = PUBLIC_POOL; - if (isLocNode) { + if (isLocNode && plc == GridIoPolicy.PUBLIC_POOL) { fut = ctx.closure().callLocalSafe( - new DataStreamerUpdateJob(ctx, log, cacheName, entries, false, skipStore, keepBinary, rcvr), - plc); + new DataStreamerUpdateJob(ctx, log, cacheName, entries, false, skipStore, keepBinary, rcvr), false); locFuts.add(fut); @@ -1675,6 +1683,19 @@ else if (ttl == CU.TTL_NOT_CHANGED) } } + /** + * Default IO policy resolver. + */ + private static class DefaultIoPolicyResolver implements IgniteClosure { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public Byte apply(ClusterNode gridNode) { + return PUBLIC_POOL; + } + } + /** * Key object wrapper. Using identity equals prevents slow down in case of hash code collision. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java index 5d38e1645e8cf..41e805ef9c515 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java @@ -123,9 +123,6 @@ public Executor poolForPolicy(byte plc) throws IgniteCheckedException { return ctx.getIgfsExecutorService(); - case GridIoPolicy.DATA_STREAMER_POOL: - return ctx.getDataStreamerExecutorService(); - default: { if (plc < 0) throw new IgniteCheckedException("Policy cannot be negative: " + plc); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java index 401b09c51b20c..9fedc35ae4c8e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java @@ -33,7 +33,6 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteDataStreamer; -import org.apache.ignite.IgniteException; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.affinity.Affinity; @@ -50,7 +49,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter; -import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; @@ -61,7 +59,6 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.stream.StreamReceiver; -import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jetbrains.annotations.Nullable; @@ -951,100 +948,6 @@ public void testCustomUserUpdater() throws Exception { } } - /** - * @throws Exception If failed. - */ - public void testLocalDataStreamerDedicatedThreadPool() throws Exception { - try { - useCache = true; - - Ignite ignite = startGrid(1); - - final IgniteCache cache = ignite.cache(null); - - IgniteDataStreamer ldr = ignite.dataStreamer(null); - try { - ldr.receiver(new StreamReceiver() { - @Override public void receive(IgniteCache cache, - Collection> entries) throws IgniteException { - String threadName = Thread.currentThread().getName(); - - cache.put("key", threadName); - } - }); - ldr.addData("key", "value"); - - ldr.tryFlush(); - - GridTestUtils.waitForCondition(new GridAbsPredicate() { - @Override public boolean apply() { - return cache.get("key") != null; - } - }, 3_000); - } - finally { - ldr.close(true); - } - - assertNotNull(cache.get("key")); - - assertTrue(cache.get("key").startsWith("data-streamer")); - - } - finally { - stopAllGrids(); - } - } - - /** - * @throws Exception If failed. - */ - public void testRemoteDataStreamerDedicatedThreadPool() throws Exception { - try { - useCache = true; - - Ignite ignite = startGrid(1); - - useCache = false; - - Ignite client = startGrid(0); - - final IgniteCache cache = ignite.cache(null); - - IgniteDataStreamer ldr = client.dataStreamer(null); - try { - ldr.receiver(new StreamReceiver() { - @Override public void receive(IgniteCache cache, - Collection> entries) throws IgniteException { - String threadName = Thread.currentThread().getName(); - - cache.put("key", threadName); - } - }); - - ldr.addData("key", "value"); - - ldr.tryFlush(); - - GridTestUtils.waitForCondition(new GridAbsPredicate() { - @Override public boolean apply() { - return cache.get("key") != null; - } - }, 3_000); - } - finally { - ldr.close(true); - } - - assertNotNull(cache.get("key")); - - assertTrue(cache.get("key").startsWith("data-streamer")); - } - finally { - stopAllGrids(); - } - } - /** * */ diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java index 1d72d4640247e..cba67e0d06005 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java @@ -62,7 +62,6 @@ public GridTestKernalContext(IgniteLogger log, IgniteConfiguration cfg) throws I null, null, null, - null, U.allPluginProviders()); GridTestUtils.setFieldValue(grid(), "cfg", config()); @@ -96,6 +95,11 @@ public void stop(boolean cancel) throws IgniteCheckedException { } } + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridTestKernalContext.class, this, super.toString()); + } + /** * Sets system executor service. * @@ -105,6 +109,7 @@ public void setSystemExecutorService(ExecutorService sysExecSvc) { this.sysExecSvc = sysExecSvc; } + /** * Sets executor service. * @@ -113,9 +118,4 @@ public void setSystemExecutorService(ExecutorService sysExecSvc) { public void setExecutorService(ExecutorService execSvc){ this.execSvc = execSvc; } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(GridTestKernalContext.class, this, super.toString()); - } } From ccae52cc1af3fa2077ece6f6a9c4146e3ba72cb4 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Tue, 18 Oct 2016 15:01:03 +0500 Subject: [PATCH 250/487] IGNITE-2355 Fixed the test HadoopClientProtocolMultipleServersSelfTest. Close FileSystem after each test to prevent using the one Client delegate for all tests. --- ...ClientProtocolMultipleServersSelfTest.java | 93 +++++++++++-------- 1 file changed, 55 insertions(+), 38 deletions(-) diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java index 04747d0b678f7..0e51938c12f29 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java @@ -24,6 +24,7 @@ import java.util.Collection; import java.util.concurrent.Callable; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; @@ -40,7 +41,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.ignite.IgniteFileSystem; -import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.mapreduce.IgniteHadoopClientProtocolProvider; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.client.GridServerUnreachableException; @@ -61,6 +62,9 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS /** Job name. */ private static final String JOB_NAME = "myJob"; + /** Rest port. */ + private static int restPort; + /** {@inheritDoc} */ @Override protected boolean igfsEnabled() { return true; @@ -71,15 +75,6 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS return true; } - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - super.beforeTest(); - - startGrids(gridCount()); - - awaitPartitionMapExchange(); - } - /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); @@ -88,10 +83,10 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS } /** {@inheritDoc} */ - @Override protected CacheConfiguration dataCacheConfiguration() { - CacheConfiguration cfg = super.dataCacheConfiguration(); + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); - cfg.setBackups(1); + cfg.getConnectorConfiguration().setPort(restPort++); return cfg; } @@ -149,15 +144,22 @@ public void checkJobSubmit(Configuration conf) throws Exception { */ @SuppressWarnings("ConstantConditions") public void testMultipleAddresses() throws Exception { - beforeJob(); + try { + restPort = REST_PORT; + + startGrids(gridCount()); - stopGrid(0); + beforeJob(); - U.sleep(5000); + U.sleep(5000); - checkJobSubmit(configMultipleAddrs(gridCount())); + checkJobSubmit(configMultipleAddrs(gridCount())); + } + finally { + FileSystem fs = FileSystem.get(configMultipleAddrs(gridCount())); - startGrid(0); + fs.close(); + } } /** @@ -165,21 +167,25 @@ public void testMultipleAddresses() throws Exception { */ @SuppressWarnings({"ConstantConditions", "ThrowableResultOfMethodCallIgnored"}) public void testSingleAddress() throws Exception { - stopGrid(0); - - U.sleep(5000); - - GridTestUtils.assertThrowsAnyCause(log, new Callable() { - @Override public Object call() throws Exception { - checkJobSubmit(configSingleAddress()); - return null; - } - }, - GridServerUnreachableException.class, "Failed to connect to any of the servers in list"); - - startGrid(0); + try { + // Don't use REST_PORT to test connection fails if the only this port is configured + restPort = REST_PORT + 1; + + startGrids(gridCount()); + + GridTestUtils.assertThrowsAnyCause(log, new Callable() { + @Override public Object call() throws Exception { + checkJobSubmit(configSingleAddress()); + return null; + } + }, + GridServerUnreachableException.class, "Failed to connect to any of the servers in list"); + } + finally { + FileSystem fs = FileSystem.get(configSingleAddress()); - awaitPartitionMapExchange(); + fs.close(); + } } /** @@ -187,17 +193,28 @@ public void testSingleAddress() throws Exception { */ @SuppressWarnings("ConstantConditions") public void testMixedAddrs() throws Exception { - beforeJob(); + try { + restPort = REST_PORT; + + startGrids(gridCount()); - stopGrid(1); + beforeJob(); - U.sleep(5000); + stopGrid(1); - checkJobSubmit(configMixed()); + U.sleep(5000); - startGrid(1); + checkJobSubmit(configMixed()); - awaitPartitionMapExchange(); + startGrid(1); + + awaitPartitionMapExchange(); + } + finally { + FileSystem fs = FileSystem.get(configMixed()); + + fs.close(); + } } /** From a863eeed9d152eed8418416291b9b19bd5b5e88b Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Tue, 18 Oct 2016 13:20:30 +0300 Subject: [PATCH 251/487] IGNITE-1924 Incomplete marshaller cache rebalancing causes Grid hangs under SSL (HOTFIX) --- .../apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java index 273df5d9d69f7..b3e820153e5dc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java @@ -472,8 +472,6 @@ private ByteBuffer allocateAppBuff() { */ private void readFromNet() throws IgniteCheckedException { try { - inNetBuf.clear(); - int read = ch.read(inNetBuf); if (read == -1) From 7290d88e14a15a3d030b7381dbd0a3f14cb65a12 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 18 Oct 2016 17:17:17 +0300 Subject: [PATCH 252/487] IGNITE-4030 Streamline PlatformTarget operation methods This closes #1167 --- .../platform/PlatformAbstractTarget.java | 54 ------------------- .../processors/platform/PlatformTarget.java | 20 ------- .../platform/cache/PlatformCache.java | 27 ++++------ .../cache/affinity/PlatformAffinity.java | 4 +- .../query/PlatformAbstractQueryCursor.java | 4 +- .../query/PlatformContinuousQueryProxy.java | 3 +- .../cluster/PlatformClusterGroup.java | 4 +- .../platform/compute/PlatformCompute.java | 9 +--- .../datastreamer/PlatformDataStreamer.java | 9 +--- .../datastructures/PlatformAtomicLong.java | 27 ++++------ .../PlatformAtomicReference.java | 8 ++- .../PlatformAtomicSequence.java | 31 +++++------ .../platform/services/PlatformServices.java | 13 +++-- .../transactions/PlatformTransactions.java | 17 ++---- .../core/src/impl/cache/query/query_impl.cpp | 6 +-- .../core/src/impl/interop/interop_target.cpp | 2 +- .../cpp/jni/include/ignite/jni/exports.h | 2 - .../cpp/jni/include/ignite/jni/java.h | 4 -- .../platforms/cpp/jni/project/vs/module.def | 3 +- modules/platforms/cpp/jni/src/exports.cpp | 8 --- modules/platforms/cpp/jni/src/java.cpp | 23 -------- .../Impl/Cache/CacheAffinityImpl.cs | 2 +- .../Impl/Cache/CacheImpl.cs | 14 ++--- .../Impl/Cache/Query/AbstractQueryCursor.cs | 4 +- .../Continuous/ContinuousQueryHandleImpl.cs | 2 +- .../Impl/Compute/ComputeImpl.cs | 4 +- .../Impl/DataStructures/AtomicLong.cs | 14 ++--- .../Impl/DataStructures/AtomicReference.cs | 4 +- .../Impl/DataStructures/AtomicSequence.cs | 14 ++--- .../Impl/Datastream/DataStreamerImpl.cs | 20 +++---- .../Apache.Ignite.Core/Impl/Events/Events.cs | 4 +- .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs | 2 +- .../Apache.Ignite.Core/Impl/PlatformTarget.cs | 41 +------------- .../Impl/Services/Services.cs | 4 +- .../Impl/Transactions/TransactionsImpl.cs | 12 ++--- .../Impl/Unmanaged/IgniteJniNativeMethods.cs | 9 +--- .../Impl/Unmanaged/UnmanagedUtils.cs | 12 +---- 37 files changed, 114 insertions(+), 326 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java index 22adef88182ca..09092c226fea0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java @@ -98,16 +98,6 @@ protected PlatformAbstractTarget(PlatformContext platformCtx) { } } - /** {@inheritDoc} */ - @Override public long outLong(int type) throws Exception { - try { - return processOutLong(type); - } - catch (Exception e) { - throw convertException(e); - } - } - /** {@inheritDoc} */ @Override public void outStream(int type, long memPtr) throws Exception { try (PlatformMemory mem = platformCtx.memory().get(memPtr)) { @@ -154,26 +144,6 @@ protected PlatformAbstractTarget(PlatformContext platformCtx) { } } - /** {@inheritDoc} */ - @Override public void inObjectStreamOutStream(int type, Object arg, long inMemPtr, long outMemPtr) throws Exception { - try (PlatformMemory inMem = platformCtx.memory().get(inMemPtr)) { - BinaryRawReaderEx reader = platformCtx.reader(inMem); - - try (PlatformMemory outMem = platformCtx.memory().get(outMemPtr)) { - PlatformOutputStream out = outMem.output(); - - BinaryRawWriterEx writer = platformCtx.writer(out); - - processInObjectStreamOutStream(type, arg, reader, writer); - - out.synchronize(); - } - } - catch (Exception e) { - throw convertException(e); - } - } - /** {@inheritDoc} */ @Override public Object inObjectStreamOutObjectStream(int type, Object arg, long inMemPtr, long outMemPtr) throws Exception { @@ -329,20 +299,6 @@ protected Object processInStreamOutObject(int type, BinaryRawReaderEx reader) th return throwUnsupported(type); } - /** - * Process IN-OUT operation. - * - * @param type Type. - * @param arg Argument. - * @param reader Binary reader. - * @param writer Binary writer. - * @throws IgniteCheckedException In case of exception. - */ - protected void processInObjectStreamOutStream(int type, @Nullable Object arg, BinaryRawReaderEx reader, - BinaryRawWriterEx writer) throws IgniteCheckedException { - throwUnsupported(type); - } - /** * Process IN-OUT operation. * @@ -357,16 +313,6 @@ protected Object processInObjectStreamOutObjectStream(int type, @Nullable Object return throwUnsupported(type); } - /** - * Process OUT operation. - * - * @param type Type. - * @throws IgniteCheckedException In case of exception. - */ - protected long processOutLong(int type) throws IgniteCheckedException { - return throwUnsupported(type); - } - /** * Process OUT operation. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java index 40773d0b09bcd..cde64024ef2a7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java @@ -66,17 +66,6 @@ public interface PlatformTarget { */ public void inStreamOutStream(int type, long inMemPtr, long outMemPtr) throws Exception; - /** - * Operation accepting an object and a memory stream and returning result to another memory stream. - * - * @param type Operation type. - * @param arg Argument (optional). - * @param inMemPtr Input memory pointer. - * @param outMemPtr Output memory pointer. - * @throws Exception In case of failure. - */ - public void inObjectStreamOutStream(int type, @Nullable Object arg, long inMemPtr, long outMemPtr) throws Exception; - /** * Operation accepting an object and a memory stream and returning result to another memory stream and an object. * @@ -90,15 +79,6 @@ public interface PlatformTarget { public Object inObjectStreamOutObjectStream(int type, @Nullable Object arg, long inMemPtr, long outMemPtr) throws Exception; - /** - * Operation returning long result. - * - * @param type Operation type. - * @return Result. - * @throws Exception In case of failure. - */ - public long outLong(int type) throws Exception; - /** * Operation returning result to memory stream. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index 05945e0f93af8..7986286b2a7cb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -273,23 +273,6 @@ public PlatformCache(PlatformContext platformCtx, IgniteCache cache, boolean kee this.keepBinary = keepBinary; } - /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { - switch (type) { - case OP_CLEAR_CACHE: - cache.clear(); - - return TRUE; - - case OP_REMOVE_ALL2: - cache.removeAll(); - - return TRUE; - } - - return super.processOutLong(type); - } - /** {@inheritDoc} */ @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException { switch (type) { @@ -785,6 +768,16 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc) { return TRUE; } + + case OP_CLEAR_CACHE: + cache.clear(); + + return TRUE; + + case OP_REMOVE_ALL2: + cache.removeAll(); + + return TRUE; } return super.processInLongOutLong(type, val); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java index 41b58aaaf2c19..12df18890889e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java @@ -292,10 +292,10 @@ public PlatformAffinity(PlatformContext platformCtx, GridKernalContext igniteCtx } /** {@inheritDoc} */ - @Override public long outLong(int type) throws Exception { + @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException { if (type == OP_PARTITIONS) return aff.partitions(); - return super.outLong(type); + return super.processInLongOutLong(type, val); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java index ff28b818066b9..6a259caa63e83 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java @@ -136,7 +136,7 @@ public PlatformAbstractQueryCursor(PlatformContext platformCtx, QueryCursorEx } /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { switch (type) { case OP_ITERATOR: iter = cursor.iterator(); @@ -154,7 +154,7 @@ public PlatformAbstractQueryCursor(PlatformContext platformCtx, QueryCursorEx return iter.hasNext() ? TRUE : FALSE; } - return super.processOutLong(type); + return super.processInLongOutLong(type, val); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java index a4d7cad5abb39..04f17ff2804c1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.platform.cache.query; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; @@ -45,7 +46,7 @@ public PlatformContinuousQueryProxy(PlatformContext platformCtx, PlatformContinu } /** {@inheritDoc} */ - @Override public long outLong(int type) throws Exception { + @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException { qry.close(); return 0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java index a94e04542516e..957eaaf74b2c9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java @@ -322,7 +322,7 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { } /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { switch (type) { case OP_RESET_METRICS: { assert prj instanceof IgniteCluster; // Can only be invoked on top-level cluster group. @@ -333,7 +333,7 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { } } - return super.processOutLong(type); + return super.processInLongOutLong(type, val); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java index 36d709a3f1356..32e484802dc68 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java @@ -135,14 +135,7 @@ public PlatformCompute(PlatformContext platformCtx, ClusterGroup grp, String pla return TRUE; } - } - - return super.processInLongOutLong(type, val); - } - /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { - switch (type) { case OP_WITH_NO_FAILOVER: { compute.withNoFailover(); computeForPlatform.withNoFailover(); @@ -151,7 +144,7 @@ public PlatformCompute(PlatformContext platformCtx, ClusterGroup grp, String pla } } - return super.processOutLong(type); + return super.processInLongOutLong(type, val); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java index 2822b7f9125b6..cd5fba059c218 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java @@ -217,14 +217,7 @@ else if (plc == PLC_FLUSH) return TRUE; } - } - - return super.processInLongOutLong(type, val); - } - /** {@inheritDoc} */ - @Override public long processOutLong(int type) throws IgniteCheckedException { - switch (type) { case OP_ALLOW_OVERWRITE: return ldr.allowOverwrite() ? TRUE : FALSE; @@ -238,7 +231,7 @@ else if (plc == PLC_FLUSH) return ldr.perNodeParallelOperations(); } - return super.processOutLong(type); + return super.processInLongOutLong(type, val); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java index 53319560f95c1..811e38bcadd79 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java @@ -99,8 +99,17 @@ public PlatformAtomicLong(PlatformContext ctx, GridCacheAtomicLongImpl atomicLon } /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { switch (type) { + case OP_ADD_AND_GET: + return atomicLong.addAndGet(val); + + case OP_GET_AND_ADD: + return atomicLong.getAndAdd(val); + + case OP_GET_AND_SET: + return atomicLong.getAndSet(val); + case OP_CLOSE: atomicLong.close(); @@ -125,22 +134,6 @@ public PlatformAtomicLong(PlatformContext ctx, GridCacheAtomicLongImpl atomicLon return atomicLong.removed() ? TRUE : FALSE; } - return super.processOutLong(type); - } - - /** {@inheritDoc} */ - @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { - switch (type) { - case OP_ADD_AND_GET: - return atomicLong.addAndGet(val); - - case OP_GET_AND_ADD: - return atomicLong.getAndAdd(val); - - case OP_GET_AND_SET: - return atomicLong.getAndSet(val); - } - return super.processInLongOutLong(type, val); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java index e5fc08d8b69e4..63b5b86ec6b69 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java @@ -24,7 +24,6 @@ import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; -import org.omg.CORBA.TRANSACTION_REQUIRED; /** * Platform atomic reference wrapper. @@ -135,7 +134,7 @@ private PlatformAtomicReference(PlatformContext ctx, GridCacheAtomicReferenceImp } /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { switch (type) { case OP_CLOSE: atomicRef.close(); @@ -146,7 +145,6 @@ private PlatformAtomicReference(PlatformContext ctx, GridCacheAtomicReferenceImp return atomicRef.removed() ? TRUE : FALSE; } - return super.processOutLong(type); + return super.processInLongOutLong(type, val); } -} - +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java index ec946cac303e1..c35273189f13f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java @@ -71,8 +71,19 @@ public PlatformAtomicSequence(PlatformContext ctx, IgniteAtomicSequence atomicSe /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { switch (type) { + case OP_ADD_AND_GET: + return atomicSeq.addAndGet(val); + + case OP_GET_AND_ADD: + return atomicSeq.getAndAdd(val); + + case OP_SET_BATCH_SIZE: + atomicSeq.batchSize((int)val); + + return TRUE; + case OP_CLOSE: atomicSeq.close(); @@ -94,24 +105,6 @@ public PlatformAtomicSequence(PlatformContext ctx, IgniteAtomicSequence atomicSe return atomicSeq.batchSize(); } - return super.processOutLong(type); - } - - /** {@inheritDoc} */ - @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { - switch (type) { - case OP_ADD_AND_GET: - return atomicSeq.addAndGet(val); - - case OP_GET_AND_ADD: - return atomicSeq.getAndAdd(val); - - case OP_SET_BATCH_SIZE: - atomicSeq.batchSize((int)val); - - return TRUE; - } - return super.processInLongOutLong(type, val); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java index 7aaf5970a9118..dc6350363bcb0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java @@ -205,7 +205,7 @@ private ServiceDescriptor findDescriptor(String name) { } /** {@inheritDoc} */ - @Override protected void processInObjectStreamOutStream(int type, Object arg, BinaryRawReaderEx reader, + @Override protected Object processInObjectStreamOutObjectStream(int type, Object arg, BinaryRawReaderEx reader, BinaryRawWriterEx writer) throws IgniteCheckedException { switch (type) { case OP_INVOKE: { @@ -234,12 +234,11 @@ private ServiceDescriptor findDescriptor(String name) { PlatformUtils.writeInvocationResult(writer, null, e); } - return; + return null; } - - default: - super.processInObjectStreamOutStream(type, arg, reader, writer); } + + return super.processInObjectStreamOutObjectStream(type, arg, reader, writer); } /** {@inheritDoc} */ @@ -298,7 +297,7 @@ private ServiceDescriptor findDescriptor(String name) { } /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { switch (type) { case OP_CANCEL_ALL: services.cancelAll(); @@ -306,7 +305,7 @@ private ServiceDescriptor findDescriptor(String name) { return TRUE; } - return super.processOutLong(type); + return super.processInLongOutLong(type, val); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java index 339937c98fc67..92d8b53decfd8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java @@ -168,18 +168,6 @@ private Transaction tx(long id) { return tx; } - /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { - switch (type) { - case OP_RESET_METRICS: - txs.resetMetrics(); - - return TRUE; - } - - return super.processOutLong(type); - } - /** {@inheritDoc} */ @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { switch (type) { @@ -201,6 +189,11 @@ private Transaction tx(long id) { case OP_STATE: return tx(val).state().ordinal(); + + case OP_RESET_METRICS: + txs.resetMetrics(); + + return TRUE; } return super.processInLongOutLong(type, val); diff --git a/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp b/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp index c65e1e8ee5958..aaeb8228367f1 100644 --- a/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp +++ b/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp @@ -67,7 +67,7 @@ namespace ignite delete batch; // 2. Close the cursor. - env.Get()->Context()->TargetOutLong(javaRef, OP_ITERATOR_CLOSE); + env.Get()->Context()->TargetInLongOutLong(javaRef, OP_ITERATOR_CLOSE, 0); // 3. Release Java reference. JniContext::Release(javaRef); @@ -199,7 +199,7 @@ namespace ignite JniErrorInfo jniErr; - env.Get()->Context()->TargetOutLong(javaRef, OP_ITERATOR, &jniErr); + env.Get()->Context()->TargetInLongOutLong(javaRef, OP_ITERATOR, 0, &jniErr); IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); @@ -249,7 +249,7 @@ namespace ignite { JniErrorInfo jniErr; - bool res = env.Get()->Context()->TargetOutLong(javaRef, OP_ITERATOR_HAS_NEXT, &jniErr) == 1; + bool res = env.Get()->Context()->TargetInLongOutLong(javaRef, OP_ITERATOR_HAS_NEXT, 0, &jniErr) == 1; IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); diff --git a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp index 592c1ba2eb93a..b0c7b893cb584 100644 --- a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp +++ b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp @@ -100,7 +100,7 @@ namespace ignite { JniErrorInfo jniErr; - long long res = env.Get()->Context()->TargetOutLong(javaRef, opType, &jniErr); + long long res = env.Get()->Context()->TargetInLongOutLong(javaRef, opType, 0, &jniErr); IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); diff --git a/modules/platforms/cpp/jni/include/ignite/jni/exports.h b/modules/platforms/cpp/jni/include/ignite/jni/exports.h index 84f5a29f0c29f..e476cf83b8b6a 100644 --- a/modules/platforms/cpp/jni/include/ignite/jni/exports.h +++ b/modules/platforms/cpp/jni/include/ignite/jni/exports.h @@ -59,9 +59,7 @@ extern "C" { long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr); void IGNITE_CALL IgniteTargetInStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, long long inMemPtr, long long outMemPtr); void* IGNITE_CALL IgniteTargetInStreamOutObject(gcj::JniContext* ctx, void* obj, int opType, long long memPtr); - void IGNITE_CALL IgniteTargetInObjectStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, void* arg, long long inMemPtr, long long outMemPtr); void* IGNITE_CALL IgniteTargetInObjectStreamOutObjectStream(gcj::JniContext* ctx, void* obj, int opType, void* arg, long long inMemPtr, long long outMemPtr); - long long IGNITE_CALL IgniteTargetOutLong(gcj::JniContext* ctx, void* obj, int opType); void IGNITE_CALL IgniteTargetOutStream(gcj::JniContext* ctx, void* obj, int opType, long long memPtr); void* IGNITE_CALL IgniteTargetOutObject(gcj::JniContext* ctx, void* obj, int opType); void IGNITE_CALL IgniteTargetListenFuture(gcj::JniContext* ctx, void* obj, long long futId, int typ); diff --git a/modules/platforms/cpp/jni/include/ignite/jni/java.h b/modules/platforms/cpp/jni/include/ignite/jni/java.h index 96daba8270f65..1f2de2e1fe778 100644 --- a/modules/platforms/cpp/jni/include/ignite/jni/java.h +++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h @@ -261,11 +261,9 @@ namespace ignite jmethodID m_PlatformTarget_inLongOutLong; jmethodID m_PlatformTarget_inStreamOutLong; jmethodID m_PlatformTarget_inStreamOutObject; - jmethodID m_PlatformTarget_outLong; jmethodID m_PlatformTarget_outStream; jmethodID m_PlatformTarget_outObject; jmethodID m_PlatformTarget_inStreamOutStream; - jmethodID m_PlatformTarget_inObjectStreamOutStream; jmethodID m_PlatformTarget_inObjectStreamOutObjectStream; jmethodID m_PlatformTarget_listenFuture; jmethodID m_PlatformTarget_listenFutureForOperation; @@ -441,9 +439,7 @@ namespace ignite long long TargetInStreamOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); void TargetInStreamOutStream(jobject obj, int opType, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL); jobject TargetInStreamOutObject(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); - void TargetInObjectStreamOutStream(jobject obj, int opType, void* arg, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL); jobject TargetInObjectStreamOutObjectStream(jobject obj, int opType, void* arg, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL); - long long TargetOutLong(jobject obj, int opType, JniErrorInfo* errInfo = NULL); void TargetOutStream(jobject obj, int opType, long long memPtr, JniErrorInfo* errInfo = NULL); jobject TargetOutObject(jobject obj, int opType, JniErrorInfo* errInfo = NULL); void TargetListenFuture(jobject obj, long long futId, int typ); diff --git a/modules/platforms/cpp/jni/project/vs/module.def b/modules/platforms/cpp/jni/project/vs/module.def index c2069b6a91e19..7930b5a0190a2 100644 --- a/modules/platforms/cpp/jni/project/vs/module.def +++ b/modules/platforms/cpp/jni/project/vs/module.def @@ -6,7 +6,6 @@ IgniteIgnitionInstance @3 IgniteIgnitionEnvironmentPointer @4 IgniteIgnitionStop @5 IgniteIgnitionStopAll @6 -IgniteTargetOutLong @7 IgniteProcessorReleaseStart @8 IgniteProcessorProjection @9 IgniteProcessorCache @10 @@ -20,7 +19,7 @@ IgniteTargetInStreamOutObject @17 IgniteTargetInStreamOutLong @18 IgniteTargetOutStream @19 IgniteTargetInStreamOutStream @20 -IgniteTargetInObjectStreamOutStream @21 +IgniteTargetInObjectStreamOutObjectStream @21 IgniteTargetListenFuture @22 IgniteTargetListenFutureForOperation @23 IgniteTargetInLongOutLong @24 diff --git a/modules/platforms/cpp/jni/src/exports.cpp b/modules/platforms/cpp/jni/src/exports.cpp index ca332be619084..76f38ce9e0505 100644 --- a/modules/platforms/cpp/jni/src/exports.cpp +++ b/modules/platforms/cpp/jni/src/exports.cpp @@ -154,18 +154,10 @@ extern "C" { return ctx->TargetInStreamOutObject(static_cast(obj), opType, memPtr); } - void IGNITE_CALL IgniteTargetInObjectStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, void* arg, long long inMemPtr, long long outMemPtr) { - ctx->TargetInObjectStreamOutStream(static_cast(obj), opType, arg, inMemPtr, outMemPtr); - } - void* IGNITE_CALL IgniteTargetInObjectStreamOutObjectStream(gcj::JniContext* ctx, void* obj, int opType, void* arg, long long inMemPtr, long long outMemPtr) { return ctx->TargetInObjectStreamOutObjectStream(static_cast(obj), opType, arg, inMemPtr, outMemPtr); } - long long IGNITE_CALL IgniteTargetOutLong(gcj::JniContext* ctx, void* obj, int opType) { - return ctx->TargetOutLong(static_cast(obj), opType); - } - void IGNITE_CALL IgniteTargetOutStream(gcj::JniContext* ctx, void* obj, int opType, long long memPtr) { ctx->TargetOutStream(static_cast(obj), opType, memPtr); } diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp index 5bbaac402ba50..4c16976be1e1f 100644 --- a/modules/platforms/cpp/jni/src/java.cpp +++ b/modules/platforms/cpp/jni/src/java.cpp @@ -218,9 +218,7 @@ namespace ignite JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_LONG = JniMethod("inStreamOutLong", "(IJ)J", false); JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_OBJECT = JniMethod("inStreamOutObject", "(IJ)Ljava/lang/Object;", false); JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_STREAM = JniMethod("inStreamOutStream", "(IJJ)V", false); - JniMethod M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_STREAM = JniMethod("inObjectStreamOutStream", "(ILjava/lang/Object;JJ)V", false); JniMethod M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_OBJECT_STREAM = JniMethod("inObjectStreamOutObjectStream", "(ILjava/lang/Object;JJ)Ljava/lang/Object;", false); - JniMethod M_PLATFORM_TARGET_OUT_LONG = JniMethod("outLong", "(I)J", false); JniMethod M_PLATFORM_TARGET_OUT_STREAM = JniMethod("outStream", "(IJ)V", false); JniMethod M_PLATFORM_TARGET_OUT_OBJECT = JniMethod("outObject", "(I)Ljava/lang/Object;", false); JniMethod M_PLATFORM_TARGET_LISTEN_FUTURE = JniMethod("listenFuture", "(JI)V", false); @@ -512,11 +510,9 @@ namespace ignite m_PlatformTarget_inLongOutLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_LONG_OUT_LONG); m_PlatformTarget_inStreamOutLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_LONG); m_PlatformTarget_inStreamOutObject = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_OBJECT); - m_PlatformTarget_outLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_OUT_LONG); m_PlatformTarget_outStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_OUT_STREAM); m_PlatformTarget_outObject = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_OUT_OBJECT); m_PlatformTarget_inStreamOutStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_STREAM); - m_PlatformTarget_inObjectStreamOutStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_STREAM); m_PlatformTarget_inObjectStreamOutObjectStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_OBJECT_STREAM); m_PlatformTarget_listenFuture = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_LISTEN_FUTURE); m_PlatformTarget_listenFutureForOperation = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_LISTEN_FOR_OPERATION); @@ -1295,14 +1291,6 @@ namespace ignite return LocalToGlobal(env, res); } - void JniContext::TargetInObjectStreamOutStream(jobject obj, int opType, void* arg, long long inMemPtr, long long outMemPtr, JniErrorInfo* err) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformTarget_inObjectStreamOutStream, opType, arg, inMemPtr, outMemPtr); - - ExceptionCheck(env, err); - } - jobject JniContext::TargetInObjectStreamOutObjectStream(jobject obj, int opType, void* arg, long long inMemPtr, long long outMemPtr, JniErrorInfo* err) { JNIEnv* env = Attach(); @@ -1313,17 +1301,6 @@ namespace ignite return LocalToGlobal(env, res); } - long long JniContext::TargetOutLong(jobject obj, int opType, JniErrorInfo* err) - { - JNIEnv* env = Attach(); - - jlong res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformTarget_outLong, opType); - - ExceptionCheck(env, err); - - return res; - } - void JniContext::TargetOutStream(jobject obj, int opType, long long memPtr, JniErrorInfo* err) { JNIEnv* env = Attach(); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs index 4ff1f2e5281c1..f09a119700014 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs @@ -104,7 +104,7 @@ internal class CacheAffinityImpl : PlatformTarget, ICacheAffinity /** */ public int Partitions { - get { return (int) DoOutOp(OpPartitions); } + get { return (int) DoOutInOp(OpPartitions); } } /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs index 3a82932bf9f54..2cc88e4142b8a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs @@ -640,7 +640,7 @@ public void LocalEvict(IEnumerable keys) /** */ public void Clear() { - DoOutOp((int) CacheOp.ClearCache); + DoOutInOp((int) CacheOp.ClearCache); } /** */ @@ -752,7 +752,7 @@ public Task RemoveAllAsync(IEnumerable keys) /** */ public void RemoveAll() { - DoOutOp((int) CacheOp.RemoveAll2); + DoOutInOp((int) CacheOp.RemoveAll2); } /** */ @@ -795,7 +795,7 @@ private int Size0(bool loc, params CachePeekMode[] modes) var op = loc ? CacheOp.SizeLoc : CacheOp.Size; - return (int) DoOutInOpLong((int) op, modes0); + return (int) DoOutInOp((int) op, modes0); } /** */ @@ -919,7 +919,7 @@ public ICacheMetrics GetMetrics() /** */ public Task Rebalance() { - return GetFuture((futId, futTyp) => DoOutInOpLong((int) CacheOp.Rebalance, futId)).Task; + return GetFuture((futId, futTyp) => DoOutInOp((int) CacheOp.Rebalance, futId)).Task; } /** */ @@ -1273,7 +1273,7 @@ private CacheResult DoOutInOpNullable(int type, Action out /** */ public void Enter(long id) { - DoOutInOpLong((int) CacheOp.EnterLock, id); + DoOutInOp((int) CacheOp.EnterLock, id); } /** */ @@ -1289,13 +1289,13 @@ public bool TryEnter(long id, TimeSpan timeout) /** */ public void Exit(long id) { - DoOutInOpLong((int) CacheOp.ExitLock, id); + DoOutInOp((int) CacheOp.ExitLock, id); } /** */ public void Close(long id) { - DoOutInOpLong((int) CacheOp.CloseLock, id); + DoOutInOp((int) CacheOp.CloseLock, id); } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs index e6092d7ddb115..95c6a36f59985 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs @@ -100,7 +100,7 @@ protected override void Dispose(bool disposing) { try { - DoOutOp(OpIteratorClose); + DoOutInOp(OpIteratorClose); } finally { @@ -125,7 +125,7 @@ public IEnumerator GetEnumerator() throw new InvalidOperationException("Failed to get enumerator entries because " + "GetAll() method has already been called."); - DoOutOp(OpIterator); + DoOutInOp(OpIterator); _iterCalled = true; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs index b66dc48b332c2..6139d8be51fdd 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs @@ -225,7 +225,7 @@ public void Dispose() try { - UU.TargetOutLong(_nativeQry, 0); + UU.TargetInLongOutLong(_nativeQry, 0, 0); } finally { diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs index 1b2e2aa914ef8..36d731d884ed0 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs @@ -106,7 +106,7 @@ public IClusterGroup ClusterGroup /// public void WithNoFailover() { - DoOutOp(OpWithNoFailover); + DoOutInOp(OpWithNoFailover); } /// @@ -116,7 +116,7 @@ public void WithNoFailover() /// Computation timeout in milliseconds. public void WithTimeout(long timeout) { - DoOutInOpLong(OpWithTimeout, timeout); + DoOutInOp(OpWithTimeout, timeout); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicLong.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicLong.cs index 571e6fd2707c0..0c4bf84b17ceb 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicLong.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicLong.cs @@ -68,31 +68,31 @@ public string Name /** */ public long Read() { - return DoOutOp((int) Op.Get); + return DoOutInOp((int) Op.Get); } /** */ public long Increment() { - return DoOutOp((int) Op.IncrementAndGet); + return DoOutInOp((int) Op.IncrementAndGet); } /** */ public long Add(long value) { - return DoOutInOpLong((int) Op.AddAndGet, value); + return DoOutInOp((int) Op.AddAndGet, value); } /** */ public long Decrement() { - return DoOutOp((int) Op.DecrementAndGet); + return DoOutInOp((int) Op.DecrementAndGet); } /** */ public long Exchange(long value) { - return DoOutInOpLong((int) Op.GetAndSet, value); + return DoOutInOp((int) Op.GetAndSet, value); } /** */ @@ -108,13 +108,13 @@ public long CompareExchange(long value, long comparand) /** */ public void Close() { - DoOutOp((int) Op.Close); + DoOutInOp((int) Op.Close); } /** */ public bool IsClosed() { - return DoOutOp((int) Op.IsClosed) == True; + return DoOutInOp((int) Op.IsClosed) == True; } } } \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs index 75e36d1c73b91..4ca4b249b3b2a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs @@ -82,13 +82,13 @@ public T CompareExchange(T value, T comparand) /** */ public bool IsClosed { - get { return DoOutOp((int) Op.IsClosed) == True; } + get { return DoOutInOp((int) Op.IsClosed) == True; } } /** */ public void Close() { - DoOutOp((int) Op.Close); + DoOutInOp((int) Op.Close); } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs index b7b924ed246a0..f7fc6b7fb2e44 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs @@ -65,38 +65,38 @@ public string Name /** */ public long Read() { - return DoOutOp((int) Op.Get); + return DoOutInOp((int) Op.Get); } /** */ public long Increment() { - return DoOutOp((int) Op.IncrementAndGet); + return DoOutInOp((int) Op.IncrementAndGet); } /** */ public long Add(long value) { - return DoOutInOpLong((int) Op.AddAndGet, value); + return DoOutInOp((int) Op.AddAndGet, value); } /** */ public int BatchSize { - get { return (int) DoOutOp((int) Op.GetBatchSize); } - set { DoOutInOpLong((int) Op.SetBatchSize, value); } + get { return (int) DoOutInOp((int) Op.GetBatchSize); } + set { DoOutInOp((int) Op.SetBatchSize, value); } } /** */ public bool IsClosed { - get { return DoOutOp((int) Op.IsClosed) == True; } + get { return DoOutInOp((int) Op.IsClosed) == True; } } /** */ public void Close() { - DoOutOp((int) Op.Close); + DoOutInOp((int) Op.Close); } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs index 8893fc5558695..b9e3030e90c4c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs @@ -156,7 +156,7 @@ public DataStreamerImpl(IUnmanagedTarget target, Marshaller marsh, string cacheN _hnd = marsh.Ignite.HandleRegistry.Allocate(thisRef); // Start topology listening. This call will ensure that buffer size member is updated. - DoOutInOpLong(OpListenTopology, _hnd); + DoOutInOp(OpListenTopology, _hnd); // Membar to ensure fields initialization before leaving constructor. Thread.MemoryBarrier(); @@ -184,7 +184,7 @@ public bool AllowOverwrite { ThrowIfDisposed(); - return DoOutOp(OpAllowOverwrite) == True; + return DoOutInOp(OpAllowOverwrite) == True; } finally { @@ -199,7 +199,7 @@ public bool AllowOverwrite { ThrowIfDisposed(); - DoOutInOpLong(OpSetAllowOverwrite, value ? True : False); + DoOutInOp(OpSetAllowOverwrite, value ? True : False); } finally { @@ -219,7 +219,7 @@ public bool SkipStore { ThrowIfDisposed(); - return DoOutOp(OpSkipStore) == True; + return DoOutInOp(OpSkipStore) == True; } finally { @@ -234,7 +234,7 @@ public bool SkipStore { ThrowIfDisposed(); - DoOutInOpLong(OpSetSkipStore, value ? True : False); + DoOutInOp(OpSetSkipStore, value ? True : False); } finally { @@ -254,7 +254,7 @@ public int PerNodeBufferSize { ThrowIfDisposed(); - return (int) DoOutOp(OpPerNodeBufferSize); + return (int) DoOutInOp(OpPerNodeBufferSize); } finally { @@ -269,7 +269,7 @@ public int PerNodeBufferSize { ThrowIfDisposed(); - DoOutInOpLong(OpSetPerNodeBufferSize, value); + DoOutInOp(OpSetPerNodeBufferSize, value); _bufSndSize = _topSize * value; } @@ -291,7 +291,7 @@ public int PerNodeParallelOperations { ThrowIfDisposed(); - return (int) DoOutOp(OpPerNodeParallelOps); + return (int) DoOutInOp(OpPerNodeParallelOps); } finally { @@ -307,7 +307,7 @@ public int PerNodeParallelOperations { ThrowIfDisposed(); - DoOutInOpLong(OpSetPerNodeParallelOps, value); + DoOutInOp(OpSetPerNodeParallelOps, value); } finally { @@ -598,7 +598,7 @@ public void TopologyChange(long topVer, int topSize) _topVer = topVer; _topSize = topSize > 0 ? topSize : 1; // Do not set to 0 to avoid 0 buffer size. - _bufSndSize = (int) (_topSize * DoOutOp(OpPerNodeBufferSize)); + _bufSndSize = (int) (_topSize * DoOutInOp(OpPerNodeBufferSize)); } } finally diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs index 5d1add640466c..db2b8d22b8c20 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs @@ -339,7 +339,7 @@ public void RecordLocal(IEvent evt) // Should do this inside lock to avoid race with subscription // ToArray is required because we are going to modify underlying dictionary during enumeration foreach (var filter in GetLocalFilters(listener, types).ToArray()) - success |= (DoOutInOpLong((int) Op.StopLocalListen, filter.Handle) == True); + success |= (DoOutInOp((int) Op.StopLocalListen, filter.Handle) == True); return success; } @@ -388,7 +388,7 @@ public ICollection GetEnabledEvents() /** */ public bool IsEnabled(int type) { - return DoOutInOpLong((int) Op.IsEnabled, type) == True; + return DoOutInOp((int) Op.IsEnabled, type) == True; } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs index 675af5e63931b..e2fd3860321fc 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs @@ -486,7 +486,7 @@ public ICollection GetTopology(long ver) /** */ public void ResetMetrics() { - UU.TargetOutLong(_prj.Target, ClusterGroupImpl.OpResetMetrics); + UU.TargetInLongOutLong(_prj.Target, ClusterGroupImpl.OpResetMetrics, 0); } /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs index 8c065bcc7249a..f7906ff5a052e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs @@ -255,16 +255,6 @@ protected static BinaryWriter WriteEnumerable(BinaryWriter writer, IEnumerabl #region OUT operations - /// - /// Perform out operation. - /// - /// Operation type. - /// Long result. - protected long DoOutOp(int type) - { - return UU.TargetOutLong(_target, type); - } - /// /// Perform out operation. /// @@ -494,35 +484,6 @@ protected TR DoOutInOp(int type, Action outAction, Func - /// Perform out-in operation. - /// - /// Operation type. - /// Out action. - /// In action. - /// Argument. - /// Result. - protected unsafe TR DoOutInOp(int type, Action outAction, Func inAction, void* arg) - { - using (PlatformMemoryStream outStream = IgniteManager.Memory.Allocate().GetStream()) - { - using (PlatformMemoryStream inStream = IgniteManager.Memory.Allocate().GetStream()) - { - BinaryWriter writer = _marsh.StartMarshal(outStream); - - outAction(writer); - - FinishMarshal(writer); - - UU.TargetInObjectStreamOutStream(_target, type, arg, outStream.SynchronizeOutput(), inStream.MemoryPointer); - - inStream.SynchronizeInput(); - - return inAction(inStream); - } - } - } - /// /// Perform out-in operation. /// @@ -672,7 +633,7 @@ protected TR DoOutInOp(int type, Action outAction) /// Operation type. /// Value. /// Result. - protected long DoOutInOpLong(int type, long val) + protected long DoOutInOp(int type, long val = 0) { return UU.TargetInLongOutLong(_target, type, val); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs index d6b1d054b74b8..3e72147b48480 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs @@ -273,7 +273,7 @@ public Task CancelAsync(string name) /** */ public void CancelAll() { - DoOutOp(OpCancelAll); + DoOutInOp(OpCancelAll); } /** */ @@ -390,7 +390,7 @@ public ICollection GetServices(string name) { return DoOutInOp(OpInvokeMethod, writer => ServiceProxySerializer.WriteProxyMethod(writer, method, args, platform), - stream => ServiceProxySerializer.ReadInvocationResult(stream, Marshaller, _keepBinary), proxy.Target); + (stream, res) => ServiceProxySerializer.ReadInvocationResult(stream, Marshaller, _keepBinary), proxy.Target); } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs index 796044d8d902f..81352e9bcae11 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs @@ -156,7 +156,7 @@ public ITransactionMetrics GetMetrics() /** */ public void ResetMetrics() { - DoOutOp(OpResetMetrics); + DoOutInOp(OpResetMetrics); } /// @@ -166,7 +166,7 @@ public void ResetMetrics() /// Final transaction state. internal TransactionState TxCommit(TransactionImpl tx) { - return (TransactionState) DoOutInOpLong(OpCommit, tx.Id); + return (TransactionState) DoOutInOp(OpCommit, tx.Id); } /// @@ -176,7 +176,7 @@ internal TransactionState TxCommit(TransactionImpl tx) /// Final transaction state. internal TransactionState TxRollback(TransactionImpl tx) { - return (TransactionState) DoOutInOpLong(OpRollback, tx.Id); + return (TransactionState) DoOutInOp(OpRollback, tx.Id); } /// @@ -186,7 +186,7 @@ internal TransactionState TxRollback(TransactionImpl tx) /// Final transaction state. internal int TxClose(TransactionImpl tx) { - return (int) DoOutInOpLong(OpClose, tx.Id); + return (int) DoOutInOp(OpClose, tx.Id); } /// @@ -196,7 +196,7 @@ internal int TxClose(TransactionImpl tx) /// Transaction current state. internal TransactionState TxState(TransactionImpl tx) { - return (TransactionState) DoOutInOpLong(OpState, tx.Id); + return (TransactionState) DoOutInOp(OpState, tx.Id); } /// @@ -206,7 +206,7 @@ internal TransactionState TxState(TransactionImpl tx) /// true if the flag was set. internal bool TxSetRollbackOnly(TransactionImpl tx) { - return DoOutInOpLong(OpSetRollbackOnly, tx.Id) == True; + return DoOutInOp(OpSetRollbackOnly, tx.Id) == True; } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs index 8de82ee04a09c..296ca38bb4305 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs @@ -124,19 +124,12 @@ internal static unsafe class IgniteJniNativeMethods long outMemPtr); [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInStreamOutObject")] - public static extern void* TargetInStreanOutObject(void* ctx, void* target, int opType, long memPtr); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInObjectStreamOutStream")] - public static extern void TargetInObjectStreamOutStream(void* ctx, void* target, int opType, - void* arg, long inMemPtr, long outMemPtr); + public static extern void* TargetInStreamOutObject(void* ctx, void* target, int opType, long memPtr); [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInObjectStreamOutObjectStream")] public static extern void* TargetInObjectStreamOutObjectStream(void* ctx, void* target, int opType, void* arg, long inMemPtr, long outMemPtr); - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetOutLong")] - public static extern long TargetOutLong(void* ctx, void* target, int opType); - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetOutStream")] public static extern void TargetOutStream(void* ctx, void* target, int opType, long memPtr); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs index 4722d0b17b92f..65e22965931ec 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs @@ -382,16 +382,11 @@ internal static void TargetInStreamOutStream(IUnmanagedTarget target, int opType internal static IUnmanagedTarget TargetInStreamOutObject(IUnmanagedTarget target, int opType, long inMemPtr) { - void* res = JNI.TargetInStreanOutObject(target.Context, target.Target, opType, inMemPtr); + void* res = JNI.TargetInStreamOutObject(target.Context, target.Target, opType, inMemPtr); return target.ChangeTarget(res); } - internal static void TargetInObjectStreamOutStream(IUnmanagedTarget target, int opType, void* arg, long inMemPtr, long outMemPtr) - { - JNI.TargetInObjectStreamOutStream(target.Context, target.Target, opType, arg, inMemPtr, outMemPtr); - } - internal static IUnmanagedTarget TargetInObjectStreamOutObjectStream(IUnmanagedTarget target, int opType, void* arg, long inMemPtr, long outMemPtr) { void* res = JNI.TargetInObjectStreamOutObjectStream(target.Context, target.Target, opType, arg, inMemPtr, outMemPtr); @@ -402,11 +397,6 @@ internal static IUnmanagedTarget TargetInObjectStreamOutObjectStream(IUnmanagedT return target.ChangeTarget(res); } - internal static long TargetOutLong(IUnmanagedTarget target, int opType) - { - return JNI.TargetOutLong(target.Context, target.Target, opType); - } - internal static void TargetOutStream(IUnmanagedTarget target, int opType, long memPtr) { JNI.TargetOutStream(target.Context, target.Target, opType, memPtr); From 66c76d1f30f024b58db8cab07ba9e7d429f596f8 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Tue, 18 Oct 2016 20:45:06 +0500 Subject: [PATCH 253/487] IGNITE-2355 Fixed the test HadoopClientProtocolMultipleServersSelfTest. Clear connection poll after the test, cosmetic. --- ...ClientProtocolMultipleServersSelfTest.java | 99 +++++++++---------- 1 file changed, 44 insertions(+), 55 deletions(-) diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java index 0e51938c12f29..51b61642ce237 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java @@ -23,8 +23,8 @@ import java.util.ArrayList; import java.util.Collection; import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; @@ -44,6 +44,8 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.mapreduce.IgniteHadoopClientProtocolProvider; import org.apache.ignite.igfs.IgfsPath; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.client.GridClient; import org.apache.ignite.internal.client.GridServerUnreachableException; import org.apache.ignite.internal.processors.hadoop.impl.HadoopAbstractSelfTest; import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils; @@ -79,6 +81,14 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS @Override protected void afterTest() throws Exception { stopAllGrids(); + ConcurrentHashMap> cliMap = + GridTestUtils.getFieldValue(IgniteHadoopClientProtocolProvider.class, "cliMap"); + + for(IgniteInternalFuture fut : cliMap.values()) + fut.get().close(); + + cliMap.clear(); + super.afterTest(); } @@ -114,7 +124,7 @@ private void beforeJob() throws Exception { * @param conf Hadoop configuration. * @throws Exception If failed. */ - public void checkJobSubmit(Configuration conf) throws Exception { + private void checkJobSubmit(Configuration conf) throws Exception { final Job job = Job.getInstance(conf); job.setJobName(JOB_NAME); @@ -144,22 +154,15 @@ public void checkJobSubmit(Configuration conf) throws Exception { */ @SuppressWarnings("ConstantConditions") public void testMultipleAddresses() throws Exception { - try { - restPort = REST_PORT; + restPort = REST_PORT; - startGrids(gridCount()); + startGrids(gridCount()); - beforeJob(); + beforeJob(); - U.sleep(5000); + U.sleep(5000); - checkJobSubmit(configMultipleAddrs(gridCount())); - } - finally { - FileSystem fs = FileSystem.get(configMultipleAddrs(gridCount())); - - fs.close(); - } + checkJobSubmit(configMultipleAddrs(gridCount())); } /** @@ -167,25 +170,18 @@ public void testMultipleAddresses() throws Exception { */ @SuppressWarnings({"ConstantConditions", "ThrowableResultOfMethodCallIgnored"}) public void testSingleAddress() throws Exception { - try { - // Don't use REST_PORT to test connection fails if the only this port is configured - restPort = REST_PORT + 1; - - startGrids(gridCount()); - - GridTestUtils.assertThrowsAnyCause(log, new Callable() { - @Override public Object call() throws Exception { - checkJobSubmit(configSingleAddress()); - return null; - } - }, - GridServerUnreachableException.class, "Failed to connect to any of the servers in list"); - } - finally { - FileSystem fs = FileSystem.get(configSingleAddress()); - - fs.close(); - } + // Don't use REST_PORT to test connection fails if the only this port is configured + restPort = REST_PORT + 1; + + startGrids(gridCount()); + + GridTestUtils.assertThrowsAnyCause(log, new Callable() { + @Override public Object call() throws Exception { + checkJobSubmit(configSingleAddress()); + return null; + } + }, + GridServerUnreachableException.class, "Failed to connect to any of the servers in list"); } /** @@ -193,28 +189,21 @@ public void testSingleAddress() throws Exception { */ @SuppressWarnings("ConstantConditions") public void testMixedAddrs() throws Exception { - try { - restPort = REST_PORT; + restPort = REST_PORT; - startGrids(gridCount()); + startGrids(gridCount()); - beforeJob(); + beforeJob(); - stopGrid(1); + stopGrid(1); - U.sleep(5000); + U.sleep(5000); - checkJobSubmit(configMixed()); + checkJobSubmit(configMixed()); - startGrid(1); + startGrid(1); - awaitPartitionMapExchange(); - } - finally { - FileSystem fs = FileSystem.get(configMixed()); - - fs.close(); - } + awaitPartitionMapExchange(); } /** @@ -234,19 +223,19 @@ private Configuration configSingleAddress() { } /** - * @param serversCnt Count ov servers. + * @param srvsCnt Count ov servers. * @return Configuration. */ - private Configuration configMultipleAddrs(int serversCnt) { + private Configuration configMultipleAddrs(int srvsCnt) { Configuration conf = HadoopUtils.safeCreateConfiguration(); setupFileSystems(conf); conf.set(MRConfig.FRAMEWORK_NAME, IgniteHadoopClientProtocolProvider.FRAMEWORK_NAME); - Collection addrs = new ArrayList<>(serversCnt); + Collection addrs = new ArrayList<>(srvsCnt); - for (int i = 0; i < serversCnt; ++i) + for (int i = 0; i < srvsCnt; ++i) addrs.add("127.0.0.1:" + Integer.toString(REST_PORT + i)); conf.set(MRConfig.MASTER_ADDRESS, F.concat(addrs, ",")); @@ -305,18 +294,18 @@ public static class TestReducer extends Reducer Date: Wed, 19 Oct 2016 13:06:42 +0300 Subject: [PATCH 254/487] IGNITE-3705: Fixed compiliation warnings. This closes #1169. --- modules/platforms/cpp/binary/project/vs/binary.vcxproj | 2 -- modules/platforms/cpp/common/project/vs/common.vcxproj | 1 - modules/platforms/cpp/core/project/vs/core.vcxproj | 2 -- modules/platforms/cpp/ignite/project/vs/ignite.vcxproj | 4 ++-- modules/platforms/cpp/jni/project/vs/jni.vcxproj | 1 - modules/platforms/cpp/odbc-test/src/queries_test.cpp | 4 ++-- modules/platforms/cpp/odbc/project/vs/odbc.vcxproj | 4 ++-- 7 files changed, 6 insertions(+), 12 deletions(-) diff --git a/modules/platforms/cpp/binary/project/vs/binary.vcxproj b/modules/platforms/cpp/binary/project/vs/binary.vcxproj index 212c5548a16f9..a29b361f0371d 100644 --- a/modules/platforms/cpp/binary/project/vs/binary.vcxproj +++ b/modules/platforms/cpp/binary/project/vs/binary.vcxproj @@ -140,7 +140,6 @@ Full true true - false $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\include AnySuitable Speed @@ -162,7 +161,6 @@ MaxSpeed true true - false $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\include AnySuitable Speed diff --git a/modules/platforms/cpp/common/project/vs/common.vcxproj b/modules/platforms/cpp/common/project/vs/common.vcxproj index e0abed237b38d..149fa48f60c5e 100644 --- a/modules/platforms/cpp/common/project/vs/common.vcxproj +++ b/modules/platforms/cpp/common/project/vs/common.vcxproj @@ -149,7 +149,6 @@ Full true true - false $(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;%(AdditionalIncludeDirectories) AnySuitable Speed diff --git a/modules/platforms/cpp/core/project/vs/core.vcxproj b/modules/platforms/cpp/core/project/vs/core.vcxproj index ca14a1d1ad7aa..6320323aae64a 100644 --- a/modules/platforms/cpp/core/project/vs/core.vcxproj +++ b/modules/platforms/cpp/core/project/vs/core.vcxproj @@ -149,7 +149,6 @@ Full true true - false $(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\jni\include;$(ProjectDir)\..\..\..\jni\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include AnySuitable Speed @@ -173,7 +172,6 @@ Full true true - false $(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\jni\include;$(ProjectDir)\..\..\..\jni\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include AnySuitable Speed diff --git a/modules/platforms/cpp/ignite/project/vs/ignite.vcxproj b/modules/platforms/cpp/ignite/project/vs/ignite.vcxproj index 6b6bed33821ad..9a912ef351de3 100644 --- a/modules/platforms/cpp/ignite/project/vs/ignite.vcxproj +++ b/modules/platforms/cpp/ignite/project/vs/ignite.vcxproj @@ -120,10 +120,10 @@ MaxSpeed true true - true $(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\jni\include;$(ProjectDir)\..\..\..\jni\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\..\core\include;$(ProjectDir)\..\..\..\core\os\win\include;$(BOOST_HOME) WIN32;NDEBUG;IGNITE_IMPL;_CRT_SECURE_NO_WARNINGS;IGNITE_FRIEND;_CRTDBG_MAP_ALLOC;%(PreprocessorDefinitions) Async + false Console @@ -138,10 +138,10 @@ MaxSpeed true true - true $(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\jni\include;$(ProjectDir)\..\..\..\jni\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\..\core\include;$(ProjectDir)\..\..\..\core\os\win\include;$(BOOST_HOME) NDEBUG;IGNITE_IMPL;_CRT_SECURE_NO_WARNINGS;IGNITE_FRIEND;_CRTDBG_MAP_ALLOC;%(PreprocessorDefinitions) Async + false Console diff --git a/modules/platforms/cpp/jni/project/vs/jni.vcxproj b/modules/platforms/cpp/jni/project/vs/jni.vcxproj index ac17ad387e6af..f080a685d0b42 100644 --- a/modules/platforms/cpp/jni/project/vs/jni.vcxproj +++ b/modules/platforms/cpp/jni/project/vs/jni.vcxproj @@ -161,7 +161,6 @@ Full true true - false $(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;%(AdditionalIncludeDirectories) AnySuitable Speed diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index b7ea967d4d692..eb6e153c71e85 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -584,12 +584,12 @@ BOOST_AUTO_TEST_CASE(TestDataAtExecution) SQLLEN len1 = SQL_DATA_AT_EXEC; SQLLEN len2 = SQL_LEN_DATA_AT_EXEC(static_cast(in1.strField.size())); - ret = SQLBindParam(stmt, 1, SQL_C_SLONG, SQL_INTEGER, 100, 100, &ind1, &len1); + ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_INTEGER, 100, 100, &ind1, sizeof(ind1), &len1); if (!SQL_SUCCEEDED(ret)) BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); - ret = SQLBindParam(stmt, 2, SQL_C_CHAR, SQL_VARCHAR, 100, 100, &ind2, &len2); + ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_CHAR, SQL_VARCHAR, 100, 100, &ind2, sizeof(ind2), &len2); if (!SQL_SUCCEEDED(ret)) BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj index 348a11aa06dec..4e4ff5b30145b 100644 --- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj +++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj @@ -122,9 +122,9 @@ MaxSpeed true true - false $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";%(PreprocessorDefinitions) + false true @@ -140,9 +140,9 @@ MaxSpeed true true - false $(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";%(PreprocessorDefinitions) + false true From 7ed2bb7e341701d052220a36a2b2f8f0a46fd644 Mon Sep 17 00:00:00 2001 From: AMRepo Date: Wed, 19 Oct 2016 18:33:59 +0300 Subject: [PATCH 255/487] IGNITE-3448 Support SQL queries with distinct aggregates added. This closes #3448. --- .../query/h2/sql/GridSqlQuerySplitter.java | 146 ++++++-- .../query/IgniteSqlSplitterSelfTest.java | 339 ++++++++++++++++++ 2 files changed, 447 insertions(+), 38 deletions(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java index 4d34ce86e97c8..62c54bd76df8b 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java @@ -33,6 +33,8 @@ import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.AVG; import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.CAST; import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.COUNT; +import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.MAX; +import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.MIN; import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.SUM; import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlPlaceholder.EMPTY; @@ -140,7 +142,8 @@ else if (expr instanceof GridSqlColumn) * @param igniteH2Indexing Indexing implementation. * @return Two step query. */ - public static GridCacheTwoStepQuery split(JdbcPreparedStatement stmt, Object[] params, boolean collocated, IgniteH2Indexing igniteH2Indexing) { + public static GridCacheTwoStepQuery split(JdbcPreparedStatement stmt, Object[] params, boolean collocated, + IgniteH2Indexing igniteH2Indexing) { if (params == null) params = GridCacheSqlQuery.EMPTY_PARAMS; @@ -169,10 +172,18 @@ public static GridCacheTwoStepQuery split(JdbcPreparedStatement stmt, Object[] p Set colNames = new HashSet<>(); - boolean aggregateFound = false; + boolean distinctAggregateFound = false; + + if (!collocated) { + for (int i = 0, len = mapExps.size(); i < len; i++) + distinctAggregateFound |= hasDistinctAggregates(mapExps.get(i)); + } + + boolean aggregateFound = distinctAggregateFound; for (int i = 0, len = mapExps.size(); i < len; i++) // Remember len because mapExps list can grow. - aggregateFound |= splitSelectExpression(mapExps, rdcExps, colNames, i, collocated, i == havingCol); + aggregateFound |= splitSelectExpression(mapExps, rdcExps, colNames, i, collocated, i == havingCol, + distinctAggregateFound); // Fill select expressions. mapQry.clearColumns(); @@ -190,9 +201,14 @@ public static GridCacheTwoStepQuery split(JdbcPreparedStatement stmt, Object[] p rdcQry.addColumn(column(((GridSqlAlias)mapExps.get(i)).alias()), false); // -- GROUP BY - if (mapQry.groupColumns() != null && !collocated) + if (mapQry.groupColumns() != null && !collocated) { rdcQry.groupColumns(mapQry.groupColumns()); + // Grouping with distinct aggregates cannot be performed on map phase + if (distinctAggregateFound) + mapQry.groupColumns(null); + } + // -- HAVING if (havingCol >= 0 && !collocated) { // TODO IGNITE-1140 - Find aggregate functions in HAVING clause or rewrite query to put all aggregates to SELECT clause. @@ -477,10 +493,11 @@ else if (el instanceof GridSqlSubquery) * @param idx Index. * @param collocated If it is a collocated query. * @param isHaving If it is a HAVING expression. + * @param hasDistinctAggregate If query has distinct aggregate expression. * @return {@code true} If aggregate was found. */ private static boolean splitSelectExpression(List mapSelect, List rdcSelect, - Set colNames, final int idx, boolean collocated, boolean isHaving) { + Set colNames, final int idx, boolean collocated, boolean isHaving, boolean hasDistinctAggregate) { GridSqlElement el = mapSelect.get(idx); GridSqlAlias alias = null; @@ -499,7 +516,7 @@ private static boolean splitSelectExpression(List mapSelect, Lis alias = alias(isHaving ? HAVING_COLUMN : columnName(idx), el); // We can update original alias here as well since it will be dropped from mapSelect. - splitAggregates(alias, 0, mapSelect, idx, true); + splitAggregates(alias, 0, mapSelect, idx, hasDistinctAggregate, true); set(rdcSelect, idx, alias); } @@ -553,11 +570,34 @@ private static boolean hasAggregates(GridSqlElement el) { return false; } + /** + * Lookup for distinct aggregates. + * Note, DISTINCT make no sense for MIN and MAX aggregates, so its will be ignored. + * + * @param el Expression. + * @return {@code true} If expression contains distinct aggregates. + */ + private static boolean hasDistinctAggregates(GridSqlElement el) { + if (el instanceof GridSqlAggregateFunction) { + GridSqlFunctionType type = ((GridSqlAggregateFunction)el).type(); + + return ((GridSqlAggregateFunction)el).distinct() && type != MIN && type != MAX; + } + + for (GridSqlElement child : el) { + if (hasDistinctAggregates(child)) + return true; + } + + return false; + } + /** * @param parentExpr Parent expression. * @param childIdx Child index to try to split. * @param mapSelect List of expressions in map SELECT clause. * @param exprIdx Index of the original expression in map SELECT clause. + * @param hasDistinctAggregate If query has distinct aggregate expression. * @param first If the first aggregate is already found in this expression. * @return {@code true} If the first aggregate is already found. */ @@ -566,17 +606,18 @@ private static boolean splitAggregates( final int childIdx, final List mapSelect, final int exprIdx, + boolean hasDistinctAggregate, boolean first) { GridSqlElement el = parentExpr.child(childIdx); if (el instanceof GridSqlAggregateFunction) { - splitAggregate(parentExpr, childIdx, mapSelect, exprIdx, first); + splitAggregate(parentExpr, childIdx, mapSelect, exprIdx, hasDistinctAggregate, first); return true; } for (int i = 0; i < el.size(); i++) { - if (splitAggregates(el, i, mapSelect, exprIdx, first)) + if (splitAggregates(el, i, mapSelect, exprIdx, hasDistinctAggregate, first)) first = false; } @@ -588,6 +629,7 @@ private static boolean splitAggregates( * @param aggIdx Index of the aggregate to split in this expression. * @param mapSelect List of expressions in map SELECT clause. * @param exprIdx Index of the original expression in map SELECT clause. + * @param hasDistinctAggregate If query has distinct aggregate expression. * @param first If this is the first aggregate found in this expression. */ private static void splitAggregate( @@ -595,6 +637,7 @@ private static void splitAggregate( int aggIdx, List mapSelect, int exprIdx, + boolean hasDistinctAggregate, boolean first ) { GridSqlAggregateFunction agg = parentExpr.child(aggIdx); @@ -612,53 +655,80 @@ private static void splitAggregate( else mapSelect.add(mapAggAlias); + /* Note Distinct aggregate can be performed only on reduce phase, so + if query contains distinct aggregate then other aggregates must be processed the same way. */ switch (agg.type()) { - case AVG: // SUM( AVG(CAST(x AS DOUBLE))*COUNT(x) )/SUM( COUNT(x) ). - //-- COUNT(x) map - GridSqlElement cntMapAgg = aggregate(agg.distinct(), COUNT) - .resultType(GridSqlType.BIGINT).addChild(agg.child()); + case AVG: // SUM( AVG(CAST(x AS DOUBLE))*COUNT(x) )/SUM( COUNT(x) ) or AVG(CAST( x AS DOUBLE)) + if (hasDistinctAggregate) /* and has no collocated group by */ { + mapAgg = agg.child(); + + rdcAgg = aggregate(agg.distinct(), agg.type()).resultType(GridSqlType.DOUBLE) + .addChild(function(CAST).resultType(GridSqlType.DOUBLE).addChild(column(mapAggAlias.alias()))); + } + else { + //-- COUNT(x) map + GridSqlElement cntMapAgg = aggregate(agg.distinct(), COUNT) + .resultType(GridSqlType.BIGINT).addChild(agg.child()); - // Add generated alias to COUNT(x). - // Using size as index since COUNT will be added as the last select element to the map query. - String cntMapAggAlias = columnName(mapSelect.size()); + // Add generated alias to COUNT(x). + // Using size as index since COUNT will be added as the last select element to the map query. + String cntMapAggAlias = columnName(mapSelect.size()); - cntMapAgg = alias(cntMapAggAlias, cntMapAgg); + cntMapAgg = alias(cntMapAggAlias, cntMapAgg); - mapSelect.add(cntMapAgg); + mapSelect.add(cntMapAgg); - //-- AVG(CAST(x AS DOUBLE)) map - mapAgg = aggregate(agg.distinct(), AVG).resultType(GridSqlType.DOUBLE).addChild( - function(CAST).resultType(GridSqlType.DOUBLE).addChild(agg.child())); + //-- AVG(CAST(x AS DOUBLE)) map + mapAgg = aggregate(agg.distinct(), AVG).resultType(GridSqlType.DOUBLE).addChild( + function(CAST).resultType(GridSqlType.DOUBLE).addChild(agg.child())); - //-- SUM( AVG(x)*COUNT(x) )/SUM( COUNT(x) ) reduce - GridSqlElement sumUpRdc = aggregate(false, SUM).addChild( - op(GridSqlOperationType.MULTIPLY, - column(mapAggAlias.alias()), - column(cntMapAggAlias))); + //-- SUM( AVG(x)*COUNT(x) )/SUM( COUNT(x) ) reduce + GridSqlElement sumUpRdc = aggregate(false, SUM).addChild( + op(GridSqlOperationType.MULTIPLY, + column(mapAggAlias.alias()), + column(cntMapAggAlias))); - GridSqlElement sumDownRdc = aggregate(false, SUM).addChild(column(cntMapAggAlias)); + GridSqlElement sumDownRdc = aggregate(false, SUM).addChild(column(cntMapAggAlias)); - rdcAgg = op(GridSqlOperationType.DIVIDE, sumUpRdc, sumDownRdc); + rdcAgg = op(GridSqlOperationType.DIVIDE, sumUpRdc, sumDownRdc); + } break; - case SUM: // SUM( SUM(x) ) - case MAX: // MAX( MAX(x) ) - case MIN: // MIN( MIN(x) ) - mapAgg = aggregate(agg.distinct(), agg.type()).resultType(agg.resultType()).addChild(agg.child()); - rdcAgg = aggregate(agg.distinct(), agg.type()).addChild(column(mapAggAlias.alias())); + case SUM: // SUM( SUM(x) ) or SUM(DISTINCT x) + case MAX: // MAX( MAX(x) ) or MAX(DISTINCT x) + case MIN: // MIN( MIN(x) ) or MIN(DISTINCT x) + if (hasDistinctAggregate) /* and has no collocated group by */ { + mapAgg = agg.child(); + + rdcAgg = aggregate(agg.distinct(), agg.type()).addChild(column(mapAggAlias.alias())); + } + else { + mapAgg = aggregate(agg.distinct(), agg.type()).resultType(agg.resultType()).addChild(agg.child()); + rdcAgg = aggregate(agg.distinct(), agg.type()).addChild(column(mapAggAlias.alias())); + } break; case COUNT_ALL: // CAST(SUM( COUNT(*) ) AS BIGINT) - case COUNT: // CAST(SUM( COUNT(x) ) AS BIGINT) - mapAgg = aggregate(agg.distinct(), agg.type()).resultType(GridSqlType.BIGINT); + case COUNT: // CAST(SUM( COUNT(x) ) AS BIGINT) or CAST(COUNT(DISTINCT x) AS BIGINT) + if (hasDistinctAggregate) /* and has no collocated group by */ { + assert agg.type() == COUNT; - if (agg.type() == COUNT) - mapAgg.addChild(agg.child()); + mapAgg = agg.child(); - rdcAgg = aggregate(false, SUM).addChild(column(mapAggAlias.alias())); - rdcAgg = function(CAST).resultType(GridSqlType.BIGINT).addChild(rdcAgg); + rdcAgg = aggregate(agg.distinct(), agg.type()).resultType(GridSqlType.BIGINT) + .addChild(column(mapAggAlias.alias())); + } + else { + mapAgg = aggregate(agg.distinct(), agg.type()).resultType(GridSqlType.BIGINT); + + if (agg.type() == COUNT) + mapAgg.addChild(agg.child()); + + rdcAgg = aggregate(false, SUM).addChild(column(mapAggAlias.alias())); + rdcAgg = function(CAST).resultType(GridSqlType.BIGINT).addChild(rdcAgg); + } break; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java index 64be936559d15..56658df9b487f 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java @@ -25,11 +25,15 @@ import java.util.Map; import java.util.Random; import java.util.concurrent.atomic.AtomicLong; +import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.util.GridRandom; @@ -98,6 +102,8 @@ public void testOffsetLimit() throws Exception { Integer.class, Integer.class)); try { + awaitPartitionMapExchange(); + List res = new ArrayList<>(); Random rnd = new GridRandom(); @@ -340,6 +346,328 @@ public void testImplicitJoinConditionGeneration() { } } + /** @throws Exception if failed. */ + public void testDistributedAggregates() throws Exception { + final String cacheName = "ints"; + + IgniteCache cache = ignite(0).getOrCreateCache(cacheConfig(cacheName, true, + Integer.class, Value.class)); + + AffinityKeyGenerator node0KeyGen = new AffinityKeyGenerator(ignite(0), cacheName); + AffinityKeyGenerator node1KeyGen = new AffinityKeyGenerator(ignite(1), cacheName); + AffinityKeyGenerator node2KeyGen = new AffinityKeyGenerator(ignite(2), cacheName); + + try { + awaitPartitionMapExchange(); + + cache.put(node0KeyGen.next(), new Value(1, 3)); + cache.put(node1KeyGen.next(), new Value(1, 3)); + cache.put(node2KeyGen.next(), new Value(1, 3)); + + cache.put(node0KeyGen.next(), new Value(2, 1)); + cache.put(node1KeyGen.next(), new Value(2, 2)); + cache.put(node2KeyGen.next(), new Value(2, 3)); + + cache.put(node0KeyGen.next(), new Value(3, 1)); + cache.put(node0KeyGen.next(), new Value(3, 1)); + cache.put(node0KeyGen.next(), new Value(3, 2)); + cache.put(node1KeyGen.next(), new Value(3, 1)); + cache.put(node1KeyGen.next(), new Value(3, 2)); + cache.put(node2KeyGen.next(), new Value(3, 2)); + + cache.put(node0KeyGen.next(), new Value(4, 2)); + cache.put(node1KeyGen.next(), new Value(5, 2)); + cache.put(node2KeyGen.next(), new Value(6, 2)); + + checkSimpleQueryWithAggr(cache); + checkSimpleQueryWithDistinctAggr(cache); + + checkQueryWithGroupsAndAggrs(cache); + checkQueryWithGroupsAndDistinctAggr(cache); + + checkSimpleQueryWithAggrMixed(cache); + checkQueryWithGroupsAndAggrMixed(cache); + } + finally { + cache.destroy(); + } + } + + /** @throws Exception if failed. */ + public void testCollocatedAggregates() throws Exception { + final String cacheName = "ints"; + + IgniteCache cache = ignite(0).getOrCreateCache(cacheConfig(cacheName, true, + Integer.class, Value.class)); + + AffinityKeyGenerator node0KeyGen = new AffinityKeyGenerator(ignite(0), cacheName); + AffinityKeyGenerator node1KeyGen = new AffinityKeyGenerator(ignite(1), cacheName); + AffinityKeyGenerator node2KeyGen = new AffinityKeyGenerator(ignite(2), cacheName); + + try { + awaitPartitionMapExchange(); + + cache.put(node0KeyGen.next(), new Value(1, 3)); + cache.put(node0KeyGen.next(), new Value(1, 3)); + cache.put(node0KeyGen.next(), new Value(1, 3)); + + cache.put(node1KeyGen.next(), new Value(2, 1)); + cache.put(node1KeyGen.next(), new Value(2, 2)); + cache.put(node1KeyGen.next(), new Value(2, 3)); + + cache.put(node2KeyGen.next(), new Value(3, 1)); + cache.put(node2KeyGen.next(), new Value(3, 1)); + cache.put(node2KeyGen.next(), new Value(3, 2)); + cache.put(node2KeyGen.next(), new Value(3, 1)); + cache.put(node2KeyGen.next(), new Value(3, 2)); + cache.put(node2KeyGen.next(), new Value(3, 2)); + + cache.put(node0KeyGen.next(), new Value(4, 2)); + cache.put(node1KeyGen.next(), new Value(5, 2)); + cache.put(node2KeyGen.next(), new Value(6, 2)); + + checkQueryWithGroupsAndAggrs(cache); + checkQueryWithGroupsAndDistinctAggr(cache); + checkQueryWithGroupsAndAggrMixed(cache); + } + finally { + cache.destroy(); + } + } + + /** Simple query with aggregates */ + private void checkSimpleQueryWithAggr(IgniteCache cache) { + try (QueryCursor> qry = cache.query(new SqlFieldsQuery( + "SELECT count(fst), sum(snd), avg(snd), min(snd), max(snd) FROM Value"))) { + List> result = qry.getAll(); + + assertEquals(1, result.size()); + + List row = result.get(0); + + assertEquals("count", 15L, ((Number)row.get(0)).longValue()); + assertEquals("sum", 30L, ((Number)row.get(1)).longValue()); + assertEquals("avg", 2.0d, ((Number)row.get(2)).doubleValue(), 0.001); + assertEquals("min", 1, ((Integer)row.get(3)).intValue()); + assertEquals("max", 3, ((Integer)row.get(4)).intValue()); + } + } + + /** Simple query with distinct aggregates */ + private void checkSimpleQueryWithDistinctAggr(IgniteCache cache) { + try (QueryCursor> qry = cache.query(new SqlFieldsQuery( + "SELECT count(distinct fst), sum(distinct snd), avg(distinct snd), min(distinct snd), max(distinct snd) " + + "FROM Value"))) { + List> result = qry.getAll(); + + assertEquals(1, result.size()); + + List row = result.get(0); + + assertEquals("count distinct", 6L, ((Number)row.get(0)).longValue()); + assertEquals("sum distinct", 6L, ((Number)row.get(1)).longValue()); + assertEquals("avg distinct", 2.0d, ((Number)row.get(2)).doubleValue(), 0.001); + assertEquals("min distinct", 1, ((Integer)row.get(3)).intValue()); + assertEquals("max distinct", 3, ((Integer)row.get(4)).intValue()); + } + } + + /** Simple query with distinct aggregates */ + private void checkSimpleQueryWithAggrMixed(IgniteCache cache) { + try (QueryCursor> qry = cache.query(new SqlFieldsQuery( + "SELECT count(fst), sum(snd), avg(snd), min(snd), max(snd)," + + "count(distinct fst), sum(distinct snd), avg(distinct snd), min(distinct snd), max(distinct snd) " + + "FROM Value"))) { + List> result = qry.getAll(); + + assertEquals(1, result.size()); + + List row = result.get(0); + + assertEquals("count", 15L, ((Number)row.get(0)).longValue()); + assertEquals("sum", 30L, ((Number)row.get(1)).longValue()); + assertEquals("avg", 2.0d, ((Number)row.get(2)).doubleValue(), 0.001); + assertEquals("min", 1, ((Integer)row.get(3)).intValue()); + assertEquals("max", 3, ((Integer)row.get(4)).intValue()); + assertEquals("count distinct", 6L, ((Number)row.get(5)).longValue()); + assertEquals("sum distinct", 6L, ((Number)row.get(6)).longValue()); + assertEquals("avg distinct", 2.0d, ((Number)row.get(7)).doubleValue(), 0.001); + assertEquals("min distinct", 1, ((Integer)row.get(8)).intValue()); + assertEquals("max distinct", 3, ((Integer)row.get(9)).intValue()); + } + } + + /** Query with aggregates and groups */ + private void checkQueryWithGroupsAndAggrs(IgniteCache cache) { + try (QueryCursor> qry = cache.query(new SqlFieldsQuery( + "SELECT fst, count(snd), sum(snd), avg(snd), min(snd), max(snd) FROM Value GROUP BY fst ORDER BY fst"))) { + List> result = qry.getAll(); + + assertEquals(6, result.size()); + + List row = result.get(0); + assertEquals("fst", 1, ((Number)row.get(0)).intValue()); + assertEquals("count", 3L, ((Number)row.get(1)).longValue()); + assertEquals("sum", 9L, ((Number)row.get(2)).longValue()); + assertEquals("avg", 3.0d, ((Number)row.get(3)).doubleValue(), 0.001); + assertEquals("min", 3, ((Integer)row.get(4)).intValue()); + assertEquals("max", 3, ((Integer)row.get(5)).intValue()); + + row = result.get(1); + assertEquals("fst", 2, ((Number)row.get(0)).intValue()); + assertEquals("count", 3L, ((Number)row.get(1)).longValue()); + assertEquals("sum", 6L, ((Number)row.get(2)).longValue()); + assertEquals("avg", 2.0d, ((Number)row.get(3)).doubleValue(), 0.001); + assertEquals("min", 1, ((Integer)row.get(4)).intValue()); + assertEquals("max", 3, ((Integer)row.get(5)).intValue()); + + row = result.get(2); + assertEquals("fst", 3, ((Number)row.get(0)).intValue()); + assertEquals("count", 6L, ((Number)row.get(1)).longValue()); + assertEquals("sum", 9L, ((Number)row.get(2)).longValue()); + assertEquals("avg", 1.5d, ((Number)row.get(3)).doubleValue(), 0.001); + assertEquals("min", 1, ((Integer)row.get(4)).intValue()); + assertEquals("max", 2, ((Integer)row.get(5)).intValue()); + } + } + + /** Query with distinct aggregates and groups */ + private void checkQueryWithGroupsAndDistinctAggr(IgniteCache cache) { + try (QueryCursor> qry = cache.query(new SqlFieldsQuery( + "SELECT count(distinct snd), sum(distinct snd), avg(distinct snd), min(distinct snd), max(distinct snd) " + + "FROM Value GROUP BY fst"))) { + List> result = qry.getAll(); + + assertEquals(6, result.size()); + + List row = result.get(0); + assertEquals("count distinct", 1L, ((Number)row.get(0)).longValue()); + assertEquals("sum distinct", 3L, ((Number)row.get(1)).longValue()); + assertEquals("avg distinct", 3.0d, ((Number)row.get(2)).doubleValue(), 0.001); + assertEquals("min distinct", 3, ((Integer)row.get(3)).intValue()); + assertEquals("max distinct", 3, ((Integer)row.get(4)).intValue()); + + row = result.get(1); + assertEquals("count distinct", 3L, ((Number)row.get(0)).longValue()); + assertEquals("sum distinct", 6L, ((Number)row.get(1)).longValue()); + assertEquals("avg distinct", 2.0d, ((Number)row.get(2)).doubleValue(), 0.001); + assertEquals("min distinct", 1, ((Integer)row.get(3)).intValue()); + assertEquals("max distinct", 3, ((Integer)row.get(4)).intValue()); + + row = result.get(2); + assertEquals("count distinct", 2L, ((Number)row.get(0)).longValue()); + assertEquals("sum distinct", 3L, ((Number)row.get(1)).longValue()); + assertEquals("avg distinct", 1.5d, ((Number)row.get(2)).doubleValue(), 0.001); + assertEquals("min distinct", 1, ((Integer)row.get(3)).intValue()); + assertEquals("max distinct", 2, ((Integer)row.get(4)).intValue()); + } + } + + /** Query with distinct aggregates and groups */ + private void checkQueryWithGroupsAndAggrMixed(IgniteCache cache) { + try (QueryCursor> qry = cache.query(new SqlFieldsQuery( + "SELECT fst, count(snd), sum(snd), avg(snd), min(snd), max(snd)," + + "count(distinct snd), sum(distinct snd), avg(distinct snd), min(distinct snd), max(distinct snd) " + + "FROM Value GROUP BY fst"))) { + List> result = qry.getAll(); + + assertEquals(6, result.size()); + + List row = result.get(0); + assertEquals("fst", 1, ((Number)row.get(0)).intValue()); + assertEquals("count", 3L, ((Number)row.get(1)).longValue()); + assertEquals("sum", 9L, ((Number)row.get(2)).longValue()); + assertEquals("avg", 3.0d, ((Number)row.get(3)).doubleValue(), 0.001); + assertEquals("min", 3, ((Integer)row.get(4)).intValue()); + assertEquals("max", 3, ((Integer)row.get(5)).intValue()); + assertEquals("count distinct", 1L, ((Number)row.get(6)).longValue()); + assertEquals("sum distinct", 3L, ((Number)row.get(7)).longValue()); + assertEquals("avg distinct", 3.0d, ((Number)row.get(8)).doubleValue(), 0.001); + assertEquals("min distinct", 3, ((Integer)row.get(9)).intValue()); + assertEquals("max distinct", 3, ((Integer)row.get(10)).intValue()); + + row = result.get(1); + assertEquals("fst", 2, ((Number)row.get(0)).intValue()); + assertEquals("count", 3L, ((Number)row.get(1)).longValue()); + assertEquals("sum", 6L, ((Number)row.get(2)).longValue()); + assertEquals("avg", 2.0d, ((Number)row.get(3)).doubleValue(), 0.001); + assertEquals("min", 1, ((Integer)row.get(4)).intValue()); + assertEquals("max", 3, ((Integer)row.get(5)).intValue()); + assertEquals("count distinct", 3L, ((Number)row.get(6)).longValue()); + assertEquals("sum distinct", 6L, ((Number)row.get(7)).longValue()); + assertEquals("avg distinct", 2.0d, ((Number)row.get(8)).doubleValue(), 0.001); + assertEquals("min distinct", 1, ((Integer)row.get(9)).intValue()); + assertEquals("max distinct", 3, ((Integer)row.get(10)).intValue()); + + row = result.get(2); + assertEquals("fst", 3, ((Number)row.get(0)).intValue()); + assertEquals("count", 6L, ((Number)row.get(1)).longValue()); + assertEquals("sum", 9L, ((Number)row.get(2)).longValue()); + assertEquals("avg", 1.5d, ((Number)row.get(3)).doubleValue(), 0.001); + assertEquals("min", 1, ((Integer)row.get(4)).intValue()); + assertEquals("max", 2, ((Integer)row.get(5)).intValue()); + assertEquals("count distinct", 2L, ((Number)row.get(6)).longValue()); + assertEquals("sum distinct", 3L, ((Number)row.get(7)).longValue()); + assertEquals("avg distinct", 1.5d, ((Number)row.get(8)).doubleValue(), 0.001); + assertEquals("min distinct", 1, ((Integer)row.get(9)).intValue()); + assertEquals("max distinct", 2, ((Integer)row.get(10)).intValue()); + } + } + + /** */ + private static class Value { + /** */ + @QuerySqlField + private final Integer fst; + + /** */ + @QuerySqlField + private final Integer snd; + + /** Constructor */ + public Value(Integer fst, Integer snd) { + this.fst = fst; + this.snd = snd; + } + } + + /** + * + */ + private static class AffinityKeyGenerator { + /** */ + private final Affinity affinity; + + /** */ + private final ClusterNode node; + + /** */ + private int start = 0; + + /** Constructor */ + AffinityKeyGenerator(Ignite node, String cacheName) { + this.affinity = node.affinity(cacheName); + this.node = node.cluster().localNode(); + } + + /** */ + public Integer next() { + int key = start; + + while (start < Integer.MAX_VALUE) { + if (affinity.isPrimary(node, key)) { + start = key + 1; + + return key; + } + + key++; + } + + throw new IllegalStateException("Can't find next key"); + } + } + /** * */ @@ -391,9 +719,11 @@ public static class Department { * Test value. */ private static class GroupIndexTestValue implements Serializable { + /** */ @QuerySqlField(orderedGroups = @QuerySqlField.Group(name = "grpIdx", order = 0)) private int a; + /** */ @QuerySqlField(orderedGroups = @QuerySqlField.Group(name = "grpIdx", order = 1)) private int b; @@ -408,22 +738,31 @@ private GroupIndexTestValue(int a, int b) { } private static class User implements Serializable { + /** */ @QuerySqlField private int id; } + /** */ private static class UserOrder implements Serializable { + /** */ @QuerySqlField private int id; + /** */ @QuerySqlField private int userId; } + /** + * + */ private static class OrderGood implements Serializable { + /** */ @QuerySqlField private int orderId; + /** */ @QuerySqlField private int goodId; } From 551a4dfae6169a07a5e28f9b266f90311f3216b7 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Fri, 21 Oct 2016 15:25:57 +0500 Subject: [PATCH 256/487] IGNITE-2355 Fixed the test HadoopClientProtocolMultipleServersSelfTest. Clear connection poll before and after the test --- ...ClientProtocolMultipleServersSelfTest.java | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java index 51b61642ce237..d6e074310e9a5 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java @@ -40,6 +40,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteFileSystem; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.mapreduce.IgniteHadoopClientProtocolProvider; @@ -81,6 +82,22 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS @Override protected void afterTest() throws Exception { stopAllGrids(); + clearConnectionMap(); + + super.afterTest(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + clearConnectionMap(); + } + + /** + * @throws IgniteCheckedException If failed. + */ + private void clearConnectionMap() throws IgniteCheckedException { ConcurrentHashMap> cliMap = GridTestUtils.getFieldValue(IgniteHadoopClientProtocolProvider.class, "cliMap"); @@ -88,8 +105,6 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS fut.get().close(); cliMap.clear(); - - super.afterTest(); } /** {@inheritDoc} */ From 958f16bb0ae0ebf5307029b9fafc2cd6f0df55c5 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 24 Oct 2016 11:26:35 +0300 Subject: [PATCH 257/487] IGNITE-4041: Ported pool processor to 1.7.3 version. --- .../ignite/internal/GridKernalContext.java | 8 +++++++ .../internal/GridKernalContextImpl.java | 12 ++++++++++ .../apache/ignite/internal/IgniteKernal.java | 3 +++ .../apache/ignite/internal/IgnitionEx.java | 22 ++++++++++++++++++- .../processors/pool/PoolProcessor.java | 5 +++++ .../junits/GridTestKernalContext.java | 1 + 6 files changed, 50 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java index e608af2a09767..ae292232e0930 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java @@ -70,6 +70,7 @@ import org.apache.ignite.plugin.PluginNotFoundException; import org.apache.ignite.plugin.PluginProvider; import org.apache.ignite.thread.IgniteStripedThreadPoolExecutor; +import org.jetbrains.annotations.Nullable; /** * @@ -547,6 +548,13 @@ public interface GridKernalContext extends Iterable { */ public ExecutorService getAffinityExecutorService(); + /** + * Get indexing executor service. + * + * @return Indexing executor service. + */ + @Nullable public ExecutorService getIndexingExecutorService(); + /** * Gets exception registry. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java index ddef345ffd151..94c64481d4ce3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java @@ -318,6 +318,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable @GridToStringExclude protected ExecutorService affExecSvc; + /** */ + @GridToStringExclude + protected ExecutorService idxExecSvc; + /** */ @GridToStringExclude protected IgniteStripedThreadPoolExecutor callbackExecSvc; @@ -382,6 +386,7 @@ public GridKernalContextImpl() { * @param igfsExecSvc IGFS executor service. * @param restExecSvc REST executor service. * @param affExecSvc Affinity executor service. + * @param idxExecSvc Indexing executor service. * @param plugins Plugin providers. * @throws IgniteCheckedException In case of error. */ @@ -400,6 +405,7 @@ protected GridKernalContextImpl( ExecutorService igfsExecSvc, ExecutorService restExecSvc, ExecutorService affExecSvc, + @Nullable ExecutorService idxExecSvc, IgniteStripedThreadPoolExecutor callbackExecSvc, List plugins) throws IgniteCheckedException { assert grid != null; @@ -418,6 +424,7 @@ protected GridKernalContextImpl( this.igfsExecSvc = igfsExecSvc; this.restExecSvc = restExecSvc; this.affExecSvc = affExecSvc; + this.idxExecSvc = idxExecSvc; this.callbackExecSvc = callbackExecSvc; String workDir = U.workDirectory(cfg.getWorkDirectory(), cfg.getIgniteHome()); @@ -965,6 +972,11 @@ protected Object readResolve() throws ObjectStreamException { return affExecSvc; } + /** {@inheritDoc} */ + @Override @Nullable public ExecutorService getIndexingExecutorService() { + return idxExecSvc; + } + /** {@inheritDoc} */ @Override public IgniteExceptionRegistry exceptionRegistry() { return IgniteExceptionRegistry.get(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 02f16af35a8f4..9f1f01ec32f29 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -668,6 +668,7 @@ private void notifyLifecycleBeansEx(LifecycleEventType evt) { * @param igfsExecSvc IGFS executor service. * @param restExecSvc Reset executor service. * @param affExecSvc Affinity executor service. + * @param idxExecSvc Indexing executor service. * @param errHnd Error handler to use for notification about startup problems. * @throws IgniteCheckedException Thrown in case of any errors. */ @@ -682,6 +683,7 @@ public void start(final IgniteConfiguration cfg, ExecutorService igfsExecSvc, ExecutorService restExecSvc, ExecutorService affExecSvc, + @Nullable ExecutorService idxExecSvc, IgniteStripedThreadPoolExecutor callbackExecSvc, GridAbsClosure errHnd) throws IgniteCheckedException @@ -788,6 +790,7 @@ public void start(final IgniteConfiguration cfg, igfsExecSvc, restExecSvc, affExecSvc, + idxExecSvc, callbackExecSvc, plugins); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index a6860b3a4ccac..5b2c3fc53faea 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -1480,6 +1480,9 @@ private static final class IgniteNamedInstance { /** Affinity executor service. */ private ThreadPoolExecutor affExecSvc; + /** Indexing pool. */ + private ThreadPoolExecutor idxExecSvc; + /** Continuous query executor service. */ private IgniteStripedThreadPoolExecutor callbackExecSvc; @@ -1747,6 +1750,19 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { affExecSvc.allowCoreThreadTimeOut(true); + if (IgniteComponentType.INDEXING.inClassPath()) { + int cpus = Runtime.getRuntime().availableProcessors(); + + idxExecSvc = new IgniteThreadPoolExecutor( + "idx", + cfg.getGridName(), + cpus, + cpus * 2, + 3000L, + new LinkedBlockingQueue(1000) + ); + } + // Register Ignite MBean for current grid instance. registerFactoryMbean(myCfg.getMBeanServer()); @@ -1759,7 +1775,7 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { grid = grid0; grid0.start(myCfg, utilityCacheExecSvc, marshCacheExecSvc, execSvc, sysExecSvc, p2pExecSvc, mgmtExecSvc, - igfsExecSvc, restExecSvc, affExecSvc, callbackExecSvc, + igfsExecSvc, restExecSvc, affExecSvc, idxExecSvc, callbackExecSvc, new CA() { @Override public void apply() { startLatch.countDown(); @@ -2398,6 +2414,10 @@ private void stopExecutors0(IgniteLogger log) { affExecSvc = null; + U.shutdownNow(getClass(), idxExecSvc, log); + + idxExecSvc = null; + U.shutdownNow(getClass(), callbackExecSvc, log); callbackExecSvc = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java index 41e805ef9c515..59e5e7dffb7e9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java @@ -108,6 +108,11 @@ public Executor poolForPolicy(byte plc) throws IgniteCheckedException { case GridIoPolicy.AFFINITY_POOL: return ctx.getAffinityExecutorService(); + case GridIoPolicy.IDX_POOL: + assert ctx.getIndexingExecutorService() != null : "Indexing pool is not configured."; + + return ctx.getIndexingExecutorService(); + case GridIoPolicy.UTILITY_CACHE_POOL: assert ctx.utilityCachePool() != null : "Utility cache pool is not configured."; diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java index cba67e0d06005..f9e2ff4e709df 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java @@ -62,6 +62,7 @@ public GridTestKernalContext(IgniteLogger log, IgniteConfiguration cfg) throws I null, null, null, + null, U.allPluginProviders()); GridTestUtils.setFieldValue(grid(), "cfg", config()); From b9bb1f6bc02cb3054b1e9b9bfbd34b886db77624 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 24 Oct 2016 11:50:51 +0300 Subject: [PATCH 258/487] Merge commit '2ab094e0' into ignite-1.7.3. --- .../ignite/testframework/junits/GridTestKernalContext.java | 1 + 1 file changed, 1 insertion(+) diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java index 1d72d4640247e..8cb32b6d9afd4 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java @@ -63,6 +63,7 @@ public GridTestKernalContext(IgniteLogger log, IgniteConfiguration cfg) throws I null, null, null, + null, U.allPluginProviders()); GridTestUtils.setFieldValue(grid(), "cfg", config()); From f97170b723e8ced39c7acae19fadb0039e7cafb7 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 24 Oct 2016 12:21:00 +0300 Subject: [PATCH 259/487] merge eaf8ae24: IGNITE-4034 Get rid of specialized methods in platform targets --- .../ignite/internal/processors/platform/cache/PlatformCache.java | 1 - 1 file changed, 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index 18a7c36cea125..6ce900ad8cb2a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -61,7 +61,6 @@ import org.jetbrains.annotations.Nullable; import javax.cache.Cache; -import javax.cache.CacheException; import javax.cache.expiry.Duration; import javax.cache.expiry.ExpiryPolicy; import javax.cache.integration.CompletionListener; From e73125d96c2a8d1f5a53a5fc3d3ed58db1dc7713 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 24 Oct 2016 12:50:22 +0300 Subject: [PATCH 260/487] merge eaf8ae24: IGNITE-4034 Get rid of specialized methods in platform targets --- .../platform/cache/PlatformCache.java | 23 +++++++++++++++---- .../cluster/PlatformClusterGroup.java | 16 ++++++------- .../platforms/cpp/jni/project/vs/module.def | 1 - .../Apache.Ignite.Core.csproj | 1 - .../Impl/Cluster/ClusterGroupImpl.cs | 12 +++++++--- 5 files changed, 36 insertions(+), 17 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index 6ce900ad8cb2a..d3fa2c825ea9e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -281,6 +281,7 @@ public class PlatformCache extends PlatformAbstractTarget { * @param cache Underlying cache. * @param keepBinary Keep binary flag. */ + @SuppressWarnings("ZeroLengthArrayAllocation") public PlatformCache(PlatformContext platformCtx, IgniteCache cache, boolean keepBinary) { this(platformCtx, cache, keepBinary, new PlatformCacheExtension[0]); } @@ -503,11 +504,25 @@ public IgniteCache rawCache() { }); } - case OP_LOCK: - return registerLock(cache.lock(reader.readObjectDetached())); + case OP_LOCK: { + long id = registerLock(cache.lock(reader.readObjectDetached())); - case OP_LOCK_ALL: - return registerLock(cache.lockAll(PlatformUtils.readCollection(reader))); + return writeResult(mem, id, new PlatformWriterClosure() { + @Override public void write(BinaryRawWriterEx writer, Long val) { + writer.writeLong(val); + } + }); + } + + case OP_LOCK_ALL: { + long id = registerLock(cache.lockAll(PlatformUtils.readCollection(reader))); + + return writeResult(mem, id, new PlatformWriterClosure() { + @Override public void write(BinaryRawWriterEx writer, Long val) { + writer.writeLong(val); + } + }); + } case OP_EXTENSION: PlatformCacheExtension ext = extension(reader.readInt()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java index 403c8ba64c4ad..d09506bff142b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.jetbrains.annotations.Nullable; /** @@ -102,6 +103,9 @@ public class PlatformClusterGroup extends PlatformAbstractTarget { /** */ private static final int OP_RESET_METRICS = 22; + /** */ + private static final int OP_FOR_SERVERS = 23; + /** Projection. */ private final ClusterGroupEx prj; @@ -316,6 +320,9 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { case OP_FOR_YOUNGEST: return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forYoungest()); + + case OP_FOR_SERVERS: + return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forServers()); } return super.processOutObject(type); @@ -336,13 +343,6 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { return super.processOutLong(type); } - /** - * @return New projection. - */ - public PlatformClusterGroup forServers() { - return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forServers()); - } - /** * @return Projection. */ @@ -367,7 +367,7 @@ private boolean pingNode(UUID nodeId) { * @return Collection of grid nodes which represented by specified topology version, * if it is present in history storage, {@code null} otherwise. * @throws UnsupportedOperationException If underlying SPI implementation does not support - * topology history. Currently only {@link org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi} + * topology history. Currently only {@link TcpDiscoverySpi} * supports topology history. */ private Collection topology(long topVer) { diff --git a/modules/platforms/cpp/jni/project/vs/module.def b/modules/platforms/cpp/jni/project/vs/module.def index 85fcac54aeeba..2e76bf7425c17 100644 --- a/modules/platforms/cpp/jni/project/vs/module.def +++ b/modules/platforms/cpp/jni/project/vs/module.def @@ -50,7 +50,6 @@ IgniteProcessorAtomicReference @128 IgniteProcessorCreateNearCache @131 IgniteProcessorGetOrCreateNearCache @132 IgniteProcessorGetCacheNames @133 -IgniteProjectionForServers @134 IgniteSetConsoleHandler @135 IgniteRemoveConsoleHandler @136 IgniteProcessorLoggerIsLevelEnabled @137 diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj index 8d81bc29369a0..08d742adaeb3b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj @@ -182,7 +182,6 @@ - diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs index e1b58618a517d..388be82525fcf 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs @@ -101,7 +101,10 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx /** */ private const int OpForRemotes = 17; - + + /** */ + public const int OpForDaemons = 18; + /** */ private const int OpForRandom = 19; @@ -114,6 +117,9 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx /** */ public const int OpResetMetrics = 22; + /** */ + public const int OpForServers = 23; + /** Initial Ignite instance. */ private readonly Ignite _ignite; @@ -299,7 +305,7 @@ public IClusterGroup ForRemotes() /** */ public IClusterGroup ForDaemons() { - return GetClusterGroup(UU.ProjectionForDaemons(Target)); + return GetClusterGroup(DoOutOpObject(OpForDaemons)); } /** */ @@ -336,7 +342,7 @@ public IClusterGroup ForYoungest() /** */ public IClusterGroup ForServers() { - return GetClusterGroup(UU.ProjectionForServers(Target)); + return GetClusterGroup(DoOutOpObject(OpForServers)); } /** */ From 6d69701038eeefaff2291a909d63d85e1ab9a376 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 24 Oct 2016 13:44:47 +0300 Subject: [PATCH 261/487] Merge commit '46dc9fc1' into ignite-1.7.3 (IGNITE-4034 Get rid of specialized methods in platform targets) --- .../cpp/core/src/impl/transactions/transactions_impl.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp b/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp index f8218827c1ee6..9f43dad86b041 100644 --- a/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp +++ b/modules/platforms/cpp/core/src/impl/transactions/transactions_impl.cpp @@ -119,6 +119,11 @@ namespace ignite val = reader.ReadInt64(); } + virtual void SetNull() + { + // No-op. + } + /** * Get value. * From 96243fbc410226002e98f15d73aeb96ecec23a2b Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 24 Oct 2016 13:53:21 +0300 Subject: [PATCH 262/487] .NET: Fix ComputeApiTestFullFooter.TestForDaemons --- .../dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs index 7b71ad4ad3a23..20290f11139e2 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs @@ -533,7 +533,7 @@ public void TestForDaemons() using (var ignite = Ignition.Start(new IgniteConfiguration(TestUtils.GetTestConfiguration()) { - SpringConfigUrl = "config\\compute\\compute-grid1.xml", + SpringConfigUrl = GetConfigs().Item1, GridName = "daemonGrid", IsDaemon = true }) From 9fc3b5095b825b9ca00b45c18f8072f998dd2adf Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 2 Aug 2016 18:37:52 +0300 Subject: [PATCH 263/487] IGNITE-3497 .NET: Improve IgniteConfigurationSection.xsd: add missing properties, enums, docs. # Conflicts: # modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs --- .../Apache.Ignite.Core.Tests.csproj | 1 + .../IgniteConfigurationSerializerTest.cs | 70 +- .../IgniteConfigurationSection.xsd | 1141 ++++++++++++++--- 3 files changed, 1054 insertions(+), 158 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj index 008229a8d197a..11a8a15a09918 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj @@ -59,6 +59,7 @@ + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs index bb703f52b05f4..2486351f7a5e7 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs @@ -28,6 +28,7 @@ namespace Apache.Ignite.Core.Tests using System.Text; using System.Threading; using System.Xml; + using System.Xml.Linq; using System.Xml.Schema; using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cache.Affinity.Fair; @@ -216,6 +217,53 @@ public void TestSerializeDeserialize() CheckSerializeDeserialize(new IgniteConfiguration()); } + /// + /// Tests that all properties are present in the schema. + /// + [Test] + public void TestAllPropertiesArePresentInSchema() + { + // ReSharper disable once PossibleNullReferenceException + var schema = XDocument.Load("IgniteConfigurationSection.xsd") + .Root.Elements() + .Single(x => x.Attribute("name").Value == "igniteConfiguration"); + + var type = typeof(IgniteConfiguration); + + CheckPropertyIsPresentInSchema(type, schema); + } + + /// + /// Checks the property is present in schema. + /// + // ReSharper disable once UnusedParameter.Local + private static void CheckPropertyIsPresentInSchema(Type type, XElement schema) + { + Func toLowerCamel = x => char.ToLowerInvariant(x[0]) + x.Substring(1); + + foreach (var prop in type.GetProperties()) + { + var propType = prop.PropertyType; + + var isCollection = propType.IsGenericType && + propType.GetGenericTypeDefinition() == typeof(ICollection<>); + + if (isCollection) + propType = propType.GetGenericArguments().First(); + + var propName = toLowerCamel(prop.Name); + + Assert.IsTrue(schema.Descendants().Select(x => x.Attribute("name")) + .Any(x => x != null && x.Value == propName), + "Property is missing in XML schema: " + propName); + + var isComplexProp = propType.Namespace != null && propType.Namespace.StartsWith("Apache.Ignite.Core"); + + if (isComplexProp) + CheckPropertyIsPresentInSchema(propType, schema); + } + } + /// /// Tests the schema validation. /// @@ -395,8 +443,8 @@ private static IgniteConfiguration GetTestConfig() Backups = 15, CacheMode = CacheMode.Replicated, CacheStoreFactory = new TestCacheStoreFactory(), - CopyOnRead = true, - EagerTtl = true, + CopyOnRead = false, + EagerTtl = false, EnableSwap = true, EvictSynchronized = true, EvictSynchronizedConcurrencyLevel = 13, @@ -431,7 +479,7 @@ private static IgniteConfiguration GetTestConfig() ValueType = typeof (long) }, }, - ReadFromBackup = true, + ReadFromBackup = false, RebalanceBatchSize = 33, RebalanceDelay = TimeSpan.MaxValue, RebalanceMode = CacheRebalanceMode.Sync, @@ -442,7 +490,7 @@ private static IgniteConfiguration GetTestConfig() StartSize = 1023, WriteBehindBatchSize = 45, WriteBehindEnabled = true, - WriteBehindFlushFrequency = TimeSpan.FromSeconds(5), + WriteBehindFlushFrequency = TimeSpan.FromSeconds(55), WriteBehindFlushSize = 66, WriteBehindFlushThreadCount = 2, WriteSynchronizationMode = CacheWriteSynchronizationMode.FullAsync, @@ -462,7 +510,7 @@ private static IgniteConfiguration GetTestConfig() { ExcludeNeighbors = true, Partitions = 48 - } + }, } }, ClientMode = true, @@ -651,21 +699,13 @@ public string GetFieldName(string name) /// public class TestSerializer : IBinarySerializer { - /// - /// Write portalbe object. - /// - /// Object. - /// Poratble writer. + /** */ public void WriteBinary(object obj, IBinaryWriter writer) { // No-op. } - /// - /// Read binary object. - /// - /// Instantiated empty object. - /// Poratble reader. + /** */ public void ReadBinary(object obj, IBinaryReader reader) { // No-op. diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd index 916fd2058ef93..40e30558dab01 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd @@ -24,44 +24,168 @@ xmlns:mstns="http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection" xmlns:xs="http://www.w3.org/2001/XMLSchema"> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Ignite configuration root. + + + Binary configuration. + + + Type configurations. + + + Name mapper for the given type. + - + + + Assembly-qualified type name. + + + + ID mapper for the given type. When it is necessary to resolve class (field) ID, then this property will be checked first. Otherwise, ID will be hash code of the class (field) simple name in lower case. + - + + + Assembly-qualified type name. + + + + + Serializer for the given type. If not provided and class implements IBinarizable then its custom logic will be used. If not provided and class doesn't implement IBinarizable then all fields of the class except of those with [NotSerialized] attribute will be serialized with help of reflection. + + - + + + Assembly-qualified type name. + + - - - - + + + Fully qualified type name. + + + + + Affinity key field name. + + + + + + Keep deserialized flag. If set to non-null value, overrides default value set in BinaryTypeConfiguration. + + + + + + Whether this instance describes an enum type. + + + + + Binarizable types. Shorthand for creating "BinaryTypeConfiguration. + + @@ -69,173 +193,528 @@ + + Default name mapper. + - + + + Assembly-qualified type name. + + + + Default ID mapper. + - + + + Assembly-qualified type name. + + + + Default serializer. + - + + + Assembly-qualified type name. + + - - + + + Default keep deserialized flag. + + + + + Compact footer flag. + + + + Cache configuration. + + + Cache configuration. + + + Factory for underlying persistent storage for read-through and write-through operations. + - + + + Assembly-qualified type name. + + + + Query entity configuration. + + + Query fields, a map from field name to Java type name. The order of fields defines the order of columns returned by the 'select *' queries. + - - - + + + Cache name. + + + + + .NET field type. + + + + + Java field type name. + + + + Field name aliases: mapping from full name in dot notation to an alias that will be used as SQL column name. Example: {"parent.name" -> "parentName"}. + - - + + + Full name of the query field. + + + + + Alias for the full name. + + + + Query indexes. + + + Query fields, a map from field name to Java type name. The order of fields defines the order of columns returned by the 'select *' queries. + - - + + + Cache name. + + + + + Whether this index is descending. + + - - + + + Index name. + + + + + Index type. + + - - - - + + + + Java key type name. + + + + + + + .NET key type. + + + + + + + Java value type name. + + + + + + + .NET value type. + + + + + Near cache configuration. + + + Eviction policy. Null value means disabled evictions. + - - - - + + + + Size of the eviction batch. Batch eviction is enabled only if maximum memory limit isn't set (MaxMemorySize == 0). + + + + + + Maximum allowed cache size (entry count). 0 for unlimited. + + + + + Maximum allowed cache size in bytes. 0 for unlimited. + + + + + Assembly-qualified type name. + + - + + + Initial cache size for near cache which will be used to pre-create internal hash table after start. + + + + Eviction policy. Null value means disabled evictions. + - - - - + + + + Size of the eviction batch. Batch eviction is enabled only if maximum memory limit isn't set (MaxMemorySize == 0). + + + + + + Maximum allowed cache size (entry count). 0 for unlimited. + + + + + Maximum allowed cache size in bytes. 0 for unlimited. + + + + + Assembly-qualified type name. + + + + Affinity function to provide mapping from keys to nodes. + - - - + + + Number of partitions. + + + + + Whether to exclude same-host-neighbors from being backups of each other. + + + + + Assembly-qualified type name. + + - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + Cache name. + + + + + Caching mode to use. + + + + + Write synchronization mode. This mode controls whether the main caller should wait for update on other nodes to complete or not. + + + + + + Flag indicating whether eviction is synchronized between primary, backup and near nodes. If this parameter is true and swap is disabled then ICache.LocalEvict will involve all nodes where an entry is kept. If this property is set to false then eviction is done independently on different cache nodes. Note that it's not recommended to set this value to true if cache store is configured since it will allow to significantly improve cache performance. + + + + + + Size of the key buffer for synchronized evictions. + + + + + + Concurrency level for synchronized evictions. This flag only makes sense with EvictSynchronized set to true. When synchronized evictions are enabled, it is possible that local eviction policy will try to evict entries faster than evictions can be synchronized with backup or near nodes. This value specifies how many concurrent synchronous eviction sessions should be allowed before the system is forced to wait and let synchronous evictions catch up with the eviction policy. + + + + + + Timeout for synchronized evictions + + + + + This value denotes the maximum size of eviction queue in percents of cache size in case of distributed cache (replicated and partitioned) and using synchronized eviction + + + + + Initial cache size which will be used to pre-create internal hash table after start. + + + + + + Flag indicating whether value should be loaded from store if it is not in the cache. + + + + + + Cache atomicity mode. + + + + + Cache write ordering mode. + + + + + + Number of nodes used to back up single partition for Partitioned cache. + + + + + + Default lock acquisition timeout. + + + + + Invalidation flag. If true, values will be invalidated (nullified) upon commit in near cache. + + + + + Cache rebalance mode. + + + + + Batch size (in number bytes) to be loaded within a single rebalance message. Rebalancing algorithm will split total data set on every node into multiple batches prior to sending data. + + + + + Flag indicating whether Ignite should use swap storage by default. + + + + + Maximum number of allowed concurrent asynchronous operations, 0 for unlimited. + + + + + Flag indicating whether Ignite should use write-behind behaviour for the cache store. + + + + + Maximum size of the write-behind cache. If cache size exceeds this value, all cached items are flushed to the cache store and write cache is cleared. + + + + + + Number of threads that will perform cache flushing. Cache flushing is performed when cache size exceeds value defined by WriteBehindFlushSize, or flush interval defined by WriteBehindFlushFrequency is elapsed. + + + + + + + Maximum batch size for write-behind cache store operations. Store operations (get or remove) are combined in a batch of this size to be passed to ICacheStore.WriteAll or ICacheStore.DeleteAll methods. + + + + + + Rebalance timeout. + + + + + Delay upon a node joining or leaving topology (or crash) after which rebalancing should be started automatically. Rebalancing should be delayed if you plan to restart nodes after they leave topology, or if you plan to start multiple nodes at once or one after another and don't want to repartition and rebalance until all nodes are started. + + + + + Time to wait between rebalance messages to avoid overloading of CPU or network. When rebalancing large data sets, the CPU or network can get over-consumed with rebalancing messages, which consecutively may slow down the application performance. This parameter helps tune the amount of time to wait between rebalance messages to make sure that rebalancing process does not have any negative performance impact. Note that application will continue to work properly while rebalancing is still in progress. + + + + + Maximum amount of memory available to off-heap storage. Possible values are -1 means that off-heap storage is disabled. 0 means that Ignite will not limit off-heap storage (it's up to user to properly add and remove entries from cache to ensure that off-heap storage does not grow indefinitely. Any positive value specifies the limit of off-heap storage in bytes. + + + + + Memory mode for cache. + + + + + Timeout after which long query warning will be printed. + + + + + If true all the SQL table and field names will be escaped with double quotes like ({ "tableName"."fieldsName"}). This enforces case sensitivity for field names and also allows having special characters in table and field names. + + + + + Number of SQL rows which will be cached onheap to avoid deserialization on each SQL index access. This setting only makes sense when offheap is enabled for this cache. + + + + + Whether read-through should be enabled for cache operations. + + + + + Whether write-through should be enabled for cache operations. + + + + + Whether expired cache entries will be eagerly removed from cache. When set to false, expired entries will be removed on next entry access. + + + + + + Whether ICacheStore is working with binary objects instead of deserialized objects. + + + + + + Whether data can be read from backup. + + + + + Whether copy of the value stored in cache should be created for cache operation implying return value. + + + + + Frequency with which write-behind cache is flushed to the cache store. This value defines the maximum time interval between object insertion/deletion from the cache at the moment when corresponding operation is applied to the cache store. + + + + Collection of options passed to JVM on Ignite start. + @@ -243,6 +722,9 @@ + + List of additional .Net assemblies to load on Ignite start. Each item can be either fully qualified assembly name, path to assembly to DLL or path to a directory when assemblies reside. + @@ -250,23 +732,39 @@ + + Lifecycle beans. + - + + + Assembly-qualified type name. + + + + Discovery service provider. Null for default discovery. + + + IP finder which defines how nodes will find each other on the network. + + + End points. + @@ -274,60 +772,254 @@ - - - - - - - + + + Assembly-qualified type name. + + + + + Local host address for socket binding. Note that one node could have additional addresses beside the loopback one. This configuration parameter is optional. + + + + + IP address of the multicast group. + + + + + Port number which multicast messages are sent to. + + + + + Number of attempts to send multicast address request. IP finder re-sends request only in case if no reply for previous request is received. + + + + + Response timeout. + + + + + Time to live for multicast packets sent out on this IP finder in order to control the scope of the multicast. + + - - - - - - - - - - - - - - - - - - - + + + Socket timeout. + + + + + Timeout for receiving acknowledgement for sent message. + + + + + Maximum timeout for receiving acknowledgement for sent message. + + + + + Network timeout. + + + + + Join timeout. + + + + + Whether client does not try to reconnect after server detected client node failure. + + + + + + Whether TcpDiscoverySpi is started in server mode regardless of IgniteConfiguration.ClientMode setting. + + + + + + Delay between issuing of heartbeat messages. SPI sends heartbeat messages in configurable time interval to other nodes to notify them about its state. + + + + + IP finder clean frequency. + + + + + Local host address for socket binding. Note that one node could have additional addresses beside the loopback one. This configuration parameter is optional. + + + + + Local port for socket binding. + + + + + + Local port range for local host ports (value must greater than or equal to 0). If provided local port LocalPort is occupied, implementation will try to increment the port number for as long as it is less than initial value plus this range. + + + + + + Maximum heartbeats count node can miss without failing client node. + + + + + Maximum heartbeats count node can miss without initiating status check. + + + + + Maximum number of reconnect attempts used when establishing connection with remote nodes. + + + + + + Statistics print frequency. TimeSpan.Zero for no statistics. + + + + + + Sets thread priority, 1 (lowest) to 10 (highest). All threads within SPI will be started with it. + + + + + Size of topology snapshots history. + + + + + Assembly-qualified type name. + + + + Communication service provider. Null for default communication. + - - - - - - - - - - - - - - - - - - + + + Number of received messages per connection to node after which acknowledgment message is sent. + + + + + Connect timeout used when establishing connection with remote nodes. + + + + + Whether to allocate direct (ByteBuffer.allocateDirect) or heap (ByteBuffer.allocate) buffer. + + + + + Value indicating whether to allocate direct (ByteBuffer.allocateDirect) or heap (ByteBuffer.allocate) send buffer. + + + + + Sets maximum idle connection timeout upon which a connection to client will be closed. + + + + + Local host address for socket binding. Note that one node could have additional addresses beside the loopback one. This configuration parameter is optional. + + + + + Local port for socket binding. + + + + + + Local port range for local host ports (value must greater than or equal to 0). If provided local port LocalPort is occupied, implementation will try to increment the port number for as long as it is less than initial value plus this range. + + + + + + Maximum connect timeout. If handshake is not established within connect timeout, then SPI tries to repeat handshake procedure with increased connect timeout. Connect timeout can grow till maximum timeout value, if maximum timeout value is reached then the handshake is considered as failed. + + + + + Message queue limit for incoming and outgoing messages. + + + + + Maximum number of reconnect attempts used when establishing connection with remote nodes. + + + + + Count of selectors to be used in TCP server. + + + + + Slow client queue limit. + + + + + Size of the socket receive buffer. + + + + + Size of the socket send buffer. + + + + + + Value for TCP_NODELAY socket option. Each socket will be opened using provided value. + + + + + + Maximum number of stored unacknowledged messages per connection to node. If number of unacknowledged messages exceeds this number then connection to node is closed and reconnect is attempted. + + + + + Assembly-qualified type name. + + + + + A set of event types to be recorded by Ignite. + + @@ -342,19 +1034,65 @@ + + Atomic data structures configuration. + - - - + + + Caching mode to use. + + + + + + Number of nodes used to back up single partition for Partitioned cache. + + + + + + + The default number of sequence values reserved for IAtomicSequence instances. After a certain number has been reserved, consequent increments of sequence will happen locally, without communication with other nodes, until the next reservation has to be made. + + + + + Transaction configuration. + - - - - - + + + Cache transaction concurrency to use when one is not explicitly specified. + + + + + Cache transaction isolation to use when one is not explicitly specified. + + + + + + Cache transaction timeout to use when one is not explicitly specified. TimeSpan.Zero for infinite timeout. + + + + + + + Size of pessimistic transactions log stored on node in order to recover transaction commit if originating node has left grid before it has sent all messages to transaction nodes. 0 for unlimited. + + + + + + Delay after which pessimistic recovery entries will be cleaned up for failed node. + + @@ -370,24 +1108,141 @@ - - - - - - - - - - - - - - - - - - + + + Local Ignite instance name to be used with Ignition.GetIgnite. + + + + + Path jvm.dll file. If not set, it's location will be determined using JAVA_HOME environment variable. If path is neither set nor determined automatically, an exception will be thrown. + + + + + Path to Ignite home. If not set environment variable IGNITE_HOME will be used. + + + + + Classpath used by JVM on Ignite start. + + + + + Whether to suppress warnings. + + + + + + Initial amount of memory in megabytes given to JVM. Maps to -Xms Java option. -1 maps to JVM defaults. + + + + + + + Maximum amount of memory in megabytes given to JVM. Maps to -Xmx Java option. -1 maps to JVM defaults. + + + + + + + Whether node should start in client mode. Client node cannot hold data in the caches. + + + + + + + Time after which a certain metric value is considered expired. + + + + + + + Number of metrics kept in history to compute totals and averages. + + + + + + + Frequency of metrics log print out. + + + + + + + Metrics update frequency. + + + + + + + Network send retry count. + + + + + + + Network send retry delay. + + + + + + Network timeout. + + + + + + Work directory. If not provided, a folder under IgniteHome will be used. + + + + + + System-wide local address or host for all Ignite components to bind to. If provided it will override all default local bind settings within Ignite. + + + + + + Whether this node should be a daemon node. Daemon nodes are the usual grid nodes that participate in topology but not visible on the main APIs. + Daemon nodes are used primarily for management and monitoring functionality. + + + + + + URL to Spring configuration file. + + + + + + Whether late affinity assignment mode should be used. + On each topology change, for each started cache, partition-to-node mapping is + calculated using AffinityFunction for cache. When late + affinity assignment mode is disabled then new affinity mapping is applied immediately. + With late affinity assignment mode, if primary node was changed for some partition, but data for this + partition is not rebalanced yet on this node, then current primary is not changed and new primary + is temporary assigned as backup. This nodes becomes primary only when rebalancing for all assigned primary + partitions is finished. This mode can show better performance for cache operations, since when cache + primary node executes some operation and data is not rebalanced yet, then it sends additional message + to force rebalancing from other nodes. + + + - + + From 04ac900a06b60fd58b7f5aa71c5371632fe03ede Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 24 Oct 2016 14:45:32 +0300 Subject: [PATCH 264/487] .NET: Fix TestCyrillicChars --- .../dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs index a030bf29080f0..2f37505fffbda 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs @@ -61,7 +61,8 @@ public void TestSlnToolsVersion() [Test] public void TestCyrillicChars() { - var srcFiles = GetDotNetSourceDir().GetFiles("*.cs", SearchOption.AllDirectories); + var srcFiles = GetDotNetSourceDir().GetFiles("*.cs", SearchOption.AllDirectories) + .Where(x => x.Name != "BinaryStringTest.cs" && x.Name != "BinarySelfTest.cs"); CheckFiles(srcFiles, x => x.Contains('\u0441') || x.Contains('\u0421'), "Files with Cyrillic 'C': "); } From f664751d2f5fda214ffa9df4f1375da4acd2f200 Mon Sep 17 00:00:00 2001 From: isapego Date: Mon, 24 Oct 2016 14:50:50 +0300 Subject: [PATCH 265/487] Merge-related fix for C++. --- .../platforms/cpp/core-test/config/cache-query.xml | 12 +----------- .../platforms/cpp/core-test/src/cache_query_test.cpp | 2 +- 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/modules/platforms/cpp/core-test/config/cache-query.xml b/modules/platforms/cpp/core-test/config/cache-query.xml index c3b5389a8fbe0..bb18f7cc56d42 100644 --- a/modules/platforms/cpp/core-test/config/cache-query.xml +++ b/modules/platforms/cpp/core-test/config/cache-query.xml @@ -69,20 +69,10 @@ - - - + diff --git a/modules/platforms/cpp/core-test/src/cache_query_test.cpp b/modules/platforms/cpp/core-test/src/cache_query_test.cpp index 9a5371bfac650..c1c26ac66743f 100644 --- a/modules/platforms/cpp/core-test/src/cache_query_test.cpp +++ b/modules/platforms/cpp/core-test/src/cache_query_test.cpp @@ -772,7 +772,7 @@ struct CacheQueryTestSuiteFixture /** Person cache accessor. */ Cache GetPersonCache() { - return grid.GetCache("cache"); + return grid.GetCache("QueryPerson"); } /** Relation cache accessor. */ From 255b3a3e199b19757a511d4340f184a9de7afddc Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 24 Oct 2016 14:52:21 +0300 Subject: [PATCH 266/487] .NET: Fix ServicesTest.TestDuckTyping --- .../dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs index f2a5fed208356..0558d1120496e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs @@ -322,8 +322,7 @@ public void TestDuckTyping([Values(true, false)] bool local) // .. but setter does not var ex = Assert.Throws(() => { prx.TestProperty = new object(); }); Assert.IsNotNull(ex.InnerException); - Assert.AreEqual("Object of type 'System.Object' cannot be converted to type 'System.Int32'.", - ex.InnerException.Message); + Assert.AreEqual("Specified cast is not valid.", ex.InnerException.Message); } /// @@ -438,6 +437,7 @@ public void TestInitException() var ex = Assert.Throws(() => Services.DeployMultiple(SvcName, svc, Grids.Length, 1)); Assert.AreEqual("Expected exception", ex.Message); + Assert.IsNotNull(ex.InnerException); Assert.IsTrue(ex.InnerException.Message.Contains("PlatformCallbackUtils.serviceInit(Native Method)")); var svc0 = Services.GetService(SvcName); From ec12a9db2265180f96be72e2217e60ced856164e Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 24 Oct 2016 17:52:36 +0300 Subject: [PATCH 267/487] Minor fix for flags passed to GridCacheMapEntry.initialValue from data streamer isolated updater. --- .../internal/processors/datastreamer/DataStreamerImpl.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java index c2f226c537025..a6065ddd76118 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java @@ -1656,13 +1656,15 @@ else if (ttl == CU.TTL_NOT_CHANGED) expiryTime = CU.toExpireTime(ttl); } + boolean primary = cctx.affinity().primary(cctx.localNode(), entry.key(), topVer); + entry.initialValue(e.getValue(), ver, ttl, expiryTime, false, topVer, - GridDrType.DR_LOAD, + primary ? GridDrType.DR_LOAD : GridDrType.DR_PRELOAD, false); cctx.evicts().touch(entry, topVer); From 02dd07a58277b357991c1f74a7dbdfdd9de2a2cc Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 25 Oct 2016 12:34:35 +0300 Subject: [PATCH 268/487] IGNITE-4074 Refactor async (*future) operations in PlatformTarget --- .../platform/PlatformAbstractTarget.java | 87 ++++- .../processors/platform/PlatformTarget.java | 21 -- .../platform/cache/PlatformCache.java | 274 +++++++++++++- .../platform/compute/PlatformCompute.java | 29 +- .../platform/events/PlatformEvents.java | 103 +++-- .../platform/messaging/PlatformMessaging.java | 53 ++- .../platform/services/PlatformServices.java | 98 +++-- .../transactions/PlatformTransactions.java | 38 +- .../cpp/jni/include/ignite/jni/exports.h | 2 - .../cpp/jni/include/ignite/jni/java.h | 4 - .../platforms/cpp/jni/project/vs/module.def | 2 - modules/platforms/cpp/jni/src/exports.cpp | 8 - modules/platforms/cpp/jni/src/java.cpp | 26 -- .../Impl/Cache/CacheImpl.cs | 353 +++++++++--------- .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs | 27 +- .../Impl/Compute/ComputeImpl.cs | 27 +- .../Apache.Ignite.Core/Impl/Events/Events.cs | 168 ++++----- .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs | 2 +- .../Impl/Messaging/Messaging.cs | 113 ++---- .../Apache.Ignite.Core/Impl/PlatformTarget.cs | 139 ++++++- .../Impl/Services/Services.cs | 125 ++++--- .../Impl/Transactions/TransactionsImpl.cs | 13 +- .../Impl/Unmanaged/IgniteJniNativeMethods.cs | 12 - .../Impl/Unmanaged/UnmanagedUtils.cs | 25 -- .../dotnet/Apache.Ignite.sln.DotSettings | 3 +- .../Apache.Ignite.sln.TeamCity.DotSettings | 1 + 26 files changed, 1060 insertions(+), 693 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java index 6197bc8280174..29b603ac964b6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java @@ -19,6 +19,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.binary.BinaryRawReader; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.binary.BinaryRawReaderEx; import org.apache.ignite.internal.binary.BinaryRawWriterEx; @@ -26,6 +27,7 @@ import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream; import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils; import org.apache.ignite.internal.processors.platform.utils.PlatformListenable; +import org.apache.ignite.lang.IgniteFuture; import org.jetbrains.annotations.Nullable; /** @@ -244,23 +246,12 @@ public PlatformContext platformContext() { /** {@inheritDoc} */ @Override public void listenFuture(final long futId, int typ) throws Exception { - listenFutureAndGet(futId, typ); + PlatformFutureUtils.listen(platformCtx, currentFuture(), futId, typ, null, this); } /** {@inheritDoc} */ @Override public void listenFutureForOperation(final long futId, int typ, int opId) throws Exception { - listenFutureForOperationAndGet(futId, typ, opId); - } - - /** {@inheritDoc} */ - @Override public PlatformListenable listenFutureAndGet(final long futId, int typ) throws Exception { - return PlatformFutureUtils.listen(platformCtx, currentFuture(), futId, typ, null, this); - } - - /** {@inheritDoc} */ - @Override public PlatformListenable listenFutureForOperationAndGet(final long futId, int typ, int opId) - throws Exception { - return PlatformFutureUtils.listen(platformCtx, currentFuture(), futId, typ, futureWriter(opId), this); + PlatformFutureUtils.listen(platformCtx, currentFuture(), futId, typ, futureWriter(opId), this); } /** @@ -413,4 +404,74 @@ protected Object processOutObject(int type) throws IgniteCheckedException { protected T throwUnsupported(int type) throws IgniteCheckedException { throw new IgniteCheckedException("Unsupported operation type: " + type); } + + /** + * Reads future information and listens. + * + * @param reader Reader. + * @param fut Future. + * @param writer Writer. + * @throws IgniteCheckedException In case of error. + */ + protected PlatformListenable readAndListenFuture(BinaryRawReader reader, IgniteInternalFuture fut, + PlatformFutureUtils.Writer writer) + throws IgniteCheckedException { + long futId = reader.readLong(); + int futTyp = reader.readInt(); + + return PlatformFutureUtils.listen(platformCtx, fut, futId, futTyp, writer, this); + } + + /** + * Reads future information and listens. + * + * @param reader Reader. + * @param fut Future. + * @param writer Writer. + * @throws IgniteCheckedException In case of error. + */ + protected PlatformListenable readAndListenFuture(BinaryRawReader reader, IgniteFuture fut, + PlatformFutureUtils.Writer writer) + throws IgniteCheckedException { + long futId = reader.readLong(); + int futTyp = reader.readInt(); + + return PlatformFutureUtils.listen(platformCtx, fut, futId, futTyp, writer, this); + } + + /** + * Reads future information and listens. + * + * @param reader Reader. + * @param fut Future. + * @throws IgniteCheckedException In case of error. + */ + protected PlatformListenable readAndListenFuture(BinaryRawReader reader, IgniteInternalFuture fut) + throws IgniteCheckedException { + return readAndListenFuture(reader, fut, null); + } + + /** + * Reads future information and listens. + * + * @param reader Reader. + * @param fut Future. + * @throws IgniteCheckedException In case of error. + */ + protected PlatformListenable readAndListenFuture(BinaryRawReader reader, IgniteFuture fut) + throws IgniteCheckedException { + return readAndListenFuture(reader, fut, null); + } + + /** + * Reads future information and listens. + * + * @param reader Reader. + * @throws IgniteCheckedException In case of error. + */ + protected long readAndListenFuture(BinaryRawReader reader) throws IgniteCheckedException { + readAndListenFuture(reader, currentFuture(), null); + + return TRUE; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java index 40773d0b09bcd..3ab5d7c76e37f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java @@ -137,25 +137,4 @@ public Object inObjectStreamOutObjectStream(int type, @Nullable Object arg, long */ @SuppressWarnings("UnusedDeclaration") public void listenFutureForOperation(final long futId, int typ, int opId) throws Exception; - - /** - * Start listening for the future. - * - * @param futId Future ID. - * @param typ Result type. - * @throws IgniteCheckedException In case of failure. - */ - @SuppressWarnings("UnusedDeclaration") - public PlatformListenable listenFutureAndGet(final long futId, int typ) throws Exception; - - /** - * Start listening for the future for specific operation type. - * - * @param futId Future ID. - * @param typ Result type. - * @param opId Operation ID required to pick correct result writer. - * @throws IgniteCheckedException In case of failure. - */ - @SuppressWarnings("UnusedDeclaration") - public PlatformListenable listenFutureForOperationAndGet(final long futId, int typ, int opId) throws Exception; } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index d3fa2c825ea9e..558a9b3538525 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -245,7 +245,82 @@ public class PlatformCache extends PlatformAbstractTarget { public static final int OP_SIZE_LOC = 56; /** */ - public static final int OP_EXTENSION = 57; + public static final int OP_PUT_ASYNC = 57; + + /** */ + public static final int OP_CLEAR_CACHE_ASYNC = 58; + + /** */ + public static final int OP_CLEAR_ALL_ASYNC = 59; + + /** */ + public static final int OP_REMOVE_ALL2_ASYNC = 60; + + /** */ + public static final int OP_SIZE_ASYNC = 61; + + /** */ + public static final int OP_CLEAR_ASYNC = 62; + + /** */ + public static final int OP_LOAD_CACHE_ASYNC = 63; + + /** */ + public static final int OP_LOC_LOAD_CACHE_ASYNC = 64; + + /** */ + public static final int OP_PUT_ALL_ASYNC = 65; + + /** */ + public static final int OP_REMOVE_ALL_ASYNC = 66; + + /** */ + public static final int OP_GET_ASYNC = 67; + + /** */ + public static final int OP_CONTAINS_KEY_ASYNC = 68; + + /** */ + public static final int OP_CONTAINS_KEYS_ASYNC = 69; + + /** */ + public static final int OP_REMOVE_BOOL_ASYNC = 70; + + /** */ + public static final int OP_REMOVE_OBJ_ASYNC = 71; + + /** */ + public static final int OP_GET_ALL_ASYNC = 72; + + /** */ + public static final int OP_GET_AND_PUT_ASYNC = 73; + + /** */ + public static final int OP_GET_AND_PUT_IF_ABSENT_ASYNC = 74; + + /** */ + public static final int OP_GET_AND_REMOVE_ASYNC = 75; + + /** */ + public static final int OP_GET_AND_REPLACE_ASYNC = 76; + + /** */ + public static final int OP_REPLACE_2_ASYNC = 77; + + /** */ + public static final int OP_REPLACE_3_ASYNC = 78; + + /** */ + public static final int OP_INVOKE_ASYNC = 79; + + /** */ + public static final int OP_INVOKE_ALL_ASYNC = 80; + + /** */ + public static final int OP_PUT_IF_ABSENT_ASYNC = 81; + + /** */ + public static final int OP_EXTENSION = 82; /** Underlying JCache in binary mode. */ private final IgniteCacheProxy cache; @@ -253,6 +328,9 @@ public class PlatformCache extends PlatformAbstractTarget { /** Initial JCache (not in binary mode). */ private final IgniteCache rawCache; + /** Underlying JCache in async mode. */ + private final IgniteCache cacheAsync; + /** Whether this cache is created with "keepBinary" flag on the other side. */ private final boolean keepBinary; @@ -302,8 +380,9 @@ public PlatformCache(PlatformContext platformCtx, IgniteCache cache, boolean kee assert exts != null; rawCache = cache; - - this.cache = (IgniteCacheProxy)cache.withKeepBinary(); + IgniteCache binCache = cache.withKeepBinary(); + cacheAsync = binCache.withAsync(); + this.cache = (IgniteCacheProxy)binCache; this.keepBinary = keepBinary; this.exts = exts; } @@ -380,12 +459,12 @@ public IgniteCache rawCache() { reader.readObjectDetached()) ? TRUE : FALSE; case OP_LOC_LOAD_CACHE: - loadCache0(reader, true); + loadCache0(reader, true, cache); return TRUE; case OP_LOAD_CACHE: - loadCache0(reader, false); + loadCache0(reader, false, cache); return TRUE; @@ -422,14 +501,17 @@ public IgniteCache rawCache() { return cache.isLocalLocked(reader.readObjectDetached(), reader.readBoolean()) ? TRUE : FALSE; case OP_LOAD_ALL: { - long futId = reader.readLong(); boolean replaceExisting = reader.readBoolean(); + Set keys = PlatformUtils.readSet(reader); + + long futId = reader.readLong(); + int futTyp = reader.readInt(); CompletionListenable fut = new CompletionListenable(); - PlatformFutureUtils.listen(platformCtx, fut, futId, PlatformFutureUtils.TYP_OBJ, null, this); + PlatformFutureUtils.listen(platformCtx, fut, futId, futTyp, null, this); - cache.loadAll(PlatformUtils.readSet(reader), replaceExisting, fut); + cache.loadAll(keys, replaceExisting, fut); return TRUE; } @@ -482,6 +564,167 @@ public IgniteCache rawCache() { }); } + + case OP_PUT_ASYNC: { + cacheAsync.put(reader.readObjectDetached(), reader.readObjectDetached()); + + return readAndListenFuture(reader); + } + + case OP_CLEAR_CACHE_ASYNC: { + cacheAsync.clear(); + + return readAndListenFuture(reader); + } + + case OP_CLEAR_ALL_ASYNC: { + cacheAsync.clearAll(PlatformUtils.readSet(reader)); + + return readAndListenFuture(reader); + } + + case OP_REMOVE_ALL2_ASYNC: { + cacheAsync.removeAll(); + + return readAndListenFuture(reader); + } + + case OP_SIZE_ASYNC: { + CachePeekMode[] modes = PlatformUtils.decodeCachePeekModes(reader.readInt()); + + cacheAsync.size(modes); + + return readAndListenFuture(reader); + } + + case OP_CLEAR_ASYNC: { + cacheAsync.clear(reader.readObjectDetached()); + + return readAndListenFuture(reader); + } + + case OP_LOAD_CACHE_ASYNC: { + loadCache0(reader, false, cacheAsync); + + return readAndListenFuture(reader); + } + + case OP_LOC_LOAD_CACHE_ASYNC: { + loadCache0(reader, true, cacheAsync); + + return readAndListenFuture(reader); + } + + case OP_PUT_ALL_ASYNC: + cacheAsync.putAll(PlatformUtils.readMap(reader)); + + return readAndListenFuture(reader); + + case OP_REMOVE_ALL_ASYNC: + cacheAsync.removeAll(PlatformUtils.readSet(reader)); + + return readAndListenFuture(reader); + + case OP_REBALANCE: + readAndListenFuture(reader, cache.rebalance()); + + return TRUE; + + case OP_GET_ASYNC: + cacheAsync.get(reader.readObjectDetached()); + + return readAndListenFuture(reader); + + case OP_CONTAINS_KEY_ASYNC: + cacheAsync.containsKey(reader.readObjectDetached()); + + return readAndListenFuture(reader); + + case OP_CONTAINS_KEYS_ASYNC: + cacheAsync.containsKeys(PlatformUtils.readSet(reader)); + + return readAndListenFuture(reader); + + case OP_REMOVE_OBJ_ASYNC: + cacheAsync.remove(reader.readObjectDetached()); + + return readAndListenFuture(reader); + + case OP_REMOVE_BOOL_ASYNC: + cacheAsync.remove(reader.readObjectDetached(), reader.readObjectDetached()); + + return readAndListenFuture(reader); + + case OP_GET_ALL_ASYNC: { + Set keys = PlatformUtils.readSet(reader); + + cacheAsync.getAll(keys); + + readAndListenFuture(reader, cacheAsync.future(), WRITER_GET_ALL); + + return TRUE; + } + + case OP_GET_AND_PUT_ASYNC: + cacheAsync.getAndPut(reader.readObjectDetached(), reader.readObjectDetached()); + + return readAndListenFuture(reader); + + case OP_GET_AND_PUT_IF_ABSENT_ASYNC: + cacheAsync.getAndPutIfAbsent(reader.readObjectDetached(), reader.readObjectDetached()); + + return readAndListenFuture(reader); + + case OP_GET_AND_REMOVE_ASYNC: + cacheAsync.getAndRemove(reader.readObjectDetached()); + + return readAndListenFuture(reader); + + case OP_GET_AND_REPLACE_ASYNC: + cacheAsync.getAndReplace(reader.readObjectDetached(), reader.readObjectDetached()); + + return readAndListenFuture(reader); + + case OP_REPLACE_2_ASYNC: + cacheAsync.replace(reader.readObjectDetached(), reader.readObjectDetached()); + + return readAndListenFuture(reader); + + case OP_REPLACE_3_ASYNC: + cacheAsync.replace(reader.readObjectDetached(), reader.readObjectDetached(), + reader.readObjectDetached()); + + return readAndListenFuture(reader); + + case OP_INVOKE_ASYNC: { + Object key = reader.readObjectDetached(); + + CacheEntryProcessor proc = platformCtx.createCacheEntryProcessor(reader.readObjectDetached(), 0); + + cacheAsync.invoke(key, proc); + + readAndListenFuture(reader, cacheAsync.future(), WRITER_INVOKE); + + return TRUE; + } + + case OP_INVOKE_ALL_ASYNC: { + Set keys = PlatformUtils.readSet(reader); + + CacheEntryProcessor proc = platformCtx.createCacheEntryProcessor(reader.readObjectDetached(), 0); + + cacheAsync.invokeAll(keys, proc); + + readAndListenFuture(reader, cacheAsync.future(), WRITER_INVOKE_ALL); + + return TRUE; + } + + case OP_PUT_IF_ABSENT_ASYNC: + cacheAsync.putIfAbsent(reader.readObjectDetached(), reader.readObjectDetached()); + + return readAndListenFuture(reader); + case OP_INVOKE: { Object key = reader.readObjectDetached(); @@ -573,12 +816,10 @@ public long writeResult(PlatformMemory mem, Object obj, PlatformWriterClosure cl return TRUE; } - - /** * Loads cache via localLoadCache or loadCache. */ - private void loadCache0(BinaryRawReaderEx reader, boolean loc) { + private void loadCache0(BinaryRawReaderEx reader, boolean loc, IgniteCache cache) { PlatformCacheEntryFilter filter = null; Object pred = reader.readObjectDetached(); @@ -836,6 +1077,15 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc) { return TRUE; } + case OP_CLEAR_CACHE: + cache.clear(); + + return TRUE; + + case OP_REMOVE_ALL2: + cache.removeAll(); + + return TRUE; case OP_REBALANCE: { PlatformFutureUtils.listen(platformCtx, cache.rebalance().chain(new C1() { @Override public Object apply(IgniteFuture fut) { @@ -916,7 +1166,7 @@ private static void writeError(BinaryRawWriterEx writer, Exception ex) { /** */ @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException { - return ((IgniteFutureImpl)cache.future()).internalFuture(); + return ((IgniteFutureImpl) cacheAsync.future()).internalFuture(); } /** */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java index 36d709a3f1356..0c10a5304fa41 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java @@ -78,8 +78,6 @@ public class PlatformCompute extends PlatformAbstractTarget { /** Compute instance for platform-only nodes. */ private final IgniteComputeImpl computeForPlatform; - /** Future for previous asynchronous operation. */ - protected ThreadLocal curFut = new ThreadLocal<>(); /** * Constructor. * @@ -121,6 +119,9 @@ public PlatformCompute(PlatformContext platformCtx, ClusterGroup grp, String pla return executeNative0(task); } + case OP_EXEC_ASYNC: + return executeJavaTask(reader, true); + default: return super.processInStreamOutObject(type, reader); } @@ -235,26 +236,11 @@ private PlatformJob nextClosureJob(PlatformAbstractTask task, BinaryRawReaderEx break; - case OP_EXEC_ASYNC: - writer.writeObjectDetached(executeJavaTask(reader, true)); - - break; - default: super.processInStreamOutStream(type, reader, writer); } } - /** */ - @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException { - IgniteInternalFuture fut = curFut.get(); - - if (fut == null) - throw new IllegalStateException("Asynchronous operation not started."); - - return fut; - } - /** * Execute task. * @@ -287,7 +273,7 @@ private PlatformListenable executeNative0(final PlatformAbstractTask task) { * @param reader Reader. * @return Task result. */ - protected Object executeJavaTask(BinaryRawReaderEx reader, boolean async) { + protected Object executeJavaTask(BinaryRawReaderEx reader, boolean async) throws IgniteCheckedException { String taskName = reader.readString(); boolean keepBinary = reader.readBoolean(); Object arg = reader.readObjectDetached(); @@ -304,11 +290,8 @@ protected Object executeJavaTask(BinaryRawReaderEx reader, boolean async) { Object res = compute0.execute(taskName, arg); - if (async) { - curFut.set(new ComputeConvertingFuture(compute0.future())); - - return null; - } + if (async) + return readAndListenFuture(reader, new ComputeConvertingFuture(compute0.future())); else return toBinary(res); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java index f1335247999cf..383e7ab655991 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java @@ -80,9 +80,18 @@ public class PlatformEvents extends PlatformAbstractTarget { /** */ private static final int OP_STOP_LOCAL_LISTEN = 14; + /** */ + private static final int OP_REMOTE_QUERY_ASYNC = 15; + + /** */ + private static final int OP_WAIT_FOR_LOCAL_ASYNC = 16; + /** */ private final IgniteEvents events; + /** */ + private final IgniteEvents eventsAsync; + /** */ private final EventResultWriter eventResWriter; @@ -101,6 +110,7 @@ public PlatformEvents(PlatformContext platformCtx, IgniteEvents events) { assert events != null; this.events = events; + eventsAsync = events.withAsync(); eventResWriter = new EventResultWriter(platformCtx); eventColResWriter = new EventCollectionResultWriter(platformCtx); @@ -136,6 +146,21 @@ public PlatformEvents(PlatformContext platformCtx, IgniteEvents events) { return TRUE; + case OP_REMOTE_QUERY_ASYNC: + startRemoteQuery(reader, eventsAsync); + + readAndListenFuture(reader, currentFuture(), eventColResWriter); + + return TRUE; + + case OP_WAIT_FOR_LOCAL_ASYNC: { + startWaitForLocal(reader, eventsAsync); + + readAndListenFuture(reader, currentFuture(), eventResWriter); + + return TRUE; + } + default: return super.processInStreamOutLong(type, reader); } @@ -159,13 +184,7 @@ public PlatformEvents(PlatformContext platformCtx, IgniteEvents events) { } case OP_WAIT_FOR_LOCAL: { - boolean hasFilter = reader.readBoolean(); - - IgnitePredicate pred = hasFilter ? localFilter(reader.readLong()) : null; - - int[] eventTypes = readEventTypes(reader); - - EventAdapter result = (EventAdapter) events.waitForLocal(pred, eventTypes); + EventAdapter result = startWaitForLocal(reader, events); platformCtx.writeEvent(writer, result); @@ -203,24 +222,9 @@ public PlatformEvents(PlatformContext platformCtx, IgniteEvents events) { } case OP_REMOTE_QUERY: { - Object pred = reader.readObjectDetached(); + Collection result = startRemoteQuery(reader, events); - long timeout = reader.readLong(); - - int[] types = readEventTypes(reader); - - PlatformEventFilterListener filter = platformCtx.createRemoteEventFilter(pred, types); - - Collection result = events.remoteQuery(filter, timeout); - - if (result == null) - writer.writeInt(-1); - else { - writer.writeInt(result.size()); - - for (Event e : result) - platformCtx.writeEvent(writer, e); - } + eventColResWriter.write(writer, result, null); break; } @@ -230,6 +234,42 @@ public PlatformEvents(PlatformContext platformCtx, IgniteEvents events) { } } + /** + * Starts the waitForLocal. + * + * @param reader Reader + * @param events Events. + * @return Result. + */ + private EventAdapter startWaitForLocal(BinaryRawReaderEx reader, IgniteEvents events) { + Long filterHnd = reader.readObject(); + + IgnitePredicate filter = filterHnd != null ? localFilter(filterHnd) : null; + + int[] eventTypes = readEventTypes(reader); + + return (EventAdapter) events.waitForLocal(filter, eventTypes); + } + + /** + * Starts the remote query. + * + * @param reader Reader. + * @param events Events. + * @return Result. + */ + private Collection startRemoteQuery(BinaryRawReaderEx reader, IgniteEvents events) { + Object pred = reader.readObjectDetached(); + + long timeout = reader.readLong(); + + int[] types = readEventTypes(reader); + + PlatformEventFilterListener filter = platformCtx.createRemoteEventFilter(pred, types); + + return events.remoteQuery(filter, timeout); + } + /** {@inheritDoc} */ @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException { switch (type) { @@ -271,7 +311,7 @@ public PlatformEvents(PlatformContext platformCtx, IgniteEvents events) { /** {@inheritDoc} */ @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException { - return ((IgniteFutureImpl)events.future()).internalFuture(); + return ((IgniteFutureImpl)eventsAsync.future()).internalFuture(); } /** {@inheritDoc} */ @@ -381,12 +421,17 @@ public EventCollectionResultWriter(PlatformContext platformCtx) { /** */ @SuppressWarnings("unchecked") @Override public void write(BinaryRawWriterEx writer, Object obj, Throwable err) { - Collection events = (Collection)obj; + Collection events = (Collection)obj; - writer.writeInt(events.size()); + if (obj != null) { + writer.writeInt(events.size()); - for (EventAdapter e : events) - platformCtx.writeEvent(writer, e); + for (Event e : events) + platformCtx.writeEvent(writer, e); + } + else { + writer.writeInt(-1); + } } /** */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java index 1b05ecaa68ca6..216427a38439f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java @@ -58,9 +58,18 @@ public class PlatformMessaging extends PlatformAbstractTarget { /** */ public static final int OP_WITH_ASYNC = 8; + /** */ + public static final int OP_REMOTE_LISTEN_ASYNC = 9; + + /** */ + public static final int OP_STOP_REMOTE_LISTEN_ASYNC = 10; + /** */ private final IgniteMessaging messaging; + /** */ + private final IgniteMessaging messagingAsync; + /** * Ctor. * @@ -73,6 +82,7 @@ public PlatformMessaging(PlatformContext platformCtx, IgniteMessaging messaging) assert messaging != null; this.messaging = messaging; + messagingAsync = messaging.withAsync(); } /** {@inheritDoc} */ @@ -120,6 +130,18 @@ public PlatformMessaging(PlatformContext platformCtx, IgniteMessaging messaging) return TRUE; } + case OP_REMOTE_LISTEN_ASYNC: { + startRemoteListen(reader, messagingAsync); + + return readAndListenFuture(reader); + } + + case OP_STOP_REMOTE_LISTEN_ASYNC: { + messagingAsync.stopRemoteListen(reader.readUuid()); + + return readAndListenFuture(reader); + } + default: return super.processInStreamOutLong(type, reader); } @@ -131,17 +153,7 @@ public PlatformMessaging(PlatformContext platformCtx, IgniteMessaging messaging) throws IgniteCheckedException { switch (type) { case OP_REMOTE_LISTEN:{ - Object nativeFilter = reader.readObjectDetached(); - - long ptr = reader.readLong(); // interop pointer - - Object topic = reader.readObjectDetached(); - - PlatformMessageFilter filter = platformCtx.createRemoteMessageFilter(nativeFilter, ptr); - - UUID listenId = messaging.remoteListen(topic, filter); - - writer.writeUuid(listenId); + writer.writeUuid(startRemoteListen(reader, messaging)); break; } @@ -151,9 +163,26 @@ public PlatformMessaging(PlatformContext platformCtx, IgniteMessaging messaging) } } + /** + * Starts the remote listener. + * @param reader Reader. + * @return Listen id. + */ + private UUID startRemoteListen(BinaryRawReaderEx reader, IgniteMessaging messaging) { + Object nativeFilter = reader.readObjectDetached(); + + long ptr = reader.readLong(); // interop pointer + + Object topic = reader.readObjectDetached(); + + PlatformMessageFilter filter = platformCtx.createRemoteMessageFilter(nativeFilter, ptr); + + return messaging.remoteListen(topic, filter); + } + /** {@inheritDoc} */ @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException { - return ((IgniteFutureImpl)messaging.future()).internalFuture(); + return ((IgniteFutureImpl)messagingAsync.future()).internalFuture(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java index 7aaf5970a9118..58989793800fb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java @@ -81,6 +81,18 @@ public class PlatformServices extends PlatformAbstractTarget { /** */ private static final int OP_CANCEL_ALL = 10; + /** */ + private static final int OP_DOTNET_DEPLOY_ASYNC = 11; + + /** */ + private static final int OP_DOTNET_DEPLOY_MULTIPLE_ASYNC = 12; + + /** */ + private static final int OP_CANCEL_ASYNC = 13; + + /** */ + private static final int OP_CANCEL_ALL_ASYNC = 14; + /** */ private static final byte PLATFORM_JAVA = 0; @@ -94,6 +106,9 @@ public class PlatformServices extends PlatformAbstractTarget { /** */ private final IgniteServices services; + /** */ + private final IgniteServices servicesAsync; + /** Server keep binary flag. */ private final boolean srvKeepBinary; @@ -110,6 +125,7 @@ public PlatformServices(PlatformContext platformCtx, IgniteServices services, bo assert services != null; this.services = services; + servicesAsync = services.withAsync(); this.srvKeepBinary = srvKeepBinary; } @@ -132,43 +148,45 @@ private ServiceDescriptor findDescriptor(String name) { throws IgniteCheckedException { switch (type) { case OP_DOTNET_DEPLOY: { - ServiceConfiguration cfg = new ServiceConfiguration(); + dotnetDeploy(reader, services); - cfg.setName(reader.readString()); - cfg.setService(new PlatformDotNetServiceImpl(reader.readObjectDetached(), platformCtx, srvKeepBinary)); - cfg.setTotalCount(reader.readInt()); - cfg.setMaxPerNodeCount(reader.readInt()); - cfg.setCacheName(reader.readString()); - cfg.setAffinityKey(reader.readObjectDetached()); + return TRUE; + } - Object filter = reader.readObjectDetached(); + case OP_DOTNET_DEPLOY_ASYNC: { + dotnetDeploy(reader, servicesAsync); - if (filter != null) - cfg.setNodeFilter(platformCtx.createClusterNodeFilter(filter)); + return readAndListenFuture(reader); + } - services.deploy(cfg); + case OP_DOTNET_DEPLOY_MULTIPLE: { + dotnetDeployMultiple(reader, services); return TRUE; } - case OP_DOTNET_DEPLOY_MULTIPLE: { - String name = reader.readString(); - Object svc = reader.readObjectDetached(); - int totalCnt = reader.readInt(); - int maxPerNodeCnt = reader.readInt(); + case OP_DOTNET_DEPLOY_MULTIPLE_ASYNC: { + dotnetDeployMultiple(reader, servicesAsync); + + return readAndListenFuture(reader); + } - services.deployMultiple(name, new PlatformDotNetServiceImpl(svc, platformCtx, srvKeepBinary), - totalCnt, maxPerNodeCnt); + case OP_CANCEL: { + services.cancel(reader.readString()); return TRUE; } - case OP_CANCEL: { - String name = reader.readString(); + case OP_CANCEL_ASYNC: { + servicesAsync.cancel(reader.readString()); - services.cancel(name); + return readAndListenFuture(reader); + } - return TRUE; + case OP_CANCEL_ALL_ASYNC: { + servicesAsync.cancelAll(); + + return readAndListenFuture(reader); } default: @@ -334,7 +352,41 @@ private ServiceDescriptor findDescriptor(String name) { /** {@inheritDoc} */ @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException { - return ((IgniteFutureImpl)services.future()).internalFuture(); + return ((IgniteFutureImpl)servicesAsync.future()).internalFuture(); + } + + /** + * Deploys multiple dotnet services. + */ + private void dotnetDeployMultiple(BinaryRawReaderEx reader, IgniteServices services) { + String name = reader.readString(); + Object svc = reader.readObjectDetached(); + int totalCnt = reader.readInt(); + int maxPerNodeCnt = reader.readInt(); + + services.deployMultiple(name, new PlatformDotNetServiceImpl(svc, platformCtx, srvKeepBinary), + totalCnt, maxPerNodeCnt); + } + + /** + * Deploys dotnet service. + */ + private void dotnetDeploy(BinaryRawReaderEx reader, IgniteServices services) { + ServiceConfiguration cfg = new ServiceConfiguration(); + + cfg.setName(reader.readString()); + cfg.setService(new PlatformDotNetServiceImpl(reader.readObjectDetached(), platformCtx, srvKeepBinary)); + cfg.setTotalCount(reader.readInt()); + cfg.setMaxPerNodeCount(reader.readInt()); + cfg.setCacheName(reader.readString()); + cfg.setAffinityKey(reader.readObjectDetached()); + + Object filter = reader.readObjectDetached(); + + if (filter != null) + cfg.setNodeFilter(platformCtx.createClusterNodeFilter(filter)); + + services.deploy(cfg); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java index 339937c98fc67..9c8ad503b9eb8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java @@ -17,9 +17,6 @@ package org.apache.ignite.internal.processors.platform.transactions; -import java.sql.Timestamp; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicLong; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteTransactions; import org.apache.ignite.configuration.TransactionConfiguration; @@ -27,7 +24,6 @@ import org.apache.ignite.internal.binary.BinaryRawWriterEx; import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; -import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils; import org.apache.ignite.internal.util.GridConcurrentFactory; import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.lang.IgniteFuture; @@ -36,6 +32,10 @@ import org.apache.ignite.transactions.TransactionIsolation; import org.apache.ignite.transactions.TransactionMetrics; +import java.sql.Timestamp; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; + /** * Native transaction wrapper implementation. */ @@ -94,21 +94,6 @@ public PlatformTransactions(PlatformContext platformCtx) { txs = platformCtx.kernalContext().grid().transactions(); } - /** - * Listens to the transaction future and notifies .NET int future. - */ - private void listenAndNotifyIntFuture(final long futId, final Transaction asyncTx) { - IgniteFuture fut = asyncTx.future().chain(new C1() { - private static final long serialVersionUID = 0L; - - @Override public Object apply(IgniteFuture fut) { - return null; - } - }); - - PlatformFutureUtils.listen(platformCtx, fut, futId, PlatformFutureUtils.TYP_OBJ, this); - } - /** * Register transaction. * @@ -138,10 +123,9 @@ private void unregisterTx(long id) { /** * @param id Transaction ID. - * @throws org.apache.ignite.IgniteCheckedException In case of error. * @return Transaction state. */ - private int txClose(long id) throws IgniteCheckedException { + private int txClose(long id) { Transaction tx = tx(id); try { @@ -209,7 +193,6 @@ private Transaction tx(long id) { /** {@inheritDoc} */ @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException { long txId = reader.readLong(); - long futId = reader.readLong(); final Transaction asyncTx = (Transaction)tx(txId).withAsync(); @@ -229,7 +212,16 @@ private Transaction tx(long id) { return super.processInStreamOutLong(type, reader); } - listenAndNotifyIntFuture(futId, asyncTx); + // Future result is the tx itself, we do not want to return it to the platform. + IgniteFuture fut = asyncTx.future().chain(new C1() { + private static final long serialVersionUID = 0L; + + @Override public Object apply(IgniteFuture fut) { + return null; + } + }); + + readAndListenFuture(reader, fut); return TRUE; } diff --git a/modules/platforms/cpp/jni/include/ignite/jni/exports.h b/modules/platforms/cpp/jni/include/ignite/jni/exports.h index 276c06aaabc0e..586c389262a8f 100644 --- a/modules/platforms/cpp/jni/include/ignite/jni/exports.h +++ b/modules/platforms/cpp/jni/include/ignite/jni/exports.h @@ -68,8 +68,6 @@ extern "C" { void* IGNITE_CALL IgniteTargetOutObject(gcj::JniContext* ctx, void* obj, int opType); void IGNITE_CALL IgniteTargetListenFuture(gcj::JniContext* ctx, void* obj, long long futId, int typ); void IGNITE_CALL IgniteTargetListenFutureForOperation(gcj::JniContext* ctx, void* obj, long long futId, int typ, int opId); - void* IGNITE_CALL IgniteTargetListenFutureAndGet(gcj::JniContext* ctx, void* obj, long long futId, int typ); - void* IGNITE_CALL IgniteTargetListenFutureForOperationAndGet(gcj::JniContext* ctx, void* obj, long long futId, int typ, int opId); void* IGNITE_CALL IgniteAcquire(gcj::JniContext* ctx, void* obj); void IGNITE_CALL IgniteRelease(void* obj); diff --git a/modules/platforms/cpp/jni/include/ignite/jni/java.h b/modules/platforms/cpp/jni/include/ignite/jni/java.h index 9e5bcaed6b10a..4c79a61342b93 100644 --- a/modules/platforms/cpp/jni/include/ignite/jni/java.h +++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h @@ -282,8 +282,6 @@ namespace ignite jmethodID m_PlatformTarget_inObjectStreamOutObjectStream; jmethodID m_PlatformTarget_listenFuture; jmethodID m_PlatformTarget_listenFutureForOperation; - jmethodID m_PlatformTarget_listenFutureAndGet; - jmethodID m_PlatformTarget_listenFutureForOperationAndGet; jclass c_PlatformUtils; jmethodID m_PlatformUtils_reallocate; @@ -465,8 +463,6 @@ namespace ignite jobject TargetOutObject(jobject obj, int opType, JniErrorInfo* errInfo = NULL); void TargetListenFuture(jobject obj, long long futId, int typ); void TargetListenFutureForOperation(jobject obj, long long futId, int typ, int opId); - void* TargetListenFutureAndGet(jobject obj, long long futId, int typ); - void* TargetListenFutureForOperationAndGet(jobject obj, long long futId, int typ, int opId); jobject CacheOutOpQueryCursor(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); jobject CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr); diff --git a/modules/platforms/cpp/jni/project/vs/module.def b/modules/platforms/cpp/jni/project/vs/module.def index 2e76bf7425c17..d9bc4113ecd7e 100644 --- a/modules/platforms/cpp/jni/project/vs/module.def +++ b/modules/platforms/cpp/jni/project/vs/module.def @@ -39,8 +39,6 @@ IgniteProcessorExtensions @97 IgniteProcessorAtomicLong @98 IgniteListenableCancel @110 IgniteListenableIsCancelled @111 -IgniteTargetListenFutureAndGet @112 -IgniteTargetListenFutureForOperationAndGet @113 IgniteProcessorCreateCacheFromConfig @114 IgniteProcessorGetOrCreateCacheFromConfig @115 IgniteProcessorGetIgniteConfiguration @116 diff --git a/modules/platforms/cpp/jni/src/exports.cpp b/modules/platforms/cpp/jni/src/exports.cpp index ab569b053281f..ee2f5c781ab56 100644 --- a/modules/platforms/cpp/jni/src/exports.cpp +++ b/modules/platforms/cpp/jni/src/exports.cpp @@ -190,14 +190,6 @@ extern "C" { ctx->TargetListenFutureForOperation(static_cast(obj), futId, typ, opId); } - void* IGNITE_CALL IgniteTargetListenFutureAndGet(gcj::JniContext* ctx, void* obj, long long futId, int typ) { - return ctx->TargetListenFutureAndGet(static_cast(obj), futId, typ); - } - - void* IGNITE_CALL IgniteTargetListenFutureForOperationAndGet(gcj::JniContext* ctx, void* obj, long long futId, int typ, int opId) { - return ctx->TargetListenFutureForOperationAndGet(static_cast(obj), futId, typ, opId); - } - void* IGNITE_CALL IgniteAcquire(gcj::JniContext* ctx, void* obj) { return ctx->Acquire(static_cast(obj)); } diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp index fbfb17e0a109a..c1efbe2df5b9b 100644 --- a/modules/platforms/cpp/jni/src/java.cpp +++ b/modules/platforms/cpp/jni/src/java.cpp @@ -231,8 +231,6 @@ namespace ignite JniMethod M_PLATFORM_TARGET_OUT_OBJECT = JniMethod("outObject", "(I)Ljava/lang/Object;", false); JniMethod M_PLATFORM_TARGET_LISTEN_FUTURE = JniMethod("listenFuture", "(JI)V", false); JniMethod M_PLATFORM_TARGET_LISTEN_FOR_OPERATION = JniMethod("listenFutureForOperation", "(JII)V", false); - JniMethod M_PLATFORM_TARGET_LISTEN_FUTURE_AND_GET = JniMethod("listenFutureAndGet", "(JI)Lorg/apache/ignite/internal/processors/platform/utils/PlatformListenable;", false); - JniMethod M_PLATFORM_TARGET_LISTEN_FOR_OPERATION_AND_GET = JniMethod("listenFutureForOperationAndGet", "(JII)Lorg/apache/ignite/internal/processors/platform/utils/PlatformListenable;", false); const char* C_PLATFORM_CALLBACK_UTILS = "org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils"; @@ -551,8 +549,6 @@ namespace ignite m_PlatformTarget_inObjectStreamOutObjectStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_OBJECT_STREAM); m_PlatformTarget_listenFuture = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_LISTEN_FUTURE); m_PlatformTarget_listenFutureForOperation = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_LISTEN_FOR_OPERATION); - m_PlatformTarget_listenFutureAndGet = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_LISTEN_FUTURE_AND_GET); - m_PlatformTarget_listenFutureForOperationAndGet = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_LISTEN_FOR_OPERATION_AND_GET); c_PlatformUtils = FindClass(env, C_PLATFORM_UTILS); m_PlatformUtils_reallocate = FindMethod(env, c_PlatformUtils, M_PLATFORM_UTILS_REALLOC); @@ -1439,28 +1435,6 @@ namespace ignite ExceptionCheck(env); } - void* JniContext::TargetListenFutureAndGet(jobject obj, long long futId, int typ) { - JNIEnv* env = Attach(); - - jobject res = env->CallObjectMethod(obj, - jvm->GetMembers().m_PlatformTarget_listenFutureAndGet, futId, typ); - - ExceptionCheck(env); - - return LocalToGlobal(env, res); - } - - void* JniContext::TargetListenFutureForOperationAndGet(jobject obj, long long futId, int typ, int opId) { - JNIEnv* env = Attach(); - - jobject res = env->CallObjectMethod(obj, - jvm->GetMembers().m_PlatformTarget_listenFutureForOperationAndGet, futId, typ, opId); - - ExceptionCheck(env); - - return LocalToGlobal(env, res); - } - jobject JniContext::CacheOutOpQueryCursor(jobject obj, int type, long long memPtr, JniErrorInfo* err) { JNIEnv* env = Attach(); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs index d9adc06f3a1a6..b1cf611abe11f 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs @@ -20,7 +20,6 @@ namespace Apache.Ignite.Core.Impl.Cache using System; using System.Collections; using System.Collections.Generic; - using System.Diagnostics; using System.Diagnostics.CodeAnalysis; using System.Threading.Tasks; using Apache.Ignite.Core.Binary; @@ -35,7 +34,6 @@ namespace Apache.Ignite.Core.Impl.Cache using Apache.Ignite.Core.Impl.Cache.Query.Continuous; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Impl.Unmanaged; - using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils; /// /// Native cache wrapper. @@ -61,15 +59,9 @@ internal class CacheImpl : PlatformTarget, ICache, ICacheInterna /** Flag: keep binary. */ private readonly bool _flagKeepBinary; - /** Flag: async mode.*/ - private readonly bool _flagAsync; - /** Flag: no-retries.*/ private readonly bool _flagNoRetries; - /** Async instance. */ - private readonly Lazy> _asyncInstance; - /// /// Constructor. /// @@ -78,72 +70,72 @@ internal class CacheImpl : PlatformTarget, ICache, ICacheInterna /// Marshaller. /// Skip store flag. /// Keep binary flag. - /// Async mode flag. /// No-retries mode flag. public CacheImpl(Ignite grid, IUnmanagedTarget target, Marshaller marsh, - bool flagSkipStore, bool flagKeepBinary, bool flagAsync, bool flagNoRetries) : base(target, marsh) + bool flagSkipStore, bool flagKeepBinary, bool flagNoRetries) : base(target, marsh) { _ignite = grid; _flagSkipStore = flagSkipStore; _flagKeepBinary = flagKeepBinary; - _flagAsync = flagAsync; _flagNoRetries = flagNoRetries; + } - _asyncInstance = new Lazy>(WithAsync); + /** */ + public IIgnite Ignite + { + get { return _ignite; } } /// - /// Returns an instance with async mode enabled. + /// Performs async operation. /// - private CacheImpl WithAsync() + private Task DoOutOpAsync(CacheOp op, T1 val1) { - var target = DoOutOpObject((int) CacheOp.WithAsync); - - return new CacheImpl(_ignite, target, Marshaller, _flagSkipStore, _flagKeepBinary, - true, _flagNoRetries); + return DoOutOpAsync((int) op, val1); } - /** */ - public IIgnite Ignite + /// + /// Performs async operation. + /// + private Task DoOutOpAsync(CacheOp op, T1 val1) { - get { return _ignite; } + return DoOutOpAsync((int) op, val1); } - /** */ - private bool IsAsync + /// + /// Performs async operation. + /// + private Task DoOutOpAsync(CacheOp op, T1 val1, T2 val2) { - get { return _flagAsync; } + return DoOutOpAsync((int) op, val1, val2); } /// - /// Gets and resets task for previous asynchronous operation. + /// Performs async operation. /// - /// The last async op id. - /// - /// Task for previous asynchronous operation. - /// - private Task GetTask(CacheOp lastAsyncOp) + private Task DoOutOpAsync(CacheOp op, T1 val1, T2 val2) { - return GetTask(lastAsyncOp); + return DoOutOpAsync((int) op, val1, val2); } /// - /// Gets and resets task for previous asynchronous operation. + /// Performs async operation. /// - /// The type of the result. - /// The last async op id. - /// The converter. - /// - /// Task for previous asynchronous operation. - /// - private Task GetTask(CacheOp lastAsyncOp, Func converter = null) + private Task DoOutOpAsync(CacheOp op, Action writeAction = null) { - Debug.Assert(_flagAsync); + return DoOutOpAsync(op, writeAction); + } - return GetFuture((futId, futTypeId) => UU.TargetListenFutureForOperation(Target, futId, futTypeId, - (int) lastAsyncOp), _flagKeepBinary, converter).Task; + /// + /// Performs async operation. + /// + private Task DoOutOpAsync(CacheOp op, Action writeAction = null, + Func convertFunc = null) + { + return DoOutOpAsync((int)op, writeAction, IsKeepBinary, convertFunc); } + /** */ public string Name { @@ -169,7 +161,7 @@ public bool IsEmpty() return this; return new CacheImpl(_ignite, DoOutOpObject((int) CacheOp.WithSkipStore), Marshaller, - true, _flagKeepBinary, _flagAsync, true); + true, _flagKeepBinary, true); } /// @@ -193,7 +185,7 @@ public bool IsEmpty() } return new CacheImpl(_ignite, DoOutOpObject((int) CacheOp.WithKeepBinary), Marshaller, - _flagSkipStore, true, _flagAsync, _flagNoRetries); + _flagSkipStore, true, _flagNoRetries); } /** */ @@ -212,7 +204,7 @@ public bool IsEmpty() w.WriteLong(access); }); - return new CacheImpl(_ignite, cache0, Marshaller, _flagSkipStore, _flagKeepBinary, _flagAsync, _flagNoRetries); + return new CacheImpl(_ignite, cache0, Marshaller, _flagSkipStore, _flagKeepBinary, _flagNoRetries); } /// @@ -244,50 +236,43 @@ public bool IsKeepBinary /** */ public void LoadCache(ICacheEntryFilter p, params object[] args) { - LoadCache0(p, args, (int)CacheOp.LoadCache); + DoOutInOpX((int) CacheOp.LoadCache, writer => WriteLoadCacheData(writer, p, args), ReadException); } /** */ public Task LoadCacheAsync(ICacheEntryFilter p, params object[] args) { - AsyncInstance.LoadCache(p, args); - - return AsyncInstance.GetTask(CacheOp.LoadCache); + return DoOutOpAsync(CacheOp.LoadCacheAsync, writer => WriteLoadCacheData(writer, p, args)); } /** */ public void LocalLoadCache(ICacheEntryFilter p, params object[] args) { - LoadCache0(p, args, (int)CacheOp.LocLoadCache); + DoOutInOpX((int) CacheOp.LocLoadCache, writer => WriteLoadCacheData(writer, p, args), ReadException); } /** */ public Task LocalLoadCacheAsync(ICacheEntryFilter p, params object[] args) { - AsyncInstance.LocalLoadCache(p, args); - - return AsyncInstance.GetTask(CacheOp.LocLoadCache); + return DoOutOpAsync(CacheOp.LocLoadCacheAsync, writer => WriteLoadCacheData(writer, p, args)); } /// - /// Loads the cache. + /// Writes the load cache data to the writer. /// - private void LoadCache0(ICacheEntryFilter p, object[] args, int opId) + private void WriteLoadCacheData(IBinaryRawWriter writer, ICacheEntryFilter p, object[] args) { - DoOutInOpX(opId, writer => + if (p != null) { - if (p != null) - { - var p0 = new CacheEntryFilterHolder(p, (k, v) => p.Invoke(new CacheEntry((TK) k, (TV) v)), - Marshaller, IsKeepBinary); + var p0 = new CacheEntryFilterHolder(p, (k, v) => p.Invoke(new CacheEntry((TK) k, (TV) v)), + Marshaller, IsKeepBinary); - writer.WriteObject(p0); - } - else - writer.WriteObject(null); + writer.WriteObject(p0); + } + else + writer.WriteObject(null); - writer.WriteArray(args); - }, ReadException); + writer.WriteArray(args); } /** */ @@ -299,12 +284,11 @@ public void LoadAll(IEnumerable keys, bool replaceExistingValues) /** */ public Task LoadAllAsync(IEnumerable keys, bool replaceExistingValues) { - return GetFuture((futId, futTyp) => DoOutOp(CacheOp.LoadAll, writer => + return DoOutOpAsync(CacheOp.LoadAll, writer => { - writer.WriteLong(futId); writer.WriteBoolean(replaceExistingValues); WriteEnumerable(writer, keys); - })).Task; + }); } /** */ @@ -318,9 +302,9 @@ public bool ContainsKey(TK key) /** */ public Task ContainsKeyAsync(TK key) { - AsyncInstance.ContainsKey(key); + IgniteArgumentCheck.NotNull(key, "key"); - return AsyncInstance.GetTask(CacheOp.ContainsKey); + return DoOutOpAsync(CacheOp.ContainsKeyAsync, key); } /** */ @@ -334,9 +318,9 @@ public bool ContainsKeys(IEnumerable keys) /** */ public Task ContainsKeysAsync(IEnumerable keys) { - AsyncInstance.ContainsKeys(keys); + IgniteArgumentCheck.NotNull(keys, "keys"); - return AsyncInstance.GetTask(CacheOp.ContainsKeys); + return DoOutOpAsync(CacheOp.ContainsKeysAsync, writer => WriteEnumerable(writer, keys)); } /** */ @@ -376,16 +360,10 @@ public bool TryLocalPeek(TK key, out TV value, params CachePeekMode[] modes) { get { - if (IsAsync) - throw new InvalidOperationException("Indexer can't be used in async mode."); - return Get(key); } set { - if (IsAsync) - throw new InvalidOperationException("Indexer can't be used in async mode."); - Put(key, value); } } @@ -399,26 +377,19 @@ public TV Get(TK key) w => w.Write(key), (stream, res) => { - if (res == True) // Not null - { - Debug.Assert(!IsAsync); - - return Unmarshal(stream); - } - - if (!IsAsync) + if (res != True) throw GetKeyNotFoundException(); - return default(TV); + return Unmarshal(stream); }, ReadException); } /** */ public Task GetAsync(TK key) { - AsyncInstance.Get(key); + IgniteArgumentCheck.NotNull(key, "key"); - return AsyncInstance.GetTask(CacheOp.Get, reader => + return DoOutOpAsync(CacheOp.GetAsync, w => w.WriteObject(key), reader => { if (reader != null) return reader.ReadObject(); @@ -432,9 +403,6 @@ public bool TryGet(TK key, out TV value) { IgniteArgumentCheck.NotNull(key, "key"); - if (IsAsync) - throw new InvalidOperationException("TryGet can't be used in async mode."); - var res = DoOutInOpNullable(CacheOp.Get, key); value = res.Value; @@ -447,9 +415,7 @@ public Task> TryGetAsync(TK key) { IgniteArgumentCheck.NotNull(key, "key"); - AsyncInstance.Get(key); - - return AsyncInstance.GetTask(CacheOp.Get, GetCacheResult); + return DoOutOpAsync(CacheOp.GetAsync, w => w.WriteObject(key), reader => GetCacheResult(reader)); } /** */ @@ -466,9 +432,9 @@ public Task> TryGetAsync(TK key) /** */ public Task> GetAllAsync(IEnumerable keys) { - AsyncInstance.GetAll(keys); + IgniteArgumentCheck.NotNull(keys, "keys"); - return AsyncInstance.GetTask(CacheOp.GetAll, r => r == null ? null : ReadGetAllDictionary(r)); + return DoOutOpAsync(CacheOp.GetAllAsync, w => WriteEnumerable(w, keys), r => ReadGetAllDictionary(r)); } /** */ @@ -484,16 +450,16 @@ public void Put(TK key, TV val) /** */ public Task PutAsync(TK key, TV val) { - AsyncInstance.Put(key, val); + IgniteArgumentCheck.NotNull(key, "key"); + IgniteArgumentCheck.NotNull(val, "val"); - return AsyncInstance.GetTask(CacheOp.Put); + return DoOutOpAsync(CacheOp.PutAsync, key, val); } /** */ public CacheResult GetAndPut(TK key, TV val) { IgniteArgumentCheck.NotNull(key, "key"); - IgniteArgumentCheck.NotNull(val, "val"); return DoOutInOpNullable(CacheOp.GetAndPut, key, val); @@ -502,9 +468,14 @@ public CacheResult GetAndPut(TK key, TV val) /** */ public Task> GetAndPutAsync(TK key, TV val) { - AsyncInstance.GetAndPut(key, val); + IgniteArgumentCheck.NotNull(key, "key"); + IgniteArgumentCheck.NotNull(val, "val"); - return AsyncInstance.GetTask(CacheOp.GetAndPut, GetCacheResult); + return DoOutOpAsync(CacheOp.GetAndPutAsync, w => + { + w.WriteObject(key); + w.WriteObject(val); + }, r => GetCacheResult(r)); } /** */ @@ -520,9 +491,14 @@ public CacheResult GetAndReplace(TK key, TV val) /** */ public Task> GetAndReplaceAsync(TK key, TV val) { - AsyncInstance.GetAndReplace(key, val); + IgniteArgumentCheck.NotNull(key, "key"); + IgniteArgumentCheck.NotNull(val, "val"); - return AsyncInstance.GetTask(CacheOp.GetAndReplace, GetCacheResult); + return DoOutOpAsync(CacheOp.GetAndReplaceAsync, w => + { + w.WriteObject(key); + w.WriteObject(val); + }, r => GetCacheResult(r)); } /** */ @@ -536,9 +512,9 @@ public CacheResult GetAndRemove(TK key) /** */ public Task> GetAndRemoveAsync(TK key) { - AsyncInstance.GetAndRemove(key); + IgniteArgumentCheck.NotNull(key, "key"); - return AsyncInstance.GetTask(CacheOp.GetAndRemove, GetCacheResult); + return DoOutOpAsync(CacheOp.GetAndRemoveAsync, w => w.WriteObject(key), r => GetCacheResult(r)); } /** */ @@ -554,9 +530,10 @@ public bool PutIfAbsent(TK key, TV val) /** */ public Task PutIfAbsentAsync(TK key, TV val) { - AsyncInstance.PutIfAbsent(key, val); + IgniteArgumentCheck.NotNull(key, "key"); + IgniteArgumentCheck.NotNull(val, "val"); - return AsyncInstance.GetTask(CacheOp.PutIfAbsent); + return DoOutOpAsync(CacheOp.PutIfAbsentAsync, key, val); } /** */ @@ -572,9 +549,14 @@ public CacheResult GetAndPutIfAbsent(TK key, TV val) /** */ public Task> GetAndPutIfAbsentAsync(TK key, TV val) { - AsyncInstance.GetAndPutIfAbsent(key, val); + IgniteArgumentCheck.NotNull(key, "key"); + IgniteArgumentCheck.NotNull(val, "val"); - return AsyncInstance.GetTask(CacheOp.GetAndPutIfAbsent, GetCacheResult); + return DoOutOpAsync(CacheOp.GetAndPutIfAbsentAsync, w => + { + w.WriteObject(key); + w.WriteObject(val); + }, r => GetCacheResult(r)); } /** */ @@ -590,9 +572,10 @@ public bool Replace(TK key, TV val) /** */ public Task ReplaceAsync(TK key, TV val) { - AsyncInstance.Replace(key, val); + IgniteArgumentCheck.NotNull(key, "key"); + IgniteArgumentCheck.NotNull(val, "val"); - return AsyncInstance.GetTask(CacheOp.Replace2); + return DoOutOpAsync(CacheOp.Replace2Async, key, val); } /** */ @@ -610,9 +593,16 @@ public bool Replace(TK key, TV oldVal, TV newVal) /** */ public Task ReplaceAsync(TK key, TV oldVal, TV newVal) { - AsyncInstance.Replace(key, oldVal, newVal); + IgniteArgumentCheck.NotNull(key, "key"); + IgniteArgumentCheck.NotNull(oldVal, "oldVal"); + IgniteArgumentCheck.NotNull(newVal, "newVal"); - return AsyncInstance.GetTask(CacheOp.Replace3); + return DoOutOpAsync(CacheOp.Replace3Async, w => + { + w.WriteObject(key); + w.WriteObject(oldVal); + w.WriteObject(newVal); + }); } /** */ @@ -626,9 +616,9 @@ public void PutAll(IDictionary vals) /** */ public Task PutAllAsync(IDictionary vals) { - AsyncInstance.PutAll(vals); + IgniteArgumentCheck.NotNull(vals, "vals"); - return AsyncInstance.GetTask(CacheOp.PutAll); + return DoOutOpAsync(CacheOp.PutAllAsync, writer => WriteDictionary(writer, vals)); } /** */ @@ -648,9 +638,7 @@ public void Clear() /** */ public Task ClearAsync() { - AsyncInstance.Clear(); - - return AsyncInstance.GetTask(); + return DoOutOpAsync(CacheOp.ClearCacheAsync); } /** */ @@ -664,9 +652,9 @@ public void Clear(TK key) /** */ public Task ClearAsync(TK key) { - AsyncInstance.Clear(key); + IgniteArgumentCheck.NotNull(key, "key"); - return AsyncInstance.GetTask(CacheOp.Clear); + return DoOutOpAsync(CacheOp.ClearAsync, key); } /** */ @@ -680,9 +668,9 @@ public void ClearAll(IEnumerable keys) /** */ public Task ClearAllAsync(IEnumerable keys) { - AsyncInstance.ClearAll(keys); + IgniteArgumentCheck.NotNull(keys, "keys"); - return AsyncInstance.GetTask(CacheOp.ClearAll); + return DoOutOpAsync(CacheOp.ClearAllAsync, writer => WriteEnumerable(writer, keys)); } /** */ @@ -712,16 +700,15 @@ public bool Remove(TK key) /** */ public Task RemoveAsync(TK key) { - AsyncInstance.Remove(key); + IgniteArgumentCheck.NotNull(key, "key"); - return AsyncInstance.GetTask(CacheOp.RemoveObj); + return DoOutOpAsync(CacheOp.RemoveObjAsync, key); } /** */ public bool Remove(TK key, TV val) { IgniteArgumentCheck.NotNull(key, "key"); - IgniteArgumentCheck.NotNull(val, "val"); return DoOutOp(CacheOp.RemoveBool, key, val); @@ -730,9 +717,10 @@ public bool Remove(TK key, TV val) /** */ public Task RemoveAsync(TK key, TV val) { - AsyncInstance.Remove(key, val); + IgniteArgumentCheck.NotNull(key, "key"); + IgniteArgumentCheck.NotNull(val, "val"); - return AsyncInstance.GetTask(CacheOp.RemoveBool); + return DoOutOpAsync(CacheOp.RemoveBoolAsync, key, val); } /** */ @@ -746,9 +734,9 @@ public void RemoveAll(IEnumerable keys) /** */ public Task RemoveAllAsync(IEnumerable keys) { - AsyncInstance.RemoveAll(keys); + IgniteArgumentCheck.NotNull(keys, "keys"); - return AsyncInstance.GetTask(CacheOp.RemoveAll); + return DoOutOpAsync(CacheOp.RemoveAllAsync, writer => WriteEnumerable(writer, keys)); } /** */ @@ -760,9 +748,7 @@ public void RemoveAll() /** */ public Task RemoveAllAsync() { - AsyncInstance.RemoveAll(); - - return AsyncInstance.GetTask(); + return DoOutOpAsync(CacheOp.RemoveAll2Async); } /** */ @@ -780,9 +766,9 @@ public int GetSize(params CachePeekMode[] modes) /** */ public Task GetSizeAsync(params CachePeekMode[] modes) { - AsyncInstance.GetSize(modes); + var modes0 = EncodePeekModes(modes); - return AsyncInstance.GetTask(); + return DoOutOpAsync(CacheOp.SizeAsync, w => w.WriteInt(modes0)); } /// @@ -831,20 +817,29 @@ public void LocalPromote(IEnumerable keys) /** */ public Task InvokeAsync(TK key, ICacheEntryProcessor processor, TArg arg) { - AsyncInstance.Invoke(key, processor, arg); + IgniteArgumentCheck.NotNull(key, "key"); + IgniteArgumentCheck.NotNull(processor, "processor"); - return AsyncInstance.GetTask(CacheOp.Invoke, r => - { - if (r == null) - return default(TRes); + var holder = new CacheEntryProcessorHolder(processor, arg, + (e, a) => processor.Process((IMutableCacheEntry)e, (TArg)a), typeof(TK), typeof(TV)); - var hasError = r.ReadBoolean(); + return DoOutOpAsync(CacheOp.InvokeAsync, writer => + { + writer.Write(key); + writer.Write(holder); + }, + r => + { + if (r == null) + return default(TRes); - if (hasError) - throw ReadException(r.Stream); + var hasError = r.ReadBoolean(); - return r.ReadObject(); - }); + if (hasError) + throw ReadException(r); + + return r.ReadObject(); + }); } /** */ @@ -864,17 +859,28 @@ public void LocalPromote(IEnumerable keys) WriteEnumerable(writer, keys); writer.Write(holder); }, - (input, res) => res == True ? ReadInvokeAllResults(input) : null, - ReadException); + (input, res) => res == True ? ReadInvokeAllResults(Marshaller.StartUnmarshal(input, IsKeepBinary)): null, ReadException); } /** */ public Task>> InvokeAllAsync(IEnumerable keys, ICacheEntryProcessor processor, TArg arg) { - AsyncInstance.InvokeAll(keys, processor, arg); + IgniteArgumentCheck.NotNull(keys, "keys"); + + IgniteArgumentCheck.NotNull(processor, "processor"); + + var holder = new CacheEntryProcessorHolder(processor, arg, + (e, a) => processor.Process((IMutableCacheEntry)e, (TArg)a), typeof(TK), typeof(TV)); + + return DoOutOpAsync(CacheOp.InvokeAllAsync, + writer => + { + WriteEnumerable(writer, keys); + writer.Write(holder); + }, + input => ReadInvokeAllResults(input)); - return AsyncInstance.GetTask(CacheOp.InvokeAll, reader => ReadInvokeAllResults(reader.Stream)); } /** */ @@ -936,7 +942,7 @@ public ICacheMetrics GetMetrics() /** */ public Task Rebalance() { - return GetFuture((futId, futTyp) => DoOutInOpLong((int) CacheOp.Rebalance, futId)).Task; + return DoOutOpAsync(CacheOp.Rebalance); } /** */ @@ -946,15 +952,7 @@ public Task Rebalance() return this; return new CacheImpl(_ignite, DoOutOpObject((int) CacheOp.WithNoRetries), Marshaller, - _flagSkipStore, _flagKeepBinary, _flagAsync, true); - } - - /// - /// Gets the asynchronous instance. - /// - private CacheImpl AsyncInstance - { - get { return _asyncInstance.Value; } + _flagSkipStore, _flagKeepBinary, true); } #region Queries @@ -1136,11 +1134,11 @@ private static int EncodePeekModes(CachePeekMode[] modes) /// Reads results of InvokeAll operation. /// /// The type of the result. - /// Stream. + /// Stream. /// Results of InvokeAll operation. - private IDictionary> ReadInvokeAllResults(IBinaryStream inStream) + private IDictionary> ReadInvokeAllResults(BinaryReader reader) { - var count = inStream.ReadInt(); + var count = reader.ReadInt(); if (count == -1) return null; @@ -1149,27 +1147,33 @@ private static int EncodePeekModes(CachePeekMode[] modes) for (var i = 0; i < count; i++) { - var key = Unmarshal(inStream); + var key = reader.ReadObject(); - var hasError = inStream.ReadBool(); + var hasError = reader.ReadBoolean(); results[key] = hasError - ? new CacheEntryProcessorResult(ReadException(inStream)) - : new CacheEntryProcessorResult(Unmarshal(inStream)); + ? new CacheEntryProcessorResult(ReadException(reader)) + : new CacheEntryProcessorResult(reader.ReadObject()); } return results; } + /// + /// Reads the exception. + /// + private Exception ReadException(IBinaryStream stream) + { + return ReadException(Marshaller.StartUnmarshal(stream)); + } + /// /// Reads the exception, either in binary wrapper form, or as a pair of strings. /// - /// The stream. + /// The stream. /// Exception. - private Exception ReadException(IBinaryStream inStream) + private Exception ReadException(BinaryReader reader) { - var reader = Marshaller.StartUnmarshal(inStream, _flagKeepBinary); - var item = reader.ReadObject(); var clsName = item as string; @@ -1177,8 +1181,8 @@ private Exception ReadException(IBinaryStream inStream) if (clsName == null) return new CacheEntryProcessorException((Exception) item); - var msg = Unmarshal(inStream); - var trace = Unmarshal(inStream); + var msg = reader.ReadObject(); + var trace = reader.ReadObject(); var inner = reader.ReadBoolean() ? reader.ReadObject() : null; return ExceptionUtils.GetException(_ignite, clsName, msg, trace, reader, inner); @@ -1191,6 +1195,9 @@ private Exception ReadException(IBinaryStream inStream) /// Dictionary. private static IDictionary ReadGetAllDictionary(BinaryReader reader) { + if (reader == null) + return null; + IBinaryStream stream = reader.Stream; if (stream.ReadBool()) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs index a43df385c0fd2..8bf39457e1215 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs @@ -78,6 +78,31 @@ internal enum CacheOp CloseLock = 54, Rebalance = 55, SizeLoc = 56, - Extension = 57 + PutAsync = 57, + ClearCacheAsync = 58, + ClearAllAsync = 59, + RemoveAll2Async = 60, + SizeAsync = 61, + ClearAsync = 62, + LoadCacheAsync = 63, + LocLoadCacheAsync = 64, + PutAllAsync = 65, + RemoveAllAsync = 66, + GetAsync = 67, + ContainsKeyAsync = 68, + ContainsKeysAsync = 69, + RemoveBoolAsync = 70, + RemoveObjAsync = 71, + GetAllAsync = 72, + GetAndPutAsync = 73, + GetAndPutIfAbsentAsync = 74, + GetAndRemoveAsync = 75, + GetAndReplaceAsync = 76, + Replace2Async = 77, + Replace3Async = 78, + InvokeAsync = 79, + InvokeAllAsync = 80, + PutIfAbsentAsync = 81, + Extension = 82 } } \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs index 1b2e2aa914ef8..df68e1cf669cd 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs @@ -25,6 +25,7 @@ namespace Apache.Ignite.Core.Impl.Compute using System.Linq; using System.Runtime.Serialization; using System.Threading; + using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Common; using Apache.Ignite.Core.Compute; @@ -141,12 +142,7 @@ public TReduceRes ExecuteJavaTask(string taskName, object taskArg) try { - TReduceRes res = DoOutInOp(OpExec, writer => - { - WriteTask(writer, taskName, taskArg, nodes); - }); - - return res; + return DoOutInOp(OpExec, writer => WriteTask(writer, taskName, taskArg, nodes)); } finally { @@ -167,18 +163,7 @@ public Future ExecuteJavaTaskAsync(string taskName, obje try { - Future fut = null; - - DoOutInOp(OpExecAsync, writer => - { - WriteTask(writer, taskName, taskArg, nodes); - }, input => - { - fut = GetFuture((futId, futTyp) => - UU.TargetListenFutureAndGet(Target, futId, futTyp), _keepBinary.Value); - }); - - return fut; + return DoOutOpObjectAsync(OpExecAsync, w => WriteTask(w, taskName, taskArg, nodes)); } finally { @@ -625,12 +610,12 @@ private long WriteJob(IComputeJob job, BinaryWriter writer) /// Task name. /// Task arg. /// Nodes. - private void WriteTask(BinaryWriter writer, string taskName, object taskArg, + private void WriteTask(IBinaryRawWriter writer, string taskName, object taskArg, ICollection nodes) { writer.WriteString(taskName); writer.WriteBoolean(_keepBinary.Value); - writer.Write(taskArg); + writer.WriteObject(taskArg); WriteNodeIds(writer, nodes); } @@ -640,7 +625,7 @@ private long WriteJob(IComputeJob job, BinaryWriter writer) /// /// Writer. /// Nodes. - private static void WriteNodeIds(BinaryWriter writer, ICollection nodes) + private static void WriteNodeIds(IBinaryRawWriter writer, ICollection nodes) { if (nodes == null) writer.WriteBoolean(false); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs index 5d1add640466c..bc1b4bbeba1ca 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs @@ -53,10 +53,11 @@ private enum Op EnableLocal = 8, DisableLocal = 9, GetEnabledEvents = 10, - WithAsync = 11, IsEnabled = 12, LocalListen = 13, - StopLocalListen = 14 + StopLocalListen = 14, + RemoteQueryAsync = 15, + WaitForLocalAsync = 16 } /** Map from user func to local wrapper, needed for invoke/unsubscribe. */ @@ -66,9 +67,6 @@ private enum Op /** Cluster group. */ private readonly IClusterGroup _clusterGroup; - /** Async instance. */ - private readonly Lazy _asyncInstance; - /// /// Initializes a new instance of the class. /// @@ -81,17 +79,6 @@ public Events(IUnmanagedTarget target, Marshaller marsh, IClusterGroup clusterGr Debug.Assert(clusterGroup != null); _clusterGroup = clusterGroup; - - _asyncInstance = new Lazy(() => new Events(this)); - } - - /// - /// Initializes a new async instance. - /// - /// The events. - private Events(Events events) : base(UU.TargetOutObject(events.Target, (int) Op.WithAsync), events.Marshaller) - { - _clusterGroup = events.ClusterGroup; } /** */ @@ -106,14 +93,6 @@ private Ignite Ignite get { return (Ignite) ClusterGroup.Ignite; } } - /// - /// Gets the asynchronous instance. - /// - private Events AsyncInstance - { - get { return _asyncInstance.Value; } - } - /** */ public ICollection RemoteQuery(IEventFilter filter, TimeSpan? timeout = null, params int[] types) where T : IEvent @@ -121,14 +100,7 @@ public ICollection RemoteQuery(IEventFilter filter, TimeSpan? timeout = IgniteArgumentCheck.NotNull(filter, "filter"); return DoOutInOp((int) Op.RemoteQuery, - writer => - { - writer.Write(filter); - - writer.WriteLong((long) (timeout == null ? 0 : timeout.Value.TotalMilliseconds)); - - WriteEventTypes(types, writer); - }, + writer => WriteRemoteQuery(filter, timeout, types, writer), reader => ReadEvents(reader)); } @@ -136,11 +108,11 @@ public ICollection RemoteQuery(IEventFilter filter, TimeSpan? timeout = public Task> RemoteQueryAsync(IEventFilter filter, TimeSpan? timeout = null, params int[] types) where T : IEvent { - AsyncInstance.RemoteQuery(filter, timeout, types); + IgniteArgumentCheck.NotNull(filter, "filter"); // ReSharper disable once RedundantTypeArgumentsOfMethod (won't compile in VS2010) - return GetFuture>((futId, futTyp) => UU.TargetListenFutureForOperation(AsyncInstance.Target, - futId, futTyp, (int) Op.RemoteQuery), convertFunc: ReadEvents).Task; + return DoOutOpAsync>((int) Op.RemoteQueryAsync, + w => WriteRemoteQuery(filter, timeout, types, w), convertFunc: ReadEvents); } /** */ @@ -234,46 +206,53 @@ public Task WaitForLocalAsync(IEnumerable types) /** */ public T WaitForLocal(IEventFilter filter, params int[] types) where T : IEvent { - long hnd = 0; + var hnd = GetFilterHandle(filter); try { - return WaitForLocal0(filter, ref hnd, types); + return DoOutInOp((int) Op.WaitForLocal, + writer => + { + writer.WriteObject(hnd); + WriteEventTypes(types, writer); + }, + reader => EventReader.Read(Marshaller.StartUnmarshal(reader))); } finally { - if (filter != null) - Ignite.HandleRegistry.Release(hnd); + if (hnd != null) + Ignite.HandleRegistry.Release(hnd.Value); } } /** */ public Task WaitForLocalAsync(IEventFilter filter, params int[] types) where T : IEvent { - long hnd = 0; + var hnd = GetFilterHandle(filter); try { - AsyncInstance.WaitForLocal0(filter, ref hnd, types); - - // ReSharper disable once RedundantTypeArgumentsOfMethod (won't compile in VS2010) - var fut = GetFuture((futId, futTyp) => UU.TargetListenFutureForOperation(AsyncInstance.Target, futId, - futTyp, (int) Op.WaitForLocal), convertFunc: reader => (T) EventReader.Read(reader)); + var task = DoOutOpAsync((int) Op.WaitForLocalAsync, writer => + { + writer.WriteObject(hnd); + WriteEventTypes(types, writer); + }, convertFunc: EventReader.Read); - if (filter != null) + if (hnd != null) { // Dispose handle as soon as future ends. - fut.Task.ContinueWith(x => Ignite.HandleRegistry.Release(hnd)); + task.ContinueWith(x => Ignite.HandleRegistry.Release(hnd.Value)); } - return fut.Task; + return task; } catch (Exception) { - Ignite.HandleRegistry.Release(hnd); + if (hnd != null) + Ignite.HandleRegistry.Release(hnd.Value); + throw; } - } /** */ @@ -392,38 +371,13 @@ public bool IsEnabled(int type) } /// - /// Waits for the specified events. + /// Gets the filter handle. /// - /// Type of events. - /// Optional filtering predicate. Event wait will end as soon as it returns false. - /// The filter handle, if applicable. - /// Types of the events to wait for. - /// If not provided, all events will be passed to the filter. - /// Ignite event. - private T WaitForLocal0(IEventFilter filter, ref long handle, params int[] types) where T : IEvent - { - if (filter != null) - handle = Ignite.HandleRegistry.Allocate(new LocalEventFilter - { - InvokeFunc = stream => InvokeLocalFilter(stream, filter) - }); - - var hnd = handle; - - return DoOutInOp((int)Op.WaitForLocal, - writer => - { - if (filter != null) - { - writer.WriteBoolean(true); - writer.WriteLong(hnd); - } - else - writer.WriteBoolean(false); - - WriteEventTypes(types, writer); - }, - reader => EventReader.Read(Marshaller.StartUnmarshal(reader))); + private long? GetFilterHandle(IEventFilter filter) where T : IEvent + { + return filter != null + ? Ignite.HandleRegistry.Allocate(new LocalEventFilter(Marshaller, filter)) + : (long?) null; } /// @@ -552,20 +506,6 @@ private IEnumerable GetLocalFilters(object listener, in return result; } - /// - /// Invokes local filter using data from specified stream. - /// - /// Event object type. - /// The stream. - /// The listener. - /// Filter invocation result. - private bool InvokeLocalFilter(IBinaryStream stream, IEventFilter listener) where T : IEvent - { - var evt = EventReader.Read(Marshaller.StartUnmarshal(stream)); - - return listener.Invoke(evt); - } - /// /// Invokes local filter using data from specified stream. /// @@ -613,18 +553,50 @@ private static int[] TypesToArray(IEnumerable types) return types as int[] ?? types.ToArray(); } + /// + /// Writes the remote query. + /// + /// The filter. + /// The timeout. + /// The types. + /// The writer. + private static void WriteRemoteQuery(IEventFilter filter, TimeSpan? timeout, int[] types, + IBinaryRawWriter writer) + where T : IEvent + { + writer.WriteObject(filter); + + writer.WriteLong((long)(timeout == null ? 0 : timeout.Value.TotalMilliseconds)); + + WriteEventTypes(types, writer); + } + /// /// Local user filter wrapper. /// - private class LocalEventFilter : IInteropCallback + private class LocalEventFilter : IInteropCallback where T : IEvent { /** */ - public Func InvokeFunc; + private readonly Marshaller _marshaller; + + /** */ + private readonly IEventFilter _listener; + + /// + /// Initializes a new instance of the class. + /// + public LocalEventFilter(Marshaller marshaller, IEventFilter listener) + { + _marshaller = marshaller; + _listener = listener; + } /** */ public int Invoke(IBinaryStream stream) { - return InvokeFunc(stream) ? 1 : 0; + var evt = EventReader.Read(_marshaller.StartUnmarshal(stream)); + + return _listener.Invoke(evt) ? 1 : 0; } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs index a8e307571ff37..79df4709df215 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs @@ -485,7 +485,7 @@ public void DestroyCache(string name) /// public ICache Cache(IUnmanagedTarget nativeCache, bool keepBinary = false) { - return new CacheImpl(this, nativeCache, _marsh, false, keepBinary, false, false); + return new CacheImpl(this, nativeCache, _marsh, false, keepBinary, false); } /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs index 2216d1a88f379..1b43438069beb 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs @@ -23,6 +23,7 @@ namespace Apache.Ignite.Core.Impl.Messaging using System.Diagnostics; using System.Linq; using System.Threading.Tasks; + using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Collections; @@ -30,7 +31,6 @@ namespace Apache.Ignite.Core.Impl.Messaging using Apache.Ignite.Core.Impl.Resource; using Apache.Ignite.Core.Impl.Unmanaged; using Apache.Ignite.Core.Messaging; - using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils; /// /// Messaging functionality. @@ -49,7 +49,8 @@ private enum Op SendOrdered = 5, StopLocalListen = 6, StopRemoteListen = 7, - WithAsync = 8 + RemoteListenAsync = 9, + StopRemoteListenAsync = 10 } /** Map from user (func+topic) -> id, needed for unsubscription. */ @@ -59,12 +60,6 @@ private enum Op /** Grid */ private readonly Ignite _ignite; - /** Async instance. */ - private readonly Lazy _asyncInstance; - - /** Async flag. */ - private readonly bool _isAsync; - /** Cluster group. */ private readonly IClusterGroup _clusterGroup; @@ -82,20 +77,6 @@ public Messaging(IUnmanagedTarget target, Marshaller marsh, IClusterGroup prj) _clusterGroup = prj; _ignite = (Ignite) prj.Ignite; - - _asyncInstance = new Lazy(() => new Messaging(this)); - } - - /// - /// Initializes a new async instance. - /// - /// The messaging. - private Messaging(Messaging messaging) : base( - UU.TargetOutObject(messaging.Target, (int) Op.WithAsync), messaging.Marshaller) - { - _isAsync = true; - _ignite = messaging._ignite; - _clusterGroup = messaging.ClusterGroup; } /** */ @@ -104,16 +85,7 @@ public IClusterGroup ClusterGroup get { return _clusterGroup; } } - /// - /// Gets the asynchronous instance. - /// - private Messaging AsyncInstance - { - get { return _asyncInstance.Value; } - } - /** */ - public void Send(object message, object topic = null) { IgniteArgumentCheck.NotNull(message, "message"); @@ -216,65 +188,28 @@ public void StopLocalListen(IMessageListener listener, object topic = null /** */ public Guid RemoteListen(IMessageListener listener, object topic = null) { - IgniteArgumentCheck.NotNull(listener, "filter"); - - var filter0 = MessageListenerHolder.CreateLocal(_ignite, listener); - var filterHnd = _ignite.HandleRegistry.AllocateSafe(filter0); - - try - { - Guid id = Guid.Empty; - - DoOutInOp((int) Op.RemoteListen, - writer => - { - writer.Write(filter0); - writer.WriteLong(filterHnd); - writer.Write(topic); - }, - input => - { - var id0 = Marshaller.StartUnmarshal(input).GetRawReader().ReadGuid(); - - Debug.Assert(_isAsync || id0.HasValue); - - if (id0.HasValue) - id = id0.Value; - }); - - return id; - } - catch (Exception) - { - _ignite.HandleRegistry.Release(filterHnd); - - throw; - } + return RemoteListen(listener, topic, + (writeAct, readAct) => DoOutInOp((int) Op.RemoteListen, writeAct, + stream => readAct(Marshaller.StartUnmarshal(stream)))); } /** */ public Task RemoteListenAsync(IMessageListener listener, object topic = null) { - AsyncInstance.RemoteListen(listener, topic); - - return AsyncInstance.GetTask(); + return RemoteListen(listener, topic, + (writeAct, readAct) => DoOutOpAsync((int) Op.RemoteListenAsync, writeAct, convertFunc: readAct)); } /** */ public void StopRemoteListen(Guid opId) { - DoOutOp((int) Op.StopRemoteListen, writer => - { - writer.WriteGuid(opId); - }); + DoOutOp((int) Op.StopRemoteListen, writer => writer.WriteGuid(opId)); } /** */ public Task StopRemoteListenAsync(Guid opId) { - AsyncInstance.StopRemoteListen(opId); - - return AsyncInstance.GetTask(); + return DoOutOpAsync((int) Op.StopRemoteListenAsync, writer => writer.WriteGuid(opId)); } /// @@ -287,5 +222,33 @@ public Task StopRemoteListenAsync(Guid opId) { return new KeyValuePair(filter, topic); } + + /// + /// Remotes listen. + /// + private TRes RemoteListen(IMessageListener filter, object topic, + Func, Func, TRes> invoker) + { + IgniteArgumentCheck.NotNull(filter, "filter"); + + var filter0 = MessageListenerHolder.CreateLocal(_ignite, filter); + var filterHnd = _ignite.HandleRegistry.AllocateSafe(filter0); + + try + { + return invoker(writer => + { + writer.WriteObject(filter0); + writer.WriteLong(filterHnd); + writer.WriteObject(topic); + }, input => input.ReadGuid() ?? Guid.Empty); + } + catch (Exception) + { + _ignite.HandleRegistry.Release(filterHnd); + + throw; + } + } } } \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs index f392830125be1..d5b69a4a67f0b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs @@ -23,6 +23,7 @@ namespace Apache.Ignite.Core.Impl using System.Diagnostics.CodeAnalysis; using System.IO; using System.Threading.Tasks; + using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Binary.IO; using Apache.Ignite.Core.Impl.Binary.Metadata; @@ -757,6 +758,102 @@ protected long DoOutInOpLong(int type, long val) #endregion + #region Async operations + + /// + /// Performs async operation. + /// + /// The type code. + /// The write action. + /// Task for async operation + protected Task DoOutOpAsync(int type, Action writeAction = null) + { + return DoOutOpAsync(type, writeAction); + } + + /// + /// Performs async operation. + /// + /// Type of the result. + /// The type code. + /// The write action. + /// Keep binary flag, only applicable to object futures. False by default. + /// The function to read future result from stream. + /// Task for async operation + protected Task DoOutOpAsync(int type, Action writeAction = null, bool keepBinary = false, + Func convertFunc = null) + { + return GetFuture((futId, futType) => DoOutOp(type, w => + { + if (writeAction != null) + writeAction(w); + w.WriteLong(futId); + w.WriteInt(futType); + }), keepBinary, convertFunc).Task; + } + + /// + /// Performs async operation. + /// + /// Type of the result. + /// The type code. + /// The write action. + /// Future for async operation + protected Future DoOutOpObjectAsync(int type, Action writeAction) + { + return GetFuture((futId, futType) => DoOutOpObject(type, w => + { + writeAction(w); + w.WriteLong(futId); + w.WriteInt(futType); + })); + } + + /// + /// Performs async operation. + /// + /// Type of the result. + /// The type of the first arg. + /// The type code. + /// First arg. + /// + /// Task for async operation + /// + protected Task DoOutOpAsync(int type, T1 val1) + { + return GetFuture((futId, futType) => DoOutOp(type, w => + { + w.WriteObject(val1); + w.WriteLong(futId); + w.WriteInt(futType); + })).Task; + } + + /// + /// Performs async operation. + /// + /// Type of the result. + /// The type of the first arg. + /// The type of the second arg. + /// The type code. + /// First arg. + /// Second arg. + /// + /// Task for async operation + /// + protected Task DoOutOpAsync(int type, T1 val1, T2 val2) + { + return GetFuture((futId, futType) => DoOutOp(type, w => + { + w.WriteObject(val1); + w.WriteObject(val2); + w.WriteLong(futId); + w.WriteInt(futType); + })).Task; + } + + #endregion + #region Miscelanneous /// @@ -846,7 +943,7 @@ protected virtual T Unmarshal(IBinaryStream stream) /// Keep binary flag, only applicable to object futures. False by default. /// The function to read future result from stream. /// Created future. - protected Future GetFuture(Func listenAction, bool keepBinary = false, + private Future GetFuture(Func listenAction, bool keepBinary = false, Func convertFunc = null) { var futType = FutureType.Object; @@ -862,7 +959,18 @@ protected virtual T Unmarshal(IBinaryStream stream) var futHnd = _marsh.Ignite.HandleRegistry.Allocate(fut); - var futTarget = listenAction(futHnd, (int) futType); + IUnmanagedTarget futTarget; + + try + { + futTarget = listenAction(futHnd, (int)futType); + } + catch (Exception) + { + _marsh.Ignite.HandleRegistry.Release(futHnd); + + throw; + } fut.SetTarget(futTarget); @@ -893,25 +1001,18 @@ protected virtual T Unmarshal(IBinaryStream stream) var futHnd = _marsh.Ignite.HandleRegistry.Allocate(fut); - listenAction(futHnd, (int)futType); - - return fut; - } + try + { + listenAction(futHnd, (int)futType); + } + catch (Exception) + { + _marsh.Ignite.HandleRegistry.Release(futHnd); - /// - /// Creates a task to listen for the last async op. - /// - protected Task GetTask() - { - return GetTask(); - } + throw; + } - /// - /// Creates a task to listen for the last async op. - /// - protected Task GetTask() - { - return GetFuture((futId, futTyp) => UU.TargetListenFuture(Target, futId, futTyp)).Task; + return fut; } #endregion diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs index 8fc973b1d137f..9d9acfd1482ec 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs @@ -17,12 +17,12 @@ namespace Apache.Ignite.Core.Impl.Services { - using System; using System.Collections.Generic; using System.Diagnostics; using System.Linq; using System.Reflection; using System.Threading.Tasks; + using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Common; @@ -50,9 +50,6 @@ internal sealed class Services : PlatformTarget, IServices /** */ private const int OpDescriptors = 5; - /** */ - private const int OpWithAsync = 6; - /** */ private const int OpWithServerKeepBinary = 7; @@ -65,6 +62,18 @@ internal sealed class Services : PlatformTarget, IServices /** */ private const int OpCancelAll = 10; + /** */ + private const int OpDeployAsync = 11; + + /** */ + private const int OpDeployMultipleAsync = 12; + + /** */ + private const int OpCancelAsync = 13; + + /** */ + private const int OpCancelAllAsync = 14; + /** */ private readonly IClusterGroup _clusterGroup; @@ -74,9 +83,6 @@ internal sealed class Services : PlatformTarget, IServices /** Server binary flag. */ private readonly bool _srvKeepBinary; - /** Async instance. */ - private readonly Lazy _asyncInstance; - /// /// Initializes a new instance of the class. /// @@ -94,20 +100,6 @@ internal sealed class Services : PlatformTarget, IServices _clusterGroup = clusterGroup; _keepBinary = keepBinary; _srvKeepBinary = srvKeepBinary; - - _asyncInstance = new Lazy(() => new Services(this)); - } - - /// - /// Initializes a new async instance. - /// - /// The services. - private Services(Services services) : base(UU.TargetOutObject(services.Target, OpWithAsync), - services.Marshaller) - { - _clusterGroup = services.ClusterGroup; - _keepBinary = services._keepBinary; - _srvKeepBinary = services._srvKeepBinary; } /** */ @@ -134,14 +126,6 @@ public IClusterGroup ClusterGroup get { return _clusterGroup; } } - /// - /// Gets the asynchronous instance. - /// - private Services AsyncInstance - { - get { return _asyncInstance.Value; } - } - /** */ public void DeployClusterSingleton(string name, IService service) { @@ -154,9 +138,10 @@ public void DeployClusterSingleton(string name, IService service) /** */ public Task DeployClusterSingletonAsync(string name, IService service) { - AsyncInstance.DeployClusterSingleton(name, service); + IgniteArgumentCheck.NotNullOrEmpty(name, "name"); + IgniteArgumentCheck.NotNull(service, "service"); - return AsyncInstance.GetTask(); + return DeployMultipleAsync(name, service, 1, 1); } /** */ @@ -171,9 +156,10 @@ public void DeployNodeSingleton(string name, IService service) /** */ public Task DeployNodeSingletonAsync(string name, IService service) { - AsyncInstance.DeployNodeSingleton(name, service); + IgniteArgumentCheck.NotNullOrEmpty(name, "name"); + IgniteArgumentCheck.NotNull(service, "service"); - return AsyncInstance.GetTask(); + return DeployMultipleAsync(name, service, 0, 1); } /** */ @@ -197,9 +183,19 @@ public void DeployKeyAffinitySingleton(string name, IService service, string /** */ public Task DeployKeyAffinitySingletonAsync(string name, IService service, string cacheName, TK affinityKey) { - AsyncInstance.DeployKeyAffinitySingleton(name, service, cacheName, affinityKey); + IgniteArgumentCheck.NotNullOrEmpty(name, "name"); + IgniteArgumentCheck.NotNull(service, "service"); + IgniteArgumentCheck.NotNull(affinityKey, "affinityKey"); - return AsyncInstance.GetTask(); + return DeployAsync(new ServiceConfiguration + { + Name = name, + Service = service, + CacheName = cacheName, + AffinityKey = affinityKey, + TotalCount = 1, + MaxPerNodeCount = 1 + }); } /** */ @@ -220,9 +216,16 @@ public void DeployMultiple(string name, IService service, int totalCount, int ma /** */ public Task DeployMultipleAsync(string name, IService service, int totalCount, int maxPerNodeCount) { - AsyncInstance.DeployMultiple(name, service, totalCount, maxPerNodeCount); + IgniteArgumentCheck.NotNullOrEmpty(name, "name"); + IgniteArgumentCheck.NotNull(service, "service"); - return AsyncInstance.GetTask(); + return DoOutOpAsync(OpDeployMultipleAsync, w => + { + w.WriteString(name); + w.WriteObject(service); + w.WriteInt(totalCount); + w.WriteInt(maxPerNodeCount); + }); } /** */ @@ -230,28 +233,15 @@ public void Deploy(ServiceConfiguration configuration) { IgniteArgumentCheck.NotNull(configuration, "configuration"); - DoOutOp(OpDeploy, w => - { - w.WriteString(configuration.Name); - w.WriteObject(configuration.Service); - w.WriteInt(configuration.TotalCount); - w.WriteInt(configuration.MaxPerNodeCount); - w.WriteString(configuration.CacheName); - w.WriteObject(configuration.AffinityKey); - - if (configuration.NodeFilter != null) - w.WriteObject(configuration.NodeFilter); - else - w.WriteObject(null); - }); + DoOutOp(OpDeploy, w => WriteServiceConfiguration(configuration, w)); } /** */ public Task DeployAsync(ServiceConfiguration configuration) { - AsyncInstance.Deploy(configuration); + IgniteArgumentCheck.NotNull(configuration, "configuration"); - return AsyncInstance.GetTask(); + return DoOutOpAsync(OpDeployAsync, w => WriteServiceConfiguration(configuration, w)); } /** */ @@ -265,9 +255,9 @@ public void Cancel(string name) /** */ public Task CancelAsync(string name) { - AsyncInstance.Cancel(name); + IgniteArgumentCheck.NotNullOrEmpty(name, "name"); - return AsyncInstance.GetTask(); + return DoOutOpAsync(OpCancelAsync, w => w.WriteString(name)); } /** */ @@ -279,9 +269,7 @@ public void CancelAll() /** */ public Task CancelAllAsync() { - AsyncInstance.CancelAll(); - - return AsyncInstance.GetTask(); + return DoOutOpAsync(OpCancelAllAsync); } /** */ @@ -391,5 +379,26 @@ public ICollection GetServices(string name) writer => ServiceProxySerializer.WriteProxyMethod(writer, method, args, platform), stream => ServiceProxySerializer.ReadInvocationResult(stream, Marshaller, _keepBinary), proxy.Target); } + + /// + /// Writes the service configuration. + /// + private static void WriteServiceConfiguration(ServiceConfiguration configuration, IBinaryRawWriter w) + { + Debug.Assert(configuration != null); + Debug.Assert(w != null); + + w.WriteString(configuration.Name); + w.WriteObject(configuration.Service); + w.WriteInt(configuration.TotalCount); + w.WriteInt(configuration.MaxPerNodeCount); + w.WriteString(configuration.CacheName); + w.WriteObject(configuration.AffinityKey); + + if (configuration.NodeFilter != null) + w.WriteObject(configuration.NodeFilter); + else + w.WriteObject(null); + } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs index 796044d8d902f..7de9be1da8d38 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs @@ -22,7 +22,6 @@ namespace Apache.Ignite.Core.Impl.Transactions using System.Threading.Tasks; using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Impl.Binary; - using Apache.Ignite.Core.Impl.Binary.IO; using Apache.Ignite.Core.Impl.Unmanaged; using Apache.Ignite.Core.Transactions; @@ -214,11 +213,7 @@ internal bool TxSetRollbackOnly(TransactionImpl tx) /// internal Task CommitAsync(TransactionImpl tx) { - return GetFuture((futId, futTyp) => DoOutOp(OpCommitAsync, (IBinaryStream s) => - { - s.WriteLong(tx.Id); - s.WriteLong(futId); - })).Task; + return DoOutOpAsync(OpCommitAsync, w => w.WriteLong(tx.Id)); } /// @@ -226,11 +221,7 @@ internal Task CommitAsync(TransactionImpl tx) /// internal Task RollbackAsync(TransactionImpl tx) { - return GetFuture((futId, futTyp) => DoOutOp(OpRollbackAsync, (IBinaryStream s) => - { - s.WriteLong(tx.Id); - s.WriteLong(futId); - })).Task; + return DoOutOpAsync(OpRollbackAsync, w => w.WriteLong(tx.Id)); } } } \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs index c352f0c0f78fa..6b867de055511 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs @@ -150,18 +150,6 @@ internal static unsafe class IgniteJniNativeMethods [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetOutObject")] public static extern void* TargetOutObject(void* ctx, void* target, int opType); - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetListenFuture")] - public static extern void TargetListenFut(void* ctx, void* target, long futId, int typ); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetListenFutureForOperation")] - public static extern void TargetListenFutForOp(void* ctx, void* target, long futId, int typ, int opId); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetListenFutureAndGet")] - public static extern void* TargetListenFutAndGet(void* ctx, void* target, long futId, int typ); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetListenFutureForOperationAndGet")] - public static extern void* TargetListenFutForOpAndGet(void* ctx, void* target, long futId, int typ, int opId); - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteAcquire")] public static extern void* Acquire(void* ctx, void* target); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs index b1d4ecdd82d8a..36dc3326e969e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs @@ -460,31 +460,6 @@ internal static IUnmanagedTarget TargetOutObject(IUnmanagedTarget target, int op return target.ChangeTarget(res); } - internal static void TargetListenFuture(IUnmanagedTarget target, long futId, int typ) - { - JNI.TargetListenFut(target.Context, target.Target, futId, typ); - } - - internal static void TargetListenFutureForOperation(IUnmanagedTarget target, long futId, int typ, int opId) - { - JNI.TargetListenFutForOp(target.Context, target.Target, futId, typ, opId); - } - - internal static IUnmanagedTarget TargetListenFutureAndGet(IUnmanagedTarget target, long futId, int typ) - { - var res = JNI.TargetListenFutAndGet(target.Context, target.Target, futId, typ); - - return target.ChangeTarget(res); - } - - internal static IUnmanagedTarget TargetListenFutureForOperationAndGet(IUnmanagedTarget target, long futId, - int typ, int opId) - { - var res = JNI.TargetListenFutForOpAndGet(target.Context, target.Target, futId, typ, opId); - - return target.ChangeTarget(res); - } - #endregion #region NATIVE METHODS: MISCELANNEOUS diff --git a/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings b/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings index ac065bcdb0f04..72ce015733d53 100644 --- a/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings +++ b/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings @@ -1,5 +1,6 @@  CSharp50 - True + True True + DO_NOT_SHOW \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.sln.TeamCity.DotSettings b/modules/platforms/dotnet/Apache.Ignite.sln.TeamCity.DotSettings index cf9e2877b757a..9672abeda535a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.sln.TeamCity.DotSettings +++ b/modules/platforms/dotnet/Apache.Ignite.sln.TeamCity.DotSettings @@ -25,6 +25,7 @@ DO_NOT_SHOW DO_NOT_SHOW DO_NOT_SHOW + DO_NOT_SHOW True True \ No newline at end of file From 2a90fcaf8e46a829306ca92e226d984111b3aefe Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 25 Oct 2016 15:15:55 +0300 Subject: [PATCH 269/487] IGNITE-4028 .NET: Get rid of OP_META in PlatformAbstractTarget --- .../platform/PlatformAbstractTarget.java | 13 +-- .../cluster/PlatformClusterGroup.java | 19 +++-- .../Impl/Cluster/ClusterGroupImpl.cs | 79 +++++++++++++++++-- .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs | 2 +- .../Apache.Ignite.Core/Impl/PlatformTarget.cs | 67 ---------------- 5 files changed, 91 insertions(+), 89 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java index 29b603ac964b6..5c7f260b12b4f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java @@ -43,9 +43,6 @@ public abstract class PlatformAbstractTarget implements PlatformTarget { /** Constant: ERROR. */ protected static final int ERROR = -1; - /** */ - private static final int OP_META = -1; - /** Context. */ protected final PlatformContext platformCtx; @@ -78,13 +75,7 @@ protected PlatformAbstractTarget(PlatformContext platformCtx) { try (PlatformMemory mem = platformCtx.memory().get(memPtr)) { BinaryRawReaderEx reader = platformCtx.reader(mem); - if (type == OP_META) { - platformCtx.processMetadata(reader); - - return TRUE; - } - else - return processInStreamOutLong(type, reader, mem); + return processInStreamOutLong(type, reader, mem); } catch (Exception e) { throw convertException(e); @@ -401,7 +392,7 @@ protected Object processOutObject(int type) throws IgniteCheckedException { * @return Dummy value which is never returned. * @throws IgniteCheckedException Exception to be thrown. */ - protected T throwUnsupported(int type) throws IgniteCheckedException { + private T throwUnsupported(int type) throws IgniteCheckedException { throw new IgniteCheckedException("Unsupported operation type: " + type); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java index d09506bff142b..04a117360ef72 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java @@ -17,21 +17,22 @@ package org.apache.ignite.internal.processors.platform.cluster; -import java.util.Collection; -import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCluster; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.cluster.ClusterGroupEx; import org.apache.ignite.internal.binary.BinaryRawReaderEx; import org.apache.ignite.internal.binary.BinaryRawWriterEx; +import org.apache.ignite.internal.cluster.ClusterGroupEx; import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.jetbrains.annotations.Nullable; +import java.util.Collection; +import java.util.UUID; + /** * Interop projection. */ @@ -59,7 +60,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget { private static final int OP_FOR_NODE_IDS = 7; /** */ - private static final int OP_METADATA = 8; + private static final int OP_GET_META = 8; /** */ private static final int OP_METRICS = 9; @@ -106,6 +107,9 @@ public class PlatformClusterGroup extends PlatformAbstractTarget { /** */ private static final int OP_FOR_SERVERS = 23; + /** */ + private static final int OP_PUT_META = 24; + /** Projection. */ private final ClusterGroupEx prj; @@ -201,7 +205,7 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { break; } - case OP_METADATA: { + case OP_GET_META: { int typeId = reader.readInt(); platformCtx.writeMetadata(writer, typeId); @@ -237,6 +241,11 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { case OP_PING_NODE: return pingNode(reader.readUuid()) ? TRUE : FALSE; + case OP_PUT_META: + platformCtx.processMetadata(reader); + + return TRUE; + default: return super.processInStreamOutLong(type, reader); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs index 388be82525fcf..912d6ed58bc7d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs @@ -76,7 +76,7 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx private const int OpForNodeIds = 7; /** */ - private const int OpMetadata = 8; + private const int OpGetMeta = 8; /** */ private const int OpMetrics = 9; @@ -103,7 +103,7 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx private const int OpForRemotes = 17; /** */ - public const int OpForDaemons = 18; + private const int OpForDaemons = 18; /** */ private const int OpForRandom = 19; @@ -115,10 +115,13 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx private const int OpForYoungest = 21; /** */ - public const int OpResetMetrics = 22; + private const int OpResetMetrics = 22; /** */ - public const int OpForServers = 23; + private const int OpForServers = 23; + + /** */ + private const int OpPutMeta = 24; /** Initial Ignite instance. */ private readonly Ignite _ignite; @@ -557,7 +560,7 @@ private IList RefreshNodes() /** */ public IBinaryType GetBinaryType(int typeId) { - return DoOutInOp(OpMetadata, + return DoOutInOp(OpGetMeta, writer => writer.WriteInt(typeId), stream => { @@ -599,5 +602,71 @@ public int[] GetSchema(int typeId, int schemaId) writer.WriteInt(schemaId); }); } + + /// + /// Resets local I/O, job, and task execution metrics. + /// + public void ResetMetrics() + { + DoOutOp(OpResetMetrics); + } + + /// + /// Put binary types to Grid. + /// + /// Binary types. + public void PutBinaryTypes(ICollection types) + { + DoOutOp(OpPutMeta, w => + { + w.WriteInt(types.Count); + + foreach (var meta in types) + { + w.WriteInt(meta.TypeId); + w.WriteString(meta.TypeName); + w.WriteString(meta.AffinityKeyFieldName); + + IDictionary fields = meta.GetFieldsMap(); + + w.WriteInt(fields.Count); + + foreach (var field in fields) + { + w.WriteString(field.Key); + w.WriteInt(field.Value); + } + + w.WriteBoolean(meta.IsEnum); + + // Send schemas + var desc = meta.Descriptor; + Debug.Assert(desc != null); + + var count = 0; + var countPos = w.Stream.Position; + w.WriteInt(0); // Reserve for count + + foreach (var schema in desc.Schema.GetAll()) + { + w.WriteInt(schema.Key); + + var ids = schema.Value; + w.WriteInt(ids.Length); + + foreach (var id in ids) + w.WriteInt(id); + + count++; + } + + w.Stream.WriteInt(countPos, count); + } + + Marshaller.FinishMarshal(w); + }); + + Marshaller.OnBinaryTypesSent(types); + } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs index 79df4709df215..70a33113fff1b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs @@ -517,7 +517,7 @@ public ICollection GetTopology(long ver) /** */ public void ResetMetrics() { - UU.TargetOutLong(_prj.Target, ClusterGroupImpl.OpResetMetrics); + _prj.ResetMetrics(); } /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs index d5b69a4a67f0b..9b80d8c670176 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs @@ -26,7 +26,6 @@ namespace Apache.Ignite.Core.Impl using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Binary.IO; - using Apache.Ignite.Core.Impl.Binary.Metadata; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Impl.Memory; using Apache.Ignite.Core.Impl.Unmanaged; @@ -49,12 +48,6 @@ internal abstract class PlatformTarget /** */ protected const int Error = -1; - /** */ - private const int OpMeta = -1; - - /** */ - public const int OpNone = -2; - /** */ private static readonly Dictionary IgniteFutureTypeMap = new Dictionary @@ -865,66 +858,6 @@ internal void FinishMarshal(BinaryWriter writer) _marsh.FinishMarshal(writer); } - /// - /// Put binary types to Grid. - /// - /// Binary types. - internal void PutBinaryTypes(ICollection types) - { - DoOutOp(OpMeta, stream => - { - BinaryWriter w = _marsh.StartMarshal(stream); - - w.WriteInt(types.Count); - - foreach (var meta in types) - { - w.WriteInt(meta.TypeId); - w.WriteString(meta.TypeName); - w.WriteString(meta.AffinityKeyFieldName); - - IDictionary fields = meta.GetFieldsMap(); - - w.WriteInt(fields.Count); - - foreach (var field in fields) - { - w.WriteString(field.Key); - w.WriteInt(field.Value); - } - - w.WriteBoolean(meta.IsEnum); - - // Send schemas - var desc = meta.Descriptor; - Debug.Assert(desc != null); - - var count = 0; - var countPos = stream.Position; - w.WriteInt(0); // Reserve for count - - foreach (var schema in desc.Schema.GetAll()) - { - w.WriteInt(schema.Key); - - var ids = schema.Value; - w.WriteInt(ids.Length); - - foreach (var id in ids) - w.WriteInt(id); - - count++; - } - - stream.WriteInt(countPos, count); - } - - _marsh.FinishMarshal(w); - }); - - _marsh.OnBinaryTypesSent(types); - } - /// /// Unmarshal object using the given stream. /// From bf266e971579cad0f40d233090c93f3acec06d0a Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Mon, 17 Oct 2016 11:26:12 +0300 Subject: [PATCH 270/487] IGNITE-4054: Hadoop: added map-reduce plan debug output. --- .../hadoop/HadoopMapReducePlanner.java | 1 + .../hadoop/HadoopExternalSplit.java | 8 +++ .../processors/hadoop/HadoopSplitWrapper.java | 9 +++ .../hadoop/jobtracker/HadoopJobTracker.java | 61 +++++++++++++++++++ 4 files changed, 79 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlanner.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlanner.java index 185994f4758fb..0009c4a80f46b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlanner.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlanner.java @@ -33,6 +33,7 @@ public interface HadoopMapReducePlanner { * @param top Topology. * @param oldPlan Old plan in case of partial failure. * @return Map reduce plan. + * @throws IgniteCheckedException If an error occurs. */ public HadoopMapReducePlan preparePlan(HadoopJob job, Collection top, @Nullable HadoopMapReducePlan oldPlan) throws IgniteCheckedException; diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java index bd767b368dba1..a9b453287c6c2 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java @@ -17,10 +17,13 @@ package org.apache.ignite.internal.processors.hadoop; +import org.apache.ignite.internal.util.typedef.internal.S; + import java.io.Externalizable; import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; +import java.util.Arrays; /** * Split serialized in external file. @@ -85,4 +88,9 @@ public long offset() { @Override public int hashCode() { return (int)(off ^ (off >>> 32)); } + + /** {@inheritDoc} */ + public String toString() { + return S.toString(HadoopExternalSplit.class, this, "hosts", Arrays.toString(hosts)); + } } \ No newline at end of file diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java index 511aa5a2ad265..fb6d0f3008d4a 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java @@ -20,7 +20,10 @@ import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; +import java.util.Arrays; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; /** @@ -33,6 +36,7 @@ public class HadoopSplitWrapper extends HadoopInputSplit { private static final long serialVersionUID = 0L; /** Native hadoop input split. */ + @GridToStringExclude private byte[] bytes; /** */ @@ -116,4 +120,9 @@ public byte[] bytes() { @Override public int hashCode() { return id; } + + /** {@inheritDoc} */ + public String toString() { + return S.toString(HadoopSplitWrapper.class, this, "hosts", Arrays.toString(hosts)); + } } \ No newline at end of file diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java index bffb82b1e9180..36782bf2febe0 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java @@ -75,6 +75,7 @@ import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteInClosure; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -313,6 +314,8 @@ public IgniteInternalFuture submit(HadoopJobId jobId, HadoopJobInfo HadoopMapReducePlan mrPlan = mrPlanner.preparePlan(job, ctx.nodes(), null); + logPlan(info, mrPlan); + HadoopJobMetadata meta = new HadoopJobMetadata(ctx.localNodeId(), jobId, info); meta.mapReducePlan(mrPlan); @@ -353,6 +356,64 @@ public IgniteInternalFuture submit(HadoopJobId jobId, HadoopJobInfo } } + /** + * Log map-reduce plan if needed. + * + * @param info Job info. + * @param plan Plan. + */ + @SuppressWarnings("StringConcatenationInsideStringBufferAppend") + private void logPlan(HadoopJobInfo info, HadoopMapReducePlan plan) { + if (log.isDebugEnabled()) { + Map, int[]>> map = new HashMap<>(); + + for (UUID nodeId : plan.mapperNodeIds()) + map.put(nodeId, new IgniteBiTuple, int[]>(plan.mappers(nodeId), null)); + + for (UUID nodeId : plan.reducerNodeIds()) { + int[] reducers = plan.reducers(nodeId); + + IgniteBiTuple, int[]> entry = map.get(nodeId); + + if (entry == null) + map.put(nodeId, new IgniteBiTuple, int[]>(null, reducers)); + else + entry.set2(reducers); + } + + StringBuilder details = new StringBuilder("["); + + boolean first = true; + + for (Map.Entry, int[]>> entry : map.entrySet()) { + if (first) + first = false; + else + details.append(", "); + + UUID nodeId = entry.getKey(); + + Collection mappers = entry.getValue().get1(); + + if (mappers == null) + mappers = Collections.emptyList(); + + int[] reducers = entry.getValue().get2(); + + if (reducers == null) + reducers = new int[0]; + + details.append("[nodeId=" + nodeId + ", mappers=" + mappers.size() + ", reducers=" + reducers.length + + ", mapperDetails=" + mappers + ", reducerDetails=" + Arrays.toString(reducers) + ']'); + } + + details.append(']'); + + log.debug("Prepared map-reduce plan [jobName=" + info.jobName() + ", mappers=" + plan.mappers() + + ", reducers=" + plan.reducers() + ", details=" + details + ']'); + } + } + /** * Convert Hadoop job metadata to job status. * From 1e21d643aaf4d78e36fccaaeaa29cd3c49c90405 Mon Sep 17 00:00:00 2001 From: sboikov Date: Mon, 17 Oct 2016 12:28:31 +0300 Subject: [PATCH 271/487] Fixed indexing test in according to changes from #80abd1b. --- .../query/h2/GridIndexingSpiAbstractSelfTest.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java index fae3188aadedb..760ee197034c9 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java @@ -40,6 +40,7 @@ import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.apache.ignite.spi.IgniteSpiCloseableIterator; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.testframework.GridStringLogger; import org.apache.ignite.testframework.GridTestUtils; @@ -354,9 +355,11 @@ public void testSpi() throws Exception { String[] aliases = {"N1", "A1", "N2", "A2"}; Object[] vals = { "Valera", 19, "Kolya", 25}; - assertTrue(fieldsRes.iterator().hasNext()); + IgniteSpiCloseableIterator> it = fieldsRes.iterator(); - List fields = fieldsRes.iterator().next(); + assertTrue(it.hasNext()); + + List fields = it.next(); assertEquals(4, fields.size()); @@ -367,7 +370,7 @@ public void testSpi() throws Exception { assertEquals(vals[i++], f); } - assertFalse(fieldsRes.iterator().hasNext()); + assertFalse(it.hasNext()); // Remove spi.remove(typeAA.space(), key(2), aa(2, "Valera", 19)); From 51cef7c740828ffcc40f411a66e630421040a2d2 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Tue, 18 Oct 2016 11:53:46 +0300 Subject: [PATCH 272/487] IGNITE-4090: CPP: Fixed compilation for GCC 4.4.7. This closes #1165. --- modules/platforms/cpp/common/Makefile.am | 4 +++- .../cpp/common/include/ignite/common/concurrent.h | 5 +++-- .../platforms/cpp/common/include/ignite/ignite_error.h | 4 ++-- modules/platforms/cpp/common/src/ignite_error.cpp | 4 ++-- modules/platforms/cpp/core-test/Makefile.am | 10 ++++++---- modules/platforms/cpp/core-test/src/decimal_test.cpp | 2 +- .../cpp/core-test/src/interop_memory_test.cpp | 5 +++-- modules/platforms/cpp/jni/src/java.cpp | 2 +- modules/platforms/cpp/odbc-test/Makefile.am | 4 +++- modules/platforms/cpp/odbc-test/src/queries_test.cpp | 4 ++-- .../platforms/cpp/odbc/src/config/configuration.cpp | 2 +- 11 files changed, 27 insertions(+), 19 deletions(-) diff --git a/modules/platforms/cpp/common/Makefile.am b/modules/platforms/cpp/common/Makefile.am index c230384820aa0..2d16c2d3f6e9c 100644 --- a/modules/platforms/cpp/common/Makefile.am +++ b/modules/platforms/cpp/common/Makefile.am @@ -26,7 +26,9 @@ SUBDIRS = \ AM_CPPFLAGS = \ -I$(srcdir)/include \ -I$(srcdir)/os/linux/include \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/common/include/ignite/common/concurrent.h b/modules/platforms/cpp/common/include/ignite/common/concurrent.h index 465e02e9ea369..35c2209e39b7b 100644 --- a/modules/platforms/cpp/common/include/ignite/common/concurrent.h +++ b/modules/platforms/cpp/common/include/ignite/common/concurrent.h @@ -138,7 +138,8 @@ namespace ignite { if (ptr) { - impl = new SharedPointerImpl(ptr, reinterpret_cast(&SharedPointerDefaultDeleter)); + void(*deleter)(T*) = (void(*)(T*)) &SharedPointerDefaultDeleter; + impl = new SharedPointerImpl(ptr, reinterpret_cast(deleter)); ImplEnableShared(ptr, impl); } else @@ -360,4 +361,4 @@ namespace ignite } } -#endif //_IGNITE_COMMON_CONCURRENT \ No newline at end of file +#endif //_IGNITE_COMMON_CONCURRENT diff --git a/modules/platforms/cpp/common/include/ignite/ignite_error.h b/modules/platforms/cpp/common/include/ignite/ignite_error.h index 17a24dc2535ff..4c5c06e3dd53d 100644 --- a/modules/platforms/cpp/common/include/ignite/ignite_error.h +++ b/modules/platforms/cpp/common/include/ignite/ignite_error.h @@ -245,7 +245,7 @@ namespace ignite /** * Destructor. */ - ~IgniteError(); + ~IgniteError() IGNITE_NO_THROW; /** * Get error code. @@ -291,4 +291,4 @@ namespace ignite # pragma warning(pop) #endif //_MSC_VER -#endif //_IGNITE_IGNITE_ERROR \ No newline at end of file +#endif //_IGNITE_IGNITE_ERROR diff --git a/modules/platforms/cpp/common/src/ignite_error.cpp b/modules/platforms/cpp/common/src/ignite_error.cpp index 8179184be8a8a..3076d7a4f4a0e 100644 --- a/modules/platforms/cpp/common/src/ignite_error.cpp +++ b/modules/platforms/cpp/common/src/ignite_error.cpp @@ -70,7 +70,7 @@ namespace ignite return *this; } - IgniteError::~IgniteError() + IgniteError::~IgniteError() IGNITE_NO_THROW { ReleaseChars(msg); } @@ -223,4 +223,4 @@ namespace ignite else if (jniCode == IGNITE_JNI_ERR_JVM_ATTACH) *err = IgniteError(IGNITE_ERR_JVM_ATTACH, "Failed to attach to JVM."); } -} \ No newline at end of file +} diff --git a/modules/platforms/cpp/core-test/Makefile.am b/modules/platforms/cpp/core-test/Makefile.am index 61c4914c5dc7a..42680cde921c5 100644 --- a/modules/platforms/cpp/core-test/Makefile.am +++ b/modules/platforms/cpp/core-test/Makefile.am @@ -33,7 +33,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ @@ -55,13 +57,13 @@ ignite_tests_SOURCES = \ src/interop_memory_test.cpp \ src/interop_test.cpp \ src/handle_registry_test.cpp \ - src/ignite_error_test.cpp \ + src/ignite_error_test.cpp \ src/binary_test_defs.cpp \ src/binary_reader_writer_raw_test.cpp \ src/binary_reader_writer_test.cpp \ src/binary_session_test.cpp \ - src/decimal_test.cpp \ - src/dynamic_size_array_test.cpp \ + src/decimal_test.cpp \ + src/dynamic_size_array_test.cpp \ src/fixed_size_array_test.cpp \ src/transactions_test.cpp \ src/teamcity_messages.cpp \ diff --git a/modules/platforms/cpp/core-test/src/decimal_test.cpp b/modules/platforms/cpp/core-test/src/decimal_test.cpp index 47fe8fc67c4e8..9d0bacfb40ef0 100644 --- a/modules/platforms/cpp/core-test/src/decimal_test.cpp +++ b/modules/platforms/cpp/core-test/src/decimal_test.cpp @@ -1098,4 +1098,4 @@ BOOST_AUTO_TEST_CASE(TestDoubleCast) CheckDoubleCast(-0.000000000000001); } -BOOST_AUTO_TEST_SUITE_END() \ No newline at end of file +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/core-test/src/interop_memory_test.cpp b/modules/platforms/cpp/core-test/src/interop_memory_test.cpp index 07e928cb4e27e..1c782b5ab2327 100644 --- a/modules/platforms/cpp/core-test/src/interop_memory_test.cpp +++ b/modules/platforms/cpp/core-test/src/interop_memory_test.cpp @@ -39,7 +39,8 @@ BOOST_AUTO_TEST_CASE(MemoryReallocationTest) SharedPointer mem = env.AllocateMemory(); - BOOST_CHECK_EQUAL(mem.Get()->Capacity(), IgniteEnvironment::DEFAULT_ALLOCATION_SIZE); + BOOST_CHECK_EQUAL(mem.Get()->Capacity(), + static_cast(IgniteEnvironment::DEFAULT_ALLOCATION_SIZE)); BOOST_CHECK(mem.Get()->Data() != NULL); @@ -92,4 +93,4 @@ BOOST_AUTO_TEST_CASE(MemoryReallocationTest) memset(mem.Get()->Data(), 0xF0F0F0F0, mem.Get()->Capacity()); } -BOOST_AUTO_TEST_SUITE_END() \ No newline at end of file +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp index c1efbe2df5b9b..18cf672028b36 100644 --- a/modules/platforms/cpp/jni/src/java.cpp +++ b/modules/platforms/cpp/jni/src/java.cpp @@ -1881,7 +1881,7 @@ namespace ignite } JNIEXPORT jlong JNICALL JniAffinityFunctionInit(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr, jobject baseFunc) { - void* baseFuncRef = baseFunc ? env->NewGlobalRef(baseFunc) : nullptr; + void* baseFuncRef = baseFunc ? env->NewGlobalRef(baseFunc) : 0; IGNITE_SAFE_FUNC(env, envPtr, AffinityFunctionInitHandler, affinityFunctionInit, memPtr, baseFuncRef); } diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index ccf1192445563..e10ef7a922770 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -34,7 +34,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/core/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 73d57c902f78d..21edf4eec25b3 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -196,10 +196,10 @@ struct QueriesTestSuiteFixture BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); } - SQLCHAR request[] = "SELECT i8Field, i16Field, i32Field, i64Field, strField, " + char request[] = "SELECT i8Field, i16Field, i32Field, i64Field, strField, " "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType"; - ret = SQLExecDirect(stmt, request, SQL_NTS); + ret = SQLExecDirect(stmt, reinterpret_cast(request), SQL_NTS); if (!SQL_SUCCEEDED(ret)) BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); diff --git a/modules/platforms/cpp/odbc/src/config/configuration.cpp b/modules/platforms/cpp/odbc/src/config/configuration.cpp index c43b1154e3ce1..07f1368e7a8c0 100644 --- a/modules/platforms/cpp/odbc/src/config/configuration.cpp +++ b/modules/platforms/cpp/odbc/src/config/configuration.cpp @@ -265,7 +265,7 @@ namespace ignite utility::IntoLower(key); - if (value.front() == '{' && value.back() == '}') + if (value[0] == '{' && value[value.size() - 1] == '}') value = value.substr(1, value.size() - 2); args[key] = value; From 442fedc17bdae43b1c87d6bb4680f724a18adb52 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Tue, 18 Oct 2016 12:25:56 +0300 Subject: [PATCH 273/487] Revert "IGNITE-3875: Added separate thread pool for data streamer. This closes #1067." This reverts commit f597aff1bdf65d3d430cf85c9932391a72c2d7dc. --- .../configuration/IgniteConfiguration.java | 31 ------ .../ignite/internal/GridKernalContext.java | 7 -- .../internal/GridKernalContextImpl.java | 12 --- .../apache/ignite/internal/IgniteKernal.java | 3 - .../apache/ignite/internal/IgnitionEx.java | 20 +--- .../managers/communication/GridIoManager.java | 2 - .../managers/communication/GridIoPolicy.java | 3 - .../closure/GridClosureProcessor.java | 3 +- .../datastreamer/DataStreamProcessor.java | 82 ++-------------- .../datastreamer/DataStreamerImpl.java | 31 +++++- .../processors/pool/PoolProcessor.java | 3 - .../DataStreamProcessorSelfTest.java | 97 ------------------- .../junits/GridTestKernalContext.java | 12 +-- 13 files changed, 44 insertions(+), 262 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java index d03958456bdba..75145a302bf8d 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java @@ -148,9 +148,6 @@ public class IgniteConfiguration { /** Default core size of public thread pool. */ public static final int DFLT_PUBLIC_THREAD_CNT = Math.max(8, AVAILABLE_PROC_CNT) * 2; - /** Default size of data streamer thread pool. */ - public static final int DFLT_DATA_STREAMER_POOL_SIZE = DFLT_PUBLIC_THREAD_CNT; - /** Default keep alive time for public thread pool. */ @Deprecated public static final long DFLT_PUBLIC_KEEP_ALIVE_TIME = 0; @@ -248,9 +245,6 @@ public class IgniteConfiguration { /** IGFS pool size. */ private int igfsPoolSize = AVAILABLE_PROC_CNT; - /** Data stream pool size. */ - private int dataStreamerPoolSize = DFLT_DATA_STREAMER_POOL_SIZE; - /** Utility cache pool size. */ private int utilityCachePoolSize = DFLT_SYSTEM_CORE_THREAD_CNT; @@ -514,7 +508,6 @@ public IgniteConfiguration(IgniteConfiguration cfg) { clockSyncFreq = cfg.getClockSyncFrequency(); clockSyncSamples = cfg.getClockSyncSamples(); consistentId = cfg.getConsistentId(); - dataStreamerPoolSize = cfg.getDataStreamerThreadPoolSize(); deployMode = cfg.getDeploymentMode(); discoStartupDelay = cfg.getDiscoveryStartupDelay(); failureDetectionTimeout = cfg.getFailureDetectionTimeout(); @@ -795,17 +788,6 @@ public int getIgfsThreadPoolSize() { return igfsPoolSize; } - /** - * Size of thread pool that is in charge of processing data stream messages. - *

          - * If not provided, executor service will have size {@link #DFLT_DATA_STREAMER_POOL_SIZE}. - * - * @return Thread pool size to be used for data stream messages. - */ - public int getDataStreamerThreadPoolSize() { - return dataStreamerPoolSize; - } - /** * Default size of thread pool that is in charge of processing utility cache messages. *

          @@ -929,19 +911,6 @@ public IgniteConfiguration setIgfsThreadPoolSize(int poolSize) { return this; } - /** - * Set thread pool size that will be used to process data stream messages. - * - * @param poolSize Executor service to use for data stream messages. - * @see IgniteConfiguration#getDataStreamerThreadPoolSize() - * @return {@code this} for chaining. - */ - public IgniteConfiguration setDataStreamerThreadPoolSize(int poolSize) { - dataStreamerPoolSize = poolSize; - - return this; - } - /** * Sets default thread pool size that will be used to process utility cache messages. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java index adace0ba35717..ae292232e0930 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java @@ -532,13 +532,6 @@ public interface GridKernalContext extends Iterable { */ public ExecutorService getIgfsExecutorService(); - /** - * Executor service that is in charge of processing data stream messages. - * - * @return Thread pool implementation to be used for data stream messages. - */ - public ExecutorService getDataStreamerExecutorService(); - /** * Should return an instance of fully configured thread pool to be used for * processing of client messages (REST requests). diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java index fb558006adee5..94c64481d4ce3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java @@ -310,10 +310,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable @GridToStringExclude private ExecutorService igfsExecSvc; - /** */ - @GridToStringExclude - private ExecutorService dataStreamExecSvc; - /** */ @GridToStringExclude protected ExecutorService restExecSvc; @@ -388,7 +384,6 @@ public GridKernalContextImpl() { * @param p2pExecSvc P2P executor service. * @param mgmtExecSvc Management executor service. * @param igfsExecSvc IGFS executor service. - * @param dataStreamExecSvc data stream executor service. * @param restExecSvc REST executor service. * @param affExecSvc Affinity executor service. * @param idxExecSvc Indexing executor service. @@ -408,7 +403,6 @@ protected GridKernalContextImpl( ExecutorService p2pExecSvc, ExecutorService mgmtExecSvc, ExecutorService igfsExecSvc, - ExecutorService dataStreamExecSvc, ExecutorService restExecSvc, ExecutorService affExecSvc, @Nullable ExecutorService idxExecSvc, @@ -428,7 +422,6 @@ protected GridKernalContextImpl( this.p2pExecSvc = p2pExecSvc; this.mgmtExecSvc = mgmtExecSvc; this.igfsExecSvc = igfsExecSvc; - this.dataStreamExecSvc = dataStreamExecSvc; this.restExecSvc = restExecSvc; this.affExecSvc = affExecSvc; this.idxExecSvc = idxExecSvc; @@ -969,11 +962,6 @@ protected Object readResolve() throws ObjectStreamException { return igfsExecSvc; } - /** {@inheritDoc} */ - @Override public ExecutorService getDataStreamerExecutorService() { - return dataStreamExecSvc; - } - /** {@inheritDoc} */ @Override public ExecutorService getRestExecutorService() { return restExecSvc; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 523bee6f387d4..1963509d62908 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -666,7 +666,6 @@ private void notifyLifecycleBeansEx(LifecycleEventType evt) { * @param p2pExecSvc P2P executor service. * @param mgmtExecSvc Management executor service. * @param igfsExecSvc IGFS executor service. - * @param dataStreamExecSvc data stream executor service. * @param restExecSvc Reset executor service. * @param affExecSvc Affinity executor service. * @param idxExecSvc Indexing executor service. @@ -682,7 +681,6 @@ public void start(final IgniteConfiguration cfg, ExecutorService p2pExecSvc, ExecutorService mgmtExecSvc, ExecutorService igfsExecSvc, - ExecutorService dataStreamExecSvc, ExecutorService restExecSvc, ExecutorService affExecSvc, @Nullable ExecutorService idxExecSvc, @@ -790,7 +788,6 @@ public void start(final IgniteConfiguration cfg, p2pExecSvc, mgmtExecSvc, igfsExecSvc, - dataStreamExecSvc, restExecSvc, affExecSvc, idxExecSvc, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java index 0653effa75449..5b2c3fc53faea 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -1468,9 +1468,6 @@ private static final class IgniteNamedInstance { /** IGFS executor service. */ private ThreadPoolExecutor igfsExecSvc; - /** Data streamer executor service. */ - private ThreadPoolExecutor dataStreamerExecSvc; - /** REST requests executor service. */ private ThreadPoolExecutor restExecSvc; @@ -1693,17 +1690,6 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { p2pExecSvc.allowCoreThreadTimeOut(true); - // Note that we do not pre-start threads here as this pool may not be needed. - dataStreamerExecSvc = new IgniteThreadPoolExecutor( - "data-streamer", - cfg.getGridName(), - cfg.getDataStreamerThreadPoolSize(), - cfg.getDataStreamerThreadPoolSize(), - DFLT_THREAD_KEEP_ALIVE_TIME, - new LinkedBlockingQueue()); - - dataStreamerExecSvc.allowCoreThreadTimeOut(true); - // Note that we do not pre-start threads here as igfs pool may not be needed. igfsExecSvc = new IgniteThreadPoolExecutor( cfg.getIgfsThreadPoolSize(), @@ -1789,7 +1775,7 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { grid = grid0; grid0.start(myCfg, utilityCacheExecSvc, marshCacheExecSvc, execSvc, sysExecSvc, p2pExecSvc, mgmtExecSvc, - igfsExecSvc, dataStreamerExecSvc, restExecSvc, affExecSvc, idxExecSvc, callbackExecSvc, + igfsExecSvc, restExecSvc, affExecSvc, idxExecSvc, callbackExecSvc, new CA() { @Override public void apply() { startLatch.countDown(); @@ -2407,10 +2393,6 @@ private void stopExecutors0(IgniteLogger log) { p2pExecSvc = null; - U.shutdownNow(getClass(), dataStreamerExecSvc, log); - - dataStreamerExecSvc = null; - U.shutdownNow(getClass(), igfsExecSvc, log); igfsExecSvc = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 77a58d38738b3..3df29cf8fb4e5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -84,7 +84,6 @@ import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.internal.GridTopic.TOPIC_COMM_USER; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.AFFINITY_POOL; -import static org.apache.ignite.internal.managers.communication.GridIoPolicy.DATA_STREAMER_POOL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IDX_POOL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IGFS_POOL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MANAGEMENT_POOL; @@ -577,7 +576,6 @@ private void onMessage0(UUID nodeId, GridIoMessage msg, IgniteRunnable msgC) { case MARSH_CACHE_POOL: case IDX_POOL: case IGFS_POOL: - case DATA_STREAMER_POOL: { if (msg.isOrdered()) processOrderedMessage(nodeId, msg, plc, msgC); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java index 18235d2cce5f1..70a73541d2ffa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java @@ -49,9 +49,6 @@ public class GridIoPolicy { /** Pool for handling distributed index range requests. */ public static final byte IDX_POOL = 8; - /** Data streamer execution pool. */ - public static final byte DATA_STREAMER_POOL = 9; - /** * Defines the range of reserved pools that are not available for plugins. * @param key The key. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java index 2c08423a71430..c5a87d4ea1985 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Callable; +import java.util.concurrent.Executor; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import org.apache.ignite.IgniteCheckedException; @@ -972,7 +973,7 @@ public IgniteInternalFuture callLocalSafe(Callable c, boolean sys) { * @param plc Policy to choose executor pool. * @return Future. */ - public IgniteInternalFuture callLocalSafe(Callable c, byte plc) { + private IgniteInternalFuture callLocalSafe(Callable c, byte plc) { try { return callLocal(c, plc); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java index bd33f62cbfc7e..7663735fc26c6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java @@ -21,12 +21,10 @@ import java.util.UUID; import java.util.concurrent.DelayQueue; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.managers.communication.GridIoManager; -import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.deployment.GridDeployment; import org.apache.ignite.internal.processors.GridProcessorAdapter; @@ -37,7 +35,6 @@ import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.GridWorker; -import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.stream.StreamReceiver; @@ -45,23 +42,12 @@ import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.GridTopic.TOPIC_DATASTREAM; +import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL; /** - * Data stream processor. + * */ public class DataStreamProcessor extends GridProcessorAdapter { - /** Data streamer separate pool feature major version. */ - private static final int DATA_STREAMER_POOL_MAJOR_VER = 1; - - /** Data streamer separate pool feature minor version. */ - private static final int DATA_STREAMER_POOL_MINOR_VER = 6; - - /** Data streamer separate pool feature maintenance version. */ - private static final int DATA_STREAMER_POOL_MAINTENANCE_VER = 10; - - /** Default pool for data streamer messages processing. */ - public static final byte DFLT_POLICY = GridIoPolicy.PUBLIC_POOL; - /** Loaders map (access is not supposed to be highly concurrent). */ private Collection ldrs = new GridConcurrentHashSet<>(); @@ -232,15 +218,13 @@ private void processRequest(final UUID nodeId, final DataStreamerRequest req) { IgniteInternalFuture fut = ctx.cache().context().exchange().affinityReadyFuture(rmtAffVer); if (fut != null && !fut.isDone()) { - final byte plc = threadIoPolicy(); - fut.listen(new CI1>() { @Override public void apply(IgniteInternalFuture t) { ctx.closure().runLocalSafe(new Runnable() { @Override public void run() { processRequest(nodeId, req); } - }, plc); + }, false); } }); @@ -356,7 +340,12 @@ private void sendResponse(UUID nodeId, Object resTopic, long reqId, @Nullable Th DataStreamerResponse res = new DataStreamerResponse(reqId, errBytes, forceLocDep); try { - ctx.io().send(nodeId, resTopic, res, threadIoPolicy()); + Byte plc = GridIoManager.currentPolicy(); + + if (plc == null) + plc = PUBLIC_POOL; + + ctx.io().send(nodeId, resTopic, res, plc); } catch (IgniteCheckedException e) { if (ctx.discovery().alive(nodeId)) @@ -366,59 +355,6 @@ else if (log.isDebugEnabled()) } } - /** - * Get IO policy. - * - * @return IO policy. - */ - private static byte threadIoPolicy() { - Byte plc = GridIoManager.currentPolicy(); - - if (plc == null) - plc = DFLT_POLICY; - - return plc; - } - - /** - * Get IO policy for particular node. - * - * @param node Node. - * @return Policy. - */ - public static byte ioPolicy(ClusterNode node) { - assert node != null; - - if (node.isLocal() || node.version().greaterThanEqual( - DATA_STREAMER_POOL_MAJOR_VER, - DATA_STREAMER_POOL_MINOR_VER, - DATA_STREAMER_POOL_MAINTENANCE_VER)) - return GridIoPolicy.DATA_STREAMER_POOL; - else - return DFLT_POLICY; - } - - /** - * Get IO policy for particular node with provided resolver. - * - * @param rslvr Resolver. - * @param node Node. - * @return IO policy. - */ - public static byte ioPolicy(@Nullable IgniteClosure rslvr, ClusterNode node) { - assert node != null; - - Byte res = null; - - if (rslvr != null) - res = rslvr.apply(node); - - if (res == null) - res = ioPolicy(node); - - return res; - } - /** {@inheritDoc} */ @Override public void printMemoryStats() { X.println(">>>"); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java index 46f6380dc22d7..c2f226c537025 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java @@ -39,6 +39,7 @@ import java.util.concurrent.atomic.AtomicReference; import javax.cache.CacheException; import javax.cache.expiry.ExpiryPolicy; + import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; @@ -59,6 +60,7 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; +import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.deployment.GridDeployment; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; @@ -104,12 +106,16 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.internal.GridTopic.TOPIC_DATASTREAM; +import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL; /** * Data streamer implementation. */ @SuppressWarnings("unchecked") public class DataStreamerImpl implements IgniteDataStreamer, Delayed { + /** Default policy reoslver. */ + private static final DefaultIoPolicyResolver DFLT_IO_PLC_RSLVR = new DefaultIoPolicyResolver(); + /** Isolated receiver. */ private static final StreamReceiver ISOLATED_UPDATER = new IsolatedUpdater(); @@ -120,7 +126,7 @@ public class DataStreamerImpl implements IgniteDataStreamer, Delayed private byte[] updaterBytes; /** IO policy resovler for data load request. */ - private IgniteClosure ioPlcRslvr; + private IgniteClosure ioPlcRslvr = DFLT_IO_PLC_RSLVR; /** Max remap count before issuing an error. */ private static final int DFLT_MAX_REMAP_CNT = 32; @@ -1307,12 +1313,14 @@ private void submit(final Collection entries, IgniteInternalFuture fut; - byte plc = DataStreamProcessor.ioPolicy(ioPlcRslvr, node); + Byte plc = ioPlcRslvr.apply(node); + + if (plc == null) + plc = PUBLIC_POOL; - if (isLocNode) { + if (isLocNode && plc == GridIoPolicy.PUBLIC_POOL) { fut = ctx.closure().callLocalSafe( - new DataStreamerUpdateJob(ctx, log, cacheName, entries, false, skipStore, keepBinary, rcvr), - plc); + new DataStreamerUpdateJob(ctx, log, cacheName, entries, false, skipStore, keepBinary, rcvr), false); locFuts.add(fut); @@ -1675,6 +1683,19 @@ else if (ttl == CU.TTL_NOT_CHANGED) } } + /** + * Default IO policy resolver. + */ + private static class DefaultIoPolicyResolver implements IgniteClosure { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public Byte apply(ClusterNode gridNode) { + return PUBLIC_POOL; + } + } + /** * Key object wrapper. Using identity equals prevents slow down in case of hash code collision. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java index 26bfc0df0faaa..59e5e7dffb7e9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java @@ -128,9 +128,6 @@ public Executor poolForPolicy(byte plc) throws IgniteCheckedException { return ctx.getIgfsExecutorService(); - case GridIoPolicy.DATA_STREAMER_POOL: - return ctx.getDataStreamerExecutorService(); - default: { if (plc < 0) throw new IgniteCheckedException("Policy cannot be negative: " + plc); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java index 401b09c51b20c..9fedc35ae4c8e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java @@ -33,7 +33,6 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteDataStreamer; -import org.apache.ignite.IgniteException; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.affinity.Affinity; @@ -50,7 +49,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter; -import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; @@ -61,7 +59,6 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.stream.StreamReceiver; -import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jetbrains.annotations.Nullable; @@ -951,100 +948,6 @@ public void testCustomUserUpdater() throws Exception { } } - /** - * @throws Exception If failed. - */ - public void testLocalDataStreamerDedicatedThreadPool() throws Exception { - try { - useCache = true; - - Ignite ignite = startGrid(1); - - final IgniteCache cache = ignite.cache(null); - - IgniteDataStreamer ldr = ignite.dataStreamer(null); - try { - ldr.receiver(new StreamReceiver() { - @Override public void receive(IgniteCache cache, - Collection> entries) throws IgniteException { - String threadName = Thread.currentThread().getName(); - - cache.put("key", threadName); - } - }); - ldr.addData("key", "value"); - - ldr.tryFlush(); - - GridTestUtils.waitForCondition(new GridAbsPredicate() { - @Override public boolean apply() { - return cache.get("key") != null; - } - }, 3_000); - } - finally { - ldr.close(true); - } - - assertNotNull(cache.get("key")); - - assertTrue(cache.get("key").startsWith("data-streamer")); - - } - finally { - stopAllGrids(); - } - } - - /** - * @throws Exception If failed. - */ - public void testRemoteDataStreamerDedicatedThreadPool() throws Exception { - try { - useCache = true; - - Ignite ignite = startGrid(1); - - useCache = false; - - Ignite client = startGrid(0); - - final IgniteCache cache = ignite.cache(null); - - IgniteDataStreamer ldr = client.dataStreamer(null); - try { - ldr.receiver(new StreamReceiver() { - @Override public void receive(IgniteCache cache, - Collection> entries) throws IgniteException { - String threadName = Thread.currentThread().getName(); - - cache.put("key", threadName); - } - }); - - ldr.addData("key", "value"); - - ldr.tryFlush(); - - GridTestUtils.waitForCondition(new GridAbsPredicate() { - @Override public boolean apply() { - return cache.get("key") != null; - } - }, 3_000); - } - finally { - ldr.close(true); - } - - assertNotNull(cache.get("key")); - - assertTrue(cache.get("key").startsWith("data-streamer")); - } - finally { - stopAllGrids(); - } - } - /** * */ diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java index 8cb32b6d9afd4..f9e2ff4e709df 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java @@ -63,7 +63,6 @@ public GridTestKernalContext(IgniteLogger log, IgniteConfiguration cfg) throws I null, null, null, - null, U.allPluginProviders()); GridTestUtils.setFieldValue(grid(), "cfg", config()); @@ -97,6 +96,11 @@ public void stop(boolean cancel) throws IgniteCheckedException { } } + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridTestKernalContext.class, this, super.toString()); + } + /** * Sets system executor service. * @@ -106,6 +110,7 @@ public void setSystemExecutorService(ExecutorService sysExecSvc) { this.sysExecSvc = sysExecSvc; } + /** * Sets executor service. * @@ -114,9 +119,4 @@ public void setSystemExecutorService(ExecutorService sysExecSvc) { public void setExecutorService(ExecutorService execSvc){ this.execSvc = execSvc; } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(GridTestKernalContext.class, this, super.toString()); - } } From 46e77f912dfd741000c32aae03449df691d522b1 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Tue, 18 Oct 2016 15:01:03 +0500 Subject: [PATCH 274/487] IGNITE-2355 Fixed the test HadoopClientProtocolMultipleServersSelfTest. Close FileSystem after each test to prevent using the one Client delegate for all tests. --- ...ClientProtocolMultipleServersSelfTest.java | 93 +++++++++++-------- 1 file changed, 55 insertions(+), 38 deletions(-) diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java index 04747d0b678f7..0e51938c12f29 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java @@ -24,6 +24,7 @@ import java.util.Collection; import java.util.concurrent.Callable; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; @@ -40,7 +41,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.ignite.IgniteFileSystem; -import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.mapreduce.IgniteHadoopClientProtocolProvider; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.internal.client.GridServerUnreachableException; @@ -61,6 +62,9 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS /** Job name. */ private static final String JOB_NAME = "myJob"; + /** Rest port. */ + private static int restPort; + /** {@inheritDoc} */ @Override protected boolean igfsEnabled() { return true; @@ -71,15 +75,6 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS return true; } - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - super.beforeTest(); - - startGrids(gridCount()); - - awaitPartitionMapExchange(); - } - /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); @@ -88,10 +83,10 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS } /** {@inheritDoc} */ - @Override protected CacheConfiguration dataCacheConfiguration() { - CacheConfiguration cfg = super.dataCacheConfiguration(); + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); - cfg.setBackups(1); + cfg.getConnectorConfiguration().setPort(restPort++); return cfg; } @@ -149,15 +144,22 @@ public void checkJobSubmit(Configuration conf) throws Exception { */ @SuppressWarnings("ConstantConditions") public void testMultipleAddresses() throws Exception { - beforeJob(); + try { + restPort = REST_PORT; + + startGrids(gridCount()); - stopGrid(0); + beforeJob(); - U.sleep(5000); + U.sleep(5000); - checkJobSubmit(configMultipleAddrs(gridCount())); + checkJobSubmit(configMultipleAddrs(gridCount())); + } + finally { + FileSystem fs = FileSystem.get(configMultipleAddrs(gridCount())); - startGrid(0); + fs.close(); + } } /** @@ -165,21 +167,25 @@ public void testMultipleAddresses() throws Exception { */ @SuppressWarnings({"ConstantConditions", "ThrowableResultOfMethodCallIgnored"}) public void testSingleAddress() throws Exception { - stopGrid(0); - - U.sleep(5000); - - GridTestUtils.assertThrowsAnyCause(log, new Callable() { - @Override public Object call() throws Exception { - checkJobSubmit(configSingleAddress()); - return null; - } - }, - GridServerUnreachableException.class, "Failed to connect to any of the servers in list"); - - startGrid(0); + try { + // Don't use REST_PORT to test connection fails if the only this port is configured + restPort = REST_PORT + 1; + + startGrids(gridCount()); + + GridTestUtils.assertThrowsAnyCause(log, new Callable() { + @Override public Object call() throws Exception { + checkJobSubmit(configSingleAddress()); + return null; + } + }, + GridServerUnreachableException.class, "Failed to connect to any of the servers in list"); + } + finally { + FileSystem fs = FileSystem.get(configSingleAddress()); - awaitPartitionMapExchange(); + fs.close(); + } } /** @@ -187,17 +193,28 @@ public void testSingleAddress() throws Exception { */ @SuppressWarnings("ConstantConditions") public void testMixedAddrs() throws Exception { - beforeJob(); + try { + restPort = REST_PORT; + + startGrids(gridCount()); - stopGrid(1); + beforeJob(); - U.sleep(5000); + stopGrid(1); - checkJobSubmit(configMixed()); + U.sleep(5000); - startGrid(1); + checkJobSubmit(configMixed()); - awaitPartitionMapExchange(); + startGrid(1); + + awaitPartitionMapExchange(); + } + finally { + FileSystem fs = FileSystem.get(configMixed()); + + fs.close(); + } } /** From f5031d2a22979869a032a1d53c097609d96e94cf Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Tue, 18 Oct 2016 13:20:30 +0300 Subject: [PATCH 275/487] IGNITE-1924 Incomplete marshaller cache rebalancing causes Grid hangs under SSL (HOTFIX) --- .../apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java index 273df5d9d69f7..b3e820153e5dc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java @@ -472,8 +472,6 @@ private ByteBuffer allocateAppBuff() { */ private void readFromNet() throws IgniteCheckedException { try { - inNetBuf.clear(); - int read = ch.read(inNetBuf); if (read == -1) From 926aa519d7dc37ac5e8960f4368646ae94171290 Mon Sep 17 00:00:00 2001 From: vozerov-gridgain Date: Wed, 26 Oct 2016 11:13:52 +0300 Subject: [PATCH 276/487] Removed unused imports from GridClosureProcessor. --- .../internal/processors/closure/GridClosureProcessor.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java index c5a87d4ea1985..9d295d3ced3b8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java @@ -27,7 +27,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Callable; -import java.util.concurrent.Executor; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import org.apache.ignite.IgniteCheckedException; @@ -80,9 +79,6 @@ import static org.apache.ignite.compute.ComputeJobResultPolicy.FAILOVER; import static org.apache.ignite.compute.ComputeJobResultPolicy.REDUCE; -import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IGFS_POOL; -import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL; -import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_NO_FAILOVER; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SUBGRID; From 3d01ee1edf72f17bf6c027afad88fc29e321970a Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 18 Oct 2016 17:17:17 +0300 Subject: [PATCH 277/487] IGNITE-4030 Streamline PlatformTarget operation methods Conflicts: modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs --- .../platform/PlatformAbstractTarget.java | 54 ------------------- .../processors/platform/PlatformTarget.java | 20 ------- .../platform/cache/PlatformCache.java | 27 ++++------ .../cache/affinity/PlatformAffinity.java | 4 +- .../query/PlatformAbstractQueryCursor.java | 4 +- .../query/PlatformContinuousQueryProxy.java | 3 +- .../cluster/PlatformClusterGroup.java | 4 +- .../platform/compute/PlatformCompute.java | 9 +--- .../datastreamer/PlatformDataStreamer.java | 9 +--- .../datastructures/PlatformAtomicLong.java | 27 ++++------ .../PlatformAtomicReference.java | 8 ++- .../PlatformAtomicSequence.java | 31 +++++------ .../platform/services/PlatformServices.java | 13 +++-- .../transactions/PlatformTransactions.java | 17 ++---- .../core/src/impl/cache/query/query_impl.cpp | 6 +-- .../core/src/impl/interop/interop_target.cpp | 2 +- .../cpp/jni/include/ignite/jni/exports.h | 2 - .../cpp/jni/include/ignite/jni/java.h | 4 -- .../platforms/cpp/jni/project/vs/module.def | 5 +- modules/platforms/cpp/jni/src/exports.cpp | 8 --- modules/platforms/cpp/jni/src/java.cpp | 23 -------- .../Impl/Cache/CacheAffinityImpl.cs | 2 +- .../Impl/Cache/CacheImpl.cs | 12 ++--- .../Impl/Cache/Query/AbstractQueryCursor.cs | 4 +- .../Continuous/ContinuousQueryHandleImpl.cs | 2 +- .../Impl/Cluster/ClusterGroupImpl.cs | 2 +- .../Impl/Compute/ComputeImpl.cs | 4 +- .../Impl/DataStructures/AtomicLong.cs | 14 ++--- .../Impl/DataStructures/AtomicReference.cs | 4 +- .../Impl/DataStructures/AtomicSequence.cs | 14 ++--- .../Impl/Datastream/DataStreamerImpl.cs | 20 +++---- .../Apache.Ignite.Core/Impl/Events/Events.cs | 4 +- .../Apache.Ignite.Core/Impl/PlatformTarget.cs | 41 +------------- .../Impl/Services/Services.cs | 4 +- .../Impl/Transactions/TransactionsImpl.cs | 12 ++--- .../Impl/Unmanaged/IgniteJniNativeMethods.cs | 9 +--- .../Impl/Unmanaged/UnmanagedUtils.cs | 12 +---- 37 files changed, 114 insertions(+), 326 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java index 5c7f260b12b4f..840c52794ae34 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java @@ -94,16 +94,6 @@ protected PlatformAbstractTarget(PlatformContext platformCtx) { } } - /** {@inheritDoc} */ - @Override public long outLong(int type) throws Exception { - try { - return processOutLong(type); - } - catch (Exception e) { - throw convertException(e); - } - } - /** {@inheritDoc} */ @Override public void outStream(int type, long memPtr) throws Exception { try (PlatformMemory mem = platformCtx.memory().get(memPtr)) { @@ -150,26 +140,6 @@ protected PlatformAbstractTarget(PlatformContext platformCtx) { } } - /** {@inheritDoc} */ - @Override public void inObjectStreamOutStream(int type, Object arg, long inMemPtr, long outMemPtr) throws Exception { - try (PlatformMemory inMem = platformCtx.memory().get(inMemPtr)) { - BinaryRawReaderEx reader = platformCtx.reader(inMem); - - try (PlatformMemory outMem = platformCtx.memory().get(outMemPtr)) { - PlatformOutputStream out = outMem.output(); - - BinaryRawWriterEx writer = platformCtx.writer(out); - - processInObjectStreamOutStream(type, arg, reader, writer); - - out.synchronize(); - } - } - catch (Exception e) { - throw convertException(e); - } - } - /** {@inheritDoc} */ @Override public Object inObjectStreamOutObjectStream(int type, Object arg, long inMemPtr, long outMemPtr) throws Exception { @@ -326,20 +296,6 @@ protected Object processInStreamOutObject(int type, BinaryRawReaderEx reader) th return throwUnsupported(type); } - /** - * Process IN-OUT operation. - * - * @param type Type. - * @param arg Argument. - * @param reader Binary reader. - * @param writer Binary writer. - * @throws IgniteCheckedException In case of exception. - */ - protected void processInObjectStreamOutStream(int type, @Nullable Object arg, BinaryRawReaderEx reader, - BinaryRawWriterEx writer) throws IgniteCheckedException { - throwUnsupported(type); - } - /** * Process IN-OUT operation. * @@ -354,16 +310,6 @@ protected Object processInObjectStreamOutObjectStream(int type, @Nullable Object return throwUnsupported(type); } - /** - * Process OUT operation. - * - * @param type Type. - * @throws IgniteCheckedException In case of exception. - */ - protected long processOutLong(int type) throws IgniteCheckedException { - return throwUnsupported(type); - } - /** * Process OUT operation. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java index 3ab5d7c76e37f..805fd5e54767e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java @@ -66,17 +66,6 @@ public interface PlatformTarget { */ public void inStreamOutStream(int type, long inMemPtr, long outMemPtr) throws Exception; - /** - * Operation accepting an object and a memory stream and returning result to another memory stream. - * - * @param type Operation type. - * @param arg Argument (optional). - * @param inMemPtr Input memory pointer. - * @param outMemPtr Output memory pointer. - * @throws Exception In case of failure. - */ - public void inObjectStreamOutStream(int type, @Nullable Object arg, long inMemPtr, long outMemPtr) throws Exception; - /** * Operation accepting an object and a memory stream and returning result to another memory stream and an object. * @@ -90,15 +79,6 @@ public interface PlatformTarget { public Object inObjectStreamOutObjectStream(int type, @Nullable Object arg, long inMemPtr, long outMemPtr) throws Exception; - /** - * Operation returning long result. - * - * @param type Operation type. - * @return Result. - * @throws Exception In case of failure. - */ - public long outLong(int type) throws Exception; - /** * Operation returning result to memory stream. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index 558a9b3538525..01790e73e089d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -387,23 +387,6 @@ public PlatformCache(PlatformContext platformCtx, IgniteCache cache, boolean kee this.exts = exts; } - /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { - switch (type) { - case OP_CLEAR_CACHE: - cache.clear(); - - return TRUE; - - case OP_REMOVE_ALL2: - cache.removeAll(); - - return TRUE; - } - - return super.processOutLong(type); - } - /** * @return Raw cache. */ @@ -1095,6 +1078,16 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc, IgniteCache cache return TRUE; } + + case OP_CLEAR_CACHE: + cache.clear(); + + return TRUE; + + case OP_REMOVE_ALL2: + cache.removeAll(); + + return TRUE; } return super.processInLongOutLong(type, val); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java index 41b58aaaf2c19..12df18890889e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java @@ -292,10 +292,10 @@ public PlatformAffinity(PlatformContext platformCtx, GridKernalContext igniteCtx } /** {@inheritDoc} */ - @Override public long outLong(int type) throws Exception { + @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException { if (type == OP_PARTITIONS) return aff.partitions(); - return super.outLong(type); + return super.processInLongOutLong(type, val); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java index ff28b818066b9..6a259caa63e83 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java @@ -136,7 +136,7 @@ public PlatformAbstractQueryCursor(PlatformContext platformCtx, QueryCursorEx } /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { switch (type) { case OP_ITERATOR: iter = cursor.iterator(); @@ -154,7 +154,7 @@ public PlatformAbstractQueryCursor(PlatformContext platformCtx, QueryCursorEx return iter.hasNext() ? TRUE : FALSE; } - return super.processOutLong(type); + return super.processInLongOutLong(type, val); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java index a4d7cad5abb39..04f17ff2804c1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.platform.cache.query; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; @@ -45,7 +46,7 @@ public PlatformContinuousQueryProxy(PlatformContext platformCtx, PlatformContinu } /** {@inheritDoc} */ - @Override public long outLong(int type) throws Exception { + @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException { qry.close(); return 0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java index 04a117360ef72..5894ff82422b0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java @@ -338,7 +338,7 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { } /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { switch (type) { case OP_RESET_METRICS: { assert prj instanceof IgniteCluster; // Can only be invoked on top-level cluster group. @@ -349,7 +349,7 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { } } - return super.processOutLong(type); + return super.processInLongOutLong(type, val); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java index 0c10a5304fa41..8ff15d5513555 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java @@ -136,14 +136,7 @@ public PlatformCompute(PlatformContext platformCtx, ClusterGroup grp, String pla return TRUE; } - } - - return super.processInLongOutLong(type, val); - } - /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { - switch (type) { case OP_WITH_NO_FAILOVER: { compute.withNoFailover(); computeForPlatform.withNoFailover(); @@ -152,7 +145,7 @@ public PlatformCompute(PlatformContext platformCtx, ClusterGroup grp, String pla } } - return super.processOutLong(type); + return super.processInLongOutLong(type, val); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java index 2822b7f9125b6..cd5fba059c218 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java @@ -217,14 +217,7 @@ else if (plc == PLC_FLUSH) return TRUE; } - } - - return super.processInLongOutLong(type, val); - } - /** {@inheritDoc} */ - @Override public long processOutLong(int type) throws IgniteCheckedException { - switch (type) { case OP_ALLOW_OVERWRITE: return ldr.allowOverwrite() ? TRUE : FALSE; @@ -238,7 +231,7 @@ else if (plc == PLC_FLUSH) return ldr.perNodeParallelOperations(); } - return super.processOutLong(type); + return super.processInLongOutLong(type, val); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java index 53319560f95c1..811e38bcadd79 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java @@ -99,8 +99,17 @@ public PlatformAtomicLong(PlatformContext ctx, GridCacheAtomicLongImpl atomicLon } /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { switch (type) { + case OP_ADD_AND_GET: + return atomicLong.addAndGet(val); + + case OP_GET_AND_ADD: + return atomicLong.getAndAdd(val); + + case OP_GET_AND_SET: + return atomicLong.getAndSet(val); + case OP_CLOSE: atomicLong.close(); @@ -125,22 +134,6 @@ public PlatformAtomicLong(PlatformContext ctx, GridCacheAtomicLongImpl atomicLon return atomicLong.removed() ? TRUE : FALSE; } - return super.processOutLong(type); - } - - /** {@inheritDoc} */ - @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { - switch (type) { - case OP_ADD_AND_GET: - return atomicLong.addAndGet(val); - - case OP_GET_AND_ADD: - return atomicLong.getAndAdd(val); - - case OP_GET_AND_SET: - return atomicLong.getAndSet(val); - } - return super.processInLongOutLong(type, val); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java index e5fc08d8b69e4..63b5b86ec6b69 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java @@ -24,7 +24,6 @@ import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; -import org.omg.CORBA.TRANSACTION_REQUIRED; /** * Platform atomic reference wrapper. @@ -135,7 +134,7 @@ private PlatformAtomicReference(PlatformContext ctx, GridCacheAtomicReferenceImp } /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { switch (type) { case OP_CLOSE: atomicRef.close(); @@ -146,7 +145,6 @@ private PlatformAtomicReference(PlatformContext ctx, GridCacheAtomicReferenceImp return atomicRef.removed() ? TRUE : FALSE; } - return super.processOutLong(type); + return super.processInLongOutLong(type, val); } -} - +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java index ec946cac303e1..c35273189f13f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java @@ -71,8 +71,19 @@ public PlatformAtomicSequence(PlatformContext ctx, IgniteAtomicSequence atomicSe /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { switch (type) { + case OP_ADD_AND_GET: + return atomicSeq.addAndGet(val); + + case OP_GET_AND_ADD: + return atomicSeq.getAndAdd(val); + + case OP_SET_BATCH_SIZE: + atomicSeq.batchSize((int)val); + + return TRUE; + case OP_CLOSE: atomicSeq.close(); @@ -94,24 +105,6 @@ public PlatformAtomicSequence(PlatformContext ctx, IgniteAtomicSequence atomicSe return atomicSeq.batchSize(); } - return super.processOutLong(type); - } - - /** {@inheritDoc} */ - @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { - switch (type) { - case OP_ADD_AND_GET: - return atomicSeq.addAndGet(val); - - case OP_GET_AND_ADD: - return atomicSeq.getAndAdd(val); - - case OP_SET_BATCH_SIZE: - atomicSeq.batchSize((int)val); - - return TRUE; - } - return super.processInLongOutLong(type, val); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java index 58989793800fb..962a4c0af00c3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java @@ -223,7 +223,7 @@ private ServiceDescriptor findDescriptor(String name) { } /** {@inheritDoc} */ - @Override protected void processInObjectStreamOutStream(int type, Object arg, BinaryRawReaderEx reader, + @Override protected Object processInObjectStreamOutObjectStream(int type, Object arg, BinaryRawReaderEx reader, BinaryRawWriterEx writer) throws IgniteCheckedException { switch (type) { case OP_INVOKE: { @@ -252,12 +252,11 @@ private ServiceDescriptor findDescriptor(String name) { PlatformUtils.writeInvocationResult(writer, null, e); } - return; + return null; } - - default: - super.processInObjectStreamOutStream(type, arg, reader, writer); } + + return super.processInObjectStreamOutObjectStream(type, arg, reader, writer); } /** {@inheritDoc} */ @@ -316,7 +315,7 @@ private ServiceDescriptor findDescriptor(String name) { } /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { + @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { switch (type) { case OP_CANCEL_ALL: services.cancelAll(); @@ -324,7 +323,7 @@ private ServiceDescriptor findDescriptor(String name) { return TRUE; } - return super.processOutLong(type); + return super.processInLongOutLong(type, val); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java index 9c8ad503b9eb8..1b41712e71793 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java @@ -152,18 +152,6 @@ private Transaction tx(long id) { return tx; } - /** {@inheritDoc} */ - @Override protected long processOutLong(int type) throws IgniteCheckedException { - switch (type) { - case OP_RESET_METRICS: - txs.resetMetrics(); - - return TRUE; - } - - return super.processOutLong(type); - } - /** {@inheritDoc} */ @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException { switch (type) { @@ -185,6 +173,11 @@ private Transaction tx(long id) { case OP_STATE: return tx(val).state().ordinal(); + + case OP_RESET_METRICS: + txs.resetMetrics(); + + return TRUE; } return super.processInLongOutLong(type, val); diff --git a/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp b/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp index c65e1e8ee5958..aaeb8228367f1 100644 --- a/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp +++ b/modules/platforms/cpp/core/src/impl/cache/query/query_impl.cpp @@ -67,7 +67,7 @@ namespace ignite delete batch; // 2. Close the cursor. - env.Get()->Context()->TargetOutLong(javaRef, OP_ITERATOR_CLOSE); + env.Get()->Context()->TargetInLongOutLong(javaRef, OP_ITERATOR_CLOSE, 0); // 3. Release Java reference. JniContext::Release(javaRef); @@ -199,7 +199,7 @@ namespace ignite JniErrorInfo jniErr; - env.Get()->Context()->TargetOutLong(javaRef, OP_ITERATOR, &jniErr); + env.Get()->Context()->TargetInLongOutLong(javaRef, OP_ITERATOR, 0, &jniErr); IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); @@ -249,7 +249,7 @@ namespace ignite { JniErrorInfo jniErr; - bool res = env.Get()->Context()->TargetOutLong(javaRef, OP_ITERATOR_HAS_NEXT, &jniErr) == 1; + bool res = env.Get()->Context()->TargetInLongOutLong(javaRef, OP_ITERATOR_HAS_NEXT, 0, &jniErr) == 1; IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err); diff --git a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp index cf120d4095012..196c3f691d99f 100644 --- a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp +++ b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp @@ -115,7 +115,7 @@ namespace ignite { JniErrorInfo jniErr; - long long res = env.Get()->Context()->TargetOutLong(javaRef, opType, &jniErr); + long long res = env.Get()->Context()->TargetInLongOutLong(javaRef, opType, 0, &jniErr); IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); diff --git a/modules/platforms/cpp/jni/include/ignite/jni/exports.h b/modules/platforms/cpp/jni/include/ignite/jni/exports.h index 586c389262a8f..3a98eda5ebddd 100644 --- a/modules/platforms/cpp/jni/include/ignite/jni/exports.h +++ b/modules/platforms/cpp/jni/include/ignite/jni/exports.h @@ -61,9 +61,7 @@ extern "C" { long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr); void IGNITE_CALL IgniteTargetInStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, long long inMemPtr, long long outMemPtr); void* IGNITE_CALL IgniteTargetInStreamOutObject(gcj::JniContext* ctx, void* obj, int opType, long long memPtr); - void IGNITE_CALL IgniteTargetInObjectStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, void* arg, long long inMemPtr, long long outMemPtr); void* IGNITE_CALL IgniteTargetInObjectStreamOutObjectStream(gcj::JniContext* ctx, void* obj, int opType, void* arg, long long inMemPtr, long long outMemPtr); - long long IGNITE_CALL IgniteTargetOutLong(gcj::JniContext* ctx, void* obj, int opType); void IGNITE_CALL IgniteTargetOutStream(gcj::JniContext* ctx, void* obj, int opType, long long memPtr); void* IGNITE_CALL IgniteTargetOutObject(gcj::JniContext* ctx, void* obj, int opType); void IGNITE_CALL IgniteTargetListenFuture(gcj::JniContext* ctx, void* obj, long long futId, int typ); diff --git a/modules/platforms/cpp/jni/include/ignite/jni/java.h b/modules/platforms/cpp/jni/include/ignite/jni/java.h index 4c79a61342b93..9dddd8cc723b0 100644 --- a/modules/platforms/cpp/jni/include/ignite/jni/java.h +++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h @@ -274,11 +274,9 @@ namespace ignite jmethodID m_PlatformTarget_inLongOutLong; jmethodID m_PlatformTarget_inStreamOutLong; jmethodID m_PlatformTarget_inStreamOutObject; - jmethodID m_PlatformTarget_outLong; jmethodID m_PlatformTarget_outStream; jmethodID m_PlatformTarget_outObject; jmethodID m_PlatformTarget_inStreamOutStream; - jmethodID m_PlatformTarget_inObjectStreamOutStream; jmethodID m_PlatformTarget_inObjectStreamOutObjectStream; jmethodID m_PlatformTarget_listenFuture; jmethodID m_PlatformTarget_listenFutureForOperation; @@ -456,9 +454,7 @@ namespace ignite long long TargetInStreamOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); void TargetInStreamOutStream(jobject obj, int opType, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL); jobject TargetInStreamOutObject(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); - void TargetInObjectStreamOutStream(jobject obj, int opType, void* arg, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL); jobject TargetInObjectStreamOutObjectStream(jobject obj, int opType, void* arg, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL); - long long TargetOutLong(jobject obj, int opType, JniErrorInfo* errInfo = NULL); void TargetOutStream(jobject obj, int opType, long long memPtr, JniErrorInfo* errInfo = NULL); jobject TargetOutObject(jobject obj, int opType, JniErrorInfo* errInfo = NULL); void TargetListenFuture(jobject obj, long long futId, int typ); diff --git a/modules/platforms/cpp/jni/project/vs/module.def b/modules/platforms/cpp/jni/project/vs/module.def index d9bc4113ecd7e..258e80cac6e6d 100644 --- a/modules/platforms/cpp/jni/project/vs/module.def +++ b/modules/platforms/cpp/jni/project/vs/module.def @@ -6,7 +6,6 @@ IgniteIgnitionInstance @3 IgniteIgnitionEnvironmentPointer @4 IgniteIgnitionStop @5 IgniteIgnitionStopAll @6 -IgniteTargetOutLong @7 IgniteProcessorReleaseStart @8 IgniteProcessorProjection @9 IgniteProcessorCache @10 @@ -20,7 +19,7 @@ IgniteTargetInStreamOutObject @17 IgniteTargetInStreamOutLong @18 IgniteTargetOutStream @19 IgniteTargetInStreamOutStream @20 -IgniteTargetInObjectStreamOutStream @21 +IgniteTargetInObjectStreamOutObjectStream @21 IgniteTargetListenFuture @22 IgniteTargetListenFutureForOperation @23 IgniteTargetInLongOutLong @24 @@ -51,4 +50,4 @@ IgniteProcessorGetCacheNames @133 IgniteSetConsoleHandler @135 IgniteRemoveConsoleHandler @136 IgniteProcessorLoggerIsLevelEnabled @137 -IgniteProcessorLoggerLog @138 \ No newline at end of file +IgniteProcessorLoggerLog @138 diff --git a/modules/platforms/cpp/jni/src/exports.cpp b/modules/platforms/cpp/jni/src/exports.cpp index ee2f5c781ab56..e87cbd3333917 100644 --- a/modules/platforms/cpp/jni/src/exports.cpp +++ b/modules/platforms/cpp/jni/src/exports.cpp @@ -162,18 +162,10 @@ extern "C" { return ctx->TargetInStreamOutObject(static_cast(obj), opType, memPtr); } - void IGNITE_CALL IgniteTargetInObjectStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, void* arg, long long inMemPtr, long long outMemPtr) { - ctx->TargetInObjectStreamOutStream(static_cast(obj), opType, arg, inMemPtr, outMemPtr); - } - void* IGNITE_CALL IgniteTargetInObjectStreamOutObjectStream(gcj::JniContext* ctx, void* obj, int opType, void* arg, long long inMemPtr, long long outMemPtr) { return ctx->TargetInObjectStreamOutObjectStream(static_cast(obj), opType, arg, inMemPtr, outMemPtr); } - long long IGNITE_CALL IgniteTargetOutLong(gcj::JniContext* ctx, void* obj, int opType) { - return ctx->TargetOutLong(static_cast(obj), opType); - } - void IGNITE_CALL IgniteTargetOutStream(gcj::JniContext* ctx, void* obj, int opType, long long memPtr) { ctx->TargetOutStream(static_cast(obj), opType, memPtr); } diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp index 18cf672028b36..e2c9bf7fa3089 100644 --- a/modules/platforms/cpp/jni/src/java.cpp +++ b/modules/platforms/cpp/jni/src/java.cpp @@ -224,9 +224,7 @@ namespace ignite JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_LONG = JniMethod("inStreamOutLong", "(IJ)J", false); JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_OBJECT = JniMethod("inStreamOutObject", "(IJ)Ljava/lang/Object;", false); JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_STREAM = JniMethod("inStreamOutStream", "(IJJ)V", false); - JniMethod M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_STREAM = JniMethod("inObjectStreamOutStream", "(ILjava/lang/Object;JJ)V", false); JniMethod M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_OBJECT_STREAM = JniMethod("inObjectStreamOutObjectStream", "(ILjava/lang/Object;JJ)Ljava/lang/Object;", false); - JniMethod M_PLATFORM_TARGET_OUT_LONG = JniMethod("outLong", "(I)J", false); JniMethod M_PLATFORM_TARGET_OUT_STREAM = JniMethod("outStream", "(IJ)V", false); JniMethod M_PLATFORM_TARGET_OUT_OBJECT = JniMethod("outObject", "(I)Ljava/lang/Object;", false); JniMethod M_PLATFORM_TARGET_LISTEN_FUTURE = JniMethod("listenFuture", "(JI)V", false); @@ -541,11 +539,9 @@ namespace ignite m_PlatformTarget_inLongOutLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_LONG_OUT_LONG); m_PlatformTarget_inStreamOutLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_LONG); m_PlatformTarget_inStreamOutObject = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_OBJECT); - m_PlatformTarget_outLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_OUT_LONG); m_PlatformTarget_outStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_OUT_STREAM); m_PlatformTarget_outObject = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_OUT_OBJECT); m_PlatformTarget_inStreamOutStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_STREAM); - m_PlatformTarget_inObjectStreamOutStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_STREAM); m_PlatformTarget_inObjectStreamOutObjectStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_OBJECT_STREAM); m_PlatformTarget_listenFuture = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_LISTEN_FUTURE); m_PlatformTarget_listenFutureForOperation = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_LISTEN_FOR_OPERATION); @@ -1371,14 +1367,6 @@ namespace ignite return LocalToGlobal(env, res); } - void JniContext::TargetInObjectStreamOutStream(jobject obj, int opType, void* arg, long long inMemPtr, long long outMemPtr, JniErrorInfo* err) { - JNIEnv* env = Attach(); - - env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformTarget_inObjectStreamOutStream, opType, arg, inMemPtr, outMemPtr); - - ExceptionCheck(env, err); - } - jobject JniContext::TargetInObjectStreamOutObjectStream(jobject obj, int opType, void* arg, long long inMemPtr, long long outMemPtr, JniErrorInfo* err) { JNIEnv* env = Attach(); @@ -1389,17 +1377,6 @@ namespace ignite return LocalToGlobal(env, res); } - long long JniContext::TargetOutLong(jobject obj, int opType, JniErrorInfo* err) - { - JNIEnv* env = Attach(); - - jlong res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformTarget_outLong, opType); - - ExceptionCheck(env, err); - - return res; - } - void JniContext::TargetOutStream(jobject obj, int opType, long long memPtr, JniErrorInfo* err) { JNIEnv* env = Attach(); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs index 4ff1f2e5281c1..f09a119700014 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs @@ -104,7 +104,7 @@ internal class CacheAffinityImpl : PlatformTarget, ICacheAffinity /** */ public int Partitions { - get { return (int) DoOutOp(OpPartitions); } + get { return (int) DoOutInOp(OpPartitions); } } /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs index b1cf611abe11f..359611d960a90 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs @@ -632,7 +632,7 @@ public void LocalEvict(IEnumerable keys) /** */ public void Clear() { - DoOutOp((int) CacheOp.ClearCache); + DoOutInOp((int) CacheOp.ClearCache); } /** */ @@ -742,7 +742,7 @@ public Task RemoveAllAsync(IEnumerable keys) /** */ public void RemoveAll() { - DoOutOp((int) CacheOp.RemoveAll2); + DoOutInOp((int) CacheOp.RemoveAll2); } /** */ @@ -783,7 +783,7 @@ private int Size0(bool loc, params CachePeekMode[] modes) var op = loc ? CacheOp.SizeLoc : CacheOp.Size; - return (int) DoOutInOpLong((int) op, modes0); + return (int) DoOutInOp((int) op, modes0); } /** */ @@ -1312,7 +1312,7 @@ private CacheResult DoOutInOpNullable(CacheOp cacheOp, TK x) /** */ public void Enter(long id) { - DoOutInOpLong((int) CacheOp.EnterLock, id); + DoOutInOp((int) CacheOp.EnterLock, id); } /** */ @@ -1328,13 +1328,13 @@ public bool TryEnter(long id, TimeSpan timeout) /** */ public void Exit(long id) { - DoOutInOpLong((int) CacheOp.ExitLock, id); + DoOutInOp((int) CacheOp.ExitLock, id); } /** */ public void Close(long id) { - DoOutInOpLong((int) CacheOp.CloseLock, id); + DoOutInOp((int) CacheOp.CloseLock, id); } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs index e6092d7ddb115..95c6a36f59985 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/AbstractQueryCursor.cs @@ -100,7 +100,7 @@ protected override void Dispose(bool disposing) { try { - DoOutOp(OpIteratorClose); + DoOutInOp(OpIteratorClose); } finally { @@ -125,7 +125,7 @@ public IEnumerator GetEnumerator() throw new InvalidOperationException("Failed to get enumerator entries because " + "GetAll() method has already been called."); - DoOutOp(OpIterator); + DoOutInOp(OpIterator); _iterCalled = true; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs index b66dc48b332c2..6139d8be51fdd 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/Continuous/ContinuousQueryHandleImpl.cs @@ -225,7 +225,7 @@ public void Dispose() try { - UU.TargetOutLong(_nativeQry, 0); + UU.TargetInLongOutLong(_nativeQry, 0, 0); } finally { diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs index 912d6ed58bc7d..9a202e1d5aacc 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs @@ -608,7 +608,7 @@ public int[] GetSchema(int typeId, int schemaId) /// public void ResetMetrics() { - DoOutOp(OpResetMetrics); + DoOutInOp(OpResetMetrics); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs index df68e1cf669cd..bc7c7d9e6d05d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs @@ -107,7 +107,7 @@ public IClusterGroup ClusterGroup /// public void WithNoFailover() { - DoOutOp(OpWithNoFailover); + DoOutInOp(OpWithNoFailover); } /// @@ -117,7 +117,7 @@ public void WithNoFailover() /// Computation timeout in milliseconds. public void WithTimeout(long timeout) { - DoOutInOpLong(OpWithTimeout, timeout); + DoOutInOp(OpWithTimeout, timeout); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicLong.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicLong.cs index 571e6fd2707c0..0c4bf84b17ceb 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicLong.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicLong.cs @@ -68,31 +68,31 @@ public string Name /** */ public long Read() { - return DoOutOp((int) Op.Get); + return DoOutInOp((int) Op.Get); } /** */ public long Increment() { - return DoOutOp((int) Op.IncrementAndGet); + return DoOutInOp((int) Op.IncrementAndGet); } /** */ public long Add(long value) { - return DoOutInOpLong((int) Op.AddAndGet, value); + return DoOutInOp((int) Op.AddAndGet, value); } /** */ public long Decrement() { - return DoOutOp((int) Op.DecrementAndGet); + return DoOutInOp((int) Op.DecrementAndGet); } /** */ public long Exchange(long value) { - return DoOutInOpLong((int) Op.GetAndSet, value); + return DoOutInOp((int) Op.GetAndSet, value); } /** */ @@ -108,13 +108,13 @@ public long CompareExchange(long value, long comparand) /** */ public void Close() { - DoOutOp((int) Op.Close); + DoOutInOp((int) Op.Close); } /** */ public bool IsClosed() { - return DoOutOp((int) Op.IsClosed) == True; + return DoOutInOp((int) Op.IsClosed) == True; } } } \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs index 75e36d1c73b91..4ca4b249b3b2a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicReference.cs @@ -82,13 +82,13 @@ public T CompareExchange(T value, T comparand) /** */ public bool IsClosed { - get { return DoOutOp((int) Op.IsClosed) == True; } + get { return DoOutInOp((int) Op.IsClosed) == True; } } /** */ public void Close() { - DoOutOp((int) Op.Close); + DoOutInOp((int) Op.Close); } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs index b7b924ed246a0..f7fc6b7fb2e44 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/DataStructures/AtomicSequence.cs @@ -65,38 +65,38 @@ public string Name /** */ public long Read() { - return DoOutOp((int) Op.Get); + return DoOutInOp((int) Op.Get); } /** */ public long Increment() { - return DoOutOp((int) Op.IncrementAndGet); + return DoOutInOp((int) Op.IncrementAndGet); } /** */ public long Add(long value) { - return DoOutInOpLong((int) Op.AddAndGet, value); + return DoOutInOp((int) Op.AddAndGet, value); } /** */ public int BatchSize { - get { return (int) DoOutOp((int) Op.GetBatchSize); } - set { DoOutInOpLong((int) Op.SetBatchSize, value); } + get { return (int) DoOutInOp((int) Op.GetBatchSize); } + set { DoOutInOp((int) Op.SetBatchSize, value); } } /** */ public bool IsClosed { - get { return DoOutOp((int) Op.IsClosed) == True; } + get { return DoOutInOp((int) Op.IsClosed) == True; } } /** */ public void Close() { - DoOutOp((int) Op.Close); + DoOutInOp((int) Op.Close); } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs index 8893fc5558695..b9e3030e90c4c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs @@ -156,7 +156,7 @@ public DataStreamerImpl(IUnmanagedTarget target, Marshaller marsh, string cacheN _hnd = marsh.Ignite.HandleRegistry.Allocate(thisRef); // Start topology listening. This call will ensure that buffer size member is updated. - DoOutInOpLong(OpListenTopology, _hnd); + DoOutInOp(OpListenTopology, _hnd); // Membar to ensure fields initialization before leaving constructor. Thread.MemoryBarrier(); @@ -184,7 +184,7 @@ public bool AllowOverwrite { ThrowIfDisposed(); - return DoOutOp(OpAllowOverwrite) == True; + return DoOutInOp(OpAllowOverwrite) == True; } finally { @@ -199,7 +199,7 @@ public bool AllowOverwrite { ThrowIfDisposed(); - DoOutInOpLong(OpSetAllowOverwrite, value ? True : False); + DoOutInOp(OpSetAllowOverwrite, value ? True : False); } finally { @@ -219,7 +219,7 @@ public bool SkipStore { ThrowIfDisposed(); - return DoOutOp(OpSkipStore) == True; + return DoOutInOp(OpSkipStore) == True; } finally { @@ -234,7 +234,7 @@ public bool SkipStore { ThrowIfDisposed(); - DoOutInOpLong(OpSetSkipStore, value ? True : False); + DoOutInOp(OpSetSkipStore, value ? True : False); } finally { @@ -254,7 +254,7 @@ public int PerNodeBufferSize { ThrowIfDisposed(); - return (int) DoOutOp(OpPerNodeBufferSize); + return (int) DoOutInOp(OpPerNodeBufferSize); } finally { @@ -269,7 +269,7 @@ public int PerNodeBufferSize { ThrowIfDisposed(); - DoOutInOpLong(OpSetPerNodeBufferSize, value); + DoOutInOp(OpSetPerNodeBufferSize, value); _bufSndSize = _topSize * value; } @@ -291,7 +291,7 @@ public int PerNodeParallelOperations { ThrowIfDisposed(); - return (int) DoOutOp(OpPerNodeParallelOps); + return (int) DoOutInOp(OpPerNodeParallelOps); } finally { @@ -307,7 +307,7 @@ public int PerNodeParallelOperations { ThrowIfDisposed(); - DoOutInOpLong(OpSetPerNodeParallelOps, value); + DoOutInOp(OpSetPerNodeParallelOps, value); } finally { @@ -598,7 +598,7 @@ public void TopologyChange(long topVer, int topSize) _topVer = topVer; _topSize = topSize > 0 ? topSize : 1; // Do not set to 0 to avoid 0 buffer size. - _bufSndSize = (int) (_topSize * DoOutOp(OpPerNodeBufferSize)); + _bufSndSize = (int) (_topSize * DoOutInOp(OpPerNodeBufferSize)); } } finally diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs index bc1b4bbeba1ca..1ec7cabe6f28b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs @@ -318,7 +318,7 @@ public void RecordLocal(IEvent evt) // Should do this inside lock to avoid race with subscription // ToArray is required because we are going to modify underlying dictionary during enumeration foreach (var filter in GetLocalFilters(listener, types).ToArray()) - success |= (DoOutInOpLong((int) Op.StopLocalListen, filter.Handle) == True); + success |= (DoOutInOp((int) Op.StopLocalListen, filter.Handle) == True); return success; } @@ -367,7 +367,7 @@ public ICollection GetEnabledEvents() /** */ public bool IsEnabled(int type) { - return DoOutInOpLong((int) Op.IsEnabled, type) == True; + return DoOutInOp((int) Op.IsEnabled, type) == True; } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs index 9b80d8c670176..37c3373bc77a7 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs @@ -252,16 +252,6 @@ protected static BinaryWriter WriteEnumerable(BinaryWriter writer, IEnumerabl #region OUT operations - /// - /// Perform out operation. - /// - /// Operation type. - /// Long result. - protected long DoOutOp(int type) - { - return UU.TargetOutLong(_target, type); - } - /// /// Perform out operation. /// @@ -566,35 +556,6 @@ protected TR DoOutInOp(int type, Action outAction, Func - /// Perform out-in operation. - /// - /// Operation type. - /// Out action. - /// In action. - /// Argument. - /// Result. - protected unsafe TR DoOutInOp(int type, Action outAction, Func inAction, void* arg) - { - using (PlatformMemoryStream outStream = IgniteManager.Memory.Allocate().GetStream()) - { - using (PlatformMemoryStream inStream = IgniteManager.Memory.Allocate().GetStream()) - { - BinaryWriter writer = _marsh.StartMarshal(outStream); - - outAction(writer); - - FinishMarshal(writer); - - UU.TargetInObjectStreamOutStream(_target, type, arg, outStream.SynchronizeOutput(), inStream.MemoryPointer); - - inStream.SynchronizeInput(); - - return inAction(inStream); - } - } - } - /// /// Perform out-in operation. /// @@ -744,7 +705,7 @@ protected TR DoOutInOp(int type, Action outAction) /// Operation type. /// Value. /// Result. - protected long DoOutInOpLong(int type, long val) + protected long DoOutInOp(int type, long val = 0) { return UU.TargetInLongOutLong(_target, type, val); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs index 9d9acfd1482ec..88d2a76f4dbf9 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs @@ -263,7 +263,7 @@ public Task CancelAsync(string name) /** */ public void CancelAll() { - DoOutOp(OpCancelAll); + DoOutInOp(OpCancelAll); } /** */ @@ -377,7 +377,7 @@ public ICollection GetServices(string name) { return DoOutInOp(OpInvokeMethod, writer => ServiceProxySerializer.WriteProxyMethod(writer, method, args, platform), - stream => ServiceProxySerializer.ReadInvocationResult(stream, Marshaller, _keepBinary), proxy.Target); + (stream, res) => ServiceProxySerializer.ReadInvocationResult(stream, Marshaller, _keepBinary), proxy.Target); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs index 7de9be1da8d38..6f8e5bfb21fe1 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs @@ -155,7 +155,7 @@ public ITransactionMetrics GetMetrics() /** */ public void ResetMetrics() { - DoOutOp(OpResetMetrics); + DoOutInOp(OpResetMetrics); } /// @@ -165,7 +165,7 @@ public void ResetMetrics() /// Final transaction state. internal TransactionState TxCommit(TransactionImpl tx) { - return (TransactionState) DoOutInOpLong(OpCommit, tx.Id); + return (TransactionState) DoOutInOp(OpCommit, tx.Id); } /// @@ -175,7 +175,7 @@ internal TransactionState TxCommit(TransactionImpl tx) /// Final transaction state. internal TransactionState TxRollback(TransactionImpl tx) { - return (TransactionState) DoOutInOpLong(OpRollback, tx.Id); + return (TransactionState) DoOutInOp(OpRollback, tx.Id); } /// @@ -185,7 +185,7 @@ internal TransactionState TxRollback(TransactionImpl tx) /// Final transaction state. internal int TxClose(TransactionImpl tx) { - return (int) DoOutInOpLong(OpClose, tx.Id); + return (int) DoOutInOp(OpClose, tx.Id); } /// @@ -195,7 +195,7 @@ internal int TxClose(TransactionImpl tx) /// Transaction current state. internal TransactionState TxState(TransactionImpl tx) { - return (TransactionState) DoOutInOpLong(OpState, tx.Id); + return (TransactionState) DoOutInOp(OpState, tx.Id); } /// @@ -205,7 +205,7 @@ internal TransactionState TxState(TransactionImpl tx) /// true if the flag was set. internal bool TxSetRollbackOnly(TransactionImpl tx) { - return DoOutInOpLong(OpSetRollbackOnly, tx.Id) == True; + return DoOutInOp(OpSetRollbackOnly, tx.Id) == True; } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs index 6b867de055511..ac41f2555a46c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs @@ -131,19 +131,12 @@ internal static unsafe class IgniteJniNativeMethods long outMemPtr); [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInStreamOutObject")] - public static extern void* TargetInStreanOutObject(void* ctx, void* target, int opType, long memPtr); - - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInObjectStreamOutStream")] - public static extern void TargetInObjectStreamOutStream(void* ctx, void* target, int opType, - void* arg, long inMemPtr, long outMemPtr); + public static extern void* TargetInStreamOutObject(void* ctx, void* target, int opType, long memPtr); [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInObjectStreamOutObjectStream")] public static extern void* TargetInObjectStreamOutObjectStream(void* ctx, void* target, int opType, void* arg, long inMemPtr, long outMemPtr); - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetOutLong")] - public static extern long TargetOutLong(void* ctx, void* target, int opType); - [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetOutStream")] public static extern void TargetOutStream(void* ctx, void* target, int opType, long memPtr); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs index 36dc3326e969e..fe1904c0974f0 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs @@ -423,16 +423,11 @@ internal static void TargetInStreamOutStream(IUnmanagedTarget target, int opType internal static IUnmanagedTarget TargetInStreamOutObject(IUnmanagedTarget target, int opType, long inMemPtr) { - void* res = JNI.TargetInStreanOutObject(target.Context, target.Target, opType, inMemPtr); + void* res = JNI.TargetInStreamOutObject(target.Context, target.Target, opType, inMemPtr); return target.ChangeTarget(res); } - internal static void TargetInObjectStreamOutStream(IUnmanagedTarget target, int opType, void* arg, long inMemPtr, long outMemPtr) - { - JNI.TargetInObjectStreamOutStream(target.Context, target.Target, opType, arg, inMemPtr, outMemPtr); - } - internal static IUnmanagedTarget TargetInObjectStreamOutObjectStream(IUnmanagedTarget target, int opType, void* arg, long inMemPtr, long outMemPtr) { void* res = JNI.TargetInObjectStreamOutObjectStream(target.Context, target.Target, opType, arg, inMemPtr, outMemPtr); @@ -443,11 +438,6 @@ internal static IUnmanagedTarget TargetInObjectStreamOutObjectStream(IUnmanagedT return target.ChangeTarget(res); } - internal static long TargetOutLong(IUnmanagedTarget target, int opType) - { - return JNI.TargetOutLong(target.Context, target.Target, opType); - } - internal static void TargetOutStream(IUnmanagedTarget target, int opType, long memPtr) { JNI.TargetOutStream(target.Context, target.Target, opType, memPtr); From e0b846758a6331e13095e9dcefa407ddfd57d6ff Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Wed, 26 Oct 2016 15:41:27 +0300 Subject: [PATCH 278/487] IGNITE-4030 Streamline PlatformTarget operation methods --- .../processors/platform/cache/PlatformCache.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index 01790e73e089d..5a6e9a2880ab3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -1060,15 +1060,6 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc, IgniteCache cache return TRUE; } - case OP_CLEAR_CACHE: - cache.clear(); - - return TRUE; - - case OP_REMOVE_ALL2: - cache.removeAll(); - - return TRUE; case OP_REBALANCE: { PlatformFutureUtils.listen(platformCtx, cache.rebalance().chain(new C1() { @Override public Object apply(IgniteFuture fut) { From 44740465677c39068dc813dabd464e60f09e5f49 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Wed, 26 Oct 2016 18:00:11 +0500 Subject: [PATCH 279/487] IGNITE-4062: fix BinaryObject.equals: compare only bytes containing the fields' data (without header and footer). This closes #1182. --- .../ignite/internal/binary/BinaryContext.java | 2 +- .../internal/binary/BinaryObjectExImpl.java | 57 ++- .../internal/binary/BinaryObjectImpl.java | 23 ++ .../binary/BinaryObjectOffheapImpl.java | 24 +- .../util/offheap/unsafe/GridUnsafeMemory.java | 33 +- .../binary/BinaryMarshallerSelfTest.java | 343 +++++++++++++++++- 6 files changed, 447 insertions(+), 35 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java index 0d66970fe9fb3..2ac8b7fc317fb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java @@ -501,7 +501,7 @@ private static BinaryInternalMapper resolveMapper(BinaryNameMapper nameMapper, B } /** - * @return Intenal mpper used as default. + * @return Internal mapper used as default. */ public static BinaryInternalMapper defaultMapper() { return DFLT_MAPPER; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java index 063bd837ac2cb..e15e7704eff50 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java @@ -76,13 +76,27 @@ public abstract class BinaryObjectExImpl implements BinaryObjectEx { throw new BinaryObjectException("Object is not enum."); } + /** + * Get offset of data begin. + * + * @return Field value. + */ + @Nullable protected abstract int dataStartOffset(); + + /** + * Get offset of the footer begin. + * + * @return Field value. + */ + @Nullable protected abstract int footerStartOffset(); + /** * Get field by offset. * - * @param fieldOffset Field offset. + * @param order Field order. * @return Field value. */ - @Nullable protected abstract F fieldByOrder(int fieldOffset); + @Nullable protected abstract F fieldByOrder(int order); /** * @param ctx Reader context. @@ -126,20 +140,29 @@ public boolean equals(Object other) { if (!(other instanceof BinaryObjectExImpl)) return false; - BinaryObjectExImpl otherPo = (BinaryObjectExImpl)other; + BinaryObjectExImpl other0 = (BinaryObjectExImpl)other; + + if (typeId() != other0.typeId()) + return false; + + int start = dataStartOffset(); + int end = footerStartOffset(); + + int otherStart = other0.dataStartOffset(); + int otherEnd = other0.footerStartOffset(); + + int len = end - start; - if (length() != otherPo.length() || typeId() != otherPo.typeId()) + if (len != otherEnd - otherStart) return false; if (hasArray()) { - if (otherPo.hasArray()) { - int len = length(); - int end = start() + len; + byte[] arr = array(); - byte[] arr = array(); - byte[] otherArr = otherPo.array(); + if (other0.hasArray()) { + byte[] otherArr = other0.array(); - for (int i = start(), j = otherPo.start(); i < end; i++, j++) { + for (int i = start, j = otherStart; i < end; i++, j++) { if (arr[i] != otherArr[j]) return false; } @@ -147,22 +170,20 @@ public boolean equals(Object other) { return true; } else { - assert otherPo.offheapAddress() > 0; + assert other0.offheapAddress() > 0; - return GridUnsafeMemory.compare(otherPo.offheapAddress() + otherPo.start(), array()); + return GridUnsafeMemory.compare(other0.offheapAddress() + otherStart, arr, start, len); } } else { assert offheapAddress() > 0; - if (otherPo.hasArray()) - return GridUnsafeMemory.compare(offheapAddress() + start(), otherPo.array()); + if (other0.hasArray()) + return GridUnsafeMemory.compare(offheapAddress() + start, other0.array(), otherStart, len); else { - assert otherPo.offheapAddress() > 0; + assert other0.offheapAddress() > 0; - return GridUnsafeMemory.compare(offheapAddress() + start(), - otherPo.offheapAddress() + otherPo.start(), - length()); + return GridUnsafeMemory.compare(offheapAddress() + start, other0.offheapAddress() + otherStart, len); } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java index f37d7c2051e36..54a7c08e7afff 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java @@ -296,6 +296,29 @@ public void context(BinaryContext ctx) { } /** {@inheritDoc} */ + @Nullable @Override protected int dataStartOffset() { + int typeId = BinaryPrimitives.readInt(arr, start + GridBinaryMarshaller.TYPE_ID_POS); + + if (typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID) { + int len = BinaryPrimitives.readInt(arr, start + GridBinaryMarshaller.DFLT_HDR_LEN + 1); + + return start + GridBinaryMarshaller.DFLT_HDR_LEN + len + 5; + } + else + return start + GridBinaryMarshaller.DFLT_HDR_LEN; + } + + /** {@inheritDoc} */ + @Nullable @Override protected int footerStartOffset() { + short flags = BinaryPrimitives.readShort(arr, start + GridBinaryMarshaller.FLAGS_POS); + + if (!BinaryUtils.hasSchema(flags)) + return start + length(); + + return start + BinaryPrimitives.readInt(arr, start + GridBinaryMarshaller.SCHEMA_OR_RAW_OFF_POS); + } + + /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Nullable @Override protected F fieldByOrder(int order) { Object val; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java index 9cbbaa2767092..7550b199210b3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java @@ -96,7 +96,7 @@ public BinaryObject heapCopy() { if (typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID) { int off = start + GridBinaryMarshaller.DFLT_HDR_LEN; - String clsName = BinaryUtils.doReadClassName(new BinaryOffheapInputStream(off, size)); + String clsName = BinaryUtils.doReadClassName(new BinaryOffheapInputStream(ptr + off, size)); typeId = ctx.typeId(clsName); } @@ -173,6 +173,28 @@ public BinaryObject heapCopy() { return (F) reader(null, false).unmarshalField(fieldId); } + /** {@inheritDoc} */ + @Nullable @Override protected int dataStartOffset() { + int typeId = BinaryPrimitives.readInt(ptr, start + GridBinaryMarshaller.TYPE_ID_POS); + + if (typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID) { + int len = BinaryPrimitives.readInt(ptr, start + GridBinaryMarshaller.DFLT_HDR_LEN + 1); + + return start + GridBinaryMarshaller.DFLT_HDR_LEN + len + 5; + } else + return start + GridBinaryMarshaller.DFLT_HDR_LEN; + } + + /** {@inheritDoc} */ + @Nullable @Override protected int footerStartOffset() { + short flags = BinaryPrimitives.readShort(ptr, start + GridBinaryMarshaller.FLAGS_POS); + + if (!BinaryUtils.hasSchema(flags)) + return start + length(); + + return start + BinaryPrimitives.readInt(ptr, start + GridBinaryMarshaller.SCHEMA_OR_RAW_OFF_POS); + } + /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Nullable @Override protected F fieldByOrder(int order) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMemory.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMemory.java index 87ba2f2c05860..718e1a6c17ac5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMemory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMemory.java @@ -153,7 +153,8 @@ public long allocateSystem(long size, boolean init) throws GridOffHeapOutOfMemor * @param reserved If {@code false}, means that memory counter was reserved and size will not * be added to counter. * @param cnt Counter to account allocated memory. - * @throws GridOffHeapOutOfMemoryException + * @throws GridOffHeapOutOfMemoryException In case of out of the off-heap memory. + * @return Pointer to the allocated memory. */ @SuppressWarnings("ErrorNotRethrown") private long allocate0(long size, boolean init, boolean reserved, @@ -507,20 +508,32 @@ public static boolean compare(long ptr1, long ptr2, int size) { * @return {@code True} if equals. */ public static boolean compare(long ptr, byte[] bytes) { + return compare(ptr, bytes, 0, bytes.length); + } + + /** + * Compares memory. + * + * @param ptr Pointer. + * @param bytes Bytes to compare. + * @param bytesOff Offset in the bytes array. + * @param len Count of compared bytes. + * @return {@code True} if equals. + */ + public static boolean compare(long ptr, byte[] bytes, int bytesOff, int len) { + assert bytesOff + len <= bytes.length : "Check compare bounds: [offset=" + bytesOff + ", len=" + len + + ", bytes.length=" + bytes.length + ']'; + final int addrSize = GridUnsafe.ADDR_SIZE; // Align reads to address size. int off = (int)(ptr % addrSize); int align = addrSize - off; - int len = bytes.length; - if (align != addrSize) { - for (int i = 0; i < align && i < len; i++) { - if (GridUnsafe.getByte(ptr) != bytes[i]) + for (int i = 0, tmpOff = bytesOff; i < align && i < len; i++, tmpOff++, ptr++) { + if (GridUnsafe.getByte(ptr) != bytes[tmpOff]) return false; - - ptr++; } } else @@ -542,7 +555,7 @@ public static boolean compare(long ptr, byte[] bytes) { int word = GridUnsafe.getInt(ptr); - int comp = GridUnsafe.getInt(bytes, GridUnsafe.BYTE_ARR_OFF + step); + int comp = GridUnsafe.getInt(bytes, GridUnsafe.BYTE_ARR_OFF + step + bytesOff); if (word != comp) return false; @@ -558,7 +571,7 @@ public static boolean compare(long ptr, byte[] bytes) { long word = GridUnsafe.getLong(ptr); - long comp = GridUnsafe.getLong(bytes, GridUnsafe.BYTE_ARR_OFF + step); + long comp = GridUnsafe.getLong(bytes, GridUnsafe.BYTE_ARR_OFF + step + bytesOff); if (word != comp) return false; @@ -572,7 +585,7 @@ public static boolean compare(long ptr, byte[] bytes) { if (left != 0) { // Compare left overs byte by byte. for (int i = 0; i < left; i++) - if (GridUnsafe.getByte(ptr + i) != bytes[i + align + words * GridUnsafe.ADDR_SIZE]) + if (GridUnsafe.getByte(ptr + i) != bytes[bytesOff + i + align + words * GridUnsafe.ADDR_SIZE]) return false; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java index f4154724bfd9a..39a4d32383f7a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java @@ -842,8 +842,6 @@ public void testVoid() throws Exception { assertEquals(clazz, marshalUnmarshal(clazz)); } - - /** * */ @@ -2921,6 +2919,209 @@ public void testUnregisteredClass() throws Exception { assertEquals(Value.class, res.cls); } + /** + * @throws Exception If failed. + */ + public void testBinaryEquals() throws Exception { + Collection excludedClasses = Arrays.asList( + ObjectRaw.class.getName(), + ObjectWithRaw.class.getName(), + Value.class.getName()); + + BinaryMarshaller m0 = binaryMarshaller(null, excludedClasses); + BinaryMarshaller m1 = binaryMarshaller(); + + Value obj = new Value(27); + ObjectWithRaw objectWithRaw = new ObjectWithRaw(27, 13); + ObjectRaw objectRaw = new ObjectRaw(27, 13); + + Value objOther = new Value(26); + ObjectWithRaw objectWithRawOther = new ObjectWithRaw(26, 13); + ObjectRaw objectRawOther = new ObjectRaw(26, 13); + + BinaryObjectImpl binObj0 = marshal(obj, m0); + BinaryObjectImpl binObj1 = marshal(obj, m1); + BinaryObjectImpl binObjWithRaw0 = marshal(objectWithRaw, m0); + BinaryObjectImpl binObjWithRaw1 = marshal(objectWithRaw, m1); + BinaryObjectImpl binObjRaw0 = marshal(objectRaw, m0); + BinaryObjectImpl binObjRaw1 = marshal(objectRaw, m1); + + assertNotEquals(binObj0.array().length, binObj1.array().length); + assertNotEquals(binObjWithRaw0.array().length, binObjWithRaw1.array().length); + assertNotEquals(binObjRaw0.array().length, binObjRaw1.array().length); + + checkEquals(binObj0, binObj1); + + checkEquals(binObjWithRaw0, binObjWithRaw1); + + checkEquals(binObjRaw0, binObjRaw1); + + BinaryObjectOffheapImpl binObjOffheap0 = null; + BinaryObjectOffheapImpl binObjOffheap1 = null; + BinaryObjectOffheapImpl binObjWithRawOffheap0 = null; + BinaryObjectOffheapImpl binObjWithRawOffheap1 = null; + BinaryObjectOffheapImpl binObjRawOffheap0 = null; + BinaryObjectOffheapImpl binObjRawOffheap1 = null; + + BinaryObjectImpl binObjOther0 = marshal(objOther, m0); + BinaryObjectImpl binObjOther1 = marshal(objOther, m1); + BinaryObjectImpl binObjWithRawOther0 = marshal(objectWithRawOther, m0); + BinaryObjectImpl binObjWithRawOther1 = marshal(objectWithRawOther, m1); + BinaryObjectImpl binObjRawOther0 = marshal(objectRawOther, m0); + BinaryObjectImpl binObjRawOther1 = marshal(objectRawOther, m1); + + assertEquals(binObjOther0.length(), binObj0.length()); + assertEquals(binObjOther1.length(), binObj1.length()); + assertEquals(binObjWithRawOther0.length(), binObjWithRaw0.length()); + assertEquals(binObjWithRawOther1.length(), binObjWithRaw1.length()); + assertEquals(binObjRawOther0.length(), binObjRaw0.length()); + assertEquals(binObjRawOther1.length(), binObjRaw1.length()); + + assertNotEquals(binObjOther0, binObj0); + assertNotEquals(binObjOther1, binObj1); + assertNotEquals(binObjWithRawOther0, binObjWithRaw0); + assertNotEquals(binObjWithRawOther1, binObjWithRaw1); + assertNotEquals(binObjRawOther0, binObjRaw0); + assertNotEquals(binObjRawOther1, binObjRaw1); + + try { + binObjOffheap0 = marshalOffHeap(binObj0, m0); + binObjOffheap1 = marshalOffHeap(binObj1, m1); + binObjWithRawOffheap0 = marshalOffHeap(binObjWithRaw0, m0); + binObjWithRawOffheap1 = marshalOffHeap(binObjWithRaw1, m1); + binObjRawOffheap0 = marshalOffHeap(binObjRaw0, m0); + binObjRawOffheap1 = marshalOffHeap(binObjRaw1, m1); + + checkEquals(binObj0, binObjOffheap0); + checkEquals(binObj1, binObjOffheap0); + checkEquals(binObj0, binObjOffheap1); + checkEquals(binObj1, binObjOffheap1); + checkEquals(binObjOffheap0, binObjOffheap1); + + checkEquals(binObjWithRaw0, binObjWithRawOffheap0); + checkEquals(binObjWithRaw0, binObjWithRawOffheap1); + checkEquals(binObjWithRaw1, binObjWithRawOffheap0); + checkEquals(binObjWithRaw1, binObjWithRawOffheap1); + checkEquals(binObjWithRawOffheap0, binObjWithRawOffheap1); + + checkEquals(binObjRaw0, binObjRawOffheap0); + checkEquals(binObjRaw1, binObjRawOffheap0); + checkEquals(binObjRaw0, binObjRawOffheap1); + checkEquals(binObjRaw1, binObjRawOffheap1); + checkEquals(binObjRawOffheap0, binObjRawOffheap1); + } + finally { + if (binObjOffheap0 != null) { + GridUnsafe.freeMemory(binObjOffheap0.offheapAddress()); + binObjOffheap0 = null; + } + + if (binObjOffheap1 != null) { + GridUnsafe.freeMemory(binObjOffheap1.offheapAddress()); + binObjOffheap1 = null; + } + + if (binObjWithRawOffheap0 != null) { + GridUnsafe.freeMemory(binObjWithRawOffheap0.offheapAddress()); + binObjOffheap1 = null; + } + + if (binObjWithRawOffheap1 != null) { + GridUnsafe.freeMemory(binObjWithRawOffheap1.offheapAddress()); + binObjOffheap1 = null; + } + + if (binObjRawOffheap0 != null) { + GridUnsafe.freeMemory(binObjRawOffheap0.offheapAddress()); + binObjOffheap1 = null; + } + + if (binObjRawOffheap1 != null) { + GridUnsafe.freeMemory(binObjRawOffheap1.offheapAddress()); + binObjOffheap1 = null; + } + } + } + + /** + * @param binObj0 Object #0. + * @param binObj1 Object #1. + */ + private void checkEquals(Object binObj0, Object binObj1) { + assertEquals(binObj0, binObj1); + assertEquals(binObj1, binObj0); + assertEquals(binObj0, binObj0); + assertEquals(binObj1, binObj1); + } + + /** + * @throws Exception If failed. + */ + public void testBinaryEqualsComplexObject() throws Exception { + List excludedClasses = Arrays.asList( + TestClass0.class.getName(), + TestClass1.class.getName(), + TestClass2.class.getName()); + + BinaryMarshaller m0 = binaryMarshaller(null, excludedClasses); + BinaryMarshaller m1 = binaryMarshaller(null); + + TestClass0 obj0 = new TestClass0(); + TestClass1 obj1 = new TestClass1(); + TestClass2 obj2 = new TestClass2(); + + BinaryObjectImpl binObj00 = marshal(obj0, m0); + BinaryObjectImpl binObj01 = marshal(obj1, m0); + BinaryObjectImpl binObj02 = marshal(obj2, m0); + + // The length of array must be equal. Object are different only by the class. + assertEquals(binObj00.array().length, binObj01.array().length); + assertEquals(binObj00.array().length, binObj02.array().length); + + BinaryObjectImpl binObj10 = marshal(obj0, m1); + BinaryObjectImpl binObj11 = marshal(obj1, m1); + BinaryObjectImpl binObj12 = marshal(obj2, m1); + + // The length of array must be equal. Object are different only by the class. + assertEquals(binObj10.array().length, binObj11.array().length); + assertEquals(binObj10.array().length, binObj12.array().length); + + assertNotEquals(binObj10.array().length, binObj00.array().length); + + assertEquals(binObj00, binObj10); + assertEquals(binObj01, binObj11); + assertEquals(binObj02, binObj12); + + assertNotEquals(binObj00, binObj01); + assertNotEquals(binObj00, binObj02); + assertNotEquals(binObj00, binObj11); + assertNotEquals(binObj00, binObj12); + + assertNotEquals(binObj01, binObj00); + assertNotEquals(binObj01, binObj02); + assertNotEquals(binObj01, binObj10); + assertNotEquals(binObj01, binObj12); + + assertNotEquals(binObj02, binObj00); + assertNotEquals(binObj02, binObj01); + assertNotEquals(binObj02, binObj00); + assertNotEquals(binObj02, binObj11); + } + + /** + * @param obj Instance of the BinaryObjectImpl to offheap marshalling. + * @param marsh Binary marshaller. + * @return Instance of BinaryObjectOffheapImpl. + */ + private BinaryObjectOffheapImpl marshalOffHeap(BinaryObjectImpl obj, BinaryMarshaller marsh) { + long ptr = copyOffheap(obj); + + return new BinaryObjectOffheapImpl(binaryContext(marsh), + ptr, + 0, + obj.array().length); + } + /** * */ @@ -3331,7 +3532,7 @@ private static class SimpleObjectWithFinal { /** * @return Simple object. */ - private SimpleObject simpleObject() { + private static SimpleObject simpleObject() { SimpleObject inner = new SimpleObject(); inner.b = 1; @@ -4903,7 +5104,8 @@ private static class SingleHandleA { /** */ - private static class SingleHandleB {} + private static class SingleHandleB { + } /** */ @@ -4918,4 +5120,135 @@ public ClassFieldObject(Class cls) { this.cls = cls; } } -} + + /** + * + */ + private static class TestClass0 { + /** */ + private int intVal = 33; + + /** */ + private String strVal = "Test string value"; + + /** */ + private SimpleObject obj = constSimpleObject(); + + /** + * @return Constant value of the SimpleObject. + */ + public static SimpleObject constSimpleObject() { + SimpleObject obj = simpleObject(); + + obj.uuid = null; + obj.date = new Date(33); + obj.ts = new Timestamp(22); + obj.uuidArr = new UUID[] {null, null, null}; + obj.dateArr = new Date[] {new Date(11111), new Date(22222), new Date(33333)}; + obj.objArr = new Object[] {null, null, null}; + + obj.inner.uuid = null; + obj.inner.date = new Date(33); + obj.inner.ts = new Timestamp(22); + obj.inner.uuidArr = new UUID[] {null, null, null}; + obj.inner.dateArr = new Date[] {new Date(11111), new Date(22222), new Date(33333)}; + obj.inner.objArr = new Object[] {null, null, null}; + + return obj; + } + } + + /** + * + */ + private static class TestClass1 { + /** */ + private int intVal = 33; + + /** */ + private String strVal = "Test string value"; + + /** */ + private SimpleObject obj = TestClass0.constSimpleObject(); + } + + /** + * + */ + private static class TestClass2 extends TestClass0 { + } + + /** */ + private static class ObjectWithRaw implements Binarylizable { + /** */ + private int val; + + /** */ + private int rawVal; + + /** + * + */ + public ObjectWithRaw() { + } + + /** + * @param val Value. + * @param rawVal Raw value. + */ + public ObjectWithRaw(int val, int rawVal) { + this.val = val; + this.rawVal = rawVal; + } + + /** {@inheritDoc} */ + @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException { + writer.writeInt("val", val); + + writer.rawWriter().writeInt(rawVal); + } + + /** {@inheritDoc} */ + @Override public void readBinary(BinaryReader reader) throws BinaryObjectException { + val = reader.readInt("val"); + + rawVal = reader.rawReader().readInt(); + } + } + + /** */ + private static class ObjectRaw implements Binarylizable { + /** */ + private int val0; + + /** */ + private int val1; + + /** + * + */ + public ObjectRaw() { + } + + /** + * @param val0 Value. + * @param val1 Raw value. + */ + public ObjectRaw(int val0, int val1) { + this.val0 = val0; + this.val1 = val1; + } + + /** {@inheritDoc} */ + @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException { + writer.rawWriter().writeInt(val0); + writer.rawWriter().writeInt(val1); + } + + /** {@inheritDoc} */ + @Override public void readBinary(BinaryReader reader) throws BinaryObjectException { + val0 = reader.rawReader().readInt(); + val1 = reader.rawReader().readInt(); + } + } +} \ No newline at end of file From 3e724fd8a8da38c955e7ab4712e0cb8b5685a16c Mon Sep 17 00:00:00 2001 From: AMRepo Date: Wed, 19 Oct 2016 18:33:59 +0300 Subject: [PATCH 280/487] IGNITE-3448 Support SQL queries with distinct aggregates added. This closes #3448. (cherry picked from commit 7ed2bb7) --- .../query/h2/sql/GridSqlQuerySplitter.java | 143 ++++++-- .../query/IgniteSqlSplitterSelfTest.java | 326 ++++++++++++++++++ 2 files changed, 432 insertions(+), 37 deletions(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java index 7205a1862f645..7d43bf647ec1a 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java @@ -39,6 +39,8 @@ import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.AVG; import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.CAST; import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.COUNT; +import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.MAX; +import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.MIN; import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.SUM; import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlPlaceholder.EMPTY; import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser.prepared; @@ -298,10 +300,18 @@ private static GridCacheSqlQuery split(GridCacheTwoStepQuery res, int splitIdx, Set colNames = new HashSet<>(); - boolean aggregateFound = false; + boolean distinctAggregateFound = false; + + if (!collocatedGroupBy) { + for (int i = 0, len = mapExps.size(); i < len; i++) + distinctAggregateFound |= hasDistinctAggregates(mapExps.get(i)); + } + + boolean aggregateFound = distinctAggregateFound; for (int i = 0, len = mapExps.size(); i < len; i++) // Remember len because mapExps list can grow. - aggregateFound |= splitSelectExpression(mapExps, rdcExps, colNames, i, collocatedGroupBy, i == havingCol); + aggregateFound |= splitSelectExpression(mapExps, rdcExps, colNames, i, collocatedGroupBy, i == havingCol, + distinctAggregateFound); // -- SELECT mapQry.clearColumns(); @@ -325,9 +335,14 @@ private static GridCacheSqlQuery split(GridCacheTwoStepQuery res, int splitIdx, findAffinityColumnConditions(mapQry.where()); // -- GROUP BY - if (mapQry.groupColumns() != null && !collocatedGroupBy) + if (mapQry.groupColumns() != null && !collocatedGroupBy) { rdcQry.groupColumns(mapQry.groupColumns()); + // Grouping with distinct aggregates cannot be performed on map phase + if (distinctAggregateFound) + mapQry.groupColumns(null); + } + // -- HAVING if (havingCol >= 0 && !collocatedGroupBy) { // TODO IGNITE-1140 - Find aggregate functions in HAVING clause or rewrite query to put all aggregates to SELECT clause. @@ -638,10 +653,11 @@ else if (el instanceof GridSqlSubquery) * @param idx Index. * @param collocated If it is a collocated query. * @param isHaving If it is a HAVING expression. + * @param hasDistinctAggregate If query has distinct aggregate expression. * @return {@code true} If aggregate was found. */ private static boolean splitSelectExpression(List mapSelect, List rdcSelect, - Set colNames, final int idx, boolean collocated, boolean isHaving) { + Set colNames, final int idx, boolean collocated, boolean isHaving, boolean hasDistinctAggregate) { GridSqlElement el = mapSelect.get(idx); GridSqlAlias alias = null; @@ -660,7 +676,7 @@ private static boolean splitSelectExpression(List mapSelect, Lis alias = alias(isHaving ? HAVING_COLUMN : columnName(idx), el); // We can update original alias here as well since it will be dropped from mapSelect. - splitAggregates(alias, 0, mapSelect, idx, true); + splitAggregates(alias, 0, mapSelect, idx, hasDistinctAggregate, true); set(rdcSelect, idx, alias); } @@ -714,11 +730,34 @@ private static boolean hasAggregates(GridSqlElement el) { return false; } + /** + * Lookup for distinct aggregates. + * Note, DISTINCT make no sense for MIN and MAX aggregates, so its will be ignored. + * + * @param el Expression. + * @return {@code true} If expression contains distinct aggregates. + */ + private static boolean hasDistinctAggregates(GridSqlElement el) { + if (el instanceof GridSqlAggregateFunction) { + GridSqlFunctionType type = ((GridSqlAggregateFunction)el).type(); + + return ((GridSqlAggregateFunction)el).distinct() && type != MIN && type != MAX; + } + + for (GridSqlElement child : el) { + if (hasDistinctAggregates(child)) + return true; + } + + return false; + } + /** * @param parentExpr Parent expression. * @param childIdx Child index to try to split. * @param mapSelect List of expressions in map SELECT clause. * @param exprIdx Index of the original expression in map SELECT clause. + * @param hasDistinctAggregate If query has distinct aggregate expression. * @param first If the first aggregate is already found in this expression. * @return {@code true} If the first aggregate is already found. */ @@ -727,17 +766,18 @@ private static boolean splitAggregates( final int childIdx, final List mapSelect, final int exprIdx, + boolean hasDistinctAggregate, boolean first) { GridSqlElement el = parentExpr.child(childIdx); if (el instanceof GridSqlAggregateFunction) { - splitAggregate(parentExpr, childIdx, mapSelect, exprIdx, first); + splitAggregate(parentExpr, childIdx, mapSelect, exprIdx, hasDistinctAggregate, first); return true; } for (int i = 0; i < el.size(); i++) { - if (splitAggregates(el, i, mapSelect, exprIdx, first)) + if (splitAggregates(el, i, mapSelect, exprIdx, hasDistinctAggregate, first)) first = false; } @@ -749,6 +789,7 @@ private static boolean splitAggregates( * @param aggIdx Index of the aggregate to split in this expression. * @param mapSelect List of expressions in map SELECT clause. * @param exprIdx Index of the original expression in map SELECT clause. + * @param hasDistinctAggregate If query has distinct aggregate expression. * @param first If this is the first aggregate found in this expression. */ private static void splitAggregate( @@ -756,6 +797,7 @@ private static void splitAggregate( int aggIdx, List mapSelect, int exprIdx, + boolean hasDistinctAggregate, boolean first ) { GridSqlAggregateFunction agg = parentExpr.child(aggIdx); @@ -773,53 +815,80 @@ private static void splitAggregate( else mapSelect.add(mapAggAlias); + /* Note Distinct aggregate can be performed only on reduce phase, so + if query contains distinct aggregate then other aggregates must be processed the same way. */ switch (agg.type()) { - case AVG: // SUM( AVG(CAST(x AS DOUBLE))*COUNT(x) )/SUM( COUNT(x) ). - //-- COUNT(x) map - GridSqlElement cntMapAgg = aggregate(agg.distinct(), COUNT) - .resultType(GridSqlType.BIGINT).addChild(agg.child()); + case AVG: // SUM( AVG(CAST(x AS DOUBLE))*COUNT(x) )/SUM( COUNT(x) ) or AVG(CAST( x AS DOUBLE)) + if (hasDistinctAggregate) /* and has no collocated group by */ { + mapAgg = agg.child(); + + rdcAgg = aggregate(agg.distinct(), agg.type()).resultType(GridSqlType.DOUBLE) + .addChild(function(CAST).resultType(GridSqlType.DOUBLE).addChild(column(mapAggAlias.alias()))); + } + else { + //-- COUNT(x) map + GridSqlElement cntMapAgg = aggregate(agg.distinct(), COUNT) + .resultType(GridSqlType.BIGINT).addChild(agg.child()); - // Add generated alias to COUNT(x). - // Using size as index since COUNT will be added as the last select element to the map query. - String cntMapAggAlias = columnName(mapSelect.size()); + // Add generated alias to COUNT(x). + // Using size as index since COUNT will be added as the last select element to the map query. + String cntMapAggAlias = columnName(mapSelect.size()); - cntMapAgg = alias(cntMapAggAlias, cntMapAgg); + cntMapAgg = alias(cntMapAggAlias, cntMapAgg); - mapSelect.add(cntMapAgg); + mapSelect.add(cntMapAgg); - //-- AVG(CAST(x AS DOUBLE)) map - mapAgg = aggregate(agg.distinct(), AVG).resultType(GridSqlType.DOUBLE).addChild( - function(CAST).resultType(GridSqlType.DOUBLE).addChild(agg.child())); + //-- AVG(CAST(x AS DOUBLE)) map + mapAgg = aggregate(agg.distinct(), AVG).resultType(GridSqlType.DOUBLE).addChild( + function(CAST).resultType(GridSqlType.DOUBLE).addChild(agg.child())); - //-- SUM( AVG(x)*COUNT(x) )/SUM( COUNT(x) ) reduce - GridSqlElement sumUpRdc = aggregate(false, SUM).addChild( - op(GridSqlOperationType.MULTIPLY, - column(mapAggAlias.alias()), - column(cntMapAggAlias))); + //-- SUM( AVG(x)*COUNT(x) )/SUM( COUNT(x) ) reduce + GridSqlElement sumUpRdc = aggregate(false, SUM).addChild( + op(GridSqlOperationType.MULTIPLY, + column(mapAggAlias.alias()), + column(cntMapAggAlias))); - GridSqlElement sumDownRdc = aggregate(false, SUM).addChild(column(cntMapAggAlias)); + GridSqlElement sumDownRdc = aggregate(false, SUM).addChild(column(cntMapAggAlias)); - rdcAgg = op(GridSqlOperationType.DIVIDE, sumUpRdc, sumDownRdc); + rdcAgg = op(GridSqlOperationType.DIVIDE, sumUpRdc, sumDownRdc); + } break; - case SUM: // SUM( SUM(x) ) - case MAX: // MAX( MAX(x) ) - case MIN: // MIN( MIN(x) ) - mapAgg = aggregate(agg.distinct(), agg.type()).resultType(agg.resultType()).addChild(agg.child()); - rdcAgg = aggregate(agg.distinct(), agg.type()).addChild(column(mapAggAlias.alias())); + case SUM: // SUM( SUM(x) ) or SUM(DISTINCT x) + case MAX: // MAX( MAX(x) ) or MAX(DISTINCT x) + case MIN: // MIN( MIN(x) ) or MIN(DISTINCT x) + if (hasDistinctAggregate) /* and has no collocated group by */ { + mapAgg = agg.child(); + + rdcAgg = aggregate(agg.distinct(), agg.type()).addChild(column(mapAggAlias.alias())); + } + else { + mapAgg = aggregate(agg.distinct(), agg.type()).resultType(agg.resultType()).addChild(agg.child()); + rdcAgg = aggregate(agg.distinct(), agg.type()).addChild(column(mapAggAlias.alias())); + } break; case COUNT_ALL: // CAST(SUM( COUNT(*) ) AS BIGINT) - case COUNT: // CAST(SUM( COUNT(x) ) AS BIGINT) - mapAgg = aggregate(agg.distinct(), agg.type()).resultType(GridSqlType.BIGINT); + case COUNT: // CAST(SUM( COUNT(x) ) AS BIGINT) or CAST(COUNT(DISTINCT x) AS BIGINT) + if (hasDistinctAggregate) /* and has no collocated group by */ { + assert agg.type() == COUNT; - if (agg.type() == COUNT) - mapAgg.addChild(agg.child()); + mapAgg = agg.child(); - rdcAgg = aggregate(false, SUM).addChild(column(mapAggAlias.alias())); - rdcAgg = function(CAST).resultType(GridSqlType.BIGINT).addChild(rdcAgg); + rdcAgg = aggregate(agg.distinct(), agg.type()).resultType(GridSqlType.BIGINT) + .addChild(column(mapAggAlias.alias())); + } + else { + mapAgg = aggregate(agg.distinct(), agg.type()).resultType(GridSqlType.BIGINT); + + if (agg.type() == COUNT) + mapAgg.addChild(agg.child()); + + rdcAgg = aggregate(false, SUM).addChild(column(mapAggAlias.alias())); + rdcAgg = function(CAST).resultType(GridSqlType.BIGINT).addChild(rdcAgg); + } break; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java index 1a2530ed014ac..50f2ef0e88f2b 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java @@ -27,14 +27,18 @@ import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicLong; import javax.cache.CacheException; +import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheKeyConfiguration; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.affinity.AffinityKeyMapped; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.util.GridRandom; @@ -1059,6 +1063,328 @@ public void testImplicitJoinConditionGeneration() { } } + /** @throws Exception if failed. */ + public void testDistributedAggregates() throws Exception { + final String cacheName = "ints"; + + IgniteCache cache = ignite(0).getOrCreateCache(cacheConfig(cacheName, true, + Integer.class, Value.class)); + + AffinityKeyGenerator node0KeyGen = new AffinityKeyGenerator(ignite(0), cacheName); + AffinityKeyGenerator node1KeyGen = new AffinityKeyGenerator(ignite(1), cacheName); + AffinityKeyGenerator node2KeyGen = new AffinityKeyGenerator(ignite(2), cacheName); + + try { + awaitPartitionMapExchange(); + + cache.put(node0KeyGen.next(), new Value(1, 3)); + cache.put(node1KeyGen.next(), new Value(1, 3)); + cache.put(node2KeyGen.next(), new Value(1, 3)); + + cache.put(node0KeyGen.next(), new Value(2, 1)); + cache.put(node1KeyGen.next(), new Value(2, 2)); + cache.put(node2KeyGen.next(), new Value(2, 3)); + + cache.put(node0KeyGen.next(), new Value(3, 1)); + cache.put(node0KeyGen.next(), new Value(3, 1)); + cache.put(node0KeyGen.next(), new Value(3, 2)); + cache.put(node1KeyGen.next(), new Value(3, 1)); + cache.put(node1KeyGen.next(), new Value(3, 2)); + cache.put(node2KeyGen.next(), new Value(3, 2)); + + cache.put(node0KeyGen.next(), new Value(4, 2)); + cache.put(node1KeyGen.next(), new Value(5, 2)); + cache.put(node2KeyGen.next(), new Value(6, 2)); + + checkSimpleQueryWithAggr(cache); + checkSimpleQueryWithDistinctAggr(cache); + + checkQueryWithGroupsAndAggrs(cache); + checkQueryWithGroupsAndDistinctAggr(cache); + + checkSimpleQueryWithAggrMixed(cache); + checkQueryWithGroupsAndAggrMixed(cache); + } + finally { + cache.destroy(); + } + } + + /** @throws Exception if failed. */ + public void testCollocatedAggregates() throws Exception { + final String cacheName = "ints"; + + IgniteCache cache = ignite(0).getOrCreateCache(cacheConfig(cacheName, true, + Integer.class, Value.class)); + + AffinityKeyGenerator node0KeyGen = new AffinityKeyGenerator(ignite(0), cacheName); + AffinityKeyGenerator node1KeyGen = new AffinityKeyGenerator(ignite(1), cacheName); + AffinityKeyGenerator node2KeyGen = new AffinityKeyGenerator(ignite(2), cacheName); + + try { + awaitPartitionMapExchange(); + + cache.put(node0KeyGen.next(), new Value(1, 3)); + cache.put(node0KeyGen.next(), new Value(1, 3)); + cache.put(node0KeyGen.next(), new Value(1, 3)); + + cache.put(node1KeyGen.next(), new Value(2, 1)); + cache.put(node1KeyGen.next(), new Value(2, 2)); + cache.put(node1KeyGen.next(), new Value(2, 3)); + + cache.put(node2KeyGen.next(), new Value(3, 1)); + cache.put(node2KeyGen.next(), new Value(3, 1)); + cache.put(node2KeyGen.next(), new Value(3, 2)); + cache.put(node2KeyGen.next(), new Value(3, 1)); + cache.put(node2KeyGen.next(), new Value(3, 2)); + cache.put(node2KeyGen.next(), new Value(3, 2)); + + cache.put(node0KeyGen.next(), new Value(4, 2)); + cache.put(node1KeyGen.next(), new Value(5, 2)); + cache.put(node2KeyGen.next(), new Value(6, 2)); + + checkQueryWithGroupsAndAggrs(cache); + checkQueryWithGroupsAndDistinctAggr(cache); + checkQueryWithGroupsAndAggrMixed(cache); + } + finally { + cache.destroy(); + } + } + + /** Simple query with aggregates */ + private void checkSimpleQueryWithAggr(IgniteCache cache) { + try (QueryCursor> qry = cache.query(new SqlFieldsQuery( + "SELECT count(fst), sum(snd), avg(snd), min(snd), max(snd) FROM Value"))) { + List> result = qry.getAll(); + + assertEquals(1, result.size()); + + List row = result.get(0); + + assertEquals("count", 15L, ((Number)row.get(0)).longValue()); + assertEquals("sum", 30L, ((Number)row.get(1)).longValue()); + assertEquals("avg", 2.0d, ((Number)row.get(2)).doubleValue(), 0.001); + assertEquals("min", 1, ((Integer)row.get(3)).intValue()); + assertEquals("max", 3, ((Integer)row.get(4)).intValue()); + } + } + + /** Simple query with distinct aggregates */ + private void checkSimpleQueryWithDistinctAggr(IgniteCache cache) { + try (QueryCursor> qry = cache.query(new SqlFieldsQuery( + "SELECT count(distinct fst), sum(distinct snd), avg(distinct snd), min(distinct snd), max(distinct snd) " + + "FROM Value"))) { + List> result = qry.getAll(); + + assertEquals(1, result.size()); + + List row = result.get(0); + + assertEquals("count distinct", 6L, ((Number)row.get(0)).longValue()); + assertEquals("sum distinct", 6L, ((Number)row.get(1)).longValue()); + assertEquals("avg distinct", 2.0d, ((Number)row.get(2)).doubleValue(), 0.001); + assertEquals("min distinct", 1, ((Integer)row.get(3)).intValue()); + assertEquals("max distinct", 3, ((Integer)row.get(4)).intValue()); + } + } + + /** Simple query with distinct aggregates */ + private void checkSimpleQueryWithAggrMixed(IgniteCache cache) { + try (QueryCursor> qry = cache.query(new SqlFieldsQuery( + "SELECT count(fst), sum(snd), avg(snd), min(snd), max(snd)," + + "count(distinct fst), sum(distinct snd), avg(distinct snd), min(distinct snd), max(distinct snd) " + + "FROM Value"))) { + List> result = qry.getAll(); + + assertEquals(1, result.size()); + + List row = result.get(0); + + assertEquals("count", 15L, ((Number)row.get(0)).longValue()); + assertEquals("sum", 30L, ((Number)row.get(1)).longValue()); + assertEquals("avg", 2.0d, ((Number)row.get(2)).doubleValue(), 0.001); + assertEquals("min", 1, ((Integer)row.get(3)).intValue()); + assertEquals("max", 3, ((Integer)row.get(4)).intValue()); + assertEquals("count distinct", 6L, ((Number)row.get(5)).longValue()); + assertEquals("sum distinct", 6L, ((Number)row.get(6)).longValue()); + assertEquals("avg distinct", 2.0d, ((Number)row.get(7)).doubleValue(), 0.001); + assertEquals("min distinct", 1, ((Integer)row.get(8)).intValue()); + assertEquals("max distinct", 3, ((Integer)row.get(9)).intValue()); + } + } + + /** Query with aggregates and groups */ + private void checkQueryWithGroupsAndAggrs(IgniteCache cache) { + try (QueryCursor> qry = cache.query(new SqlFieldsQuery( + "SELECT fst, count(snd), sum(snd), avg(snd), min(snd), max(snd) FROM Value GROUP BY fst ORDER BY fst"))) { + List> result = qry.getAll(); + + assertEquals(6, result.size()); + + List row = result.get(0); + assertEquals("fst", 1, ((Number)row.get(0)).intValue()); + assertEquals("count", 3L, ((Number)row.get(1)).longValue()); + assertEquals("sum", 9L, ((Number)row.get(2)).longValue()); + assertEquals("avg", 3.0d, ((Number)row.get(3)).doubleValue(), 0.001); + assertEquals("min", 3, ((Integer)row.get(4)).intValue()); + assertEquals("max", 3, ((Integer)row.get(5)).intValue()); + + row = result.get(1); + assertEquals("fst", 2, ((Number)row.get(0)).intValue()); + assertEquals("count", 3L, ((Number)row.get(1)).longValue()); + assertEquals("sum", 6L, ((Number)row.get(2)).longValue()); + assertEquals("avg", 2.0d, ((Number)row.get(3)).doubleValue(), 0.001); + assertEquals("min", 1, ((Integer)row.get(4)).intValue()); + assertEquals("max", 3, ((Integer)row.get(5)).intValue()); + + row = result.get(2); + assertEquals("fst", 3, ((Number)row.get(0)).intValue()); + assertEquals("count", 6L, ((Number)row.get(1)).longValue()); + assertEquals("sum", 9L, ((Number)row.get(2)).longValue()); + assertEquals("avg", 1.5d, ((Number)row.get(3)).doubleValue(), 0.001); + assertEquals("min", 1, ((Integer)row.get(4)).intValue()); + assertEquals("max", 2, ((Integer)row.get(5)).intValue()); + } + } + + /** Query with distinct aggregates and groups */ + private void checkQueryWithGroupsAndDistinctAggr(IgniteCache cache) { + try (QueryCursor> qry = cache.query(new SqlFieldsQuery( + "SELECT count(distinct snd), sum(distinct snd), avg(distinct snd), min(distinct snd), max(distinct snd) " + + "FROM Value GROUP BY fst"))) { + List> result = qry.getAll(); + + assertEquals(6, result.size()); + + List row = result.get(0); + assertEquals("count distinct", 1L, ((Number)row.get(0)).longValue()); + assertEquals("sum distinct", 3L, ((Number)row.get(1)).longValue()); + assertEquals("avg distinct", 3.0d, ((Number)row.get(2)).doubleValue(), 0.001); + assertEquals("min distinct", 3, ((Integer)row.get(3)).intValue()); + assertEquals("max distinct", 3, ((Integer)row.get(4)).intValue()); + + row = result.get(1); + assertEquals("count distinct", 3L, ((Number)row.get(0)).longValue()); + assertEquals("sum distinct", 6L, ((Number)row.get(1)).longValue()); + assertEquals("avg distinct", 2.0d, ((Number)row.get(2)).doubleValue(), 0.001); + assertEquals("min distinct", 1, ((Integer)row.get(3)).intValue()); + assertEquals("max distinct", 3, ((Integer)row.get(4)).intValue()); + + row = result.get(2); + assertEquals("count distinct", 2L, ((Number)row.get(0)).longValue()); + assertEquals("sum distinct", 3L, ((Number)row.get(1)).longValue()); + assertEquals("avg distinct", 1.5d, ((Number)row.get(2)).doubleValue(), 0.001); + assertEquals("min distinct", 1, ((Integer)row.get(3)).intValue()); + assertEquals("max distinct", 2, ((Integer)row.get(4)).intValue()); + } + } + + /** Query with distinct aggregates and groups */ + private void checkQueryWithGroupsAndAggrMixed(IgniteCache cache) { + try (QueryCursor> qry = cache.query(new SqlFieldsQuery( + "SELECT fst, count(snd), sum(snd), avg(snd), min(snd), max(snd)," + + "count(distinct snd), sum(distinct snd), avg(distinct snd), min(distinct snd), max(distinct snd) " + + "FROM Value GROUP BY fst"))) { + List> result = qry.getAll(); + + assertEquals(6, result.size()); + + List row = result.get(0); + assertEquals("fst", 1, ((Number)row.get(0)).intValue()); + assertEquals("count", 3L, ((Number)row.get(1)).longValue()); + assertEquals("sum", 9L, ((Number)row.get(2)).longValue()); + assertEquals("avg", 3.0d, ((Number)row.get(3)).doubleValue(), 0.001); + assertEquals("min", 3, ((Integer)row.get(4)).intValue()); + assertEquals("max", 3, ((Integer)row.get(5)).intValue()); + assertEquals("count distinct", 1L, ((Number)row.get(6)).longValue()); + assertEquals("sum distinct", 3L, ((Number)row.get(7)).longValue()); + assertEquals("avg distinct", 3.0d, ((Number)row.get(8)).doubleValue(), 0.001); + assertEquals("min distinct", 3, ((Integer)row.get(9)).intValue()); + assertEquals("max distinct", 3, ((Integer)row.get(10)).intValue()); + + row = result.get(1); + assertEquals("fst", 2, ((Number)row.get(0)).intValue()); + assertEquals("count", 3L, ((Number)row.get(1)).longValue()); + assertEquals("sum", 6L, ((Number)row.get(2)).longValue()); + assertEquals("avg", 2.0d, ((Number)row.get(3)).doubleValue(), 0.001); + assertEquals("min", 1, ((Integer)row.get(4)).intValue()); + assertEquals("max", 3, ((Integer)row.get(5)).intValue()); + assertEquals("count distinct", 3L, ((Number)row.get(6)).longValue()); + assertEquals("sum distinct", 6L, ((Number)row.get(7)).longValue()); + assertEquals("avg distinct", 2.0d, ((Number)row.get(8)).doubleValue(), 0.001); + assertEquals("min distinct", 1, ((Integer)row.get(9)).intValue()); + assertEquals("max distinct", 3, ((Integer)row.get(10)).intValue()); + + row = result.get(2); + assertEquals("fst", 3, ((Number)row.get(0)).intValue()); + assertEquals("count", 6L, ((Number)row.get(1)).longValue()); + assertEquals("sum", 9L, ((Number)row.get(2)).longValue()); + assertEquals("avg", 1.5d, ((Number)row.get(3)).doubleValue(), 0.001); + assertEquals("min", 1, ((Integer)row.get(4)).intValue()); + assertEquals("max", 2, ((Integer)row.get(5)).intValue()); + assertEquals("count distinct", 2L, ((Number)row.get(6)).longValue()); + assertEquals("sum distinct", 3L, ((Number)row.get(7)).longValue()); + assertEquals("avg distinct", 1.5d, ((Number)row.get(8)).doubleValue(), 0.001); + assertEquals("min distinct", 1, ((Integer)row.get(9)).intValue()); + assertEquals("max distinct", 2, ((Integer)row.get(10)).intValue()); + } + } + + /** */ + private static class Value { + /** */ + @QuerySqlField + private final Integer fst; + + /** */ + @QuerySqlField + private final Integer snd; + + /** Constructor */ + public Value(Integer fst, Integer snd) { + this.fst = fst; + this.snd = snd; + } + } + + /** + * + */ + private static class AffinityKeyGenerator { + /** */ + private final Affinity affinity; + + /** */ + private final ClusterNode node; + + /** */ + private int start = 0; + + /** Constructor */ + AffinityKeyGenerator(Ignite node, String cacheName) { + this.affinity = node.affinity(cacheName); + this.node = node.cluster().localNode(); + } + + /** */ + public Integer next() { + int key = start; + + while (start < Integer.MAX_VALUE) { + if (affinity.isPrimary(node, key)) { + start = key + 1; + + return key; + } + + key++; + } + + throw new IllegalStateException("Can't find next key"); + } + } + /** * */ From b9d28caad2670557e381d2d67823fc58b4dfd97d Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Thu, 27 Oct 2016 17:04:01 +0300 Subject: [PATCH 281/487] Revert "IGNITE-4028 .NET: Get rid of OP_META in PlatformAbstractTarget" This reverts commit 2a90fcaf8e46a829306ca92e226d984111b3aefe. --- .../platform/PlatformAbstractTarget.java | 13 ++- .../cluster/PlatformClusterGroup.java | 19 ++--- .../Impl/Cluster/ClusterGroupImpl.cs | 79 ++----------------- .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs | 2 +- .../Apache.Ignite.Core/Impl/PlatformTarget.cs | 67 ++++++++++++++++ 5 files changed, 89 insertions(+), 91 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java index 5c7f260b12b4f..29b603ac964b6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java @@ -43,6 +43,9 @@ public abstract class PlatformAbstractTarget implements PlatformTarget { /** Constant: ERROR. */ protected static final int ERROR = -1; + /** */ + private static final int OP_META = -1; + /** Context. */ protected final PlatformContext platformCtx; @@ -75,7 +78,13 @@ protected PlatformAbstractTarget(PlatformContext platformCtx) { try (PlatformMemory mem = platformCtx.memory().get(memPtr)) { BinaryRawReaderEx reader = platformCtx.reader(mem); - return processInStreamOutLong(type, reader, mem); + if (type == OP_META) { + platformCtx.processMetadata(reader); + + return TRUE; + } + else + return processInStreamOutLong(type, reader, mem); } catch (Exception e) { throw convertException(e); @@ -392,7 +401,7 @@ protected Object processOutObject(int type) throws IgniteCheckedException { * @return Dummy value which is never returned. * @throws IgniteCheckedException Exception to be thrown. */ - private T throwUnsupported(int type) throws IgniteCheckedException { + protected T throwUnsupported(int type) throws IgniteCheckedException { throw new IgniteCheckedException("Unsupported operation type: " + type); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java index 04a117360ef72..d09506bff142b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java @@ -17,22 +17,21 @@ package org.apache.ignite.internal.processors.platform.cluster; +import java.util.Collection; +import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCluster; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.cluster.ClusterGroupEx; import org.apache.ignite.internal.binary.BinaryRawReaderEx; import org.apache.ignite.internal.binary.BinaryRawWriterEx; -import org.apache.ignite.internal.cluster.ClusterGroupEx; import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.jetbrains.annotations.Nullable; -import java.util.Collection; -import java.util.UUID; - /** * Interop projection. */ @@ -60,7 +59,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget { private static final int OP_FOR_NODE_IDS = 7; /** */ - private static final int OP_GET_META = 8; + private static final int OP_METADATA = 8; /** */ private static final int OP_METRICS = 9; @@ -107,9 +106,6 @@ public class PlatformClusterGroup extends PlatformAbstractTarget { /** */ private static final int OP_FOR_SERVERS = 23; - /** */ - private static final int OP_PUT_META = 24; - /** Projection. */ private final ClusterGroupEx prj; @@ -205,7 +201,7 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { break; } - case OP_GET_META: { + case OP_METADATA: { int typeId = reader.readInt(); platformCtx.writeMetadata(writer, typeId); @@ -241,11 +237,6 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { case OP_PING_NODE: return pingNode(reader.readUuid()) ? TRUE : FALSE; - case OP_PUT_META: - platformCtx.processMetadata(reader); - - return TRUE; - default: return super.processInStreamOutLong(type, reader); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs index 912d6ed58bc7d..388be82525fcf 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs @@ -76,7 +76,7 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx private const int OpForNodeIds = 7; /** */ - private const int OpGetMeta = 8; + private const int OpMetadata = 8; /** */ private const int OpMetrics = 9; @@ -103,7 +103,7 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx private const int OpForRemotes = 17; /** */ - private const int OpForDaemons = 18; + public const int OpForDaemons = 18; /** */ private const int OpForRandom = 19; @@ -115,13 +115,10 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx private const int OpForYoungest = 21; /** */ - private const int OpResetMetrics = 22; + public const int OpResetMetrics = 22; /** */ - private const int OpForServers = 23; - - /** */ - private const int OpPutMeta = 24; + public const int OpForServers = 23; /** Initial Ignite instance. */ private readonly Ignite _ignite; @@ -560,7 +557,7 @@ private IList RefreshNodes() /** */ public IBinaryType GetBinaryType(int typeId) { - return DoOutInOp(OpGetMeta, + return DoOutInOp(OpMetadata, writer => writer.WriteInt(typeId), stream => { @@ -602,71 +599,5 @@ public int[] GetSchema(int typeId, int schemaId) writer.WriteInt(schemaId); }); } - - /// - /// Resets local I/O, job, and task execution metrics. - /// - public void ResetMetrics() - { - DoOutOp(OpResetMetrics); - } - - /// - /// Put binary types to Grid. - /// - /// Binary types. - public void PutBinaryTypes(ICollection types) - { - DoOutOp(OpPutMeta, w => - { - w.WriteInt(types.Count); - - foreach (var meta in types) - { - w.WriteInt(meta.TypeId); - w.WriteString(meta.TypeName); - w.WriteString(meta.AffinityKeyFieldName); - - IDictionary fields = meta.GetFieldsMap(); - - w.WriteInt(fields.Count); - - foreach (var field in fields) - { - w.WriteString(field.Key); - w.WriteInt(field.Value); - } - - w.WriteBoolean(meta.IsEnum); - - // Send schemas - var desc = meta.Descriptor; - Debug.Assert(desc != null); - - var count = 0; - var countPos = w.Stream.Position; - w.WriteInt(0); // Reserve for count - - foreach (var schema in desc.Schema.GetAll()) - { - w.WriteInt(schema.Key); - - var ids = schema.Value; - w.WriteInt(ids.Length); - - foreach (var id in ids) - w.WriteInt(id); - - count++; - } - - w.Stream.WriteInt(countPos, count); - } - - Marshaller.FinishMarshal(w); - }); - - Marshaller.OnBinaryTypesSent(types); - } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs index 70a33113fff1b..79df4709df215 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs @@ -517,7 +517,7 @@ public ICollection GetTopology(long ver) /** */ public void ResetMetrics() { - _prj.ResetMetrics(); + UU.TargetOutLong(_prj.Target, ClusterGroupImpl.OpResetMetrics); } /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs index 9b80d8c670176..d5b69a4a67f0b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs @@ -26,6 +26,7 @@ namespace Apache.Ignite.Core.Impl using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Binary.IO; + using Apache.Ignite.Core.Impl.Binary.Metadata; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Impl.Memory; using Apache.Ignite.Core.Impl.Unmanaged; @@ -48,6 +49,12 @@ internal abstract class PlatformTarget /** */ protected const int Error = -1; + /** */ + private const int OpMeta = -1; + + /** */ + public const int OpNone = -2; + /** */ private static readonly Dictionary IgniteFutureTypeMap = new Dictionary @@ -858,6 +865,66 @@ internal void FinishMarshal(BinaryWriter writer) _marsh.FinishMarshal(writer); } + /// + /// Put binary types to Grid. + /// + /// Binary types. + internal void PutBinaryTypes(ICollection types) + { + DoOutOp(OpMeta, stream => + { + BinaryWriter w = _marsh.StartMarshal(stream); + + w.WriteInt(types.Count); + + foreach (var meta in types) + { + w.WriteInt(meta.TypeId); + w.WriteString(meta.TypeName); + w.WriteString(meta.AffinityKeyFieldName); + + IDictionary fields = meta.GetFieldsMap(); + + w.WriteInt(fields.Count); + + foreach (var field in fields) + { + w.WriteString(field.Key); + w.WriteInt(field.Value); + } + + w.WriteBoolean(meta.IsEnum); + + // Send schemas + var desc = meta.Descriptor; + Debug.Assert(desc != null); + + var count = 0; + var countPos = stream.Position; + w.WriteInt(0); // Reserve for count + + foreach (var schema in desc.Schema.GetAll()) + { + w.WriteInt(schema.Key); + + var ids = schema.Value; + w.WriteInt(ids.Length); + + foreach (var id in ids) + w.WriteInt(id); + + count++; + } + + stream.WriteInt(countPos, count); + } + + _marsh.FinishMarshal(w); + }); + + _marsh.OnBinaryTypesSent(types); + } + /// /// Unmarshal object using the given stream. /// From 2e7f59bf113bb76af0ddb5029b6af4900a6b1d54 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Thu, 27 Oct 2016 17:04:01 +0300 Subject: [PATCH 282/487] Revert "IGNITE-4028 .NET: Get rid of OP_META in PlatformAbstractTarget" This reverts commit 2a90fcaf8e46a829306ca92e226d984111b3aefe. # Conflicts: # modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs --- .../platform/PlatformAbstractTarget.java | 13 ++- .../cluster/PlatformClusterGroup.java | 19 ++--- .../Impl/Cluster/ClusterGroupImpl.cs | 79 ++----------------- .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs | 2 +- .../Apache.Ignite.Core/Impl/PlatformTarget.cs | 67 ++++++++++++++++ 5 files changed, 89 insertions(+), 91 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java index 840c52794ae34..cba0031eeffdd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java @@ -43,6 +43,9 @@ public abstract class PlatformAbstractTarget implements PlatformTarget { /** Constant: ERROR. */ protected static final int ERROR = -1; + /** */ + private static final int OP_META = -1; + /** Context. */ protected final PlatformContext platformCtx; @@ -75,7 +78,13 @@ protected PlatformAbstractTarget(PlatformContext platformCtx) { try (PlatformMemory mem = platformCtx.memory().get(memPtr)) { BinaryRawReaderEx reader = platformCtx.reader(mem); - return processInStreamOutLong(type, reader, mem); + if (type == OP_META) { + platformCtx.processMetadata(reader); + + return TRUE; + } + else + return processInStreamOutLong(type, reader, mem); } catch (Exception e) { throw convertException(e); @@ -338,7 +347,7 @@ protected Object processOutObject(int type) throws IgniteCheckedException { * @return Dummy value which is never returned. * @throws IgniteCheckedException Exception to be thrown. */ - private T throwUnsupported(int type) throws IgniteCheckedException { + protected T throwUnsupported(int type) throws IgniteCheckedException { throw new IgniteCheckedException("Unsupported operation type: " + type); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java index 5894ff82422b0..dde98f5be5641 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java @@ -17,22 +17,21 @@ package org.apache.ignite.internal.processors.platform.cluster; +import java.util.Collection; +import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCluster; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.cluster.ClusterGroupEx; import org.apache.ignite.internal.binary.BinaryRawReaderEx; import org.apache.ignite.internal.binary.BinaryRawWriterEx; -import org.apache.ignite.internal.cluster.ClusterGroupEx; import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; import org.apache.ignite.internal.processors.platform.PlatformContext; import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.jetbrains.annotations.Nullable; -import java.util.Collection; -import java.util.UUID; - /** * Interop projection. */ @@ -60,7 +59,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget { private static final int OP_FOR_NODE_IDS = 7; /** */ - private static final int OP_GET_META = 8; + private static final int OP_METADATA = 8; /** */ private static final int OP_METRICS = 9; @@ -107,9 +106,6 @@ public class PlatformClusterGroup extends PlatformAbstractTarget { /** */ private static final int OP_FOR_SERVERS = 23; - /** */ - private static final int OP_PUT_META = 24; - /** Projection. */ private final ClusterGroupEx prj; @@ -205,7 +201,7 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { break; } - case OP_GET_META: { + case OP_METADATA: { int typeId = reader.readInt(); platformCtx.writeMetadata(writer, typeId); @@ -241,11 +237,6 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { case OP_PING_NODE: return pingNode(reader.readUuid()) ? TRUE : FALSE; - case OP_PUT_META: - platformCtx.processMetadata(reader); - - return TRUE; - default: return super.processInStreamOutLong(type, reader); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs index 9a202e1d5aacc..277d61f6fac57 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs @@ -76,7 +76,7 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx private const int OpForNodeIds = 7; /** */ - private const int OpGetMeta = 8; + private const int OpMetadata = 8; /** */ private const int OpMetrics = 9; @@ -103,7 +103,7 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx private const int OpForRemotes = 17; /** */ - private const int OpForDaemons = 18; + public const int OpForDaemons = 18; /** */ private const int OpForRandom = 19; @@ -115,13 +115,10 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx private const int OpForYoungest = 21; /** */ - private const int OpResetMetrics = 22; + public const int OpResetMetrics = 22; /** */ - private const int OpForServers = 23; - - /** */ - private const int OpPutMeta = 24; + public const int OpForServers = 23; /** Initial Ignite instance. */ private readonly Ignite _ignite; @@ -560,7 +557,7 @@ private IList RefreshNodes() /** */ public IBinaryType GetBinaryType(int typeId) { - return DoOutInOp(OpGetMeta, + return DoOutInOp(OpMetadata, writer => writer.WriteInt(typeId), stream => { @@ -602,71 +599,5 @@ public int[] GetSchema(int typeId, int schemaId) writer.WriteInt(schemaId); }); } - - /// - /// Resets local I/O, job, and task execution metrics. - /// - public void ResetMetrics() - { - DoOutInOp(OpResetMetrics); - } - - /// - /// Put binary types to Grid. - /// - /// Binary types. - public void PutBinaryTypes(ICollection types) - { - DoOutOp(OpPutMeta, w => - { - w.WriteInt(types.Count); - - foreach (var meta in types) - { - w.WriteInt(meta.TypeId); - w.WriteString(meta.TypeName); - w.WriteString(meta.AffinityKeyFieldName); - - IDictionary fields = meta.GetFieldsMap(); - - w.WriteInt(fields.Count); - - foreach (var field in fields) - { - w.WriteString(field.Key); - w.WriteInt(field.Value); - } - - w.WriteBoolean(meta.IsEnum); - - // Send schemas - var desc = meta.Descriptor; - Debug.Assert(desc != null); - - var count = 0; - var countPos = w.Stream.Position; - w.WriteInt(0); // Reserve for count - - foreach (var schema in desc.Schema.GetAll()) - { - w.WriteInt(schema.Key); - - var ids = schema.Value; - w.WriteInt(ids.Length); - - foreach (var id in ids) - w.WriteInt(id); - - count++; - } - - w.Stream.WriteInt(countPos, count); - } - - Marshaller.FinishMarshal(w); - }); - - Marshaller.OnBinaryTypesSent(types); - } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs index 70a33113fff1b..79df4709df215 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs @@ -517,7 +517,7 @@ public ICollection GetTopology(long ver) /** */ public void ResetMetrics() { - _prj.ResetMetrics(); + UU.TargetOutLong(_prj.Target, ClusterGroupImpl.OpResetMetrics); } /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs index 37c3373bc77a7..f74138902088b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs @@ -26,6 +26,7 @@ namespace Apache.Ignite.Core.Impl using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Binary.IO; + using Apache.Ignite.Core.Impl.Binary.Metadata; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Impl.Memory; using Apache.Ignite.Core.Impl.Unmanaged; @@ -48,6 +49,12 @@ internal abstract class PlatformTarget /** */ protected const int Error = -1; + /** */ + private const int OpMeta = -1; + + /** */ + public const int OpNone = -2; + /** */ private static readonly Dictionary IgniteFutureTypeMap = new Dictionary @@ -819,6 +826,66 @@ internal void FinishMarshal(BinaryWriter writer) _marsh.FinishMarshal(writer); } + /// + /// Put binary types to Grid. + /// + /// Binary types. + internal void PutBinaryTypes(ICollection types) + { + DoOutOp(OpMeta, stream => + { + BinaryWriter w = _marsh.StartMarshal(stream); + + w.WriteInt(types.Count); + + foreach (var meta in types) + { + w.WriteInt(meta.TypeId); + w.WriteString(meta.TypeName); + w.WriteString(meta.AffinityKeyFieldName); + + IDictionary fields = meta.GetFieldsMap(); + + w.WriteInt(fields.Count); + + foreach (var field in fields) + { + w.WriteString(field.Key); + w.WriteInt(field.Value); + } + + w.WriteBoolean(meta.IsEnum); + + // Send schemas + var desc = meta.Descriptor; + Debug.Assert(desc != null); + + var count = 0; + var countPos = stream.Position; + w.WriteInt(0); // Reserve for count + + foreach (var schema in desc.Schema.GetAll()) + { + w.WriteInt(schema.Key); + + var ids = schema.Value; + w.WriteInt(ids.Length); + + foreach (var id in ids) + w.WriteInt(id); + + count++; + } + + stream.WriteInt(countPos, count); + } + + _marsh.FinishMarshal(w); + }); + + _marsh.OnBinaryTypesSent(types); + } + /// /// Unmarshal object using the given stream. /// From 160e37f1bf83289b1295f0bf772be1f18bc342b8 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Thu, 27 Oct 2016 17:36:11 +0300 Subject: [PATCH 283/487] Revert "IGNITE-4028 .NET: Get rid of OP_META in PlatformAbstractTarget" This reverts commit 2a90fcaf8e46a829306ca92e226d984111b3aefe. # Conflicts: # modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs --- modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs index 79df4709df215..e8171cb5efe64 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs @@ -517,7 +517,7 @@ public ICollection GetTopology(long ver) /** */ public void ResetMetrics() { - UU.TargetOutLong(_prj.Target, ClusterGroupImpl.OpResetMetrics); + UU.TargetInLongOutLong(_prj.Target, ClusterGroupImpl.OpResetMetrics, 0); } /** */ From 9ddb8be1243df8e489f7ebc716d315415775439a Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Thu, 27 Oct 2016 17:52:22 +0300 Subject: [PATCH 284/487] IGNITE-2079 GridCacheIoManager eats exception trail if it falls into the directed case merger from ignite-2079-2 # Conflicts: # modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java --- .../org/apache/ignite/events/EventType.java | 6 + .../events/UnhandledExceptionEvent.java | 61 ++++ .../processors/cache/GridCacheIoManager.java | 70 ++-- .../cache/query/GridCacheQueryManager.java | 10 + .../query/GridCacheQueryMetricsAdapter.java | 9 +- .../cache/query/GridCacheQueryResponse.java | 2 +- .../CacheContinuousQueryHandler.java | 50 +-- ...UnmarshallingContinuousQueryErrorTest.java | 302 ++++++++++++++++++ ...teCacheP2pUnmarshallingErrorTestSuite.java | 6 +- 9 files changed, 455 insertions(+), 61 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/events/UnhandledExceptionEvent.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingContinuousQueryErrorTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/events/EventType.java b/modules/core/src/main/java/org/apache/ignite/events/EventType.java index 103dbd4dde4bd..7778f67b60f36 100644 --- a/modules/core/src/main/java/org/apache/ignite/events/EventType.java +++ b/modules/core/src/main/java/org/apache/ignite/events/EventType.java @@ -858,6 +858,12 @@ public interface EventType { */ public static final int EVT_IGFS_FILE_PURGED = 127; + /** + * Built-in event type: event for unhandled exception. + * + */ + public static final int EVT_UNHANDLED_EXCEPTION = 128; + /** * All checkpoint events. This array can be directly passed into * {@link IgniteEvents#localListen(IgnitePredicate, int...)} method to diff --git a/modules/core/src/main/java/org/apache/ignite/events/UnhandledExceptionEvent.java b/modules/core/src/main/java/org/apache/ignite/events/UnhandledExceptionEvent.java new file mode 100644 index 0000000000000..cb6cd85f12605 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/events/UnhandledExceptionEvent.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.events; + +import org.apache.ignite.cluster.ClusterNode; + +/** + * Cache fail event. + */ +public class UnhandledExceptionEvent extends EventAdapter { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private Exception ex; + + /** + * Default constructor. + */ + public UnhandledExceptionEvent() { + } + + /** + * @param node Node. + * @param msg Message. + * @param ex Exception. + * @param type Type. + */ + public UnhandledExceptionEvent(ClusterNode node, String msg, Exception ex, int type) { + super(node, msg, type); + this.ex = ex; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "msg=" + message() + ", type=" + type() + "ex=" + ex; + } + + /** + * + * @return inner exception + */ + public Exception getException() { + return ex; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java index 78dddd3340270..5d7cb00dffec9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java @@ -17,50 +17,26 @@ package org.apache.ignite.internal.processors.cache; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.events.UnhandledExceptionEvent; +import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.deployment.GridDeploymentInfo; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.distributed.dht.CacheGetFuture; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.*; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockResponse; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetRequest; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetResponse; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishResponse; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.*; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryResponse; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -79,6 +55,12 @@ import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; +import java.util.*; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.ignite.events.EventType.EVT_UNHANDLED_EXCEPTION; import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE; /** @@ -693,6 +675,11 @@ private void processFailedMessage(UUID nodeId, GridCacheMessage msg, IgniteBiInC break; + case 59: + // No additional actions required, just skipping default switch section, + // since UnhandledException already registered. + break; + case 114: { processMessage(nodeId,msg,c);// Will be handled by Rebalance Demander. } @@ -737,12 +724,33 @@ private void processFailedMessage(UUID nodeId, GridCacheMessage msg, IgniteBiInC break; - default: - throw new IgniteCheckedException("Failed to send response to node. Unsupported direct type [message=" - + msg + "]", msg.classError()); + default:{ + String shortMsg = "Failed to send response to node. Unsupported direct type [message=" + msg + "]"; + + IgniteCheckedException e = new IgniteCheckedException(shortMsg, msg.classError()); + + registerUnhandledException(ctx, shortMsg, e); + } } } + /** + * @param ctx Grid cache context. + * @param shortMsg Short message. + * @param ex Original Exception. + */ + public static void registerUnhandledException(GridCacheContext ctx, String shortMsg, IgniteCheckedException ex) { + GridKernalContext kctx = ctx.kernalContext(); + + kctx.exceptionRegistry().onException(shortMsg, ex); + + ClusterNode node = ctx.discovery().localNode(); + + UnhandledExceptionEvent evt = new UnhandledExceptionEvent(node, shortMsg, ex, EVT_UNHANDLED_EXCEPTION); + + kctx.event().record(evt); + } + /** * @param nodeId Node ID. * @param msg Message. 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 7bd1a51b314d3..97e59c1d7d5c8 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 @@ -325,6 +325,16 @@ void onWaitAtStop() { // No-op. } + /** + * Increment fails counter. + */ + public void onUnhandledException() { + final boolean statsEnabled = cctx.config().isStatisticsEnabled(); + + if (statsEnabled) + metrics.incrementOnFails(); + } + /** * Processes cache query request. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java index e70ea9ff46222..d25b7c47770c8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java @@ -172,4 +172,11 @@ public GridCacheQueryMetricsAdapter copy() { @Override public String toString() { return S.toString(GridCacheQueryMetricsAdapter.class, this); } -} + + /** + * Increment fails counter. + */ + public void incrementOnFails() { + fails.increment(); + } +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java index 8492c380b1a75..2b86efea768cd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java @@ -357,6 +357,6 @@ public boolean fields() { /** {@inheritDoc} */ @Override public String toString() { - return S.toString(GridCacheQueryResponse.class, this); + return S.toString(GridCacheQueryResponse.class, this, super.toString()); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 304d031d81fc0..4c91ea779809c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -17,28 +17,6 @@ package org.apache.ignite.internal.processors.cache.query.continuous; -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NavigableSet; -import java.util.Set; -import java.util.TreeMap; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicLong; -import javax.cache.event.CacheEntryEvent; -import javax.cache.event.CacheEntryEventFilter; -import javax.cache.event.CacheEntryUpdatedListener; -import javax.cache.event.EventType; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; @@ -61,6 +39,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheDeploymentManager; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateFuture; import org.apache.ignite.internal.processors.cache.query.CacheQueryType; +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager.JCacheQueryLocalListener; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager.JCacheQueryRemoteFilter; import org.apache.ignite.internal.processors.continuous.GridContinuousBatch; @@ -82,8 +61,22 @@ import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentLinkedDeque8; +import javax.cache.event.CacheEntryEvent; +import javax.cache.event.CacheEntryEventFilter; +import javax.cache.event.CacheEntryUpdatedListener; +import javax.cache.event.EventType; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.*; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; + import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED; import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_OBJECT_READ; +import static org.apache.ignite.internal.processors.cache.GridCacheIoManager.registerUnhandledException; /** * Continuous query handler. @@ -688,8 +681,17 @@ private void notifyCallback0(UUID nodeId, catch (IgniteCheckedException ex) { if (ignoreClsNotFound) assert internal; - else - U.error(ctx.log(getClass()), "Failed to unmarshal entry.", ex); + else { + String shortMsg = "Failed to unmarshal entry."; + + U.error(ctx.log(getClass()), shortMsg, ex); + + GridCacheQueryManager qryMgr = cctx.queries(); + + qryMgr.onUnhandledException(); + + registerUnhandledException(cctx, shortMsg, ex); + } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingContinuousQueryErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingContinuousQueryErrorTest.java new file mode 100644 index 0000000000000..82f5f0916ab1a --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingContinuousQueryErrorTest.java @@ -0,0 +1,302 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.Cache; +import javax.cache.event.CacheEntryEvent; +import javax.cache.event.CacheEntryUpdatedListener; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.query.ContinuousQuery; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.events.Event; +import org.apache.ignite.events.EventType; +import org.apache.ignite.events.UnhandledExceptionEvent; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMetricsAdapter; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.thread.IgniteThread; + +/** + * Checks behavior on exception while unmarshalling key for continuous query. + */ +public class IgniteCacheP2pUnmarshallingContinuousQueryErrorTest extends IgniteCacheP2pUnmarshallingErrorTest { + /** + * {@inheritDoc} + */ + @Override protected int gridCount() { + return 3; + } + + /** Used inside InitialQuery listener. */ + private static final CountDownLatch latch = new CountDownLatch(1); + + /** Node where unmarshalling fails with exceptions. */ + private static volatile String failNode; + + /** Used to count UnhandledExceptionEvents at client node. */ + private static final AtomicInteger cnt = new AtomicInteger(); + + /** + * {@inheritDoc} + */ + @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception { + CacheConfiguration cacheCfg = super.cacheConfiguration(gridName); + + cacheCfg.setStatisticsEnabled(true); + + return cacheCfg; + } + + /** + * {@inheritDoc} + */ + @Override public void testResponseMessageOnUnmarshallingFailed() throws Exception { + IgniteEx client = grid(0); + IgniteEx node1 = grid(1); + IgniteEx node2 = grid(2); + + assert client.configuration().isClientMode() && + !node1.configuration().isClientMode() && + !node2.configuration().isClientMode(); + + failNode = client.name(); + + client.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + UnhandledExceptionEvent uex = (UnhandledExceptionEvent)evt; + + assertTrue(X.getFullStackTrace(uex.getException()). + contains("IOException: Class can not be unmarshalled")); + + cnt.incrementAndGet(); + + return true; + } + }, EventType.EVT_UNHANDLED_EXCEPTION); + + node1.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + fail("This line should newer calls."); + + return true; + } + }, EventType.EVT_UNHANDLED_EXCEPTION); + + ContinuousQuery qry = new ContinuousQuery<>(); + + qry.setInitialQuery(new ScanQuery<>(new IgniteBiPredicate() { + @Override public boolean apply(TestKey key, String val) { + latch.countDown(); // Gives guarantee query initialized. + + return true; + } + })); + + qry.setLocalListener(new CacheEntryUpdatedListener() { + @Override public void onUpdated(Iterable> evts) { + fail("This line should newer calls."); + } + }); + + validate( + 0,//execs + 0,//evts + 0,//fails + client, + node1, + node2); + + // Put element before creating QueryCursor. + putPrimary(node1); + + try (QueryCursor> cur = client.cache(null).query(qry)) { + latch.await(); + + validate( + 1,//execs + 0,//evts + 0,//fails + client, + node1, + node2); + + putPrimary(node1); + + validate( + 1,//execs + 1,//evts + 1,//fails + client, + node1, + node2); + + putPrimary(node2); + + validate( + 1,//execs + 2,//evts + 2,//fails + client, + node1, + node2); + } + } + + /** + * @param ignite Ignite. + */ + private void putPrimary(IgniteEx ignite) { + IgniteCache cache = ignite.cache(null); + + cache.put(generateNodeKeys(ignite, cache), "value"); + } + + /** + * @param execs Executions. + * @param evts Events. + * @param failsNum Fails number. + * @param client Client. + * @param node1 Node 1. + * @param node2 Node 2. + */ + private void validate(final int execs, final int evts, final int failsNum, final IgniteEx client, IgniteEx node1, + IgniteEx node2) throws Exception { + assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return client.cache(null).queryMetrics().fails() == failsNum; + } + }, 5_000)); + + assertEquals(evts, cnt.intValue()); + + validateCacheQueryMetrics(client, execs, failsNum); + validateCacheQueryMetrics(node1, 0, 0); + validateCacheQueryMetrics(node2, 0, 0); + } + + /** + * @param ignite Ignite. + * @param executions Executions. + * @param fails Fails. + */ + private void validateCacheQueryMetrics(IgniteEx ignite, int executions, int fails) { + IgniteCache cache = ignite.cache(null); + + GridCacheQueryMetricsAdapter metr = (GridCacheQueryMetricsAdapter)cache.queryMetrics(); + + assertEquals(metr.executions(), executions); + + assertEquals(metr.fails(), fails); + } + + /** + * @param node Node. + * @param cache Cache. + */ + private TestKey generateNodeKeys(IgniteEx node, IgniteCache cache) { + + ClusterNode locNode = node.localNode(); + + for (int ind = 0; ind < 100_000; ind++) { + TestKey key = new TestKey("key" + ind); + + if (affinity(cache).isPrimary(locNode, key)) + return key; + } + + throw new IgniteException("Unable to find key keys as primary for cache."); + } + + /** + * + * */ + private static class TestKey implements Externalizable { + /** + * Field. + */ + @QuerySqlField(index = true) + private String field; + + /** + * Required by {@link Externalizable}. + */ + public TestKey() { + } + + /** + * @param field Test key 1. + */ + public TestKey(String field) { + this.field = field; + } + + /** + * {@inheritDoc} + */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + IgniteCacheP2pUnmarshallingContinuousQueryErrorTest.TestKey key = (IgniteCacheP2pUnmarshallingContinuousQueryErrorTest.TestKey)o; + + return !(field != null ? !field.equals(key.field) : key.field != null); + } + + /** + * {@inheritDoc} + */ + @Override public int hashCode() { + return field != null ? field.hashCode() : 0; + } + + /** + * {@inheritDoc} + */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + out.writeObject(field); + } + + /** + * {@inheritDoc} + */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + field = (String)in.readObject(); + + if (((IgniteThread)Thread.currentThread()).getGridName().equals(failNode)) + throw new IOException("Class can not be unmarshalled."); + + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuite.java index dfc96dca72337..b45d134f6956e 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuite.java @@ -19,10 +19,7 @@ import java.util.Set; import junit.framework.TestSuite; -import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingErrorTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingNearErrorTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingRebalanceErrorTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingTxErrorTest; +import org.apache.ignite.internal.processors.cache.*; import org.apache.ignite.testframework.GridTestUtils; /** @@ -49,6 +46,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { GridTestUtils.addTestIfNeeded(suite, IgniteCacheP2pUnmarshallingNearErrorTest.class, ignoredTests); GridTestUtils.addTestIfNeeded(suite, IgniteCacheP2pUnmarshallingRebalanceErrorTest.class, ignoredTests); GridTestUtils.addTestIfNeeded(suite, IgniteCacheP2pUnmarshallingTxErrorTest.class, ignoredTests); + GridTestUtils.addTestIfNeeded(suite, IgniteCacheP2pUnmarshallingContinuousQueryErrorTest.class, ignoredTests); return suite; } From 087f6405d21ffe33181ced407ef67bd78583900d Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Fri, 28 Oct 2016 17:26:33 +0700 Subject: [PATCH 285/487] Web console beta-5. --- modules/web-console/DEVNOTES.txt | 2 +- modules/web-console/backend/app/agent.js | 26 +- modules/web-console/backend/app/browser.js | 14 +- modules/web-console/backend/app/mongo.js | 207 +- .../backend/routes/demo/domains.json | 15 +- .../web-console/backend/services/agents.js | 1 + modules/web-console/backend/services/auth.js | 8 +- .../web-console/backend/services/caches.js | 14 +- .../web-console/backend/services/clusters.js | 19 +- .../web-console/backend/services/domains.js | 12 +- modules/web-console/backend/services/igfss.js | 12 +- .../backend/test/unit/AuthService.test.js | 4 +- .../backend/test/unit/CacheService.test.js | 135 +- .../backend/test/unit/ClusterService.test.js | 132 +- .../backend/test/unit/DomainService.test.js | 144 +- .../backend/test/unit/IgfsService.test.js | 122 +- modules/web-console/frontend/.eslintrc | 6 +- .../web-console/frontend/app/app.config.js | 10 + modules/web-console/frontend/app/app.js | 37 +- .../frontend/app/data/dialects.json | 9 + .../frontend/app/data/java-classes.json | 4 +- .../frontend/app/decorator/tooltip.js | 13 +- .../app/directives/on-focus-out.directive.js | 37 + .../restore-input-focus.directive.js} | 9 +- .../ui-ace-java/ui-ace-java.controller.js | 78 +- .../ui-ace-java/ui-ace-java.directive.js | 100 +- .../ui-ace-pojos/ui-ace-pojos.controller.js | 2 +- .../ui-ace-pom/ui-ace-pom.controller.js | 2 +- .../ui-ace-sharp.controller.js} | 9 +- .../ui-ace-sharp.directive.js} | 28 +- .../directives/ui-ace-sharp/ui-ace-sharp.jade | 22 + .../ui-ace-spring/ui-ace-spring.controller.js | 88 + .../ui-ace-spring/ui-ace-spring.directive.js | 66 + .../ui-ace-spring.jade} | 0 .../ui-grid-settings/ui-grid-settings.jade | 33 + .../ui-grid-settings/ui-grid-settings.scss | 38 + .../default-name.filter.js} | 8 +- .../frontend/app/filters/hasPojo.filter.js | 5 +- .../jade/form/form-field-feedback.jade | 5 +- .../frontend/app/helpers/jade/mixins.jade | 219 +- .../app/modules/agent/agent.module.js | 10 +- .../modules/configuration/Version.service.js | 82 +- .../configuration/configuration.module.js | 30 +- .../generator/AbstractTransformer.js | 341 ++++ .../modules/configuration/generator/Beans.js | 379 ++++ .../generator/ConfigurationGenerator.js | 1785 +++++++++++++++++ .../generator/JavaTransformer.service.js | 1721 ++++++++++++++++ .../generator/PlatformGenerator.js | 522 +++++ .../configuration/generator/Pom.service.js | 173 +- .../generator/Properties.service.js | 74 + .../configuration/generator/Readme.service.js | 79 + .../generator/SharpTransformer.service.js | 243 +++ .../generator/SpringTransformer.service.js | 325 +++ .../configuration/generator/StringBuilder.js | 76 + .../defaults/cache.platform.provider.js | 60 + .../generator/defaults/cache.provider.js | 129 ++ .../defaults/cluster.platform.provider.js | 49 + .../generator/defaults/cluster.provider.js | 293 +++ .../generator/defaults/igfs.provider.js | 68 + .../generator/generator-common.js | 0 .../generator/generator-java.js | 14 +- .../generator/generator-optional.js | 0 .../generator/generator-spring.js} | 723 +++---- .../app/modules/form/field/input/text.scss | 1 - .../frontend/app/modules/form/form.module.js | 10 +- .../modules/form/panel/revert.directive.js | 4 +- .../services/FormGUID.service.js} | 7 +- .../java-built-in-class.directive.js | 6 +- .../validator/java-identifier.directive.js | 6 +- .../form/validator/java-keywords.directive.js | 15 +- .../java-package-specified.directive.js | 6 +- .../app/modules/nodes/Nodes.service.js | 69 + .../modules/nodes/nodes-dialog.controller.js | 68 + .../app/modules/nodes/nodes-dialog.jade | 35 + .../app/modules/nodes/nodes-dialog.scss | 20 + .../app/modules/nodes/nodes.module.js | 27 + .../app/modules/sql/sql.controller.js | 269 ++- .../app/modules/states/configuration.state.js | 2 +- .../caches/client-near-cache.jade | 50 + .../states/configuration/caches/general.jade | 3 + .../states/configuration/caches/memory.jade | 14 +- .../caches/near-cache-client.jade | 51 + ...near-cache.jade => near-cache-server.jade} | 9 +- .../configuration/caches/node-filter.jade | 49 - .../states/configuration/caches/query.jade | 30 +- .../configuration/caches/rebalance.jade | 3 +- .../states/configuration/caches/store.jade | 84 +- .../states/configuration/clusters/atomic.jade | 3 +- .../configuration/clusters/attributes.jade | 4 +- .../states/configuration/clusters/binary.jade | 28 +- .../configuration/clusters/cache-key-cfg.jade | 9 +- .../configuration/clusters/checkpoint.jade | 85 + .../configuration/clusters/checkpoint/fs.jade | 66 + .../clusters/checkpoint/jdbc.jade | 45 + .../configuration/clusters/checkpoint/s3.jade | 174 ++ .../configuration/clusters/collision.jade | 13 +- .../clusters/collision/job-stealing.jade | 2 +- .../configuration/clusters/communication.jade | 3 +- .../configuration/clusters/connector.jade | 3 +- .../configuration/clusters/deployment.jade | 3 +- .../configuration/clusters/discovery.jade | 3 +- .../states/configuration/clusters/events.jade | 37 +- .../configuration/clusters/failover.jade | 19 +- .../configuration/clusters/general.jade | 3 + .../clusters/general/discovery/zookeeper.jade | 2 + .../states/configuration/clusters/igfs.jade | 3 +- .../clusters/load-balancing.jade | 104 + .../configuration/clusters/marshaller.jade | 3 +- .../states/configuration/clusters/odbc.jade | 3 +- .../states/configuration/clusters/ssl.jade | 7 +- .../states/configuration/clusters/swap.jade | 3 +- .../configuration/clusters/transactions.jade | 6 +- .../states/configuration/domains/general.jade | 12 +- .../states/configuration/domains/query.jade | 40 +- .../states/configuration/domains/store.jade | 15 +- .../states/configuration/igfs/general.jade | 3 + .../states/configuration/igfs/ipc.jade | 2 +- .../states/configuration/igfs/misc.jade | 4 +- .../states/configuration/igfs/secondary.jade | 3 +- .../summary/summary.controller.js | 106 +- .../app/services/ErrorPopover.service.js | 2 +- .../app/services/FormUtils.service.js | 6 +- .../app/services/JavaTypes.service.js | 70 +- .../app/services/LegacyTable.service.js | 24 +- .../app/services/LegacyUtils.service.js | 128 +- .../frontend/app/services/SqlTypes.service.js | 6 +- modules/web-console/frontend/app/vendor.js | 3 +- .../frontend/controllers/caches-controller.js | 121 +- .../controllers/clusters-controller.js | 177 +- .../controllers/domains-controller.js | 148 +- .../frontend/controllers/igfs-controller.js | 6 +- .../generator/generator-properties.js | 175 -- .../frontend/generator/generator-readme.js | 85 - .../frontend/gulpfile.babel.js/paths.js | 31 +- .../frontend/gulpfile.babel.js/tasks/jade.js | 21 +- .../frontend/gulpfile.babel.js/tasks/watch.js | 6 +- .../gulpfile.babel.js/webpack/common.js | 2 +- .../webpack/environments/development.js | 17 +- .../webpack/environments/test.js | 52 + .../gulpfile.babel.js/webpack/index.js | 4 +- modules/web-console/frontend/package.json | 16 +- .../frontend/public/stylesheets/style.scss | 75 +- .../frontend/test/karma.conf.babel.js | 91 + .../web-console/frontend/test/karma.conf.js | 98 +- .../test/unit/JavaTransformer.test.js | 57 + .../frontend/test/unit/JavaTypes.test.js | 23 +- .../test/unit/SharpTransformer.test.js | 55 + .../test/unit/SpringTransformer.test.js | 57 + .../frontend/test/unit/SqlTypes.test.js | 17 - .../frontend/test/unit/Version.test.js | 82 + .../test/unit/defaultName.filter.test.js | 38 + .../frontend/views/configuration/caches.jade | 3 +- .../views/configuration/clusters.jade | 3 +- .../views/configuration/domains-import.jade | 14 +- .../summary-project-structure.jade | 2 +- .../frontend/views/configuration/summary.jade | 57 +- .../frontend/views/sql/cache-metadata.jade | 2 +- .../frontend/views/sql/notebook-new.jade | 4 +- .../web-console/frontend/views/sql/sql.jade | 80 +- .../views/templates/agent-download.jade | 6 +- .../views/templates/batch-confirm.jade | 4 +- .../frontend/views/templates/clone.jade | 4 +- .../frontend/views/templates/confirm.jade | 4 +- .../frontend/views/templates/demo-info.jade | 4 +- .../views/templates/getting-started.jade | 4 +- .../frontend/views/templates/message.jade | 4 +- .../frontend/views/templates/select.jade | 2 +- .../ignite/console/demo/AgentClusterDemo.java | 2 +- 168 files changed, 10271 insertions(+), 2146 deletions(-) create mode 100644 modules/web-console/frontend/app/data/dialects.json create mode 100644 modules/web-console/frontend/app/directives/on-focus-out.directive.js rename modules/web-console/frontend/app/{modules/configuration/generator/Java.service.js => directives/restore-input-focus.directive.js} (81%) rename modules/web-console/frontend/app/directives/{ui-ace-xml/ui-ace-xml.controller.js => ui-ace-sharp/ui-ace-sharp.controller.js} (74%) rename modules/web-console/frontend/app/directives/{ui-ace-xml/ui-ace-xml.directive.js => ui-ace-sharp/ui-ace-sharp.directive.js} (80%) create mode 100644 modules/web-console/frontend/app/directives/ui-ace-sharp/ui-ace-sharp.jade create mode 100644 modules/web-console/frontend/app/directives/ui-ace-spring/ui-ace-spring.controller.js create mode 100644 modules/web-console/frontend/app/directives/ui-ace-spring/ui-ace-spring.directive.js rename modules/web-console/frontend/app/directives/{ui-ace-xml/ui-ace-xml.jade => ui-ace-spring/ui-ace-spring.jade} (100%) create mode 100644 modules/web-console/frontend/app/directives/ui-grid-settings/ui-grid-settings.jade create mode 100644 modules/web-console/frontend/app/directives/ui-grid-settings/ui-grid-settings.scss rename modules/web-console/frontend/app/{modules/form/field/input/select.scss => filters/default-name.filter.js} (81%) create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/Beans.js create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/PlatformGenerator.js create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/StringBuilder.js create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js rename modules/web-console/frontend/{ => app/modules/configuration}/generator/generator-common.js (100%) rename modules/web-console/frontend/{ => app/modules/configuration}/generator/generator-java.js (99%) rename modules/web-console/frontend/{ => app/modules/configuration}/generator/generator-optional.js (100%) rename modules/web-console/frontend/{generator/generator-xml.js => app/modules/configuration/generator/generator-spring.js} (63%) rename modules/web-console/frontend/app/modules/{configuration/generator/Xml.service.js => form/services/FormGUID.service.js} (86%) create mode 100644 modules/web-console/frontend/app/modules/nodes/Nodes.service.js create mode 100644 modules/web-console/frontend/app/modules/nodes/nodes-dialog.controller.js create mode 100644 modules/web-console/frontend/app/modules/nodes/nodes-dialog.jade create mode 100644 modules/web-console/frontend/app/modules/nodes/nodes-dialog.scss create mode 100644 modules/web-console/frontend/app/modules/nodes/nodes.module.js create mode 100644 modules/web-console/frontend/app/modules/states/configuration/caches/client-near-cache.jade create mode 100644 modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.jade rename modules/web-console/frontend/app/modules/states/configuration/caches/{server-near-cache.jade => near-cache-server.jade} (90%) create mode 100644 modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade create mode 100644 modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade create mode 100644 modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade create mode 100644 modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade create mode 100644 modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade delete mode 100644 modules/web-console/frontend/generator/generator-properties.js delete mode 100644 modules/web-console/frontend/generator/generator-readme.js create mode 100644 modules/web-console/frontend/gulpfile.babel.js/webpack/environments/test.js create mode 100644 modules/web-console/frontend/test/karma.conf.babel.js create mode 100644 modules/web-console/frontend/test/unit/JavaTransformer.test.js create mode 100644 modules/web-console/frontend/test/unit/SharpTransformer.test.js create mode 100644 modules/web-console/frontend/test/unit/SpringTransformer.test.js create mode 100644 modules/web-console/frontend/test/unit/Version.test.js create mode 100644 modules/web-console/frontend/test/unit/defaultName.filter.test.js diff --git a/modules/web-console/DEVNOTES.txt b/modules/web-console/DEVNOTES.txt index cf2b6507cb7d6..85ec9587fd24f 100644 --- a/modules/web-console/DEVNOTES.txt +++ b/modules/web-console/DEVNOTES.txt @@ -9,7 +9,7 @@ How to deploy locally: run "npm install --no-optional" for download backend dependencies. 4. Change directory to '$IGNITE_HOME/modules/web-console/frontend' and run "npm install --no-optional" for download frontend dependencies. -5. Build ignite-web-agent module follow instructions from 'modules/web-agent/README.txt'. +5. Build ignite-web-agent module follow instructions from 'modules/web-console/web-agent/README.txt'. 6. Copy ignite-web-agent-.zip from '$IGNITE_HOME/modules/web-console/web-agent/target' to '$IGNITE_HOME/modules/web-console/backend/agent_dists' folder. diff --git a/modules/web-console/backend/app/agent.js b/modules/web-console/backend/app/agent.js index aa06c84b77cde..6aa9a1293e3d3 100644 --- a/modules/web-console/backend/app/agent.js +++ b/modules/web-console/backend/app/agent.js @@ -225,20 +225,32 @@ module.exports.factory = function(_, fs, path, JSZip, socketio, settings, mongo) * @param {String} nid Node id. * @param {String} cacheName Cache name. * @param {String} query Query. + * @param {Boolean} nonCollocatedJoins Flag whether to execute non collocated joins. * @param {Boolean} local Flag whether to execute query locally. * @param {int} pageSize Page size. * @returns {Promise} */ - fieldsQuery(demo, nid, cacheName, query, local, pageSize) { + fieldsQuery(demo, nid, cacheName, query, nonCollocatedJoins, local, pageSize) { const cmd = new Command(demo, 'exe') .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask') .addParam('p1', nid) - .addParam('p2', 'org.apache.ignite.internal.visor.query.VisorQueryTask') - .addParam('p3', 'org.apache.ignite.internal.visor.query.VisorQueryArg') - .addParam('p4', cacheName) - .addParam('p5', query) - .addParam('p6', local) - .addParam('p7', pageSize); + .addParam('p2', 'org.apache.ignite.internal.visor.query.VisorQueryTask'); + + if (nonCollocatedJoins) { + cmd.addParam('p3', 'org.apache.ignite.internal.visor.query.VisorQueryArgV2') + .addParam('p4', cacheName) + .addParam('p5', query) + .addParam('p6', true) + .addParam('p7', local) + .addParam('p8', pageSize); + } + else { + cmd.addParam('p3', 'org.apache.ignite.internal.visor.query.VisorQueryArg') + .addParam('p4', cacheName) + .addParam('p5', query) + .addParam('p6', local) + .addParam('p7', pageSize); + } return this.executeRest(cmd); } diff --git a/modules/web-console/backend/app/browser.js b/modules/web-console/backend/app/browser.js index d3a673ba06121..254851d0bb6b7 100644 --- a/modules/web-console/backend/app/browser.js +++ b/modules/web-console/backend/app/browser.js @@ -98,9 +98,9 @@ module.exports.factory = (_, socketio, agentMgr, configure) => { }); // Execute query on node and return first page to browser. - socket.on('node:query', (nid, cacheName, query, local, pageSize, cb) => { + socket.on('node:query', (nid, cacheName, query, distributedJoins, local, pageSize, cb) => { agentMgr.findAgent(accountId()) - .then((agent) => agent.fieldsQuery(demo, nid, cacheName, query, local, pageSize)) + .then((agent) => agent.fieldsQuery(demo, nid, cacheName, query, distributedJoins, local, pageSize)) .then((res) => cb(null, res)) .catch((err) => cb(_errorToJson(err))); }); @@ -114,13 +114,13 @@ module.exports.factory = (_, socketio, agentMgr, configure) => { }); // Execute query on node and return full result to browser. - socket.on('node:query:getAll', (nid, cacheName, query, local, cb) => { + socket.on('node:query:getAll', (nid, cacheName, query, distributedJoins, local, cb) => { // Set page size for query. const pageSize = 1024; agentMgr.findAgent(accountId()) .then((agent) => { - const firstPage = agent.fieldsQuery(demo, nid, cacheName, query, local, pageSize) + const firstPage = agent.fieldsQuery(demo, nid, cacheName, query, distributedJoins, local, pageSize) .then(({result}) => { if (result.key) return Promise.reject(result.key); @@ -133,10 +133,10 @@ module.exports.factory = (_, socketio, agentMgr, configure) => { return acc; return agent.queryFetch(demo, acc.responseNodeId, acc.queryId, pageSize) - .then((res) => { - acc.rows = acc.rows.concat(res.rows); + .then(({result}) => { + acc.rows = acc.rows.concat(result.rows); - acc.hasMore = res.hasMore; + acc.hasMore = result.hasMore; return fetchResult(acc); }); diff --git a/modules/web-console/backend/app/mongo.js b/modules/web-console/backend/app/mongo.js index ba7ed09a663ca..e12af2a6c8dc1 100644 --- a/modules/web-console/backend/app/mongo.js +++ b/modules/web-console/backend/app/mongo.js @@ -123,7 +123,7 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose indexType: {type: String, enum: ['SORTED', 'FULLTEXT', 'GEOSPATIAL']}, fields: [{name: String, direction: Boolean}] }], - demo: Boolean + generatePojo: Boolean }); DomainModelSchema.index({valueType: 1, space: 1}, {unique: true}); @@ -148,9 +148,6 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose IGFS: { igfs: {type: ObjectId, ref: 'Igfs'} }, - OnNodes: { - nodeIds: [String] - }, Custom: { className: String } @@ -199,7 +196,8 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose dialect: { type: String, enum: ['Generic', 'Oracle', 'DB2', 'SQLServer', 'MySQL', 'PostgreSQL', 'H2'] - } + }, + sqlEscapeAll: Boolean }, CacheJdbcBlobStoreFactory: { connectVia: {type: String, enum: ['URL', 'DataSource']}, @@ -218,7 +216,7 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose deleteQuery: String }, CacheHibernateBlobStoreFactory: { - hibernateProperties: [String] + hibernateProperties: [{name: String, value: String}] } }, storeKeepBinary: Boolean, @@ -248,8 +246,8 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose readFromBackup: Boolean, copyOnRead: Boolean, maxConcurrentAsyncOperations: Number, - nearCacheEnabled: Boolean, nearConfiguration: { + enabled: Boolean, nearStartSize: Number, nearEvictionPolicy: { kind: {type: String, enum: ['LRU', 'FIFO', 'SORTED']}, @@ -270,7 +268,28 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose } } }, - demo: Boolean + clientNearConfiguration: { + enabled: Boolean, + nearStartSize: Number, + nearEvictionPolicy: { + kind: {type: String, enum: ['LRU', 'FIFO', 'SORTED']}, + LRU: { + batchSize: Number, + maxMemorySize: Number, + maxSize: Number + }, + FIFO: { + batchSize: Number, + maxMemorySize: Number, + maxSize: Number + }, + SORTED: { + batchSize: Number, + maxMemorySize: Number, + maxSize: Number + } + } + } }); CacheSchema.index({name: 1, space: 1}, {unique: true}); @@ -370,7 +389,50 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose addresses: [String] }, S3: { - bucketName: String + bucketName: String, + clientConfiguration: { + protocol: {type: String, enum: ['HTTP', 'HTTPS']}, + maxConnections: Number, + userAgent: String, + localAddress: String, + proxyHost: String, + proxyPort: Number, + proxyUsername: String, + proxyDomain: String, + proxyWorkstation: String, + retryPolicy: { + kind: {type: String, enum: ['Default', 'DefaultMaxRetries', 'DynamoDB', 'DynamoDBMaxRetries', 'Custom', 'CustomClass']}, + DefaultMaxRetries: { + maxErrorRetry: Number + }, + DynamoDBMaxRetries: { + maxErrorRetry: Number + }, + Custom: { + retryCondition: String, + backoffStrategy: String, + maxErrorRetry: Number, + honorMaxErrorRetryInClientConfig: Boolean + }, + CustomClass: { + className: String + } + }, + maxErrorRetry: Number, + socketTimeout: Number, + connectionTimeout: Number, + requestTimeout: Number, + useReaper: Boolean, + useGzip: Boolean, + signerOverride: String, + preemptiveBasicProxyAuth: Boolean, + connectionTTL: Number, + connectionMaxIdleMillis: Number, + useTcpKeepAlive: Boolean, + dnsResolver: String, + responseMetadataCacheSize: Number, + secureRandom: String + } }, Cloud: { credential: String, @@ -463,6 +525,17 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose igfsThreadPoolSize: Number, igfss: [{type: ObjectId, ref: 'Igfs'}], includeEventTypes: [String], + eventStorage: { + kind: {type: String, enum: ['Memory', 'Custom']}, + Memory: { + expireAgeMs: Number, + expireCount: Number, + filter: String + }, + Custom: { + className: String + } + }, managementThreadPoolSize: Number, marshaller: { kind: {type: String, enum: ['OptimizedMarshaller', 'JdkMarshaller']}, @@ -628,6 +701,122 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose cacheKeyConfiguration: [{ typeName: String, affinityKeyFieldName: String + }], + checkpointSpi: [{ + kind: {type: String, enum: ['FS', 'Cache', 'S3', 'JDBC', 'Custom']}, + FS: { + directoryPaths: [String], + checkpointListener: String + }, + Cache: { + cache: {type: ObjectId, ref: 'Cache'}, + checkpointListener: String + }, + S3: { + awsCredentials: { + kind: {type: String, enum: ['Basic', 'Properties', 'Anonymous', 'BasicSession', 'Custom']}, + Properties: { + path: String + }, + Custom: { + className: String + } + }, + bucketNameSuffix: String, + clientConfiguration: { + protocol: {type: String, enum: ['HTTP', 'HTTPS']}, + maxConnections: Number, + userAgent: String, + localAddress: String, + proxyHost: String, + proxyPort: Number, + proxyUsername: String, + proxyDomain: String, + proxyWorkstation: String, + retryPolicy: { + kind: {type: String, enum: ['Default', 'DefaultMaxRetries', 'DynamoDB', 'DynamoDBMaxRetries', 'Custom']}, + DefaultMaxRetries: { + maxErrorRetry: Number + }, + DynamoDBMaxRetries: { + maxErrorRetry: Number + }, + Custom: { + retryCondition: String, + backoffStrategy: String, + maxErrorRetry: Number, + honorMaxErrorRetryInClientConfig: Boolean + } + }, + maxErrorRetry: Number, + socketTimeout: Number, + connectionTimeout: Number, + requestTimeout: Number, + useReaper: Boolean, + useGzip: Boolean, + signerOverride: String, + preemptiveBasicProxyAuth: Boolean, + connectionTTL: Number, + connectionMaxIdleMillis: Number, + useTcpKeepAlive: Boolean, + dnsResolver: String, + responseMetadataCacheSize: Number, + secureRandom: String + }, + checkpointListener: String + }, + JDBC: { + dataSourceBean: String, + dialect: { + type: String, + enum: ['Generic', 'Oracle', 'DB2', 'SQLServer', 'MySQL', 'PostgreSQL', 'H2'] + }, + user: String, + checkpointTableName: String, + keyFieldName: String, + keyFieldType: String, + valueFieldName: String, + valueFieldType: String, + expireDateFieldName: String, + expireDateFieldType: String, + numberOfRetries: Number, + checkpointListener: String + }, + Custom: { + className: String + } + }], + loadBalancingSpi: [{ + kind: {type: String, enum: ['RoundRobin', 'Adaptive', 'WeightedRandom', 'Custom']}, + RoundRobin: { + perTask: Boolean + }, + Adaptive: { + loadProbe: { + kind: {type: String, enum: ['Job', 'CPU', 'ProcessingTime', 'Custom']}, + Job: { + useAverage: Boolean + }, + CPU: { + useAverage: Boolean, + useProcessors: Boolean, + processorCoefficient: Number + }, + ProcessingTime: { + useAverage: Boolean + }, + Custom: { + className: String + } + } + }, + WeightedRandom: { + nodeWeight: Number, + useWeights: Boolean + }, + Custom: { + className: String + } }] }); diff --git a/modules/web-console/backend/routes/demo/domains.json b/modules/web-console/backend/routes/demo/domains.json index 980d8d1059a94..ae46758e5ddba 100644 --- a/modules/web-console/backend/routes/demo/domains.json +++ b/modules/web-console/backend/routes/demo/domains.json @@ -39,7 +39,8 @@ "javaFieldType": "int" } ], - "caches": [] + "caches": [], + "generatePojo": true }, { "keyType": "Integer", @@ -81,7 +82,8 @@ "javaFieldType": "int" } ], - "caches": [] + "caches": [], + "generatePojo": true }, { "keyType": "Integer", @@ -218,7 +220,8 @@ "javaFieldType": "int" } ], - "caches": [] + "caches": [], + "generatePojo": true }, { "keyType": "Integer", @@ -260,7 +263,8 @@ "javaFieldType": "int" } ], - "caches": [] + "caches": [], + "generatePojo": true }, { "keyType": "Integer", @@ -302,6 +306,7 @@ "javaFieldType": "int" } ], - "caches": [] + "caches": [], + "generatePojo": true } ] diff --git a/modules/web-console/backend/services/agents.js b/modules/web-console/backend/services/agents.js index 8a657395f8538..5c0b6a7800653 100644 --- a/modules/web-console/backend/services/agents.js +++ b/modules/web-console/backend/services/agents.js @@ -38,6 +38,7 @@ module.exports.factory = (_, fs, path, JSZip, settings, agentMgr, errors) => { class AgentsService { /** * Get agent archive with user agent configuration. + * * @returns {*} - readable stream for further piping. (http://stuk.github.io/jszip/documentation/api_jszip/generate_node_stream.html) */ static getArchive(host, token) { diff --git a/modules/web-console/backend/services/auth.js b/modules/web-console/backend/services/auth.js index 67a3f2f991971..c3423da95687c 100644 --- a/modules/web-console/backend/services/auth.js +++ b/modules/web-console/backend/services/auth.js @@ -36,6 +36,7 @@ module.exports.factory = (_, mongo, settings, errors) => { class AuthService { /** * Generate token string. + * * @param length - length of string * @returns {string} - generated token */ @@ -53,7 +54,8 @@ module.exports.factory = (_, mongo, settings, errors) => { } /** - * Reset password reset token for user + * Reset password reset token for user. + * * @param email - user email * @returns {Promise.} - that resolves account found by email with new reset password token. */ @@ -71,6 +73,7 @@ module.exports.factory = (_, mongo, settings, errors) => { /** * Reset password by reset token. + * * @param {string} token - reset token * @param {string} newPassword - new password * @returns {Promise.} - that resolves account with new password @@ -95,7 +98,8 @@ module.exports.factory = (_, mongo, settings, errors) => { } /** - * Find account by token + * Find account by token. + * * @param {string} token - reset token * @returns {Promise.<{token, email}>} - that resolves token and user email */ diff --git a/modules/web-console/backend/services/caches.js b/modules/web-console/backend/services/caches.js index f8cc2edd0790b..e59d51df0281f 100644 --- a/modules/web-console/backend/services/caches.js +++ b/modules/web-console/backend/services/caches.js @@ -34,14 +34,16 @@ module.exports = { module.exports.factory = (_, mongo, spaceService, errors) => { /** * Convert remove status operation to own presentation. + * * @param {RemoveResult} result - The results of remove operation. */ const convertRemoveStatus = ({result}) => ({rowsAffected: result.n}); /** - * Update existing cache - * @param {Object} cache - The cache for updating - * @returns {Promise.} that resolves cache id + * Update existing cache. + * + * @param {Object} cache - The cache for updating. + * @returns {Promise.} that resolves cache id. */ const update = (cache) => { const cacheId = cache._id; @@ -60,6 +62,7 @@ module.exports.factory = (_, mongo, spaceService, errors) => { /** * Create new cache. + * * @param {Object} cache - The cache for creation. * @returns {Promise.} that resolves cache id. */ @@ -78,6 +81,7 @@ module.exports.factory = (_, mongo, spaceService, errors) => { /** * Remove all caches by space ids. + * * @param {Number[]} spaceIds - The space ids for cache deletion. * @returns {Promise.} - that resolves results of remove operation. */ @@ -93,6 +97,7 @@ module.exports.factory = (_, mongo, spaceService, errors) => { class CachesService { /** * Create or update cache. + * * @param {Object} cache - The cache. * @returns {Promise.} that resolves cache id of merge operation. */ @@ -105,6 +110,7 @@ module.exports.factory = (_, mongo, spaceService, errors) => { /** * Get caches by spaces. + * * @param {mongo.ObjectId|String} spaceIds - The spaces ids that own caches. * @returns {Promise.} - contains requested caches. */ @@ -114,6 +120,7 @@ module.exports.factory = (_, mongo, spaceService, errors) => { /** * Remove cache. + * * @param {mongo.ObjectId|String} cacheId - The cache id for remove. * @returns {Promise.<{rowsAffected}>} - The number of affected rows. */ @@ -129,6 +136,7 @@ module.exports.factory = (_, mongo, spaceService, errors) => { /** * Remove all caches by user. + * * @param {mongo.ObjectId|String} userId - The user id that own caches. * @param {Boolean} demo - The flag indicates that need lookup in demo space. * @returns {Promise.<{rowsAffected}>} - The number of affected rows. diff --git a/modules/web-console/backend/services/clusters.js b/modules/web-console/backend/services/clusters.js index 6c2722b9b5fb0..06e413a96c08b 100644 --- a/modules/web-console/backend/services/clusters.js +++ b/modules/web-console/backend/services/clusters.js @@ -34,14 +34,16 @@ module.exports = { module.exports.factory = (_, mongo, spacesService, errors) => { /** * Convert remove status operation to own presentation. + * * @param {RemoveResult} result - The results of remove operation. */ const convertRemoveStatus = ({result}) => ({rowsAffected: result.n}); /** - * Update existing cluster - * @param {Object} cluster - The cluster for updating - * @returns {Promise.} that resolves cluster id + * Update existing cluster. + * + * @param {Object} cluster - The cluster for updating. + * @returns {Promise.} that resolves cluster id. */ const update = (cluster) => { const clusterId = cluster._id; @@ -60,6 +62,7 @@ module.exports.factory = (_, mongo, spacesService, errors) => { /** * Create new cluster. + * * @param {Object} cluster - The cluster for creation. * @returns {Promise.} that resolves cluster id. */ @@ -78,6 +81,7 @@ module.exports.factory = (_, mongo, spacesService, errors) => { /** * Remove all caches by space ids. + * * @param {Number[]} spaceIds - The space ids for cache deletion. * @returns {Promise.} - that resolves results of remove operation. */ @@ -90,7 +94,8 @@ module.exports.factory = (_, mongo, spacesService, errors) => { class ClustersService { /** * Create or update cluster. - * @param {Object} cluster - The cluster + * + * @param {Object} cluster - The cluster. * @returns {Promise.} that resolves cluster id of merge operation. */ static merge(cluster) { @@ -102,8 +107,9 @@ module.exports.factory = (_, mongo, spacesService, errors) => { /** * Get clusters and linked objects by space. - * @param {mongo.ObjectId|String} spaceIds - The spaces id that own cluster. - * @returns {Promise.<[mongo.Cache[], mongo.Cluster[], mongo.DomainModel[], mongo.Space[]]>} - contains requested caches and array of linked objects: clusters, domains, spaces. + * + * @param {mongo.ObjectId|String} spaceIds The spaces ids that own clusters. + * @returns {Promise.>} Requested clusters. */ static listBySpaces(spaceIds) { return mongo.Cluster.find({space: {$in: spaceIds}}).sort('name').lean().exec(); @@ -111,6 +117,7 @@ module.exports.factory = (_, mongo, spacesService, errors) => { /** * Remove cluster. + * * @param {mongo.ObjectId|String} clusterId - The cluster id for remove. * @returns {Promise.<{rowsAffected}>} - The number of affected rows. */ diff --git a/modules/web-console/backend/services/domains.js b/modules/web-console/backend/services/domains.js index 791e229597f62..22582c17f6e74 100644 --- a/modules/web-console/backend/services/domains.js +++ b/modules/web-console/backend/services/domains.js @@ -35,6 +35,7 @@ module.exports = { module.exports.factory = (_, mongo, spacesService, cachesService, errors) => { /** * Convert remove status operation to own presentation. + * * @param {RemoveResult} result - The results of remove operation. */ const convertRemoveStatus = ({result}) => ({rowsAffected: result.n}); @@ -43,7 +44,8 @@ module.exports.factory = (_, mongo, spacesService, cachesService, errors) => { Promise.all(_.map(cacheStoreChanges, (change) => mongo.Cache.update({_id: {$eq: change.cacheId}}, change.change, {}).exec())); /** - * Update existing domain + * Update existing domain. + * * @param {Object} domain - The domain for updating * @param savedDomains List of saved domains. * @returns {Promise.} that resolves domain id @@ -67,6 +69,7 @@ module.exports.factory = (_, mongo, spacesService, cachesService, errors) => { /** * Create new domain. + * * @param {Object} domain - The domain for creation. * @param savedDomains List of saved domains. * @returns {Promise.} that resolves cluster id. @@ -131,6 +134,7 @@ module.exports.factory = (_, mongo, spacesService, cachesService, errors) => { /** * Remove all caches by space ids. + * * @param {Array.} spaceIds - The space ids for cache deletion. * @returns {Promise.} - that resolves results of remove operation. */ @@ -142,6 +146,7 @@ module.exports.factory = (_, mongo, spacesService, cachesService, errors) => { class DomainsService { /** * Batch merging domains. + * * @param {Array.} domains */ static batchMerge(domains) { @@ -150,9 +155,9 @@ module.exports.factory = (_, mongo, spacesService, cachesService, errors) => { /** * Get domain and linked objects by space. + * * @param {mongo.ObjectId|String} spaceIds - The space id that own domain. - * @returns {Promise.<[mongo.Cache[], mongo.Cluster[], mongo.DomainModel[], mongo.Space[]]>} - * contains requested domains and array of linked objects: caches, spaces. + * @returns {Promise.>} contains requested domains. */ static listBySpaces(spaceIds) { return mongo.DomainModel.find({space: {$in: spaceIds}}).sort('valueType').lean().exec(); @@ -160,6 +165,7 @@ module.exports.factory = (_, mongo, spacesService, cachesService, errors) => { /** * Remove domain. + * * @param {mongo.ObjectId|String} domainId - The domain id for remove. * @returns {Promise.<{rowsAffected}>} - The number of affected rows. */ diff --git a/modules/web-console/backend/services/igfss.js b/modules/web-console/backend/services/igfss.js index 20f0121fa3599..578c6428cace4 100644 --- a/modules/web-console/backend/services/igfss.js +++ b/modules/web-console/backend/services/igfss.js @@ -34,12 +34,14 @@ module.exports = { module.exports.factory = (_, mongo, spacesService, errors) => { /** * Convert remove status operation to own presentation. + * * @param {RemoveResult} result - The results of remove operation. */ const convertRemoveStatus = ({result}) => ({rowsAffected: result.n}); /** - * Update existing IGFS + * Update existing IGFS. + * * @param {Object} igfs - The IGFS for updating * @returns {Promise.} that resolves IGFS id */ @@ -58,6 +60,7 @@ module.exports.factory = (_, mongo, spacesService, errors) => { /** * Create new IGFS. + * * @param {Object} igfs - The IGFS for creation. * @returns {Promise.} that resolves IGFS id. */ @@ -75,6 +78,7 @@ module.exports.factory = (_, mongo, spacesService, errors) => { /** * Remove all IGFSs by space ids. + * * @param {Number[]} spaceIds - The space ids for IGFS deletion. * @returns {Promise.} - that resolves results of remove operation. */ @@ -86,6 +90,7 @@ module.exports.factory = (_, mongo, spacesService, errors) => { class IgfssService { /** * Create or update IGFS. + * * @param {Object} igfs - The IGFS * @returns {Promise.} that resolves IGFS id of merge operation. */ @@ -98,8 +103,9 @@ module.exports.factory = (_, mongo, spacesService, errors) => { /** * Get IGFS by spaces. + * * @param {mongo.ObjectId|String} spacesIds - The spaces ids that own IGFSs. - * @returns {Promise.} - contains requested IGFSs. + * @returns {Promise.>} - contains requested IGFSs. */ static listBySpaces(spacesIds) { return mongo.Igfs.find({space: {$in: spacesIds}}).sort('name').lean().exec(); @@ -107,6 +113,7 @@ module.exports.factory = (_, mongo, spacesService, errors) => { /** * Remove IGFS. + * * @param {mongo.ObjectId|String} igfsId - The IGFS id for remove. * @returns {Promise.<{rowsAffected}>} - The number of affected rows. */ @@ -121,6 +128,7 @@ module.exports.factory = (_, mongo, spacesService, errors) => { /** * Remove all IGFSes by user. + * * @param {mongo.ObjectId|String} userId - The user id that own IGFS. * @param {Boolean} demo - The flag indicates that need lookup in demo space. * @returns {Promise.<{rowsAffected}>} - The number of affected rows. diff --git a/modules/web-console/backend/test/unit/AuthService.test.js b/modules/web-console/backend/test/unit/AuthService.test.js index eec60c41ea5d1..5ce473d272013 100644 --- a/modules/web-console/backend/test/unit/AuthService.test.js +++ b/modules/web-console/backend/test/unit/AuthService.test.js @@ -35,9 +35,7 @@ suite('AuthServiceTestsSuite', () => { }); }); - setup(() => { - return db.init(); - }); + setup(() => db.init()); test('Check token generator', () => { const tokenLength = 16; diff --git a/modules/web-console/backend/test/unit/CacheService.test.js b/modules/web-console/backend/test/unit/CacheService.test.js index 980f47afb0e9b..304f62c547346 100644 --- a/modules/web-console/backend/test/unit/CacheService.test.js +++ b/modules/web-console/backend/test/unit/CacheService.test.js @@ -19,56 +19,37 @@ const assert = require('chai').assert; const injector = require('../injector'); const testCaches = require('../data/caches.json'); const testAccounts = require('../data/accounts.json'); +const testSpaces = require('../data/spaces.json'); let cacheService; let mongo; let errors; +let db; suite('CacheServiceTestsSuite', () => { - const prepareUserSpaces = () => { - return mongo.Account.create(testAccounts) - .then((accounts) => { - return Promise.all( - accounts.map((account) => mongo.Space.create( - [ - {name: 'Personal space', owner: account._id, demo: false}, - {name: 'Demo space', owner: account._id, demo: true} - ] - ))) - .then((spaces) => [accounts, spaces]); - }); - }; - suiteSetup(() => { return Promise.all([injector('services/caches'), injector('mongo'), - injector('errors')]) - .then(([_cacheService, _mongo, _errors]) => { + injector('errors'), + injector('dbHelper')]) + .then(([_cacheService, _mongo, _errors, _db]) => { mongo = _mongo; cacheService = _cacheService; errors = _errors; + db = _db; }); }); - setup(() => { - return Promise.all([ - mongo.Cache.remove().exec(), - mongo.Account.remove().exec(), - mongo.Space.remove().exec() - ]); - }); + setup(() => db.init()); test('Create new cache', (done) => { - cacheService.merge(testCaches[0]) - .then((cache) => { - assert.isNotNull(cache._id); - - return cache._id; - }) - .then((cacheId) => mongo.Cache.findById(cacheId)) - .then((cache) => { - assert.isNotNull(cache); - }) + const dupleCache = Object.assign({}, testCaches[0], {name: 'Other name'}); + + delete dupleCache._id; + + cacheService.merge(dupleCache) + .then((cache) => mongo.Cache.findById(cache._id)) + .then((cache) => assert.isNotNull(cache)) .then(done) .catch(done); }); @@ -76,52 +57,43 @@ suite('CacheServiceTestsSuite', () => { test('Update existed cache', (done) => { const newName = 'NewUniqueName'; - cacheService.merge(testCaches[0]) - .then((cache) => { - const cacheBeforeMerge = Object.assign({}, testCaches[0], {_id: cache._id, name: newName}); + const cacheBeforeMerge = Object.assign({}, testCaches[0], {name: newName}); - return cacheService.merge(cacheBeforeMerge); - }) + cacheService.merge(cacheBeforeMerge) .then((cache) => mongo.Cache.findById(cache._id)) - .then((cacheAfterMerge) => { - assert.equal(cacheAfterMerge.name, newName); - }) + .then((cacheAfterMerge) => assert.equal(cacheAfterMerge.name, newName)) .then(done) .catch(done); }); test('Create duplicated cache', (done) => { - const dupleCache = Object.assign({}, testCaches[0], {_id: null}); + const dupleCache = Object.assign({}, testCaches[0]); - cacheService.merge(testCaches[0]) - .then(() => cacheService.merge(dupleCache)) + delete dupleCache._id; + + cacheService.merge(dupleCache) .catch((err) => { assert.instanceOf(err, errors.DuplicateKeyException); + done(); }); }); test('Remove existed cache', (done) => { - cacheService.merge(testCaches[0]) - .then((createdCache) => { - return mongo.Cache.findById(createdCache._id) - .then((foundCache) => foundCache._id) - .then(cacheService.remove) - .then(({rowsAffected}) => { - assert.equal(rowsAffected, 1); - }) - .then(() => mongo.Cache.findById(createdCache._id)) - .then((notFoundCache) => { - assert.isNull(notFoundCache); - }); - }) + cacheService.remove(testCaches[0]._id) + .then(({rowsAffected}) => + assert.equal(rowsAffected, 1) + ) + .then(() => mongo.Cache.findById(testCaches[0]._id)) + .then((notFoundCache) => + assert.isNull(notFoundCache) + ) .then(done) .catch(done); }); test('Remove cache without identifier', (done) => { - cacheService.merge(testCaches[0]) - .then(() => cacheService.remove()) + cacheService.remove() .catch((err) => { assert.instanceOf(err, errors.IllegalArgumentException); @@ -132,45 +104,28 @@ suite('CacheServiceTestsSuite', () => { test('Remove missed cache', (done) => { const validNoExistingId = 'FFFFFFFFFFFFFFFFFFFFFFFF'; - cacheService.merge(testCaches[0]) - .then(() => cacheService.remove(validNoExistingId)) - .then(({rowsAffected}) => { - assert.equal(rowsAffected, 0); - }) + cacheService.remove(validNoExistingId) + .then(({rowsAffected}) => + assert.equal(rowsAffected, 0) + ) .then(done) .catch(done); }); - test('Remove all caches in space', (done) => { - prepareUserSpaces() - .then(([accounts, spaces]) => { - const currentUser = accounts[0]; - const userCache = Object.assign({}, testCaches[0], {space: spaces[0][0]._id}); - - return cacheService.merge(userCache) - .then(() => cacheService.removeAll(currentUser._id, false)); - }) - .then(({rowsAffected}) => { - assert.equal(rowsAffected, 1); - }) + test('Get all caches by space', (done) => { + cacheService.listBySpaces(testSpaces[0]._id) + .then((caches) => + assert.equal(caches.length, 5) + ) .then(done) .catch(done); }); - test('Get all caches by space', (done) => { - prepareUserSpaces() - .then(([accounts, spaces]) => { - const userCache = Object.assign({}, testCaches[0], {space: spaces[0][0]._id}); - - return cacheService.merge(userCache) - .then((cache) => { - return cacheService.listBySpaces(spaces[0][0]._id) - .then((caches) => { - assert.equal(caches.length, 1); - assert.equal(caches[0]._id.toString(), cache._id.toString()); - }); - }); - }) + test('Remove all caches in space', (done) => { + cacheService.removeAll(testAccounts[0]._id, false) + .then(({rowsAffected}) => + assert.equal(rowsAffected, 5) + ) .then(done) .catch(done); }); diff --git a/modules/web-console/backend/test/unit/ClusterService.test.js b/modules/web-console/backend/test/unit/ClusterService.test.js index aff54f7a4ed0b..ed04c452fe574 100644 --- a/modules/web-console/backend/test/unit/ClusterService.test.js +++ b/modules/web-console/backend/test/unit/ClusterService.test.js @@ -19,55 +19,37 @@ const assert = require('chai').assert; const injector = require('../injector'); const testClusters = require('../data/clusters.json'); const testAccounts = require('../data/accounts.json'); +const testSpaces = require('../data/spaces.json'); let clusterService; let mongo; let errors; +let db; suite('ClusterServiceTestsSuite', () => { - const prepareUserSpaces = () => { - return mongo.Account.create(testAccounts) - .then((accounts) => { - return Promise.all(accounts.map((account) => mongo.Space.create( - [ - {name: 'Personal space', owner: account._id, demo: false}, - {name: 'Demo space', owner: account._id, demo: true} - ] - ))) - .then((spaces) => [accounts, spaces]); - }); - }; - suiteSetup(() => { return Promise.all([injector('services/clusters'), injector('mongo'), - injector('errors')]) - .then(([_clusterService, _mongo, _errors]) => { + injector('errors'), + injector('dbHelper')]) + .then(([_clusterService, _mongo, _errors, _db]) => { mongo = _mongo; clusterService = _clusterService; errors = _errors; + db = _db; }); }); - setup(() => { - return Promise.all([ - mongo.Cluster.remove().exec(), - mongo.Account.remove().exec(), - mongo.Space.remove().exec() - ]); - }); + setup(() => db.init()); test('Create new cluster', (done) => { - clusterService.merge(testClusters[0]) - .then((cluster) => { - assert.isNotNull(cluster._id); - - return cluster._id; - }) - .then((clusterId) => mongo.Cluster.findById(clusterId)) - .then((cluster) => { - assert.isNotNull(cluster); - }) + const dupleCluster = Object.assign({}, testClusters[0], {name: 'Other name'}); + + delete dupleCluster._id; + + clusterService.merge(dupleCluster) + .then((cluster) => mongo.Cluster.findById(cluster._id)) + .then((cluster) => assert.isNotNull(cluster)) .then(done) .catch(done); }); @@ -75,25 +57,21 @@ suite('ClusterServiceTestsSuite', () => { test('Update existed cluster', (done) => { const newName = 'NewUniqueName'; - clusterService.merge(testClusters[0]) - .then((cluster) => { - const clusterBeforeMerge = Object.assign({}, testClusters[0], {_id: cluster._id, name: newName}); + const clusterBeforeMerge = Object.assign({}, testClusters[0], {name: newName}); - return clusterService.merge(clusterBeforeMerge); - }) + clusterService.merge(clusterBeforeMerge) .then((cluster) => mongo.Cluster.findById(cluster._id)) - .then((clusterAfterMerge) => { - assert.equal(clusterAfterMerge.name, newName); - }) + .then((clusterAfterMerge) => assert.equal(clusterAfterMerge.name, newName)) .then(done) .catch(done); }); test('Create duplicated cluster', (done) => { - const dupleCluster = Object.assign({}, testClusters[0], {_id: null}); + const dupleCluster = Object.assign({}, testClusters[0]); - clusterService.merge(testClusters[0]) - .then(() => clusterService.merge(dupleCluster)) + delete dupleCluster._id; + + clusterService.merge(dupleCluster) .catch((err) => { assert.instanceOf(err, errors.DuplicateKeyException); @@ -102,25 +80,20 @@ suite('ClusterServiceTestsSuite', () => { }); test('Remove existed cluster', (done) => { - clusterService.merge(testClusters[0]) - .then((existCluster) => { - return mongo.Cluster.findById(existCluster._id) - .then((foundCluster) => clusterService.remove(foundCluster._id)) - .then(({rowsAffected}) => { - assert.equal(rowsAffected, 1); - }) - .then(() => mongo.Cluster.findById(existCluster._id)) - .then((notFoundCluster) => { - assert.isNull(notFoundCluster); - }); - }) + clusterService.remove(testClusters[0]._id) + .then(({rowsAffected}) => + assert.equal(rowsAffected, 1) + ) + .then(() => mongo.Cluster.findById(testClusters[0]._id)) + .then((notFoundCluster) => + assert.isNull(notFoundCluster) + ) .then(done) .catch(done); }); test('Remove cluster without identifier', (done) => { - clusterService.merge(testClusters[0]) - .then(() => clusterService.remove()) + clusterService.remove() .catch((err) => { assert.instanceOf(err, errors.IllegalArgumentException); @@ -131,45 +104,28 @@ suite('ClusterServiceTestsSuite', () => { test('Remove missed cluster', (done) => { const validNoExistingId = 'FFFFFFFFFFFFFFFFFFFFFFFF'; - clusterService.merge(testClusters[0]) - .then(() => clusterService.remove(validNoExistingId)) - .then(({rowsAffected}) => { - assert.equal(rowsAffected, 0); - }) + clusterService.remove(validNoExistingId) + .then(({rowsAffected}) => + assert.equal(rowsAffected, 0) + ) .then(done) .catch(done); }); - test('Remove all clusters in space', (done) => { - prepareUserSpaces() - .then(([accounts, spaces]) => { - const currentUser = accounts[0]; - const userCluster = Object.assign({}, testClusters[0], {space: spaces[0][0]._id}); - - return clusterService.merge(userCluster) - .then(() => clusterService.removeAll(currentUser._id, false)); - }) - .then(({rowsAffected}) => { - assert.equal(rowsAffected, 1); - }) + test('Get all clusters by space', (done) => { + clusterService.listBySpaces(testSpaces[0]._id) + .then((clusters) => + assert.equal(clusters.length, 2) + ) .then(done) .catch(done); }); - test('Get all clusters by space', (done) => { - prepareUserSpaces() - .then(([accounts, spaces]) => { - const userCluster = Object.assign({}, testClusters[0], {space: spaces[0][0]._id}); - - return clusterService.merge(userCluster) - .then((existCluster) => { - return clusterService.listBySpaces(spaces[0][0]._id) - .then((clusters) => { - assert.equal(clusters.length, 1); - assert.equal(clusters[0]._id.toString(), existCluster._id.toString()); - }); - }); - }) + test('Remove all clusters in space', (done) => { + clusterService.removeAll(testAccounts[0]._id, false) + .then(({rowsAffected}) => + assert.equal(rowsAffected, 2) + ) .then(done) .catch(done); }); diff --git a/modules/web-console/backend/test/unit/DomainService.test.js b/modules/web-console/backend/test/unit/DomainService.test.js index 8ce3fef3b0bea..c7cf1499c926b 100644 --- a/modules/web-console/backend/test/unit/DomainService.test.js +++ b/modules/web-console/backend/test/unit/DomainService.test.js @@ -19,56 +19,46 @@ const assert = require('chai').assert; const injector = require('../injector'); const testDomains = require('../data/domains.json'); const testAccounts = require('../data/accounts.json'); +const testSpaces = require('../data/spaces.json'); let domainService; let mongo; let errors; +let db; suite('DomainsServiceTestsSuite', () => { - const prepareUserSpaces = () => { - return mongo.Account.create(testAccounts) - .then((accounts) => { - return Promise.all(accounts.map((account) => mongo.Space.create( - [ - {name: 'Personal space', owner: account._id, demo: false}, - {name: 'Demo space', owner: account._id, demo: true} - ] - ))) - .then((spaces) => [accounts, spaces]); - }); - }; - suiteSetup(() => { return Promise.all([injector('services/domains'), injector('mongo'), - injector('errors')]) - .then(([_domainService, _mongo, _errors]) => { + injector('errors'), + injector('dbHelper')]) + .then(([_domainService, _mongo, _errors, _db]) => { mongo = _mongo; domainService = _domainService; errors = _errors; + db = _db; }); }); - setup(() => { - return Promise.all([ - mongo.DomainModel.remove().exec(), - mongo.Account.remove().exec(), - mongo.Space.remove().exec() - ]); - }); + setup(() => db.init()); test('Create new domain', (done) => { - domainService.batchMerge([testDomains[0]]) + const dupleDomain = Object.assign({}, testDomains[0], {valueType: 'other.Type'}); + + delete dupleDomain._id; + + domainService.batchMerge([dupleDomain]) .then((results) => { const domain = results.savedDomains[0]; - assert.isNotNull(domain._id); + assert.isObject(domain); + assert.isDefined(domain._id); - return domain._id; + return mongo.DomainModel.findById(domain._id); }) - .then((domainId) => mongo.DomainModel.findById(domainId)) .then((domain) => { - assert.isNotNull(domain); + assert.isObject(domain); + assert.isDefined(domain._id); }) .then(done) .catch(done); @@ -77,27 +67,31 @@ suite('DomainsServiceTestsSuite', () => { test('Update existed domain', (done) => { const newValType = 'value.Type'; - domainService.batchMerge([testDomains[0]]) - .then((results) => { - const domain = results.savedDomains[0]; + const domainBeforeMerge = Object.assign({}, testDomains[0], {valueType: newValType}); - const domainBeforeMerge = Object.assign({}, testDomains[0], {_id: domain._id, valueType: newValType}); + domainService.batchMerge([domainBeforeMerge]) + .then(({savedDomains, generatedCaches}) => { + assert.isArray(savedDomains); + assert.isArray(generatedCaches); - return domainService.batchMerge([domainBeforeMerge]); - }) - .then((results) => mongo.DomainModel.findById(results.savedDomains[0]._id)) - .then((domainAfterMerge) => { - assert.equal(domainAfterMerge.valueType, newValType); + assert.equal(1, savedDomains.length); + assert.equal(0, generatedCaches.length); + + return mongo.DomainModel.findById(savedDomains[0]._id); }) + .then((domainAfterMerge) => + assert.equal(domainAfterMerge.valueType, newValType) + ) .then(done) .catch(done); }); test('Create duplicated domain', (done) => { - const dupleDomain = Object.assign({}, testDomains[0], {_id: null}); + const dupleDomain = Object.assign({}, testDomains[0]); - domainService.batchMerge([testDomains[0]]) - .then(() => domainService.batchMerge([dupleDomain])) + delete dupleDomain._id; + + domainService.batchMerge([dupleDomain]) .catch((err) => { assert.instanceOf(err, errors.DuplicateKeyException); @@ -106,27 +100,20 @@ suite('DomainsServiceTestsSuite', () => { }); test('Remove existed domain', (done) => { - domainService.batchMerge([testDomains[0]]) - .then((results) => { - const domain = results.savedDomains[0]; - - return mongo.DomainModel.findById(domain._id) - .then((foundDomain) => domainService.remove(foundDomain._id)) - .then(({rowsAffected}) => { - assert.equal(rowsAffected, 1); - }) - .then(() => mongo.DomainModel.findById(domain._id)) - .then((notFoundDomain) => { - assert.isNull(notFoundDomain); - }); - }) + domainService.remove(testDomains[0]._id) + .then(({rowsAffected}) => + assert.equal(rowsAffected, 1) + ) + .then(() => mongo.DomainModel.findById(testDomains[0]._id)) + .then((notFoundDomain) => + assert.isNull(notFoundDomain) + ) .then(done) .catch(done); }); test('Remove domain without identifier', (done) => { - domainService.batchMerge([testDomains[0]]) - .then(() => domainService.remove()) + domainService.remove() .catch((err) => { assert.instanceOf(err, errors.IllegalArgumentException); @@ -137,47 +124,28 @@ suite('DomainsServiceTestsSuite', () => { test('Remove missed domain', (done) => { const validNoExistingId = 'FFFFFFFFFFFFFFFFFFFFFFFF'; - domainService.batchMerge([testDomains[0]]) - .then(() => domainService.remove(validNoExistingId)) - .then(({rowsAffected}) => { - assert.equal(rowsAffected, 0); - }) + domainService.remove(validNoExistingId) + .then(({rowsAffected}) => + assert.equal(rowsAffected, 0) + ) .then(done) .catch(done); }); - test('Remove all domains in space', (done) => { - prepareUserSpaces() - .then(([accounts, spaces]) => { - const currentUser = accounts[0]; - const userDomain = Object.assign({}, testDomains[0], {space: spaces[0][0]._id}); - - return domainService.batchMerge([userDomain]) - .then(() => domainService.removeAll(currentUser._id, false)); - }) - .then(({rowsAffected}) => { - assert.equal(rowsAffected, 1); - }) + test('Get all domains by space', (done) => { + domainService.listBySpaces(testSpaces[0]._id) + .then((domains) => + assert.equal(domains.length, 5) + ) .then(done) .catch(done); }); - test('Get all domains by space', (done) => { - prepareUserSpaces() - .then(([accounts, spaces]) => { - const userDomain = Object.assign({}, testDomains[0], {space: spaces[0][0]._id}); - - return domainService.batchMerge([userDomain]) - .then((results) => { - const domain = results.savedDomains[0]; - - return domainService.listBySpaces(spaces[0][0]._id) - .then((domains) => { - assert.equal(domains.length, 1); - assert.equal(domains[0]._id.toString(), domain._id.toString()); - }); - }); - }) + test('Remove all domains in space', (done) => { + domainService.removeAll(testAccounts[0]._id, false) + .then(({rowsAffected}) => + assert.equal(rowsAffected, 5) + ) .then(done) .catch(done); }); diff --git a/modules/web-console/backend/test/unit/IgfsService.test.js b/modules/web-console/backend/test/unit/IgfsService.test.js index 4fab27947a27f..67d9e087e58d7 100644 --- a/modules/web-console/backend/test/unit/IgfsService.test.js +++ b/modules/web-console/backend/test/unit/IgfsService.test.js @@ -19,55 +19,43 @@ const assert = require('chai').assert; const injector = require('../injector'); const testIgfss = require('../data/igfss.json'); const testAccounts = require('../data/accounts.json'); +const testSpaces = require('../data/spaces.json'); let igfsService; let mongo; let errors; +let db; suite('IgfsServiceTestsSuite', () => { - const prepareUserSpaces = () => { - return mongo.Account.create(testAccounts) - .then((accounts) => { - return Promise.all(accounts.map((account) => mongo.Space.create( - [ - {name: 'Personal space', owner: account._id, demo: false}, - {name: 'Demo space', owner: account._id, demo: true} - ] - ))) - .then((spaces) => [accounts, spaces]); - }); - }; - suiteSetup(() => { return Promise.all([injector('services/igfss'), injector('mongo'), - injector('errors')]) - .then(([_igfsService, _mongo, _errors]) => { + injector('errors'), + injector('dbHelper')]) + .then(([_igfsService, _mongo, _errors, _db]) => { mongo = _mongo; igfsService = _igfsService; errors = _errors; + db = _db; }); }); - setup(() => { - return Promise.all([ - mongo.Igfs.remove().exec(), - mongo.Account.remove().exec(), - mongo.Space.remove().exec() - ]); - }); + setup(() => db.init()); test('Create new igfs', (done) => { - igfsService.merge(testIgfss[0]) + const dupleIgfs = Object.assign({}, testIgfss[0], {name: 'Other name'}); + + delete dupleIgfs._id; + + igfsService.merge(dupleIgfs) .then((igfs) => { assert.isNotNull(igfs._id); - return igfs._id; - }) - .then((igfsId) => mongo.Igfs.findById(igfsId)) - .then((igfs) => { - assert.isNotNull(igfs); + return mongo.Igfs.findById(igfs._id); }) + .then((igfs) => + assert.isNotNull(igfs) + ) .then(done) .catch(done); }); @@ -75,25 +63,21 @@ suite('IgfsServiceTestsSuite', () => { test('Update existed igfs', (done) => { const newName = 'NewUniqueName'; - igfsService.merge(testIgfss[0]) - .then((existIgfs) => { - const igfsBeforeMerge = Object.assign({}, testIgfss[0], {_id: existIgfs._id, name: newName}); + const igfsBeforeMerge = Object.assign({}, testIgfss[0], {name: newName}); - return igfsService.merge(igfsBeforeMerge); - }) + igfsService.merge(igfsBeforeMerge) .then((igfs) => mongo.Igfs.findById(igfs._id)) - .then((igfsAfterMerge) => { - assert.equal(igfsAfterMerge.name, newName); - }) + .then((igfsAfterMerge) => assert.equal(igfsAfterMerge.name, newName)) .then(done) .catch(done); }); test('Create duplicated igfs', (done) => { - const dupleIfgs = Object.assign({}, testIgfss[0], {_id: null}); + const dupleIfgs = Object.assign({}, testIgfss[0]); + + delete dupleIfgs._id; - igfsService.merge(testIgfss[0]) - .then(() => igfsService.merge(dupleIfgs)) + igfsService.merge(dupleIfgs) .catch((err) => { assert.instanceOf(err, errors.DuplicateKeyException); @@ -102,25 +86,18 @@ suite('IgfsServiceTestsSuite', () => { }); test('Remove existed igfs', (done) => { - igfsService.merge(testIgfss[0]) - .then((existIgfs) => { - return mongo.Igfs.findById(existIgfs._id) - .then((foundIgfs) => igfsService.remove(foundIgfs._id)) - .then(({rowsAffected}) => { - assert.equal(rowsAffected, 1); - }) - .then(() => mongo.Igfs.findById(existIgfs._id)) - .then((notFoundIgfs) => { - assert.isNull(notFoundIgfs); - }); - }) + igfsService.remove(testIgfss[0]._id) + .then(({rowsAffected}) => assert.equal(rowsAffected, 1)) + .then(() => mongo.Igfs.findById(testIgfss[0]._id)) + .then((notFoundIgfs) => + assert.isNull(notFoundIgfs) + ) .then(done) .catch(done); }); test('Remove igfs without identifier', (done) => { - igfsService.merge(testIgfss[0]) - .then(() => igfsService.remove()) + igfsService.remove() .catch((err) => { assert.instanceOf(err, errors.IllegalArgumentException); @@ -131,45 +108,22 @@ suite('IgfsServiceTestsSuite', () => { test('Remove missed igfs', (done) => { const validNoExistingId = 'FFFFFFFFFFFFFFFFFFFFFFFF'; - igfsService.merge(testIgfss[0]) - .then(() => igfsService.remove(validNoExistingId)) - .then(({rowsAffected}) => { - assert.equal(rowsAffected, 0); - }) + igfsService.remove(validNoExistingId) + .then(({rowsAffected}) => assert.equal(rowsAffected, 0)) .then(done) .catch(done); }); - test('Remove all igfss in space', (done) => { - prepareUserSpaces() - .then(([accounts, spaces]) => { - const currentUser = accounts[0]; - const userIgfs = Object.assign({}, testIgfss[0], {space: spaces[0][0]._id}); - - return igfsService.merge(userIgfs) - .then(() => igfsService.removeAll(currentUser._id, false)); - }) - .then(({rowsAffected}) => { - assert.equal(rowsAffected, 1); - }) + test('Get all igfss by space', (done) => { + igfsService.listBySpaces(testSpaces[0]._id) + .then((igfss) => assert.equal(igfss.length, 1)) .then(done) .catch(done); }); - test('Get all igfss by space', (done) => { - prepareUserSpaces() - .then(([accounts, spaces]) => { - const userIgfs = Object.assign({}, testIgfss[0], {space: spaces[0][0]._id}); - - return igfsService.merge(userIgfs) - .then((existIgfs) => { - return igfsService.listBySpaces(spaces[0][0]._id) - .then((igfss) => { - assert.equal(igfss.length, 1); - assert.equal(igfss[0]._id.toString(), existIgfs._id.toString()); - }); - }); - }) + test('Remove all igfss in space', (done) => { + igfsService.removeAll(testAccounts[0]._id, false) + .then(({rowsAffected}) => assert.equal(rowsAffected, 1)) .then(done) .catch(done); }); diff --git a/modules/web-console/frontend/.eslintrc b/modules/web-console/frontend/.eslintrc index a8a3bb8ff7fa3..988cfa07c0888 100644 --- a/modules/web-console/frontend/.eslintrc +++ b/modules/web-console/frontend/.eslintrc @@ -27,12 +27,8 @@ globals: global: true angular: true $generatorCommon: true - $generatorProperties: true - $generatorXml: true + $generatorSpring: true $generatorJava: true - $generatorPom: true - $generatorReadme: true - $generatorDocker: true $generatorOptional: true saveAs: true process: true diff --git a/modules/web-console/frontend/app/app.config.js b/modules/web-console/frontend/app/app.config.js index 7adfc94fdd6e8..7416ce9e3a283 100644 --- a/modules/web-console/frontend/app/app.config.js +++ b/modules/web-console/frontend/app/app.config.js @@ -15,6 +15,16 @@ * limitations under the License. */ +import _ from 'lodash'; + +const nonNil = _.negate(_.isNil); +const nonEmpty = _.negate(_.isEmpty); + +_.mixin({ + nonNil, + nonEmpty +}); + import alertTemplateUrl from '../views/templates/alert.jade'; const igniteConsoleCfg = angular.module('ignite-console.config', ['ngAnimate', 'mgcrea.ngStrap']); diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js index 830d067be936f..3510743e129c3 100644 --- a/modules/web-console/frontend/app/app.js +++ b/modules/web-console/frontend/app/app.js @@ -16,6 +16,7 @@ */ import '../public/stylesheets/style.scss'; +import '../app/directives/ui-grid-settings/ui-grid-settings.scss'; import './helpers/jade/mixins.jade'; import './app.config'; @@ -26,6 +27,7 @@ import './decorator/tooltip'; import './modules/form/form.module'; import './modules/agent/agent.module.js'; import './modules/sql/sql.module'; +import './modules/nodes/nodes.module'; import './modules/Demo/Demo.module.js'; import './modules/states/signin.state'; @@ -60,12 +62,15 @@ import igniteOnClickFocus from './directives/on-click-focus.directive.js'; import igniteOnEnter from './directives/on-enter.directive.js'; import igniteOnEnterFocusMove from './directives/on-enter-focus-move.directive.js'; import igniteOnEscape from './directives/on-escape.directive.js'; -import igniteUiAceDocker from './directives/ui-ace-docker/ui-ace-docker.directive'; +import igniteOnFocusOut from './directives/on-focus-out.directive.js'; +import igniteRestoreInputFocus from './directives/restore-input-focus.directive.js'; import igniteUiAceJava from './directives/ui-ace-java/ui-ace-java.directive'; +import igniteUiAceSpring from './directives/ui-ace-spring/ui-ace-spring.directive'; +import igniteUiAceCSharp from './directives/ui-ace-sharp/ui-ace-sharp.directive'; import igniteUiAcePojos from './directives/ui-ace-pojos/ui-ace-pojos.directive'; import igniteUiAcePom from './directives/ui-ace-pom/ui-ace-pom.directive'; +import igniteUiAceDocker from './directives/ui-ace-docker/ui-ace-docker.directive'; import igniteUiAceTabs from './directives/ui-ace-tabs.directive'; -import igniteUiAceXml from './directives/ui-ace-xml/ui-ace-xml.directive'; import igniteRetainSelection from './directives/retain-selection.directive'; // Services. @@ -89,24 +94,15 @@ import UnsavedChangesGuard from './services/UnsavedChangesGuard.service'; // Filters. import byName from './filters/byName.filter'; +import defaultName from './filters/default-name.filter'; import domainsValidation from './filters/domainsValidation.filter'; -import hasPojo from './filters/hasPojo.filter'; import duration from './filters/duration.filter'; +import hasPojo from './filters/hasPojo.filter'; // Generators -import $generatorCommon from 'generator/generator-common'; -import $generatorJava from 'generator/generator-java'; -import $generatorOptional from 'generator/generator-optional'; -import $generatorProperties from 'generator/generator-properties'; -import $generatorReadme from 'generator/generator-readme'; -import $generatorXml from 'generator/generator-xml'; +import $generatorOptional from './modules/configuration/generator/generator-optional'; -window.$generatorCommon = $generatorCommon; -window.$generatorJava = $generatorJava; window.$generatorOptional = $generatorOptional; -window.$generatorProperties = $generatorProperties; -window.$generatorReadme = $generatorReadme; -window.$generatorXml = $generatorXml; // Controllers import admin from 'controllers/admin-controller'; @@ -152,6 +148,7 @@ angular 'ignite-console.socket', 'ignite-console.agent', 'ignite-console.sql', + 'ignite-console.nodes', 'ignite-console.demo', // States. 'ignite-console.states.login', @@ -184,13 +181,16 @@ angular .directive(...igniteOnEnter) .directive(...igniteOnEnterFocusMove) .directive(...igniteOnEscape) -.directive(...igniteUiAceDocker) +.directive(...igniteUiAceSpring) .directive(...igniteUiAceJava) +.directive(...igniteUiAceCSharp) .directive(...igniteUiAcePojos) .directive(...igniteUiAcePom) +.directive(...igniteUiAceDocker) .directive(...igniteUiAceTabs) -.directive(...igniteUiAceXml) .directive(...igniteRetainSelection) +.directive('igniteOnFocusOut', igniteOnFocusOut) +.directive('igniteRestoreInputFocus', igniteRestoreInputFocus) // Services. .service('IgniteErrorPopover', ErrorPopover) .service('JavaTypes', JavaTypes) @@ -219,10 +219,11 @@ angular .controller(...igfs) .controller(...profile) // Filters. -.filter(...hasPojo) -.filter(...domainsValidation) .filter(...byName) +.filter('defaultName', defaultName) +.filter(...domainsValidation) .filter(...duration) +.filter(...hasPojo) .config(['$stateProvider', '$locationProvider', '$urlRouterProvider', ($stateProvider, $locationProvider, $urlRouterProvider) => { // Set up the states. $stateProvider diff --git a/modules/web-console/frontend/app/data/dialects.json b/modules/web-console/frontend/app/data/dialects.json new file mode 100644 index 0000000000000..007fbc6a74470 --- /dev/null +++ b/modules/web-console/frontend/app/data/dialects.json @@ -0,0 +1,9 @@ +{ + "Generic": "org.apache.ignite.cache.store.jdbc.dialect.BasicJdbcDialect", + "Oracle": "org.apache.ignite.cache.store.jdbc.dialect.OracleDialect", + "DB2": "org.apache.ignite.cache.store.jdbc.dialect.DB2Dialect", + "SQLServer": "org.apache.ignite.cache.store.jdbc.dialect.SQLServerDialect", + "MySQL": "org.apache.ignite.cache.store.jdbc.dialect.MySQLDialect", + "PostgreSQL": "org.apache.ignite.cache.store.jdbc.dialect.BasicJdbcDialect", + "H2": "org.apache.ignite.cache.store.jdbc.dialect.H2Dialect" +} \ No newline at end of file diff --git a/modules/web-console/frontend/app/data/java-classes.json b/modules/web-console/frontend/app/data/java-classes.json index b0ec9fb4b7559..6704457a81e7d 100644 --- a/modules/web-console/frontend/app/data/java-classes.json +++ b/modules/web-console/frontend/app/data/java-classes.json @@ -15,5 +15,7 @@ {"short": "String", "full": "java.lang.String"}, {"short": "Time", "full": "java.sql.Time"}, {"short": "Timestamp", "full": "java.sql.Timestamp"}, - {"short": "UUID", "full": "java.util.UUID"} + {"short": "UUID", "full": "java.util.UUID"}, + {"short": "Serializable", "full": "java.io.Serializable"}, + {"short": "Class", "full": "java.lang.Class"} ] diff --git a/modules/web-console/frontend/app/decorator/tooltip.js b/modules/web-console/frontend/app/decorator/tooltip.js index 221cf7c15a565..71ea69478948a 100644 --- a/modules/web-console/frontend/app/decorator/tooltip.js +++ b/modules/web-console/frontend/app/decorator/tooltip.js @@ -28,12 +28,15 @@ angular.module('mgcrea.ngStrap.tooltip') let tipElementEntered = false; config.onShow = ($tooltip) => { - $tooltip.$element.on('mouseenter', () => tipElementEntered = true); - $tooltip.$element.on('mouseleave', () => { - tipElementEntered = false; + // Workaround for tooltip detection. + if ($tooltip.$element && $tooltip.$options.trigger === 'click hover') { + $tooltip.$element.on('mouseenter', () => tipElementEntered = true); + $tooltip.$element.on('mouseleave', () => { + tipElementEntered = false; - $tooltip.leave(); - }); + $tooltip.leave(); + }); + } }; const $tooltip = $delegate(element, config); diff --git a/modules/web-console/frontend/app/directives/on-focus-out.directive.js b/modules/web-console/frontend/app/directives/on-focus-out.directive.js new file mode 100644 index 0000000000000..2e6bfc6a84089 --- /dev/null +++ b/modules/web-console/frontend/app/directives/on-focus-out.directive.js @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +export default ['$timeout', '$parse', ($timeout, $parse) => { + return ($scope, $element, $attrs) => { + const handlerCheckFocusOut = (FocusClick) => { + if ($element.find(FocusClick.target).length) + return; + + $parse($attrs.igniteOnFocusOut)($scope); + + $timeout(); + }; + + window.addEventListener('click', handlerCheckFocusOut, true); + window.addEventListener('focusin', handlerCheckFocusOut, true); + + $scope.$on('$destroy', () => { + window.removeEventListener('click', handlerCheckFocusOut, true); + window.removeEventListener('focusin', handlerCheckFocusOut, true); + }); + }; +}]; diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Java.service.js b/modules/web-console/frontend/app/directives/restore-input-focus.directive.js similarity index 81% rename from modules/web-console/frontend/app/modules/configuration/generator/Java.service.js rename to modules/web-console/frontend/app/directives/restore-input-focus.directive.js index 67e19b9abbf32..32e66224482ad 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/Java.service.js +++ b/modules/web-console/frontend/app/directives/restore-input-focus.directive.js @@ -15,7 +15,10 @@ * limitations under the License. */ -// TODO IGNITE-2054: need move $generatorJava to services. -export default ['GeneratorJava', () => { - return $generatorJava; +export default [() => { + return ($scope, $element) => { + $element.on('click', () => { + $element.siblings('.input-tip').find('input').focus(); + }); + }; }]; diff --git a/modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.controller.js b/modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.controller.js index f869e65eb0a86..7e7a0e42b6488 100644 --- a/modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.controller.js +++ b/modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.controller.js @@ -15,18 +15,78 @@ * limitations under the License. */ -const SERVER_CFG = 'ServerConfigurationFactory'; -const CLIENT_CFG = 'ClientConfigurationFactory'; - -export default ['$scope', 'GeneratorJava', function($scope, generator) { +export default ['$scope', 'JavaTransformer', function($scope, java) { const ctrl = this; delete ctrl.data; - // Set default generator - ctrl.generator = (cluster) => { - const type = $scope.cfg ? CLIENT_CFG : SERVER_CFG; + const client = ctrl.client === 'true'; + + // Setup generator. + switch (ctrl.generator) { + case 'igniteConfiguration': + const clsName = client ? 'ClientConfigurationFactory' : 'ServerConfigurationFactory'; + + ctrl.generate = (cluster) => java.cluster(cluster, 'config', clsName, client); + + break; + case 'clusterCaches': + ctrl.generate = (cluster, caches) => { + const clusterCaches = _.reduce(caches, (acc, cache) => { + if (_.includes(cluster.caches, cache.value)) + acc.push(cache.cache); + + return acc; + }, []); + + const cfg = java.generator.clusterGeneral(cluster); + + java.generator.clusterCaches(cluster, clusterCaches, null, false, cfg); + + return java.toSection(cfg); + }; + + break; + case 'cacheStore': + case 'cacheQuery': + ctrl.generate = (cache, domains) => { + const cacheDomains = _.reduce(domains, (acc, domain) => { + if (_.includes(cache.domains, domain.value)) + acc.push(domain.meta); + + return acc; + }, []); + + return java[ctrl.generator](cache, cacheDomains); + }; + + break; + case 'cacheNodeFilter': + ctrl.generate = (cache, igfss) => { + const cacheIgfss = _.reduce(igfss, (acc, igfs) => { + acc.push(igfs.igfs); + + return acc; + }, []); + + return java.cacheNodeFilter(cache, cacheIgfss); + }; + + break; + case 'igfss': + ctrl.generate = (cluster, igfss) => { + const clusterIgfss = _.reduce(igfss, (acc, igfs) => { + if (_.includes(cluster.igfss, igfs.value)) + acc.push(igfs.igfs); + + return acc; + }, []); + + return java.clusterIgfss(clusterIgfss); + }; - return generator.cluster(cluster, 'config', type, $scope.cfg); - }; + break; + default: + ctrl.generate = (master, detail) => java[ctrl.generator](master, detail); + } }]; diff --git a/modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.directive.js b/modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.directive.js index fbb143135899c..c21bff7279b59 100644 --- a/modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.directive.js +++ b/modules/web-console/frontend/app/directives/ui-ace-java/ui-ace-java.directive.js @@ -18,7 +18,7 @@ import templateUrl from './ui-ace-java.jade'; import controller from './ui-ace-java.controller'; -export default ['igniteUiAceJava', ['GeneratorJava', (generator) => { +export default ['igniteUiAceJava', [() => { const link = (scope, $el, attrs, [ctrl, igniteUiAceTabs, formCtrl, ngModelCtrl]) => { if (formCtrl && ngModelCtrl) formCtrl.$removeControl(ngModelCtrl); @@ -34,96 +34,12 @@ export default ['igniteUiAceJava', ['GeneratorJava', (generator) => { if (igniteUiAceTabs && igniteUiAceTabs.onChange) scope.onChange = igniteUiAceTabs.onChange; - const render = (data) => { - delete ctrl.data; - - if (!data) - return; - - return ctrl.generator(scope.master); - }; - - // Setup generator. - if (scope.generator) { - const method = scope.generator; - - switch (method) { - case 'clusterCaches': - ctrl.generator = (cluster) => { - const caches = _.reduce(scope.detail, (acc, cache) => { - if (_.includes(cluster.caches, cache.value)) - acc.push(cache.cache); - - return acc; - }, []); - - return generator.clusterCaches(caches, null, true, generator.clusterGeneral(cluster)).asString(); - }; - - break; - - case 'igfss': - ctrl.generator = (cluster) => { - const igfss = _.reduce(scope.detail, (acc, igfs) => { - if (_.includes(cluster.igfss, igfs.value)) - acc.push(igfs.igfs); - - return acc; - }, []); - - return generator.igfss(igfss, 'cfg').asString(); - }; - - break; - - case 'cacheStore': - case 'cacheQuery': - ctrl.generator = (cache) => { - const domains = _.reduce(scope.detail, (acc, domain) => { - if (_.includes(cache.domains, domain.value)) - acc.push(domain.meta); - - return acc; - }, []); - - return generator[method](cache, domains).asString(); - }; - - break; - - case 'cacheNodeFilter': - ctrl.generator = (cache) => { - const igfss = _.reduce(scope.detail, (acc, igfs) => { - acc.push(igfs.igfs); - - return acc; - }, []); - - return generator.cacheNodeFilter(cache, igfss).asString(); - }; - - break; - - default: - ctrl.generator = (data) => generator[method](data).asString(); - } - } - - if (!_.isUndefined(attrs.clusterCfg)) { - scope.$watch('cfg', (cfg) => { - if (!_.isUndefined(cfg)) - return; - - scope.cfg = {}; - }); - - scope.$watch('cfg', (data) => ctrl.data = render(data), true); - } - const noDeepWatch = !(typeof attrs.noDeepWatch !== 'undefined'); // Setup watchers. - scope.$watch('master', (data) => ctrl.data = render(data), noDeepWatch); + scope.$watch('master', () => { + ctrl.data = _.isNil(scope.master) ? null : ctrl.generate(scope.master, scope.detail).asString(); + }, noDeepWatch); }; return { @@ -131,12 +47,12 @@ export default ['igniteUiAceJava', ['GeneratorJava', (generator) => { restrict: 'E', scope: { master: '=', - detail: '=', - generator: '@', - cfg: '=?clusterCfg' + detail: '=' }, bindToController: { - data: '=?ngModel' + data: '=?ngModel', + generator: '@', + client: '@' }, link, templateUrl, diff --git a/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js index bc185b38daeb4..4e1187447096c 100644 --- a/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js +++ b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js @@ -15,7 +15,7 @@ * limitations under the License. */ -export default ['$scope', 'JavaTypes', 'GeneratorJava', function($scope, JavaTypes, generator) { +export default ['$scope', 'JavaTypes', 'JavaTransformer', function($scope, JavaTypes, generator) { const ctrl = this; // Watchers definition. diff --git a/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js index 82afeac245ad9..2bf78c359230d 100644 --- a/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js +++ b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js @@ -25,7 +25,7 @@ export default ['$scope', 'GeneratorPom', 'IgniteVersion', function($scope, pom, if (!value) return; - ctrl.data = pom.generate($scope.cluster, Version.ignite).asString(); + ctrl.data = pom.generate($scope.cluster, Version.productVersion().ignite).asString(); }; // Setup watchers. diff --git a/modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.controller.js b/modules/web-console/frontend/app/directives/ui-ace-sharp/ui-ace-sharp.controller.js similarity index 74% rename from modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.controller.js rename to modules/web-console/frontend/app/directives/ui-ace-sharp/ui-ace-sharp.controller.js index 3233757e7c32e..e600773bf680b 100644 --- a/modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.controller.js +++ b/modules/web-console/frontend/app/directives/ui-ace-sharp/ui-ace-sharp.controller.js @@ -15,13 +15,18 @@ * limitations under the License. */ -export default ['$scope', 'GeneratorXml', function($scope, generator) { +const SERVER_CFG = 'ServerConfigurationFactory'; +const CLIENT_CFG = 'ClientConfigurationFactory'; + +export default ['$scope', 'IgniteSharpTransformer', function($scope, generator) { const ctrl = this; delete ctrl.data; // Set default generator ctrl.generator = (cluster) => { - return generator.cluster(cluster, $scope.cfg); + const type = $scope.cfg ? CLIENT_CFG : SERVER_CFG; + + return generator.cluster(cluster, 'config', type, $scope.cfg); }; }]; diff --git a/modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.directive.js b/modules/web-console/frontend/app/directives/ui-ace-sharp/ui-ace-sharp.directive.js similarity index 80% rename from modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.directive.js rename to modules/web-console/frontend/app/directives/ui-ace-sharp/ui-ace-sharp.directive.js index 3bd834f14366a..5d9ad7988b23f 100644 --- a/modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.directive.js +++ b/modules/web-console/frontend/app/directives/ui-ace-sharp/ui-ace-sharp.directive.js @@ -15,10 +15,10 @@ * limitations under the License. */ -import templateUrl from './ui-ace-xml.jade'; -import controller from './ui-ace-xml.controller'; +import templateUrl from './ui-ace-sharp.jade'; +import controller from './ui-ace-sharp.controller'; -export default ['igniteUiAceXml', ['GeneratorXml', (generator) => { +export default ['igniteUiAceSharp', ['IgniteSharpTransformer', (generator) => { const link = (scope, $el, attrs, [ctrl, igniteUiAceTabs, formCtrl, ngModelCtrl]) => { if (formCtrl && ngModelCtrl) formCtrl.$removeControl(ngModelCtrl); @@ -57,7 +57,7 @@ export default ['igniteUiAceXml', ['GeneratorXml', (generator) => { return acc; }, []); - return generator.clusterCaches(caches, null, true, generator.clusterGeneral(cluster)).asString(); + return generator.clusterCaches(cluster, caches, null, true).asString(); }; break; @@ -71,13 +71,12 @@ export default ['igniteUiAceXml', ['GeneratorXml', (generator) => { return acc; }, []); - return generator.igfss(igfss).asString(); + return generator.igfss(igfss, 'cfg').asString(); }; break; case 'cacheStore': - case 'cacheQuery': ctrl.generator = (cache) => { const domains = _.reduce(scope.detail, (acc, domain) => { if (_.includes(cache.domains, domain.value)) @@ -86,20 +85,7 @@ export default ['igniteUiAceXml', ['GeneratorXml', (generator) => { return acc; }, []); - return generator[method](cache, domains).asString(); - }; - - break; - - case 'cacheNodeFilter': - ctrl.generator = (cache) => { - const igfss = _.reduce(scope.detail, (acc, igfs) => { - acc.push(igfs.igfs); - - return acc; - }, []); - - return generator.cacheNodeFilter(cache, igfss).asString(); + return generator.cacheStore(cache, domains).asString(); }; break; @@ -142,6 +128,6 @@ export default ['igniteUiAceXml', ['GeneratorXml', (generator) => { templateUrl, controller, controllerAs: 'ctrl', - require: ['igniteUiAceXml', '?^igniteUiAceTabs', '?^form', '?ngModel'] + require: ['igniteUiAceSharp', '?^igniteUiAceTabs', '?^form', '?ngModel'] }; }]]; diff --git a/modules/web-console/frontend/app/directives/ui-ace-sharp/ui-ace-sharp.jade b/modules/web-console/frontend/app/directives/ui-ace-sharp/ui-ace-sharp.jade new file mode 100644 index 0000000000000..d3f9e446e2d57 --- /dev/null +++ b/modules/web-console/frontend/app/directives/ui-ace-sharp/ui-ace-sharp.jade @@ -0,0 +1,22 @@ +//- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +div(ng-if='ctrl.data' + ignite-ace='{onLoad: onLoad, \ + onChange: onChange, \ + renderOptions: renderOptions, \ + mode: "csharp"}' + ng-model='ctrl.data') diff --git a/modules/web-console/frontend/app/directives/ui-ace-spring/ui-ace-spring.controller.js b/modules/web-console/frontend/app/directives/ui-ace-spring/ui-ace-spring.controller.js new file mode 100644 index 0000000000000..50b71cbf8a69a --- /dev/null +++ b/modules/web-console/frontend/app/directives/ui-ace-spring/ui-ace-spring.controller.js @@ -0,0 +1,88 @@ +/* + * 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. + */ + +export default ['$scope', 'SpringTransformer', function($scope, spring) { + const ctrl = this; + + delete ctrl.data; + + // Setup generator. + switch (ctrl.generator) { + case 'igniteConfiguration': + ctrl.generate = (cluster) => spring.cluster(cluster, ctrl.client === 'true'); + + break; + case 'clusterCaches': + ctrl.generate = (cluster, caches) => { + const clusterCaches = _.reduce(caches, (acc, cache) => { + if (_.includes(cluster.caches, cache.value)) + acc.push(cache.cache); + + return acc; + }, []); + + const cfg = spring.generator.clusterGeneral(cluster); + + spring.generator.clusterCaches(cluster, clusterCaches, null, false, cfg); + + return spring.toSection(cfg); + }; + + break; + case 'cacheStore': + case 'cacheQuery': + ctrl.generate = (cache, domains) => { + const cacheDomains = _.reduce(domains, (acc, domain) => { + if (_.includes(cache.domains, domain.value)) + acc.push(domain.meta); + + return acc; + }, []); + + return spring[ctrl.generator](cache, cacheDomains); + }; + + break; + case 'cacheNodeFilter': + ctrl.generate = (cache, igfss) => { + const cacheIgfss = _.reduce(igfss, (acc, igfs) => { + acc.push(igfs.igfs); + + return acc; + }, []); + + return spring.cacheNodeFilter(cache, cacheIgfss); + }; + + break; + case 'igfss': + ctrl.generate = (cluster, igfss) => { + const clusterIgfss = _.reduce(igfss, (acc, igfs) => { + if (_.includes(cluster.igfss, igfs.value)) + acc.push(igfs.igfs); + + return acc; + }, []); + + return spring.clusterIgfss(clusterIgfss); + }; + + break; + default: + ctrl.generate = (master, detail) => spring[ctrl.generator](master, detail); + } +}]; diff --git a/modules/web-console/frontend/app/directives/ui-ace-spring/ui-ace-spring.directive.js b/modules/web-console/frontend/app/directives/ui-ace-spring/ui-ace-spring.directive.js new file mode 100644 index 0000000000000..42d25b65f3993 --- /dev/null +++ b/modules/web-console/frontend/app/directives/ui-ace-spring/ui-ace-spring.directive.js @@ -0,0 +1,66 @@ +/* + * 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. + */ + +import _ from 'lodash'; + +import templateUrl from './ui-ace-spring.jade'; +import controller from './ui-ace-spring.controller'; + +export default ['igniteUiAceSpring', [() => { + const link = (scope, $el, attrs, [ctrl, igniteUiAceTabs, formCtrl, ngModelCtrl]) => { + if (formCtrl && ngModelCtrl) + formCtrl.$removeControl(ngModelCtrl); + + if (igniteUiAceTabs && igniteUiAceTabs.onLoad) { + scope.onLoad = (editor) => { + igniteUiAceTabs.onLoad(editor); + + // Disable highlight in model switch. + scope.$watch('master', () => editor.attractAttention = false); + }; + } + + if (igniteUiAceTabs && igniteUiAceTabs.onChange) + scope.onChange = igniteUiAceTabs.onChange; + + const noDeepWatch = !(typeof attrs.noDeepWatch !== 'undefined'); + + // Setup watchers. + scope.$watch('master', () => { + ctrl.data = _.isNil(scope.master) ? null : ctrl.generate(scope.master, scope.detail).asString(); + }, noDeepWatch); + }; + + return { + priority: 1, + restrict: 'E', + scope: { + master: '=', + detail: '=' + }, + bindToController: { + data: '=?ngModel', + generator: '@', + client: '@' + }, + link, + templateUrl, + controller, + controllerAs: 'ctrl', + require: ['igniteUiAceSpring', '?^igniteUiAceTabs', '?^form', '?ngModel'] + }; +}]]; diff --git a/modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.jade b/modules/web-console/frontend/app/directives/ui-ace-spring/ui-ace-spring.jade similarity index 100% rename from modules/web-console/frontend/app/directives/ui-ace-xml/ui-ace-xml.jade rename to modules/web-console/frontend/app/directives/ui-ace-spring/ui-ace-spring.jade diff --git a/modules/web-console/frontend/app/directives/ui-grid-settings/ui-grid-settings.jade b/modules/web-console/frontend/app/directives/ui-grid-settings/ui-grid-settings.jade new file mode 100644 index 0000000000000..8f1487e4f4e52 --- /dev/null +++ b/modules/web-console/frontend/app/directives/ui-grid-settings/ui-grid-settings.jade @@ -0,0 +1,33 @@ +//- + 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. + +mixin ui-grid-settings() + .ui-grid-settings + i.fa.fa-bars(data-animation='am-flip-x' bs-dropdown='' aria-haspopup='true' aria-expanded='expanded' data-auto-close='1' data-trigger='click') + ul.select.dropdown-menu(role='menu') + li(ng-repeat='item in paragraph.gridOptions.categories|filter:{selectable:true}') + a(ng-click='paragraph.toggleColumns(item, !item.visible)') + i.fa.fa-check-square-o.pull-left(ng-if='item.visible') + i.fa.fa-square-o.pull-left(ng-if='!item.visible') + span {{::item.name}} + li.divider + li + a(ng-click='paragraph.selectAllColumns()') Select all + li + a(ng-click='paragraph.clearAllColumns()') Clear all + li.divider + li + a(ng-click='$hide()') Close diff --git a/modules/web-console/frontend/app/directives/ui-grid-settings/ui-grid-settings.scss b/modules/web-console/frontend/app/directives/ui-grid-settings/ui-grid-settings.scss new file mode 100644 index 0000000000000..6517a603e137a --- /dev/null +++ b/modules/web-console/frontend/app/directives/ui-grid-settings/ui-grid-settings.scss @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +.ui-grid-settings { + ul.select.dropdown-menu > li > a { + padding-top: 0; + padding-bottom: 0; + } + + ul.select.dropdown-menu > li > a > i { + position: relative; + line-height: 26px; + width: 14px; + margin-left: 0; + color: inherit; + } + + ul.select.dropdown-menu > li > a > span { + line-height: 26px; + padding-left: 5px; + padding-right: 8px; + cursor: pointer; + } +} diff --git a/modules/web-console/frontend/app/modules/form/field/input/select.scss b/modules/web-console/frontend/app/filters/default-name.filter.js similarity index 81% rename from modules/web-console/frontend/app/modules/form/field/input/select.scss rename to modules/web-console/frontend/app/filters/default-name.filter.js index 55bbd58ccc7bd..687ff84c36ad0 100644 --- a/modules/web-console/frontend/app/modules/form/field/input/select.scss +++ b/modules/web-console/frontend/app/filters/default-name.filter.js @@ -15,7 +15,7 @@ * limitations under the License. */ -.select.dropdown-menu.ng-leave { - transition: none !important; /* disable transitions */ - animation: none 0s !important; /* disable keyframe animations */ -} +// Filter that will check name and return `` if needed. +export default [() => { + return (name, html) => _.isEmpty(name) ? (html ? '<default>' : '') : name; +}]; diff --git a/modules/web-console/frontend/app/filters/hasPojo.filter.js b/modules/web-console/frontend/app/filters/hasPojo.filter.js index a17942371ef47..ee50051f73ac9 100644 --- a/modules/web-console/frontend/app/filters/hasPojo.filter.js +++ b/modules/web-console/frontend/app/filters/hasPojo.filter.js @@ -15,4 +15,7 @@ * limitations under the License. */ -export default ['hasPojo', [() => ({caches} = []) => _.find(caches, (cache) => cache.domains && cache.domains.length)]]; +// Filter that return 'true' if caches has at least one domain with 'generatePojo' flag. +export default ['hasPojo', [() => ({caches} = []) => + _.find(caches, (cache) => cache.domains && cache.domains.length && + cache.domains.find((domain) => domain.generatePojo))]]; diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-feedback.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-feedback.jade index bf012db9f7504..ac1f0802683c4 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-feedback.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-feedback.jade @@ -21,9 +21,12 @@ mixin form-field-feedback(name, error, message) i.fa.fa-exclamation-triangle.form-field-feedback( ng-if='!#{__pristine} && #{__error}' + name='{{ #{name} }}' + bs-tooltip data-title=message + ignite-error=error ignite-error-message=message - name='{{ #{name} }}' + ignite-restore-input-focus ) diff --git a/modules/web-console/frontend/app/helpers/jade/mixins.jade b/modules/web-console/frontend/app/helpers/jade/mixins.jade index 4979c1234bc08..9d46883e2074d 100644 --- a/modules/web-console/frontend/app/helpers/jade/mixins.jade +++ b/modules/web-console/frontend/app/helpers/jade/mixins.jade @@ -97,28 +97,30 @@ mixin ipaddress-port-range-feedback(name) mixin uuid-feedback(name) +form-field-feedback(name, 'uuid', 'Invalid node ID!') -//- Mixin for checkbox. -mixin checkbox(lbl, model, name, tip) - +form-field-checkbox(lbl, model, name, false, false, tip) - //- Function that convert enabled state to corresponding disabled state. -var enabledToDisabled = function (enabled) { - return (enabled === false || enabled === true) ? !enabled : '!(' + enabled + ')'; -} +//- Mixin for checkbox. +mixin checkbox(lbl, model, name, tip) + +form-field-checkbox(lbl, model, name, false, false, tip) + //- Mixin for checkbox with enabled condition. mixin checkbox-enabled(lbl, model, name, enabled, tip) +form-field-checkbox(lbl, model, name, enabledToDisabled(enabled), false, tip) -//- Mixin for java name field with enabled condition. -mixin java-class(lbl, model, name, enabled, required, tip) +//- Mixin for Java class name field with auto focus condition. +mixin java-class-autofocus-placholder(lbl, model, name, enabled, required, autofocus, placeholder, tip, validationActive) -var errLbl = lbl.substring(0, lbl.length - 1) - +ignite-form-field-text(lbl, model, name, enabledToDisabled(enabled), required, 'Enter fully qualified class name', tip)( + +ignite-form-field-text(lbl, model, name, enabledToDisabled(enabled), required, placeholder, tip)( data-java-identifier='true' data-java-package-specified='true' data-java-keywords='true' data-java-built-in-class='true' + data-ignite-form-field-input-autofocus=autofocus + data-validation-active=validationActive ? '{{ #{validationActive} }}' : '"always"' ) if block block @@ -128,14 +130,27 @@ mixin java-class(lbl, model, name, enabled, required, tip) +form-field-feedback(name, 'javaPackageSpecified', errLbl + ' does not have package specified!') +form-field-feedback(name, 'javaIdentifier', errLbl + ' is invalid Java identifier!') +//- Mixin for Java class name field with auto focus condition. +mixin java-class-autofocus(lbl, model, name, enabled, required, autofocus, tip, validationActive) + +java-class-autofocus-placholder(lbl, model, name, enabled, required, autofocus, 'Enter fully qualified class name', tip, validationActive) + if block + block + +//- Mixin for Java class name field. +mixin java-class(lbl, model, name, enabled, required, tip, validationActive) + +java-class-autofocus(lbl, model, name, enabled, required, 'false', tip, validationActive) + if block + block + //- Mixin for text field with enabled condition with options. -mixin java-class-typeahead(lbl, model, name, options, enabled, required, placeholder, tip) +mixin java-class-typeahead(lbl, model, name, options, enabled, required, placeholder, tip, validationActive) -var errLbl = lbl.substring(0, lbl.length - 1) +form-field-datalist(lbl, model, name, enabledToDisabled(enabled), required, placeholder, options, tip)( data-java-identifier='true' data-java-package-specified='allow-built-in' data-java-keywords='true' + data-validation-active=validationActive ? '{{ #{validationActive} }}' : '"always"' ) +form-field-feedback(name, 'javaKeywords', errLbl + ' could not contains reserved Java keyword!') +form-field-feedback(name, 'javaPackageSpecified', errLbl + ' does not have package specified!') @@ -234,22 +249,21 @@ mixin table-text-field(name, model, items, valid, save, placeholder, newItem) -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false' -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';' - if block - block + div(ignite-on-focus-out=onBlur) + if block + block + + .input-tip + +ignite-form-field-input(name, model, false, 'true', placeholder)( + data-ignite-unique=items + data-ignite-form-field-input-autofocus='true' - .input-tip - +ignite-form-field-input(name, model, false, 'true', placeholder)( - data-ignite-unique=items - data-ignite-form-field-input-autofocus='true' - ng-blur=onBlur - ignite-on-enter=onEnter - ignite-on-escape=onEscape - ) + ignite-on-enter=onEnter + ignite-on-escape=onEscape + ) //- Mixin for table java class field. mixin table-java-class-field(lbl, name, model, items, valid, save, newItem) - -var errLbl = lbl.substring(0, lbl.length - 1) - -var resetOnEnter = newItem ? '(stopblur = true) && (group.add = [{}])' : '(field.edit = false)' -var onEnter = valid + ' && (' + save + '); ' + valid + ' && ' + resetOnEnter + ';' @@ -258,31 +272,28 @@ mixin table-java-class-field(lbl, name, model, items, valid, save, newItem) -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false' -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';' - if block - block - - +form-field-feedback(name, 'javaBuiltInClass', errLbl + ' should not be the Java built-in class!') - +form-field-feedback(name, 'javaKeywords', errLbl + ' could not contains reserved Java keyword!') - +form-field-feedback(name, 'javaPackageSpecified', errLbl + ' does not have package specified!') - +form-field-feedback(name, 'javaIdentifier', errLbl + ' is invalid Java identifier!') + div(ignite-on-focus-out=onBlur) + +form-field-feedback(name, 'javaBuiltInClass', lbl + ' should not be the Java built-in class!') + +form-field-feedback(name, 'javaKeywords', lbl + ' could not contains reserved Java keyword!') + +form-field-feedback(name, 'javaPackageSpecified', lbl + ' does not have package specified!') + +form-field-feedback(name, 'javaIdentifier', lbl + ' is invalid Java identifier!') - if block - block + if block + block - .input-tip - +ignite-form-field-input(name, model, false, 'true', 'Enter fully qualified class name')( - data-java-identifier='true' - data-java-package-specified='true' - data-java-keywords='true' - data-java-built-in-class='true' + .input-tip + +ignite-form-field-input(name, model, false, 'true', 'Enter fully qualified class name')( + data-java-identifier='true' + data-java-package-specified='true' + data-java-keywords='true' + data-java-built-in-class='true' - data-ignite-unique=items - data-ignite-form-field-input-autofocus='true' + data-ignite-unique=items + data-ignite-form-field-input-autofocus='true' - ng-blur=onBlur - ignite-on-enter=onEnter - ignite-on-escape=onEscape - ) + ignite-on-enter=onEnter + ignite-on-escape=onEscape + ) //- Mixin for table java package field. mixin table-java-package-field(name, model, items, valid, save, newItem) @@ -294,24 +305,24 @@ mixin table-java-package-field(name, model, items, valid, save, newItem) -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false' -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';' - +form-field-feedback(name, 'javaKeywords', 'Package name could not contains reserved Java keyword!') - +form-field-feedback(name, 'javaPackageName', 'Package name is invalid!') + div(ignite-on-focus-out=onBlur) + +form-field-feedback(name, 'javaKeywords', 'Package name could not contains reserved Java keyword!') + +form-field-feedback(name, 'javaPackageName', 'Package name is invalid!') - if block - block + if block + block - .input-tip - +ignite-form-field-input(name, model, false, 'true', 'Enter package name')( - data-java-keywords='true' - data-java-package-name='package-only' + .input-tip + +ignite-form-field-input(name, model, false, 'true', 'Enter package name')( + data-java-keywords='true' + data-java-package-name='package-only' - data-ignite-unique=items - data-ignite-form-field-input-autofocus='true' + data-ignite-unique=items + data-ignite-form-field-input-autofocus='true' - ng-blur=onBlur - ignite-on-enter=onEnter - ignite-on-escape=onEscape - ) + ignite-on-enter=onEnter + ignite-on-escape=onEscape + ) //- Mixin for table address field. @@ -324,24 +335,26 @@ mixin table-address-field(name, model, items, valid, save, newItem, portRange) -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false' -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';' - +ipaddress-feedback(name) - +ipaddress-port-feedback(name) - +ipaddress-port-range-feedback(name) - - if block - block - - .input-tip - +ignite-form-field-input(name, model, false, 'true', 'IP address:port')( - data-ipaddress='true' - data-ipaddress-with-port='true' - data-ipaddress-with-port-range=portRange ? 'true' : null - data-ignite-unique=items - data-ignite-form-field-input-autofocus='true' - ng-blur=onBlur - ignite-on-enter=onEnter - ignite-on-escape=onEscape - ) + div(ignite-on-focus-out=onBlur) + +ipaddress-feedback(name) + +ipaddress-port-feedback(name) + +ipaddress-port-range-feedback(name) + +form-field-feedback(name, 'required', 'IP address:port could not be empty!') + + if block + block + + .input-tip + +ignite-form-field-input(name, model, false, 'true', 'IP address:port')( + data-ipaddress='true' + data-ipaddress-with-port='true' + data-ipaddress-with-port-range=portRange ? 'true' : null + data-ignite-unique=items + data-ignite-form-field-input-autofocus='true' + + ignite-on-enter=onEnter + ignite-on-escape=onEscape + ) //- Mixin for table UUID field. mixin table-uuid-field(name, model, items, valid, save, newItem) @@ -353,18 +366,19 @@ mixin table-uuid-field(name, model, items, valid, save, newItem) -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false' -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';' - if block - block + div(ignite-on-focus-out=onBlur) + if block + block + + .input-tip + +ignite-form-field-input(name, model, false, 'true', 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx')( + data-uuid='true' + data-ignite-unique=items + data-ignite-form-field-input-autofocus='true' - .input-tip - +ignite-form-field-input(name, model, false, 'true', 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx')( - data-uuid='true' - data-ignite-unique=items - data-ignite-form-field-input-autofocus='true' - ng-blur=onBlur - ignite-on-enter=onEnter - ignite-on-escape=onEscape - ) + ignite-on-enter=onEnter + ignite-on-escape=onEscape + ) //- Mixin for table save button. "||" used instead of "&&" to workaround escaping of "&&" to "&&" @@ -424,7 +438,7 @@ mixin evictionPolicy(model, name, enabled, required, tip) a.customize(ng-hide='__.expanded' ng-click='__.expanded = true') Show settings .panel-details(ng-if='__.expanded') .details-row - +number('Batch size', policy + '.batchSize', name + '+ "batchSize"', enabled, '1', '0', + +number('Batch size', policy + '.batchSize', name + '+ "batchSize"', enabled, '1', '1', 'Number of entries to remove on shrink') .details-row +number('Max memory size', policy + '.maxMemorySize', name + '+ "maxMemorySize"', enabled, '0', '0', @@ -443,18 +457,35 @@ mixin caches(model, tip) +dropdown-multiple('Caches:' + ' (add)', model + '.caches', '"caches"', true, 'Choose caches', 'No caches configured', 'caches', tip) +//- Mixin for XML, Java, .Net preview. +mixin preview(master, generator, detail) + ignite-ui-ace-tabs + .preview-panel(ng-init='mode = "spring"') + .preview-legend + a(ng-class='{active: mode === "spring"}' ng-click='mode = "spring"') Spring + a(ng-class='{active: mode === "java"}' ng-click='mode = "java"') Java + a(ng-class='{active: mode === "csharp"}' ng-click='mode = "csharp"') C# + //a(ng-class='{active: mode === "app.config"}' ng-click='mode = "app.config"') app.config + .preview-content(ng-switch='mode') + ignite-ui-ace-spring(ng-switch-when="spring" data-master=master data-generator=generator ng-model='$parent.data' data-detail=detail) + ignite-ui-ace-java(ng-switch-when="java" data-master=master data-generator=generator ng-model='$parent.data' data-detail=detail) + ignite-ui-ace-sharp(ng-switch-when="csharp" data-master=master data-generator=generator ng-model='$parent.data' data-detail=detail) + .preview-content-empty(ng-if='!data') + label All Defaults + //- Mixin for XML and Java preview. mixin preview-xml-java(master, generator, detail) ignite-ui-ace-tabs - .preview-panel + .preview-panel(ng-init='mode = "spring"') .preview-legend - a(ng-class='{active: !mode, inactive: mode}' ng-click='mode = false') XML - |   - a(ng-class='{active: mode, inactive: !mode}' ng-click='mode = true') Java - .preview-content(ng-if='mode') - ignite-ui-ace-java(data-master=master data-generator=generator ng-model='$parent.data' data-detail=detail) - .preview-content(ng-if='!mode') - ignite-ui-ace-xml(data-master=master data-generator=generator ng-model='$parent.data' data-detail=detail) + a(ng-class='{active: mode === "spring"}' ng-click='mode = "spring"') Spring + a(ng-class='{active: mode === "java"}' ng-click='mode = "java"') Java + //a(ng-class='{active: mode === "csharp"}' ng-click='mode = "csharp"') C# + //a(ng-class='{active: mode === "app.config"}' ng-click='mode = "app.config"') app.config + .preview-content(ng-switch='mode') + ignite-ui-ace-spring(ng-switch-when="spring" data-master=master data-generator=generator ng-model='$parent.data' data-detail=detail) + ignite-ui-ace-java(ng-switch-when="java" data-master=master data-generator=generator ng-model='$parent.data' data-detail=detail) + //ignite-ui-ace-sharp(ng-switch-when="csharp" data-master=master data-generator=generator ng-model='$parent.data' data-detail=detail) .preview-content-empty(ng-if='!data') label All Defaults @@ -503,7 +534,7 @@ mixin table-pair-edit(tbl, prefix, keyPlaceholder, valPlaceholder, keyJavaBuiltI //- Mixin for DB dialect. mixin dialect(lbl, model, name, required, tipTitle, genericDialectName, placeholder) - +dropdown(lbl, model, name, required, placeholder, '[\ + +dropdown-required(lbl, model, name, 'true', required, placeholder, '[\ {value: "Generic", label: "' + genericDialectName + '"},\ {value: "Oracle", label: "Oracle"},\ {value: "DB2", label: "IBM DB2"},\ diff --git a/modules/web-console/frontend/app/modules/agent/agent.module.js b/modules/web-console/frontend/app/modules/agent/agent.module.js index 22ced13600f67..d6fc86311cf24 100644 --- a/modules/web-console/frontend/app/modules/agent/agent.module.js +++ b/modules/web-console/frontend/app/modules/agent/agent.module.js @@ -277,12 +277,13 @@ class IgniteAgentMonitor { * @param {String} nid Node id. * @param {String} cacheName Cache name. * @param {String} [query] Query if null then scan query. + * @param {Boolean} nonCollocatedJoins Flag whether to execute non collocated joins. * @param {Boolean} local Flag whether to execute query locally. * @param {int} pageSize * @returns {Promise} */ - query(nid, cacheName, query, local, pageSize) { - return this._rest('node:query', nid, maskNull(cacheName), maskNull(query), local, pageSize) + query(nid, cacheName, query, nonCollocatedJoins, local, pageSize) { + return this._rest('node:query', nid, maskNull(cacheName), maskNull(query), nonCollocatedJoins, local, pageSize) .then(({result}) => { if (_.isEmpty(result.key)) return result.value; @@ -295,11 +296,12 @@ class IgniteAgentMonitor { * @param {String} nid Node id. * @param {String} cacheName Cache name. * @param {String} [query] Query if null then scan query. + * @param {Boolean} nonCollocatedJoins Flag whether to execute non collocated joins. * @param {Boolean} local Flag whether to execute query locally. * @returns {Promise} */ - queryGetAll(nid, cacheName, query, local) { - return this._rest('node:query:getAll', nid, maskNull(cacheName), maskNull(query), local); + queryGetAll(nid, cacheName, query, nonCollocatedJoins, local) { + return this._rest('node:query:getAll', nid, maskNull(cacheName), maskNull(query), nonCollocatedJoins, local); } /** diff --git a/modules/web-console/frontend/app/modules/configuration/Version.service.js b/modules/web-console/frontend/app/modules/configuration/Version.service.js index c36a98b921e5d..06efdda42b83b 100644 --- a/modules/web-console/frontend/app/modules/configuration/Version.service.js +++ b/modules/web-console/frontend/app/modules/configuration/Version.service.js @@ -16,10 +16,80 @@ */ /** - * Utility service for version. + * Utility service for version parsing and comparing */ -export default () => { - return { - ignite: '1.7.0' - }; -}; +const VERSION_MATCHER = /(\d+)\.(\d+)\.(\d+)([-.]([^0123456789][^-]+)(-SNAPSHOT)?)?(-(\d+))?(-([\da-f]+))?/i; + +const numberComparator = (a, b) => a > b ? 1 : a < b ? -1 : 0; + +export default class Version { + /** + * Tries to parse product version from it's string representation. + * + * @param {String} ver - String representation of version. + * @returns {{major: Number, minor: Number, maintenance: Number, stage: String, revTs: Number, revHash: String}} - Object that contains product version fields. + */ + parse(ver) { + // Development or built from source ZIP. + ver = ver.replace(/(-DEV|-n\/a)$/i, ''); + + const [, major, minor, maintenance, stage, ...chunks] = ver.match(VERSION_MATCHER); + + return { + major: parseInt(major, 10), + minor: parseInt(minor, 10), + maintenance: parseInt(maintenance, 10), + stage: (stage || '').substring(1), + revTs: chunks[2] ? parseInt(chunks[3], 10) : 0, + revHash: chunks[4] ? chunks[5] : null + }; + } + + /** + * Compare to version. + * @param a {String} first compared version. + * @param b {String} second compared version. + * @returns {Number} 1 if a > b, 0 if versions equals, -1 if a < b + */ + compare(a, b) { + const pa = this.parse(a); + const pb = this.parse(b); + + let res = numberComparator(pa.major, pb.major); + + if (res !== 0) + return res; + + res = numberComparator(pa.minor, pb.minor); + + if (res !== 0) + return res; + + res = numberComparator(pa.maintenance, pb.maintenance); + + if (res !== 0) + return res; + + return numberComparator(pa.revTs, pb.maintenance); + } + + /** + * Return current product version. + * @returns {{ignite: string}} + */ + productVersion() { + return { + ignite: '1.7.0' + }; + } + + /** + * Check if node version is newer or same + * @param {String} nodeVer + * @param {String} sinceVer + * @returns {Boolean} + */ + since(nodeVer, sinceVer) { + return this.compare(nodeVer, sinceVer) >= 0; + } +} diff --git a/modules/web-console/frontend/app/modules/configuration/configuration.module.js b/modules/web-console/frontend/app/modules/configuration/configuration.module.js index dc6fa2fa31a95..27f7befbeb4f5 100644 --- a/modules/web-console/frontend/app/modules/configuration/configuration.module.js +++ b/modules/web-console/frontend/app/modules/configuration/configuration.module.js @@ -21,10 +21,22 @@ import igniteEventGroups from './EventGroups.provider'; import igniteSidebar from './Sidebar.provider'; import Version from './Version.service'; -import GeneratorXml from './generator/Xml.service'; -import GeneratorJava from './generator/Java.service'; +import clusterDefaults from './generator/defaults/cluster.provider'; +import clusterPlatformDefaults from './generator/defaults/cluster.platform.provider'; +import cacheDefaults from './generator/defaults/cache.provider'; +import cachePlatformDefaults from './generator/defaults/cache.platform.provider'; +import igfsDefaults from './generator/defaults/igfs.provider'; + +import ConfigurationGenerator from './generator/ConfigurationGenerator'; +import PlatformGenerator from './generator/PlatformGenerator'; + +import SpringTransformer from './generator/SpringTransformer.service'; +import JavaTransformer from './generator/JavaTransformer.service'; +import SharpTransformer from './generator/SharpTransformer.service'; import GeneratorDocker from './generator/Docker.service'; import GeneratorPom from './generator/Pom.service'; +import GeneratorProperties from './generator/Properties.service'; +import GeneratorReadme from './generator/Readme.service'; import igniteSidebarDirective from './sidebar.directive'; @@ -33,11 +45,21 @@ angular .module('ignite-console.configuration', [ ]) +.provider('igniteClusterDefaults', clusterDefaults) +.provider('igniteClusterPlatformDefaults', clusterPlatformDefaults) +.provider('igniteCacheDefaults', cacheDefaults) +.provider('igniteCachePlatformDefaults', cachePlatformDefaults) +.provider('igniteIgfsDefaults', igfsDefaults) .provider(...igniteEventGroups) .provider(...igniteSidebar) .directive(...igniteSidebarDirective) .service('IgniteVersion', Version) -.service(...GeneratorXml) -.service(...GeneratorJava) +.service('IgniteConfigurationGenerator', ConfigurationGenerator) +.service('IgnitePlatformGenerator', PlatformGenerator) +.service('SpringTransformer', SpringTransformer) +.service('JavaTransformer', JavaTransformer) +.service('IgniteSharpTransformer', SharpTransformer) +.service('IgnitePropertiesGenerator', GeneratorProperties) +.service('IgniteReadmeGenerator', GeneratorReadme) .service(...GeneratorDocker) .service(...GeneratorPom); diff --git a/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js b/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js new file mode 100644 index 0000000000000..6244a531ea85b --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js @@ -0,0 +1,341 @@ +/* + * 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. + */ + +import StringBuilder from './StringBuilder'; + +export default class AbstractTransformer { + // Append comment with time stamp. + static mainComment(sb, ...lines) { + lines.push(sb.generatedBy()); + + return this.commentBlock(sb, ...lines); + } + + // Append line before and after property. + static _emptyLineIfNeeded(sb, props, curIdx) { + if (curIdx === props.length - 1) + return; + + const cur = props[curIdx]; + + // Empty line after. + if (_.includes(['MAP', 'COLLECTION', 'ARRAY'], cur.clsName) || (cur.clsName === 'BEAN' && cur.value.isComplex())) + return sb.emptyLine(); + + const next = props[curIdx + 1]; + + // Empty line before. + if (_.includes(['MAP', 'COLLECTION', 'ARRAY'], next.clsName) || (next.clsName === 'BEAN' && next.value.isComplex())) + return sb.emptyLine(); + } + + // Generate general section. + static clusterGeneral(cluster) { + return this.toSection(this.generator.clusterGeneral(cluster)); + } + + // Generate atomics group. + static clusterAtomics(atomics) { + return this.toSection(this.generator.clusterAtomics(atomics)); + } + + // Generate binary group. + static clusterBinary(binary) { + return this.toSection(this.generator.clusterBinary(binary)); + } + + // Generate cache key configurations. + static clusterCacheKeyConfiguration(keyCfgs) { + return this.toSection(this.generator.clusterCacheKeyConfiguration(keyCfgs)); + } + + // Generate collision group. + static clusterCollision(collision) { + return this.toSection(this.generator.clusterCollision(collision)); + } + + // Generate communication group. + static clusterCommunication(cluster) { + return this.toSection(this.generator.clusterCommunication(cluster)); + } + + // Generate REST access configuration. + static clusterConnector(connector) { + return this.toSection(this.generator.clusterConnector(connector)); + } + + // Generate deployment group. + static clusterDeployment(cluster) { + return this.toSection(this.generator.clusterDeployment(cluster)); + } + + // Generate discovery group. + static clusterDiscovery(disco) { + return this.toSection(this.generator.clusterDiscovery(disco)); + } + + // Generate events group. + static clusterEvents(cluster) { + return this.toSection(this.generator.clusterEvents(cluster)); + } + + // Generate failover group. + static clusterFailover(cluster) { + return this.toSection(this.generator.clusterFailover(cluster)); + } + + // Generate cluster IGFSs group. + static clusterIgfss(igfss) { + return this.toSection(this.generator.clusterIgfss(igfss)); + } + + // Generate load balancing SPI group. + static clusterLoadBalancing(cluster) { + return this.toSection(this.generator.clusterLoadBalancing(cluster)); + } + + // Generate logger group. + static clusterLogger(cluster) { + return this.toSection(this.generator.clusterLogger(cluster)); + } + + // Generate marshaller group. + static clusterMarshaller(cluster) { + return this.toSection(this.generator.clusterMarshaller(cluster)); + } + + // Generate metrics group. + static clusterMetrics(cluster) { + return this.toSection(this.generator.clusterMetrics(cluster)); + } + + // Generate ODBC group. + static clusterODBC(odbc) { + return this.toSection(this.generator.clusterODBC(odbc)); + } + + // Generate ssl group. + static clusterSsl(cluster) { + return this.toSection(this.generator.clusterSsl(cluster)); + } + + // Generate swap group. + static clusterSwap(cluster) { + return this.toSection(this.generator.clusterSwap(cluster)); + } + + // Generate time group. + static clusterTime(cluster) { + return this.toSection(this.generator.clusterTime(cluster)); + } + + // Generate thread pools group. + static clusterPools(cluster) { + return this.toSection(this.generator.clusterPools(cluster)); + } + + // Generate transactions group. + static clusterTransactions(transactionConfiguration) { + return this.toSection(this.generator.clusterTransactions(transactionConfiguration)); + } + + // Generate user attributes group. + static clusterUserAttributes(cluster) { + return this.toSection(this.generator.clusterUserAttributes(cluster)); + } + + // Generate IGFS general group. + static igfsGeneral(igfs) { + return this.toSection(this.generator.igfsGeneral(igfs)); + } + + // Generate IGFS secondary file system group. + static igfsSecondFS(igfs) { + return this.toSection(this.generator.igfsSecondFS(igfs)); + } + + // Generate IGFS IPC group. + static igfsIPC(igfs) { + return this.toSection(this.generator.igfsIPC(igfs)); + } + + // Generate IGFS fragmentizer group. + static igfsFragmentizer(igfs) { + return this.toSection(this.generator.igfsFragmentizer(igfs)); + } + + // Generate IGFS Dual mode group. + static igfsDualMode(igfs) { + return this.toSection(this.generator.igfsDualMode(igfs)); + } + + // Generate IGFS miscellaneous group. + static igfsMisc(igfs) { + return this.toSection(this.generator.igfsMisc(igfs)); + } + + // Generate cache general group. + static cacheGeneral(cache) { + return this.toSection(this.generator.cacheGeneral(cache)); + } + + // Generate cache memory group. + static cacheMemory(cache) { + return this.toSection(this.generator.cacheMemory(cache)); + } + + // Generate cache queries & Indexing group. + static cacheQuery(cache, domains) { + return this.toSection(this.generator.cacheQuery(cache, domains)); + } + + // Generate cache store group. + static cacheStore(cache, domains) { + return this.toSection(this.generator.cacheStore(cache, domains)); + } + + // Generate cache concurrency control group. + static cacheConcurrency(cache) { + return this.toSection(this.generator.cacheConcurrency(cache)); + } + + // Generate cache node filter group. + static cacheNodeFilter(cache, igfss) { + return this.toSection(this.generator.cacheNodeFilter(cache, igfss)); + } + + // Generate cache rebalance group. + static cacheRebalance(cache) { + return this.toSection(this.generator.cacheRebalance(cache)); + } + + // Generate server near cache group. + static cacheNearServer(cache) { + return this.toSection(this.generator.cacheNearServer(cache)); + } + + // Generate client near cache group. + static cacheNearClient(cache) { + return this.toSection(this.generator.cacheNearClient(cache)); + } + + // Generate cache statistics group. + static cacheStatistics(cache) { + return this.toSection(this.generator.cacheStatistics(cache)); + } + + // Generate caches configs. + static clusterCaches(cluster, caches, igfss, client) { + return this.toSection(this.generator.clusterCaches(cluster, caches, igfss, client)); + } + + // Generate caches configs. + static clusterCheckpoint(cluster, caches) { + return this.toSection(this.generator.clusterCheckpoint(cluster, caches)); + } + + // Generate domain model for general group. + static domainModelGeneral(domain) { + return this.toSection(this.generator.domainModelGeneral(domain)); + } + + // Generate domain model for query group. + static domainModelQuery(domain) { + return this.toSection(this.generator.domainModelQuery(domain)); + } + + // Generate domain model for store group. + static domainStore(domain) { + return this.toSection(this.generator.domainStore(domain)); + } + + /** + * Check if configuration contains properties. + * + * @param {Bean} bean + * @returns {Boolean} + */ + static hasProperties(bean) { + const searchProps = (prop) => { + switch (prop.clsName) { + case 'BEAN': + if (this.hasProperties(prop.value)) + return true; + + break; + case 'ARRAY': + case 'COLLECTION': + if (_.find(prop.items, (item) => this.hasProperties(item))) + return true; + + break; + case 'DATA_SOURCE': + case 'PROPERTY': + case 'PROPERTY_CHAR': + return true; + default: + } + + return false; + }; + + return _.isObject(bean) && (!!_.find(bean.arguments, searchProps) || !!_.find(bean.properties, searchProps)); + } + + /** + * Collect datasource beans. + * + * @param {Bean} bean + */ + static collectDataSources(bean) { + const dataSources = _.reduce(bean.properties, (acc, prop) => { + switch (prop.clsName.toUpperCase()) { + case 'ARRAY': + if (this._isBean(prop.typeClsName)) + _.forEach(prop.items, (item) => acc.push(...this.collectDataSources(item))); + + break; + case 'BEAN': + acc.push(...this.collectDataSources(prop.value)); + + break; + case 'DATA_SOURCE': + acc.push(prop.value); + + break; + default: + } + + return acc; + }, []); + + return _.uniqBy(dataSources, (ds) => ds.id); + } + + /** + * Transform to section. + * + * @param cfg + * @param sb + * @return {StringBuilder} + */ + static toSection(cfg, sb = new StringBuilder()) { + this._setProperties(sb, cfg); + + return sb; + } +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Beans.js b/modules/web-console/frontend/app/modules/configuration/generator/Beans.js new file mode 100644 index 0000000000000..546f38b086651 --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/Beans.js @@ -0,0 +1,379 @@ +/* + * 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. + */ + +import _ from 'lodash'; + +export class EmptyBean { + /** + * @param {String} clsName + */ + constructor(clsName) { + this.properties = []; + this.arguments = []; + + this.clsName = clsName; + } + + isEmpty() { + return false; + } + + nonEmpty() { + return !this.isEmpty(); + } + + isComplex() { + return _.nonEmpty(this.properties) || !!_.find(this.arguments, (arg) => arg.clsName === 'MAP'); + } + + nonComplex() { + return !this.isComplex(); + } + + findProperty(name) { + return _.find(this.properties, {name}); + } +} + +export class Bean extends EmptyBean { + /** + * @param {String} clsName + * @param {String} id + * @param {Object} src + * @param {Object} dflts + */ + constructor(clsName, id, src, dflts = {}) { + super(clsName); + + this.id = id; + + this.src = src; + this.dflts = dflts; + } + + factoryMethod(name) { + this.factoryMtd = name; + + return this; + } + + /** + * @param acc + * @param clsName + * @param model + * @param name + * @param {Function} nonEmpty Non empty function. + * @param {Function} mapper Mapper function. + * @returns {Bean} + * @private + */ + _property(acc, clsName, model, name, nonEmpty = () => true, mapper = (val) => val) { + if (!this.src) + return this; + + const value = mapper(_.get(this.src, model)); + + if (nonEmpty(value) && value !== _.get(this.dflts, model)) + acc.push({clsName, name, value}); + + return this; + } + + isEmpty() { + return _.isEmpty(this.arguments) && _.isEmpty(this.properties); + } + + constructorArgument(clsName, value) { + this.arguments.push({clsName, value}); + + return this; + } + + stringConstructorArgument(model) { + return this._property(this.arguments, 'java.lang.String', model, null, _.nonEmpty); + } + + intConstructorArgument(model) { + return this._property(this.arguments, 'int', model, null, _.nonNil); + } + + classConstructorArgument(model) { + return this._property(this.arguments, 'java.lang.Class', model, null, _.nonEmpty); + } + + pathConstructorArgument(model) { + return this._property(this.arguments, 'PATH', model, null, _.nonEmpty); + } + + constantConstructorArgument(model) { + if (!this.src) + return this; + + const value = _.get(this.src, model); + const dflt = _.get(this.dflts, model); + + if (_.nonNil(value) && _.nonNil(dflt) && value !== dflt.value) + this.arguments.push({clsName: dflt.clsName, constant: true, value}); + + return this; + } + + propertyConstructorArgument(value, hint = '') { + this.arguments.push({clsName: 'PROPERTY', value, hint}); + + return this; + } + + /** + * @param {String} id + * @param {EmptyBean|Bean} value + * @returns {Bean} + */ + beanConstructorArgument(id, value) { + this.arguments.push({clsName: 'BEAN', id, value}); + + return this; + } + + /** + * @param {String} id + * @param {String} model + * @param {Array.} entries + * @returns {Bean} + */ + mapConstructorArgument(id, model, entries) { + if (!this.src) + return this; + + const dflt = _.get(this.dflts, model); + + if (_.nonEmpty(entries) && _.nonNil(dflt) && entries !== dflt.entries) { + this.arguments.push({ + clsName: 'MAP', + id, + keyClsName: dflt.keyClsName, + keyField: dflt.keyField || 'name', + valClsName: dflt.valClsName, + valField: dflt.valField || 'value', + entries + }); + } + + return this; + } + + valueOf(path) { + return _.get(this.src, path) || _.get(this.dflts, path + '.value') || _.get(this.dflts, path); + } + + includes(...paths) { + return this.src && _.every(paths, (path) => { + const value = _.get(this.src, path); + const dflt = _.get(this.dflts, path); + + return _.nonNil(value) && value !== dflt; + }); + } + + prop(clsName, name, value) { + this.properties.push({clsName, name, value}); + } + + boolProperty(model, name = model) { + return this._property(this.properties, 'boolean', model, name, _.nonNil); + } + + byteProperty(model, name = model) { + return this._property(this.properties, 'byte', model, name, _.nonNil); + } + + intProperty(model, name = model) { + return this._property(this.properties, 'int', model, name, _.nonNil); + } + + floatProperty(model, name = model) { + return this._property(this.properties, 'float', model, name, _.nonNil); + } + + property(name, value, hint) { + this.properties.push({clsName: 'PROPERTY', name, value, hint}); + + return this; + } + + propertyChar(name, value, hint) { + this.properties.push({clsName: 'PROPERTY_CHAR', name, value, hint}); + + return this; + } + + stringProperty(model, name = model, mapper) { + return this._property(this.properties, 'java.lang.String', model, name, _.nonEmpty, mapper); + } + + pathProperty(model, name = model) { + return this._property(this.properties, 'PATH', model, name, _.nonEmpty); + } + + classProperty(model, name = model) { + return this._property(this.properties, 'java.lang.Class', model, name, _.nonEmpty); + } + + enumProperty(model, name = model) { + if (!this.src) + return this; + + const value = _.get(this.src, model); + const dflt = _.get(this.dflts, model); + + if (_.nonNil(value) && _.nonNil(dflt) && value !== dflt.value) + this.properties.push({clsName: dflt.clsName, name, value: dflt.mapper ? dflt.mapper(value) : value}); + + return this; + } + + emptyBeanProperty(model, name = model) { + if (!this.src) + return this; + + const cls = _.get(this.src, model); + const dflt = _.get(this.dflts, model); + + if (_.nonEmpty(cls) && cls !== dflt) + this.properties.push({clsName: 'BEAN', name, value: new EmptyBean(cls)}); + + return this; + } + + /** + * @param {String} name + * @param {EmptyBean|Bean} value + * @returns {Bean} + */ + beanProperty(name, value) { + this.properties.push({clsName: 'BEAN', name, value}); + + return this; + } + + /** + * @param {String} id + * @param {String} name + * @param {Array} items + * @param {String} typeClsName + * @returns {Bean} + */ + arrayProperty(id, name, items, typeClsName = 'java.lang.String') { + if (items && items.length) + this.properties.push({clsName: 'ARRAY', id, name, items, typeClsName}); + + return this; + } + + /** + * @param {String} id + * @param {String} name + * @param {Array} items + * @param {String} typeClsName + * @returns {Bean} + */ + varArgProperty(id, name, items, typeClsName = 'java.lang.String') { + if (items && items.length) + this.properties.push({clsName: 'ARRAY', id, name, items, typeClsName, varArg: true}); + + return this; + } + + /** + * @param {String} id + * @param {String} name + * @param {Array} items + * @param {String} typeClsName + * @param {String} implClsName + * @returns {Bean} + */ + collectionProperty(id, name, items, typeClsName = 'java.lang.String', implClsName = 'java.util.ArrayList') { + if (items && items.length) + this.properties.push({id, name, items, clsName: 'COLLECTION', typeClsName, implClsName}); + + return this; + } + + /** + * @param {String} id + * @param {String} model + * @param {String} [name] + * @param {Boolean} [ordered] + * @returns {Bean} + */ + mapProperty(id, model, name = model, ordered = false) { + if (!this.src) + return this; + + const entries = _.isString(model) ? _.get(this.src, model) : model; + const dflt = _.isString(model) ? _.get(this.dflts, model) : _.get(this.dflts, name); + + if (_.nonEmpty(entries) && _.nonNil(dflt) && entries !== dflt.entries) { + this.properties.push({ + clsName: 'MAP', + id, + name, + ordered, + keyClsName: dflt.keyClsName, + keyField: dflt.keyField || 'name', + valClsName: dflt.valClsName, + valField: dflt.valField || 'value', + entries + }); + } + + return this; + } + + propsProperty(id, model, name = model) { + if (!this.src) + return this; + + const entries = _.get(this.src, model); + + if (_.nonEmpty(entries)) + this.properties.push({clsName: 'java.util.Properties', id, name, entries}); + + return this; + } + + /** + * @param {String} id + * @param {String} name + * @param {EmptyBean|Bean} value + */ + dataSource(id, name, value) { + if (value) + this.properties.push({clsName: 'DATA_SOURCE', id, name, value}); + + return this; + } + + /** + * @param {String} id + * @param {String} name + * @param {Array} eventTypes + */ + eventTypes(id, name, eventTypes) { + this.properties.push({clsName: 'EVENT_TYPES', id, name, eventTypes}); + } +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js new file mode 100644 index 0000000000000..869e3dfb52063 --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js @@ -0,0 +1,1785 @@ +/* + * 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. + */ + +import DFLT_DIALECTS from 'app/data/dialects.json'; + +import { EmptyBean, Bean } from './Beans'; + +export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'igniteIgfsDefaults', (JavaTypes, clusterDflts, cacheDflts, igfsDflts) => { + class ConfigurationGenerator { + static igniteConfigurationBean(cluster) { + return new Bean('org.apache.ignite.configuration.IgniteConfiguration', 'cfg', cluster, clusterDflts); + } + + static igfsConfigurationBean(igfs) { + return new Bean('org.apache.ignite.configuration.FileSystemConfiguration', 'igfs', igfs, igfsDflts); + } + + static cacheConfigurationBean(cache) { + return new Bean('org.apache.ignite.configuration.CacheConfiguration', 'ccfg', cache, cacheDflts); + } + + static domainConfigurationBean(domain) { + return new Bean('org.apache.ignite.cache.QueryEntity', 'qryEntity', domain, cacheDflts); + } + + static discoveryConfigurationBean(discovery) { + return new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery', discovery, clusterDflts.discovery); + } + + /** + * Function to generate ignite configuration. + * + * @param {Object} cluster Cluster to process. + * @param {Boolean} client + * @return {Bean} Generated ignite configuration. + */ + static igniteConfiguration(cluster, client) { + const cfg = this.igniteConfigurationBean(cluster); + + this.clusterGeneral(cluster, cfg, client); + this.clusterAtomics(cluster.atomicConfiguration, cfg); + this.clusterBinary(cluster.binaryConfiguration, cfg); + this.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, cfg); + this.clusterCheckpoint(cluster, cluster.caches, cfg); + this.clusterCollision(cluster.collision, cfg); + this.clusterCommunication(cluster, cfg); + this.clusterConnector(cluster.connector, cfg); + this.clusterDeployment(cluster, cfg); + this.clusterEvents(cluster, cfg); + this.clusterFailover(cluster, cfg); + this.clusterLoadBalancing(cluster, cfg); + this.clusterLogger(cluster.logger, cfg); + this.clusterODBC(cluster.odbc, cfg); + this.clusterMarshaller(cluster, cfg); + this.clusterMetrics(cluster, cfg); + this.clusterSwap(cluster, cfg); + this.clusterTime(cluster, cfg); + this.clusterPools(cluster, cfg); + this.clusterTransactions(cluster.transactionConfiguration, cfg); + this.clusterSsl(cluster, cfg); + this.clusterUserAttributes(cluster, cfg); + + this.clusterCaches(cluster, cluster.caches, cluster.igfss, client, cfg); + + if (!client) + this.clusterIgfss(cluster.igfss, cfg); + + return cfg; + } + + static dialectClsName(dialect) { + return DFLT_DIALECTS[dialect] || 'Unknown database: ' + (dialect || 'Choose JDBC dialect'); + } + + static dataSourceBean(id, dialect) { + let dsBean; + + switch (dialect) { + case 'Generic': + dsBean = new Bean('com.mchange.v2.c3p0.ComboPooledDataSource', id, {}) + .property('jdbcUrl', `${id}.jdbc.url`, 'jdbc:your_database'); + + break; + case 'Oracle': + dsBean = new Bean('oracle.jdbc.pool.OracleDataSource', id, {}) + .property('URL', `${id}.jdbc.url`, 'jdbc:oracle:thin:@[host]:[port]:[database]'); + + break; + case 'DB2': + dsBean = new Bean('com.ibm.db2.jcc.DB2DataSource', id, {}) + .property('serverName', `${id}.jdbc.server_name`, 'YOUR_DATABASE_SERVER_NAME') + .property('portNumber', `${id}.jdbc.port_number`, 'YOUR_JDBC_PORT_NUMBER') + .property('databaseName', `${id}.jdbc.database_name`, 'YOUR_JDBC_DRIVER_TYPE') + .property('driverType', `${id}.jdbc.driver_type`, 'YOUR_DATABASE_NAME'); + + break; + case 'SQLServer': + dsBean = new Bean('com.microsoft.sqlserver.jdbc.SQLServerDataSource', id, {}) + .property('URL', `${id}.jdbc.url`, 'jdbc:sqlserver://[host]:[port][;databaseName=database]'); + + break; + case 'MySQL': + dsBean = new Bean('com.mysql.jdbc.jdbc2.optional.MysqlDataSource', id, {}) + .property('URL', `${id}.jdbc.url`, 'jdbc:mysql://[host]:[port]/[database]'); + + break; + case 'PostgreSQL': + dsBean = new Bean('org.postgresql.ds.PGPoolingDataSource', id, {}) + .property('url', `${id}.jdbc.url`, 'jdbc:postgresql://[host]:[port]/[database]'); + + break; + case 'H2': + dsBean = new Bean('org.h2.jdbcx.JdbcDataSource', id, {}) + .property('URL', `${id}.jdbc.url`, 'jdbc:h2:tcp://[host]/[database]'); + + break; + default: + } + + if (dsBean) { + dsBean.property('user', `${id}.jdbc.username`, 'YOUR_USER_NAME') + .property('password', `${id}.jdbc.password`, 'YOUR_PASSWORD'); + } + + return dsBean; + } + + // Generate general section. + static clusterGeneral(cluster, cfg = this.igniteConfigurationBean(cluster), client = false) { + if (client) + cfg.prop('boolean', 'clientMode', true); + + cfg.stringProperty('name', 'gridName') + .stringProperty('localHost'); + + if (_.isNil(cluster.discovery)) + return cfg; + + const discovery = new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery', + cluster.discovery, clusterDflts.discovery); + + let ipFinder; + + switch (discovery.valueOf('kind')) { + case 'Vm': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder', + 'ipFinder', cluster.discovery.Vm, clusterDflts.discovery.Vm); + + ipFinder.collectionProperty('addrs', 'addresses', cluster.discovery.Vm.addresses); + + break; + case 'Multicast': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder', + 'ipFinder', cluster.discovery.Multicast, clusterDflts.discovery.Multicast); + + ipFinder.stringProperty('multicastGroup') + .intProperty('multicastPort') + .intProperty('responseWaitTime') + .intProperty('addressRequestAttempts') + .stringProperty('localAddress') + .collectionProperty('addrs', 'addresses', cluster.discovery.Multicast.addresses); + + break; + case 'S3': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinder', + 'ipFinder', cluster.discovery.S3, clusterDflts.discovery.S3); + + ipFinder.stringProperty('bucketName'); + + break; + case 'Cloud': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinder', + 'ipFinder', cluster.discovery.Cloud, clusterDflts.discovery.Cloud); + + ipFinder.stringProperty('credential') + .pathProperty('credentialPath') + .stringProperty('identity') + .stringProperty('provider') + .collectionProperty('regions', 'regions', cluster.discovery.Cloud.regions) + .collectionProperty('zones', 'zones', cluster.discovery.Cloud.zones); + + break; + case 'GoogleStorage': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.gce.TcpDiscoveryGoogleStorageIpFinder', + 'ipFinder', cluster.discovery.GoogleStorage, clusterDflts.discovery.GoogleStorage); + + ipFinder.stringProperty('projectName') + .stringProperty('bucketName') + .pathProperty('serviceAccountP12FilePath') + .stringProperty('serviceAccountId'); + + break; + case 'Jdbc': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder', + 'ipFinder', cluster.discovery.Jdbc, clusterDflts.discovery.Jdbc); + + ipFinder.intProperty('initSchema'); + + if (ipFinder.includes('dataSourceBean', 'dialect')) { + const id = ipFinder.valueOf('dataSourceBean'); + + ipFinder.dataSource(id, 'dataSource', this.dataSourceBean(id, ipFinder.valueOf('dialect'))); + } + + break; + case 'SharedFs': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder', + 'ipFinder', cluster.discovery.SharedFs, clusterDflts.discovery.SharedFs); + + ipFinder.pathProperty('path'); + + break; + case 'ZooKeeper': + const src = cluster.discovery.ZooKeeper; + const dflt = clusterDflts.discovery.ZooKeeper; + + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.zk.TcpDiscoveryZookeeperIpFinder', + 'ipFinder', src, dflt); + + ipFinder.emptyBeanProperty('curator') + .stringProperty('zkConnectionString'); + + if (_.get(src, 'retryPolicy.kind')) { + const policy = src.retryPolicy; + + let retryPolicyBean; + + switch (policy.kind) { + case 'ExponentialBackoff': + retryPolicyBean = new Bean('org.apache.curator.retry.ExponentialBackoffRetry', null, + policy.ExponentialBackoff, dflt.ExponentialBackoff) + .intConstructorArgument('baseSleepTimeMs') + .intConstructorArgument('maxRetries') + .intConstructorArgument('maxSleepMs'); + + break; + case 'BoundedExponentialBackoff': + retryPolicyBean = new Bean('org.apache.curator.retry.BoundedExponentialBackoffRetry', + null, policy.BoundedExponentialBackoffRetry, dflt.BoundedExponentialBackoffRetry) + .intConstructorArgument('baseSleepTimeMs') + .intConstructorArgument('maxSleepTimeMs') + .intConstructorArgument('maxRetries'); + + break; + case 'UntilElapsed': + retryPolicyBean = new Bean('org.apache.curator.retry.RetryUntilElapsed', null, + policy.UntilElapsed, dflt.UntilElapsed) + .intConstructorArgument('maxElapsedTimeMs') + .intConstructorArgument('sleepMsBetweenRetries'); + + break; + + case 'NTimes': + retryPolicyBean = new Bean('org.apache.curator.retry.RetryNTimes', null, + policy.NTimes, dflt.NTimes) + .intConstructorArgument('n') + .intConstructorArgument('sleepMsBetweenRetries'); + + break; + case 'OneTime': + retryPolicyBean = new Bean('org.apache.curator.retry.RetryOneTime', null, + policy.OneTime, dflt.OneTime) + .intConstructorArgument('sleepMsBetweenRetry'); + + break; + case 'Forever': + retryPolicyBean = new Bean('org.apache.curator.retry.RetryForever', null, + policy.Forever, dflt.Forever) + .intConstructorArgument('retryIntervalMs'); + + break; + case 'Custom': + if (_.nonEmpty(policy.Custom.className)) + retryPolicyBean = new EmptyBean(policy.Custom.className); + + break; + default: + } + + if (retryPolicyBean) + ipFinder.beanProperty('retryPolicy', retryPolicyBean); + } + + ipFinder.pathProperty('basePath', '/services') + .stringProperty('serviceName') + .boolProperty('allowDuplicateRegistrations'); + + break; + default: + } + + if (ipFinder) + discovery.beanProperty('ipFinder', ipFinder); + + this.clusterDiscovery(cluster.discovery, cfg, discovery); + + return cfg; + } + + static igfsDataCache(igfs) { + return this.cacheConfiguration({ + name: igfs.name + '-data', + cacheMode: 'PARTITIONED', + atomicityMode: 'TRANSACTIONAL', + writeSynchronizationMode: 'FULL_SYNC', + backups: 0, + igfsAffinnityGroupSize: igfs.affinnityGroupSize || 512 + }); + } + + static igfsMetaCache(igfs) { + return this.cacheConfiguration({ + name: igfs.name + '-meta', + cacheMode: 'REPLICATED', + atomicityMode: 'TRANSACTIONAL', + writeSynchronizationMode: 'FULL_SYNC' + }); + } + + static clusterCaches(cluster, caches, igfss, client, cfg = this.igniteConfigurationBean(cluster)) { + const ccfgs = _.map(caches, (cache) => this.cacheConfiguration(cache)); + + if (!client) { + _.forEach(igfss, (igfs) => { + ccfgs.push(this.igfsDataCache(igfs)); + ccfgs.push(this.igfsMetaCache(igfs)); + }); + } + + cfg.varArgProperty('ccfgs', 'cacheConfiguration', ccfgs, 'org.apache.ignite.configuration.CacheConfiguration'); + + return cfg; + } + + // Generate atomics group. + static clusterAtomics(atomics, cfg = this.igniteConfigurationBean()) { + const acfg = new Bean('org.apache.ignite.configuration.AtomicConfiguration', 'atomicCfg', + atomics, clusterDflts.atomics); + + acfg.enumProperty('cacheMode') + .intProperty('atomicSequenceReserveSize'); + + if (acfg.valueOf('cacheMode') === 'PARTITIONED') + acfg.intProperty('backups'); + + if (acfg.isEmpty()) + return cfg; + + cfg.beanProperty('atomicConfiguration', acfg); + + return cfg; + } + + // Generate binary group. + static clusterBinary(binary, cfg = this.igniteConfigurationBean()) { + const binaryCfg = new Bean('org.apache.ignite.configuration.BinaryConfiguration', 'binaryCfg', + binary, clusterDflts.binary); + + binaryCfg.emptyBeanProperty('idMapper') + .emptyBeanProperty('nameMapper') + .emptyBeanProperty('serializer'); + + const typeCfgs = []; + + _.forEach(binary.typeConfigurations, (type) => { + const typeCfg = new Bean('org.apache.ignite.binary.BinaryTypeConfiguration', + JavaTypes.toJavaName('binaryType', type.typeName), type, clusterDflts.binary.typeConfigurations); + + typeCfg.stringProperty('typeName') + .emptyBeanProperty('idMapper') + .emptyBeanProperty('nameMapper') + .emptyBeanProperty('serializer') + .intProperty('enum'); + + if (typeCfg.nonEmpty()) + typeCfgs.push(typeCfg); + }); + + binaryCfg.collectionProperty('types', 'typeConfigurations', typeCfgs, 'org.apache.ignite.binary.BinaryTypeConfiguration') + .boolProperty('compactFooter'); + + if (binaryCfg.isEmpty()) + return cfg; + + cfg.beanProperty('binaryConfiguration', binaryCfg); + + return cfg; + } + + // Generate cache key configurations. + static clusterCacheKeyConfiguration(keyCfgs, cfg = this.igniteConfigurationBean()) { + const items = _.reduce(keyCfgs, (acc, keyCfg) => { + if (keyCfg.typeName && keyCfg.affinityKeyFieldName) { + acc.push(new Bean('org.apache.ignite.cache.CacheKeyConfiguration', null, keyCfg) + .stringConstructorArgument('typeName') + .stringConstructorArgument('affinityKeyFieldName')); + } + + return acc; + }, []); + + if (_.isEmpty(items)) + return cfg; + + cfg.arrayProperty('cacheKeyConfiguration', 'cacheKeyConfiguration', items, + 'org.apache.ignite.cache.CacheKeyConfiguration'); + + return cfg; + } + + // Generate checkpoint configurations. + static clusterCheckpoint(cluster, caches, cfg = this.igniteConfigurationBean()) { + const cfgs = _.filter(_.map(cluster.checkpointSpi, (spi) => { + switch (spi.kind) { + case 'FS': + const fsBean = new Bean('org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi', + 'checkpointSpi', spi.FS); + + fsBean.collectionProperty('directoryPaths', 'directoryPaths', _.get(spi, 'FS.directoryPaths')) + .emptyBeanProperty('checkpointListener'); + + return fsBean; + + case 'Cache': + const cacheBean = new Bean('org.apache.ignite.spi.checkpoint.cache.CacheCheckpointSpi', + 'checkpointSpi', spi.Cache); + + const cache = _.find(caches, (c) => c._id === _.get(spi, 'Cache.cache') || c.cache._id === _.get(spi, 'Cache.cache')); + + if (cache) + cacheBean.prop('java.lang.String', 'cacheName', cache.name || cache.cache.name); + + cacheBean.stringProperty('cacheName') + .emptyBeanProperty('checkpointListener'); + + return cacheBean; + + case 'S3': + const s3Bean = new Bean('org.apache.ignite.spi.checkpoint.s3.S3CheckpointSpi', + 'checkpointSpi', spi.S3, clusterDflts.checkpointSpi.S3); + + let credentialsBean = null; + + switch (_.get(spi.S3, 'awsCredentials.kind')) { + case 'Basic': + credentialsBean = new Bean('com.amazonaws.auth.BasicAWSCredentials', 'awsCredentials', {}); + + credentialsBean.constructorArgument('PROPERTY', 'checkpoint.s3.credentials.accessKey') + .constructorArgument('PROPERTY', 'checkpoint.s3.credentials.secretKey'); + + break; + + case 'Properties': + credentialsBean = new Bean('com.amazonaws.auth.PropertiesCredentials', 'awsCredentials', {}); + + const fileBean = new Bean('java.io.File', '', spi.S3.awsCredentials.Properties) + .pathConstructorArgument('path'); + + if (fileBean.nonEmpty()) + credentialsBean.beanConstructorArgument('file', fileBean); + + break; + + case 'Anonymous': + credentialsBean = new Bean('com.amazonaws.auth.AnonymousAWSCredentials', 'awsCredentials', {}); + + break; + + case 'BasicSession': + credentialsBean = new Bean('com.amazonaws.auth.BasicSessionCredentials', 'awsCredentials', {}); + + // TODO 2054 Arguments in one line is very long string. + credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey') + .propertyConstructorArgument('checkpoint.s3.credentials.secretKey') + .propertyConstructorArgument('checkpoint.s3.credentials.sessionToken'); + + break; + + case 'Custom': + const className = _.get(spi.S3.awsCredentials, 'Custom.className'); + + credentialsBean = new Bean(className, 'awsCredentials', {}); + + break; + + default: + break; + } + + if (credentialsBean) + s3Bean.beanProperty('awsCredentials', credentialsBean); + + s3Bean.stringProperty('bucketNameSuffix'); + + const clientBean = new Bean('com.amazonaws.ClientConfiguration', 'clientCfg', spi.S3.clientConfiguration, + clusterDflts.checkpointSpi.S3.clientConfiguration); + + clientBean.enumProperty('protocol') + .intProperty('maxConnections') + .stringProperty('userAgent'); + + const locAddr = new Bean('java.net.InetAddress', '', spi.S3.clientConfiguration) + .factoryMethod('getByName') + .stringConstructorArgument('localAddress'); + + if (locAddr.nonEmpty()) + clientBean.beanProperty('localAddress', locAddr); + + clientBean.stringProperty('proxyHost') + .intProperty('proxyPort') + .stringProperty('proxyUsername'); + + const userName = clientBean.valueOf('proxyUsername'); + + if (userName) + clientBean.property('proxyPassword', `checkpoint.s3.proxy.${userName}.password`); + + clientBean.stringProperty('proxyDomain') + .stringProperty('proxyWorkstation'); + + const retryPolicy = spi.S3.clientConfiguration.retryPolicy; + + if (retryPolicy) { + const kind = retryPolicy.kind; + + const policy = retryPolicy[kind]; + + let retryBean; + + switch (kind) { + case 'Default': + retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', { + retryCondition: 'DEFAULT_RETRY_CONDITION', + backoffStrategy: 'DEFAULT_BACKOFF_STRATEGY', + maxErrorRetry: 'DEFAULT_MAX_ERROR_RETRY', + honorMaxErrorRetryInClientConfig: true + }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy); + + retryBean.constantConstructorArgument('retryCondition') + .constantConstructorArgument('backoffStrategy') + .constantConstructorArgument('maxErrorRetry') + .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); + + break; + + case 'DefaultMaxRetries': + retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', { + retryCondition: 'DEFAULT_RETRY_CONDITION', + backoffStrategy: 'DEFAULT_BACKOFF_STRATEGY', + maxErrorRetry: _.get(policy, 'maxErrorRetry') || -1, + honorMaxErrorRetryInClientConfig: false + }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy); + + retryBean.constantConstructorArgument('retryCondition') + .constantConstructorArgument('backoffStrategy') + .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry')) + .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); + + break; + + case 'DynamoDB': + retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', { + retryCondition: 'DEFAULT_RETRY_CONDITION', + backoffStrategy: 'DYNAMODB_DEFAULT_BACKOFF_STRATEGY', + maxErrorRetry: 'DYNAMODB_DEFAULT_MAX_ERROR_RETRY', + honorMaxErrorRetryInClientConfig: true + }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy); + + retryBean.constantConstructorArgument('retryCondition') + .constantConstructorArgument('backoffStrategy') + .constantConstructorArgument('maxErrorRetry') + .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); + + break; + + case 'DynamoDBMaxRetries': + retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', { + retryCondition: 'DEFAULT_RETRY_CONDITION', + backoffStrategy: 'DYNAMODB_DEFAULT_BACKOFF_STRATEGY', + maxErrorRetry: _.get(policy, 'maxErrorRetry') || -1, + honorMaxErrorRetryInClientConfig: false + }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy); + + retryBean.constantConstructorArgument('retryCondition') + .constantConstructorArgument('backoffStrategy') + .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry')) + .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); + + break; + + case 'Custom': + retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', policy); + + retryBean.beanConstructorArgument('retryCondition', retryBean.valueOf('retryCondition') ? new EmptyBean(retryBean.valueOf('retryCondition')) : null) + .beanConstructorArgument('backoffStrategy', retryBean.valueOf('backoffStrategy') ? new EmptyBean(retryBean.valueOf('backoffStrategy')) : null) + .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry')) + .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); + + break; + + default: + break; + } + + if (retryBean) + clientBean.beanProperty('retryPolicy', retryBean); + } + + clientBean.intProperty('maxErrorRetry') + .intProperty('socketTimeout') + .intProperty('connectionTimeout') + .intProperty('requestTimeout') + .intProperty('socketSendBufferSizeHints') + .stringProperty('signerOverride') + .intProperty('connectionTTL') + .intProperty('connectionMaxIdleMillis') + .emptyBeanProperty('dnsResolver') + .intProperty('responseMetadataCacheSize') + .emptyBeanProperty('secureRandom') + .boolProperty('useReaper') + .boolProperty('useGzip') + .boolProperty('preemptiveBasicProxyAuth') + .boolProperty('useTcpKeepAlive'); + + if (clientBean.nonEmpty()) + s3Bean.beanProperty('clientConfiguration', clientBean); + + s3Bean.emptyBeanProperty('checkpointListener'); + + return s3Bean; + + case 'JDBC': + const jdbcBean = new Bean('org.apache.ignite.spi.checkpoint.jdbc.JdbcCheckpointSpi', + 'checkpointSpi', spi.JDBC, clusterDflts.checkpointSpi.JDBC); + + const id = jdbcBean.valueOf('dataSourceBean'); + const dialect = _.get(spi.JDBC, 'dialect'); + + jdbcBean.dataSource(id, 'dataSource', this.dataSourceBean(id, dialect)); + + if (!_.isEmpty(jdbcBean.valueOf('user'))) { + jdbcBean.stringProperty('user') + .property('pwd', `checkpoint.${jdbcBean.valueOf('dataSourceBean')}.${jdbcBean.valueOf('user')}.jdbc.password`, 'YOUR_PASSWORD'); + } + + jdbcBean.stringProperty('checkpointTableName') + .stringProperty('keyFieldName') + .stringProperty('keyFieldType') + .stringProperty('valueFieldName') + .stringProperty('valueFieldType') + .stringProperty('expireDateFieldName') + .stringProperty('expireDateFieldType') + .intProperty('numberOfRetries') + .emptyBeanProperty('checkpointListener'); + + return jdbcBean; + + case 'Custom': + const clsName = _.get(spi, 'Custom.className'); + + if (clsName) + return new Bean(clsName, 'checkpointSpi', spi.Cache); + + return null; + + default: + return null; + } + }), (checkpointBean) => _.nonNil(checkpointBean)); + + cfg.arrayProperty('checkpointSpi', 'checkpointSpi', cfgs, 'org.apache.ignite.spi.checkpoint.CheckpointSpi'); + + return cfg; + } + + // Generate collision group. + static clusterCollision(collision, cfg = this.igniteConfigurationBean()) { + let colSpi; + + switch (collision.kind) { + case 'JobStealing': + colSpi = new Bean('org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi', + 'colSpi', collision.JobStealing, clusterDflts.collision.JobStealing); + + colSpi.intProperty('activeJobsThreshold') + .intProperty('waitJobsThreshold') + .intProperty('messageExpireTime') + .intProperty('maximumStealingAttempts') + .boolProperty('stealingEnabled') + .emptyBeanProperty('externalCollisionListener') + .mapProperty('stealingAttrs', 'stealingAttributes'); + + break; + case 'FifoQueue': + colSpi = new Bean('org.apache.ignite.spi.collision.fifoqueue.FifoQueueCollisionSpi', + 'colSpi', collision.FifoQueue, clusterDflts.collision.FifoQueue); + + colSpi.intProperty('parallelJobsNumber') + .intProperty('waitingJobsNumber'); + + break; + case 'PriorityQueue': + colSpi = new Bean('org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi', + 'colSpi', collision.PriorityQueue, clusterDflts.collision.PriorityQueue); + + colSpi.intProperty('parallelJobsNumber') + .intProperty('waitingJobsNumber') + .intProperty('priorityAttributeKey') + .intProperty('jobPriorityAttributeKey') + .intProperty('defaultPriority') + .intProperty('starvationIncrement') + .boolProperty('starvationPreventionEnabled'); + + break; + case 'Custom': + colSpi = new Bean(collision.Custom.class, + 'colSpi', collision.PriorityQueue, clusterDflts.collision.PriorityQueue); + + break; + default: + return cfg; + } + + if (colSpi.isEmpty()) + return cfg; + + cfg.beanProperty('collisionSpi', colSpi); + + return cfg; + } + + // Generate communication group. + static clusterCommunication(cluster, cfg = this.igniteConfigurationBean(cluster)) { + const commSpi = new Bean('org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi', 'communicationSpi', + cluster.communication, clusterDflts.communication); + + commSpi.emptyBeanProperty('listener') + .stringProperty('localAddress') + .intProperty('localPort') + .intProperty('localPortRange') + .intProperty('sharedMemoryPort') + .intProperty('directBuffer') + .intProperty('directSendBuffer') + .intProperty('idleConnectionTimeout') + .intProperty('connectTimeout') + .intProperty('maxConnectTimeout') + .intProperty('reconnectCount') + .intProperty('socketSendBuffer') + .intProperty('socketReceiveBuffer') + .intProperty('messageQueueLimit') + .intProperty('slowClientQueueLimit') + .intProperty('tcpNoDelay') + .intProperty('ackSendThreshold') + .intProperty('unacknowledgedMessagesBufferSize') + .intProperty('socketWriteTimeout') + .intProperty('selectorsCount') + .emptyBeanProperty('addressResolver'); + + if (commSpi.nonEmpty()) + cfg.beanProperty('communicationSpi', commSpi); + + cfg.intProperty('networkTimeout') + .intProperty('networkSendRetryDelay') + .intProperty('networkSendRetryCount') + .intProperty('discoveryStartupDelay'); + + return cfg; + } + + // Generate REST access configuration. + static clusterConnector(connector, cfg = this.igniteConfigurationBean()) { + const connCfg = new Bean('org.apache.ignite.configuration.ConnectorConfiguration', + 'connectorConfiguration', connector, clusterDflts.connector); + + if (connCfg.valueOf('enabled')) { + connCfg.pathProperty('jettyPath') + .stringProperty('host') + .intProperty('port') + .intProperty('portRange') + .intProperty('idleTimeout') + .intProperty('idleQueryCursorTimeout') + .intProperty('idleQueryCursorCheckFrequency') + .intProperty('receiveBufferSize') + .intProperty('sendBufferSize') + .intProperty('sendQueueLimit') + .intProperty('directBuffer') + .intProperty('noDelay') + .intProperty('selectorCount') + .intProperty('threadPoolSize') + .emptyBeanProperty('messageInterceptor') + .stringProperty('secretKey'); + + if (connCfg.valueOf('sslEnabled')) { + connCfg.intProperty('sslClientAuth') + .emptyBeanProperty('sslFactory'); + } + + if (connCfg.nonEmpty()) + cfg.beanProperty('connectorConfiguration', connCfg); + } + + return cfg; + } + + // Generate deployment group. + static clusterDeployment(cluster, cfg = this.igniteConfigurationBean(cluster)) { + cfg.enumProperty('deploymentMode') + .boolProperty('peerClassLoadingEnabled'); + + if (cfg.valueOf('peerClassLoadingEnabled')) { + cfg.intProperty('peerClassLoadingMissedResourcesCacheSize') + .intProperty('peerClassLoadingThreadPoolSize') + .varArgProperty('p2pLocClsPathExcl', 'peerClassLoadingLocalClassPathExclude', + cluster.peerClassLoadingLocalClassPathExclude); + } + + return cfg; + } + + // Generate discovery group. + static clusterDiscovery(discovery, cfg = this.igniteConfigurationBean(), discoSpi = this.discoveryConfigurationBean(discovery)) { + discoSpi.stringProperty('localAddress') + .intProperty('localPort') + .intProperty('localPortRange') + .emptyBeanProperty('addressResolver') + .intProperty('socketTimeout') + .intProperty('ackTimeout') + .intProperty('maxAckTimeout') + .intProperty('networkTimeout') + .intProperty('joinTimeout') + .intProperty('threadPriority') + .intProperty('heartbeatFrequency') + .intProperty('maxMissedHeartbeats') + .intProperty('maxMissedClientHeartbeats') + .intProperty('topHistorySize') + .emptyBeanProperty('listener') + .emptyBeanProperty('dataExchange') + .emptyBeanProperty('metricsProvider') + .intProperty('reconnectCount') + .intProperty('statisticsPrintFrequency') + .intProperty('ipFinderCleanFrequency') + .emptyBeanProperty('authenticator') + .intProperty('forceServerMode') + .intProperty('clientReconnectDisabled'); + + if (discoSpi.nonEmpty()) + cfg.beanProperty('discoverySpi', discoSpi); + + return discoSpi; + } + + // Generate events group. + static clusterEvents(cluster, cfg = this.igniteConfigurationBean(cluster)) { + const eventStorage = cluster.eventStorage; + + let eventStorageBean = null; + + switch (_.get(eventStorage, 'kind')) { + case 'Memory': + eventStorageBean = new Bean('org.apache.ignite.spi.eventstorage.memory.MemoryEventStorageSpi', 'eventStorage', eventStorage.Memory, clusterDflts.eventStorage.Memory); + + eventStorageBean.intProperty('expireAgeMs') + .intProperty('expireCount') + .emptyBeanProperty('filter'); + + break; + + case 'Custom': + const className = _.get(eventStorage, 'Custom.className'); + + if (className) + eventStorageBean = new EmptyBean(className); + + break; + + default: + // No-op. + } + + if (eventStorageBean && eventStorageBean.nonEmpty()) + cfg.beanProperty('eventStorageSpi', eventStorageBean); + + if (_.nonEmpty(cluster.includeEventTypes)) + cfg.eventTypes('evts', 'includeEventTypes', cluster.includeEventTypes); + + return cfg; + } + + // Generate failover group. + static clusterFailover(cluster, cfg = this.igniteConfigurationBean(cluster)) { + const spis = []; + + _.forEach(cluster.failoverSpi, (spi) => { + let failoverSpi; + + switch (spi.kind) { + case 'JobStealing': + failoverSpi = new Bean('org.apache.ignite.spi.failover.jobstealing.JobStealingFailoverSpi', + 'failoverSpi', spi.JobStealing, clusterDflts.failoverSpi.JobStealing); + + failoverSpi.intProperty('maximumFailoverAttempts'); + + break; + case 'Never': + failoverSpi = new Bean('org.apache.ignite.spi.failover.never.NeverFailoverSpi', + 'failoverSpi', spi.Never); + + break; + case 'Always': + failoverSpi = new Bean('org.apache.ignite.spi.failover.always.AlwaysFailoverSpi', + 'failoverSpi', spi.Always, clusterDflts.failoverSpi.Always); + + failoverSpi.intProperty('maximumFailoverAttempts'); + + break; + case 'Custom': + const className = _.get(spi, 'Custom.class'); + + if (className) + failoverSpi = new EmptyBean(className); + + break; + default: + // No-op. + } + + if (failoverSpi) + spis.push(failoverSpi); + }); + + if (spis.length) + cfg.arrayProperty('failoverSpi', 'failoverSpi', spis, 'org.apache.ignite.spi.failover.FailoverSpi'); + + return cfg; + } + + // Generate load balancing configuration group. + static clusterLoadBalancing(cluster, cfg = this.igniteConfigurationBean(cluster)) { + const spis = []; + + _.forEach(cluster.loadBalancingSpi, (spi) => { + let loadBalancingSpi; + + switch (spi.kind) { + case 'RoundRobin': + loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi', 'loadBalancingSpi', spi.RoundRobin, clusterDflts.loadBalancingSpi.RoundRobin); + + loadBalancingSpi.boolProperty('perTask'); + + break; + case 'Adaptive': + loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveLoadBalancingSpi', 'loadBalancingSpi', spi.Adaptive); + + let probeBean; + + switch (spi.Adaptive.loadProbe.kind) { + case 'Job': + probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveJobCountLoadProbe', 'jobProbe', spi.Adaptive.loadProbe.Job, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.Job); + + probeBean.boolProperty('useAverage'); + + break; + + case 'CPU': + probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveCpuLoadProbe', 'cpuProbe', spi.Adaptive.loadProbe.CPU, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.CPU); + + probeBean.boolProperty('useAverage') + .boolProperty('useProcessors') + .intProperty('processorCoefficient'); + + break; + + case 'ProcessingTime': + probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveProcessingTimeLoadProbe', 'timeProbe', spi.Adaptive.loadProbe.ProcessingTime, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.ProcessingTime); + + probeBean.boolProperty('useAverage'); + + break; + + case 'Custom': + const className = _.get(spi, 'Adaptive.loadProbe.Custom.className'); + + if (className) + probeBean = new Bean(className, 'probe', spi.Adaptive.loadProbe.Job.Custom); + + break; + + default: + // No-op. + } + + if (probeBean) + loadBalancingSpi.beanProperty('loadProbe', probeBean); + + break; + case 'WeightedRandom': + loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.weightedrandom.WeightedRandomLoadBalancingSpi', 'loadBalancingSpi', spi.WeightedRandom, clusterDflts.loadBalancingSpi.WeightedRandom); + + loadBalancingSpi.intProperty('nodeWeight') + .boolProperty('useWeights'); + + break; + case 'Custom': + const className = _.get(spi, 'Custom.className'); + + if (className) + loadBalancingSpi = new Bean(className, 'loadBalancingSpi', spi.Custom); + + break; + default: + // No-op. + } + + if (loadBalancingSpi) + spis.push(loadBalancingSpi); + }); + + if (spis.length) + cfg.arrayProperty('loadBalancingSpi', 'loadBalancingSpi', spis, 'org.apache.ignite.spi.loadbalancing.LoadBalancingSpi'); + + return cfg; + } + + // Generate logger group. + static clusterLogger(logger, cfg = this.igniteConfigurationBean()) { + if (_.isNil(logger)) + return cfg; + + let loggerBean; + + switch (logger.kind) { + case 'Log4j': + if (logger.Log4j && (logger.Log4j.mode === 'Default' || logger.Log4j.mode === 'Path' && _.nonEmpty(logger.Log4j.path))) { + loggerBean = new Bean('org.apache.ignite.logger.log4j.Log4JLogger', + 'logger', logger.Log4j, clusterDflts.logger.Log4j); + + if (loggerBean.valueOf('mode') === 'Path') + loggerBean.pathConstructorArgument('path'); + + loggerBean.enumProperty('level'); + } + + break; + case 'Log4j2': + if (logger.Log4j2 && _.nonEmpty(logger.Log4j2.path)) { + loggerBean = new Bean('org.apache.ignite.logger.log4j2.Log4J2Logger', + 'logger', logger.Log4j2, clusterDflts.logger.Log4j2); + + loggerBean.pathConstructorArgument('path') + .enumProperty('level'); + } + + break; + case 'Null': + loggerBean = new EmptyBean('org.apache.ignite.logger.NullLogger'); + + break; + case 'Java': + loggerBean = new EmptyBean('org.apache.ignite.logger.java.JavaLogger'); + + break; + case 'JCL': + loggerBean = new EmptyBean('org.apache.ignite.logger.jcl.JclLogger'); + + break; + case 'SLF4J': + loggerBean = new EmptyBean('org.apache.ignite.logger.slf4j.Slf4jLogger'); + + break; + case 'Custom': + if (logger.Custom && _.nonEmpty(logger.Custom.class)) + loggerBean = new EmptyBean(logger.Custom.class); + + break; + default: + } + + if (loggerBean) + cfg.beanProperty('gridLogger', loggerBean); + + return cfg; + } + + // Generate IGFSs configs. + static clusterIgfss(igfss, cfg = this.igniteConfigurationBean()) { + const igfsCfgs = _.map(igfss, (igfs) => { + const igfsCfg = this.igfsGeneral(igfs); + + this.igfsIPC(igfs, igfsCfg); + this.igfsFragmentizer(igfs, igfsCfg); + this.igfsDualMode(igfs, igfsCfg); + this.igfsSecondFS(igfs, igfsCfg); + this.igfsMisc(igfs, igfsCfg); + + return igfsCfg; + }); + + cfg.varArgProperty('igfsCfgs', 'fileSystemConfiguration', igfsCfgs, 'org.apache.ignite.configuration.FileSystemConfiguration'); + + return cfg; + } + + // Generate marshaller group. + static clusterMarshaller(cluster, cfg = this.igniteConfigurationBean(cluster)) { + const marshaller = cluster.marshaller; + + if (marshaller && marshaller.kind) { + let bean; + + switch (marshaller.kind) { + case 'OptimizedMarshaller': + bean = new Bean('org.apache.ignite.marshaller.optimized.OptimizedMarshaller', 'marshaller', + marshaller[marshaller.kind]); + + bean.intProperty('poolSize') + .intProperty('requireSerializable'); + + break; + + case 'JdkMarshaller': + bean = new Bean('org.apache.ignite.marshaller.jdk.JdkMarshaller', 'marshaller', + marshaller[marshaller.kind]); + + break; + + default: + } + + if (bean) + cfg.beanProperty('marshaller', bean); + } + + cfg.intProperty('marshalLocalJobs') + .intProperty('marshallerCacheKeepAliveTime') + .intProperty('marshallerCacheThreadPoolSize', 'marshallerCachePoolSize'); + + return cfg; + } + + // Generate metrics group. + static clusterMetrics(cluster, cfg = this.igniteConfigurationBean(cluster)) { + cfg.intProperty('metricsExpireTime') + .intProperty('metricsHistorySize') + .intProperty('metricsLogFrequency') + .intProperty('metricsUpdateFrequency'); + + return cfg; + } + + // Generate ODBC group. + static clusterODBC(odbc, cfg = this.igniteConfigurationBean()) { + if (_.get(odbc, 'odbcEnabled') !== true) + return cfg; + + const bean = new Bean('org.apache.ignite.configuration.OdbcConfiguration', 'odbcConfiguration', + odbc, clusterDflts.odbcConfiguration); + + bean.stringProperty('endpointAddress') + .intProperty('maxOpenCursors'); + + cfg.beanProperty('odbcConfiguration', bean); + + return cfg; + } + + // Java code generator for cluster's SSL configuration. + static clusterSsl(cluster, cfg = this.igniteConfigurationBean(cluster)) { + if (cluster.sslEnabled && _.nonNil(cluster.sslContextFactory)) { + const bean = new Bean('org.apache.ignite.ssl.SslContextFactory', 'sslCtxFactory', + cluster.sslContextFactory); + + bean.intProperty('keyAlgorithm') + .pathProperty('keyStoreFilePath'); + + if (_.nonEmpty(bean.valueOf('keyStoreFilePath'))) + bean.propertyChar('keyStorePassword', 'ssl.key.storage.password', 'YOUR_SSL_KEY_STORAGE_PASSWORD'); + + bean.intProperty('keyStoreType') + .intProperty('protocol'); + + if (_.nonEmpty(cluster.sslContextFactory.trustManagers)) { + bean.arrayProperty('trustManagers', 'trustManagers', + _.map(cluster.sslContextFactory.trustManagers, (clsName) => new EmptyBean(clsName)), + 'javax.net.ssl.TrustManager'); + } + else { + bean.pathProperty('trustStoreFilePath'); + + if (_.nonEmpty(bean.valueOf('trustStoreFilePath'))) + bean.propertyChar('trustStorePassword', 'ssl.trust.storage.password', 'YOUR_SSL_TRUST_STORAGE_PASSWORD'); + + bean.intProperty('trustStoreType'); + } + + cfg.beanProperty('sslContextFactory', bean); + } + + return cfg; + } + + // Generate swap group. + static clusterSwap(cluster, cfg = this.igniteConfigurationBean(cluster)) { + if (cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind === 'FileSwapSpaceSpi') { + const bean = new Bean('org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi', 'swapSpaceSpi', + cluster.swapSpaceSpi.FileSwapSpaceSpi); + + bean.pathProperty('baseDirectory') + .intProperty('readStripesNumber') + .floatProperty('maximumSparsity') + .intProperty('maxWriteQueueSize') + .intProperty('writeBufferSize'); + + cfg.beanProperty('swapSpaceSpi', bean); + } + + return cfg; + } + + // Generate time group. + static clusterTime(cluster, cfg = this.igniteConfigurationBean(cluster)) { + cfg.intProperty('clockSyncSamples') + .intProperty('clockSyncFrequency') + .intProperty('timeServerPortBase') + .intProperty('timeServerPortRange'); + + return cfg; + } + + // Generate thread pools group. + static clusterPools(cluster, cfg = this.igniteConfigurationBean(cluster)) { + cfg.intProperty('publicThreadPoolSize') + .intProperty('systemThreadPoolSize') + .intProperty('managementThreadPoolSize') + .intProperty('igfsThreadPoolSize') + .intProperty('rebalanceThreadPoolSize'); + + return cfg; + } + + // Generate transactions group. + static clusterTransactions(transactionConfiguration, cfg = this.igniteConfigurationBean()) { + const bean = new Bean('org.apache.ignite.configuration.TransactionConfiguration', 'transactionConfiguration', + transactionConfiguration, clusterDflts.transactionConfiguration); + + bean.enumProperty('defaultTxConcurrency') + .enumProperty('defaultTxIsolation') + .intProperty('defaultTxTimeout') + .intProperty('pessimisticTxLogLinger') + .intProperty('pessimisticTxLogSize') + .boolProperty('txSerializableEnabled') + .emptyBeanProperty('txManagerFactory'); + + if (bean.nonEmpty()) + cfg.beanProperty('transactionConfiguration', bean); + + return cfg; + } + + // Generate user attributes group. + static clusterUserAttributes(cluster, cfg = this.igniteConfigurationBean(cluster)) { + cfg.mapProperty('attrs', 'attributes', 'userAttributes'); + + return cfg; + } + + // Generate domain model for general group. + static domainModelGeneral(domain, cfg = this.domainConfigurationBean(domain)) { + switch (cfg.valueOf('queryMetadata')) { + case 'Annotations': + if (_.nonNil(domain.keyType) && _.nonNil(domain.valueType)) + cfg.varArgProperty('indexedTypes', 'indexedTypes', [domain.keyType, domain.valueType], 'java.lang.Class'); + + break; + case 'Configuration': + cfg.stringProperty('keyType', 'keyType', (val) => JavaTypes.fullClassName(val)) + .stringProperty('valueType', 'valueType', (val) => JavaTypes.fullClassName(val)); + + break; + default: + } + + return cfg; + } + + // Generate domain model for query group. + static domainModelQuery(domain, cfg = this.domainConfigurationBean(domain)) { + if (cfg.valueOf('queryMetadata') === 'Configuration') { + const fields = _.map(domain.fields, + (e) => ({name: e.name, className: JavaTypes.fullClassName(e.className)})); + + cfg.mapProperty('fields', fields, 'fields', true) + .mapProperty('aliases', 'aliases'); + + const indexes = _.map(domain.indexes, (index) => + new Bean('org.apache.ignite.cache.QueryIndex', 'index', index, cacheDflts.indexes) + .stringProperty('name') + .enumProperty('indexType') + .mapProperty('indFlds', 'fields', 'fields', true) + ); + + cfg.collectionProperty('indexes', 'indexes', indexes, 'org.apache.ignite.cache.QueryIndex'); + } + + return cfg; + } + + // Generate domain model db fields. + static _domainModelDatabaseFields(cfg, propName, domain) { + const fields = _.map(domain[propName], (field) => { + return new Bean('org.apache.ignite.cache.store.jdbc.JdbcTypeField', 'typeField', field, cacheDflts.typeField) + .constantConstructorArgument('databaseFieldType') + .stringConstructorArgument('databaseFieldName') + .classConstructorArgument('javaFieldType') + .stringConstructorArgument('javaFieldName'); + }); + + cfg.varArgProperty(propName, propName, fields, 'org.apache.ignite.cache.store.jdbc.JdbcTypeField'); + + return cfg; + } + + // Generate domain model for store group. + static domainStore(domain, cfg = this.domainConfigurationBean(domain)) { + cfg.stringProperty('databaseSchema') + .stringProperty('databaseTable'); + + this._domainModelDatabaseFields(cfg, 'keyFields', domain); + this._domainModelDatabaseFields(cfg, 'valueFields', domain); + + return cfg; + } + + /** + * Generate eviction policy object. + * @param {Object} ccfg Parent configuration. + * @param {String} name Property name. + * @param {Object} src Source. + * @param {Object} dflt Default. + * @returns {Object} Parent configuration. + * @private + */ + static _evictionPolicy(ccfg, name, src, dflt) { + let bean; + + switch (_.get(src, 'kind')) { + case 'LRU': + bean = new Bean('org.apache.ignite.cache.eviction.lru.LruEvictionPolicy', 'evictionPlc', + src.LRU, dflt.LRU); + + break; + case 'FIFO': + bean = new Bean('org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy', 'evictionPlc', + src.FIFO, dflt.FIFO); + + break; + case 'SORTED': + bean = new Bean('org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy', 'evictionPlc', + src.SORTED, dflt.SORTED); + + break; + default: + return ccfg; + } + + bean.intProperty('batchSize') + .intProperty('maxMemorySize') + .intProperty('maxSize'); + + ccfg.beanProperty(name, bean); + + return ccfg; + } + + // Generate cache general group. + static cacheGeneral(cache, ccfg = this.cacheConfigurationBean(cache)) { + ccfg.stringProperty('name') + .enumProperty('cacheMode') + .enumProperty('atomicityMode'); + + if (ccfg.valueOf('cacheMode') === 'PARTITIONED' && ccfg.valueOf('backups')) { + ccfg.intProperty('backups') + .intProperty('readFromBackup'); + } + + ccfg.intProperty('copyOnRead'); + + if (ccfg.valueOf('cacheMode') === 'PARTITIONED' && ccfg.valueOf('atomicityMode') === 'TRANSACTIONAL') + ccfg.intProperty('invalidate'); + + return ccfg; + } + + // Generate cache memory group. + static cacheMemory(cache, ccfg = this.cacheConfigurationBean(cache)) { + ccfg.enumProperty('memoryMode'); + + if (ccfg.valueOf('memoryMode') !== 'OFFHEAP_VALUES') + ccfg.intProperty('offHeapMaxMemory'); + + this._evictionPolicy(ccfg, 'evictionPolicy', cache.evictionPolicy, cacheDflts.evictionPolicy); + + ccfg.intProperty('startSize') + .boolProperty('swapEnabled'); + + return ccfg; + } + + // Generate cache queries & Indexing group. + static cacheQuery(cache, domains, ccfg = this.cacheConfigurationBean(cache)) { + const indexedTypes = _.reduce(domains, (acc, domain) => { + if (domain.queryMetadata === 'Annotations') + acc.push(domain.keyType, domain.valueType); + + return acc; + }, []); + + ccfg.stringProperty('sqlSchema') + .intProperty('sqlOnheapRowCacheSize') + .intProperty('longQueryWarningTimeout') + .arrayProperty('indexedTypes', 'indexedTypes', indexedTypes, 'java.lang.Class') + .arrayProperty('sqlFunctionClasses', 'sqlFunctionClasses', cache.sqlFunctionClasses, 'java.lang.Class') + .intProperty('snapshotableIndex') + .intProperty('sqlEscapeAll'); + + return ccfg; + } + + // Generate cache store group. + static cacheStore(cache, domains, ccfg = this.cacheConfigurationBean(cache)) { + const kind = _.get(cache, 'cacheStoreFactory.kind'); + + if (kind && cache.cacheStoreFactory[kind]) { + let bean = null; + + const storeFactory = cache.cacheStoreFactory[kind]; + + switch (kind) { + case 'CacheJdbcPojoStoreFactory': + bean = new Bean('org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory', 'cacheStoreFactory', + storeFactory); + + const jdbcId = bean.valueOf('dataSourceBean'); + + bean.dataSource(jdbcId, 'dataSourceBean', this.dataSourceBean(jdbcId, storeFactory.dialect)) + .beanProperty('dialect', new EmptyBean(this.dialectClsName(storeFactory.dialect))); + + bean.boolProperty('sqlEscapeAll'); + + const setType = (typeBean, propName) => { + if (JavaTypes.nonBuiltInClass(typeBean.valueOf(propName))) + typeBean.stringProperty(propName); + else + typeBean.classProperty(propName); + }; + + const types = _.reduce(domains, (acc, domain) => { + if (_.isNil(domain.databaseTable)) + return acc; + + const typeBean = new Bean('org.apache.ignite.cache.store.jdbc.JdbcType', 'type', + _.merge({}, domain, {cacheName: cache.name})) + .stringProperty('cacheName'); + + setType(typeBean, 'keyType'); + setType(typeBean, 'valueType'); + + this.domainStore(domain, typeBean); + + acc.push(typeBean); + + return acc; + }, []); + + bean.varArgProperty('types', 'types', types, 'org.apache.ignite.cache.store.jdbc.JdbcType'); + + break; + case 'CacheJdbcBlobStoreFactory': + bean = new Bean('org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory', 'cacheStoreFactory', + storeFactory); + + if (bean.valueOf('connectVia') === 'DataSource') { + const blobId = bean.valueOf('dataSourceBean'); + + bean.dataSource(blobId, 'dataSourceBean', this.dataSourceBean(blobId, storeFactory.dialect)); + } + else { + ccfg.stringProperty('connectionUrl') + .stringProperty('user') + .property('password', `ds.${storeFactory.user}.password`, 'YOUR_PASSWORD'); + } + + bean.boolProperty('initSchema') + .stringProperty('createTableQuery') + .stringProperty('loadQuery') + .stringProperty('insertQuery') + .stringProperty('updateQuery') + .stringProperty('deleteQuery'); + + break; + case 'CacheHibernateBlobStoreFactory': + bean = new Bean('org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory', + 'cacheStoreFactory', storeFactory); + + bean.propsProperty('props', 'hibernateProperties'); + + break; + default: + } + + if (bean) + ccfg.beanProperty('cacheStoreFactory', bean); + } + + ccfg.boolProperty('storeKeepBinary') + .boolProperty('loadPreviousValue') + .boolProperty('readThrough') + .boolProperty('writeThrough'); + + if (ccfg.valueOf('writeBehindEnabled')) { + ccfg.boolProperty('writeBehindEnabled') + .intProperty('writeBehindBatchSize') + .intProperty('writeBehindFlushSize') + .intProperty('writeBehindFlushFrequency') + .intProperty('writeBehindFlushThreadCount'); + } + + return ccfg; + } + + // Generate cache concurrency control group. + static cacheConcurrency(cache, ccfg = this.cacheConfigurationBean(cache)) { + ccfg.intProperty('maxConcurrentAsyncOperations') + .intProperty('defaultLockTimeout') + .enumProperty('atomicWriteOrderMode') + .enumProperty('writeSynchronizationMode'); + + return ccfg; + } + + // Generate cache node filter group. + static cacheNodeFilter(cache, igfss, ccfg = this.cacheConfigurationBean(cache)) { + const kind = _.get(cache, 'nodeFilter.kind'); + + if (kind && cache.nodeFilter[kind]) { + let bean = null; + + switch (kind) { + case 'IGFS': + const foundIgfs = _.find(igfss, (igfs) => igfs._id === cache.nodeFilter.IGFS.igfs); + + if (foundIgfs) { + bean = new Bean('org.apache.ignite.internal.processors.igfs.IgfsNodePredicate', 'nodeFilter', foundIgfs) + .stringConstructorArgument('name'); + } + + break; + case 'Custom': + bean = new Bean(cache.nodeFilter.Custom.className, 'nodeFilter'); + + break; + default: + return ccfg; + } + + if (bean) + ccfg.beanProperty('nodeFilter', bean); + } + + return ccfg; + } + + // Generate cache rebalance group. + static cacheRebalance(cache, ccfg = this.cacheConfigurationBean(cache)) { + if (ccfg.valueOf('cacheMode') !== 'LOCAL') { + ccfg.enumProperty('rebalanceMode') + .intProperty('rebalanceThreadPoolSize') + .intProperty('rebalanceBatchSize') + .intProperty('rebalanceBatchesPrefetchCount') + .intProperty('rebalanceOrder') + .intProperty('rebalanceDelay') + .intProperty('rebalanceTimeout') + .intProperty('rebalanceThrottle'); + } + + if (ccfg.includes('igfsAffinnityGroupSize')) { + const bean = new Bean('org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper', 'affinityMapper', cache) + .intConstructorArgument('igfsAffinnityGroupSize'); + + ccfg.beanProperty('affinityMapper', bean); + } + + return ccfg; + } + + // Generate server near cache group. + static cacheNearServer(cache, ccfg = this.cacheConfigurationBean(cache)) { + if (ccfg.valueOf('cacheMode') === 'PARTITIONED' && _.get(cache, 'nearConfiguration.enabled')) { + const bean = new Bean('org.apache.ignite.configuration.NearCacheConfiguration', 'nearConfiguration', + cache.nearConfiguration, cacheDflts.nearConfiguration); + + bean.intProperty('nearStartSize'); + + this._evictionPolicy(bean, 'nearEvictionPolicy', + bean.valueOf('nearEvictionPolicy'), cacheDflts.evictionPolicy); + + ccfg.beanProperty('nearConfiguration', bean); + } + + return ccfg; + } + + // Generate client near cache group. + static cacheNearClient(cache, ccfg = this.cacheConfigurationBean(cache)) { + if (ccfg.valueOf('cacheMode') === 'PARTITIONED' && _.get(cache, 'clientNearConfiguration.enabled')) { + const bean = new Bean('org.apache.ignite.configuration.NearCacheConfiguration', + JavaTypes.toJavaName('nearConfiguration', ccfg.valueOf('name')), + cache.clientNearConfiguration, cacheDflts.clientNearConfiguration); + + bean.intProperty('nearStartSize'); + + this._evictionPolicy(bean, 'nearEvictionPolicy', + bean.valueOf('nearEvictionPolicy'), cacheDflts.evictionPolicy); + + return bean; + } + + return ccfg; + } + + // Generate cache statistics group. + static cacheStatistics(cache, ccfg = this.cacheConfigurationBean(cache)) { + ccfg.boolProperty('statisticsEnabled') + .boolProperty('managementEnabled'); + + return ccfg; + } + + // Generate domain models configs. + static cacheDomains(domains, ccfg) { + const qryEntities = _.reduce(domains, (acc, domain) => { + if (_.isNil(domain.queryMetadata) || domain.queryMetadata === 'Configuration') { + const qryEntity = this.domainModelGeneral(domain); + + this.domainModelQuery(domain, qryEntity); + + acc.push(qryEntity); + } + + return acc; + }, []); + + ccfg.collectionProperty('qryEntities', 'queryEntities', qryEntities, 'org.apache.ignite.cache.QueryEntity'); + } + + static cacheConfiguration(cache, ccfg = this.cacheConfigurationBean(cache)) { + this.cacheGeneral(cache, ccfg); + this.cacheMemory(cache, ccfg); + this.cacheQuery(cache, cache.domains, ccfg); + this.cacheStore(cache, cache.domains, ccfg); + + const igfs = _.get(cache, 'nodeFilter.IGFS.instance'); + this.cacheNodeFilter(cache, igfs ? [igfs] : [], ccfg); + this.cacheConcurrency(cache, ccfg); + this.cacheRebalance(cache, ccfg); + this.cacheNearServer(cache, ccfg); + this.cacheStatistics(cache, ccfg); + this.cacheDomains(cache.domains, ccfg); + + return ccfg; + } + + // Generate IGFS general group. + static igfsGeneral(igfs, cfg = this.igfsConfigurationBean(igfs)) { + if (_.isEmpty(igfs.name)) + return cfg; + + cfg.stringProperty('name') + .stringProperty('name', 'dataCacheName', (name) => name + '-data') + .stringProperty('name', 'metaCacheName', (name) => name + '-meta') + .enumProperty('defaultMode'); + + return cfg; + } + + // Generate IGFS secondary file system group. + static igfsSecondFS(igfs, cfg = this.igfsConfigurationBean(igfs)) { + if (igfs.secondaryFileSystemEnabled) { + const secondFs = igfs.secondaryFileSystem || {}; + + const bean = new Bean('org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem', + 'secondaryFileSystem', secondFs, igfsDflts.secondaryFileSystem); + + bean.stringProperty('userName', 'defaultUserName'); + + const factoryBean = new Bean('org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory', + 'fac', secondFs); + + factoryBean.stringProperty('uri') + .pathProperty('cfgPath', 'configPaths'); + + bean.beanProperty('fileSystemFactory', factoryBean); + + cfg.beanProperty('secondaryFileSystem', bean); + } + + return cfg; + } + + // Generate IGFS IPC group. + static igfsIPC(igfs, cfg = this.igfsConfigurationBean(igfs)) { + if (igfs.ipcEndpointEnabled) { + const bean = new Bean('org.apache.ignite.igfs.IgfsIpcEndpointConfiguration', 'ipcEndpointConfiguration', + igfs.ipcEndpointConfiguration, igfsDflts.ipcEndpointConfiguration); + + bean.enumProperty('type') + .stringProperty('host') + .intProperty('port') + .intProperty('memorySize') + .pathProperty('tokenDirectoryPath') + .intProperty('threadCount'); + + cfg.beanProperty('ipcEndpointConfiguration', bean); + } + + return cfg; + } + + // Generate IGFS fragmentizer group. + static igfsFragmentizer(igfs, cfg = this.igfsConfigurationBean(igfs)) { + if (igfs.fragmentizerEnabled) { + cfg.intProperty('fragmentizerConcurrentFiles') + .intProperty('fragmentizerThrottlingBlockLength') + .intProperty('fragmentizerThrottlingDelay'); + } + else + cfg.boolProperty('fragmentizerEnabled'); + + return cfg; + } + + // Generate IGFS Dual mode group. + static igfsDualMode(igfs, cfg = this.igfsConfigurationBean(igfs)) { + cfg.intProperty('dualModeMaxPendingPutsSize') + .emptyBeanProperty('dualModePutExecutorService') + .intProperty('dualModePutExecutorServiceShutdown'); + + return cfg; + } + + // Generate IGFS miscellaneous group. + static igfsMisc(igfs, cfg = this.igfsConfigurationBean(igfs)) { + cfg.intProperty('blockSize') + .intProperty('streamBufferSize') + .intProperty('maxSpaceSize') + .intProperty('maximumTaskRangeLength') + .intProperty('managementPort') + .intProperty('perNodeBatchSize') + .intProperty('perNodeParallelBatchCount') + .intProperty('prefetchBlocks') + .intProperty('sequentialReadsBeforePrefetch') + .intProperty('trashPurgeTimeout') + .intProperty('colocateMetadata') + .intProperty('relaxedConsistency') + .mapProperty('pathModes', 'pathModes'); + + return cfg; + } + } + + return ConfigurationGenerator; +}]; diff --git a/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js new file mode 100644 index 0000000000000..64d43d8ff8091 --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js @@ -0,0 +1,1721 @@ +/* + * 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. + */ + +import AbstractTransformer from './AbstractTransformer'; +import StringBuilder from './StringBuilder'; + +const STORE_FACTORY = ['org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory']; + +// Descriptors for generation of demo data. +const PREDEFINED_QUERIES = [ + { + schema: 'CARS', + type: 'PARKING', + create: [ + 'CREATE TABLE IF NOT EXISTS CARS.PARKING (', + 'ID INTEGER NOT NULL PRIMARY KEY,', + 'NAME VARCHAR(50) NOT NULL,', + 'CAPACITY INTEGER NOT NULL)' + ], + clearQuery: ['DELETE FROM CARS.PARKING'], + insertCntConsts: [{name: 'DEMO_MAX_PARKING_CNT', val: 5, comment: 'How many parkings to generate.'}], + insertPattern: ['INSERT INTO CARS.PARKING(ID, NAME, CAPACITY) VALUES(?, ?, ?)'], + fillInsertParameters(sb) { + sb.append('stmt.setInt(1, id);'); + sb.append('stmt.setString(2, "Parking #" + (id + 1));'); + sb.append('stmt.setInt(3, 10 + rnd.nextInt(20));'); + }, + selectQuery: ['SELECT * FROM PARKING WHERE CAPACITY >= 20'] + }, + { + schema: 'CARS', + type: 'CAR', + create: [ + 'CREATE TABLE IF NOT EXISTS CARS.CAR (', + 'ID INTEGER NOT NULL PRIMARY KEY,', + 'PARKING_ID INTEGER NOT NULL,', + 'NAME VARCHAR(50) NOT NULL);' + ], + clearQuery: ['DELETE FROM CARS.CAR'], + rndRequired: true, + insertCntConsts: [ + {name: 'DEMO_MAX_CAR_CNT', val: 10, comment: 'How many cars to generate.'}, + {name: 'DEMO_MAX_PARKING_CNT', val: 5, comment: 'How many parkings to generate.'} + ], + insertPattern: ['INSERT INTO CARS.CAR(ID, PARKING_ID, NAME) VALUES(?, ?, ?)'], + fillInsertParameters(sb) { + sb.append('stmt.setInt(1, id);'); + sb.append('stmt.setInt(2, rnd.nextInt(DEMO_MAX_PARKING_CNT));'); + sb.append('stmt.setString(3, "Car #" + (id + 1));'); + }, + selectQuery: ['SELECT * FROM CAR WHERE PARKINGID = 2'] + }, + { + type: 'COUNTRY', + create: [ + 'CREATE TABLE IF NOT EXISTS COUNTRY (', + 'ID INTEGER NOT NULL PRIMARY KEY,', + 'NAME VARCHAR(50),', + 'POPULATION INTEGER NOT NULL);' + ], + clearQuery: ['DELETE FROM COUNTRY'], + insertCntConsts: [{name: 'DEMO_MAX_COUNTRY_CNT', val: 5, comment: 'How many countries to generate.'}], + insertPattern: ['INSERT INTO COUNTRY(ID, NAME, POPULATION) VALUES(?, ?, ?)'], + fillInsertParameters(sb) { + sb.append('stmt.setInt(1, id);'); + sb.append('stmt.setString(2, "Country #" + (id + 1));'); + sb.append('stmt.setInt(3, 10000000 + rnd.nextInt(100000000));'); + }, + selectQuery: ['SELECT * FROM COUNTRY WHERE POPULATION BETWEEN 15000000 AND 25000000'] + }, + { + type: 'DEPARTMENT', + create: [ + 'CREATE TABLE IF NOT EXISTS DEPARTMENT (', + 'ID INTEGER NOT NULL PRIMARY KEY,', + 'COUNTRY_ID INTEGER NOT NULL,', + 'NAME VARCHAR(50) NOT NULL);' + ], + clearQuery: ['DELETE FROM DEPARTMENT'], + rndRequired: true, + insertCntConsts: [ + {name: 'DEMO_MAX_DEPARTMENT_CNT', val: 5, comment: 'How many departments to generate.'}, + {name: 'DEMO_MAX_COUNTRY_CNT', val: 5, comment: 'How many countries to generate.'} + ], + insertPattern: ['INSERT INTO DEPARTMENT(ID, COUNTRY_ID, NAME) VALUES(?, ?, ?)'], + fillInsertParameters(sb) { + sb.append('stmt.setInt(1, id);'); + sb.append('stmt.setInt(2, rnd.nextInt(DEMO_MAX_COUNTRY_CNT));'); + sb.append('stmt.setString(3, "Department #" + (id + 1));'); + }, + selectQuery: ['SELECT * FROM DEPARTMENT'] + }, + { + type: 'EMPLOYEE', + create: [ + 'CREATE TABLE IF NOT EXISTS EMPLOYEE (', + 'ID INTEGER NOT NULL PRIMARY KEY,', + 'DEPARTMENT_ID INTEGER NOT NULL,', + 'MANAGER_ID INTEGER,', + 'FIRST_NAME VARCHAR(50) NOT NULL,', + 'LAST_NAME VARCHAR(50) NOT NULL,', + 'EMAIL VARCHAR(50) NOT NULL,', + 'PHONE_NUMBER VARCHAR(50),', + 'HIRE_DATE DATE NOT NULL,', + 'JOB VARCHAR(50) NOT NULL,', + 'SALARY DOUBLE);' + ], + clearQuery: ['DELETE FROM EMPLOYEE'], + rndRequired: true, + insertCntConsts: [ + {name: 'DEMO_MAX_EMPLOYEE_CNT', val: 10, comment: 'How many employees to generate.'}, + {name: 'DEMO_MAX_DEPARTMENT_CNT', val: 5, comment: 'How many departments to generate.'} + ], + customGeneration(sb, conVar, stmtVar) { + sb.append(`${stmtVar} = ${conVar}.prepareStatement("INSERT INTO EMPLOYEE(ID, DEPARTMENT_ID, MANAGER_ID, FIRST_NAME, LAST_NAME, EMAIL, PHONE_NUMBER, HIRE_DATE, JOB, SALARY) VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?)");`); + + sb.emptyLine(); + + sb.startBlock('for (int id = 0; id < DEMO_MAX_EMPLOYEE_CNT; id ++) {'); + + sb.append('int depId = rnd.nextInt(DEMO_MAX_DEPARTMENT_CNT);'); + + sb.emptyLine(); + + sb.append('stmt.setInt(1, DEMO_MAX_DEPARTMENT_CNT + id);'); + sb.append('stmt.setInt(2, depId);'); + sb.append('stmt.setInt(3, depId);'); + sb.append('stmt.setString(4, "First name manager #" + (id + 1));'); + sb.append('stmt.setString(5, "Last name manager#" + (id + 1));'); + sb.append('stmt.setString(6, "Email manager#" + (id + 1));'); + sb.append('stmt.setString(7, "Phone number manager#" + (id + 1));'); + sb.append('stmt.setString(8, "2014-01-01");'); + sb.append('stmt.setString(9, "Job manager #" + (id + 1));'); + sb.append('stmt.setDouble(10, 600.0 + rnd.nextInt(300));'); + + sb.emptyLine(); + + sb.append('stmt.executeUpdate();'); + + sb.endBlock('}'); + }, + selectQuery: ['SELECT * FROM EMPLOYEE WHERE SALARY > 700'] + } +]; + +// Var name generator function. +const beenNameSeed = () => { + let idx = ''; + const names = []; + + return (bean) => { + let name; + + while (_.includes(names, name = `${bean.id}${idx ? '_' + idx : idx}`)) + idx++; + + names.push(name); + + return name; + }; +}; + +export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator', (JavaTypes, eventGroups, generator) => { + class JavaTransformer extends AbstractTransformer { + static generator = generator; + + // Mapping for objects to method call. + static METHOD_MAPPING = { + 'org.apache.ignite.configuration.CacheConfiguration': { + id: (ccfg) => JavaTypes.toJavaName('cache', ccfg.findProperty('name').value), + args: '', + generator: (sb, id, ccfg) => { + const cacheName = ccfg.findProperty('name').value; + const dataSources = JavaTransformer.collectDataSources(ccfg); + + const javadoc = [ + `Create configuration for cache "${cacheName}".`, + '', + '@return Configured cache.' + ]; + + if (dataSources.length) + javadoc.push('@throws Exception if failed to create cache configuration.'); + + JavaTransformer.commentBlock(sb, ...javadoc); + sb.startBlock(`public static CacheConfiguration ${id}()${dataSources.length ? ' throws Exception' : ''} {`); + + JavaTransformer.constructBean(sb, ccfg, [], true); + + sb.emptyLine(); + sb.append(`return ${ccfg.id};`); + + sb.endBlock('}'); + + return sb; + } + }, + 'org.apache.ignite.cache.store.jdbc.JdbcType': { + id: (type) => JavaTypes.toJavaName('jdbcType', JavaTypes.shortClassName(type.findProperty('valueType').value)), + args: 'ccfg.getName()', + generator: (sb, name, jdbcType) => { + const javadoc = [ + `Create JDBC type for "${name}".`, + '', + '@param cacheName Cache name.', + '@return Configured JDBC type.' + ]; + + JavaTransformer.commentBlock(sb, ...javadoc); + sb.startBlock(`private static JdbcType ${name}(String cacheName) {`); + + const cacheName = jdbcType.findProperty('cacheName'); + + cacheName.clsName = 'var'; + cacheName.value = 'cacheName'; + + JavaTransformer.constructBean(sb, jdbcType); + + sb.emptyLine(); + sb.append(`return ${jdbcType.id};`); + + sb.endBlock('}'); + + return sb; + } + } + }; + + // Append comment line. + static comment(sb, ...lines) { + _.forEach(lines, (line) => sb.append(`// ${line}`)); + } + + // Append comment block. + static commentBlock(sb, ...lines) { + if (lines.length === 1) + sb.append(`/** ${_.head(lines)} **/`); + else { + sb.append('/**'); + + _.forEach(lines, (line) => sb.append(` * ${line}`)); + + sb.append(' **/'); + } + } + + /** + * @param {Bean} bean + */ + static _newBean(bean) { + const shortClsName = JavaTypes.shortClassName(bean.clsName); + + if (_.isEmpty(bean.arguments)) + return `new ${shortClsName}()`; + + const args = _.map(bean.arguments, (arg) => { + switch (arg.clsName) { + case 'MAP': + return arg.id; + case 'BEAN': + return this._newBean(arg.value); + default: + return this._toObject(arg.clsName, arg.value); + } + }); + + if (bean.factoryMtd) + return `${shortClsName}.${bean.factoryMtd}(${args.join(', ')})`; + + return `new ${shortClsName}(${args.join(', ')})`; + } + + /** + * @param {StringBuilder} sb + * @param {String} parentId + * @param {String} propertyName + * @param {String} value + * @private + */ + static _setProperty(sb, parentId, propertyName, value) { + sb.append(`${parentId}.set${_.upperFirst(propertyName)}(${value});`); + } + + /** + * @param {StringBuilder} sb + * @param {Array.} vars + * @param {Boolean} limitLines + * @param {Bean} bean + * @param {String} id + + * @private + */ + static constructBean(sb, bean, vars = [], limitLines = false, id = bean.id) { + _.forEach(bean.arguments, (arg) => { + switch (arg.clsName) { + case 'MAP': + this._constructMap(sb, arg, vars); + + sb.emptyLine(); + + break; + default: + if (this._isBean(arg.clsName) && arg.value.isComplex()) { + this.constructBean(sb, arg.value, vars, limitLines); + + sb.emptyLine(); + } + } + }); + + const clsName = JavaTypes.shortClassName(bean.clsName); + + sb.append(`${this.varInit(clsName, id, vars)} = ${this._newBean(bean)};`); + + if (_.nonEmpty(bean.properties)) { + sb.emptyLine(); + + this._setProperties(sb, bean, vars, limitLines, id); + } + } + + /** + * @param {StringBuilder} sb + * @param {Bean} bean + * @param {Array.} vars + * @param {Boolean} limitLines + * @private + */ + static constructStoreFactory(sb, bean, vars, limitLines = false) { + const shortClsName = JavaTypes.shortClassName(bean.clsName); + + if (_.includes(vars, bean.id)) + sb.append(`${bean.id} = ${this._newBean(bean)};`); + else { + vars.push(bean.id); + + sb.append(`${shortClsName} ${bean.id} = ${this._newBean(bean)};`); + } + + sb.emptyLine(); + + sb.startBlock(`${bean.id}.setDataSourceFactory(new Factory() {`); + this.commentBlock(sb, '{@inheritDoc}'); + sb.startBlock('@Override public DataSource create() {'); + + sb.append(`return DataSources.INSTANCE_${bean.findProperty('dataSourceBean').id};`); + + sb.endBlock('};'); + sb.endBlock('});'); + + const storeFactory = _.cloneDeep(bean); + + _.remove(storeFactory.properties, (p) => _.includes(['dataSourceBean'], p.name)); + + if (storeFactory.properties.length) { + sb.emptyLine(); + + this._setProperties(sb, storeFactory, vars, limitLines); + } + } + + static _isBean(clsName) { + return JavaTypes.nonBuiltInClass(clsName) && JavaTypes.nonEnum(clsName) && _.includes(clsName, '.'); + } + + static _toObject(clsName, val) { + const items = _.isArray(val) ? val : [val]; + + return _.map(items, (item) => { + if (_.isNil(item)) + return 'null'; + + switch (clsName) { + case 'var': + return item; + case 'byte': + return `(byte) ${item}`; + case 'float': + return `${item}f`; + case 'long': + return `${item}L`; + case 'java.io.Serializable': + case 'java.lang.String': + return `"${item}"`; + case 'PATH': + return `"${item.replace(/\\/g, '\\\\')}"`; + case 'java.lang.Class': + return `${JavaTypes.shortClassName(item)}.class`; + case 'java.util.UUID': + return `UUID.fromString("${item}")`; + case 'PROPERTY_CHAR': + return `props.getProperty("${item}").toCharArray()`; + case 'PROPERTY': + return `props.getProperty("${item}")`; + default: + if (this._isBean(clsName)) { + if (item.isComplex()) + return item.id; + + return this._newBean(item); + } + + if (JavaTypes.nonEnum(clsName)) + return item; + + return `${JavaTypes.shortClassName(clsName)}.${item}`; + } + }); + } + + static _constructBeans(sb, type, items, vars, limitLines) { + if (this._isBean(type)) { + // Construct objects inline for preview or simple objects. + const mapper = this.METHOD_MAPPING[type]; + + const nextId = mapper ? mapper.id : beenNameSeed(); + + // Prepare objects refs. + return _.map(items, (item) => { + if (limitLines && mapper) + return mapper.id(item) + (limitLines ? `(${mapper.args})` : ''); + + if (item.isComplex()) { + const id = nextId(item); + + this.constructBean(sb, item, vars, limitLines, id); + + sb.emptyLine(); + + return id; + } + + return this._newBean(item); + }); + } + + return this._toObject(type, items); + } + + /** + * + * @param sb + * @param parentId + * @param arrProp + * @param vars + * @param limitLines + * @private + */ + static _setVarArg(sb, parentId, arrProp, vars, limitLines) { + const refs = this._constructBeans(sb, arrProp.typeClsName, arrProp.items, vars, limitLines); + + // Set refs to property. + if (refs.length === 1) + this._setProperty(sb, parentId, arrProp.name, _.head(refs)); + else { + sb.startBlock(`${parentId}.set${_.upperFirst(arrProp.name)}(`); + + const lastIdx = refs.length - 1; + + _.forEach(refs, (ref, idx) => { + sb.append(ref + (lastIdx !== idx ? ',' : '')); + }); + + sb.endBlock(');'); + } + } + + /** + * + * @param sb + * @param parentId + * @param arrProp + * @param vars + * @param limitLines + * @private + */ + static _setArray(sb, parentId, arrProp, vars, limitLines) { + const refs = this._constructBeans(sb, arrProp.typeClsName, arrProp.items, vars, limitLines); + + const arrType = JavaTypes.shortClassName(arrProp.typeClsName); + + // Set refs to property. + sb.startBlock(`${parentId}.set${_.upperFirst(arrProp.name)}(new ${arrType}[] {`); + + const lastIdx = refs.length - 1; + + _.forEach(refs, (ref, idx) => sb.append(ref + (lastIdx !== idx ? ',' : ''))); + + sb.endBlock('});'); + } + + static _constructMap(sb, map, vars = []) { + const keyClsName = JavaTypes.shortClassName(map.keyClsName); + const valClsName = JavaTypes.shortClassName(map.valClsName); + + const mapClsName = map.ordered ? 'LinkedHashMap' : 'HashMap'; + + const type = `${mapClsName}<${keyClsName}, ${valClsName}>`; + + sb.append(`${this.varInit(type, map.id, vars)} = new ${mapClsName}<>();`); + + sb.emptyLine(); + + _.forEach(map.entries, (entry) => { + const key = this._toObject(map.keyClsName, entry[map.keyField]); + const val = entry[map.valField]; + + if (_.isArray(val) && map.valClsName === 'java.lang.String') { + if (val.length > 1) { + sb.startBlock(`${map.id}.put(${key},`); + + _.forEach(val, (line, idx) => { + sb.append(`"${line}"${idx !== val.length - 1 ? ' +' : ''}`); + }); + + sb.endBlock(');'); + } + else + sb.append(`${map.id}.put(${key}, ${this._toObject(map.valClsName, _.head(val))});`); + } + else + sb.append(`${map.id}.put(${key}, ${this._toObject(map.valClsName, val)});`); + }); + } + + static varInit(type, id, vars) { + if (_.includes(vars, id)) + return id; + + vars.push(id); + + return `${type} ${id}`; + } + + /** + * + * @param {StringBuilder} sb + * @param {Bean} bean + * @param {String} id + * @param {Array.} vars + * @param {Boolean} limitLines + * @returns {StringBuilder} + */ + static _setProperties(sb = new StringBuilder(), bean, vars = [], limitLines = false, id = bean.id) { + _.forEach(bean.properties, (prop, idx) => { + switch (prop.clsName) { + case 'DATA_SOURCE': + this._setProperty(sb, id, 'dataSource', `DataSources.INSTANCE_${prop.id}`); + + break; + case 'EVENT_TYPES': + if (prop.eventTypes.length === 1) + this._setProperty(sb, id, prop.name, _.head(prop.eventTypes)); + else { + sb.append(`int[] ${prop.id} = new int[${_.head(prop.eventTypes)}.length`); + + _.forEach(_.tail(prop.eventTypes), (evtGrp) => { + sb.append(` + ${evtGrp}.length`); + }); + + sb.append('];'); + + sb.emptyLine(); + + sb.append('int k = 0;'); + + _.forEach(prop.eventTypes, (evtGrp, evtIdx) => { + sb.emptyLine(); + + sb.append(`System.arraycopy(${evtGrp}, 0, ${prop.id}, k, ${evtGrp}.length);`); + + if (evtIdx < prop.eventTypes.length - 1) + sb.append(`k += ${evtGrp}.length;`); + }); + + sb.emptyLine(); + + sb.append(`cfg.setIncludeEventTypes(${prop.id});`); + } + + break; + case 'ARRAY': + if (prop.varArg) + this._setVarArg(sb, id, prop, vars, limitLines); + else + this._setArray(sb, id, prop, vars, limitLines); + + break; + case 'COLLECTION': + const nonBean = !this._isBean(prop.typeClsName); + + if (nonBean && prop.implClsName === 'java.util.ArrayList') { + const items = _.map(prop.items, (item) => this._toObject(prop.typeClsName, item)); + + if (items.length > 1) { + sb.startBlock(`${id}.set${_.upperFirst(prop.name)}(Arrays.asList(`); + + _.forEach(items, (item, i) => sb.append(item + (i !== items.length - 1 ? ',' : ''))); + + sb.endBlock('));'); + } + else + this._setProperty(sb, id, prop.name, `Arrays.asList(${items})`); + } + else { + const colTypeClsName = JavaTypes.shortClassName(prop.typeClsName); + const implClsName = JavaTypes.shortClassName(prop.implClsName); + + sb.append(`${this.varInit(`${implClsName}<${colTypeClsName}>`, prop.id, vars)} = new ${implClsName}<>();`); + + sb.emptyLine(); + + if (nonBean) { + _.forEach(this._toObject(colTypeClsName, prop.items), (item) => { + sb.append(`${prop.id}.add("${item}");`); + + sb.emptyLine(); + }); + } + else { + _.forEach(prop.items, (item) => { + this.constructBean(sb, item, vars, limitLines); + + sb.append(`${prop.id}.add(${item.id});`); + + sb.emptyLine(); + }); + } + + this._setProperty(sb, id, prop.name, prop.id); + } + + break; + case 'MAP': + this._constructMap(sb, prop, vars); + + if (_.nonEmpty(prop.entries)) + sb.emptyLine(); + + this._setProperty(sb, id, prop.name, prop.id); + + break; + case 'java.util.Properties': + sb.append(`${this.varInit('Properties', prop.id, vars)} = new Properties();`); + + if (_.nonEmpty(prop.entries)) + sb.emptyLine(); + + _.forEach(prop.entries, (entry) => { + const key = this._toObject('java.lang.String', entry.name); + const val = this._toObject('java.lang.String', entry.value); + + sb.append(`${prop.id}.setProperty(${key}, ${val});`); + }); + + sb.emptyLine(); + + this._setProperty(sb, id, prop.name, prop.id); + + break; + case 'BEAN': + const embedded = prop.value; + + if (_.includes(STORE_FACTORY, embedded.clsName)) { + this.constructStoreFactory(sb, embedded, vars, limitLines); + + sb.emptyLine(); + + this._setProperty(sb, id, prop.name, embedded.id); + } + else if (embedded.isComplex()) { + this.constructBean(sb, embedded, vars, limitLines); + + sb.emptyLine(); + + this._setProperty(sb, id, prop.name, embedded.id); + } + else + this._setProperty(sb, id, prop.name, this._newBean(embedded)); + + break; + default: + this._setProperty(sb, id, prop.name, this._toObject(prop.clsName, prop.value)); + } + + this._emptyLineIfNeeded(sb, bean.properties, idx); + }); + + return sb; + } + + static collectBeanImports(bean) { + const imports = [bean.clsName]; + + _.forEach(bean.arguments, (arg) => { + switch (arg.clsName) { + case 'BEAN': + imports.push(...this.collectPropertiesImports(arg.value.properties)); + + break; + case 'java.lang.Class': + imports.push(JavaTypes.fullClassName(arg.value)); + + break; + default: + imports.push(arg.clsName); + } + }); + + imports.push(...this.collectPropertiesImports(bean.properties)); + + if (_.includes(STORE_FACTORY, bean.clsName)) + imports.push('javax.sql.DataSource', 'javax.cache.configuration.Factory'); + + return imports; + } + + /** + * @param {Array.} props + * @returns {Array.} + */ + static collectPropertiesImports(props) { + const imports = []; + + _.forEach(props, (prop) => { + switch (prop.clsName) { + case 'DATA_SOURCE': + imports.push(prop.value.clsName); + + break; + case 'PROPERTY': + case 'PROPERTY_CHAR': + imports.push('java.io.InputStream', 'java.util.Properties'); + + break; + case 'BEAN': + imports.push(...this.collectBeanImports(prop.value)); + + break; + case 'ARRAY': + imports.push(prop.typeClsName); + + if (this._isBean(prop.typeClsName)) + _.forEach(prop.items, (item) => imports.push(...this.collectBeanImports(item))); + + break; + case 'COLLECTION': + imports.push(prop.typeClsName); + + if (this._isBean(prop.typeClsName)) { + _.forEach(prop.items, (item) => imports.push(...this.collectBeanImports(item))); + + imports.push(prop.implClsName); + } + else if (prop.implClsName === 'java.util.ArrayList') + imports.push('java.util.Arrays'); + else + imports.push(prop.implClsName); + + break; + case 'MAP': + imports.push(prop.ordered ? 'java.util.LinkedHashMap' : 'java.util.HashMap'); + imports.push(prop.keyClsName); + imports.push(prop.valClsName); + + break; + default: + if (!JavaTypes.nonEnum(prop.clsName)) + imports.push(prop.clsName); + } + }); + + return imports; + } + + static _prepareImports(imports) { + return _.sortedUniq(_.sortBy(_.filter(imports, (cls) => !cls.startsWith('java.lang.') && _.includes(cls, '.')))); + } + + /** + * @param {Bean} bean + * @returns {Array.} + */ + static collectStaticImports(bean) { + const imports = []; + + _.forEach(bean.properties, (prop) => { + switch (prop.clsName) { + case 'EVENT_TYPES': + _.forEach(prop.eventTypes, (value) => { + const evtGrp = _.find(eventGroups, {value}); + + imports.push(`${evtGrp.class}.${evtGrp.value}`); + }); + + break; + default: + // No-op. + } + }); + + return imports; + } + + /** + * @param {Bean} bean + * @returns {Object} + */ + static collectBeansWithMapping(bean) { + const beans = {}; + + _.forEach(bean.properties, (prop) => { + switch (prop.clsName) { + case 'BEAN': + _.merge(beans, this.collectBeansWithMapping(prop.value)); + + break; + case 'ARRAY': + if (this._isBean(prop.typeClsName)) { + const mapping = this.METHOD_MAPPING[prop.typeClsName]; + + _.reduce(prop.items, (acc, item) => { + if (mapping) { + acc[mapping.id(item)] = item; + + _.merge(acc, this.collectBeansWithMapping(item)); + } + return acc; + }, beans); + } + + break; + default: + // No-op. + } + }); + + return beans; + } + + /** + * Build Java startup class with configuration. + * + * @param {Bean} cfg + * @param pkg Package name. + * @param {String} clsName Class name for generate factory class otherwise generate code snippet. + * @param {Array.} clientNearCaches Is client node. + * @returns {StringBuilder} + */ + static igniteConfiguration(cfg, pkg, clsName, clientNearCaches) { + const sb = new StringBuilder(); + + sb.append(`package ${pkg};`); + sb.emptyLine(); + + const imports = this.collectBeanImports(cfg); + + if (_.nonEmpty(clientNearCaches)) + imports.push('org.apache.ignite.configuration.NearCacheConfiguration'); + + if (_.includes(imports, 'oracle.jdbc.pool.OracleDataSource')) + imports.push('java.sql.SQLException'); + + const hasProps = this.hasProperties(cfg); + + if (hasProps) + imports.push('java.util.Properties', 'java.io.InputStream'); + + _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`)); + + sb.emptyLine(); + + const staticImports = this._prepareImports(this.collectStaticImports(cfg)); + + if (staticImports.length) { + _.forEach(this._prepareImports(staticImports), (cls) => sb.append(`import static ${cls};`)); + + sb.emptyLine(); + } + + this.mainComment(sb); + sb.startBlock(`public class ${clsName} {`); + + // 2. Add external property file + if (hasProps) { + this.commentBlock(sb, 'Secret properties loading.'); + sb.append('private static final Properties props = new Properties();'); + sb.emptyLine(); + sb.startBlock('static {'); + sb.startBlock('try (InputStream in = IgniteConfiguration.class.getClassLoader().getResourceAsStream("secret.properties")) {'); + sb.append('props.load(in);'); + sb.endBlock('}'); + sb.startBlock('catch (Exception ignored) {'); + sb.append('// No-op.'); + sb.endBlock('}'); + sb.endBlock('}'); + sb.emptyLine(); + } + + // 3. Add data sources. + const dataSources = this.collectDataSources(cfg); + + if (dataSources.length) { + this.commentBlock(sb, 'Helper class for datasource creation.'); + sb.startBlock('public static class DataSources {'); + + _.forEach(dataSources, (ds, idx) => { + const dsClsName = JavaTypes.shortClassName(ds.clsName); + + if (idx !== 0) + sb.emptyLine(); + + sb.append(`public static final ${dsClsName} INSTANCE_${ds.id} = create${ds.id}();`); + sb.emptyLine(); + + sb.startBlock(`private static ${dsClsName} create${ds.id}() {`); + + if (dsClsName === 'OracleDataSource') + sb.startBlock('try {'); + + this.constructBean(sb, ds); + + sb.emptyLine(); + sb.append(`return ${ds.id};`); + + if (dsClsName === 'OracleDataSource') { + sb.endBlock('}'); + sb.startBlock('catch (SQLException ex) {'); + sb.append('throw new Error(ex);'); + sb.endBlock('}'); + } + + sb.endBlock('}'); + }); + + sb.endBlock('}'); + + sb.emptyLine(); + } + + _.forEach(clientNearCaches, (cache) => { + this.commentBlock(sb, `Configuration of near cache for cache: ${cache.name}.`, + '', + '@return Near cache configuration.', + '@throws Exception If failed to construct near cache configuration instance.' + ); + + const nearCacheBean = generator.cacheNearClient(cache); + + sb.startBlock(`public static NearCacheConfiguration ${nearCacheBean.id}() throws Exception {`); + + this.constructBean(sb, nearCacheBean); + sb.emptyLine(); + + sb.append(`return ${nearCacheBean.id};`); + sb.endBlock('}'); + + sb.emptyLine(); + }); + + this.commentBlock(sb, 'Configure grid.', + '', + '@return Ignite configuration.', + '@throws Exception If failed to construct Ignite configuration instance.' + ); + sb.startBlock('public static IgniteConfiguration createConfiguration() throws Exception {'); + + this.constructBean(sb, cfg, [], true); + + sb.emptyLine(); + + sb.append(`return ${cfg.id};`); + + sb.endBlock('}'); + + const beans = this.collectBeansWithMapping(cfg); + + _.forEach(beans, (bean, id) => { + sb.emptyLine(); + + this.METHOD_MAPPING[bean.clsName].generator(sb, id, bean); + }); + + sb.endBlock('}'); + + return sb; + } + + static cluster(cluster, pkg, clsName, client) { + const cfg = this.generator.igniteConfiguration(cluster, client); + + const clientNearCaches = client ? _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')) : []; + + return this.igniteConfiguration(cfg, pkg, clsName, clientNearCaches); + } + + /** + * Generate source code for type by its domain model. + * + * @param fullClsName Full class name. + * @param fields Fields. + * @param addConstructor If 'true' then empty and full constructors should be generated. + * @returns {StringBuilder} + */ + static pojo(fullClsName, fields, addConstructor) { + const dotIdx = fullClsName.lastIndexOf('.'); + + const pkg = fullClsName.substring(0, dotIdx); + const clsName = fullClsName.substring(dotIdx + 1); + + const sb = new StringBuilder(); + + sb.append(`package ${pkg};`); + sb.emptyLine(); + + const imports = ['java.io.Serializable']; + + _.forEach(fields, (field) => imports.push(JavaTypes.fullClassName(field.javaFieldType))); + + _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`)); + + sb.emptyLine(); + + this.mainComment(sb, + `${clsName} definition.`, + '' + ); + sb.startBlock(`public class ${clsName} implements Serializable {`); + sb.append('/** */'); + sb.append('private static final long serialVersionUID = 0L;'); + sb.emptyLine(); + + // Generate fields declaration. + _.forEach(fields, (field) => { + const fldName = field.javaFieldName; + const fldType = JavaTypes.shortClassName(field.javaFieldType); + + sb.append(`/** Value for ${fldName}. */`); + sb.append(`private ${fldType} ${fldName};`); + + sb.emptyLine(); + }); + + // Generate constructors. + if (addConstructor) { + this.commentBlock(sb, 'Empty constructor.'); + sb.startBlock(`public ${clsName}() {`); + this.comment(sb, 'No-op.'); + sb.endBlock('}'); + + sb.emptyLine(); + + this.commentBlock(sb, 'Full constructor.'); + + const arg = (field) => { + const fldType = JavaTypes.shortClassName(field.javaFieldType); + + return `${fldType} ${field.javaFieldName}`; + }; + + sb.startBlock(`public ${clsName}(${arg(_.head(fields))}${fields.length === 1 ? ') {' : ','}`); + + _.forEach(_.tail(fields), (field, idx) => { + sb.append(`${arg(field)}${idx !== fields.length - 1 ? ',' : ') {'}`); + }); + + _.forEach(fields, (field) => sb.append(`this.${field.javaFieldName} = ${field.javaFieldName};`)); + + sb.endBlock('}'); + + sb.emptyLine(); + } + + // Generate getters and setters methods. + _.forEach(fields, (field) => { + const fldType = JavaTypes.shortClassName(field.javaFieldType); + const fldName = field.javaFieldName; + + this.commentBlock(sb, + `Gets ${fldName}`, + '', + `@return Value for ${fldName}.` + ); + sb.startBlock(`public ${fldType} ${JavaTypes.toJavaName('get', fldName)}() {`); + sb.append('return ' + fldName + ';'); + sb.endBlock('}'); + + sb.emptyLine(); + + this.commentBlock(sb, + `Sets ${fldName}`, + '', + `@param ${fldName} New value for ${fldName}.` + ); + sb.startBlock(`public void ${JavaTypes.toJavaName('set', fldName)}(${fldType} ${fldName}) {`); + sb.append(`this.${fldName} = ${fldName};`); + sb.endBlock('}'); + + sb.emptyLine(); + }); + + // Generate equals() method. + this.commentBlock(sb, '{@inheritDoc}'); + sb.startBlock('@Override public boolean equals(Object o) {'); + sb.startBlock('if (this == o)'); + sb.append('return true;'); + + sb.endBlock(''); + + sb.startBlock(`if (!(o instanceof ${clsName}))`); + sb.append('return false;'); + + sb.endBlock(''); + + sb.append(`${clsName} that = (${clsName})o;`); + + _.forEach(fields, (field) => { + sb.emptyLine(); + + const javaName = field.javaFieldName; + const javaType = field.javaFieldType; + + switch (javaType) { + case 'float': + sb.startBlock(`if (Float.compare(${javaName}, that.${javaName}) != 0)`); + + break; + case 'double': + sb.startBlock(`if (Double.compare(${javaName}, that.${javaName}) != 0)`); + + break; + default: + if (JavaTypes.isJavaPrimitive(javaType)) + sb.startBlock('if (' + javaName + ' != that.' + javaName + ')'); + else + sb.startBlock('if (' + javaName + ' != null ? !' + javaName + '.equals(that.' + javaName + ') : that.' + javaName + ' != null)'); + } + + sb.append('return false;'); + + sb.endBlock(''); + }); + + sb.append('return true;'); + sb.endBlock('}'); + + sb.emptyLine(); + + // Generate hashCode() method. + this.commentBlock(sb, '{@inheritDoc}'); + sb.startBlock('@Override public int hashCode() {'); + + let first = true; + let tempVar = false; + + _.forEach(fields, (field) => { + const javaName = field.javaFieldName; + const javaType = field.javaFieldType; + + let fldHashCode; + + switch (javaType) { + case 'boolean': + fldHashCode = `${javaName} ? 1 : 0`; + + break; + case 'byte': + case 'short': + fldHashCode = `(int)${javaName}`; + + break; + case 'int': + fldHashCode = `${javaName}`; + + break; + case 'long': + fldHashCode = `(int)(${javaName} ^ (${javaName} >>> 32))`; + + break; + case 'float': + fldHashCode = `${javaName} != +0.0f ? Float.floatToIntBits(${javaName}) : 0`; + + break; + case 'double': + sb.append(`${tempVar ? 'ig_hash_temp' : 'long ig_hash_temp'} = Double.doubleToLongBits(${javaName});`); + + tempVar = true; + + fldHashCode = `${javaName} != +0.0f ? Float.floatToIntBits(${javaName}) : 0`; + + break; + default: + fldHashCode = `${javaName} != null ? ${javaName}.hashCode() : 0`; + } + + sb.append(first ? `int res = ${fldHashCode};` : `res = 31 * res + ${fldHashCode.startsWith('(') ? fldHashCode : `(${fldHashCode})`};`); + + first = false; + + sb.emptyLine(); + }); + + sb.append('return res;'); + sb.endBlock('}'); + + sb.emptyLine(); + + this.commentBlock(sb, '{@inheritDoc}'); + sb.startBlock('@Override public String toString() {'); + sb.startBlock(`return "${clsName} [" + `); + + _.forEach(fields, (field, idx) => { + sb.append(`"${field.javaFieldName}=" + ${field.javaFieldName}${idx < fields.length - 1 ? ' + ", " + ' : ' +'}`); + }); + + sb.endBlock('"]";'); + sb.endBlock('}'); + + sb.endBlock('}'); + + return sb.asString(); + } + + /** + * Generate source code for type by its domain models. + * + * @param caches List of caches to generate POJOs for. + * @param addConstructor If 'true' then generate constructors. + * @param includeKeyFields If 'true' then include key fields into value POJO. + */ + static pojos(caches, addConstructor, includeKeyFields) { + const pojos = []; + + _.forEach(caches, (cache) => { + _.forEach(cache.domains, (domain) => { + // Process only domains with 'generatePojo' flag and skip already generated classes. + if (domain.generatePojo && !_.find(pojos, {valueType: domain.valueType}) && + // Skip domain models without value fields. + _.nonEmpty(domain.valueFields)) { + const pojo = {}; + + // Key class generation only if key is not build in java class. + if (_.nonNil(domain.keyFields) && domain.keyFields.length > 0) { + pojo.keyType = domain.keyType; + pojo.keyClass = this.pojo(domain.keyType, domain.keyFields, addConstructor); + } + + const valueFields = _.clone(domain.valueFields); + + if (includeKeyFields) { + _.forEach(domain.keyFields, ({fld}) => { + if (!_.find(valueFields, {name: fld.name})) + valueFields.push(fld); + }); + } + + pojo.valueType = domain.valueType; + pojo.valueClass = this.pojo(domain.valueType, valueFields, addConstructor); + + pojos.push(pojo); + } + }); + }); + + return pojos; + } + + // Generate creation and execution of cache query. + static _multilineQuery(sb, query, prefix, postfix) { + if (_.isEmpty(query)) + return; + + _.forEach(query, (line, ix) => { + if (ix === 0) { + if (query.length === 1) + sb.append(`${prefix}"${line}"${postfix}`); + else + sb.startBlock(`${prefix}"${line}" +`); + } + else + sb.append(`"${line}"${ix === query.length - 1 ? postfix : ' +'}`); + }); + + if (query.length > 1) + sb.endBlock(''); + else + sb.emptyLine(); + } + + // Generate creation and execution of prepared statement. + static _prepareStatement(sb, conVar, query) { + this._multilineQuery(sb, query, `${conVar}.prepareStatement(`, ').executeUpdate();'); + } + + static demoStartup(sb, cluster, shortFactoryCls) { + const cachesWithDataSource = _.filter(cluster.caches, (cache) => { + const kind = _.get(cache, 'cacheStoreFactory.kind'); + + if (kind) { + const store = cache.cacheStoreFactory[kind]; + + return (store.connectVia === 'DataSource' || _.isNil(store.connectVia)) && store.dialect; + } + + return false; + }); + + const uniqDomains = []; + + // Prepare array of cache and his demo domain model list. Every domain is contained only in first cache. + const demoTypes = _.reduce(cachesWithDataSource, (acc, cache) => { + const domains = _.filter(cache.domains, (domain) => _.nonEmpty(domain.valueFields) && + !_.includes(uniqDomains, domain)); + + if (_.nonEmpty(domains)) { + uniqDomains.push(...domains); + + acc.push({ + cache, + domains + }); + } + + return acc; + }, []); + + if (_.nonEmpty(demoTypes)) { + // Group domain modes by data source + const typeByDs = _.groupBy(demoTypes, ({cache}) => cache.cacheStoreFactory[cache.cacheStoreFactory.kind].dataSourceBean); + + let rndNonDefined = true; + + const generatedConsts = []; + + _.forEach(typeByDs, (types) => { + _.forEach(types, (type) => { + _.forEach(type.domains, (domain) => { + const valType = domain.valueType.toUpperCase(); + + const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type)); + + if (desc) { + if (rndNonDefined && desc.rndRequired) { + this.commentBlock(sb, 'Random generator for demo data.'); + sb.append('private static final Random rnd = new Random();'); + + sb.emptyLine(); + + rndNonDefined = false; + } + + _.forEach(desc.insertCntConsts, (cnt) => { + if (!_.includes(generatedConsts, cnt.name)) { + this.commentBlock(sb, cnt.comment); + sb.append(`private static final int ${cnt.name} = ${cnt.val};`); + + sb.emptyLine(); + + generatedConsts.push(cnt.name); + } + }); + } + }); + }); + }); + + // Generation of fill database method + this.commentBlock(sb, 'Fill data for Demo.'); + sb.startBlock('private static void prepareDemoData() throws SQLException {'); + + let firstDs = true; + + _.forEach(typeByDs, (types, ds) => { + const conVar = ds + 'Con'; + + if (firstDs) + firstDs = false; + else + sb.emptyLine(); + + sb.startBlock(`try (Connection ${conVar} = ${shortFactoryCls}.DataSources.INSTANCE_${ds}.getConnection()) {`); + + let first = true; + let stmtFirst = true; + + _.forEach(types, (type) => { + _.forEach(type.domains, (domain) => { + const valType = domain.valueType.toUpperCase(); + + const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type)); + + if (desc) { + if (first) + first = false; + else + sb.emptyLine(); + + this.comment(sb, `Generate ${desc.type}.`); + + if (desc.schema) + this._prepareStatement(sb, conVar, [`CREATE SCHEMA IF NOT EXISTS ${desc.schema}`]); + + this._prepareStatement(sb, conVar, desc.create); + + this._prepareStatement(sb, conVar, desc.clearQuery); + + let stmtVar = 'stmt'; + + if (stmtFirst) { + stmtFirst = false; + + stmtVar = 'PreparedStatement stmt'; + } + + if (_.isFunction(desc.customGeneration)) + desc.customGeneration(sb, conVar, stmtVar); + else { + sb.append(`${stmtVar} = ${conVar}.prepareStatement("${desc.insertPattern}");`); + + sb.emptyLine(); + + sb.startBlock(`for (int id = 0; id < ${desc.insertCntConsts[0].name}; id ++) {`); + + desc.fillInsertParameters(sb); + + sb.emptyLine(); + + sb.append('stmt.executeUpdate();'); + + sb.endBlock('}'); + } + + sb.emptyLine(); + + sb.append(`${conVar}.commit();`); + } + }); + }); + + sb.endBlock('}'); + }); + + sb.endBlock('}'); + + sb.emptyLine(); + + this.commentBlock(sb, 'Print result table to console.'); + sb.startBlock('private static void printResult(List> rows) {'); + sb.append('for (Cache.Entry row: rows)'); + sb.append(' System.out.println(row);'); + sb.endBlock('}'); + + sb.emptyLine(); + + // Generation of execute queries method. + this.commentBlock(sb, 'Run demo.'); + sb.startBlock('private static void runDemo(Ignite ignite) throws SQLException {'); + + const getType = (fullType) => fullType.substr(fullType.lastIndexOf('.') + 1); + + const cacheLoaded = []; + let rowVariableDeclared = false; + firstDs = true; + + _.forEach(typeByDs, (types, ds) => { + const conVar = ds + 'Con'; + + if (firstDs) + firstDs = false; + else + sb.emptyLine(); + + sb.startBlock(`try (Connection ${conVar} = ${shortFactoryCls}.DataSources.INSTANCE_${ds}.getConnection()) {`); + + let first = true; + + _.forEach(types, (type) => { + _.forEach(type.domains, (domain) => { + const valType = domain.valueType.toUpperCase(); + + const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type)); + + if (desc) { + if (_.isEmpty(desc.selectQuery)) + return; + + if (first) + first = false; + else + sb.emptyLine(); + + const cacheName = type.cache.name; + + if (!_.includes(cacheLoaded, cacheName)) { + sb.append(`ignite.cache("${cacheName}").loadCache(null);`); + + sb.emptyLine(); + + cacheLoaded.push(cacheName); + } + + const varRows = rowVariableDeclared ? 'rows' : 'List> rows'; + + this._multilineQuery(sb, desc.selectQuery, `${varRows} = ignite.cache("${cacheName}").query(new SqlQuery<>("${getType(domain.valueType)}", `, ')).getAll();'); + + sb.append('printResult(rows);'); + + rowVariableDeclared = true; + } + }); + }); + + sb.endBlock('}'); + }); + + sb.endBlock('}'); + } + } + + /** + * Function to generate java class for node startup with cluster configuration. + * + * @param {Object} cluster Cluster to process. + * @param {String} fullClsName Full class name. + * @param {String} cfgRef Config. + * @param {String} [factoryCls] fully qualified class name of configuration factory. + * @param {Array.} [clientNearCaches] Is client node. + */ + static nodeStartup(cluster, fullClsName, cfgRef, factoryCls, clientNearCaches) { + const dotIdx = fullClsName.lastIndexOf('.'); + + const pkg = fullClsName.substring(0, dotIdx); + const clsName = fullClsName.substring(dotIdx + 1); + + const demo = clsName === 'DemoStartup'; + + const sb = new StringBuilder(); + + const imports = ['org.apache.ignite.Ignition', 'org.apache.ignite.Ignite']; + + if (demo) { + imports.push('org.h2.tools.Server', 'java.sql.Connection', 'java.sql.PreparedStatement', + 'java.sql.SQLException', 'java.util.Random', 'java.util.List', 'javax.cache.Cache', + 'org.apache.ignite.cache.query.SqlQuery'); + } + + let shortFactoryCls; + + if (factoryCls) { + imports.push(factoryCls); + + shortFactoryCls = JavaTypes.shortClassName(factoryCls); + } + + sb.append(`package ${pkg};`) + .emptyLine(); + + _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`)); + sb.emptyLine(); + + if (demo) { + this.mainComment(sb, + 'To start demo configure data sources in secret.properties file.', + 'For H2 database it should be like following:', + 'dsH2.jdbc.url=jdbc:h2:tcp://localhost/mem:DemoDB;DB_CLOSE_DELAY=-1', + 'dsH2.jdbc.username=sa', + 'dsH2.jdbc.password=', + '' + ); + } + else + this.mainComment(sb); + + sb.startBlock(`public class ${clsName} {`); + + if (demo && shortFactoryCls) + this.demoStartup(sb, cluster, shortFactoryCls); + + this.commentBlock(sb, + 'Start up node with specified configuration.', + '', + '@param args Command line arguments, none required.', + '@throws Exception If failed.' + ); + sb.startBlock('public static void main(String[] args) throws Exception {'); + + if (demo) { + sb.startBlock('try {'); + sb.append('// Start H2 database server.'); + sb.append('Server.createTcpServer("-tcpDaemon").start();'); + sb.endBlock('}'); + sb.startBlock('catch (SQLException ignore) {'); + sb.append('// No-op.'); + sb.endBlock('}'); + + sb.emptyLine(); + } + + if ((_.nonEmpty(clientNearCaches) || demo) && shortFactoryCls) { + sb.append(`Ignite ignite = Ignition.start(${cfgRef});`); + + _.forEach(clientNearCaches, (cache, idx) => { + sb.emptyLine(); + + if (idx === 0) + sb.append('// Demo of near cache creation on client node.'); + + const nearCacheMtd = JavaTypes.toJavaName('nearConfiguration', cache.name); + + sb.append(`ignite.getOrCreateCache(${shortFactoryCls}.${cache.name}(), ${shortFactoryCls}.${nearCacheMtd}());`); + }); + } + else + sb.append(`Ignition.start(${cfgRef});`); + + if (demo) { + sb.emptyLine(); + + sb.append('prepareDemoData();'); + + sb.emptyLine(); + + sb.append('runDemo(ignite);'); + } + + sb.endBlock('}'); + + sb.endBlock('}'); + + return sb.asString(); + } + + /** + * Function to generate java class for load caches. + * + * @param caches Caches to load. + * @param pkg Class package name. + * @param clsName Class name. + * @param {String} cfgRef Config. + */ + static loadCaches(caches, pkg, clsName, cfgRef) { + const sb = new StringBuilder(); + + sb.append(`package ${pkg};`) + .emptyLine(); + + const imports = ['org.apache.ignite.Ignition', 'org.apache.ignite.Ignite']; + + _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`)); + sb.emptyLine(); + + this.mainComment(sb); + sb.startBlock(`public class ${clsName} {`); + + this.commentBlock(sb, + '

          ', + 'Utility to load caches from database.', + '

          ', + 'How to use:', + '

            ', + '
          • Start cluster.
          • ', + '
          • Start this utility and wait while load complete.
          • ', + '
          ', + '', + '@param args Command line arguments, none required.', + '@throws Exception If failed.' + ); + sb.startBlock('public static void main(String[] args) throws Exception {'); + + sb.startBlock(`try (Ignite ignite = Ignition.start(${cfgRef})) {`); + + sb.append('System.out.println(">>> Loading caches...");'); + + sb.emptyLine(); + + _.forEach(caches, (cache) => { + sb.append('System.out.println(">>> Loading cache: ' + cache.name + '");'); + sb.append('ignite.cache("' + cache.name + '").loadCache(null);'); + + sb.emptyLine(); + }); + + sb.append('System.out.println(">>> All caches loaded!");'); + + sb.endBlock('}'); + + sb.endBlock('}'); + + sb.endBlock('}'); + + return sb.asString(); + } + + /** + * Checks if cluster has demo types. + * + * @param cluster Cluster to check. + * @param demo Is demo enabled. + * @returns {boolean} True if cluster has caches with demo types. + */ + static isDemoConfigured(cluster, demo) { + return demo && _.find(cluster.caches, (cache) => _.find(cache.domains, (domain) => _.find(PREDEFINED_QUERIES, (desc) => domain.valueType.toUpperCase().endsWith(desc.type)))); + } + } + + return JavaTransformer; +}]; diff --git a/modules/web-console/frontend/app/modules/configuration/generator/PlatformGenerator.js b/modules/web-console/frontend/app/modules/configuration/generator/PlatformGenerator.js new file mode 100644 index 0000000000000..b07619331715e --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/PlatformGenerator.js @@ -0,0 +1,522 @@ +/* + * 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. + */ + +// import _ from 'lodash'; +import { EmptyBean, Bean } from './Beans'; + +export default ['JavaTypes', 'igniteClusterPlatformDefaults', 'igniteCachePlatformDefaults', (JavaTypes, clusterDflts, cacheDflts) => { + class PlatformGenerator { + static igniteConfigurationBean(cluster) { + return new Bean('Apache.Ignite.Core.IgniteConfiguration', 'cfg', cluster, clusterDflts); + } + + static cacheConfigurationBean(cache) { + return new Bean('Apache.Ignite.Core.Cache.Configuration.CacheConfiguration', 'ccfg', cache, cacheDflts); + } + + /** + * Function to generate ignite configuration. + * + * @param {Object} cluster Cluster to process. + * @return {String} Generated ignite configuration. + */ + static igniteConfiguration(cluster) { + const cfg = this.igniteConfigurationBean(cluster); + + this.clusterAtomics(cluster.atomics, cfg); + + return cfg; + } + + // Generate general section. + static clusterGeneral(cluster, cfg = this.igniteConfigurationBean(cluster)) { + cfg.stringProperty('name', 'GridName') + .stringProperty('localHost', 'Localhost'); + + if (_.isNil(cluster.discovery)) + return cfg; + + const discovery = new Bean('Apache.Ignite.Core.Discovery.Tcp.TcpDiscoverySpi', 'discovery', + cluster.discovery, clusterDflts.discovery); + + let ipFinder; + + switch (discovery.valueOf('kind')) { + case 'Vm': + ipFinder = new Bean('Apache.Ignite.Core.Discovery.Tcp.Static.TcpDiscoveryStaticIpFinder', + 'ipFinder', cluster.discovery.Vm, clusterDflts.discovery.Vm); + + ipFinder.collectionProperty('addrs', 'addresses', cluster.discovery.Vm.addresses, 'ICollection'); + + break; + case 'Multicast': + ipFinder = new Bean('Apache.Ignite.Core.Discovery.Tcp.Multicast.TcpDiscoveryMulticastIpFinder', + 'ipFinder', cluster.discovery.Multicast, clusterDflts.discovery.Multicast); + + ipFinder.stringProperty('MulticastGroup') + .intProperty('multicastPort', 'MulticastPort') + .intProperty('responseWaitTime', 'ResponseTimeout') + .intProperty('addressRequestAttempts', 'AddressRequestAttempts') + .stringProperty('localAddress', 'LocalAddress') + .collectionProperty('addrs', 'Endpoints', cluster.discovery.Multicast.addresses, 'ICollection'); + + break; + default: + } + + if (ipFinder) + discovery.beanProperty('IpFinder', ipFinder); + + cfg.beanProperty('DiscoverySpi', discovery); + + + return cfg; + } + + static clusterAtomics(atomics, cfg = this.igniteConfigurationBean()) { + const acfg = new Bean('Apache.Ignite.Core.DataStructures.Configuration.AtomicConfiguration', 'atomicCfg', + atomics, clusterDflts.atomics); + + acfg.enumProperty('cacheMode', 'CacheMode') + .intProperty('atomicSequenceReserveSize', 'AtomicSequenceReserveSize'); + + if (acfg.valueOf('cacheMode') === 'PARTITIONED') + acfg.intProperty('backups', 'Backups'); + + if (acfg.isEmpty()) + return cfg; + + cfg.beanProperty('AtomicConfiguration', acfg); + + return cfg; + } + + // Generate binary group. + static clusterBinary(binary, cfg = this.igniteConfigurationBean()) { + const binaryCfg = new Bean('Apache.Ignite.Core.Binary.BinaryConfiguration', 'binaryCfg', + binary, clusterDflts.binary); + + binaryCfg.emptyBeanProperty('idMapper', 'DefaultIdMapper') + .emptyBeanProperty('nameMapper', 'DefaultNameMapper') + .emptyBeanProperty('serializer', 'DefaultSerializer'); + + // const typeCfgs = []; + // + // _.forEach(binary.typeConfigurations, (type) => { + // const typeCfg = new MethodBean('Apache.Ignite.Core.Binary.BinaryTypeConfiguration', + // JavaTypes.toJavaName('binaryType', type.typeName), type, clusterDflts.binary.typeConfigurations); + // + // typeCfg.stringProperty('typeName', 'TypeName') + // .emptyBeanProperty('idMapper', 'IdMapper') + // .emptyBeanProperty('nameMapper', 'NameMapper') + // .emptyBeanProperty('serializer', 'Serializer') + // .intProperty('enum', 'IsEnum'); + // + // if (typeCfg.nonEmpty()) + // typeCfgs.push(typeCfg); + // }); + // + // binaryCfg.collectionProperty('types', 'TypeConfigurations', typeCfgs, 'ICollection', + // 'Apache.Ignite.Core.Binary.BinaryTypeConfiguration'); + // + // binaryCfg.boolProperty('compactFooter', 'CompactFooter'); + // + // if (binaryCfg.isEmpty()) + // return cfg; + // + // cfg.beanProperty('binaryConfiguration', binaryCfg); + + return cfg; + } + + // Generate communication group. + static clusterCommunication(cluster, cfg = this.igniteConfigurationBean(cluster)) { + const commSpi = new Bean('Apache.Ignite.Core.Communication.Tcp.TcpCommunicationSpi', 'communicationSpi', + cluster.communication, clusterDflts.communication); + + commSpi.emptyBeanProperty('listener') + .stringProperty('localAddress') + .intProperty('localPort') + .intProperty('localPortRange') + // .intProperty('sharedMemoryPort') + .intProperty('directBuffer') + .intProperty('directSendBuffer') + .intProperty('idleConnectionTimeout') + .intProperty('connectTimeout') + .intProperty('maxConnectTimeout') + .intProperty('reconnectCount') + .intProperty('socketSendBuffer') + .intProperty('socketReceiveBuffer') + .intProperty('messageQueueLimit') + .intProperty('slowClientQueueLimit') + .intProperty('tcpNoDelay') + .intProperty('ackSendThreshold') + .intProperty('unacknowledgedMessagesBufferSize') + // .intProperty('socketWriteTimeout') + .intProperty('selectorsCount'); + // .emptyBeanProperty('addressResolver'); + + if (commSpi.nonEmpty()) + cfg.beanProperty('CommunicationSpi', commSpi); + + cfg.intProperty('networkTimeout', 'NetworkTimeout') + .intProperty('networkSendRetryDelay') + .intProperty('networkSendRetryCount'); + // .intProperty('discoveryStartupDelay'); + + return cfg; + } + + // Generate discovery group. + static clusterDiscovery(discovery, cfg = this.igniteConfigurationBean()) { + if (discovery) { + let discoveryCfg = cfg.findProperty('discovery'); + + if (_.isNil(discoveryCfg)) { + discoveryCfg = new Bean('Apache.Ignite.Core.Discovery.Tcp.TcpDiscoverySpi', 'discovery', + discovery, clusterDflts.discovery); + } + + discoveryCfg.stringProperty('localAddress') + .intProperty('localPort') + .intProperty('localPortRange') + .intProperty('socketTimeout') + .intProperty('ackTimeout') + .intProperty('maxAckTimeout') + .intProperty('networkTimeout') + .intProperty('joinTimeout') + .intProperty('threadPriority') + .intProperty('heartbeatFrequency') + .intProperty('maxMissedHeartbeats') + .intProperty('maxMissedClientHeartbeats') + .intProperty('topHistorySize') + .intProperty('reconnectCount') + .intProperty('statisticsPrintFrequency') + .intProperty('ipFinderCleanFrequency') + .intProperty('forceServerMode') + .intProperty('clientReconnectDisabled'); + + if (discoveryCfg.nonEmpty()) + cfg.beanProperty('discoverySpi', discoveryCfg); + } + + return cfg; + } + + // Generate events group. + static clusterEvents(cluster, cfg = this.igniteConfigurationBean(cluster)) { + if (_.nonEmpty(cluster.includeEventTypes)) + cfg.eventTypes('events', 'includeEventTypes', cluster.includeEventTypes); + + return cfg; + } + + // Generate metrics group. + static clusterMetrics(cluster, cfg = this.igniteConfigurationBean(cluster)) { + cfg.intProperty('metricsExpireTime') + .intProperty('metricsHistorySize') + .intProperty('metricsLogFrequency') + .intProperty('metricsUpdateFrequency'); + + return cfg; + } + + // Generate transactions group. + static clusterTransactions(transactionConfiguration, cfg = this.igniteConfigurationBean()) { + const bean = new Bean('Apache.Ignite.Core.Transactions.TransactionConfiguration', 'TransactionConfiguration', + transactionConfiguration, clusterDflts.transactionConfiguration); + + bean.enumProperty('defaultTxConcurrency', 'DefaultTransactionConcurrency') + .enumProperty('defaultTxIsolation', 'DefaultTransactionIsolation') + .intProperty('defaultTxTimeout', 'DefaultTimeout') + .intProperty('pessimisticTxLogLinger', 'PessimisticTransactionLogLinger') + .intProperty('pessimisticTxLogSize', 'PessimisticTransactionLogSize'); + + if (bean.nonEmpty()) + cfg.beanProperty('transactionConfiguration', bean); + + return cfg; + } + + // Generate user attributes group. + static clusterUserAttributes(cluster, cfg = this.igniteConfigurationBean(cluster)) { + cfg.mapProperty('attributes', 'attributes', 'UserAttributes'); + + return cfg; + } + + static clusterCaches(cluster, caches, igfss, isSrvCfg, cfg = this.igniteConfigurationBean(cluster)) { + // const cfg = this.clusterGeneral(cluster, cfg); + // + // if (_.nonEmpty(caches)) { + // const ccfgs = _.map(caches, (cache) => this.cacheConfiguration(cache)); + // + // cfg.collectionProperty('', '', ccfgs, ); + // } + + return this.clusterGeneral(cluster, cfg); + } + + // Generate cache general group. + static cacheGeneral(cache, ccfg = this.cacheConfigurationBean(cache)) { + ccfg.stringProperty('name') + .enumProperty('cacheMode') + .enumProperty('atomicityMode'); + + if (ccfg.valueOf('cacheMode') === 'PARTITIONED' && ccfg.valueOf('backups')) { + ccfg.intProperty('backups') + .intProperty('readFromBackup'); + } + + ccfg.intProperty('copyOnRead'); + + if (ccfg.valueOf('cacheMode') === 'PARTITIONED' && ccfg.valueOf('atomicityMode') === 'TRANSACTIONAL') + ccfg.intProperty('invalidate'); + + return ccfg; + } + + // Generate cache memory group. + static cacheMemory(cache, ccfg = this.cacheConfigurationBean(cache)) { + ccfg.enumProperty('memoryMode'); + + if (ccfg.valueOf('memoryMode') !== 'OFFHEAP_VALUES') + ccfg.intProperty('offHeapMaxMemory'); + + // this._evictionPolicy(ccfg, 'evictionPolicy', cache.evictionPolicy, cacheDflts.evictionPolicy); + + ccfg.intProperty('startSize') + .boolProperty('swapEnabled', 'EnableSwap'); + + return ccfg; + } + + // Generate cache queries & Indexing group. + static cacheQuery(cache, domains, ccfg = this.cacheConfigurationBean(cache)) { + ccfg.intProperty('sqlOnheapRowCacheSize') + .intProperty('longQueryWarningTimeout'); + + return ccfg; + } + + // Generate cache store group. + static cacheStore(cache, domains, ccfg = this.cacheConfigurationBean(cache)) { + const kind = _.get(cache, 'cacheStoreFactory.kind'); + + if (kind && cache.cacheStoreFactory[kind]) { + let bean = null; + + const storeFactory = cache.cacheStoreFactory[kind]; + + switch (kind) { + case 'CacheJdbcPojoStoreFactory': + bean = new Bean('org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory', 'cacheStoreFactory', + storeFactory); + + const id = bean.valueOf('dataSourceBean'); + + bean.dataSource(id, 'dataSourceBean', this.dataSourceBean(id, storeFactory.dialect)) + .beanProperty('dialect', new EmptyBean(this.dialectClsName(storeFactory.dialect))); + + const setType = (typeBean, propName) => { + if (JavaTypes.nonBuiltInClass(typeBean.valueOf(propName))) + typeBean.stringProperty(propName); + else + typeBean.classProperty(propName); + }; + + const types = _.reduce(domains, (acc, domain) => { + if (_.isNil(domain.databaseTable)) + return acc; + + const typeBean = new Bean('org.apache.ignite.cache.store.jdbc.JdbcType', 'type', + _.merge({}, domain, {cacheName: cache.name})) + .stringProperty('cacheName'); + + setType(typeBean, 'keyType'); + setType(typeBean, 'valueType'); + + this.domainStore(domain, typeBean); + + acc.push(typeBean); + + return acc; + }, []); + + bean.arrayProperty('types', 'types', types, 'org.apache.ignite.cache.store.jdbc.JdbcType'); + + break; + case 'CacheJdbcBlobStoreFactory': + bean = new Bean('org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory', 'cacheStoreFactory', + storeFactory); + + if (bean.valueOf('connectVia') === 'DataSource') + bean.dataSource(bean.valueOf('dataSourceBean'), 'dataSourceBean', this.dialectClsName(storeFactory.dialect)); + else { + ccfg.stringProperty('connectionUrl') + .stringProperty('user') + .property('password', `ds.${storeFactory.user}.password`, 'YOUR_PASSWORD'); + } + + bean.boolProperty('initSchema') + .stringProperty('createTableQuery') + .stringProperty('loadQuery') + .stringProperty('insertQuery') + .stringProperty('updateQuery') + .stringProperty('deleteQuery'); + + break; + case 'CacheHibernateBlobStoreFactory': + bean = new Bean('org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory', + 'cacheStoreFactory', storeFactory); + + bean.propsProperty('props', 'hibernateProperties'); + + break; + default: + } + + if (bean) + ccfg.beanProperty('cacheStoreFactory', bean); + } + + ccfg.boolProperty('storeKeepBinary') + .boolProperty('loadPreviousValue') + .boolProperty('readThrough') + .boolProperty('writeThrough'); + + if (ccfg.valueOf('writeBehindEnabled')) { + ccfg.boolProperty('writeBehindEnabled') + .intProperty('writeBehindBatchSize') + .intProperty('writeBehindFlushSize') + .intProperty('writeBehindFlushFrequency') + .intProperty('writeBehindFlushThreadCount'); + } + + return ccfg; + } + + // Generate cache concurrency control group. + static cacheConcurrency(cache, ccfg = this.cacheConfigurationBean(cache)) { + ccfg.intProperty('maxConcurrentAsyncOperations') + .intProperty('defaultLockTimeout') + .enumProperty('atomicWriteOrderMode') + .enumProperty('writeSynchronizationMode'); + + return ccfg; + } + + // Generate cache node filter group. + static cacheNodeFilter(cache, igfss, ccfg = this.cacheConfigurationBean(cache)) { + const kind = _.get(cache, 'nodeFilter.kind'); + + if (kind && cache.nodeFilter[kind]) { + let bean = null; + + switch (kind) { + case 'IGFS': + const foundIgfs = _.find(igfss, (igfs) => igfs._id === cache.nodeFilter.IGFS.igfs); + + if (foundIgfs) { + bean = new Bean('org.apache.ignite.internal.processors.igfs.IgfsNodePredicate', 'nodeFilter', foundIgfs) + .stringConstructorArgument('name'); + } + + break; + case 'Custom': + bean = new Bean(cache.nodeFilter.Custom.className, 'nodeFilter'); + + break; + default: + return ccfg; + } + + if (bean) + ccfg.beanProperty('nodeFilter', bean); + } + + return ccfg; + } + + // Generate cache rebalance group. + static cacheRebalance(cache, ccfg = this.cacheConfigurationBean(cache)) { + if (ccfg.valueOf('cacheMode') !== 'LOCAL') { + ccfg.enumProperty('rebalanceMode') + .intProperty('rebalanceThreadPoolSize') + .intProperty('rebalanceBatchSize') + .intProperty('rebalanceBatchesPrefetchCount') + .intProperty('rebalanceOrder') + .intProperty('rebalanceDelay') + .intProperty('rebalanceTimeout') + .intProperty('rebalanceThrottle'); + } + + if (ccfg.includes('igfsAffinnityGroupSize')) { + const bean = new Bean('org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper', 'affinityMapper', cache) + .intConstructorArgument('igfsAffinnityGroupSize'); + + ccfg.beanProperty('affinityMapper', bean); + } + + return ccfg; + } + + // Generate server near cache group. + static cacheServerNearCache(cache, ccfg = this.cacheConfigurationBean(cache)) { + if (cache.cacheMode === 'PARTITIONED' && cache.nearCacheEnabled) { + const bean = new Bean('org.apache.ignite.configuration.NearCacheConfiguration', 'nearConfiguration', + cache.nearConfiguration, {nearStartSize: 375000}); + + bean.intProperty('nearStartSize'); + + this._evictionPolicy(bean, 'nearEvictionPolicy', + bean.valueOf('nearEvictionPolicy'), cacheDflts.evictionPolicy); + + ccfg.beanProperty('nearConfiguration', bean); + } + + return ccfg; + } + + // Generate cache statistics group. + static cacheStatistics(cache, ccfg = this.cacheConfigurationBean(cache)) { + ccfg.boolProperty('statisticsEnabled') + .boolProperty('managementEnabled'); + + return ccfg; + } + + static cacheConfiguration(cache, ccfg = this.cacheConfigurationBean(cache)) { + this.cacheGeneral(cache, ccfg); + this.cacheMemory(cache, ccfg); + this.cacheQuery(cache, cache.domains, ccfg); + this.cacheStore(cache, cache.domains, ccfg); + + const igfs = _.get(cache, 'nodeFilter.IGFS.instance'); + this.cacheNodeFilter(cache, igfs ? [igfs] : [], ccfg); + this.cacheConcurrency(cache, ccfg); + this.cacheRebalance(cache, ccfg); + this.cacheServerNearCache(cache, ccfg); + this.cacheStatistics(cache, ccfg); + // this.cacheDomains(cache.domains, cfg); + + return ccfg; + } + } + + return PlatformGenerator; +}]; diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js index f2d04b08b2719..db58532187780 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js @@ -15,6 +15,8 @@ * limitations under the License. */ +import StringBuilder from './StringBuilder'; + // Java built-in class names. import POM_DEPENDENCIES from 'app/data/pom-dependencies.json'; @@ -29,8 +31,8 @@ class GeneratorPom { return s.replace(/[^A-Za-z0-9_\-.]+/g, '_'); } - addProperty(res, tag, val) { - res.line('<' + tag + '>' + val + ''); + addProperty(sb, tag, val) { + sb.append('<' + tag + '>' + val + ''); } addDependency(deps, groupId, artifactId, version, jar) { @@ -38,92 +40,89 @@ class GeneratorPom { deps.push({groupId, artifactId, version, jar}); } - addResource(res, dir, exclude) { - res.startBlock(''); + addResource(sb, dir, exclude) { + sb.startBlock(''); if (dir) - this.addProperty(res, 'directory', dir); + this.addProperty(sb, 'directory', dir); if (exclude) { - res.startBlock(''); - this.addProperty(res, 'exclude', exclude); - res.endBlock(''); + sb.startBlock(''); + this.addProperty(sb, 'exclude', exclude); + sb.endBlock(''); } - res.endBlock(''); + sb.endBlock(''); } - artifact(res, cluster, version) { - this.addProperty(res, 'groupId', 'org.apache.ignite'); - this.addProperty(res, 'artifactId', this.escapeId(cluster.name) + '-project'); - this.addProperty(res, 'version', version); + artifact(sb, cluster, version) { + this.addProperty(sb, 'groupId', 'org.apache.ignite'); + this.addProperty(sb, 'artifactId', this.escapeId(cluster.name) + '-project'); + this.addProperty(sb, 'version', version); - res.needEmptyLine = true; + sb.emptyLine(); } - dependencies(res, cluster, deps) { - if (!res) - res = $generatorCommon.builder(); - - res.startBlock(''); + dependencies(sb, cluster, deps) { + sb.startBlock(''); _.forEach(deps, (dep) => { - res.startBlock(''); + sb.startBlock(''); - this.addProperty(res, 'groupId', dep.groupId); - this.addProperty(res, 'artifactId', dep.artifactId); - this.addProperty(res, 'version', dep.version); + this.addProperty(sb, 'groupId', dep.groupId); + this.addProperty(sb, 'artifactId', dep.artifactId); + this.addProperty(sb, 'version', dep.version); if (dep.jar) { - this.addProperty(res, 'scope', 'system'); - this.addProperty(res, 'systemPath', '${project.basedir}/jdbc-drivers/' + dep.jar); + this.addProperty(sb, 'scope', 'system'); + this.addProperty(sb, 'systemPath', '${project.basedir}/jdbc-drivers/' + dep.jar); } - res.endBlock(''); + sb.endBlock(''); }); - res.endBlock(''); + sb.endBlock(''); - return res; + return sb; } - build(res, cluster, excludeGroupIds) { - res.startBlock(''); - res.startBlock(''); - this.addResource(res, 'src/main/java', '**/*.java'); - this.addResource(res, 'src/main/resources'); - res.endBlock(''); - - res.startBlock(''); - res.startBlock(''); - this.addProperty(res, 'artifactId', 'maven-dependency-plugin'); - res.startBlock(''); - res.startBlock(''); - this.addProperty(res, 'id', 'copy-libs'); - this.addProperty(res, 'phase', 'test-compile'); - res.startBlock(''); - this.addProperty(res, 'goal', 'copy-dependencies'); - res.endBlock(''); - res.startBlock(''); - this.addProperty(res, 'excludeGroupIds', excludeGroupIds.join(',')); - this.addProperty(res, 'outputDirectory', 'target/libs'); - this.addProperty(res, 'includeScope', 'compile'); - this.addProperty(res, 'excludeTransitive', 'true'); - res.endBlock(''); - res.endBlock(''); - res.endBlock(''); - res.endBlock(''); - res.startBlock(''); - this.addProperty(res, 'artifactId', 'maven-compiler-plugin'); - this.addProperty(res, 'version', '3.1'); - res.startBlock(''); - this.addProperty(res, 'source', '1.7'); - this.addProperty(res, 'target', '1.7'); - res.endBlock(''); - res.endBlock(''); - res.endBlock(''); - res.endBlock(''); - - res.endBlock(''); + build(sb = new StringBuilder(), cluster, excludeGroupIds) { + sb.startBlock(''); + sb.startBlock(''); + this.addResource(sb, 'src/main/java', '**/*.java'); + this.addResource(sb, 'src/main/resources'); + sb.endBlock(''); + + sb.startBlock(''); + sb.startBlock(''); + this.addProperty(sb, 'artifactId', 'maven-dependency-plugin'); + sb.startBlock(''); + sb.startBlock(''); + this.addProperty(sb, 'id', 'copy-libs'); + this.addProperty(sb, 'phase', 'test-compile'); + sb.startBlock(''); + this.addProperty(sb, 'goal', 'copy-dependencies'); + sb.endBlock(''); + sb.startBlock(''); + this.addProperty(sb, 'excludeGroupIds', excludeGroupIds.join(',')); + this.addProperty(sb, 'outputDirectory', 'target/libs'); + this.addProperty(sb, 'includeScope', 'compile'); + this.addProperty(sb, 'excludeTransitive', 'true'); + sb.endBlock(''); + sb.endBlock(''); + sb.endBlock(''); + sb.endBlock(''); + sb.startBlock(''); + this.addProperty(sb, 'artifactId', 'maven-compiler-plugin'); + this.addProperty(sb, 'version', '3.1'); + sb.startBlock(''); + this.addProperty(sb, 'source', '1.7'); + this.addProperty(sb, 'target', '1.7'); + sb.endBlock(''); + sb.endBlock(''); + sb.endBlock(''); + sb.endBlock(''); + + sb.endBlock(''); } /** @@ -144,10 +143,10 @@ class GeneratorPom { * * @param cluster Cluster to take info about dependencies. * @param version Ignite version for Ignite dependencies. - * @param res Resulting output with generated pom. + * @param sb Resulting output with generated pom. * @returns {string} Generated content. */ - generate(cluster, version, res) { + generate(cluster, version, sb = new StringBuilder()) { const caches = cluster.caches; const deps = []; const storeDeps = []; @@ -155,9 +154,6 @@ class GeneratorPom { const blobStoreFactory = {cacheStoreFactory: {kind: 'CacheHibernateBlobStoreFactory'}}; - if (!res) - res = $generatorCommon.builder(); - _.forEach(caches, (cache) => { if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) this.storeFactoryDependency(storeDeps, cache.cacheStoreFactory[cache.cacheStoreFactory.kind]); @@ -166,21 +162,21 @@ class GeneratorPom { this.addDependency(deps, 'org.apache.ignite', 'ignite-extdata-p2p', version); }); - res.line(''); + sb.append(''); - res.needEmptyLine = true; + sb.emptyLine(); - res.line(''); + sb.append(``); - res.needEmptyLine = true; + sb.emptyLine(); - res.startBlock(''); + sb.startBlock(''); - res.line('4.0.0'); + sb.append('4.0.0'); - res.needEmptyLine = true; + sb.emptyLine(); - this.artifact(res, cluster, version); + this.artifact(sb, cluster, version); this.addDependency(deps, 'org.apache.ignite', 'ignite-core', version); @@ -200,6 +196,17 @@ class GeneratorPom { this.storeFactoryDependency(storeDeps, cluster.discovery.Jdbc); } + _.forEach(cluster.checkpointSpi, (spi) => { + if (spi.kind === 'S3') { + dep = POM_DEPENDENCIES.S3; + + if (dep) + this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version); + } + else if (spi.kind === 'JDBC') + this.storeFactoryDependency(storeDeps, spi.JDBC); + }); + if (_.find(cluster.igfss, (igfs) => igfs.secondaryFileSystemEnabled)) this.addDependency(deps, 'org.apache.ignite', 'ignite-hadoop', version); @@ -213,13 +220,13 @@ class GeneratorPom { this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version); } - this.dependencies(res, cluster, deps.concat(storeDeps)); + this.dependencies(sb, cluster, deps.concat(storeDeps)); - res.needEmptyLine = true; + sb.emptyLine(); - this.build(res, cluster, excludeGroupIds); + this.build(sb, cluster, excludeGroupIds); - return res; + return sb; } } diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js new file mode 100644 index 0000000000000..07f8dffe7eb5e --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js @@ -0,0 +1,74 @@ +/* + * 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. + */ + +import StringBuilder from './StringBuilder'; + +/** + * Properties generation entry point. + */ +export default class PropertiesGenerator { + _collectProperties(bean) { + const props = []; + + _.forEach(bean.properties, (prop) => { + switch (prop.clsName) { + case 'DATA_SOURCE': + props.push(...this._collectProperties(prop.value)); + props.push(''); + + break; + case 'BEAN': + props.push(...this._collectProperties(prop.value)); + + break; + case 'PROPERTY': + case 'PROPERTY_CHAR': + props.push(`${prop.value}=${prop.hint}`); + + break; + case 'ARRAY': + case 'COLLECTION': + _.forEach(prop.items, (item) => { + const itemLines = this._collectProperties(item); + + if (_.intersection(props, itemLines).length !== itemLines.length) + props.push(...this._collectProperties(item)); + }); + + break; + default: + } + }); + + return props; + } + + generate(cfg) { + const lines = this._collectProperties(cfg); + + if (_.isEmpty(lines)) + return null; + + const sb = new StringBuilder(); + + sb.append(`# ${sb.generatedBy()}`); + + _.forEach(lines, (line) => sb.append(line)); + + return sb.asString(); + } +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js new file mode 100644 index 0000000000000..7043807fb818c --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js @@ -0,0 +1,79 @@ +/* + * 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. + */ + +import StringBuilder from './StringBuilder'; + +/** + * Properties generation entry point. + */ +export default class ReadmeGenerator { + header(sb) { + sb.append('Content of this folder was generated by Apache Ignite Web Console'); + sb.append('================================================================='); + } + + /** + * Generate README.txt for jdbc folder. + * + * @param sb Resulting output with generated readme. + * @returns {string} Generated content. + */ + generateJDBC(sb = new StringBuilder()) { + sb.append('Proprietary JDBC drivers for databases like Oracle, IBM DB2, Microsoft SQL Server are not available on Maven Central repository.'); + sb.append('Drivers should be downloaded manually and copied to this folder.'); + + return sb.asString(); + } + + /** + * Generate README.txt. + * + * @returns {string} Generated content. + */ + generate(sb = new StringBuilder()) { + this.header(sb); + sb.emptyLine(); + + sb.append('Project structure:'); + sb.append(' /config - this folder contains client and server XML configurations.'); + sb.append(' /jdbc-drivers - this folder should contains proprietary JDBC drivers.'); + sb.append(' /src - this folder contains generated java code.'); + sb.append(' /src/main/java/config - this folder contains generated java classes with cluster configuration from code.'); + sb.append(' /src/main/java/startup - this folder contains generated java classes with server and client nodes startup code.'); + sb.append(' /src/main/java/[model] - this optional folder will be named as package name for your POJO classes and contain generated POJO files.'); + sb.append(' /src/main/resources - this optional folder contains generated secret.properties file with security sensitive information if any.'); + sb.append(' Dockerfile - sample Docker file. With this file you could package Ignite deployment with all the dependencies into a standard container.'); + sb.append(' pom.xml - generated Maven project description, could be used to open generated project in IDE or build with Maven.'); + sb.append(' README.txt - this file.'); + + sb.emptyLine(); + + sb.append('Ignite ships with CacheJdbcPojoStore, which is out-of-the-box JDBC implementation of the IgniteCacheStore '); + sb.append('interface, and automatically handles all the write-through and read-through logic.'); + + sb.emptyLine(); + + sb.append('You can use generated configuration and POJO classes as part of your application.'); + + sb.emptyLine(); + + sb.append('Note, in case of using proprietary JDBC drivers (Oracle, IBM DB2, Microsoft SQL Server)'); + sb.append('you should download them manually and copy into ./jdbc-drivers folder.'); + + return sb.asString(); + } +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js new file mode 100644 index 0000000000000..19043f6f8a57a --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js @@ -0,0 +1,243 @@ +/* + * 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. + */ + +import _ from 'lodash'; +import AbstractTransformer from './AbstractTransformer'; +import StringBuilder from './StringBuilder'; + +export default ['JavaTypes', 'IgnitePlatformGenerator', (JavaTypes, generator) => { + return class SharpTransformer extends AbstractTransformer { + static generator = generator; + + static commentBlock(sb, ...lines) { + _.forEach(lines, (line) => sb.append(`// ${line}`)); + } + + static doc(sb, ...lines) { + sb.append('/// '); + _.forEach(lines, (line) => sb.append(`/// ${line}`)); + sb.append('/// '); + } + + static mainComment(sb) { + return this.doc(sb, sb.generatedBy()); + } + + /** + * + * @param {Array.} sb + * @param {Bean} bean + */ + static _defineBean(sb, bean) { + const shortClsName = JavaTypes.shortClassName(bean.clsName); + + sb.append(`var ${bean.id} = new ${shortClsName}();`); + } + + /** + * @param {StringBuilder} sb + * @param {Bean} parent + * @param {Bean} propertyName + * @param {String|Bean} value + * @private + */ + static _setProperty(sb, parent, propertyName, value) { + sb.append(`${parent.id}.${_.upperFirst(propertyName)} = ${value};`); + } + + /** + * + * @param {StringBuilder} sb + * @param {Bean} parent + * @param {String} propertyName + * @param {Bean} bean + * @private + */ + static _setBeanProperty(sb, parent, propertyName, bean) { + sb.append(`${parent.id}.${_.upperFirst(propertyName)} = ${bean.id};`); + } + + static _toObject(clsName, val) { + const items = _.isArray(val) ? val : [val]; + + return _.map(items, (item, idx) => { + if (_.isNil(item)) + return 'null'; + + const shortClsName = JavaTypes.shortClassName(clsName); + + switch (shortClsName) { + // case 'byte': + // return `(byte) ${item}`; + // case 'Serializable': + case 'String': + if (items.length > 1) + return `"${item}"${idx !== items.length - 1 ? ' +' : ''}`; + + return `"${item}"`; + // case 'Path': + // return `"${item.replace(/\\/g, '\\\\')}"`; + // case 'Class': + // return `${this.shortClassName(item)}.class`; + // case 'UUID': + // return `UUID.fromString("${item}")`; + // case 'PropertyChar': + // return `props.getProperty("${item}").toCharArray()`; + // case 'Property': + // return `props.getProperty("${item}")`; + // case 'Bean': + // if (item.isComplex()) + // return item.id; + // + // return this._newBean(item); + default: + if (JavaTypes.nonEnum(shortClsName)) + return item; + + return `${shortClsName}.${item}`; + } + }); + } + + /** + * + * @param {StringBuilder} sb + * @param {Bean} bean + * @returns {Array} + */ + static _setProperties(sb = new StringBuilder(), bean) { + _.forEach(bean.properties, (prop) => { + switch (prop.clsName) { + case 'ICollection': + // const implClsName = JavaTypes.shortClassName(prop.implClsName); + + const colTypeClsName = JavaTypes.shortClassName(prop.typeClsName); + + if (colTypeClsName === 'String') { + const items = this._toObject(colTypeClsName, prop.items); + + sb.append(`${bean.id}.${_.upperFirst(prop.name)} = new {${items.join(', ')}};`); + } + // else { + // if (_.includes(vars, prop.id)) + // sb.append(`${prop.id} = new ${implClsName}<>();`); + // else { + // vars.push(prop.id); + // + // sb.append(`${clsName}<${colTypeClsName}> ${prop.id} = new ${implClsName}<>();`); + // } + // + // sb.emptyLine(); + // + // if (nonBean) { + // const items = this._toObject(colTypeClsName, prop.items); + // + // _.forEach(items, (item) => { + // sb.append(`${prop.id}.add("${item}");`); + // + // sb.emptyLine(); + // }); + // } + // else { + // _.forEach(prop.items, (item) => { + // this.constructBean(sb, item, vars, limitLines); + // + // sb.append(`${prop.id}.add(${item.id});`); + // + // sb.emptyLine(); + // }); + // + // this._setProperty(sb, bean.id, prop.name, prop.id); + // } + // } + + break; + + case 'Bean': + const nestedBean = prop.value; + + this._defineBean(sb, nestedBean); + + sb.emptyLine(); + + this._setProperties(sb, nestedBean); + + sb.emptyLine(); + + this._setBeanProperty(sb, bean, prop.name, nestedBean); + + break; + default: + this._setProperty(sb, bean, prop.name, this._toObject(prop.clsName, prop.value)); + } + }); + + return sb; + } + + /** + * Build Java startup class with configuration. + * + * @param {Bean} cfg + * @param pkg Package name. + * @param clsName Class name for generate factory class otherwise generate code snippet. + * @param clientNearCfg Optional near cache configuration for client node. + * @returns {String} + */ + static toClassFile(cfg, pkg, clsName) { + const sb = new StringBuilder(); + + sb.startBlock(`namespace ${pkg}`, '{'); + + _.forEach(_.sortBy(cfg.collectClasses()), (cls) => sb.append(`using ${cls};`)); + sb.emptyLine(); + + + this.mainComment(sb); + sb.startBlock(`public class ${clsName}`, '{'); + + this.doc(sb, 'Configure grid.'); + sb.startBlock('public static IgniteConfiguration CreateConfiguration()', '{'); + + this._defineBean(sb, cfg); + + sb.emptyLine(); + + this._setProperties(sb, cfg); + + sb.emptyLine(); + + sb.append(`return ${cfg.id};`); + + sb.endBlock('}'); + + sb.endBlock('}'); + + sb.endBlock('}'); + + return sb.asString(); + } + + static generateSection(bean) { + const sb = new StringBuilder(); + + this._setProperties(sb, bean); + + return sb.asString(); + } + }; +}]; diff --git a/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js new file mode 100644 index 0000000000000..91a85fa65c998 --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js @@ -0,0 +1,325 @@ +/* + * 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. + */ + +import _ from 'lodash'; + +import AbstractTransformer from './AbstractTransformer'; +import StringBuilder from './StringBuilder'; + +export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator', (JavaTypes, eventGroups, generator) => { + return class SpringTransformer extends AbstractTransformer { + static generator = generator; + + static commentBlock(sb, ...lines) { + if (lines.length > 1) { + sb.append(''); + } + else + sb.append(``); + } + + static appendBean(sb, bean, appendId) { + const beanTags = []; + + if (appendId) + beanTags.push(`id="${bean.id}"`); + + beanTags.push(`class="${bean.clsName}"`); + + if (bean.factoryMtd) + beanTags.push(`factory-method="${bean.factoryMtd}"`); + + sb.startBlock(``); + + _.forEach(bean.arguments, (arg) => { + if (arg.clsName === 'MAP') { + sb.startBlock(''); + this._constructMap(sb, arg); + sb.endBlock(''); + } + else if (_.isNil(arg.value)) { + sb.startBlock(''); + sb.append(''); + sb.endBlock(''); + } + else if (arg.constant) { + sb.startBlock(''); + sb.append(``); + sb.endBlock(''); + } + else if (arg.clsName === 'BEAN') { + sb.startBlock(''); + this.appendBean(sb, arg.value); + sb.endBlock(''); + } + else + sb.append(``); + }); + + this._setProperties(sb, bean); + + sb.endBlock(''); + } + + static _toObject(clsName, items) { + return _.map(_.isArray(items) ? items : [items], (item) => { + switch (clsName) { + case 'PROPERTY': + case 'PROPERTY_CHAR': + return `\${${item}}`; + case 'java.lang.Class': + return JavaTypes.fullClassName(item); + case 'long': + return `${item}L`; + default: + return item; + } + }); + } + + static _isBean(clsName) { + return JavaTypes.nonBuiltInClass(clsName) && JavaTypes.nonEnum(clsName) && _.includes(clsName, '.'); + } + + static _setCollection(sb, prop, tag) { + sb.startBlock(``); + sb.startBlock(`<${tag}>`); + + _.forEach(prop.items, (item, idx) => { + if (this._isBean(prop.typeClsName)) { + if (idx !== 0) + sb.emptyLine(); + + this.appendBean(sb, item); + } + else + sb.append(`${item}`); + }); + + sb.endBlock(``); + sb.endBlock(''); + } + + static _constructMap(sb, map) { + sb.startBlock(''); + + _.forEach(map.entries, (entry) => { + const key = entry[map.keyField]; + const val = entry[map.valField]; + + const isKeyBean = this._isBean(map.keyClsName); + const isValBean = this._isBean(map.valClsName); + + + if (isKeyBean || isValBean) { + sb.startBlock(''); + + sb.startBlock(''); + if (isKeyBean) + this.appendBean(sb, key); + else + sb.append(this._toObject(map.keyClsName, key)); + sb.endBlock(''); + + sb.startBlock(''); + if (isValBean) + this.appendBean(sb, val); + else + sb.append(this._toObject(map.valClsName, val)); + sb.endBlock(''); + + sb.endBlock(''); + } + else + sb.append(``); + }); + + sb.endBlock(''); + } + + /** + * + * @param {StringBuilder} sb + * @param {Bean} bean + * @returns {StringBuilder} + */ + static _setProperties(sb, bean) { + _.forEach(bean.properties, (prop, idx) => { + switch (prop.clsName) { + case 'DATA_SOURCE': + const valAttr = prop.name === 'dataSource' ? 'ref' : 'value'; + + sb.append(``); + + break; + case 'EVENT_TYPES': + sb.startBlock(``); + + if (prop.eventTypes.length === 1) { + const evtGrp = _.find(eventGroups, {value: _.head(prop.eventTypes)}); + + evtGrp && sb.append(``); + } + else { + sb.startBlock(''); + + _.forEach(prop.eventTypes, (item, ix) => { + ix > 0 && sb.emptyLine(); + + const evtGrp = _.find(eventGroups, {value: item}); + + if (evtGrp) { + sb.append(``); + + _.forEach(evtGrp.events, (event) => + sb.append(``)); + } + }); + + sb.endBlock(''); + } + + sb.endBlock(''); + + break; + case 'ARRAY': + this._setCollection(sb, prop, 'array'); + + break; + case 'COLLECTION': + this._setCollection(sb, prop, 'list'); + + break; + case 'MAP': + sb.startBlock(``); + + this._constructMap(sb, prop); + + sb.endBlock(''); + + break; + case 'java.util.Properties': + sb.startBlock(``); + sb.startBlock(''); + + _.forEach(prop.entries, (entry) => { + sb.append(`${entry.value}`); + }); + + sb.endBlock(''); + sb.endBlock(''); + + break; + case 'BEAN': + sb.startBlock(``); + + this.appendBean(sb, prop.value); + + sb.endBlock(''); + + break; + default: + sb.append(``); + } + + this._emptyLineIfNeeded(sb, bean.properties, idx); + }); + + return sb; + } + + /** + * Build final XML. + * + * @param {Bean} cfg Ignite configuration. + * @param {Boolean} clientNearCaches + * @returns {StringBuilder} + */ + static igniteConfiguration(cfg, clientNearCaches) { + const sb = new StringBuilder(); + + // 0. Add header. + sb.append(''); + sb.emptyLine(); + + this.mainComment(sb); + sb.emptyLine(); + + // 1. Start beans section. + sb.startBlock([ + '']); + + // 2. Add external property file + if (this.hasProperties(cfg)) { + this.commentBlock(sb, 'Load external properties file.'); + + sb.startBlock(''); + sb.append(''); + sb.endBlock(''); + + sb.emptyLine(); + } + + // 3. Add data sources. + const dataSources = this.collectDataSources(cfg); + + if (dataSources.length) { + this.commentBlock(sb, 'Data source beans will be initialized from external properties file.'); + + _.forEach(dataSources, (ds) => { + this.appendBean(sb, ds, true); + + sb.emptyLine(); + }); + } + + _.forEach(clientNearCaches, (cache) => { + this.commentBlock(sb, 'Configuration of near cache for cache "' + cache.name + '"'); + + this.appendBean(sb, generator.cacheNearClient(cache), true); + + sb.emptyLine(); + }); + + // 3. Add main content. + this.appendBean(sb, cfg); + + // 4. Close beans section. + sb.endBlock(''); + + return sb; + } + + static cluster(cluster, client) { + const cfg = generator.igniteConfiguration(cluster, client); + + const clientNearCaches = client ? _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')) : []; + + return this.igniteConfiguration(cfg, clientNearCaches); + } + }; +}]; diff --git a/modules/web-console/frontend/app/modules/configuration/generator/StringBuilder.js b/modules/web-console/frontend/app/modules/configuration/generator/StringBuilder.js new file mode 100644 index 0000000000000..338bf1ebc809d --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/StringBuilder.js @@ -0,0 +1,76 @@ +/* + * 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. + */ + +const DATE_OPTS = { + month: '2-digit', + day: '2-digit', + year: 'numeric', + hour: '2-digit', + minute: '2-digit', + hour12: false +}; + +export default class StringBuilder { + generatedBy() { + return `This file was generated by Ignite Web Console (${new Date().toLocaleString('en-US', DATE_OPTS)})`; + } + + /** + * @param deep + * @param indent + */ + constructor(deep = 0, indent = 4) { + this.indent = indent; + this.deep = deep; + this.lines = []; + } + + emptyLine() { + this.lines.push(''); + + return this; + } + + append(lines) { + if (_.isArray(lines)) + _.forEach(lines, (line) => this.lines.push(_.repeat(' ', this.indent * this.deep) + line)); + else + this.lines.push(_.repeat(' ', this.indent * this.deep) + lines); + + return this; + } + + startBlock(lines) { + this.append(lines); + + this.deep++; + + return this; + } + + endBlock(line) { + this.deep--; + + this.append(line); + + return this; + } + + asString() { + return this.lines.join('\n'); + } +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js new file mode 100644 index 0000000000000..f06e11b5f57be --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js @@ -0,0 +1,60 @@ +/* + * 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. + */ + +import _ from 'lodash'; + +const enumValueMapper = (val) => _.capitalize(val); + +const DFLT_CACHE = { + cacheMode: { + clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMode', + mapper: enumValueMapper + }, + atomicityMode: { + clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheAtomicityMode', + mapper: enumValueMapper + }, + memoryMode: { + clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMemoryMode', + value: 'ONHEAP_TIERED', + mapper: enumValueMapper + }, + atomicWriteOrderMode: { + clsName: 'org.apache.ignite.cache.CacheAtomicWriteOrderMode', + mapper: enumValueMapper + }, + writeSynchronizationMode: { + clsName: 'org.apache.ignite.cache.CacheWriteSynchronizationMode', + value: 'PRIMARY_SYNC', + mapper: enumValueMapper + }, + rebalanceMode: { + clsName: 'org.apache.ignite.cache.CacheRebalanceMode', + value: 'ASYNC', + mapper: enumValueMapper + } +}; + +export default function() { + this.append = (dflts) => { + _.merge(DFLT_CACHE, dflts); + }; + + this.$get = ['igniteCacheDefaults', (cacheDefaults) => { + return _.merge({}, cacheDefaults, DFLT_CACHE); + }]; +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js new file mode 100644 index 0000000000000..5ff1e02c63a4d --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import _ from 'lodash'; + +const DFLT_CACHE = { + cacheMode: { + clsName: 'org.apache.ignite.cache.CacheMode' + }, + atomicityMode: { + clsName: 'org.apache.ignite.cache.CacheAtomicityMode' + }, + memoryMode: { + clsName: 'org.apache.ignite.cache.CacheMemoryMode', + value: 'ONHEAP_TIERED' + }, + offHeapMaxMemory: -1, + startSize: 1500000, + swapEnabled: false, + sqlOnheapRowCacheSize: 10240, + longQueryWarningTimeout: 3000, + snapshotableIndex: false, + sqlEscapeAll: false, + storeKeepBinary: false, + loadPreviousValue: false, + readThrough: false, + writeThrough: false, + writeBehindEnabled: false, + writeBehindBatchSize: 512, + writeBehindFlushSize: 10240, + writeBehindFlushFrequency: 5000, + writeBehindFlushThreadCount: 1, + maxConcurrentAsyncOperations: 500, + defaultLockTimeout: 0, + atomicWriteOrderMode: { + clsName: 'org.apache.ignite.cache.CacheAtomicWriteOrderMode' + }, + writeSynchronizationMode: { + clsName: 'org.apache.ignite.cache.CacheWriteSynchronizationMode', + value: 'PRIMARY_SYNC' + }, + rebalanceMode: { + clsName: 'org.apache.ignite.cache.CacheRebalanceMode', + value: 'ASYNC' + }, + rebalanceThreadPoolSize: 1, + rebalanceBatchSize: 524288, + rebalanceBatchesPrefetchCount: 2, + rebalanceOrder: 0, + rebalanceDelay: 0, + rebalanceTimeout: 10000, + rebalanceThrottle: 0, + statisticsEnabled: false, + managementEnabled: false, + nearConfiguration: { + nearStartSize: 375000 + }, + clientNearConfiguration: { + nearStartSize: 375000 + }, + evictionPolicy: { + LRU: { + batchSize: 1, + maxSize: 100000 + }, + FIFO: { + batchSize: 1, + maxSize: 100000 + }, + SORTED: { + batchSize: 1, + maxSize: 100000 + } + }, + queryMetadata: 'Configuration', + fields: { + keyClsName: 'java.lang.String', + valClsName: 'java.lang.String', + valField: 'className', + entries: [] + }, + aliases: { + keyClsName: 'java.lang.String', + valClsName: 'java.lang.String', + keyField: 'field', + valField: 'alias', + entries: [] + }, + indexes: { + indexType: { + clsName: 'org.apache.ignite.cache.QueryIndexType' + }, + fields: { + keyClsName: 'java.lang.String', + valClsName: 'java.lang.Boolean', + valField: 'direction', + entries: [] + } + }, + typeField: { + databaseFieldType: { + clsName: 'java.sql.Types' + } + } +}; + +export default function() { + this.append = (dflts) => { + _.merge(DFLT_CACHE, dflts); + }; + + this.$get = [() => { + return DFLT_CACHE; + }]; +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js new file mode 100644 index 0000000000000..582426ec8e7b4 --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js @@ -0,0 +1,49 @@ +/* + * 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. + */ + +import _ from 'lodash'; + +const enumValueMapper = (val) => _.capitalize(val); + +const DFLT_CLUSTER = { + atomics: { + cacheMode: { + clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMode', + mapper: enumValueMapper + } + }, + transactionConfiguration: { + defaultTxConcurrency: { + clsName: 'Apache.Ignite.Core.Transactions.TransactionConcurrency', + mapper: enumValueMapper + }, + defaultTxIsolation: { + clsName: 'Apache.Ignite.Core.Transactions.TransactionIsolation', + mapper: enumValueMapper + } + } +}; + +export default function() { + this.append = (dflts) => { + _.merge(DFLT_CLUSTER, dflts); + }; + + this.$get = ['igniteClusterDefaults', (clusterDefaults) => { + return _.merge({}, clusterDefaults, DFLT_CLUSTER); + }]; +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js new file mode 100644 index 0000000000000..1be4b701bda73 --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js @@ -0,0 +1,293 @@ +/* + * 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. + */ + +const DFLT_CLUSTER = { + localHost: '0.0.0.0', + discovery: { + localPort: 47500, + localPortRange: 100, + socketTimeout: 5000, + ackTimeout: 5000, + maxAckTimeout: 600000, + networkTimeout: 5000, + joinTimeout: 0, + threadPriority: 10, + heartbeatFrequency: 2000, + maxMissedHeartbeats: 1, + maxMissedClientHeartbeats: 5, + topHistorySize: 1000, + reconnectCount: 10, + statisticsPrintFrequency: 0, + ipFinderCleanFrequency: 60000, + forceServerMode: false, + clientReconnectDisabled: false, + Multicast: { + multicastGroup: '228.1.2.4', + multicastPort: 47400, + responseWaitTime: 500, + addressRequestAttempts: 2, + localAddress: '0.0.0.0' + }, + Jdbc: { + initSchema: false + }, + SharedFs: { + path: 'disco/tcp' + }, + ZooKeeper: { + basePath: '/services', + serviceName: 'ignite', + allowDuplicateRegistrations: false, + ExponentialBackoff: { + baseSleepTimeMs: 1000, + maxRetries: 10 + }, + BoundedExponentialBackoffRetry: { + baseSleepTimeMs: 1000, + maxSleepTimeMs: 2147483647, + maxRetries: 10 + }, + UntilElapsed: { + maxElapsedTimeMs: 60000, + sleepMsBetweenRetries: 1000 + }, + RetryNTimes: { + n: 10, + sleepMsBetweenRetries: 1000 + }, + OneTime: { + sleepMsBetweenRetry: 1000 + }, + Forever: { + retryIntervalMs: 1000 + } + } + }, + atomics: { + atomicSequenceReserveSize: 1000, + backups: 0, + cacheMode: { + clsName: 'org.apache.ignite.cache.CacheMode', + value: 'PARTITIONED' + } + }, + binary: { + compactFooter: true, + typeConfigurations: { + enum: false + } + }, + collision: { + kind: null, + JobStealing: { + activeJobsThreshold: 95, + waitJobsThreshold: 0, + messageExpireTime: 1000, + maximumStealingAttempts: 5, + stealingEnabled: true, + stealingAttributes: { + keyClsName: 'java.lang.String', + valClsName: 'java.io.Serializable', + items: [] + } + }, + PriorityQueue: { + priorityAttributeKey: 'grid.task.priority', + jobPriorityAttributeKey: 'grid.job.priority', + defaultPriority: 0, + starvationIncrement: 1, + starvationPreventionEnabled: true + } + }, + communication: { + localPort: 47100, + localPortRange: 100, + sharedMemoryPort: 48100, + directBuffer: false, + directSendBuffer: false, + idleConnectionTimeout: 30000, + connectTimeout: 5000, + maxConnectTimeout: 600000, + reconnectCount: 10, + socketSendBuffer: 32768, + socketReceiveBuffer: 32768, + messageQueueLimit: 1024, + tcpNoDelay: true, + ackSendThreshold: 16, + unacknowledgedMessagesBufferSize: 0, + socketWriteTimeout: 2000 + }, + networkTimeout: 5000, + networkSendRetryDelay: 1000, + networkSendRetryCount: 3, + discoveryStartupDelay: 60000, + connector: { + port: 11211, + portRange: 100, + idleTimeout: 7000, + idleQueryCursorTimeout: 600000, + idleQueryCursorCheckFrequency: 60000, + receiveBufferSize: 32768, + sendBufferSize: 32768, + sendQueueLimit: 0, + directBuffer: false, + noDelay: true, + sslEnabled: false, + sslClientAuth: false + }, + deploymentMode: { + clsName: 'org.apache.ignite.configuration.DeploymentMode', + value: 'SHARED' + }, + peerClassLoadingEnabled: false, + peerClassLoadingMissedResourcesCacheSize: 100, + peerClassLoadingThreadPoolSize: 2, + failoverSpi: { + JobStealing: { + maximumFailoverAttempts: 5 + }, + Always: { + maximumFailoverAttempts: 5 + } + }, + logger: { + Log4j: { + level: { + clsName: 'org.apache.logging.log4j.Level' + } + }, + Log4j2: { + level: { + clsName: 'org.apache.logging.log4j.Level' + } + } + }, + marshalLocalJobs: false, + marshallerCacheKeepAliveTime: 10000, + metricsHistorySize: 10000, + metricsLogFrequency: 60000, + metricsUpdateFrequency: 2000, + clockSyncSamples: 8, + clockSyncFrequency: 120000, + timeServerPortBase: 31100, + timeServerPortRange: 100, + transactionConfiguration: { + defaultTxConcurrency: { + clsName: 'org.apache.ignite.transactions.TransactionConcurrency', + value: 'PESSIMISTIC' + }, + defaultTxIsolation: { + clsName: 'org.apache.ignite.transactions.TransactionIsolation', + value: 'REPEATABLE_READ' + }, + defaultTxTimeout: 0, + pessimisticTxLogLinger: 10000 + }, + attributes: { + keyClsName: 'java.lang.String', + valClsName: 'java.lang.String', + items: [] + }, + odbcConfiguration: { + endpointAddress: '0.0.0.0:10800..10810', + maxOpenCursors: 128 + }, + eventStorage: { + Memory: { + expireCount: 10000 + } + }, + checkpointSpi: { + S3: { + bucketNameSuffix: 'default-bucket', + clientConfiguration: { + protocol: { + clsName: 'com.amazonaws.Protocol', + value: 'HTTPS' + }, + maxConnections: 50, + retryPolicy: { + retryCondition: { + clsName: 'com.amazonaws.retry.PredefinedRetryPolicies' + }, + backoffStrategy: { + clsName: 'com.amazonaws.retry.PredefinedRetryPolicies' + }, + maxErrorRetry: { + clsName: 'com.amazonaws.retry.PredefinedRetryPolicies' + } + }, + maxErrorRetry: -1, + socketTimeout: 50000, + connectionTimeout: 50000, + requestTimeout: 0, + socketSendBufferSizeHints: 0, + connectionTTL: -1, + connectionMaxIdleMillis: 60000, + responseMetadataCacheSize: 50, + useReaper: true, + useGzip: false, + preemptiveBasicProxyAuth: false, + useTcpKeepAlive: false + } + }, + JDBC: { + checkpointTableName: 'CHECKPOINTS', + keyFieldName: 'NAME', + keyFieldType: 'VARCHAR', + valueFieldName: 'VALUE', + valueFieldType: 'BLOB', + expireDateFieldName: 'EXPIRE_DATE', + expireDateFieldType: 'DATETIME', + numberOfRetries: 2 + } + }, + loadBalancingSpi: { + RoundRobin: { + perTask: false + }, + Adaptive: { + loadProbe: { + Job: { + useAverage: true + }, + CPU: { + useAverage: true, + useProcessors: true, + processorCoefficient: 1 + }, + ProcessingTime: { + useAverage: true + } + } + }, + WeightedRandom: { + nodeWeight: 10, + useWeights: false + } + } +}; + +export default function() { + this.append = (dflts) => { + _.merge(DFLT_CLUSTER, dflts); + }; + + this.$get = [() => { + return DFLT_CLUSTER; + }]; +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js new file mode 100644 index 0000000000000..c556336a3953b --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js @@ -0,0 +1,68 @@ +/* + * 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. + */ + +const DFLT_IGFS = { + defaultMode: { + clsName: 'org.apache.ignite.igfs.IgfsMode', + value: 'DUAL_ASYNC' + }, + secondaryFileSystem: { + + }, + ipcEndpointConfiguration: { + type: { + clsName: 'org.apache.ignite.igfs.IgfsIpcEndpointType' + }, + host: '127.0.0.1', + port: 10500, + memorySize: 262144, + tokenDirectoryPath: 'ipc/shmem' + }, + fragmentizerConcurrentFiles: 0, + fragmentizerThrottlingBlockLength: 16777216, + fragmentizerThrottlingDelay: 200, + dualModeMaxPendingPutsSize: 0, + dualModePutExecutorServiceShutdown: false, + blockSize: 65536, + streamBufferSize: 65536, + maxSpaceSize: 0, + maximumTaskRangeLength: 0, + managementPort: 11400, + perNodeBatchSize: 100, + perNodeParallelBatchCount: 8, + prefetchBlocks: 0, + sequentialReadsBeforePrefetch: 0, + trashPurgeTimeout: 1000, + colocateMetadata: true, + relaxedConsistency: true, + pathModes: { + keyClsName: 'java.lang.String', + keyField: 'path', + valClsName: 'org.apache.ignite.igfs.IgfsMode', + valField: 'mode' + } +}; + +export default function() { + this.append = (dflts) => { + _.merge(DFLT_IGFS, dflts); + }; + + this.$get = [() => { + return DFLT_IGFS; + }]; +} diff --git a/modules/web-console/frontend/generator/generator-common.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-common.js similarity index 100% rename from modules/web-console/frontend/generator/generator-common.js rename to modules/web-console/frontend/app/modules/configuration/generator/generator-common.js diff --git a/modules/web-console/frontend/generator/generator-java.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-java.js similarity index 99% rename from modules/web-console/frontend/generator/generator-java.js rename to modules/web-console/frontend/app/modules/configuration/generator/generator-java.js index db54928e4f53e..296b942c42ec0 100644 --- a/modules/web-console/frontend/generator/generator-java.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/generator-java.js @@ -1637,6 +1637,12 @@ $generatorJava.cacheStore = function(cache, domains, cacheVarName, res) { res.needEmptyLine = true; + if (storeFactory.sqlEscapeAll) { + res.line(varName + '.setSqlEscapeAll(true);'); + + res.needEmptyLine = true; + } + const domainConfigs = _.filter(domains, function(domain) { return $generatorCommon.domainQueryMetadata(domain) === 'Configuration' && $generatorCommon.isDefinedAndNotEmpty(domain.databaseTable); @@ -2565,10 +2571,10 @@ $generatorJava.javaClassCode = function(domain, key, pkg, useConstructor, includ $generatorJava.pojos = function(caches, useConstructor, includeKeyFields) { const pojos = []; - _.forEach(caches, function(cache) { - _.forEach(cache.domains, function(domain) { - // Skip already generated classes. - if (!_.find(pojos, {valueType: domain.valueType}) && + _.forEach(caches, (cache) => { + _.forEach(cache.domains, (domain) => { + // Process only domains with 'generatePojo' flag and skip already generated classes. + if (domain.generatePojo && !_.find(pojos, {valueType: domain.valueType}) && // Skip domain models without value fields. $generatorCommon.isDefinedAndNotEmpty(domain.valueFields)) { const pojo = {}; diff --git a/modules/web-console/frontend/generator/generator-optional.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-optional.js similarity index 100% rename from modules/web-console/frontend/generator/generator-optional.js rename to modules/web-console/frontend/app/modules/configuration/generator/generator-optional.js diff --git a/modules/web-console/frontend/generator/generator-xml.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js similarity index 63% rename from modules/web-console/frontend/generator/generator-xml.js rename to modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js index ded93af048294..f70c66f5c91e1 100644 --- a/modules/web-console/frontend/generator/generator-xml.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js @@ -16,10 +16,10 @@ */ // XML generation entry point. -const $generatorXml = {}; +const $generatorSpring = {}; // Do XML escape. -$generatorXml.escape = function(s) { +$generatorSpring.escape = function(s) { if (typeof (s) !== 'string') return s; @@ -27,7 +27,7 @@ $generatorXml.escape = function(s) { }; // Add constructor argument -$generatorXml.constructorArg = function(res, ix, obj, propName, dflt, opt) { +$generatorSpring.constructorArg = function(res, ix, obj, propName, dflt, opt) { const v = (obj ? obj[propName] : null) || dflt; if ($generatorCommon.isDefinedAndNotEmpty(v)) @@ -40,7 +40,7 @@ $generatorXml.constructorArg = function(res, ix, obj, propName, dflt, opt) { }; // Add XML element. -$generatorXml.element = function(res, tag, attr1, val1, attr2, val2) { +$generatorSpring.element = function(res, tag, attr1, val1, attr2, val2) { let elem = '<' + tag; if (attr1) @@ -56,7 +56,7 @@ $generatorXml.element = function(res, tag, attr1, val1, attr2, val2) { }; // Add property. -$generatorXml.property = function(res, obj, propName, setterName, dflt) { +$generatorSpring.property = function(res, obj, propName, setterName, dflt) { if (!_.isNil(obj)) { const val = obj[propName]; @@ -65,7 +65,7 @@ $generatorXml.property = function(res, obj, propName, setterName, dflt) { // Add to result if no default provided or value not equals to default. if (missDflt || (!missDflt && val !== dflt)) { - $generatorXml.element(res, 'property', 'name', setterName ? setterName : propName, 'value', $generatorXml.escape(val)); + $generatorSpring.element(res, 'property', 'name', setterName ? setterName : propName, 'value', $generatorSpring.escape(val)); return true; } @@ -76,15 +76,15 @@ $generatorXml.property = function(res, obj, propName, setterName, dflt) { }; // Add property for class name. -$generatorXml.classNameProperty = function(res, obj, propName) { +$generatorSpring.classNameProperty = function(res, obj, propName) { const val = obj[propName]; if (!_.isNil(val)) - $generatorXml.element(res, 'property', 'name', propName, 'value', $generatorCommon.JavaTypes.fullClassName(val)); + $generatorSpring.element(res, 'property', 'name', propName, 'value', $generatorCommon.JavaTypes.fullClassName(val)); }; // Add list property. -$generatorXml.listProperty = function(res, obj, propName, listType, rowFactory) { +$generatorSpring.listProperty = function(res, obj, propName, listType, rowFactory) { const val = obj[propName]; if (val && val.length > 0) { @@ -94,7 +94,7 @@ $generatorXml.listProperty = function(res, obj, propName, listType, rowFactory) listType = 'list'; if (!rowFactory) - rowFactory = (v) => '' + $generatorXml.escape(v) + ''; + rowFactory = (v) => '' + $generatorSpring.escape(v) + ''; res.startBlock(''); res.startBlock('<' + listType + '>'); @@ -109,7 +109,7 @@ $generatorXml.listProperty = function(res, obj, propName, listType, rowFactory) }; // Add array property -$generatorXml.arrayProperty = function(res, obj, propName, descr, rowFactory) { +$generatorSpring.arrayProperty = function(res, obj, propName, descr, rowFactory) { const val = obj[propName]; if (val && val.length > 0) { @@ -137,7 +137,7 @@ $generatorXml.arrayProperty = function(res, obj, propName, descr, rowFactory) { * @param desc Bean metadata object. * @param createBeanAlthoughNoProps Always generate bean even it has no properties defined. */ -$generatorXml.beanProperty = function(res, bean, beanPropName, desc, createBeanAlthoughNoProps) { +$generatorSpring.beanProperty = function(res, bean, beanPropName, desc, createBeanAlthoughNoProps) { const props = desc.fields; if (bean && $generatorCommon.hasProperty(bean, props)) { @@ -159,12 +159,12 @@ $generatorXml.beanProperty = function(res, bean, beanPropName, desc, createBeanA if (descr) { switch (descr.type) { case 'list': - $generatorXml.listProperty(res, bean, propName, descr.setterName); + $generatorSpring.listProperty(res, bean, propName, descr.setterName); break; case 'array': - $generatorXml.arrayProperty(res, bean, propName, descr); + $generatorSpring.arrayProperty(res, bean, propName, descr); break; @@ -178,8 +178,8 @@ $generatorXml.beanProperty = function(res, bean, beanPropName, desc, createBeanA _.forEach(val, function(nameAndValue) { const eqIndex = nameAndValue.indexOf('='); if (eqIndex >= 0) { - res.line('' + - $generatorXml.escape(nameAndValue.substr(eqIndex + 1)) + ''); + res.line('' + + $generatorSpring.escape(nameAndValue.substr(eqIndex + 1)) + ''); } }); @@ -203,12 +203,12 @@ $generatorXml.beanProperty = function(res, bean, beanPropName, desc, createBeanA break; default: - if ($generatorXml.property(res, bean, propName, descr.setterName, descr.dflt)) + if ($generatorSpring.property(res, bean, propName, descr.setterName, descr.dflt)) hasData = true; } } else - if ($generatorXml.property(res, bean, propName)) + if ($generatorSpring.property(res, bean, propName)) hasData = true; } }); @@ -241,7 +241,7 @@ $generatorXml.beanProperty = function(res, bean, beanPropName, desc, createBeanA * @param obj Object to take bean class name. * @param propName Property name. */ -$generatorXml.simpleBeanProperty = function(res, obj, propName) { +$generatorSpring.simpleBeanProperty = function(res, obj, propName) { if (!_.isNil(obj)) { const val = obj[propName]; @@ -256,20 +256,20 @@ $generatorXml.simpleBeanProperty = function(res, obj, propName) { }; // Generate eviction policy. -$generatorXml.evictionPolicy = function(res, evtPlc, propName) { +$generatorSpring.evictionPolicy = function(res, evtPlc, propName) { if (evtPlc && evtPlc.kind) { - $generatorXml.beanProperty(res, evtPlc[evtPlc.kind.toUpperCase()], propName, + $generatorSpring.beanProperty(res, evtPlc[evtPlc.kind.toUpperCase()], propName, $generatorCommon.EVICTION_POLICIES[evtPlc.kind], true); } }; // Generate discovery. -$generatorXml.clusterGeneral = function(cluster, res) { +$generatorSpring.clusterGeneral = function(cluster, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.property(res, cluster, 'name', 'gridName'); - $generatorXml.property(res, cluster, 'localHost'); + $generatorSpring.property(res, cluster, 'name', 'gridName'); + $generatorSpring.property(res, cluster, 'localHost'); if (cluster.discovery) { res.startBlock(''); @@ -283,12 +283,12 @@ $generatorXml.clusterGeneral = function(cluster, res) { res.startBlock(''); if (d.Multicast) { - $generatorXml.property(res, d.Multicast, 'multicastGroup'); - $generatorXml.property(res, d.Multicast, 'multicastPort'); - $generatorXml.property(res, d.Multicast, 'responseWaitTime'); - $generatorXml.property(res, d.Multicast, 'addressRequestAttempts'); - $generatorXml.property(res, d.Multicast, 'localAddress'); - $generatorXml.listProperty(res, d.Multicast, 'addresses'); + $generatorSpring.property(res, d.Multicast, 'multicastGroup'); + $generatorSpring.property(res, d.Multicast, 'multicastPort'); + $generatorSpring.property(res, d.Multicast, 'responseWaitTime'); + $generatorSpring.property(res, d.Multicast, 'addressRequestAttempts'); + $generatorSpring.property(res, d.Multicast, 'localAddress'); + $generatorSpring.listProperty(res, d.Multicast, 'addresses'); } res.endBlock(''); @@ -299,7 +299,7 @@ $generatorXml.clusterGeneral = function(cluster, res) { res.startBlock(''); if (d.Vm) - $generatorXml.listProperty(res, d.Vm, 'addresses'); + $generatorSpring.listProperty(res, d.Vm, 'addresses'); res.endBlock(''); @@ -310,7 +310,7 @@ $generatorXml.clusterGeneral = function(cluster, res) { if (d.S3) { if (d.S3.bucketName) - res.line(''); + res.line(''); } res.endBlock(''); @@ -321,12 +321,12 @@ $generatorXml.clusterGeneral = function(cluster, res) { res.startBlock(''); if (d.Cloud) { - $generatorXml.property(res, d.Cloud, 'credential'); - $generatorXml.property(res, d.Cloud, 'credentialPath'); - $generatorXml.property(res, d.Cloud, 'identity'); - $generatorXml.property(res, d.Cloud, 'provider'); - $generatorXml.listProperty(res, d.Cloud, 'regions'); - $generatorXml.listProperty(res, d.Cloud, 'zones'); + $generatorSpring.property(res, d.Cloud, 'credential'); + $generatorSpring.property(res, d.Cloud, 'credentialPath'); + $generatorSpring.property(res, d.Cloud, 'identity'); + $generatorSpring.property(res, d.Cloud, 'provider'); + $generatorSpring.listProperty(res, d.Cloud, 'regions'); + $generatorSpring.listProperty(res, d.Cloud, 'zones'); } res.endBlock(''); @@ -337,10 +337,10 @@ $generatorXml.clusterGeneral = function(cluster, res) { res.startBlock(''); if (d.GoogleStorage) { - $generatorXml.property(res, d.GoogleStorage, 'projectName'); - $generatorXml.property(res, d.GoogleStorage, 'bucketName'); - $generatorXml.property(res, d.GoogleStorage, 'serviceAccountP12FilePath'); - $generatorXml.property(res, d.GoogleStorage, 'serviceAccountId'); + $generatorSpring.property(res, d.GoogleStorage, 'projectName'); + $generatorSpring.property(res, d.GoogleStorage, 'bucketName'); + $generatorSpring.property(res, d.GoogleStorage, 'serviceAccountP12FilePath'); + $generatorSpring.property(res, d.GoogleStorage, 'serviceAccountId'); } res.endBlock(''); @@ -358,7 +358,7 @@ $generatorXml.clusterGeneral = function(cluster, res) { if (datasource.dataSourceBean && datasource.dialect) { res.line(''); - if (_.findIndex(res.datasources, (ds) => ds.dataSourceBean === datasource.dataSourceBean) < 0) { + if (!_.find(res.datasources, { dataSourceBean: datasource.dataSourceBean })) { res.datasources.push({ dataSourceBean: datasource.dataSourceBean, dialect: datasource.dialect @@ -375,7 +375,7 @@ $generatorXml.clusterGeneral = function(cluster, res) { res.startBlock(''); if (d.SharedFs) - $generatorXml.property(res, d.SharedFs, 'path'); + $generatorSpring.property(res, d.SharedFs, 'path'); res.endBlock(''); @@ -391,7 +391,7 @@ $generatorXml.clusterGeneral = function(cluster, res) { res.endBlock(''); } - $generatorXml.property(res, d.ZooKeeper, 'zkConnectionString'); + $generatorSpring.property(res, d.ZooKeeper, 'zkConnectionString'); if (d.ZooKeeper.retryPolicy && d.ZooKeeper.retryPolicy.kind) { const kind = d.ZooKeeper.retryPolicy.kind; @@ -404,48 +404,48 @@ $generatorXml.clusterGeneral = function(cluster, res) { switch (kind) { case 'ExponentialBackoff': res.startBlock(''); - $generatorXml.constructorArg(res, 0, retryPolicy, 'baseSleepTimeMs', 1000); - $generatorXml.constructorArg(res, 1, retryPolicy, 'maxRetries', 10); - $generatorXml.constructorArg(res, 2, retryPolicy, 'maxSleepMs', null, true); + $generatorSpring.constructorArg(res, 0, retryPolicy, 'baseSleepTimeMs', 1000); + $generatorSpring.constructorArg(res, 1, retryPolicy, 'maxRetries', 10); + $generatorSpring.constructorArg(res, 2, retryPolicy, 'maxSleepMs', null, true); res.endBlock(''); break; case 'BoundedExponentialBackoff': res.startBlock(''); - $generatorXml.constructorArg(res, 0, retryPolicy, 'baseSleepTimeMs', 1000); - $generatorXml.constructorArg(res, 1, retryPolicy, 'maxSleepTimeMs', 2147483647); - $generatorXml.constructorArg(res, 2, retryPolicy, 'maxRetries', 10); + $generatorSpring.constructorArg(res, 0, retryPolicy, 'baseSleepTimeMs', 1000); + $generatorSpring.constructorArg(res, 1, retryPolicy, 'maxSleepTimeMs', 2147483647); + $generatorSpring.constructorArg(res, 2, retryPolicy, 'maxRetries', 10); res.endBlock(''); break; case 'UntilElapsed': res.startBlock(''); - $generatorXml.constructorArg(res, 0, retryPolicy, 'maxElapsedTimeMs', 60000); - $generatorXml.constructorArg(res, 1, retryPolicy, 'sleepMsBetweenRetries', 1000); + $generatorSpring.constructorArg(res, 0, retryPolicy, 'maxElapsedTimeMs', 60000); + $generatorSpring.constructorArg(res, 1, retryPolicy, 'sleepMsBetweenRetries', 1000); res.endBlock(''); break; case 'NTimes': res.startBlock(''); - $generatorXml.constructorArg(res, 0, retryPolicy, 'n', 10); - $generatorXml.constructorArg(res, 1, retryPolicy, 'sleepMsBetweenRetries', 1000); + $generatorSpring.constructorArg(res, 0, retryPolicy, 'n', 10); + $generatorSpring.constructorArg(res, 1, retryPolicy, 'sleepMsBetweenRetries', 1000); res.endBlock(''); break; case 'OneTime': res.startBlock(''); - $generatorXml.constructorArg(res, 0, retryPolicy, 'sleepMsBetweenRetry', 1000); + $generatorSpring.constructorArg(res, 0, retryPolicy, 'sleepMsBetweenRetry', 1000); res.endBlock(''); break; case 'Forever': res.startBlock(''); - $generatorXml.constructorArg(res, 0, retryPolicy, 'retryIntervalMs', 1000); + $generatorSpring.constructorArg(res, 0, retryPolicy, 'retryIntervalMs', 1000); res.endBlock(''); break; @@ -463,9 +463,9 @@ $generatorXml.clusterGeneral = function(cluster, res) { res.endBlock(''); } - $generatorXml.property(res, d.ZooKeeper, 'basePath', null, '/services'); - $generatorXml.property(res, d.ZooKeeper, 'serviceName', null, 'ignite'); - $generatorXml.property(res, d.ZooKeeper, 'allowDuplicateRegistrations', null, false); + $generatorSpring.property(res, d.ZooKeeper, 'basePath', null, '/services'); + $generatorSpring.property(res, d.ZooKeeper, 'serviceName', null, 'ignite'); + $generatorSpring.property(res, d.ZooKeeper, 'allowDuplicateRegistrations', null, false); } res.endBlock(''); @@ -478,7 +478,7 @@ $generatorXml.clusterGeneral = function(cluster, res) { res.endBlock(''); - $generatorXml.clusterDiscovery(d, res); + $generatorSpring.clusterDiscovery(d, res); res.endBlock(''); res.endBlock(''); @@ -490,7 +490,7 @@ $generatorXml.clusterGeneral = function(cluster, res) { }; // Generate atomics group. -$generatorXml.clusterAtomics = function(atomics, res) { +$generatorSpring.clusterAtomics = function(atomics, res) { if (!res) res = $generatorCommon.builder(); @@ -506,12 +506,12 @@ $generatorXml.clusterAtomics = function(atomics, res) { let hasData = cacheMode !== 'PARTITIONED'; - $generatorXml.property(res, atomics, 'cacheMode', null, 'PARTITIONED'); + $generatorSpring.property(res, atomics, 'cacheMode', null, 'PARTITIONED'); - hasData = $generatorXml.property(res, atomics, 'atomicSequenceReserveSize', null, 1000) || hasData; + hasData = $generatorSpring.property(res, atomics, 'atomicSequenceReserveSize', null, 1000) || hasData; if (cacheMode === 'PARTITIONED') - hasData = $generatorXml.property(res, atomics, 'backups', null, 0) || hasData; + hasData = $generatorSpring.property(res, atomics, 'backups', null, 0) || hasData; res.endBlock(''); res.endBlock(''); @@ -526,7 +526,7 @@ $generatorXml.clusterAtomics = function(atomics, res) { }; // Generate binary group. -$generatorXml.clusterBinary = function(binary, res) { +$generatorSpring.clusterBinary = function(binary, res) { if (!res) res = $generatorCommon.builder(); @@ -534,9 +534,9 @@ $generatorXml.clusterBinary = function(binary, res) { res.startBlock(''); res.startBlock(''); - $generatorXml.simpleBeanProperty(res, binary, 'idMapper'); - $generatorXml.simpleBeanProperty(res, binary, 'nameMapper'); - $generatorXml.simpleBeanProperty(res, binary, 'serializer'); + $generatorSpring.simpleBeanProperty(res, binary, 'idMapper'); + $generatorSpring.simpleBeanProperty(res, binary, 'nameMapper'); + $generatorSpring.simpleBeanProperty(res, binary, 'serializer'); if ($generatorCommon.isDefinedAndNotEmpty(binary.typeConfigurations)) { res.startBlock(''); @@ -545,11 +545,11 @@ $generatorXml.clusterBinary = function(binary, res) { _.forEach(binary.typeConfigurations, function(type) { res.startBlock(''); - $generatorXml.property(res, type, 'typeName'); - $generatorXml.simpleBeanProperty(res, type, 'idMapper'); - $generatorXml.simpleBeanProperty(res, type, 'nameMapper'); - $generatorXml.simpleBeanProperty(res, type, 'serializer'); - $generatorXml.property(res, type, 'enum', null, false); + $generatorSpring.property(res, type, 'typeName'); + $generatorSpring.simpleBeanProperty(res, type, 'idMapper'); + $generatorSpring.simpleBeanProperty(res, type, 'nameMapper'); + $generatorSpring.simpleBeanProperty(res, type, 'serializer'); + $generatorSpring.property(res, type, 'enum', null, false); res.endBlock(''); }); @@ -558,7 +558,7 @@ $generatorXml.clusterBinary = function(binary, res) { res.endBlock(''); } - $generatorXml.property(res, binary, 'compactFooter', null, true); + $generatorSpring.property(res, binary, 'compactFooter', null, true); res.endBlock(''); res.endBlock(''); @@ -570,7 +570,7 @@ $generatorXml.clusterBinary = function(binary, res) { }; // Generate cache key configurations. -$generatorXml.clusterCacheKeyConfiguration = function(keyCfgs, res) { +$generatorSpring.clusterCacheKeyConfiguration = function(keyCfgs, res) { if (!res) res = $generatorCommon.builder(); @@ -585,8 +585,8 @@ $generatorXml.clusterCacheKeyConfiguration = function(keyCfgs, res) { _.forEach(keyCfgs, (cfg) => { res.startBlock(''); - $generatorXml.constructorArg(res, -1, cfg, 'typeName'); - $generatorXml.constructorArg(res, -1, cfg, 'affinityKeyFieldName'); + $generatorSpring.constructorArg(res, -1, cfg, 'typeName'); + $generatorSpring.constructorArg(res, -1, cfg, 'affinityKeyFieldName'); res.endBlock(''); }); @@ -598,7 +598,7 @@ $generatorXml.clusterCacheKeyConfiguration = function(keyCfgs, res) { }; // Generate collision group. -$generatorXml.clusterCollision = function(collision, res) { +$generatorSpring.clusterCollision = function(collision, res) { if (!res) res = $generatorCommon.builder(); @@ -611,11 +611,11 @@ $generatorXml.clusterCollision = function(collision, res) { switch (collision.kind) { case 'JobStealing': res.startBlock(''); - $generatorXml.property(res, spi, 'activeJobsThreshold', null, 95); - $generatorXml.property(res, spi, 'waitJobsThreshold', null, 0); - $generatorXml.property(res, spi, 'messageExpireTime', null, 1000); - $generatorXml.property(res, spi, 'maximumStealingAttempts', null, 5); - $generatorXml.property(res, spi, 'stealingEnabled', null, true); + $generatorSpring.property(res, spi, 'activeJobsThreshold', null, 95); + $generatorSpring.property(res, spi, 'waitJobsThreshold', null, 0); + $generatorSpring.property(res, spi, 'messageExpireTime', null, 1000); + $generatorSpring.property(res, spi, 'maximumStealingAttempts', null, 5); + $generatorSpring.property(res, spi, 'stealingEnabled', null, true); if ($generatorCommon.isDefinedAndNotEmpty(spi.externalCollisionListener)) { res.needEmptyLine = true; @@ -632,7 +632,7 @@ $generatorXml.clusterCollision = function(collision, res) { res.startBlock(''); _.forEach(spi.stealingAttributes, function(attr) { - $generatorXml.element(res, 'entry', 'key', attr.name, 'value', attr.value); + $generatorSpring.element(res, 'entry', 'key', attr.name, 'value', attr.value); }); res.endBlock(''); @@ -645,21 +645,21 @@ $generatorXml.clusterCollision = function(collision, res) { case 'FifoQueue': res.startBlock(''); - $generatorXml.property(res, spi, 'parallelJobsNumber'); - $generatorXml.property(res, spi, 'waitingJobsNumber'); + $generatorSpring.property(res, spi, 'parallelJobsNumber'); + $generatorSpring.property(res, spi, 'waitingJobsNumber'); res.endBlock(''); break; case 'PriorityQueue': res.startBlock(''); - $generatorXml.property(res, spi, 'parallelJobsNumber'); - $generatorXml.property(res, spi, 'waitingJobsNumber'); - $generatorXml.property(res, spi, 'priorityAttributeKey', null, 'grid.task.priority'); - $generatorXml.property(res, spi, 'jobPriorityAttributeKey', null, 'grid.job.priority'); - $generatorXml.property(res, spi, 'defaultPriority', null, 0); - $generatorXml.property(res, spi, 'starvationIncrement', null, 1); - $generatorXml.property(res, spi, 'starvationPreventionEnabled', null, true); + $generatorSpring.property(res, spi, 'parallelJobsNumber'); + $generatorSpring.property(res, spi, 'waitingJobsNumber'); + $generatorSpring.property(res, spi, 'priorityAttributeKey', null, 'grid.task.priority'); + $generatorSpring.property(res, spi, 'jobPriorityAttributeKey', null, 'grid.job.priority'); + $generatorSpring.property(res, spi, 'defaultPriority', null, 0); + $generatorSpring.property(res, spi, 'starvationIncrement', null, 1); + $generatorSpring.property(res, spi, 'starvationPreventionEnabled', null, true); res.endBlock(''); break; @@ -680,18 +680,18 @@ $generatorXml.clusterCollision = function(collision, res) { }; // Generate communication group. -$generatorXml.clusterCommunication = function(cluster, res) { +$generatorSpring.clusterCommunication = function(cluster, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.beanProperty(res, cluster.communication, 'communicationSpi', $generatorCommon.COMMUNICATION_CONFIGURATION); + $generatorSpring.beanProperty(res, cluster.communication, 'communicationSpi', $generatorCommon.COMMUNICATION_CONFIGURATION); - $generatorXml.property(res, cluster, 'networkTimeout', null, 5000); - $generatorXml.property(res, cluster, 'networkSendRetryDelay', null, 1000); - $generatorXml.property(res, cluster, 'networkSendRetryCount', null, 3); - $generatorXml.property(res, cluster, 'segmentCheckFrequency'); - $generatorXml.property(res, cluster, 'waitForSegmentOnStart', null, false); - $generatorXml.property(res, cluster, 'discoveryStartupDelay', null, 60000); + $generatorSpring.property(res, cluster, 'networkTimeout', null, 5000); + $generatorSpring.property(res, cluster, 'networkSendRetryDelay', null, 1000); + $generatorSpring.property(res, cluster, 'networkSendRetryCount', null, 3); + $generatorSpring.property(res, cluster, 'segmentCheckFrequency'); + $generatorSpring.property(res, cluster, 'waitForSegmentOnStart', null, false); + $generatorSpring.property(res, cluster, 'discoveryStartupDelay', null, 60000); res.needEmptyLine = true; @@ -705,7 +705,7 @@ $generatorXml.clusterCommunication = function(cluster, res) { * @param res Optional configuration presentation builder object. * @returns Configuration presentation builder object */ -$generatorXml.clusterConnector = function(connector, res) { +$generatorSpring.clusterConnector = function(connector, res) { if (!res) res = $generatorCommon.builder(); @@ -717,7 +717,7 @@ $generatorXml.clusterConnector = function(connector, res) { cfg.fields.sslFactory = {type: 'bean'}; } - $generatorXml.beanProperty(res, connector, 'connectorConfiguration', cfg, true); + $generatorSpring.beanProperty(res, connector, 'connectorConfiguration', cfg, true); res.needEmptyLine = true; } @@ -726,22 +726,22 @@ $generatorXml.clusterConnector = function(connector, res) { }; // Generate deployment group. -$generatorXml.clusterDeployment = function(cluster, res) { +$generatorSpring.clusterDeployment = function(cluster, res) { if (!res) res = $generatorCommon.builder(); - if ($generatorXml.property(res, cluster, 'deploymentMode', null, 'SHARED')) + if ($generatorSpring.property(res, cluster, 'deploymentMode', null, 'SHARED')) res.needEmptyLine = true; const p2pEnabled = cluster.peerClassLoadingEnabled; if (!_.isNil(p2pEnabled)) { - $generatorXml.property(res, cluster, 'peerClassLoadingEnabled', null, false); + $generatorSpring.property(res, cluster, 'peerClassLoadingEnabled', null, false); if (p2pEnabled) { - $generatorXml.property(res, cluster, 'peerClassLoadingMissedResourcesCacheSize', null, 100); - $generatorXml.property(res, cluster, 'peerClassLoadingThreadPoolSize', null, 2); - $generatorXml.listProperty(res, cluster, 'peerClassLoadingLocalClassPathExclude'); + $generatorSpring.property(res, cluster, 'peerClassLoadingMissedResourcesCacheSize', null, 100); + $generatorSpring.property(res, cluster, 'peerClassLoadingThreadPoolSize', null, 2); + $generatorSpring.listProperty(res, cluster, 'peerClassLoadingLocalClassPathExclude'); } res.needEmptyLine = true; @@ -751,39 +751,39 @@ $generatorXml.clusterDeployment = function(cluster, res) { }; // Generate discovery group. -$generatorXml.clusterDiscovery = function(disco, res) { +$generatorSpring.clusterDiscovery = function(disco, res) { if (!res) res = $generatorCommon.builder(); if (disco) { - $generatorXml.property(res, disco, 'localAddress'); - $generatorXml.property(res, disco, 'localPort', null, 47500); - $generatorXml.property(res, disco, 'localPortRange', null, 100); + $generatorSpring.property(res, disco, 'localAddress'); + $generatorSpring.property(res, disco, 'localPort', null, 47500); + $generatorSpring.property(res, disco, 'localPortRange', null, 100); if ($generatorCommon.isDefinedAndNotEmpty(disco.addressResolver)) - $generatorXml.beanProperty(res, disco, 'addressResolver', {className: disco.addressResolver}, true); - $generatorXml.property(res, disco, 'socketTimeout', null, 5000); - $generatorXml.property(res, disco, 'ackTimeout', null, 5000); - $generatorXml.property(res, disco, 'maxAckTimeout', null, 600000); - $generatorXml.property(res, disco, 'networkTimeout', null, 5000); - $generatorXml.property(res, disco, 'joinTimeout', null, 0); - $generatorXml.property(res, disco, 'threadPriority', null, 10); - $generatorXml.property(res, disco, 'heartbeatFrequency', null, 2000); - $generatorXml.property(res, disco, 'maxMissedHeartbeats', null, 1); - $generatorXml.property(res, disco, 'maxMissedClientHeartbeats', null, 5); - $generatorXml.property(res, disco, 'topHistorySize', null, 1000); + $generatorSpring.beanProperty(res, disco, 'addressResolver', {className: disco.addressResolver}, true); + $generatorSpring.property(res, disco, 'socketTimeout', null, 5000); + $generatorSpring.property(res, disco, 'ackTimeout', null, 5000); + $generatorSpring.property(res, disco, 'maxAckTimeout', null, 600000); + $generatorSpring.property(res, disco, 'networkTimeout', null, 5000); + $generatorSpring.property(res, disco, 'joinTimeout', null, 0); + $generatorSpring.property(res, disco, 'threadPriority', null, 10); + $generatorSpring.property(res, disco, 'heartbeatFrequency', null, 2000); + $generatorSpring.property(res, disco, 'maxMissedHeartbeats', null, 1); + $generatorSpring.property(res, disco, 'maxMissedClientHeartbeats', null, 5); + $generatorSpring.property(res, disco, 'topHistorySize', null, 1000); if ($generatorCommon.isDefinedAndNotEmpty(disco.listener)) - $generatorXml.beanProperty(res, disco, 'listener', {className: disco.listener}, true); + $generatorSpring.beanProperty(res, disco, 'listener', {className: disco.listener}, true); if ($generatorCommon.isDefinedAndNotEmpty(disco.dataExchange)) - $generatorXml.beanProperty(res, disco, 'dataExchange', {className: disco.dataExchange}, true); + $generatorSpring.beanProperty(res, disco, 'dataExchange', {className: disco.dataExchange}, true); if ($generatorCommon.isDefinedAndNotEmpty(disco.metricsProvider)) - $generatorXml.beanProperty(res, disco, 'metricsProvider', {className: disco.metricsProvider}, true); - $generatorXml.property(res, disco, 'reconnectCount', null, 10); - $generatorXml.property(res, disco, 'statisticsPrintFrequency', null, 0); - $generatorXml.property(res, disco, 'ipFinderCleanFrequency', null, 60000); + $generatorSpring.beanProperty(res, disco, 'metricsProvider', {className: disco.metricsProvider}, true); + $generatorSpring.property(res, disco, 'reconnectCount', null, 10); + $generatorSpring.property(res, disco, 'statisticsPrintFrequency', null, 0); + $generatorSpring.property(res, disco, 'ipFinderCleanFrequency', null, 60000); if ($generatorCommon.isDefinedAndNotEmpty(disco.authenticator)) - $generatorXml.beanProperty(res, disco, 'authenticator', {className: disco.authenticator}, true); - $generatorXml.property(res, disco, 'forceServerMode', null, false); - $generatorXml.property(res, disco, 'clientReconnectDisabled', null, false); + $generatorSpring.beanProperty(res, disco, 'authenticator', {className: disco.authenticator}, true); + $generatorSpring.property(res, disco, 'forceServerMode', null, false); + $generatorSpring.property(res, disco, 'clientReconnectDisabled', null, false); res.needEmptyLine = true; } @@ -792,7 +792,7 @@ $generatorXml.clusterDiscovery = function(disco, res) { }; // Generate events group. -$generatorXml.clusterEvents = function(cluster, res) { +$generatorSpring.clusterEvents = function(cluster, res) { if (!res) res = $generatorCommon.builder(); @@ -836,7 +836,7 @@ $generatorXml.clusterEvents = function(cluster, res) { }; // Generate failover group. -$generatorXml.clusterFailover = function(cluster, res) { +$generatorSpring.clusterFailover = function(cluster, res) { if (!res) res = $generatorCommon.builder(); @@ -853,7 +853,7 @@ $generatorXml.clusterFailover = function(cluster, res) { if ((spi.kind === 'JobStealing' || spi.kind === 'Always') && $generatorCommon.isDefinedAndNotEmpty(maxAttempts) && maxAttempts !== 5) { res.startBlock(''); - $generatorXml.property(res, spi[spi.kind], 'maximumFailoverAttempts', null, 5); + $generatorSpring.property(res, spi[spi.kind], 'maximumFailoverAttempts', null, 5); res.endBlock(''); } @@ -874,7 +874,7 @@ $generatorXml.clusterFailover = function(cluster, res) { }; // Generate marshaller group. -$generatorXml.clusterLogger = function(logger, res) { +$generatorSpring.clusterLogger = function(logger, res) { if (!res) res = $generatorCommon.builder(); @@ -886,8 +886,8 @@ $generatorXml.clusterLogger = function(logger, res) { switch (logger.kind) { case 'Log4j2': res.startBlock(''); - res.line(''); - $generatorXml.property(res, log, 'level'); + res.line(''); + $generatorSpring.property(res, log, 'level'); res.endBlock(''); break; @@ -919,9 +919,9 @@ $generatorXml.clusterLogger = function(logger, res) { res.startBlock(''); if (log.mode === 'Path') - res.line(''); + res.line(''); - $generatorXml.property(res, log, 'level'); + $generatorSpring.property(res, log, 'level'); res.endBlock(''); } @@ -944,20 +944,20 @@ $generatorXml.clusterLogger = function(logger, res) { }; // Generate marshaller group. -$generatorXml.clusterMarshaller = function(cluster, res) { +$generatorSpring.clusterMarshaller = function(cluster, res) { if (!res) res = $generatorCommon.builder(); const marshaller = cluster.marshaller; if (marshaller && marshaller.kind) - $generatorXml.beanProperty(res, marshaller[marshaller.kind], 'marshaller', $generatorCommon.MARSHALLERS[marshaller.kind], true); + $generatorSpring.beanProperty(res, marshaller[marshaller.kind], 'marshaller', $generatorCommon.MARSHALLERS[marshaller.kind], true); res.softEmptyLine(); - $generatorXml.property(res, cluster, 'marshalLocalJobs', null, false); - $generatorXml.property(res, cluster, 'marshallerCacheKeepAliveTime', null, 10000); - $generatorXml.property(res, cluster, 'marshallerCacheThreadPoolSize', 'marshallerCachePoolSize'); + $generatorSpring.property(res, cluster, 'marshalLocalJobs', null, false); + $generatorSpring.property(res, cluster, 'marshallerCacheKeepAliveTime', null, 10000); + $generatorSpring.property(res, cluster, 'marshallerCacheThreadPoolSize', 'marshallerCachePoolSize'); res.needEmptyLine = true; @@ -965,14 +965,14 @@ $generatorXml.clusterMarshaller = function(cluster, res) { }; // Generate metrics group. -$generatorXml.clusterMetrics = function(cluster, res) { +$generatorSpring.clusterMetrics = function(cluster, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.property(res, cluster, 'metricsExpireTime'); - $generatorXml.property(res, cluster, 'metricsHistorySize', null, 10000); - $generatorXml.property(res, cluster, 'metricsLogFrequency', null, 60000); - $generatorXml.property(res, cluster, 'metricsUpdateFrequency', null, 2000); + $generatorSpring.property(res, cluster, 'metricsExpireTime'); + $generatorSpring.property(res, cluster, 'metricsHistorySize', null, 10000); + $generatorSpring.property(res, cluster, 'metricsLogFrequency', null, 60000); + $generatorSpring.property(res, cluster, 'metricsUpdateFrequency', null, 2000); res.needEmptyLine = true; @@ -980,12 +980,12 @@ $generatorXml.clusterMetrics = function(cluster, res) { }; // Generate swap group. -$generatorXml.clusterSwap = function(cluster, res) { +$generatorSpring.clusterSwap = function(cluster, res) { if (!res) res = $generatorCommon.builder(); if (cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind === 'FileSwapSpaceSpi') { - $generatorXml.beanProperty(res, cluster.swapSpaceSpi.FileSwapSpaceSpi, 'swapSpaceSpi', + $generatorSpring.beanProperty(res, cluster.swapSpaceSpi.FileSwapSpaceSpi, 'swapSpaceSpi', $generatorCommon.SWAP_SPACE_SPI, true); res.needEmptyLine = true; @@ -995,14 +995,14 @@ $generatorXml.clusterSwap = function(cluster, res) { }; // Generate time group. -$generatorXml.clusterTime = function(cluster, res) { +$generatorSpring.clusterTime = function(cluster, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.property(res, cluster, 'clockSyncSamples', null, 8); - $generatorXml.property(res, cluster, 'clockSyncFrequency', null, 120000); - $generatorXml.property(res, cluster, 'timeServerPortBase', null, 31100); - $generatorXml.property(res, cluster, 'timeServerPortRange', null, 100); + $generatorSpring.property(res, cluster, 'clockSyncSamples', null, 8); + $generatorSpring.property(res, cluster, 'clockSyncFrequency', null, 120000); + $generatorSpring.property(res, cluster, 'timeServerPortBase', null, 31100); + $generatorSpring.property(res, cluster, 'timeServerPortRange', null, 100); res.needEmptyLine = true; @@ -1010,12 +1010,12 @@ $generatorXml.clusterTime = function(cluster, res) { }; // Generate OBC configuration group. -$generatorXml.clusterODBC = function(odbc, res) { +$generatorSpring.clusterODBC = function(odbc, res) { if (!res) res = $generatorCommon.builder(); if (odbc && odbc.odbcEnabled) - $generatorXml.beanProperty(res, odbc, 'odbcConfiguration', $generatorCommon.ODBC_CONFIGURATION, true); + $generatorSpring.beanProperty(res, odbc, 'odbcConfiguration', $generatorCommon.ODBC_CONFIGURATION, true); res.needEmptyLine = true; @@ -1023,15 +1023,15 @@ $generatorXml.clusterODBC = function(odbc, res) { }; // Generate thread pools group. -$generatorXml.clusterPools = function(cluster, res) { +$generatorSpring.clusterPools = function(cluster, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.property(res, cluster, 'publicThreadPoolSize'); - $generatorXml.property(res, cluster, 'systemThreadPoolSize'); - $generatorXml.property(res, cluster, 'managementThreadPoolSize'); - $generatorXml.property(res, cluster, 'igfsThreadPoolSize'); - $generatorXml.property(res, cluster, 'rebalanceThreadPoolSize'); + $generatorSpring.property(res, cluster, 'publicThreadPoolSize'); + $generatorSpring.property(res, cluster, 'systemThreadPoolSize'); + $generatorSpring.property(res, cluster, 'managementThreadPoolSize'); + $generatorSpring.property(res, cluster, 'igfsThreadPoolSize'); + $generatorSpring.property(res, cluster, 'rebalanceThreadPoolSize'); res.needEmptyLine = true; @@ -1039,11 +1039,11 @@ $generatorXml.clusterPools = function(cluster, res) { }; // Generate transactions group. -$generatorXml.clusterTransactions = function(transactionConfiguration, res) { +$generatorSpring.clusterTransactions = function(transactionConfiguration, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.beanProperty(res, transactionConfiguration, 'transactionConfiguration', $generatorCommon.TRANSACTION_CONFIGURATION, false); + $generatorSpring.beanProperty(res, transactionConfiguration, 'transactionConfiguration', $generatorCommon.TRANSACTION_CONFIGURATION, false); res.needEmptyLine = true; @@ -1051,7 +1051,7 @@ $generatorXml.clusterTransactions = function(transactionConfiguration, res) { }; // Generate user attributes group. -$generatorXml.clusterUserAttributes = function(cluster, res) { +$generatorSpring.clusterUserAttributes = function(cluster, res) { if (!res) res = $generatorCommon.builder(); @@ -1060,7 +1060,7 @@ $generatorXml.clusterUserAttributes = function(cluster, res) { res.startBlock(''); _.forEach(cluster.attributes, function(attr) { - $generatorXml.element(res, 'entry', 'key', attr.name, 'value', attr.value); + $generatorSpring.element(res, 'entry', 'key', attr.name, 'value', attr.value); }); res.endBlock(''); @@ -1079,7 +1079,7 @@ $generatorXml.clusterUserAttributes = function(cluster, res) { * @param res Optional configuration presentation builder object. * @returns Configuration presentation builder object */ -$generatorXml.clusterSsl = function(cluster, res) { +$generatorSpring.clusterSsl = function(cluster, res) { if (!res) res = $generatorCommon.builder(); @@ -1099,7 +1099,7 @@ $generatorXml.clusterSsl = function(cluster, res) { $generatorCommon.SSL_CONFIGURATION_TRUST_MANAGER_FACTORY : $generatorCommon.SSL_CONFIGURATION_TRUST_FILE_FACTORY; - $generatorXml.beanProperty(res, sslFactory, 'sslContextFactory', propsDesc, true); + $generatorSpring.beanProperty(res, sslFactory, 'sslContextFactory', propsDesc, true); res.needEmptyLine = true; } @@ -1108,22 +1108,22 @@ $generatorXml.clusterSsl = function(cluster, res) { }; // Generate cache general group. -$generatorXml.cacheGeneral = function(cache, res) { +$generatorSpring.cacheGeneral = function(cache, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.property(res, cache, 'name'); + $generatorSpring.property(res, cache, 'name'); - $generatorXml.property(res, cache, 'cacheMode'); - $generatorXml.property(res, cache, 'atomicityMode'); + $generatorSpring.property(res, cache, 'cacheMode'); + $generatorSpring.property(res, cache, 'atomicityMode'); - if (cache.cacheMode === 'PARTITIONED' && $generatorXml.property(res, cache, 'backups')) - $generatorXml.property(res, cache, 'readFromBackup'); + if (cache.cacheMode === 'PARTITIONED' && $generatorSpring.property(res, cache, 'backups')) + $generatorSpring.property(res, cache, 'readFromBackup'); - $generatorXml.property(res, cache, 'copyOnRead'); + $generatorSpring.property(res, cache, 'copyOnRead'); if (cache.cacheMode === 'PARTITIONED' && cache.atomicityMode === 'TRANSACTIONAL') - $generatorXml.property(res, cache, 'invalidate'); + $generatorSpring.property(res, cache, 'invalidate'); res.needEmptyLine = true; @@ -1131,23 +1131,23 @@ $generatorXml.cacheGeneral = function(cache, res) { }; // Generate cache memory group. -$generatorXml.cacheMemory = function(cache, res) { +$generatorSpring.cacheMemory = function(cache, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.property(res, cache, 'memoryMode', null, 'ONHEAP_TIERED'); + $generatorSpring.property(res, cache, 'memoryMode', null, 'ONHEAP_TIERED'); if (cache.memoryMode !== 'OFFHEAP_VALUES') - $generatorXml.property(res, cache, 'offHeapMaxMemory', null, -1); + $generatorSpring.property(res, cache, 'offHeapMaxMemory', null, -1); res.softEmptyLine(); - $generatorXml.evictionPolicy(res, cache.evictionPolicy, 'evictionPolicy'); + $generatorSpring.evictionPolicy(res, cache.evictionPolicy, 'evictionPolicy'); res.softEmptyLine(); - $generatorXml.property(res, cache, 'startSize', null, 1500000); - $generatorXml.property(res, cache, 'swapEnabled', null, false); + $generatorSpring.property(res, cache, 'startSize', null, 1500000); + $generatorSpring.property(res, cache, 'swapEnabled', null, false); res.needEmptyLine = true; @@ -1155,13 +1155,13 @@ $generatorXml.cacheMemory = function(cache, res) { }; // Generate cache query & indexing group. -$generatorXml.cacheQuery = function(cache, domains, res) { +$generatorSpring.cacheQuery = function(cache, domains, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.property(res, cache, 'sqlSchema'); - $generatorXml.property(res, cache, 'sqlOnheapRowCacheSize', null, 10240); - $generatorXml.property(res, cache, 'longQueryWarningTimeout', null, 3000); + $generatorSpring.property(res, cache, 'sqlSchema'); + $generatorSpring.property(res, cache, 'sqlOnheapRowCacheSize', null, 10240); + $generatorSpring.property(res, cache, 'longQueryWarningTimeout', null, 3000); const indexedTypes = _.filter(domains, (domain) => domain.queryMetadata === 'Annotations'); @@ -1182,12 +1182,12 @@ $generatorXml.cacheQuery = function(cache, domains, res) { res.softEmptyLine(); - $generatorXml.listProperty(res, cache, 'sqlFunctionClasses'); + $generatorSpring.listProperty(res, cache, 'sqlFunctionClasses'); res.softEmptyLine(); - $generatorXml.property(res, cache, 'snapshotableIndex', null, false); - $generatorXml.property(res, cache, 'sqlEscapeAll', null, false); + $generatorSpring.property(res, cache, 'snapshotableIndex', null, false); + $generatorSpring.property(res, cache, 'sqlEscapeAll', null, false); res.needEmptyLine = true; @@ -1195,7 +1195,7 @@ $generatorXml.cacheQuery = function(cache, domains, res) { }; // Generate cache store group. -$generatorXml.cacheStore = function(cache, domains, res) { +$generatorSpring.cacheStore = function(cache, domains, res) { if (!res) res = $generatorCommon.builder(); @@ -1209,12 +1209,15 @@ $generatorXml.cacheStore = function(cache, domains, res) { res.startBlock(''); res.startBlock(''); - $generatorXml.property(res, storeFactory, 'dataSourceBean'); + $generatorSpring.property(res, storeFactory, 'dataSourceBean'); res.startBlock(''); res.line(''); res.endBlock(''); + if (storeFactory.sqlEscapeAll) + $generatorSpring.property(res, storeFactory, 'sqlEscapeAll'); + const domainConfigs = _.filter(domains, function(domain) { return $generatorCommon.isDefinedAndNotEmpty(domain.databaseTable); }); @@ -1226,12 +1229,12 @@ $generatorXml.cacheStore = function(cache, domains, res) { _.forEach(domainConfigs, function(domain) { res.startBlock(''); - $generatorXml.property(res, cache, 'name', 'cacheName'); + $generatorSpring.property(res, cache, 'name', 'cacheName'); - $generatorXml.classNameProperty(res, domain, 'keyType'); - $generatorXml.property(res, domain, 'valueType'); + $generatorSpring.classNameProperty(res, domain, 'keyType'); + $generatorSpring.property(res, domain, 'valueType'); - $generatorXml.domainStore(domain, res); + $generatorSpring.domainStore(domain, res); res.endBlock(''); }); @@ -1248,28 +1251,28 @@ $generatorXml.cacheStore = function(cache, domains, res) { res.startBlock(''); if (storeFactory.connectVia === 'DataSource') - $generatorXml.property(res, storeFactory, 'dataSourceBean'); + $generatorSpring.property(res, storeFactory, 'dataSourceBean'); else { - $generatorXml.property(res, storeFactory, 'connectionUrl'); - $generatorXml.property(res, storeFactory, 'user'); + $generatorSpring.property(res, storeFactory, 'connectionUrl'); + $generatorSpring.property(res, storeFactory, 'user'); res.line(''); } - $generatorXml.property(res, storeFactory, 'initSchema'); - $generatorXml.property(res, storeFactory, 'createTableQuery'); - $generatorXml.property(res, storeFactory, 'loadQuery'); - $generatorXml.property(res, storeFactory, 'insertQuery'); - $generatorXml.property(res, storeFactory, 'updateQuery'); - $generatorXml.property(res, storeFactory, 'deleteQuery'); + $generatorSpring.property(res, storeFactory, 'initSchema'); + $generatorSpring.property(res, storeFactory, 'createTableQuery'); + $generatorSpring.property(res, storeFactory, 'loadQuery'); + $generatorSpring.property(res, storeFactory, 'insertQuery'); + $generatorSpring.property(res, storeFactory, 'updateQuery'); + $generatorSpring.property(res, storeFactory, 'deleteQuery'); res.endBlock(''); res.endBlock(''); } else - $generatorXml.beanProperty(res, storeFactory, 'cacheStoreFactory', $generatorCommon.STORE_FACTORIES[factoryKind], true); + $generatorSpring.beanProperty(res, storeFactory, 'cacheStoreFactory', $generatorCommon.STORE_FACTORIES[factoryKind], true); if (storeFactory.dataSourceBean && (storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : null) : storeFactory.dialect)) { - if (_.findIndex(res.datasources, (ds) => ds.dataSourceBean === storeFactory.dataSourceBean) < 0) { + if (!_.find(res.datasources, { dataSourceBean: storeFactory.dataSourceBean})) { res.datasources.push({ dataSourceBean: storeFactory.dataSourceBean, dialect: storeFactory.dialect @@ -1281,19 +1284,19 @@ $generatorXml.cacheStore = function(cache, domains, res) { res.softEmptyLine(); - $generatorXml.property(res, cache, 'storeKeepBinary', null, false); - $generatorXml.property(res, cache, 'loadPreviousValue', null, false); - $generatorXml.property(res, cache, 'readThrough', null, false); - $generatorXml.property(res, cache, 'writeThrough', null, false); + $generatorSpring.property(res, cache, 'storeKeepBinary', null, false); + $generatorSpring.property(res, cache, 'loadPreviousValue', null, false); + $generatorSpring.property(res, cache, 'readThrough', null, false); + $generatorSpring.property(res, cache, 'writeThrough', null, false); res.softEmptyLine(); if (cache.writeBehindEnabled) { - $generatorXml.property(res, cache, 'writeBehindEnabled', null, false); - $generatorXml.property(res, cache, 'writeBehindBatchSize', null, 512); - $generatorXml.property(res, cache, 'writeBehindFlushSize', null, 10240); - $generatorXml.property(res, cache, 'writeBehindFlushFrequency', null, 5000); - $generatorXml.property(res, cache, 'writeBehindFlushThreadCount', null, 1); + $generatorSpring.property(res, cache, 'writeBehindEnabled', null, false); + $generatorSpring.property(res, cache, 'writeBehindBatchSize', null, 512); + $generatorSpring.property(res, cache, 'writeBehindFlushSize', null, 10240); + $generatorSpring.property(res, cache, 'writeBehindFlushFrequency', null, 5000); + $generatorSpring.property(res, cache, 'writeBehindFlushThreadCount', null, 1); } res.needEmptyLine = true; @@ -1302,7 +1305,7 @@ $generatorXml.cacheStore = function(cache, domains, res) { }; // Generate cache node filter group. -$generatorXml.cacheNodeFilter = function(cache, igfss, res) { +$generatorSpring.cacheNodeFilter = function(cache, igfss, res) { if (!res) res = $generatorCommon.builder(); @@ -1364,14 +1367,14 @@ $generatorXml.cacheNodeFilter = function(cache, igfss, res) { }; // Generate cache concurrency group. -$generatorXml.cacheConcurrency = function(cache, res) { +$generatorSpring.cacheConcurrency = function(cache, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.property(res, cache, 'maxConcurrentAsyncOperations', null, 500); - $generatorXml.property(res, cache, 'defaultLockTimeout', null, 0); - $generatorXml.property(res, cache, 'atomicWriteOrderMode'); - $generatorXml.property(res, cache, 'writeSynchronizationMode', null, 'PRIMARY_SYNC'); + $generatorSpring.property(res, cache, 'maxConcurrentAsyncOperations', null, 500); + $generatorSpring.property(res, cache, 'defaultLockTimeout', null, 0); + $generatorSpring.property(res, cache, 'atomicWriteOrderMode'); + $generatorSpring.property(res, cache, 'writeSynchronizationMode', null, 'PRIMARY_SYNC'); res.needEmptyLine = true; @@ -1379,19 +1382,19 @@ $generatorXml.cacheConcurrency = function(cache, res) { }; // Generate cache rebalance group. -$generatorXml.cacheRebalance = function(cache, res) { +$generatorSpring.cacheRebalance = function(cache, res) { if (!res) res = $generatorCommon.builder(); if (cache.cacheMode !== 'LOCAL') { - $generatorXml.property(res, cache, 'rebalanceMode', null, 'ASYNC'); - $generatorXml.property(res, cache, 'rebalanceThreadPoolSize', null, 1); - $generatorXml.property(res, cache, 'rebalanceBatchSize', null, 524288); - $generatorXml.property(res, cache, 'rebalanceBatchesPrefetchCount', null, 2); - $generatorXml.property(res, cache, 'rebalanceOrder', null, 0); - $generatorXml.property(res, cache, 'rebalanceDelay', null, 0); - $generatorXml.property(res, cache, 'rebalanceTimeout', null, 10000); - $generatorXml.property(res, cache, 'rebalanceThrottle', null, 0); + $generatorSpring.property(res, cache, 'rebalanceMode', null, 'ASYNC'); + $generatorSpring.property(res, cache, 'rebalanceThreadPoolSize', null, 1); + $generatorSpring.property(res, cache, 'rebalanceBatchSize', null, 524288); + $generatorSpring.property(res, cache, 'rebalanceBatchesPrefetchCount', null, 2); + $generatorSpring.property(res, cache, 'rebalanceOrder', null, 0); + $generatorSpring.property(res, cache, 'rebalanceDelay', null, 0); + $generatorSpring.property(res, cache, 'rebalanceTimeout', null, 10000); + $generatorSpring.property(res, cache, 'rebalanceThrottle', null, 0); } res.softEmptyLine(); @@ -1399,7 +1402,7 @@ $generatorXml.cacheRebalance = function(cache, res) { if (cache.igfsAffinnityGroupSize) { res.startBlock(''); res.startBlock(''); - $generatorXml.constructorArg(res, -1, cache, 'igfsAffinnityGroupSize'); + $generatorSpring.constructorArg(res, -1, cache, 'igfsAffinnityGroupSize'); res.endBlock(''); res.endBlock(''); } @@ -1408,7 +1411,7 @@ $generatorXml.cacheRebalance = function(cache, res) { }; // Generate cache server near cache group. -$generatorXml.cacheServerNearCache = function(cache, res) { +$generatorSpring.cacheServerNearCache = function(cache, res) { if (!res) res = $generatorCommon.builder(); @@ -1420,9 +1423,9 @@ $generatorXml.cacheServerNearCache = function(cache, res) { if (cache.nearConfiguration) { if (cache.nearConfiguration.nearStartSize) - $generatorXml.property(res, cache.nearConfiguration, 'nearStartSize', null, 375000); + $generatorSpring.property(res, cache.nearConfiguration, 'nearStartSize', null, 375000); - $generatorXml.evictionPolicy(res, cache.nearConfiguration.nearEvictionPolicy, 'nearEvictionPolicy'); + $generatorSpring.evictionPolicy(res, cache.nearConfiguration.nearEvictionPolicy, 'nearEvictionPolicy'); } res.endBlock(''); @@ -1435,12 +1438,12 @@ $generatorXml.cacheServerNearCache = function(cache, res) { }; // Generate cache statistics group. -$generatorXml.cacheStatistics = function(cache, res) { +$generatorSpring.cacheStatistics = function(cache, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.property(res, cache, 'statisticsEnabled', null, false); - $generatorXml.property(res, cache, 'managementEnabled', null, false); + $generatorSpring.property(res, cache, 'statisticsEnabled', null, false); + $generatorSpring.property(res, cache, 'managementEnabled', null, false); res.needEmptyLine = true; @@ -1448,7 +1451,7 @@ $generatorXml.cacheStatistics = function(cache, res) { }; // Generate domain model query fields. -$generatorXml.domainModelQueryFields = function(res, domain) { +$generatorSpring.domainModelQueryFields = function(res, domain) { const fields = domain.fields; if (fields && fields.length > 0) { @@ -1458,7 +1461,7 @@ $generatorXml.domainModelQueryFields = function(res, domain) { res.startBlock(''); _.forEach(fields, function(field) { - $generatorXml.element(res, 'entry', 'key', field.name, 'value', $generatorCommon.JavaTypes.fullClassName(field.className)); + $generatorSpring.element(res, 'entry', 'key', field.name, 'value', $generatorCommon.JavaTypes.fullClassName(field.className)); }); res.endBlock(''); @@ -1469,7 +1472,7 @@ $generatorXml.domainModelQueryFields = function(res, domain) { }; // Generate domain model query fields. -$generatorXml.domainModelQueryAliases = function(res, domain) { +$generatorSpring.domainModelQueryAliases = function(res, domain) { const aliases = domain.aliases; if (aliases && aliases.length > 0) { @@ -1479,7 +1482,7 @@ $generatorXml.domainModelQueryAliases = function(res, domain) { res.startBlock(''); _.forEach(aliases, function(alias) { - $generatorXml.element(res, 'entry', 'key', alias.field, 'value', alias.alias); + $generatorSpring.element(res, 'entry', 'key', alias.field, 'value', alias.alias); }); res.endBlock(''); @@ -1490,7 +1493,7 @@ $generatorXml.domainModelQueryAliases = function(res, domain) { }; // Generate domain model indexes. -$generatorXml.domainModelQueryIndexes = function(res, domain) { +$generatorSpring.domainModelQueryIndexes = function(res, domain) { const indexes = domain.indexes; if (indexes && indexes.length > 0) { @@ -1502,8 +1505,8 @@ $generatorXml.domainModelQueryIndexes = function(res, domain) { _.forEach(indexes, function(index) { res.startBlock(''); - $generatorXml.property(res, index, 'name'); - $generatorXml.property(res, index, 'indexType'); + $generatorSpring.property(res, index, 'name'); + $generatorSpring.property(res, index, 'indexType'); const fields = index.fields; @@ -1512,7 +1515,7 @@ $generatorXml.domainModelQueryIndexes = function(res, domain) { res.startBlock(''); _.forEach(fields, function(field) { - $generatorXml.element(res, 'entry', 'key', field.name, 'value', field.direction); + $generatorSpring.element(res, 'entry', 'key', field.name, 'value', field.direction); }); res.endBlock(''); @@ -1530,7 +1533,7 @@ $generatorXml.domainModelQueryIndexes = function(res, domain) { }; // Generate domain model db fields. -$generatorXml.domainModelDatabaseFields = function(res, domain, fieldProp) { +$generatorSpring.domainModelDatabaseFields = function(res, domain, fieldProp) { const fields = domain[fieldProp]; if (fields && fields.length > 0) { @@ -1543,15 +1546,15 @@ $generatorXml.domainModelDatabaseFields = function(res, domain, fieldProp) { _.forEach(fields, function(field) { res.startBlock(''); - $generatorXml.property(res, field, 'databaseFieldName'); + $generatorSpring.property(res, field, 'databaseFieldName'); res.startBlock(''); res.line(''); res.endBlock(''); - $generatorXml.property(res, field, 'javaFieldName'); + $generatorSpring.property(res, field, 'javaFieldName'); - $generatorXml.classNameProperty(res, field, 'javaFieldType'); + $generatorSpring.classNameProperty(res, field, 'javaFieldType'); res.endBlock(''); }); @@ -1564,7 +1567,7 @@ $generatorXml.domainModelDatabaseFields = function(res, domain, fieldProp) { }; // Generate domain model general group. -$generatorXml.domainModelGeneral = function(domain, res) { +$generatorSpring.domainModelGeneral = function(domain, res) { if (!res) res = $generatorCommon.builder(); @@ -1591,8 +1594,8 @@ $generatorXml.domainModelGeneral = function(domain, res) { break; case 'Configuration': - $generatorXml.classNameProperty(res, domain, 'keyType'); - $generatorXml.property(res, domain, 'valueType'); + $generatorSpring.classNameProperty(res, domain, 'keyType'); + $generatorSpring.property(res, domain, 'valueType'); break; @@ -1605,14 +1608,14 @@ $generatorXml.domainModelGeneral = function(domain, res) { }; // Generate domain model for query group. -$generatorXml.domainModelQuery = function(domain, res) { +$generatorSpring.domainModelQuery = function(domain, res) { if (!res) res = $generatorCommon.builder(); if ($generatorCommon.domainQueryMetadata(domain) === 'Configuration') { - $generatorXml.domainModelQueryFields(res, domain); - $generatorXml.domainModelQueryAliases(res, domain); - $generatorXml.domainModelQueryIndexes(res, domain); + $generatorSpring.domainModelQueryFields(res, domain); + $generatorSpring.domainModelQueryAliases(res, domain); + $generatorSpring.domainModelQueryIndexes(res, domain); res.needEmptyLine = true; } @@ -1621,33 +1624,33 @@ $generatorXml.domainModelQuery = function(domain, res) { }; // Generate domain model for store group. -$generatorXml.domainStore = function(domain, res) { +$generatorSpring.domainStore = function(domain, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.property(res, domain, 'databaseSchema'); - $generatorXml.property(res, domain, 'databaseTable'); + $generatorSpring.property(res, domain, 'databaseSchema'); + $generatorSpring.property(res, domain, 'databaseTable'); res.softEmptyLine(); - $generatorXml.domainModelDatabaseFields(res, domain, 'keyFields'); - $generatorXml.domainModelDatabaseFields(res, domain, 'valueFields'); + $generatorSpring.domainModelDatabaseFields(res, domain, 'keyFields'); + $generatorSpring.domainModelDatabaseFields(res, domain, 'valueFields'); res.needEmptyLine = true; return res; }; -$generatorXml.cacheQueryMetadata = function(domain, res) { +$generatorSpring.cacheQueryMetadata = function(domain, res) { if (!res) res = $generatorCommon.builder(); res.startBlock(''); - $generatorXml.classNameProperty(res, domain, 'keyType'); - $generatorXml.property(res, domain, 'valueType'); + $generatorSpring.classNameProperty(res, domain, 'keyType'); + $generatorSpring.property(res, domain, 'valueType'); - $generatorXml.domainModelQuery(domain, res); + $generatorSpring.domainModelQuery(domain, res); res.endBlock(''); @@ -1657,7 +1660,7 @@ $generatorXml.cacheQueryMetadata = function(domain, res) { }; // Generate domain models configs. -$generatorXml.cacheDomains = function(domains, res) { +$generatorSpring.cacheDomains = function(domains, res) { if (!res) res = $generatorCommon.builder(); @@ -1673,7 +1676,7 @@ $generatorXml.cacheDomains = function(domains, res) { res.startBlock(''); _.forEach(domainConfigs, function(domain) { - $generatorXml.cacheQueryMetadata(domain, res); + $generatorSpring.cacheQueryMetadata(domain, res); }); res.endBlock(''); @@ -1684,13 +1687,13 @@ $generatorXml.cacheDomains = function(domains, res) { }; // Generate cache configs. -$generatorXml.cache = function(cache, res) { +$generatorSpring.cache = function(cache, res) { if (!res) res = $generatorCommon.builder(); res.startBlock(''); - $generatorXml.cacheConfiguration(cache, res); + $generatorSpring.cacheConfiguration(cache, res); res.endBlock(''); @@ -1698,29 +1701,29 @@ $generatorXml.cache = function(cache, res) { }; // Generate cache configs. -$generatorXml.cacheConfiguration = function(cache, res) { +$generatorSpring.cacheConfiguration = function(cache, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.cacheGeneral(cache, res); - $generatorXml.cacheMemory(cache, res); - $generatorXml.cacheQuery(cache, cache.domains, res); - $generatorXml.cacheStore(cache, cache.domains, res); + $generatorSpring.cacheGeneral(cache, res); + $generatorSpring.cacheMemory(cache, res); + $generatorSpring.cacheQuery(cache, cache.domains, res); + $generatorSpring.cacheStore(cache, cache.domains, res); const igfs = _.get(cache, 'nodeFilter.IGFS.instance'); - $generatorXml.cacheNodeFilter(cache, igfs ? [igfs] : [], res); - $generatorXml.cacheConcurrency(cache, res); - $generatorXml.cacheRebalance(cache, res); - $generatorXml.cacheServerNearCache(cache, res); - $generatorXml.cacheStatistics(cache, res); - $generatorXml.cacheDomains(cache.domains, res); + $generatorSpring.cacheNodeFilter(cache, igfs ? [igfs] : [], res); + $generatorSpring.cacheConcurrency(cache, res); + $generatorSpring.cacheRebalance(cache, res); + $generatorSpring.cacheServerNearCache(cache, res); + $generatorSpring.cacheStatistics(cache, res); + $generatorSpring.cacheDomains(cache.domains, res); return res; }; // Generate caches configs. -$generatorXml.clusterCaches = function(caches, igfss, isSrvCfg, res) { +$generatorSpring.clusterCaches = function(caches, igfss, isSrvCfg, res) { if (!res) res = $generatorCommon.builder(); @@ -1731,18 +1734,18 @@ $generatorXml.clusterCaches = function(caches, igfss, isSrvCfg, res) { res.startBlock(''); _.forEach(caches, function(cache) { - $generatorXml.cache(cache, res); + $generatorSpring.cache(cache, res); res.needEmptyLine = true; }); if (isSrvCfg) { _.forEach(igfss, (igfs) => { - $generatorXml.cache($generatorCommon.igfsDataCache(igfs), res); + $generatorSpring.cache($generatorCommon.igfsDataCache(igfs), res); res.needEmptyLine = true; - $generatorXml.cache($generatorCommon.igfsMetaCache(igfs), res); + $generatorSpring.cache($generatorCommon.igfsMetaCache(igfs), res); res.needEmptyLine = true; }); @@ -1758,7 +1761,7 @@ $generatorXml.clusterCaches = function(caches, igfss, isSrvCfg, res) { }; // Generate IGFSs configs. -$generatorXml.igfss = function(igfss, res) { +$generatorSpring.igfss = function(igfss, res) { if (!res) res = $generatorCommon.builder(); @@ -1771,12 +1774,12 @@ $generatorXml.igfss = function(igfss, res) { _.forEach(igfss, function(igfs) { res.startBlock(''); - $generatorXml.igfsGeneral(igfs, res); - $generatorXml.igfsIPC(igfs, res); - $generatorXml.igfsFragmentizer(igfs, res); - $generatorXml.igfsDualMode(igfs, res); - $generatorXml.igfsSecondFS(igfs, res); - $generatorXml.igfsMisc(igfs, res); + $generatorSpring.igfsGeneral(igfs, res); + $generatorSpring.igfsIPC(igfs, res); + $generatorSpring.igfsFragmentizer(igfs, res); + $generatorSpring.igfsDualMode(igfs, res); + $generatorSpring.igfsSecondFS(igfs, res); + $generatorSpring.igfsMisc(igfs, res); res.endBlock(''); @@ -1793,12 +1796,12 @@ $generatorXml.igfss = function(igfss, res) { }; // Generate IGFS IPC configuration. -$generatorXml.igfsIPC = function(igfs, res) { +$generatorSpring.igfsIPC = function(igfs, res) { if (!res) res = $generatorCommon.builder(); if (igfs.ipcEndpointEnabled) { - $generatorXml.beanProperty(res, igfs.ipcEndpointConfiguration, 'ipcEndpointConfiguration', $generatorCommon.IGFS_IPC_CONFIGURATION, true); + $generatorSpring.beanProperty(res, igfs.ipcEndpointConfiguration, 'ipcEndpointConfiguration', $generatorCommon.IGFS_IPC_CONFIGURATION, true); res.needEmptyLine = true; } @@ -1807,29 +1810,29 @@ $generatorXml.igfsIPC = function(igfs, res) { }; // Generate IGFS fragmentizer configuration. -$generatorXml.igfsFragmentizer = function(igfs, res) { +$generatorSpring.igfsFragmentizer = function(igfs, res) { if (!res) res = $generatorCommon.builder(); if (igfs.fragmentizerEnabled) { - $generatorXml.property(res, igfs, 'fragmentizerConcurrentFiles', null, 0); - $generatorXml.property(res, igfs, 'fragmentizerThrottlingBlockLength', null, 16777216); - $generatorXml.property(res, igfs, 'fragmentizerThrottlingDelay', null, 200); + $generatorSpring.property(res, igfs, 'fragmentizerConcurrentFiles', null, 0); + $generatorSpring.property(res, igfs, 'fragmentizerThrottlingBlockLength', null, 16777216); + $generatorSpring.property(res, igfs, 'fragmentizerThrottlingDelay', null, 200); res.needEmptyLine = true; } else - $generatorXml.property(res, igfs, 'fragmentizerEnabled'); + $generatorSpring.property(res, igfs, 'fragmentizerEnabled'); return res; }; // Generate IGFS dual mode configuration. -$generatorXml.igfsDualMode = function(igfs, res) { +$generatorSpring.igfsDualMode = function(igfs, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.property(res, igfs, 'dualModeMaxPendingPutsSize', null, 0); + $generatorSpring.property(res, igfs, 'dualModeMaxPendingPutsSize', null, 0); if ($generatorCommon.isDefinedAndNotEmpty(igfs.dualModePutExecutorService)) { res.startBlock(''); @@ -1837,14 +1840,14 @@ $generatorXml.igfsDualMode = function(igfs, res) { res.endBlock(''); } - $generatorXml.property(res, igfs, 'dualModePutExecutorServiceShutdown', null, false); + $generatorSpring.property(res, igfs, 'dualModePutExecutorServiceShutdown', null, false); res.needEmptyLine = true; return res; }; -$generatorXml.igfsSecondFS = function(igfs, res) { +$generatorSpring.igfsSecondFS = function(igfs, res) { if (!res) res = $generatorCommon.builder(); @@ -1858,12 +1861,12 @@ $generatorXml.igfsSecondFS = function(igfs, res) { const nameDefined = $generatorCommon.isDefinedAndNotEmpty(secondFs.userName); const cfgDefined = $generatorCommon.isDefinedAndNotEmpty(secondFs.cfgPath); - $generatorXml.constructorArg(res, 0, secondFs, 'uri'); + $generatorSpring.constructorArg(res, 0, secondFs, 'uri'); if (cfgDefined || nameDefined) - $generatorXml.constructorArg(res, 1, secondFs, 'cfgPath'); + $generatorSpring.constructorArg(res, 1, secondFs, 'cfgPath'); - $generatorXml.constructorArg(res, 2, secondFs, 'userName', null, true); + $generatorSpring.constructorArg(res, 2, secondFs, 'userName', null, true); res.endBlock(''); res.endBlock(''); @@ -1875,7 +1878,7 @@ $generatorXml.igfsSecondFS = function(igfs, res) { }; // Generate IGFS general configuration. -$generatorXml.igfsGeneral = function(igfs, res) { +$generatorSpring.igfsGeneral = function(igfs, res) { if (!res) res = $generatorCommon.builder(); @@ -1883,10 +1886,10 @@ $generatorXml.igfsGeneral = function(igfs, res) { igfs.dataCacheName = $generatorCommon.igfsDataCache(igfs).name; igfs.metaCacheName = $generatorCommon.igfsMetaCache(igfs).name; - $generatorXml.property(res, igfs, 'name'); - $generatorXml.property(res, igfs, 'dataCacheName'); - $generatorXml.property(res, igfs, 'metaCacheName'); - $generatorXml.property(res, igfs, 'defaultMode', null, 'DUAL_ASYNC'); + $generatorSpring.property(res, igfs, 'name'); + $generatorSpring.property(res, igfs, 'dataCacheName'); + $generatorSpring.property(res, igfs, 'metaCacheName'); + $generatorSpring.property(res, igfs, 'defaultMode', null, 'DUAL_ASYNC'); res.needEmptyLine = true; } @@ -1895,22 +1898,22 @@ $generatorXml.igfsGeneral = function(igfs, res) { }; // Generate IGFS misc configuration. -$generatorXml.igfsMisc = function(igfs, res) { +$generatorSpring.igfsMisc = function(igfs, res) { if (!res) res = $generatorCommon.builder(); - $generatorXml.property(res, igfs, 'blockSize', null, 65536); - $generatorXml.property(res, igfs, 'streamBufferSize', null, 65536); - $generatorXml.property(res, igfs, 'maxSpaceSize', null, 0); - $generatorXml.property(res, igfs, 'maximumTaskRangeLength', null, 0); - $generatorXml.property(res, igfs, 'managementPort', null, 11400); - $generatorXml.property(res, igfs, 'perNodeBatchSize', null, 100); - $generatorXml.property(res, igfs, 'perNodeParallelBatchCount', null, 8); - $generatorXml.property(res, igfs, 'prefetchBlocks', null, 0); - $generatorXml.property(res, igfs, 'sequentialReadsBeforePrefetch', null, 0); - $generatorXml.property(res, igfs, 'trashPurgeTimeout', null, 1000); - $generatorXml.property(res, igfs, 'colocateMetadata', null, true); - $generatorXml.property(res, igfs, 'relaxedConsistency', null, true); + $generatorSpring.property(res, igfs, 'blockSize', null, 65536); + $generatorSpring.property(res, igfs, 'streamBufferSize', null, 65536); + $generatorSpring.property(res, igfs, 'maxSpaceSize', null, 0); + $generatorSpring.property(res, igfs, 'maximumTaskRangeLength', null, 0); + $generatorSpring.property(res, igfs, 'managementPort', null, 11400); + $generatorSpring.property(res, igfs, 'perNodeBatchSize', null, 100); + $generatorSpring.property(res, igfs, 'perNodeParallelBatchCount', null, 8); + $generatorSpring.property(res, igfs, 'prefetchBlocks', null, 0); + $generatorSpring.property(res, igfs, 'sequentialReadsBeforePrefetch', null, 0); + $generatorSpring.property(res, igfs, 'trashPurgeTimeout', null, 1000); + $generatorSpring.property(res, igfs, 'colocateMetadata', null, true); + $generatorSpring.property(res, igfs, 'relaxedConsistency', null, true); res.softEmptyLine(); @@ -1930,14 +1933,14 @@ $generatorXml.igfsMisc = function(igfs, res) { }; // Generate DataSource beans. -$generatorXml.generateDataSources = function(datasources, res) { +$generatorSpring.generateDataSources = function(datasources, res) { if (!res) res = $generatorCommon.builder(); if (datasources.length > 0) { res.line(''); - _.forEach(datasources, (datasource) => $generatorXml.generateDataSource(datasource, res)); + _.forEach(datasources, (datasource) => $generatorSpring.generateDataSource(datasource, res)); res.needEmptyLine = true; @@ -1947,7 +1950,7 @@ $generatorXml.generateDataSources = function(datasources, res) { return res; }; -$generatorXml.generateDataSource = function(datasource, res) { +$generatorSpring.generateDataSource = function(datasource, res) { const beanId = datasource.dataSourceBean; res.startBlock(''); @@ -1985,7 +1988,7 @@ $generatorXml.generateDataSource = function(datasource, res) { res.emptyLineIfNeeded(); }; -$generatorXml.clusterConfiguration = function(cluster, clientNearCfg, res) { +$generatorSpring.clusterConfiguration = function(cluster, clientNearCfg, res) { const isSrvCfg = _.isNil(clientNearCfg); if (!isSrvCfg) { @@ -1994,55 +1997,55 @@ $generatorXml.clusterConfiguration = function(cluster, clientNearCfg, res) { res.needEmptyLine = true; } - $generatorXml.clusterGeneral(cluster, res); + $generatorSpring.clusterGeneral(cluster, res); - $generatorXml.clusterAtomics(cluster.atomicConfiguration, res); + $generatorSpring.clusterAtomics(cluster.atomicConfiguration, res); - $generatorXml.clusterBinary(cluster.binaryConfiguration, res); + $generatorSpring.clusterBinary(cluster.binaryConfiguration, res); - $generatorXml.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, res); + $generatorSpring.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, res); - $generatorXml.clusterCollision(cluster.collision, res); + $generatorSpring.clusterCollision(cluster.collision, res); - $generatorXml.clusterCommunication(cluster, res); + $generatorSpring.clusterCommunication(cluster, res); - $generatorXml.clusterConnector(cluster.connector, res); + $generatorSpring.clusterConnector(cluster.connector, res); - $generatorXml.clusterDeployment(cluster, res); + $generatorSpring.clusterDeployment(cluster, res); - $generatorXml.clusterEvents(cluster, res); + $generatorSpring.clusterEvents(cluster, res); - $generatorXml.clusterFailover(cluster, res); + $generatorSpring.clusterFailover(cluster, res); - $generatorXml.clusterLogger(cluster.logger, res); + $generatorSpring.clusterLogger(cluster.logger, res); - $generatorXml.clusterODBC(cluster.odbc, res); + $generatorSpring.clusterODBC(cluster.odbc, res); - $generatorXml.clusterMarshaller(cluster, res); + $generatorSpring.clusterMarshaller(cluster, res); - $generatorXml.clusterMetrics(cluster, res); + $generatorSpring.clusterMetrics(cluster, res); - $generatorXml.clusterSwap(cluster, res); + $generatorSpring.clusterSwap(cluster, res); - $generatorXml.clusterTime(cluster, res); + $generatorSpring.clusterTime(cluster, res); - $generatorXml.clusterPools(cluster, res); + $generatorSpring.clusterPools(cluster, res); - $generatorXml.clusterTransactions(cluster.transactionConfiguration, res); + $generatorSpring.clusterTransactions(cluster.transactionConfiguration, res); - $generatorXml.clusterCaches(cluster.caches, cluster.igfss, isSrvCfg, res); + $generatorSpring.clusterCaches(cluster.caches, cluster.igfss, isSrvCfg, res); - $generatorXml.clusterSsl(cluster, res); + $generatorSpring.clusterSsl(cluster, res); if (isSrvCfg) - $generatorXml.igfss(cluster.igfss, res); + $generatorSpring.igfss(cluster.igfss, res); - $generatorXml.clusterUserAttributes(cluster, res); + $generatorSpring.clusterUserAttributes(cluster, res); return res; }; -$generatorXml.cluster = function(cluster, clientNearCfg) { +$generatorSpring.cluster = function(cluster, clientNearCfg) { if (cluster) { const res = $generatorCommon.builder(1); @@ -2050,10 +2053,10 @@ $generatorXml.cluster = function(cluster, clientNearCfg) { res.startBlock(''); if (clientNearCfg.nearStartSize) - $generatorXml.property(res, clientNearCfg, 'nearStartSize'); + $generatorSpring.property(res, clientNearCfg, 'nearStartSize'); if (clientNearCfg.nearEvictionPolicy && clientNearCfg.nearEvictionPolicy.kind) - $generatorXml.evictionPolicy(res, clientNearCfg.nearEvictionPolicy, 'nearEvictionPolicy'); + $generatorSpring.evictionPolicy(res, clientNearCfg.nearEvictionPolicy, 'nearEvictionPolicy'); res.endBlock(''); @@ -2065,7 +2068,7 @@ $generatorXml.cluster = function(cluster, clientNearCfg) { // Generate Ignite Configuration. res.startBlock(''); - $generatorXml.clusterConfiguration(cluster, clientNearCfg, res); + $generatorSpring.clusterConfiguration(cluster, clientNearCfg, res); res.endBlock(''); @@ -2091,7 +2094,7 @@ $generatorXml.cluster = function(cluster, clientNearCfg) { } // 3. Add data sources. - xml += $generatorXml.generateDataSources(res.datasources, $generatorCommon.builder(1)).asString(); + xml += $generatorSpring.generateDataSources(res.datasources, $generatorCommon.builder(1)).asString(); // 3. Add main content. xml += res.asString(); @@ -2105,4 +2108,4 @@ $generatorXml.cluster = function(cluster, clientNearCfg) { return ''; }; -export default $generatorXml; +export default $generatorSpring; diff --git a/modules/web-console/frontend/app/modules/form/field/input/text.scss b/modules/web-console/frontend/app/modules/form/field/input/text.scss index c76bebdd9813f..6882469de8ab7 100644 --- a/modules/web-console/frontend/app/modules/form/field/input/text.scss +++ b/modules/web-console/frontend/app/modules/form/field/input/text.scss @@ -38,4 +38,3 @@ .input-tip .form-control-feedback { height: auto; } - diff --git a/modules/web-console/frontend/app/modules/form/form.module.js b/modules/web-console/frontend/app/modules/form/form.module.js index 23eafcd6de831..01dd57cdfbb8f 100644 --- a/modules/web-console/frontend/app/modules/form/form.module.js +++ b/modules/web-console/frontend/app/modules/form/form.module.js @@ -21,7 +21,6 @@ import angular from 'angular'; import './field/field.scss'; import './field/feedback.scss'; import './field/input/text.scss'; -import './field/input/select.scss'; // Panel. import igniteFormPanel from './panel/panel.directive'; @@ -56,6 +55,8 @@ import igniteFormControlFeedback from './field/form-control-feedback.directive'; import igniteFormFieldUp from './field/up.directive'; import igniteFormFieldDown from './field/down.directive'; +import IgniteFormGUID from './services/FormGUID.service.js'; + angular .module('ignite-console.Form', [ @@ -88,9 +89,6 @@ angular .directive(...igniteFormControlFeedback) .directive(...igniteFormFieldUp) .directive(...igniteFormFieldDown) -// Generator of globally unique identifier. -.factory('IgniteFormGUID', [() => { - let guid = 0; - return () => `form-field-${guid++}`; -}]); +// Generator of globally unique identifier. +.service('IgniteFormGUID', IgniteFormGUID); diff --git a/modules/web-console/frontend/app/modules/form/panel/revert.directive.js b/modules/web-console/frontend/app/modules/form/panel/revert.directive.js index 2076b0d428217..c2454fdeb4000 100644 --- a/modules/web-console/frontend/app/modules/form/panel/revert.directive.js +++ b/modules/web-console/frontend/app/modules/form/panel/revert.directive.js @@ -33,9 +33,9 @@ export default ['igniteFormRevert', ['$tooltip', 'IgniteLegacyTable', ($tooltip, if (field) { field.$viewValue = value; - field.$setViewValue(value); + field.$setViewValue && field.$setViewValue(value); field.$setPristine(); - field.$render(); + field.$render && field.$render(); } }); diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Xml.service.js b/modules/web-console/frontend/app/modules/form/services/FormGUID.service.js similarity index 86% rename from modules/web-console/frontend/app/modules/configuration/generator/Xml.service.js rename to modules/web-console/frontend/app/modules/form/services/FormGUID.service.js index 58d1ce0a75237..b886851766ffb 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/Xml.service.js +++ b/modules/web-console/frontend/app/modules/form/services/FormGUID.service.js @@ -15,7 +15,8 @@ * limitations under the License. */ -// TODO IGNITE-2052: need move $generatorXml to services. -export default ['GeneratorXml', () => { - return $generatorXml; +export default [() => { + let guid = 0; + + return () => `form-field-${guid++}`; }]; diff --git a/modules/web-console/frontend/app/modules/form/validator/java-built-in-class.directive.js b/modules/web-console/frontend/app/modules/form/validator/java-built-in-class.directive.js index 1a4b504cf212b..f9aadc4aba0e1 100644 --- a/modules/web-console/frontend/app/modules/form/validator/java-built-in-class.directive.js +++ b/modules/web-console/frontend/app/modules/form/validator/java-built-in-class.directive.js @@ -20,7 +20,11 @@ export default ['javaBuiltInClass', ['JavaTypes', (JavaTypes) => { if (_.isUndefined(attrs.javaBuiltInClass) || !attrs.javaBuiltInClass) return; - ngModel.$validators.javaBuiltInClass = (value) => JavaTypes.nonBuiltInClass(value); + ngModel.$validators.javaBuiltInClass = (value) => attrs.validationActive === 'false' || + JavaTypes.nonBuiltInClass(value); + + if (attrs.validationActive !== 'always') + attrs.$observe('validationActive', () => ngModel.$validate()); }; return { diff --git a/modules/web-console/frontend/app/modules/form/validator/java-identifier.directive.js b/modules/web-console/frontend/app/modules/form/validator/java-identifier.directive.js index 2abe1df0db33d..21ebfa0f8a4ce 100644 --- a/modules/web-console/frontend/app/modules/form/validator/java-identifier.directive.js +++ b/modules/web-console/frontend/app/modules/form/validator/java-identifier.directive.js @@ -20,7 +20,11 @@ export default ['javaIdentifier', ['JavaTypes', (JavaTypes) => { if (_.isNil(attrs.javaIdentifier) || attrs.javaIdentifier !== 'true') return; - ngModel.$validators.javaIdentifier = (value) => _.isEmpty(value) || JavaTypes.validClassName(value); + ngModel.$validators.javaIdentifier = (value) => attrs.validationActive === 'false' || + _.isEmpty(value) || JavaTypes.validClassName(value); + + if (attrs.validationActive !== 'always') + attrs.$observe('validationActive', () => ngModel.$validate()); }; return { diff --git a/modules/web-console/frontend/app/modules/form/validator/java-keywords.directive.js b/modules/web-console/frontend/app/modules/form/validator/java-keywords.directive.js index c9eef414e37f7..9d5c2aa65f78d 100644 --- a/modules/web-console/frontend/app/modules/form/validator/java-keywords.directive.js +++ b/modules/web-console/frontend/app/modules/form/validator/java-keywords.directive.js @@ -22,16 +22,13 @@ export default ['javaKeywords', ['JavaTypes', (JavaTypes) => { const packageOnly = attrs.javaPackageName === 'package-only'; - ngModel.$validators.javaKeywords = (value) => { - if (value) { - if (_.isEmpty(value) || !JavaTypes.validClassName(value) || (!packageOnly && !JavaTypes.packageSpecified(value))) - return true; + ngModel.$validators.javaKeywords = (value) => attrs.validationActive === 'false' || + _.isEmpty(value) || !JavaTypes.validClassName(value) || + (!packageOnly && !JavaTypes.packageSpecified(value)) || + _.findIndex(value.split('.'), JavaTypes.isKeyword) < 0; - return _.findIndex(value.split('.'), JavaTypes.isKeyword) < 0; - } - - return true; - }; + if (attrs.validationActive !== 'always') + attrs.$observe('validationActive', () => ngModel.$validate()); }; return { diff --git a/modules/web-console/frontend/app/modules/form/validator/java-package-specified.directive.js b/modules/web-console/frontend/app/modules/form/validator/java-package-specified.directive.js index 5f24eb7eddf7b..d3274ae3bbf59 100644 --- a/modules/web-console/frontend/app/modules/form/validator/java-package-specified.directive.js +++ b/modules/web-console/frontend/app/modules/form/validator/java-package-specified.directive.js @@ -22,9 +22,13 @@ export default ['javaPackageSpecified', ['JavaTypes', (JavaTypes) => { const allowBuiltIn = attrs.javaPackageSpecified === 'allow-built-in'; - ngModel.$validators.javaPackageSpecified = (value) => _.isEmpty(value) || + ngModel.$validators.javaPackageSpecified = (value) => attrs.validationActive === 'false' || + _.isEmpty(value) || !JavaTypes.validClassName(value) || JavaTypes.packageSpecified(value) || (allowBuiltIn && !JavaTypes.nonBuiltInClass(value)); + + if (attrs.validationActive !== 'always') + attrs.$observe('validationActive', () => ngModel.$validate()); }; return { diff --git a/modules/web-console/frontend/app/modules/nodes/Nodes.service.js b/modules/web-console/frontend/app/modules/nodes/Nodes.service.js new file mode 100644 index 0000000000000..b320ae4461a31 --- /dev/null +++ b/modules/web-console/frontend/app/modules/nodes/Nodes.service.js @@ -0,0 +1,69 @@ +/* + * 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. + */ + +import nodesDialogTemplate from './nodes-dialog.jade'; + +const DEFAULT_OPTIONS = { + grid: { + multiSelect: false + } +}; + +class Nodes { + static $inject = ['$q', '$modal']; + + /** + * @param $q + * @param $modal + */ + constructor($q, $modal) { + this.$q = $q; + this.$modal = $modal; + } + + selectNode(nodes, cacheName, options = DEFAULT_OPTIONS) { + const { $q, $modal } = this; + const defer = $q.defer(); + options.target = cacheName; + + const modalInstance = $modal({ + templateUrl: nodesDialogTemplate, + show: true, + resolve: { + nodes: () => nodes || [], + options: () => options + }, + placement: 'center', + controller: 'nodesDialogController', + controllerAs: '$ctrl' + }); + + modalInstance.$scope.$ok = (data) => { + defer.resolve(data); + modalInstance.$scope.$hide(); + }; + + modalInstance.$scope.$cancel = () => { + defer.reject(); + modalInstance.$scope.$hide(); + }; + + return defer.promise; + } +} + +export default Nodes; diff --git a/modules/web-console/frontend/app/modules/nodes/nodes-dialog.controller.js b/modules/web-console/frontend/app/modules/nodes/nodes-dialog.controller.js new file mode 100644 index 0000000000000..3e588acdd2bf9 --- /dev/null +++ b/modules/web-console/frontend/app/modules/nodes/nodes-dialog.controller.js @@ -0,0 +1,68 @@ +/* + * 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. + */ + +const NID_TEMPLATE = '
          {{ COL_FIELD | limitTo:8 }}
          '; + +const COLUMNS_DEFS = [ + {displayName: 'Node ID8', field: 'nid', headerTooltip: 'Node ID8', cellTemplate: NID_TEMPLATE, minWidth: 85, width: 85, pinnedLeft: true}, + {displayName: 'Node IP', field: 'ip', headerTooltip: 'Primary IP address of node', minWidth: 75, width: 120}, + {displayName: 'Grid name', field: 'gridName', headerTooltip: 'Name of node grid cluster', minWidth: 75, width: 120}, + {displayName: 'Version', field: 'version', headerTooltip: 'Node version', minWidth: 75, width: 140}, + {displayName: 'OS information', field: 'os', headerTooltip: 'OS information for node\'s host', minWidth: 125} +]; + +export default ['$scope', '$animate', 'uiGridConstants', 'nodes', 'options', function($scope, $animate, uiGridConstants, nodes, options) { + const $ctrl = this; + + const updateSelected = () => { + const nids = $ctrl.gridApi.selection.getSelectedRows().map((node) => node.nid).sort(); + + if (!_.isEqual(nids, $ctrl.selected)) + $ctrl.selected = nids; + }; + + $ctrl.nodes = nodes; + $ctrl.options = options; + $ctrl.selected = []; + + $ctrl.gridOptions = { + data: nodes, + columnVirtualizationThreshold: 30, + columnDefs: COLUMNS_DEFS, + enableRowSelection: true, + enableRowHeaderSelection: false, + enableColumnMenus: false, + multiSelect: true, + modifierKeysToMultiSelect: true, + noUnselect: false, + flatEntityAccess: true, + fastWatch: true, + onRegisterApi: (api) => { + $animate.enabled(api.grid.element, false); + + $ctrl.gridApi = api; + + api.selection.on.rowSelectionChanged($scope, updateSelected); + api.selection.on.rowSelectionChangedBatch($scope, updateSelected); + + $ctrl.gridApi.grid.element.css('height', '270px'); + + setTimeout(() => $ctrl.gridApi.core.notifyDataChange(uiGridConstants.dataChange.COLUMN), 300); + }, + ...options.grid + }; +}]; diff --git a/modules/web-console/frontend/app/modules/nodes/nodes-dialog.jade b/modules/web-console/frontend/app/modules/nodes/nodes-dialog.jade new file mode 100644 index 0000000000000..d9ea68cdff2a4 --- /dev/null +++ b/modules/web-console/frontend/app/modules/nodes/nodes-dialog.jade @@ -0,0 +1,35 @@ +//- + 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. + +.modal.ignite-nodes-dialog(tabindex='-1' role='dialog') + .modal-dialog + .modal-content + .modal-header + button.close(ng-click='$cancel()' aria-hidden='true') × + h4.modal-title Select Node + .modal-body.modal-body-with-scroll + p Choose node to execute query for cache: #[strong {{ $ctrl.options.target }}] + + .panel.panel-default.nodes-grid + .panel-heading + label Cache Nodes: {{ $ctrl.nodes.length }} + + .panel-body.panel-body_collapse + .grid(ui-grid='$ctrl.gridOptions' ui-grid-resize-columns ui-grid-selection ui-grid-pinning) + + .modal-footer + button.btn.btn-primary(id='confirm-btn-confirm' ng-click='$ok($ctrl.selected)' ng-disabled='$ctrl.selected.length === 0') Select node + button.btn.btn-default(id='confirm-btn-close' ng-click='$cancel()') Cancel diff --git a/modules/web-console/frontend/app/modules/nodes/nodes-dialog.scss b/modules/web-console/frontend/app/modules/nodes/nodes-dialog.scss new file mode 100644 index 0000000000000..0c65e54ef3f2f --- /dev/null +++ b/modules/web-console/frontend/app/modules/nodes/nodes-dialog.scss @@ -0,0 +1,20 @@ +.ignite-nodes-dialog { + label { + font-size: 18px; + margin-right: 20px; + } + + .ui-grid-pinned-container.ui-grid-pinned-container-left .ui-grid-cell:last-child, + .ui-grid-pinned-container.ui-grid-pinned-container-left .ui-grid-header-cell:last-child, + .ui-grid-header-cell:last-child .ui-grid-column-resizer.right { + //border-right: none; + } + + .nodes-grid { + height: 320px; + } + .panel-body_collapse { + padding: 0; + margin: 0; + } +} diff --git a/modules/web-console/frontend/app/modules/nodes/nodes.module.js b/modules/web-console/frontend/app/modules/nodes/nodes.module.js new file mode 100644 index 0000000000000..4e68b39cc5de2 --- /dev/null +++ b/modules/web-console/frontend/app/modules/nodes/nodes.module.js @@ -0,0 +1,27 @@ +/* + * 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. + */ + +import angular from 'angular'; + +import './nodes-dialog.scss'; + +import Nodes from './Nodes.service'; +import nodesDialogController from './nodes-dialog.controller'; + +angular.module('ignite-console.nodes', []) + .service('IgniteNodes', Nodes) + .controller('nodesDialogController', nodesDialogController); diff --git a/modules/web-console/frontend/app/modules/sql/sql.controller.js b/modules/web-console/frontend/app/modules/sql/sql.controller.js index 92eb7be39bdae..0c2be01e833b8 100644 --- a/modules/web-console/frontend/app/modules/sql/sql.controller.js +++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js @@ -27,6 +27,8 @@ const SCAN_CACHE_WITH_FILTER = 'VISOR_SCAN_CACHE_WITH_FILTER'; /** Prefix for node local key for SCAN near queries. */ const SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE = 'VISOR_SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE'; +const NON_COLLOCATED_JOINS_SINCE = '1.7.0'; + const _fullColName = (col) => { const res = []; @@ -51,6 +53,25 @@ class Paragraph { _.assign(this, paragraph); + const _enableColumns = (categories, visible) => { + _.forEach(categories, (cat) => { + cat.visible = visible; + + _.forEach(this.gridOptions.columnDefs, (col) => { + if (col.displayName === cat.name) + col.visible = visible; + }); + }); + + this.gridOptions.api.grid.refresh(); + }; + + const _selectableColumns = () => _.filter(this.gridOptions.categories, (cat) => cat.selectable); + + this.toggleColumns = (category, visible) => _enableColumns([category], visible); + this.selectAllColumns = () => _enableColumns(_selectableColumns(), true); + this.clearAllColumns = () => _enableColumns(_selectableColumns(), false); + Object.defineProperty(this, 'gridOptions', {value: { enableGridMenu: false, enableColumnMenus: false, @@ -60,6 +81,7 @@ class Paragraph { if (_.isNil(this.api)) return; + this.categories = []; this.columnDefs = _.reduce(self.meta, (cols, col, idx) => { if (self.columnFilter(col)) { cols.push({ @@ -69,6 +91,12 @@ class Paragraph { minWidth: 50, cellClass: 'cell-left' }); + + this.categories.push({ + name: col.fieldName, + visible: true, + selectable: true + }); } return cols; @@ -133,7 +161,7 @@ class Paragraph { } queryExecuted() { - return !_.isEmpty(this.meta); + return !_.isEmpty(this.meta) || !_.isEmpty(this.errMsg); } scanExplain() { @@ -154,8 +182,8 @@ class Paragraph { } // Controller for SQL notebook screen. -export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', '$animate', '$location', '$anchorScroll', '$state', '$modal', '$popover', 'IgniteLoading', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteAgentMonitor', 'IgniteChartColors', 'IgniteNotebook', 'IgniteScanFilterInput', 'uiGridExporterConstants', - function($root, $scope, $http, $q, $timeout, $interval, $animate, $location, $anchorScroll, $state, $modal, $popover, Loading, LegacyUtils, Messages, Confirm, agentMonitor, IgniteChartColors, Notebook, ScanFilterInput, uiGridExporterConstants) { +export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', '$animate', '$location', '$anchorScroll', '$state', '$filter', '$modal', '$popover', 'IgniteLoading', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteAgentMonitor', 'IgniteChartColors', 'IgniteNotebook', 'IgniteScanFilterInput', 'IgniteNodes', 'uiGridExporterConstants', 'IgniteVersion', + function($root, $scope, $http, $q, $timeout, $interval, $animate, $location, $anchorScroll, $state, $filter, $modal, $popover, Loading, LegacyUtils, Messages, Confirm, agentMonitor, IgniteChartColors, Notebook, ScanFilterInput, Nodes, uiGridExporterConstants, Version) { let stopTopology = null; const _tryStopRefresh = function(paragraph) { @@ -212,7 +240,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', } }; - $scope.maskCacheName = (cacheName) => _.isEmpty(cacheName) ? '' : cacheName; + const maskCacheName = $filter('defaultName'); // We need max 1800 items to hold history for 30 mins in case of refresh every second. const HISTORY_LENGTH = 1800; @@ -781,24 +809,30 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', * @private */ const _refreshFn = () => - agentMonitor.topology() - .then((clusters) => { - $scope.caches = _.sortBy(_.reduce(clusters, (items, cluster) => { - _.forEach(cluster.caches, (cache) => { - let item = _.find(items, {name: cache.name}); + agentMonitor.topology(true) + .then((nodes) => { + $scope.caches = _.sortBy(_.reduce(nodes, (cachesAcc, node) => { + _.forEach(node.caches, (cache) => { + let item = _.find(cachesAcc, {name: cache.name}); if (_.isNil(item)) { - cache.label = $scope.maskCacheName(cache.name); + cache.label = maskCacheName(cache.name); - cache.nodeIds = []; + cache.nodes = []; - items.push(item = cache); + cachesAcc.push(item = cache); } - item.nodeIds.push(cluster.nodeId); + item.nodes.push({ + nid: node.nodeId.toUpperCase(), + ip: _.head(node.attributes['org.apache.ignite.ips'].split(', ')), + version: node.attributes['org.apache.ignite.build.ver'], + gridName: node.attributes['org.apache.ignite.ignite.name'], + os: `${node.attributes['os.name']} ${node.attributes['os.arch']} ${node.attributes['os.version']}` + }); }); - return items; + return cachesAcc; }, []), 'label'); if (_.isEmpty($scope.caches)) @@ -830,7 +864,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', .then(_refreshFn) .then(() => Loading.finish('sqlLoading')) .then(() => { - $root.IgniteDemoMode && _.forEach($scope.notebook.paragraphs, $scope.execute); + $root.IgniteDemoMode && _.forEach($scope.notebook.paragraphs, (paragraph) => $scope.execute(paragraph)); stopTopology = $interval(_refreshFn, 5000, 0, false); }); @@ -1072,7 +1106,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', paragraph.gridOptions.rebuildColumns(); paragraph.chartColumns = _.reduce(paragraph.meta, (acc, col, idx) => { - if (paragraph.columnFilter(col) && _notObjectType(col.fieldTypeName)) { + if (_notObjectType(col.fieldTypeName)) { acc.push({ label: col.fieldName, type: col.fieldTypeName, @@ -1100,8 +1134,6 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', if (paragraph.disabledSystemColumns) return; - paragraph.systemColumns = !paragraph.systemColumns; - paragraph.columnFilter = _columnFilter(paragraph); paragraph.chartColumns = []; @@ -1113,10 +1145,11 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', /** * @param {Object} paragraph Query + * @param {Boolean} clearChart Flag is need clear chart model. * @param {{columns: Array, rows: Array, responseNodeId: String, queryId: int, hasMore: Boolean}} res Query results. * @private */ - const _processQueryResult = (paragraph, res) => { + const _processQueryResult = (paragraph, clearChart, res) => { const prevKeyCols = paragraph.chartKeyCols; const prevValCols = paragraph.chartValCols; @@ -1131,11 +1164,12 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', if (!LegacyUtils.isDefined(paragraph.chartValCols)) paragraph.chartValCols = []; - if (res.columns.length <= 2) { + if (res.columns.length) { const _key = _.find(res.columns, {fieldName: '_KEY'}); const _val = _.find(res.columns, {fieldName: '_VAL'}); - paragraph.disabledSystemColumns = (res.columns.length === 2 && _key && _val) || + paragraph.disabledSystemColumns = !(_key && _val) || + (res.columns.length === 2 && _key && _val) || (res.columns.length === 1 && (_key || _val)); } @@ -1175,12 +1209,8 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', const chartHistory = paragraph.chartHistory; - // Clear history on query change. - const queryChanged = paragraph.prevQuery !== paragraph.query; - - if (queryChanged) { - paragraph.prevQuery = paragraph.query; - + // Clear history on query change. + if (clearChart) { chartHistory.length = 0; _.forEach(paragraph.charts, (chart) => chart.data.length = 0); @@ -1198,7 +1228,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', if (_.isNil(paragraph.result) || paragraph.result === 'none' || paragraph.scanExplain()) paragraph.result = 'table'; else if (paragraph.chart()) { - let resetCharts = queryChanged; + let resetCharts = clearChart; if (!resetCharts) { const curKeyCols = paragraph.chartKeyCols; @@ -1214,15 +1244,34 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', }; const _closeOldQuery = (paragraph) => { - const queryId = paragraph.queryArgs && paragraph.queryArgs.queryId; + if (paragraph.queryId) + return agentMonitor.queryClose(paragraph.resNodeId, paragraph.queryId); - return queryId ? agentMonitor.queryClose(queryId) : $q.when(); + return $q.when(); }; - const cacheNode = (name) => { - const cache = _.find($scope.caches, {name}); + /** + * @param {String} name Cache name. + * @return {Array.} Nids + */ + const cacheNodes = (name) => { + return _.find($scope.caches, {name}).nodes; + }; + + /** + * @param {String} name Cache name. + * @param {Boolean} local Local query. + * @return {String} Nid + */ + const _chooseNode = (name, local) => { + const nodes = cacheNodes(name); + + if (local) { + return Nodes.selectNode(nodes, name) + .then((selectedNids) => _.head(selectedNids)); + } - return cache.nodeIds[_.random(0, cache.nodeIds.length - 1)]; + return Promise.resolve(nodes[_.random(0, nodes.length - 1)].nid); }; const _executeRefresh = (paragraph) => { @@ -1230,8 +1279,9 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', agentMonitor.awaitAgent() .then(() => _closeOldQuery(paragraph)) - .then(() => agentMonitor.query(cacheNode(args.cacheName), args.cacheName, args.query, false, args.pageSize)) - .then(_processQueryResult.bind(this, paragraph)) + .then(() => args.localNid || _chooseNode(args.cacheName, false)) + .then((nid) => agentMonitor.query(nid, args.cacheName, args.query, args.nonCollocatedJoins, !!args.localNid, args.pageSize)) + .then(_processQueryResult.bind(this, paragraph, false)) .catch((err) => paragraph.errMsg = err.message); }; @@ -1249,41 +1299,62 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', } }; - $scope.execute = (paragraph) => { - if (!$scope.actionAvailable(paragraph, true)) - return; + const addLimit = (query, limitSize) => + `SELECT * FROM ( + ${query} + ) LIMIT ${limitSize}`; - Notebook.save($scope.notebook) - .catch(Messages.showError); + $scope.nonCollocatedJoinsAvailable = (paragraph) => { + const cache = _.find($scope.caches, {name: paragraph.cacheName}); - paragraph.prevQuery = paragraph.queryArgs ? paragraph.queryArgs.query : paragraph.query; + if (cache) + return !!_.find(cache.nodes, (node) => Version.since(node.version, NON_COLLOCATED_JOINS_SINCE)); - _showLoading(paragraph, true); + return false; + }; - _closeOldQuery(paragraph) - .then(() => { - const args = paragraph.queryArgs = { - cacheName: paragraph.cacheName, - pageSize: paragraph.pageSize, - query: paragraph.query, - type: 'QUERY' - }; + $scope.execute = (paragraph, nonCollocatedJoins = false) => { + const local = !!paragraph.localQry; - return agentMonitor.query(cacheNode(paragraph.cacheName), args.cacheName, args.query, false, args.pageSize); - }) - .then((res) => { - _processQueryResult(paragraph, res); + $scope.actionAvailable(paragraph, true) && _chooseNode(paragraph.cacheName, local) + .then((nid) => { + Notebook.save($scope.notebook) + .catch(Messages.showError); - _tryStartRefresh(paragraph); - }) - .catch((err) => { - paragraph.errMsg = err.message; + paragraph.prevQuery = paragraph.queryArgs ? paragraph.queryArgs.query : paragraph.query; - _showLoading(paragraph, false); + _showLoading(paragraph, true); - $scope.stopRefresh(paragraph); - }) - .then(() => paragraph.ace.focus()); + return _closeOldQuery(paragraph) + .then(() => { + const args = paragraph.queryArgs = { + cacheName: paragraph.cacheName, + pageSize: paragraph.pageSize, + query: paragraph.query, + firstPageOnly: paragraph.firstPageOnly, + nonCollocatedJoins, + type: 'QUERY', + localNid: local ? nid : null + }; + + const qry = args.firstPageOnly ? addLimit(args.query, args.pageSize) : paragraph.query; + + return agentMonitor.query(nid, args.cacheName, qry, nonCollocatedJoins, local, args.pageSize); + }) + .then((res) => { + _processQueryResult(paragraph, true, res); + + _tryStartRefresh(paragraph); + }) + .catch((err) => { + paragraph.errMsg = err.message; + + _showLoading(paragraph, false); + + $scope.stopRefresh(paragraph); + }) + .then(() => paragraph.ace.focus()); + }); }; const _cancelRefresh = (paragraph) => { @@ -1310,7 +1381,8 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', _showLoading(paragraph, true); _closeOldQuery(paragraph) - .then(() => { + .then(() => _chooseNode(paragraph.cacheName, false)) + .then((nid) => { const args = paragraph.queryArgs = { cacheName: paragraph.cacheName, pageSize: paragraph.pageSize, @@ -1318,9 +1390,9 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', type: 'EXPLAIN' }; - return agentMonitor.query(cacheNode(paragraph.cacheName), args.cacheName, args.query, false, args.pageSize); + return agentMonitor.query(nid, args.cacheName, args.query, false, false, args.pageSize); }) - .then(_processQueryResult.bind(this, paragraph)) + .then(_processQueryResult.bind(this, paragraph, true)) .catch((err) => { paragraph.errMsg = err.message; @@ -1330,34 +1402,48 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', }; $scope.scan = (paragraph, query = null) => { - if (!$scope.actionAvailable(paragraph, false)) - return; + const local = !!paragraph.localQry; - Notebook.save($scope.notebook) - .catch(Messages.showError); + $scope.actionAvailable(paragraph, false) && _chooseNode(paragraph.cacheName, local) + .then((nid) => { + Notebook.save($scope.notebook) + .catch(Messages.showError); - _cancelRefresh(paragraph); + _cancelRefresh(paragraph); - _showLoading(paragraph, true); + _showLoading(paragraph, true); - _closeOldQuery(paragraph) - .then(() => { - const args = paragraph.queryArgs = { - cacheName: paragraph.cacheName, - pageSize: paragraph.pageSize, - query, - type: 'SCAN' - }; + _closeOldQuery(paragraph) + .then(() => { + const args = paragraph.queryArgs = { + cacheName: paragraph.cacheName, + pageSize: paragraph.pageSize, + firstPageOnly: paragraph.firstPageOnly, + query, + type: 'SCAN', + localNid: local ? nid : null + }; - return agentMonitor.query(cacheNode(paragraph.cacheName), args.cacheName, query, false, args.pageSize); - }) - .then(_processQueryResult.bind(this, paragraph)) - .catch((err) => { - paragraph.errMsg = err.message; + return agentMonitor.query(nid, args.cacheName, query, false, local, args.pageSize); + }) + .then((res) => { + if (paragraph.firstPageOnly) { + res.hasMore = false; - _showLoading(paragraph, false); - }) - .then(() => paragraph.ace.focus()); + _processQueryResult(paragraph, true, res); + + _closeOldQuery(paragraph); + } + else + _processQueryResult(paragraph, true, res); + }) + .catch((err) => { + paragraph.errMsg = err.message; + + _showLoading(paragraph, false); + }) + .then(() => paragraph.ace.focus()); + }); }; $scope.scanWithFilter = (paragraph) => { @@ -1480,10 +1566,11 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', paragraph.gridOptions.api.exporter.pdfExport(uiGridExporterConstants.ALL, uiGridExporterConstants.VISIBLE); }; - $scope.exportCsvAll = function(paragraph) { + $scope.exportCsvAll = (paragraph) => { const args = paragraph.queryArgs; - agentMonitor.queryGetAll(cacheNode(args.cacheName), args.cacheName, args.query, false) + return Promise.resolve(args.localNid || _chooseNode(args.cacheName, false)) + .then((nid) => agentMonitor.queryGetAll(nid, args.cacheName, args.query, !!args.nonCollocatedJoins, !!args.localNid)) .then((res) => _export(paragraph.name + '-all.csv', paragraph.columnFilter, res.columns, res.rows)) .catch(Messages.showError) .then(() => paragraph.ace.focus()); @@ -1601,7 +1688,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', if (_.isNil(paragraph.queryArgs.query)) { scope.title = 'SCAN query'; - scope.content = [`SCAN query for cache: ${$scope.maskCacheName(paragraph.queryArgs.cacheName)}`]; + scope.content = [`SCAN query for cache: ${maskCacheName(paragraph.queryArgs.cacheName, true)}`]; } else if (paragraph.queryArgs.query.startsWith(SCAN_CACHE_WITH_FILTER)) { scope.title = 'SCAN query'; @@ -1613,7 +1700,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', else filter = paragraph.queryArgs.query.substr(SCAN_CACHE_WITH_FILTER.length); - scope.content = [`SCAN query for cache: ${$scope.maskCacheName(paragraph.queryArgs.cacheName)} with filter: ${filter}`]; + scope.content = [`SCAN query for cache: ${maskCacheName(paragraph.queryArgs.cacheName, true)} with filter: ${filter}`]; } else if (paragraph.queryArgs.query .startsWith('EXPLAIN ')) { scope.title = 'Explain query'; diff --git a/modules/web-console/frontend/app/modules/states/configuration.state.js b/modules/web-console/frontend/app/modules/states/configuration.state.js index 7fd7541f28582..888c804b37af4 100644 --- a/modules/web-console/frontend/app/modules/states/configuration.state.js +++ b/modules/web-console/frontend/app/modules/states/configuration.state.js @@ -30,7 +30,7 @@ angular.module('ignite-console.states.configuration', ['ui.router']) // Summary screen .directive(...summaryTabs) // Services. - .service('igniteConfigurationResource', ConfigurationResource) + .service('IgniteConfigurationResource', ConfigurationResource) // Configure state provider. .config(['$stateProvider', 'AclRouteProvider', ($stateProvider, AclRoute) => { // Setup the states. diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/client-near-cache.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/client-near-cache.jade new file mode 100644 index 0000000000000..9d8ccbea64769 --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/client-near-cache.jade @@ -0,0 +1,50 @@ +//- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +include ../../../../../app/helpers/jade/mixins.jade + +-var form = 'clientNearCache' +-var model = 'backupItem.clientNearConfiguration' + +.panel.panel-default(ng-form=form novalidate ng-show='backupItem.cacheMode === "PARTITIONED"') + .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') + ignite-form-panel-chevron + label Client near cache + ignite-form-field-tooltip.tipLabel + | Near cache settings for client nodes#[br] + | Near cache is a small local cache that stores most recently or most frequently accessed data#[br] + | Should be used in case when it is impossible to send computations to remote nodes + ignite-form-revert + .panel-collapse(role='tabpanel' bs-collapse-target id=form) + .panel-body(ng-if='ui.isPanelLoaded("#{form}")') + .col-sm-6 + -var enabled = model + '.clientNearCacheEnabled' + + .settings-row + +checkbox('Enabled', enabled, '"clientNacheEnabled"', 'Flag indicating whether to configure near cache') + .settings-row + +number('Start size:', model + '.nearStartSize', '"clientNearStartSize"', enabled, '375000', '0', + 'Initial cache size for near cache which will be used to pre-create internal hash table after start') + .settings-row + +evictionPolicy(model + '.nearEvictionPolicy', '"clientNearCacheEvictionPolicy"', enabled, 'false', + 'Near cache eviction policy\ +
            \ +
          • Least Recently Used (LRU) - Eviction policy based on LRU algorithm and supports batch eviction
          • \ +
          • First In First Out (FIFO) - Eviction policy based on FIFO algorithm and supports batch eviction
          • \ +
          • SORTED - Eviction policy which will select the minimum cache entry for eviction
          • \ +
          ') + .col-sm-6 + +preview-xml-java('backupItem', 'cacheClientNearCache') diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade index e3147b19e60a4..e9ff14358db27 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade @@ -23,6 +23,9 @@ include ../../../../../app/helpers/jade/mixins.jade .panel-heading(bs-collapse-toggle) ignite-form-panel-chevron label General + ignite-form-field-tooltip.tipLabel + | Common cache configuration#[br] + | #[a(href="https://apacheignite.readme.io/docs/data-grid" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id='general') .panel-body diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade index e9f29fd5007ac..724418f79fb29 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade @@ -24,7 +24,8 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label Memory ignite-form-field-tooltip.tipLabel - | Cache memory settings + | Cache memory settings#[br] + | #[a(href="https://apacheignite.readme.io/docs/off-heap-memory" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') @@ -77,7 +78,7 @@ include ../../../../../app/helpers/jade/mixins.jade ') .settings-row(data-ng-if=model + '.offHeapMode === 1 && ' + model + '.memoryMode !== "OFFHEAP_VALUES"') +number-required('Off-heap memory max size:', model + '.offHeapMaxMemory', '"offHeapMaxMemory"', 'true', - model + '.offHeapMode === 1', '', 1, + model + '.offHeapMode === 1', 'Enter off-heap memory size', '1', 'Maximum amount of memory available to off-heap storage in bytes') .settings-row -var onHeapTired = model + '.memoryMode === "ONHEAP_TIERED"' @@ -94,7 +95,14 @@ include ../../../../../app/helpers/jade/mixins.jade
        • SORTED - Eviction policy which will select the minimum cache entry for eviction
        • \ ') .settings-row - +number('Start size:', model + '.startSize', '"startSize"', 'true', '1500000', '0', 'Initial cache size in entries number') + +number('Start size:', model + '.startSize', '"startSize"', 'true', '1500000', '0', + 'In terms of size and capacity, Ignite internal cache map acts exactly like a normal Java HashMap: it has some initial capacity\ + (which is pretty small by default), which doubles as data arrives. The process of internal cache map resizing is CPU-intensive\ + and time-consuming, and if you load a huge dataset into cache (which is a normal use case), the map will have to resize a lot of times.\ + To avoid that, you can specify the initial cache map capacity, comparable to the expected size of your dataset.\ + This will save a lot of CPU resources during the load time, because the map would not have to resize.\ + For example, if you expect to load 10 million entries into cache, you can set this property to 10 000 000.\ + This will save you from cache internal map resizes.') .settings-row +checkbox('Swap enabled', model + '.swapEnabled', '"swapEnabled"', 'Flag indicating whether swap storage is enabled or not for this cache') .col-sm-6 diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.jade new file mode 100644 index 0000000000000..ba538c20007ba --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.jade @@ -0,0 +1,51 @@ +//- + 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. + +include ../../../../../app/helpers/jade/mixins.jade + +-var form = 'clientNearCache' +-var model = 'backupItem' + +.panel.panel-default(ng-form=form novalidate ng-show='backupItem.cacheMode === "PARTITIONED"') + .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') + ignite-form-panel-chevron + label Near cache on client node + ignite-form-field-tooltip.tipLabel + | Near cache settings for client nodes#[br] + | Near cache is a small local cache that stores most recently or most frequently accessed data#[br] + | Should be used in case when it is impossible to send computations to remote nodes + ignite-form-revert + .panel-collapse(role='tabpanel' bs-collapse-target id=form) + .panel-body(ng-if='ui.isPanelLoaded("#{form}")') + .col-sm-6 + -var nearCfg = model + '.clientNearConfiguration' + -var enabled = nearCfg + '.enabled' + + .settings-row + +checkbox('Enabled', enabled, '"clientNearEnabled"', 'Flag indicating whether to configure near cache') + .settings-row + +number('Start size:', nearCfg + '.nearStartSize', '"clientNearStartSize"', enabled, '375000', '0', + 'Initial cache size for near cache which will be used to pre-create internal hash table after start') + .settings-row + +evictionPolicy(nearCfg + '.nearEvictionPolicy', '"clientNearCacheEvictionPolicy"', enabled, 'false', + 'Near cache eviction policy\ +
            \ +
          • Least Recently Used (LRU) - Eviction policy based on LRU algorithm and supports batch eviction
          • \ +
          • First In First Out (FIFO) - Eviction policy based on FIFO algorithm and supports batch eviction
          • \ +
          • SORTED - Eviction policy which will select the minimum cache entry for eviction
          • \ +
          ') + .col-sm-6 + +preview-xml-java(model, 'cacheNearClient') diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/server-near-cache.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-server.jade similarity index 90% rename from modules/web-console/frontend/app/modules/states/configuration/caches/server-near-cache.jade rename to modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-server.jade index 74f500bac4960..a96b947971f40 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/server-near-cache.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-server.jade @@ -22,17 +22,18 @@ include ../../../../../app/helpers/jade/mixins.jade .panel.panel-default(ng-form=form novalidate ng-show='#{model}.cacheMode === "PARTITIONED"') .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron - label Server near cache + label Near cache on server node ignite-form-field-tooltip.tipLabel | Near cache settings#[br] | Near cache is a small local cache that stores most recently or most frequently accessed data#[br] - | Should be used in case when it is impossible to send computations to remote nodes + | Should be used in case when it is impossible to send computations to remote nodes#[br] + | #[a(href="https://apacheignite.readme.io/docs/near-caches" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 - -var enabled = model + '.nearCacheEnabled' -var nearCfg = model + '.nearConfiguration' + -var enabled = nearCfg + '.enabled' .settings-row +checkbox('Enabled', enabled, '"nearCacheEnabled"', 'Flag indicating whether to configure near cache') @@ -48,4 +49,4 @@ include ../../../../../app/helpers/jade/mixins.jade
        • SORTED - Eviction policy which will select the minimum cache entry for eviction
        • \ ') .col-sm-6 - +preview-xml-java(model, 'cacheServerNearCache') + +preview-xml-java(model, 'cacheNearServer') diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade index ee28c876e0ee7..eb74736738361 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade @@ -36,7 +36,6 @@ include ../../../../../app/helpers/jade/mixins.jade +dropdown('Node filter:', nodeFilterKind, '"nodeFilter"', 'true', 'Not set', '[\ {value: "IGFS", label: "IGFS nodes"},\ - {value: "OnNodes", label: "Specified nodes"},\ {value: "Custom", label: "Custom"},\ {value: undefined, label: "Not set"}\ ]', @@ -56,53 +55,5 @@ include ../../../../../app/helpers/jade/mixins.jade +java-class('Class name:', customNodeFilter + '.className', '"customNodeFilter"', 'true', required, 'Class name of custom node filter implementation') - div(ng-show='#{nodeFilterKind} === "OnNodes"') - -var nodeSetFilter = nodeFilter + '.OnNodes.nodeIds' - - +ignite-form-group(ng-form=form ng-model=nodeSetFilter) - -var uniqueTip = 'Such node ID already exists!' - - ignite-form-field-label - | Node IDs - ignite-form-group-tooltip - | Set of node IDs to deploy cache - ignite-form-group-add(ng-click='group.add = [{}]') - | Add new node ID - - .group-content(ng-if='#{nodeSetFilter}.length') - -var model = 'obj.model'; - -var name = '"edit" + $index' - -var valid = form + '[' + name + '].$valid' - -var save = nodeSetFilter + '[$index] = ' + model - - div(ng-repeat='model in #{nodeSetFilter} track by $index' ng-init='obj = {}') - label.col-xs-12.col-sm-12.col-md-12 - .indexField - | {{ $index+1 }}) - +table-remove-button(nodeSetFilter, 'Remove node ID') - - span(ng-hide='field.edit') - a.labelFormField(ng-click='field.edit = true; #{model} = model;') {{ model }} - span(ng-if='field.edit' ng-init='#{field} = model') - +table-uuid-field(name, model, nodeSetFilter, valid, save, false, true) - +table-save-button(valid, save, false) - +unique-feedback(name, uniqueTip) - +uuid-feedback(name) - - .group-content(ng-repeat='field in group.add') - -var model = 'new'; - -var name = '"new"' - -var valid = form + '[' + name + '].$valid' - -var save = nodeSetFilter + '.push(' + model + ')' - - div - label.col-xs-12.col-sm-12.col-md-12 - +table-uuid-field(name, model, nodeSetFilter, valid, save, true, true) - +table-save-button(valid, save, true) - +unique-feedback(name, uniqueTip) - +uuid-feedback(name) - .group-content-empty(id='nodeSetFilter' ng-if='!(#{nodeSetFilter}.length) && !group.add.length') - | Not defined - .col-sm-6 +preview-xml-java(model, 'cacheNodeFilter', 'igfss') diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade index c709135672559..ae13166c3fa94 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade @@ -24,21 +24,28 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label Queries & Indexing ignite-form-field-tooltip.tipLabel - | Cache queries settings + | Cache queries settings#[br] + | #[a(href="https://apacheignite.readme.io/docs/sql-queries" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row +text('SQL schema name:', model + '.sqlSchema', '"sqlSchema"', 'false', 'Input schema name', - 'Schema name allow to use existing database queries in your application and according to SQL ANSI-99
          \ - Cache is reffered by schema name in cross-cache queries
          \ - Nonquoted identifiers are not case sensitive. Quoted identifiers are case sensitive
          \ - When SQL schema is not specified, quoted cache name should be used
          \ - Query example without schema name:
          \ - SELECT .... FROM "cache1".Type1 JOIN "cache2".Type2 ...
          \ - The same query using schema name:
          \ - SELECT .... FROM cache1.Type1 JOIN cache2.Type2 ...') + 'Specify any custom name to be used as SQL schema for current cache. This name will correspond to SQL ANSI-99 standard.\ + Nonquoted identifiers are not case sensitive. Quoted identifiers are case sensitive.\ + When SQL schema is not specified, quoted cache name should used instead.
          \ + For example:\ +
            \ +
          • \ + Query without schema names (quoted cache names will be used):\ + SELECT * FROM "PersonsCache".Person p INNER JOIN "OrganizationsCache".Organization o on p.org = o.id\ +
          • \ +
          • \ + The same query using schema names "Persons" and "Organizations":\ + SELECT * FROM Persons.Person p INNER JOIN Organizations.Organization o on p.org = o.id\ +
          • \ +
          ') .settings-row +number('On-heap cache for off-heap indexes:', model + '.sqlOnheapRowCacheSize', '"sqlOnheapRowCacheSize"', 'true', '10240', '1', 'Number of SQL rows which will be cached onheap to avoid deserialization on each SQL index access') @@ -97,7 +104,8 @@ include ../../../../../app/helpers/jade/mixins.jade 'Flag indicating whether SQL indexes should support snapshots') .settings-row +checkbox('Escape table and filed names', model + '.sqlEscapeAll', '"sqlEscapeAll"', - 'If enabled than all the SQL table and field names will be escaped with double quotes like "tableName"."fieldName"
          \ - This enforces case sensitivity for field names and also allows having special characters in table and field names') + 'If enabled than all schema, table and field names will be escaped with double quotes (for example: "tableName"."fieldName").
          \ + This enforces case sensitivity for field names and also allows having special characters in table and field names.
          \ + Escaped names will be used for creation internal structures in Ignite SQL engine.') .col-sm-6 +preview-xml-java(model, 'cacheQuery', 'domains') diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade index 6cf2d33f93f66..824442c35d02a 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade @@ -24,7 +24,8 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label Rebalance ignite-form-field-tooltip.tipLabel - | Cache rebalance settings + | Cache rebalance settings#[br] + | #[a(href="https://apacheignite.readme.io/docs/rebalancing" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade index 84752d6330214..a7d8f1453777b 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade @@ -20,34 +20,35 @@ include ../../../../../app/helpers/jade/mixins.jade -var model = 'backupItem' mixin hibernateField(name, model, items, valid, save, newItem) - -var reset = newItem ? 'group.add = []' : 'field.edit = false' - -var resetOnEnter = newItem ? '(stopblur = true) && (group.add = [{}])' : '(field.edit = false)' -var onEnter = valid + ' && (' + save + '); ' + valid + ' && ' + resetOnEnter + ';' + -var onEscape = newItem ? 'group.add = []' : 'field.edit = false' + -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false' -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';' - if block - block + div(ignite-on-focus-out=onBlur) + if block + block - .input-tip - +ignite-form-field-input(name, model, false, 'true', 'key=value')( - data-ignite-property-unique=items - data-ignite-property-value-specified - data-ignite-form-field-input-autofocus='true' + .input-tip + +ignite-form-field-input(name, model, false, 'true', 'key=value')( + data-ignite-property-unique=items + data-ignite-property-value-specified + data-ignite-form-field-input-autofocus='true' - ng-blur=onBlur - ignite-on-enter=onEnter - ignite-on-escape=onEscape - ) + ignite-on-enter=onEnter + ignite-on-escape=onEscape + ) .panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Store ignite-form-field-tooltip.tipLabel - | Cache store settings + | Cache store settings#[br] + | #[a(href="https://apacheignite.readme.io/docs/persistent-store" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') @@ -86,6 +87,11 @@ mixin hibernateField(name, model, items, valid, save, newItem) +dialect('Dialect:', pojoStoreFactory + '.dialect', '"pojoDialect"', required, 'Dialect of SQL implemented by a particular RDBMS:', 'Generic JDBC dialect', 'Choose JDBC dialect') + .details-row + +checkbox('Escape table and filed names', pojoStoreFactory + '.sqlEscapeAll', '"sqlEscapeAll"', + 'If enabled than all schema, table and field names will be escaped with double quotes (for example: "tableName"."fieldName").
          \ + This enforces case sensitivity for field names and also allows having special characters in table and field names.
          \ + Escaped names will be used for CacheJdbcPojoStore internal SQL queries.') div(ng-show='#{storeFactoryKind} === "CacheJdbcBlobStoreFactory"') -var blobStoreFactory = storeFactory + '.CacheJdbcBlobStoreFactory' -var blobStoreFactoryVia = blobStoreFactory + '.connectVia' @@ -154,46 +160,28 @@ mixin hibernateField(name, model, items, valid, save, newItem) ignite-form-group-tooltip | List of Hibernate properties#[br] | For example: connection.url=jdbc:h2:mem:exampleDb - ignite-form-group-add(ng-click='group.add = [{}]') + ignite-form-group-add(ng-click='tableNewItem(hibernatePropsTbl)') | Add new Hibernate property -var tipUnique = 'Property with such key already exists!' -var tipPropertySpecified = 'Property should be present in format key=value!' - .group-content(ng-if='#{hibernateProperties}.length') - -var model = 'obj.model'; - -var name = '"edit" + $index' - -var valid = form + '[' + name + '].$valid' - -var save = hibernateProperties + '[$index] = ' + model - - div(ng-repeat='model in #{hibernateProperties} track by $index' ng-init='obj = {}') - label.col-xs-12.col-sm-12.col-md-12 - .indexField - | {{ $index+1 }}) - +table-remove-button(hibernateProperties, 'Remove Hibernate property') - - span(ng-hide='field.edit') - a.labelFormField(ng-click='field.edit = true; #{model} = model;') {{ model }} - span(ng-if='field.edit') - +hibernateField(name, model, hibernateProperties, valid, save, false) - +table-save-button(valid, save, false) - +form-field-feedback(name, 'ignitePropertyUnique', tipUnique) - +form-field-feedback(name, 'ignitePropertyValueSpecified', tipPropertySpecified) - - .group-content(ng-repeat='field in group.add') - -var model = 'new'; - -var name = '"new"' - -var valid = form + '[' + name + '].$valid' - -var save = hibernateProperties + '.push(' + model + ')' - - div - label.col-xs-12.col-sm-12.col-md-12 - +hibernateField(name, model, hibernateProperties, valid, save, true) - +table-save-button(valid, save, true) - +form-field-feedback(name, 'ignitePropertyUnique', tipUnique) - +form-field-feedback(name, 'ignitePropertyValueSpecified', tipPropertySpecified) - .group-content-empty(ng-if='!(#{hibernateProperties}.length) && !group.add.length') + .group-content-empty(ng-if='!((#{hibernateProperties} && #{hibernateProperties}.length > 0) || tableNewItemActive(hibernatePropsTbl))') | Not defined + .group-content(ng-show='(#{hibernateProperties} && #{hibernateProperties}.length > 0) || tableNewItemActive(hibernatePropsTbl)') + table.links-edit(id='hibernateProps' st-table=hibernateProperties) + tbody + tr(ng-repeat='item in #{hibernateProperties}') + td.col-sm-12(ng-hide='tableEditing(hibernatePropsTbl, $index)') + a.labelFormField(ng-click='tableStartEdit(backupItem, hibernatePropsTbl, $index)') {{item.name}} = {{item.value}} + +btn-remove('tableRemove(backupItem, hibernatePropsTbl, $index)', '"Remove Property"') + td.col-sm-12(ng-if='tableEditing(hibernatePropsTbl, $index)') + +table-pair-edit('hibernatePropsTbl', 'cur', 'Property name', 'Property value', false, false, '{{::hibernatePropsTbl.focusId + $index}}', '$index', '=') + tfoot(ng-show='tableNewItemActive(hibernatePropsTbl)') + tr + td.col-sm-12 + +table-pair-edit('hibernatePropsTbl', 'new', 'Property name', 'Property value', false, false, '{{::hibernatePropsTbl.focusId + $index}}', '-1', '=') + .settings-row +checkbox('Keep binary in store', model + '.storeKeepBinary', '"storeKeepBinary"', diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade index c4ef88ee2f569..ef83356527a31 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade @@ -25,7 +25,8 @@ include ../../../../../app/helpers/jade/mixins.jade label Atomic configuration ignite-form-field-tooltip.tipLabel | Configuration for atomic data structures#[br] - | Atomics are distributed across the cluster, essentially enabling performing atomic operations (such as increment-and-get or compare-and-set) with the same globally-visible value + | Atomics are distributed across the cluster, essentially enabling performing atomic operations (such as increment-and-get or compare-and-set) with the same globally-visible value#[br] + | #[a(href="https://apacheignite.readme.io/docs/atomic-types" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target='' id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade index 29e7a79b41fb7..b41b97c2099da 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade @@ -43,8 +43,8 @@ include ../../../../../app/helpers/jade/mixins.jade .group-content(ng-show='(#{userAttributes} && #{userAttributes}.length > 0) || tableNewItemActive(attributesTbl)') table.links-edit(id='attributes' st-table=userAttributes) tbody - tr(ng-repeat='item in #{userAttributes}') - td.col-sm-12(ng-show='!tableEditing(attributesTbl, $index)') + tr(ng-repeat='item in #{userAttributes} track by $index') + td.col-sm-12(ng-hide='tableEditing(attributesTbl, $index)') a.labelFormField(ng-click='tableStartEdit(backupItem, attributesTbl, $index)') {{item.name}} = {{item.value}} +btn-remove('tableRemove(backupItem, attributesTbl, $index)', '"Remove attribute"') td.col-sm-12(ng-show='tableEditing(attributesTbl, $index)') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade index c63e2d9e8836c..9994087dde6dd 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade @@ -20,18 +20,13 @@ include ../../../../../app/helpers/jade/mixins.jade -var model = 'backupItem.binaryConfiguration' -var types = model + '.typeConfigurations' -//- Mixin for java name field with enabled condition. -mixin binary-types-java-class(lbl, model, name, enabled, required, remove, autofocus, tip) - +java-class(lbl, model, name, enabled, required, tip) - if (remove) - +table-remove-button(types, 'Remove type configuration') - .panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Binary configuration ignite-form-field-tooltip.tipLabel - | Configuration of specific binary types + | Configuration of specific binary types#[br] + | #[a(href="https://apacheignite.readme.io/docs/binary-marshaller" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') @@ -45,8 +40,6 @@ mixin binary-types-java-class(lbl, model, name, enabled, required, remove, autof .settings-row +java-class('Serializer:', model + '.serializer', '"serializer"', 'true', 'false', 'Class with custom serialization logic for binary objects') .settings-row - -var form = 'binaryTypeConfigurations'; - +ignite-form-group() ignite-form-field-label | Type configurations @@ -59,15 +52,22 @@ mixin binary-types-java-class(lbl, model, name, enabled, required, remove, autof .group-content(ng-repeat='model in #{types} track by $index') hr(ng-if='$index !== 0') .settings-row - +binary-types-java-class('Type name:', 'model.typeName', '"typeName" + $index', 'true', 'true', true, 'true', 'Type name') + +java-class-autofocus('Type name:', 'model.typeName', '"typeName" + $index', 'true', 'true', 'true', 'Type name') + +table-remove-button(types, 'Remove type configuration') .settings-row - +binary-types-java-class('ID mapper:', 'model.idMapper', '"idMapper" + $index', 'true', 'false', false, 'false', + +java-class('ID mapper:', 'model.idMapper', '"idMapper" + $index', 'true', 'false', 'Maps given from BinaryNameMapper type and filed name to ID that will be used by Ignite in internals
          \ - Ignite never writes full strings for field or type/class names. Instead, for performance reasons, Ignite writes integer hash codes for type/class and field names. It has been tested that hash code conflicts for the type/class names or the field names within the same type are virtually non - existent and, to gain performance, it is safe to work with hash codes. For the cases when hash codes for different types or fields actually do collide BinaryIdMapper allows to override the automatically generated hash code IDs for the type and field names') + Ignite never writes full strings for field or type/class names.\ + Instead, for performance reasons, Ignite writes integer hash codes for type/class and field names.\ + It has been tested that hash code conflicts for the type/class names or the field names within the same type are virtually non - existent and,\ + to gain performance, it is safe to work with hash codes.\ + For the cases when hash codes for different types or fields actually do collide BinaryIdMapper allows to override the automatically generated hash code IDs for the type and field names') .settings-row - +binary-types-java-class('Name mapper:', 'model.nameMapper', '"nameMapper" + $index', 'true', 'false', false, 'false', 'Maps type/class and field names to different names') + +java-class('Name mapper:', 'model.nameMapper', '"nameMapper" + $index', 'true', 'false', + 'Maps type/class and field names to different names') .settings-row - +binary-types-java-class('Serializer:', 'model.serializer', '"serializer" + $index', 'true', 'false', false, 'false', 'Class with custom serialization logic for binary object') + +java-class('Serializer:', 'model.serializer', '"serializer" + $index', 'true', 'false', + 'Class with custom serialization logic for binary object') .settings-row +checkbox('Enum', 'model.enum', 'enum', 'Flag indicating that this type is the enum') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade index 3531c779a69be..45ccc13b385d5 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade @@ -18,7 +18,6 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'cacheKeyCfg' -var model = 'backupItem.cacheKeyConfiguration' --var items = model; .panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') @@ -31,9 +30,7 @@ include ../../../../../app/helpers/jade/mixins.jade .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row - -var form = form + 'TypeConfigurations' - - +ignite-form-group(ng-form=form ng-model=model) + +ignite-form-group() ignite-form-field-label | Cache key configuration ignite-form-group-tooltip @@ -45,8 +42,8 @@ include ../../../../../app/helpers/jade/mixins.jade .group-content(ng-repeat='model in #{model} track by $index') hr(ng-if='$index !== 0') .settings-row - +java-class('Type name:', 'model.typeName', '"cacheKeyTypeName" + $index', 'true', 'true', 'Type name') - +table-remove-button(items, 'Remove cache key configuration') + +java-class-autofocus('Type name:', 'model.typeName', '"cacheKeyTypeName" + $index', 'true', 'true', 'true', 'Type name') + +table-remove-button(model, 'Remove cache key configuration') .settings-row +text('Affinity key field name:', 'model.affinityKeyFieldName', '"affinityKeyFieldName" + $index', true, 'Enter field name', 'Affinity key field name') .col-sm-6 diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade new file mode 100644 index 0000000000000..d2552a802634b --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade @@ -0,0 +1,85 @@ +//- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +include ../../../../../app/helpers/jade/mixins.jade + +-var form = 'checkpoint' +-var model = 'backupItem.checkpointSpi' +-var CustomCheckpoint = 'model.kind === "Custom"' + +.panel.panel-default(ng-form=form novalidate) + .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') + ignite-form-panel-chevron + label Checkpointing + ignite-form-field-tooltip.tipLabel + | Checkpointing provides an ability to save an intermediate job state#[br] + | #[a(href="http://apacheignite.gridgain.org/docs/checkpointing" target="_blank") More info] + ignite-form-revert + .panel-collapse(role='tabpanel' bs-collapse-target id=form) + .panel-body(ng-if='ui.isPanelLoaded("#{form}")') + .col-sm-6 + .settings-row(ng-init='checkpointSpiTbl={type: "checkpointSpi", model: "checkpointSpi", focusId: "kind", ui: "checkpoint-table"}') + +ignite-form-group() + ignite-form-field-label + | Checkpoint SPI configurations + ignite-form-group-tooltip + | Checkpoint SPI configurations + ignite-form-group-add(ng-click='tableNewItem(checkpointSpiTbl)') + | Add checkpoint SPI + .group-content-empty(ng-if='!(#{model} && #{model}.length > 0)') + | Not defined + .group-content(ng-show='#{model} && #{model}.length > 0' ng-repeat='model in #{model} track by $index') + hr(ng-if='$index != 0') + .settings-row + +dropdown-required('Checkpoint SPI:', 'model.kind', '"checkpointKind" + $index', 'true', 'true', 'Choose checkpoint configuration variant', '[\ + {value: "FS", label: "File System"},\ + {value: "Cache", label: "Cache"},\ + {value: "S3", label: "Amazon S3"},\ + {value: "JDBC", label: "Database"},\ + {value: "Custom", label: "Custom"}\ + ]', + 'Provides an ability to save an intermediate job state\ +
            \ +
          • File System - Uses a shared file system to store checkpoints
          • \ +
          • Cache - Uses a cache to store checkpoints
          • \ +
          • Amazon S3 - Uses Amazon S3 to store checkpoints
          • \ +
          • Database - Uses a database to store checkpoints
          • \ +
          • Custom - Custom checkpoint SPI implementation
          • \ +
          ') + +table-remove-button(model, 'Remove Checkpoint SPI') + + div(ng-if='model.kind === "FS"') + include ./checkpoint/fs.jade + + div(ng-if='model.kind === "Cache"') + .settings-row + +dropdown-required-empty('Cache:', 'model.Cache.cache', '"checkpointCacheCache"+ $index', 'true', 'true', + 'Choose cache', 'No caches configured for current cluster', 'clusterCaches', 'Cache to use for storing checkpoints') + .settings-row + +java-class('Listener:', 'model.Cache.checkpointListener', '"checkpointCacheListener" + $index', 'true', 'false', + 'Checkpoint listener implementation class name') + + div(ng-show='model.kind === "S3"') + include ./checkpoint/s3.jade + + div(ng-show='model.kind === "JDBC"') + include ./checkpoint/jdbc.jade + + .settings-row(ng-show=CustomCheckpoint) + +java-class('Class name:', 'model.Custom.className', '"checkpointCustomClassName" + $index', 'true', CustomCheckpoint, + 'Custom CheckpointSpi implementation class') + .col-sm-6 + +preview-xml-java('backupItem', 'clusterCheckpoint', 'caches') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade new file mode 100644 index 0000000000000..01f14f4613bcc --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade @@ -0,0 +1,66 @@ +//- + 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. + +include ../../../../../../app/helpers/jade/mixins.jade + +-var form = 'checkpointFsPaths' +-var dirPaths = 'model.FS.directoryPaths' + +.details-row + +ignite-form-group(ng-form=form ng-model=dirPaths) + -var uniqueTip = 'Such path already exists!' + + ignite-form-field-label + | Paths + ignite-form-group-tooltip + | Paths to a shared directory where checkpoints will be stored + ignite-form-group-add(ng-click='(group.add = [{}])') + | Add new path + + .group-content(ng-if='#{dirPaths}.length') + -var model = 'obj.model'; + -var name = '"edit" + $index' + -var valid = form + '[' + name + '].$valid' + -var save = dirPaths + '[$index] = ' + model + + div(ng-repeat='model in #{dirPaths} track by $index' ng-init='obj = {}') + label.col-xs-12.col-sm-12.col-md-12 + .indexField + | {{ $index+1 }}) + +table-remove-conditional-button(dirPaths, 'true', 'Remove path') + span(ng-hide='field.edit') + a.labelFormField(ng-click='(field.edit = true) && (#{model} = model)') {{ model }} + span(ng-if='field.edit') + +table-text-field(name, model, dirPaths, valid, save, 'Input directory path', false) + +table-save-button(valid, save, false) + +unique-feedback(name, uniqueTip) + .group-content(ng-repeat='field in group.add') + -var model = 'new'; + -var name = '"new"' + -var valid = form + '[' + name + '].$valid' + -var save = dirPaths + '.push(' + model + ')' + + div + label.col-xs-12.col-sm-12.col-md-12 + +table-text-field(name, model, dirPaths, valid, save, 'Input directory path', true) + +table-save-button(valid, save, true) + +unique-feedback(name, uniqueTip) + .group-content-empty(ng-if='!(#{dirPaths}.length) && !group.add.length') + | Not defined + +.settings-row + +java-class('Listener:', 'model.FS.checkpointListener', '"checkpointFsListener" + $index', 'true', 'false', + 'Checkpoint listener implementation class name') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade new file mode 100644 index 0000000000000..a7b217f14240a --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade @@ -0,0 +1,45 @@ +//- + 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. + +include ../../../../../../app/helpers/jade/mixins.jade + +.settings-row + +text('Data source bean name:', 'model.JDBC.dataSourceBean', '"checkpointJdbcDataSourceBean" + $index', 'model.kind === "JDBC"', 'Input bean name', + 'Name of the data source bean in Spring context') +.settings-row + +dialect('Dialect:', 'model.JDBC.dialect', '"checkpointJdbcDialect" + $index', 'model.kind === "JDBC"', + 'Dialect of SQL implemented by a particular RDBMS:', 'Generic JDBC dialect', 'Choose JDBC dialect') +.settings-row + +text('User:', 'model.JDBC.user', '"checkpointJdbcUser" + $index', 'false', 'Input user name', 'Checkpoint jdbc user name') +.settings-row + +text('Table name:', 'model.JDBC.checkpointTableName', '"checkpointJdbcCheckpointTableName" + $index', 'false', 'CHECKPOINTS', 'Checkpoint table name') +.settings-row + +text('Key field name:', 'model.JDBC.keyFieldName', '"checkpointJdbcKeyFieldName" + $index', 'false', 'NAME', 'Checkpoint key field name') +.settings-row + +dropdown('Key field type:', 'model.JDBC.keyFieldType', '"checkpointJdbcKeyFieldType" + $index', 'true', 'VARCHAR', 'supportedJdbcTypes', 'Checkpoint key field type') +.settings-row + +text('Value field name:', 'model.JDBC.valueFieldName', '"checkpointJdbcValueFieldName" + $index', 'false', 'VALUE', 'Checkpoint value field name') +.settings-row + +dropdown('Value field type:', 'model.JDBC.valueFieldType', '"checkpointJdbcValueFieldType" + $index', 'true', 'BLOB', 'supportedJdbcTypes', 'Checkpoint value field type') +.settings-row + +text('Expire date field name:', 'model.JDBC.expireDateFieldName', '"checkpointJdbcExpireDateFieldName" + $index', 'false', 'EXPIRE_DATE', 'Checkpoint expire date field name') +.settings-row + +dropdown('Expire date field type:', 'model.JDBC.expireDateFieldType', '"checkpointJdbcExpireDateFieldType"', 'true', 'DATETIME', 'supportedJdbcTypes', 'Checkpoint expire date field type') +.settings-row + +number('Number of retries:', 'model.JDBC.numberOfRetries', '"checkpointJdbcNumberOfRetries"', 'true', '2', '0', 'Number of retries in case of DB failure') +.settings-row + +java-class('Listener:', 'model.JDBC.checkpointListener', '"checkpointJdbcListener" + $index', 'true', 'false', + 'Checkpoint listener implementation class name') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade new file mode 100644 index 0000000000000..8373b03ad2c4a --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade @@ -0,0 +1,174 @@ +//- + 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. + +include ../../../../../../app/helpers/jade/mixins.jade + +-var credentialsModel = 'model.S3.awsCredentials' +-var clientCfgModel = 'model.S3.clientConfiguration' +-var checkpointS3Path = 'model.S3.awsCredentials.kind === "Properties"' +-var checkpointS3Custom = 'model.S3.awsCredentials.kind === "Custom"' + +-var clientRetryModel = clientCfgModel + '.retryPolicy' +-var checkpointS3DefaultMaxRetry = clientRetryModel + '.kind === "DefaultMaxRetries"' +-var checkpointS3DynamoDbMaxRetry = clientRetryModel + '.kind === "DynamoDBMaxRetries"' +-var checkpointS3CustomRetry = clientRetryModel + '.kind === "Custom"' + +.settings-row + +dropdown-required('AWS credentials:', 'model.S3.awsCredentials.kind', '"checkpointS3AwsCredentials"', 'true', 'model.kind === "S3"', 'Custom', '[\ + {value: "Basic", label: "Basic"},\ + {value: "Properties", label: "Properties"},\ + {value: "Anonymous", label: "Anonymous"},\ + {value: "BasicSession", label: "Basic with session"},\ + {value: "Custom", label: "Custom"}\ + ]', + 'AWS credentials\ +
            \ +
          • Basic - Allows callers to pass in the AWS access key and secret access in the constructor
          • \ +
          • Properties - Reads in AWS access keys from a properties file
          • \ +
          • Anonymous - Allows use of "anonymous" credentials
          • \ +
          • Database - Session credentials with keys and session token
          • \ +
          • Custom - Custom AWS credentials provider
          • \ +
          ') +.panel-details(ng-show=checkpointS3Path) + .details-row + +text('Path:', credentialsModel + '.Properties.path', '"checkpointS3PropertiesPath"', checkpointS3Path, 'Input properties file path', + 'The file from which to read the AWS credentials properties') +.panel-details(ng-show=checkpointS3Custom) + .details-row + +java-class('Class name:', credentialsModel + '.Custom.className', '"checkpointS3CustomClassName" + $index', 'true', checkpointS3Custom, + 'Custom AWS credentials provider implementation class') +.settings-row + +text('Bucket name suffix:', 'model.S3.bucketNameSuffix', '"checkpointS3BucketNameSuffix"', 'false', 'default-bucket', 'Bucket name suffix') +.settings-row + +dropdown('Protocol:', clientCfgModel + '.protocol', '"checkpointS3Protocol"', 'true', 'HTTPS', '[\ + {value: "HTTP", label: "HTTP"},\ + {value: "HTTPS", label: "HTTPS"}\ + ]', + 'Provides an ability to save an intermediate job state\ +
            \ +
          • HTTP - Using the HTTP protocol is less secure than HTTPS, but can slightly reduce\ + the system resources used when communicating with AWS
          • \ +
          • HTTPS - Using the HTTPS protocol is more secure than using the HTTP protocol, but\ + may use slightly more system resources. AWS recommends using HTTPS for maximize security
          • \ +
          ') +.settings-row + +number('Maximum connections:', clientCfgModel + '.maxConnections', '"checkpointS3MaxConnections"', + 'true', '50', '1', 'Maximum number of allowed open HTTP connections') +.settings-row + +text('User agent:', clientCfgModel + '.userAgent', '"checkpointS3UserAgent"', 'false', 'System specific header', + 'HTTP user agent header to send with all requests') +.settings-row + +text-ip-address('Local address:', clientCfgModel + '.localAddress', '"checkpointS3LocalAddress"', 'true', 'Not specified', + 'Optionally specifies the local address to bind to') +.settings-row + +text('Proxy host:', clientCfgModel + '.proxyHost', '"checkpointS3ProxyHost"', 'false', 'Not specified', + 'Optional proxy host the client will connect through') +.settings-row + +number('Proxy port:', clientCfgModel + '.proxyPort', '"checkpointS3ProxyPort"', 'true', 'Not specified', '0', + 'Optional proxy port the client will connect through') +.settings-row + +text('Proxy user:', clientCfgModel + '.proxyUsername', '"checkpointS3ProxyUsername"', 'false', 'Not specified', + 'Optional proxy user name to use if connecting through a proxy') +.settings-row + +text('Proxy domain:', clientCfgModel + '.proxyDomain', '"checkpointS3ProxyDomain"', 'false', 'Not specified', + 'Optional Windows domain name for configuring an NTLM proxy') +.settings-row + +text('Proxy workstation:', clientCfgModel + '.proxyWorkstation', '"checkpointS3ProxyWorkstation"', 'false', 'Not specified', + 'Optional Windows workstation name for configuring NTLM proxy support') +.settings-row + +dropdown('Retry policy:', clientRetryModel + '.kind', '"checkpointS3RetryPolicy"', 'true', 'Default', '[\ + {value: "Default", label: "Default SDK retry policy"},\ + {value: "DefaultMaxRetries", label: "Default with the specified max retry count"},\ + {value: "DynamoDB", label: "Default for DynamoDB client"},\ + {value: "DynamoDBMaxRetries", label: "DynamoDB with the specified max retry count"},\ + {value: "Custom", label: "Custom configured"}\ + ]', + 'Provides an ability to save an intermediate job state\ +
            \ +
          • SDK default retry policy - This policy will honor the maxErrorRetry set in ClientConfiguration
          • \ +
          • Default with the specified max retry count - Default SDK retry policy with the specified max retry count
          • \ +
          • Default for DynamoDB client - This policy will honor the maxErrorRetry set in ClientConfiguration
          • \ +
          • DynamoDB with the specified max retry count - This policy will honor the maxErrorRetry set in ClientConfiguration with the specified max retry count
          • \ +
          • Custom configured - Custom configured SDK retry policy
          • \ +
          ') +.panel-details(ng-show=checkpointS3DefaultMaxRetry) + .details-row + +number-required('Maximum retry attempts:', clientRetryModel + '.DefaultMaxRetries.maxErrorRetry', '"checkpointS3DefaultMaxErrorRetry"', 'true', checkpointS3DefaultMaxRetry, '-1', '1', + 'Maximum number of retry attempts for failed requests') +.panel-details(ng-show=checkpointS3DynamoDbMaxRetry) + .details-row + +number-required('Maximum retry attempts:', clientRetryModel + '.DynamoDBMaxRetries.maxErrorRetry', '"checkpointS3DynamoDBMaxErrorRetry"', 'true', checkpointS3DynamoDbMaxRetry, '-1', '1', + 'Maximum number of retry attempts for failed requests') +.panel-details(ng-show=checkpointS3CustomRetry) + .details-row + +java-class('Retry condition:', clientRetryModel + '.Custom.retryCondition', '"checkpointS3CustomRetryPolicy" + $index', 'true', checkpointS3CustomRetry, + 'Retry condition on whether a specific request and exception should be retried') + .details-row + +java-class('Backoff strategy:', clientRetryModel + '.Custom.backoffStrategy', '"checkpointS3CustomBackoffStrategy" + $index', 'true', checkpointS3CustomRetry, + 'Back-off strategy for controlling how long the next retry should wait') + .details-row + +number-required('Maximum retry attempts:', clientRetryModel + '.Custom.maxErrorRetry', '"checkpointS3CustomMaxErrorRetry"', 'true', checkpointS3CustomRetry, '-1', '1', + 'Maximum number of retry attempts for failed requests') + .details-row + +checkbox('Honor the max error retry set', clientRetryModel + '.Custom.honorMaxErrorRetryInClientConfig', '"checkpointS3CustomHonorMaxErrorRetryInClientConfig"', + 'Whether this retry policy should honor the max error retry set by ClientConfiguration#setMaxErrorRetry(int)') +.settings-row + +number('Maximum retry attempts:', clientCfgModel + '.maxErrorRetry', '"checkpointS3MaxErrorRetry"', 'true', '-1', '0', + 'Maximum number of retry attempts for failed retryable requests
          \ + If -1 the configured RetryPolicy will be used to control the retry count') +.settings-row + +number('Socket timeout:', clientCfgModel + '.socketTimeout', '"checkpointS3SocketTimeout"', 'true', '50000', '0', + 'Amount of time in milliseconds to wait for data to be transfered over an established, open connection before the connection times out and is closed
          \ + A value of 0 means infinity') +.settings-row + +number('Connection timeout:', clientCfgModel + '.connectionTimeout', '"checkpointS3ConnectionTimeout"', 'true', '50000', '0', + 'Amount of time in milliseconds to wait when initially establishing a connection before giving up and timing out
          \ + A value of 0 means infinity') +.settings-row + +number('Request timeout:', clientCfgModel + '.requestTimeout', '"checkpointS3RequestTimeout"', 'true', '0', '-1', + 'Amount of time in milliseconds to wait for the request to complete before giving up and timing out
          \ + A non - positive value means infinity') +.settings-row + +text('Signature algorithm:', clientCfgModel + '.signerOverride', '"checkpointS3SignerOverride"', 'false', 'Not specified', + 'Name of the signature algorithm to use for signing requests made by this client') +.settings-row + +number('Connection TTL:', clientCfgModel + '.connectionTTL', '"checkpointS3ConnectionTTL"', 'true', '-1', '-1', + 'Expiration time in milliseconds for a connection in the connection pool
          \ + By default, it is set to -1, i.e. connections do not expire') +.settings-row + +number('Idle timeout:', clientCfgModel + '.connectionMaxIdleMillis', '"checkpointS3ConnectionMaxIdleMillis"', 'true', '60000', '0', + 'Maximum amount of time that an idle connection may sit in the connection pool and still be eligible for reuse') +.settings-row + +java-class('DNS resolver:', clientCfgModel + '.dnsResolver', '"checkpointS3DnsResolver" + $index', 'true', 'false', + 'DNS Resolver that should be used to for resolving AWS IP addresses') +.settings-row + +number('Response metadata cache size:', clientCfgModel + '.responseMetadataCacheSize', '"checkpointS3ResponseMetadataCacheSize"', 'true', '50', '0', + 'Response metadata cache size') +.settings-row + +java-class('SecureRandom class name:', clientCfgModel + '.secureRandom', '"checkpointS3SecureRandom" + $index', 'true', 'false', + 'SecureRandom to be used by the SDK class name') +.settings-row + +checkbox('Use reaper', clientCfgModel + '.useReaper', '"checkpointS3UseReaper"', 'Checks if the IdleConnectionReaper is to be started') +.settings-row + +checkbox('Use GZIP', clientCfgModel + '.useGzip', '"checkpointS3UseGzip"', 'Checks if gzip compression is used') +.settings-row + +checkbox('Preemptively basic authentication', clientCfgModel + '.preemptiveBasicProxyAuth', '"checkpointS3PreemptiveBasicProxyAuth"', + 'Attempt to authenticate preemptively against proxy servers using basic authentication') +.settings-row + +checkbox('TCP KeepAlive', clientCfgModel + '.useTcpKeepAlive', '"checkpointS3UseTcpKeepAlive"', 'TCP KeepAlive support is enabled') +.settings-row + +java-class('Listener:', 'model.S3.checkpointListener', '"checkpointS3Listener" + $index', 'true', 'false', + 'Checkpoint listener implementation class name') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade index 491e4f19a379e..1a5d6d6eb0ee8 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade @@ -25,7 +25,8 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label Collision configuration ignite-form-field-tooltip.tipLabel - | Configuration Collision SPI allows to regulate how grid jobs get executed when they arrive on a destination node for execution + | Configuration Collision SPI allows to regulate how grid jobs get executed when they arrive on a destination node for execution#[br] + | #[a(href="https://apacheignite.readme.io/docs/job-scheduling" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') @@ -47,15 +48,15 @@ include ../../../../../app/helpers/jade/mixins.jade
        • Custom - custom CollisionSpi implementation
        • \
        • Default - jobs are activated immediately on arrival to mapped node
        • \ ') - .settings-row(ng-show='#{modelCollisionKind} !== "Noop"') + .settings-row(ng-if='#{modelCollisionKind} !== "Noop"') .panel-details - div(ng-show='#{modelCollisionKind} === "JobStealing"') + div(ng-if='#{modelCollisionKind} === "JobStealing"') include ./collision/job-stealing.jade - div(ng-show='#{modelCollisionKind} === "FifoQueue"') + div(ng-if='#{modelCollisionKind} === "FifoQueue"') include ./collision/fifo-queue.jade - div(ng-show='#{modelCollisionKind} === "PriorityQueue"') + div(ng-if='#{modelCollisionKind} === "PriorityQueue"') include ./collision/priority-queue.jade - div(ng-show='#{modelCollisionKind} === "Custom"') + div(ng-if='#{modelCollisionKind} === "Custom"') include ./collision/custom.jade .col-sm-6 -var model = 'backupItem.collision' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade index 3e6d42823798b..365911554d437 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade @@ -39,7 +39,7 @@ div +java-class('External listener:', model + '.externalCollisionListener', '"jsExternalCollisionListener"', 'true', 'false', 'Listener to be set for notification of external collision events') .details-row - +ignite-form-group(ng-model='#{stealingAttributes}' ng-form='#{form}') + +ignite-form-group ignite-form-field-label | Stealing attributes ignite-form-group-tooltip diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade index 2c60478f45ca1..55bf909e5b271 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade @@ -26,7 +26,8 @@ include ../../../../../app/helpers/jade/mixins.jade label Communication ignite-form-field-tooltip.tipLabel | Configuration of communication with other nodes by TCP/IP - | Provide basic plumbing to send and receive grid messages and is utilized for all distributed grid operations + | Provide basic plumbing to send and receive grid messages and is utilized for all distributed grid operations#[br] + | #[a(href="https://apacheignite.readme.io/docs/network-config" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade index baec54fe2ed00..6e7528331e0e1 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade @@ -26,7 +26,8 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label Connector configuration ignite-form-field-tooltip.tipLabel - | Configure HTTP REST configuration to enable HTTP server features + | Configure HTTP REST configuration to enable HTTP server features#[br] + | #[a(href="https://apacheignite.readme.io/docs/configuration" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade index 6cfa82d8984b7..67b6b4f368445 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade @@ -26,7 +26,8 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label Class deployment ignite-form-field-tooltip.tipLabel - | Task and resources deployment in cluster + | Task and resources deployment in cluster#[br] + | #[a(href="https://apacheignite.readme.io/docs/deployment-modes" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id='deployment') .panel-body(ng-if='ui.isPanelLoaded("#{form}")') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade index 1fdcbec44d998..c9a2be3168245 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade @@ -24,7 +24,8 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label Discovery ignite-form-field-tooltip.tipLabel - | TCP/IP discovery configuration + | TCP/IP discovery configuration#[br] + | #[a(href="https://apacheignite.readme.io/docs/cluster-config" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade index 412714c55657e..42c319c4aa5ee 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade @@ -18,20 +18,51 @@ include ../../../../../app/helpers/jade/mixins.jade -var form = 'events' -var model = 'backupItem' +-var modelEventStorage = model + '.eventStorage' +-var modelEventStorageKind = modelEventStorage + '.kind' +-var eventStorageMemory = modelEventStorageKind + ' === "Memory"' +-var eventStorageCustom = modelEventStorageKind + ' === "Custom"' .panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Events ignite-form-field-tooltip.tipLabel - | Grid events are used for notification about what happens within the grid + | Grid events are used for notification about what happens within the grid#[br] + | #[a(href="https://apacheignite.readme.io/docs/events" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') .col-sm-6 .settings-row +dropdown-multiple('Include type:', model + '.includeEventTypes', '"includeEventTypes"', true, 'Choose recorded event types', '', 'eventGroups', - 'Array of event types, which will be recorded by GridEventStorageManager#record(Event)
          \ - Note, that either the include event types or the exclude event types can be established') + 'Array of event types, which will be recorded by GridEventStorageManager#record(Event)
          \ + Note, that either the include event types or the exclude event types can be established') + + .settings-row + +dropdown('Event storage:', modelEventStorageKind, '"eventStorageKind"', 'true', '', + '[\ + {value: "Memory", label: "Memory"},\ + {value: "Custom", label: "Custom"}\ + ]', + 'Regulate how grid store events locally on node\ +
            \ +
          • Memory - All events are kept in the FIFO queue in-memory
          • \ +
          • Custom - Custom implementation of event storage SPI
          • \ +
          ') + + div(ng-show=eventStorageMemory) + .settings-row + +number('Events expiration time:', modelEventStorage + '.Memory.expireAgeMs', '"EventStorageExpireAgeMs"', 'true', 'Long.MAX_VALUE', '1', 'All events that exceed this value will be removed from the queue when next event comes') + .settings-row + +number('Events queue size:', modelEventStorage + '.Memory.expireCount', '"EventStorageExpireCount"', 'true', '10000', '1', 'Events will be filtered out when new request comes') + .settings-row + +java-class('Filter:', modelEventStorage + '.Memory.filter', '"EventStorageFilter"', 'true', 'false', + 'Filter for events to be recorded
          \ + Should be implementation of o.a.i.lang.IgnitePredicate<o.a.i.events.Event>') + + .settings-row(ng-show=eventStorageCustom) + +java-class('Class:', modelEventStorage + '.Custom.className', '"EventStorageCustom"', 'true', eventStorageCustom, 'Event storage implementation class name') + .col-sm-6 +preview-xml-java(model, 'clusterEvents') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade index 85f0f5439c9eb..b29701d38820c 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade @@ -19,14 +19,15 @@ include ../../../../../app/helpers/jade/mixins.jade -var model = 'backupItem' -var form = 'failoverSpi' -var failoverSpi = model + '.failoverSpi' --var failoverCustom = 'failover.kind === "Custom"' +-var failoverCustom = 'model.kind === "Custom"' .panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron label Failover configuration ignite-form-field-tooltip.tipLabel - | Failover SPI provides ability to supply custom logic for handling failed execution of a grid job + | Failover SPI provides ability to supply custom logic for handling failed execution of a grid job#[br] + | #[a(href="https://apacheignite.readme.io/docs/fault-tolerance" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') @@ -41,10 +42,10 @@ include ../../../../../app/helpers/jade/mixins.jade | Add failover SPI .group-content-empty(ng-if='!(#{failoverSpi} && #{failoverSpi}.length > 0)') | Not defined - .group-content(ng-show='#{failoverSpi} && #{failoverSpi}.length > 0' ng-repeat='failover in #{failoverSpi} track by $index') + .group-content(ng-show='#{failoverSpi} && #{failoverSpi}.length > 0' ng-repeat='model in #{failoverSpi} track by $index') hr(ng-if='$index != 0') .settings-row - +dropdown('Failover SPI:', 'failover.kind', '"failoverKind" + $index', 'true', 'Choose Failover SPI', '[\ + +dropdown-required('Failover SPI:', 'model.kind', '"failoverKind" + $index', 'true', 'true', 'Choose Failover SPI', '[\ {value: "JobStealing", label: "Job stealing"},\ {value: "Never", label: "Never"},\ {value: "Always", label: "Always"},\ @@ -59,14 +60,14 @@ include ../../../../../app/helpers/jade/mixins.jade ') +table-remove-button(failoverSpi, 'Remove Failover SPI') - .settings-row(ng-show='failover.kind === "JobStealing"') - +number('Maximum failover attempts:', 'failover.JobStealing.maximumFailoverAttempts', '"jsMaximumFailoverAttempts" + $index', 'true', '5', '0', + .settings-row(ng-show='model.kind === "JobStealing"') + +number('Maximum failover attempts:', 'model.JobStealing.maximumFailoverAttempts', '"jsMaximumFailoverAttempts" + $index', 'true', '5', '0', 'Maximum number of attempts to execute a failed job on another node') - .settings-row(ng-show='failover.kind === "Always"') - +number('Maximum failover attempts:', 'failover.Always.maximumFailoverAttempts', '"alwaysMaximumFailoverAttempts" + $index', 'true', '5', '0', + .settings-row(ng-show='model.kind === "Always"') + +number('Maximum failover attempts:', 'model.Always.maximumFailoverAttempts', '"alwaysMaximumFailoverAttempts" + $index', 'true', '5', '0', 'Maximum number of attempts to execute a failed job on another node') .settings-row(ng-show=failoverCustom) - +java-class('SPI implementation', 'failover.Custom.class', '"failoverSpiClass" + $index', 'true', failoverCustom, + +java-class('SPI implementation', 'model.Custom.class', '"failoverSpiClass" + $index', 'true', failoverCustom, 'Custom FailoverSpi implementation class name.') .col-sm-6 +preview-xml-java(model, 'clusterFailover') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade index d0d390f820495..8ec9b1383f677 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade @@ -24,6 +24,9 @@ include ../../../../../app/helpers/jade/mixins.jade .panel-heading(bs-collapse-toggle) ignite-form-panel-chevron label General + ignite-form-field-tooltip.tipLabel + | Common cluster configuration#[br] + | #[a(href="https://apacheignite.readme.io/docs/clustering" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade index afd3ecd87c3ad..da369937a553f 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade @@ -71,6 +71,8 @@ div div(ng-show='#{modelRetryPolicyKind} === "Custom"') include ./zookeeper/retrypolicy/custom.jade .details-row + -var model = 'backupItem.discovery.ZooKeeper' + +text('Base path:', model + '.basePath', '"basePath"', 'false', '/services', 'Base path for service registration') .details-row +text('Service name:', model + '.serviceName', '"serviceName"', 'false', 'ignite', diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade index 181b44fdfddac..7eb150558daa2 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade @@ -24,7 +24,8 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label IGFS ignite-form-field-tooltip.tipLabel - | IGFS (Ignite In-Memory File System) configurations assigned to cluster + | IGFS (Ignite In-Memory File System) configurations assigned to cluster#[br] + | #[a(href="https://apacheignite-fs.readme.io/docs/in-memory-file-system" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade new file mode 100644 index 0000000000000..916504bc91c95 --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade @@ -0,0 +1,104 @@ +//- + 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. + +include ../../../../../app/helpers/jade/mixins.jade + +-var model = 'backupItem' +-var form = 'loadBalancing' +-var loadBalancingSpi = model + '.loadBalancingSpi' +-var loadBalancingCustom = 'model.kind === "Custom"' +-var loadProbeCustom = 'model.kind === "Adaptive" && model.Adaptive.loadProbe.kind === "Custom"' + +.panel.panel-default(ng-form=form novalidate) + .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') + ignite-form-panel-chevron + label Load balancing configuration + ignite-form-field-tooltip.tipLabel + | Load balancing component balances job distribution among cluster nodes#[br] + | #[a(href="https://apacheignite.readme.io/docs/load-balancing" target="_blank") More info] + ignite-form-revert + .panel-collapse(role='tabpanel' bs-collapse-target id=form) + .panel-body(ng-if='ui.isPanelLoaded("#{form}")') + .col-sm-6 + .settings-row(ng-init='loadBalancingSpiTbl={type: "loadBalancingSpi", model: "loadBalancingSpi", focusId: "kind", ui: "load-balancing-table"}') + +ignite-form-group() + ignite-form-field-label + | Load balancing configurations + ignite-form-group-tooltip + | Load balancing component balances job distribution among cluster nodes + ignite-form-group-add(ng-click='tableNewItem(loadBalancingSpiTbl)') + | Add load balancing configuration + .group-content-empty(ng-if='!(#{loadBalancingSpi} && #{loadBalancingSpi}.length > 0)') + | Not defined + .group-content(ng-show='#{loadBalancingSpi} && #{loadBalancingSpi}.length > 0' ng-repeat='model in #{loadBalancingSpi} track by $index') + hr(ng-if='$index != 0') + .settings-row + +dropdown-required('Load balancing:', 'model.kind', '"loadBalancingKind" + $index', 'true', 'true', 'Choose load balancing SPI', '[\ + {value: "RoundRobin", label: "Round-robin"},\ + {value: "Adaptive", label: "Adaptive"},\ + {value: "WeightedRandom", label: "Random"},\ + {value: "Custom", label: "Custom"}\ + ]', 'Provides the next best balanced node for job execution\ +
            \ +
          • Round-robin - Iterates through nodes in round-robin fashion and pick the next sequential node
          • \ +
          • Adaptive - Adapts to overall node performance
          • \ +
          • Random - Picks a random node for job execution
          • \ +
          • Custom - Custom load balancing implementation
          • \ +
          ') + + +table-remove-button(loadBalancingSpi, 'Remove load balancing SPI') + .settings-row(ng-show='model.kind === "RoundRobin"') + +checkbox('Per task', 'model.RoundRobin.perTask', '"loadBalancingRRPerTask" + $index', 'A new round robin order should be created for every task flag') + .settings-row(ng-show='model.kind === "Adaptive"') + +dropdown('Load probe:', 'model.Adaptive.loadProbe.kind', '"loadBalancingAdaptiveLoadProbeKind" + $index', 'true', 'Default', '[\ + {value: "Job", label: "Job count"},\ + {value: "CPU", label: "CPU load"},\ + {value: "ProcessingTime", label: "Processing time"},\ + {value: "Custom", label: "Custom"},\ + {value: undefined, label: "Default"}\ + ]', 'Implementation of node load probing\ +
            \ +
          • Job count - Based on active and waiting job count
          • \ +
          • CPU load - Based on CPU load
          • \ +
          • Processing time - Based on total job processing time
          • \ +
          • Custom - Custom load probing implementation
          • \ +
          • Default - Default load probing implementation
          • \ +
          ') + .settings-row(ng-show='model.kind === "Adaptive" && model.Adaptive.loadProbe.kind') + .panel-details + .details-row(ng-show='model.Adaptive.loadProbe.kind === "Job"') + +checkbox('Use average', 'model.Adaptive.loadProbe.Job.useAverage', '"loadBalancingAdaptiveJobUseAverage" + $index', 'Use average CPU load vs. current') + .details-row(ng-show='model.Adaptive.loadProbe.kind === "CPU"') + +checkbox('Use average', 'model.Adaptive.loadProbe.CPU.useAverage', '"loadBalancingAdaptiveCPUUseAverage" + $index', 'Use average CPU load vs. current') + .details-row(ng-show='model.Adaptive.loadProbe.kind === "CPU"') + +checkbox('Use processors', 'model.Adaptive.loadProbe.CPU.useProcessors', '"loadBalancingAdaptiveCPUUseProcessors" + $index', "divide each node's CPU load by the number of processors on that node") + .details-row(ng-show='model.Adaptive.loadProbe.kind === "CPU"') + +number-min-max-step('Processor coefficient:', 'model.Adaptive.loadProbe.CPU.processorCoefficient', + '"loadBalancingAdaptiveCPUProcessorCoefficient" + $index', 'true', '1', '0.001', '1', '0.05', 'Coefficient of every CPU') + .details-row(ng-show='model.Adaptive.loadProbe.kind === "ProcessingTime"') + +checkbox('Use average', 'model.Adaptive.loadProbe.ProcessingTime.useAverage', '"loadBalancingAdaptiveJobUseAverage" + $index', 'Use average execution time vs. current') + .details-row(ng-show=loadProbeCustom) + +java-class('Load brobe implementation:', 'model.Adaptive.loadProbe.Custom.className', '"loadBalancingAdaptiveJobUseClass" + $index', 'true', loadProbeCustom, + 'Custom load balancing SPI implementation class name.') + .settings-row(ng-show='model.kind === "WeightedRandom"') + +number('Node weight:', 'model.WeightedRandom.nodeWeight', '"loadBalancingWRNodeWeight" + $index', 'true', 10, '1', 'Weight of node') + .settings-row(ng-show='model.kind === "WeightedRandom"') + +checkbox('Use weights', 'model.WeightedRandom.useWeights', '"loadBalancingWRUseWeights" + $index', 'Node weights should be checked when doing random load balancing') + .settings-row(ng-show=loadBalancingCustom) + +java-class('Load balancing SPI implementation:', 'model.Custom.className', '"loadBalancingClass" + $index', 'true', loadBalancingCustom, + 'Custom load balancing SPI implementation class name.') + .col-sm-6 + +preview-xml-java(model, 'clusterLoadBalancing') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade index 1fc3ce7c3f7d3..3f3661cd93b8c 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade @@ -28,7 +28,8 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-field-tooltip.tipLabel | Marshaller allows to marshal or unmarshal objects in grid#[br] | It provides serialization/deserialization mechanism for all instances that are sent across networks or are otherwise serialized - | By default BinaryMarshaller will be used + | By default BinaryMarshaller will be used#[br] + | #[a(href="https://apacheignite.readme.io/docs/binary-marshaller" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade index c267891c69ad3..dd46d2abed19d 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade @@ -25,7 +25,8 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label ODBC configuration ignite-form-field-tooltip.tipLabel - | ODBC server configuration + | ODBC server configuration#[br] + | #[a(href="https://apacheignite.readme.io/docs/odbc-driver" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade index 726438672136b..0a51cf7b57b1f 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade @@ -27,7 +27,8 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label(id='sslConfiguration-title') SSL configuration ignite-form-field-tooltip.tipLabel - | Settings for SSL configuration for creating a secure socket layer + | Settings for SSL configuration for creating a secure socket layer#[br] + | #[a(href="https://apacheignite.readme.io/docs/ssltls" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') @@ -58,7 +59,7 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-group-tooltip | Pre-configured trust managers ignite-form-group-add(ng-show='#{enabled}' ng-click='(group.add = [{}])') - | Add new trust manager. + | Add new trust manager .group-content(ng-if='#{trust}.length') -var model = 'obj.model'; @@ -75,7 +76,7 @@ include ../../../../../app/helpers/jade/mixins.jade span(ng-hide='field.edit') a.labelFormField(ng-click='#{enabled} && (field.edit = true) && (#{model} = model)') {{ model }} span(ng-if='field.edit') - +table-java-class-field('Trust manager:', name, model, trust, valid, save, false) + +table-java-class-field('Trust manager', name, model, trust, valid, save, false) +table-save-button(valid, save, false) +unique-feedback(name, uniqueTip) div(ng-hide=enabled) diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade index 1c75c4f33443c..c76391c6ac0e6 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade @@ -26,7 +26,8 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label Swap ignite-form-field-tooltip.tipLabel - | Settings for overflow data to disk if it cannot fit in memory + | Settings for overflow data to disk if it cannot fit in memory#[br] + | #[a(href="https://apacheignite.readme.io/docs/off-heap-memory#swap-space" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade index d9611a5f01509..294615895e9d9 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade @@ -24,7 +24,8 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label Transactions ignite-form-field-tooltip.tipLabel - | Settings for transactions + | Settings for transactions#[br] + | #[a(href="https://apacheignite.readme.io/docs/transactions" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') @@ -39,8 +40,7 @@ include ../../../../../app/helpers/jade/mixins.jade
            \
          • OPTIMISTIC - All cache operations are not distributed to other nodes until commit is called
          • \
          • PESSIMISTIC - A lock is acquired on all cache operations with exception of read operations in READ_COMMITTED mode
          • \ -
          \ - ') + ') .settings-row +dropdown('Isolation:', model + '.defaultTxIsolation', '"defaultTxIsolation"', 'true', 'REPEATABLE_READ', '[\ diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade b/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade index 74895f5a7c443..bf21ee11399b8 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade @@ -24,11 +24,15 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label General ignite-form-field-tooltip.tipLabel - | Domain model properties common for Query and Store + | Domain model properties common for Query and Store#[br] + | #[a(href="https://apacheignite.readme.io/docs/cache-queries" target="_blank") More info about query configuration]#[br] + | #[a(href="https://apacheignite.readme.io/docs/persistent-store" target="_blank") More info about store] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body .col-sm-6 + .settings-row + +checkbox('Generate POJO classes', model + '.generatePojo', '"generatePojo"', 'If selected then POJO classes will be generated from database tables') .settings-row +caches(model, 'Select caches to associate domain model with cache') .settings-row @@ -38,9 +42,11 @@ include ../../../../../app/helpers/jade/mixins.jade
        • Java annotations like @QuerySqlField
        • \
        • Configuration via QueryEntity class
        • \ ') + -var generatePojo = model + '.generatePojo' .settings-row - +java-class-typeahead('Key type:', model + '.keyType', '"keyType"', 'javaBuiltInClasses', 'true', 'true', 'Full class name for Key', 'Key class used to store key in cache') + +java-class-typeahead('Key type:', model + '.keyType', '"keyType"', 'javaBuiltInClasses', 'true', 'true', '{{ ' + generatePojo + ' ? "Full class name for Key" : "Key type name" }}', 'Key class used to store key in cache', generatePojo) .settings-row - +java-class('Value type:', model + '.valueType', '"valueType"', 'true', 'true', 'Value class used to store value in cache') + +java-class-autofocus-placholder('Value type:', model + '.valueType', '"valueType"', 'true', 'true', 'false', '{{ ' + generatePojo +' ? "Enter fully qualified class name" : "Value type name" }}', 'Value class used to store value in cache', generatePojo) + .col-sm-6 +preview-xml-java(model, 'domainModelGeneral') diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade b/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade index 0cf21f5130beb..45051b23e8768 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade @@ -37,25 +37,27 @@ mixin table-index-item-edit(prefix, index, sortAvailable, idAddition) -var btnSave = 'tableIndexItemSave(indexesTbl, itemIndex, ' + index + ')' -var btnVisibleAndSave = btnVisible + ' && ' + btnSave - .col-xs-8.col-sm-8.col-md-8(ng-show=sortAvailable) - label.fieldSep / - .input-tip - input.form-control(id='{{::"#{fieldName}S" + #{idAddition}}}' ignite-on-enter-focus-move='{{::"#{direction}S" + #{idAddition}}}' type='text' ng-model=fieldNameModel placeholder='Field name' ignite-on-escape='tableReset()') - .col-xs-4.col-sm-4.col-md-4(ng-show=sortAvailable) - +btn-save(btnVisible, btnSave) - .input-tip - button.select-toggle.form-control(id='{{::"#{direction}S" + #{idAddition}}}' ng-model=directionModel bs-select bs-options='item.value as item.label for item in {{sortDirections}}' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()' tabindex='0') - .col-xs-12(ng-show='!(#{sortAvailable})') + div(ng-if=sortAvailable) + .col-xs-8.col-sm-8.col-md-8 + label.fieldSep / + .input-tip + button.select-toggle.form-control(id='{{::"#{fieldName}" + #{idAddition}}}' ignite-on-enter-focus-move='{{::"#{direction}S" + #{idAddition}}}' ng-model=fieldNameModel placeholder='{{fields("#{prefix}", #{fieldNameModel}).length > 0 ? "Choose field" : "No fields configured"}}' bs-select bs-options='item.value as item.label for item in fields("#{prefix}", #{fieldNameModel})' ng-disabled='fields("#{prefix}", #{fieldNameModel}).length === 0' ignite-on-escape='tableReset()' tabindex='0') + .col-xs-4.col-sm-4.col-md-4 + +btn-save(btnVisible, btnSave) + .input-tip + button.select-toggle.form-control(id='{{::"#{direction}" + #{idAddition}}}' ng-model=directionModel bs-select bs-options='item.value as item.label for item in {{sortDirections}}' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()' tabindex='0') + .col-xs-12(ng-if='!(#{sortAvailable})') +btn-save(btnVisible, btnSave) .input-tip - input.form-control(id='{{::"#{fieldName}" + #{idAddition}}}' type='text' ng-model=fieldNameModel placeholder='Field name' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()') + button.select-toggle.form-control(id='{{::"#{fieldName}" + #{idAddition}}}' ng-model=fieldNameModel placeholder='{{fields("#{prefix}", #{fieldNameModel}).length > 0 ? "Choose index field" : "No fields configured"}}' bs-select bs-options='item.value as item.label for item in fields("#{prefix}", #{fieldNameModel})' ng-disabled='fields("#{prefix}", #{fieldNameModel}).length === 0' ignite-on-escape='tableReset()' tabindex='0') .panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle) ignite-form-panel-chevron label(id='query-title') Domain model for SQL query ignite-form-field-tooltip.tipLabel - | Domain model properties for fields queries + | Domain model properties for fields queries#[br] + | #[a(href="https://apacheignite.readme.io/docs/cache-queries" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id='query') .panel-body @@ -76,8 +78,8 @@ mixin table-index-item-edit(prefix, index, sortAvailable, idAddition) .group-content(ng-show='(#{queryFields} && #{queryFields}.length > 0) || tableNewItemActive(queryFieldsTbl)') table.links-edit(id='fields' st-table=queryFields) tbody - tr(ng-repeat='item in #{queryFields}') - td.col-sm-12(ng-show='!tableEditing(queryFieldsTbl, $index)') + tr(ng-repeat='item in #{queryFields} track by $index') + td.col-sm-12(ng-hide='tableEditing(queryFieldsTbl, $index)') a.labelFormField(ng-click='tableStartEdit(backupItem, queryFieldsTbl, $index)') {{item.name}} / {{item.className}} +btn-remove('tableRemove(backupItem, queryFieldsTbl, $index)', '"Remove path"') td.col-sm-12(ng-show='tableEditing(queryFieldsTbl, $index)') @@ -100,8 +102,8 @@ mixin table-index-item-edit(prefix, index, sortAvailable, idAddition) .group-content(ng-show='(#{queryAliases} && #{queryAliases}.length > 0) || tableNewItemActive(aliasesTbl)') table.links-edit(id='aliases' st-table=queryAliases) tbody - tr(ng-repeat='item in #{queryAliases}') - td.col-sm-12(ng-show='!tableEditing(aliasesTbl, $index)') + tr(ng-repeat='item in #{queryAliases} track by $index') + td.col-sm-12(ng-hide='tableEditing(aliasesTbl, $index)') a.labelFormField(ng-click='tableStartEdit(backupItem, aliasesTbl, $index)') {{item.field}} → {{item.alias}} +btn-remove('tableRemove(backupItem, aliasesTbl, $index)', '"Remove alias"') td.col-sm-12(ng-show='tableEditing(aliasesTbl, $index)') @@ -125,9 +127,9 @@ mixin table-index-item-edit(prefix, index, sortAvailable, idAddition) table.links-edit(st-table=queryIndexes ng-init='newDirection = false') tbody - tr(ng-repeat='item in #{queryIndexes}') + tr(ng-repeat='item in #{queryIndexes} track by $index') td - .col-sm-12(ng-show='!tableEditing(indexesTbl, $index)') + .col-sm-12(ng-hide='tableEditing(indexesTbl, $index)') a.labelFormField(id='indexes{{$index}}' ng-click='tableStartEdit(backupItem, indexesTbl, $index)') {{$index + 1}}) {{item.name}} [{{item.indexType}}] +btn-remove('tableRemove(backupItem, indexesTbl, $index)', '"Remove index"') +btn-add('tableIndexNewItem(indexesTbl, $index)', '"Add new field to index"') @@ -143,9 +145,9 @@ mixin table-index-item-edit(prefix, index, sortAvailable, idAddition) .margin-left-dflt table.links-edit-sub(st-table='item.fields' ng-init='itemIndex = $index') tbody - tr(ng-repeat='itemItem in item.fields') + tr(ng-repeat='itemItem in item.fields track by $index') td - div(ng-show='!tableIndexItemEditing(indexesTbl, itemIndex, $index)') + div(ng-hide='tableIndexItemEditing(indexesTbl, itemIndex, $index)') a.labelFormField(ng-if='item.indexType == "SORTED"' ng-click='tableIndexItemStartEdit(indexesTbl, itemIndex, $index)') {{$index + 1}}) {{itemItem.name}} / {{itemItem.direction ? "ASC" : "DESC"}} a.labelFormField(ng-if='item.indexType != "SORTED"' ng-click='tableIndexItemStartEdit(indexesTbl, itemIndex, $index)') {{$index + 1}}) {{itemItem.name}} +btn-remove('tableRemoveIndexItem(item, $index)', '"Remove field from index"') diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade b/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade index 96913bbab0bed..ab569e706a5e1 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade @@ -66,7 +66,8 @@ mixin table-db-field-edit(tbl, prefix, focusId, index) ignite-form-panel-chevron label Domain model for cache store ignite-form-field-tooltip.tipLabel - | Domain model properties for binding database with cache via POJO cache store + | Domain model properties for binding database with cache via POJO cache store#[br] + | #[a(href="https://apacheignite.readme.io/docs/persistent-store" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') @@ -87,12 +88,12 @@ mixin table-db-field-edit(tbl, prefix, focusId, index) .group-content(ng-show='(#{keyFields} && #{keyFields}.length > 0) || tableNewItemActive(keysTbl)') table.links-edit(st-table=keyFields) tbody - tr(ng-repeat='item in #{keyFields}') + tr(ng-repeat='item in #{keyFields} track by $index') td - div(ng-show='!tableEditing(keysTbl, $index)') + div(ng-hide='tableEditing(keysTbl, $index)') a.labelFormField(ng-click='tableStartEdit(backupItem, keysTbl, $index)') {{$index + 1}}) {{item.databaseFieldName}} / {{item.databaseFieldType}} / {{item.javaFieldName}} / {{item.javaFieldType}} +btn-remove('tableRemove(backupItem, keysTbl, $index)', '"Remove key field"') - div(ng-show='tableEditing(keysTbl, $index)') + div(ng-if='tableEditing(keysTbl, $index)') +table-db-field-edit('keysTbl', 'cur', '{{::keysTbl.focusId + $index}}', '$index') tfoot(ng-show='tableNewItemActive(keysTbl)') tr @@ -110,12 +111,12 @@ mixin table-db-field-edit(tbl, prefix, focusId, index) .group-content(ng-show='(#{valueFields} && #{valueFields}.length > 0) || tableNewItemActive(valuesTbl)') table.links-edit(st-table=valueFields) tbody - tr(ng-repeat='item in #{valueFields}') + tr(ng-repeat='item in #{valueFields} track by $index') td - div(ng-show='!tableEditing(valuesTbl, $index)') + div(ng-hide='tableEditing(valuesTbl, $index)') a.labelFormField(ng-click='tableStartEdit(backupItem, valuesTbl, $index)') {{$index + 1}}) {{item.databaseFieldName}} / {{item.databaseFieldType}} / {{item.javaFieldName}} / {{item.javaFieldType}} +btn-remove('tableRemove(backupItem, valuesTbl, $index)', '"Remove key field"') - div(ng-show='tableEditing(valuesTbl, $index)') + div(ng-if='tableEditing(valuesTbl, $index)') +table-db-field-edit('valuesTbl', 'cur', '{{::valuesTbl.focusId + $index}}', '$index') tfoot(ng-show='tableNewItemActive(valuesTbl)') tr diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade index b0871208e18ef..d12a6e8e17fdd 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade @@ -23,6 +23,9 @@ include ../../../../../app/helpers/jade/mixins.jade .panel-heading(bs-collapse-toggle) ignite-form-panel-chevron label General + ignite-form-field-tooltip.tipLabel + | General IGFS configuration#[br] + | #[a(href="https://apacheignite-fs.readme.io/docs/in-memory-file-system" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id='general') .panel-body diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade index bb5e00b54b724..f8da2f9d05c8e 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade @@ -54,7 +54,7 @@ include ../../../../../app/helpers/jade/mixins.jade .settings-row +text-enabled('Token directory:', ipcEndpointConfiguration + '.tokenDirectoryPath', '"ipcEndpointConfigurationTokenDirectoryPath"', enabled, 'false', 'ipc/shmem', 'Directory where shared memory tokens are stored') .settings-row - +number('Thread count:', ipcEndpointConfiguration + '.threadCount', 'ipcEndpointConfigurationThreadCount', enabled, 'availableProcessors', '1', + +number('Thread count:', ipcEndpointConfiguration + '.threadCount', '"ipcEndpointConfigurationThreadCount"', enabled, 'availableProcessors', '1', 'Number of threads used by this endpoint to process incoming requests') .col-sm-6 +preview-xml-java(model, 'igfsIPC') diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade index cb4687addf5a7..ca22dc5a20fe8 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade @@ -93,8 +93,8 @@ mixin table-igfs-path-mode-edit(prefix, focusId, index) .group-content(ng-show='(#{pathModes} && #{pathModes}.length > 0) || tableNewItemActive(tblPathModes)') table.links-edit(id='pathModes' st-table=pathModes) tbody - tr(ng-repeat='item in #{pathModes}') - td.col-sm-12(ng-show='!tableEditing(tblPathModes, $index)') + tr(ng-repeat='item in #{pathModes} track by $index') + td.col-sm-12(ng-hide='tableEditing(tblPathModes, $index)') a.labelFormField(ng-click='tableStartEdit(backupItem, tblPathModes, $index)') {{item.path + " [" + item.mode + "]"}} +btn-remove('tableRemove(backupItem, tblPathModes, $index)', '"Remove path"') td.col-sm-12(ng-show='tableEditing(tblPathModes, $index)') diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade index 0649527222759..114366217659d 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade @@ -24,7 +24,8 @@ include ../../../../../app/helpers/jade/mixins.jade ignite-form-panel-chevron label(id="secondaryFileSystem-title") Secondary file system ignite-form-field-tooltip.tipLabel - | Secondary file system is provided for pass-through, write-through, and read-through purposes + | Secondary file system is provided for pass-through, write-through, and read-through purposes#[br] + | #[a(href="https://apacheignite-fs.readme.io/docs/secondary-file-system" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js index 4aad7e24f380f..4bfbf486b3763 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js +++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js @@ -20,8 +20,8 @@ import JSZip from 'jszip'; import saver from 'file-saver'; export default [ - '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteLoading', '$filter', 'igniteConfigurationResource', 'JavaTypes', 'IgniteVersion', 'GeneratorDocker', 'GeneratorPom', 'IgniteFormUtils', - function($root, $scope, $http, LegacyUtils, Messages, Loading, $filter, Resource, JavaTypes, Version, docker, pom, FormUtils) { + '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteLoading', '$filter', 'IgniteConfigurationResource', 'JavaTypes', 'IgniteVersion', 'IgniteConfigurationGenerator', 'SpringTransformer', 'JavaTransformer', 'GeneratorDocker', 'GeneratorPom', 'IgnitePropertiesGenerator', 'IgniteReadmeGenerator', 'IgniteFormUtils', + function($root, $scope, $http, LegacyUtils, Messages, Loading, $filter, Resource, JavaTypes, Version, generator, spring, java, docker, pom, propsGenerator, readme, FormUtils) { const ctrl = this; $scope.ui = { ready: false }; @@ -108,11 +108,18 @@ export default [ ] }; + const clnCfg = { type: 'file', name: 'client.xml' }; + const srvCfg = { type: 'file', name: 'server.xml' }; + const resourcesFolder = { type: 'folder', name: 'resources', children: [ - { type: 'file', name: 'secret.properties' } + { + type: 'folder', + name: 'META-INF', + children: [clnCfg, srvCfg] + } ] }; @@ -131,10 +138,6 @@ export default [ ] }; - const clnCfg = { type: 'file', name: 'client.xml' }; - - const srvCfg = { type: 'file', name: 'server.xml' }; - const mainFolder = { type: 'folder', name: 'main', @@ -145,11 +148,6 @@ export default [ type: 'folder', name: 'project.zip', children: [ - { - type: 'folder', - name: 'config', - children: [clnCfg, srvCfg] - }, { type: 'folder', name: 'jdbc-drivers', @@ -215,6 +213,16 @@ export default [ folder.children.push(leaf); } + function cacheHasDatasource(cache) { + if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) { + const storeFactory = cache.cacheStoreFactory[cache.cacheStoreFactory.kind]; + + return !!(storeFactory && (storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : false) : storeFactory.dialect)); // eslint-disable-line no-nested-ternary + } + + return false; + } + $scope.selectItem = (cluster) => { delete ctrl.cluster; @@ -231,17 +239,17 @@ export default [ sessionStorage.summarySelectedId = $scope.clusters.indexOf(cluster); - mainFolder.children = [javaFolder]; + mainFolder.children = [javaFolder, resourcesFolder]; if (_.find(cluster.caches, (cache) => !_.isNil(cache.cacheStoreFactory))) javaFolder.children = [javaConfigFolder, loadFolder, javaStartupFolder]; else javaFolder.children = [javaConfigFolder, javaStartupFolder]; - if ($generatorCommon.secretPropertiesNeeded(cluster)) - mainFolder.children.push(resourcesFolder); + if (_.nonNil(_.find(cluster.caches, cacheHasDatasource)) || cluster.sslEnabled) + resourcesFolder.children.push({ type: 'file', name: 'secret.properties' }); - if ($generatorJava.isDemoConfigured(cluster, $root.IgniteDemoMode)) + if (java.isDemoConfigured(cluster, $root.IgniteDemoMode)) javaFolder.children.push(demoFolder); if (cluster.discovery.kind === 'Jdbc' && cluster.discovery.Jdbc.dialect) @@ -286,7 +294,6 @@ export default [ // TODO IGNITE-2114: implemented as independent logic for download. $scope.downloadConfiguration = function() { const cluster = $scope.cluster; - const clientNearCfg = cluster.clientNearCfg; const zip = new JSZip(); @@ -299,54 +306,65 @@ export default [ zip.file('Dockerfile', ctrl.data.docker); zip.file('.dockerignore', docker.ignoreFile()); - const builder = $generatorProperties.generateProperties(cluster); + const cfg = generator.igniteConfiguration(cluster, false); + const clientCfg = generator.igniteConfiguration(cluster, true); + const clientNearCaches = _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')); + + const secProps = propsGenerator.generate(cfg); - if (builder) - zip.file('src/main/resources/secret.properties', builder.asString()); + if (secProps) + zip.file('src/main/resources/secret.properties', secProps); - const srcPath = 'src/main/java/'; + const srcPath = 'src/main/java'; + const resourcesPath = 'src/main/resources'; - const serverXml = 'config/' + cluster.name + '-server.xml'; - const clientXml = 'config/' + cluster.name + '-client.xml'; + const serverXml = `${cluster.name}-server.xml`; + const clientXml = `${cluster.name}-client.xml`; - zip.file(serverXml, $generatorXml.cluster(cluster)); - zip.file(clientXml, $generatorXml.cluster(cluster, clientNearCfg)); + const metaPath = `${resourcesPath}/META-INF`; - zip.file(srcPath + 'config/ServerConfigurationFactory.java', $generatorJava.cluster(cluster, 'config', 'ServerConfigurationFactory', null)); - zip.file(srcPath + 'config/ClientConfigurationFactory.java', $generatorJava.cluster(cluster, 'config', 'ClientConfigurationFactory', clientNearCfg)); + zip.file(`${metaPath}/${serverXml}`, spring.igniteConfiguration(cfg).asString()); + zip.file(`${metaPath}/${clientXml}`, spring.igniteConfiguration(clientCfg, clientNearCaches).asString()); - if ($generatorJava.isDemoConfigured(cluster, $root.IgniteDemoMode)) { - zip.file(srcPath + 'demo/DemoStartup.java', $generatorJava.nodeStartup(cluster, 'demo', 'DemoStartup', + const cfgPath = `${srcPath}/config`; + + zip.file(`${cfgPath}/ServerConfigurationFactory.java`, java.igniteConfiguration(cfg, 'config', 'ServerConfigurationFactory').asString()); + zip.file(`${cfgPath}/ClientConfigurationFactory.java`, java.igniteConfiguration(cfg, 'config', 'ClientConfigurationFactory', clientNearCaches).asString()); + + if (java.isDemoConfigured(cluster, $root.IgniteDemoMode)) { + zip.file(`${srcPath}/demo/DemoStartup.java`, java.nodeStartup(cluster, 'demo.DemoStartup', 'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory')); } // Generate loader for caches with configured store. - const cachesToLoad = _.filter(cluster.caches, (cache) => !_.isNil(cache.cacheStoreFactory)); + const cachesToLoad = _.filter(cluster.caches, (cache) => _.nonNil(cache.cacheStoreFactory)); + + if (_.nonEmpty(cachesToLoad)) + zip.file(`${srcPath}/load/LoadCaches.java`, java.loadCaches(cachesToLoad, 'load', 'LoadCaches', `"${clientXml}"`)); - if (!_.isEmpty(cachesToLoad)) - zip.file(srcPath + 'load/LoadCaches.java', $generatorJava.loadCaches(cachesToLoad, 'load', 'LoadCaches', '"' + clientXml + '"')); + const startupPath = `${srcPath}/startup`; - zip.file(srcPath + 'startup/ServerNodeSpringStartup.java', $generatorJava.nodeStartup(cluster, 'startup', 'ServerNodeSpringStartup', '"' + serverXml + '"')); - zip.file(srcPath + 'startup/ClientNodeSpringStartup.java', $generatorJava.nodeStartup(cluster, 'startup', 'ClientNodeSpringStartup', '"' + clientXml + '"')); + zip.file(`${startupPath}/ServerNodeSpringStartup.java`, java.nodeStartup(cluster, 'startup.ServerNodeSpringStartup', `"${serverXml}"`)); + zip.file(`${startupPath}/ClientNodeSpringStartup.java`, java.nodeStartup(cluster, 'startup.ClientNodeSpringStartup', `"${clientXml}"`)); - zip.file(srcPath + 'startup/ServerNodeCodeStartup.java', $generatorJava.nodeStartup(cluster, 'startup', 'ServerNodeCodeStartup', + zip.file(`${startupPath}/ServerNodeCodeStartup.java`, java.nodeStartup(cluster, 'startup.ServerNodeCodeStartup', 'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory')); - zip.file(srcPath + 'startup/ClientNodeCodeStartup.java', $generatorJava.nodeStartup(cluster, 'startup', 'ClientNodeCodeStartup', - 'ClientConfigurationFactory.createConfiguration()', 'config.ClientConfigurationFactory', clientNearCfg)); + zip.file(`${startupPath}/ClientNodeCodeStartup.java`, java.nodeStartup(cluster, 'startup.ClientNodeCodeStartup', + 'ClientConfigurationFactory.createConfiguration()', 'config.ClientConfigurationFactory', clientNearCaches)); - zip.file('pom.xml', pom.generate(cluster, Version.ignite).asString()); + zip.file('pom.xml', pom.generate(cluster, Version.productVersion().ignite).asString()); - zip.file('README.txt', $generatorReadme.readme().asString()); - zip.file('jdbc-drivers/README.txt', $generatorReadme.readmeJdbc().asString()); + zip.file('README.txt', readme.generate()); + zip.file('jdbc-drivers/README.txt', readme.generateJDBC()); - if (!ctrl.data.pojos) - ctrl.data.pojos = $generatorJava.pojos(cluster.caches); + if (_.isEmpty(ctrl.data.pojos)) + ctrl.data.pojos = java.pojos(cluster.caches); for (const pojo of ctrl.data.pojos) { if (pojo.keyClass && JavaTypes.nonBuiltInClass(pojo.keyType)) - zip.file(srcPath + pojo.keyType.replace(/\./g, '/') + '.java', pojo.keyClass); + zip.file(`${srcPath}/${pojo.keyType.replace(/\./g, '/')}.java`, pojo.keyClass); - zip.file(srcPath + pojo.valueType.replace(/\./g, '/') + '.java', pojo.valueClass); + zip.file(`${srcPath}/${pojo.valueType.replace(/\./g, '/')}.java`, pojo.valueClass); } $generatorOptional.optionalContent(zip, cluster); diff --git a/modules/web-console/frontend/app/services/ErrorPopover.service.js b/modules/web-console/frontend/app/services/ErrorPopover.service.js index 313043137a055..5132d5092e5eb 100644 --- a/modules/web-console/frontend/app/services/ErrorPopover.service.js +++ b/modules/web-console/frontend/app/services/ErrorPopover.service.js @@ -108,7 +108,7 @@ export default class ErrorPopover { if (this._popover) this._popover.hide(); - if (ui) { + if (ui && ui.isPanelLoaded) { this.FormUtils.ensureActivePanel(ui, panelId, id); this.$timeout(() => this._show(id, message, showTime), ui.isPanelLoaded(panelId) ? 200 : 500); diff --git a/modules/web-console/frontend/app/services/FormUtils.service.js b/modules/web-console/frontend/app/services/FormUtils.service.js index 5e7943a015faf..6ccc3c68446ec 100644 --- a/modules/web-console/frontend/app/services/FormUtils.service.js +++ b/modules/web-console/frontend/app/services/FormUtils.service.js @@ -17,7 +17,7 @@ export default ['IgniteFormUtils', ['$window', 'IgniteFocus', ($window, Focus) => { function ensureActivePanel(ui, pnl, focusId) { - if (ui) { + if (ui && ui.loadPanel) { const collapses = $('div.panel-collapse'); ui.loadPanel(pnl); @@ -430,6 +430,10 @@ export default ['IgniteFormUtils', ['$window', 'IgniteFocus', ($window, Focus) = return _.includes(this.loadedPanels, pnl); } }; + }, + markPristineInvalidAsDirty(ngModelCtrl) { + if (ngModelCtrl && ngModelCtrl.$invalid && ngModelCtrl.$pristine) + ngModelCtrl.$setDirty(); } }; }]]; diff --git a/modules/web-console/frontend/app/services/JavaTypes.service.js b/modules/web-console/frontend/app/services/JavaTypes.service.js index 8cb87be6b9f9f..679914f0c5b49 100644 --- a/modules/web-console/frontend/app/services/JavaTypes.service.js +++ b/modules/web-console/frontend/app/services/JavaTypes.service.js @@ -15,8 +15,6 @@ * limitations under the License. */ -import _ from 'lodash'; - // Java built-in class names. import JAVA_CLASSES from '../data/java-classes.json'; @@ -42,6 +40,42 @@ const VALID_UUID = /^[0-9a-f]{8}-[0-9a-f]{4}-[1-5][0-9a-f]{3}-[89ab][0-9a-f]{3}- * Utility service for various check on java types. */ export default class JavaTypes { + static $inject = ['igniteClusterDefaults', 'igniteCacheDefaults', 'igniteIgfsDefaults']; + + constructor(clusterDflts, cacheDflts, igfsDflts) { + this.enumClasses = _.uniq(this._enumClassesAcc(_.merge(clusterDflts, cacheDflts, igfsDflts), [])); + this.shortEnumClasses = _.map(this.enumClasses, (cls) => this.shortClassName(cls)); + } + + /** + * Collects recursive enum classes. + * + * @param root Root object. + * @param classes Collected classes. + * @return {Array.} + * @private + */ + _enumClassesAcc(root, classes) { + return _.reduce(root, (acc, val, key) => { + if (key === 'clsName') + acc.push(val); + else if (_.isObject(val)) + this._enumClassesAcc(val, acc); + + return acc; + }, classes); + } + + /** + * Check if class name is non enum class in Ignite configuration. + * + * @param clsName + * @return {boolean} + */ + nonEnum(clsName) { + return !_.includes(this.shortEnumClasses, clsName) && !_.includes(this.enumClasses, clsName); + } + /** * @param clsName {String} Class name to check. * @returns {boolean} 'true' if provided class name is a not Java built in class. @@ -52,7 +86,7 @@ export default class JavaTypes { /** * @param clsName Class name to check. - * @returns Full class name for java build-in types or source class otherwise. + * @returns {String} Full class name for java build-in types or source class otherwise. */ fullClassName(clsName) { const type = _.find(JAVA_CLASSES, (clazz) => clsName === clazz.short); @@ -60,6 +94,23 @@ export default class JavaTypes { return type ? type.full : clsName; } + /** + * Extract class name from full class name. + * + * @param clsName full class name. + * @return {String} Class name. + */ + shortClassName(clsName) { + if (this.isJavaPrimitive(clsName)) + return clsName; + + const fullClsName = this.fullClassName(clsName); + + const dotIdx = fullClsName.lastIndexOf('.'); + + return dotIdx > 0 ? fullClsName.substr(dotIdx + 1) : fullClsName; + } + /** * @param value {String} Value text to check. * @returns {boolean} 'true' if given text is valid Java class name. @@ -115,4 +166,17 @@ export default class JavaTypes { isJavaPrimitive(clsName) { return _.includes(JAVA_PRIMITIVES, clsName); } + + /** + * Convert some name to valid java name. + * + * @param prefix To append to java name. + * @param name to convert. + * @returns {string} Valid java name. + */ + toJavaName(prefix, name) { + const javaName = name ? this.shortClassName(name).replace(/[^A-Za-z_0-9]+/g, '_') : 'dflt'; + + return prefix + javaName.charAt(0).toLocaleUpperCase() + javaName.slice(1); + } } diff --git a/modules/web-console/frontend/app/services/LegacyTable.service.js b/modules/web-console/frontend/app/services/LegacyTable.service.js index 5d9ec9d436215..a024a3b7a858f 100644 --- a/modules/web-console/frontend/app/services/LegacyTable.service.js +++ b/modules/web-console/frontend/app/services/LegacyTable.service.js @@ -19,7 +19,27 @@ export default ['IgniteLegacyTable', ['IgniteLegacyUtils', 'IgniteFocus', 'IgniteErrorPopover', (LegacyUtils, Focus, ErrorPopover) => { function _model(item, field) { - return LegacyUtils.getModel(item, field); + let path = field.path; + + if (_.isNil(path) || _.isNil(item)) + return item; + + path = path.replace(/\[(\w+)\]/g, '.$1'); // convert indexes to properties + path = path.replace(/^\./, ''); // strip a leading dot + + const segs = path.split('.'); + let root = item; + + while (segs.length > 0) { + const pathStep = segs.shift(); + + if (typeof root[pathStep] === 'undefined') + root[pathStep] = {}; + + root = root[pathStep]; + } + + return root; } const table = {name: 'none', editIndex: -1}; @@ -190,7 +210,7 @@ export default ['IgniteLegacyTable', } } - return valid; + return valid || stopEdit; }, tablePairSaveVisible(field, index) { const pairValue = _tablePairValue(field, index); diff --git a/modules/web-console/frontend/app/services/LegacyUtils.service.js b/modules/web-console/frontend/app/services/LegacyUtils.service.js index dcf0bc869e9c8..e7c064bcc4505 100644 --- a/modules/web-console/frontend/app/services/LegacyUtils.service.js +++ b/modules/web-console/frontend/app/services/LegacyUtils.service.js @@ -182,43 +182,19 @@ export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { const VALID_JAVA_IDENTIFIER = new RegExp('^[a-zA-Z_$][a-zA-Z\\d_$]*$'); - function isValidJavaIdentifier(msg, ident, elemId, panels, panelId) { + function isValidJavaIdentifier(msg, ident, elemId, panels, panelId, stopEdit) { if (isEmptyString(ident)) - return ErrorPopover.show(elemId, msg + ' is invalid!', panels, panelId); + return !stopEdit && ErrorPopover.show(elemId, msg + ' is invalid!', panels, panelId); if (_.includes(JAVA_KEYWORDS, ident)) - return ErrorPopover.show(elemId, msg + ' could not contains reserved java keyword: "' + ident + '"!', panels, panelId); + return !stopEdit && ErrorPopover.show(elemId, msg + ' could not contains reserved java keyword: "' + ident + '"!', panels, panelId); if (!VALID_JAVA_IDENTIFIER.test(ident)) - return ErrorPopover.show(elemId, msg + ' contains invalid identifier: "' + ident + '"!', panels, panelId); + return !stopEdit && ErrorPopover.show(elemId, msg + ' contains invalid identifier: "' + ident + '"!', panels, panelId); return true; } - function getModel(obj, field) { - let path = field.path; - - if (!isDefined(path) || !isDefined(obj)) - return obj; - - path = path.replace(/\[(\w+)\]/g, '.$1'); // convert indexes to properties - path = path.replace(/^\./, ''); // strip a leading dot - - const segs = path.split('.'); - let root = obj; - - while (segs.length > 0) { - const pathStep = segs.shift(); - - if (typeof root[pathStep] === 'undefined') - root[pathStep] = {}; - - root = root[pathStep]; - } - - return root; - } - /** * Extract datasource from cache or cluster. * @@ -226,18 +202,26 @@ export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { * @returns {*} Datasource object or null if not set. */ function extractDataSource(object) { + let datasource = null; + // Extract from cluster object if (_.get(object, 'discovery.kind') === 'Jdbc') { - const datasource = object.discovery.Jdbc; + datasource = object.discovery.Jdbc; + + if (datasource.dataSourceBean && datasource.dialect) + return datasource; + } // Extract from JDBC checkpoint configuration. + else if (_.get(object, 'kind') === 'JDBC') { + datasource = object.JDBC; if (datasource.dataSourceBean && datasource.dialect) return datasource; } // Extract from cache object else if (_.get(object, 'cacheStoreFactory.kind')) { - const storeFactory = object.cacheStoreFactory[object.cacheStoreFactory.kind]; + datasource = object.cacheStoreFactory[object.cacheStoreFactory.kind]; - if (storeFactory.dialect || (storeFactory.connectVia === 'DataSource')) - return storeFactory; + if (datasource.dialect || (datasource.connectVia === 'DataSource')) + return datasource; } return null; @@ -268,10 +252,13 @@ export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { * Compare datasources of caches or clusters. * * @param firstObj First cache or cluster. + * @param firstType Type of first object to compare. * @param secondObj Second cache or cluster. + * @param secondType Type of first object to compare. + * @param index Index of invalid object when check is failed. * @returns {*} Check result object. */ - function compareDataSources(firstObj, secondObj) { + function compareDataSources(firstObj, firstType, secondObj, secondType, index) { const firstDs = extractDataSource(firstObj); const secondDs = extractDataSource(secondObj); @@ -280,7 +267,7 @@ export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { const secondDB = secondDs.dialect; if (firstDs.dataSourceBean === secondDs.dataSourceBean && firstDB !== secondDB) - return {checked: false, firstObj, firstDB, secondObj, secondDB}; + return {checked: false, firstObj, firstDs, firstType, secondObj, secondDs, secondType, index}; } return DS_CHECK_SUCCESS; @@ -303,7 +290,6 @@ export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { } return { - getModel, mkOptions(options) { return _.map(options, (option) => { return {value: option, label: isDefined(option) ? option : 'Not set'}; @@ -326,24 +312,24 @@ export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { javaBuiltInTypes, isJavaBuiltInClass, isValidJavaIdentifier, - isValidJavaClass(msg, ident, allowBuiltInClass, elemId, packageOnly, panels, panelId) { + isValidJavaClass(msg, ident, allowBuiltInClass, elemId, packageOnly, panels, panelId, stopEdit = false) { if (isEmptyString(ident)) - return ErrorPopover.show(elemId, msg + ' could not be empty!', panels, panelId); + return !stopEdit && ErrorPopover.show(elemId, msg + ' could not be empty!', panels, panelId); const parts = ident.split('.'); const len = parts.length; if (!allowBuiltInClass && isJavaBuiltInClass(ident)) - return ErrorPopover.show(elemId, msg + ' should not be the Java build-in class!', panels, panelId); + return !stopEdit && ErrorPopover.show(elemId, msg + ' should not be the Java build-in class!', panels, panelId); if (len < 2 && !isJavaBuiltInClass(ident) && !packageOnly) - return ErrorPopover.show(elemId, msg + ' does not have package specified!', panels, panelId); + return !stopEdit && ErrorPopover.show(elemId, msg + ' does not have package specified!', panels, panelId); for (let i = 0; i < parts.length; i++) { const part = parts[i]; - if (!isValidJavaIdentifier(msg, part, elemId, panels, panelId)) + if (!isValidJavaIdentifier(msg, part, elemId, panels, panelId, stopEdit)) return false; } @@ -394,14 +380,25 @@ export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { let res = DS_CHECK_SUCCESS; _.find(caches, (curCache, curIx) => { - res = compareDataSources(curCache, cluster); + // Check datasources of cluster JDBC ip finder and cache store factory datasource. + res = compareDataSources(curCache, 'cache', cluster, 'cluster'); + + if (!res.checked) + return true; + + _.find(cluster.checkpointSpi, (spi, spiIx) => { + res = compareDataSources(curCache, 'cache', spi, 'checkpoint', spiIx); + + return !res.checked; + }); if (!res.checked) return true; + // Check datasource of current saved cache and datasource of other cache in cluster. if (isDefined(checkCacheExt)) { if (checkCacheExt._id !== curCache._id) { - res = compareDataSources(checkCacheExt, curCache); + res = compareDataSources(checkCacheExt, 'cache', curCache, 'cache'); return !res.checked; } @@ -409,9 +406,10 @@ export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { return false; } + // Check datasources of specified list of caches. return _.find(caches, (checkCache, checkIx) => { if (checkIx < curIx) { - res = compareDataSources(checkCache, curCache); + res = compareDataSources(checkCache, 'cache', curCache, 'cache'); return !res.checked; } @@ -420,6 +418,26 @@ export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { }); }); + if (res.checked) { + _.find(cluster.checkpointSpi, (curSpi, curIx) => { + // Check datasources of cluster JDBC ip finder and cache store factory datasource. + res = compareDataSources(cluster, 'cluster', curSpi, 'checkpoint', curIx); + + if (!res.checked) + return true; + + _.find(cluster.checkpointSpi, (spi, spiIx) => { + if (spiIx < curIx) { + res = compareDataSources(curSpi, 'checkpoint', spi, 'checkpoint', curIx); + + return !res.checked; + } + + return false; + }); + }); + } + return res; }, checkCacheSQLSchemas(caches, checkCacheExt) { @@ -469,14 +487,8 @@ export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { writeThrough: dflt || cache.writeThrough }; } - }, - autoClusterSwapSpiConfiguration(cluster, caches) { - const swapConfigured = cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind; - - if (!swapConfigured && _.find(caches, (cache) => cache.swapEnabled)) - return {swapSpaceSpi: {kind: 'FileSwapSpaceSpi'}}; - return null; + return {}; }, randomString(len) { const possible = 'ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789'; @@ -498,7 +510,10 @@ export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { const firstErrorKey = errKeys[0]; const firstError = errors[firstErrorKey][0]; - const actualError = firstError.$error[firstErrorKey][0]; + + const err = firstError.$error[firstErrorKey]; + + const actualError = _.isArray(err) ? err[0] : firstError; const errNameFull = actualError.$name; const errNameShort = errNameFull.endsWith('TextInput') ? errNameFull.substring(0, errNameFull.length - 9) : errNameFull; @@ -507,12 +522,17 @@ export default ['IgniteLegacyUtils', ['IgniteErrorPopover', (ErrorPopover) => { try { return errors[firstErrorKey][0].$errorMessages[errName][firstErrorKey]; } - catch (ignored) { + catch (ignored1) { try { return form[firstError.$name].$errorMessages[errName][firstErrorKey]; } - catch (ignited) { - return false; + catch (ignored2) { + try { + return form.$errorMessages[errName][firstErrorKey]; + } + catch (ignored3) { + return false; + } } } }; diff --git a/modules/web-console/frontend/app/services/SqlTypes.service.js b/modules/web-console/frontend/app/services/SqlTypes.service.js index 2a16e9d559f0c..e42d903fd3a2c 100644 --- a/modules/web-console/frontend/app/services/SqlTypes.service.js +++ b/modules/web-console/frontend/app/services/SqlTypes.service.js @@ -15,13 +15,11 @@ * limitations under the License. */ -import _ from 'lodash'; - // List of H2 reserved SQL keywords. -import H2_SQL_KEYWORDS from '../data/sql-keywords.json'; +import H2_SQL_KEYWORDS from 'app/data/sql-keywords.json'; // List of JDBC type descriptors. -import JDBC_TYPES from '../data/jdbc-types.json'; +import JDBC_TYPES from 'app/data/jdbc-types.json'; // Regular expression to check H2 SQL identifier. const VALID_IDENTIFIER = /^[a-zA-Z_][a-zA-Z0-9_$]*$/im; diff --git a/modules/web-console/frontend/app/vendor.js b/modules/web-console/frontend/app/vendor.js index 0322887de9f68..a9e88442cf0a3 100644 --- a/modules/web-console/frontend/app/vendor.js +++ b/modules/web-console/frontend/app/vendor.js @@ -38,6 +38,7 @@ import 'brace'; import 'brace/mode/xml'; import 'brace/mode/sql'; import 'brace/mode/java'; +import 'brace/mode/csharp'; import 'brace/mode/dockerfile'; import 'brace/mode/snippets'; import 'brace/theme/chrome'; @@ -46,7 +47,7 @@ import 'brace/ext/searchbox'; import 'file-saver'; import 'jszip'; import 'nvd3'; -import 'query-command-supported'; +import 'lodash'; import 'angular-gridster/dist/angular-gridster.min.css'; import 'angular-tree-control/css/tree-control-attribute.css'; import 'angular-tree-control/css/tree-control.css'; diff --git a/modules/web-console/frontend/controllers/caches-controller.js b/modules/web-console/frontend/controllers/caches-controller.js index 8c3290694ee88..8c01173ae509c 100644 --- a/modules/web-console/frontend/controllers/caches-controller.js +++ b/modules/web-console/frontend/controllers/caches-controller.js @@ -17,8 +17,8 @@ // Controller for Caches screen. export default ['cachesController', [ - '$scope', '$http', '$state', '$filter', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'igniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', - function($scope, $http, $state, $filter, $timeout, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, Resource, ErrorPopover, FormUtils) { + '$scope', '$http', '$state', '$filter', '$timeout', '$modal', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', 'IgniteLegacyTable', + function($scope, $http, $state, $filter, $timeout, $modal, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, Resource, ErrorPopover, FormUtils, LegacyTable) { UnsavedChangesGuard.install($scope); const emptyCache = {empty: true}; @@ -96,6 +96,73 @@ export default ['cachesController', [ item.offHeapMaxMemory = item.offHeapMaxMemory > 0 ? item.offHeapMaxMemory : null; }; + $scope.tablePairSave = LegacyTable.tablePairSave; + $scope.tablePairSaveVisible = LegacyTable.tablePairSaveVisible; + $scope.tableNewItem = LegacyTable.tableNewItem; + $scope.tableNewItemActive = LegacyTable.tableNewItemActive; + + $scope.tableStartEdit = function(item, field, index) { + if ($scope.tableReset(true)) + LegacyTable.tableStartEdit(item, field, index, $scope.tableSave); + }; + + $scope.tableEditing = LegacyTable.tableEditing; + + $scope.tableSave = function(field, index, stopEdit) { + if (LegacyTable.tablePairSaveVisible(field, index)) + return LegacyTable.tablePairSave($scope.tablePairValid, $scope.backupItem, field, index, stopEdit); + + return true; + }; + + $scope.tableRemove = function(item, field, index) { + if ($scope.tableReset(true)) + LegacyTable.tableRemove(item, field, index); + }; + + $scope.tableReset = (trySave) => { + const field = LegacyTable.tableField(); + + if (trySave && LegacyUtils.isDefined(field) && !$scope.tableSave(field, LegacyTable.tableEditedRowIndex(), true)) + return false; + + LegacyTable.tableReset(); + + return true; + }; + + $scope.hibernatePropsTbl = { + type: 'hibernate', + model: 'cacheStoreFactory.CacheHibernateBlobStoreFactory.hibernateProperties', + focusId: 'Property', + ui: 'table-pair', + keyName: 'name', + valueName: 'value', + save: $scope.tableSave + }; + + $scope.tablePairValid = function(item, field, index, stopEdit) { + const pairValue = LegacyTable.tablePairValue(field, index); + + const model = _.get(item, field.model); + + if (!_.isNil(model)) { + const idx = _.findIndex(model, (pair) => { + return pair.name === pairValue.key; + }); + + // Found duplicate by key. + if (idx >= 0 && idx !== index) { + if (stopEdit) + return false; + + return ErrorPopover.show(LegacyTable.tableFieldId(index, 'KeyProperty'), 'Property with such name already exists!', $scope.ui, 'query'); + } + } + + return true; + }; + Loading.start('loadingCachesScreen'); // When landing on the page, get caches and show them. @@ -117,6 +184,7 @@ export default ['cachesController', [ value: cluster._id, label: cluster.name, discovery: cluster.discovery, + checkpointSpi: cluster.checkpointSpi, caches: cluster.caches })); @@ -204,7 +272,7 @@ export default ['cachesController', [ else $scope.backupItem = emptyCache; - $scope.backupItem = angular.merge({}, blank, $scope.backupItem); + $scope.backupItem = _.merge({}, blank, $scope.backupItem); if ($scope.ui.inputForm) { $scope.ui.inputForm.$error = {}; @@ -258,6 +326,15 @@ export default ['cachesController', [ return caches; } + const _objToString = (type, name, prefix = '') => { + if (type === 'checkpoint') + return `${prefix} checkpoint configuration in cluster "${name}"`; + if (type === 'cluster') + return `${prefix} discovery IP finder in cluster "${name}"`; + + return `${prefix} ${type} "${name}"`; + }; + function checkDataSources() { const clusters = cacheClusters(); @@ -272,20 +349,11 @@ export default ['cachesController', [ }); if (!checkRes.checked) { - if (_.get(checkRes.secondObj, 'discovery.kind') === 'Jdbc') { - return ErrorPopover.show(checkRes.firstObj.cacheStoreFactory.kind === 'CacheJdbcPojoStoreFactory' ? 'pojoDialectInput' : 'blobDialectInput', - 'Found cluster "' + failCluster.label + '" with the same data source bean name "' + - checkRes.secondObj.discovery.Jdbc.dataSourceBean + '" and different database: "' + - LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.firstDB) + '" in current cache and "' + - LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.secondDB) + '" in"' + checkRes.secondObj.label + '" cluster', - $scope.ui, 'store', 10000); - } - return ErrorPopover.show(checkRes.firstObj.cacheStoreFactory.kind === 'CacheJdbcPojoStoreFactory' ? 'pojoDialectInput' : 'blobDialectInput', - 'Found cache "' + checkRes.secondObj.name + '" in cluster "' + failCluster.label + '" ' + - 'with the same data source bean name "' + checkRes.firstObj.cacheStoreFactory[checkRes.firstObj.cacheStoreFactory.kind].dataSourceBean + - '" and different database: "' + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.firstDB) + '" in current cache and "' + - LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.secondDB) + '" in "' + checkRes.secondObj.name + '" cache', + 'Found ' + _objToString(checkRes.secondType, checkRes.secondObj.name || failCluster.label) + ' with the same data source bean name "' + + checkRes.firstDs.dataSourceBean + '" and different database: "' + + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.firstDs.dialect) + '" in ' + _objToString(checkRes.firstType, checkRes.firstObj.name, 'current') + ' and "' + + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.secondDs.dialect) + '" in ' + _objToString(checkRes.secondType, checkRes.secondObj.name || failCluster.label), $scope.ui, 'store', 10000); } @@ -296,7 +364,7 @@ export default ['cachesController', [ if (evictionPlc && evictionPlc.kind) { const plc = evictionPlc[evictionPlc.kind]; - if (plc.maxMemorySize === 0 && plc.maxSize === 0) + if (plc && !plc.maxMemorySize && !plc.maxSize) return ErrorPopover.show('evictionPolicymaxMemorySizeInput', 'Either maximum memory size or maximum size should be great than 0!', $scope.ui, 'memory'); } @@ -409,7 +477,7 @@ export default ['cachesController', [ }); if (idx >= 0) - angular.merge($scope.caches[idx], item); + _.assign($scope.caches[idx], item); else { item._id = _id; $scope.caches.push(item); @@ -440,7 +508,7 @@ export default ['cachesController', [ $scope.saveItem = function() { const item = $scope.backupItem; - angular.extend(item, LegacyUtils.autoCacheStoreConfiguration(item, cacheDomains(item))); + _.merge(item, LegacyUtils.autoCacheStoreConfiguration(item, cacheDomains(item))); if (validate(item)) save(item); @@ -462,7 +530,20 @@ export default ['cachesController', [ item.name = newName; - delete item.sqlSchema; + if (!_.isEmpty(item.clusters) && !_.isNil(item.sqlSchema)) { + delete item.sqlSchema; + + const scope = $scope.$new(); + + scope.title = 'Info'; + scope.content = [ + 'Use the same SQL schema name in one cluster in not allowed', + 'SQL schema name will be reset' + ]; + + // Show a basic modal from a controller + $modal({scope, template: '/templates/message.html', placement: 'center', show: true}); + } save(item); }); diff --git a/modules/web-console/frontend/controllers/clusters-controller.js b/modules/web-console/frontend/controllers/clusters-controller.js index 5a3c7e2717602..f9096f7f61618 100644 --- a/modules/web-console/frontend/controllers/clusters-controller.js +++ b/modules/web-console/frontend/controllers/clusters-controller.js @@ -17,7 +17,7 @@ // Controller for Clusters screen. export default ['clustersController', [ - '$rootScope', '$scope', '$http', '$state', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'igniteEventGroups', 'DemoInfo', 'IgniteLegacyTable', 'igniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', + '$rootScope', '$scope', '$http', '$state', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'igniteEventGroups', 'DemoInfo', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', function($root, $scope, $http, $state, $timeout, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, igniteEventGroups, DemoInfo, LegacyTable, Resource, ErrorPopover, FormUtils) { UnsavedChangesGuard.install($scope); @@ -103,6 +103,46 @@ export default ['clustersController', [ else $scope.backupItem.failoverSpi = {}; } + else if (field.type === 'loadBalancingSpi') { + const newLoadBalancing = {Adaptive: { + loadProbe: { + Job: {useAverage: true}, + CPU: { + useAverage: true, + useProcessors: true + }, + ProcessingTime: {useAverage: true} + } + }}; + + if (LegacyUtils.isDefined($scope.backupItem.loadBalancingSpi)) + $scope.backupItem.loadBalancingSpi.push(newLoadBalancing); + else + $scope.backupItem.loadBalancingSpi = [newLoadBalancing]; + } + else if (field.type === 'checkpointSpi') { + const newCheckpointCfg = { + FS: { + directoryPaths: [] + }, + S3: { + awsCredentials: { + kind: 'Basic' + }, + clientConfiguration: { + retryPolicy: { + kind: 'Default' + }, + useReaper: true + } + } + }; + + if (LegacyUtils.isDefined($scope.backupItem.checkpointSpi)) + $scope.backupItem.checkpointSpi.push(newCheckpointCfg); + else + $scope.backupItem.checkpointSpi = [newCheckpointCfg]; + } else LegacyTable.tableNewItem(field); } @@ -149,6 +189,8 @@ export default ['clustersController', [ $scope.backupItem.failoverSpi.splice(idx, 1); }; + $scope.supportedJdbcTypes = LegacyUtils.mkOptions(LegacyUtils.SUPPORTED_JDBC_TYPES); + // We need to initialize backupItem with empty object in order to properly used from angular directives. $scope.backupItem = emptyCluster; @@ -204,11 +246,20 @@ export default ['clustersController', [ // When landing on the page, get clusters and show them. Resource.read() - .then(({spaces, clusters, caches, igfss}) => { + .then(({spaces, clusters, caches, domains, igfss}) => { $scope.spaces = spaces; + $scope.clusters = clusters; - $scope.caches = _.map(caches, (cache) => ({value: cache._id, label: cache.name, cache})); + $scope.caches = _.map(caches, (cache) => { + cache.domains = _.filter(domains, ({_id}) => _.includes(cache.domains, _id)); + + if (_.get(cache, 'nodeFilter.kind') === 'IGFS') + cache.nodeFilter.IGFS.instance = _.find(igfss, {_id: cache.nodeFilter.IGFS.igfs}); + + return {value: cache._id, label: cache.name, cache}; + }); + $scope.igfss = _.map(igfss, (igfs) => ({value: igfs._id, label: igfs.name, igfs})); _.forEach($scope.clusters, (cluster) => { @@ -222,6 +273,9 @@ export default ['clustersController', [ if (!cluster.logger) cluster.logger = {Log4j: { mode: 'Default'}}; + + if (!cluster.eventStorage) + cluster.eventStorage = { kind: 'Memory' }; }); if ($state.params.linkId) @@ -259,6 +313,12 @@ export default ['clustersController', [ form.$setPristine(); else form.$setDirty(); + + $scope.clusterCaches = _.filter($scope.caches, + (cache) => _.find($scope.backupItem.caches, + (selCache) => selCache === cache.value + ) + ); }, true); $scope.$watch('ui.activePanels.length', () => { @@ -279,6 +339,8 @@ export default ['clustersController', [ Loading.finish('loadingClustersScreen'); }); + $scope.clusterCaches = []; + $scope.selectItem = function(item, backup) { function selectItem() { $scope.selectedItem = item; @@ -300,7 +362,7 @@ export default ['clustersController', [ else $scope.backupItem = emptyCluster; - $scope.backupItem = angular.merge({}, blank, $scope.backupItem); + $scope.backupItem = _.merge({}, blank, $scope.backupItem); if ($scope.ui.inputForm) { $scope.ui.inputForm.$error = {}; @@ -319,7 +381,7 @@ export default ['clustersController', [ $scope.linkId = () => $scope.backupItem._id ? $scope.backupItem._id : 'create'; function prepareNewItem(linkId) { - return angular.merge({}, blank, { + return _.merge({}, blank, { space: $scope.spaces[0]._id, discovery: { kind: 'Multicast', @@ -331,6 +393,7 @@ export default ['clustersController', [ communication: {tcpNoDelay: true}, connector: {noDelay: true}, collision: {kind: 'Noop', JobStealing: {stealingEnabled: true}, PriorityQueue: {starvationPreventionEnabled: true}}, + eventStorage: {kind: 'Memory'}, failoverSpi: [], logger: {Log4j: { mode: 'Default'}}, caches: linkId && _.find($scope.caches, {value: linkId}) ? [linkId] : [], @@ -354,27 +417,45 @@ export default ['clustersController', [ (cache) => _.includes(item.caches, cache._id)); } + const _objToString = (type, name, prefix = '') => { + if (type === 'checkpoint') + return prefix + ' checkpoint configuration'; + if (type === 'cluster') + return prefix + ' discovery IP finder'; + + return `${prefix} ${type} "${name}"`; + }; + function checkCacheDatasources(item) { const caches = clusterCaches(item); const checkRes = LegacyUtils.checkDataSources(item, caches); if (!checkRes.checked) { - if (_.get(checkRes.secondObj, 'discovery.kind') === 'Jdbc') { - return ErrorPopover.show('dialectInput', - 'Found cache "' + checkRes.firstObj.name + '" with the same data source bean name "' + - item.discovery.Jdbc.dataSourceBean + '" and different database: "' + - LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.secondDB) + '" in current cluster and "' + - LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.firstDB) + '" in "' + checkRes.firstObj.name + '" cache', - $scope.ui, 'general', 10000); + let ids; + + if (checkRes.secondType === 'cluster') + ids = { section: 'general', fieldId: 'dialectInput' }; + else if (checkRes.secondType === 'cache') + ids = { section: 'general', fieldId: 'cachesInput' }; + else if (checkRes.secondType === 'checkpoint') + ids = { section: 'checkpoint', fieldId: `checkpointJdbcDialect${checkRes.index}Input` }; + else + return true; + + if (checkRes.firstType === checkRes.secondType && checkRes.firstType === 'cache') { + return ErrorPopover.show(ids.fieldId, 'Found caches "' + checkRes.firstObj.name + '" and "' + checkRes.secondObj.name + '" with the same data source bean name "' + + checkRes.firstDs.dataSourceBean + '" and different database: "' + + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.secondDs.dialect) + '" in ' + _objToString(checkRes.secondType, checkRes.secondObj.name) + ' and "' + + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.firstDs.dialect) + '" in ' + _objToString(checkRes.firstType, checkRes.firstObj.name), + $scope.ui, ids.section, 10000); } - return ErrorPopover.show('cachesInput', - 'Found caches "' + checkRes.firstObj.name + '" and "' + checkRes.secondObj.name + '" ' + - 'with the same data source bean name "' + checkRes.firstObj.cacheStoreFactory[checkRes.firstObj.cacheStoreFactory.kind].dataSourceBean + - '" and different databases: "' + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.firstDB) + '" in "' + checkRes.firstObj.name + '" and "' + - LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.secondDB) + '" in "' + checkRes.secondObj.name + '" cache', - $scope.ui, 'general', 10000); + return ErrorPopover.show(ids.fieldId, 'Found ' + _objToString(checkRes.firstType, checkRes.firstObj.name) + ' with the same data source bean name "' + + checkRes.firstDs.dataSourceBean + '" and different database: "' + + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.secondDs.dialect) + '" in ' + _objToString(checkRes.secondType, checkRes.secondObj.name, 'current') + ' and "' + + LegacyUtils.cacheStoreJdbcDialectsLabel(checkRes.firstDs.dialect) + '" in ' + _objToString(checkRes.firstType, checkRes.firstObj.name), + $scope.ui, ids.section, 10000); } return true; @@ -434,6 +515,38 @@ export default ['clustersController', [ return true; } + function checkCheckpointSpis(item) { + const cfgs = item.checkpointSpi; + + if (_.isEmpty(cfgs)) + return true; + + return _.isNil(_.find(cfgs, (cfg, ix) => { + if (_.isNil(cfg.kind)) { + ErrorPopover.show('checkpointKind' + ix, 'Choose checkpoint implementation variant', $scope.ui, 'checkpoint'); + + return true; + } + + switch (cfg.kind) { + case 'Cache': + const cache = _.get(cfg, 'Cache.cache'); + + if (_.isNil(cache) || !_.find($scope.backupItem.caches, (selCache) => cache === selCache)) { + ErrorPopover.show('checkpointCacheCache' + ix, 'Choose cache from configured cluster caches', $scope.ui, 'checkpoint'); + + return true; + } + + break; + + default: break; + } + + return false; + })); + } + function checkCommunicationConfiguration(item) { const c = item.communication; @@ -467,6 +580,20 @@ export default ['clustersController', [ return true; } + function checkLoadBalancingConfiguration(item) { + const balancingSpis = item.loadBalancingSpi; + + return _.isNil(_.find(balancingSpis, (curSpi, curIx) => { + if (_.find(balancingSpis, (spi, ix) => curIx > ix && curSpi.kind === spi.kind)) { + ErrorPopover.show('loadBalancingKind' + curIx, 'Load balancing SPI of that type is already configured', $scope.ui, 'loadBalancing'); + + return true; + } + + return false; + })); + } + function checkSwapConfiguration(item) { const swapKind = item.swapSpaceSpi && item.swapSpaceSpi.kind; @@ -535,12 +662,18 @@ export default ['clustersController', [ if (!checkCacheKeyConfiguration(item)) return false; + if (!checkCheckpointSpis(item)) + return false; + if (!checkCommunicationConfiguration(item)) return false; if (!checkDiscoveryConfiguration(item)) return false; + if (!checkLoadBalancingConfiguration(item)) + return false; + if (!checkSwapConfiguration(item)) return false; @@ -564,7 +697,7 @@ export default ['clustersController', [ const idx = _.findIndex($scope.clusters, (cluster) => cluster._id === _id); if (idx >= 0) - angular.merge($scope.clusters[idx], item); + _.assign($scope.clusters[idx], item); else { item._id = _id; $scope.clusters.push(item); @@ -595,10 +728,10 @@ export default ['clustersController', [ $scope.saveItem = function() { const item = $scope.backupItem; - const swapSpi = LegacyUtils.autoClusterSwapSpiConfiguration(item, clusterCaches(item)); + const swapConfigured = item.swapSpaceSpi && item.swapSpaceSpi.kind; - if (swapSpi) - angular.extend(item, swapSpi); + if (!swapConfigured && _.find(clusterCaches(item), (cache) => cache.swapEnabled)) + _.merge(item, {swapSpaceSpi: {kind: 'FileSwapSpaceSpi'}}); if (validate(item)) save(item); diff --git a/modules/web-console/frontend/controllers/domains-controller.js b/modules/web-console/frontend/controllers/domains-controller.js index 2d450dbe91f25..0a79d82603e69 100644 --- a/modules/web-console/frontend/controllers/domains-controller.js +++ b/modules/web-console/frontend/controllers/domains-controller.js @@ -17,7 +17,7 @@ // Controller for Domain model screen. export default ['domainsController', [ - '$rootScope', '$scope', '$http', '$state', '$filter', '$timeout', '$modal', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteConfirmBatch', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteAgentMonitor', 'IgniteLegacyTable', 'igniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', 'JavaTypes', 'SqlTypes', + '$rootScope', '$scope', '$http', '$state', '$filter', '$timeout', '$modal', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteConfirmBatch', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteAgentMonitor', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', 'JavaTypes', 'SqlTypes', function($root, $scope, $http, $state, $filter, $timeout, $modal, LegacyUtils, Messages, Focus, Confirm, ConfirmBatch, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, IgniteAgentMonitor, LegacyTable, Resource, ErrorPopover, FormUtils, JavaTypes, SqlTypes) { UnsavedChangesGuard.install($scope); @@ -58,6 +58,7 @@ export default ['domainsController', [ $scope.$on('$destroy', $root.$on('user', _packageNameUpdate)); + $scope.ui.generatePojo = true; $scope.ui.builtinKeys = true; $scope.ui.usePrimitives = true; $scope.ui.generateAliases = true; @@ -75,7 +76,6 @@ export default ['domainsController', [ return !item.empty && (!item._id || _.find($scope.displayedRows, {_id: item._id})); }; - $scope.getModel = LegacyUtils.getModel; $scope.javaBuiltInClasses = LegacyUtils.javaBuiltInClasses; $scope.compactJavaName = FormUtils.compactJavaName; $scope.widthIsSufficient = FormUtils.widthIsSufficient; @@ -91,7 +91,7 @@ export default ['domainsController', [ case 'fields': case 'aliases': if (LegacyTable.tablePairSaveVisible(field, index)) - return LegacyTable.tablePairSave($scope.tablePairValid, $scope.backupItem, field, index, stopEdit); + return LegacyTable.tablePairSave($scope.tablePairValid, $scope.backupItem, field, index, stopEdit) || stopEdit; break; @@ -140,11 +140,43 @@ export default ['domainsController', [ $scope.tableEditing = LegacyTable.tableEditing; $scope.tableRemove = function(item, field, index) { - if ($scope.tableReset(true)) + if ($scope.tableReset(true)) { + // Remove field from indexes. + if (field.type === 'fields') { + _.forEach($scope.backupItem.indexes, (modelIndex) => { + modelIndex.fields = _.filter(modelIndex.fields, (indexField) => { + return indexField.name !== $scope.backupItem.fields[index].name; + }); + }); + } + LegacyTable.tableRemove(item, field, index); + } + }; + + $scope.tablePairSave = (pairValid, item, field, index, stopEdit) => { + // On change of field name update that field in index fields. + if (index >= 0 && field.type === 'fields') { + const newName = LegacyTable.tablePairValue(field, index).key; + const oldName = _.get(item, field.model)[index][field.keyName]; + + const saved = LegacyTable.tablePairSave(pairValid, item, field, index, stopEdit); + + if (saved && oldName !== newName) { + _.forEach($scope.backupItem.indexes, (idx) => { + _.forEach(idx.fields, (fld) => { + if (fld.name === oldName) + fld.name = newName; + }); + }); + } + + return saved; + } + + return LegacyTable.tablePairSave(pairValid, item, field, index, stopEdit); }; - $scope.tablePairSave = LegacyTable.tablePairSave; $scope.tablePairSaveVisible = LegacyTable.tablePairSaveVisible; $scope.queryFieldsTbl = { @@ -169,6 +201,19 @@ export default ['domainsController', [ $scope.queryMetadataVariants = LegacyUtils.mkOptions(['Annotations', 'Configuration']); + // Create list of fields to show in index fields dropdown. + $scope.fields = (prefix, cur) => { + const fields = _.map($scope.backupItem.fields, (field) => ({value: field.name, label: field.name})); + + if (prefix === 'new') + return fields; + + if (cur && !_.find(fields, {value: cur})) + fields.push({value: cur, label: cur + ' (Unknown field)'}); + + return fields; + }; + const INFO_CONNECT_TO_DB = 'Configure connection to database'; const INFO_SELECT_SCHEMAS = 'Select schemas to load tables from'; const INFO_SELECT_TABLES = 'Select tables to import as domain model'; @@ -214,6 +259,12 @@ export default ['domainsController', [ jdbcUrl: 'jdbc:mysql://[host]:[port]/[database]', user: 'root' }, + { + db: 'MySQL', + jdbcDriverClass: 'org.mariadb.jdbc.Driver', + jdbcUrl: 'jdbc:mariadb://[host]:[port]/[database]', + user: 'root' + }, { db: 'H2', jdbcDriverClass: 'org.h2.Driver', @@ -223,7 +274,7 @@ export default ['domainsController', [ ]; $scope.selectedPreset = { - db: 'General', + db: 'Generic', jdbcDriverJar: '', jdbcDriverClass: '', jdbcUrl: 'jdbc:[database]', @@ -266,7 +317,7 @@ export default ['domainsController', [ const oldPreset = _.find(_dbPresets, {jdbcDriverClass: preset.jdbcDriverClass}); if (oldPreset) - angular.extend(oldPreset, preset); + _.assign(oldPreset, preset); else _dbPresets.push(preset); @@ -283,7 +334,7 @@ export default ['domainsController', [ }); if (!result) - result = {db: 'General', jdbcUrl: 'jdbc:[database]', user: 'admin'}; + result = {db: 'Generic', jdbcUrl: 'jdbc:[database]', user: 'admin'}; result.jdbcDriverJar = selectedJdbcJar.jdbcDriverJar; result.jdbcDriverClass = selectedJdbcJar.jdbcDriverClass; @@ -373,6 +424,7 @@ export default ['domainsController', [ function prepareNewItem(cacheId) { return { space: $scope.spaces[0]._id, + generatePojo: true, caches: cacheId && _.find($scope.caches, {value: cacheId}) ? [cacheId] : // eslint-disable-line no-nested-ternary (_.isEmpty($scope.caches) ? [] : [$scope.caches[0].value]), queryMetadata: 'Configuration' @@ -751,20 +803,12 @@ export default ['domainsController', [ importDomainModal.hide(); } - function _saveDomainModel() { - if (LegacyUtils.isEmptyString($scope.ui.packageName)) { - ErrorPopover.show('domainPackageNameInput', 'Package could not be empty'); - - Focus.move('domainPackageNameInput'); - - return false; - } - - if (!LegacyUtils.isValidJavaClass('Package', $scope.ui.packageName, false, 'domainPackageNameInput', true)) { - Focus.move('domainPackageNameInput'); + function _saveDomainModel(optionsForm) { + const generatePojo = $scope.ui.generatePojo; + const packageName = $scope.ui.packageName; + if (generatePojo && !LegacyUtils.checkFieldValidators({inputForm: optionsForm})) return false; - } const batch = []; const checkedCaches = []; @@ -803,7 +847,7 @@ export default ['domainsController', [ containDup = true; } - const valType = _toJavaPackage($scope.ui.packageName) + '.' + typeName; + const valType = generatePojo ? _toJavaPackage(packageName) + '.' + typeName : tableName; let _containKey = false; @@ -852,7 +896,8 @@ export default ['domainsController', [ confirm: false, skip: false, space: $scope.spaces[0], - caches: [] + caches: [], + generatePojo }; if (LegacyUtils.isDefined(domainFound)) { @@ -985,7 +1030,7 @@ export default ['domainsController', [ } } - $scope.importDomainNext = function() { + $scope.importDomainNext = function(form) { if (!$scope.importDomainNextAvailable()) return; @@ -1000,7 +1045,7 @@ export default ['domainsController', [ else if (act === 'tables') _selectOptions(); else if (act === 'options') - _saveDomainModel(); + _saveDomainModel(form); }; $scope.nextTooltipText = function() { @@ -1101,11 +1146,14 @@ export default ['domainsController', [ Resource.read() .then(({spaces, clusters, caches, domains}) => { $scope.spaces = spaces; + $scope.clusters = _.map(clusters, (cluster) => ({ label: cluster.name, value: cluster._id })); + $scope.caches = _mapCaches(caches); + $scope.domains = _.sortBy(domains, 'valueType'); _.forEach($scope.clusters, (cluster) => $scope.ui.generatedCachesClusters.push(cluster.value)); @@ -1157,8 +1205,14 @@ export default ['domainsController', [ if (form.$valid && ModelNormalizer.isEqual(__original_value, val)) form.$setPristine(); - else + else { form.$setDirty(); + + const general = form.general; + + FormUtils.markPristineInvalidAsDirty(general.keyType); + FormUtils.markPristineInvalidAsDirty(general.valueType); + } }, true); $scope.$watch('ui.activePanels.length', () => { @@ -1210,7 +1264,7 @@ export default ['domainsController', [ else $scope.backupItem = emptyDomain; - $scope.backupItem = angular.merge({}, blank, $scope.backupItem); + $scope.backupItem = _.merge({}, blank, $scope.backupItem); if ($scope.ui.inputForm) { $scope.ui.inputForm.$error = {}; @@ -1252,9 +1306,12 @@ export default ['domainsController', [ const indexes = item.indexes; if (indexes && indexes.length > 0) { - if (_.find(indexes, function(index, i) { + if (_.find(indexes, function(index, idx) { if (_.isEmpty(index.fields)) - return !ErrorPopover.show('indexes' + i, 'Index fields are not specified', $scope.ui, 'query'); + return !ErrorPopover.show('indexes' + idx, 'Index fields are not specified', $scope.ui, 'query'); + + if (_.find(index.fields, (field) => !_.find(item.fields, (configuredField) => configuredField.name === field.name))) + return !ErrorPopover.show('indexes' + idx, 'Index contains not configured fields', $scope.ui, 'query'); })) return false; } @@ -1332,7 +1389,7 @@ export default ['domainsController', [ }); if (idx >= 0) - angular.extend($scope.domains[idx], savedMeta); + _.assign($scope.domains[idx], savedMeta); else $scope.domains.push(savedMeta); @@ -1503,15 +1560,11 @@ export default ['domainsController', [ }); // Found duplicate by key. - if (idx >= 0 && idx !== index) { - if (stopEdit) - return false; - - return ErrorPopover.show(LegacyTable.tableFieldId(index, pairField.idPrefix + pairField.id), 'Field with such ' + pairField.dupObjName + ' already exists!', $scope.ui, 'query'); - } + if (idx >= 0 && idx !== index) + return !stopEdit && ErrorPopover.show(LegacyTable.tableFieldId(index, pairField.idPrefix + pairField.id), 'Field with such ' + pairField.dupObjName + ' already exists!', $scope.ui, 'query'); } - if (pairField.classValidation && !LegacyUtils.isValidJavaClass(pairField.msg, pairValue.value, true, LegacyTable.tableFieldId(index, 'Value' + pairField.id), false, $scope.ui, 'query')) { + if (pairField.classValidation && !LegacyUtils.isValidJavaClass(pairField.msg, pairValue.value, true, LegacyTable.tableFieldId(index, 'Value' + pairField.id), false, $scope.ui, 'query', stopEdit)) { if (stopEdit) return false; @@ -1560,8 +1613,8 @@ export default ['domainsController', [ let model = item[field.model]; - if (!LegacyUtils.isValidJavaIdentifier(dbFieldTable.msg + ' java name', dbFieldValue.javaFieldName, LegacyTable.tableFieldId(index, 'JavaFieldName' + dbFieldTable.id))) - return false; + if (!LegacyUtils.isValidJavaIdentifier(dbFieldTable.msg + ' java name', dbFieldValue.javaFieldName, LegacyTable.tableFieldId(index, 'JavaFieldName' + dbFieldTable.id), $scope.ui, 'store', stopEdit)) + return stopEdit; if (LegacyUtils.isDefined(model)) { let idx = _.findIndex(model, function(dbMeta) { @@ -1570,7 +1623,7 @@ export default ['domainsController', [ // Found duplicate. if (idx >= 0 && index !== idx) - return ErrorPopover.show(LegacyTable.tableFieldId(index, 'DatabaseFieldName' + dbFieldTable.id), 'Field with such database name already exists!', $scope.ui, 'store'); + return stopEdit || ErrorPopover.show(LegacyTable.tableFieldId(index, 'DatabaseFieldName' + dbFieldTable.id), 'Field with such database name already exists!', $scope.ui, 'store'); idx = _.findIndex(model, function(dbMeta) { return dbMeta.javaFieldName === dbFieldValue.javaFieldName; @@ -1578,7 +1631,7 @@ export default ['domainsController', [ // Found duplicate. if (idx >= 0 && index !== idx) - return ErrorPopover.show(LegacyTable.tableFieldId(index, 'JavaFieldName' + dbFieldTable.id), 'Field with such java name already exists!', $scope.ui, 'store'); + return stopEdit || ErrorPopover.show(LegacyTable.tableFieldId(index, 'JavaFieldName' + dbFieldTable.id), 'Field with such java name already exists!', $scope.ui, 'store'); if (index < 0) model.push(dbFieldValue); @@ -1639,7 +1692,7 @@ export default ['domainsController', [ // Found duplicate. if (idx >= 0 && idx !== curIdx) - return ErrorPopover.show(LegacyTable.tableFieldId(curIdx, 'IndexName'), 'Index with such name already exists!', $scope.ui, 'query'); + return !stopEdit && ErrorPopover.show(LegacyTable.tableFieldId(curIdx, 'IndexName'), 'Index with such name already exists!', $scope.ui, 'query'); } LegacyTable.tableReset(); @@ -1675,10 +1728,8 @@ export default ['domainsController', [ $scope.tableIndexNewItem = function(field, indexIdx) { if ($scope.tableReset(true)) { - const index = $scope.backupItem.indexes[indexIdx]; - LegacyTable.tableState(field, -1, 'table-index-fields'); - LegacyTable.tableFocusInvalidField(-1, 'FieldName' + (index.indexType === 'SORTED' ? 'S' : '') + indexIdx); + LegacyTable.tableFocusInvalidField(-1, 'FieldName' + indexIdx); field.newFieldName = null; field.newDirection = true; @@ -1734,7 +1785,7 @@ export default ['domainsController', [ field.curDirection = indexItem.direction; field.indexIdx = indexIdx; - Focus.move('curFieldName' + (index.indexType === 'SORTED' ? 'S' : '') + field.indexIdx + '-' + curIdx); + Focus.move('curFieldName' + field.indexIdx + '-' + curIdx); } }; @@ -1753,8 +1804,11 @@ export default ['domainsController', [ const idx = _.findIndex(fields, (fld) => fld.name === indexItemValue.name); // Found duplicate. - if (idx >= 0 && idx !== curIdx) - return ErrorPopover.show(LegacyTable.tableFieldId(curIdx, 'FieldName' + (index.indexType === 'SORTED' ? 'S' : '') + indexIdx + (curIdx >= 0 ? '-' : '')), 'Field with such name already exists in index!', $scope.ui, 'query'); + if (idx >= 0 && idx !== curIdx) { + return !stopEdit && ErrorPopover.show(LegacyTable.tableFieldId(curIdx, + 'FieldName' + indexIdx + (curIdx >= 0 ? '-' : '')), + 'Field with such name already exists in index!', $scope.ui, 'query'); + } } LegacyTable.tableReset(); diff --git a/modules/web-console/frontend/controllers/igfs-controller.js b/modules/web-console/frontend/controllers/igfs-controller.js index 761771207dc32..e505f1c450948 100644 --- a/modules/web-console/frontend/controllers/igfs-controller.js +++ b/modules/web-console/frontend/controllers/igfs-controller.js @@ -17,7 +17,7 @@ // Controller for IGFS screen. export default ['igfsController', [ - '$scope', '$http', '$state', '$filter', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteLegacyTable', 'igniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', + '$scope', '$http', '$state', '$filter', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', function($scope, $http, $state, $filter, $timeout, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, LegacyTable, Resource, ErrorPopover, FormUtils) { UnsavedChangesGuard.install($scope); @@ -231,7 +231,7 @@ export default ['igfsController', [ else $scope.backupItem = emptyIgfs; - $scope.backupItem = angular.merge({}, blank, $scope.backupItem); + $scope.backupItem = _.merge({}, blank, $scope.backupItem); if ($scope.ui.inputForm) { $scope.ui.inputForm.$error = {}; @@ -304,7 +304,7 @@ export default ['igfsController', [ }); if (idx >= 0) - angular.merge($scope.igfss[idx], item); + _.assign($scope.igfss[idx], item); else { item._id = _id; $scope.igfss.push(item); diff --git a/modules/web-console/frontend/generator/generator-properties.js b/modules/web-console/frontend/generator/generator-properties.js deleted file mode 100644 index 57427138f43a3..0000000000000 --- a/modules/web-console/frontend/generator/generator-properties.js +++ /dev/null @@ -1,175 +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. - */ - -// Properties generation entry point. -const $generatorProperties = {}; - -$generatorProperties.jdbcUrlTemplate = function(dialect) { - switch (dialect) { - case 'Oracle': - return 'jdbc:oracle:thin:@[host]:[port]:[database]'; - case 'DB2': - return 'jdbc:db2://[host]:[port]/[database]'; - case 'SQLServer': - return 'jdbc:sqlserver://[host]:[port][;databaseName=database]'; - case 'MySQL': - return 'jdbc:mysql://[host]:[port]/[database]'; - case 'PostgreSQL': - return 'jdbc:postgresql://[host]:[port]/[database]'; - case 'H2': - return 'jdbc:h2:tcp://[host]/[database]'; - default: - } - - return 'jdbc:your_database'; -}; - -$generatorProperties.createBuilder = function() { - const res = $generatorCommon.builder(); - - res.line('# ' + $generatorCommon.mainComment('list of properties')); - - return res; -}; - -/** - * Generate properties file with properties stubs for stores data sources. - * - * @param res Resulting output with generated properties. - * @param datasources Already added datasources. - * @param storeFactory Current datasource factory. - * @param dialect Current dialect. - * @returns {string} Generated content. - */ -$generatorProperties.dataSourceProperties = function(res, datasources, storeFactory, dialect) { - const beanId = storeFactory.dataSourceBean; - - const dsClsName = $generatorCommon.dataSourceClassName(dialect); - - const varType = res.importClass(dsClsName); - - const beanClassName = $generatorCommon.toJavaName(varType, storeFactory.dataSourceBean); - - if (!_.includes(datasources, beanClassName)) { - datasources.push(beanClassName); - - res.needEmptyLine = true; - - switch (dialect) { - case 'DB2': - res.line(beanId + '.jdbc.server_name=YOUR_DATABASE_SERVER_NAME'); - res.line(beanId + '.jdbc.port_number=YOUR_JDBC_PORT_NUMBER'); - res.line(beanId + '.jdbc.driver_type=YOUR_JDBC_DRIVER_TYPE'); - res.line(beanId + '.jdbc.database_name=YOUR_DATABASE_NAME'); - - break; - - default: - res.line(beanId + '.jdbc.url=' + $generatorProperties.jdbcUrlTemplate(dialect)); - } - - res.line(beanId + '.jdbc.username=YOUR_USER_NAME'); - res.line(beanId + '.jdbc.password=YOUR_PASSWORD'); - res.line(''); - } -}; - -/** - * Generate properties file with properties stubs for stores data sources. - * - * @param cluster Configuration to process. - * @param res Resulting output with generated properties. - * @returns {string} Generated content. - */ -$generatorProperties.dataSourcesProperties = function(cluster, res) { - const datasources = []; - - if (cluster.caches && cluster.caches.length > 0) { - _.forEach(cluster.caches, function(cache) { - if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) { - const storeFactory = cache.cacheStoreFactory[cache.cacheStoreFactory.kind]; - - const dialect = storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : null) : storeFactory.dialect; - - const connectViaUrl = cache.cacheStoreFactory.kind === 'CacheJdbcBlobStoreFactory' && storeFactory.connectVia === 'URL'; - - if (!res && (dialect || connectViaUrl)) - res = $generatorProperties.createBuilder(); - - if (dialect) - $generatorProperties.dataSourceProperties(res, datasources, storeFactory, dialect); - - if (connectViaUrl) - res.line('ds.' + storeFactory.user + '.password=YOUR_PASSWORD'); - } - }); - } - - if (cluster.discovery.kind === 'Jdbc') { - const ds = cluster.discovery.Jdbc; - - if (ds.dataSourceBean && ds.dialect) { - if (!res) - res = $generatorProperties.createBuilder(); - - $generatorProperties.dataSourceProperties(res, datasources, ds, ds.dialect); - } - } - - return res; -}; - -/** - * Generate properties file with properties stubs for cluster SSL configuration. - * - * @param cluster Cluster to get SSL configuration. - * @param res Optional configuration presentation builder object. - * @returns Configuration presentation builder object - */ -$generatorProperties.sslProperties = function(cluster, res) { - if (cluster.sslEnabled && cluster.sslContextFactory) { - if (!res) - res = $generatorProperties.createBuilder(); - - res.needEmptyLine = true; - - if (_.isEmpty(cluster.sslContextFactory.keyStoreFilePath)) - res.line('ssl.key.storage.password=YOUR_SSL_KEY_STORAGE_PASSWORD'); - - if (_.isEmpty(cluster.sslContextFactory.trustStoreFilePath)) - res.line('ssl.trust.storage.password=YOUR_SSL_TRUST_STORAGE_PASSWORD'); - } - - return res; -}; - -/** - * Generate properties file with all possible properties. - * - * @param cluster Cluster to get configurations. - * @param res Optional configuration presentation builder object. - * @returns Configuration presentation builder object - */ -$generatorProperties.generateProperties = function(cluster, res) { - res = $generatorProperties.dataSourcesProperties(cluster, res); - - res = $generatorProperties.sslProperties(cluster, res); - - return res; -}; - -export default $generatorProperties; diff --git a/modules/web-console/frontend/generator/generator-readme.js b/modules/web-console/frontend/generator/generator-readme.js deleted file mode 100644 index 432f1e6e82993..0000000000000 --- a/modules/web-console/frontend/generator/generator-readme.js +++ /dev/null @@ -1,85 +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. - */ - -// README.txt generation entry point. -const $generatorReadme = {}; - -$generatorReadme.generatedBy = function(res) { - res.line('Content of this folder was generated by Apache Ignite Web Console'); - res.line('================================================================='); - - res.needEmptyLine = true; -}; - -/** - * Generate README.txt. - * - * @param res Resulting output with generated readme. - * @returns {string} Generated content. - */ -$generatorReadme.readme = function(res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorReadme.generatedBy(res); - - res.line('Project structure:'); - res.line(' /config - this folder contains client and server XML configurations.'); - res.line(' /jdbc-drivers - this folder should contains proprietary JDBC drivers.'); - res.line(' /src - this folder contains generated java code.'); - res.line(' /src/main/java/config - this folder contains generated java classes with cluster configuration from code.'); - res.line(' /src/main/java/startup - this folder contains generated java classes with server and client nodes startup code.'); - res.line(' /src/main/java/[model] - this optional folder will be named as package name for your POJO classes and contain generated POJO files.'); - res.line(' /src/main/resources - this optional folder contains generated secret.properties file with security sensitive information if any.'); - res.line(' Dockerfile - sample Docker file. With this file you could package Ignite deployment with all the dependencies into a standard container.'); - res.line(' pom.xml - generated Maven project description, could be used to open generated project in IDE or build with Maven.'); - res.line(' README.txt - this file.'); - - res.needEmptyLine = true; - - res.line('Ignite ships with CacheJdbcPojoStore, which is out-of-the-box JDBC implementation of the IgniteCacheStore '); - res.line('interface, and automatically handles all the write-through and read-through logic.'); - - res.needEmptyLine = true; - - res.line('You can use generated configuration and POJO classes as part of your application.'); - - res.needEmptyLine = true; - - res.line('Note, in case of using proprietary JDBC drivers (Oracle, IBM DB2, Microsoft SQL Server)'); - res.line('you should download them manually and copy into ./jdbc-drivers folder.'); - - return res; -}; - -/** - * Generate README.txt for jdbc folder. - * - * @param res Resulting output with generated readme. - * @returns {string} Generated content. - */ -$generatorReadme.readmeJdbc = function(res) { - if (!res) - res = $generatorCommon.builder(); - - res.line('Proprietary JDBC drivers for databases like Oracle, IBM DB2, Microsoft SQL Server are not available on Maven Central repository.'); - res.line('Drivers should be downloaded manually and copied to this folder.'); - - return res; -}; - -export default $generatorReadme; diff --git a/modules/web-console/frontend/gulpfile.babel.js/paths.js b/modules/web-console/frontend/gulpfile.babel.js/paths.js index 9134e446b069c..6ebfbad42ba84 100644 --- a/modules/web-console/frontend/gulpfile.babel.js/paths.js +++ b/modules/web-console/frontend/gulpfile.babel.js/paths.js @@ -24,17 +24,20 @@ const destDir = path.resolve('build'); const igniteModulesDir = process.env.IGNITE_MODULES ? path.join(path.normalize(process.env.IGNITE_MODULES), 'frontend') : './ignite_modules'; const igniteModulesTemp = path.resolve('ignite_modules_temp'); -const jadePaths = [ - './views/*.jade', +const jadeViewsPaths = [ './views/**/*.jade', - './app/helpers/**/*.jade', - './app/modules/states/configuration/**/*.jade', - './app/modules/sql/*.jade' + '!./views/configuration/*.jade' ]; -const resourcePaths = [ - './public/**/*.png', - './public/*.ico' +const jadeAppModulePaths = [ + './app/modules/states/configuration/**/*.jade', + './app/modules/sql/*.jade', + './views/**/*.jade', + '!./views/*.jade', + '!./views/includes/*.jade', + '!./views/settings/*.jade', + '!./views/sql/*.jade', + '!./views/templates/*.jade' ]; const jadeModulePaths = [ @@ -53,6 +56,11 @@ const appModulePaths = [ igniteModulesDir + '/**/app/data/*.json' ]; +const resourcePaths = [ + './public/**/*.png', + './public/*.ico' +]; + const resourceModulePaths = [ igniteModulesDir + '/**/images/*.png', igniteModulesDir + '/*.ico' @@ -65,10 +73,11 @@ export { igniteModulesDir, igniteModulesTemp, - jadePaths, - resourcePaths, - + jadeViewsPaths, + jadeAppModulePaths, jadeModulePaths, + + resourcePaths, resourceModulePaths, appModulePaths }; diff --git a/modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js b/modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js index b1503733a0cf9..fe8bb09616996 100644 --- a/modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js +++ b/modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js @@ -16,19 +16,28 @@ */ import gulp from 'gulp'; +import ll from 'gulp-ll'; import jade from 'gulp-jade'; -import sequence from 'gulp-sequence'; -import { jadePaths, jadeModulePaths, destDir } from '../paths'; +import { jadeViewsPaths, jadeAppModulePaths, jadeModulePaths, destDir } from '../paths'; const jadeOptions = { - basedir: './' + basedir: './', + cache: true }; -gulp.task('jade', (cb) => sequence('jade:source', 'jade:ignite_modules', cb)); +ll.tasks(['jade:views', 'jade:app', 'jade:ignite_modules']); -gulp.task('jade:source', () => - gulp.src(jadePaths) +gulp.task('jade', ['jade:views', 'jade:app', 'jade:ignite_modules']); + +gulp.task('jade:views', () => + gulp.src(jadeViewsPaths) + .pipe(jade(jadeOptions)) + .pipe(gulp.dest(destDir)) +); + +gulp.task('jade:app', () => + gulp.src(jadeAppModulePaths) .pipe(jade(jadeOptions)) .pipe(gulp.dest(destDir)) ); diff --git a/modules/web-console/frontend/gulpfile.babel.js/tasks/watch.js b/modules/web-console/frontend/gulpfile.babel.js/tasks/watch.js index c179f9c91f98d..dfaa1a3a88c35 100644 --- a/modules/web-console/frontend/gulpfile.babel.js/tasks/watch.js +++ b/modules/web-console/frontend/gulpfile.babel.js/tasks/watch.js @@ -18,13 +18,15 @@ import gulp from 'gulp'; import sequence from 'gulp-sequence'; -import { jadePaths, jadeModulePaths, resourcePaths, resourceModulePaths, appModulePaths } from '../paths'; +import { jadeViewsPaths, jadeAppModulePaths, jadeModulePaths, resourcePaths, resourceModulePaths, appModulePaths } from '../paths'; gulp.task('watch:ignite-modules', (cb) => sequence('clean:ignite-modules-temp', 'ignite:modules', cb)); // Build + watch task. gulp.task('watch', ['build'], () => { - gulp.watch(jadePaths.concat(jadeModulePaths), ['jade']); + gulp.watch(jadeViewsPaths, ['jade:views']); + gulp.watch(jadeAppModulePaths.concat('./app/helpers/**/*.jade'), ['jade:app']); + gulp.watch(jadeModulePaths.concat('./app/helpers/**/*.jade'), ['jade:ignite_modules']); gulp.watch(resourcePaths, ['copy:resource:app']); gulp.watch(resourceModulePaths, ['copy:resource:ignite_modules']); gulp.watch(appModulePaths, ['watch:ignite-modules']); diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js index e3d88b3106945..f6c2d87c2f4a2 100644 --- a/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js +++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js @@ -43,7 +43,7 @@ try { } export default () => { - const assetsLoader = development ? 'url-loader' : 'file-loader'; + const assetsLoader = 'file-loader'; return { cache: true, diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js index 229760e8af067..1550dfad54775 100644 --- a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js +++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js @@ -20,6 +20,10 @@ import webpack from 'webpack'; import {destDir, rootDir, srcDir} from '../../paths'; +const devServerHost = 'localhost'; +const devServerPort = 9000; +const devServerUrl = `http://${devServerHost}:${devServerPort}/`; + export default () => { const plugins = [ new webpack.HotModuleReplacementPlugin() @@ -27,9 +31,12 @@ export default () => { return { entry: { - webpack: 'webpack-dev-server/client?http://localhost:9000/', + webpack: `webpack-dev-server/client?${devServerUrl}`, app: [path.join(srcDir, 'app.js'), 'webpack/hot/only-dev-server'] }, + output: { + publicPath: devServerUrl + }, context: rootDir, debug: true, devtool: 'source-map', @@ -60,10 +67,12 @@ export default () => { aggregateTimeout: 1000, poll: 2000 }, - stats: {colors: true}, - port: 9000 + stats: { + colors: true, + chunks: false + }, + port: devServerPort }, - stats: {colors: true}, plugins }; }; diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/test.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/test.js new file mode 100644 index 0000000000000..1c37196838ae4 --- /dev/null +++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/test.js @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import webpack from 'webpack'; + +const NODE_ENV = process.env.NODE_ENV || 'production'; + +export default () => { + + return { + cache: true, + node: { + fs: 'empty' + }, + + module: { + preLoaders: null + }, + + // Entry points. + entry: null, + + // Output system. + output: null, + eslint: null, + + // Load plugins. + plugins: [ + new webpack.ProvidePlugin({ + $: 'jquery', + jQuery: 'jquery', + _: 'lodash', + nv: 'nvd3' + }), + new webpack.DefinePlugin({NODE_ENV: JSON.stringify(NODE_ENV)}) + ] + }; +}; diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/index.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/index.js index 6682f9cd78f60..9b344dd6e1cab 100644 --- a/modules/web-console/frontend/gulpfile.babel.js/webpack/index.js +++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/index.js @@ -19,13 +19,15 @@ import _ from 'lodash'; import commonConfig from './common'; import devConfig from './environments/development'; import prodConfig from './environments/production'; +import testConfig from './environments/test'; const env = process.env.NODE_ENV || 'production'; // Config by environments. const configs = { production: prodConfig, - development: devConfig + development: devConfig, + test: testConfig }; // Load config file by environment diff --git a/modules/web-console/frontend/package.json b/modules/web-console/frontend/package.json index 7ea289346cddf..fe8c2953e048e 100644 --- a/modules/web-console/frontend/package.json +++ b/modules/web-console/frontend/package.json @@ -5,9 +5,9 @@ "private": true, "scripts": { "dev": "cross-env NODE_ENV=development gulp watch", - "build": "cross-env NODE_ENV=production gulp build", - "test": "karma start ./test/karma.conf.js", - "eslint": "eslint --format node_modules/eslint-friendly-formatter gulpfile.babel.js/ app/ controllers/ generator/ ignite_modules/ ignite_modules_temp/ -- --eff-by-issue" + "build": "cross-env NODE_ENV=production gulp build --no-ll", + "test": "cross-env NODE_ENV=test karma start ./test/karma.conf.js", + "eslint": "eslint --format node_modules/eslint-friendly-formatter gulpfile.babel.js/ app/ controllers/ ignite_modules/ ignite_modules_temp/ -- --eff-by-issue" }, "author": "", "contributors": [ @@ -57,7 +57,6 @@ "jszip": "^3.0.0", "lodash": "^4.8.2", "nvd3": "^1.8.3", - "query-command-supported": "^1.0.0", "raleway-webfont": "^3.0.1", "roboto-font": "^0.1.0", "socket.io-client": "^1.4.6", @@ -67,7 +66,7 @@ "assets-webpack-plugin": "^3.2.0", "autoprefixer-core": "^6.0.1", "babel-core": "^6.7.6", - "babel-eslint": "^6.0.4", + "babel-eslint": "^7.0.0", "babel-loader": "^6.2.4", "babel-plugin-add-module-exports": "^0.2.1", "babel-plugin-transform-builtin-extend": "^1.1.0", @@ -79,16 +78,17 @@ "chai": "^3.5.0", "cross-env": "^1.0.7", "css-loader": "^0.23.0", - "eslint": "^2.9.0", + "eslint": "^3.0.0", "eslint-friendly-formatter": "^2.0.5", "eslint-loader": "^1.0.0", "expose-loader": "^0.7.1", "extract-text-webpack-plugin": "^1.0.1", "file-loader": "^0.9.0", "gulp": "^3.9.1", - "gulp-eslint": "^2.0.0", + "gulp-eslint": "^3.0.0", "gulp-inject": "^4.0.0", "gulp-jade": "^1.1.0", + "gulp-ll": "^1.0.4", "gulp-rimraf": "^0.2.0", "gulp-sequence": "^0.4.1", "gulp-util": "^3.0.7", @@ -102,6 +102,7 @@ "karma-babel-preprocessor": "^6.0.1", "karma-jasmine": "^1.0.2", "karma-mocha": "^1.0.1", + "karma-mocha-reporter": "^2.2.0", "karma-phantomjs-launcher": "^1.0.0", "karma-teamcity-reporter": "^1.0.0", "karma-webpack": "^1.7.0", @@ -115,7 +116,6 @@ "require-dir": "^0.3.0", "resolve-url-loader": "^1.4.3", "sass-loader": "^3.1.1", - "should": "^9.0.2", "style-loader": "^0.13.1", "url": "^0.11.0", "url-loader": "^0.5.6", diff --git a/modules/web-console/frontend/public/stylesheets/style.scss b/modules/web-console/frontend/public/stylesheets/style.scss index 07845607b060c..0f8f49a360b32 100644 --- a/modules/web-console/frontend/public/stylesheets/style.scss +++ b/modules/web-console/frontend/public/stylesheets/style.scss @@ -82,6 +82,11 @@ hr { vertical-align: middle; } +.table .ui-grid-settings { + float: left; + padding-right: 10px; +} + ul.navbar-nav, .sidebar-nav { li.active > a { color: $link-color; @@ -359,6 +364,13 @@ h1, h2, h3, h4, h5, h6 { margin-right: -2px; } +// Modal icon +.modal-header h4 > i.fa { + cursor: default; + float: left; + line-height: $modal-title-line-height; +} + .modal .modal-dialog { width: 650px; } @@ -623,7 +635,7 @@ button.form-control { margin-top: 30px; } - .btn-group { + .btn-group:last-of-type { margin-right: 0; } @@ -665,6 +677,16 @@ button.form-control { border-top: 1px solid $ignite-border-color; padding: 10px 10px; + + input[type="checkbox"] { + line-height: 20px; + margin-right: 5px; + } + + label { + line-height: 20px !important; + vertical-align: middle; + } } .sql-result { @@ -686,6 +708,16 @@ button.form-control { .total { padding: 10px 10px; + + input[type="checkbox"] { + line-height: 20px; + margin-right: 5px; + } + + label { + line-height: 20px !important; + vertical-align: middle; + } } .table { @@ -1318,12 +1350,6 @@ label { margin: 0 5px; } -.fieldButton { - float: right; - margin-left: 5px; - margin-right: 0; -} - .fa { cursor: pointer; } @@ -1639,13 +1665,18 @@ th[st-sort] { z-index: 900; a { + color: $input-color-placeholder; background-color: white; margin-left: 5px; font-size: 0.9em; } - .inactive { - color: $input-color-placeholder; + a + a { + margin-left: 10px + } + + a.active { + color: $brand-primary; } } @@ -2170,3 +2201,29 @@ html,body,.splash-screen { padding: 0; } } + +// Fix for incorrect tooltip placement after fast show|hide. +.tooltip.ng-leave { + transition: none !important; /* Disable transitions. */ + animation: none 0s !important; /* Disable keyframe animations. */ +} + +// Fix for incorrect dropdown placement. +.select.dropdown-menu.ng-leave { + transition: none !important; /* Disable transitions. */ + animation: none 0s !important; /* Disable keyframe animations. */ +} +.disable-animations { + // Use this for transitions + &.ng-enter, + &.ng-leave, + &.ng-animate { + -webkit-transition: none !important; + transition: none !important; + } + // Use this for keyframe animations + &.ng-animate { + -webkit-animation: none 0s; + animation: none 0s; + } +} diff --git a/modules/web-console/frontend/test/karma.conf.babel.js b/modules/web-console/frontend/test/karma.conf.babel.js new file mode 100644 index 0000000000000..76a0ba0a240a5 --- /dev/null +++ b/modules/web-console/frontend/test/karma.conf.babel.js @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import webpackConfig from '../gulpfile.babel.js/webpack'; +import path from 'path'; + +const basePath = path.resolve('./'); + +export default (config) => { + config.set({ + // Base path that will be used to resolve all patterns (eg. files, exclude). + basePath: basePath, + + // Frameworks to use available frameworks: https://npmjs.org/browse/keyword/karma-adapter + frameworks: ['mocha'], + + // List of files / patterns to load in the browser. + files: [ + 'test/**/*.test.js' + ], + + plugins: [ + require('karma-phantomjs-launcher'), + require('karma-teamcity-reporter'), + require('karma-mocha-reporter'), + require('karma-webpack'), + require('karma-mocha') + ], + + // Preprocess matching files before serving them to the browser + // available preprocessors: https://npmjs.org/browse/keyword/karma-preprocessor. + preprocessors: { + 'test/**/*.js': ['webpack'] + }, + webpack: webpackConfig, + + webpackMiddleware: { + noInfo: true + }, + + // Test results reporter to use + // possible values: 'dots', 'progress' + // available reporters: https://npmjs.org/browse/keyword/karma-reporter. + reporters: ['mocha'], + + // web server port + port: 9876, + + // enable / disable colors in the output (reporters and logs) + colors: true, + + // level of logging + // possible values: config.LOG_DISABLE || config.LOG_ERROR || config.LOG_WARN || config.LOG_INFO || config.LOG_DEBUG + logLevel: config.LOG_INFO, + + // enable / disable watching file and executing tests whenever any file changes + autoWatch: true, + + // start these browsers + // available browser launchers: https://npmjs.org/browse/keyword/karma-launcher + browsers: ['PhantomJS'], + + // Continuous Integration mode + // if true, Karma captures browsers, runs the tests and exits + singleRun: true, + + // Concurrency level + // how many browser should be started simultaneous + concurrency: Infinity, + + client: { + mocha: { + ui: 'tdd' + } + } + }); +}; diff --git a/modules/web-console/frontend/test/karma.conf.js b/modules/web-console/frontend/test/karma.conf.js index e13ba00031908..a8b1b11fcb47d 100644 --- a/modules/web-console/frontend/test/karma.conf.js +++ b/modules/web-console/frontend/test/karma.conf.js @@ -15,99 +15,5 @@ * limitations under the License. */ -const path = require('path'); -const webpack = require('webpack'); - -const basePath = path.resolve('./'); - -module.exports = function(config) { - config.set({ - // Base path that will be used to resolve all patterns (eg. files, exclude). - basePath: basePath, - - // Frameworks to use available frameworks: https://npmjs.org/browse/keyword/karma-adapter - frameworks: ['mocha'], - - // List of files / patterns to load in the browser. - files: [ - 'test/**/*.test.js' - ], - - plugins: [ - require('karma-phantomjs-launcher'), - require('karma-teamcity-reporter'), - require('karma-webpack'), - require('karma-mocha') - ], - - // Preprocess matching files before serving them to the browser - // available preprocessors: https://npmjs.org/browse/keyword/karma-preprocessor. - preprocessors: { - 'test/**/*.js': ['webpack'] - }, - - webpack: { - module: { - loaders: [ - { - test: /\.json$/, - loader: 'json' - }, - { - test: /\.js$/, - loader: 'babel', - exclude: /node_modules/ - } - ] - }, - resolve: { - extensions: ["", ".js"] - }, - plugins: [ - new webpack.ProvidePlugin({ - _: 'lodash' - }) - ] - }, - - webpackMiddleware: { - noInfo: true - }, - - // Test results reporter to use - // possible values: 'dots', 'progress' - // available reporters: https://npmjs.org/browse/keyword/karma-reporter. - reporters: ['teamcity'], - - // web server port - port: 9876, - - // enable / disable colors in the output (reporters and logs) - colors: true, - - // level of logging - // possible values: config.LOG_DISABLE || config.LOG_ERROR || config.LOG_WARN || config.LOG_INFO || config.LOG_DEBUG - logLevel: config.LOG_INFO, - - // enable / disable watching file and executing tests whenever any file changes - autoWatch: true, - - // start these browsers - // available browser launchers: https://npmjs.org/browse/keyword/karma-launcher - browsers: ['PhantomJS'], - - // Continuous Integration mode - // if true, Karma captures browsers, runs the tests and exits - singleRun: true, - - // Concurrency level - // how many browser should be started simultaneous - concurrency: Infinity, - - client: { - mocha: { - ui: 'tdd' - } - } - }); -}; +require('babel-core/register'); +module.exports = require('./karma.conf.babel.js').default; diff --git a/modules/web-console/frontend/test/unit/JavaTransformer.test.js b/modules/web-console/frontend/test/unit/JavaTransformer.test.js new file mode 100644 index 0000000000000..3f390003722da --- /dev/null +++ b/modules/web-console/frontend/test/unit/JavaTransformer.test.js @@ -0,0 +1,57 @@ +/* + * 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. + */ + +import JavaTypes from '../../app/services/JavaTypes.service.js'; + +import generator from '../../app/modules/configuration/generator/ConfigurationGenerator'; +import transformer from '../../app/modules/configuration/generator/JavaTransformer.service'; + +import { assert } from 'chai'; + +suite.skip('JavaTransformerTestsSuite', () => { + test('AtomicConfiguration', () => { + const ConfigurationGenerator = generator[1](); + const JavaTransformer = transformer[1][2](JavaTypes[1](), ConfigurationGenerator); + + const acfg = { + atomicSequenceReserveSize: 1001, + backups: 1, + cacheMode: 'LOCAL' + }; + + const bean = ConfigurationGenerator.clusterAtomics(acfg); + + console.log(JavaTransformer.generateSection(bean)); + }); + + test('IgniteConfiguration', () => { + const ConfigurationGenerator = generator[1](); + const JavaTransformer = transformer[1][2](JavaTypes[1](), ConfigurationGenerator); + + const clusterCfg = { + atomics: { + atomicSequenceReserveSize: 1001, + backups: 1, + cacheMode: 'LOCAL' + } + }; + + const bean = ConfigurationGenerator.igniteConfiguration(clusterCfg); + + console.log(JavaTransformer.toClassFile(bean, 'config', 'ServerConfigurationFactory', null)); + }); +}); diff --git a/modules/web-console/frontend/test/unit/JavaTypes.test.js b/modules/web-console/frontend/test/unit/JavaTypes.test.js index 7b12168558c85..2df8c6ac2426e 100644 --- a/modules/web-console/frontend/test/unit/JavaTypes.test.js +++ b/modules/web-console/frontend/test/unit/JavaTypes.test.js @@ -17,7 +17,11 @@ import JavaTypes from '../../app/services/JavaTypes.service.js'; -const INSTANCE = new JavaTypes(); +import ClusterDflts from '../../app/modules/configuration/generator/defaults/cluster.provider'; +import CacheDflts from '../../app/modules/configuration/generator/defaults/cache.provider'; +import IgfsDflts from '../../app/modules/configuration/generator/defaults/igfs.provider'; + +const INSTANCE = new JavaTypes((new ClusterDflts()).$get[0](), (new CacheDflts()).$get[0](), (new IgfsDflts()).$get[0]()); import { assert } from 'chai'; @@ -43,6 +47,23 @@ suite('JavaTypesTestsSuite', () => { assert.equal(INSTANCE.nonBuiltInClass('my.package.CustomClass'), true); }); + test('nonEnum', () => { + assert.equal(INSTANCE.nonEnum('org.apache.ignite.cache.CacheMode'), false); + assert.equal(INSTANCE.nonEnum('org.apache.ignite.transactions.TransactionConcurrency'), false); + assert.equal(INSTANCE.nonEnum('org.apache.ignite.cache.CacheWriteSynchronizationMode'), false); + assert.equal(INSTANCE.nonEnum('org.apache.ignite.igfs.IgfsIpcEndpointType'), false); + assert.equal(INSTANCE.nonEnum('java.io.Serializable'), true); + assert.equal(INSTANCE.nonEnum('BigDecimal'), true); + }); + + test('shortClassName', () => { + assert.equal(INSTANCE.shortClassName('java.math.BigDecimal'), 'BigDecimal'); + assert.equal(INSTANCE.shortClassName('int'), 'int'); + assert.equal(INSTANCE.shortClassName('java.lang.Integer'), 'Integer'); + assert.equal(INSTANCE.shortClassName('java.util.UUID'), 'UUID'); + assert.equal(INSTANCE.shortClassName('Abstract'), 'Abstract'); + }); + test('fullClassName', () => { assert.equal(INSTANCE.fullClassName('BigDecimal'), 'java.math.BigDecimal'); }); diff --git a/modules/web-console/frontend/test/unit/SharpTransformer.test.js b/modules/web-console/frontend/test/unit/SharpTransformer.test.js new file mode 100644 index 0000000000000..20de266af6bb1 --- /dev/null +++ b/modules/web-console/frontend/test/unit/SharpTransformer.test.js @@ -0,0 +1,55 @@ +/* + * 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. + */ + +import generator from '../../app/modules/configuration/generator/PlatformGenerator'; +import transformer from '../../app/modules/configuration/generator/SharpTransformer.service'; + +import { assert } from 'chai'; + +suite.skip('SharpTransformerTestsSuite', () => { + test('AtomicConfiguration', () => { + const PlatformGenerator = generator[1](); + const SharpTransformer = transformer[1](PlatformGenerator); + + const acfg = { + atomicSequenceReserveSize: 1001, + backups: 1, + cacheMode: 'LOCAL' + }; + + const bean = PlatformGenerator.clusterAtomics(acfg); + + console.log(SharpTransformer.generateSection(bean)); + }); + + test('IgniteConfiguration', () => { + const PlatformGenerator = generator[1](); + const SharpTransformer = transformer[1](PlatformGenerator); + + const clusterCfg = { + atomics: { + atomicSequenceReserveSize: 1001, + backups: 1, + cacheMode: 'LOCAL' + } + }; + + const bean = PlatformGenerator.igniteConfiguration(clusterCfg); + + console.log(SharpTransformer.toClassFile(bean, 'config', 'ServerConfigurationFactory', null)); + }); +}); diff --git a/modules/web-console/frontend/test/unit/SpringTransformer.test.js b/modules/web-console/frontend/test/unit/SpringTransformer.test.js new file mode 100644 index 0000000000000..7998f666befe4 --- /dev/null +++ b/modules/web-console/frontend/test/unit/SpringTransformer.test.js @@ -0,0 +1,57 @@ +/* + * 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. + */ + +import JavaTypes from '../../app/services/JavaTypes.service.js'; + +import generator from '../../app/modules/configuration/generator/ConfigurationGenerator'; +import transformer from '../../app/modules/configuration/generator/SpringTransformer.service'; + +import { assert } from 'chai'; + +suite.skip('SpringTransformerTestsSuite', () => { + test('AtomicConfiguration', () => { + const ConfigurationGenerator = generator[1](); + const SpringTransformer = transformer[1][2](JavaTypes[1](), ConfigurationGenerator); + + const acfg = { + atomicSequenceReserveSize: 1001, + backups: 1, + cacheMode: 'LOCAL' + }; + + const bean = ConfigurationGenerator.clusterAtomics(acfg); + + console.log(SpringTransformer.generateSection(bean)); + }); + + test('IgniteConfiguration', () => { + const ConfigurationGenerator = generator[1](); + const SpringTransformer = transformer[1][2](JavaTypes[1](), ConfigurationGenerator); + + const cfg = { + atomics: { + atomicSequenceReserveSize: 1001, + backups: 1, + cacheMode: 'LOCAL' + } + }; + + const bean = ConfigurationGenerator.igniteConfiguration(cfg); + + console.log(SpringTransformer.generate(bean)); + }); +}); diff --git a/modules/web-console/frontend/test/unit/SqlTypes.test.js b/modules/web-console/frontend/test/unit/SqlTypes.test.js index 3cfaafc1212f3..2d54bdfdb1f25 100644 --- a/modules/web-console/frontend/test/unit/SqlTypes.test.js +++ b/modules/web-console/frontend/test/unit/SqlTypes.test.js @@ -15,23 +15,6 @@ * limitations under the License. */ -/* - * 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. - */ - import SqlTypes from '../../app/services/SqlTypes.service.js'; const INSTANCE = new SqlTypes(); diff --git a/modules/web-console/frontend/test/unit/Version.test.js b/modules/web-console/frontend/test/unit/Version.test.js new file mode 100644 index 0000000000000..a67fde8b2e13d --- /dev/null +++ b/modules/web-console/frontend/test/unit/Version.test.js @@ -0,0 +1,82 @@ +/* + * 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. + */ + +import VersionService from '../../app/modules/configuration/Version.service.js'; + +const INSTANCE = new VersionService(); + +import { assert } from 'chai'; + +suite('VersionServiceTestsSuite', () => { + test('Check patch version', () => { + assert.equal(INSTANCE.compare('1.7.2', '1.7.1'), 1); + }); + + test('Check minor version', () => { + assert.equal(INSTANCE.compare('1.8.1', '1.7.1'), 1); + }); + + test('Check major version', () => { + assert.equal(INSTANCE.compare('2.7.1', '1.7.1'), 1); + }); + + test('Version a > b', () => { + assert.equal(INSTANCE.compare('1.7.0', '1.5.0'), 1); + }); + + test('Version a = b', () => { + assert.equal(INSTANCE.compare('1.7.0', '1.7.0'), 0); + }); + + test('Version a < b', () => { + assert.equal(INSTANCE.compare('1.5.1', '1.5.2'), -1); + }); + + test('Check since call', () => { + assert.equal(INSTANCE.since('1.6.0', '1.5.0'), true); + }); + + test('Check wrong since call', () => { + assert.equal(INSTANCE.since('1.3.0', '1.5.0'), false); + }); + + test('Parse 1.7.0-SNAPSHOT', () => { + const version = INSTANCE.parse('1.7.0-SNAPSHOT'); + assert.equal(version.major, 1); + assert.equal(version.minor, 7); + assert.equal(version.maintenance, 0); + assert.equal(version.stage, 'SNAPSHOT'); + assert.equal(version.revTs, 0); + assert.isNull(version.revHash); + }); + + test('Parse strip -DEV 1.7.0-DEV', () => { + const version = INSTANCE.parse('1.7.0-DEV'); + assert.equal(version.major, 1); + assert.equal(version.minor, 7); + assert.equal(version.maintenance, 0); + assert.equal(version.stage, ''); + }); + + test('Parse strip -n/a 1.7.0-n/a', () => { + const version = INSTANCE.parse('1.7.0-n/a'); + assert.equal(version.major, 1); + assert.equal(version.minor, 7); + assert.equal(version.maintenance, 0); + assert.equal(version.stage, ''); + }); +}); diff --git a/modules/web-console/frontend/test/unit/defaultName.filter.test.js b/modules/web-console/frontend/test/unit/defaultName.filter.test.js new file mode 100644 index 0000000000000..5f282900b4f97 --- /dev/null +++ b/modules/web-console/frontend/test/unit/defaultName.filter.test.js @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import defaultName from '../../app/filters/default-name.filter'; + +import { assert } from 'chai'; + +const INSTANCE = defaultName[0](); + +suite('defaultName', () => { + test('defaultName filter', () => { + assert.equal(INSTANCE(''), ''); + assert.equal(INSTANCE(null), ''); + assert.equal(INSTANCE(undefined), ''); + assert.equal(INSTANCE('', false), ''); + assert.equal(INSTANCE(null, false), ''); + assert.equal(INSTANCE(undefined, false), ''); + assert.equal(INSTANCE('', true), '<default>'); + assert.equal(INSTANCE(null, true), '<default>'); + assert.equal(INSTANCE(undefined, true), '<default>'); + assert.equal(INSTANCE("name", false), 'name'); + assert.equal(INSTANCE("name", true), 'name'); + }); +}); diff --git a/modules/web-console/frontend/views/configuration/caches.jade b/modules/web-console/frontend/views/configuration/caches.jade index a1218ec7b83f2..74d5505bc0444 100644 --- a/modules/web-console/frontend/views/configuration/caches.jade +++ b/modules/web-console/frontend/views/configuration/caches.jade @@ -45,9 +45,10 @@ include ../../app/helpers/jade/mixins.jade div(ng-show='ui.expanded') include ../../app/modules/states/configuration/caches/concurrency.jade + include ../../app/modules/states/configuration/caches/near-cache-client.jade + include ../../app/modules/states/configuration/caches/near-cache-server.jade include ../../app/modules/states/configuration/caches/node-filter.jade include ../../app/modules/states/configuration/caches/rebalance.jade - include ../../app/modules/states/configuration/caches/server-near-cache.jade include ../../app/modules/states/configuration/caches/statistics.jade +advanced-options-toggle-default diff --git a/modules/web-console/frontend/views/configuration/clusters.jade b/modules/web-console/frontend/views/configuration/clusters.jade index b79b1ea13e86c..8a3a19408e9f8 100644 --- a/modules/web-console/frontend/views/configuration/clusters.jade +++ b/modules/web-console/frontend/views/configuration/clusters.jade @@ -41,10 +41,10 @@ include ../../app/helpers/jade/mixins.jade +advanced-options-toggle-default div(ng-show='ui.expanded') - include ../../app/modules/states/configuration/clusters/atomic.jade include ../../app/modules/states/configuration/clusters/binary.jade include ../../app/modules/states/configuration/clusters/cache-key-cfg.jade + include ../../app/modules/states/configuration/clusters/checkpoint.jade include ../../app/modules/states/configuration/clusters/collision.jade include ../../app/modules/states/configuration/clusters/communication.jade include ../../app/modules/states/configuration/clusters/connector.jade @@ -53,6 +53,7 @@ include ../../app/helpers/jade/mixins.jade include ../../app/modules/states/configuration/clusters/events.jade include ../../app/modules/states/configuration/clusters/failover.jade include ../../app/modules/states/configuration/clusters/igfs.jade + include ../../app/modules/states/configuration/clusters/load-balancing.jade include ../../app/modules/states/configuration/clusters/logger.jade include ../../app/modules/states/configuration/clusters/marshaller.jade include ../../app/modules/states/configuration/clusters/metrics.jade diff --git a/modules/web-console/frontend/views/configuration/domains-import.jade b/modules/web-console/frontend/views/configuration/domains-import.jade index e2eaf97b958c5..99014bce6c1d2 100644 --- a/modules/web-console/frontend/views/configuration/domains-import.jade +++ b/modules/web-console/frontend/views/configuration/domains-import.jade @@ -28,8 +28,10 @@ mixin td-ellipses-lbl(w, lbl) .modal-content(ignite-loading='importDomainFromDb' ignite-loading-text='{{importDomain.loadingOptions.text}}') #errors-container.modal-header.header button.close(ng-click='$hide()' aria-hidden='true') × - h4.modal-title(ng-if='!importDomain.demo') Import domain models from database - h4.modal-title(ng-if='importDomain.demo') Import domain models from demo database + h4.modal-title() + i.fa.fa-database + span(ng-if='!importDomain.demo') Import domain models from database + span(ng-if='importDomain.demo') Import domain models from demo database .modal-body .import-domain-model-wizard-page(ng-if='importDomain.action == "drivers" && !importDomain.jdbcDriversNotFound') .import-domain-model-wizard-page(ng-if='importDomain.action == "drivers" && importDomain.jdbcDriversNotFound') @@ -144,14 +146,16 @@ mixin td-ellipses-lbl(w, lbl) -var form = 'optionsForm' form.form-horizontal(name=form novalidate) - .settings-row.settings-row_small-label - +java-package('Package:', 'ui.packageName', '"domainPackageName"', true, true, 'Package that will be used for POJOs generation')(data-container='.modal-domain-import') .settings-row +checkbox('Use Java built-in types for keys', 'ui.builtinKeys', '"domainBuiltinKeys"', 'Use Java built-in types like "Integer", "Long", "String" instead of POJO generation in case when table primary key contains only one field') .settings-row +checkbox('Use primitive types for NOT NULL table columns', 'ui.usePrimitives', '"domainUsePrimitives"', 'Use primitive types like "int", "long", "double" for POJOs fields generation in case of NOT NULL columns') .settings-row +checkbox('Generate aliases for query fields', 'ui.generateAliases', '"domainGenerateAliases"', 'Generate aliases for query fields with database field names when database field name differ from Java field name') + .settings-row + +checkbox('Generate POJO classes', 'ui.generatePojo', '"domainGeneratePojo"', 'If selected then POJO classes will be generated from database tables') + .settings-row.settings-row_small-label(ng-show='ui.generatePojo') + +java-package('Package:', 'ui.packageName', '"domainPackageName"', true, true, 'Package that will be used for POJOs generation')(data-container='.modal-domain-import') .settings-row.settings-row_small-label +ignite-form-field-dropdown('Clusters:', 'ui.generatedCachesClusters', '"generatedCachesClusters"', false, false, true, 'Choose clusters for generated caches', '', 'clusters', @@ -160,4 +164,4 @@ mixin td-ellipses-lbl(w, lbl) .modal-footer label(ng-hide='importDomain.action == "drivers" || (importDomain.action == "connect" && importDomain.demo)').labelField {{importDomain.info}} a.btn.btn-primary(ng-hide='importDomain.action == "drivers" || importDomain.action == "connect"' ng-click='importDomainPrev()' bs-tooltip='' data-title='{{prevTooltipText()}}' data-placement='bottom') Prev - a.btn.btn-primary(ng-click='importDomainNext()' ng-disabled='!importDomainNextAvailable()' bs-tooltip='' data-title='{{nextTooltipText()}}' data-placement='bottom') {{importDomain.button}} + a.btn.btn-primary(ng-click='importDomainNext(optionsForm)' ng-disabled='!importDomainNextAvailable()' bs-tooltip='' data-title='{{nextTooltipText()}}' data-placement='bottom') {{importDomain.button}} diff --git a/modules/web-console/frontend/views/configuration/summary-project-structure.jade b/modules/web-console/frontend/views/configuration/summary-project-structure.jade index aa094376be658..29d4538c7cb90 100644 --- a/modules/web-console/frontend/views/configuration/summary-project-structure.jade +++ b/modules/web-console/frontend/views/configuration/summary-project-structure.jade @@ -19,7 +19,7 @@ button.close(id='summary-project-structure-close' ng-click='$hide()') × .popover-content treecontrol.tree-classic(tree-model='projectStructure' options='projectStructureOptions' expanded-nodes='projectStructureExpanded') - span(ng-switch='' on='node.type') + span(ng-switch='node.type') span(ng-switch-when='folder') label {{node.name}} span(ng-switch-when='file') diff --git a/modules/web-console/frontend/views/configuration/summary.jade b/modules/web-console/frontend/views/configuration/summary.jade index 0d30df86eafc6..d041fea0133fb 100644 --- a/modules/web-console/frontend/views/configuration/summary.jade +++ b/modules/web-console/frontend/views/configuration/summary.jade @@ -53,16 +53,16 @@ mixin hard-link(ref, txt) label Server .panel-collapse(id='server' role='tabpanel' bs-collapse-target) - ignite-ui-ace-tabs.summary-tabs + .summary-tabs(ignite-ui-ace-tabs) div(bs-tabs data-bs-active-pane="tabsServer.activeTab" template='configuration/summary-tabs.html') div(bs-pane title='XML') - ignite-ui-ace-xml(ng-if='tabsServer.activeTab == 0 || tabsServer.init[0]' ng-init='tabsServer.init[0] = true' data-master='cluster' data-no-deep-watch) + ignite-ui-ace-spring(ng-if='tabsServer.activeTab == 0 || tabsServer.init[0]' ng-init='tabsServer.init[0] = true' data-master='cluster' data-generator='igniteConfiguration' data-no-deep-watch) div(bs-pane title='Java') - ignite-ui-ace-java(ng-if='tabsServer.activeTab == 1 || tabsServer.init[1]' ng-init='tabsServer.init[1] = true' data-master='cluster' data-no-deep-watch) + ignite-ui-ace-java(ng-if='tabsServer.activeTab == 1 || tabsServer.init[1]' ng-init='tabsServer.init[1] = true' data-master='cluster' data-generator='igniteConfiguration' data-no-deep-watch) div(bs-pane title='POM') - ignite-ui-ace-pom(ng-if='tabsServer.activeTab == 2 || tabsServer.init[2]' ng-init='tabsServer.init[2] = true' data-cluster='cluster' data-no-deep-watch) + ignite-ui-ace-pom(ng-if='tabsServer.activeTab == 2 || tabsServer.init[2]' ng-init='tabsServer.init[2] = true' data-cluster='cluster' data-generator='igniteConfiguration' data-no-deep-watch) div(bs-pane title='Dockerfile') - ignite-ui-ace-docker(ng-if='tabsServer.activeTab == 3 || tabsServer.init[3]' ng-init='tabsServer.init[3] = true' data-cluster='cluster' data-no-deep-watch ng-model='ctrl.data.docker') + ignite-ui-ace-docker(ng-if='tabsServer.activeTab == 3 || tabsServer.init[3]' ng-init='tabsServer.init[3] = true' data-cluster='cluster' data-generator='igniteConfiguration' data-no-deep-watch ng-model='ctrl.data.docker') .panel.panel-default .panel-heading(role='tab' bs-collapse-toggle) @@ -70,53 +70,14 @@ mixin hard-link(ref, txt) label Client .panel-collapse(id='client' role='tabpanel' bs-collapse-target) - -var form = 'clientForm' - form(name=form novalidate) - -var nearCfg = 'ctrl.cluster.clientNearCfg' - -var nearCfgEvictionPolicy = nearCfg + '.nearEvictionPolicy[' + nearCfg + '.nearEvictionPolicy.kind]' - - .group-content - .settings-row(ng-if='true') - .col-xs-8.col-sm-8.col-md-7 - +ignite-form-field-number('Near cache start size:', nearCfg + '.nearStartSize', '"nearStartSize"', false, false, '375000', false, false, false, 'Initial cache size for near cache which will be used to pre-create internal hash table after start') - - .settings-row(ng-if='true') - .col-xs-8.col-sm-8.col-md-7 - +ignite-form-field-dropdown('Near cache eviction policy', nearCfg + '.nearEvictionPolicy.kind', '"evictionPolicies"', false, false, false, 'Not set', false, '[\ - {value: "LRU", label: "LRU"},\ - {value: "FIFO", label: "FIFO"},\ - {value: "SORTED", label: "Sorted"},\ - {value: undefined, label: "Not set"}\ - ]', 'Near cache eviction policy') - - span(ng-if='#{nearCfg}.nearEvictionPolicy.kind') - a.customize( - ng-show='ctrl.__form.expanded' - ng-click='ctrl.__form.expanded = false' - ) Hide settings - a.customize( - ng-hide='ctrl.__form.expanded' - ng-click='ctrl.__form.expanded = true' - ) Show settings - - .settings-row - .panel-details.col-xs-12.col-sm-12.col-md-7(ng-if='ctrl.__form.expanded && #{nearCfg}.nearEvictionPolicy.kind') - .details-row - +ignite-form-field-number('Batch size:', nearCfgEvictionPolicy + '.batchSize', '"batchSize"', false, false, '1', false, false, false, 'Number of entries to remove on shrink') - - .details-row - +ignite-form-field-number('Max memory size:', nearCfgEvictionPolicy + '.maxMemorySize', '"maxMemorySize"', false, false, '0', false, false, false, 'Maximum allowed cache size in bytes') - - .details-row - +ignite-form-field-number('Max size:', nearCfgEvictionPolicy + '.maxSize', '"maxSize"', false, false, '100000', false, false, false, 'Maximum allowed size of cache before entry will start getting evicted') - .summary-tabs(ignite-ui-ace-tabs) div(bs-tabs data-bs-active-pane="tabsClient.activeTab" template='configuration/summary-tabs.html') div(bs-pane title='XML') - ignite-ui-ace-xml(ng-if='tabsClient.activeTab == 0 || tabsClient.init[0]' ng-init='tabsClient.init[0] = true' data-master='cluster' data-no-deep-watch data-cluster-cfg='#{nearCfg}') + ignite-ui-ace-spring(ng-if='tabsClient.activeTab == 0 || tabsClient.init[0]' ng-init='tabsClient.init[0] = true' data-master='cluster' data-generator='igniteConfiguration' data-client='true' data-no-deep-watch) div(bs-pane title='Java') - ignite-ui-ace-java(ng-if='tabsClient.activeTab == 1 || tabsClient.init[1]' ng-init='tabsClient.init[1] = true' data-master='cluster' data-no-deep-watch data-cluster-cfg='#{nearCfg}') + ignite-ui-ace-java(ng-if='tabsClient.activeTab == 1 || tabsClient.init[1]' ng-init='tabsClient.init[1] = true' data-master='cluster' data-generator='igniteConfiguration' data-client='true' data-no-deep-watch) div(bs-pane title='POM') - ignite-ui-ace-pom(ng-if='tabsClient.activeTab == 2 || tabsClient.init[2]' ng-init='tabsClient.init[2] = true' data-cluster='cluster' data-no-deep-watch) + ignite-ui-ace-pom(ng-if='tabsClient.activeTab == 2 || tabsClient.init[2]' ng-init='tabsClient.init[2] = true' data-cluster='cluster' data-generator='igniteConfiguration' data-client='true' data-no-deep-watch) div(bs-pane title='POJO' ng-if='cluster | hasPojo') ignite-ui-ace-pojos(ng-if='tabsClient.activeTab == 3 || tabsClient.init[3]' ng-init='tabsClient.init[3] = true' data-cluster='cluster' data-no-deep-watch ng-model='ctrl.data.pojos') + diff --git a/modules/web-console/frontend/views/sql/cache-metadata.jade b/modules/web-console/frontend/views/sql/cache-metadata.jade index 450c178f99ca0..385960a056421 100644 --- a/modules/web-console/frontend/views/sql/cache-metadata.jade +++ b/modules/web-console/frontend/views/sql/cache-metadata.jade @@ -21,7 +21,7 @@ input.form-control(type='text' ng-model='metaFilter' placeholder='Filter metadata...') .popover-content(ng-if='metadata && metadata.length > 0') treecontrol.tree-classic(tree-model='metadata' options='metaOptions' filter-expression='metaFilter') - span(ng-switch='' on='node.type') + span(ng-switch='node.type') span(ng-switch-when='type' ng-dblclick='dblclickMetadata(paragraph, node)') i.fa.fa-table label.clickable(ng-bind='node.displayName') diff --git a/modules/web-console/frontend/views/sql/notebook-new.jade b/modules/web-console/frontend/views/sql/notebook-new.jade index 09b2daed618a7..8d9e8c4d9aaa9 100644 --- a/modules/web-console/frontend/views/sql/notebook-new.jade +++ b/modules/web-console/frontend/views/sql/notebook-new.jade @@ -19,7 +19,9 @@ .modal-content .modal-header button.close(ng-click='$hide()') × - h4.modal-title New SQL notebook + h4.modal-title + i.fa.fa-file-o + | New SQL notebook form.form-horizontal.modal-body.row(name='ui.inputForm' novalidate) div .col-sm-2 diff --git a/modules/web-console/frontend/views/sql/sql.jade b/modules/web-console/frontend/views/sql/sql.jade index 81acdfd5b53cb..4a596b96e1c1d 100644 --- a/modules/web-console/frontend/views/sql/sql.jade +++ b/modules/web-console/frontend/views/sql/sql.jade @@ -14,6 +14,8 @@ See the License for the specific language governing permissions and limitations under the License. +include ../../app/directives/ui-grid-settings/ui-grid-settings.jade + mixin btn-toolbar(btn, click, tip, focusId) i.btn.btn-default.fa(class=btn ng-click=click bs-tooltip='' data-title=tip ignite-on-click-focus=focusId data-trigger='hover' data-placement='bottom') @@ -82,36 +84,70 @@ mixin paragraph-rename .input-tip input.form-control(id='paragraph-name-{{paragraph.id}}' ng-model='paragraph.editName' required ng-click='$event.stopPropagation();' ignite-on-enter='renameParagraph(paragraph, paragraph.editName)' ignite-on-escape='paragraph.edit = false') +mixin query-settings + label.tipLabel Refresh rate: + button.btn.btn-default.fa.fa-clock-o.tipLabel(title='Click to show refresh rate dialog' ng-class='{"btn-info": paragraph.rate && paragraph.rate.installed}' bs-popover data-template-url='/sql/paragraph-rate.html' data-placement='left' data-auto-close='1' data-trigger='click') {{rateAsString(paragraph)}} + label.tipLabel Page size: + button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.pageSize' bs-options='item for item in pageSizes' bs-select bs-tooltip data-placement='bottom-right' data-title='Max number of rows to show in query result as one page') + label.margin-left-dflt(title='Fetch first page of results only') + input(type='checkbox' ng-model='paragraph.firstPageOnly') + span Fetch first page only + label.margin-left-dflt(title='Execute query locally on selected node.\nNode selection dialog will be shown before query execution.') + input(type='checkbox' ng-model='paragraph.localQry') + span Local query + +mixin query-actions + .btn-group(bs-tooltip='' data-title='{{actionTooltip(paragraph, "execute", true)}}' data-placement='bottom') + button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph)') Execute + button.btn.btn-primary.dropdown-toggle( + ng-disabled='!actionAvailable(paragraph, true)' + bs-dropdown='' + data-container='body' + data-placement='bottom-right' + ) + span.caret + ul.dropdown-menu(role='menu') + li #[a(href='javascript:void(0)' ng-click='execute(paragraph)') Execute] + li #[a(href='javascript:void(0)' ng-if='nonCollocatedJoinsAvailable(paragraph)' ng-click='execute(paragraph, true)') Execute non collocated joins] + .btn-group(bs-tooltip='' data-title='{{actionTooltip(paragraph, "execute scan", false)}}' data-placement='bottom') + button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' ng-click='scan(paragraph)') Scan + button.btn.btn-primary.dropdown-toggle( + ng-disabled='!actionAvailable(paragraph, false)' + bs-dropdown='' + data-container='body' + data-placement='bottom-right' + ) + span.caret + ul.dropdown-menu(role='menu') + li #[a(href='javascript:void(0)' ng-click='scan(paragraph)') Scan] + li #[a(href='javascript:void(0)' ng-click='actionAvailable(paragraph, false) && scanWithFilter(paragraph)') Scan with filter] + a.btn.btn-default(ng-disabled='!actionAvailable(paragraph, true)' ng-click='explain(paragraph)' data-placement='bottom' bs-tooltip='' data-title='{{actionTooltip(paragraph, "explain", true)}}') Explain + mixin query-controls .sql-controls - a.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph)' data-placement='bottom' bs-tooltip='' data-title='{{actionTooltip(paragraph, "execute", true)}}') Execute - a.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='explain(paragraph)' data-placement='bottom' bs-tooltip='' data-title='{{actionTooltip(paragraph, "explain", true)}}') Explain - .btn-group(bs-tooltip='' data-title='{{actionTooltip(paragraph, "execute scan", false)}}' data-placement='bottom') - a.btn.btn-primary.fieldButton(ng-disabled='!actionAvailable(paragraph, false)' ng-click='scan(paragraph)') Scan - a.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' data-toggle='dropdown' data-container='body' bs-dropdown='[{ text: "Scan with filter", click: "actionAvailable(paragraph, false) && scanWithFilter(paragraph)" }]') - span.caret - + +query-actions() .pull-right - label.tipLabel System columns: - a.btn.btn-default.fa.fa-bars.tipLabel(ng-class='{"btn-info": paragraph.systemColumns}' ng-click='toggleSystemColumns(paragraph)' ng-disabled='paragraph.disabledSystemColumns' bs-tooltip data-title='Show "_KEY", "_VAL" columns') - label.tipLabel Refresh rate: - button.btn.btn-default.fa.fa-clock-o.tipLabel(title='Click to show refresh rate dialog' ng-class='{"btn-info": paragraph.rate && paragraph.rate.installed}' bs-popover data-template-url='/sql/paragraph-rate.html' data-placement='left' data-auto-close='1' data-trigger='click') {{rateAsString(paragraph)}} - label.tipLabel Page size: - button.select-toggle.fieldButton.btn.btn-default(ng-model='paragraph.pageSize' bs-options='item for item in pageSizes' bs-select bs-tooltip data-placement='bottom-right' data-title='Max number of rows to show in query result as one page') + +query-settings() mixin table-result .total.row .col-xs-4 + +ui-grid-settings label Page: #[b {{paragraph.page}}] label.margin-left-dflt Results so far: #[b {{paragraph.rows.length + paragraph.total}}] label.margin-left-dflt Duration: #[b {{paragraph.duration | duration}}] .col-xs-4 +result-toolbar .col-xs-4 - .btn-group.pull-right(ng-disabled='paragraph.loading') - button.btn.btn-primary.fieldButton(ng-click='exportCsv(paragraph)' bs-tooltip data-title='{{actionTooltip(paragraph, "export", false)}}') Export - button.btn.btn-primary(id='export-item-dropdown' data-toggle='dropdown' data-container='body' bs-dropdown='exportDropdown' data-placement='bottom-right') - span.caret + .pull-right + label(style='margin-right: 10px;') + input(type='checkbox' ng-model='paragraph.systemColumns' ng-change='toggleSystemColumns(paragraph)' ng-disabled='paragraph.disabledSystemColumns') + span Show _KEY, _VAL columns + .btn-group(ng-disabled='paragraph.loading') + button.btn.btn-primary(ng-click='exportCsv(paragraph)' bs-tooltip data-title='{{actionTooltip(paragraph, "export", false)}}') Export + button.btn.btn-primary.dropdown-toggle(id='export-item-dropdown' data-toggle='dropdown' data-container='body' bs-dropdown='exportDropdown' data-placement='bottom-right') + span.caret + .grid(ui-grid='paragraph.gridOptions' ui-grid-resize-columns ui-grid-exporter) mixin chart-result @@ -161,15 +197,15 @@ mixin chart-result .col-xs-4.col-sm-3 div(ng-show='caches.length > 0' style='padding: 5px 10px' st-table='displayedCaches' st-safe-src='caches') lable.labelField.labelFormField Caches: - i.fa.fa-database.tipField(title='Click to show cache types metadata dialog' bs-popover data-template-url='/sql/cache-metadata.html' data-placement='bottom' data-trigger='click' data-container='#{{ paragraph.id}}') + i.fa.fa-database.tipField(title='Click to show cache types metadata dialog' bs-popover data-template-url='/sql/cache-metadata.html' data-placement='bottom' data-trigger='click' data-container='#{{ paragraph.id }}') .input-tip input.form-control(type='text' st-search='label' placeholder='Filter caches...') table.links tbody.scrollable-y(style='max-height: 15em; display: block;') tr(ng-repeat='cache in displayedCaches track by cache.name') td(style='width: 100%') - input.labelField(id='cache{{$index}}' type='radio' value='{{cache.name}}' ng-model='paragraph.cacheName') - label(for='cache{{$index}}' ng-bind='cache.label') + input.labelField(id='cache_{{ [paragraph.id, $index].join("_") }}' type='radio' value='{{cache.name}}' ng-model='paragraph.cacheName') + label(for='cache_{{ [paragraph.id, $index].join("_") }} ' ng-bind='cache.label') .empty-caches(ng-show='displayedCaches.length == 0 && caches.length != 0') label Wrong caches filter .empty-caches(ng-show='caches.length == 0') @@ -183,10 +219,10 @@ mixin chart-result +table-result .chart(ng-switch-when='chart') +chart-result - .footer.clearfix(ng-show='paragraph.nonRefresh()') + .footer.clearfix a.pull-left(ng-click='showResultQuery(paragraph)') Show query - -var nextVisibleCondition = 'paragraph.queryId && (paragraph.table() || paragraph.chart() && (paragraph.timeLineSupported() || !paragraph.chartTimeLineEnabled()))' + -var nextVisibleCondition = 'paragraph.resultType() != "error" && paragraph.queryId && paragraph.nonRefresh() && (paragraph.table() || paragraph.chart() && !paragraph.scanExplain())' .pull-right(ng-show=nextVisibleCondition ng-class='{disabled: paragraph.loading}' ng-click='!paragraph.loading && nextPage(paragraph)') i.fa.fa-chevron-circle-right diff --git a/modules/web-console/frontend/views/templates/agent-download.jade b/modules/web-console/frontend/views/templates/agent-download.jade index 864694b8bc603..a6da3d10d64f8 100644 --- a/modules/web-console/frontend/views/templates/agent-download.jade +++ b/modules/web-console/frontend/views/templates/agent-download.jade @@ -18,8 +18,10 @@ .modal-dialog .modal-content #errors-container.modal-header.header - h4.modal-title(ng-if='!hasAgents') Connection to Ignite Web Agent is not established - h4.modal-title(ng-if='hasAgents') Connection to Ignite Node is not established + h4.modal-title + i.fa.fa-download + span(ng-if='!hasAgents') Connection to Ignite Web Agent is not established + span(ng-if='hasAgents') Connection to Ignite Node is not established .agent-download(ng-if='!hasAgents') p Please download and run #[a(href='javascript:void(0)' ng-click='downloadAgent()') ignite-web-agent] in order to {{::agentGoal}} p For run: diff --git a/modules/web-console/frontend/views/templates/batch-confirm.jade b/modules/web-console/frontend/views/templates/batch-confirm.jade index 74513144b0ce3..c5d377565ce33 100644 --- a/modules/web-console/frontend/views/templates/batch-confirm.jade +++ b/modules/web-console/frontend/views/templates/batch-confirm.jade @@ -19,7 +19,9 @@ .modal-content .modal-header button.close(ng-click='cancel()' aria-hidden='true') × - h4.modal-title Confirmation + h4.modal-title + i.fa.fa-question + | Confirmation .modal-body(ng-show='content') p(ng-bind-html='content' style='text-align: center') .modal-footer diff --git a/modules/web-console/frontend/views/templates/clone.jade b/modules/web-console/frontend/views/templates/clone.jade index d68bf458d134c..6cf93ca94f20b 100644 --- a/modules/web-console/frontend/views/templates/clone.jade +++ b/modules/web-console/frontend/views/templates/clone.jade @@ -21,7 +21,9 @@ include ../../app/helpers/jade/mixins.jade .modal-content .modal-header button.close(ng-click='$hide()') × - h4.modal-title Clone + h4.modal-title + i.fa.fa-clone + | Clone form.form-horizontal.modal-body.row(name='ui.inputForm' novalidate) div .col-sm-2 diff --git a/modules/web-console/frontend/views/templates/confirm.jade b/modules/web-console/frontend/views/templates/confirm.jade index 26af061fa5939..f9f966befd67c 100644 --- a/modules/web-console/frontend/views/templates/confirm.jade +++ b/modules/web-console/frontend/views/templates/confirm.jade @@ -19,7 +19,9 @@ .modal-content .modal-header button.close(ng-click='confirmCancel()' aria-hidden='true') × - h4.modal-title Confirmation + h4.modal-title + i.fa.fa-question-circle-o + | Confirmation .modal-body(ng-show='content') p(ng-bind-html='content' style='text-align: center;') .modal-footer diff --git a/modules/web-console/frontend/views/templates/demo-info.jade b/modules/web-console/frontend/views/templates/demo-info.jade index 100e80660defd..44c091c3a0896 100644 --- a/modules/web-console/frontend/views/templates/demo-info.jade +++ b/modules/web-console/frontend/views/templates/demo-info.jade @@ -19,7 +19,9 @@ .modal-content #errors-container.modal-header.header button.close(ng-click='close()' aria-hidden='true') × - h4.modal-title {{title}} + h4.modal-title + i.fa.fa-info-circle + | {{title}} .modal-body div(ng-bind-html='message') div(ng-hide='hasAgents') diff --git a/modules/web-console/frontend/views/templates/getting-started.jade b/modules/web-console/frontend/views/templates/getting-started.jade index 98bc265b8c7f9..3a89035d27ab0 100644 --- a/modules/web-console/frontend/views/templates/getting-started.jade +++ b/modules/web-console/frontend/views/templates/getting-started.jade @@ -19,7 +19,9 @@ .modal-content #errors-container.modal-header.header button.close(ng-click='close()' aria-hidden='true') × - h4.modal-title {{title}} + h4.modal-title + i.fa.fa-book + | {{title}} .getting-started .col-xs-12(ng-bind-html='message') .modal-footer diff --git a/modules/web-console/frontend/views/templates/message.jade b/modules/web-console/frontend/views/templates/message.jade index 6dcf445d941d0..6eff74bba5173 100644 --- a/modules/web-console/frontend/views/templates/message.jade +++ b/modules/web-console/frontend/views/templates/message.jade @@ -19,7 +19,9 @@ .modal-content .modal-header button.close(ng-click='$hide()' aria-hidden='true') × - h4.modal-title {{title}} + h4.modal-title + i.fa.fa-info-circle + | {{title}} .modal-body(ng-show='content') p(ng-bind-html='content.join("
          ")' style='text-align: left;') .modal-footer diff --git a/modules/web-console/frontend/views/templates/select.jade b/modules/web-console/frontend/views/templates/select.jade index 3feee61c20c8c..5b6cc0170f5f0 100644 --- a/modules/web-console/frontend/views/templates/select.jade +++ b/modules/web-console/frontend/views/templates/select.jade @@ -21,6 +21,6 @@ ul.select.dropdown-menu(tabindex='-1' ng-show='$isVisible()' role='select') hr(style='margin: 5px 0') li(role='presentation' ng-repeat='match in $matches') hr(ng-if='match.value == undefined' style='margin: 5px 0') - a(id='li-dropdown-item-{{$index}}' role='menuitem' tabindex='-1' ng-class='{active: $isActive($index)}' ng-click='$select($index, $event)' bs-tooltip='widthIsSufficient && !widthIsSufficient("li-dropdown-item-{{$index}}", $index, match.label) ? match.label : ""' data-placement='bottom') + a(id='li-dropdown-item-{{$index}}' role='menuitem' tabindex='-1' ng-class='{active: $isActive($index)}' ng-click='$select($index, $event)' bs-tooltip='widthIsSufficient && !widthIsSufficient("li-dropdown-item-{{$index}}", $index, match.label) ? match.label : ""' data-placement='right auto') i(class='{{$iconCheckmark}}' ng-if='$isActive($index)' ng-class='{active: $isActive($index)}') span(ng-bind='match.label') diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java index 09189b5992398..2fb9f562bdbe6 100644 --- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java +++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java @@ -333,7 +333,7 @@ private static CacheConfiguration cacheCar() { private static IgniteConfiguration igniteConfiguration(int gridIdx, boolean client) { IgniteConfiguration cfg = new IgniteConfiguration(); - cfg.setGridName((client ? "demo-server-" : "demo-client-") + gridIdx); + cfg.setGridName((client ? "demo-client-" : "demo-server-" ) + gridIdx); cfg.setLocalHost("127.0.0.1"); cfg.setIncludeEventTypes(EVTS_DISCOVERY); From 44db38f3c56107badcf4948afff967ae12d115d6 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Fri, 28 Oct 2016 17:23:52 +0300 Subject: [PATCH 286/487] IGNITE-4115 Ignite.NET: Clean up instructions in README.txt file --- modules/platforms/dotnet/README.txt | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/modules/platforms/dotnet/README.txt b/modules/platforms/dotnet/README.txt index 783e19e72d02d..7521dfe2a1374 100644 --- a/modules/platforms/dotnet/README.txt +++ b/modules/platforms/dotnet/README.txt @@ -9,8 +9,14 @@ create distributed locks, subscribe for event listeners, etc. Files list: * Apache.Ignite.exe - executable to start standalone Ignite.NET node. + * Apache.Ignite.exe.config - standalone node configuration file. * Apache.Ignite.Core.dll - Ignite.NET API library. * Apache.Ignite.Core.xml - Library XML documentation. + * Apache.Ignite.Linq.dll - Ignite LINQ Provider library. + * Apache.Ignite.AspNet.dll - Ignite ASP.NET integration. + * Apache.Ignite.NLog.dll - Ignite NLog logger. + * Apache.Ignite.Log4Net.dll - Ignite Log4Net logger. + * IgniteConfigurationSection.xsd - Configuration XML schema. Development @@ -21,13 +27,10 @@ Development * Download and install Microsoft Visual C++ 2010 Redistributable Package: http://www.microsoft.com/en-us/download/details.aspx?id=14632 - * Download and install the latest Java Development Kit: + * Download and install the latest Java Runtime Environment: https://java.com/en/download/index.jsp - * Set JAVA_HOME environment variable as per this tutorial: - http://docs.oracle.com/cd/E19182-01/820-7851/inst_cli_jdk_javahome_t/index.html - - * Add Apache.Ignite.Core.dll to your project references (or x86\Apache.Ignite.Core.dll if you need 32-bit version). + * Add Apache.Ignite.Core.dll to your project references. * To start Apache Ignite as a standalone node or Windows service use Apache.Ignite.exe. From a801f01c3933b992ae8b5282f438f06059c22523 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Fri, 28 Oct 2016 18:24:48 +0300 Subject: [PATCH 287/487] IGNITE-4151 .NET: Fix project build settings consistency --- .../Apache.Ignite.AspNet.csproj | 3 +- .../ProjectFilesTest.cs | 62 +++++++++++++++++++ .../Apache.Ignite.Core.csproj | 1 + .../Apache.Ignite.Linq.csproj | 2 +- .../Apache.Ignite.Log4Net.csproj | 4 +- .../Apache.Ignite.NLog.csproj | 4 +- 6 files changed, 69 insertions(+), 7 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj b/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj index 1ac452ffd8f35..2e501c1d657bb 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj @@ -31,9 +31,8 @@ bin\Release\ - TRACE true - pdbonly + none AnyCPU prompt Apache.Ignite.AspNet.ruleset diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs index 2f37505fffbda..b95fead7a7164 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs @@ -22,6 +22,7 @@ namespace Apache.Ignite.Core.Tests using System.IO; using System.Linq; using System.Reflection; + using System.Text.RegularExpressions; using NUnit.Framework; /// @@ -41,6 +42,67 @@ public void TestCsprojToolsVersion() CheckFiles(projFiles, x => !x.Contains("ToolsVersion=\"4.0\""), "Invalid csproj files: "); } + /// + /// Tests that release build settings are correct: XML docs are generated. + /// + [Test] + public void TestCsprojReleaseDocs() + { + CheckFiles(GetReleaseCsprojFiles(), x => !GetReleaseSection(x).Contains("DocumentationFile"), + "Missing XML doc in release mode: "); + } + + /// + /// Tests that release build settings are correct: there are no DEBUG/TRACE constants. + /// + [Test] + public void TestCsprojBuildSettings() + { + CheckFiles(GetReleaseCsprojFiles(), x => GetReleaseSection(x).Contains("DefineConstants"), + "Invalid constants in release mode: "); + } + + /// + /// Tests that release build settings are correct: debug information is disabled. + /// + [Test] + public void TestCsprojPdbSettings() + { + CheckFiles(GetReleaseCsprojFiles(), x => !GetReleaseSection(x).Contains("none"), + "Invalid DebugType in release mode: "); + } + + /// + /// Tests that release build settings are correct: debug information is disabled. + /// + [Test] + public void TestCsprojOptimizeCode() + { + CheckFiles(GetReleaseCsprojFiles(), x => !GetReleaseSection(x).Contains("true"), + "Invalid optimize setting in release mode: "); + } + + /// + /// Gets the csproj files that go to the release binary package. + /// + private static IEnumerable GetReleaseCsprojFiles() + { + return GetDotNetSourceDir().GetFiles("*.csproj", SearchOption.AllDirectories) + .Where(x => x.Name != "Apache.Ignite.csproj" && + !x.Name.Contains("Test") && + !x.Name.Contains("Example") && + !x.Name.Contains("Benchmark")); + } + + /// + /// Gets the release section. + /// + private static string GetReleaseSection(string csproj) + { + return Regex.Match(csproj, @"]*Release\|AnyCPU(.*?)<\/PropertyGroup>", + RegexOptions.Singleline).Value; + } + /// /// Tests that tools version is compatible with VS2010. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj index 08d742adaeb3b..6fa13789c89b9 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj @@ -75,6 +75,7 @@ true AnyCPU Apache.Ignite.Core.ruleset + none diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj b/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj index 79b52bdf0cb83..e935f3f714c3c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj @@ -24,12 +24,12 @@ bin\Release\ - TRACE true none AnyCPU prompt MinimumRecommendedRules.ruleset + bin\Release\Apache.Ignite.Linq.XML true diff --git a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj index 0fdd6116bf3d9..af5b7e45255e4 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj @@ -24,12 +24,12 @@ AllRules.ruleset - pdbonly + none true bin\Release\ - TRACE prompt 4 + bin\Release\Apache.Ignite.Log4Net.XML true diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj b/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj index c8d87057ad00a..147dc3747ca00 100644 --- a/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj @@ -22,12 +22,12 @@ 4 - pdbonly + none true bin\Release\ - TRACE prompt 4 + bin\Release\Apache.Ignite.NLog.XML true From 6a2ecdbf5dbc86ff0c25cca579a669e90b3cfffd Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Fri, 28 Oct 2016 19:45:29 +0300 Subject: [PATCH 288/487] IGNITE-4131 .NET: Improve "Unsupported type exception" with additional information --- .../Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs | 12 ++++++++++++ .../Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs | 6 +++--- 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs index 860fd9e74f4f9..cc5d8a109ce86 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs @@ -1948,6 +1948,18 @@ public static void ReadConfiguration(BinaryReader reader, out ICollection + /// Gets the unsupported type exception. + /// + public static BinaryObjectException GetUnsupportedTypeException(Type type, object obj) + { + return new BinaryObjectException(string.Format( + "Unsupported object type [type={0}, object={1}].\nSpecified type " + + "can not be serialized by Ignite: it is neither [Serializable], " + + "nor registered in IgniteConfiguration.BinaryConfiguration." + + "\nSee https://apacheignite-net.readme.io/docs/serialization for more details.", type, obj)); + } + /// /// Creates and instance from the type name in reader. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs index 063aa9dce3144..585ccd32fbfb1 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs @@ -1259,8 +1259,8 @@ public void Write(T obj) // Are we dealing with a well-known type? var handler = BinarySystemHandlers.GetWriteHandler(type); - if (handler == null) // We did our best, object cannot be marshalled. - throw new BinaryObjectException("Unsupported object type [type=" + type + ", object=" + obj + ']'); + if (handler == null) // We did our best, object cannot be marshalled. + throw BinaryUtils.GetUnsupportedTypeException(type, obj); if (handler.SupportsHandles && WriteHandle(_stream.Position, obj)) return; @@ -1316,7 +1316,7 @@ private unsafe void WritePrimitive(T val, Type type) WriteLongField(*(long*)&val0); } else - throw new BinaryObjectException("Unsupported object type [type=" + type.FullName + ", object=" + val + ']'); + throw BinaryUtils.GetUnsupportedTypeException(type, val); } /// From 9ebbaea586d9ba360a1325a840fc7d81c93a95fc Mon Sep 17 00:00:00 2001 From: Valentin Kulichenko Date: Fri, 28 Oct 2016 16:08:44 -0700 Subject: [PATCH 289/487] IGNITE-4110 - Fixed BUFFER_UNDERFLOW and BUFFER_OVERFLOW handling in BlockingSslHandler --- .../util/nio/ssl/BlockingSslHandler.java | 50 ++++++++++--------- ...mmunicationSpiSslSmallBuffersSelfTest.java | 43 ++++++++++++++++ .../IgniteSpiCommunicationSelfTestSuite.java | 7 ++- 3 files changed, 75 insertions(+), 25 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSmallBuffersSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java index b3e820153e5dc..638106f667a3c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java @@ -35,6 +35,7 @@ import static javax.net.ssl.SSLEngineResult.HandshakeStatus.NEED_TASK; import static javax.net.ssl.SSLEngineResult.HandshakeStatus.NEED_UNWRAP; import static javax.net.ssl.SSLEngineResult.HandshakeStatus.NOT_HANDSHAKING; +import static javax.net.ssl.SSLEngineResult.Status.BUFFER_OVERFLOW; import static javax.net.ssl.SSLEngineResult.Status.BUFFER_UNDERFLOW; import static javax.net.ssl.SSLEngineResult.Status.CLOSED; import static javax.net.ssl.SSLEngineResult.Status.OK; @@ -92,7 +93,10 @@ public BlockingSslHandler(SSLEngine sslEngine, this.order = order; // Allocate a little bit more so SSL engine would not return buffer overflow status. - int netBufSize = sslEngine.getSession().getPacketBufferSize() + 50; + // + // System property override is for test purposes only. + int netBufSize = Integer.getInteger("BlockingSslHandler.netBufSize", + sslEngine.getSession().getPacketBufferSize() + 50); outNetBuf = directBuf ? ByteBuffer.allocateDirect(netBufSize) : ByteBuffer.allocate(netBufSize); outNetBuf.order(order); @@ -173,15 +177,22 @@ public boolean handshake() throws IgniteCheckedException, SSLException { SSLEngineResult res = sslEngine.wrap(handshakeBuf, outNetBuf); - outNetBuf.flip(); + if (res.getStatus() == BUFFER_OVERFLOW) { + outNetBuf = expandBuffer(outNetBuf, outNetBuf.capacity() * 2); + + outNetBuf.flip(); + } + else { + outNetBuf.flip(); + + writeNetBuffer(); + } handshakeStatus = res.getHandshakeStatus(); if (log.isDebugEnabled()) log.debug("Wrapped handshake data [status=" + res.getStatus() + ", handshakeStatus=" + - handshakeStatus + ']'); - - writeNetBuffer(); + handshakeStatus + ']'); break; } @@ -368,6 +379,7 @@ private Status unwrapHandshake() throws SSLException, IgniteCheckedException { inNetBuf.flip(); SSLEngineResult res = unwrap0(); + handshakeStatus = res.getHandshakeStatus(); checkStatus(res); @@ -384,6 +396,11 @@ private Status unwrapHandshake() throws SSLException, IgniteCheckedException { renegotiateIfNeeded(res); } + else if (res.getStatus() == BUFFER_UNDERFLOW) { + inNetBuf.compact(); + + inNetBuf = expandBuffer(inNetBuf, inNetBuf.capacity() * 2); + } else // prepare to be written again inNetBuf.compact(); @@ -462,6 +479,7 @@ private ByteBuffer allocateAppBuff() { int appBufSize = Math.max(sslEngine.getSession().getApplicationBufferSize() + 50, netBufSize * 2); ByteBuffer buf = ByteBuffer.allocate(appBufSize); + buf.order(order); return buf; @@ -504,9 +522,9 @@ private void writeNetBuffer() throws IgniteCheckedException { * @return Expanded byte buffer. */ private ByteBuffer expandBuffer(ByteBuffer original, int cap) { - ByteBuffer res = ByteBuffer.allocate(cap); + ByteBuffer res = original.isDirect() ? ByteBuffer.allocateDirect(cap) : ByteBuffer.allocate(cap); - res.order(ByteOrder.nativeOrder()); + res.order(original.order()); original.flip(); @@ -514,20 +532,4 @@ private ByteBuffer expandBuffer(ByteBuffer original, int cap) { return res; } - - /** - * Copies the given byte buffer. - * - * @param original Byte buffer to copy. - * @return Copy of the original byte buffer. - */ - private ByteBuffer copy(ByteBuffer original) { - ByteBuffer cp = ByteBuffer.allocate(original.remaining()); - - cp.put(original); - - cp.flip(); - - return cp; - } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSmallBuffersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSmallBuffersSelfTest.java new file mode 100644 index 0000000000000..1ccb9bbf0fa44 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSmallBuffersSelfTest.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.communication.tcp; + +import org.apache.ignite.internal.util.nio.ssl.BlockingSslHandler; +import org.apache.ignite.testframework.junits.spi.GridSpiTest; + +/** + * Copy of {@link GridTcpCommunicationSpiSslSelfTest}, but overriding initial buffer + * sizes in {@link BlockingSslHandler}. This checks that {@code BUFFER_UNDERFLOW} and + * {@code BUFFER_OVERFLOW} conditions are properly handled. + */ +@GridSpiTest(spi = TcpCommunicationSpi.class, group = "Communication SPI") +public class GridTcpCommunicationSpiSslSmallBuffersSelfTest extends GridTcpCommunicationSpiSslSelfTest { + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty("BlockingSslHandler.netBufSize", "1000"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty("BlockingSslHandler.netBufSize"); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java index eb88524cddbda..c557fbbcad791 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java @@ -28,6 +28,8 @@ import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoverySelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoverySslSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiShmemSelfTest; +import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiSslSelfTest; +import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiSslSmallBuffersSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiStartStopSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiTcpFailureDetectionSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiTcpNoDelayOffSelfTest; @@ -53,6 +55,9 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(GridTcpCommunicationSpiConcurrentConnectSelfTest.class)); suite.addTest(new TestSuite(GridTcpCommunicationSpiConcurrentConnectSslSelfTest.class)); + suite.addTest(new TestSuite(GridTcpCommunicationSpiSslSelfTest.class)); + suite.addTest(new TestSuite(GridTcpCommunicationSpiSslSmallBuffersSelfTest.class)); + suite.addTest(new TestSuite(GridTcpCommunicationSpiTcpSelfTest.class)); suite.addTest(new TestSuite(GridTcpCommunicationSpiTcpNoDelayOffSelfTest.class)); suite.addTest(new TestSuite(GridTcpCommunicationSpiShmemSelfTest.class)); @@ -69,4 +74,4 @@ public static TestSuite suite() throws Exception { return suite; } -} \ No newline at end of file +} From 6f160728c544d252f77bdb85c0ff2857559707a3 Mon Sep 17 00:00:00 2001 From: Valentin Kulichenko Date: Fri, 28 Oct 2016 16:18:14 -0700 Subject: [PATCH 290/487] IGNITE-4110 - Fixed BUFFER_UNDERFLOW and BUFFER_OVERFLOW handling in BlockingSslHandler --- .../util/nio/ssl/BlockingSslHandler.java | 50 ++++++++++--------- ...mmunicationSpiSslSmallBuffersSelfTest.java | 43 ++++++++++++++++ .../IgniteSpiCommunicationSelfTestSuite.java | 7 ++- 3 files changed, 75 insertions(+), 25 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSmallBuffersSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java index b3e820153e5dc..638106f667a3c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java @@ -35,6 +35,7 @@ import static javax.net.ssl.SSLEngineResult.HandshakeStatus.NEED_TASK; import static javax.net.ssl.SSLEngineResult.HandshakeStatus.NEED_UNWRAP; import static javax.net.ssl.SSLEngineResult.HandshakeStatus.NOT_HANDSHAKING; +import static javax.net.ssl.SSLEngineResult.Status.BUFFER_OVERFLOW; import static javax.net.ssl.SSLEngineResult.Status.BUFFER_UNDERFLOW; import static javax.net.ssl.SSLEngineResult.Status.CLOSED; import static javax.net.ssl.SSLEngineResult.Status.OK; @@ -92,7 +93,10 @@ public BlockingSslHandler(SSLEngine sslEngine, this.order = order; // Allocate a little bit more so SSL engine would not return buffer overflow status. - int netBufSize = sslEngine.getSession().getPacketBufferSize() + 50; + // + // System property override is for test purposes only. + int netBufSize = Integer.getInteger("BlockingSslHandler.netBufSize", + sslEngine.getSession().getPacketBufferSize() + 50); outNetBuf = directBuf ? ByteBuffer.allocateDirect(netBufSize) : ByteBuffer.allocate(netBufSize); outNetBuf.order(order); @@ -173,15 +177,22 @@ public boolean handshake() throws IgniteCheckedException, SSLException { SSLEngineResult res = sslEngine.wrap(handshakeBuf, outNetBuf); - outNetBuf.flip(); + if (res.getStatus() == BUFFER_OVERFLOW) { + outNetBuf = expandBuffer(outNetBuf, outNetBuf.capacity() * 2); + + outNetBuf.flip(); + } + else { + outNetBuf.flip(); + + writeNetBuffer(); + } handshakeStatus = res.getHandshakeStatus(); if (log.isDebugEnabled()) log.debug("Wrapped handshake data [status=" + res.getStatus() + ", handshakeStatus=" + - handshakeStatus + ']'); - - writeNetBuffer(); + handshakeStatus + ']'); break; } @@ -368,6 +379,7 @@ private Status unwrapHandshake() throws SSLException, IgniteCheckedException { inNetBuf.flip(); SSLEngineResult res = unwrap0(); + handshakeStatus = res.getHandshakeStatus(); checkStatus(res); @@ -384,6 +396,11 @@ private Status unwrapHandshake() throws SSLException, IgniteCheckedException { renegotiateIfNeeded(res); } + else if (res.getStatus() == BUFFER_UNDERFLOW) { + inNetBuf.compact(); + + inNetBuf = expandBuffer(inNetBuf, inNetBuf.capacity() * 2); + } else // prepare to be written again inNetBuf.compact(); @@ -462,6 +479,7 @@ private ByteBuffer allocateAppBuff() { int appBufSize = Math.max(sslEngine.getSession().getApplicationBufferSize() + 50, netBufSize * 2); ByteBuffer buf = ByteBuffer.allocate(appBufSize); + buf.order(order); return buf; @@ -504,9 +522,9 @@ private void writeNetBuffer() throws IgniteCheckedException { * @return Expanded byte buffer. */ private ByteBuffer expandBuffer(ByteBuffer original, int cap) { - ByteBuffer res = ByteBuffer.allocate(cap); + ByteBuffer res = original.isDirect() ? ByteBuffer.allocateDirect(cap) : ByteBuffer.allocate(cap); - res.order(ByteOrder.nativeOrder()); + res.order(original.order()); original.flip(); @@ -514,20 +532,4 @@ private ByteBuffer expandBuffer(ByteBuffer original, int cap) { return res; } - - /** - * Copies the given byte buffer. - * - * @param original Byte buffer to copy. - * @return Copy of the original byte buffer. - */ - private ByteBuffer copy(ByteBuffer original) { - ByteBuffer cp = ByteBuffer.allocate(original.remaining()); - - cp.put(original); - - cp.flip(); - - return cp; - } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSmallBuffersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSmallBuffersSelfTest.java new file mode 100644 index 0000000000000..1ccb9bbf0fa44 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSmallBuffersSelfTest.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.communication.tcp; + +import org.apache.ignite.internal.util.nio.ssl.BlockingSslHandler; +import org.apache.ignite.testframework.junits.spi.GridSpiTest; + +/** + * Copy of {@link GridTcpCommunicationSpiSslSelfTest}, but overriding initial buffer + * sizes in {@link BlockingSslHandler}. This checks that {@code BUFFER_UNDERFLOW} and + * {@code BUFFER_OVERFLOW} conditions are properly handled. + */ +@GridSpiTest(spi = TcpCommunicationSpi.class, group = "Communication SPI") +public class GridTcpCommunicationSpiSslSmallBuffersSelfTest extends GridTcpCommunicationSpiSslSelfTest { + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty("BlockingSslHandler.netBufSize", "1000"); + + super.beforeTestsStarted(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.clearProperty("BlockingSslHandler.netBufSize"); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java index eb88524cddbda..c557fbbcad791 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java @@ -28,6 +28,8 @@ import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoverySelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoverySslSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiShmemSelfTest; +import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiSslSelfTest; +import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiSslSmallBuffersSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiStartStopSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiTcpFailureDetectionSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiTcpNoDelayOffSelfTest; @@ -53,6 +55,9 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(GridTcpCommunicationSpiConcurrentConnectSelfTest.class)); suite.addTest(new TestSuite(GridTcpCommunicationSpiConcurrentConnectSslSelfTest.class)); + suite.addTest(new TestSuite(GridTcpCommunicationSpiSslSelfTest.class)); + suite.addTest(new TestSuite(GridTcpCommunicationSpiSslSmallBuffersSelfTest.class)); + suite.addTest(new TestSuite(GridTcpCommunicationSpiTcpSelfTest.class)); suite.addTest(new TestSuite(GridTcpCommunicationSpiTcpNoDelayOffSelfTest.class)); suite.addTest(new TestSuite(GridTcpCommunicationSpiShmemSelfTest.class)); @@ -69,4 +74,4 @@ public static TestSuite suite() throws Exception { return suite; } -} \ No newline at end of file +} From 6b78ad0cbbcf286cb083136c49cebd5dd85de58c Mon Sep 17 00:00:00 2001 From: sboikov Date: Mon, 31 Oct 2016 10:35:44 +0300 Subject: [PATCH 291/487] TcoDiscovery: reduced amount of debug logging (heartbeat/connection check messages are logged trace level). --- .../GridCachePartitionExchangeManager.java | 10 +- .../processors/cache/GridCacheUtils.java | 67 ------- .../ignite/spi/discovery/tcp/ClientImpl.java | 12 +- .../ignite/spi/discovery/tcp/ServerImpl.java | 164 ++++++++++++------ .../spi/discovery/tcp/TcpDiscoveryImpl.java | 59 ++++++- .../messages/TcpDiscoveryAbstractMessage.java | 7 + .../TcpDiscoveryClientAckResponse.java | 5 + .../TcpDiscoveryClientHeartbeatMessage.java | 7 +- .../TcpDiscoveryConnectionCheckMessage.java | 5 + .../TcpDiscoveryHeartbeatMessage.java | 5 + 10 files changed, 211 insertions(+), 130 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index 4eb61e3922047..a901e2aca7825 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -718,7 +718,7 @@ public void scheduleResendPartitions() { /** * Partition refresh callback. */ - void refreshPartitions() { + private void refreshPartitions() { ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, AffinityTopologyVersion.NONE); if (oldest == null) { @@ -735,7 +735,13 @@ void refreshPartitions() { // If this is the oldest node. if (oldest.id().equals(cctx.localNodeId())) { - rmts = CU.remoteNodes(cctx, AffinityTopologyVersion.NONE); + GridDhtPartitionsExchangeFuture lastFut = lastInitializedFut; + + // No need to send to nodes which did not finish their first exchange. + AffinityTopologyVersion rmtTopVer = + lastFut != null ? lastFut.topologyVersion() : AffinityTopologyVersion.NONE; + + rmts = CU.remoteNodes(cctx, rmtTopVer); if (log.isDebugEnabled()) log.debug("Refreshing partitions from oldest node: " + cctx.localNodeId()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 4c18f213bb131..90e428ca10b80 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -409,23 +409,6 @@ public static String cacheNameForSwapSpaceName(String swapSpaceName) { return "gg-swap-cache-dflt".equals(swapSpaceName) ? null : swapSpaceName.substring("gg-swap-cache-".length()); } - /** - * Gets public cache name substituting null name by {@code 'default'}. - * - * @return Public cache name substituting null name by {@code 'default'}. - */ - public static String namexx(@Nullable String name) { - return name == null ? "default" : name; - } - - /** - * @return Partition to state transformer. - */ - @SuppressWarnings({"unchecked"}) - public static IgniteClosure part2state() { - return PART2STATE; - } - /** * Gets all nodes on which cache with the same name is started. * @@ -461,18 +444,6 @@ public static Collection remoteNodes(final GridCacheSharedContext c return ctx.discovery().remoteCacheNodes(topVer); } - /** - * Gets alive remote nodes with at least one cache configured. - * - * @param ctx Cache context. - * @param topOrder Maximum allowed node order. - * @return Affinity nodes. - */ - public static Collection aliveRemoteServerNodesWithCaches(final GridCacheSharedContext ctx, - AffinityTopologyVersion topOrder) { - return ctx.discovery().aliveRemoteServerNodesWithCaches(topOrder); - } - /** * Gets all nodes on which cache with the same name is started and the local DHT storage is enabled. * @@ -643,44 +614,6 @@ public static IgnitePredicate writes() { return WRITE_FILTER; } - /** - * Gets type filter for projections. - * - * @param keyType Key type. - * @param valType Value type. - * @param Key type. - * @param Value type. - * @return Type filter. - */ - public static IgniteBiPredicate typeFilter(final Class keyType, final Class valType) { - return new P2() { - @Override public boolean apply(K k, V v) { - return keyType.isAssignableFrom(k.getClass()) && valType.isAssignableFrom(v.getClass()); - } - - @Override public String toString() { - return "Type filter [keyType=" + keyType + ", valType=" + valType + ']'; - } - }; - } - - /** - * @param keyType Key type. - * @param valType Value type. - * @return Type filter. - */ - public static CacheEntryPredicate typeFilter0(final Class keyType, final Class valType) { - return new CacheEntrySerializablePredicate(new CacheEntryPredicateAdapter() { - @Override public boolean apply(GridCacheEntryEx e) { - Object val = CU.value(peekVisibleValue(e), e.context(), false); - - return val == null || - valType.isAssignableFrom(val.getClass()) && - keyType.isAssignableFrom(e.key().value(e.context().cacheObjectContext(), false).getClass()); - } - }); - } - /** * @return Boolean reducer. */ diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 2d948da3801a8..f9291219e66d4 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -929,8 +929,10 @@ public void setSocket(SocketStream sockStream, UUID rmtNodeId) { msg.senderNodeId(rmtNodeId); - if (log.isDebugEnabled()) - log.debug("Message has been received: " + msg); + DebugLogger debugLog = messageLogger(msg); + + if (debugLog.isDebugEnabled()) + debugLog.debug("Message has been received: " + msg); spi.stats.onMessageReceived(msg); @@ -2079,6 +2081,8 @@ private void notifyDiscovery(int type, long topVer, ClusterNode node, Collection @Nullable DiscoverySpiCustomMessage data) { DiscoverySpiListener lsnr = spi.lsnr; + DebugLogger log = type == EVT_NODE_METRICS_UPDATED ? traceLog : debugLog; + if (lsnr != null) { if (log.isDebugEnabled()) log.debug("Discovery notification [node=" + node + ", type=" + U.gridEventName(type) + @@ -2094,14 +2098,14 @@ else if (log.isDebugEnabled()) /** * @param msg Message. */ - public void addMessage(Object msg) { + void addMessage(Object msg) { queue.add(msg); } /** * @return Queue size. */ - public int queueSize() { + int queueSize() { return queue.size(); } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 78a5f39100c6a..55e5c8976b61c 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -214,6 +214,12 @@ class ServerImpl extends TcpDiscoveryImpl { /** Leaving nodes (but still in topology). */ private final Collection leavingNodes = new HashSet<>(); + /** Collection to track joining nodes. */ + private Set joiningNodes = new HashSet<>(); + + /** Pending custom messages that should not be sent between NodeAdded and NodeAddFinished messages. */ + private Queue pendingCustomMsgs = new ArrayDeque<>(); + /** If non-shared IP finder is used this flag shows whether IP finder contains local address. */ private boolean ipFinderHasLocAddr; @@ -296,7 +302,7 @@ class ServerImpl extends TcpDiscoveryImpl { throw new IgniteSpiException("Info log level should be enabled for TCP discovery to work " + "in debug mode."); - debugLog = new ConcurrentLinkedDeque<>(); + debugLogQ = new ConcurrentLinkedDeque<>(); U.quietAndWarn(log, "TCP discovery SPI is configured in debug mode."); } @@ -1288,6 +1294,8 @@ private void notifyDiscovery(int type, long topVer, TcpDiscoveryNode node) { TcpDiscoverySpiState spiState = spiStateCopy(); + DebugLogger log = type == EVT_NODE_METRICS_UPDATED ? traceLog : debugLog; + if (lsnr != null && node.visible() && (spiState == CONNECTED || spiState == DISCONNECTING)) { if (log.isDebugEnabled()) log.debug("Discovery notification [node=" + node + ", spiState=" + spiState + @@ -1410,10 +1418,14 @@ private void printStatistics() { if (log.isInfoEnabled() && spi.statsPrintFreq > 0) { int failedNodesSize; int leavingNodesSize; + int joiningNodesSize; + int pendingCustomMsgsSize; synchronized (mux) { failedNodesSize = failedNodes.size(); leavingNodesSize = leavingNodes.size(); + joiningNodesSize = joiningNodes.size(); + pendingCustomMsgsSize = pendingCustomMsgs.size(); } Runtime runtime = Runtime.getRuntime(); @@ -1422,8 +1434,13 @@ private void printStatistics() { log.info("Discovery SPI statistics [statistics=" + spi.stats + ", spiState=" + spiStateCopy() + ", coord=" + coord + + ", next=" + (msgWorker != null ? msgWorker.next : "N/A") + + ", intOrder=" + (locNode != null ? locNode.internalOrder() : "N/A") + ", topSize=" + ring.allNodes().size() + - ", leavingNodesSize=" + leavingNodesSize + ", failedNodesSize=" + failedNodesSize + + ", leavingNodesSize=" + leavingNodesSize + + ", failedNodesSize=" + failedNodesSize + + ", joiningNodesSize=" + joiningNodesSize + + ", pendingCustomMsgs=" + pendingCustomMsgsSize + ", msgWorker.queue.size=" + (msgWorker != null ? msgWorker.queueSize() : "N/A") + ", clients=" + ring.clientNodes().size() + ", clientWorkers=" + clientMsgWorkers.size() + @@ -1612,7 +1629,7 @@ TcpDiscoveryNodesRing ring() { b.append("In-memory log messages: ").append(U.nl()); - for (String msg : debugLog) + for (String msg : debugLogQ) b.append(" ").append(msg).append(U.nl()); b.append(U.nl()); @@ -2177,12 +2194,6 @@ private class RingMessageWorker extends MessageWorkerAdapter pendingCustomMsgs = new ArrayDeque<>(); - - /** Collection to track joining nodes. */ - private Set joiningNodes = new HashSet<>(); - /** */ protected RingMessageWorker() { @@ -2197,6 +2208,8 @@ protected RingMessageWorker() { * @param msg Message to add. */ void addMessage(TcpDiscoveryAbstractMessage msg) { + DebugLogger log = messageLogger(msg); + if ((msg instanceof TcpDiscoveryStatusCheckMessage || msg instanceof TcpDiscoveryJoinRequestMessage || msg instanceof TcpDiscoveryCustomEventMessage || @@ -2278,6 +2291,8 @@ private void initConnectionCheckFrequency() { * @param msg Message to process. */ @Override protected void processMessage(TcpDiscoveryAbstractMessage msg) { + DebugLogger log = messageLogger(msg); + if (log.isDebugEnabled()) log.debug("Processing message [cls=" + msg.getClass().getSimpleName() + ", id=" + msg.id() + ']'); @@ -2285,12 +2300,12 @@ private void initConnectionCheckFrequency() { debugLog(msg, "Processing message [cls=" + msg.getClass().getSimpleName() + ", id=" + msg.id() + ']'); if (locNode.internalOrder() == 0) { - boolean process = false; + boolean proc = false; if (msg instanceof TcpDiscoveryNodeAddedMessage) - process = ((TcpDiscoveryNodeAddedMessage)msg).node().equals(locNode); + proc = ((TcpDiscoveryNodeAddedMessage)msg).node().equals(locNode); - if (!process) { + if (!proc) { if (log.isDebugEnabled()) { log.debug("Ignore message, local node order is not initialized [msg=" + msg + ", locNode=" + locNode + ']'); @@ -2488,8 +2503,8 @@ private void sendMessageAcrossRing(TcpDiscoveryAbstractMessage msg) { newNextNode = true; } - else if (log.isDebugEnabled()) - log.debug("Next node remains the same [nextId=" + next.id() + + else if (log.isTraceEnabled()) + log.trace("Next node remains the same [nextId=" + next.id() + ", nextOrder=" + next.internalOrder() + ']'); // Flag that shows whether next node exists and accepts incoming connections. @@ -2752,8 +2767,10 @@ else if (!spi.failureDetectionTimeoutEnabled() && (e instanceof onMessageExchanged(); - if (log.isDebugEnabled()) { - log.debug("Message has been sent to next node [msg=" + msg + + DebugLogger debugLog = messageLogger(msg); + + if (debugLog.isDebugEnabled()) { + debugLog.debug("Message has been sent to next node [msg=" + msg + ", next=" + next.id() + ", res=" + res + ']'); } @@ -3783,7 +3800,9 @@ else if (!locNodeId.equals(node.id()) && ring.node(node.id()) != null) { return; } - joiningNodes.add(node.id()); + synchronized (mux) { + joiningNodes.add(node.id()); + } if (!isLocalNodeCoordinator() && spi.nodeAuth != null && spi.nodeAuth.isGlobalNodeAuthentication()) { boolean authFailed = true; @@ -3895,7 +3914,9 @@ else if (!locNodeId.equals(node.id()) && ring.node(node.id()) != null) { n.visible(true); } - joiningNodes.clear(); + synchronized (mux) { + joiningNodes.clear(); + } locNode.setAttributes(node.attributes()); @@ -4021,7 +4042,9 @@ private void processNodeAddFinishedMessage(TcpDiscoveryNodeAddFinishedMessage ms } } - joiningNodes.remove(nodeId); + synchronized (mux) { + joiningNodes.remove(nodeId); + } TcpDiscoverySpiState state = spiStateCopy(); @@ -4240,7 +4263,9 @@ else if (leftNode.equals(next) && sock != null) { } } - joiningNodes.remove(leftNode.id()); + synchronized (mux) { + joiningNodes.remove(leftNode.id()); + } spi.stats.onNodeLeft(); @@ -4418,7 +4443,9 @@ private void processNodeFailedMessage(TcpDiscoveryNodeFailedMessage msg) { ", msg=" + msg.warning() + ']'); } - joiningNodes.remove(node.id()); + synchronized (mux) { + joiningNodes.remove(node.id()); + } notifyDiscovery(EVT_NODE_FAILED, topVer, node); @@ -4619,8 +4646,8 @@ private void processHeartbeatMessage(TcpDiscoveryHeartbeatMessage msg) { } if (locNodeId.equals(msg.creatorNodeId()) && !hasMetrics(msg, locNodeId) && msg.senderNodeId() != null) { - if (log.isDebugEnabled()) - log.debug("Discarding heartbeat message that has made two passes: " + msg); + if (log.isTraceEnabled()) + log.trace("Discarding heartbeat message that has made two passes: " + msg); return; } @@ -4821,18 +4848,28 @@ private void processCustomMessage(TcpDiscoveryCustomEventMessage msg) { assert ring.minimumNodeVersion() != null : ring; + boolean joiningEmpty; + + synchronized (mux) { + joiningEmpty = joiningNodes.isEmpty(); + } + if (ring.minimumNodeVersion().compareTo(CUSTOM_MSG_ALLOW_JOINING_FOR_VERIFIED_SINCE) >= 0) - delayMsg = msg.topologyVersion() == 0L && !joiningNodes.isEmpty(); + delayMsg = msg.topologyVersion() == 0L && !joiningEmpty; else - delayMsg = !joiningNodes.isEmpty(); + delayMsg = !joiningEmpty; if (delayMsg) { if (log.isDebugEnabled()) { - log.debug("Delay custom message processing, there are joining nodes [msg=" + msg + - ", joiningNodes=" + joiningNodes + ']'); + synchronized (mux) { + log.debug("Delay custom message processing, there are joining nodes [msg=" + msg + + ", joiningNodes=" + joiningNodes + ']'); + } } - pendingCustomMsgs.add(msg); + synchronized (mux) { + pendingCustomMsgs.add(msg); + } return; } @@ -4973,10 +5010,16 @@ private void checkFailedNodesList() { * Checks and flushes custom event messages if no nodes are attempting to join the grid. */ private void checkPendingCustomMessages() { - if (joiningNodes.isEmpty() && isLocalNodeCoordinator()) { + boolean joiningEmpty; + + synchronized (mux) { + joiningEmpty = joiningNodes.isEmpty(); + } + + if (joiningEmpty && isLocalNodeCoordinator()) { TcpDiscoveryCustomEventMessage msg; - while ((msg = pendingCustomMsgs.poll()) != null) { + while ((msg = pollPendingCustomeMessage()) != null) { processCustomMessage(msg); if (msg.verified()) @@ -4985,6 +5028,15 @@ private void checkPendingCustomMessages() { } } + /** + * @return Pending custom message. + */ + @Nullable private TcpDiscoveryCustomEventMessage pollPendingCustomeMessage() { + synchronized (mux) { + return pendingCustomMsgs.poll(); + } + } + /** * @param msg Custom message. */ @@ -5134,8 +5186,12 @@ private class TcpServer extends IgniteSpiThread { else srvrSock = new ServerSocket(port, 0, spi.locHost); - if (log.isInfoEnabled()) - log.info("Successfully bound to TCP port [port=" + port + ", localHost=" + spi.locHost + ']'); + if (log.isInfoEnabled()) { + log.info("Successfully bound to TCP port [port=" + port + + ", localHost=" + spi.locHost + + ", locNodeId=" + spi.ignite().configuration().getNodeId() + + ']'); + } return; } @@ -5450,7 +5506,7 @@ else if (e.hasCause(ObjectStreamException.class) || return; } - long socketTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() : + long sockTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() : spi.getSocketTimeout(); while (!isInterrupted()) { @@ -5460,8 +5516,10 @@ else if (e.hasCause(ObjectStreamException.class) || msg.senderNodeId(nodeId); - if (log.isDebugEnabled()) - log.debug("Message has been received: " + msg); + DebugLogger debugLog = messageLogger(msg); + + if (debugLog.isDebugEnabled()) + debugLog.debug("Message has been received: " + msg); spi.stats.onMessageReceived(msg); @@ -5469,7 +5527,7 @@ else if (e.hasCause(ObjectStreamException.class) || debugLog(msg, "Message has been received: " + msg); if (msg instanceof TcpDiscoveryConnectionCheckMessage) { - spi.writeToSocket(msg, sock, RES_OK, socketTimeout); + spi.writeToSocket(msg, sock, RES_OK, sockTimeout); continue; } @@ -5491,7 +5549,7 @@ else if (msg instanceof TcpDiscoveryClientReconnectMessage) { TcpDiscoverySpiState state = spiStateCopy(); if (state == CONNECTED) { - spi.writeToSocket(msg, sock, RES_OK, socketTimeout); + spi.writeToSocket(msg, sock, RES_OK, sockTimeout); if (clientMsgWrk.getState() == State.NEW) clientMsgWrk.start(); @@ -5501,7 +5559,7 @@ else if (msg instanceof TcpDiscoveryClientReconnectMessage) { continue; } else { - spi.writeToSocket(msg, sock, RES_CONTINUE_JOIN, socketTimeout); + spi.writeToSocket(msg, sock, RES_CONTINUE_JOIN, sockTimeout); break; } @@ -5509,7 +5567,7 @@ else if (msg instanceof TcpDiscoveryClientReconnectMessage) { } else if (msg instanceof TcpDiscoveryDuplicateIdMessage) { // Send receipt back. - spi.writeToSocket(msg, sock, RES_OK, socketTimeout); + spi.writeToSocket(msg, sock, RES_OK, sockTimeout); boolean ignored = false; @@ -5538,7 +5596,7 @@ else if (msg instanceof TcpDiscoveryDuplicateIdMessage) { } else if (msg instanceof TcpDiscoveryAuthFailedMessage) { // Send receipt back. - spi.writeToSocket(msg, sock, RES_OK, socketTimeout); + spi.writeToSocket(msg, sock, RES_OK, sockTimeout); boolean ignored = false; @@ -5567,7 +5625,7 @@ else if (msg instanceof TcpDiscoveryAuthFailedMessage) { } else if (msg instanceof TcpDiscoveryCheckFailedMessage) { // Send receipt back. - spi.writeToSocket(msg, sock, RES_OK, socketTimeout); + spi.writeToSocket(msg, sock, RES_OK, sockTimeout); boolean ignored = false; @@ -5596,7 +5654,7 @@ else if (msg instanceof TcpDiscoveryCheckFailedMessage) { } else if (msg instanceof TcpDiscoveryLoopbackProblemMessage) { // Send receipt back. - spi.writeToSocket(msg, sock, RES_OK, socketTimeout); + spi.writeToSocket(msg, sock, RES_OK, sockTimeout); boolean ignored = false; @@ -5650,7 +5708,7 @@ else if (msg instanceof TcpDiscoveryLoopbackProblemMessage) { clientMsgWrk.addMessage(ack); } else - spi.writeToSocket(msg, sock, RES_OK, socketTimeout); + spi.writeToSocket(msg, sock, RES_OK, sockTimeout); if (heartbeatMsg != null) processClientHeartbeatMessage(heartbeatMsg); @@ -5914,7 +5972,7 @@ void metrics(ClusterMetrics metrics) { /** * @param msg Message. */ - public void addMessage(TcpDiscoveryAbstractMessage msg) { + void addMessage(TcpDiscoveryAbstractMessage msg) { addMessage(msg, null); } @@ -5922,7 +5980,7 @@ public void addMessage(TcpDiscoveryAbstractMessage msg) { * @param msg Message. * @param msgBytes Optional message bytes. */ - public void addMessage(TcpDiscoveryAbstractMessage msg, @Nullable byte[] msgBytes) { + void addMessage(TcpDiscoveryAbstractMessage msg, @Nullable byte[] msgBytes) { T2 t = new T2<>(msg, msgBytes); if (msg.highPriority()) @@ -5930,6 +5988,8 @@ public void addMessage(TcpDiscoveryAbstractMessage msg, @Nullable byte[] msgByte else queue.add(t); + DebugLogger log = messageLogger(msg); + if (log.isDebugEnabled()) log.debug("Message has been added to client queue: " + msg); } @@ -5948,22 +6008,24 @@ public void addMessage(TcpDiscoveryAbstractMessage msg, @Nullable byte[] msgByte if (msgBytes == null) msgBytes = U.marshal(spi.marshaller(), msg); + DebugLogger msgLog = messageLogger(msg); + if (msg instanceof TcpDiscoveryClientAckResponse) { if (clientVer == null) { ClusterNode node = spi.getNode(clientNodeId); if (node != null) clientVer = IgniteUtils.productVersion(node); - else if (log.isDebugEnabled()) - log.debug("Skip sending message ack to client, fail to get client node " + + else if (msgLog.isDebugEnabled()) + msgLog.debug("Skip sending message ack to client, fail to get client node " + "[sock=" + sock + ", locNodeId=" + getLocalNodeId() + ", rmtNodeId=" + clientNodeId + ", msg=" + msg + ']'); } if (clientVer != null && clientVer.compareTo(TcpDiscoveryClientAckResponse.CLIENT_ACK_SINCE_VERSION) >= 0) { - if (log.isDebugEnabled()) - log.debug("Sending message ack to client [sock=" + sock + ", locNodeId=" + if (msgLog.isDebugEnabled()) + msgLog.debug("Sending message ack to client [sock=" + sock + ", locNodeId=" + getLocalNodeId() + ", rmtNodeId=" + clientNodeId + ", msg=" + msg + ']'); spi.writeToSocket(sock, msg, msgBytes, spi.failureDetectionTimeoutEnabled() ? @@ -5971,8 +6033,8 @@ else if (log.isDebugEnabled()) } } else { - if (log.isDebugEnabled()) - log.debug("Redirecting message to client [sock=" + sock + ", locNodeId=" + if (msgLog.isDebugEnabled()) + msgLog.debug("Redirecting message to client [sock=" + sock + ", locNodeId=" + getLocalNodeId() + ", rmtNodeId=" + clientNodeId + ", msg=" + msg + ']'); assert topologyInitialized(msg) : msg; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java index 30b83e54b2731..0816cbca120eb 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java @@ -70,7 +70,33 @@ abstract class TcpDiscoveryImpl { /** Received messages. */ @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - protected ConcurrentLinkedDeque debugLog; + protected ConcurrentLinkedDeque debugLogQ; + + /** */ + protected final ServerImpl.DebugLogger debugLog = new DebugLogger() { + /** {@inheritDoc} */ + @Override public boolean isDebugEnabled() { + return log.isDebugEnabled(); + } + + /** {@inheritDoc} */ + @Override public void debug(String msg) { + log.debug(msg); + } + }; + + /** */ + protected final ServerImpl.DebugLogger traceLog = new DebugLogger() { + /** {@inheritDoc} */ + @Override public boolean isDebugEnabled() { + return log.isTraceEnabled(); + } + + /** {@inheritDoc} */ + @Override public void debug(String msg) { + log.trace(msg); + } + }; /** * @param spi Adapter. @@ -111,12 +137,12 @@ protected void debugLog(@Nullable TcpDiscoveryAbstractMessage discoMsg, String m "-" + locNode.internalOrder() + "] " + msg; - debugLog.add(msg0); + debugLogQ.add(msg0); - int delta = debugLog.size() - debugMsgHist; + int delta = debugLogQ.size() - debugMsgHist; - for (int i = 0; i < delta && debugLog.size() > debugMsgHist; i++) - debugLog.poll(); + for (int i = 0; i < delta && debugLogQ.size() > debugMsgHist; i++) + debugLogQ.poll(); } /** @@ -326,4 +352,27 @@ protected static List toOrderedList(Collection addrs) return res; } + + /** + * @param msg Message. + * @return Message logger. + */ + protected final DebugLogger messageLogger(TcpDiscoveryAbstractMessage msg) { + return msg.traceLogLevel() ? traceLog : debugLog; + } + + /** + * + */ + interface DebugLogger { + /** + * @return {@code True} if debug logging is enabled. + */ + boolean isDebugEnabled(); + + /** + * @param msg Message to log. + */ + void debug(String msg); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java index 24f2a5af3b4c9..39170eadf7cf0 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java @@ -98,6 +98,13 @@ protected TcpDiscoveryAbstractMessage(TcpDiscoveryAbstractMessage msg) { this.pendingIdx = msg.pendingIdx; } + /** + * @return + */ + public boolean traceLogLevel() { + return false; + } + /** * Gets creator node. * diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java index 65057654e6d8a..0a656d57128e8 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java @@ -52,6 +52,11 @@ public IgniteUuid messageId() { return msgId; } + /** {@inheritDoc} */ + @Override public boolean traceLogLevel() { + return true; + } + /** {@inheritDoc} */ @Override public boolean highPriority() { return true; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientHeartbeatMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientHeartbeatMessage.java index 3993de0d8bf24..ade54685cac4e 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientHeartbeatMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientHeartbeatMessage.java @@ -25,7 +25,7 @@ /** * Heartbeat message. *

          - * Client sends his hearbeats in this message. + * Client sends his heartbeats in this message. */ public class TcpDiscoveryClientHeartbeatMessage extends TcpDiscoveryAbstractMessage { /** */ @@ -60,6 +60,11 @@ public ClusterMetrics metrics() { return true; } + /** {@inheritDoc} */ + @Override public boolean traceLogLevel() { + return true; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(TcpDiscoveryClientHeartbeatMessage.class, this, "super", super.toString()); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java index a1529365fd097..7793a3a054424 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java @@ -49,6 +49,11 @@ public TcpDiscoveryConnectionCheckMessage(TcpDiscoveryNode creatorNode) { super(creatorNode.id()); } + /** {@inheritDoc} */ + @Override public boolean traceLogLevel() { + return true; + } + /** {@inheritDoc} */ @Override public void writeExternal(ObjectOutput out) throws IOException { // This method has been left empty intentionally to keep message size at min. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHeartbeatMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHeartbeatMessage.java index 338e3f5a2ff1a..0ae253ab51c0a 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHeartbeatMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHeartbeatMessage.java @@ -213,6 +213,11 @@ public void addClientNodeId(UUID clientNodeId) { clientNodeIds.add(clientNodeId); } + /** {@inheritDoc} */ + @Override public boolean traceLogLevel() { + return true; + } + /** {@inheritDoc} */ @Override public boolean highPriority() { return true; From b1b87036fa180df80e07db00f3674eba6089fe71 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 31 Oct 2016 18:26:25 +0300 Subject: [PATCH 292/487] .NET: Fix ASP.NET session state provider test failures due to incorrect merge --- .../processors/platform/cache/PlatformCache.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index 558a9b3538525..c552bae6902aa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -873,9 +873,9 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc, IgniteCache cache long update = reader.readLong(); long access = reader.readLong(); - IgniteCache cache0 = cache.withExpiryPolicy(new InteropExpiryPolicy(create, update, access)); + IgniteCache cache0 = rawCache.withExpiryPolicy(new InteropExpiryPolicy(create, update, access)); - return new PlatformCache(platformCtx, cache0, keepBinary); + return copy(cache0, keepBinary); } case OP_LOC_ITERATOR: { @@ -1001,14 +1001,14 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc, IgniteCache cache if (cache.isAsync()) return this; - return new PlatformCache(platformCtx, (IgniteCache)cache.withAsync(), keepBinary); + return copy(rawCache.withAsync(), keepBinary); } case OP_WITH_KEEP_BINARY: { if (keepBinary) return this; - return new PlatformCache(platformCtx, cache.withKeepBinary(), true); + return copy(rawCache.withKeepBinary(), true); } case OP_WITH_NO_RETRIES: { @@ -1017,14 +1017,14 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc, IgniteCache cache if (opCtx != null && opCtx.noRetries()) return this; - return new PlatformCache(platformCtx, cache.withNoRetries(), keepBinary); + return copy(rawCache.withNoRetries(), keepBinary); } case OP_WITH_SKIP_STORE: { if (cache.delegate().skipStore()) return this; - return new PlatformCache(platformCtx, cache.withSkipStore(), keepBinary); + return copy(rawCache.withSkipStore(), keepBinary); } case OP_ITERATOR: { From 8add10e2ac9a91b0d7e473e6f40ff14d650abf96 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 31 Oct 2016 18:26:25 +0300 Subject: [PATCH 293/487] .NET: Fix ASP.NET session state provider test failures due to incorrect merge --- .../processors/platform/cache/PlatformCache.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index 5a6e9a2880ab3..be50e5d8f8366 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -856,9 +856,9 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc, IgniteCache cache long update = reader.readLong(); long access = reader.readLong(); - IgniteCache cache0 = cache.withExpiryPolicy(new InteropExpiryPolicy(create, update, access)); + IgniteCache cache0 = rawCache.withExpiryPolicy(new InteropExpiryPolicy(create, update, access)); - return new PlatformCache(platformCtx, cache0, keepBinary); + return copy(cache0, keepBinary); } case OP_LOC_ITERATOR: { @@ -984,14 +984,14 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc, IgniteCache cache if (cache.isAsync()) return this; - return new PlatformCache(platformCtx, (IgniteCache)cache.withAsync(), keepBinary); + return copy(rawCache.withAsync(), keepBinary); } case OP_WITH_KEEP_BINARY: { if (keepBinary) return this; - return new PlatformCache(platformCtx, cache.withKeepBinary(), true); + return copy(rawCache.withKeepBinary(), true); } case OP_WITH_NO_RETRIES: { @@ -1000,14 +1000,14 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc, IgniteCache cache if (opCtx != null && opCtx.noRetries()) return this; - return new PlatformCache(platformCtx, cache.withNoRetries(), keepBinary); + return copy(rawCache.withNoRetries(), keepBinary); } case OP_WITH_SKIP_STORE: { if (cache.delegate().skipStore()) return this; - return new PlatformCache(platformCtx, cache.withSkipStore(), keepBinary); + return copy(rawCache.withSkipStore(), keepBinary); } case OP_ITERATOR: { From e00d53173e7e217f4e94388fda34ebd813c4e7c3 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 31 Oct 2016 18:37:49 +0300 Subject: [PATCH 294/487] .NET: Fix license header --- .../dotnet/Apache.Ignite.NLog/packages.config | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/packages.config b/modules/platforms/dotnet/Apache.Ignite.NLog/packages.config index 50aeb52c2b59a..15659a4dae2d3 100644 --- a/modules/platforms/dotnet/Apache.Ignite.NLog/packages.config +++ b/modules/platforms/dotnet/Apache.Ignite.NLog/packages.config @@ -1,4 +1,20 @@  + \ No newline at end of file From 260f9b2d01b16a98fcb7bc721573b95a72adade2 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 31 Oct 2016 18:37:49 +0300 Subject: [PATCH 295/487] .NET: Fix license header --- .../dotnet/Apache.Ignite.NLog/packages.config | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/packages.config b/modules/platforms/dotnet/Apache.Ignite.NLog/packages.config index 50aeb52c2b59a..15659a4dae2d3 100644 --- a/modules/platforms/dotnet/Apache.Ignite.NLog/packages.config +++ b/modules/platforms/dotnet/Apache.Ignite.NLog/packages.config @@ -1,4 +1,20 @@  + \ No newline at end of file From e7d7a59eff6da1cd62e8ffb64405cdf12607b49d Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 31 Oct 2016 18:43:15 +0300 Subject: [PATCH 296/487] IGNITE-4153 .NET: IgniteConfiguration.FailureDetectionTimeout property is missing --- .../utils/PlatformConfigurationUtils.java | 2 ++ .../IgniteConfigurationSerializerTest.cs | 3 ++- .../IgniteConfigurationTest.cs | 6 +++++- .../Apache.Ignite.Core/IgniteConfiguration.cs | 21 +++++++++++++++++++ .../IgniteConfigurationSection.xsd | 7 +++++++ 5 files changed, 37 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java index 1d2baf2b2596a..26374f0c220ba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java @@ -461,6 +461,7 @@ public static void readIgniteConfiguration(BinaryRawReaderEx in, IgniteConfigura String localHost = in.readString(); if (localHost != null) cfg.setLocalHost(localHost); if (in.readBoolean()) cfg.setDaemon(in.readBoolean()); if (in.readBoolean()) cfg.setLateAffinityAssignment(in.readBoolean()); + if (in.readBoolean()) cfg.setFailureDetectionTimeout(in.readLong()); readCacheConfigurations(in, cfg); readDiscoveryConfiguration(in, cfg); @@ -829,6 +830,7 @@ public static void writeIgniteConfiguration(BinaryRawWriter w, IgniteConfigurati w.writeString(cfg.getLocalHost()); w.writeBoolean(true); w.writeBoolean(cfg.isDaemon()); w.writeBoolean(true); w.writeBoolean(cfg.isLateAffinityAssignment()); + w.writeBoolean(true); w.writeLong(cfg.getFailureDetectionTimeout()); CacheConfiguration[] cacheCfg = cfg.getCacheConfiguration(); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs index 2486351f7a5e7..9fb5bbd96b1c2 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs @@ -598,7 +598,8 @@ private static IgniteConfiguration GetTestConfig() }, IsLateAffinityAssignment = false, SpringConfigUrl = "test", - Logger = new IgniteNLogLogger() + Logger = new IgniteNLogLogger(), + FailureDetectionTimeout = TimeSpan.FromMinutes(2) }; } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs index 0e2e49637fece..30bae478574ca 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs @@ -169,6 +169,8 @@ public void TestAllConfigurationProperties() Assert.AreEqual(com.SocketSendBufferSize, resCom.SocketSendBufferSize); Assert.AreEqual(com.TcpNoDelay, resCom.TcpNoDelay); Assert.AreEqual(com.UnacknowledgedMessagesBufferSize, resCom.UnacknowledgedMessagesBufferSize); + + Assert.AreEqual(cfg.FailureDetectionTimeout, resCfg.FailureDetectionTimeout); } } @@ -385,6 +387,7 @@ private static void CheckDefaultProperties(IgniteConfiguration cfg) Assert.AreEqual(IgniteConfiguration.DefaultNetworkTimeout, cfg.NetworkTimeout); Assert.AreEqual(IgniteConfiguration.DefaultNetworkSendRetryCount, cfg.NetworkSendRetryCount); Assert.AreEqual(IgniteConfiguration.DefaultNetworkSendRetryDelay, cfg.NetworkSendRetryDelay); + Assert.AreEqual(IgniteConfiguration.DefaultFailureDetectionTimeout, cfg.FailureDetectionTimeout); } ///

          @@ -491,7 +494,8 @@ private static IgniteConfiguration GetCustomConfig() SlowClientQueueLimit = 98, SocketSendBufferSize = 2045, UnacknowledgedMessagesBufferSize = 3450 - } + }, + FailureDetectionTimeout = TimeSpan.FromSeconds(3.5) }; } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs index b045c5a24b16a..68433f74591b1 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs @@ -90,6 +90,11 @@ public class IgniteConfiguration /// public static readonly TimeSpan DefaultNetworkSendRetryDelay = TimeSpan.FromMilliseconds(1000); + /// + /// Default failure detection timeout. + /// + public static readonly TimeSpan DefaultFailureDetectionTimeout = TimeSpan.FromSeconds(10); + /** */ private TimeSpan? _metricsExpireTime; @@ -120,6 +125,9 @@ public class IgniteConfiguration /** */ private bool? _clientMode; + /** */ + private TimeSpan? _failureDetectionTimeout; + /// /// Default network retry count. /// @@ -195,6 +203,7 @@ internal void Write(BinaryWriter writer) writer.WriteString(Localhost); writer.WriteBooleanNullable(_isDaemon); writer.WriteBooleanNullable(_isLateAffinityAssignment); + writer.WriteTimeSpanAsLongNullable(_failureDetectionTimeout); // Cache config var caches = CacheConfiguration; @@ -329,6 +338,7 @@ private void ReadCore(BinaryReader r) Localhost = r.ReadString(); _isDaemon = r.ReadBooleanNullable(); _isLateAffinityAssignment = r.ReadBooleanNullable(); + _failureDetectionTimeout = r.ReadTimeSpanNullable(); // Cache config var cacheCfgCount = r.ReadInt(); @@ -769,5 +779,16 @@ public static IgniteConfiguration FromXml(string xml) /// or logs to console otherwise. ///
          public ILogger Logger { get; set; } + + /// + /// Gets or sets the failure detection timeout used by + /// and . + /// + [DefaultValue(typeof(TimeSpan), "00:00:10")] + public TimeSpan FailureDetectionTimeout + { + get { return _failureDetectionTimeout ?? DefaultFailureDetectionTimeout; } + set { _failureDetectionTimeout = value; } + } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd index 40e30558dab01..8764b60e54497 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd @@ -1242,6 +1242,13 @@ + + + + Failure detection timeout used by discovery and communication subsystems. + + + From d0f4b23600c655dfb94f6fe0205d1cca7cf19d36 Mon Sep 17 00:00:00 2001 From: AKuznetsov Date: Mon, 31 Oct 2016 23:06:58 +0700 Subject: [PATCH 297/487] Fixed license. --- .../app/modules/nodes/nodes-dialog.scss | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/modules/web-console/frontend/app/modules/nodes/nodes-dialog.scss b/modules/web-console/frontend/app/modules/nodes/nodes-dialog.scss index 0c65e54ef3f2f..8145dbc4a3f98 100644 --- a/modules/web-console/frontend/app/modules/nodes/nodes-dialog.scss +++ b/modules/web-console/frontend/app/modules/nodes/nodes-dialog.scss @@ -1,3 +1,20 @@ +/* + * 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. + */ + .ignite-nodes-dialog { label { font-size: 18px; From b8b9abe863ed8139553a9ad7013dfad5a363b4da Mon Sep 17 00:00:00 2001 From: devozerov Date: Mon, 31 Oct 2016 21:31:22 +0300 Subject: [PATCH 298/487] Fixes after merge. --- .../apache/ignite/cache/query/SqlQuery.java | 5 +- .../processors/cache/QueryCursorImpl.java | 18 +- .../closure/GridClosureProcessor.java | 1 - .../processors/query/GridQueryCancel.java | 60 ++-- .../messages/GridQueryFailResponse.java | 13 +- .../junits/GridTestKernalContext.java | 14 +- .../query/h2/twostep/GridMergeIndex.java | 12 +- .../h2/twostep/msg/GridH2QueryRequest.java | 42 ++- ...tedQueryStopOnCancelOrTimeoutSelfTest.java | 57 ++-- ...eQueryAbstractDistributedJoinSelfTest.java | 290 +++++++++++++++++- ...eryNodeRestartDistributedJoinSelfTest.java | 262 +--------------- .../IgniteCacheQueryNodeRestartSelfTest2.java | 2 +- ...ancelOrTimeoutDistributedJoinSelfTest.java | 137 ++++++++- .../IgniteCacheQuerySelfTestSuite.java | 4 +- .../IgniteCacheQuerySelfTestSuite2.java | 2 + 15 files changed, 564 insertions(+), 355 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java index 83e171d4ab43f..3b8fe6d55c577 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java @@ -17,14 +17,15 @@ package org.apache.ignite.cache.query; -import java.util.concurrent.TimeUnit; -import javax.cache.Cache; import org.apache.ignite.IgniteCache; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; +import javax.cache.Cache; +import java.util.concurrent.TimeUnit; + /** * SQL Query. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java index f68426ee3c93a..f93a74735a7cc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java @@ -17,11 +17,6 @@ package org.apache.ignite.internal.processors.cache; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; -import javax.cache.CacheException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.cache.query.QueryCancelledException; @@ -29,10 +24,13 @@ import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; -import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.CLOSED; -import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.EXECUTION; -import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.IDLE; -import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.RESULT_READY; +import javax.cache.CacheException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; + +import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.*; /** * Query cursor implementation. @@ -40,7 +38,7 @@ public class QueryCursorImpl implements QueryCursorEx { /** */ private final static AtomicReferenceFieldUpdater STATE_UPDATER = - AtomicReferenceFieldUpdater.newUpdater(QueryCursorImpl.class, State.class, "state"); + AtomicReferenceFieldUpdater.newUpdater(QueryCursorImpl.class, State.class, "state"); /** Query executor. */ private Iterable iterExec; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java index 252540e56b9a5..9d295d3ced3b8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java @@ -27,7 +27,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Callable; -import java.util.concurrent.Executor; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import org.apache.ignite.IgniteCheckedException; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java index 47f1208645055..7391f396e3a2f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java @@ -18,49 +18,57 @@ package org.apache.ignite.internal.processors.query; import org.apache.ignite.cache.query.QueryCancelledException; -import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.util.typedef.internal.U; + +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; /** * Holds query cancel state. */ public class GridQueryCancel { - /** */ - private volatile boolean cancelled; + /** No-op runnable indicating cancelled state. */ + private static final Runnable CANCELLED = new Runnable() { + @Override public void run() { + // No-op. + } + }; /** */ - private volatile boolean completed; + private static final AtomicReferenceFieldUpdater STATE_UPDATER = + AtomicReferenceFieldUpdater.newUpdater(GridQueryCancel.class, Runnable.class, "clo"); /** */ private volatile Runnable clo; /** - * Sets a cancel closure. The closure must be idempotent to multiple invocations. + * Sets a cancel closure. * * @param clo Clo. */ - public void set(Runnable clo) throws QueryCancelledException{ - checkCancelled(); + public void set(Runnable clo) throws QueryCancelledException { + assert clo != null; - this.clo = clo; + while(true) { + Runnable tmp = this.clo; + + if (tmp == CANCELLED) + throw new QueryCancelledException(); + + if (STATE_UPDATER.compareAndSet(this, tmp, clo)) + return; + } } /** - * Spins until a query is completed. - * Only one thread can enter this method. - * This is guaranteed by {@link org.apache.ignite.internal.processors.cache.QueryCursorImpl} + * Executes cancel closure. */ public void cancel() { - cancelled = true; - - int attempt = 0; + while(true) { + Runnable tmp = this.clo; - while (!completed) { - if (clo != null) clo.run(); + if (STATE_UPDATER.compareAndSet(this, tmp, CANCELLED)) { + if (tmp != null) + tmp.run(); - try { - U.sleep(++attempt * 10); - } catch (IgniteInterruptedCheckedException ignored) { return; } } @@ -69,16 +77,8 @@ public void cancel() { /** * Stops query execution if a user requested cancel. */ - public void checkCancelled() throws QueryCancelledException{ - if (cancelled) + public void checkCancelled() throws QueryCancelledException { + if (clo == CANCELLED) throw new QueryCancelledException(); } - - /** - * Sets completed state. - * The method must be called then a query is completed by any reason, typically in final block. - */ - public void setCompleted() { - completed = true; - } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java index 261241e58543e..7554ae9f0946a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java @@ -17,13 +17,14 @@ package org.apache.ignite.internal.processors.query.h2.twostep.messages; -import java.nio.ByteBuffer; import org.apache.ignite.cache.query.QueryCancelledException; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import java.nio.ByteBuffer; + /** * Error message. */ @@ -113,13 +114,13 @@ public byte failCode() { writer.incrementState(); case 1: - if (!writer.writeByte("failCode", failCode)) + if (!writer.writeLong("qryReqId", qryReqId)) return false; writer.incrementState(); case 2: - if (!writer.writeLong("qryReqId", qryReqId)) + if (!writer.writeByte("failCode", failCode)) return false; writer.incrementState(); @@ -146,7 +147,7 @@ public byte failCode() { reader.incrementState(); case 1: - failCode = reader.readByte("failCode"); + qryReqId = reader.readLong("qryReqId"); if (!reader.isLastRead()) return false; @@ -154,7 +155,7 @@ public byte failCode() { reader.incrementState(); case 2: - qryReqId = reader.readLong("qryReqId"); + failCode = reader.readByte("failCode"); if (!reader.isLastRead()) return false; @@ -175,4 +176,4 @@ public byte failCode() { @Override public byte fieldsCount() { return 3; } -} +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java index cba67e0d06005..03138c32049eb 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java @@ -17,21 +17,18 @@ package org.apache.ignite.testframework.junits; -import java.util.List; -import java.util.ListIterator; -import java.util.concurrent.ExecutorService; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.GridComponent; -import org.apache.ignite.internal.GridKernalContextImpl; -import org.apache.ignite.internal.GridKernalGatewayImpl; -import org.apache.ignite.internal.GridLoggerProxy; -import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.*; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; +import java.util.List; +import java.util.ListIterator; +import java.util.concurrent.ExecutorService; + /** * Test context. */ @@ -62,6 +59,7 @@ public GridTestKernalContext(IgniteLogger log, IgniteConfiguration cfg) throws I null, null, null, + null, U.allPluginProviders()); GridTestUtils.setFieldValue(grid(), "cfg", config()); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java index 796ea66a28202..444ea82a012a2 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java @@ -20,7 +20,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.ConcurrentModificationException; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -113,7 +112,7 @@ public Set sources() { protected final void checkSourceNodesAlive() { for (UUID nodeId : sources()) { if (!ctx.discovery().alive(nodeId)) { - fail(nodeId); + fail(nodeId, null); return; } @@ -174,11 +173,18 @@ public void fail(final CacheException e) { /** * @param nodeId Node ID. */ - public void fail(UUID nodeId) { + public void fail(UUID nodeId, final CacheException e) { addPage0(new GridResultPage(null, nodeId, null) { @Override public boolean isFail() { return true; } + + @Override public void fetchNextPage() { + if (e == null) + super.fetchNextPage(); + else + throw e; + } }); } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java index dc82b2c8f2607..884173f469452 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java @@ -45,8 +45,8 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable { private static final long serialVersionUID = 0L; /** - * Map query will not destroy context until explicit query cancel request - * will be received because distributed join requests can be received. + * Map query will not destroy context until explicit query cancel request will be received because distributed join + * requests can be received. */ public static int FLAG_DISTRIBUTED_JOINS = 1; @@ -82,6 +82,9 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable { @GridDirectCollection(String.class) private Collection tbls; + /** */ + private int timeout; + /** * @param tbls Tables. * @return {@code this}. @@ -153,7 +156,7 @@ public AffinityTopologyVersion topologyVersion() { /** * @return Explicit partitions mapping. */ - public Map partitions() { + public Map partitions() { return parts; } @@ -161,7 +164,7 @@ public Map partitions() { * @param parts Explicit partitions mapping. * @return {@code this}. */ - public GridH2QueryRequest partitions(Map parts) { + public GridH2QueryRequest partitions(Map parts) { this.parts = parts; return this; @@ -219,6 +222,23 @@ public boolean isFlagSet(int flags) { return (this.flags & flags) == flags; } + /** + * @return Timeout. + */ + public int timeout() { + return timeout; + } + + /** + * @param timeout New timeout. + * @return {@code this}. + */ + public GridH2QueryRequest timeout(int timeout) { + this.timeout = timeout; + + return this; + } + /** {@inheritDoc} */ @Override public void marshall(Marshaller m) { if (F.isEmpty(qrys)) @@ -297,6 +317,11 @@ public boolean isFlagSet(int flags) { writer.incrementState(); + case 8: + if (!writer.writeInt("timeout", timeout)) + return false; + + writer.incrementState(); } return true; @@ -374,6 +399,13 @@ public boolean isFlagSet(int flags) { reader.incrementState(); + case 8: + timeout = reader.readInt("timeout"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); } return reader.afterMessageRead(GridH2QueryRequest.class); @@ -386,7 +418,7 @@ public boolean isFlagSet(int flags) { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 8; + return 9; } /** {@inheritDoc} */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java index 0f60db25834fd..a92bf2b7a1bc6 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java @@ -19,6 +19,7 @@ import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.UUID; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; @@ -43,7 +44,7 @@ */ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends GridCommonAbstractTest { /** Grids count. */ - private static final int GRIDS_COUNT = 3; + private static final int GRIDS_CNT = 3; /** IP finder. */ private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); @@ -55,19 +56,19 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr public static final int VAL_SIZE = 16; /** */ - private static final String QUERY_1 = "select a._val, b._val from String a, String b"; + private static final String QRY_1 = "select a._val, b._val from String a, String b"; /** */ - private static final String QUERY_2 = "select a._key, count(*) from String a group by a._key"; + private static final String QRY_2 = "select a._key, count(*) from String a group by a._key"; /** */ - private static final String QUERY_3 = "select a._val from String a"; + private static final String QRY_3 = "select a._val from String a"; /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); - startGridsMultiThreaded(GRIDS_COUNT); + startGridsMultiThreaded(GRIDS_CNT); } /** {@inheritDoc} */ @@ -97,82 +98,82 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr /** */ public void testRemoteQueryExecutionTimeout() throws Exception { - testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 500, TimeUnit.MILLISECONDS, true); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 500, TimeUnit.MILLISECONDS, true); } /** */ public void testRemoteQueryWithMergeTableTimeout() throws Exception { - testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 500, TimeUnit.MILLISECONDS, true); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, true); } /** */ public void testRemoteQueryExecutionCancel0() throws Exception { - testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 1, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.MILLISECONDS, false); } /** */ public void testRemoteQueryExecutionCancel1() throws Exception { - testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 500, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 500, TimeUnit.MILLISECONDS, false); } /** */ public void testRemoteQueryExecutionCancel2() throws Exception { - testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 1, TimeUnit.SECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.SECONDS, false); } /** */ public void testRemoteQueryExecutionCancel3() throws Exception { - testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 3, TimeUnit.SECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 3, TimeUnit.SECONDS, false); } /** */ public void testRemoteQueryWithMergeTableCancel0() throws Exception { - testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 1, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1, TimeUnit.MILLISECONDS, false); } /** */ public void testRemoteQueryWithMergeTableCancel1() throws Exception { - testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 500, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, false); } /** */ public void testRemoteQueryWithMergeTableCancel2() throws Exception { - testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 1_500, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1_500, TimeUnit.MILLISECONDS, false); } /** */ public void testRemoteQueryWithMergeTableCancel3() throws Exception { - testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 3, TimeUnit.SECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.SECONDS, false); } /** */ public void testRemoteQueryWithoutMergeTableCancel0() throws Exception { - testQuery(CACHE_SIZE, VAL_SIZE, QUERY_3, 1, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1, TimeUnit.MILLISECONDS, false); } /** */ public void testRemoteQueryWithoutMergeTableCancel1() throws Exception { - testQuery(CACHE_SIZE, VAL_SIZE, QUERY_3, 500, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 500, TimeUnit.MILLISECONDS, false); } /** */ public void testRemoteQueryWithoutMergeTableCancel2() throws Exception { - testQuery(CACHE_SIZE, VAL_SIZE, QUERY_3, 1_000, TimeUnit.MILLISECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1_000, TimeUnit.MILLISECONDS, false); } /** */ public void testRemoteQueryWithoutMergeTableCancel3() throws Exception { - testQuery(CACHE_SIZE, VAL_SIZE, QUERY_3, 3, TimeUnit.SECONDS, false); + testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false); } /** */ public void testRemoteQueryAlreadyFinishedStop() throws Exception { - testQuery(100, VAL_SIZE, QUERY_3, 3, TimeUnit.SECONDS, false); + testQueryCancel(100, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false); } /** */ - private void testQuery(int keyCnt, int valSize, String sql, int timeoutUnits, TimeUnit timeUnit, - boolean timeout) throws Exception { + private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUnits, TimeUnit timeUnit, + boolean timeout) throws Exception { try (Ignite client = startGrid("client")) { IgniteCache cache = client.cache(null); @@ -230,19 +231,23 @@ private void testQuery(int keyCnt, int valSize, String sql, int timeoutUnits, Ti /** * Validates clean state on all participating nodes after query cancellation. */ + @SuppressWarnings("unchecked") private void checkCleanState() { - for (int i = 0; i < GRIDS_COUNT; i++) { + for (int i = 0; i < GRIDS_CNT; i++) { IgniteEx grid = grid(i); // Validate everything was cleaned up. - ConcurrentMap> map = U.field(((IgniteH2Indexing)U.field((Object)U.field( + ConcurrentMap map = U.field(((IgniteH2Indexing)U.field(U.field( grid.context(), "qryProc"), "idx")).mapQueryExecutor(), "qryRess"); String msg = "Map executor state is not cleared"; // TODO FIXME Current implementation leaves map entry for each node that's ever executed a query. - for (ConcurrentMap results : map.values()) - assertEquals(msg, 0, results.size()); + for (Object result : map.values()) { + Map m = U.field(result, "res"); + + assertEquals(msg, 0, m.size()); + } } } } \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java index be34a09865838..339e0d35b8719 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java @@ -1,7 +1,291 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.ignite.internal.processors.cache.distributed.near; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.cache.query.annotations.QuerySqlFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.util.GridRandom; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import java.io.Serializable; +import java.util.Random; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheRebalanceMode.SYNC; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; + /** - * Created by vozerov on 31.10.2016. + * Test for distributed queries with node restarts. */ -public class IgniteCacheQueryAbstractDistributedJoinSelfTest { -} +public class IgniteCacheQueryAbstractDistributedJoinSelfTest extends GridCommonAbstractTest { + /** */ + protected static final String QRY_0 = "select co._key, count(*) cnt\n" + + "from \"pe\".Person pe, \"pr\".Product pr, \"co\".Company co, \"pu\".Purchase pu\n" + + "where pe._key = pu.personId and pu.productId = pr._key and pr.companyId = co._key \n" + + "group by co._key order by cnt desc, co._key"; + + /** */ + protected static final String QRY_0_BROADCAST = "select co._key, count(*) cnt\n" + + "from \"co\".Company co, \"pr\".Product pr, \"pu\".Purchase pu, \"pe\".Person pe \n" + + "where pe._key = pu.personId and pu.productId = pr._key and pr.companyId = co._key \n" + + "group by co._key order by cnt desc, co._key"; + + /** */ + protected static final String QRY_1 = "select pr._key, co._key\n" + + "from \"pr\".Product pr, \"co\".Company co\n" + + "where pr.companyId = co._key\n" + + "order by co._key, pr._key "; + + /** */ + protected static final String QRY_1_BROADCAST = "select pr._key, co._key\n" + + "from \"co\".Company co, \"pr\".Product pr \n" + + "where pr.companyId = co._key\n" + + "order by co._key, pr._key "; + + /** */ + protected static final int GRID_CNT = 2; + + /** */ + private static final int PERS_CNT = 600; + + /** */ + private static final int PURCHASE_CNT = 6_000; + + /** */ + private static final int COMPANY_CNT = 25; + + /** */ + private static final int PRODUCT_CNT = 100; + + /** */ + private static TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration c = super.getConfiguration(gridName); + + if ("client".equals(gridName)) + c.setClientMode(true); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setIpFinder(IP_FINDER); + + c.setDiscoverySpi(disco); + + int i = 0; + + CacheConfiguration[] ccs = new CacheConfiguration[4]; + + for (String name : F.asList("pe", "pu", "co", "pr")) { + CacheConfiguration cc = defaultCacheConfiguration(); + + cc.setName(name); + cc.setCacheMode(PARTITIONED); + cc.setBackups(2); + cc.setWriteSynchronizationMode(FULL_SYNC); + cc.setAtomicityMode(TRANSACTIONAL); + cc.setRebalanceMode(SYNC); + cc.setLongQueryWarningTimeout(15_000); + cc.setAffinity(new RendezvousAffinityFunction(false, 60)); + + switch (name) { + case "pe": + cc.setIndexedTypes( + Integer.class, Person.class + ); + + break; + + case "pu": + cc.setIndexedTypes( + Integer.class, Purchase.class + ); + + break; + + case "co": + cc.setIndexedTypes( + Integer.class, Company.class + ); + + break; + + case "pr": + cc.setIndexedTypes( + Integer.class, Product.class + ); + + break; + } + + ccs[i++] = cc; + } + + c.setCacheConfiguration(ccs); + + return c; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGridsMultiThreaded(GRID_CNT); + + fillCaches(); + } + + /** + * + */ + private void fillCaches() { + IgniteCache co = grid(0).cache("co"); + + for (int i = 0; i < COMPANY_CNT; i++) + co.put(i, new Company(i)); + + IgniteCache pr = grid(0).cache("pr"); + + Random rnd = new GridRandom(); + + for (int i = 0; i < PRODUCT_CNT; i++) + pr.put(i, new Product(i, rnd.nextInt(COMPANY_CNT))); + + IgniteCache pe = grid(0).cache("pe"); + + for (int i = 0; i < PERS_CNT; i++) + pe.put(i, new Person(i)); + + IgniteCache pu = grid(0).cache("pu"); + + for (int i = 0; i < PURCHASE_CNT; i++) { + int persId = rnd.nextInt(PERS_CNT); + int prodId = rnd.nextInt(PRODUCT_CNT); + + pu.put(i, new Purchase(persId, prodId)); + } + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** + * + */ + protected static class Person implements Serializable { + /** */ + @QuerySqlField(index = true) + int id; + + /** + * @param id ID. + */ + Person(int id) { + this.id = id; + } + } + + /** + * + */ + protected static class Purchase implements Serializable { + /** */ + @QuerySqlField(index = true) + int personId; + + /** */ + @QuerySqlField(index = true) + int productId; + + /** + * @param personId Person ID. + * @param productId Product ID. + */ + Purchase(int personId, int productId) { + this.personId = personId; + this.productId = productId; + } + } + + /** + * + */ + protected static class Company implements Serializable { + /** */ + @QuerySqlField(index = true) + int id; + + /** + * @param id ID. + */ + Company(int id) { + this.id = id; + } + } + + /** + * + */ + protected static class Product implements Serializable { + /** */ + @QuerySqlField(index = true) + int id; + + /** */ + @QuerySqlField(index = true) + int companyId; + + /** + * @param id ID. + * @param companyId Company ID. + */ + Product(int id, int companyId) { + this.id = id; + this.companyId = companyId; + } + } + + /** */ + public static class Functions { + /** */ + @QuerySqlFunction + public static int sleep() { + try { + U.sleep(1_000); + } catch (IgniteInterruptedCheckedException ignored) { + // No-op. + } + + return 0; + } + } +} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java index 0e6806f519bf9..ced28bcc7a483 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java @@ -17,185 +17,25 @@ package org.apache.ignite.internal.processors.cache.distributed.near; -import java.io.Serializable; -import java.util.List; -import java.util.Random; -import java.util.concurrent.Callable; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicIntegerArray; -import javax.cache.CacheException; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.cache.query.SqlFieldsQuery; -import org.apache.ignite.cache.query.annotations.QuerySqlField; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.util.GridRandom; import org.apache.ignite.internal.util.typedef.CAX; -import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; -import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; -import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; -import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; -import static org.apache.ignite.cache.CacheMode.PARTITIONED; -import static org.apache.ignite.cache.CacheRebalanceMode.SYNC; -import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import javax.cache.CacheException; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicIntegerArray; /** * Test for distributed queries with node restarts. */ -public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends GridCommonAbstractTest { - /** */ - private static final String QRY_0 = "select co._key, count(*) cnt\n" + - "from \"pe\".Person pe, \"pr\".Product pr, \"co\".Company co, \"pu\".Purchase pu\n" + - "where pe._key = pu.personId and pu.productId = pr._key and pr.companyId = co._key \n" + - "group by co._key order by cnt desc, co._key"; - - /** */ - private static final String QRY_0_BROADCAST = "select co._key, count(*) cnt\n" + - "from \"co\".Company co, \"pr\".Product pr, \"pu\".Purchase pu, \"pe\".Person pe \n" + - "where pe._key = pu.personId and pu.productId = pr._key and pr.companyId = co._key \n" + - "group by co._key order by cnt desc, co._key"; - - /** */ - private static final String QRY_1 = "select pr._key, co._key\n" + - "from \"pr\".Product pr, \"co\".Company co\n" + - "where pr.companyId = co._key\n" + - "order by co._key, pr._key "; - - /** */ - private static final String QRY_1_BROADCAST = "select pr._key, co._key\n" + - "from \"co\".Company co, \"pr\".Product pr \n" + - "where pr.companyId = co._key\n" + - "order by co._key, pr._key "; - - /** */ - private static final int GRID_CNT = 6; - - /** */ - private static final int PERS_CNT = 600; - - /** */ - private static final int PURCHASE_CNT = 6000; - - /** */ - private static final int COMPANY_CNT = 25; - - /** */ - private static final int PRODUCT_CNT = 100; - - /** */ - private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration c = super.getConfiguration(gridName); - - TcpDiscoverySpi disco = new TcpDiscoverySpi(); - - disco.setIpFinder(ipFinder); - - c.setDiscoverySpi(disco); - - int i = 0; - - CacheConfiguration[] ccs = new CacheConfiguration[4]; - - for (String name : F.asList("pe", "pu", "co", "pr")) { - CacheConfiguration cc = defaultCacheConfiguration(); - - cc.setName(name); - cc.setCacheMode(PARTITIONED); - cc.setBackups(2); - cc.setWriteSynchronizationMode(FULL_SYNC); - cc.setAtomicityMode(TRANSACTIONAL); - cc.setRebalanceMode(SYNC); - cc.setLongQueryWarningTimeout(15_000); - cc.setAffinity(new RendezvousAffinityFunction(false, 60)); - - switch (name) { - case "pe": - cc.setIndexedTypes( - Integer.class, Person.class - ); - - break; - - case "pu": - cc.setIndexedTypes( - Integer.class, Purchase.class - ); - - break; - - case "co": - cc.setIndexedTypes( - Integer.class, Company.class - ); - - break; - - case "pr": - cc.setIndexedTypes( - Integer.class, Product.class - ); - - break; - } - - ccs[i++] = cc; - } - - c.setCacheConfiguration(ccs); - - return c; - } - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - super.beforeTestsStarted(); - - startGridsMultiThreaded(GRID_CNT); - - fillCaches(); - } - - /** - * - */ - private void fillCaches() { - IgniteCache co = grid(0).cache("co"); - - for (int i = 0; i < COMPANY_CNT; i++) - co.put(i, new Company(i)); - - IgniteCache pr = grid(0).cache("pr"); - - Random rnd = new GridRandom(); - - for (int i = 0; i < PRODUCT_CNT; i++) - pr.put(i, new Product(i, rnd.nextInt(COMPANY_CNT))); - - IgniteCache pe = grid(0).cache("pe"); - - for (int i = 0; i < PERS_CNT; i++) - pe.put(i, new Person(i)); - - IgniteCache pu = grid(0).cache("pu"); - - for (int i = 0; i < PURCHASE_CNT; i++) { - int persId = rnd.nextInt(PERS_CNT); - int prodId = rnd.nextInt(PRODUCT_CNT); - - pu.put(i, new Purchase(persId, prodId)); - } - } +public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends IgniteCacheQueryAbstractDistributedJoinSelfTest { /** * @throws Exception If failed. */ @@ -319,13 +159,6 @@ private void restarts(final boolean broadcastQry) throws Exception { else { IgniteCache cache = grid(g).cache("co"); - SqlFieldsQuery qry; - - if (broadcastQry) - qry = new SqlFieldsQuery(QRY_1_BROADCAST).setDistributedJoins(true).setEnforceJoinOrder(true); - else - qry = new SqlFieldsQuery(QRY_1).setDistributedJoins(true); - assertEquals(rRes, cache.query(qry1).getAll()); } @@ -392,85 +225,4 @@ private void restarts(final boolean broadcastQry) throws Exception { info("Stopped."); } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - stopAllGrids(); - } - - /** - * - */ - private static class Person implements Serializable { - /** */ - @QuerySqlField(index = true) - int id; - - /** - * @param id ID. - */ - Person(int id) { - this.id = id; - } - } - - /** - * - */ - private static class Purchase implements Serializable { - /** */ - @QuerySqlField(index = true) - int personId; - - /** */ - @QuerySqlField(index = true) - int productId; - - /** - * @param personId Person ID. - * @param productId Product ID. - */ - Purchase(int personId, int productId) { - this.personId = personId; - this.productId = productId; - } - } - - /** - * - */ - private static class Company implements Serializable { - /** */ - @QuerySqlField(index = true) - int id; - - /** - * @param id ID. - */ - Company(int id) { - this.id = id; - } - } - - /** - * - */ - private static class Product implements Serializable { - /** */ - @QuerySqlField(index = true) - int id; - - /** */ - @QuerySqlField(index = true) - int companyId; - - /** - * @param id ID. - * @param companyId Company ID. - */ - Product(int id, int companyId) { - this.id = id; - this.companyId = companyId; - } - } } \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java index 8b33a46b836da..154daa0a7a47c 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java @@ -267,7 +267,7 @@ public void testRestarts() throws Exception { continue; if (th.getMessage() != null && - th.getMessage().startsWith("Failed to fetch data from node:")) { + th.getMessage().startsWith("Failed to fetch data from node:")) { failedOnRemoteFetch = true; break; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java index 80bd62e533937..4baaf8f650d81 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java @@ -1,7 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.ignite.internal.processors.cache.distributed.near; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import javax.cache.CacheException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.query.QueryCancelledException; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; +import org.apache.ignite.internal.util.typedef.internal.U; + /** - * Created by vozerov on 31.10.2016. + * Test for cancel of query containing distributed joins. */ -public class IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest { -} +public class IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest extends IgniteCacheQueryAbstractDistributedJoinSelfTest { + /** */ + public void testCancel1() throws Exception { + testQueryCancel(grid(0), "pe", QRY_0, 1, TimeUnit.MILLISECONDS, false); + } + + /** */ + public void testCancel2() throws Exception { + testQueryCancel(grid(0), "pe", QRY_0, 50, TimeUnit.MILLISECONDS, false); + } + + /** */ + public void testCancel3() throws Exception { + testQueryCancel(grid(0), "pe", QRY_0, 100, TimeUnit.MILLISECONDS, false); + } + + /** */ + public void testCancel4() throws Exception { + testQueryCancel(grid(0), "pe", QRY_0, 500, TimeUnit.MILLISECONDS, false); + } + + /** */ + public void testTimeout1() throws Exception { + testQueryCancel(grid(0), "pe", QRY_0, 1, TimeUnit.MILLISECONDS, true); + } + + /** */ + public void testTimeout2() throws Exception { + testQueryCancel(grid(0), "pe", QRY_0, 50, TimeUnit.MILLISECONDS, true); + } + + /** */ + public void testTimeout3() throws Exception { + testQueryCancel(grid(0), "pe", QRY_0, 100, TimeUnit.MILLISECONDS, true); + } + + /** */ + public void testTimeout4() throws Exception { + testQueryCancel(grid(0), "pe", QRY_0, 500, TimeUnit.MILLISECONDS, true); + } + + /** */ + private void testQueryCancel(Ignite ignite, String cacheName, String sql, int timeoutUnits, TimeUnit timeUnit, + boolean timeout) throws Exception { + SqlFieldsQuery qry = new SqlFieldsQuery(sql).setDistributedJoins(true); + + IgniteCache cache = ignite.cache(cacheName); + + final QueryCursor> cursor; + if (timeout) { + qry.setTimeout(timeoutUnits, timeUnit); + + cursor = cache.query(qry); + } else { + cursor = cache.query(qry); + + ignite.scheduler().runLocal(new Runnable() { + @Override public void run() { + cursor.close(); + } + }, timeoutUnits, timeUnit); + } + + try (QueryCursor> ignored = cursor) { + cursor.iterator(); + } + catch (CacheException ex) { + log().error("Got expected exception", ex); + + assertTrue("Must throw correct exception", ex.getCause() instanceof QueryCancelledException); + } + + // Give some time to clean up. + Thread.sleep(TimeUnit.MILLISECONDS.convert(timeoutUnits, timeUnit) + 3_000); + + checkCleanState(); + } + + /** + * Validates clean state on all participating nodes after query cancellation. + */ + @SuppressWarnings("unchecked") + private void checkCleanState() { + for (int i = 0; i < GRID_CNT; i++) { + IgniteEx grid = grid(i); + + // Validate everything was cleaned up. + ConcurrentMap map = U.field(((IgniteH2Indexing) U.field(U.field( + grid.context(), "qryProc"), "idx")).mapQueryExecutor(), "qryRess"); + + String msg = "Map executor state is not cleared"; + + // TODO FIXME Current implementation leaves map entry for each node that's ever executed a query. + for (Object result : map.values()) { + Map m = U.field(result, "res"); + + assertEquals(msg, 0, m.size()); + } + } + } +} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java index b7e64034e073e..7f98d0a74d468 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java @@ -63,7 +63,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedSnapshotEnabledQuerySelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNoRebalanceSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeFailTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartDistributedJoinSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryAbstractDistributedJoinSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest2; import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQueryP2PDisabledSelfTest; @@ -125,7 +125,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCacheOffheapIndexScanTest.class); suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class); suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest2.class); - suite.addTestSuite(IgniteCacheQueryNodeRestartDistributedJoinSelfTest.class); + suite.addTestSuite(IgniteCacheQueryAbstractDistributedJoinSelfTest.class); suite.addTestSuite(IgniteCacheQueryNodeFailTest.class); suite.addTestSuite(IgniteCacheClientQueryReplicatedNodeRestartSelfTest.class); suite.addTestSuite(GridCacheCrossCacheQuerySelfTest.class); diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java index 5722c010d501c..be7523fe3a501 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java @@ -38,6 +38,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQuerySelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQuerySelfTest; import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest; @@ -100,6 +101,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.class); suite.addTestSuite(IgniteCacheDistributedQueryCancelSelfTest.class); suite.addTestSuite(IgniteCacheLocalQueryCancelOrTimeoutSelfTest.class); + suite.addTestSuite(IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.class); // Other. suite.addTestSuite(CacheQueryNewClientSelfTest.class); From 61ab650ecd353b4014a30f793090e9dab3519ad5 Mon Sep 17 00:00:00 2001 From: devozerov Date: Mon, 31 Oct 2016 21:33:59 +0300 Subject: [PATCH 299/487] Fixes after merge. --- .../org/apache/ignite/cache/query/SqlQuery.java | 5 ++--- .../processors/cache/QueryCursorImpl.java | 16 +++++++++------- .../processors/query/GridQueryCancel.java | 3 +-- .../twostep/messages/GridQueryFailResponse.java | 3 +-- .../junits/GridTestKernalContext.java | 13 ++++++++----- ...acheQueryAbstractDistributedJoinSelfTest.java | 5 ++--- 6 files changed, 23 insertions(+), 22 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java index 3b8fe6d55c577..83e171d4ab43f 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java @@ -17,15 +17,14 @@ package org.apache.ignite.cache.query; +import java.util.concurrent.TimeUnit; +import javax.cache.Cache; import org.apache.ignite.IgniteCache; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; -import javax.cache.Cache; -import java.util.concurrent.TimeUnit; - /** * SQL Query. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java index f93a74735a7cc..5a46d653d4113 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java @@ -17,6 +17,11 @@ package org.apache.ignite.internal.processors.cache; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import javax.cache.CacheException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.cache.query.QueryCancelledException; @@ -24,13 +29,10 @@ import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; -import javax.cache.CacheException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; - -import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.*; +import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.CLOSED; +import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.EXECUTION; +import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.IDLE; +import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.RESULT_READY; /** * Query cursor implementation. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java index 7391f396e3a2f..d49bcc7ae77f0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java @@ -17,9 +17,8 @@ package org.apache.ignite.internal.processors.query; -import org.apache.ignite.cache.query.QueryCancelledException; - import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import org.apache.ignite.cache.query.QueryCancelledException; /** * Holds query cancel state. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java index 7554ae9f0946a..0baf6ea88deea 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java @@ -17,14 +17,13 @@ package org.apache.ignite.internal.processors.query.h2.twostep.messages; +import java.nio.ByteBuffer; import org.apache.ignite.cache.query.QueryCancelledException; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; -import java.nio.ByteBuffer; - /** * Error message. */ diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java index 03138c32049eb..f9e2ff4e709df 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java @@ -17,18 +17,21 @@ package org.apache.ignite.testframework.junits; +import java.util.List; +import java.util.ListIterator; +import java.util.concurrent.ExecutorService; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.*; +import org.apache.ignite.internal.GridComponent; +import org.apache.ignite.internal.GridKernalContextImpl; +import org.apache.ignite.internal.GridKernalGatewayImpl; +import org.apache.ignite.internal.GridLoggerProxy; +import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; -import java.util.List; -import java.util.ListIterator; -import java.util.concurrent.ExecutorService; - /** * Test context. */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java index 339e0d35b8719..be3e22d5d07f8 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java @@ -17,6 +17,8 @@ package org.apache.ignite.internal.processors.cache.distributed.near; +import java.io.Serializable; +import java.util.Random; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.cache.query.annotations.QuerySqlField; @@ -32,9 +34,6 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import java.io.Serializable; -import java.util.Random; - import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheMode.PARTITIONED; import static org.apache.ignite.cache.CacheRebalanceMode.SYNC; From 4495efc2aa37ca31214558968fa83598317e830a Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 1 Nov 2016 13:22:30 +0300 Subject: [PATCH 300/487] IGNITE-4117 .NET: Fix ClientReconnectTask completion timing This closes #1195 --- .../platform/PlatformProcessorImpl.java | 15 +++- .../Apache.Ignite.Core.Tests/ReconnectTest.cs | 81 +++++++++++++++++-- 2 files changed, 88 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java index 548145e56639a..d875c7e332433 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java @@ -54,6 +54,7 @@ import org.apache.ignite.internal.processors.platform.transactions.PlatformTransactions; import org.apache.ignite.internal.processors.platform.utils.PlatformConfigurationUtils; import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; +import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFuture; @@ -101,6 +102,9 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf /** Cache extensions. */ private final PlatformCacheExtension[] cacheExts; + /** Cluster restart flag for the reconnect callback. */ + private volatile boolean clusterRestarted; + /** * Constructor. * @@ -378,11 +382,20 @@ public PlatformProcessorImpl(GridKernalContext ctx) { /** {@inheritDoc} */ @Override public void onDisconnected(IgniteFuture reconnectFut) throws IgniteCheckedException { platformCtx.gateway().onClientDisconnected(); + + // 1) onReconnected is called on all grid components. + // 2) After all of grid components have completed their reconnection, reconnectFut is completed. + reconnectFut.listen(new CI1>() { + @Override public void apply(IgniteFuture future) { + platformCtx.gateway().onClientReconnected(clusterRestarted); + } + }); } /** {@inheritDoc} */ @Override public IgniteInternalFuture onReconnected(boolean clusterRestarted) throws IgniteCheckedException { - platformCtx.gateway().onClientReconnected(clusterRestarted); + // Save the flag value for callback of reconnectFut. + this.clusterRestarted = clusterRestarted; return null; } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs index 35bbca5bade71..91e4c061a518e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs @@ -17,8 +17,12 @@ namespace Apache.Ignite.Core.Tests { + using System; + using System.Threading; using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Configuration; using Apache.Ignite.Core.Common; + using Apache.Ignite.Core.Lifecycle; using Apache.Ignite.Core.Tests.Process; using NUnit.Framework; @@ -28,11 +32,71 @@ namespace Apache.Ignite.Core.Tests [Category(TestUtils.CategoryIntensive)] public class ReconnectTest { + /** */ + private const string CacheName = "cache"; + /// - /// Tests the disconnected exception. + /// Tests the cluster restart scenario, where client is alive, but all servers restart. /// [Test] - public void TestDisconnectedException() + public void TestClusterRestart() + { + var serverCfg = new IgniteConfiguration(TestUtils.GetTestConfiguration()) + { + CacheConfiguration = new[] {new CacheConfiguration(CacheName)} + }; + + var clientCfg = new IgniteConfiguration(TestUtils.GetTestConfiguration()) + { + GridName = "client", + ClientMode = true + }; + + var server = Ignition.Start(serverCfg); + var client = Ignition.Start(clientCfg); + + ClientReconnectEventArgs eventArgs = null; + + client.ClientReconnected += (sender, args) => { eventArgs = args; }; + + var cache = client.GetCache(CacheName); + + cache[1] = 1; + + Ignition.Stop(server.Name, true); + + var cacheEx = Assert.Throws(() => cache.Get(1)); + var ex = cacheEx.InnerException as ClientDisconnectedException; + + Assert.IsNotNull(ex); + + // Start the server and wait for reconnect. + Ignition.Start(serverCfg); + Assert.IsTrue(ex.ClientReconnectTask.Result); + + // Check the event args. + Thread.Sleep(1); // Wait for event handler + + Assert.IsNotNull(eventArgs); + Assert.IsTrue(eventArgs.HasClusterRestarted); + + // Refresh the cache instance and check that it works. + var cache1 = client.GetCache(CacheName); + Assert.AreEqual(0, cache1.GetSize()); + + cache1[1] = 2; + Assert.AreEqual(2, cache1[1]); + + // Check that old cache instance does not work. + var cacheEx1 = Assert.Throws(() => cache.Get(1)); + Assert.AreEqual("Cache has been closed or destroyed: " + CacheName, cacheEx1.Message); + } + + /// + /// Tests the failed connection scenario, where servers are alive, but can't be contacted. + /// + [Test] + public void TestFailedConnection() { var cfg = new IgniteConfiguration { @@ -54,7 +118,7 @@ public void TestDisconnectedException() Assert.IsTrue(ignite.GetCluster().ClientReconnectTask.IsCompleted); - var cache = ignite.CreateCache("c"); + var cache = ignite.CreateCache(CacheName); cache[1] = 1; @@ -69,6 +133,8 @@ public void TestDisconnectedException() var inner = (ClientDisconnectedException) ex.InnerException; + Assert.IsNotNull(inner); + var clientReconnectTask = inner.ClientReconnectTask; Assert.AreEqual(ignite.GetCluster().ClientReconnectTask, clientReconnectTask); @@ -78,7 +144,7 @@ public void TestDisconnectedException() // Resume process to reconnect proc.Resume(); - clientReconnectTask.Wait(); + Assert.IsFalse(clientReconnectTask.Result); Assert.AreEqual(1, cache[1]); Assert.AreEqual(1, disconnected); @@ -97,11 +163,12 @@ private static IgniteProcess StartServerProcess(IgniteConfiguration cfg) } /// - /// Fixture tear down. + /// Test tear down. /// - [TestFixtureTearDown] - public void FixtureTearDown() + [TearDown] + public void TearDown() { + Ignition.StopAll(true); IgniteProcess.KillAll(); Ignition.ClientMode = false; } From dc0adf641f0ddac19af84ba0d701b33ee520b067 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 16 Aug 2016 18:11:34 +0300 Subject: [PATCH 301/487] IGNITE-3368 .NET: Improve test coverage This closes #953 # Conflicts: # modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs --- .../Apache.Ignite.Core.Tests.csproj | 10 + .../Binary/BinaryBuilderSelfTest.cs | 24 ++- .../Binary/BinaryReaderWriterTest.cs | 171 ++++++++++++++++ .../Binary/IO/BinaryStreamsTest.cs | 151 ++++++++++++++ .../Cache/Affinity/AffinityKeyTest.cs | 66 ++++++ .../Affinity/AffinityTopologyVersionTest.cs | 59 ++++++ .../Cache/CacheAbstractTest.cs | 20 +- .../Cache/CacheResultTest.cs | 75 +++++++ .../Continuous/ContinuousQueryAbstractTest.cs | 42 ++-- .../Cache/Store/CacheParallelLoadStoreTest.cs | 2 +- .../Cache/Store/CacheStoreAdapterTest.cs | 90 ++++++++ .../Cache/Store/CacheTestParallelLoadStore.cs | 9 + .../Collections/MultiValueDictionaryTest.cs} | 57 +++--- .../Collections/ReadOnlyCollectionTest.cs | 59 ++++++ .../Collections/ReadOnlyDictionaryTest.cs | 70 +++++++ .../Common/IgniteGuidTest.cs | 62 ++++++ .../Compute/AbstractTaskTest.cs | 40 ++-- .../Compute/BinarizableClosureTaskTest.cs | 18 +- .../Compute/BinarizableTaskTest.cs | 18 +- .../Compute/CancellationTest.cs | 10 + .../Compute/ClosureTaskTest.cs | 192 ++++-------------- .../Compute/ComputeApiTest.cs | 53 +++-- .../Compute/FailoverTaskSelfTest.cs | 5 +- .../Compute/IgniteExceptionTaskSelfTest.cs | 43 ++-- .../Compute/ResourceTaskTest.cs | 51 +++++ .../Compute/SerializableClosureTaskTest.cs | 5 + .../Compute/TaskAdapterTest.cs | 5 +- .../Compute/TaskResultTest.cs | 14 +- .../Config/Compute/compute-standalone.xml | 1 + .../Apache.Ignite.Core.Tests/EventsTest.cs | 192 ++++++++++++++---- .../Examples/ExamplesTest.cs | 2 +- .../ExceptionsTest.cs | 46 +++++ .../Apache.Ignite.Core.Tests/TestUtils.cs | 3 + .../Apache.Ignite.Core.csproj | 3 - .../Cache/Affinity/AffinityKey.cs | 11 + .../Cache/CachePartialUpdateException.cs | 6 +- .../Store/CacheParallelLoadStoreAdapter.cs | 7 + .../Impl/Binary/BinaryObject.cs | 2 + .../Impl/Binary/BinaryObjectBuilder.cs | 2 +- .../Impl/Binary/BinaryObjectHeader.cs | 5 +- .../Binary/BinarySurrogateTypeDescriptor.cs | 2 +- .../Impl/Binary/BinaryWriter.cs | 20 -- .../Impl/Binary/Io/BinaryStreamAdapter.cs | 5 + .../Impl/Binary/Io/BinaryStreamBase.cs | 11 +- .../Impl/Binary/Io/IBinaryStream.cs | 2 +- .../Impl/Binary/JavaTypes.cs | 11 - .../Cache/Event/JavaCacheEntryEventFilter.cs | 2 + .../Impl/Collections/MultiValueDictionary.cs | 26 --- .../Impl/Collections/ReadOnlyDictionary.cs | 2 +- .../Impl/Common/ResizeableArray.cs | 64 ------ .../Impl/Events/EventTypeConverter.cs | 3 + .../Apache.Ignite.Core/Impl/Events/Events.cs | 3 + .../Apache.Ignite.Core/Impl/ExceptionUtils.cs | 1 + .../Apache.Ignite.Core/Impl/Handle/Handle.cs | 12 +- .../Impl/Handle/HandleRegistry.cs | 20 +- .../Apache.Ignite.Core/Impl/Handle/IHandle.cs | 5 - .../Apache.Ignite.Core/Impl/IgniteProxy.cs | 1 + .../Apache.Ignite.Core/Impl/IgniteUtils.cs | 22 +- .../Impl/InteropExceptionHolder.cs | 88 -------- .../Impl/Memory/PlatformMemoryUtils.cs | 30 --- .../Impl/Memory/PlatformRawMemory.cs | 4 + .../Apache.Ignite.Core/Impl/PlatformTarget.cs | 8 - 62 files changed, 1390 insertions(+), 653 deletions(-) create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryReaderWriterTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/IO/BinaryStreamsTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityKeyTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityTopologyVersionTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheResultTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreAdapterTest.cs rename modules/platforms/dotnet/{Apache.Ignite.Core/Impl/Binary/BinaryObjectHandle.cs => Apache.Ignite.Core.Tests/Collections/MultiValueDictionaryTest.cs} (50%) create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyCollectionTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyDictionaryTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Common/IgniteGuidTest.cs delete mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ResizeableArray.cs delete mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Impl/InteropExceptionHolder.cs diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj index 11a8a15a09918..34d93d7f37481 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj @@ -62,6 +62,16 @@ + + + + + + + + + + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs index 80788a63b02f4..c28025591de5e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs @@ -24,6 +24,7 @@ namespace Apache.Ignite.Core.Tests.Binary using System.Collections; using System.Collections.Generic; using System.Linq; + using System.Text.RegularExpressions; using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Impl; using Apache.Ignite.Core.Impl.Binary; @@ -589,6 +590,23 @@ public void TestHashCodeChange() Assert.AreEqual(100, binObj.GetHashCode()); } + /// + /// Tests equality and formatting members. + /// + [Test] + public void TestEquality() + { + var bin = _grid.GetBinary(); + + var obj1 = bin.GetBuilder("myType").SetStringField("str", "foo").SetIntField("int", 1).Build(); + var obj2 = bin.GetBuilder("myType").SetStringField("str", "foo").SetIntField("int", 1).Build(); + + Assert.AreEqual(obj1, obj2); + Assert.AreEqual(obj1.GetHashCode(), obj2.GetHashCode()); + + Assert.IsTrue(Regex.IsMatch(obj1.ToString(), @"myType \[idHash=[0-9]+, str=foo, int=1\]")); + } + /// /// Test primitive fields setting. /// @@ -1617,9 +1635,13 @@ public void TestBuildEnum() foreach (var binEnum in binEnums) { Assert.IsTrue(binEnum.GetBinaryType().IsEnum); + Assert.AreEqual(val, binEnum.EnumValue); - Assert.AreEqual((TestEnumRegistered)val, binEnum.Deserialize()); + + Assert.AreEqual((TestEnumRegistered) val, binEnum.Deserialize()); } + + Assert.AreEqual(binEnums[0], binEnums[1]); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryReaderWriterTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryReaderWriterTest.cs new file mode 100644 index 0000000000000..e4cff1bc0bfe5 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryReaderWriterTest.cs @@ -0,0 +1,171 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Tests.Binary +{ + using System; + using Apache.Ignite.Core.Binary; + using Apache.Ignite.Core.Impl.Binary; + using NUnit.Framework; + + /// + /// Tests the and classes. + /// + public class BinaryReaderWriterTest + { + /// + /// Tests all read/write methods. + /// + [Test] + public void TestWriteRead() + { + var marsh = new Marshaller(new BinaryConfiguration(typeof(ReadWriteAll))); + + marsh.Unmarshal(marsh.Marshal(new ReadWriteAll())); + } + + private class ReadWriteAll : IBinarizable + { + private static readonly DateTime Date = DateTime.UtcNow; + + private static readonly Guid Guid = Guid.NewGuid(); + + public void WriteBinary(IBinaryWriter writer) + { + writer.WriteByte("Byte", 1); + writer.WriteByteArray("ByteArray", new byte[] {1}); + writer.WriteChar("Char", '1'); + writer.WriteCharArray("CharArray", new[] {'1'}); + writer.WriteShort("Short", 1); + writer.WriteShortArray("ShortArray", new short[] {1}); + writer.WriteInt("Int", 1); + writer.WriteIntArray("IntArray", new[] {1}); + writer.WriteLong("Long", 1); + writer.WriteLongArray("LongArray", new long[] {1}); + writer.WriteBoolean("Boolean", true); + writer.WriteBooleanArray("BooleanArray", new[] {true}); + writer.WriteFloat("Float", 1); + writer.WriteFloatArray("FloatArray", new float[] {1}); + writer.WriteDouble("Double", 1); + writer.WriteDoubleArray("DoubleArray", new double[] {1}); + writer.WriteDecimal("Decimal", 1); + writer.WriteDecimalArray("DecimalArray", new decimal?[] {1}); + writer.WriteTimestamp("Timestamp", Date); + writer.WriteTimestampArray("TimestampArray", new DateTime?[] {Date}); + writer.WriteString("String", "1"); + writer.WriteStringArray("StringArray", new[] {"1"}); + writer.WriteGuid("Guid", Guid); + writer.WriteGuidArray("GuidArray", new Guid?[] {Guid}); + writer.WriteEnum("Enum", MyEnum.Bar); + writer.WriteEnumArray("EnumArray", new[] {MyEnum.Bar}); + + var raw = writer.GetRawWriter(); + + raw.WriteByte(1); + raw.WriteByteArray(new byte[] {1}); + raw.WriteChar('1'); + raw.WriteCharArray(new[] {'1'}); + raw.WriteShort(1); + raw.WriteShortArray(new short[] {1}); + raw.WriteInt(1); + raw.WriteIntArray(new[] {1}); + raw.WriteLong(1); + raw.WriteLongArray(new long[] {1}); + raw.WriteBoolean(true); + raw.WriteBooleanArray(new[] {true}); + raw.WriteFloat(1); + raw.WriteFloatArray(new float[] {1}); + raw.WriteDouble(1); + raw.WriteDoubleArray(new double[] {1}); + raw.WriteDecimal(1); + raw.WriteDecimalArray(new decimal?[] {1}); + raw.WriteTimestamp(Date); + raw.WriteTimestampArray(new DateTime?[] {Date}); + raw.WriteString("1"); + raw.WriteStringArray(new[] {"1"}); + raw.WriteGuid(Guid); + raw.WriteGuidArray(new Guid?[] {Guid}); + raw.WriteEnum(MyEnum.Bar); + raw.WriteEnumArray(new[] {MyEnum.Bar}); + } + + public void ReadBinary(IBinaryReader reader) + { + Assert.AreEqual(1, reader.ReadByte("Byte")); + Assert.AreEqual(new byte[] {1}, reader.ReadByteArray("ByteArray")); + Assert.AreEqual('1', reader.ReadChar("Char")); + Assert.AreEqual(new[] {'1'}, reader.ReadCharArray("CharArray")); + Assert.AreEqual(1, reader.ReadShort("Short")); + Assert.AreEqual(new short[] {1}, reader.ReadShortArray("ShortArray")); + Assert.AreEqual(1, reader.ReadInt("Int")); + Assert.AreEqual(new[] {1}, reader.ReadIntArray("IntArray")); + Assert.AreEqual(1, reader.ReadLong("Long")); + Assert.AreEqual(new long[] {1}, reader.ReadLongArray("LongArray")); + Assert.AreEqual(true, reader.ReadBoolean("Boolean")); + Assert.AreEqual(new[] {true}, reader.ReadBooleanArray("BooleanArray")); + Assert.AreEqual(1, reader.ReadFloat("Float")); + Assert.AreEqual(new float[] {1}, reader.ReadFloatArray("FloatArray")); + Assert.AreEqual(1, reader.ReadDouble("Double")); + Assert.AreEqual(new double[] {1}, reader.ReadDoubleArray("DoubleArray")); + Assert.AreEqual(1, reader.ReadDecimal("Decimal")); + Assert.AreEqual(new decimal?[] {1}, reader.ReadDecimalArray("DecimalArray")); + Assert.AreEqual(Date, reader.ReadTimestamp("Timestamp")); + Assert.AreEqual(new DateTime?[] {Date}, reader.ReadTimestampArray("TimestampArray")); + Assert.AreEqual("1", reader.ReadString("String")); + Assert.AreEqual(new[] {"1"}, reader.ReadStringArray("StringArray")); + Assert.AreEqual(Guid, reader.ReadGuid("Guid")); + Assert.AreEqual(new Guid?[] {Guid}, reader.ReadGuidArray("GuidArray")); + Assert.AreEqual(MyEnum.Bar, reader.ReadEnum("Enum")); + Assert.AreEqual(new[] {MyEnum.Bar}, reader.ReadEnumArray("EnumArray")); + + var raw = reader.GetRawReader(); + + Assert.AreEqual(1, raw.ReadByte()); + Assert.AreEqual(new byte[] { 1 }, raw.ReadByteArray()); + Assert.AreEqual('1', raw.ReadChar()); + Assert.AreEqual(new[] { '1' }, raw.ReadCharArray()); + Assert.AreEqual(1, raw.ReadShort()); + Assert.AreEqual(new short[] { 1 }, raw.ReadShortArray()); + Assert.AreEqual(1, raw.ReadInt()); + Assert.AreEqual(new[] { 1 }, raw.ReadIntArray()); + Assert.AreEqual(1, raw.ReadLong()); + Assert.AreEqual(new long[] { 1 }, raw.ReadLongArray()); + Assert.AreEqual(true, raw.ReadBoolean()); + Assert.AreEqual(new[] { true }, raw.ReadBooleanArray()); + Assert.AreEqual(1, raw.ReadFloat()); + Assert.AreEqual(new float[] { 1 }, raw.ReadFloatArray()); + Assert.AreEqual(1, raw.ReadDouble()); + Assert.AreEqual(new double[] { 1 }, raw.ReadDoubleArray()); + Assert.AreEqual(1, raw.ReadDecimal()); + Assert.AreEqual(new decimal?[] { 1 }, raw.ReadDecimalArray()); + Assert.AreEqual(Date, raw.ReadTimestamp()); + Assert.AreEqual(new DateTime?[] { Date }, raw.ReadTimestampArray()); + Assert.AreEqual("1", raw.ReadString()); + Assert.AreEqual(new[] { "1" }, raw.ReadStringArray()); + Assert.AreEqual(Guid, raw.ReadGuid()); + Assert.AreEqual(new Guid?[] { Guid }, raw.ReadGuidArray()); + Assert.AreEqual(MyEnum.Bar, raw.ReadEnum()); + Assert.AreEqual(new[] { MyEnum.Bar }, raw.ReadEnumArray()); + } + } + + private enum MyEnum + { + Bar + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/IO/BinaryStreamsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/IO/BinaryStreamsTest.cs new file mode 100644 index 0000000000000..ad5358db40597 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/IO/BinaryStreamsTest.cs @@ -0,0 +1,151 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Tests.Binary.IO +{ + using System; + using System.IO; + using System.Text; + using Apache.Ignite.Core.Impl.Binary.IO; + using Apache.Ignite.Core.Impl.Memory; + using NUnit.Framework; + + /// + /// Tests binary streams. + /// + public class BinaryStreamsTest + { + /// + /// Tests the platform memory stream. + /// + [Test] + public void TestPlatformMemoryStream() + { + var stream = new PlatformMemoryStream(GetMemory()); + TestStream(stream, false, () => stream.SynchronizeOutput()); + } + + /// + /// Tests the platform big endian memory stream. + /// + [Test] + public void TestPlatformBigEndianMemoryStream() + { + var stream = new PlatformBigEndianMemoryStream(GetMemory()); + TestStream(stream, false, () => stream.SynchronizeOutput()); + } + + /// + /// Tests the binary heap stream. + /// + [Test] + public void TestBinaryHeapStream() + { + TestStream(new BinaryHeapStream(1), true, () => { }); + } + + /// + /// Gets the memory. + /// + private static PlatformMemory GetMemory() + { + return new PlatformMemoryPool().Allocate(10); + } + + /// + /// Tests the stream. + /// + private static unsafe void TestStream(IBinaryStream stream, bool sameArr, Action flush) + { + Action seek = () => Assert.AreEqual(0, stream.Seek(0, SeekOrigin.Begin)); + + Action, object> check = (write, read, expectedResult) => + { + seek(); + write(); + flush(); + seek(); + Assert.AreEqual(expectedResult, read()); + }; + + // Arrays. + Assert.AreEqual(sameArr, stream.IsSameArray(stream.GetArray())); + Assert.IsFalse(stream.IsSameArray(new byte[1])); + Assert.IsFalse(stream.IsSameArray(stream.GetArrayCopy())); + + // byte* + byte* bytes = stackalloc byte[10]; + *bytes = 1; + *(bytes + 1) = 2; + + stream.Write(bytes, 2); + Assert.AreEqual(2, stream.Position); + flush(); + + seek(); + Assert.AreEqual(sameArr ? 256 : 2, stream.Remaining); + byte* bytes2 = stackalloc byte[2]; + stream.Read(bytes2, 2); + Assert.AreEqual(1, *bytes2); + Assert.AreEqual(2, *(bytes2 + 1)); + + // char* + seek(); + char* chars = stackalloc char[10]; + *chars = 'a'; + *(chars + 1) = 'b'; + + Assert.AreEqual(2, stream.WriteString(chars, 2, 2, Encoding.ASCII)); + flush(); + + seek(); + stream.Read(bytes2, 2); + Assert.AreEqual('a', *bytes2); + Assert.AreEqual('b', *(bytes2 + 1)); + + // Others. + check(() => stream.Write(new byte[] {3, 4, 5}, 1, 2), () => stream.ReadByteArray(2), new byte[] {4, 5}); + + check(() => stream.WriteBool(true), () => stream.ReadBool(), true); + check(() => stream.WriteBoolArray(new[] {true, false}), () => stream.ReadBoolArray(2), + new[] {true, false}); + + check(() => stream.WriteByte(4), () => stream.ReadByte(), 4); + check(() => stream.WriteByteArray(new byte[] {4, 5, 6}), () => stream.ReadByteArray(3), + new byte[] {4, 5, 6}); + + check(() => stream.WriteChar('x'), () => stream.ReadChar(), 'x'); + check(() => stream.WriteCharArray(new[] {'a', 'b'}), () => stream.ReadCharArray(2), new[] {'a', 'b'}); + + check(() => stream.WriteDouble(4), () => stream.ReadDouble(), 4d); + check(() => stream.WriteDoubleArray(new[] {4d}), () => stream.ReadDoubleArray(1), new[] {4d}); + + check(() => stream.WriteFloat(4), () => stream.ReadFloat(), 4f); + check(() => stream.WriteFloatArray(new[] {4f}), () => stream.ReadFloatArray(1), new[] {4f}); + + check(() => stream.WriteInt(4), () => stream.ReadInt(), 4); + check(() => stream.WriteInt(0, 4), () => stream.ReadInt(), 4); + check(() => stream.WriteIntArray(new[] {4}), () => stream.ReadIntArray(1), new[] {4}); + + check(() => stream.WriteLong(4), () => stream.ReadLong(), 4L); + check(() => stream.WriteLongArray(new[] {4L}), () => stream.ReadLongArray(1), new[] {4L}); + + check(() => stream.WriteShort(4), () => stream.ReadShort(), (short)4); + check(() => stream.WriteShortArray(new short[] {4}), () => stream.ReadShortArray(1), new short[] {4}); + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityKeyTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityKeyTest.cs new file mode 100644 index 0000000000000..cf86273e39657 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityKeyTest.cs @@ -0,0 +1,66 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Tests.Cache.Affinity +{ + using Apache.Ignite.Core.Cache.Affinity; + using NUnit.Framework; + + /// + /// Tests for + /// + public class AffinityKeyTest + { + /// + /// Tests the equality. + /// + [Test] + public void TestEquality() + { + // Default. + var key = new AffinityKey(); + + Assert.IsNull(key.Key); + Assert.IsNull(key.Affinity); + Assert.AreEqual(0, key.GetHashCode()); + Assert.AreEqual(new AffinityKey(), key); + + // Ctor 1. + const string myKey = "myKey"; + key = new AffinityKey(myKey); + + Assert.AreEqual(myKey, key.Key); + Assert.AreEqual(myKey, key.Affinity); + Assert.AreNotEqual(0, key.GetHashCode()); + + // Ctor 2. + var ver1 = new AffinityKey(long.MaxValue, int.MaxValue); + var ver2 = new AffinityKey(long.MaxValue, int.MaxValue); + + Assert.AreEqual(ver1, ver2); + Assert.IsTrue(ver1 == ver2); + Assert.IsFalse(ver1 != ver2); + + Assert.AreNotEqual(key, ver1); + Assert.IsTrue(key != ver1); + Assert.IsFalse(key == ver1); + + // ToString. + Assert.AreEqual("AffinityKey [Key=1, Affinity=2]", new AffinityKey(1, 2).ToString()); + } + } +} \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityTopologyVersionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityTopologyVersionTest.cs new file mode 100644 index 0000000000000..de25ea4639598 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityTopologyVersionTest.cs @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +namespace Apache.Ignite.Core.Tests.Cache.Affinity +{ + using Apache.Ignite.Core.Cache.Affinity; + using NUnit.Framework; + + /// + /// Tests for + /// + public class AffinityTopologyVersionTest + { + /// + /// Tests the equality. + /// + [Test] + public void TestEquality() + { + // Default. + var ver = new AffinityTopologyVersion(); + + Assert.AreEqual(0, ver.Version); + Assert.AreEqual(0, ver.MinorVersion); + Assert.AreEqual(0, ver.GetHashCode()); + Assert.AreEqual(new AffinityTopologyVersion(), ver); + + // Custom. + var ver1 = new AffinityTopologyVersion(long.MaxValue, int.MaxValue); + var ver2 = new AffinityTopologyVersion(long.MaxValue, int.MaxValue); + + Assert.AreEqual(ver1, ver2); + Assert.IsTrue(ver1 == ver2); + Assert.IsFalse(ver1 != ver2); + + Assert.AreNotEqual(ver, ver1); + Assert.IsTrue(ver != ver1); + Assert.IsFalse(ver == ver1); + + // ToString. + Assert.AreEqual("AffinityTopologyVersion [Version=1, MinorVersion=2]", + new AffinityTopologyVersion(1, 2).ToString()); + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs index 8878d2e7f0d07..9fd1f1de5282b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs @@ -29,7 +29,6 @@ namespace Apache.Ignite.Core.Tests.Cache using Apache.Ignite.Core.Cache.Expiry; using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Common; - using Apache.Ignite.Core.Impl; using Apache.Ignite.Core.Impl.Cache; using Apache.Ignite.Core.Tests.Query; using Apache.Ignite.Core.Transactions; @@ -991,10 +990,8 @@ public void TestWithExpiryPolicyZeroNegative() key1 = PrimaryKeyForCache(Cache(1)); } - var cache = cache0.WithExpiryPolicy(new ExpiryPolicy(null, null, null)); - // Test zero expiration. - cache = cache0.WithExpiryPolicy(new ExpiryPolicy(TimeSpan.Zero, TimeSpan.Zero, TimeSpan.Zero)); + var cache = cache0.WithExpiryPolicy(new ExpiryPolicy(TimeSpan.Zero, TimeSpan.Zero, TimeSpan.Zero)); cache.Put(key0, key0); cache.Put(key1, key1); @@ -1930,7 +1927,7 @@ public void TestLockSimple() }; checkLock(cache.Lock(key)); - checkLock(cache.LockAll(new[] { key, 1, 2, 3 })); + checkLock(cache.LockAll(new[] {key, 1, 2, 3})); } [Test] @@ -2038,7 +2035,7 @@ private static void CheckLock(ICache cache, int key, Func } /// - /// ENsure taht lock cannot be obtained by other threads. + /// Ensure that lock cannot be obtained by other threads. /// /// Get lock function. /// Shared lock. @@ -2419,10 +2416,15 @@ public void TestTxStateAndExceptions() return; var tx = Transactions.TxStart(); - + Assert.AreEqual(TransactionState.Active, tx.State); + Assert.AreEqual(Thread.CurrentThread.ManagedThreadId, tx.ThreadId); - tx.Rollback(); + tx.AddMeta("myMeta", 42); + Assert.AreEqual(42, tx.Meta("myMeta")); + Assert.AreEqual(42, tx.RemoveMeta("myMeta")); + + tx.RollbackAsync().Wait(); Assert.AreEqual(TransactionState.RolledBack, tx.State); @@ -3346,7 +3348,7 @@ protected virtual bool TxEnabled() return true; } - protected virtual bool LockingEnabled() + protected bool LockingEnabled() { return TxEnabled(); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheResultTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheResultTest.cs new file mode 100644 index 0000000000000..6be26bf1c0448 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheResultTest.cs @@ -0,0 +1,75 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Tests.Cache +{ + using System.Collections.Generic; + using Apache.Ignite.Core.Cache; + using NUnit.Framework; + + /// + /// tests. + /// + public class CacheResultTest + { + /// + /// Tests equality members. + /// + [Test] + public void TestEquality() + { + var entry1 = new CacheResult(2); + var entry2 = new CacheResult(2); + var entry3 = new CacheResult(3); + + Assert.AreEqual(entry1, entry2); + Assert.AreNotEqual(entry1, entry3); + + Assert.IsTrue(entry1 == entry2); + Assert.IsFalse(entry1 != entry2); + + Assert.IsTrue(entry1 != entry3); + Assert.IsFalse(entry1 == entry3); + + var boxedEntry1 = (object) entry1; + var boxedEntry2 = (object) entry2; + var boxedEntry3 = (object) entry3; + + Assert.IsFalse(ReferenceEquals(boxedEntry1, boxedEntry2)); + + Assert.AreEqual(boxedEntry1, boxedEntry2); + Assert.AreNotEqual(boxedEntry1, boxedEntry3); + } + + /// + /// Tests with hash data structures. + /// + [Test] + public void TestHashCode() + { + var entry1 = new CacheResult(2); + var entry2 = new CacheResult(2); + var entry3 = new CacheResult(3); + + var set = new HashSet {entry1}; + + Assert.IsTrue(set.Contains(entry1)); + Assert.IsTrue(set.Contains(entry2)); + Assert.IsFalse(set.Contains(entry3)); + } + } +} \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAbstractTest.cs index 4b285f9a4c70c..270c3fc55add1 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAbstractTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAbstractTest.cs @@ -181,11 +181,11 @@ public void TestMultipleClose() { // Put from local node. cache1.GetAndPut(key1, Entry(key1)); - CheckCallbackSingle(key1, null, Entry(key1)); + CheckCallbackSingle(key1, null, Entry(key1), CacheEntryEventType.Created); // Put from remote node. cache2.GetAndPut(key2, Entry(key2)); - CheckCallbackSingle(key2, null, Entry(key2)); + CheckCallbackSingle(key2, null, Entry(key2), CacheEntryEventType.Created); } qryHnd.Dispose(); @@ -217,13 +217,13 @@ protected void CheckCallback(bool loc) { // Put from local node. cache1.GetAndPut(key1, Entry(key1)); - CheckCallbackSingle(key1, null, Entry(key1)); + CheckCallbackSingle(key1, null, Entry(key1), CacheEntryEventType.Created); cache1.GetAndPut(key1, Entry(key1 + 1)); - CheckCallbackSingle(key1, Entry(key1), Entry(key1 + 1)); + CheckCallbackSingle(key1, Entry(key1), Entry(key1 + 1), CacheEntryEventType.Updated); cache1.Remove(key1); - CheckCallbackSingle(key1, Entry(key1 + 1), null); + CheckCallbackSingle(key1, Entry(key1 + 1), null, CacheEntryEventType.Removed); // Put from remote node. cache2.GetAndPut(key2, Entry(key2)); @@ -231,21 +231,21 @@ protected void CheckCallback(bool loc) if (loc) CheckNoCallback(100); else - CheckCallbackSingle(key2, null, Entry(key2)); + CheckCallbackSingle(key2, null, Entry(key2), CacheEntryEventType.Created); cache1.GetAndPut(key2, Entry(key2 + 1)); if (loc) CheckNoCallback(100); else - CheckCallbackSingle(key2, Entry(key2), Entry(key2 + 1)); + CheckCallbackSingle(key2, Entry(key2), Entry(key2 + 1), CacheEntryEventType.Updated); cache1.Remove(key2); if (loc) CheckNoCallback(100); else - CheckCallbackSingle(key2, Entry(key2 + 1), null); + CheckCallbackSingle(key2, Entry(key2 + 1), null, CacheEntryEventType.Removed); } cache1.Put(key1, Entry(key1)); @@ -310,7 +310,7 @@ protected void CheckFilter(bool binarizable, bool loc) int key1 = PrimaryKey(cache1); cache1.GetAndPut(key1, Entry(key1)); CheckFilterSingle(key1, null, Entry(key1)); - CheckCallbackSingle(key1, null, Entry(key1)); + CheckCallbackSingle(key1, null, Entry(key1), CacheEntryEventType.Created); // Put from remote node. int key2 = PrimaryKey(cache2); @@ -324,7 +324,7 @@ protected void CheckFilter(bool binarizable, bool loc) else { CheckFilterSingle(key2, null, Entry(key2)); - CheckCallbackSingle(key2, null, Entry(key2)); + CheckCallbackSingle(key2, null, Entry(key2), CacheEntryEventType.Created); } AbstractFilter.res = false; @@ -755,12 +755,12 @@ public void TestTimeout() { // Put from local node. cache1.GetAndPut(key1, Entry(key1)); - CheckCallbackSingle(key1, null, Entry(key1)); + CheckCallbackSingle(key1, null, Entry(key1), CacheEntryEventType.Created); // Put from remote node. cache1.GetAndPut(key2, Entry(key2)); CheckNoCallback(100); - CheckCallbackSingle(key2, null, Entry(key2), 1000); + CheckCallbackSingle(key2, null, Entry(key2), CacheEntryEventType.Created); } } @@ -851,7 +851,7 @@ public void TestInitialQuery() // Check continuous query cache1.Put(44, Entry(44)); - CheckCallbackSingle(44, null, Entry(44)); + CheckCallbackSingle(44, null, Entry(44), CacheEntryEventType.Created); } Assert.Throws(() => contQry.GetInitialQueryCursor()); @@ -922,19 +922,10 @@ private static void CheckNoFilter(int timeout) /// Expected key. /// Expected old value. /// Expected new value. - private static void CheckCallbackSingle(int expKey, BinarizableEntry expOldVal, BinarizableEntry expVal) - { - CheckCallbackSingle(expKey, expOldVal, expVal, 1000); - } - - /// - /// Check single callback event. - /// - /// Expected key. - /// Expected old value. - /// Expected new value. + /// Expected type. /// Timeout. - private static void CheckCallbackSingle(int expKey, BinarizableEntry expOldVal, BinarizableEntry expVal, int timeout) + private static void CheckCallbackSingle(int expKey, BinarizableEntry expOldVal, BinarizableEntry expVal, + CacheEntryEventType expType, int timeout = 1000) { CallbackEvent evt; @@ -946,6 +937,7 @@ private static void CheckCallbackSingle(int expKey, BinarizableEntry expOldVal, Assert.AreEqual(expKey, e.Key); Assert.AreEqual(expOldVal, e.OldValue); Assert.AreEqual(expVal, e.Value); + Assert.AreEqual(expType, e.EventType); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheParallelLoadStoreTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheParallelLoadStoreTest.cs index 4aa910ca7e366..105dea23f11cb 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheParallelLoadStoreTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheParallelLoadStoreTest.cs @@ -96,7 +96,7 @@ public void TestLoadCache() } // check that items were processed in parallel - Assert.GreaterOrEqual(CacheTestParallelLoadStore.UniqueThreadCount, Environment.ProcessorCount); + Assert.GreaterOrEqual(CacheTestParallelLoadStore.UniqueThreadCount, Environment.ProcessorCount - 1); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreAdapterTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreAdapterTest.cs new file mode 100644 index 0000000000000..6690584ed6760 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreAdapterTest.cs @@ -0,0 +1,90 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Tests.Cache.Store +{ + using System.Collections.Generic; + using System.Linq; + using Apache.Ignite.Core.Cache.Store; + using NUnit.Framework; + + /// + /// Tests for . + /// + public class CacheStoreAdapterTest + { + /// + /// Tests the load write delete. + /// + [Test] + public void TestLoadWriteDelete() + { + var store = new Store(); + + store.LoadCache(null); + Assert.IsEmpty(store.Map); + + var data = Enumerable.Range(1, 5).ToDictionary(x => x, x => x.ToString()); + + // Write. + store.WriteAll(data); + Assert.AreEqual(data, store.Map); + + // Load. + CollectionAssert.AreEqual(data, store.LoadAll(data.Keys)); + CollectionAssert.AreEqual(data.Where(x => x.Key < 3).ToDictionary(x => x.Key, x => x.Value), + store.LoadAll(data.Keys.Where(x => x < 3).ToList())); + + // Delete. + var removed = new[] {3, 5}; + + foreach (var key in removed) + data.Remove(key); + + store.DeleteAll(removed); + CollectionAssert.AreEqual(data, store.LoadAll(data.Keys)); + } + + /// + /// Test store. + /// + private class Store : CacheStoreAdapter + { + /** */ + public readonly Dictionary Map = new Dictionary(); + + /** */ + public override object Load(object key) + { + object res; + return Map.TryGetValue(key, out res) ? res : null; + } + + /** */ + public override void Write(object key, object val) + { + Map[key] = val; + } + + /** */ + public override void Delete(object key) + { + Map.Remove(key); + } + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestParallelLoadStore.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestParallelLoadStore.cs index 770ca838ea1f3..81b4697a0a82a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestParallelLoadStore.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestParallelLoadStore.cs @@ -17,6 +17,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Store { + using System; using System.Collections; using System.Collections.Concurrent; using System.Collections.Generic; @@ -35,6 +36,14 @@ public class CacheTestParallelLoadStore : CacheParallelLoadStoreAdapter /** list of thread ids where Parse has been executed */ private static readonly ConcurrentDictionary ThreadIds = new ConcurrentDictionary(); + /// + /// Initializes a new instance of the class. + /// + public CacheTestParallelLoadStore() + { + MaxDegreeOfParallelism -= 1; + } + /// /// Gets the count of unique threads that entered Parse method. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHandle.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/MultiValueDictionaryTest.cs similarity index 50% rename from modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHandle.cs rename to modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/MultiValueDictionaryTest.cs index 35735feac33ff..aa3e2aadc189e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHandle.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/MultiValueDictionaryTest.cs @@ -1,4 +1,4 @@ -/* +/* * 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. @@ -15,45 +15,44 @@ * limitations under the License. */ -namespace Apache.Ignite.Core.Impl.Binary +namespace Apache.Ignite.Core.Tests.Collections { + using Apache.Ignite.Core.Impl.Collections; + using NUnit.Framework; + /// - /// Object handle. Wraps a single value. + /// Tests the . /// - internal class BinaryObjectHandle + public class MultiValueDictionaryTest { - /** Value. */ - private readonly object _val; - /// - /// Initializes a new instance of the class. + /// Tests the dictionary. /// - /// The value. - public BinaryObjectHandle(object val) + [Test] + public void TestMultiValueDictionary() { - _val = val; - } + var dict = new MultiValueDictionary(); - /// - /// Gets the value. - /// - public object Value - { - get { return _val; } - } + dict.Add(1, 1); + dict.Add(1, 2); - /** */ - public override bool Equals(object obj) - { - var that = obj as BinaryObjectHandle; + int val; - return that != null && _val == that._val; - } + Assert.IsTrue(dict.TryRemove(1, out val)); + Assert.AreEqual(2, val); - /** */ - public override int GetHashCode() - { - return _val != null ? _val.GetHashCode() : 0; + Assert.IsTrue(dict.TryRemove(1, out val)); + Assert.AreEqual(1, val); + + Assert.IsFalse(dict.TryRemove(1, out val)); + + dict.Add(2, 1); + dict.Add(2, 2); + dict.Remove(2, 3); + dict.Remove(2, 2); + + Assert.IsTrue(dict.TryRemove(2, out val)); + Assert.AreEqual(1, val); } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyCollectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyCollectionTest.cs new file mode 100644 index 0000000000000..27991a352fddc --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyCollectionTest.cs @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +namespace Apache.Ignite.Core.Tests.Collections +{ + using System; + using System.Collections; + using System.Linq; + using Apache.Ignite.Core.Impl.Collections; + using NUnit.Framework; + + /// + /// Tests for + /// + public class ReadOnlyCollectionTest + { + /// + /// Tests the disctionary. + /// + [Test] + public void TestCollection() + { + // Default ctor. + var data = Enumerable.Range(1, 5).ToArray(); + var col = new ReadOnlyCollection(data); + + Assert.AreEqual(5, col.Count); + Assert.IsTrue(col.IsReadOnly); + CollectionAssert.AreEqual(data, col); + + Assert.IsTrue(col.GetEnumerator().MoveNext()); + Assert.IsTrue(((IEnumerable) col).GetEnumerator().MoveNext()); + + Assert.IsTrue(col.Contains(4)); + + var arr = new int[5]; + col.CopyTo(arr, 0); + CollectionAssert.AreEqual(data, arr); + + Assert.Throws(() => col.Add(1)); + Assert.Throws(() => col.Clear()); + Assert.Throws(() => col.Remove(1)); + } + } +} \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyDictionaryTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyDictionaryTest.cs new file mode 100644 index 0000000000000..294251a66e742 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyDictionaryTest.cs @@ -0,0 +1,70 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Tests.Collections +{ + using System; + using System.Collections; + using System.Collections.Generic; + using System.Linq; + using Apache.Ignite.Core.Impl.Collections; + using NUnit.Framework; + + /// + /// Tests for + /// + public class ReadOnlyDictionaryTest + { + /// + /// Tests the disctionary. + /// + [Test] + public void TestDictionary() + { + // Default ctor. + var data = Enumerable.Range(1, 5).ToDictionary(x => x, x => x.ToString()); + var dict = new ReadOnlyDictionary(data); + + Assert.AreEqual(5, dict.Count); + Assert.IsTrue(dict.IsReadOnly); + CollectionAssert.AreEqual(data, dict); + CollectionAssert.AreEqual(data.Keys, dict.Keys); + CollectionAssert.AreEqual(data.Values, dict.Values); + + Assert.IsTrue(dict.GetEnumerator().MoveNext()); + Assert.IsTrue(((IEnumerable) dict).GetEnumerator().MoveNext()); + + Assert.IsTrue(dict.ContainsKey(1)); + Assert.IsTrue(dict.Contains(new KeyValuePair(4, "4"))); + Assert.AreEqual("3", dict[3]); + + string val; + Assert.IsTrue(dict.TryGetValue(2, out val)); + Assert.AreEqual("2", val); + + var arr = new KeyValuePair[5]; + dict.CopyTo(arr, 0); + CollectionAssert.AreEqual(data, arr); + + Assert.Throws(() => dict.Add(1, "2")); + Assert.Throws(() => dict.Add(new KeyValuePair(1, "2"))); + Assert.Throws(() => dict.Clear()); + Assert.Throws(() => dict.Remove(1)); + Assert.Throws(() => dict.Remove(new KeyValuePair(1, "2"))); + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Common/IgniteGuidTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Common/IgniteGuidTest.cs new file mode 100644 index 0000000000000..705faf46d7de7 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Common/IgniteGuidTest.cs @@ -0,0 +1,62 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Tests.Common +{ + using System; + using Apache.Ignite.Core.Common; + using NUnit.Framework; + + /// + /// Tests the . + /// + public class IgniteGuidTest + { + /// + /// Tests the . + /// + [Test] + public void TestIgniteGuid() + { + var guid = Guid.NewGuid(); + + var id1 = new IgniteGuid(guid, 1); + var id2 = new IgniteGuid(guid, 1); + var id3 = new IgniteGuid(guid, 2); + var id4 = new IgniteGuid(Guid.NewGuid(), 2); + + // Properties. + Assert.AreEqual(guid, id1.GlobalId); + Assert.AreEqual(1, id1.LocalId); + Assert.AreEqual(id1.GetHashCode(), id2.GetHashCode()); + + // Equality. + Assert.AreEqual(id1, id2); + Assert.IsTrue(id1 == id2); + Assert.IsFalse(id1 != id2); + + // Inequality. + Assert.AreNotEqual(id1, id3); + Assert.IsFalse(id1 == id3); + Assert.IsTrue(id1 != id3); + + Assert.AreNotEqual(id4, id3); + Assert.IsFalse(id4 == id3); + Assert.IsTrue(id4 != id3); + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs index d31ad43a209c3..6bcd01042e598 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs @@ -162,30 +162,21 @@ public void StopClient() /// /// Path to Java XML configuration. /// Node configuration. - protected IgniteConfiguration Configuration(string path) + private IgniteConfiguration Configuration(string path) { - IgniteConfiguration cfg = new IgniteConfiguration(); - - if (!_fork) + return new IgniteConfiguration { - BinaryConfiguration portCfg = new BinaryConfiguration(); - - ICollection portTypeCfgs = new List(); - - GetBinaryTypeConfigurations(portTypeCfgs); - - portCfg.TypeConfigurations = portTypeCfgs; - - cfg.BinaryConfiguration = portCfg; - } - - cfg.JvmClasspath = TestUtils.CreateTestClasspath(); - - cfg.JvmOptions = TestUtils.TestJavaOptions(); - - cfg.SpringConfigUrl = path; - - return cfg; + JvmClasspath = TestUtils.CreateTestClasspath(), + JvmOptions = TestUtils.TestJavaOptions(), + SpringConfigUrl = path, + BinaryConfiguration = _fork + ? null + : new BinaryConfiguration + { + TypeConfigurations = + (GetBinaryTypes() ?? new Type[0]).Select(t => new BinaryTypeConfiguration(t)).ToList() + } + }; } /// @@ -209,10 +200,9 @@ private static IgniteProcess Fork(string path) /// /// Define binary types. /// - /// Binary type configurations. - protected virtual void GetBinaryTypeConfigurations(ICollection portTypeCfgs) + protected virtual ICollection GetBinaryTypes() { - // No-op. + return null; } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableClosureTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableClosureTaskTest.cs index b88158231f129..c169f1e0b1583 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableClosureTaskTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableClosureTaskTest.cs @@ -40,12 +40,15 @@ public class BinarizableClosureTaskTest : ClosureTaskTest protected BinarizableClosureTaskTest(bool fork) : base(fork) { } /** */ - protected override void GetBinaryTypeConfigurations(ICollection portTypeCfgs) + protected override ICollection GetBinaryTypes() { - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableOutFunc))); - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableFunc))); - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableResult))); - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableException))); + return new [] + { + typeof(BinarizableOutFunc), + typeof(BinarizableFunc), + typeof(BinarizableResult), + typeof(BinarizableException) + }; } /** */ @@ -76,6 +79,11 @@ protected override void CheckError(Exception err) { Assert.IsTrue(err != null); + var aggregate = err as AggregateException; + + if (aggregate != null) + err = aggregate.InnerException; + BinarizableException err0 = err as BinarizableException; Assert.IsTrue(err0 != null); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableTaskTest.cs index 8aa28defab5fb..6bdfd9c252d85 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableTaskTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableTaskTest.cs @@ -17,6 +17,7 @@ namespace Apache.Ignite.Core.Tests.Compute { + using System; using System.Collections.Generic; using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cluster; @@ -71,14 +72,17 @@ private static IBinaryObject ToBinary(IIgnite grid, object obj) } /** */ - override protected void GetBinaryTypeConfigurations(ICollection portTypeCfgs) + protected override ICollection GetBinaryTypes() { - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableJobArgument))); - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableJobResult))); - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableTaskArgument))); - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableTaskResult))); - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableJob))); - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableWrapper))); + return new[] + { + typeof(BinarizableJobResult), + typeof(BinarizableTaskArgument), + typeof(BinarizableTaskResult), + typeof(BinarizableJobArgument), + typeof(BinarizableJob), + typeof(BinarizableWrapper) + }; } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/CancellationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/CancellationTest.cs index bbd116916647d..19bb40d8016a1 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/CancellationTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/CancellationTest.cs @@ -68,11 +68,21 @@ public void TestJavaTask() public void TestClosures() { TestClosure((c, t) => c.BroadcastAsync(new ComputeAction(), t)); + TestClosure((c, t) => c.BroadcastAsync(new ComputeFunc(), t)); + TestClosure((c, t) => c.BroadcastAsync(new ComputeBiFunc(), 10, t)); + TestClosure((c, t) => c.AffinityRunAsync(null, 0, new ComputeAction(), t)); + TestClosure((c, t) => c.RunAsync(new ComputeAction(), t)); TestClosure((c, t) => c.RunAsync(Enumerable.Range(1, 10).Select(x => new ComputeAction()), t)); + TestClosure((c, t) => c.CallAsync(new ComputeFunc(), t)); + TestClosure((c, t) => c.CallAsync(Enumerable.Range(1, 10).Select(x => new ComputeFunc()), t)); + TestClosure((c, t) => c.CallAsync(Enumerable.Range(1, 10).Select(x => new ComputeFunc()), + new ComputeReducer(), t)); + TestClosure((c, t) => c.AffinityCallAsync(null, 0, new ComputeFunc(), t)); + TestClosure((c, t) => c.ApplyAsync(new ComputeBiFunc(), 10, t)); TestClosure((c, t) => c.ApplyAsync(new ComputeBiFunc(), Enumerable.Range(1, 100), t)); TestClosure((c, t) => c.ApplyAsync(new ComputeBiFunc(), Enumerable.Range(1, 100), new ComputeReducer(), t)); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs index 8664413a7c684..ffb28445762d6 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs @@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Tests.Compute { using System; using System.Collections.Generic; + using System.Linq; using Apache.Ignite.Core.Compute; using NUnit.Framework; @@ -27,7 +28,7 @@ namespace Apache.Ignite.Core.Tests.Compute /// public abstract class ClosureTaskTest : AbstractTaskTest { - /** Amount of multiple clousres. */ + /** Amount of multiple closures. */ private const int MultiCloCnt = 5; /** */ @@ -45,9 +46,8 @@ public abstract class ClosureTaskTest : AbstractTaskTest [Test] public void TestExecuteSingle() { - var res = Grid1.GetCompute().Call(OutFunc(false)); - - CheckResult(res); + CheckResult(Grid1.GetCompute().Call(OutFunc(false))); + CheckResult(Grid1.GetCompute().CallAsync(OutFunc(false)).Result); } /// @@ -56,16 +56,8 @@ public void TestExecuteSingle() [Test] public void TestExecuteSingleException() { - try - { - Grid1.GetCompute().Call(OutFunc(true)); - - Assert.Fail(); - } - catch (Exception e) - { - CheckError(e); - } + CheckError(Assert.Catch(() => Grid1.GetCompute().Call(OutFunc(true)))); + CheckError(Assert.Catch(() => Grid1.GetCompute().CallAsync(OutFunc(true)).Wait())); } /// @@ -74,15 +66,10 @@ public void TestExecuteSingleException() [Test] public void TestExecuteMultiple() { - var clos = new List>(MultiCloCnt); - - for (int i = 0; i < MultiCloCnt; i++) - clos.Add(OutFunc(false)); - - ICollection ress = Grid1.GetCompute().Call(clos); + var clos = Enumerable.Range(0, MultiCloCnt).Select(x => OutFunc(false)).ToArray(); - foreach (object res in ress) - CheckResult(res); + Grid1.GetCompute().Call(clos).ToList().ForEach(CheckResult); + Grid1.GetCompute().CallAsync(clos).Result.ToList().ForEach(CheckResult); } /// @@ -91,15 +78,10 @@ public void TestExecuteMultiple() [Test] public void TestExecuteMultipleReduced() { - var clos = new List>(MultiCloCnt); + var clos = Enumerable.Range(0, MultiCloCnt).Select(x => OutFunc(false)).ToArray(); - for (int i = 0; i < MultiCloCnt; i++) - clos.Add(OutFunc(false)); - - ICollection ress = Grid1.GetCompute().Call(clos, new Reducer(false)); - - foreach (object res in ress) - CheckResult(res); + Grid1.GetCompute().Call(clos, new Reducer(false)).ToList().ForEach(CheckResult); + Grid1.GetCompute().CallAsync(clos, new Reducer(false)).Result.ToList().ForEach(CheckResult); } /// @@ -108,21 +90,11 @@ public void TestExecuteMultipleReduced() [Test] public void TestExecuteMultipleException() { - var clos = new List>(MultiCloCnt); + // Some closures will be faulty. + var clos = Enumerable.Range(0, MultiCloCnt).Select(x => OutFunc(x % 2 == 0)).ToArray(); - for (int i = 0; i < MultiCloCnt; i++) - clos.Add(OutFunc(i % 2 == 0)); // Some closures will be faulty. - - try - { - Grid1.GetCompute().Call(clos); - - Assert.Fail(); - } - catch (Exception e) - { - CheckError(e); - } + CheckError(Assert.Catch(() => Grid1.GetCompute().Call(clos))); + CheckError(Assert.Catch(() => Grid1.GetCompute().CallAsync(clos).Wait())); } /// @@ -131,10 +103,8 @@ public void TestExecuteMultipleException() [Test] public void TestBroadcastOut() { - ICollection ress = Grid1.GetCompute().Broadcast(OutFunc(false)); - - foreach (object res in ress) - CheckResult(res); + Grid1.GetCompute().Broadcast(OutFunc(false)).ToList().ForEach(CheckResult); + Grid1.GetCompute().BroadcastAsync(OutFunc(false)).Result.ToList().ForEach(CheckResult); } /// @@ -143,16 +113,8 @@ public void TestBroadcastOut() [Test] public void TestBroadcastOutException() { - try - { - Grid1.GetCompute().Broadcast(OutFunc(true)); - - Assert.Fail(); - } - catch (Exception e) - { - CheckError(e); - } + CheckError(Assert.Catch(() => Grid1.GetCompute().Broadcast(OutFunc(true)))); + CheckError(Assert.Catch(() => Grid1.GetCompute().BroadcastAsync(OutFunc(true)).Wait())); } /// @@ -161,10 +123,8 @@ public void TestBroadcastOutException() [Test] public void TestBroadcastInOut() { - ICollection ress = Grid1.GetCompute().Broadcast(Func(false), 1); - - foreach (object res in ress) - CheckResult(res); + Grid1.GetCompute().Broadcast(Func(false), 1).ToList().ForEach(CheckResult); + Grid1.GetCompute().BroadcastAsync(Func(false), 1).Result.ToList().ForEach(CheckResult); } /// @@ -173,16 +133,8 @@ public void TestBroadcastInOut() [Test] public void TestBroadcastInOutException() { - try - { - Grid1.GetCompute().Broadcast(Func(true), 1); - - Assert.Fail(); - } - catch (Exception e) - { - CheckError(e); - } + CheckError(Assert.Catch(() => Grid1.GetCompute().Broadcast(Func(true), 1))); + CheckError(Assert.Catch(() => Grid1.GetCompute().BroadcastAsync(Func(true), 1).Wait())); } /// @@ -191,9 +143,8 @@ public void TestBroadcastInOutException() [Test] public void TestApply() { - object res = Grid1.GetCompute().Apply(Func(false), 1); - - CheckResult(res); + CheckResult(Grid1.GetCompute().Apply(Func(false), 1)); + CheckResult(Grid1.GetCompute().ApplyAsync(Func(false), 1).Result); } /// @@ -202,16 +153,8 @@ public void TestApply() [Test] public void TestApplyException() { - try - { - Grid1.GetCompute().Apply(Func(true), 1); - - Assert.Fail(); - } - catch (Exception e) - { - CheckError(e); - } + CheckError(Assert.Catch(() => Grid1.GetCompute().Apply(Func(true), 1))); + CheckError(Assert.Catch(() => Grid1.GetCompute().ApplyAsync(Func(true), 1).Wait())); } /// @@ -220,19 +163,10 @@ public void TestApplyException() [Test] public void TestApplyMultiple() { - var args = new List(MultiCloCnt); - - for (int i = 0; i < MultiCloCnt; i++) - args.Add(1); - - Console.WriteLine("START TASK"); - - var ress = Grid1.GetCompute().Apply(Func(false), args); + var args = Enumerable.Repeat(1, MultiCloCnt).Cast().ToArray(); - Console.WriteLine("END TASK."); - - foreach (object res in ress) - CheckResult(res); + Grid1.GetCompute().Apply(Func(false), args).ToList().ForEach(CheckResult); + Grid1.GetCompute().ApplyAsync(Func(false), args).Result.ToList().ForEach(CheckResult); } /// @@ -241,21 +175,10 @@ public void TestApplyMultiple() [Test] public void TestApplyMultipleException() { - ICollection args = new List(MultiCloCnt); - - for (int i = 0; i < MultiCloCnt; i++) - args.Add(1); - - try - { - Grid1.GetCompute().Apply(Func(true), args); + var args = Enumerable.Repeat(1, MultiCloCnt).Cast().ToArray(); - Assert.Fail(); - } - catch (Exception e) - { - CheckError(e); - } + CheckError(Assert.Catch(() => Grid1.GetCompute().Apply(Func(true), args))); + CheckError(Assert.Catch(() => Grid1.GetCompute().ApplyAsync(Func(true), args).Wait())); } /// @@ -264,16 +187,10 @@ public void TestApplyMultipleException() [Test] public void TestApplyMultipleReducer() { - var args = new List(MultiCloCnt); + var args = Enumerable.Repeat(1, MultiCloCnt).Cast().ToArray(); - for (int i = 0; i < MultiCloCnt; i++) - args.Add(1); - - ICollection ress = - Grid1.GetCompute().Apply(Func(false), args, new Reducer(false)); - - foreach (object res in ress) - CheckResult(res); + Grid1.GetCompute().Apply(Func(false), args, new Reducer(false)).ToList().ForEach(CheckResult); + Grid1.GetCompute().ApplyAsync(Func(false), args, new Reducer(false)).Result.ToList().ForEach(CheckResult); } /// @@ -282,21 +199,10 @@ public void TestApplyMultipleReducer() [Test] public void TestAppylMultipleReducerJobException() { - List args = new List(MultiCloCnt); + var args = Enumerable.Repeat(1, MultiCloCnt).Cast().ToArray(); - for (int i = 0; i < MultiCloCnt; i++) - args.Add(1); - - try - { - Grid1.GetCompute().Apply(Func(true), args, new Reducer(false)); - - Assert.Fail(); - } - catch (Exception e) - { - CheckError(e); - } + CheckError(Assert.Catch(() => Grid1.GetCompute().Apply(Func(true), args, new Reducer(false)))); + CheckError(Assert.Catch(() => Grid1.GetCompute().ApplyAsync(Func(true), args, new Reducer(false)).Wait())); } /// @@ -305,23 +211,11 @@ public void TestAppylMultipleReducerJobException() [Test] public void TestAppylMultipleReducerReduceException() { - var args = new List(MultiCloCnt); - - for (int i = 0; i < MultiCloCnt; i++) - args.Add(1); + var args = Enumerable.Repeat(1, MultiCloCnt).Cast().ToArray(); - try - { - Grid1.GetCompute().Apply(Func(false), args, new Reducer(true)); + var e = Assert.Throws(() => Grid1.GetCompute().Apply(Func(false), args, new Reducer(true))); - Assert.Fail(); - } - catch (Exception e) - { - Assert.AreEqual(typeof(Exception), e.GetType()); - - Assert.AreEqual(ErrMsg, e.Message); - } + Assert.AreEqual(ErrMsg, e.Message); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs index 20290f11139e2..1e0287fe56e8f 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs @@ -187,7 +187,10 @@ public void TestProjection() Assert.NotNull(prj); - Assert.IsTrue(prj == prj.Ignite); + Assert.AreEqual(prj, prj.Ignite); + + // Check that default Compute projection excludes client nodes. + CollectionAssert.AreEquivalent(prj.ForServers().GetNodes(), prj.GetCompute().ClusterGroup.GetNodes()); } /// @@ -991,9 +994,7 @@ public void TestBinarizableArgTask() compute.WithKeepBinary(); - PlatformComputeNetBinarizable arg = new PlatformComputeNetBinarizable(); - - arg.Field = 100; + PlatformComputeNetBinarizable arg = new PlatformComputeNetBinarizable {Field = 100}; int res = compute.ExecuteJavaTask(BinaryArgTask, arg); @@ -1036,9 +1037,11 @@ public void TestBroadcastTask([Values(false, true)] bool isAsync) public void TestBroadcastAction() { var id = Guid.NewGuid(); - _grid1.GetCompute().Broadcast(new ComputeAction(id)); + Assert.AreEqual(2, ComputeAction.InvokeCount(id)); + id = Guid.NewGuid(); + _grid1.GetCompute().BroadcastAsync(new ComputeAction(id)).Wait(); Assert.AreEqual(2, ComputeAction.InvokeCount(id)); } @@ -1049,9 +1052,11 @@ public void TestBroadcastAction() public void TestRunAction() { var id = Guid.NewGuid(); - _grid1.GetCompute().Run(new ComputeAction(id)); + Assert.AreEqual(1, ComputeAction.InvokeCount(id)); + id = Guid.NewGuid(); + _grid1.GetCompute().RunAsync(new ComputeAction(id)).Wait(); Assert.AreEqual(1, ComputeAction.InvokeCount(id)); } @@ -1081,12 +1086,12 @@ public void TestRunActionAsyncCancel() public void TestRunActions() { var id = Guid.NewGuid(); - - var actions = Enumerable.Range(0, 10).Select(x => new ComputeAction(id)); - - _grid1.GetCompute().Run(actions); - + _grid1.GetCompute().Run(Enumerable.Range(0, 10).Select(x => new ComputeAction(id))); Assert.AreEqual(10, ComputeAction.InvokeCount(id)); + + var id2 = Guid.NewGuid(); + _grid1.GetCompute().RunAsync(Enumerable.Range(0, 10).Select(x => new ComputeAction(id2))).Wait(); + Assert.AreEqual(10, ComputeAction.InvokeCount(id2)); } /// @@ -1109,7 +1114,9 @@ public void TestAffinityRun() var affinityKey = _grid1.GetAffinity(cacheName).GetAffinityKey(primaryKey); _grid1.GetCompute().AffinityRun(cacheName, affinityKey, new ComputeAction()); + Assert.AreEqual(node.Id, ComputeAction.LastNodeId); + _grid1.GetCompute().AffinityRunAsync(cacheName, affinityKey, new ComputeAction()).Wait(); Assert.AreEqual(node.Id, ComputeAction.LastNodeId); } } @@ -1138,6 +1145,15 @@ public void TestAffinityCall() Assert.AreEqual(result, ComputeFunc.InvokeCount); Assert.AreEqual(node.Id, ComputeFunc.LastNodeId); + + // Async. + ComputeFunc.InvokeCount = 0; + + result = _grid1.GetCompute().AffinityCallAsync(cacheName, affinityKey, new ComputeFunc()).Result; + + Assert.AreEqual(result, ComputeFunc.InvokeCount); + + Assert.AreEqual(node.Id, ComputeFunc.LastNodeId); } } @@ -1175,10 +1191,18 @@ public void TestWithTimeout() [Test] public void TestNetTaskSimple() { - int res = _grid1.GetCompute().Execute( - typeof(NetSimpleTask), new NetSimpleJobArgument(1)).Res; + Assert.AreEqual(2, _grid1.GetCompute() + .Execute( + typeof(NetSimpleTask), new NetSimpleJobArgument(1)).Res); + + Assert.AreEqual(2, _grid1.GetCompute() + .ExecuteAsync( + typeof(NetSimpleTask), new NetSimpleJobArgument(1)).Result.Res); - Assert.AreEqual(2, res); + Assert.AreEqual(4, _grid1.GetCompute().Execute(new NetSimpleTask(), new NetSimpleJobArgument(2)).Res); + + Assert.AreEqual(6, _grid1.GetCompute().ExecuteAsync(new NetSimpleTask(), new NetSimpleJobArgument(3)) + .Result.Res); } /// @@ -1407,6 +1431,7 @@ class ComputeFunc : INestedComputeFunc, IUserInterface int IComputeFunc.Invoke() { + Thread.Sleep(10); InvokeCount++; LastNodeId = _grid.GetCluster().GetLocalNode().Id; return InvokeCount; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/FailoverTaskSelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/FailoverTaskSelfTest.cs index 45af88887df1d..19872451c0ac2 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/FailoverTaskSelfTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/FailoverTaskSelfTest.cs @@ -19,7 +19,6 @@ namespace Apache.Ignite.Core.Tests.Compute { using System; using System.Collections.Generic; - using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Compute; using Apache.Ignite.Core.Resource; @@ -111,9 +110,9 @@ public void Cleanup() } /** */ - override protected void GetBinaryTypeConfigurations(ICollection portTypeCfgs) + protected override ICollection GetBinaryTypes() { - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(TestBinarizableJob))); + return new[] {typeof(TestBinarizableJob)}; } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs index 0c983fd23a8db..912102cb1b708 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs @@ -105,7 +105,7 @@ public void TestLocalJobError() Assert.AreEqual(1, res); - Assert.AreEqual(1, JobErrs.Count); + Assert.AreEqual(4, JobErrs.Count); Assert.IsNotNull(JobErrs.First() as GoodException); Assert.AreEqual(ErrorMode.LocJobErr, ((GoodException) JobErrs.First()).Mode); } @@ -122,7 +122,7 @@ public void TestLocalJobErrorNotMarshalable() Assert.AreEqual(1, res); - Assert.AreEqual(1, JobErrs.Count); + Assert.AreEqual(4, JobErrs.Count); Assert.IsNotNull(JobErrs.First() as BadException); // Local job exception is not marshalled. } @@ -153,7 +153,7 @@ public void TestRemoteJobError() Assert.AreEqual(1, res); - Assert.AreEqual(1, JobErrs.Count); + Assert.AreEqual(4, JobErrs.Count); Assert.IsNotNull(JobErrs.ElementAt(0) as GoodException); @@ -172,7 +172,7 @@ public void TestRemoteJobErrorNotMarshalable() Assert.AreEqual(1, res); - Assert.AreEqual(1, JobErrs.Count); + Assert.AreEqual(4, JobErrs.Count); Assert.IsNotNull(JobErrs.ElementAt(0) as IgniteException); } @@ -189,7 +189,7 @@ public void TestRemoteJobResultNotMarshalable() Assert.AreEqual(1, res); - Assert.AreEqual(1, JobErrs.Count); + Assert.AreEqual(4, JobErrs.Count); Assert.IsNotNull(JobErrs.ElementAt(0) as IgniteException); } @@ -305,9 +305,19 @@ private int Execute() { JobErrs.Clear(); - object res = Grid1.GetCompute().Execute(new Task()); + Func getRes = r => r is GoodTaskResult ? ((GoodTaskResult) r).Res : ((BadTaskResult) r).Res; - return res is GoodTaskResult ? ((GoodTaskResult)res).Res : ((BadTaskResult)res).Res; + var res1 = getRes(Grid1.GetCompute().Execute(new Task())); + var res2 = getRes(Grid1.GetCompute().Execute(typeof(Task))); + + var resAsync1 = getRes(Grid1.GetCompute().ExecuteAsync(new Task()).Result); + var resAsync2 = getRes(Grid1.GetCompute().ExecuteAsync(typeof(Task)).Result); + + Assert.AreEqual(res1, res2); + Assert.AreEqual(res2, resAsync1); + Assert.AreEqual(resAsync1, resAsync2); + + return res1; } /// @@ -318,20 +328,7 @@ private Exception ExecuteWithError() { JobErrs.Clear(); - Exception err = null; - - try - { - Grid1.GetCompute().Execute(new Task()); - - Assert.Fail(); - } - catch (Exception e) - { - err = e; - } - - return err; + return Assert.Catch(() => Grid1.GetCompute().Execute(new Task())); } /// @@ -391,11 +388,11 @@ public enum ErrorMode /// /// Task. /// - public class Task : IComputeTask + private class Task : IComputeTask { /** Grid. */ [InstanceResource] - private IIgnite _grid = null; + private readonly IIgnite _grid = null; /** Result. */ private int _res; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ResourceTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ResourceTaskTest.cs index 522341a3a02a0..433b635e37e12 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ResourceTaskTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ResourceTaskTest.cs @@ -53,6 +53,17 @@ public void TestTaskInjection() Assert.AreEqual(GetServerCount(), res); } + /// + /// Test Ignite injection into the task. + /// + [Test] + public void TestTaskInjectionBinarizable() + { + int res = Grid1.GetCompute().Execute(new InjectionTaskBinarizable(), 0); + + Assert.AreEqual(GetServerCount(), res); + } + /// /// Test Ignite injection into the closure. /// @@ -86,6 +97,12 @@ public void TestNoResultCache() Assert.AreEqual(GetServerCount(), res); } + /** */ + protected override ICollection GetBinaryTypes() + { + return new[] {typeof(InjectionJobBinarizable)}; + } + /// /// Injection task. /// @@ -112,6 +129,40 @@ public int Reduce(IList> results) } } + /// + /// Injection task. + /// + private class InjectionTaskBinarizable : Injectee, IComputeTask + { + /** */ + public IDictionary, IClusterNode> Map(IList subgrid, object arg) + { + CheckInjection(); + + return subgrid.ToDictionary(x => (IComputeJob) new InjectionJobBinarizable(), x => x); + } + + /** */ + public ComputeJobResultPolicy OnResult(IComputeJobResult res, IList> rcvd) + { + return ComputeJobResultPolicy.Wait; + } + + /** */ + public int Reduce(IList> results) + { + return results.Sum(res => res.Data); + } + } + + /// + /// Binarizable job. + /// + public class InjectionJobBinarizable : InjectionJob + { + // No-op. + } + /// /// Injection job. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs index ded56edbd8b8d..8db4876fc1c8d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs @@ -66,6 +66,11 @@ protected override void CheckError(Exception err) { Assert.IsTrue(err != null); + var aggregate = err as AggregateException; + + if (aggregate != null) + err = aggregate.InnerException; + SerializableException err0 = err as SerializableException; Assert.IsTrue(err0 != null); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskAdapterTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskAdapterTest.cs index 7789ac4d3b255..32a28a77505e3 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskAdapterTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskAdapterTest.cs @@ -22,7 +22,6 @@ namespace Apache.Ignite.Core.Tests.Compute using System; using System.Collections.Generic; using System.Linq; - using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Compute; using Apache.Ignite.Core.Resource; using NUnit.Framework; @@ -102,9 +101,9 @@ public void TestBinarizableJobAdapter() } /** */ - override protected void GetBinaryTypeConfigurations(ICollection portTypeCfgs) + protected override ICollection GetBinaryTypes() { - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableJob))); + return new[] { typeof(BinarizableJob) }; } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskResultTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskResultTest.cs index 26286de4f51e2..289b68b361080 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskResultTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskResultTest.cs @@ -19,7 +19,6 @@ namespace Apache.Ignite.Core.Tests.Compute { using System; using System.Collections.Generic; - using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Compute; using Apache.Ignite.Core.Resource; @@ -156,12 +155,15 @@ public void TestTaskResultLarge() } /** */ - override protected void GetBinaryTypeConfigurations(ICollection portTypeCfgs) + protected override ICollection GetBinaryTypes() { - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableResult))); - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(TestBinarizableJob))); - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableOutFunc))); - portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableFunc))); + return new[] + { + typeof(BinarizableResult), + typeof(TestBinarizableJob), + typeof(BinarizableOutFunc), + typeof(BinarizableFunc) + }; } [Test] diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-standalone.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-standalone.xml index af5f49951555b..3990e3b22b701 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-standalone.xml +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-standalone.xml @@ -68,6 +68,7 @@ Apache.Ignite.Core.Tests.Compute.BinarizableClosureTaskTest+BinarizableFunc Apache.Ignite.Core.Tests.Compute.BinarizableClosureTaskTest+BinarizableResult Apache.Ignite.Core.Tests.Compute.BinarizableClosureTaskTest+BinarizableException + Apache.Ignite.Core.Tests.Compute.ResourceTaskTest+InjectionJobBinarizable diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs index c2ccd1a8b7ba5..cc214906d09cf 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs @@ -15,6 +15,7 @@ * limitations under the License. */ +#pragma warning disable 618 namespace Apache.Ignite.Core.Tests { using System; @@ -25,6 +26,7 @@ namespace Apache.Ignite.Core.Tests using System.Threading.Tasks; using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cache.Query; + using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Common; using Apache.Ignite.Core.Events; using Apache.Ignite.Core.Impl; @@ -141,7 +143,7 @@ public void TestLocalListen() CheckSend(2); // Unsubscribe from all events - events.StopLocalListen(listener); + events.StopLocalListen(listener, Enumerable.Empty()); CheckNoEvent(); @@ -298,11 +300,20 @@ public void TestLocalQuery() Assert.AreEqual(3, qryResult.Count); } + /// + /// Tests the record local. + /// + [Test] + public void TestRecordLocal() + { + Assert.Throws(() => _grid1.GetEvents().RecordLocal(new MyEvent())); + } + /// /// Tests the WaitForLocal. /// [Test] - public void TestWaitForLocal([Values(true, false)] bool async) + public void TestWaitForLocal() { var events = _grid1.GetEvents(); @@ -312,50 +323,80 @@ public void TestWaitForLocal([Values(true, false)] bool async) events.EnableLocal(eventType); - Func, int[], Task> getWaitTask; + var taskFuncs = GetWaitTasks(events).Select( + func => (Func, int[], Task>) ( + (filter, types) => + { + var task = func(filter, types); - if (async) - getWaitTask = (filter, types) => - { - var task = events.WaitForLocalAsync(filter, types); - GenerateTaskEvent(); - return task; - }; - else - getWaitTask = (filter, types) => + Thread.Sleep(100); // allow task to start and begin waiting for events + + GenerateTaskEvent(); + + return task; + })).ToArray(); + + for (int i = 0; i < taskFuncs.Length; i++) + { + var getWaitTask = taskFuncs[i]; + + // No params + var waitTask = getWaitTask(null, new int[0]); + + waitTask.Wait(timeout); + + // Event types + waitTask = getWaitTask(null, new[] {EventType.TaskReduced}); + + Assert.IsTrue(waitTask.Wait(timeout)); + Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result); + Assert.AreEqual(EventType.TaskReduced, waitTask.Result.Type); + + if (i > 3) { - var task = Task.Factory.StartNew(() => events.WaitForLocal(filter, types)); - Thread.Sleep(500); // allow task to start and begin waiting for events - GenerateTaskEvent(); - return task; - }; + // Filter + waitTask = getWaitTask(new EventFilter(e => e.Type == EventType.TaskReduced), new int[0]); - // No params - var waitTask = getWaitTask(null, new int[0]); + Assert.IsTrue(waitTask.Wait(timeout)); + Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result); + Assert.AreEqual(EventType.TaskReduced, waitTask.Result.Type); - waitTask.Wait(timeout); + // Filter & types + waitTask = getWaitTask(new EventFilter(e => e.Type == EventType.TaskReduced), + new[] {EventType.TaskReduced}); - // Event types - waitTask = getWaitTask(null, new[] {EventType.TaskReduced}); + Assert.IsTrue(waitTask.Wait(timeout)); + Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result); + Assert.AreEqual(EventType.TaskReduced, waitTask.Result.Type); + } + } + } - Assert.IsTrue(waitTask.Wait(timeout)); - Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result); - Assert.AreEqual(EventType.TaskReduced, waitTask.Result.Type); + /// + /// Gets the wait tasks for different overloads of WaitForLocal. + /// + private static IEnumerable, int[], Task>> GetWaitTasks(IEvents events) + { + yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(types)); + yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(types.ToList())); - // Filter - waitTask = getWaitTask(new EventFilter(e => e.Type == EventType.TaskReduced), new int[0]); + yield return (filter, types) => events.WaitForLocalAsync(types); + yield return (filter, types) => events.WaitForLocalAsync(types.ToList()); - Assert.IsTrue(waitTask.Wait(timeout)); - Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result); - Assert.AreEqual(EventType.TaskReduced, waitTask.Result.Type); + yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(filter, types)); + yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(filter, types.ToList())); - // Filter & types - waitTask = getWaitTask(new EventFilter(e => e.Type == EventType.TaskReduced), - new[] {EventType.TaskReduced}); + yield return (filter, types) => events.WaitForLocalAsync(filter, types); + yield return (filter, types) => events.WaitForLocalAsync(filter, types.ToList()); + } - Assert.IsTrue(waitTask.Wait(timeout)); - Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result); - Assert.AreEqual(EventType.TaskReduced, waitTask.Result.Type); + /// + /// Tests the wait for local overloads. + /// + [Test] + public void TestWaitForLocalOverloads() + { + } /* @@ -492,6 +533,7 @@ public void TestSerialization() Assert.AreEqual(expectedGuid, cacheEvent.SubjectId); Assert.AreEqual("cloClsName", cacheEvent.ClosureClassName); Assert.AreEqual("taskName", cacheEvent.TaskName); + Assert.IsTrue(cacheEvent.ToShortString().StartsWith("SWAP_SPACE_CLEARED: IsNear=")); var qryExecEvent = EventReader.Read(reader); CheckEventBase(qryExecEvent); @@ -501,6 +543,9 @@ public void TestSerialization() Assert.AreEqual("clause", qryExecEvent.Clause); Assert.AreEqual(expectedGuid, qryExecEvent.SubjectId); Assert.AreEqual("taskName", qryExecEvent.TaskName); + Assert.AreEqual( + "SWAP_SPACE_CLEARED: QueryType=qryType, CacheName=cacheName, ClassName=clsName, Clause=clause, " + + "SubjectId=00000000-0000-0001-0000-000000000002, TaskName=taskName", qryExecEvent.ToShortString()); var qryReadEvent = EventReader.Read(reader); CheckEventBase(qryReadEvent); @@ -514,6 +559,10 @@ public void TestSerialization() Assert.AreEqual(2, qryReadEvent.Value); Assert.AreEqual(3, qryReadEvent.OldValue); Assert.AreEqual(4, qryReadEvent.Row); + Assert.AreEqual( + "SWAP_SPACE_CLEARED: QueryType=qryType, CacheName=cacheName, ClassName=clsName, Clause=clause, " + + "SubjectId=00000000-0000-0001-0000-000000000002, TaskName=taskName, Key=1, Value=2, " + + "OldValue=3, Row=4", qryReadEvent.ToShortString()); var cacheRebalancingEvent = EventReader.Read(reader); CheckEventBase(cacheRebalancingEvent); @@ -522,15 +571,19 @@ public void TestSerialization() Assert.AreEqual(locNode, cacheRebalancingEvent.DiscoveryNode); Assert.AreEqual(2, cacheRebalancingEvent.DiscoveryEventType); Assert.AreEqual(3, cacheRebalancingEvent.DiscoveryTimestamp); - + Assert.IsTrue(cacheRebalancingEvent.ToShortString().StartsWith( + "SWAP_SPACE_CLEARED: CacheName=cacheName, Partition=1, DiscoveryNode=GridNode")); + var checkpointEvent = EventReader.Read(reader); CheckEventBase(checkpointEvent); Assert.AreEqual("cpKey", checkpointEvent.Key); - + Assert.AreEqual("SWAP_SPACE_CLEARED: Key=cpKey", checkpointEvent.ToShortString()); + var discoEvent = EventReader.Read(reader); CheckEventBase(discoEvent); Assert.AreEqual(grid.TopologyVersion, discoEvent.TopologyVersion); Assert.AreEqual(grid.GetNodes(), discoEvent.TopologyNodes); + Assert.IsTrue(discoEvent.ToShortString().StartsWith("SWAP_SPACE_CLEARED: EventNode=GridNode")); var jobEvent = EventReader.Read(reader); CheckEventBase(jobEvent); @@ -540,10 +593,12 @@ public void TestSerialization() Assert.AreEqual(locNode, jobEvent.TaskNode); Assert.AreEqual(expectedGridGuid, jobEvent.TaskSessionId); Assert.AreEqual(expectedGuid, jobEvent.TaskSubjectId); + Assert.IsTrue(jobEvent.ToShortString().StartsWith("SWAP_SPACE_CLEARED: TaskName=taskName")); var spaceEvent = EventReader.Read(reader); CheckEventBase(spaceEvent); Assert.AreEqual("space", spaceEvent.Space); + Assert.IsTrue(spaceEvent.ToShortString().StartsWith("SWAP_SPACE_CLEARED: Space=space")); var taskEvent = EventReader.Read(reader); CheckEventBase(taskEvent); @@ -552,6 +607,7 @@ public void TestSerialization() Assert.AreEqual("taskClsName", taskEvent.TaskClassName); Assert.AreEqual("taskName", taskEvent.TaskName); Assert.AreEqual(expectedGridGuid, taskEvent.TaskSessionId); + Assert.IsTrue(taskEvent.ToShortString().StartsWith("SWAP_SPACE_CLEARED: TaskName=taskName")); } } @@ -570,6 +626,11 @@ private void CheckEventBase(IEvent evt) Assert.AreNotEqual(Guid.Empty, evt.Id.GlobalId); Assert.IsTrue(Math.Abs((evt.Timestamp - DateTime.UtcNow).TotalSeconds) < 20, "Invalid event timestamp: '{0}', current time: '{1}'", evt.Timestamp, DateTime.Now); + + Assert.Greater(evt.LocalOrder, 0); + + Assert.IsTrue(evt.ToString().Contains("[Name=SWAP_SPACE_CLEARED")); + Assert.IsTrue(evt.ToShortString().StartsWith("SWAP_SPACE_CLEARED")); } /// @@ -855,6 +916,7 @@ bool IEventListener.Invoke(T evt) } /** */ + // ReSharper disable once UnusedMember.Global public bool Invoke(T evt) { throw new Exception("Invalid method"); @@ -954,4 +1016,58 @@ public override string ToString() return EventObjectType.ToString(); } } + + /// + /// Custom event. + /// + public class MyEvent : IEvent + { + /** */ + public IgniteGuid Id + { + get { throw new NotImplementedException(); } + } + + /** */ + public long LocalOrder + { + get { throw new NotImplementedException(); } + } + + /** */ + public IClusterNode Node + { + get { throw new NotImplementedException(); } + } + + /** */ + public string Message + { + get { throw new NotImplementedException(); } + } + + /** */ + public int Type + { + get { throw new NotImplementedException(); } + } + + /** */ + public string Name + { + get { throw new NotImplementedException(); } + } + + /** */ + public DateTime Timestamp + { + get { throw new NotImplementedException(); } + } + + /** */ + public string ToShortString() + { + throw new NotImplementedException(); + } + } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs index 02a5d0bf9370f..f6730d7bf4a0b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs @@ -27,7 +27,7 @@ namespace Apache.Ignite.Core.Tests.Examples /// /// Tests all examples in various modes. /// - [Category(TestUtils.CategoryIntensive)] + [Category(TestUtils.CategoryExamples)] public class ExamplesTest { /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs index a3241910144c9..e766f5a4f4fa1 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs @@ -121,6 +121,52 @@ public void TestPartialUpdateExceptionSerialization() })); } + /// + /// Tests that all exceptions have mandatory constructors and are serializable. + /// + [Test] + public void TestAllExceptionsConstructors() + { + var types = typeof(IIgnite).Assembly.GetTypes().Where(x => x.IsSubclassOf(typeof(Exception))); + + foreach (var type in types) + { + Assert.IsTrue(type.IsSerializable, "Exception is not serializable: " + type); + + // Default ctor. + var defCtor = type.GetConstructor(new Type[0]); + Assert.IsNotNull(defCtor); + + var ex = (Exception) defCtor.Invoke(new object[0]); + Assert.AreEqual(string.Format("Exception of type '{0}' was thrown.", type.FullName), ex.Message); + + // Message ctor. + var msgCtor = type.GetConstructor(new[] {typeof(string)}); + Assert.IsNotNull(msgCtor); + + ex = (Exception) msgCtor.Invoke(new object[] {"myMessage"}); + Assert.AreEqual("myMessage", ex.Message); + + // Serialization. + var stream = new MemoryStream(); + var formatter = new BinaryFormatter(); + + formatter.Serialize(stream, ex); + stream.Seek(0, SeekOrigin.Begin); + + ex = (Exception) formatter.Deserialize(stream); + Assert.AreEqual("myMessage", ex.Message); + + // Message+cause ctor. + var msgCauseCtor = type.GetConstructor(new[] { typeof(string), typeof(Exception) }); + Assert.IsNotNull(msgCauseCtor); + + ex = (Exception) msgCauseCtor.Invoke(new object[] {"myMessage", new Exception("innerEx")}); + Assert.AreEqual("myMessage", ex.Message); + Assert.AreEqual("innerEx", ex.InnerException.Message); + } + } + /// /// Tests CachePartialUpdateException serialization. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs index d3851dbca33e1..88a2b522fdaa1 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs @@ -39,6 +39,9 @@ public static class TestUtils /** Indicates long running and/or memory/cpu intensive test. */ public const string CategoryIntensive = "LONG_TEST"; + /** Indicates examples tests. */ + public const string CategoryExamples = "EXAMPLES_TEST"; + /** */ public const int DfltBusywaitSleepInterval = 200; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj index 6fa13789c89b9..8e8f8caadc89e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj @@ -310,7 +310,6 @@ - @@ -351,7 +350,6 @@ - @@ -386,7 +384,6 @@ - diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityKey.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityKey.cs index 1d27d65ce9528..1b7fcb0466367 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityKey.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityKey.cs @@ -158,5 +158,16 @@ public override int GetHashCode() { return !left.Equals(right); } + + /// + /// Returns a that represents this instance. + /// + /// + /// A that represents this instance. + /// + public override string ToString() + { + return string.Format("AffinityKey [Key={0}, Affinity={1}]", _key, _affinity); + } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs index 907af142f4907..484fcebce4f6e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs @@ -71,7 +71,8 @@ protected CachePartialUpdateException(SerializationInfo info, StreamingContext c /// /// Exception message. /// Exception occurred during failed keys read/write. - public CachePartialUpdateException(string msg, Exception failedKeysException) : this(msg, null, failedKeysException) + public CachePartialUpdateException(string msg, Exception failedKeysException) + : this(msg, null, failedKeysException) { // No-op. } @@ -92,7 +93,8 @@ public CachePartialUpdateException(string msg, IList failedKeys) : this( /// Exception message. /// Failed keys. /// Exception occurred during failed keys read/write. - private CachePartialUpdateException(string msg, IList failedKeys, Exception failedKeysException) : base(msg) + private CachePartialUpdateException(string msg, IList failedKeys, Exception failedKeysException) + : base(msg, failedKeysException) { _failedKeys = failedKeys; _failedKeysException = failedKeysException; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Store/CacheParallelLoadStoreAdapter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Store/CacheParallelLoadStoreAdapter.cs index 147f1bd529968..c5068386dfcaa 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Store/CacheParallelLoadStoreAdapter.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Store/CacheParallelLoadStoreAdapter.cs @@ -110,6 +110,7 @@ public virtual void LoadCache(Action act, params object[] args) /// The value for the entry that is to be stored in the cache /// or null if the object can't be loaded /// + [ExcludeFromCodeCoverage] public virtual object Load(object key) { return null; @@ -124,6 +125,7 @@ public virtual object Load(object key) /// /// A map of key, values to be stored in the cache. /// + [ExcludeFromCodeCoverage] public virtual IDictionary LoadAll(ICollection keys) { return null; @@ -136,6 +138,7 @@ public virtual IDictionary LoadAll(ICollection keys) /// /// Key to write. /// Value to write. + [ExcludeFromCodeCoverage] public virtual void Write(object key, object val) { // No-op. @@ -154,6 +157,7 @@ public virtual void Write(object key, object val) /// a mutable collection to write. Upon invocation, it contains the entries /// to write for write-through. Upon return the collection must only contain entries /// that were not successfully written. (see partial success above). + [ExcludeFromCodeCoverage] public virtual void WriteAll(IDictionary entries) { // No-op. @@ -167,6 +171,7 @@ public virtual void WriteAll(IDictionary entries) /// This method is invoked even if no mapping for the key exists. /// /// The key that is used for the delete operation. + [ExcludeFromCodeCoverage] public virtual void Delete(object key) { // No-op. @@ -189,6 +194,7 @@ public virtual void Delete(object key) /// a mutable collection of keys for entries to delete. Upon invocation, /// it contains the keys to delete for write-through. Upon return the collection must only contain /// the keys that were not successfully deleted. + [ExcludeFromCodeCoverage] public virtual void DeleteAll(ICollection keys) { // No-op. @@ -199,6 +205,7 @@ public virtual void DeleteAll(ICollection keys) /// commit parameter. /// /// True if transaction should commit, false for rollback. + [ExcludeFromCodeCoverage] public virtual void SessionEnd(bool commit) { // No-op. diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs index 13d31331d2c3b..39a2f8ba6f1cc 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs @@ -21,6 +21,7 @@ namespace Apache.Ignite.Core.Impl.Binary using System.Collections; using System.Collections.Generic; using System.Diagnostics; + using System.Diagnostics.CodeAnalysis; using System.IO; using System.Runtime.CompilerServices; using System.Text; @@ -125,6 +126,7 @@ public T Deserialize() } /** */ + [ExcludeFromCodeCoverage] public int EnumValue { get diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs index 646d5631f8836..1626a2df9d006 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs @@ -1027,7 +1027,7 @@ private static void TransferBytes(BinaryHeapStream inStream, IBinaryStream outSt } /// - /// Mutation ocntext. + /// Mutation context. /// private class Context { diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs index 2624d52225bb1..bb5c207210278 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs @@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Impl.Binary { using System; using System.Diagnostics; + using System.Diagnostics.CodeAnalysis; using System.IO; using System.Runtime.InteropServices; using Apache.Ignite.Core.Impl.Binary.IO; @@ -100,6 +101,7 @@ public BinaryObjectHeader(int typeId, int hashCode, int length, int schemaId, in /// Initializes a new instance of the struct from specified stream. /// /// The stream. + [ExcludeFromCodeCoverage] // big-endian only private BinaryObjectHeader(IBinaryStream stream) { Header = stream.ReadByte(); @@ -116,6 +118,7 @@ private BinaryObjectHeader(IBinaryStream stream) /// Writes this instance to the specified stream. /// /// The stream. + [ExcludeFromCodeCoverage] // big-endian only private void Write(IBinaryStream stream) { stream.WriteByte(Header); @@ -292,7 +295,7 @@ public bool Equals(BinaryObjectHeader other) public override bool Equals(object obj) { if (ReferenceEquals(null, obj)) return false; - + return obj is BinaryObjectHeader && Equals((BinaryObjectHeader) obj); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs index 16e3032b31f2b..b572e7c6d76d1 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs @@ -58,7 +58,7 @@ public BinarySurrogateTypeDescriptor(BinaryConfiguration cfg, int id) } /// - /// Constrcutor. + /// Constructor. /// /// Configuration. /// Type name. diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs index 585ccd32fbfb1..77a22ddd2586e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs @@ -1441,26 +1441,6 @@ internal ICollection GetBinaryTypes() return _metas == null ? null : _metas.Values; } - /// - /// Check whether the given object is binarizeble, i.e. it can be serialized with binary marshaller. - /// - /// Object. - /// True if binarizable. - internal bool IsBinarizable(object obj) - { - if (obj != null) - { - Type type = obj.GetType(); - - // We assume object as binarizable only in case it has descriptor. - // Collections, Enums and non-primitive arrays do not have descriptors - // and this is fine here because we cannot know whether their members are binarizable. - return _marsh.GetDescriptor(type) != null || BinarySystemHandlers.GetWriteHandler(type) != null; - } - - return true; - } - /// /// Write field ID. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamAdapter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamAdapter.cs index dcbff81d6967f..b062689becbb8 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamAdapter.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamAdapter.cs @@ -18,6 +18,7 @@ namespace Apache.Ignite.Core.Impl.Binary.IO { using System; + using System.Diagnostics.CodeAnalysis; using System.IO; /// @@ -78,12 +79,14 @@ public override bool CanSeek } /** */ + [ExcludeFromCodeCoverage] public override long Seek(long offset, SeekOrigin origin) { throw new NotSupportedException("Stream is not seekable."); } /** */ + [ExcludeFromCodeCoverage] public override long Position { get @@ -97,6 +100,7 @@ public override long Position } /** */ + [ExcludeFromCodeCoverage] public override long Length { get @@ -106,6 +110,7 @@ public override long Length } /** */ + [ExcludeFromCodeCoverage] public override void SetLength(long value) { throw new NotSupportedException("Stream is not seekable."); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamBase.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamBase.cs index a5e140d2a4dfd..62866024941a7 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamBase.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamBase.cs @@ -25,7 +25,7 @@ namespace Apache.Ignite.Core.Impl.Binary.IO /// /// Base class for managed and unmanaged data streams. /// - internal unsafe abstract class BinaryStreamBase : IBinaryStream + internal abstract unsafe class BinaryStreamBase : IBinaryStream { /** Byte: zero. */ private const byte ByteZero = 0; @@ -1070,13 +1070,10 @@ public int Position /// /// True if they are same. /// - public virtual bool IsSameArray(byte[] arr) - { - return false; - } - + public abstract bool IsSameArray(byte[] arr); + /// - /// Seek to the given positoin. + /// Seek to the given position. /// /// Offset. /// Seek origin. diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs index d530713a29c26..cd509c6e04a0f 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs @@ -312,7 +312,7 @@ int Position bool IsSameArray(byte[] arr); /// - /// Seek to the given positoin. + /// Seek to the given position. /// /// Offset. /// Seek origin. diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs index a8d94f279df12..109d55f8c5fe2 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs @@ -62,9 +62,6 @@ internal static class JavaTypes private static readonly Dictionary JavaToNet = NetToJava.GroupBy(x => x.Value).ToDictionary(g => g.Key, g => g.First().Key); - /** */ - private static readonly string MappedTypes = string.Join(", ", NetToJava.Keys.Select(x => x.Name)); - /// /// Gets the corresponding Java type name. /// @@ -110,13 +107,5 @@ public static Type GetDotNetType(string javaTypeName) return JavaToNet.TryGetValue(javaTypeName, out res) ? res : null; } - - /// - /// Gets the supported types as a comma-separated string. - /// - public static string SupportedTypesString - { - get { return MappedTypes; } - } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Event/JavaCacheEntryEventFilter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Event/JavaCacheEntryEventFilter.cs index b5c2ece981dc6..a56e12a18622a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Event/JavaCacheEntryEventFilter.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Event/JavaCacheEntryEventFilter.cs @@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Impl.Cache.Event { using System; using System.Collections.Generic; + using System.Diagnostics.CodeAnalysis; using Apache.Ignite.Core.Cache.Event; using Apache.Ignite.Core.Impl.Common; @@ -30,6 +31,7 @@ namespace Apache.Ignite.Core.Impl.Cache.Event internal class JavaCacheEntryEventFilter : PlatformJavaObjectFactoryProxy, ICacheEntryEventFilter { /** */ + [ExcludeFromCodeCoverage] public bool Evaluate(ICacheEntryEvent evt) { throw new InvalidOperationException(GetType() + " cannot be invoked directly."); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/MultiValueDictionary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/MultiValueDictionary.cs index 2adb021fe2ba5..a4e9c93552048 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/MultiValueDictionary.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/MultiValueDictionary.cs @@ -51,32 +51,6 @@ public void Add(TKey key, TValue val) _dict[key] = val; } - /// - /// Tries the get a value. In case of multiple values for a key, returns the last one. - /// - /// The key. - /// The value. - /// True if value has been found for specified key; otherwise false. - public bool TryGetValue(TKey key, out TValue val) - { - object val0; - - if (!_dict.TryGetValue(key, out val0)) - { - val = default(TValue); - return false; - } - - var list = val0 as List; - - if (list != null) - val = list[list.Count - 1]; - else - val = (TValue) val0; - - return true; - } - /// /// Removes the specified value for the specified key. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyDictionary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyDictionary.cs index 60ec9d0863e6f..a51a14960da3e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyDictionary.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyDictionary.cs @@ -110,7 +110,7 @@ public void Add(TKey key, TValue value) /** */ public bool Remove(TKey key) { - return _dict.Remove(key); + throw GetReadonlyException(); } /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ResizeableArray.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ResizeableArray.cs deleted file mode 100644 index 82a8eeef5d352..0000000000000 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ResizeableArray.cs +++ /dev/null @@ -1,64 +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. - */ - -namespace Apache.Ignite.Core.Impl.Common -{ - using System.Collections.Generic; - - /// - /// Simple append-only alternative which exposes internal array. - /// - internal class ResizeableArray - { - /** Array. */ - private T[] _arr; - - /// - /// Constructor. - /// - /// Capacity. - public ResizeableArray(int capacity) - { - _arr = new T[capacity]; - } - - /// - /// Array. - /// - public T[] Array - { - get { return _arr; } - } - - /// - /// Count. - /// - public int Count { get; private set; } - - /// - /// Add element. - /// - /// Element. - public void Add(T element) - { - if (Count == _arr.Length) - System.Array.Resize(ref _arr, _arr.Length*2); - - _arr[Count++] = element; - } - } -} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventTypeConverter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventTypeConverter.cs index 6b8f9357035d5..f0d15640c61f7 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventTypeConverter.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventTypeConverter.cs @@ -20,6 +20,7 @@ namespace Apache.Ignite.Core.Impl.Events using System; using System.Collections.Generic; using System.ComponentModel; + using System.Diagnostics.CodeAnalysis; using System.Globalization; using System.Linq; using Apache.Ignite.Core.Events; @@ -57,6 +58,7 @@ internal class EventTypeConverter : TypeConverter /// /// true if this converter can perform the conversion; otherwise, false. /// + [ExcludeFromCodeCoverage] // not called public override bool CanConvertFrom(ITypeDescriptorContext context, Type sourceType) { return sourceType == typeof(string); @@ -72,6 +74,7 @@ public override bool CanConvertFrom(ITypeDescriptorContext context, Type sourceT /// /// true if this converter can perform the conversion; otherwise, false. /// + [ExcludeFromCodeCoverage] // not called public override bool CanConvertTo(ITypeDescriptorContext context, Type destinationType) { return destinationType == typeof(string); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs index 1ec7cabe6f28b..eb454d62f3b12 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs @@ -130,6 +130,7 @@ public ICollection RemoteQuery(IEventFilter filter, TimeSpan? timeout = } /** */ + [ExcludeFromCodeCoverage] public Guid? RemoteListen(int bufSize = 1, TimeSpan? interval = null, bool autoUnsubscribe = true, IEventFilter localListener = null, IEventFilter remoteFilter = null, params int[] types) where T : IEvent @@ -163,6 +164,7 @@ public ICollection RemoteQuery(IEventFilter filter, TimeSpan? timeout = } /** */ + [ExcludeFromCodeCoverage] public Guid? RemoteListen(int bufSize = 1, TimeSpan? interval = null, bool autoUnsubscribe = true, IEventFilter localListener = null, IEventFilter remoteFilter = null, IEnumerable types = null) where T : IEvent @@ -171,6 +173,7 @@ public ICollection RemoteQuery(IEventFilter filter, TimeSpan? timeout = } /** */ + [ExcludeFromCodeCoverage] public void StopRemoteListen(Guid opId) { DoOutOp((int) Op.StopRemoteListen, writer => diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs index 461872f8f7929..a59ca5f31d327 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs @@ -205,6 +205,7 @@ static ExceptionUtils() /// Message. /// Stack trace. /// Exception. + [ExcludeFromCodeCoverage] // Covered by a test in a separate process. public static Exception GetJvmInitializeException(string clsName, string msg, string stackTrace) { if (clsName != null) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/Handle.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/Handle.cs index fb56891589e15..7791c914dc163 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/Handle.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/Handle.cs @@ -39,7 +39,7 @@ public class Handle : IHandle /// /// Target. /// Release action. - public Handle(T target, Action releaseAction) + protected Handle(T target, Action releaseAction) { _target = target; _releaseAction = releaseAction; @@ -48,7 +48,7 @@ public Handle(T target, Action releaseAction) /// /// Target. /// - public T Target + protected T Target { get { return _target; } } @@ -61,13 +61,5 @@ public void Release() if (Interlocked.CompareExchange(ref _released, 1, 0) == 0) _releaseAction(_target); } - - /// - /// Resource released flag. - /// - public bool Released - { - get { return Thread.VolatileRead(ref _released) == 1; } - } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs index e18970f8561cb..4e1135a06f9f2 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs @@ -101,16 +101,6 @@ public long AllocateCritical(object target) return Allocate0(target, true, false); } - /// - /// Allocate a handle for critical resource in safe mode. - /// - /// Target. - /// Pointer. - public long AllocateCriticalSafe(object target) - { - return Allocate0(target, true, true); - } - /// /// Internal allocation routine. /// @@ -121,7 +111,7 @@ public long AllocateCriticalSafe(object target) private long Allocate0(object target, bool critical, bool safe) { if (Closed) - throw ClosedException(); + throw GetClosedException(); // Try allocating on critical path. if (critical) @@ -140,7 +130,7 @@ private long Allocate0(object target, bool critical, bool safe) Release0(target, true); - throw ClosedException(); + throw GetClosedException(); } return fastIdx; @@ -159,7 +149,7 @@ private long Allocate0(object target, bool critical, bool safe) Release0(target, true); - throw ClosedException(); + throw GetClosedException(); } return slowIdx; @@ -320,6 +310,7 @@ public int Count /// Gets a snapshot of currently referenced objects list. /// [SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate", Justification = "Semantics.")] + [ExcludeFromCodeCoverage] public IList> GetItems() { Thread.MemoryBarrier(); @@ -335,7 +326,8 @@ public int Count /// Create new exception for closed state. /// /// Exception. - private static Exception ClosedException() + [ExcludeFromCodeCoverage] + private static Exception GetClosedException() { return new InvalidOperationException("Cannot allocate a resource handle because Ignite is stopping."); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/IHandle.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/IHandle.cs index d147f8be54dba..700ab5f8c8b03 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/IHandle.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/IHandle.cs @@ -26,10 +26,5 @@ public interface IHandle /// Release the resource. /// void Release(); - - /// - /// Resource released flag. - /// - bool Released { get; } } } \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs index 914a87dedaa49..8dc63bd02a9f9 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs @@ -41,6 +41,7 @@ namespace Apache.Ignite.Core.Impl /// Grid proxy with fake serialization. /// [Serializable] + [ExcludeFromCodeCoverage] internal class IgniteProxy : IIgnite, IClusterGroupEx, IBinaryWriteAware, ICluster { /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs index 0a69e4a8ab0f5..de9daaee5b66c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs @@ -20,7 +20,6 @@ namespace Apache.Ignite.Core.Impl using System; using System.Collections.Generic; using System.ComponentModel; - using System.Diagnostics; using System.Diagnostics.CodeAnalysis; using System.Globalization; using System.IO; @@ -238,6 +237,7 @@ private static void LoadJvmDll(string configJvmDllPath, ILogger log) /// /// Formats the Win32 error. /// + [ExcludeFromCodeCoverage] private static string FormatWin32Error(int errorCode) { if (errorCode == NativeMethods.ERROR_BAD_EXE_FORMAT) @@ -493,25 +493,5 @@ public static List ReadNodes(IBinaryRawReader reader, Func - /// Writes the node collection to a stream. - /// - /// The writer. - /// The nodes. - public static void WriteNodes(IBinaryRawWriter writer, ICollection nodes) - { - Debug.Assert(writer != null); - - if (nodes != null) - { - writer.WriteInt(nodes.Count); - - foreach (var node in nodes) - writer.WriteGuid(node.Id); - } - else - writer.WriteInt(-1); - } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/InteropExceptionHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/InteropExceptionHolder.cs deleted file mode 100644 index 9edcb03ee0079..0000000000000 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/InteropExceptionHolder.cs +++ /dev/null @@ -1,88 +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. - */ - -namespace Apache.Ignite.Core.Impl -{ - using System; - using System.Diagnostics.CodeAnalysis; - using System.Runtime.Serialization.Formatters.Binary; - using Apache.Ignite.Core.Binary; - using Apache.Ignite.Core.Impl.Binary; - using Apache.Ignite.Core.Impl.Binary.IO; - - /// - /// Holder of exception which must be serialized to Java and then backwards to the native platform. - /// - internal class InteropExceptionHolder : IBinarizable - { - /** Initial exception. */ - private readonly Exception _err; - - /// - /// Constructor. - /// - public InteropExceptionHolder() - { - // No-op. - } - - /// - /// Constructor. - /// - /// Error. - public InteropExceptionHolder(Exception err) - { - _err = err; - } - - /// - /// Underlying exception. - /// - public Exception Error - { - get { return _err; } - } - - /** */ - [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")] - public void WriteBinary(IBinaryWriter writer) - { - var writer0 = (BinaryWriter) writer.GetRawWriter(); - - if (writer0.IsBinarizable(_err)) - { - writer0.WriteBoolean(true); - writer0.WriteObject(_err); - } - else - { - writer0.WriteBoolean(false); - - using (var streamAdapter = new BinaryStreamAdapter(writer0.Stream)) - { - new BinaryFormatter().Serialize(streamAdapter, _err); - } - } - } - - /** */ - public void ReadBinary(IBinaryReader reader) - { - throw new NotImplementedException(); - } - } -} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs index a991b3d51c01e..3aa54902aa96a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs @@ -87,16 +87,6 @@ public static int GetCapacity(long memPtr) return *((int*)(memPtr + MemHdrOffCap)); } - /// - /// Sets capacity for the given memory chunk. - /// - /// Memory pointer. - /// CalculateCapacity. - public static void SetCapacity(long memPtr, int cap) - { - *((int*)(memPtr + MemHdrOffCap)) = cap; - } - /// /// Gets length for the given memory chunk. /// @@ -137,16 +127,6 @@ public static void SetFlags(long memPtr, int flags) *((int*)(memPtr + MemHdrOffFlags)) = flags; } - /// - /// Check whether this memory chunk is external. - /// - /// Memory pointer. - /// True if owned by Java. - public static bool IsExternal(long memPtr) - { - return IsExternal(GetFlags(memPtr)); - } - /// /// Check whether flags denote that this memory chunk is external. /// @@ -157,16 +137,6 @@ public static bool IsExternal(int flags) return (flags & FlagExt) != FlagExt; } - /// - /// Check whether this memory chunk is pooled. - /// - /// Memory pointer. - /// True if pooled. - public static bool IsPooled(long memPtr) - { - return IsPooled(GetFlags(memPtr)); - } - /// /// Check whether flags denote pooled memory chunk. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs index 8e54261e0b21a..f252ef31b380d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs @@ -52,6 +52,7 @@ public PlatformMemoryStream GetStream() } /** */ + [ExcludeFromCodeCoverage] public long Pointer { get { throw new NotSupportedException(); } @@ -73,10 +74,13 @@ public int Capacity public int Length { get { return _size; } + + [ExcludeFromCodeCoverage] set { throw new NotSupportedException(); } } /** */ + [ExcludeFromCodeCoverage] public void Reallocate(int cap) { throw new NotSupportedException(); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs index f74138902088b..fb9d890dface9 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs @@ -1032,13 +1032,5 @@ protected void ThrowIfDisposed() if (_disposed) throw new ObjectDisposedException(GetType().Name, "Object has been disposed."); } - - /// - /// Gets a value indicating whether this instance is disposed. - /// - protected bool IsDisposed - { - get { return _disposed; } - } } } From cd0275d95dc6524464a154c74b1e62559b3d8af4 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Fri, 19 Aug 2016 17:34:00 +0300 Subject: [PATCH 302/487] IGNITE-3711 .NET: Fix PlatformRawMemory coverage issue --- .../Impl/Memory/PlatformRawMemory.cs | 1 + .../Impl/Unmanaged/UnmanagedCallbacks.cs | 14 +++++--------- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs index f252ef31b380d..c3cbee29fbd4b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs @@ -24,6 +24,7 @@ namespace Apache.Ignite.Core.Impl.Memory /// Non-resizeable raw memory chunk without metadata header. /// [CLSCompliant(false)] + [ExcludeFromCodeCoverage] public class PlatformRawMemory : IPlatformMemory { /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs index fcc1652a1bdff..95a46331af46e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs @@ -1097,6 +1097,11 @@ private void OnStop(void* target) private void Error(void* target, int errType, sbyte* errClsChars, int errClsCharsLen, sbyte* errMsgChars, int errMsgCharsLen, sbyte* stackTraceChars, int stackTraceCharsLen, void* errData, int errDataLen) { + // errData mechanism is only needed for CachePartialUpdateException and is no longer used, + // since CacheImpl handles all errors itself. + Debug.Assert(errDataLen == 0); + Debug.Assert(errData == null); + string errCls = IgniteUtils.Utf8UnmanagedToString(errClsChars, errClsCharsLen); string errMsg = IgniteUtils.Utf8UnmanagedToString(errMsgChars, errMsgCharsLen); string stackTrace = IgniteUtils.Utf8UnmanagedToString(stackTraceChars, stackTraceCharsLen); @@ -1104,15 +1109,6 @@ private void OnStop(void* target) switch (errType) { case ErrGeneric: - if (_ignite != null && errDataLen > 0) - { - // Stream disposal intentionally omitted: IGNITE-1598 - var stream = new PlatformRawMemory(errData, errDataLen).GetStream(); - - throw ExceptionUtils.GetException(_ignite, errCls, errMsg, stackTrace, - _ignite.Marshaller.StartUnmarshal(stream)); - } - throw ExceptionUtils.GetException(_ignite, errCls, errMsg, stackTrace); case ErrJvmInit: From d7dee5248332c6c6276c9e21aed4afb3abdd4340 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Fri, 29 Jul 2016 15:40:42 +0300 Subject: [PATCH 303/487] IGNITE-3604 .NET: Fix inconsistent namespace imports in examples Conflicts: modules/platforms/dotnet/Apache.Ignite.sln.DotSettings --- .../dotnet/Apache.Ignite.sln.DotSettings | 4 +++ .../Compute/ClosureExample.cs | 12 ++++---- .../Compute/TaskExample.cs | 14 ++++----- .../Datagrid/ContinuousQueryExample.cs | 16 +++++----- .../Datagrid/DataStreamerExample.cs | 12 ++++---- .../Datagrid/LinqExample.cs | 29 +++++++++---------- .../Datagrid/PutGetExample.cs | 12 ++++---- .../Datagrid/QueryExample.cs | 18 ++++++------ .../Datagrid/StoreExample.cs | 20 ++++++------- .../Datagrid/TransactionExample.cs | 14 ++++----- .../Events/EventsExample.cs | 16 +++++----- .../Messaging/MessagingExample.cs | 10 +++---- .../Misc/LifecycleExample.cs | 13 ++++----- .../Services/IMapService.cs | 4 +-- .../Services/ServicesExample.cs | 10 +++---- .../Binary/Account.cs | 6 ++-- .../Binary/Address.cs | 2 +- .../Binary/Employee.cs | 8 ++--- .../Binary/EmployeeKey.cs | 4 +-- .../Binary/Organization.cs | 2 +- .../Compute/AverageSalaryJob.cs | 11 ++++--- .../Compute/AverageSalaryTask.cs | 9 +++--- .../Compute/CharacterCountClosure.cs | 6 ++-- .../Compute/CharacterCountReducer.cs | 4 +-- .../Datagrid/ContinuousQueryFilter.cs | 6 ++-- .../Datagrid/EmployeeStore.cs | 13 ++++----- .../Datagrid/EmployeeStorePredicate.cs | 5 ++-- .../Events/LocalListener.cs | 8 ++--- .../Messaging/LocalListener.cs | 8 ++--- .../Messaging/RemoteOrderedListener.cs | 10 +++---- .../Messaging/RemoteUnorderedListener.cs | 10 +++---- .../Services/MapService.cs | 12 ++++---- 32 files changed, 163 insertions(+), 165 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings b/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings index 72ce015733d53..078e9fbdb9dbb 100644 --- a/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings +++ b/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings @@ -1,5 +1,9 @@  CSharp50 + <?xml version="1.0" encoding="utf-16"?><Profile name="silent"><CSReorderTypeMembers>True</CSReorderTypeMembers><CSCodeStyleAttributes ArrangeTypeAccessModifier="True" ArrangeTypeMemberAccessModifier="True" SortModifiers="True" RemoveRedundantParentheses="False" AddMissingParentheses="False" ArrangeBraces="False" ArrangeAttributes="False" ArrangeArgumentsStyle="False" /><RemoveCodeRedundancies>True</RemoveCodeRedundancies><CSArrangeQualifiers>True</CSArrangeQualifiers><CSOptimizeUsings><OptimizeUsings>True</OptimizeUsings><EmbraceInRegion>False</EmbraceInRegion><RegionName></RegionName></CSOptimizeUsings><CSShortenReferences>True</CSShortenReferences><CSReformatCode>True</CSReformatCode></Profile> + silent + DoNotTouch + DoNotTouch True True DO_NOT_SHOW diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/ClosureExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/ClosureExample.cs index 34e32b91d57c1..0b5ac464b018f 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/ClosureExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/ClosureExample.cs @@ -15,14 +15,14 @@ * limitations under the License. */ -using System; -using System.Collections.Generic; -using System.Linq; -using Apache.Ignite.Core; -using Apache.Ignite.ExamplesDll.Compute; - namespace Apache.Ignite.Examples.Compute { + using System; + using System.Collections.Generic; + using System.Linq; + using Apache.Ignite.Core; + using Apache.Ignite.ExamplesDll.Compute; + /// /// Example demonstrating closure execution. /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/TaskExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/TaskExample.cs index a7873cb3c2617..71e76a9191c93 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/TaskExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/TaskExample.cs @@ -15,14 +15,14 @@ * limitations under the License. */ -using System; -using System.Collections.Generic; -using Apache.Ignite.Core; -using Apache.Ignite.ExamplesDll.Compute; -using Apache.Ignite.ExamplesDll.Binary; - namespace Apache.Ignite.Examples.Compute { + using System; + using System.Collections.Generic; + using Apache.Ignite.Core; + using Apache.Ignite.ExamplesDll.Binary; + using Apache.Ignite.ExamplesDll.Compute; + /// /// Example demonstrating task execution. /// @@ -78,7 +78,7 @@ public static void Main() /// Collection of employees. private static ICollection Employees() { - return new [] + return new[] { new Employee( "James Wilson", diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/ContinuousQueryExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/ContinuousQueryExample.cs index e3fc5834ed148..d2e27269f2f3a 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/ContinuousQueryExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/ContinuousQueryExample.cs @@ -15,16 +15,16 @@ * limitations under the License. */ -using System; -using System.Collections.Generic; -using System.Threading; -using Apache.Ignite.Core; -using Apache.Ignite.Core.Cache.Event; -using Apache.Ignite.Core.Cache.Query.Continuous; -using Apache.Ignite.ExamplesDll.Datagrid; - namespace Apache.Ignite.Examples.Datagrid { + using System; + using System.Collections.Generic; + using System.Threading; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Cache.Event; + using Apache.Ignite.Core.Cache.Query.Continuous; + using Apache.Ignite.ExamplesDll.Datagrid; + /// /// This example demonstrates continuous query API. /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/DataStreamerExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/DataStreamerExample.cs index 88a052992817b..c4bf3420e52b6 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/DataStreamerExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/DataStreamerExample.cs @@ -15,14 +15,14 @@ * limitations under the License. */ -using System; -using System.Diagnostics; -using Apache.Ignite.Core; -using Apache.Ignite.Core.Datastream; -using Apache.Ignite.ExamplesDll.Binary; - namespace Apache.Ignite.Examples.Datagrid { + using System; + using System.Diagnostics; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Datastream; + using Apache.Ignite.ExamplesDll.Binary; + /// /// Demonstrates how cache can be populated with data utilizing . /// Data streamer is a lot more efficient to use than standard cache put operation diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs index b4a8493c01b7c..22236003be195 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs @@ -15,19 +15,18 @@ * limitations under the License. */ -using System; -using System.Linq; -using System.Collections.Generic; - -using Apache.Ignite.Core; -using Apache.Ignite.Linq; -using Apache.Ignite.Core.Cache; -using Apache.Ignite.Core.Cache.Configuration; -using Apache.Ignite.Core.Cache.Query; -using Apache.Ignite.ExamplesDll.Binary; - namespace Apache.Ignite.Examples.Datagrid { + using System; + using System.Collections.Generic; + using System.Linq; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Cache.Query; + using Apache.Ignite.ExamplesDll.Binary; + using Apache.Ignite.Linq; + /// /// This example populates cache with sample data and runs several LINQ queries over this data. /// @@ -103,7 +102,7 @@ private static void QueryExample(ICache cache) { const int zip = 94109; - IQueryable> qry = + IQueryable> qry = cache.AsCacheQueryable().Where(emp => emp.Value.Address.Zip == zip); Console.WriteLine(); @@ -122,7 +121,7 @@ private static void CompiledQueryExample(ICache cache) const int zip = 94109; // Compile cache query to eliminate LINQ overhead on multiple runs. - Func>> qry = + Func>> qry = CompiledQuery.Compile((int z) => cache.AsCacheQueryable().Where(emp => emp.Value.Address.Zip == z)); Console.WriteLine(); @@ -137,7 +136,7 @@ private static void CompiledQueryExample(ICache cache) /// /// Employee cache. /// Organization cache. - private static void JoinQueryExample(ICache employeeCache, + private static void JoinQueryExample(ICache employeeCache, ICache organizationCache) { const string orgName = "Apache"; @@ -145,7 +144,7 @@ private static void CompiledQueryExample(ICache cache) IQueryable> employees = employeeCache.AsCacheQueryable(); IQueryable> organizations = organizationCache.AsCacheQueryable(); - IQueryable> qry = + IQueryable> qry = from employee in employees from organization in organizations where employee.Key.OrganizationId == organization.Key && organization.Value.Name == orgName diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/PutGetExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/PutGetExample.cs index ebf77ec5081dd..10a75fae8a9db 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/PutGetExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/PutGetExample.cs @@ -15,14 +15,14 @@ * limitations under the License. */ -using System; -using System.Collections.Generic; -using Apache.Ignite.Core; -using Apache.Ignite.Core.Binary; -using Apache.Ignite.ExamplesDll.Binary; - namespace Apache.Ignite.Examples.Datagrid { + using System; + using System.Collections.Generic; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Binary; + using Apache.Ignite.ExamplesDll.Binary; + /// /// This example demonstrates several put-get operations on Ignite cache /// with binary values. Note that binary object can be retrieved in diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs index 7880eac5fa167..ccd6fd97905e5 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs @@ -15,17 +15,17 @@ * limitations under the License. */ -using System; -using System.Collections; -using System.Collections.Generic; -using Apache.Ignite.Core; -using Apache.Ignite.Core.Cache.Configuration; -using Apache.Ignite.Core.Cache; -using Apache.Ignite.Core.Cache.Query; -using Apache.Ignite.ExamplesDll.Binary; - namespace Apache.Ignite.Examples.Datagrid { + using System; + using System.Collections; + using System.Collections.Generic; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Cache.Query; + using Apache.Ignite.ExamplesDll.Binary; + /// /// This example populates cache with sample data and runs several SQL and /// full text queries over this data. diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/StoreExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/StoreExample.cs index 25721b85cc7a1..62da647e303a4 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/StoreExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/StoreExample.cs @@ -15,15 +15,15 @@ * limitations under the License. */ -using System; -using System.Collections.Generic; -using Apache.Ignite.Core; -using Apache.Ignite.Core.Cache.Configuration; -using Apache.Ignite.ExamplesDll.Binary; -using Apache.Ignite.ExamplesDll.Datagrid; - namespace Apache.Ignite.Examples.Datagrid { + using System; + using System.Collections.Generic; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.ExamplesDll.Binary; + using Apache.Ignite.ExamplesDll.Datagrid; + /// /// Example demonstrating cache store. /// @@ -38,7 +38,7 @@ namespace Apache.Ignite.Examples.Datagrid /// Apache.Ignite.exe -configFileName=platforms\dotnet\examples\apache.ignite.examples\app.config -assembly=[path_to_Apache.Ignite.ExamplesDll.dll] /// 2) Start example. /// - class StoreExample + public class StoreExample { /// Cache name. private const string CacheName = "dotnet_cache_with_store"; @@ -75,7 +75,7 @@ public static void Main() Console.WriteLine(); Console.WriteLine(">>> Loaded entry from store through ICache.LoadCache()."); Console.WriteLine(">>> Current cache size: " + cache.GetSize()); - + // Load entry from store calling ICache.Get() method. Employee emp = cache.Get(2); @@ -97,7 +97,7 @@ public static void Main() // Clear values again. cache.Clear(); - + Console.WriteLine(); Console.WriteLine(">>> Cleared values from cache again."); Console.WriteLine(">>> Current cache size: " + cache.GetSize()); diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/TransactionExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/TransactionExample.cs index b0e953c752d84..f90cf9642ed2b 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/TransactionExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/TransactionExample.cs @@ -15,14 +15,14 @@ * limitations under the License. */ -using System; -using Apache.Ignite.Core; -using Apache.Ignite.Core.Cache.Configuration; -using Apache.Ignite.ExamplesDll.Binary; -using Apache.Ignite.Core.Transactions; - namespace Apache.Ignite.Examples.Datagrid { + using System; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Transactions; + using Apache.Ignite.ExamplesDll.Binary; + /// /// This example demonstrates how to use transactions on Apache cache. /// @@ -37,7 +37,7 @@ namespace Apache.Ignite.Examples.Datagrid /// Apache.Ignite.exe -configFileName=platforms\dotnet\examples\apache.ignite.examples\app.config -assembly=[path_to_Apache.Ignite.ExamplesDll.dll] /// 2) Start example. /// - class TransactionExample + public class TransactionExample { /// Cache name. private const string CacheName = "dotnet_cache_tx"; diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Events/EventsExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Events/EventsExample.cs index 3b804f662227d..938b132641305 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Events/EventsExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Events/EventsExample.cs @@ -15,16 +15,16 @@ * limitations under the License. */ -using System; -using System.Linq; -using Apache.Ignite.Core; -using Apache.Ignite.Core.Events; -using Apache.Ignite.ExamplesDll.Compute; -using Apache.Ignite.ExamplesDll.Events; -using Apache.Ignite.ExamplesDll.Binary; - namespace Apache.Ignite.Examples.Events { + using System; + using System.Linq; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Events; + using Apache.Ignite.ExamplesDll.Binary; + using Apache.Ignite.ExamplesDll.Compute; + using Apache.Ignite.ExamplesDll.Events; + /// /// Example demonstrating Ignite events. /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Messaging/MessagingExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Messaging/MessagingExample.cs index be267dfa2669f..6728e38f69a8d 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Messaging/MessagingExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Messaging/MessagingExample.cs @@ -15,13 +15,13 @@ * limitations under the License. */ -using System; -using System.Threading; -using Apache.Ignite.Core; -using Apache.Ignite.ExamplesDll.Messaging; - namespace Apache.Ignite.Examples.Messaging { + using System; + using System.Threading; + using Apache.Ignite.Core; + using Apache.Ignite.ExamplesDll.Messaging; + /// /// Example demonstrating Ignite messaging. Should be run with standalone Apache Ignite.NET node. /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Misc/LifecycleExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Misc/LifecycleExample.cs index b7acb61fd56e0..4831b2cd8d729 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Misc/LifecycleExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Misc/LifecycleExample.cs @@ -15,16 +15,15 @@ * limitations under the License. */ -using System; -using System.Collections.Generic; -using Apache.Ignite.Core; -using Apache.Ignite.Core.Lifecycle; -using Apache.Ignite.Core.Resource; - namespace Apache.Ignite.Examples.Misc { + using System; + using System.Collections.Generic; + using Apache.Ignite.Core; using Apache.Ignite.Core.Discovery.Tcp; using Apache.Ignite.Core.Discovery.Tcp.Static; + using Apache.Ignite.Core.Lifecycle; + using Apache.Ignite.Core.Resource; /// /// This example shows how to provide your own implementation @@ -101,7 +100,7 @@ public void OnLifecycleEvent(LifecycleEventType evt) if (evt == LifecycleEventType.AfterNodeStart) Started = true; else if (evt == LifecycleEventType.AfterNodeStop) - Started = false; + Started = false; } /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/IMapService.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/IMapService.cs index 7253a0bb71344..4e2b883a97ac0 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/IMapService.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/IMapService.cs @@ -15,10 +15,10 @@ * limitations under the License. */ -using Apache.Ignite.ExamplesDll.Services; - namespace Apache.Ignite.Examples.Services { + using Apache.Ignite.ExamplesDll.Services; + /// /// Interface for service proxy interaction. /// Actual service class () does not have to implement this interface. diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/ServicesExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/ServicesExample.cs index fcfce14c88ad3..245356d16e54b 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/ServicesExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/ServicesExample.cs @@ -15,12 +15,12 @@ * limitations under the License. */ -using System; -using Apache.Ignite.Core; -using Apache.Ignite.ExamplesDll.Services; - namespace Apache.Ignite.Examples.Services { + using System; + using Apache.Ignite.Core; + using Apache.Ignite.ExamplesDll.Services; + /// /// Example demonstrating Ignite services. /// @@ -55,7 +55,7 @@ public static void Main() // Get a sticky service proxy so that we will always be contacting the same remote node. var prx = ignite.GetServices().GetServiceProxy>("service", true); - + for (var i = 0; i < 10; i++) prx.Put(i, i.ToString()); diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Account.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Account.cs index 4d3fc35577e3a..8d659e18b6788 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Account.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Account.cs @@ -34,12 +34,12 @@ public Account(int id, decimal balance) Id = id; Balance = balance; } - + /// /// Account ID. /// public int Id { get; set; } - + /// /// Account balance. /// @@ -51,7 +51,7 @@ public Account(int id, decimal balance) /// /// A string that represents the current object. /// - override public String ToString() + public override String ToString() { return string.Format("{0} [id={1}, balance={2}]", typeof(Account).Name, Id, Balance); } diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Address.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Address.cs index bb6ed7f96217f..7053e780324c8 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Address.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Address.cs @@ -76,7 +76,7 @@ public void ReadBinary(IBinaryReader reader) /// /// A string that represents the current object. /// - override public string ToString() + public override string ToString() { return string.Format("{0} [street={1}, zip={2}]", typeof(Address).Name, Street, Zip); } diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs index 44a3f116ff691..0fc3230af03c9 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs @@ -72,9 +72,9 @@ public Employee(string name, long salary, Address address, ICollection d /// /// A string that represents the current object. /// - override public string ToString() + public override string ToString() { - return string.Format("{0} [name={1}, salary={2}, address={3}, departments={4}]", typeof(Employee).Name, + return string.Format("{0} [name={1}, salary={2}, address={3}, departments={4}]", typeof(Employee).Name, Name, Salary, Address, CollectionToString(Departments)); } @@ -87,8 +87,8 @@ private static string CollectionToString(ICollection col) if (col == null) return "null"; - var elements = col.Any() - ? col.Select(x => x.ToString()).Aggregate((x, y) => x + ", " + y) + var elements = col.Any() + ? col.Select(x => x.ToString()).Aggregate((x, y) => x + ", " + y) : string.Empty; return string.Format("[{0}]", elements); diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs index 9c1d8cc81b58a..e69964838f29f 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs @@ -47,7 +47,7 @@ public EmployeeKey(int id, int orgId) [AffinityKeyMapped] [QuerySqlField(IsIndexed = true)] public int OrganizationId { get; private set; } - + /// /// Determines whether the specified is equal to the current . /// @@ -82,7 +82,7 @@ public override int GetHashCode() /// public override string ToString() { - return string.Format("{0} [id={1}, organizationId={2}]", typeof (EmployeeKey).Name, Id, OrganizationId); + return string.Format("{0} [id={1}, organizationId={2}]", typeof(EmployeeKey).Name, Id, OrganizationId); } } } diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Organization.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Organization.cs index 16d4113309c3b..7495ee91ad731 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Organization.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Organization.cs @@ -70,7 +70,7 @@ public Organization(string name, Address address, OrganizationType type, DateTim /// 2 public override string ToString() { - return string.Format("{0} [name={1}, address={2}, type={3}, lastUpdated={4}]", typeof (Organization).Name, + return string.Format("{0} [name={1}, address={2}, type={3}, lastUpdated={4}]", typeof(Organization).Name, Name, Address, Type, LastUpdated); } } diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs index e05a4360282ae..2685df09bba7d 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs @@ -15,12 +15,11 @@ * limitations under the License. */ -using System; -using System.Collections.Generic; -using Apache.Ignite.Core.Compute; - namespace Apache.Ignite.ExamplesDll.Compute { + using System; + using System.Collections.Generic; + using Apache.Ignite.Core.Compute; using Apache.Ignite.ExamplesDll.Binary; /// @@ -33,7 +32,7 @@ public class AverageSalaryJob : ComputeJobAdapter> private readonly ICollection _employees = new List(); /// - /// Adds employee. + /// Adds employee. /// /// Employee. public void Add(Employee employee) @@ -45,7 +44,7 @@ public void Add(Employee employee) /// Execute the job. /// /// Job result: tuple with total salary in the first item and employees count in the second. - override public Tuple Execute() + public override Tuple Execute() { long sum = 0; int count = 0; diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryTask.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryTask.cs index 3af166fac9747..6ef9798119b3f 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryTask.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryTask.cs @@ -15,13 +15,12 @@ * limitations under the License. */ -using System; -using System.Collections.Generic; -using System.Linq; -using Apache.Ignite.Core.Compute; - namespace Apache.Ignite.ExamplesDll.Compute { + using System; + using System.Collections.Generic; + using System.Linq; + using Apache.Ignite.Core.Compute; using Apache.Ignite.ExamplesDll.Binary; /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountClosure.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountClosure.cs index 28232213aae77..a34c3fab48c60 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountClosure.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountClosure.cs @@ -15,11 +15,11 @@ * limitations under the License. */ -using System; -using Apache.Ignite.Core.Compute; - namespace Apache.Ignite.ExamplesDll.Compute { + using System; + using Apache.Ignite.Core.Compute; + /// /// Closure counting characters in a string. /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountReducer.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountReducer.cs index 68250469fb58e..9ebf7eb5a16ea 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountReducer.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountReducer.cs @@ -15,10 +15,10 @@ * limitations under the License. */ -using Apache.Ignite.Core.Compute; - namespace Apache.Ignite.ExamplesDll.Compute { + using Apache.Ignite.Core.Compute; + /// /// Character count reducer which collects individual string lengths and aggregate them. /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ContinuousQueryFilter.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ContinuousQueryFilter.cs index 8c05f425970f0..f569f3e68a92a 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ContinuousQueryFilter.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ContinuousQueryFilter.cs @@ -15,11 +15,11 @@ * limitations under the License. */ -using System; -using Apache.Ignite.Core.Cache.Event; - namespace Apache.Ignite.ExamplesDll.Datagrid { + using System; + using Apache.Ignite.Core.Cache.Event; + /// /// Filter for continuous query example. /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStore.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStore.cs index 561d83f30c905..7049011e864d0 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStore.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStore.cs @@ -15,15 +15,14 @@ * limitations under the License. */ -using System; -using System.Collections; -using System.Collections.Concurrent; -using System.Collections.Generic; -using Apache.Ignite.Core.Cache; -using Apache.Ignite.Core.Cache.Store; - namespace Apache.Ignite.ExamplesDll.Datagrid { + using System; + using System.Collections; + using System.Collections.Concurrent; + using System.Collections.Generic; + using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Store; using Apache.Ignite.ExamplesDll.Binary; /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStorePredicate.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStorePredicate.cs index c25b2fa76fd11..5c3bf892e91b4 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStorePredicate.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStorePredicate.cs @@ -15,11 +15,10 @@ * limitations under the License. */ -using System; -using Apache.Ignite.Core.Cache; - namespace Apache.Ignite.ExamplesDll.Datagrid { + using System; + using Apache.Ignite.Core.Cache; using Apache.Ignite.ExamplesDll.Binary; /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs index 46524a1698ca9..77af7a953cdc4 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs @@ -15,12 +15,12 @@ * limitations under the License. */ -using System; -using System.Threading; -using Apache.Ignite.Core.Events; - namespace Apache.Ignite.ExamplesDll.Events { + using System; + using System.Threading; + using Apache.Ignite.Core.Events; + /// /// Local event listener. /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/LocalListener.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/LocalListener.cs index 591d426cdf8cb..e8b2f4bb957af 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/LocalListener.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/LocalListener.cs @@ -15,12 +15,12 @@ * limitations under the License. */ -using System; -using System.Threading; -using Apache.Ignite.Core.Messaging; - namespace Apache.Ignite.ExamplesDll.Messaging { + using System; + using System.Threading; + using Apache.Ignite.Core.Messaging; + /// /// Local message listener which signals countdown event on each received message. /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteOrderedListener.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteOrderedListener.cs index 85538c293b9f8..edf38f2436b86 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteOrderedListener.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteOrderedListener.cs @@ -15,13 +15,13 @@ * limitations under the License. */ -using System; -using Apache.Ignite.Core; -using Apache.Ignite.Core.Messaging; -using Apache.Ignite.Core.Resource; - namespace Apache.Ignite.ExamplesDll.Messaging { + using System; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Messaging; + using Apache.Ignite.Core.Resource; + /// /// Listener for Ordered topic. /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteUnorderedListener.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteUnorderedListener.cs index ab23e8b23c7a8..8054d36a58b22 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteUnorderedListener.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteUnorderedListener.cs @@ -15,13 +15,13 @@ * limitations under the License. */ -using System; -using Apache.Ignite.Core; -using Apache.Ignite.Core.Messaging; -using Apache.Ignite.Core.Resource; - namespace Apache.Ignite.ExamplesDll.Messaging { + using System; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Messaging; + using Apache.Ignite.Core.Resource; + /// /// Listener for Unordered topic. /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs index d577ff75aec45..958d91dbf7070 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs @@ -15,14 +15,14 @@ * limitations under the License. */ -using System; -using Apache.Ignite.Core; -using Apache.Ignite.Core.Cache; -using Apache.Ignite.Core.Resource; -using Apache.Ignite.Core.Services; - namespace Apache.Ignite.ExamplesDll.Services { + using System; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Resource; + using Apache.Ignite.Core.Services; + /// /// Service implementation. /// From 390c8d59cf8d44d9703e5fbf8d9c0f8028a4a172 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Thu, 11 Aug 2016 16:18:14 +0300 Subject: [PATCH 304/487] IGNITE-3675 .NET: Use separate caches for different entities in QueryExample. --- .../Datagrid/LinqExample.cs | 38 +++++++++++------- .../Datagrid/QueryExample.cs | 39 ++++++++++++------- 2 files changed, 51 insertions(+), 26 deletions(-) diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs index 22236003be195..848d8f510c67d 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs @@ -43,8 +43,11 @@ namespace Apache.Ignite.Examples.Datagrid /// public class LinqExample { - /// Cache name. - private const string CacheName = "dotnet_cache_query"; + /// Organization cache name. + private const string OrganizationCacheName = "dotnet_cache_query_organization"; + + /// Employee cache name. + private const string EmployeeCacheName = "dotnet_cache_query_employee"; [STAThread] public static void Main() @@ -54,25 +57,27 @@ public static void Main() Console.WriteLine(); Console.WriteLine(">>> Cache LINQ example started."); - var cache = ignite.GetOrCreateCache(new CacheConfiguration + var employeeCache = ignite.GetOrCreateCache(new CacheConfiguration { - Name = CacheName, + Name = EmployeeCacheName, QueryEntities = new[] { - new QueryEntity(typeof(int), typeof(Organization)), new QueryEntity(typeof(EmployeeKey), typeof(Employee)) } }); - // Clean up caches on all nodes before run. - cache.Clear(); + var organizationCache = ignite.GetOrCreateCache(new CacheConfiguration + { + Name = OrganizationCacheName, + QueryEntities = new[] + { + new QueryEntity(typeof(int), typeof(Organization)) + } + }); // Populate cache with sample data entries. - PopulateCache(cache); - - // Create cache that will work with specific types. - var employeeCache = ignite.GetCache(CacheName); - var organizationCache = ignite.GetCache(CacheName); + PopulateCache(employeeCache); + PopulateCache(organizationCache); // Run SQL query example. QueryExample(employeeCache); @@ -177,7 +182,7 @@ private static void FieldsQueryExample(ICache cache) /// Populate cache with data for this example. /// /// Cache. - private static void PopulateCache(ICache cache) + private static void PopulateCache(ICache cache) { cache.Put(1, new Organization( "Apache", @@ -192,7 +197,14 @@ private static void PopulateCache(ICache cache) OrganizationType.Private, DateTime.Now )); + } + /// + /// Populate cache with data for this example. + /// + /// Cache. + private static void PopulateCache(ICache cache) + { cache.Put(new EmployeeKey(1, 1), new Employee( "James Wilson", 12500, diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs index ccd6fd97905e5..8b5e6f37e83c8 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs @@ -43,8 +43,11 @@ namespace Apache.Ignite.Examples.Datagrid /// public class QueryExample { - /// Cache name. - private const string CacheName = "dotnet_cache_query"; + /// Organization cache name. + private const string OrganizationCacheName = "dotnet_cache_query_organization"; + + /// Employee cache name. + private const string EmployeeCacheName = "dotnet_cache_query_employee"; [STAThread] public static void Main() @@ -54,24 +57,27 @@ public static void Main() Console.WriteLine(); Console.WriteLine(">>> Cache query example started."); - var cache = ignite.GetOrCreateCache(new CacheConfiguration + var employeeCache = ignite.GetOrCreateCache(new CacheConfiguration { - Name = CacheName, + Name = EmployeeCacheName, QueryEntities = new[] { - new QueryEntity(typeof(int), typeof(Organization)), new QueryEntity(typeof(EmployeeKey), typeof(Employee)) } }); - // Clean up caches on all nodes before run. - cache.Clear(); + var organizationCache = ignite.GetOrCreateCache(new CacheConfiguration + { + Name = OrganizationCacheName, + QueryEntities = new[] + { + new QueryEntity(typeof(int), typeof(Organization)) + } + }); // Populate cache with sample data entries. - PopulateCache(cache); - - // Create cache that will work with specific types. - var employeeCache = ignite.GetCache(CacheName); + PopulateCache(employeeCache); + PopulateCache(organizationCache); // Run SQL query example. SqlQueryExample(employeeCache); @@ -119,7 +125,7 @@ private static void SqlJoinQueryExample(ICache cache) const string orgName = "Apache"; var qry = cache.Query(new SqlQuery("Employee", - "from Employee, Organization " + + "from Employee, \"dotnet_cache_query_organization\".Organization " + "where Employee.organizationId = Organization._key and Organization.name = ?", orgName)); Console.WriteLine(); @@ -163,7 +169,7 @@ private static void FullTextQueryExample(ICache cache) /// Populate cache with data for this example. /// /// Cache. - private static void PopulateCache(ICache cache) + private static void PopulateCache(ICache cache) { cache.Put(1, new Organization( "Apache", @@ -178,7 +184,14 @@ private static void PopulateCache(ICache cache) OrganizationType.Private, DateTime.Now )); + } + /// + /// Populate cache with data for this example. + /// + /// Cache. + private static void PopulateCache(ICache cache) + { cache.Put(new EmployeeKey(1, 1), new Employee( "James Wilson", 12500, From 861c1736facdf76c71e93ddada0d8e15b5b2950c Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Thu, 11 Aug 2016 17:32:13 +0300 Subject: [PATCH 305/487] IGNITE-3673 .NET: Add examples for distributed joins. --- .../Datagrid/LinqExample.cs | 195 +++++++++++++----- .../Datagrid/QueryExample.cs | 174 +++++++++++----- .../Apache.Ignite.ExamplesDll.csproj | 1 - .../Binary/Employee.cs | 10 +- .../Binary/EmployeeKey.cs | 88 -------- 5 files changed, 271 insertions(+), 197 deletions(-) delete mode 100644 modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs index 848d8f510c67d..86739b4dd34a1 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs @@ -18,10 +18,10 @@ namespace Apache.Ignite.Examples.Datagrid { using System; - using System.Collections.Generic; using System.Linq; using Apache.Ignite.Core; using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Affinity; using Apache.Ignite.Core.Cache.Configuration; using Apache.Ignite.Core.Cache.Query; using Apache.Ignite.ExamplesDll.Binary; @@ -49,6 +49,9 @@ public class LinqExample /// Employee cache name. private const string EmployeeCacheName = "dotnet_cache_query_employee"; + /// Colocated employee cache name. + private const string EmployeeCacheNameColocated = "dotnet_cache_query_employee_colocated"; + [STAThread] public static void Main() { @@ -57,26 +60,18 @@ public static void Main() Console.WriteLine(); Console.WriteLine(">>> Cache LINQ example started."); - var employeeCache = ignite.GetOrCreateCache(new CacheConfiguration - { - Name = EmployeeCacheName, - QueryEntities = new[] - { - new QueryEntity(typeof(EmployeeKey), typeof(Employee)) - } - }); - - var organizationCache = ignite.GetOrCreateCache(new CacheConfiguration - { - Name = OrganizationCacheName, - QueryEntities = new[] - { - new QueryEntity(typeof(int), typeof(Organization)) - } - }); + var employeeCache = ignite.GetOrCreateCache( + new CacheConfiguration(EmployeeCacheName, typeof(Employee))); + + var employeeCacheColocated = ignite.GetOrCreateCache( + new CacheConfiguration(EmployeeCacheNameColocated, typeof(Employee))); + + var organizationCache = ignite.GetOrCreateCache( + new CacheConfiguration(OrganizationCacheName, new QueryEntity(typeof(int), typeof(Organization)))); // Populate cache with sample data entries. PopulateCache(employeeCache); + PopulateCache(employeeCacheColocated); PopulateCache(organizationCache); // Run SQL query example. @@ -86,7 +81,10 @@ public static void Main() CompiledQueryExample(employeeCache); // Run SQL query with join example. - JoinQueryExample(employeeCache, organizationCache); + JoinQueryExample(employeeCacheColocated, organizationCache); + + // Run SQL query with distributed join example. + DistributedJoinQueryExample(employeeCache, organizationCache); // Run SQL fields query example. FieldsQueryExample(employeeCache); @@ -103,17 +101,17 @@ public static void Main() /// Queries employees that have provided ZIP code in address. /// /// Cache. - private static void QueryExample(ICache cache) + private static void QueryExample(ICache cache) { const int zip = 94109; - IQueryable> qry = + IQueryable> qry = cache.AsCacheQueryable().Where(emp => emp.Value.Address.Zip == zip); Console.WriteLine(); Console.WriteLine(">>> Employees with zipcode " + zip + ":"); - foreach (ICacheEntry entry in qry) + foreach (ICacheEntry entry in qry) Console.WriteLine(">>> " + entry.Value); } @@ -121,18 +119,18 @@ private static void QueryExample(ICache cache) /// Queries employees that have provided ZIP code in address with a compiled query. /// /// Cache. - private static void CompiledQueryExample(ICache cache) + private static void CompiledQueryExample(ICache cache) { const int zip = 94109; // Compile cache query to eliminate LINQ overhead on multiple runs. - Func>> qry = + Func>> qry = CompiledQuery.Compile((int z) => cache.AsCacheQueryable().Where(emp => emp.Value.Address.Zip == z)); Console.WriteLine(); Console.WriteLine(">>> Employees with zipcode using compiled query " + zip + ":"); - foreach (ICacheEntry entry in qry(zip)) + foreach (ICacheEntry entry in qry(zip)) Console.WriteLine(">>> " + entry.Value); } @@ -141,25 +139,54 @@ private static void CompiledQueryExample(ICache cache) /// /// Employee cache. /// Organization cache. - private static void JoinQueryExample(ICache employeeCache, + private static void JoinQueryExample(ICache employeeCache, ICache organizationCache) { const string orgName = "Apache"; - IQueryable> employees = employeeCache.AsCacheQueryable(); + IQueryable> employees = employeeCache.AsCacheQueryable(); IQueryable> organizations = organizationCache.AsCacheQueryable(); - IQueryable> qry = + IQueryable> qry = + from employee in employees + from organization in organizations + where employee.Value.OrganizationId == organization.Key && organization.Value.Name == orgName + select employee; + + + Console.WriteLine(); + Console.WriteLine(">>> Employees working for " + orgName + ":"); + + foreach (ICacheEntry entry in qry) + Console.WriteLine(">>> " + entry.Value); + } + + /// + /// Queries employees that work for organization with provided name. + /// + /// Employee cache. + /// Organization cache. + private static void DistributedJoinQueryExample(ICache employeeCache, + ICache organizationCache) + { + const string orgName = "Apache"; + + var queryOptions = new QueryOptions {EnableDistributedJoins = true}; + + IQueryable> employees = employeeCache.AsCacheQueryable(queryOptions); + IQueryable> organizations = organizationCache.AsCacheQueryable(queryOptions); + + IQueryable> qry = from employee in employees from organization in organizations - where employee.Key.OrganizationId == organization.Key && organization.Value.Name == orgName + where employee.Value.OrganizationId == organization.Key && organization.Value.Name == orgName select employee; Console.WriteLine(); Console.WriteLine(">>> Employees working for " + orgName + ":"); - foreach (ICacheEntry entry in qry) + foreach (ICacheEntry entry in qry) Console.WriteLine(">>> " + entry.Value); } @@ -167,7 +194,7 @@ private static void CompiledQueryExample(ICache cache) /// Queries names and salaries for all employees. /// /// Cache. - private static void FieldsQueryExample(ICache cache) + private static void FieldsQueryExample(ICache cache) { var qry = cache.AsCacheQueryable().Select(entry => new {entry.Value.Name, entry.Value.Salary}); @@ -188,71 +215,125 @@ private static void PopulateCache(ICache cache) "Apache", new Address("1065 East Hillsdale Blvd, Foster City, CA", 94404), OrganizationType.Private, - DateTime.Now - )); + DateTime.Now)); cache.Put(2, new Organization( "Microsoft", new Address("1096 Eddy Street, San Francisco, CA", 94109), OrganizationType.Private, - DateTime.Now - )); + DateTime.Now)); + } + + /// + /// Populate cache with data for this example. + /// + /// Cache. + private static void PopulateCache(ICache cache) + { + cache.Put(new AffinityKey(1, 1), new Employee( + "James Wilson", + 12500, + new Address("1096 Eddy Street, San Francisco, CA", 94109), + new[] {"Human Resources", "Customer Service"}, + 1)); + + cache.Put(new AffinityKey(2, 1), new Employee( + "Daniel Adams", + 11000, + new Address("184 Fidler Drive, San Antonio, TX", 78130), + new[] {"Development", "QA"}, + 1)); + + cache.Put(new AffinityKey(3, 1), new Employee( + "Cristian Moss", + 12500, + new Address("667 Jerry Dove Drive, Florence, SC", 29501), + new[] {"Logistics"}, + 1)); + + cache.Put(new AffinityKey(4, 2), new Employee( + "Allison Mathis", + 25300, + new Address("2702 Freedom Lane, San Francisco, CA", 94109), + new[] {"Development"}, + 2)); + + cache.Put(new AffinityKey(5, 2), new Employee( + "Breana Robbin", + 6500, + new Address("3960 Sundown Lane, Austin, TX", 78130), + new[] {"Sales"}, + 2)); + + cache.Put(new AffinityKey(6, 2), new Employee( + "Philip Horsley", + 19800, + new Address("2803 Elsie Drive, Sioux Falls, SD", 57104), + new[] {"Sales"}, + 2)); + + cache.Put(new AffinityKey(7, 2), new Employee( + "Brian Peters", + 10600, + new Address("1407 Pearlman Avenue, Boston, MA", 12110), + new[] {"Development", "QA"}, + 2)); } /// /// Populate cache with data for this example. /// /// Cache. - private static void PopulateCache(ICache cache) + private static void PopulateCache(ICache cache) { - cache.Put(new EmployeeKey(1, 1), new Employee( + cache.Put(1, new Employee( "James Wilson", 12500, new Address("1096 Eddy Street, San Francisco, CA", 94109), - new List { "Human Resources", "Customer Service" } - )); + new[] {"Human Resources", "Customer Service"}, + 1)); - cache.Put(new EmployeeKey(2, 1), new Employee( + cache.Put(2, new Employee( "Daniel Adams", 11000, new Address("184 Fidler Drive, San Antonio, TX", 78130), - new List { "Development", "QA" } - )); + new[] {"Development", "QA"}, + 1)); - cache.Put(new EmployeeKey(3, 1), new Employee( + cache.Put(3, new Employee( "Cristian Moss", 12500, new Address("667 Jerry Dove Drive, Florence, SC", 29501), - new List { "Logistics" } - )); + new[] {"Logistics"}, + 1)); - cache.Put(new EmployeeKey(4, 2), new Employee( + cache.Put(4, new Employee( "Allison Mathis", 25300, new Address("2702 Freedom Lane, San Francisco, CA", 94109), - new List { "Development" } - )); + new[] {"Development"}, + 2)); - cache.Put(new EmployeeKey(5, 2), new Employee( + cache.Put(5, new Employee( "Breana Robbin", 6500, new Address("3960 Sundown Lane, Austin, TX", 78130), - new List { "Sales" } - )); + new[] {"Sales"}, + 2)); - cache.Put(new EmployeeKey(6, 2), new Employee( + cache.Put(6, new Employee( "Philip Horsley", 19800, new Address("2803 Elsie Drive, Sioux Falls, SD", 57104), - new List { "Sales" } - )); + new[] {"Sales"}, + 2)); - cache.Put(new EmployeeKey(7, 2), new Employee( + cache.Put(7, new Employee( "Brian Peters", 10600, new Address("1407 Pearlman Avenue, Boston, MA", 12110), - new List { "Development", "QA" } - )); + new[] {"Development", "QA"}, + 2)); } } } diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs index 8b5e6f37e83c8..1c351492fa8e2 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs @@ -22,6 +22,7 @@ namespace Apache.Ignite.Examples.Datagrid using System.Collections.Generic; using Apache.Ignite.Core; using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Affinity; using Apache.Ignite.Core.Cache.Configuration; using Apache.Ignite.Core.Cache.Query; using Apache.Ignite.ExamplesDll.Binary; @@ -49,6 +50,9 @@ public class QueryExample /// Employee cache name. private const string EmployeeCacheName = "dotnet_cache_query_employee"; + /// Employee cache name. + private const string EmployeeCacheNameColocated = "dotnet_cache_query_employee_colocated"; + [STAThread] public static void Main() { @@ -57,33 +61,28 @@ public static void Main() Console.WriteLine(); Console.WriteLine(">>> Cache query example started."); - var employeeCache = ignite.GetOrCreateCache(new CacheConfiguration - { - Name = EmployeeCacheName, - QueryEntities = new[] - { - new QueryEntity(typeof(EmployeeKey), typeof(Employee)) - } - }); - - var organizationCache = ignite.GetOrCreateCache(new CacheConfiguration - { - Name = OrganizationCacheName, - QueryEntities = new[] - { - new QueryEntity(typeof(int), typeof(Organization)) - } - }); + var employeeCache = ignite.GetOrCreateCache( + new CacheConfiguration(EmployeeCacheName, typeof(Employee))); + + var employeeCacheColocated = ignite.GetOrCreateCache( + new CacheConfiguration(EmployeeCacheNameColocated, typeof(Employee))); + + var organizationCache = ignite.GetOrCreateCache( + new CacheConfiguration(OrganizationCacheName, new QueryEntity(typeof(int), typeof(Organization)))); // Populate cache with sample data entries. PopulateCache(employeeCache); + PopulateCache(employeeCacheColocated); PopulateCache(organizationCache); // Run SQL query example. SqlQueryExample(employeeCache); // Run SQL query with join example. - SqlJoinQueryExample(employeeCache); + SqlJoinQueryExample(employeeCacheColocated); + + // Run SQL query with distributed join example. + SqlDistributedJoinQueryExample(employeeCache); // Run SQL fields query example. SqlFieldsQueryExample(employeeCache); @@ -103,7 +102,7 @@ public static void Main() /// Queries employees that have provided ZIP code in address. /// /// Cache. - private static void SqlQueryExample(ICache cache) + private static void SqlQueryExample(ICache cache) { const int zip = 94109; @@ -120,7 +119,7 @@ private static void SqlQueryExample(ICache cache) /// Queries employees that work for organization with provided name. /// /// Cache. - private static void SqlJoinQueryExample(ICache cache) + private static void SqlJoinQueryExample(ICache cache) { const string orgName = "Apache"; @@ -135,11 +134,33 @@ private static void SqlJoinQueryExample(ICache cache) Console.WriteLine(">>> " + entry.Value); } + /// + /// Queries employees that work for organization with provided name. + /// + /// Cache. + private static void SqlDistributedJoinQueryExample(ICache cache) + { + const string orgName = "Apache"; + + var qry = cache.Query(new SqlQuery("Employee", + "from Employee, \"dotnet_cache_query_organization\".Organization " + + "where Employee.organizationId = Organization._key and Organization.name = ?", orgName) + { + EnableDistributedJoins = true + }); + + Console.WriteLine(); + Console.WriteLine(">>> Employees working for " + orgName + ":"); + + foreach (var entry in qry) + Console.WriteLine(">>> " + entry.Value); + } + /// /// Queries names and salaries for all employees. /// /// Cache. - private static void SqlFieldsQueryExample(ICache cache) + private static void SqlFieldsQueryExample(ICache cache) { var qry = cache.QueryFields(new SqlFieldsQuery("select name, salary from Employee")); @@ -154,7 +175,7 @@ private static void SqlFieldsQueryExample(ICache cache) /// Queries employees that live in Texas using full-text query API. /// /// Cache. - private static void FullTextQueryExample(ICache cache) + private static void FullTextQueryExample(ICache cache) { var qry = cache.Query(new TextQuery("Employee", "TX")); @@ -175,71 +196,124 @@ private static void PopulateCache(ICache cache) "Apache", new Address("1065 East Hillsdale Blvd, Foster City, CA", 94404), OrganizationType.Private, - DateTime.Now - )); + DateTime.Now)); - cache.Put(2, new Organization( - "Microsoft", + cache.Put(2, new Organization("Microsoft", new Address("1096 Eddy Street, San Francisco, CA", 94109), OrganizationType.Private, - DateTime.Now - )); + DateTime.Now)); + } + + /// + /// Populate cache with data for this example. + /// + /// Cache. + private static void PopulateCache(ICache cache) + { + cache.Put(new AffinityKey(1, 1), new Employee( + "James Wilson", + 12500, + new Address("1096 Eddy Street, San Francisco, CA", 94109), + new[] {"Human Resources", "Customer Service"}, + 1)); + + cache.Put(new AffinityKey(2, 1), new Employee( + "Daniel Adams", + 11000, + new Address("184 Fidler Drive, San Antonio, TX", 78130), + new[] {"Development", "QA"}, + 1)); + + cache.Put(new AffinityKey(3, 1), new Employee( + "Cristian Moss", + 12500, + new Address("667 Jerry Dove Drive, Florence, SC", 29501), + new[] {"Logistics"}, + 1)); + + cache.Put(new AffinityKey(4, 2), new Employee( + "Allison Mathis", + 25300, + new Address("2702 Freedom Lane, San Francisco, CA", 94109), + new[] {"Development"}, + 2)); + + cache.Put(new AffinityKey(5, 2), new Employee( + "Breana Robbin", + 6500, + new Address("3960 Sundown Lane, Austin, TX", 78130), + new[] {"Sales"}, + 2)); + + cache.Put(new AffinityKey(6, 2), new Employee( + "Philip Horsley", + 19800, + new Address("2803 Elsie Drive, Sioux Falls, SD", 57104), + new[] {"Sales"}, + 2)); + + cache.Put(new AffinityKey(7, 2), new Employee( + "Brian Peters", + 10600, + new Address("1407 Pearlman Avenue, Boston, MA", 12110), + new[] {"Development", "QA"}, + 2)); } /// /// Populate cache with data for this example. /// /// Cache. - private static void PopulateCache(ICache cache) + private static void PopulateCache(ICache cache) { - cache.Put(new EmployeeKey(1, 1), new Employee( + cache.Put(1, new Employee( "James Wilson", 12500, new Address("1096 Eddy Street, San Francisco, CA", 94109), - new List { "Human Resources", "Customer Service" } - )); + new[] {"Human Resources", "Customer Service"}, + 1)); - cache.Put(new EmployeeKey(2, 1), new Employee( + cache.Put(2, new Employee( "Daniel Adams", 11000, new Address("184 Fidler Drive, San Antonio, TX", 78130), - new List { "Development", "QA" } - )); + new[] {"Development", "QA"}, + 1)); - cache.Put(new EmployeeKey(3, 1), new Employee( + cache.Put(3, new Employee( "Cristian Moss", 12500, new Address("667 Jerry Dove Drive, Florence, SC", 29501), - new List { "Logistics" } - )); + new[] {"Logistics"}, + 1)); - cache.Put(new EmployeeKey(4, 2), new Employee( + cache.Put(4, new Employee( "Allison Mathis", 25300, new Address("2702 Freedom Lane, San Francisco, CA", 94109), - new List { "Development" } - )); + new[] {"Development"}, + 2)); - cache.Put(new EmployeeKey(5, 2), new Employee( + cache.Put(5, new Employee( "Breana Robbin", 6500, new Address("3960 Sundown Lane, Austin, TX", 78130), - new List { "Sales" } - )); + new[] {"Sales"}, + 2)); - cache.Put(new EmployeeKey(6, 2), new Employee( + cache.Put(6, new Employee( "Philip Horsley", 19800, new Address("2803 Elsie Drive, Sioux Falls, SD", 57104), - new List { "Sales" } - )); + new[] {"Sales"}, + 2)); - cache.Put(new EmployeeKey(7, 2), new Employee( + cache.Put(7, new Employee( "Brian Peters", 10600, new Address("1407 Pearlman Avenue, Boston, MA", 12110), - new List { "Development", "QA" } - )); + new[] {"Development", "QA"}, + 2)); } } } diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj index 16d0be305f4e9..41981d8c87c59 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj @@ -57,7 +57,6 @@ - diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs index 0fc3230af03c9..4cff2a883cbaf 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs @@ -35,12 +35,14 @@ public class Employee /// Salary. /// Address. /// Departments. - public Employee(string name, long salary, Address address, ICollection departments) + public Employee(string name, long salary, Address address, ICollection departments, + int organizationId = 0) { Name = name; Salary = salary; Address = address; Departments = departments; + OrganizationId = organizationId; } /// @@ -49,6 +51,12 @@ public Employee(string name, long salary, Address address, ICollection d [QuerySqlField] public string Name { get; set; } + /// + /// Organization id. + /// + [QuerySqlField(IsIndexed = true)] + public int OrganizationId { get; set; } + /// /// Salary. /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs deleted file mode 100644 index e69964838f29f..0000000000000 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs +++ /dev/null @@ -1,88 +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. - */ - -namespace Apache.Ignite.ExamplesDll.Binary -{ - using Apache.Ignite.Core.Cache.Affinity; - using Apache.Ignite.Core.Cache.Configuration; - - /// - /// Employee key. Used in query example to co-locate employees with their organizations. - /// - public class EmployeeKey - { - /// - /// Constructor. - /// - /// ID. - /// Organization ID. - public EmployeeKey(int id, int orgId) - { - Id = id; - OrganizationId = orgId; - } - - /// - /// ID. - /// - public int Id { get; private set; } - - /// - /// Organization ID. - /// - [AffinityKeyMapped] - [QuerySqlField(IsIndexed = true)] - public int OrganizationId { get; private set; } - - /// - /// Determines whether the specified is equal to the current . - /// - /// - /// true if the specified is equal to the current ; otherwise, false. - /// - /// The object to compare with the current object. 2 - public override bool Equals(object obj) - { - EmployeeKey other = obj as EmployeeKey; - - return other != null && Id == other.Id && OrganizationId == other.OrganizationId; - } - - /// - /// Serves as a hash function for a particular type. - /// - /// - /// A hash code for the current . - /// - /// 2 - public override int GetHashCode() - { - return 31 * Id + OrganizationId; - } - - /// - /// Returns a string that represents the current object. - /// - /// - /// A string that represents the current object. - /// - public override string ToString() - { - return string.Format("{0} [id={1}, organizationId={2}]", typeof(EmployeeKey).Name, Id, OrganizationId); - } - } -} From 3a9cbed4e692e76ad884e60a4f5edc670e6b3c8f Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 8 Aug 2016 17:00:04 +0300 Subject: [PATCH 306/487] IGNITE-3630 .NET: Add pure binary mode example with put-get and queries. --- .../Apache.Ignite.Examples.csproj | 1 + .../Datagrid/BinaryModeExample.cs | 272 ++++++++++++++++++ 2 files changed, 273 insertions(+) create mode 100644 modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/BinaryModeExample.cs diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj index ab0a9f21f96af..9b1d3230d4744 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj @@ -54,6 +54,7 @@ + diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/BinaryModeExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/BinaryModeExample.cs new file mode 100644 index 0000000000000..1c430b3aa9628 --- /dev/null +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/BinaryModeExample.cs @@ -0,0 +1,272 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Examples.Datagrid +{ + using System; + using System.Collections; + using Apache.Ignite.Core; + using Apache.Ignite.Core.Binary; + using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Cache.Query; + + /// + /// This example works with cache entirely in binary mode: no classes or configurations are needed. + /// + /// 1) Set this class as startup object (Apache.Ignite.Examples project -> right-click -> Properties -> + /// Application -> Startup object); + /// 2) Start example (F5 or Ctrl+F5). + /// + /// This example can be run with standalone Apache Ignite.NET node: + /// 1) Run %IGNITE_HOME%/platforms/dotnet/bin/Apache.Ignite.exe: + /// Apache.Ignite.exe -configFileName=platforms\dotnet\examples\apache.ignite.examples\app.config + /// 2) Start example. + /// + public class BinaryModeExample + { + /// Cache name. + private const string CacheName = "dotnet_binary_cache"; + + /// Person type name. + private const string PersonType = "Person"; + + /// Company type name. + private const string CompanyType = "Company"; + + /// Name field name. + private const string NameField = "Name"; + + /// Company ID field name. + private const string CompanyIdField = "CompanyId"; + + /// ID field name. + private const string IdField = "Id"; + + [STAThread] + public static void Main() + { + using (var ignite = Ignition.StartFromApplicationConfiguration()) + { + Console.WriteLine(); + Console.WriteLine(">>> Binary mode example started."); + + // Create new cache and configure queries for Person and Company binary types. + // Note that there are no such classes defined. + var cache0 = ignite.GetOrCreateCache(new CacheConfiguration + { + Name = CacheName, + QueryEntities = new[] + { + new QueryEntity + { + KeyType = typeof(int), + ValueTypeName = PersonType, + Fields = new[] + { + new QueryField(NameField, typeof(string)), + new QueryField(CompanyIdField, typeof(int)) + }, + Indexes = new[] {new QueryIndex(false, QueryIndexType.FullText, NameField)} + }, + new QueryEntity + { + KeyType = typeof(int), + ValueTypeName = CompanyType, + Fields = new[] + { + new QueryField(IdField, typeof(int)), + new QueryField(NameField, typeof(string)) + } + } + } + }); + + // Switch to binary mode to work with data in serialized form. + var cache = cache0.WithKeepBinary(); + + // Clean up caches on all nodes before run. + cache.Clear(); + + // Populate cache with sample data entries. + PopulateCache(cache); + + // Run read & modify example. + ReadModifyExample(cache); + + // Run SQL query example. + SqlQueryExample(cache); + + // Run SQL query with join example. + SqlJoinQueryExample(cache); + + // Run SQL fields query example. + SqlFieldsQueryExample(cache); + + // Run full text query example. + FullTextQueryExample(cache); + + Console.WriteLine(); + } + + Console.WriteLine(); + Console.WriteLine(">>> Example finished, press any key to exit ..."); + Console.ReadKey(); + } + + /// + /// Reads binary object fields and modifies them. + /// + /// Cache. + private static void ReadModifyExample(ICache cache) + { + const int id = 1; + + IBinaryObject person = cache[id]; + + string name = person.GetField(NameField); + + Console.WriteLine(); + Console.WriteLine(">>> Name of the person with id {0}: {1}", id, name); + + // Modify the binary object. + cache[id] = person.ToBuilder().SetField("Name", name + " Jr.").Build(); + + Console.WriteLine(">>> Modified person with id {0}: {1}", id, cache[1]); + } + + /// + /// Queries persons that have a specific name using SQL. + /// + /// Cache. + private static void SqlQueryExample(ICache cache) + { + var qry = cache.Query(new SqlQuery(PersonType, "name like 'James%'")); + + Console.WriteLine(); + Console.WriteLine(">>> Persons named James:"); + + foreach (var entry in qry) + Console.WriteLine(">>> " + entry.Value); + } + + /// + /// Queries persons that work for company with provided name. + /// + /// Cache. + private static void SqlJoinQueryExample(ICache cache) + { + const string orgName = "Apache"; + + var qry = cache.Query(new SqlQuery(PersonType, + "from Person, Company " + + "where Person.CompanyId = Company.Id and Company.Name = ?", orgName)); + + Console.WriteLine(); + Console.WriteLine(">>> Persons working for " + orgName + ":"); + + foreach (var entry in qry) + Console.WriteLine(">>> " + entry.Value); + } + + /// + /// Queries names for all persons. + /// + /// Cache. + private static void SqlFieldsQueryExample(ICache cache) + { + var qry = cache.QueryFields(new SqlFieldsQuery("select name from Person order by name")); + + Console.WriteLine(); + Console.WriteLine(">>> All person names:"); + + foreach (IList row in qry) + Console.WriteLine(">>> " + row[0]); + } + + /// + /// Queries persons that have a specific name using full-text query API. + /// + /// Cache. + private static void FullTextQueryExample(ICache cache) + { + var qry = cache.Query(new TextQuery(PersonType, "Peters")); + + Console.WriteLine(); + Console.WriteLine(">>> Persons named Peters:"); + + foreach (var entry in qry) + Console.WriteLine(">>> " + entry.Value); + } + + /// + /// Populate cache with data for this example. + /// + /// Cache. + private static void PopulateCache(ICache cache) + { + IBinary binary = cache.Ignite.GetBinary(); + + // Populate persons. + cache[1] = binary.GetBuilder(PersonType) + .SetField(NameField, "James Wilson") + .SetField(CompanyIdField, -1) + .Build(); + + cache[2] = binary.GetBuilder(PersonType) + .SetField(NameField, "Daniel Adams") + .SetField(CompanyIdField, -1) + .Build(); + + cache[3] = binary.GetBuilder(PersonType) + .SetField(NameField, "Cristian Moss") + .SetField(CompanyIdField, -1) + .Build(); + + cache[4] = binary.GetBuilder(PersonType) + .SetField(NameField, "Allison Mathis") + .SetField(CompanyIdField, -2) + .Build(); + + cache[5] = binary.GetBuilder(PersonType) + .SetField(NameField, "Breana Robbin") + .SetField(CompanyIdField, -2) + .Build(); + + cache[6] = binary.GetBuilder(PersonType) + .SetField(NameField, "Philip Horsley") + .SetField(CompanyIdField, -2) + .Build(); + + cache[7] = binary.GetBuilder(PersonType) + .SetField(NameField, "James Peters") + .SetField(CompanyIdField, -2) + .Build(); + + // Populate companies. + cache[-1] = binary.GetBuilder(CompanyType) + .SetField(NameField, "Apache") + .SetField(IdField, -1) + .Build(); + + cache[-2] = binary.GetBuilder(CompanyType) + .SetField(NameField, "Microsoft") + .SetField(IdField, -2) + .Build(); + } + } +} From d5e15af76044cf65385672f8528d48ecdeca3cb6 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Wed, 2 Nov 2016 12:02:00 +0300 Subject: [PATCH 307/487] IGNITE-4121 .NET: add ScanQuery example --- .../Datagrid/QueryExample.cs | 24 ++++++++- .../Apache.Ignite.ExamplesDll.csproj | 1 + .../Datagrid/ScanQueryFilter.cs | 50 +++++++++++++++++++ 3 files changed, 73 insertions(+), 2 deletions(-) create mode 100644 modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ScanQueryFilter.cs diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs index 1c351492fa8e2..98f9f50b1ef96 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs @@ -19,13 +19,13 @@ namespace Apache.Ignite.Examples.Datagrid { using System; using System.Collections; - using System.Collections.Generic; using Apache.Ignite.Core; using Apache.Ignite.Core.Cache; using Apache.Ignite.Core.Cache.Affinity; using Apache.Ignite.Core.Cache.Configuration; using Apache.Ignite.Core.Cache.Query; using Apache.Ignite.ExamplesDll.Binary; + using Apache.Ignite.ExamplesDll.Datagrid; /// /// This example populates cache with sample data and runs several SQL and @@ -75,6 +75,9 @@ public static void Main() PopulateCache(employeeCacheColocated); PopulateCache(organizationCache); + // Run scan query example. + ScanQueryExample(employeeCache); + // Run SQL query example. SqlQueryExample(employeeCache); @@ -98,6 +101,23 @@ public static void Main() Console.ReadKey(); } + /// + /// Queries employees that have provided ZIP code in address. + /// + /// Cache. + private static void ScanQueryExample(ICache cache) + { + const int zip = 94109; + + var qry = cache.Query(new ScanQuery(new ScanQueryFilter(zip))); + + Console.WriteLine(); + Console.WriteLine(">>> Employees with zipcode {0} (scan):", zip); + + foreach (var entry in qry) + Console.WriteLine(">>> " + entry.Value); + } + /// /// Queries employees that have provided ZIP code in address. /// @@ -109,7 +129,7 @@ private static void SqlQueryExample(ICache cache) var qry = cache.Query(new SqlQuery(typeof(Employee), "zip = ?", zip)); Console.WriteLine(); - Console.WriteLine(">>> Employees with zipcode " + zip + ":"); + Console.WriteLine(">>> Employees with zipcode {0} (SQL):", zip); foreach (var entry in qry) Console.WriteLine(">>> " + entry.Value); diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj index 41981d8c87c59..a41c2f4d089b3 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj @@ -49,6 +49,7 @@ + diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ScanQueryFilter.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ScanQueryFilter.cs new file mode 100644 index 0000000000000..369b5d05ecf7b --- /dev/null +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ScanQueryFilter.cs @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +namespace Apache.Ignite.ExamplesDll.Datagrid +{ + using System; + using Apache.Ignite.Core.Cache; + using Apache.Ignite.ExamplesDll.Binary; + + /// + /// Filter for scan query example. + /// + [Serializable] + public class ScanQueryFilter : ICacheEntryFilter + { + /** Zip code to filter on. */ + private readonly int _zipCode; + + /// + /// Initializes a new instance of the class. + /// + /// The zip code. + public ScanQueryFilter(int zipCode) + { + _zipCode = zipCode; + } + + /// + /// Returns a value indicating whether provided cache entry satisfies this predicate. + /// + public bool Invoke(ICacheEntry entry) + { + return entry.Value.Address.Zip == _zipCode; + } + } +} From 74f8308d10fc011c00e52efcdb315b35cc79e60a Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Wed, 2 Nov 2016 15:59:15 +0300 Subject: [PATCH 308/487] IGNITE-4123 .NET: Remove [Serializable] from Employee in examples --- .../Impl/Messaging/MessageListenerHolder.cs | 6 ------ .../dotnet/examples/Apache.Ignite.Examples/App.config | 1 + .../examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs | 3 +-- .../Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs | 5 ++--- 4 files changed, 4 insertions(+), 11 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs index 8b674624b51c1..1a2c18a4f9b45 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs @@ -102,12 +102,6 @@ public void Release() DestroyAction(); } - /** */ - public bool Released - { - get { return false; } // Multiple releases are allowed. - } - /// /// Creates local holder instance. /// diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/App.config b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/App.config index 1d180e81f6a23..8b16df3bb1998 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/App.config +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/App.config @@ -37,6 +37,7 @@ Apache.Ignite.ExamplesDll.Binary.Employee Apache.Ignite.ExamplesDll.Binary.EmployeeKey Apache.Ignite.ExamplesDll.Binary.Organization + Apache.Ignite.ExamplesDll.Compute.AverageSalaryJob diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs index 4cff2a883cbaf..b746bdf0ad566 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs @@ -17,7 +17,6 @@ namespace Apache.Ignite.ExamplesDll.Binary { - using System; using System.Collections.Generic; using System.Linq; using Apache.Ignite.Core.Cache.Configuration; @@ -25,7 +24,6 @@ namespace Apache.Ignite.ExamplesDll.Binary /// /// Employee. /// - [Serializable] public class Employee { /// @@ -35,6 +33,7 @@ public class Employee /// Salary. /// Address. /// Departments. + /// The organization identifier. public Employee(string name, long salary, Address address, ICollection departments, int organizationId = 0) { diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs index 2685df09bba7d..491f3c1d60805 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs @@ -18,18 +18,17 @@ namespace Apache.Ignite.ExamplesDll.Compute { using System; - using System.Collections.Generic; + using System.Collections; using Apache.Ignite.Core.Compute; using Apache.Ignite.ExamplesDll.Binary; /// /// Average salary job. /// - [Serializable] public class AverageSalaryJob : ComputeJobAdapter> { /// Employees. - private readonly ICollection _employees = new List(); + private readonly ArrayList _employees = new ArrayList(); /// /// Adds employee. From 92fff630fbf36c82f93bbd9ddd53d11bed44e772 Mon Sep 17 00:00:00 2001 From: devozerov Date: Wed, 2 Nov 2016 17:50:51 +0300 Subject: [PATCH 309/487] Restored services compatibility. --- .../processors/job/GridJobWorker.java | 10 +- .../service/GridServiceProcessor.java | 197 +++++++++++------- .../internal/util/SerializableTransient.java | 58 ++++++ .../ignite/marshaller/MarshallerUtils.java | 22 ++ .../optimized/OptimizedClassDescriptor.java | 90 +++++++- 5 files changed, 296 insertions(+), 81 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/util/SerializableTransient.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java index 8169eb1101f4f..5f38b29025584 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java @@ -57,6 +57,7 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.marshaller.MarshallerUtils; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.events.EventType.EVT_JOB_CANCELLED; @@ -421,7 +422,14 @@ boolean initialize(GridDeployment dep, Class taskCls) { try { if (job == null) { - job = U.unmarshal(marsh, jobBytes, U.resolveClassLoader(dep.classLoader(), ctx.config())); + MarshallerUtils.jobSenderVersion(taskNode.version()); + + try { + job = U.unmarshal(marsh, jobBytes, U.resolveClassLoader(dep.classLoader(), ctx.config())); + } + finally { + MarshallerUtils.jobSenderVersion(null); + } // No need to hold reference any more. jobBytes = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 527d360e9578f..8489875badbcb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -20,11 +20,14 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; @@ -87,6 +90,7 @@ import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.internal.util.SerializableTransient; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.JobContextResource; import org.apache.ignite.resources.LoggerResource; @@ -115,6 +119,9 @@ public class GridServiceProcessor extends GridProcessorAdapter { /** */ public static final IgniteProductVersion LAZY_SERVICES_CFG_SINCE = IgniteProductVersion.fromString("1.5.22"); + /** Versions that only compatible with each other, and from 1.5.33. */ + private static final Set SERVICE_TOP_CALLABLE_VER1; + /** */ private final Boolean srvcCompatibilitySysProp; @@ -162,6 +169,31 @@ public class GridServiceProcessor extends GridProcessorAdapter { /** Topology listener. */ private GridLocalEventListener topLsnr = new TopologyListener(); + static { + Set versions = new TreeSet<>(new Comparator() { + @Override public int compare(final IgniteProductVersion o1, final IgniteProductVersion o2) { + return o1.compareToIgnoreTimestamp(o2); + } + }); + + versions.add(IgniteProductVersion.fromString("1.5.30")); + versions.add(IgniteProductVersion.fromString("1.5.31")); + versions.add(IgniteProductVersion.fromString("1.5.32")); + versions.add(IgniteProductVersion.fromString("1.6.3")); + versions.add(IgniteProductVersion.fromString("1.6.4")); + versions.add(IgniteProductVersion.fromString("1.6.5")); + versions.add(IgniteProductVersion.fromString("1.6.6")); + versions.add(IgniteProductVersion.fromString("1.6.7")); + versions.add(IgniteProductVersion.fromString("1.6.8")); + versions.add(IgniteProductVersion.fromString("1.6.9")); + versions.add(IgniteProductVersion.fromString("1.6.10")); + versions.add(IgniteProductVersion.fromString("1.7.0")); + versions.add(IgniteProductVersion.fromString("1.7.1")); + versions.add(IgniteProductVersion.fromString("1.7.2")); + + SERVICE_TOP_CALLABLE_VER1 = Collections.unmodifiableSet(versions); + } + /** * @param ctx Kernal context. */ @@ -668,9 +700,13 @@ public Map serviceTopology(String name) throws IgniteCheckedExcep ClusterNode node = cache.affinity().mapKeyToNode(name); if (node.version().compareTo(ServiceTopologyCallable.SINCE_VER) >= 0) { + final ServiceTopologyCallable call = new ServiceTopologyCallable(name); + + call.serialize = SERVICE_TOP_CALLABLE_VER1.contains(node.version()); + return ctx.closure().callAsyncNoFailover( GridClosureCallMode.BROADCAST, - new ServiceTopologyCallable(name), + call, Collections.singletonList(node), false ).get(); @@ -815,7 +851,7 @@ public T serviceProxy(ClusterGroup prj, String name, Class svcItf } } - return new GridServiceProxy(prj, name, svcItf, sticky, ctx).proxy(); + return new GridServiceProxy<>(prj, name, svcItf, sticky, ctx).proxy(); } /** @@ -868,7 +904,7 @@ public Collection services(String name) { * @param topVer Topology version. * @throws IgniteCheckedException If failed. */ - private void reassign(GridServiceDeployment dep, AffinityTopologyVersion topVer) throws IgniteCheckedException { + private void reassign(GridServiceDeployment dep, long topVer) throws IgniteCheckedException { ServiceConfiguration cfg = dep.configuration(); Object nodeFilter = cfg.getNodeFilter(); @@ -882,7 +918,7 @@ private void reassign(GridServiceDeployment dep, AffinityTopologyVersion topVer) Object affKey = cfg.getAffinityKey(); while (true) { - GridServiceAssignments assigns = new GridServiceAssignments(cfg, dep.nodeId(), topVer.topologyVersion()); + GridServiceAssignments assigns = new GridServiceAssignments(cfg, dep.nodeId(), topVer); Collection nodes; @@ -912,7 +948,7 @@ private void reassign(GridServiceDeployment dep, AffinityTopologyVersion topVer) Map cnts = new HashMap<>(); if (affKey != null) { - ClusterNode n = ctx.affinity().mapKeyToNode(cacheName, affKey, topVer); + ClusterNode n = ctx.affinity().mapKeyToNode(cacheName, affKey, new AffinityTopologyVersion(topVer)); if (n != null) { int cnt = maxPerNodeCnt == 0 ? totalCnt == 0 ? 1 : totalCnt : maxPerNodeCnt; @@ -1144,7 +1180,7 @@ private Service copyAndInject(ServiceConfiguration cfg) throws IgniteCheckedExce if (cfg instanceof LazyServiceConfiguration) { byte[] bytes = ((LazyServiceConfiguration)cfg).serviceBytes(); - Service srvc = U.unmarshal(m, bytes, U.resolveClassLoader(null, ctx.config())); + Service srvc = m.unmarshal(bytes, U.resolveClassLoader(null, ctx.config())); ctx.resource().inject(srvc); @@ -1154,9 +1190,10 @@ private Service copyAndInject(ServiceConfiguration cfg) throws IgniteCheckedExce Service svc = cfg.getService(); try { - byte[] bytes = U.marshal(m, svc); + byte[] bytes = m.marshal(svc); - Service cp = U.unmarshal(m, bytes, U.resolveClassLoader(svc.getClass().getClassLoader(), ctx.config())); + Service cp = m.unmarshal(bytes, + U.resolveClassLoader(svc.getClass().getClassLoader(), ctx.config())); ctx.resource().inject(cp); @@ -1231,8 +1268,8 @@ private Iterator> serviceEntries(IgniteBiPredicate(); + if (oldestSrvNode == null) + return F.emptyIterator(); GridCacheQueryManager qryMgr = cache.context().queries(); @@ -1418,7 +1455,7 @@ private void processDeployment(CacheEntryEvent 0; + if (newTopVer != topVer) { + assert newTopVer > topVer; - // Reassignment will happen from topology event. - return; - } + // Reassignment will happen from topology event. + return; + } - ctx.timeout().addTimeoutObject(new GridTimeoutObject() { - private IgniteUuid id = IgniteUuid.randomUuid(); + ctx.timeout().addTimeoutObject(new GridTimeoutObject() { + private IgniteUuid id = IgniteUuid.randomUuid(); - private long start = System.currentTimeMillis(); + private long start = System.currentTimeMillis(); - @Override public IgniteUuid timeoutId() { - return id; - } + @Override public IgniteUuid timeoutId() { + return id; + } - @Override public long endTime() { - return start + RETRY_TIMEOUT; - } + @Override public long endTime() { + return start + RETRY_TIMEOUT; + } - @Override public void onTimeout() { - if (!busyLock.enterBusy()) - return; + @Override public void onTimeout() { + if (!busyLock.enterBusy()) + return; - try { - // Try again. - onDeployment(dep, topVer); - } - finally { - busyLock.leaveBusy(); - } + try { + // Try again. + onDeployment(dep, topVer); } - }); + finally { + busyLock.leaveBusy(); + } + } + }); } } @@ -1531,28 +1568,16 @@ private void onDeployment(final GridServiceDeployment dep, final AffinityTopolog */ private class TopologyListener implements GridLocalEventListener { /** {@inheritDoc} */ - @Override public void onEvent(Event evt) { + @Override public void onEvent(final Event evt) { if (!busyLock.enterBusy()) return; try { - final AffinityTopologyVersion topVer; - - if (evt instanceof DiscoveryCustomEvent) { - DiscoveryCustomMessage msg = ((DiscoveryCustomEvent)evt).customMessage(); - - topVer = ((DiscoveryCustomEvent)evt).affinityTopologyVersion(); - - if (msg instanceof CacheAffinityChangeMessage) { - if (!((CacheAffinityChangeMessage)msg).exchangeNeeded()) - return; - } - } - else - topVer = new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion(), 0); - depExe.submit(new BusyRunnable() { @Override public void run0() { + AffinityTopologyVersion topVer = + new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion()); + ClusterNode oldest = CU.oldestAliveCacheServerNode(cache.context().shared(), topVer); if (oldest != null && oldest.isLocal()) { @@ -1587,7 +1612,7 @@ private class TopologyListener implements GridLocalEventListener { ctx.cache().internalCache(UTILITY_CACHE_NAME).context().affinity(). affinityReadyFuture(topVer).get(); - reassign(dep, topVer); + reassign(dep, topVer.topologyVersion()); } catch (IgniteCheckedException ex) { if (!(e instanceof ClusterTopologyCheckedException)) @@ -1604,7 +1629,7 @@ private class TopologyListener implements GridLocalEventListener { } if (!retries.isEmpty()) - onReassignmentFailed(topVer, retries); + onReassignmentFailed(topVer.topologyVersion(), retries); } // Clean up zombie assignments. @@ -1641,14 +1666,13 @@ private class TopologyListener implements GridLocalEventListener { * @param topVer Topology version. * @param retries Retries. */ - private void onReassignmentFailed(final AffinityTopologyVersion topVer, - final Collection retries) { + private void onReassignmentFailed(final long topVer, final Collection retries) { if (!busyLock.enterBusy()) return; try { // If topology changed again, let next event handle it. - if (ctx.discovery().topologyVersionEx().equals(topVer)) + if (ctx.discovery().topologyVersion() != topVer) return; for (Iterator it = retries.iterator(); it.hasNext(); ) { @@ -1829,6 +1853,7 @@ static class ServiceAssignmentsPredicate implements IgniteBiPredicate> { /** */ private static final long serialVersionUID = 0L; @@ -1836,11 +1861,14 @@ private static class ServiceTopologyCallable implements IgniteCallable + * Works only for jobs. For other messages node version is not available. + *

          + */ +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.TYPE) +public @interface SerializableTransient { + /** + * Name of the private static method that returns list of transient fields + * that should be serialized (String[]), and accepts itself (before serialization) + * and {@link IgniteProductVersion}, e.g. + *
          +     *     private static String[] fields(Object self, IgniteProductVersion ver){
          +     *         return ver.compareTo("1.5.30") > 0 ? SERIALIZABLE_FIELDS : null;
          +     *     }
          +     * 
          + *

          + * On serialization version argument ver is null, on deserialization - self is null. + *

          + *

          + * If it returns empty array or null all transient fields will be normally + * ignored. + *

          + * + * @return Name of the method. + */ + String methodName(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java index 9668bafb1c7ff..ad63702de00d5 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java @@ -17,6 +17,7 @@ package org.apache.ignite.marshaller; +import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.jetbrains.annotations.Nullable; @@ -24,6 +25,9 @@ * Utility marshaller methods. */ public class MarshallerUtils { + /** Job sender node version. */ + private static final ThreadLocal JOB_SND_NODE_VER = new ThreadLocal<>(); + /** * Set node name to marshaller context if possible. * @@ -55,4 +59,22 @@ public static JdkMarshaller jdkMarshaller(@Nullable String nodeName) { private MarshallerUtils() { // No-op. } + + /** + * Sets thread local job sender node version. + * + * @param ver Thread local job sender node version. + */ + public static void jobSenderVersion(IgniteProductVersion ver) { + JOB_SND_NODE_VER.set(ver); + } + + /** + * Returns thread local job sender node version. + * + * @return Thread local job sender node version. + */ + public static IgniteProductVersion jobSenderVersion() { + return JOB_SND_NODE_VER.get(); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java index 5a5b54d001f29..160f2c129de78 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java @@ -47,8 +47,11 @@ import java.util.concurrent.ConcurrentMap; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.marshaller.MarshallerContext; import org.apache.ignite.marshaller.MarshallerExclusions; +import org.apache.ignite.internal.util.SerializableTransient; +import org.apache.ignite.marshaller.MarshallerUtils; import static java.lang.reflect.Modifier.isFinal; import static java.lang.reflect.Modifier.isPrivate; @@ -166,6 +169,9 @@ class OptimizedClassDescriptor { /** Proxy interfaces. */ private Class[] proxyIntfs; + /** Method returns serializable transient fields. */ + private Method serTransMtd; + /** * Creates descriptor for class. * @@ -441,6 +447,27 @@ else if (Proxy.class.isAssignableFrom(cls)) { readObjMtds.add(mtd); + final SerializableTransient serTransAn = c.getAnnotation(SerializableTransient.class); + + // Custom serialization policy for transient fields. + if (serTransAn != null) { + try { + serTransMtd = c.getDeclaredMethod(serTransAn.methodName(), cls, IgniteProductVersion.class); + + int mod = serTransMtd.getModifiers(); + + if (isStatic(mod) && isPrivate(mod) + && serTransMtd.getReturnType() == String[].class) + serTransMtd.setAccessible(true); + else + // Set method back to null if it has incorrect signature. + serTransMtd = null; + } + catch (NoSuchMethodException ignored) { + serTransMtd = null; + } + } + Field[] clsFields0 = c.getDeclaredFields(); Map fieldNames = new HashMap<>(); @@ -797,7 +824,7 @@ void write(OptimizedObjectOutputStream out, Object obj) throws IOException { writeTypeData(out); out.writeShort(checksum); - out.writeSerializable(obj, writeObjMtds, fields); + out.writeSerializable(obj, writeObjMtds, serializableFields(obj.getClass(), obj, null)); break; @@ -806,6 +833,60 @@ void write(OptimizedObjectOutputStream out, Object obj) throws IOException { } } + /** + * Gets list of serializable fields. If {@link #serTransMtd} method + * returns list of transient fields, they will be added to other fields. + * Transient fields that are not included in that list will be normally + * ignored. + * + * @param cls Class. + * @param obj Object. + * @param ver Job sender version. + * @return Serializable fields. + */ + @SuppressWarnings("ForLoopReplaceableByForEach") + private Fields serializableFields(Class cls, Object obj, IgniteProductVersion ver) { + if (serTransMtd == null) + return fields; + + try { + final String[] transFields = (String[])serTransMtd.invoke(cls, obj, ver); + + if (transFields == null || transFields.length == 0) + return fields; + + List clsFields = new ArrayList<>(); + + clsFields.addAll(fields.fields.get(0).fields); + + for (int i = 0; i < transFields.length; i++) { + final String fieldName = transFields[i]; + + final Field f = cls.getDeclaredField(fieldName); + + FieldInfo fieldInfo = new FieldInfo(f, f.getName(), + GridUnsafe.objectFieldOffset(f), fieldType(f.getType())); + + clsFields.add(fieldInfo); + } + + Collections.sort(clsFields, new Comparator() { + @Override public int compare(FieldInfo t1, FieldInfo t2) { + return t1.name().compareTo(t2.name()); + } + }); + + List fields = new ArrayList<>(); + + fields.add(new ClassFields(clsFields)); + + return new Fields(fields); + } + catch (Exception e) { + return fields; + } + } + /** * @param out Output stream. * @throws IOException In case of error. @@ -838,7 +919,12 @@ Object read(OptimizedObjectInputStream in) throws ClassNotFoundException, IOExce case SERIALIZABLE: verifyChecksum(in.readShort()); - return in.readSerializable(cls, readObjMtds, readResolveMtd, fields); + // If no serialize method, then unmarshal as usual. + if (serTransMtd != null) + return in.readSerializable(cls, readObjMtds, readResolveMtd, + serializableFields(cls, null, MarshallerUtils.jobSenderVersion())); + else + return in.readSerializable(cls, readObjMtds, readResolveMtd, fields); default: assert false : "Unexpected type: " + type; From a62a0136d295486d95c6e2ab5bba88270d831753 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 2 Nov 2016 19:07:45 +0300 Subject: [PATCH 310/487] GG-11655 - Fix merge --- .../service/GridServiceProcessor.java | 136 ++++++++++-------- 1 file changed, 74 insertions(+), 62 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 8489875badbcb..6c263638de7c5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -851,7 +851,7 @@ public T serviceProxy(ClusterGroup prj, String name, Class svcItf } } - return new GridServiceProxy<>(prj, name, svcItf, sticky, ctx).proxy(); + return new GridServiceProxy(prj, name, svcItf, sticky, ctx).proxy(); } /** @@ -904,7 +904,7 @@ public Collection services(String name) { * @param topVer Topology version. * @throws IgniteCheckedException If failed. */ - private void reassign(GridServiceDeployment dep, long topVer) throws IgniteCheckedException { + private void reassign(GridServiceDeployment dep, AffinityTopologyVersion topVer) throws IgniteCheckedException { ServiceConfiguration cfg = dep.configuration(); Object nodeFilter = cfg.getNodeFilter(); @@ -918,7 +918,7 @@ private void reassign(GridServiceDeployment dep, long topVer) throws IgniteCheck Object affKey = cfg.getAffinityKey(); while (true) { - GridServiceAssignments assigns = new GridServiceAssignments(cfg, dep.nodeId(), topVer); + GridServiceAssignments assigns = new GridServiceAssignments(cfg, dep.nodeId(), topVer.topologyVersion()); Collection nodes; @@ -948,7 +948,7 @@ private void reassign(GridServiceDeployment dep, long topVer) throws IgniteCheck Map cnts = new HashMap<>(); if (affKey != null) { - ClusterNode n = ctx.affinity().mapKeyToNode(cacheName, affKey, new AffinityTopologyVersion(topVer)); + ClusterNode n = ctx.affinity().mapKeyToNode(cacheName, affKey, topVer); if (n != null) { int cnt = maxPerNodeCnt == 0 ? totalCnt == 0 ? 1 : totalCnt : maxPerNodeCnt; @@ -1180,7 +1180,7 @@ private Service copyAndInject(ServiceConfiguration cfg) throws IgniteCheckedExce if (cfg instanceof LazyServiceConfiguration) { byte[] bytes = ((LazyServiceConfiguration)cfg).serviceBytes(); - Service srvc = m.unmarshal(bytes, U.resolveClassLoader(null, ctx.config())); + Service srvc = U.unmarshal(m, bytes, U.resolveClassLoader(null, ctx.config())); ctx.resource().inject(srvc); @@ -1190,10 +1190,9 @@ private Service copyAndInject(ServiceConfiguration cfg) throws IgniteCheckedExce Service svc = cfg.getService(); try { - byte[] bytes = m.marshal(svc); + byte[] bytes = U.marshal(m, svc); - Service cp = m.unmarshal(bytes, - U.resolveClassLoader(svc.getClass().getClassLoader(), ctx.config())); + Service cp = U.unmarshal(m, bytes, U.resolveClassLoader(svc.getClass().getClassLoader(), ctx.config())); ctx.resource().inject(cp); @@ -1268,8 +1267,8 @@ private Iterator> serviceEntries(IgniteBiPredicate(); GridCacheQueryManager qryMgr = cache.context().queries(); @@ -1455,7 +1454,7 @@ private void processDeployment(CacheEntryEvent topVer; + /** + * Deployment callback. + * + * @param dep Service deployment. + * @param topVer Topology version. + */ + private void onDeployment(final GridServiceDeployment dep, final AffinityTopologyVersion topVer) { + // Retry forever. + try { + AffinityTopologyVersion newTopVer = ctx.discovery().topologyVersionEx(); - // Reassignment will happen from topology event. - return; + // If topology version changed, reassignment will happen from topology event. + if (newTopVer.equals(topVer)) + reassign(dep, topVer); } + catch (IgniteCheckedException e) { + if (!(e instanceof ClusterTopologyCheckedException)) + log.error("Failed to do service reassignment (will retry): " + dep.configuration().getName(), e); - ctx.timeout().addTimeoutObject(new GridTimeoutObject() { - private IgniteUuid id = IgniteUuid.randomUuid(); + AffinityTopologyVersion newTopVer = ctx.discovery().topologyVersionEx(); - private long start = System.currentTimeMillis(); + if (!newTopVer.equals(topVer)) { + assert newTopVer.compareTo(topVer) > 0; - @Override public IgniteUuid timeoutId() { - return id; + // Reassignment will happen from topology event. + return; } - @Override public long endTime() { - return start + RETRY_TIMEOUT; - } + ctx.timeout().addTimeoutObject(new GridTimeoutObject() { + private IgniteUuid id = IgniteUuid.randomUuid(); - @Override public void onTimeout() { - if (!busyLock.enterBusy()) - return; + private long start = System.currentTimeMillis(); - try { - // Try again. - onDeployment(dep, topVer); + @Override public IgniteUuid timeoutId() { + return id; } - finally { - busyLock.leaveBusy(); + + @Override public long endTime() { + return start + RETRY_TIMEOUT; } - } - }); + + @Override public void onTimeout() { + if (!busyLock.enterBusy()) + return; + + try { + // Try again. + onDeployment(dep, topVer); + } + finally { + busyLock.leaveBusy(); + } + } + }); } } @@ -1568,16 +1567,28 @@ private void onDeployment(final GridServiceDeployment dep, final long topVer) { */ private class TopologyListener implements GridLocalEventListener { /** {@inheritDoc} */ - @Override public void onEvent(final Event evt) { + @Override public void onEvent(Event evt) { if (!busyLock.enterBusy()) return; try { + final AffinityTopologyVersion topVer; + + if (evt instanceof DiscoveryCustomEvent) { + DiscoveryCustomMessage msg = ((DiscoveryCustomEvent)evt).customMessage(); + + topVer = ((DiscoveryCustomEvent)evt).affinityTopologyVersion(); + + if (msg instanceof CacheAffinityChangeMessage) { + if (!((CacheAffinityChangeMessage)msg).exchangeNeeded()) + return; + } + } + else + topVer = new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion(), 0); + depExe.submit(new BusyRunnable() { @Override public void run0() { - AffinityTopologyVersion topVer = - new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion()); - ClusterNode oldest = CU.oldestAliveCacheServerNode(cache.context().shared(), topVer); if (oldest != null && oldest.isLocal()) { @@ -1612,7 +1623,7 @@ private class TopologyListener implements GridLocalEventListener { ctx.cache().internalCache(UTILITY_CACHE_NAME).context().affinity(). affinityReadyFuture(topVer).get(); - reassign(dep, topVer.topologyVersion()); + reassign(dep, topVer); } catch (IgniteCheckedException ex) { if (!(e instanceof ClusterTopologyCheckedException)) @@ -1629,7 +1640,7 @@ private class TopologyListener implements GridLocalEventListener { } if (!retries.isEmpty()) - onReassignmentFailed(topVer.topologyVersion(), retries); + onReassignmentFailed(topVer, retries); } // Clean up zombie assignments. @@ -1666,13 +1677,14 @@ private class TopologyListener implements GridLocalEventListener { * @param topVer Topology version. * @param retries Retries. */ - private void onReassignmentFailed(final long topVer, final Collection retries) { + private void onReassignmentFailed(final AffinityTopologyVersion topVer, + final Collection retries) { if (!busyLock.enterBusy()) return; try { // If topology changed again, let next event handle it. - if (ctx.discovery().topologyVersion() != topVer) + if (ctx.discovery().topologyVersionEx().equals(topVer)) return; for (Iterator it = retries.iterator(); it.hasNext(); ) { From 175da6b7e394dd76c27d5155ff98a5b2ef03bb9d Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Mon, 7 Nov 2016 11:16:58 +0500 Subject: [PATCH 311/487] IGNITE-3432: check data/meta cache names are different for different IGFS instances. This closes #1201 --- .../processors/igfs/IgfsProcessor.java | 15 ++++ .../igfs/IgfsProcessorValidationSelfTest.java | 30 ++++++++ ...cEndpointRegistrationAbstractSelfTest.java | 76 +++++++++++++++++-- ...ointRegistrationOnLinuxAndMacSelfTest.java | 11 ++- 4 files changed, 122 insertions(+), 10 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java index 5c0e030b1b8c9..c72140d87b5b6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.igfs; +import java.util.Set; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteFileSystem; import org.apache.ignite.cache.affinity.AffinityKeyMapper; @@ -295,6 +296,9 @@ private String maskName(@Nullable String name) { private void validateLocalIgfsConfigurations(FileSystemConfiguration[] cfgs) throws IgniteCheckedException { Collection cfgNames = new HashSet<>(); + Collection dataCacheNames = new HashSet<>(); + Collection metaCacheNames = new HashSet<>(); + for (FileSystemConfiguration cfg : cfgs) { String name = cfg.getName(); @@ -327,6 +331,14 @@ private void validateLocalIgfsConfigurations(FileSystemConfiguration[] cfgs) thr if (F.eq(cfg.getDataCacheName(), cfg.getMetaCacheName())) throw new IgniteCheckedException("Cannot use same cache as both data and meta cache: " + cfg.getName()); + if (dataCacheNames.contains(cfg.getDataCacheName())) + throw new IgniteCheckedException("Data cache names should be different for different IGFS instances: " + + cfg.getName()); + + if (metaCacheNames.contains(cfg.getMetaCacheName())) + throw new IgniteCheckedException("Meta cache names should be different for different IGFS instances: " + + cfg.getName()); + if (!(dataCacheCfg.getAffinityMapper() instanceof IgfsGroupDataBlocksKeyMapper)) throw new IgniteCheckedException("Invalid IGFS data cache configuration (key affinity mapper class should be " + IgfsGroupDataBlocksKeyMapper.class.getSimpleName() + "): " + cfg); @@ -360,6 +372,9 @@ private void validateLocalIgfsConfigurations(FileSystemConfiguration[] cfgs) thr "secondaryFileSystem cannot be null when mode is not " + IgfsMode.PRIMARY); } + dataCacheNames.add(cfg.getDataCacheName()); + metaCacheNames.add(cfg.getMetaCacheName()); + cfgNames.add(name); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java index 97334da8bcb50..28ec1da1d056b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java @@ -236,6 +236,36 @@ public void testLocalIfNonPrimaryModeAndHadoopFileSystemUriIsNull() throws Excep checkGridStartFails(g1Cfg, "secondaryFileSystem cannot be null when mode is not PRIMARY", true); } + /** + * @throws Exception If failed. + */ + public void testLocalIfMetaCacheNameEquals() throws Exception { + g1Cfg.setCacheConfiguration(concat(dataCaches(1024), metaCaches(), CacheConfiguration.class)); + + g1IgfsCfg1.setDataCacheName(dataCache1Name); + g1IgfsCfg1.setMetaCacheName(metaCache1Name); + + g1IgfsCfg2.setDataCacheName(dataCache2Name); + g1IgfsCfg2.setMetaCacheName(metaCache1Name); + + checkGridStartFails(g1Cfg, "Meta cache names should be different for different IGFS instances", true); + } + + /** + * @throws Exception If failed. + */ + public void testLocalIfDataCacheNameEquals() throws Exception { + g1Cfg.setCacheConfiguration(concat(dataCaches(1024), metaCaches(), CacheConfiguration.class)); + + g1IgfsCfg1.setDataCacheName(dataCache1Name); + g1IgfsCfg1.setMetaCacheName(metaCache1Name); + + g1IgfsCfg2.setDataCacheName(dataCache1Name); + g1IgfsCfg2.setMetaCacheName(metaCache2Name); + + checkGridStartFails(g1Cfg, "Data cache names should be different for different IGFS instances", true); + } + /** * @throws Exception If failed. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java index 13a19fea86439..e58d5038bc650 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java @@ -17,6 +17,8 @@ package org.apache.ignite.internal.processors.igfs; +import java.util.ArrayList; +import java.util.List; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.IgniteCheckedException; @@ -48,7 +50,7 @@ */ public abstract class IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest extends IgfsCommonAbstractTest { /** IP finder. */ - private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + protected static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); private static final AtomicInteger mgmtPort = new AtomicInteger(DFLT_MGMT_PORT); @@ -80,12 +82,13 @@ public void testLoopbackEndpointsRegistration() throws Exception { * @throws Exception If failed. */ public void testLoopbackEndpointsCustomHostRegistration() throws Exception { - IgniteConfiguration cfg = gridConfiguration(); + IgniteConfiguration cfg = gridConfigurationManyIgfsCaches(2); cfg.setFileSystemConfiguration( - igfsConfiguration(IgfsIpcEndpointType.TCP, IgfsIpcEndpointConfiguration.DFLT_PORT, "127.0.0.1"), + igfsConfiguration(IgfsIpcEndpointType.TCP, IgfsIpcEndpointConfiguration.DFLT_PORT, "127.0.0.1", + "partitioned0", "replicated0"), igfsConfiguration(IgfsIpcEndpointType.TCP, IgfsIpcEndpointConfiguration.DFLT_PORT + 1, - U.getLocalHost().getHostName())); + U.getLocalHost().getHostName(), "partitioned1", "replicated1")); G.start(cfg); @@ -100,6 +103,7 @@ public void testLoopbackEndpointsCustomHostRegistration() throws Exception { * Counts all registered IPC endpoints. * * @return Tuple2 where (tcp endpoints count, shmem endpoints count). + * @throws Exception If failed. */ protected T2 checkRegisteredIpcEndpoints() throws Exception { GridKernalContext ctx = ((IgniteKernal)grid()).context(); @@ -150,6 +154,48 @@ protected IgniteConfiguration gridConfiguration() throws Exception { return cfg; } + /** + * Creates base grid configuration. + * + * @param cacheCtn Caches count. + * + * @return Base grid configuration. + * @throws Exception In case of any error. + */ + IgniteConfiguration gridConfigurationManyIgfsCaches(int cacheCtn) throws Exception { + IgniteConfiguration cfg = getConfiguration(getTestGridName()); + + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + discoSpi.setIpFinder(IP_FINDER); + + cfg.setDiscoverySpi(discoSpi); + + List cachesCfg = new ArrayList<>(); + + for (int i = 0; i < cacheCtn; ++i) { + CacheConfiguration dataCacheCfg = defaultCacheConfiguration(); + + dataCacheCfg.setName("partitioned" + i); + dataCacheCfg.setCacheMode(CacheMode.PARTITIONED); + dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128)); + dataCacheCfg.setBackups(0); + dataCacheCfg.setAtomicityMode(TRANSACTIONAL); + + CacheConfiguration metaCacheCfg = defaultCacheConfiguration(); + + metaCacheCfg.setName("replicated" + i); + metaCacheCfg.setCacheMode(CacheMode.REPLICATED); + metaCacheCfg.setAtomicityMode(TRANSACTIONAL); + + cachesCfg.add(dataCacheCfg); + cachesCfg.add(metaCacheCfg); + } + + cfg.setCacheConfiguration(cachesCfg.toArray(new CacheConfiguration[cachesCfg.size()])); + + return cfg; + } + /** * Creates test-purposed IgfsConfiguration. * @@ -157,9 +203,27 @@ protected IgniteConfiguration gridConfiguration() throws Exception { * @param endPntPort End point port. * @param endPntHost End point host. * @return test-purposed IgfsConfiguration. + * @throws IgniteCheckedException If failed. */ protected FileSystemConfiguration igfsConfiguration(@Nullable IgfsIpcEndpointType endPntType, @Nullable Integer endPntPort, @Nullable String endPntHost) throws IgniteCheckedException { + + return igfsConfiguration(endPntType, endPntPort, endPntHost, "partitioned", "replicated"); + } + + /** + * Creates test-purposed IgfsConfiguration. + * + * @param endPntType End point type. + * @param endPntPort End point port. + * @param endPntHost End point host. + * @param dataCacheName Data cache name. + * @param metaCacheName Meta cache name. + * @return test-purposed IgfsConfiguration. + * @throws IgniteCheckedException If failed. + */ + protected FileSystemConfiguration igfsConfiguration(@Nullable IgfsIpcEndpointType endPntType, + @Nullable Integer endPntPort, @Nullable String endPntHost, String dataCacheName, String metaCacheName) throws IgniteCheckedException { IgfsIpcEndpointConfiguration endPntCfg = null; if (endPntType != null) { @@ -176,8 +240,8 @@ protected FileSystemConfiguration igfsConfiguration(@Nullable IgfsIpcEndpointTyp FileSystemConfiguration igfsConfiguration = new FileSystemConfiguration(); - igfsConfiguration.setDataCacheName("partitioned"); - igfsConfiguration.setMetaCacheName("replicated"); + igfsConfiguration.setDataCacheName(dataCacheName); + igfsConfiguration.setMetaCacheName(metaCacheName); igfsConfiguration.setName("igfs" + UUID.randomUUID()); igfsConfiguration.setManagementPort(mgmtPort.getAndIncrement()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.java index 3d898745b2ec5..030c852e520fd 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.java @@ -33,12 +33,15 @@ public class IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest * @throws Exception If failed. */ public void testLoopbackAndShmemEndpointsRegistration() throws Exception { - IgniteConfiguration cfg = gridConfiguration(); + IgniteConfiguration cfg = gridConfigurationManyIgfsCaches(3); cfg.setFileSystemConfiguration( - igfsConfiguration(null, null, null), // Check null IPC endpoint config won't bring any hassles. - igfsConfiguration(IgfsIpcEndpointType.TCP, IgfsIpcEndpointConfiguration.DFLT_PORT + 1, null), - igfsConfiguration(IgfsIpcEndpointType.SHMEM, IgfsIpcEndpointConfiguration.DFLT_PORT + 2, null)); + // Check null IPC endpoint config won't bring any hassles. + igfsConfiguration(null, null, null, "partitioned0", "replicated0"), + igfsConfiguration(IgfsIpcEndpointType.TCP, IgfsIpcEndpointConfiguration.DFLT_PORT + 1, null, + "partitioned1", "replicated1"), + igfsConfiguration(IgfsIpcEndpointType.SHMEM, IgfsIpcEndpointConfiguration.DFLT_PORT + 2, null, + "partitioned2", "replicated2")); G.start(cfg); From ead15193899d08f41491166003cabed0560f0c59 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 7 Nov 2016 10:49:03 +0300 Subject: [PATCH 312/487] IGNITE-4028 Get rid of OP_META in PlatformAbstractTarget This closes #1192 --- .../platform/PlatformAbstractTarget.java | 11 +- .../platform/PlatformNoopProcessor.java | 5 + .../platform/PlatformProcessor.java | 7 + .../platform/PlatformProcessorImpl.java | 6 + .../binary/PlatformBinaryProcessor.java | 96 +++++++++++ .../cluster/PlatformClusterGroup.java | 31 ---- .../ignite/impl/binary/binary_type_manager.h | 14 +- .../impl/binary/binary_type_updater_impl.h | 12 +- .../include/ignite/impl/ignite_environment.h | 49 ++++-- modules/platforms/cpp/core/src/ignition.cpp | 8 +- .../impl/binary/binary_type_updater_impl.cpp | 13 +- .../cpp/core/src/impl/ignite_environment.cpp | 45 +++-- .../core/src/impl/interop/interop_target.cpp | 4 +- .../cpp/jni/include/ignite/jni/exports.h | 1 + .../cpp/jni/include/ignite/jni/java.h | 2 + .../cpp/jni/include/ignite/jni/utils.h | 91 +++++++++- .../platforms/cpp/jni/project/vs/module.def | 1 + modules/platforms/cpp/jni/src/exports.cpp | 4 + modules/platforms/cpp/jni/src/java.cpp | 13 ++ .../Apache.Ignite.Core.csproj | 2 +- .../Apache.Ignite.Core/Impl/Binary/Binary.cs | 2 +- .../Binary/BinaryObjectSchemaSerializer.cs | 2 +- .../Impl/Binary/BinaryProcessor.cs | 156 ++++++++++++++++++ .../Impl/Binary/BinaryReader.cs | 2 +- .../Impl/Binary/Marshaller.cs | 11 +- .../Impl/Cluster/ClusterGroupImpl.cs | 72 ++------ .../Impl/Cluster/IClusterGroupEx.cs | 35 ---- .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs | 29 ++-- .../Apache.Ignite.Core/Impl/IgniteProxy.cs | 9 +- .../Apache.Ignite.Core/Impl/PlatformTarget.cs | 64 ------- .../Impl/Unmanaged/IgniteJniNativeMethods.cs | 3 + .../Impl/Unmanaged/UnmanagedUtils.cs | 7 + 32 files changed, 526 insertions(+), 281 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs delete mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java index cba0031eeffdd..2df86acee9ce8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java @@ -43,9 +43,6 @@ public abstract class PlatformAbstractTarget implements PlatformTarget { /** Constant: ERROR. */ protected static final int ERROR = -1; - /** */ - private static final int OP_META = -1; - /** Context. */ protected final PlatformContext platformCtx; @@ -78,13 +75,7 @@ protected PlatformAbstractTarget(PlatformContext platformCtx) { try (PlatformMemory mem = platformCtx.memory().get(memPtr)) { BinaryRawReaderEx reader = platformCtx.reader(mem); - if (type == OP_META) { - platformCtx.processMetadata(reader); - - return TRUE; - } - else - return processInStreamOutLong(type, reader, mem); + return processInStreamOutLong(type, reader, mem); } catch (Exception e) { throw convertException(e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java index a7b7a8d9b028c..fd357ec77f67c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java @@ -185,4 +185,9 @@ public PlatformNoopProcessor(GridKernalContext ctx) { @Override public void loggerLog(int level, String message, String category, String errorInfo) { // No-op. } + + /** {@inheritDoc} */ + @Override public PlatformTarget binaryProcessor() { + return null; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java index 1d9d3cdc1eee9..f01175eaf9dd6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java @@ -271,4 +271,11 @@ public interface PlatformProcessor extends GridProcessor { * @param errorInfo Error info. */ public void loggerLog(int level, String message, String category, String errorInfo); + + /** + * Gets the binary processor. + * + * @return Binary processor. + */ + public PlatformTarget binaryProcessor(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java index d875c7e332433..f775987024135 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java @@ -35,6 +35,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl; import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicLongImpl; +import org.apache.ignite.internal.processors.platform.binary.PlatformBinaryProcessor; import org.apache.ignite.internal.processors.platform.cache.PlatformCache; import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension; import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinity; @@ -502,6 +503,11 @@ private PlatformTarget createPlatformCache(IgniteCacheProxy cache) { } } + /** {@inheritDoc} */ + @Override public PlatformTarget binaryProcessor() { + return new PlatformBinaryProcessor(platformCtx); + } + /** * Gets the near cache config. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java new file mode 100644 index 0000000000000..1bb577e2b8a04 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java @@ -0,0 +1,96 @@ +/* + * 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.platform.binary; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.binary.BinaryRawReaderEx; +import org.apache.ignite.internal.binary.BinaryRawWriterEx; +import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget; +import org.apache.ignite.internal.processors.platform.PlatformContext; + +/** + * Platform binary processor. + */ +public class PlatformBinaryProcessor extends PlatformAbstractTarget { + /** */ + private static final int OP_GET_META = 1; + + /** */ + private static final int OP_GET_ALL_META = 2; + + /** */ + private static final int OP_PUT_META = 3; + + /** */ + private static final int OP_GET_SCHEMA = 4; + + /** + * Constructor. + * + * @param platformCtx Context. + */ + public PlatformBinaryProcessor(PlatformContext platformCtx) { + super(platformCtx); + } + + /** {@inheritDoc} */ + @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException { + if (type == OP_PUT_META) { + platformCtx.processMetadata(reader); + + return TRUE; + } + + return super.processInStreamOutLong(type, reader); + } + + /** {@inheritDoc} */ + @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException { + if (type == OP_GET_ALL_META) + platformCtx.writeAllMetadata(writer); + else + super.processOutStream(type, writer); + } + + /** {@inheritDoc} */ + @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, + BinaryRawWriterEx writer) throws IgniteCheckedException { + switch (type) { + case OP_GET_META: { + int typeId = reader.readInt(); + + platformCtx.writeMetadata(writer, typeId); + + break; + } + + case OP_GET_SCHEMA: { + int typeId = reader.readInt(); + int schemaId = reader.readInt(); + + platformCtx.writeSchema(writer, typeId, schemaId); + + break; + } + + default: + super.processInStreamOutStream(type, reader, writer); + break; + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java index dde98f5be5641..724eea0293db7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java @@ -37,9 +37,6 @@ */ @SuppressWarnings({"UnusedDeclaration"}) public class PlatformClusterGroup extends PlatformAbstractTarget { - /** */ - private static final int OP_ALL_METADATA = 1; - /** */ private static final int OP_FOR_ATTRIBUTE = 2; @@ -58,9 +55,6 @@ public class PlatformClusterGroup extends PlatformAbstractTarget { /** */ private static final int OP_FOR_NODE_IDS = 7; - /** */ - private static final int OP_METADATA = 8; - /** */ private static final int OP_METRICS = 9; @@ -79,9 +73,6 @@ public class PlatformClusterGroup extends PlatformAbstractTarget { /** */ private static final int OP_TOPOLOGY = 14; - /** */ - private static final int OP_SCHEMA = 15; - /** */ private static final int OP_FOR_OTHERS = 16; @@ -130,11 +121,6 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { break; - case OP_ALL_METADATA: - platformCtx.writeAllMetadata(writer); - - break; - default: super.processOutStream(type, writer); } @@ -201,14 +187,6 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { break; } - case OP_METADATA: { - int typeId = reader.readInt(); - - platformCtx.writeMetadata(writer, typeId); - - break; - } - case OP_TOPOLOGY: { long topVer = reader.readLong(); @@ -217,15 +195,6 @@ public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) { break; } - case OP_SCHEMA: { - int typeId = reader.readInt(); - int schemaId = reader.readInt(); - - platformCtx.writeSchema(writer, typeId, schemaId); - - break; - } - default: super.processInStreamOutStream(type, reader, writer); } diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h index 6019a2c17a037..8aca1a04c8f86 100644 --- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h +++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h @@ -65,7 +65,7 @@ namespace ignite /** * Get current type manager version. * - * @param Version. + * @return Version. */ int32_t GetVersion(); @@ -80,7 +80,7 @@ namespace ignite /** * Process pending updates. * - * @param updated Updater. + * @param updater Updater. * @param err Error. * @return In case of success. */ @@ -89,18 +89,18 @@ namespace ignite private: /** Current snapshots. */ ignite::common::concurrent::SharedPointer> snapshots; - + /** Pending snapshots. */ - std::vector* pending; + std::vector* pending; /** Critical section. */ ignite::common::concurrent::CriticalSection* cs; /** Version of pending changes. */ - int32_t pendingVer; - + int32_t pendingVer; + /** Latest version. */ - int32_t ver; + int32_t ver; IGNITE_NO_COPY_ASSIGNMENT(BinaryTypeManager); diff --git a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_type_updater_impl.h b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_type_updater_impl.h index 4edad4fcbebea..bd21751096702 100644 --- a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_type_updater_impl.h +++ b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_type_updater_impl.h @@ -24,7 +24,7 @@ #include "ignite/impl/binary/binary_type_updater.h" namespace ignite -{ +{ namespace impl { namespace binary @@ -41,7 +41,7 @@ namespace ignite * @param env Environment. * @param javaRef Reference to Java object which is able to process type request. */ - BinaryTypeUpdaterImpl(ignite::common::concurrent::SharedPointer env, jobject javaRef); + BinaryTypeUpdaterImpl(IgniteEnvironment& env, jobject javaRef); /** * Destructor. @@ -51,15 +51,15 @@ namespace ignite bool Update(Snap* snapshot, IgniteError* err); private: /** Environment. */ - ignite::common::concurrent::SharedPointer env; - + IgniteEnvironment& env; + /** Handle to Java object. */ - jobject javaRef; + jobject javaRef; IGNITE_NO_COPY_ASSIGNMENT(BinaryTypeUpdaterImpl) }; } - } + } } #endif //_IGNITE_IMPL_BINARY_BINARY_TYPE_UPDATER_IMPL \ No newline at end of file diff --git a/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h b/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h index d2486bb535b42..fb6f6572fc943 100644 --- a/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h +++ b/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h @@ -23,6 +23,7 @@ #include "ignite/impl/interop/interop_memory.h" #include "ignite/impl/binary/binary_type_manager.h" +#include "ignite/jni/utils.h" namespace ignite { @@ -52,24 +53,29 @@ namespace ignite /** * Populate callback handlers. * - * @param Target (current env wrapped into a shared pointer). + * @param target (current env wrapped into a shared pointer). * @return JNI handlers. */ - ignite::jni::java::JniHandlers GetJniHandlers(ignite::common::concurrent::SharedPointer* target); + jni::java::JniHandlers GetJniHandlers(common::concurrent::SharedPointer* target); /** - * Perform initialization on successful start. + * Set context. * * @param ctx Context. */ - void Initialize(ignite::common::concurrent::SharedPointer ctx); + void SetContext(common::concurrent::SharedPointer ctx); + + /** + * Perform initialization on successful start. + */ + void Initialize(); /** * Start callback. * * @param memPtr Memory pointer. */ - void OnStartCallback(long long memPtr); + void OnStartCallback(long long memPtr, jobject proc); /** * Get name of Ignite instance. @@ -83,14 +89,14 @@ namespace ignite * * @return Context. */ - ignite::jni::java::JniContext* Context(); + jni::java::JniContext* Context(); /** * Get memory for interop operations. * * @return Memory. */ - ignite::common::concurrent::SharedPointer AllocateMemory(); + common::concurrent::SharedPointer AllocateMemory(); /** * Get memory chunk for interop operations with desired capacity. @@ -98,7 +104,7 @@ namespace ignite * @param cap Capacity. * @return Memory. */ - ignite::common::concurrent::SharedPointer AllocateMemory(int32_t cap); + common::concurrent::SharedPointer AllocateMemory(int32_t cap); /** * Get memory chunk located at the given pointer. @@ -106,27 +112,46 @@ namespace ignite * @param memPtr Memory pointer. * @retrun Memory. */ - ignite::common::concurrent::SharedPointer GetMemory(int64_t memPtr); + common::concurrent::SharedPointer GetMemory(int64_t memPtr); /** * Get type manager. * - * @param Type manager. + * @return Type manager. */ binary::BinaryTypeManager* GetTypeManager(); + + /** + * Get type updater. + * + * @return Type updater. + */ + binary::BinaryTypeUpdater* GetTypeUpdater(); + + /** + * Notify processor that Ignite instance has started. + */ + void ProcessorReleaseStart(); + private: /** Context to access Java. */ - ignite::common::concurrent::SharedPointer ctx; + common::concurrent::SharedPointer ctx; /** Startup latch. */ - ignite::common::concurrent::SingleLatch* latch; + common::concurrent::SingleLatch* latch; /** Ignite name. */ char* name; + /** Processor instance. */ + jni::JavaGlobalRef proc; + /** Type manager. */ binary::BinaryTypeManager* metaMgr; + /** Type updater. */ + binary::BinaryTypeUpdater* metaUpdater; + IGNITE_NO_COPY_ASSIGNMENT(IgniteEnvironment); }; } diff --git a/modules/platforms/cpp/core/src/ignition.cpp b/modules/platforms/cpp/core/src/ignition.cpp index 83adb4c38aaeb..72fbf7a8f0c13 100644 --- a/modules/platforms/cpp/core/src/ignition.cpp +++ b/modules/platforms/cpp/core/src/ignition.cpp @@ -230,10 +230,12 @@ namespace ignite if (!ctx.Get()) { IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); - + failed = true; } + env.Get()->SetContext(ctx); + // 5. Start Ignite. if (!failed) { @@ -263,7 +265,7 @@ namespace ignite } else { // 6. Ignite is started at this point. - env.Get()->Initialize(ctx); + env.Get()->Initialize(); started = true; } @@ -288,6 +290,8 @@ namespace ignite } else { + env.Get()->ProcessorReleaseStart(); + IgniteImpl* impl = new IgniteImpl(env, javaRef); return Ignite(impl); diff --git a/modules/platforms/cpp/core/src/impl/binary/binary_type_updater_impl.cpp b/modules/platforms/cpp/core/src/impl/binary/binary_type_updater_impl.cpp index e6375a62302b2..ff6df9e204c8b 100644 --- a/modules/platforms/cpp/core/src/impl/binary/binary_type_updater_impl.cpp +++ b/modules/platforms/cpp/core/src/impl/binary/binary_type_updater_impl.cpp @@ -34,24 +34,25 @@ namespace ignite namespace binary { /** Operation: metadata update. */ - const int32_t OP_METADATA = -1; + const int32_t OP_PUT_META = 3; - BinaryTypeUpdaterImpl::BinaryTypeUpdaterImpl(SharedPointer env, - jobject javaRef) : env(env), javaRef(javaRef) + BinaryTypeUpdaterImpl::BinaryTypeUpdaterImpl(IgniteEnvironment& env, jobject javaRef) : + env(env), + javaRef(javaRef) { // No-op. } BinaryTypeUpdaterImpl::~BinaryTypeUpdaterImpl() { - // No-op. + JniContext::Release(javaRef); } bool BinaryTypeUpdaterImpl::Update(Snap* snap, IgniteError* err) { JniErrorInfo jniErr; - SharedPointer mem = env.Get()->AllocateMemory(); + SharedPointer mem = env.AllocateMemory(); InteropOutputStream out(mem.Get()); BinaryWriterImpl writer(&out, NULL); @@ -85,7 +86,7 @@ namespace ignite out.Synchronize(); - long long res = env.Get()->Context()->TargetInStreamOutLong(javaRef, OP_METADATA, mem.Get()->PointerLong(), &jniErr); + long long res = env.Context()->TargetInStreamOutLong(javaRef, OP_PUT_META, mem.Get()->PointerLong(), &jniErr); IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err); diff --git a/modules/platforms/cpp/core/src/impl/ignite_environment.cpp b/modules/platforms/cpp/core/src/impl/ignite_environment.cpp index ab50866f6b182..32c855bc18bea 100644 --- a/modules/platforms/cpp/core/src/impl/ignite_environment.cpp +++ b/modules/platforms/cpp/core/src/impl/ignite_environment.cpp @@ -19,6 +19,7 @@ #include "ignite/impl/binary/binary_reader_impl.h" #include "ignite/impl/ignite_environment.h" #include "ignite/binary/binary.h" +#include "ignite/impl/binary/binary_type_updater_impl.h" using namespace ignite::common::concurrent; using namespace ignite::jni::java; @@ -34,14 +35,14 @@ namespace ignite * OnStart callback. * * @param target Target environment. - * @param proc Processor instance (not used for now). + * @param proc Processor instance. * @param memPtr Memory pointer. */ void IGNITE_CALL OnStart(void* target, void* proc, long long memPtr) { SharedPointer* ptr = static_cast*>(target); - ptr->Get()->OnStartCallback(memPtr); + ptr->Get()->OnStartCallback(memPtr, reinterpret_cast(proc)); } /** @@ -72,8 +73,8 @@ namespace ignite mem.Get()->Reallocate(cap); } - IgniteEnvironment::IgniteEnvironment() : ctx(SharedPointer()), latch(new SingleLatch), name(NULL), - metaMgr(new BinaryTypeManager()) + IgniteEnvironment::IgniteEnvironment() : ctx(SharedPointer()), latch(new SingleLatch), name(0), + proc(), metaMgr(new BinaryTypeManager()), metaUpdater(0) { // No-op. } @@ -81,11 +82,9 @@ namespace ignite IgniteEnvironment::~IgniteEnvironment() { delete latch; - - if (name) - delete name; - + delete name; delete metaMgr; + delete metaUpdater; } JniHandlers IgniteEnvironment::GetJniHandlers(SharedPointer* target) @@ -99,16 +98,23 @@ namespace ignite hnds.memRealloc = MemoryReallocate; - hnds.error = NULL; + hnds.error = 0; return hnds; } - void IgniteEnvironment::Initialize(SharedPointer ctx) + void IgniteEnvironment::SetContext(SharedPointer ctx) { this->ctx = ctx; + } + void IgniteEnvironment::Initialize() + { latch->CountDown(); + + jobject binaryProc = Context()->ProcessorBinaryProcessor(proc.Get()); + + metaUpdater = new BinaryTypeUpdaterImpl(*this, binaryProc); } const char* IgniteEnvironment::InstanceName() const @@ -160,14 +166,27 @@ namespace ignite return metaMgr; } - void IgniteEnvironment::OnStartCallback(long long memPtr) + BinaryTypeUpdater* IgniteEnvironment::GetTypeUpdater() { + return metaUpdater; + } + + void IgniteEnvironment::ProcessorReleaseStart() + { + if (proc.Get()) + ctx.Get()->ProcessorReleaseStart(proc.Get()); + } + + void IgniteEnvironment::OnStartCallback(long long memPtr, jobject proc) + { + this->proc = jni::JavaGlobalRef(*ctx.Get(), proc); + InteropExternalMemory mem(reinterpret_cast(memPtr)); InteropInputStream stream(&mem); BinaryReaderImpl reader(&stream); - int32_t nameLen = reader.ReadString(NULL, 0); + int32_t nameLen = reader.ReadString(0, 0); if (nameLen >= 0) { @@ -175,7 +194,7 @@ namespace ignite reader.ReadString(name, nameLen + 1); } else - name = NULL; + name = 0; } } } diff --git a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp index 196c3f691d99f..4992ccb82aa2a 100644 --- a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp +++ b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp @@ -72,9 +72,7 @@ namespace ignite if (metaMgr->IsUpdatedSince(metaVer)) { - BinaryTypeUpdaterImpl metaUpdater(env, javaRef); - - if (!metaMgr->ProcessPendingUpdates(&metaUpdater, err)) + if (!metaMgr->ProcessPendingUpdates(env.Get()->GetTypeUpdater(), err)) return 0; } diff --git a/modules/platforms/cpp/jni/include/ignite/jni/exports.h b/modules/platforms/cpp/jni/include/ignite/jni/exports.h index 3a98eda5ebddd..3052435298ea5 100644 --- a/modules/platforms/cpp/jni/include/ignite/jni/exports.h +++ b/modules/platforms/cpp/jni/include/ignite/jni/exports.h @@ -56,6 +56,7 @@ extern "C" { void IGNITE_CALL IgniteProcessorGetCacheNames(gcj::JniContext* ctx, void* obj, long long memPtr); bool IGNITE_CALL IgniteProcessorLoggerIsLevelEnabled(gcj::JniContext* ctx, void* obj, int level); void IGNITE_CALL IgniteProcessorLoggerLog(gcj::JniContext* ctx, void* obj, int level, char* message, char* category, char* errorInfo); + void* IGNITE_CALL IgniteProcessorBinaryProcessor(gcj::JniContext* ctx, void* obj); long long IGNITE_CALL IgniteTargetInLongOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr); long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr); diff --git a/modules/platforms/cpp/jni/include/ignite/jni/java.h b/modules/platforms/cpp/jni/include/ignite/jni/java.h index 9dddd8cc723b0..07df001c43b83 100644 --- a/modules/platforms/cpp/jni/include/ignite/jni/java.h +++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h @@ -269,6 +269,7 @@ namespace ignite jmethodID m_PlatformProcessor_atomicReference; jmethodID m_PlatformProcessor_loggerIsLevelEnabled; jmethodID m_PlatformProcessor_loggerLog; + jmethodID m_PlatformProcessor_binaryProcessor; jclass c_PlatformTarget; jmethodID m_PlatformTarget_inLongOutLong; @@ -449,6 +450,7 @@ namespace ignite void ProcessorGetCacheNames(jobject obj, long long memPtr); bool ProcessorLoggerIsLevelEnabled(jobject obj, int level); void ProcessorLoggerLog(jobject obj, int level, char* message, char* category, char* errorInfo); + jobject ProcessorBinaryProcessor(jobject obj); long long TargetInLongOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); long long TargetInStreamOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL); diff --git a/modules/platforms/cpp/jni/include/ignite/jni/utils.h b/modules/platforms/cpp/jni/include/ignite/jni/utils.h index ee402484a79df..2b22cf97e67e1 100644 --- a/modules/platforms/cpp/jni/include/ignite/jni/utils.h +++ b/modules/platforms/cpp/jni/include/ignite/jni/utils.h @@ -19,6 +19,7 @@ #include +#include #include namespace ignite @@ -35,13 +36,101 @@ namespace ignite * Destructor. */ ~AttachHelper(); - + /** * Callback invoked on successful thread attach ot JVM. */ static void OnThreadAttach(); }; + /** + * Represents global reference to Java object. + */ + class IGNITE_IMPORT_EXPORT JavaGlobalRef + { + public: + /** + * Default constructor + */ + JavaGlobalRef() : + ctx(0), + obj(0) + { + // No-op. + } + + /** + * Constructor + * + * @param ctx JNI context. + * @param obj Java object. + */ + JavaGlobalRef(java::JniContext& ctx, jobject obj) : + ctx(&ctx), + obj(ctx.Acquire(obj)) + { + // No-op. + } + + /** + * Copy constructor + * + * @param other Other instance. + */ + JavaGlobalRef(const JavaGlobalRef& other) : + ctx(other.ctx), + obj(ctx->Acquire(other.obj)) + { + // No-op. + } + + /** + * Assignment operator. + * + * @param other Other instance. + * @return *this. + */ + JavaGlobalRef& operator=(const JavaGlobalRef& other) + { + if (this != &other) + { + if (ctx) + ctx->Release(obj); + + ctx = other.ctx; + obj = ctx->Acquire(other.obj); + } + + return *this; + } + + /** + * Destructor. + */ + ~JavaGlobalRef() + { + if (ctx) + ctx->Release(obj); + } + + /** + * Get object. + * + * @return Object. + */ + jobject Get() + { + return obj; + } + + private: + /** Context. */ + java::JniContext* ctx; + + /** Object. */ + jobject obj; + }; + /** * Attempts to find JVM library to load it into the process later. * First search is performed using the passed path argument (is not NULL). diff --git a/modules/platforms/cpp/jni/project/vs/module.def b/modules/platforms/cpp/jni/project/vs/module.def index 258e80cac6e6d..e58ac3ba0dbe4 100644 --- a/modules/platforms/cpp/jni/project/vs/module.def +++ b/modules/platforms/cpp/jni/project/vs/module.def @@ -51,3 +51,4 @@ IgniteSetConsoleHandler @135 IgniteRemoveConsoleHandler @136 IgniteProcessorLoggerIsLevelEnabled @137 IgniteProcessorLoggerLog @138 +IgniteProcessorBinaryProcessor @139 \ No newline at end of file diff --git a/modules/platforms/cpp/jni/src/exports.cpp b/modules/platforms/cpp/jni/src/exports.cpp index e87cbd3333917..dde98fbb6af91 100644 --- a/modules/platforms/cpp/jni/src/exports.cpp +++ b/modules/platforms/cpp/jni/src/exports.cpp @@ -150,6 +150,10 @@ extern "C" { ctx->ProcessorLoggerLog(static_cast(obj), level, message, category, errorInfo); } + void* IGNITE_CALL IgniteProcessorBinaryProcessor(gcj::JniContext* ctx, void* obj) { + return ctx->ProcessorBinaryProcessor(static_cast(obj)); + } + long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr) { return ctx->TargetInStreamOutLong(static_cast(obj), opType, memPtr); } diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp index e2c9bf7fa3089..2d3cf729467f1 100644 --- a/modules/platforms/cpp/jni/src/java.cpp +++ b/modules/platforms/cpp/jni/src/java.cpp @@ -218,6 +218,7 @@ namespace ignite JniMethod M_PLATFORM_PROCESSOR_GET_CACHE_NAMES = JniMethod("getCacheNames", "(J)V", false); JniMethod M_PLATFORM_PROCESSOR_LOGGER_IS_LEVEL_ENABLED = JniMethod("loggerIsLevelEnabled", "(I)Z", false); JniMethod M_PLATFORM_PROCESSOR_LOGGER_LOG = JniMethod("loggerLog", "(ILjava/lang/String;Ljava/lang/String;Ljava/lang/String;)V", false); + JniMethod M_PLATFORM_PROCESSOR_BINARY_PROCESSOR = JniMethod("binaryProcessor", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false); const char* C_PLATFORM_TARGET = "org/apache/ignite/internal/processors/platform/PlatformTarget"; JniMethod M_PLATFORM_TARGET_IN_LONG_OUT_LONG = JniMethod("inLongOutLong", "(IJ)J", false); @@ -534,6 +535,7 @@ namespace ignite m_PlatformProcessor_getCacheNames = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_GET_CACHE_NAMES); m_PlatformProcessor_loggerIsLevelEnabled = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_LOGGER_IS_LEVEL_ENABLED); m_PlatformProcessor_loggerLog = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_LOGGER_LOG); + m_PlatformProcessor_binaryProcessor = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_BINARY_PROCESSOR); c_PlatformTarget = FindClass(env, C_PLATFORM_TARGET); m_PlatformTarget_inLongOutLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_LONG_OUT_LONG); @@ -1339,6 +1341,17 @@ namespace ignite ExceptionCheck(env); } + jobject JniContext::ProcessorBinaryProcessor(jobject obj) + { + JNIEnv* env = Attach(); + + jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_binaryProcessor); + + ExceptionCheck(env); + + return LocalToGlobal(env, res); + } + long long JniContext::TargetInStreamOutLong(jobject obj, int opType, long long memPtr, JniErrorInfo* err) { JNIEnv* env = Attach(); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj index 8e8f8caadc89e..2973bb3bf2760 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj @@ -91,6 +91,7 @@ + @@ -287,7 +288,6 @@ - diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs index 1deac07de7aac..3d55acd6f4962 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs @@ -120,7 +120,7 @@ public int GetTypeId(string typeName) /** */ public ICollection GetBinaryTypes() { - return Marshaller.Ignite.ClusterGroup.GetBinaryTypes(); + return Marshaller.Ignite.BinaryProcessor.GetBinaryTypes(); } /** */ diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs index da86c07ff46a5..f1d2f6a9068b6 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs @@ -250,7 +250,7 @@ private static int[] GetFieldIds(BinaryObjectHeader hdr, BinaryObjectSchema sche if (fieldIds == null) { if (marsh.Ignite != null) - fieldIds = marsh.Ignite.ClusterGroup.GetSchema(hdr.TypeId, hdr.SchemaId); + fieldIds = marsh.Ignite.BinaryProcessor.GetSchema(hdr.TypeId, hdr.SchemaId); if (fieldIds == null) throw new BinaryObjectException("Cannot find schema for object with compact footer [" + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs new file mode 100644 index 0000000000000..6935fa2a348c5 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs @@ -0,0 +1,156 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Impl.Binary +{ + using System.Collections.Generic; + using System.Diagnostics; + using Apache.Ignite.Core.Binary; + using Apache.Ignite.Core.Impl.Binary.Metadata; + using Apache.Ignite.Core.Impl.Unmanaged; + + /// + /// Binary metadata processor. + /// + internal class BinaryProcessor : PlatformTarget + { + /// + /// Op codes. + /// + private enum Op + { + GetMeta = 1, + GetAllMeta = 2, + PutMeta = 3, + GetSchema = 4 + } + + /// + /// Initializes a new instance of the class. + /// + /// Target. + /// Marshaller. + public BinaryProcessor(IUnmanagedTarget target, Marshaller marsh) : base(target, marsh) + { + // No-op. + } + + /// + /// Gets metadata for specified type. + /// + public IBinaryType GetBinaryType(int typeId) + { + return DoOutInOp((int) Op.GetMeta, + writer => writer.WriteInt(typeId), + stream => + { + var reader = Marshaller.StartUnmarshal(stream, false); + + return reader.ReadBoolean() ? new BinaryType(reader) : null; + } + ); + } + + /// + /// Gets metadata for all known types. + /// + public List GetBinaryTypes() + { + return DoInOp((int) Op.GetAllMeta, s => + { + var reader = Marshaller.StartUnmarshal(s); + + var size = reader.ReadInt(); + + var res = new List(size); + + for (var i = 0; i < size; i++) + res.Add(reader.ReadBoolean() ? new BinaryType(reader) : null); + + return res; + }); + } + + /// + /// Gets the schema. + /// + public int[] GetSchema(int typeId, int schemaId) + { + return DoOutInOp((int) Op.GetSchema, writer => + { + writer.WriteInt(typeId); + writer.WriteInt(schemaId); + }); + } + + /// + /// Put binary types to Grid. + /// + /// Binary types. + internal void PutBinaryTypes(ICollection types) + { + DoOutOp((int) Op.PutMeta, w => + { + w.WriteInt(types.Count); + + foreach (var meta in types) + { + w.WriteInt(meta.TypeId); + w.WriteString(meta.TypeName); + w.WriteString(meta.AffinityKeyFieldName); + + var fields = meta.GetFieldsMap(); + + w.WriteInt(fields.Count); + + foreach (var field in fields) + { + w.WriteString(field.Key); + w.WriteInt(field.Value); + } + + w.WriteBoolean(meta.IsEnum); + + // Send schemas + var desc = meta.Descriptor; + Debug.Assert(desc != null); + + var count = 0; + var countPos = w.Stream.Position; + w.WriteInt(0); // Reserve for count + + foreach (var schema in desc.Schema.GetAll()) + { + w.WriteInt(schema.Key); + + var ids = schema.Value; + w.WriteInt(ids.Length); + + foreach (var id in ids) + w.WriteInt(id); + + count++; + } + + w.Stream.WriteInt(countPos, count); + } + }); + + Marshaller.OnBinaryTypesSent(types); + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs index f048e9733e859..4c34f73078350 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs @@ -768,7 +768,7 @@ private int[] ReadSchema() if (_curHdr.IsCompactFooter) { // Get schema from Java - var schema = Marshaller.Ignite.ClusterGroup.GetSchema(_curHdr.TypeId, _curHdr.SchemaId); + var schema = Marshaller.Ignite.BinaryProcessor.GetSchema(_curHdr.TypeId, _curHdr.SchemaId); if (schema == null) throw new BinaryObjectException("Cannot find schema for object with compact footer [" + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs index 6e63e9aaf3b61..7acdfaaaeb725 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs @@ -161,7 +161,9 @@ public void FinishMarshal(BinaryWriter writer) var ignite = Ignite; if (ignite != null && metas != null && metas.Count > 0) - ignite.PutBinaryTypes(metas); + { + ignite.BinaryProcessor.PutBinaryTypes(metas); + } } /// @@ -270,7 +272,7 @@ public IBinaryType GetBinaryType(int typeId) { if (Ignite != null) { - IBinaryType meta = Ignite.GetBinaryType(typeId); + IBinaryType meta = Ignite.BinaryProcessor.GetBinaryType(typeId); if (meta != null) return meta; @@ -290,7 +292,10 @@ public void PutBinaryType(IBinaryTypeDescriptor desc) GetBinaryTypeHandler(desc); // ensure that handler exists if (Ignite != null) - Ignite.PutBinaryTypes(new[] {new BinaryType(desc)}); + { + ICollection metas = new[] {new BinaryType(desc)}; + Ignite.BinaryProcessor.PutBinaryTypes(metas); + } } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs index 277d61f6fac57..6c8779c3aa829 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs @@ -29,7 +29,6 @@ namespace Apache.Ignite.Core.Impl.Cluster using Apache.Ignite.Core.Compute; using Apache.Ignite.Core.Events; using Apache.Ignite.Core.Impl.Binary; - using Apache.Ignite.Core.Impl.Binary.Metadata; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Impl.Compute; using Apache.Ignite.Core.Impl.Events; @@ -43,7 +42,7 @@ namespace Apache.Ignite.Core.Impl.Cluster /// /// Ignite projection implementation. /// - internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx + internal class ClusterGroupImpl : PlatformTarget, IClusterGroup { /** Attribute: platform. */ private const string AttrPlatform = "org.apache.ignite.platform"; @@ -54,9 +53,6 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx /** Initial topver; invalid from Java perspective, so update will be triggered when this value is met. */ private const int TopVerInit = 0; - /** */ - private const int OpAllMetadata = 1; - /** */ private const int OpForAttribute = 2; @@ -75,9 +71,6 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx /** */ private const int OpForNodeIds = 7; - /** */ - private const int OpMetadata = 8; - /** */ private const int OpMetrics = 9; @@ -96,14 +89,11 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx /** */ private const int OpTopology = 14; - /** */ - private const int OpSchema = 15; - /** */ private const int OpForRemotes = 17; /** */ - public const int OpForDaemons = 18; + private const int OpForDaemons = 18; /** */ private const int OpForRandom = 19; @@ -115,10 +105,10 @@ internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx private const int OpForYoungest = 21; /** */ - public const int OpResetMetrics = 22; + private const int OpResetMetrics = 22; /** */ - public const int OpForServers = 23; + private const int OpForServers = 23; /** Initial Ignite instance. */ private readonly Ignite _ignite; @@ -507,6 +497,14 @@ internal IList NodesNoRefresh() return _nodes; } + /// + /// Resets the metrics. + /// + public void ResetMetrics() + { + DoOutInOp(OpResetMetrics); + } + /// /// Creates new Cluster Group from given native projection. /// @@ -553,51 +551,5 @@ private IList RefreshNodes() return _nodes; } - - /** */ - public IBinaryType GetBinaryType(int typeId) - { - return DoOutInOp(OpMetadata, - writer => writer.WriteInt(typeId), - stream => - { - var reader = Marshaller.StartUnmarshal(stream, false); - - return reader.ReadBoolean() ? new BinaryType(reader) : null; - } - ); - } - - /// - /// Gets metadata for all known types. - /// - public List GetBinaryTypes() - { - return DoInOp(OpAllMetadata, s => - { - var reader = Marshaller.StartUnmarshal(s); - - var size = reader.ReadInt(); - - var res = new List(size); - - for (var i = 0; i < size; i++) - res.Add(reader.ReadBoolean() ? new BinaryType(reader) : null); - - return res; - }); - } - - /// - /// Gets the schema. - /// - public int[] GetSchema(int typeId, int schemaId) - { - return DoOutInOp(OpSchema, writer => - { - writer.WriteInt(typeId); - writer.WriteInt(schemaId); - }); - } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs deleted file mode 100644 index 59162e60def21..0000000000000 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs +++ /dev/null @@ -1,35 +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. - */ - -namespace Apache.Ignite.Core.Impl.Cluster -{ - using Apache.Ignite.Core.Binary; - using Apache.Ignite.Core.Cluster; - - /// - /// Extended internal Ignite interface. - /// - internal interface IClusterGroupEx : IClusterGroup - { - /// - /// Gets protable metadata for type. - /// - /// Type ID. - /// Metadata. - IBinaryType GetBinaryType(int typeId); - } -} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs index e8171cb5efe64..1747df2849bd6 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs @@ -33,7 +33,6 @@ namespace Apache.Ignite.Core.Impl using Apache.Ignite.Core.DataStructures; using Apache.Ignite.Core.Events; using Apache.Ignite.Core.Impl.Binary; - using Apache.Ignite.Core.Impl.Binary.Metadata; using Apache.Ignite.Core.Impl.Cache; using Apache.Ignite.Core.Impl.Cluster; using Apache.Ignite.Core.Impl.Common; @@ -52,7 +51,7 @@ namespace Apache.Ignite.Core.Impl /// /// Native Ignite wrapper. /// - internal class Ignite : IIgnite, IClusterGroupEx, ICluster + internal class Ignite : IIgnite, ICluster { /** */ private readonly IgniteConfiguration _cfg; @@ -72,6 +71,9 @@ internal class Ignite : IIgnite, IClusterGroupEx, ICluster /** Binary. */ private readonly Binary.Binary _binary; + /** Binary processor. */ + private readonly BinaryProcessor _binaryProc; + /** Cached proxy. */ private readonly IgniteProxy _proxy; @@ -126,6 +128,8 @@ internal class Ignite : IIgnite, IClusterGroupEx, ICluster _binary = new Binary.Binary(marsh); + _binaryProc = new BinaryProcessor(UU.ProcessorBinaryProcessor(proc), marsh); + _proxy = new IgniteProxy(this); cbs.Initialize(this); @@ -517,7 +521,7 @@ public ICollection GetTopology(long ver) /** */ public void ResetMetrics() { - UU.TargetInLongOutLong(_prj.Target, ClusterGroupImpl.OpResetMetrics, 0); + _prj.ResetMetrics(); } /** */ @@ -732,26 +736,19 @@ internal Marshaller Marshaller } /// - /// Configuration. + /// Gets the binary processor. /// - internal IgniteConfiguration Configuration + internal BinaryProcessor BinaryProcessor { - get { return _cfg; } + get { return _binaryProc; } } /// - /// Put metadata to Grid. + /// Configuration. /// - /// Metadata. - internal void PutBinaryTypes(ICollection metas) - { - _prj.PutBinaryTypes(metas); - } - - /** */ - public IBinaryType GetBinaryType(int typeId) + internal IgniteConfiguration Configuration { - return _prj.GetBinaryType(typeId); + get { return _cfg; } } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs index 8dc63bd02a9f9..98a2d476bfbfa 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs @@ -30,7 +30,6 @@ namespace Apache.Ignite.Core.Impl using Apache.Ignite.Core.DataStructures; using Apache.Ignite.Core.Events; using Apache.Ignite.Core.Impl.Binary; - using Apache.Ignite.Core.Impl.Cluster; using Apache.Ignite.Core.Log; using Apache.Ignite.Core.Lifecycle; using Apache.Ignite.Core.Messaging; @@ -42,7 +41,7 @@ namespace Apache.Ignite.Core.Impl /// [Serializable] [ExcludeFromCodeCoverage] - internal class IgniteProxy : IIgnite, IClusterGroupEx, IBinaryWriteAware, ICluster + internal class IgniteProxy : IIgnite, IBinaryWriteAware, ICluster { /** */ [NonSerialized] @@ -455,11 +454,5 @@ internal Ignite Target return _ignite; } } - - /** */ - public IBinaryType GetBinaryType(int typeId) - { - return _ignite.GetBinaryType(typeId); - } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs index fb9d890dface9..bafc759cbd087 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs @@ -26,7 +26,6 @@ namespace Apache.Ignite.Core.Impl using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Binary.IO; - using Apache.Ignite.Core.Impl.Binary.Metadata; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Impl.Memory; using Apache.Ignite.Core.Impl.Unmanaged; @@ -49,9 +48,6 @@ internal abstract class PlatformTarget /** */ protected const int Error = -1; - /** */ - private const int OpMeta = -1; - /** */ public const int OpNone = -2; @@ -826,66 +822,6 @@ internal void FinishMarshal(BinaryWriter writer) _marsh.FinishMarshal(writer); } - /// - /// Put binary types to Grid. - /// - /// Binary types. - internal void PutBinaryTypes(ICollection types) - { - DoOutOp(OpMeta, stream => - { - BinaryWriter w = _marsh.StartMarshal(stream); - - w.WriteInt(types.Count); - - foreach (var meta in types) - { - w.WriteInt(meta.TypeId); - w.WriteString(meta.TypeName); - w.WriteString(meta.AffinityKeyFieldName); - - IDictionary fields = meta.GetFieldsMap(); - - w.WriteInt(fields.Count); - - foreach (var field in fields) - { - w.WriteString(field.Key); - w.WriteInt(field.Value); - } - - w.WriteBoolean(meta.IsEnum); - - // Send schemas - var desc = meta.Descriptor; - Debug.Assert(desc != null); - - var count = 0; - var countPos = stream.Position; - w.WriteInt(0); // Reserve for count - - foreach (var schema in desc.Schema.GetAll()) - { - w.WriteInt(schema.Key); - - var ids = schema.Value; - w.WriteInt(ids.Length); - - foreach (var id in ids) - w.WriteInt(id); - - count++; - } - - stream.WriteInt(countPos, count); - } - - _marsh.FinishMarshal(w); - }); - - _marsh.OnBinaryTypesSent(types); - } - /// /// Unmarshal object using the given stream. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs index ac41f2555a46c..c746866da100d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs @@ -123,6 +123,9 @@ internal static unsafe class IgniteJniNativeMethods [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorLoggerLog")] public static extern void ProcessorLoggerLog(void* ctx, void* obj, int level, sbyte* messsage, sbyte* category, sbyte* errorInfo); + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorBinaryProcessor")] + public static extern void* ProcessorBinaryProcessor(void* ctx, void* obj); + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInStreamOutLong")] public static extern long TargetInStreamOutLong(void* ctx, void* target, int opType, long memPtr); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs index fe1904c0974f0..f36c35ff0c913 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs @@ -402,6 +402,13 @@ internal static bool ProcessorLoggerIsLevelEnabled(IUnmanagedTarget target, int } } + internal static IUnmanagedTarget ProcessorBinaryProcessor(IUnmanagedTarget target) + { + void* res = JNI.ProcessorBinaryProcessor(target.Context, target.Target); + + return target.ChangeTarget(res); + } + #endregion #region NATIVE METHODS: TARGET From 40ef2f5ae42826fe8fd077e3013e8f55c8512bdd Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Mon, 7 Nov 2016 12:09:41 +0300 Subject: [PATCH 313/487] ignite-4178 support permission builder --- .../security/SecurityBasicPermissionSet.java | 107 +++++++++ .../SecurityPermissionSetBuilder.java | 222 ++++++++++++++++++ .../SecurityPermissionSetBuilderTest.java | 93 ++++++++ .../testsuites/IgniteBasicTestSuite.java | 3 + 4 files changed, 425 insertions(+) create mode 100644 modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java create mode 100644 modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java create mode 100644 modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java new file mode 100644 index 0000000000000..5b50c56321377 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java @@ -0,0 +1,107 @@ +/* + * 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.plugin.security; + +import java.util.Map; +import java.util.HashMap; +import java.util.ArrayList; +import java.util.Collection; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; + +/** + * Simple implementation of {@link SecurityPermissionSet} interface. Provides + * convenient way to specify permission set in the XML configuration. + */ +public class SecurityBasicPermissionSet implements SecurityPermissionSet { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Cache permissions. */ + private Map> cachePerms = new HashMap<>(); + + /** Task permissions. */ + private Map> taskPerms = new HashMap<>(); + + /** System permissions. */ + private Collection sysPerms = new ArrayList<>(); + + /** Default allow all. */ + private boolean dfltAllowAll; + + /** + * Setter for set cache permission map. + * + * @param cachePerms Cache permissions. + */ + public void setCachePermissions(Map> cachePerms) { + this.cachePerms = cachePerms; + } + + /** + * Setter for set task permission map. + * + * @param taskPerms Task permissions. + */ + public void setTaskPermissions(Map> taskPerms) { + this.taskPerms = taskPerms; + } + + /** + * Setter for set collection system permission. + * + * @param sysPerms System permissions. + */ + public void setSystemPermissions(Collection sysPerms) { + this.sysPerms = sysPerms; + } + + /** + * Setter for set default allow all. + * + * @param dfltAllowAll Default allow all. + */ + public void setDefaultAllowAll(boolean dfltAllowAll) { + this.dfltAllowAll = dfltAllowAll; + } + + /** {@inheritDoc} */ + @Override public Map> cachePermissions() { + return cachePerms; + } + + /** {@inheritDoc} */ + @Override public Map> taskPermissions() { + return taskPerms; + } + + /** {@inheritDoc} */ + @Nullable @Override public Collection systemPermissions() { + return sysPerms; + } + + /** {@inheritDoc} */ + @Override public boolean defaultAllowAll() { + return dfltAllowAll; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SecurityBasicPermissionSet.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java new file mode 100644 index 0000000000000..61ad77ce08f5e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java @@ -0,0 +1,222 @@ +/* + * 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.plugin.security; + +import java.util.Map; +import java.util.List; +import java.util.HashMap; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import org.apache.ignite.IgniteException; + +import static java.util.Collections.unmodifiableList; +import static java.util.Collections.unmodifiableMap; + +/** + * Provides a convenient way to create a permission set. + *

          + * Here is example: + *

          + *      SecurityPermissionSet permsSet = new SecurityPermissionSetBuilder()
          + *          .appendCachePermissions("cache1", CACHE_PUT, CACHE_REMOVE)
          + *          .appendCachePermissions("cache2", CACHE_READ)
          + *          .appendTaskPermissions("task1", TASK_CANCEL)
          + *          .appendTaskPermissions("task2", TASK_EXECUTE)
          + *          .appendSystemPermissions(ADMIN_VIEW, EVENTS_ENABLE)
          + *          .build();
          + * 
          + *

          + * The builder also does additional validation. For example, if you try to + * append {@code EVENTS_ENABLE} permission for a cache, exception will be thrown: + *

          + *      SecurityPermissionSet permsSet = new SecurityPermissionSetBuilder()
          + *          .appendCachePermissions("cache1", EVENTS_ENABLE)
          + *          .build();
          + * 
          + */ +public class SecurityPermissionSetBuilder { + /** Cache permissions.*/ + private Map> cachePerms = new HashMap<>(); + + /** Task permissions.*/ + private Map> taskPerms = new HashMap<>(); + + /** System permissions.*/ + private List sysPerms = new ArrayList<>(); + + /** Default allow all.*/ + private boolean dfltAllowAll; + + /** + * Static factory method for create new permission builder. + * + * @return SecurityPermissionSetBuilder + */ + public static SecurityPermissionSetBuilder create(){ + return new SecurityPermissionSetBuilder(); + } + + /** + * Append default all flag. + * + * @param dfltAllowAll Default allow all. + * @return SecurityPermissionSetBuilder refer to same permission builder. + */ + public SecurityPermissionSetBuilder defaultAllowAll(boolean dfltAllowAll) { + this.dfltAllowAll = dfltAllowAll; + + return this; + } + + /** + * Append permission set form {@link org.apache.ignite.IgniteCompute task} with {@code name}. + * + * @param name String for map some task to permission set. + * @param perms Permissions. + * @return SecurityPermissionSetBuilder refer to same permission builder. + */ + public SecurityPermissionSetBuilder appendTaskPermissions(String name, SecurityPermission... perms) { + validate(toCollection("TASK_"), perms); + + append(taskPerms, name, toCollection(perms)); + + return this; + } + + /** + * Append permission set form {@link org.apache.ignite.IgniteCache cache} with {@code name}. + * + * @param name String for map some cache to permission set. + * @param perms Permissions. + * @return {@link SecurityPermissionSetBuilder} refer to same permission builder. + */ + public SecurityPermissionSetBuilder appendCachePermissions(String name, SecurityPermission... perms) { + validate(toCollection("CACHE_"), perms); + + append(cachePerms, name, toCollection(perms)); + + return this; + } + + /** + * Append system permission set. + * + * @param perms Permission. + * @return {@link SecurityPermissionSetBuilder} refer to same permission builder. + */ + public SecurityPermissionSetBuilder appendSystemPermissions(SecurityPermission... perms) { + validate(toCollection("EVENTS_", "ADMIN_"), perms); + + sysPerms.addAll(toCollection(perms)); + + return this; + } + + /** + * Validate method use patterns. + * + * @param ptrns Pattern. + * @param perms Permissions. + */ + private void validate(Collection ptrns, SecurityPermission... perms) { + assert ptrns != null; + assert perms != null; + + for (SecurityPermission perm : perms) + validate(ptrns, perm); + } + + /** + * @param ptrns Patterns. + * @param perm Permission. + */ + private void validate(Collection ptrns, SecurityPermission perm) { + assert ptrns != null; + assert perm != null; + + boolean ex = true; + + String name = perm.name(); + + for (String ptrn : ptrns) { + if (name.startsWith(ptrn)) { + ex = false; + + break; + } + } + + if (ex) + throw new IgniteException("you can assign permission only start with " + ptrns + ", but you try " + name); + } + + /** + * Convert vararg to {@link Collection}. + * + * @param perms Permissions. + */ + @SafeVarargs + private final Collection toCollection(T... perms) { + assert perms != null; + + Collection col = new ArrayList<>(perms.length); + + Collections.addAll(col, perms); + + return col; + } + + /** + * @param permsMap Permissions map. + * @param name Name. + * @param perms Permission. + */ + private void append( + Map> permsMap, + String name, + Collection perms + ) { + assert permsMap != null; + assert name != null; + assert perms != null; + + Collection col = permsMap.get(name); + + if (col == null) + permsMap.put(name, perms); + else + col.addAll(perms); + } + + /** + * Builds the {@link SecurityPermissionSet}. + * + * @return {@link SecurityPermissionSet} instance. + */ + public SecurityPermissionSet build() { + SecurityBasicPermissionSet permSet = new SecurityBasicPermissionSet(); + + permSet.setDefaultAllowAll(dfltAllowAll); + permSet.setCachePermissions(unmodifiableMap(cachePerms)); + permSet.setTaskPermissions(unmodifiableMap(taskPerms)); + permSet.setSystemPermissions(unmodifiableList(sysPerms)); + + return permSet; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java b/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java new file mode 100644 index 0000000000000..1d951cf7cb161 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java @@ -0,0 +1,93 @@ +package org.apache.ignite.plugin.security; + +import java.util.Map; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Collection; +import java.util.concurrent.Callable; +import org.apache.ignite.IgniteException; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_PUT; +import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_READ; +import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_REMOVE; +import static org.apache.ignite.plugin.security.SecurityPermission.TASK_CANCEL; +import static org.apache.ignite.plugin.security.SecurityPermission.TASK_EXECUTE; +import static org.apache.ignite.plugin.security.SecurityPermission.EVENTS_ENABLE; +import static org.apache.ignite.plugin.security.SecurityPermission.ADMIN_VIEW; +import static org.apache.ignite.testframework.GridTestUtils.assertThrows; + +/** + * Test for check correct work {@link SecurityPermissionSetBuilder permission builder} + */ +public class SecurityPermissionSetBuilderTest extends GridCommonAbstractTest { + /** + * + */ + public void testPermissionBuilder() { + SecurityBasicPermissionSet exp = new SecurityBasicPermissionSet(); + + Map> permCache = new HashMap<>(); + permCache.put("cache1", Arrays.asList(CACHE_PUT, CACHE_REMOVE)); + permCache.put("cache2", Arrays.asList(CACHE_READ)); + + exp.setCachePermissions(permCache); + + Map> permTask = new HashMap<>(); + permTask.put("task1", Arrays.asList(TASK_CANCEL)); + permTask.put("task2", Arrays.asList(TASK_EXECUTE)); + + exp.setTaskPermissions(permTask); + + exp.setSystemPermissions(Arrays.asList(ADMIN_VIEW, EVENTS_ENABLE)); + + final SecurityPermissionSetBuilder permsBuilder = new SecurityPermissionSetBuilder(); + + assertThrows(log, new Callable() { + @Override + public Object call() throws Exception { + permsBuilder.appendCachePermissions("cache", ADMIN_VIEW); + return null; + } + }, IgniteException.class, + "you can assign permission only start with [CACHE_], but you try ADMIN_VIEW" + ); + + assertThrows(log, new Callable() { + @Override + public Object call() throws Exception { + permsBuilder.appendTaskPermissions("task", CACHE_READ); + return null; + } + }, IgniteException.class, + "you can assign permission only start with [TASK_], but you try CACHE_READ" + ); + + assertThrows(log, new Callable() { + @Override + public Object call() throws Exception { + permsBuilder.appendSystemPermissions(TASK_EXECUTE, CACHE_PUT); + return null; + } + }, IgniteException.class, + "you can assign permission only start with [EVENTS_, ADMIN_], but you try TASK_EXECUTE" + ); + + permsBuilder.appendCachePermissions( + "cache1", CACHE_PUT, CACHE_REMOVE + ).appendCachePermissions( + "cache2", CACHE_READ + ).appendTaskPermissions( + "task1", TASK_CANCEL + ).appendTaskPermissions( + "task2", TASK_EXECUTE + ).appendSystemPermissions(ADMIN_VIEW, EVENTS_ENABLE); + + SecurityPermissionSet actual = permsBuilder.build(); + + assertEquals(exp.cachePermissions(), actual.cachePermissions()); + assertEquals(exp.taskPermissions(), actual.taskPermissions()); + assertEquals(exp.systemPermissions(), actual.systemPermissions()); + assertEquals(exp.defaultAllowAll(), actual.defaultAllowAll()); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java index 62c2eb3b7a383..6ab0885976f10 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java @@ -53,6 +53,7 @@ import org.apache.ignite.messaging.GridMessagingNoPeerClassLoadingSelfTest; import org.apache.ignite.messaging.GridMessagingSelfTest; import org.apache.ignite.messaging.IgniteMessagingWithClientTest; +import org.apache.ignite.plugin.security.SecurityPermissionSetBuilderTest; import org.apache.ignite.spi.GridSpiLocalHostInjectionTest; import org.apache.ignite.startup.properties.NotStringSystemPropertyTest; import org.apache.ignite.testframework.GridTestUtils; @@ -143,6 +144,8 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(MarshallerContextLockingSelfTest.class); + suite.addTestSuite(SecurityPermissionSetBuilderTest.class); + return suite; } } From df670c7d64046d282c053f296c47a4743c58c8b1 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 7 Nov 2016 12:40:00 +0300 Subject: [PATCH 314/487] IGNITE-4118 .NET: Optimistic transaction example This closes #1200 --- .../Apache.Ignite.Examples.csproj | 1 + .../Datagrid/OptimisticTransactionExample.cs | 110 ++++++++++++++++++ 2 files changed, 111 insertions(+) create mode 100644 modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/OptimisticTransactionExample.cs diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj index 9b1d3230d4744..2ee91da46d189 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj @@ -52,6 +52,7 @@ + diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/OptimisticTransactionExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/OptimisticTransactionExample.cs new file mode 100644 index 0000000000000..2f9d16493fc72 --- /dev/null +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/OptimisticTransactionExample.cs @@ -0,0 +1,110 @@ +/* + * 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. + */ + +using System; +using System.Threading; +using System.Threading.Tasks; +using Apache.Ignite.Core; +using Apache.Ignite.Core.Cache; +using Apache.Ignite.Core.Cache.Configuration; +using Apache.Ignite.Core.Transactions; + +namespace Apache.Ignite.Examples.Datagrid +{ + /// + /// This example demonstrates optimistic transaction concurrency control. + /// + /// 1) Build the project Apache.Ignite.ExamplesDll (select it -> right-click -> Build). + /// Apache.Ignite.ExamplesDll.dll must appear in %IGNITE_HOME%/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/bin/${Platform]/${Configuration} folder. + /// 2) Set this class as startup object (Apache.Ignite.Examples project -> right-click -> Properties -> + /// Application -> Startup object); + /// 3) Start example (F5 or Ctrl+F5). + /// + /// This example can be run with standalone Apache Ignite.NET node: + /// 1) Run %IGNITE_HOME%/platforms/dotnet/bin/Apache.Ignite.exe: + /// Apache.Ignite.exe -configFileName=platforms\dotnet\examples\apache.ignite.examples\app.config -assembly=[path_to_Apache.Ignite.ExamplesDll.dll] + /// 2) Start example. + /// + public class OptimisticTransactionExample + { + /// Cache name. + private const string CacheName = "dotnet_optimistic_tx_example"; + + [STAThread] + public static void Main() + { + using (var ignite = Ignition.StartFromApplicationConfiguration()) + { + Console.WriteLine(); + Console.WriteLine(">>> Optimistic transaction example started."); + + // Create Transactional cache. + var cacheCfg = new CacheConfiguration(CacheName) { AtomicityMode = CacheAtomicityMode.Transactional }; + + var cache = ignite.GetOrCreateCache(cacheCfg); + + // Put a value. + cache[1] = 0; + + // Increment a value in parallel within a transaction. + var task1 = Task.Factory.StartNew(() => IncrementCacheValue(cache, 1)); + var task2 = Task.Factory.StartNew(() => IncrementCacheValue(cache, 2)); + + Task.WaitAll(task1, task2); + + Console.WriteLine(); + Console.WriteLine(">>> Resulting value in cache: " + cache[1]); + + Console.WriteLine(); + Console.WriteLine(">>> Example finished, press any key to exit ..."); + Console.ReadKey(); + } + } + + /// + /// Increments the cache value within a transaction. + /// + /// The cache. + /// The thread identifier. + private static void IncrementCacheValue(ICache cache, int threadId) + { + try + { + var transactions = cache.Ignite.GetTransactions(); + + using (var tx = transactions.TxStart(TransactionConcurrency.Optimistic, + TransactionIsolation.Serializable)) + { + // Increment cache value. + cache[1]++; + + // Introduce a delay to ensure lock conflict. + Thread.Sleep(TimeSpan.FromSeconds(2.5)); + + tx.Commit(); + } + + Console.WriteLine("\n>>> Thread {0} successfully incremented cached value.", threadId); + } + catch (TransactionOptimisticException ex) + { + Console.WriteLine("\n>>> Thread {0} failed to increment cached value. " + + "Caught an expected optimistic exception: {1}", threadId, ex.Message); + } + } + } +} \ No newline at end of file From 474f22fda4c7cf4d7b2623c451cd7c10f0d8c636 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 7 Nov 2016 12:55:20 +0300 Subject: [PATCH 315/487] IGNITE-4119 .NET: add TransactionDeadlockException --- .../platform/cache/PlatformCache.java | 12 ++++ .../Cache/CacheAbstractTest.cs | 44 +++++++++++- .../ExceptionsTest.cs | 3 + .../Apache.Ignite.Core.csproj | 3 +- .../Apache.Ignite.Core/Impl/ExceptionUtils.cs | 1 + .../TransactionDeadlockException.cs | 71 +++++++++++++++++++ 6 files changed, 132 insertions(+), 2 deletions(-) create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionDeadlockException.cs diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index be50e5d8f8366..6f23682955a60 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -58,6 +58,8 @@ import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.transactions.TransactionDeadlockException; +import org.apache.ignite.transactions.TransactionTimeoutException; import org.jetbrains.annotations.Nullable; import javax.cache.Cache; @@ -1095,6 +1097,16 @@ private void loadCache0(BinaryRawReaderEx reader, boolean loc, IgniteCache cache if (e.getCause() instanceof EntryProcessorException) return (Exception)e.getCause(); + TransactionDeadlockException deadlockException = X.cause(e, TransactionDeadlockException.class); + + if (deadlockException != null) + return deadlockException; + + TransactionTimeoutException timeoutException = X.cause(e, TransactionTimeoutException.class); + + if (timeoutException != null) + return timeoutException; + return super.convertException(e); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs index 9fd1f1de5282b..63e236a070418 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs @@ -2458,7 +2458,49 @@ public void TestTxStateAndExceptions() // Expected } } - + + /// + /// Tests the transaction deadlock detection. + /// + [Test] + public void TestTxDeadlockDetection() + { + if (!TxEnabled()) + return; + + var cache = Cache(); + + var keys0 = Enumerable.Range(1, 100).ToArray(); + + cache.PutAll(keys0.ToDictionary(x => x, x => x)); + + var barrier = new Barrier(2); + + Action increment = keys => + { + using (var tx = Transactions.TxStart(TransactionConcurrency.Pessimistic, + TransactionIsolation.RepeatableRead, TimeSpan.FromSeconds(0.5), 0)) + { + foreach (var key in keys) + cache[key]++; + + barrier.SignalAndWait(500); + + tx.Commit(); + } + }; + + // Increment keys within tx in different order to cause a deadlock. + var aex = Assert.Throws(() => + Task.WaitAll(Task.Factory.StartNew(() => increment(keys0)), + Task.Factory.StartNew(() => increment(keys0.Reverse().ToArray())))); + + Assert.AreEqual(2, aex.InnerExceptions.Count); + + var deadlockEx = aex.InnerExceptions.OfType().First(); + Assert.IsTrue(deadlockEx.Message.Trim().StartsWith("Deadlock detected:"), deadlockEx.Message); + } + /// /// Test thraed-locals leak. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs index e766f5a4f4fa1..052ff6ffc730d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs @@ -65,6 +65,8 @@ public void TestExceptions() var e = Assert.Throws(() => grid.GetCluster().ForRemotes().GetMetrics()); + Assert.IsNotNull(e.InnerException); + Assert.IsTrue(e.InnerException.Message.StartsWith( "class org.apache.ignite.cluster.ClusterGroupEmptyException: Cluster group is empty.")); @@ -163,6 +165,7 @@ public void TestAllExceptionsConstructors() ex = (Exception) msgCauseCtor.Invoke(new object[] {"myMessage", new Exception("innerEx")}); Assert.AreEqual("myMessage", ex.Message); + Assert.IsNotNull(ex.InnerException); Assert.AreEqual("innerEx", ex.InnerException.Message); } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj index 2973bb3bf2760..66253f6c700bc 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj @@ -75,7 +75,7 @@ true AnyCPU Apache.Ignite.Core.ruleset - none + none @@ -477,6 +477,7 @@ + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs index a59ca5f31d327..ddbdd86fc559b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs @@ -97,6 +97,7 @@ static ExceptionUtils() Exs["org.apache.ignite.transactions.TransactionTimeoutException"] = (i, m, e) => new TransactionTimeoutException(m, e); Exs["org.apache.ignite.transactions.TransactionRollbackException"] = (i, m, e) => new TransactionRollbackException(m, e); Exs["org.apache.ignite.transactions.TransactionHeuristicException"] = (i, m, e) => new TransactionHeuristicException(m, e); + Exs["org.apache.ignite.transactions.TransactionDeadlockException"] = (i, m, e) => new TransactionDeadlockException(m, e); // Security exceptions. Exs["org.apache.ignite.IgniteAuthenticationException"] = (i, m, e) => new SecurityException(m, e); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionDeadlockException.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionDeadlockException.cs new file mode 100644 index 0000000000000..b0ba5de397d11 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionDeadlockException.cs @@ -0,0 +1,71 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Transactions +{ + using System; + using System.Runtime.Serialization; + using Apache.Ignite.Core.Common; + + /// + /// Indicates a deadlock within Ignite transaction. + /// + /// This exception can be thrown from any cache method that modifies or reads data within a transaction + /// with timeout (see + /// overload). + /// + [Serializable] + public class TransactionDeadlockException : IgniteException + { + /// + /// Initializes a new instance of the class. + /// + public TransactionDeadlockException() + { + // No-op. + } + + /// + /// Initializes a new instance of the class. + /// + /// The message that describes the error. + public TransactionDeadlockException(string message) : base(message) + { + // No-op. + } + + /// + /// Initializes a new instance of the class. + /// + /// The message. + /// The cause. + public TransactionDeadlockException(string message, Exception cause) : base(message, cause) + { + // No-op. + } + + /// + /// Initializes a new instance of the class. + /// + /// Serialization information. + /// Streaming context. + protected TransactionDeadlockException(SerializationInfo info, StreamingContext ctx) : base(info, ctx) + { + // No-op. + } + } +} From fc7ce5a4d72145f2e8a86debeda264ef0a5b37e3 Mon Sep 17 00:00:00 2001 From: isapego Date: Mon, 7 Nov 2016 13:26:05 +0300 Subject: [PATCH 316/487] IGNITE-4090: Added flags so stdint and limits can be used in C++. --- modules/platforms/cpp/binary/Makefile.am | 4 +++- modules/platforms/cpp/core/Makefile.am | 4 +++- modules/platforms/cpp/examples/odbc-example/Makefile.am | 4 +++- modules/platforms/cpp/examples/putget-example/Makefile.am | 4 +++- modules/platforms/cpp/examples/query-example/Makefile.am | 4 +++- modules/platforms/cpp/ignite/Makefile.am | 4 +++- modules/platforms/cpp/jni/Makefile.am | 4 +++- modules/platforms/cpp/odbc/Makefile.am | 4 +++- 8 files changed, 24 insertions(+), 8 deletions(-) diff --git a/modules/platforms/cpp/binary/Makefile.am b/modules/platforms/cpp/binary/Makefile.am index 4876776c811b2..ca301a6395434 100644 --- a/modules/platforms/cpp/binary/Makefile.am +++ b/modules/platforms/cpp/binary/Makefile.am @@ -26,7 +26,9 @@ AM_CPPFLAGS = \ -I$(srcdir)/include \ -I@top_srcdir@/common/include \ -I@top_srcdir@/common/os/linux/include \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/core/Makefile.am b/modules/platforms/cpp/core/Makefile.am index bbb77203da624..97523cf64992f 100644 --- a/modules/platforms/cpp/core/Makefile.am +++ b/modules/platforms/cpp/core/Makefile.am @@ -31,7 +31,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/examples/odbc-example/Makefile.am b/modules/platforms/cpp/examples/odbc-example/Makefile.am index e58410527a4f4..83cc63e207366 100644 --- a/modules/platforms/cpp/examples/odbc-example/Makefile.am +++ b/modules/platforms/cpp/examples/odbc-example/Makefile.am @@ -30,7 +30,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/../jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/examples/putget-example/Makefile.am b/modules/platforms/cpp/examples/putget-example/Makefile.am index 5301ea1a0aaeb..cf39002f7ae54 100644 --- a/modules/platforms/cpp/examples/putget-example/Makefile.am +++ b/modules/platforms/cpp/examples/putget-example/Makefile.am @@ -30,7 +30,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/../jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/examples/query-example/Makefile.am b/modules/platforms/cpp/examples/query-example/Makefile.am index 14373034ae446..01231ec38f977 100644 --- a/modules/platforms/cpp/examples/query-example/Makefile.am +++ b/modules/platforms/cpp/examples/query-example/Makefile.am @@ -30,7 +30,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/../jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/ignite/Makefile.am b/modules/platforms/cpp/ignite/Makefile.am index 625f1df670e45..2dbc4d614e23f 100644 --- a/modules/platforms/cpp/ignite/Makefile.am +++ b/modules/platforms/cpp/ignite/Makefile.am @@ -30,7 +30,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/jni/Makefile.am b/modules/platforms/cpp/jni/Makefile.am index b9b39133d1e09..2cb4b90d8afa4 100644 --- a/modules/platforms/cpp/jni/Makefile.am +++ b/modules/platforms/cpp/jni/Makefile.am @@ -29,7 +29,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/common/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am index 3c8b37a5e97f2..b0cc5f87133fc 100644 --- a/modules/platforms/cpp/odbc/Makefile.am +++ b/modules/platforms/cpp/odbc/Makefile.am @@ -27,7 +27,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/common/include \ -I@top_srcdir@/common/os/linux/include \ -I@top_srcdir@/binary/include \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ From a98804a249496ba9bafbc96daa7aaf25b3d36724 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Mon, 7 Nov 2016 14:00:00 +0300 Subject: [PATCH 317/487] IGNITE-4113: Added tests. Added Statement::Set/GetAttribute. --- .../cpp/odbc-test/src/api_robustness_test.cpp | 63 ++++++++ .../cpp/odbc-test/src/queries_test.cpp | 9 +- .../cpp/odbc/include/ignite/odbc/statement.h | 42 +++++ modules/platforms/cpp/odbc/src/odbc.cpp | 116 +------------- modules/platforms/cpp/odbc/src/statement.cpp | 151 ++++++++++++++++++ 5 files changed, 268 insertions(+), 113 deletions(-) diff --git a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp index 008cf25f26c7e..fbd5f12532c28 100644 --- a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp @@ -160,6 +160,54 @@ struct ApiRobustnessTestSuiteFixture testCache = grid.GetCache("cache"); } + /** + * Check that SQLFetchScroll does not crash with unsupported orientation. + * + * @param orientation Fetch orientation. + */ + void CheckFetchScrollUnsupportedOrientation(SQLUSMALLINT orientation) + { + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); + + SQLRETURN ret; + + const int64_t recordsNum = 100; + + for (int i = 0; i < recordsNum; ++i) + { + TestType val; + + val.i32Field = i * 10; + + testCache.Put(i, val); + } + + int32_t i32Field = -1; + + // Binding column. + ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &i32Field, 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + SQLCHAR request[] = "SELECT i32Field FROM TestType ORDER BY _key"; + + ret = SQLExecDirect(stmt, request, SQL_NTS); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFetchScroll(stmt, SQL_FETCH_NEXT, 0); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_EQUAL(i32Field, 0); + + ret = SQLFetchScroll(stmt, orientation, 0); + + // Operation is not supported. However, there should be no crash. + BOOST_CHECK(ret == SQL_ERROR); + } + /** * Destructor. */ @@ -1003,4 +1051,19 @@ BOOST_AUTO_TEST_CASE(TestSQLSpecialColumns) SQLCloseCursor(stmt); } +BOOST_AUTO_TEST_CASE(TestFetchScrollLast) +{ + CheckFetchScrollUnsupportedOrientation(SQL_FETCH_LAST); +} + +BOOST_AUTO_TEST_CASE(TestFetchScrollPrior) +{ + CheckFetchScrollUnsupportedOrientation(SQL_FETCH_PRIOR); +} + +BOOST_AUTO_TEST_CASE(TestFetchScrollFirst) +{ + CheckFetchScrollUnsupportedOrientation(SQL_FETCH_FIRST); +} + BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index eb6e153c71e85..82e9972b5ca2d 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -200,6 +200,8 @@ struct QueriesTestSuiteFixture "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType"; ret = SQLExecDirect(stmt, reinterpret_cast(request), SQL_NTS); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); if (!SQL_SUCCEEDED(ret)) BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); @@ -364,6 +366,8 @@ BOOST_AUTO_TEST_CASE(TestTwoRowsString) "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType"; ret = SQLExecDirect(stmt, request, SQL_NTS); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); if (!SQL_SUCCEEDED(ret)) BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); @@ -460,6 +464,8 @@ BOOST_AUTO_TEST_CASE(TestOneRowString) "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType"; ret = SQLExecDirect(stmt, request, SQL_NTS); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); ret = SQLFetch(stmt); if (!SQL_SUCCEEDED(ret)) @@ -522,6 +528,8 @@ BOOST_AUTO_TEST_CASE(TestOneRowStringLen) "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType"; ret = SQLExecDirect(stmt, request, SQL_NTS); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); ret = SQLFetch(stmt); if (!SQL_SUCCEEDED(ret)) @@ -666,5 +674,4 @@ BOOST_AUTO_TEST_CASE(TestDataAtExecution) BOOST_CHECK(ret == SQL_NO_DATA); } - BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h index 35f1e98b4f140..db6205ed3d307 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h @@ -117,6 +117,25 @@ namespace ignite */ void UnbindAllParameters(); + /** + * Set statement attribute. + * + * @param attr Attribute type. + * @param value Value pointer. + * @param valueLen Value length. + */ + void SetAttribute(int attr, void* value, SQLINTEGER valueLen); + + /** + * Get statement attribute. + * + * @param attr Attribute type. + * @param buf Buffer for value. + * @param bufLen Buffer length. + * @param valueLen Resulting value length. + */ + void GetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER *valueLen); + /** * Get number of binded parameters. * @@ -355,6 +374,29 @@ namespace ignite */ SqlResult InternalBindParameter(uint16_t paramIdx, const app::Parameter& param); + /** + * Set statement attribute. + * Internal call. + * + * @param attr Attribute type. + * @param value Value pointer. + * @param valueLen Value length. + * @return Operation result. + */ + SqlResult InternalSetAttribute(int attr, void* value, SQLINTEGER valueLen); + + /** + * Get statement attribute. + * Internal call. + * + * @param attr Attribute type. + * @param buf Buffer for value. + * @param bufLen Buffer length. + * @param valueLen Resulting value length. + * @return Operation result. + */ + SqlResult InternalGetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER* valueLen); + /** * Get value of the column in the result set. * diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp index 7416ad2b5b663..612d51a5b4b88 100644 --- a/modules/platforms/cpp/odbc/src/odbc.cpp +++ b/modules/platforms/cpp/odbc/src/odbc.cpp @@ -854,73 +854,9 @@ namespace ignite if (!statement) return SQL_INVALID_HANDLE; - if (!valueBuf) - return SQL_ERROR; - - switch (attr) - { - case SQL_ATTR_APP_ROW_DESC: - case SQL_ATTR_APP_PARAM_DESC: - case SQL_ATTR_IMP_ROW_DESC: - case SQL_ATTR_IMP_PARAM_DESC: - { - SQLPOINTER *val = reinterpret_cast(valueBuf); - - *val = static_cast(stmt); - - break; - } - - case SQL_ATTR_ROW_ARRAY_SIZE: - { - SQLINTEGER *val = reinterpret_cast(valueBuf); - - *val = static_cast(1); - - break; - } - - case SQL_ATTR_ROWS_FETCHED_PTR: - { - SQLULEN** val = reinterpret_cast(valueBuf); - - *val = reinterpret_cast(statement->GetRowsFetchedPtr()); - - break; - } + statement->GetAttribute(attr, valueBuf, valueBufLen, valueResLen); - case SQL_ATTR_ROW_STATUS_PTR: - { - SQLUSMALLINT** val = reinterpret_cast(valueBuf); - - *val = reinterpret_cast(statement->GetRowStatusesPtr()); - - break; - } - - case SQL_ATTR_PARAM_BIND_OFFSET_PTR: - { - SQLULEN** val = reinterpret_cast(valueBuf); - - *val = reinterpret_cast(statement->GetParamBindOffsetPtr()); - - break; - } - - case SQL_ATTR_ROW_BIND_OFFSET_PTR: - { - SQLULEN** val = reinterpret_cast(valueBuf); - - *val = reinterpret_cast(statement->GetColumnBindOffsetPtr()); - - break; - } - - default: - return SQL_ERROR; - } - - return SQL_SUCCESS; + return statement->GetDiagnosticRecords().GetReturnCode(); } SQLRETURN SQLSetStmtAttr(SQLHSTMT stmt, @@ -943,53 +879,9 @@ namespace ignite if (!statement) return SQL_INVALID_HANDLE; - switch (attr) - { - case SQL_ATTR_ROW_ARRAY_SIZE: - { - SQLULEN val = reinterpret_cast(value); - - LOG_MSG("Value: %d\n", val); - - if (val != 1) - return SQL_ERROR; - - break; - } - - case SQL_ATTR_ROWS_FETCHED_PTR: - { - statement->SetRowsFetchedPtr(reinterpret_cast(value)); - - break; - } - - case SQL_ATTR_ROW_STATUS_PTR: - { - statement->SetRowStatusesPtr(reinterpret_cast(value)); - - break; - } - - case SQL_ATTR_PARAM_BIND_OFFSET_PTR: - { - statement->SetParamBindOffsetPtr(reinterpret_cast(value)); - - break; - } - - case SQL_ATTR_ROW_BIND_OFFSET_PTR: - { - statement->SetColumnBindOffsetPtr(reinterpret_cast(value)); + statement->SetAttribute(attr, value, valueLen); - break; - } - - default: - return SQL_ERROR; - } - - return SQL_SUCCESS; + return statement->GetDiagnosticRecords().GetReturnCode(); } SQLRETURN SQLPrimaryKeys(SQLHSTMT stmt, diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp index 32f7c3f0c33d2..f1a577a2adde3 100644 --- a/modules/platforms/cpp/odbc/src/statement.cpp +++ b/modules/platforms/cpp/odbc/src/statement.cpp @@ -145,6 +145,157 @@ namespace ignite paramBindings.clear(); } + void Statement::SetAttribute(int attr, void* value, SQLINTEGER valueLen) + { + IGNITE_ODBC_API_CALL(InternalSetAttribute(attr, value, valueLen)); + } + + SqlResult Statement::InternalSetAttribute(int attr, void* value, SQLINTEGER valueLen) + { + switch (attr) + { + case SQL_ATTR_ROW_ARRAY_SIZE: + { + SQLULEN val = reinterpret_cast(value); + + LOG_MSG("SQL_ATTR_ROW_ARRAY_SIZE: %d\n", val); + + if (val != 1) + { + AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, + "Fetching of more than one row by call is not supported."); + + return SQL_RESULT_ERROR; + } + + break; + } + + case SQL_ATTR_ROWS_FETCHED_PTR: + { + SetRowsFetchedPtr(reinterpret_cast(value)); + + break; + } + + case SQL_ATTR_ROW_STATUS_PTR: + { + SetRowStatusesPtr(reinterpret_cast(value)); + + break; + } + + case SQL_ATTR_PARAM_BIND_OFFSET_PTR: + { + SetParamBindOffsetPtr(reinterpret_cast(value)); + + break; + } + + case SQL_ATTR_ROW_BIND_OFFSET_PTR: + { + SetColumnBindOffsetPtr(reinterpret_cast(value)); + + break; + } + + default: + { + AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, + "Specified attribute is not supported."); + + return SQL_RESULT_ERROR; + } + } + + return SQL_RESULT_SUCCESS; + } + + void Statement::GetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER* valueLen) + { + IGNITE_ODBC_API_CALL(InternalGetAttribute(attr, buf, bufLen, valueLen)); + } + + SqlResult Statement::InternalGetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER* valueLen) + { + if (!buf) + { + AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, "Data buffer is NULL."); + + return SQL_RESULT_ERROR; + } + + switch (attr) + { + case SQL_ATTR_APP_ROW_DESC: + case SQL_ATTR_APP_PARAM_DESC: + case SQL_ATTR_IMP_ROW_DESC: + case SQL_ATTR_IMP_PARAM_DESC: + { + SQLPOINTER *val = reinterpret_cast(buf); + + *val = static_cast(this); + + break; + } + + case SQL_ATTR_ROW_ARRAY_SIZE: + { + SQLINTEGER *val = reinterpret_cast(buf); + + *val = static_cast(1); + + break; + } + + case SQL_ATTR_ROWS_FETCHED_PTR: + { + SQLULEN** val = reinterpret_cast(buf); + + *val = reinterpret_cast(GetRowsFetchedPtr()); + + break; + } + + case SQL_ATTR_ROW_STATUS_PTR: + { + SQLUSMALLINT** val = reinterpret_cast(buf); + + *val = reinterpret_cast(GetRowStatusesPtr()); + + break; + } + + case SQL_ATTR_PARAM_BIND_OFFSET_PTR: + { + SQLULEN** val = reinterpret_cast(buf); + + *val = reinterpret_cast(GetParamBindOffsetPtr()); + + break; + } + + case SQL_ATTR_ROW_BIND_OFFSET_PTR: + { + SQLULEN** val = reinterpret_cast(buf); + + *val = reinterpret_cast(GetColumnBindOffsetPtr()); + + break; + } + + default: + { + AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, + "Specified attribute is not supported."); + + return SQL_RESULT_ERROR; + } + } + + return SQL_RESULT_SUCCESS; + } + uint16_t Statement::GetParametersNumber() { IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS; From b1c7c9bb95c900083702d0ba0362edf3aea5a7b4 Mon Sep 17 00:00:00 2001 From: sboikov Date: Mon, 7 Nov 2016 15:40:36 +0300 Subject: [PATCH 318/487] GG-11360 - Implement SQL queries cancellation Fix for commit 80abd1b: for distributed joins need always send cancel request. --- .../h2/twostep/GridReduceQueryExecutor.java | 25 +++++++++++++------ 1 file changed, 18 insertions(+), 7 deletions(-) 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 3847373ea14a0..3b851a6a63e15 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 @@ -756,7 +756,7 @@ public Iterator> query( } finally { // Make sure any activity related to current attempt is cancelled. - cancelRemoteQueriesIfNeeded(nodes, r, qryReqId); + cancelRemoteQueriesIfNeeded(nodes, r, qryReqId, qry.distributedJoins()); if (!runs.remove(qryReqId, r)) U.warn(log, "Query run was already removed: " + qryReqId); @@ -793,15 +793,26 @@ private boolean wasCancelled(CacheException e) { } /** + * @param nodes Query nodes. * @param r Query run. * @param qryReqId Query id. + * @param distributedJoins Distributed join flag. */ - private void cancelRemoteQueriesIfNeeded(Collection nodes, QueryRun r, long qryReqId) { - for (GridMergeIndex idx : r.idxs) { - if (!idx.fetchedAll()) { - send(nodes, new GridQueryCancelRequest(qryReqId), null, false); - - break; + private void cancelRemoteQueriesIfNeeded(Collection nodes, + QueryRun r, + long qryReqId, + boolean distributedJoins) + { + // For distributedJoins need always send cancel request to cleanup resources. + if (distributedJoins) + send(nodes, new GridQueryCancelRequest(qryReqId), null, false); + else { + for (GridMergeIndex idx : r.idxs) { + if (!idx.fetchedAll()) { + send(nodes, new GridQueryCancelRequest(qryReqId), null, false); + + break; + } } } } From 319014de075c80fb15e58172cc24e35ce16b56cf Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 7 Nov 2016 17:53:40 +0300 Subject: [PATCH 319/487] IGNITE-4132 .NET: Improve BinaryConfiguration documentation --- .../dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs | 5 ++++- .../dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs index 3a9d86c4f78db..51df90786613c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs @@ -88,7 +88,10 @@ public BinaryConfiguration(params Type[] binaryTypes) public ICollection TypeConfigurations { get; set; } /// - /// Binarizable types. Shorthand for creating . + /// Gets or sets a collection of assembly-qualified type names + /// (the result of ) for binarizable types. + /// + /// Shorthand for creating . /// [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")] public ICollection Types { get; set; } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd index 8764b60e54497..e9fb991853de9 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd @@ -183,7 +183,7 @@ - Binarizable types. Shorthand for creating "BinaryTypeConfiguration. + Assembly-qualified type names (Type.AssemblyQualifiedName) for binarizable types. Shorthand for creating BinaryTypeConfiguration. From 950bad474ef29f9b808e74034c49a69d57eb2740 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 8 Nov 2016 14:03:34 +0300 Subject: [PATCH 320/487] GG-11655 - Restore service compatibility with releases before 1.5.30. --- .../processors/job/GridJobWorker.java | 10 ++- .../service/GridServiceProcessor.java | 61 ++++++++++++- .../internal/util/SerializableTransient.java | 58 ++++++++++++ .../ignite/marshaller/MarshallerUtils.java | 22 +++++ .../optimized/OptimizedClassDescriptor.java | 90 ++++++++++++++++++- 5 files changed, 234 insertions(+), 7 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/util/SerializableTransient.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java index 8169eb1101f4f..5f38b29025584 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java @@ -57,6 +57,7 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.marshaller.MarshallerUtils; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.events.EventType.EVT_JOB_CANCELLED; @@ -421,7 +422,14 @@ boolean initialize(GridDeployment dep, Class taskCls) { try { if (job == null) { - job = U.unmarshal(marsh, jobBytes, U.resolveClassLoader(dep.classLoader(), ctx.config())); + MarshallerUtils.jobSenderVersion(taskNode.version()); + + try { + job = U.unmarshal(marsh, jobBytes, U.resolveClassLoader(dep.classLoader(), ctx.config())); + } + finally { + MarshallerUtils.jobSenderVersion(null); + } // No need to hold reference any more. jobBytes = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 527d360e9578f..6c263638de7c5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -20,11 +20,14 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; @@ -87,6 +90,7 @@ import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.internal.util.SerializableTransient; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.JobContextResource; import org.apache.ignite.resources.LoggerResource; @@ -115,6 +119,9 @@ public class GridServiceProcessor extends GridProcessorAdapter { /** */ public static final IgniteProductVersion LAZY_SERVICES_CFG_SINCE = IgniteProductVersion.fromString("1.5.22"); + /** Versions that only compatible with each other, and from 1.5.33. */ + private static final Set SERVICE_TOP_CALLABLE_VER1; + /** */ private final Boolean srvcCompatibilitySysProp; @@ -162,6 +169,31 @@ public class GridServiceProcessor extends GridProcessorAdapter { /** Topology listener. */ private GridLocalEventListener topLsnr = new TopologyListener(); + static { + Set versions = new TreeSet<>(new Comparator() { + @Override public int compare(final IgniteProductVersion o1, final IgniteProductVersion o2) { + return o1.compareToIgnoreTimestamp(o2); + } + }); + + versions.add(IgniteProductVersion.fromString("1.5.30")); + versions.add(IgniteProductVersion.fromString("1.5.31")); + versions.add(IgniteProductVersion.fromString("1.5.32")); + versions.add(IgniteProductVersion.fromString("1.6.3")); + versions.add(IgniteProductVersion.fromString("1.6.4")); + versions.add(IgniteProductVersion.fromString("1.6.5")); + versions.add(IgniteProductVersion.fromString("1.6.6")); + versions.add(IgniteProductVersion.fromString("1.6.7")); + versions.add(IgniteProductVersion.fromString("1.6.8")); + versions.add(IgniteProductVersion.fromString("1.6.9")); + versions.add(IgniteProductVersion.fromString("1.6.10")); + versions.add(IgniteProductVersion.fromString("1.7.0")); + versions.add(IgniteProductVersion.fromString("1.7.1")); + versions.add(IgniteProductVersion.fromString("1.7.2")); + + SERVICE_TOP_CALLABLE_VER1 = Collections.unmodifiableSet(versions); + } + /** * @param ctx Kernal context. */ @@ -668,9 +700,13 @@ public Map serviceTopology(String name) throws IgniteCheckedExcep ClusterNode node = cache.affinity().mapKeyToNode(name); if (node.version().compareTo(ServiceTopologyCallable.SINCE_VER) >= 0) { + final ServiceTopologyCallable call = new ServiceTopologyCallable(name); + + call.serialize = SERVICE_TOP_CALLABLE_VER1.contains(node.version()); + return ctx.closure().callAsyncNoFailover( GridClosureCallMode.BROADCAST, - new ServiceTopologyCallable(name), + call, Collections.singletonList(node), false ).get(); @@ -1829,6 +1865,7 @@ static class ServiceAssignmentsPredicate implements IgniteBiPredicate> { /** */ private static final long serialVersionUID = 0L; @@ -1836,11 +1873,14 @@ private static class ServiceTopologyCallable implements IgniteCallable + * Works only for jobs. For other messages node version is not available. + *

          + */ +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.TYPE) +public @interface SerializableTransient { + /** + * Name of the private static method that returns list of transient fields + * that should be serialized (String[]), and accepts itself (before serialization) + * and {@link IgniteProductVersion}, e.g. + *
          +     *     private static String[] fields(Object self, IgniteProductVersion ver){
          +     *         return ver.compareTo("1.5.30") > 0 ? SERIALIZABLE_FIELDS : null;
          +     *     }
          +     * 
          + *

          + * On serialization version argument ver is null, on deserialization - self is null. + *

          + *

          + * If it returns empty array or null all transient fields will be normally + * ignored. + *

          + * + * @return Name of the method. + */ + String methodName(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java index 9668bafb1c7ff..ad63702de00d5 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java @@ -17,6 +17,7 @@ package org.apache.ignite.marshaller; +import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.jetbrains.annotations.Nullable; @@ -24,6 +25,9 @@ * Utility marshaller methods. */ public class MarshallerUtils { + /** Job sender node version. */ + private static final ThreadLocal JOB_SND_NODE_VER = new ThreadLocal<>(); + /** * Set node name to marshaller context if possible. * @@ -55,4 +59,22 @@ public static JdkMarshaller jdkMarshaller(@Nullable String nodeName) { private MarshallerUtils() { // No-op. } + + /** + * Sets thread local job sender node version. + * + * @param ver Thread local job sender node version. + */ + public static void jobSenderVersion(IgniteProductVersion ver) { + JOB_SND_NODE_VER.set(ver); + } + + /** + * Returns thread local job sender node version. + * + * @return Thread local job sender node version. + */ + public static IgniteProductVersion jobSenderVersion() { + return JOB_SND_NODE_VER.get(); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java index 5a5b54d001f29..160f2c129de78 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java @@ -47,8 +47,11 @@ import java.util.concurrent.ConcurrentMap; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.marshaller.MarshallerContext; import org.apache.ignite.marshaller.MarshallerExclusions; +import org.apache.ignite.internal.util.SerializableTransient; +import org.apache.ignite.marshaller.MarshallerUtils; import static java.lang.reflect.Modifier.isFinal; import static java.lang.reflect.Modifier.isPrivate; @@ -166,6 +169,9 @@ class OptimizedClassDescriptor { /** Proxy interfaces. */ private Class[] proxyIntfs; + /** Method returns serializable transient fields. */ + private Method serTransMtd; + /** * Creates descriptor for class. * @@ -441,6 +447,27 @@ else if (Proxy.class.isAssignableFrom(cls)) { readObjMtds.add(mtd); + final SerializableTransient serTransAn = c.getAnnotation(SerializableTransient.class); + + // Custom serialization policy for transient fields. + if (serTransAn != null) { + try { + serTransMtd = c.getDeclaredMethod(serTransAn.methodName(), cls, IgniteProductVersion.class); + + int mod = serTransMtd.getModifiers(); + + if (isStatic(mod) && isPrivate(mod) + && serTransMtd.getReturnType() == String[].class) + serTransMtd.setAccessible(true); + else + // Set method back to null if it has incorrect signature. + serTransMtd = null; + } + catch (NoSuchMethodException ignored) { + serTransMtd = null; + } + } + Field[] clsFields0 = c.getDeclaredFields(); Map fieldNames = new HashMap<>(); @@ -797,7 +824,7 @@ void write(OptimizedObjectOutputStream out, Object obj) throws IOException { writeTypeData(out); out.writeShort(checksum); - out.writeSerializable(obj, writeObjMtds, fields); + out.writeSerializable(obj, writeObjMtds, serializableFields(obj.getClass(), obj, null)); break; @@ -806,6 +833,60 @@ void write(OptimizedObjectOutputStream out, Object obj) throws IOException { } } + /** + * Gets list of serializable fields. If {@link #serTransMtd} method + * returns list of transient fields, they will be added to other fields. + * Transient fields that are not included in that list will be normally + * ignored. + * + * @param cls Class. + * @param obj Object. + * @param ver Job sender version. + * @return Serializable fields. + */ + @SuppressWarnings("ForLoopReplaceableByForEach") + private Fields serializableFields(Class cls, Object obj, IgniteProductVersion ver) { + if (serTransMtd == null) + return fields; + + try { + final String[] transFields = (String[])serTransMtd.invoke(cls, obj, ver); + + if (transFields == null || transFields.length == 0) + return fields; + + List clsFields = new ArrayList<>(); + + clsFields.addAll(fields.fields.get(0).fields); + + for (int i = 0; i < transFields.length; i++) { + final String fieldName = transFields[i]; + + final Field f = cls.getDeclaredField(fieldName); + + FieldInfo fieldInfo = new FieldInfo(f, f.getName(), + GridUnsafe.objectFieldOffset(f), fieldType(f.getType())); + + clsFields.add(fieldInfo); + } + + Collections.sort(clsFields, new Comparator() { + @Override public int compare(FieldInfo t1, FieldInfo t2) { + return t1.name().compareTo(t2.name()); + } + }); + + List fields = new ArrayList<>(); + + fields.add(new ClassFields(clsFields)); + + return new Fields(fields); + } + catch (Exception e) { + return fields; + } + } + /** * @param out Output stream. * @throws IOException In case of error. @@ -838,7 +919,12 @@ Object read(OptimizedObjectInputStream in) throws ClassNotFoundException, IOExce case SERIALIZABLE: verifyChecksum(in.readShort()); - return in.readSerializable(cls, readObjMtds, readResolveMtd, fields); + // If no serialize method, then unmarshal as usual. + if (serTransMtd != null) + return in.readSerializable(cls, readObjMtds, readResolveMtd, + serializableFields(cls, null, MarshallerUtils.jobSenderVersion())); + else + return in.readSerializable(cls, readObjMtds, readResolveMtd, fields); default: assert false : "Unexpected type: " + type; From 1612b6d66fed032182a41e90da71e6b986ae087b Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 8 Nov 2016 14:07:54 +0300 Subject: [PATCH 321/487] .NET: Fix minor analysis warnings --- .../Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs | 2 +- .../dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs index f6a2f071e4711..86b1406b7ec55 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs @@ -62,7 +62,7 @@ public abstract class ComputeTaskSplitAdapter : Compute /// /// Split returned no jobs. [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")] - override public IDictionary, IClusterNode> Map(IList subgrid, TArg arg) + public override IDictionary, IClusterNode> Map(IList subgrid, TArg arg) { var jobs = Split(subgrid.Count, arg); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs index e54a199de29c6..a5e2cb008e393 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs @@ -24,7 +24,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged /// /// Base unmanaged target implementation. /// - internal unsafe sealed class UnmanagedTarget : CriticalHandle, IUnmanagedTarget + internal sealed unsafe class UnmanagedTarget : CriticalHandle, IUnmanagedTarget { /** Context. */ private readonly UnmanagedContext _ctx; From e821dc0083003bc81058b1cb223d8a8a2ee44daf Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Tue, 8 Nov 2016 15:09:21 +0300 Subject: [PATCH 322/487] IGNITE-2079 (revert commit) GridCacheIoManager eats exception trail if it falls into the directed case --- .../org/apache/ignite/events/EventType.java | 6 - .../events/UnhandledExceptionEvent.java | 61 ---- .../processors/cache/GridCacheIoManager.java | 70 ++-- .../cache/query/GridCacheQueryManager.java | 10 - .../query/GridCacheQueryMetricsAdapter.java | 7 - .../cache/query/GridCacheQueryResponse.java | 2 +- .../CacheContinuousQueryHandler.java | 50 ++- ...UnmarshallingContinuousQueryErrorTest.java | 302 ------------------ ...teCacheP2pUnmarshallingErrorTestSuite.java | 6 +- 9 files changed, 60 insertions(+), 454 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/events/UnhandledExceptionEvent.java delete mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingContinuousQueryErrorTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/events/EventType.java b/modules/core/src/main/java/org/apache/ignite/events/EventType.java index 7778f67b60f36..103dbd4dde4bd 100644 --- a/modules/core/src/main/java/org/apache/ignite/events/EventType.java +++ b/modules/core/src/main/java/org/apache/ignite/events/EventType.java @@ -858,12 +858,6 @@ public interface EventType { */ public static final int EVT_IGFS_FILE_PURGED = 127; - /** - * Built-in event type: event for unhandled exception. - * - */ - public static final int EVT_UNHANDLED_EXCEPTION = 128; - /** * All checkpoint events. This array can be directly passed into * {@link IgniteEvents#localListen(IgnitePredicate, int...)} method to diff --git a/modules/core/src/main/java/org/apache/ignite/events/UnhandledExceptionEvent.java b/modules/core/src/main/java/org/apache/ignite/events/UnhandledExceptionEvent.java deleted file mode 100644 index cb6cd85f12605..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/events/UnhandledExceptionEvent.java +++ /dev/null @@ -1,61 +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.events; - -import org.apache.ignite.cluster.ClusterNode; - -/** - * Cache fail event. - */ -public class UnhandledExceptionEvent extends EventAdapter { - /** */ - private static final long serialVersionUID = 0L; - - /** */ - private Exception ex; - - /** - * Default constructor. - */ - public UnhandledExceptionEvent() { - } - - /** - * @param node Node. - * @param msg Message. - * @param ex Exception. - * @param type Type. - */ - public UnhandledExceptionEvent(ClusterNode node, String msg, Exception ex, int type) { - super(node, msg, type); - this.ex = ex; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "msg=" + message() + ", type=" + type() + "ex=" + ex; - } - - /** - * - * @return inner exception - */ - public Exception getException() { - return ex; - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java index 5d7cb00dffec9..78dddd3340270 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java @@ -17,26 +17,50 @@ package org.apache.ignite.internal.processors.cache; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.events.UnhandledExceptionEvent; -import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.managers.deployment.GridDeploymentInfo; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.distributed.dht.*; +import org.apache.ignite.internal.processors.cache.distributed.dht.CacheGetFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse; -import org.apache.ignite.internal.processors.cache.distributed.near.*; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryResponse; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -55,12 +79,6 @@ import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; -import java.util.*; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; - -import static org.apache.ignite.events.EventType.EVT_UNHANDLED_EXCEPTION; import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE; /** @@ -675,11 +693,6 @@ private void processFailedMessage(UUID nodeId, GridCacheMessage msg, IgniteBiInC break; - case 59: - // No additional actions required, just skipping default switch section, - // since UnhandledException already registered. - break; - case 114: { processMessage(nodeId,msg,c);// Will be handled by Rebalance Demander. } @@ -724,33 +737,12 @@ private void processFailedMessage(UUID nodeId, GridCacheMessage msg, IgniteBiInC break; - default:{ - String shortMsg = "Failed to send response to node. Unsupported direct type [message=" + msg + "]"; - - IgniteCheckedException e = new IgniteCheckedException(shortMsg, msg.classError()); - - registerUnhandledException(ctx, shortMsg, e); - } + default: + throw new IgniteCheckedException("Failed to send response to node. Unsupported direct type [message=" + + msg + "]", msg.classError()); } } - /** - * @param ctx Grid cache context. - * @param shortMsg Short message. - * @param ex Original Exception. - */ - public static void registerUnhandledException(GridCacheContext ctx, String shortMsg, IgniteCheckedException ex) { - GridKernalContext kctx = ctx.kernalContext(); - - kctx.exceptionRegistry().onException(shortMsg, ex); - - ClusterNode node = ctx.discovery().localNode(); - - UnhandledExceptionEvent evt = new UnhandledExceptionEvent(node, shortMsg, ex, EVT_UNHANDLED_EXCEPTION); - - kctx.event().record(evt); - } - /** * @param nodeId Node ID. * @param msg Message. 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 97e59c1d7d5c8..7bd1a51b314d3 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 @@ -325,16 +325,6 @@ void onWaitAtStop() { // No-op. } - /** - * Increment fails counter. - */ - public void onUnhandledException() { - final boolean statsEnabled = cctx.config().isStatisticsEnabled(); - - if (statsEnabled) - metrics.incrementOnFails(); - } - /** * Processes cache query request. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java index d25b7c47770c8..99056a20b9345 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java @@ -172,11 +172,4 @@ public GridCacheQueryMetricsAdapter copy() { @Override public String toString() { return S.toString(GridCacheQueryMetricsAdapter.class, this); } - - /** - * Increment fails counter. - */ - public void incrementOnFails() { - fails.increment(); - } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java index 2b86efea768cd..8492c380b1a75 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java @@ -357,6 +357,6 @@ public boolean fields() { /** {@inheritDoc} */ @Override public String toString() { - return S.toString(GridCacheQueryResponse.class, this, super.toString()); + return S.toString(GridCacheQueryResponse.class, this); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 4c91ea779809c..304d031d81fc0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -17,6 +17,28 @@ package org.apache.ignite.internal.processors.cache.query.continuous; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; +import javax.cache.event.CacheEntryEvent; +import javax.cache.event.CacheEntryEventFilter; +import javax.cache.event.CacheEntryUpdatedListener; +import javax.cache.event.EventType; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; @@ -39,7 +61,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheDeploymentManager; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateFuture; import org.apache.ignite.internal.processors.cache.query.CacheQueryType; -import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager.JCacheQueryLocalListener; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager.JCacheQueryRemoteFilter; import org.apache.ignite.internal.processors.continuous.GridContinuousBatch; @@ -61,22 +82,8 @@ import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentLinkedDeque8; -import javax.cache.event.CacheEntryEvent; -import javax.cache.event.CacheEntryEventFilter; -import javax.cache.event.CacheEntryUpdatedListener; -import javax.cache.event.EventType; -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.*; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicLong; - import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED; import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_OBJECT_READ; -import static org.apache.ignite.internal.processors.cache.GridCacheIoManager.registerUnhandledException; /** * Continuous query handler. @@ -681,17 +688,8 @@ private void notifyCallback0(UUID nodeId, catch (IgniteCheckedException ex) { if (ignoreClsNotFound) assert internal; - else { - String shortMsg = "Failed to unmarshal entry."; - - U.error(ctx.log(getClass()), shortMsg, ex); - - GridCacheQueryManager qryMgr = cctx.queries(); - - qryMgr.onUnhandledException(); - - registerUnhandledException(cctx, shortMsg, ex); - } + else + U.error(ctx.log(getClass()), "Failed to unmarshal entry.", ex); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingContinuousQueryErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingContinuousQueryErrorTest.java deleted file mode 100644 index 82f5f0916ab1a..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingContinuousQueryErrorTest.java +++ /dev/null @@ -1,302 +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; - -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicInteger; -import javax.cache.Cache; -import javax.cache.event.CacheEntryEvent; -import javax.cache.event.CacheEntryUpdatedListener; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteException; -import org.apache.ignite.cache.query.ContinuousQuery; -import org.apache.ignite.cache.query.QueryCursor; -import org.apache.ignite.cache.query.ScanQuery; -import org.apache.ignite.cache.query.annotations.QuerySqlField; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.events.Event; -import org.apache.ignite.events.EventType; -import org.apache.ignite.events.UnhandledExceptionEvent; -import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMetricsAdapter; -import org.apache.ignite.internal.util.lang.GridAbsPredicate; -import org.apache.ignite.internal.util.typedef.X; -import org.apache.ignite.lang.IgniteBiPredicate; -import org.apache.ignite.lang.IgnitePredicate; -import org.apache.ignite.testframework.GridTestUtils; -import org.apache.ignite.thread.IgniteThread; - -/** - * Checks behavior on exception while unmarshalling key for continuous query. - */ -public class IgniteCacheP2pUnmarshallingContinuousQueryErrorTest extends IgniteCacheP2pUnmarshallingErrorTest { - /** - * {@inheritDoc} - */ - @Override protected int gridCount() { - return 3; - } - - /** Used inside InitialQuery listener. */ - private static final CountDownLatch latch = new CountDownLatch(1); - - /** Node where unmarshalling fails with exceptions. */ - private static volatile String failNode; - - /** Used to count UnhandledExceptionEvents at client node. */ - private static final AtomicInteger cnt = new AtomicInteger(); - - /** - * {@inheritDoc} - */ - @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception { - CacheConfiguration cacheCfg = super.cacheConfiguration(gridName); - - cacheCfg.setStatisticsEnabled(true); - - return cacheCfg; - } - - /** - * {@inheritDoc} - */ - @Override public void testResponseMessageOnUnmarshallingFailed() throws Exception { - IgniteEx client = grid(0); - IgniteEx node1 = grid(1); - IgniteEx node2 = grid(2); - - assert client.configuration().isClientMode() && - !node1.configuration().isClientMode() && - !node2.configuration().isClientMode(); - - failNode = client.name(); - - client.events().localListen(new IgnitePredicate() { - @Override public boolean apply(Event evt) { - UnhandledExceptionEvent uex = (UnhandledExceptionEvent)evt; - - assertTrue(X.getFullStackTrace(uex.getException()). - contains("IOException: Class can not be unmarshalled")); - - cnt.incrementAndGet(); - - return true; - } - }, EventType.EVT_UNHANDLED_EXCEPTION); - - node1.events().localListen(new IgnitePredicate() { - @Override public boolean apply(Event evt) { - fail("This line should newer calls."); - - return true; - } - }, EventType.EVT_UNHANDLED_EXCEPTION); - - ContinuousQuery qry = new ContinuousQuery<>(); - - qry.setInitialQuery(new ScanQuery<>(new IgniteBiPredicate() { - @Override public boolean apply(TestKey key, String val) { - latch.countDown(); // Gives guarantee query initialized. - - return true; - } - })); - - qry.setLocalListener(new CacheEntryUpdatedListener() { - @Override public void onUpdated(Iterable> evts) { - fail("This line should newer calls."); - } - }); - - validate( - 0,//execs - 0,//evts - 0,//fails - client, - node1, - node2); - - // Put element before creating QueryCursor. - putPrimary(node1); - - try (QueryCursor> cur = client.cache(null).query(qry)) { - latch.await(); - - validate( - 1,//execs - 0,//evts - 0,//fails - client, - node1, - node2); - - putPrimary(node1); - - validate( - 1,//execs - 1,//evts - 1,//fails - client, - node1, - node2); - - putPrimary(node2); - - validate( - 1,//execs - 2,//evts - 2,//fails - client, - node1, - node2); - } - } - - /** - * @param ignite Ignite. - */ - private void putPrimary(IgniteEx ignite) { - IgniteCache cache = ignite.cache(null); - - cache.put(generateNodeKeys(ignite, cache), "value"); - } - - /** - * @param execs Executions. - * @param evts Events. - * @param failsNum Fails number. - * @param client Client. - * @param node1 Node 1. - * @param node2 Node 2. - */ - private void validate(final int execs, final int evts, final int failsNum, final IgniteEx client, IgniteEx node1, - IgniteEx node2) throws Exception { - assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() { - @Override public boolean apply() { - return client.cache(null).queryMetrics().fails() == failsNum; - } - }, 5_000)); - - assertEquals(evts, cnt.intValue()); - - validateCacheQueryMetrics(client, execs, failsNum); - validateCacheQueryMetrics(node1, 0, 0); - validateCacheQueryMetrics(node2, 0, 0); - } - - /** - * @param ignite Ignite. - * @param executions Executions. - * @param fails Fails. - */ - private void validateCacheQueryMetrics(IgniteEx ignite, int executions, int fails) { - IgniteCache cache = ignite.cache(null); - - GridCacheQueryMetricsAdapter metr = (GridCacheQueryMetricsAdapter)cache.queryMetrics(); - - assertEquals(metr.executions(), executions); - - assertEquals(metr.fails(), fails); - } - - /** - * @param node Node. - * @param cache Cache. - */ - private TestKey generateNodeKeys(IgniteEx node, IgniteCache cache) { - - ClusterNode locNode = node.localNode(); - - for (int ind = 0; ind < 100_000; ind++) { - TestKey key = new TestKey("key" + ind); - - if (affinity(cache).isPrimary(locNode, key)) - return key; - } - - throw new IgniteException("Unable to find key keys as primary for cache."); - } - - /** - * - * */ - private static class TestKey implements Externalizable { - /** - * Field. - */ - @QuerySqlField(index = true) - private String field; - - /** - * Required by {@link Externalizable}. - */ - public TestKey() { - } - - /** - * @param field Test key 1. - */ - public TestKey(String field) { - this.field = field; - } - - /** - * {@inheritDoc} - */ - @Override public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - - IgniteCacheP2pUnmarshallingContinuousQueryErrorTest.TestKey key = (IgniteCacheP2pUnmarshallingContinuousQueryErrorTest.TestKey)o; - - return !(field != null ? !field.equals(key.field) : key.field != null); - } - - /** - * {@inheritDoc} - */ - @Override public int hashCode() { - return field != null ? field.hashCode() : 0; - } - - /** - * {@inheritDoc} - */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - out.writeObject(field); - } - - /** - * {@inheritDoc} - */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - field = (String)in.readObject(); - - if (((IgniteThread)Thread.currentThread()).getGridName().equals(failNode)) - throw new IOException("Class can not be unmarshalled."); - - } - } -} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuite.java index b45d134f6956e..dfc96dca72337 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuite.java @@ -19,7 +19,10 @@ import java.util.Set; import junit.framework.TestSuite; -import org.apache.ignite.internal.processors.cache.*; +import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingErrorTest; +import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingNearErrorTest; +import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingRebalanceErrorTest; +import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingTxErrorTest; import org.apache.ignite.testframework.GridTestUtils; /** @@ -46,7 +49,6 @@ public static TestSuite suite(Set ignoredTests) throws Exception { GridTestUtils.addTestIfNeeded(suite, IgniteCacheP2pUnmarshallingNearErrorTest.class, ignoredTests); GridTestUtils.addTestIfNeeded(suite, IgniteCacheP2pUnmarshallingRebalanceErrorTest.class, ignoredTests); GridTestUtils.addTestIfNeeded(suite, IgniteCacheP2pUnmarshallingTxErrorTest.class, ignoredTests); - GridTestUtils.addTestIfNeeded(suite, IgniteCacheP2pUnmarshallingContinuousQueryErrorTest.class, ignoredTests); return suite; } From 865bbcf0f41a0c4944e0928f1758d43a0eae82c5 Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Tue, 8 Nov 2016 15:18:29 +0300 Subject: [PATCH 323/487] Revert "Merge remote-tracking branch 'professional/ignite-1.6.11' into ignite-1.6.11" This reverts commit c2c82ca44befe4570325dd6cf2ba885e0d90596c, reversing changes made to e821dc0083003bc81058b1cb223d8a8a2ee44daf. --- .../processors/job/GridJobWorker.java | 10 +- .../service/GridServiceProcessor.java | 61 +------ .../internal/util/SerializableTransient.java | 58 ------- .../ignite/marshaller/MarshallerUtils.java | 22 --- .../optimized/OptimizedClassDescriptor.java | 90 +---------- modules/platforms/cpp/binary/Makefile.am | 4 +- modules/platforms/cpp/core/Makefile.am | 4 +- .../cpp/examples/odbc-example/Makefile.am | 4 +- .../cpp/examples/putget-example/Makefile.am | 4 +- .../cpp/examples/query-example/Makefile.am | 4 +- modules/platforms/cpp/ignite/Makefile.am | 4 +- modules/platforms/cpp/jni/Makefile.am | 4 +- .../cpp/odbc-test/src/api_robustness_test.cpp | 63 -------- .../cpp/odbc-test/src/queries_test.cpp | 9 +- modules/platforms/cpp/odbc/Makefile.am | 4 +- .../cpp/odbc/include/ignite/odbc/statement.h | 42 ----- modules/platforms/cpp/odbc/src/odbc.cpp | 116 +++++++++++++- modules/platforms/cpp/odbc/src/statement.cpp | 151 ------------------ 18 files changed, 128 insertions(+), 526 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/util/SerializableTransient.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java index 5f38b29025584..8169eb1101f4f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java @@ -57,7 +57,6 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; -import org.apache.ignite.marshaller.MarshallerUtils; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.events.EventType.EVT_JOB_CANCELLED; @@ -422,14 +421,7 @@ boolean initialize(GridDeployment dep, Class taskCls) { try { if (job == null) { - MarshallerUtils.jobSenderVersion(taskNode.version()); - - try { - job = U.unmarshal(marsh, jobBytes, U.resolveClassLoader(dep.classLoader(), ctx.config())); - } - finally { - MarshallerUtils.jobSenderVersion(null); - } + job = U.unmarshal(marsh, jobBytes, U.resolveClassLoader(dep.classLoader(), ctx.config())); // No need to hold reference any more. jobBytes = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 6c263638de7c5..527d360e9578f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -20,14 +20,11 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; @@ -90,7 +87,6 @@ import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; -import org.apache.ignite.internal.util.SerializableTransient; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.JobContextResource; import org.apache.ignite.resources.LoggerResource; @@ -119,9 +115,6 @@ public class GridServiceProcessor extends GridProcessorAdapter { /** */ public static final IgniteProductVersion LAZY_SERVICES_CFG_SINCE = IgniteProductVersion.fromString("1.5.22"); - /** Versions that only compatible with each other, and from 1.5.33. */ - private static final Set SERVICE_TOP_CALLABLE_VER1; - /** */ private final Boolean srvcCompatibilitySysProp; @@ -169,31 +162,6 @@ public class GridServiceProcessor extends GridProcessorAdapter { /** Topology listener. */ private GridLocalEventListener topLsnr = new TopologyListener(); - static { - Set versions = new TreeSet<>(new Comparator() { - @Override public int compare(final IgniteProductVersion o1, final IgniteProductVersion o2) { - return o1.compareToIgnoreTimestamp(o2); - } - }); - - versions.add(IgniteProductVersion.fromString("1.5.30")); - versions.add(IgniteProductVersion.fromString("1.5.31")); - versions.add(IgniteProductVersion.fromString("1.5.32")); - versions.add(IgniteProductVersion.fromString("1.6.3")); - versions.add(IgniteProductVersion.fromString("1.6.4")); - versions.add(IgniteProductVersion.fromString("1.6.5")); - versions.add(IgniteProductVersion.fromString("1.6.6")); - versions.add(IgniteProductVersion.fromString("1.6.7")); - versions.add(IgniteProductVersion.fromString("1.6.8")); - versions.add(IgniteProductVersion.fromString("1.6.9")); - versions.add(IgniteProductVersion.fromString("1.6.10")); - versions.add(IgniteProductVersion.fromString("1.7.0")); - versions.add(IgniteProductVersion.fromString("1.7.1")); - versions.add(IgniteProductVersion.fromString("1.7.2")); - - SERVICE_TOP_CALLABLE_VER1 = Collections.unmodifiableSet(versions); - } - /** * @param ctx Kernal context. */ @@ -700,13 +668,9 @@ public Map serviceTopology(String name) throws IgniteCheckedExcep ClusterNode node = cache.affinity().mapKeyToNode(name); if (node.version().compareTo(ServiceTopologyCallable.SINCE_VER) >= 0) { - final ServiceTopologyCallable call = new ServiceTopologyCallable(name); - - call.serialize = SERVICE_TOP_CALLABLE_VER1.contains(node.version()); - return ctx.closure().callAsyncNoFailover( GridClosureCallMode.BROADCAST, - call, + new ServiceTopologyCallable(name), Collections.singletonList(node), false ).get(); @@ -1865,7 +1829,6 @@ static class ServiceAssignmentsPredicate implements IgniteBiPredicate> { /** */ private static final long serialVersionUID = 0L; @@ -1873,14 +1836,11 @@ private static class ServiceTopologyCallable implements IgniteCallable - * Works only for jobs. For other messages node version is not available. - *

          - */ -@Retention(RetentionPolicy.RUNTIME) -@Target(ElementType.TYPE) -public @interface SerializableTransient { - /** - * Name of the private static method that returns list of transient fields - * that should be serialized (String[]), and accepts itself (before serialization) - * and {@link IgniteProductVersion}, e.g. - *
          -     *     private static String[] fields(Object self, IgniteProductVersion ver){
          -     *         return ver.compareTo("1.5.30") > 0 ? SERIALIZABLE_FIELDS : null;
          -     *     }
          -     * 
          - *

          - * On serialization version argument ver is null, on deserialization - self is null. - *

          - *

          - * If it returns empty array or null all transient fields will be normally - * ignored. - *

          - * - * @return Name of the method. - */ - String methodName(); -} diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java index ad63702de00d5..9668bafb1c7ff 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java @@ -17,7 +17,6 @@ package org.apache.ignite.marshaller; -import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.jetbrains.annotations.Nullable; @@ -25,9 +24,6 @@ * Utility marshaller methods. */ public class MarshallerUtils { - /** Job sender node version. */ - private static final ThreadLocal JOB_SND_NODE_VER = new ThreadLocal<>(); - /** * Set node name to marshaller context if possible. * @@ -59,22 +55,4 @@ public static JdkMarshaller jdkMarshaller(@Nullable String nodeName) { private MarshallerUtils() { // No-op. } - - /** - * Sets thread local job sender node version. - * - * @param ver Thread local job sender node version. - */ - public static void jobSenderVersion(IgniteProductVersion ver) { - JOB_SND_NODE_VER.set(ver); - } - - /** - * Returns thread local job sender node version. - * - * @return Thread local job sender node version. - */ - public static IgniteProductVersion jobSenderVersion() { - return JOB_SND_NODE_VER.get(); - } } diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java index 160f2c129de78..5a5b54d001f29 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java @@ -47,11 +47,8 @@ import java.util.concurrent.ConcurrentMap; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.marshaller.MarshallerContext; import org.apache.ignite.marshaller.MarshallerExclusions; -import org.apache.ignite.internal.util.SerializableTransient; -import org.apache.ignite.marshaller.MarshallerUtils; import static java.lang.reflect.Modifier.isFinal; import static java.lang.reflect.Modifier.isPrivate; @@ -169,9 +166,6 @@ class OptimizedClassDescriptor { /** Proxy interfaces. */ private Class[] proxyIntfs; - /** Method returns serializable transient fields. */ - private Method serTransMtd; - /** * Creates descriptor for class. * @@ -447,27 +441,6 @@ else if (Proxy.class.isAssignableFrom(cls)) { readObjMtds.add(mtd); - final SerializableTransient serTransAn = c.getAnnotation(SerializableTransient.class); - - // Custom serialization policy for transient fields. - if (serTransAn != null) { - try { - serTransMtd = c.getDeclaredMethod(serTransAn.methodName(), cls, IgniteProductVersion.class); - - int mod = serTransMtd.getModifiers(); - - if (isStatic(mod) && isPrivate(mod) - && serTransMtd.getReturnType() == String[].class) - serTransMtd.setAccessible(true); - else - // Set method back to null if it has incorrect signature. - serTransMtd = null; - } - catch (NoSuchMethodException ignored) { - serTransMtd = null; - } - } - Field[] clsFields0 = c.getDeclaredFields(); Map fieldNames = new HashMap<>(); @@ -824,7 +797,7 @@ void write(OptimizedObjectOutputStream out, Object obj) throws IOException { writeTypeData(out); out.writeShort(checksum); - out.writeSerializable(obj, writeObjMtds, serializableFields(obj.getClass(), obj, null)); + out.writeSerializable(obj, writeObjMtds, fields); break; @@ -833,60 +806,6 @@ void write(OptimizedObjectOutputStream out, Object obj) throws IOException { } } - /** - * Gets list of serializable fields. If {@link #serTransMtd} method - * returns list of transient fields, they will be added to other fields. - * Transient fields that are not included in that list will be normally - * ignored. - * - * @param cls Class. - * @param obj Object. - * @param ver Job sender version. - * @return Serializable fields. - */ - @SuppressWarnings("ForLoopReplaceableByForEach") - private Fields serializableFields(Class cls, Object obj, IgniteProductVersion ver) { - if (serTransMtd == null) - return fields; - - try { - final String[] transFields = (String[])serTransMtd.invoke(cls, obj, ver); - - if (transFields == null || transFields.length == 0) - return fields; - - List clsFields = new ArrayList<>(); - - clsFields.addAll(fields.fields.get(0).fields); - - for (int i = 0; i < transFields.length; i++) { - final String fieldName = transFields[i]; - - final Field f = cls.getDeclaredField(fieldName); - - FieldInfo fieldInfo = new FieldInfo(f, f.getName(), - GridUnsafe.objectFieldOffset(f), fieldType(f.getType())); - - clsFields.add(fieldInfo); - } - - Collections.sort(clsFields, new Comparator() { - @Override public int compare(FieldInfo t1, FieldInfo t2) { - return t1.name().compareTo(t2.name()); - } - }); - - List fields = new ArrayList<>(); - - fields.add(new ClassFields(clsFields)); - - return new Fields(fields); - } - catch (Exception e) { - return fields; - } - } - /** * @param out Output stream. * @throws IOException In case of error. @@ -919,12 +838,7 @@ Object read(OptimizedObjectInputStream in) throws ClassNotFoundException, IOExce case SERIALIZABLE: verifyChecksum(in.readShort()); - // If no serialize method, then unmarshal as usual. - if (serTransMtd != null) - return in.readSerializable(cls, readObjMtds, readResolveMtd, - serializableFields(cls, null, MarshallerUtils.jobSenderVersion())); - else - return in.readSerializable(cls, readObjMtds, readResolveMtd, fields); + return in.readSerializable(cls, readObjMtds, readResolveMtd, fields); default: assert false : "Unexpected type: " + type; diff --git a/modules/platforms/cpp/binary/Makefile.am b/modules/platforms/cpp/binary/Makefile.am index ca301a6395434..4876776c811b2 100644 --- a/modules/platforms/cpp/binary/Makefile.am +++ b/modules/platforms/cpp/binary/Makefile.am @@ -26,9 +26,7 @@ AM_CPPFLAGS = \ -I$(srcdir)/include \ -I@top_srcdir@/common/include \ -I@top_srcdir@/common/os/linux/include \ - -DIGNITE_IMPL \ - -D__STDC_LIMIT_MACROS \ - -D__STDC_CONSTANT_MACROS + -DIGNITE_IMPL AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/core/Makefile.am b/modules/platforms/cpp/core/Makefile.am index 97523cf64992f..bbb77203da624 100644 --- a/modules/platforms/cpp/core/Makefile.am +++ b/modules/platforms/cpp/core/Makefile.am @@ -31,9 +31,7 @@ AM_CPPFLAGS = \ -I@top_srcdir@/jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL \ - -D__STDC_LIMIT_MACROS \ - -D__STDC_CONSTANT_MACROS + -DIGNITE_IMPL AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/examples/odbc-example/Makefile.am b/modules/platforms/cpp/examples/odbc-example/Makefile.am index 83cc63e207366..e58410527a4f4 100644 --- a/modules/platforms/cpp/examples/odbc-example/Makefile.am +++ b/modules/platforms/cpp/examples/odbc-example/Makefile.am @@ -30,9 +30,7 @@ AM_CPPFLAGS = \ -I@top_srcdir@/../jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL \ - -D__STDC_LIMIT_MACROS \ - -D__STDC_CONSTANT_MACROS + -DIGNITE_IMPL AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/examples/putget-example/Makefile.am b/modules/platforms/cpp/examples/putget-example/Makefile.am index cf39002f7ae54..5301ea1a0aaeb 100644 --- a/modules/platforms/cpp/examples/putget-example/Makefile.am +++ b/modules/platforms/cpp/examples/putget-example/Makefile.am @@ -30,9 +30,7 @@ AM_CPPFLAGS = \ -I@top_srcdir@/../jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL \ - -D__STDC_LIMIT_MACROS \ - -D__STDC_CONSTANT_MACROS + -DIGNITE_IMPL AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/examples/query-example/Makefile.am b/modules/platforms/cpp/examples/query-example/Makefile.am index 01231ec38f977..14373034ae446 100644 --- a/modules/platforms/cpp/examples/query-example/Makefile.am +++ b/modules/platforms/cpp/examples/query-example/Makefile.am @@ -30,9 +30,7 @@ AM_CPPFLAGS = \ -I@top_srcdir@/../jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL \ - -D__STDC_LIMIT_MACROS \ - -D__STDC_CONSTANT_MACROS + -DIGNITE_IMPL AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/ignite/Makefile.am b/modules/platforms/cpp/ignite/Makefile.am index 2dbc4d614e23f..625f1df670e45 100644 --- a/modules/platforms/cpp/ignite/Makefile.am +++ b/modules/platforms/cpp/ignite/Makefile.am @@ -30,9 +30,7 @@ AM_CPPFLAGS = \ -I@top_srcdir@/jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL \ - -D__STDC_LIMIT_MACROS \ - -D__STDC_CONSTANT_MACROS + -DIGNITE_IMPL AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/jni/Makefile.am b/modules/platforms/cpp/jni/Makefile.am index 2cb4b90d8afa4..b9b39133d1e09 100644 --- a/modules/platforms/cpp/jni/Makefile.am +++ b/modules/platforms/cpp/jni/Makefile.am @@ -29,9 +29,7 @@ AM_CPPFLAGS = \ -I@top_srcdir@/common/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL \ - -D__STDC_LIMIT_MACROS \ - -D__STDC_CONSTANT_MACROS + -DIGNITE_IMPL AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp index fbd5f12532c28..008cf25f26c7e 100644 --- a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp @@ -160,54 +160,6 @@ struct ApiRobustnessTestSuiteFixture testCache = grid.GetCache("cache"); } - /** - * Check that SQLFetchScroll does not crash with unsupported orientation. - * - * @param orientation Fetch orientation. - */ - void CheckFetchScrollUnsupportedOrientation(SQLUSMALLINT orientation) - { - Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); - - SQLRETURN ret; - - const int64_t recordsNum = 100; - - for (int i = 0; i < recordsNum; ++i) - { - TestType val; - - val.i32Field = i * 10; - - testCache.Put(i, val); - } - - int32_t i32Field = -1; - - // Binding column. - ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &i32Field, 0, 0); - - if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); - - SQLCHAR request[] = "SELECT i32Field FROM TestType ORDER BY _key"; - - ret = SQLExecDirect(stmt, request, SQL_NTS); - if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); - - ret = SQLFetchScroll(stmt, SQL_FETCH_NEXT, 0); - if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); - - BOOST_CHECK_EQUAL(i32Field, 0); - - ret = SQLFetchScroll(stmt, orientation, 0); - - // Operation is not supported. However, there should be no crash. - BOOST_CHECK(ret == SQL_ERROR); - } - /** * Destructor. */ @@ -1051,19 +1003,4 @@ BOOST_AUTO_TEST_CASE(TestSQLSpecialColumns) SQLCloseCursor(stmt); } -BOOST_AUTO_TEST_CASE(TestFetchScrollLast) -{ - CheckFetchScrollUnsupportedOrientation(SQL_FETCH_LAST); -} - -BOOST_AUTO_TEST_CASE(TestFetchScrollPrior) -{ - CheckFetchScrollUnsupportedOrientation(SQL_FETCH_PRIOR); -} - -BOOST_AUTO_TEST_CASE(TestFetchScrollFirst) -{ - CheckFetchScrollUnsupportedOrientation(SQL_FETCH_FIRST); -} - BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 82e9972b5ca2d..eb6e153c71e85 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -200,8 +200,6 @@ struct QueriesTestSuiteFixture "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType"; ret = SQLExecDirect(stmt, reinterpret_cast(request), SQL_NTS); - if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); if (!SQL_SUCCEEDED(ret)) BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); @@ -366,8 +364,6 @@ BOOST_AUTO_TEST_CASE(TestTwoRowsString) "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType"; ret = SQLExecDirect(stmt, request, SQL_NTS); - if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); if (!SQL_SUCCEEDED(ret)) BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); @@ -464,8 +460,6 @@ BOOST_AUTO_TEST_CASE(TestOneRowString) "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType"; ret = SQLExecDirect(stmt, request, SQL_NTS); - if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); ret = SQLFetch(stmt); if (!SQL_SUCCEEDED(ret)) @@ -528,8 +522,6 @@ BOOST_AUTO_TEST_CASE(TestOneRowStringLen) "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType"; ret = SQLExecDirect(stmt, request, SQL_NTS); - if (!SQL_SUCCEEDED(ret)) - BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); ret = SQLFetch(stmt); if (!SQL_SUCCEEDED(ret)) @@ -674,4 +666,5 @@ BOOST_AUTO_TEST_CASE(TestDataAtExecution) BOOST_CHECK(ret == SQL_NO_DATA); } + BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am index b0cc5f87133fc..3c8b37a5e97f2 100644 --- a/modules/platforms/cpp/odbc/Makefile.am +++ b/modules/platforms/cpp/odbc/Makefile.am @@ -27,9 +27,7 @@ AM_CPPFLAGS = \ -I@top_srcdir@/common/include \ -I@top_srcdir@/common/os/linux/include \ -I@top_srcdir@/binary/include \ - -DIGNITE_IMPL \ - -D__STDC_LIMIT_MACROS \ - -D__STDC_CONSTANT_MACROS + -DIGNITE_IMPL AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h index db6205ed3d307..35f1e98b4f140 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h @@ -117,25 +117,6 @@ namespace ignite */ void UnbindAllParameters(); - /** - * Set statement attribute. - * - * @param attr Attribute type. - * @param value Value pointer. - * @param valueLen Value length. - */ - void SetAttribute(int attr, void* value, SQLINTEGER valueLen); - - /** - * Get statement attribute. - * - * @param attr Attribute type. - * @param buf Buffer for value. - * @param bufLen Buffer length. - * @param valueLen Resulting value length. - */ - void GetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER *valueLen); - /** * Get number of binded parameters. * @@ -374,29 +355,6 @@ namespace ignite */ SqlResult InternalBindParameter(uint16_t paramIdx, const app::Parameter& param); - /** - * Set statement attribute. - * Internal call. - * - * @param attr Attribute type. - * @param value Value pointer. - * @param valueLen Value length. - * @return Operation result. - */ - SqlResult InternalSetAttribute(int attr, void* value, SQLINTEGER valueLen); - - /** - * Get statement attribute. - * Internal call. - * - * @param attr Attribute type. - * @param buf Buffer for value. - * @param bufLen Buffer length. - * @param valueLen Resulting value length. - * @return Operation result. - */ - SqlResult InternalGetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER* valueLen); - /** * Get value of the column in the result set. * diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp index 612d51a5b4b88..7416ad2b5b663 100644 --- a/modules/platforms/cpp/odbc/src/odbc.cpp +++ b/modules/platforms/cpp/odbc/src/odbc.cpp @@ -854,9 +854,73 @@ namespace ignite if (!statement) return SQL_INVALID_HANDLE; - statement->GetAttribute(attr, valueBuf, valueBufLen, valueResLen); + if (!valueBuf) + return SQL_ERROR; - return statement->GetDiagnosticRecords().GetReturnCode(); + switch (attr) + { + case SQL_ATTR_APP_ROW_DESC: + case SQL_ATTR_APP_PARAM_DESC: + case SQL_ATTR_IMP_ROW_DESC: + case SQL_ATTR_IMP_PARAM_DESC: + { + SQLPOINTER *val = reinterpret_cast(valueBuf); + + *val = static_cast(stmt); + + break; + } + + case SQL_ATTR_ROW_ARRAY_SIZE: + { + SQLINTEGER *val = reinterpret_cast(valueBuf); + + *val = static_cast(1); + + break; + } + + case SQL_ATTR_ROWS_FETCHED_PTR: + { + SQLULEN** val = reinterpret_cast(valueBuf); + + *val = reinterpret_cast(statement->GetRowsFetchedPtr()); + + break; + } + + case SQL_ATTR_ROW_STATUS_PTR: + { + SQLUSMALLINT** val = reinterpret_cast(valueBuf); + + *val = reinterpret_cast(statement->GetRowStatusesPtr()); + + break; + } + + case SQL_ATTR_PARAM_BIND_OFFSET_PTR: + { + SQLULEN** val = reinterpret_cast(valueBuf); + + *val = reinterpret_cast(statement->GetParamBindOffsetPtr()); + + break; + } + + case SQL_ATTR_ROW_BIND_OFFSET_PTR: + { + SQLULEN** val = reinterpret_cast(valueBuf); + + *val = reinterpret_cast(statement->GetColumnBindOffsetPtr()); + + break; + } + + default: + return SQL_ERROR; + } + + return SQL_SUCCESS; } SQLRETURN SQLSetStmtAttr(SQLHSTMT stmt, @@ -879,9 +943,53 @@ namespace ignite if (!statement) return SQL_INVALID_HANDLE; - statement->SetAttribute(attr, value, valueLen); + switch (attr) + { + case SQL_ATTR_ROW_ARRAY_SIZE: + { + SQLULEN val = reinterpret_cast(value); - return statement->GetDiagnosticRecords().GetReturnCode(); + LOG_MSG("Value: %d\n", val); + + if (val != 1) + return SQL_ERROR; + + break; + } + + case SQL_ATTR_ROWS_FETCHED_PTR: + { + statement->SetRowsFetchedPtr(reinterpret_cast(value)); + + break; + } + + case SQL_ATTR_ROW_STATUS_PTR: + { + statement->SetRowStatusesPtr(reinterpret_cast(value)); + + break; + } + + case SQL_ATTR_PARAM_BIND_OFFSET_PTR: + { + statement->SetParamBindOffsetPtr(reinterpret_cast(value)); + + break; + } + + case SQL_ATTR_ROW_BIND_OFFSET_PTR: + { + statement->SetColumnBindOffsetPtr(reinterpret_cast(value)); + + break; + } + + default: + return SQL_ERROR; + } + + return SQL_SUCCESS; } SQLRETURN SQLPrimaryKeys(SQLHSTMT stmt, diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp index f1a577a2adde3..32f7c3f0c33d2 100644 --- a/modules/platforms/cpp/odbc/src/statement.cpp +++ b/modules/platforms/cpp/odbc/src/statement.cpp @@ -145,157 +145,6 @@ namespace ignite paramBindings.clear(); } - void Statement::SetAttribute(int attr, void* value, SQLINTEGER valueLen) - { - IGNITE_ODBC_API_CALL(InternalSetAttribute(attr, value, valueLen)); - } - - SqlResult Statement::InternalSetAttribute(int attr, void* value, SQLINTEGER valueLen) - { - switch (attr) - { - case SQL_ATTR_ROW_ARRAY_SIZE: - { - SQLULEN val = reinterpret_cast(value); - - LOG_MSG("SQL_ATTR_ROW_ARRAY_SIZE: %d\n", val); - - if (val != 1) - { - AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, - "Fetching of more than one row by call is not supported."); - - return SQL_RESULT_ERROR; - } - - break; - } - - case SQL_ATTR_ROWS_FETCHED_PTR: - { - SetRowsFetchedPtr(reinterpret_cast(value)); - - break; - } - - case SQL_ATTR_ROW_STATUS_PTR: - { - SetRowStatusesPtr(reinterpret_cast(value)); - - break; - } - - case SQL_ATTR_PARAM_BIND_OFFSET_PTR: - { - SetParamBindOffsetPtr(reinterpret_cast(value)); - - break; - } - - case SQL_ATTR_ROW_BIND_OFFSET_PTR: - { - SetColumnBindOffsetPtr(reinterpret_cast(value)); - - break; - } - - default: - { - AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, - "Specified attribute is not supported."); - - return SQL_RESULT_ERROR; - } - } - - return SQL_RESULT_SUCCESS; - } - - void Statement::GetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER* valueLen) - { - IGNITE_ODBC_API_CALL(InternalGetAttribute(attr, buf, bufLen, valueLen)); - } - - SqlResult Statement::InternalGetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER* valueLen) - { - if (!buf) - { - AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, "Data buffer is NULL."); - - return SQL_RESULT_ERROR; - } - - switch (attr) - { - case SQL_ATTR_APP_ROW_DESC: - case SQL_ATTR_APP_PARAM_DESC: - case SQL_ATTR_IMP_ROW_DESC: - case SQL_ATTR_IMP_PARAM_DESC: - { - SQLPOINTER *val = reinterpret_cast(buf); - - *val = static_cast(this); - - break; - } - - case SQL_ATTR_ROW_ARRAY_SIZE: - { - SQLINTEGER *val = reinterpret_cast(buf); - - *val = static_cast(1); - - break; - } - - case SQL_ATTR_ROWS_FETCHED_PTR: - { - SQLULEN** val = reinterpret_cast(buf); - - *val = reinterpret_cast(GetRowsFetchedPtr()); - - break; - } - - case SQL_ATTR_ROW_STATUS_PTR: - { - SQLUSMALLINT** val = reinterpret_cast(buf); - - *val = reinterpret_cast(GetRowStatusesPtr()); - - break; - } - - case SQL_ATTR_PARAM_BIND_OFFSET_PTR: - { - SQLULEN** val = reinterpret_cast(buf); - - *val = reinterpret_cast(GetParamBindOffsetPtr()); - - break; - } - - case SQL_ATTR_ROW_BIND_OFFSET_PTR: - { - SQLULEN** val = reinterpret_cast(buf); - - *val = reinterpret_cast(GetColumnBindOffsetPtr()); - - break; - } - - default: - { - AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, - "Specified attribute is not supported."); - - return SQL_RESULT_ERROR; - } - } - - return SQL_RESULT_SUCCESS; - } - uint16_t Statement::GetParametersNumber() { IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS; From 9726421ff9efb2b19813b2fd6ad27a3728b5ab1a Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Tue, 8 Nov 2016 15:59:00 +0300 Subject: [PATCH 324/487] Revert Revert Merge remote-tracking branch 'professional/ignite-1.6.11' --- .../processors/job/GridJobWorker.java | 10 +- .../service/GridServiceProcessor.java | 61 ++++++- .../internal/util/SerializableTransient.java | 58 +++++++ .../ignite/marshaller/MarshallerUtils.java | 22 +++ .../optimized/OptimizedClassDescriptor.java | 90 ++++++++++- modules/platforms/cpp/binary/Makefile.am | 4 +- modules/platforms/cpp/core/Makefile.am | 4 +- .../cpp/examples/odbc-example/Makefile.am | 4 +- .../cpp/examples/putget-example/Makefile.am | 4 +- .../cpp/examples/query-example/Makefile.am | 4 +- modules/platforms/cpp/ignite/Makefile.am | 4 +- modules/platforms/cpp/jni/Makefile.am | 4 +- .../cpp/odbc-test/src/api_robustness_test.cpp | 63 ++++++++ .../cpp/odbc-test/src/queries_test.cpp | 9 +- modules/platforms/cpp/odbc/Makefile.am | 4 +- .../cpp/odbc/include/ignite/odbc/statement.h | 42 +++++ modules/platforms/cpp/odbc/src/odbc.cpp | 116 +------------- modules/platforms/cpp/odbc/src/statement.cpp | 151 ++++++++++++++++++ 18 files changed, 526 insertions(+), 128 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/util/SerializableTransient.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java index 8169eb1101f4f..5f38b29025584 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java @@ -57,6 +57,7 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.marshaller.MarshallerUtils; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.events.EventType.EVT_JOB_CANCELLED; @@ -421,7 +422,14 @@ boolean initialize(GridDeployment dep, Class taskCls) { try { if (job == null) { - job = U.unmarshal(marsh, jobBytes, U.resolveClassLoader(dep.classLoader(), ctx.config())); + MarshallerUtils.jobSenderVersion(taskNode.version()); + + try { + job = U.unmarshal(marsh, jobBytes, U.resolveClassLoader(dep.classLoader(), ctx.config())); + } + finally { + MarshallerUtils.jobSenderVersion(null); + } // No need to hold reference any more. jobBytes = null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 527d360e9578f..6c263638de7c5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -20,11 +20,14 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; @@ -87,6 +90,7 @@ import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.internal.util.SerializableTransient; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.JobContextResource; import org.apache.ignite.resources.LoggerResource; @@ -115,6 +119,9 @@ public class GridServiceProcessor extends GridProcessorAdapter { /** */ public static final IgniteProductVersion LAZY_SERVICES_CFG_SINCE = IgniteProductVersion.fromString("1.5.22"); + /** Versions that only compatible with each other, and from 1.5.33. */ + private static final Set SERVICE_TOP_CALLABLE_VER1; + /** */ private final Boolean srvcCompatibilitySysProp; @@ -162,6 +169,31 @@ public class GridServiceProcessor extends GridProcessorAdapter { /** Topology listener. */ private GridLocalEventListener topLsnr = new TopologyListener(); + static { + Set versions = new TreeSet<>(new Comparator() { + @Override public int compare(final IgniteProductVersion o1, final IgniteProductVersion o2) { + return o1.compareToIgnoreTimestamp(o2); + } + }); + + versions.add(IgniteProductVersion.fromString("1.5.30")); + versions.add(IgniteProductVersion.fromString("1.5.31")); + versions.add(IgniteProductVersion.fromString("1.5.32")); + versions.add(IgniteProductVersion.fromString("1.6.3")); + versions.add(IgniteProductVersion.fromString("1.6.4")); + versions.add(IgniteProductVersion.fromString("1.6.5")); + versions.add(IgniteProductVersion.fromString("1.6.6")); + versions.add(IgniteProductVersion.fromString("1.6.7")); + versions.add(IgniteProductVersion.fromString("1.6.8")); + versions.add(IgniteProductVersion.fromString("1.6.9")); + versions.add(IgniteProductVersion.fromString("1.6.10")); + versions.add(IgniteProductVersion.fromString("1.7.0")); + versions.add(IgniteProductVersion.fromString("1.7.1")); + versions.add(IgniteProductVersion.fromString("1.7.2")); + + SERVICE_TOP_CALLABLE_VER1 = Collections.unmodifiableSet(versions); + } + /** * @param ctx Kernal context. */ @@ -668,9 +700,13 @@ public Map serviceTopology(String name) throws IgniteCheckedExcep ClusterNode node = cache.affinity().mapKeyToNode(name); if (node.version().compareTo(ServiceTopologyCallable.SINCE_VER) >= 0) { + final ServiceTopologyCallable call = new ServiceTopologyCallable(name); + + call.serialize = SERVICE_TOP_CALLABLE_VER1.contains(node.version()); + return ctx.closure().callAsyncNoFailover( GridClosureCallMode.BROADCAST, - new ServiceTopologyCallable(name), + call, Collections.singletonList(node), false ).get(); @@ -1829,6 +1865,7 @@ static class ServiceAssignmentsPredicate implements IgniteBiPredicate> { /** */ private static final long serialVersionUID = 0L; @@ -1836,11 +1873,14 @@ private static class ServiceTopologyCallable implements IgniteCallable + * Works only for jobs. For other messages node version is not available. + *

          + */ +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.TYPE) +public @interface SerializableTransient { + /** + * Name of the private static method that returns list of transient fields + * that should be serialized (String[]), and accepts itself (before serialization) + * and {@link IgniteProductVersion}, e.g. + *
          +     *     private static String[] fields(Object self, IgniteProductVersion ver){
          +     *         return ver.compareTo("1.5.30") > 0 ? SERIALIZABLE_FIELDS : null;
          +     *     }
          +     * 
          + *

          + * On serialization version argument ver is null, on deserialization - self is null. + *

          + *

          + * If it returns empty array or null all transient fields will be normally + * ignored. + *

          + * + * @return Name of the method. + */ + String methodName(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java index 9668bafb1c7ff..ad63702de00d5 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java @@ -17,6 +17,7 @@ package org.apache.ignite.marshaller; +import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.jetbrains.annotations.Nullable; @@ -24,6 +25,9 @@ * Utility marshaller methods. */ public class MarshallerUtils { + /** Job sender node version. */ + private static final ThreadLocal JOB_SND_NODE_VER = new ThreadLocal<>(); + /** * Set node name to marshaller context if possible. * @@ -55,4 +59,22 @@ public static JdkMarshaller jdkMarshaller(@Nullable String nodeName) { private MarshallerUtils() { // No-op. } + + /** + * Sets thread local job sender node version. + * + * @param ver Thread local job sender node version. + */ + public static void jobSenderVersion(IgniteProductVersion ver) { + JOB_SND_NODE_VER.set(ver); + } + + /** + * Returns thread local job sender node version. + * + * @return Thread local job sender node version. + */ + public static IgniteProductVersion jobSenderVersion() { + return JOB_SND_NODE_VER.get(); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java index 5a5b54d001f29..160f2c129de78 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java @@ -47,8 +47,11 @@ import java.util.concurrent.ConcurrentMap; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.marshaller.MarshallerContext; import org.apache.ignite.marshaller.MarshallerExclusions; +import org.apache.ignite.internal.util.SerializableTransient; +import org.apache.ignite.marshaller.MarshallerUtils; import static java.lang.reflect.Modifier.isFinal; import static java.lang.reflect.Modifier.isPrivate; @@ -166,6 +169,9 @@ class OptimizedClassDescriptor { /** Proxy interfaces. */ private Class[] proxyIntfs; + /** Method returns serializable transient fields. */ + private Method serTransMtd; + /** * Creates descriptor for class. * @@ -441,6 +447,27 @@ else if (Proxy.class.isAssignableFrom(cls)) { readObjMtds.add(mtd); + final SerializableTransient serTransAn = c.getAnnotation(SerializableTransient.class); + + // Custom serialization policy for transient fields. + if (serTransAn != null) { + try { + serTransMtd = c.getDeclaredMethod(serTransAn.methodName(), cls, IgniteProductVersion.class); + + int mod = serTransMtd.getModifiers(); + + if (isStatic(mod) && isPrivate(mod) + && serTransMtd.getReturnType() == String[].class) + serTransMtd.setAccessible(true); + else + // Set method back to null if it has incorrect signature. + serTransMtd = null; + } + catch (NoSuchMethodException ignored) { + serTransMtd = null; + } + } + Field[] clsFields0 = c.getDeclaredFields(); Map fieldNames = new HashMap<>(); @@ -797,7 +824,7 @@ void write(OptimizedObjectOutputStream out, Object obj) throws IOException { writeTypeData(out); out.writeShort(checksum); - out.writeSerializable(obj, writeObjMtds, fields); + out.writeSerializable(obj, writeObjMtds, serializableFields(obj.getClass(), obj, null)); break; @@ -806,6 +833,60 @@ void write(OptimizedObjectOutputStream out, Object obj) throws IOException { } } + /** + * Gets list of serializable fields. If {@link #serTransMtd} method + * returns list of transient fields, they will be added to other fields. + * Transient fields that are not included in that list will be normally + * ignored. + * + * @param cls Class. + * @param obj Object. + * @param ver Job sender version. + * @return Serializable fields. + */ + @SuppressWarnings("ForLoopReplaceableByForEach") + private Fields serializableFields(Class cls, Object obj, IgniteProductVersion ver) { + if (serTransMtd == null) + return fields; + + try { + final String[] transFields = (String[])serTransMtd.invoke(cls, obj, ver); + + if (transFields == null || transFields.length == 0) + return fields; + + List clsFields = new ArrayList<>(); + + clsFields.addAll(fields.fields.get(0).fields); + + for (int i = 0; i < transFields.length; i++) { + final String fieldName = transFields[i]; + + final Field f = cls.getDeclaredField(fieldName); + + FieldInfo fieldInfo = new FieldInfo(f, f.getName(), + GridUnsafe.objectFieldOffset(f), fieldType(f.getType())); + + clsFields.add(fieldInfo); + } + + Collections.sort(clsFields, new Comparator() { + @Override public int compare(FieldInfo t1, FieldInfo t2) { + return t1.name().compareTo(t2.name()); + } + }); + + List fields = new ArrayList<>(); + + fields.add(new ClassFields(clsFields)); + + return new Fields(fields); + } + catch (Exception e) { + return fields; + } + } + /** * @param out Output stream. * @throws IOException In case of error. @@ -838,7 +919,12 @@ Object read(OptimizedObjectInputStream in) throws ClassNotFoundException, IOExce case SERIALIZABLE: verifyChecksum(in.readShort()); - return in.readSerializable(cls, readObjMtds, readResolveMtd, fields); + // If no serialize method, then unmarshal as usual. + if (serTransMtd != null) + return in.readSerializable(cls, readObjMtds, readResolveMtd, + serializableFields(cls, null, MarshallerUtils.jobSenderVersion())); + else + return in.readSerializable(cls, readObjMtds, readResolveMtd, fields); default: assert false : "Unexpected type: " + type; diff --git a/modules/platforms/cpp/binary/Makefile.am b/modules/platforms/cpp/binary/Makefile.am index 4876776c811b2..ca301a6395434 100644 --- a/modules/platforms/cpp/binary/Makefile.am +++ b/modules/platforms/cpp/binary/Makefile.am @@ -26,7 +26,9 @@ AM_CPPFLAGS = \ -I$(srcdir)/include \ -I@top_srcdir@/common/include \ -I@top_srcdir@/common/os/linux/include \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/core/Makefile.am b/modules/platforms/cpp/core/Makefile.am index bbb77203da624..97523cf64992f 100644 --- a/modules/platforms/cpp/core/Makefile.am +++ b/modules/platforms/cpp/core/Makefile.am @@ -31,7 +31,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/examples/odbc-example/Makefile.am b/modules/platforms/cpp/examples/odbc-example/Makefile.am index e58410527a4f4..83cc63e207366 100644 --- a/modules/platforms/cpp/examples/odbc-example/Makefile.am +++ b/modules/platforms/cpp/examples/odbc-example/Makefile.am @@ -30,7 +30,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/../jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/examples/putget-example/Makefile.am b/modules/platforms/cpp/examples/putget-example/Makefile.am index 5301ea1a0aaeb..cf39002f7ae54 100644 --- a/modules/platforms/cpp/examples/putget-example/Makefile.am +++ b/modules/platforms/cpp/examples/putget-example/Makefile.am @@ -30,7 +30,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/../jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/examples/query-example/Makefile.am b/modules/platforms/cpp/examples/query-example/Makefile.am index 14373034ae446..01231ec38f977 100644 --- a/modules/platforms/cpp/examples/query-example/Makefile.am +++ b/modules/platforms/cpp/examples/query-example/Makefile.am @@ -30,7 +30,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/../jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/ignite/Makefile.am b/modules/platforms/cpp/ignite/Makefile.am index 625f1df670e45..2dbc4d614e23f 100644 --- a/modules/platforms/cpp/ignite/Makefile.am +++ b/modules/platforms/cpp/ignite/Makefile.am @@ -30,7 +30,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/jni/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/jni/Makefile.am b/modules/platforms/cpp/jni/Makefile.am index b9b39133d1e09..2cb4b90d8afa4 100644 --- a/modules/platforms/cpp/jni/Makefile.am +++ b/modules/platforms/cpp/jni/Makefile.am @@ -29,7 +29,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/common/os/linux/include \ -I$(JAVA_HOME)/include \ -I$(JAVA_HOME)/include/linux \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp index 008cf25f26c7e..fbd5f12532c28 100644 --- a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp @@ -160,6 +160,54 @@ struct ApiRobustnessTestSuiteFixture testCache = grid.GetCache("cache"); } + /** + * Check that SQLFetchScroll does not crash with unsupported orientation. + * + * @param orientation Fetch orientation. + */ + void CheckFetchScrollUnsupportedOrientation(SQLUSMALLINT orientation) + { + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); + + SQLRETURN ret; + + const int64_t recordsNum = 100; + + for (int i = 0; i < recordsNum; ++i) + { + TestType val; + + val.i32Field = i * 10; + + testCache.Put(i, val); + } + + int32_t i32Field = -1; + + // Binding column. + ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &i32Field, 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + SQLCHAR request[] = "SELECT i32Field FROM TestType ORDER BY _key"; + + ret = SQLExecDirect(stmt, request, SQL_NTS); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLFetchScroll(stmt, SQL_FETCH_NEXT, 0); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + BOOST_CHECK_EQUAL(i32Field, 0); + + ret = SQLFetchScroll(stmt, orientation, 0); + + // Operation is not supported. However, there should be no crash. + BOOST_CHECK(ret == SQL_ERROR); + } + /** * Destructor. */ @@ -1003,4 +1051,19 @@ BOOST_AUTO_TEST_CASE(TestSQLSpecialColumns) SQLCloseCursor(stmt); } +BOOST_AUTO_TEST_CASE(TestFetchScrollLast) +{ + CheckFetchScrollUnsupportedOrientation(SQL_FETCH_LAST); +} + +BOOST_AUTO_TEST_CASE(TestFetchScrollPrior) +{ + CheckFetchScrollUnsupportedOrientation(SQL_FETCH_PRIOR); +} + +BOOST_AUTO_TEST_CASE(TestFetchScrollFirst) +{ + CheckFetchScrollUnsupportedOrientation(SQL_FETCH_FIRST); +} + BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index eb6e153c71e85..82e9972b5ca2d 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -200,6 +200,8 @@ struct QueriesTestSuiteFixture "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType"; ret = SQLExecDirect(stmt, reinterpret_cast(request), SQL_NTS); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); if (!SQL_SUCCEEDED(ret)) BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); @@ -364,6 +366,8 @@ BOOST_AUTO_TEST_CASE(TestTwoRowsString) "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType"; ret = SQLExecDirect(stmt, request, SQL_NTS); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); if (!SQL_SUCCEEDED(ret)) BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); @@ -460,6 +464,8 @@ BOOST_AUTO_TEST_CASE(TestOneRowString) "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType"; ret = SQLExecDirect(stmt, request, SQL_NTS); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); ret = SQLFetch(stmt); if (!SQL_SUCCEEDED(ret)) @@ -522,6 +528,8 @@ BOOST_AUTO_TEST_CASE(TestOneRowStringLen) "floatField, doubleField, boolField, guidField, dateField, timestampField FROM TestType"; ret = SQLExecDirect(stmt, request, SQL_NTS); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); ret = SQLFetch(stmt); if (!SQL_SUCCEEDED(ret)) @@ -666,5 +674,4 @@ BOOST_AUTO_TEST_CASE(TestDataAtExecution) BOOST_CHECK(ret == SQL_NO_DATA); } - BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am index 3c8b37a5e97f2..b0cc5f87133fc 100644 --- a/modules/platforms/cpp/odbc/Makefile.am +++ b/modules/platforms/cpp/odbc/Makefile.am @@ -27,7 +27,9 @@ AM_CPPFLAGS = \ -I@top_srcdir@/common/include \ -I@top_srcdir@/common/os/linux/include \ -I@top_srcdir@/binary/include \ - -DIGNITE_IMPL + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS AM_CXXFLAGS = \ -Wall \ diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h index 35f1e98b4f140..db6205ed3d307 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h @@ -117,6 +117,25 @@ namespace ignite */ void UnbindAllParameters(); + /** + * Set statement attribute. + * + * @param attr Attribute type. + * @param value Value pointer. + * @param valueLen Value length. + */ + void SetAttribute(int attr, void* value, SQLINTEGER valueLen); + + /** + * Get statement attribute. + * + * @param attr Attribute type. + * @param buf Buffer for value. + * @param bufLen Buffer length. + * @param valueLen Resulting value length. + */ + void GetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER *valueLen); + /** * Get number of binded parameters. * @@ -355,6 +374,29 @@ namespace ignite */ SqlResult InternalBindParameter(uint16_t paramIdx, const app::Parameter& param); + /** + * Set statement attribute. + * Internal call. + * + * @param attr Attribute type. + * @param value Value pointer. + * @param valueLen Value length. + * @return Operation result. + */ + SqlResult InternalSetAttribute(int attr, void* value, SQLINTEGER valueLen); + + /** + * Get statement attribute. + * Internal call. + * + * @param attr Attribute type. + * @param buf Buffer for value. + * @param bufLen Buffer length. + * @param valueLen Resulting value length. + * @return Operation result. + */ + SqlResult InternalGetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER* valueLen); + /** * Get value of the column in the result set. * diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp index 7416ad2b5b663..612d51a5b4b88 100644 --- a/modules/platforms/cpp/odbc/src/odbc.cpp +++ b/modules/platforms/cpp/odbc/src/odbc.cpp @@ -854,73 +854,9 @@ namespace ignite if (!statement) return SQL_INVALID_HANDLE; - if (!valueBuf) - return SQL_ERROR; - - switch (attr) - { - case SQL_ATTR_APP_ROW_DESC: - case SQL_ATTR_APP_PARAM_DESC: - case SQL_ATTR_IMP_ROW_DESC: - case SQL_ATTR_IMP_PARAM_DESC: - { - SQLPOINTER *val = reinterpret_cast(valueBuf); - - *val = static_cast(stmt); - - break; - } - - case SQL_ATTR_ROW_ARRAY_SIZE: - { - SQLINTEGER *val = reinterpret_cast(valueBuf); - - *val = static_cast(1); - - break; - } - - case SQL_ATTR_ROWS_FETCHED_PTR: - { - SQLULEN** val = reinterpret_cast(valueBuf); - - *val = reinterpret_cast(statement->GetRowsFetchedPtr()); - - break; - } + statement->GetAttribute(attr, valueBuf, valueBufLen, valueResLen); - case SQL_ATTR_ROW_STATUS_PTR: - { - SQLUSMALLINT** val = reinterpret_cast(valueBuf); - - *val = reinterpret_cast(statement->GetRowStatusesPtr()); - - break; - } - - case SQL_ATTR_PARAM_BIND_OFFSET_PTR: - { - SQLULEN** val = reinterpret_cast(valueBuf); - - *val = reinterpret_cast(statement->GetParamBindOffsetPtr()); - - break; - } - - case SQL_ATTR_ROW_BIND_OFFSET_PTR: - { - SQLULEN** val = reinterpret_cast(valueBuf); - - *val = reinterpret_cast(statement->GetColumnBindOffsetPtr()); - - break; - } - - default: - return SQL_ERROR; - } - - return SQL_SUCCESS; + return statement->GetDiagnosticRecords().GetReturnCode(); } SQLRETURN SQLSetStmtAttr(SQLHSTMT stmt, @@ -943,53 +879,9 @@ namespace ignite if (!statement) return SQL_INVALID_HANDLE; - switch (attr) - { - case SQL_ATTR_ROW_ARRAY_SIZE: - { - SQLULEN val = reinterpret_cast(value); - - LOG_MSG("Value: %d\n", val); - - if (val != 1) - return SQL_ERROR; - - break; - } - - case SQL_ATTR_ROWS_FETCHED_PTR: - { - statement->SetRowsFetchedPtr(reinterpret_cast(value)); - - break; - } - - case SQL_ATTR_ROW_STATUS_PTR: - { - statement->SetRowStatusesPtr(reinterpret_cast(value)); - - break; - } - - case SQL_ATTR_PARAM_BIND_OFFSET_PTR: - { - statement->SetParamBindOffsetPtr(reinterpret_cast(value)); - - break; - } - - case SQL_ATTR_ROW_BIND_OFFSET_PTR: - { - statement->SetColumnBindOffsetPtr(reinterpret_cast(value)); + statement->SetAttribute(attr, value, valueLen); - break; - } - - default: - return SQL_ERROR; - } - - return SQL_SUCCESS; + return statement->GetDiagnosticRecords().GetReturnCode(); } SQLRETURN SQLPrimaryKeys(SQLHSTMT stmt, diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp index 32f7c3f0c33d2..f1a577a2adde3 100644 --- a/modules/platforms/cpp/odbc/src/statement.cpp +++ b/modules/platforms/cpp/odbc/src/statement.cpp @@ -145,6 +145,157 @@ namespace ignite paramBindings.clear(); } + void Statement::SetAttribute(int attr, void* value, SQLINTEGER valueLen) + { + IGNITE_ODBC_API_CALL(InternalSetAttribute(attr, value, valueLen)); + } + + SqlResult Statement::InternalSetAttribute(int attr, void* value, SQLINTEGER valueLen) + { + switch (attr) + { + case SQL_ATTR_ROW_ARRAY_SIZE: + { + SQLULEN val = reinterpret_cast(value); + + LOG_MSG("SQL_ATTR_ROW_ARRAY_SIZE: %d\n", val); + + if (val != 1) + { + AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, + "Fetching of more than one row by call is not supported."); + + return SQL_RESULT_ERROR; + } + + break; + } + + case SQL_ATTR_ROWS_FETCHED_PTR: + { + SetRowsFetchedPtr(reinterpret_cast(value)); + + break; + } + + case SQL_ATTR_ROW_STATUS_PTR: + { + SetRowStatusesPtr(reinterpret_cast(value)); + + break; + } + + case SQL_ATTR_PARAM_BIND_OFFSET_PTR: + { + SetParamBindOffsetPtr(reinterpret_cast(value)); + + break; + } + + case SQL_ATTR_ROW_BIND_OFFSET_PTR: + { + SetColumnBindOffsetPtr(reinterpret_cast(value)); + + break; + } + + default: + { + AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, + "Specified attribute is not supported."); + + return SQL_RESULT_ERROR; + } + } + + return SQL_RESULT_SUCCESS; + } + + void Statement::GetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER* valueLen) + { + IGNITE_ODBC_API_CALL(InternalGetAttribute(attr, buf, bufLen, valueLen)); + } + + SqlResult Statement::InternalGetAttribute(int attr, void* buf, SQLINTEGER bufLen, SQLINTEGER* valueLen) + { + if (!buf) + { + AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, "Data buffer is NULL."); + + return SQL_RESULT_ERROR; + } + + switch (attr) + { + case SQL_ATTR_APP_ROW_DESC: + case SQL_ATTR_APP_PARAM_DESC: + case SQL_ATTR_IMP_ROW_DESC: + case SQL_ATTR_IMP_PARAM_DESC: + { + SQLPOINTER *val = reinterpret_cast(buf); + + *val = static_cast(this); + + break; + } + + case SQL_ATTR_ROW_ARRAY_SIZE: + { + SQLINTEGER *val = reinterpret_cast(buf); + + *val = static_cast(1); + + break; + } + + case SQL_ATTR_ROWS_FETCHED_PTR: + { + SQLULEN** val = reinterpret_cast(buf); + + *val = reinterpret_cast(GetRowsFetchedPtr()); + + break; + } + + case SQL_ATTR_ROW_STATUS_PTR: + { + SQLUSMALLINT** val = reinterpret_cast(buf); + + *val = reinterpret_cast(GetRowStatusesPtr()); + + break; + } + + case SQL_ATTR_PARAM_BIND_OFFSET_PTR: + { + SQLULEN** val = reinterpret_cast(buf); + + *val = reinterpret_cast(GetParamBindOffsetPtr()); + + break; + } + + case SQL_ATTR_ROW_BIND_OFFSET_PTR: + { + SQLULEN** val = reinterpret_cast(buf); + + *val = reinterpret_cast(GetColumnBindOffsetPtr()); + + break; + } + + default: + { + AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, + "Specified attribute is not supported."); + + return SQL_RESULT_ERROR; + } + } + + return SQL_RESULT_SUCCESS; + } + uint16_t Statement::GetParametersNumber() { IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS; From 5a3a1960fff1dcf32961c45c0ba5149d6748d2fc Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Tue, 8 Nov 2016 17:36:35 +0300 Subject: [PATCH 325/487] Added license header. --- .../SecurityPermissionSetBuilderTest.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java b/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java index 1d951cf7cb161..f63f9a7da0a01 100644 --- a/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java @@ -1,3 +1,20 @@ +/* + * 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.plugin.security; import java.util.Map; From f697fb5786fb4ce15f581c465ff0dcb3d2bb7b14 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 8 Nov 2016 19:13:48 +0300 Subject: [PATCH 326/487] IGNITE-4185 .NET: Fix NullReferenceException in IgniteOutputCacheProvider when igniteConfiguration is missing --- .../Apache.Ignite.AspNet.Tests/App.config | 1 + .../IgniteSessionStateStoreProviderTest.cs | 21 ++++++++++++++++--- .../Apache.Ignite.AspNet/Impl/ConfigUtil.cs | 5 +++++ 3 files changed, 24 insertions(+), 3 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config index 86ee3d4544845..7d2c1d0d5d5cb 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config @@ -22,6 +22,7 @@
          +
          diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs index 2c73359c0d850..9c3b07c1a0762 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs @@ -19,6 +19,7 @@ namespace Apache.Ignite.AspNet.Tests { using System; using System.Collections.Specialized; + using System.Configuration; using System.Linq; using System.Reflection; using System.Threading; @@ -116,14 +117,28 @@ public void TestInitialization() Assert.Throws(() => stateProvider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions)); - // Invalid section. - Assert.Throws(() => + // Missing section. + var ex = Assert.Throws(() => stateProvider.Initialize("testName", new NameValueCollection { - {SectionNameAttr, "invalidSection"}, + {SectionNameAttr, "missingSection"}, {CacheNameAttr, CacheName} })); + Assert.IsInstanceOf(ex.InnerException); + + // Invalid section with missing content. + stateProvider = new IgniteSessionStateStoreProvider(); + + ex = Assert.Throws(() => + stateProvider.Initialize("testName", new NameValueCollection + { + {SectionNameAttr, "igniteConfigurationInvalid"}, + {CacheNameAttr, CacheName} + })); + + Assert.IsInstanceOf(ex.InnerException); + // Valid grid. stateProvider = GetProvider(); diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs index a162d81677ae2..fc93c7e17af34 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs @@ -86,6 +86,11 @@ private static IIgnite StartFromApplicationConfiguration(string sectionName, str "Could not find {0} with name '{1}'", typeof(IgniteConfigurationSection).Name, sectionName)); config = section.IgniteConfiguration; + + if (config == null) + throw new ConfigurationErrorsException(string.Format(CultureInfo.InvariantCulture, + "{0} with name '{1}' is defined in , but not present in configuration", + typeof(IgniteConfigurationSection).Name, sectionName)); } else config = new IgniteConfiguration {GridName = gridName}; From 69487f2c375010737311af65750a519b403fc17f Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 8 Nov 2016 19:38:28 +0300 Subject: [PATCH 327/487] .NET: Fix error messages when IgniteConfigurationSection content is missing --- .../Apache.Ignite.AspNet/Impl/ConfigUtil.cs | 2 +- .../IgniteConfigurationSectionTest.cs | 20 +++++++++++++++++++ .../Apache.Ignite.Core.Tests/app.config | 1 + .../custom_app.config | 1 + .../dotnet/Apache.Ignite.Core/Ignition.cs | 12 +++++++++++ 5 files changed, 35 insertions(+), 1 deletion(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs index fc93c7e17af34..b967518ead467 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs @@ -89,7 +89,7 @@ private static IIgnite StartFromApplicationConfiguration(string sectionName, str if (config == null) throw new ConfigurationErrorsException(string.Format(CultureInfo.InvariantCulture, - "{0} with name '{1}' is defined in , but not present in configuration", + "{0} with name '{1}' is defined in , but not present in configuration.", typeof(IgniteConfigurationSection).Name, sectionName)); } else diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs index 4e10a2bb81e36..13bd9a251aef1 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs @@ -78,6 +78,7 @@ public void TestIgniteStart() [Test] public void TestIgniteStartError() { + // Missing section in default file. var ex = Assert.Throws(() => Ignition.StartFromApplicationConfiguration("igniteConfiguration111")); @@ -85,17 +86,36 @@ public void TestIgniteStartError() ex.Message); + // Missing section body. + ex = Assert.Throws(() => + Ignition.StartFromApplicationConfiguration("igniteConfigurationMissing")); + + Assert.AreEqual("IgniteConfigurationSection with name 'igniteConfigurationMissing' " + + "is defined in , but not present in configuration.", ex.Message); + + + // Missing custom file. ex = Assert.Throws(() => Ignition.StartFromApplicationConfiguration("igniteConfiguration", "somefile")); Assert.AreEqual("Specified config file does not exist: somefile", ex.Message); + // Missing section in custom file. ex = Assert.Throws(() => Ignition.StartFromApplicationConfiguration("igniteConfiguration", "custom_app.config")); Assert.AreEqual("Could not find IgniteConfigurationSection with name 'igniteConfiguration' " + "in file 'custom_app.config'", ex.Message); + + + // Missing section body in custom file. + ex = Assert.Throws(() => + Ignition.StartFromApplicationConfiguration("igniteConfigurationMissing", "custom_app.config")); + + Assert.AreEqual("IgniteConfigurationSection with name 'igniteConfigurationMissing' in file " + + "'custom_app.config' is defined in , but not present in configuration.", + ex.Message); } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config index d22bea4468cd5..f928ed491d4ab 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config @@ -21,6 +21,7 @@
          +
          diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/custom_app.config b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/custom_app.config index 41ea39eb6576f..aec91dc8f0c34 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/custom_app.config +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/custom_app.config @@ -20,6 +20,7 @@
          +
          diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs index 7b023f3e44220..515902f166061 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs @@ -152,6 +152,12 @@ public static IIgnite StartFromApplicationConfiguration(string sectionName) throw new ConfigurationErrorsException(string.Format("Could not find {0} with name '{1}'", typeof(IgniteConfigurationSection).Name, sectionName)); + if (section.IgniteConfiguration == null) + throw new ConfigurationErrorsException( + string.Format("{0} with name '{1}' is defined in , " + + "but not present in configuration.", + typeof(IgniteConfigurationSection).Name, sectionName)); + return Start(section.IgniteConfiguration); } @@ -177,6 +183,12 @@ public static IIgnite StartFromApplicationConfiguration(string sectionName, stri string.Format("Could not find {0} with name '{1}' in file '{2}'", typeof(IgniteConfigurationSection).Name, sectionName, configPath)); + if (section.IgniteConfiguration == null) + throw new ConfigurationErrorsException( + string.Format("{0} with name '{1}' in file '{2}' is defined in , " + + "but not present in configuration.", + typeof(IgniteConfigurationSection).Name, sectionName, configPath)); + return Start(section.IgniteConfiguration); } From d88f422aeb02738d676d86ce416551b805ad154e Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Wed, 9 Nov 2016 14:25:38 +0700 Subject: [PATCH 328/487] GG-11028 Fixed resolving of host name. --- bin/ignite.bat | 5 ++ .../ignite/internal/util/IgniteUtils.java | 47 +++++++++++++++---- .../visor/misc/VisorResolveHostNameTask.java | 4 +- 3 files changed, 44 insertions(+), 12 deletions(-) diff --git a/bin/ignite.bat b/bin/ignite.bat index b999b8673b289..db686cc145a60 100644 --- a/bin/ignite.bat +++ b/bin/ignite.bat @@ -183,6 +183,11 @@ if %ERRORLEVEL% equ 0 ( :: :: set JVM_OPTS=%JVM_OPTS% -Djava.net.preferIPv4Stack=true +:: +:: Uncomment to enable reverse DNS lookup. +:: +:: set JVM_OPTS=%JVM_OPTS% -Dsun.net.spi.nameservice.provider.1=default -Dsun.net.spi.nameservice.provider.2=dns,sun + :: :: Assertions are disabled by default since version 3.5. :: If you want to enable them - set 'ENABLE_ASSERTIONS' flag to '1'. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 1e8d648e0e27e..7b011ddc15629 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -476,6 +476,9 @@ public abstract class IgniteUtils { /** */ private static volatile IgniteBiTuple, Collection> cachedLocalAddr; + /** */ + private static volatile IgniteBiTuple, Collection> cachedLocalAddrAllHostNames; + /** */ private static final ConcurrentMap> classCache = new ConcurrentHashMap8<>(); @@ -1845,41 +1848,61 @@ public void run() { */ public static IgniteBiTuple, Collection> resolveLocalAddresses(InetAddress locAddr) throws IOException, IgniteCheckedException { + return resolveLocalAddresses(locAddr, false); + } + + /** + * Returns host names consistent with {@link #resolveLocalHost(String)}. So when it returns + * a common address this method returns single host name, and when a wildcard address passed + * this method tries to collect addresses of all available interfaces. + * + * @param locAddr Local address to resolve. + * @param allHostNames If {@code true} then include host names for all addresses. + * @return Resolved available addresses and host names of given local address. + * @throws IOException If failed. + * @throws IgniteCheckedException If no network interfaces found. + */ + public static IgniteBiTuple, Collection> resolveLocalAddresses(InetAddress locAddr, + boolean allHostNames) throws IOException, IgniteCheckedException { assert locAddr != null; Collection addrs = new ArrayList<>(); Collection hostNames = new ArrayList<>(); if (locAddr.isAnyLocalAddress()) { - IgniteBiTuple, Collection> res = cachedLocalAddr; + IgniteBiTuple, Collection> res = + allHostNames ? cachedLocalAddrAllHostNames : cachedLocalAddr; if (res == null) { - List localAddrs = new ArrayList<>(); + List locAddrs = new ArrayList<>(); for (NetworkInterface itf : asIterable(NetworkInterface.getNetworkInterfaces())) { for (InetAddress addr : asIterable(itf.getInetAddresses())) { if (!addr.isLinkLocalAddress()) - localAddrs.add(addr); + locAddrs.add(addr); } } - localAddrs = filterReachable(localAddrs); + locAddrs = filterReachable(locAddrs); - for (InetAddress addr : localAddrs) - addresses(addr, addrs, hostNames); + for (InetAddress addr : locAddrs) + addresses(addr, addrs, hostNames, allHostNames); if (F.isEmpty(addrs)) throw new IgniteCheckedException("No network addresses found (is networking enabled?)."); res = F.t(addrs, hostNames); - cachedLocalAddr = res; + if (allHostNames) + cachedLocalAddrAllHostNames = res; + else + cachedLocalAddr = res; } return res; } - addresses(locAddr, addrs, hostNames); + addresses(locAddr, addrs, hostNames, allHostNames); return F.t(addrs, hostNames); } @@ -1887,16 +1910,20 @@ public static IgniteBiTuple, Collection> resolveLocal /** * @param addr Address. * @param addrs Addresses. + * @param allHostNames If {@code true} then include host names for all addresses. * @param hostNames Host names. */ - private static void addresses(InetAddress addr, Collection addrs, Collection hostNames) { + private static void addresses(InetAddress addr, Collection addrs, Collection hostNames, + boolean allHostNames) { String hostName = addr.getHostName(); String ipAddr = addr.getHostAddress(); addrs.add(ipAddr); - if (!F.isEmpty(hostName) && !addr.isLoopbackAddress()) + if (allHostNames) + hostNames.add(hostName); + else if (!F.isEmpty(hostName) && !addr.isLoopbackAddress()) hostNames.add(hostName); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorResolveHostNameTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorResolveHostNameTask.java index 0f9e0387681ea..f851e262abe96 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorResolveHostNameTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorResolveHostNameTask.java @@ -66,7 +66,7 @@ private VisorResolveHostNameJob(Void arg, boolean debug) { try { IgniteBiTuple, Collection> addrs = - IgniteUtils.resolveLocalAddresses(InetAddress.getByName("0.0.0.0")); + IgniteUtils.resolveLocalAddresses(InetAddress.getByName("0.0.0.0"), true); assert (addrs.get1() != null); assert (addrs.get2() != null); @@ -104,4 +104,4 @@ else if (!hostName.equals(ip)) return S.toString(VisorResolveHostNameJob.class, this); } } -} \ No newline at end of file +} From ac660dcaa5bf8eb20e7dd4e442e97c1cf548a827 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Wed, 9 Nov 2016 15:29:06 +0300 Subject: [PATCH 329/487] IGNITE-4183: ODBC Fixed null-values fetching issue. --- .../cpp/odbc-test/include/test_type.h | 42 ++++-- .../cpp/odbc-test/src/queries_test.cpp | 139 ++++++++++++++++-- .../cpp/odbc-test/src/sql_outer_join_test.cpp | 2 +- .../odbc/src/app/application_data_buffer.cpp | 34 ++++- 4 files changed, 187 insertions(+), 30 deletions(-) diff --git a/modules/platforms/cpp/odbc-test/include/test_type.h b/modules/platforms/cpp/odbc-test/include/test_type.h index 2a4a9798703a3..0e082516caf92 100644 --- a/modules/platforms/cpp/odbc-test/include/test_type.h +++ b/modules/platforms/cpp/odbc-test/include/test_type.h @@ -28,6 +28,7 @@ namespace ignite struct TestType { TestType() : + allNulls(false), i8Field(0), i16Field(0), i32Field(0), @@ -45,6 +46,7 @@ namespace ignite int64_t i64Field, const std::string& strField, float floatField, double doubleField, bool boolField, const Guid& guidField, const Date& dateField, const Timestamp& timestampField) : + allNulls(false), i8Field(i8Field), i16Field(i16Field), i32Field(i32Field), @@ -60,6 +62,7 @@ namespace ignite // No-op. } + bool allNulls; int8_t i8Field; int16_t i16Field; int32_t i32Field; @@ -91,17 +94,34 @@ namespace ignite void Write(BinaryWriter& writer, TestType obj) { - writer.WriteInt8("i8Field", obj.i8Field); - writer.WriteInt16("i16Field", obj.i16Field); - writer.WriteInt32("i32Field", obj.i32Field); - writer.WriteInt64("i64Field", obj.i64Field); - writer.WriteString("strField", obj.strField); - writer.WriteFloat("floatField", obj.floatField); - writer.WriteDouble("doubleField", obj.doubleField); - writer.WriteBool("boolField", obj.boolField); - writer.WriteGuid("guidField", obj.guidField); - writer.WriteDate("dateField", obj.dateField); - writer.WriteTimestamp("timestampField", obj.timestampField); + if (!obj.allNulls) + { + writer.WriteInt8("i8Field", obj.i8Field); + writer.WriteInt16("i16Field", obj.i16Field); + writer.WriteInt32("i32Field", obj.i32Field); + writer.WriteInt64("i64Field", obj.i64Field); + writer.WriteString("strField", obj.strField); + writer.WriteFloat("floatField", obj.floatField); + writer.WriteDouble("doubleField", obj.doubleField); + writer.WriteBool("boolField", obj.boolField); + writer.WriteGuid("guidField", obj.guidField); + writer.WriteDate("dateField", obj.dateField); + writer.WriteTimestamp("timestampField", obj.timestampField); + } + else + { + writer.WriteNull("i8Field"); + writer.WriteNull("i16Field"); + writer.WriteNull("i32Field"); + writer.WriteNull("i64Field"); + writer.WriteNull("strField"); + writer.WriteNull("floatField"); + writer.WriteNull("doubleField"); + writer.WriteNull("boolField"); + writer.WriteNull("guidField"); + writer.WriteNull("dateField"); + writer.WriteNull("timestampField"); + } } TestType Read(BinaryReader& reader) diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 82e9972b5ca2d..a7fc7a9eb55fa 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -250,14 +250,14 @@ struct QueriesTestSuiteFixture BOOST_CHECK_EQUAL(columns[9], 0); BOOST_CHECK_EQUAL(columns[10], 0); - BOOST_CHECK_EQUAL(columnLens[0], 0); - BOOST_CHECK_EQUAL(columnLens[1], 0); - BOOST_CHECK_EQUAL(columnLens[2], 0); - BOOST_CHECK_EQUAL(columnLens[3], 0); - BOOST_CHECK_EQUAL(columnLens[4], 0); - BOOST_CHECK_EQUAL(columnLens[5], 0); - BOOST_CHECK_EQUAL(columnLens[6], 0); - BOOST_CHECK_EQUAL(columnLens[7], 0); + BOOST_CHECK_EQUAL(columnLens[0], static_cast(sizeof(T))); + BOOST_CHECK_EQUAL(columnLens[1], static_cast(sizeof(T))); + BOOST_CHECK_EQUAL(columnLens[2], static_cast(sizeof(T))); + BOOST_CHECK_EQUAL(columnLens[3], static_cast(sizeof(T))); + BOOST_CHECK_EQUAL(columnLens[4], static_cast(sizeof(T))); + BOOST_CHECK_EQUAL(columnLens[5], static_cast(sizeof(T))); + BOOST_CHECK_EQUAL(columnLens[6], static_cast(sizeof(T))); + BOOST_CHECK_EQUAL(columnLens[7], static_cast(sizeof(T))); BOOST_CHECK_EQUAL(columnLens[8], SQL_NO_TOTAL); BOOST_CHECK_EQUAL(columnLens[9], SQL_NO_TOTAL); BOOST_CHECK_EQUAL(columnLens[10], SQL_NO_TOTAL); @@ -296,32 +296,32 @@ BOOST_AUTO_TEST_CASE(TestConnectionProtocolVersion_1_6_0) BOOST_AUTO_TEST_CASE(TestTwoRowsInt8) { - CheckTwoRowsInt(SQL_C_STINYINT); + CheckTwoRowsInt(SQL_C_STINYINT); } BOOST_AUTO_TEST_CASE(TestTwoRowsUint8) { - CheckTwoRowsInt(SQL_C_UTINYINT); + CheckTwoRowsInt(SQL_C_UTINYINT); } BOOST_AUTO_TEST_CASE(TestTwoRowsInt16) { - CheckTwoRowsInt(SQL_C_SSHORT); + CheckTwoRowsInt(SQL_C_SSHORT); } BOOST_AUTO_TEST_CASE(TestTwoRowsUint16) { - CheckTwoRowsInt(SQL_C_USHORT); + CheckTwoRowsInt(SQL_C_USHORT); } BOOST_AUTO_TEST_CASE(TestTwoRowsInt32) { - CheckTwoRowsInt(SQL_C_SLONG); + CheckTwoRowsInt(SQL_C_SLONG); } BOOST_AUTO_TEST_CASE(TestTwoRowsUint32) { - CheckTwoRowsInt(SQL_C_ULONG); + CheckTwoRowsInt(SQL_C_ULONG); } BOOST_AUTO_TEST_CASE(TestTwoRowsInt64) @@ -674,4 +674,115 @@ BOOST_AUTO_TEST_CASE(TestDataAtExecution) BOOST_CHECK(ret == SQL_NO_DATA); } +BOOST_AUTO_TEST_CASE(TestNullFields) +{ + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache"); + + SQLRETURN ret; + + TestType in(1, 2, 3, 4, "5", 6.0f, 7.0, true, Guid(8, 9), BinaryUtils::MakeDateGmt(1987, 6, 5), + BinaryUtils::MakeTimestampGmt(1998, 12, 27, 1, 2, 3, 456)); + + TestType inNull; + + inNull.allNulls = true; + + testCache.Put(1, in); + testCache.Put(2, inNull); + testCache.Put(3, in); + + const size_t columnsCnt = 10; + + SQLLEN columnLens[columnsCnt] = { 0 }; + + int8_t i8Column; + int16_t i16Column; + int32_t i32Column; + int64_t i64Column; + char strColumn[ODBC_BUFFER_SIZE]; + float floatColumn; + double doubleColumn; + bool boolColumn; + SQL_DATE_STRUCT dateColumn; + SQL_TIMESTAMP_STRUCT timestampColumn; + + // Binding columns. + ret = SQLBindCol(stmt, 1, SQL_C_STINYINT, &i8Column, 0, &columnLens[0]); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 2, SQL_C_SSHORT, &i16Column, 0, &columnLens[1]); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 3, SQL_C_SLONG, &i32Column, 0, &columnLens[2]); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 4, SQL_C_SBIGINT, &i64Column, 0, &columnLens[3]); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 5, SQL_C_CHAR, &strColumn, ODBC_BUFFER_SIZE, &columnLens[4]); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 6, SQL_C_FLOAT, &floatColumn, 0, &columnLens[5]); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 7, SQL_C_DOUBLE, &doubleColumn, 0, &columnLens[6]); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 8, SQL_C_BIT, &boolColumn, 0, &columnLens[7]); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 9, SQL_C_DATE, &dateColumn, 0, &columnLens[8]); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + ret = SQLBindCol(stmt, 10, SQL_C_TIMESTAMP, ×tampColumn, 0, &columnLens[9]); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + SQLCHAR request[] = "SELECT i8Field, i16Field, i32Field, i64Field, strField, " + "floatField, doubleField, boolField, dateField, timestampField FROM TestType ORDER BY _key"; + + ret = SQLExecDirect(stmt, request, SQL_NTS); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + // Fetching the first non-null row. + ret = SQLFetch(stmt); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + // Checking that columns are not null. + for (SQLSMALLINT i = 0; i < columnsCnt; ++i) + BOOST_CHECK_NE(columnLens[i], SQL_NULL_DATA); + + // Fetching null row. + ret = SQLFetch(stmt); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + // Checking that columns are null. + for (SQLSMALLINT i = 0; i < columnsCnt; ++i) + BOOST_CHECK_EQUAL(columnLens[i], SQL_NULL_DATA); + + // Fetching the last non-null row. + ret = SQLFetch(stmt); + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + // Checking that columns are not null. + for (SQLSMALLINT i = 0; i < columnsCnt; ++i) + BOOST_CHECK_NE(columnLens[i], SQL_NULL_DATA); + + ret = SQLFetch(stmt); + BOOST_CHECK(ret == SQL_NO_DATA); +} + BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc-test/src/sql_outer_join_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_outer_join_test.cpp index 426041b8602e5..56f521939046b 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_outer_join_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_outer_join_test.cpp @@ -211,7 +211,7 @@ BOOST_AUTO_TEST_CASE(TestOuterJoinOpsLess) BOOST_CHECK_NE(columnsLen[0], SQL_NULL_DATA); BOOST_CHECK_EQUAL(columns[0], 30); - BOOST_CHECK_EQUAL(columnsLen[1], SQL_NULL_DATA); + BOOST_CHECK_NE(columnsLen[1], SQL_NULL_DATA); ret = SQLFetch(stmt); BOOST_CHECK(ret == SQL_NO_DATA); diff --git a/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp b/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp index 1438b0c9e238b..078e6916b6f77 100644 --- a/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp +++ b/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp @@ -182,6 +182,10 @@ namespace ignite memcpy(out->val, &uval, std::min(SQL_MAX_NUMERIC_LEN, sizeof(uval))); } + + if (resLenPtr) + *resLenPtr = static_cast(sizeof(SQL_NUMERIC_STRUCT)); + break; } @@ -237,12 +241,16 @@ namespace ignite void ApplicationDataBuffer::PutNumToNumBuffer(Tin value) { void* dataPtr = GetData(); + SqlLen* resLenPtr = GetResLen(); if (dataPtr) { Tbuf* out = reinterpret_cast(dataPtr); *out = static_cast(value); } + + if (resLenPtr) + *resLenPtr = static_cast(sizeof(Tbuf)); } template @@ -448,6 +456,8 @@ namespace ignite { using namespace type_traits; + SqlLen* resLenPtr = GetResLen(); + switch (type) { case IGNITE_ODBC_C_TYPE_CHAR: @@ -476,13 +486,14 @@ namespace ignite for (size_t i = 0; i < sizeof(guid->Data4); ++i) guid->Data4[i] = (lsb >> (sizeof(guid->Data4) - i - 1) * 8) & 0xFF; + if (resLenPtr) + *resLenPtr = static_cast(sizeof(SQLGUID)); + break; } default: { - SqlLen* resLenPtr = GetResLen(); - if (resLenPtr) *resLenPtr = SQL_NO_TOTAL; } @@ -573,6 +584,8 @@ namespace ignite { using namespace type_traits; + SqlLen* resLenPtr = GetResLen(); + switch (type) { case IGNITE_ODBC_C_TYPE_SIGNED_TINYINT: @@ -637,6 +650,9 @@ namespace ignite numeric->sign = unscaled.GetSign() < 0 ? 0 : 1; numeric->precision = unscaled.GetPrecision(); + if (resLenPtr) + *resLenPtr = static_cast(sizeof(SQL_NUMERIC_STRUCT)); + break; } @@ -644,8 +660,6 @@ namespace ignite case IGNITE_ODBC_C_TYPE_BINARY: default: { - SqlLen* resLenPtr = GetResLen(); - if (resLenPtr) *resLenPtr = SQL_NO_TOTAL; } @@ -716,6 +730,9 @@ namespace ignite buffer->month = tmTime.tm_mon + 1; buffer->day = tmTime.tm_mday; + if (resLenPtr) + *resLenPtr = static_cast(sizeof(SQL_DATE_STRUCT)); + break; } @@ -731,6 +748,9 @@ namespace ignite buffer->second = tmTime.tm_sec; buffer->fraction = 0; + if (resLenPtr) + *resLenPtr = static_cast(sizeof(SQL_TIMESTAMP_STRUCT)); + break; } @@ -830,6 +850,9 @@ namespace ignite buffer->month = tmTime.tm_mon + 1; buffer->day = tmTime.tm_mday; + if (resLenPtr) + *resLenPtr = static_cast(sizeof(SQL_DATE_STRUCT)); + break; } @@ -845,6 +868,9 @@ namespace ignite buffer->second = tmTime.tm_sec; buffer->fraction = value.GetSecondFraction(); + if (resLenPtr) + *resLenPtr = static_cast(sizeof(SQL_TIMESTAMP_STRUCT)); + break; } From cdae2ab76d403aef9a0bd209fc7497dc6cfdfc08 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Wed, 9 Nov 2016 16:25:30 +0300 Subject: [PATCH 330/487] IGNITE-3873: Added WiX script to generate ODBC installer. --- modules/platforms/cpp/DEVNOTES.txt | 12 ++ modules/platforms/cpp/odbc/README.txt | 23 ++-- .../cpp/odbc/install/ignite-odbc-amd64.wxs | 114 ++++++++++++++++++ .../cpp/odbc/install/ignite-odbc-x86.wxs | 114 ++++++++++++++++++ pom.xml | 8 ++ 5 files changed, 263 insertions(+), 8 deletions(-) create mode 100644 modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs create mode 100644 modules/platforms/cpp/odbc/install/ignite-odbc-x86.wxs diff --git a/modules/platforms/cpp/DEVNOTES.txt b/modules/platforms/cpp/DEVNOTES.txt index cd0a1547579c6..924b0d8281134 100644 --- a/modules/platforms/cpp/DEVNOTES.txt +++ b/modules/platforms/cpp/DEVNOTES.txt @@ -58,6 +58,18 @@ Building binaries: * If you want to build ODBC driver then you should explicitly build it as it is disabled in the solution file by default. In IDE it can be done by clicking on the ODBC project with the right mouse button and choosing "Build" option. + +Building installers: + * Install WiX Toolset if you do not have it yet. + * Add WiX Toolset "bin" directory to your PATH environmental variable. + * Build ODBC drivers: Release|x64 for 64-bit version and Release|Win32 for 32-bit version. + * Open terminal and navigate to the directory $IGNITE_HOME/platforms/cpp/odbc/install + * Execute the following commands one by one to build 32-bit driver: + * candle.exe ignite-odbc-x86.wxs + * light.exe -ext WixUIExtension ignite-odbc-x86.wixobj + * Execute the following commands one by one to build 64-bit driver: + * candle.exe ignite-odbc-amd64.wxs + * light.exe -ext WixUIExtension ignite-odbc-amd64.wixobj Building in later versions of Visual Studio: * Open project\vs\ignite.sln or project\vs\ignite_86.sln in Visual Studio diff --git a/modules/platforms/cpp/odbc/README.txt b/modules/platforms/cpp/odbc/README.txt index 8f0c805bdb078..fe40a5cf950c2 100644 --- a/modules/platforms/cpp/odbc/README.txt +++ b/modules/platforms/cpp/odbc/README.txt @@ -58,14 +58,21 @@ ODBC driver you should perfrom the following steps: Installing ODBC driver on Windows ======================================= -For 32-bit Windows you should use 32-bit version of the driver while for the -64-bit Windows you can use 64-bit driver as well as 32-bit. - -To install driver on Windows you should first choose a directory on your -filesystem where your driver or drivers will be located. Once you have -choosen the place you should put your driver there and ensure that all driver -dependencies can be resolved i.e. they can be found either in the %PATH% or -in the same directory as the driver. +Note, that for 32-bit Windows you should use 32-bit version of the driver +while for the 64-bit Windows you can use both 64-bit and 32-bit versions of the +driver. You may want to use 32-bit driver on 64-bit system for 32-bit +applications. + +There are two ways to install ODBC driver currently. The first one is to use +32-bit or 64-bit installer. This is the most simple way and you are recommended +to stick to it by default. + +However there is also another way to install driver manually using scripts. If +you choose this method you should first choose a directory on your filesystem +where your driver or drivers will be located. Once you have choosen the place +you should put your driver there and ensure that all driver dependencies can be +resolved i.e. they can be found either in the %PATH% or in the same directory +as the driver. After that you should use one of the install scripts from the directory %IGNITE_HOME%/platforms/cpp/odbc/install. Note that most likely you will diff --git a/modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs b/modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs new file mode 100644 index 0000000000000..cb6ab5410b205 --- /dev/null +++ b/modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs @@ -0,0 +1,114 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 1 + "1"]]> + + 1 + + NOT Installed + Installed AND PATCH + + 1 + 1 + NOT WIXUI_DONTVALIDATEPATH + "1"]]> + WIXUI_DONTVALIDATEPATH OR WIXUI_INSTALLDIR_VALID="1" + 1 + 1 + + NOT Installed + Installed AND NOT PATCH + Installed AND PATCH + + 1 + + 1 + 1 + 1 + + + + + + + + + \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/install/ignite-odbc-x86.wxs b/modules/platforms/cpp/odbc/install/ignite-odbc-x86.wxs new file mode 100644 index 0000000000000..6ee1cdf10e2ea --- /dev/null +++ b/modules/platforms/cpp/odbc/install/ignite-odbc-x86.wxs @@ -0,0 +1,114 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 1 + "1"]]> + + 1 + + NOT Installed + Installed AND PATCH + + 1 + 1 + NOT WIXUI_DONTVALIDATEPATH + "1"]]> + WIXUI_DONTVALIDATEPATH OR WIXUI_INSTALLDIR_VALID="1" + 1 + 1 + + NOT Installed + Installed AND NOT PATCH + Installed AND PATCH + + 1 + + 1 + 1 + 1 + + + + + + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index 36051b4aca0b4..8069e381a0cba 100644 --- a/pom.xml +++ b/pom.xml @@ -891,6 +891,14 @@ + + + + + + + + From 1093819ac0f3e7a0faacde59919117b8977e6d5b Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Wed, 9 Nov 2016 18:19:01 +0300 Subject: [PATCH 331/487] IGNITE-4201: Fixed version fix maven step. --- pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/pom.xml b/pom.xml index 4f4198ccf3c42..942410fea47ca 100644 --- a/pom.xml +++ b/pom.xml @@ -888,6 +888,7 @@ + From bac0cba7fddd412dfbff98163afbc15d81d5e0d4 Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Thu, 10 Nov 2016 09:02:41 +0300 Subject: [PATCH 332/487] ignite-4044 always authenticate local node --- .../ignite/spi/discovery/tcp/ServerImpl.java | 112 ++++++++++++++---- 1 file changed, 89 insertions(+), 23 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 55e5c8976b61c..0de787dbf8111 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -815,33 +815,24 @@ private void joinTopology() throws IgniteSpiException { SecurityCredentials locCred = (SecurityCredentials)locNode.getAttributes() .get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS); + boolean auth = false; + + if (spi.nodeAuth != null && spi.nodeAuth.isGlobalNodeAuthentication()) { + localAuthentication(locCred); + + auth = true; + } + // Marshal credentials for backward compatibility and security. - marshalCredentials(locNode); + marshalCredentials(locNode, locCred); while (true) { if (!sendJoinRequestMessage()) { if (log.isDebugEnabled()) log.debug("Join request message has not been sent (local node is the first in the topology)."); - if (spi.nodeAuth != null) { - // Authenticate local node. - try { - SecurityContext subj = spi.nodeAuth.authenticateNode(locNode, locCred); - - if (subj == null) - throw new IgniteSpiException("Authentication failed for local node: " + locNode.id()); - - Map attrs = new HashMap<>(locNode.attributes()); - - attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, U.marshal(spi.marshaller(), subj)); - attrs.remove(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS); - - locNode.setAttributes(attrs); - } - catch (IgniteException | IgniteCheckedException e) { - throw new IgniteSpiException("Failed to authenticate local node (will shutdown local node).", e); - } - } + if (!auth && spi.nodeAuth != null) + localAuthentication(locCred); locNode.order(1); locNode.internalOrder(1); @@ -920,6 +911,8 @@ else if (spiState == LOOPBACK_PROBLEM) { } } + locNode.attributes().remove(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS); + assert locNode.order() != 0; assert locNode.internalOrder() != 0; @@ -927,6 +920,33 @@ else if (spiState == LOOPBACK_PROBLEM) { log.debug("Discovery SPI has been connected to topology with order: " + locNode.internalOrder()); } + /** + * Authenticate local node. + * + * @param locCred Local security credentials for authentication. + * @throws IgniteSpiException If any error occurs. + */ + private void localAuthentication(SecurityCredentials locCred){ + assert spi.nodeAuth != null; + assert locCred != null; + + try { + SecurityContext subj = spi.nodeAuth.authenticateNode(locNode, locCred); + + if (subj == null) + throw new IgniteSpiException("Authentication failed for local node: " + locNode.id()); + + Map attrs = new HashMap<>(locNode.attributes()); + + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, U.marshal(spi.marshaller(), subj)); + + locNode.setAttributes(attrs); + + } catch (IgniteException | IgniteCheckedException e) { + throw new IgniteSpiException("Failed to authenticate local node (will shutdown local node).", e); + } + } + /** * Tries to send join request message to a random node presenting in topology. * Address is provided by {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder} and message is @@ -1241,15 +1261,15 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) * Marshalls credentials with discovery SPI marshaller (will replace attribute value). * * @param node Node to marshall credentials for. + * @param cred Credentials for marshall. * @throws IgniteSpiException If marshalling failed. */ - private void marshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException { + private void marshalCredentials(TcpDiscoveryNode node, SecurityCredentials cred) throws IgniteSpiException { try { // Use security-unsafe getter. Map attrs = new HashMap<>(node.getAttributes()); - attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS, - U.marshal(spi.marshaller(), attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS))); + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS, spi.marshaller().marshal(cred)); node.setAttributes(attrs); } @@ -3906,6 +3926,52 @@ else if (!locNodeId.equals(node.id()) && ring.node(node.id()) != null) { if (top != null && !top.isEmpty()) { spi.gridStartTime = msg.gridStartTime(); + if (spi.nodeAuth != null && spi.nodeAuth.isGlobalNodeAuthentication()) { + TcpDiscoveryAbstractMessage authFail = + new TcpDiscoveryAuthFailedMessage(locNodeId, spi.locHost); + + try { + ClassLoader cl = U.resolveClassLoader(spi.ignite().configuration()); + + byte[] rmSubj = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT); + byte[] locSubj = locNode.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT); + + SecurityContext rmCrd = spi.marshaller().unmarshal(rmSubj, cl); + SecurityContext locCrd = spi.marshaller().unmarshal(locSubj, cl); + + if (!permissionsEqual(locCrd.subject().permissions(), + rmCrd.subject().permissions())) { + // Node has not pass authentication. + LT.warn(log, + null, + "Failed to authenticate local node " + + "(local authentication result is different from rest of topology) " + + "[nodeId=" + node.id() + ", addrs=" + U.addressesAsString(node) + ']', + "Authentication failed [nodeId=" + U.id8(node.id()) + + ", addrs=" + U.addressesAsString(node) + ']'); + + joinRes.set(authFail); + + spiState = AUTH_FAILED; + + mux.notifyAll(); + + return; + } + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to verify node permissions consistency (will drop the node): " + node, e); + + joinRes.set(authFail); + + spiState = AUTH_FAILED; + + mux.notifyAll(); + + return; + } + } + for (TcpDiscoveryNode n : top) { assert n.internalOrder() < node.internalOrder() : "Invalid node [topNode=" + n + ", added=" + node + ']'; From 26daa57ca82254d68ac04a7b33223c6eb5ade0e4 Mon Sep 17 00:00:00 2001 From: sboikov Date: Thu, 10 Nov 2016 11:17:29 +0300 Subject: [PATCH 333/487] Fixed javadoc. --- .../spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java index 39170eadf7cf0..783a113ea1ff6 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java @@ -99,7 +99,7 @@ protected TcpDiscoveryAbstractMessage(TcpDiscoveryAbstractMessage msg) { } /** - * @return + * @return {@code True} if need use trace logging for this message (to reduce amount of logging with debug level). */ public boolean traceLogLevel() { return false; From 8b59f4e76138e08e80aa219c1a9cf0c3df6fdb4b Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 10 Nov 2016 14:43:00 +0300 Subject: [PATCH 334/487] Backport commit of the following: commit 612eb3daffe608995aac28eed019b3e6ef9d66d3 Author: Aleksei Scherbakov Date: Fri Aug 19 13:28:39 2016 +0300 ignite-2795 Support 'copyOnRead' for SQL queries --- .../processors/cache/GridCacheContext.java | 8 + .../query/h2/GridH2ResultSetIterator.java | 62 ++++- .../processors/query/h2/IgniteH2Indexing.java | 4 +- .../query/h2/opt/GridH2ValueCacheObject.java | 10 +- .../h2/twostep/GridMapQueryExecutor.java | 34 ++- .../h2/twostep/GridReduceQueryExecutor.java | 2 +- .../cache/CacheSqlQueryValueCopySelfTest.java | 226 ++++++++++++++++++ .../IgniteCacheQuerySelfTestSuite2.java | 2 + 8 files changed, 338 insertions(+), 10 deletions(-) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.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 dc9c766dcd993..30f1c4bd63f18 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 @@ -1698,6 +1698,14 @@ public boolean useOffheapEntry() { (cacheCfg.getMemoryMode() == OFFHEAP_TIERED || cacheCfg.getMemoryMode() == OFFHEAP_VALUES); } + /** + * @return {@code True} if the value for the cache object has to be copied because + * of {@link CacheConfiguration#isCopyOnRead()}. + */ + public boolean needValueCopy() { + return affNode && cacheCfg.isCopyOnRead() && cacheCfg.getMemoryMode() != OFFHEAP_VALUES; + } + /** * Converts temporary offheap object to heap-based. * diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java index 3603bb5237633..e0680d35291a5 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java @@ -17,23 +17,48 @@ package org.apache.ignite.internal.processors.query.h2; +import java.lang.reflect.Field; import java.sql.ResultSet; import java.sql.SQLException; import java.util.NoSuchElementException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; - +import org.h2.jdbc.JdbcResultSet; +import org.h2.result.ResultInterface; +import org.h2.value.Value; /** * Iterator over result set. */ public abstract class GridH2ResultSetIterator extends GridCloseableIteratorAdapter { + /** */ + private static final Field RESULT_FIELD; + + /** + * Initialize. + */ + static { + try { + RESULT_FIELD = JdbcResultSet.class.getDeclaredField("result"); + + RESULT_FIELD.setAccessible(true); + } + catch (NoSuchFieldException e) { + throw new IllegalStateException("Check H2 version in classpath.", e); + } + } + /** */ private static final long serialVersionUID = 0L; + /** */ + private final ResultInterface res; + /** */ private final ResultSet data; @@ -49,12 +74,20 @@ public abstract class GridH2ResultSetIterator extends GridCloseableIteratorAd /** * @param data Data array. * @param closeStmt If {@code true} closes result set statement when iterator is closed. + * @param needCpy {@code True} if need copy cache object's value. * @throws IgniteCheckedException If failed. */ - protected GridH2ResultSetIterator(ResultSet data, boolean closeStmt) throws IgniteCheckedException { + protected GridH2ResultSetIterator(ResultSet data, boolean closeStmt, boolean needCpy) throws IgniteCheckedException { this.data = data; this.closeStmt = closeStmt; + try { + res = needCpy ? (ResultInterface)RESULT_FIELD.get(data) : null; + } + catch (IllegalAccessException e) { + throw new IllegalStateException(e); // Must not happen. + } + if (data != null) { try { row = new Object[data.getMetaData().getColumnCount()]; @@ -78,8 +111,27 @@ private boolean fetchNext() { if (!data.next()) return false; - for (int c = 0; c < row.length; c++) - row[c] = data.getObject(c + 1); + if (res != null) { + Value[] values = res.currentRow(); + + for (int c = 0; c < row.length; c++) { + Value val = values[c]; + + if (val instanceof GridH2ValueCacheObject) { + GridH2ValueCacheObject valCacheObj = (GridH2ValueCacheObject)values[c]; + + GridCacheContext cctx = valCacheObj.getCacheContext(); + + row[c] = valCacheObj.getObject(cctx != null && cctx.needValueCopy()); + } + else + row[c] = val.getObject(); + } + } + else { + for (int c = 0; c < row.length; c++) + row[c] = data.getObject(c + 1); + } return true; } @@ -134,6 +186,6 @@ private boolean fetchNext() { /** {@inheritDoc} */ @Override public String toString() { - return S.toString((Class)getClass(), this); + return S.toString(GridH2ResultSetIterator.class, this); } } \ No newline at end of file diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java index ab332c123b87b..5c2fab564e421 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java @@ -2230,7 +2230,7 @@ private static class FieldsIterator extends GridH2ResultSetIterator> { * @throws IgniteCheckedException If failed. */ protected FieldsIterator(ResultSet data) throws IgniteCheckedException { - super(data, false); + super(data, false, true); } /** {@inheritDoc} */ @@ -2255,7 +2255,7 @@ private static class KeyValIterator extends GridH2ResultSetIterator cctx, UUID qrySrcNodeId, GridCacheSqlQuery qry) { + private QueryResult(ResultSet rs, GridCacheContext cctx, UUID qrySrcNodeId, GridCacheSqlQuery qry) { this.rs = rs; this.cctx = cctx; this.qry = qry; this.qrySrcNodeId = qrySrcNodeId; + this.cpNeeded = cctx.isLocalNode(qrySrcNodeId); try { res = (ResultInterface)RESULT_FIELD.get(rs); @@ -803,6 +808,33 @@ synchronized boolean fetchNextPage(List rows, int pageSize) { Value[] row = res.currentRow(); + if (cpNeeded) { + boolean copied = false; + + for (int j = 0; j < row.length; j++) { + Value val = row[j]; + + if (val instanceof GridH2ValueCacheObject) { + GridH2ValueCacheObject valCacheObj = (GridH2ValueCacheObject)val; + + GridCacheContext cctx = valCacheObj.getCacheContext(); + + if (cctx != null && cctx.needValueCopy()) { + row[j] = new GridH2ValueCacheObject(valCacheObj.getCacheContext(), valCacheObj.getCacheObject()) { + @Override public Object getObject() { + return getObject(true); + } + }; + + copied = true; + } + } + } + + if (i == 0 && !copied) + cpNeeded = false; // No copy on read caches, skip next checks. + } + assert row != null; if (readEvt) { 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 3fdbf42bf6a39..29ad94a041e69 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 @@ -1289,7 +1289,7 @@ private static class Iter extends GridH2ResultSetIterator> { * @throws IgniteCheckedException If failed. */ protected Iter(ResultSet data) throws IgniteCheckedException { - super(data, true); + super(data, true, false); } /** {@inheritDoc} */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java new file mode 100644 index 0000000000000..e47e893530384 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.List; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests modification of values returned by query iterators with enabled copy on read. + */ +public class CacheSqlQueryValueCopySelfTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int KEYS = 100; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + if ("client".equals(cfg.getGridName())) + cfg.setClientMode(true); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + + CacheConfiguration cc = new CacheConfiguration<>(); + + cc.setCopyOnRead(true); + cc.setIndexedTypes(Integer.class, Value.class); + + cfg.setCacheConfiguration(cc); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGridsMultiThreaded(3); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + IgniteCache cache = grid(0).cache(null); + + for (int i = 0; i < KEYS; i++) + cache.put(i, new Value("before")); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + IgniteCache cache = grid(0).cache(null); + + cache.removeAll(); + + super.afterTest(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** + * Tests two step query from dedicated client. + * + * @throws Exception If failed. + */ + public void testTwoStepSqlClientQuery() throws Exception { + try (Ignite client = startGrid("client")) { + IgniteCache cache = client.cache(null); + + List> all = cache.query( + new SqlQuery(Value.class, "select * from Value")).getAll(); + + assertEquals(KEYS, all.size()); + + for (Cache.Entry entry : all) + entry.getValue().str = "after"; + + check(cache); + + QueryCursor> qry = cache.query(new SqlFieldsQuery("select _val from Value")); + + List> all0 = qry.getAll(); + + assertEquals(KEYS, all0.size()); + + for (List entry : all0) + ((Value)entry.get(0)).str = "after"; + + check(cache); + } + } + + /** + * Test two step query without local reduce phase. + */ + public void testTwoStepSkipReduceSqlQuery() { + IgniteCache cache = grid(0).cache(null); + + List> all = cache.query( + new SqlQuery(Value.class, "select * from Value").setPageSize(3)).getAll(); + + assertEquals(KEYS, all.size()); + + for (Cache.Entry entry : all) + entry.getValue().str = "after"; + + check(cache); + } + + /** + * Test two step query value copy. + */ + public void testTwoStepReduceSqlQuery() { + IgniteCache cache = grid(0).cache(null); + + QueryCursor> qry = cache.query(new SqlFieldsQuery("select _val from Value order by _key")); + + List> all = qry.getAll(); + + assertEquals(KEYS, all.size()); + + for (List entry : all) + ((Value)entry.get(0)).str = "after"; + + check(cache); + } + + /** + * Tests local sql query. + */ + public void testLocalSqlQuery() { + IgniteCache cache = grid(0).cache(null); + + SqlQuery qry = new SqlQuery<>(Value.class.getSimpleName(), "select * from Value"); + qry.setLocal(true); + + List> all = cache.query(qry).getAll(); + + assertFalse(all.isEmpty()); + + for (Cache.Entry entry : all) + entry.getValue().str = "after"; + + check(cache); + } + + /** + * Tests local sql query. + */ + public void testLocalSqlFieldsQuery() { + IgniteCache cache = grid(0).cache(null); + + QueryCursor> cur = cache.query(new SqlFieldsQuery("select _val from Value").setLocal(true)); + + List> all = cur.getAll(); + + assertFalse(all.isEmpty()); + + for (List entry : all) + ((Value)entry.get(0)).str = "after"; + + check(cache); + } + + /** */ + private static class Value { + /** */ + private String str; + + /** + * @param str String. + */ + public Value(String str) { + this.str = str; + } + } + + /** + * @param cache Cache. + */ + private void check(IgniteCache cache) { + int cnt = 0; + + // Value should be not modified by previous assignment. + for (Cache.Entry entry : cache) { + cnt++; + + assertEquals("before", entry.getValue().str); + } + + assertEquals(KEYS, cnt); + } +} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java index 9128f76e45b5b..4b4a576aed674 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java @@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest; import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest; import org.apache.ignite.internal.processors.cache.CacheScanPartitionQueryFallbackSelfTest; +import org.apache.ignite.internal.processors.cache.CacheSqlQueryValueCopySelfTest; import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheNoClassQuerySelfTest; @@ -114,6 +115,7 @@ public static TestSuite suite() throws Exception { // Other. suite.addTestSuite(CacheQueryNewClientSelfTest.class); suite.addTestSuite(CacheOffheapBatchIndexingSingleTypeTest.class); + suite.addTestSuite(CacheSqlQueryValueCopySelfTest.class); return suite; } From b7499828c928e02e8e554f960f3754e4d08bfbe0 Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Thu, 10 Nov 2016 16:10:21 +0300 Subject: [PATCH 335/487] IGNITE-500 CacheLoadingConcurrentGridStartSelfTest fails (DataStreamer data loss at unstable topology in !allowOverwrite mode fixed) --- .../org/apache/ignite/IgniteDataStreamer.java | 2 +- .../processors/cache/GridCacheMapEntry.java | 5 +- .../cache/GridCacheMvccManager.java | 77 +++ .../GridCachePartitionExchangeManager.java | 5 + .../cache/GridCacheSharedContext.java | 1 + .../datastreamer/DataStreamProcessor.java | 104 ++- .../datastreamer/DataStreamerImpl.java | 603 +++++++++++++----- .../ignite/internal/util/GridLogThrottle.java | 29 +- .../cache/IgniteCacheDynamicStopSelfTest.java | 48 +- ...cheLoadingConcurrentGridStartSelfTest.java | 251 +++++++- ...urrentGridStartSelfTestAllowOverwrite.java | 30 + .../DataStreamProcessorSelfTest.java | 4 +- .../DataStreamerImplSelfTest.java | 170 +++-- .../DataStreamerMultiThreadedSelfTest.java | 2 - .../datastreamer/DataStreamerTimeoutTest.java | 92 ++- .../testsuites/IgniteCacheTestSuite2.java | 2 + 16 files changed, 1120 insertions(+), 305 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTestAllowOverwrite.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteDataStreamer.java b/modules/core/src/main/java/org/apache/ignite/IgniteDataStreamer.java index 484fee9e9abda..4e00d661081eb 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteDataStreamer.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteDataStreamer.java @@ -291,7 +291,7 @@ public interface IgniteDataStreamer extends AutoCloseable { * @throws IllegalStateException If grid has been concurrently stopped or * {@link #close(boolean)} has already been called on streamer. */ - public IgniteFuture removeData(K key) throws CacheException, IgniteInterruptedException, IllegalStateException; + public IgniteFuture removeData(K key) throws CacheException, IgniteInterruptedException, IllegalStateException; /** * Adds data for streaming on remote node. This method can be called from multiple diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 59966728eb8d7..950153f6c9049 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -3444,11 +3444,8 @@ protected boolean hasValueUnlocked() { if (val == null) { skipQryNtf = true; - if (cctx.deferredDelete() && !isInternal()) { - assert !deletedUnlocked(); - + if (cctx.deferredDelete() && !deletedUnlocked() && !isInternal()) deletedUnlocked(true); - } } else if (deletedUnlocked()) deletedUnlocked(false); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java index c4db01e855220..c57e17cf8846f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java @@ -46,6 +46,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashSet; import org.apache.ignite.internal.util.GridConcurrentFactory; +import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -108,6 +109,9 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter { private final ConcurrentMap> atomicFuts = new ConcurrentHashMap8<>(); + /** Pending data streamer futures. */ + private final GridConcurrentHashSet dataStreamerFuts = new GridConcurrentHashSet<>(); + /** */ private final ConcurrentMap> futs = new ConcurrentHashMap8<>(); @@ -445,6 +449,13 @@ public Collection> atomicFutures() { return atomicFuts.values(); } + /** + * @return Collection of pending data streamer futures. + */ + public Collection dataStreamerFutures() { + return dataStreamerFuts; + } + /** * Gets future by given future ID. * @@ -475,6 +486,21 @@ public void addFuture(final GridCacheFuture fut, final IgniteUuid futId) { onFutureAdded(fut); } + /** + * @param topVer Topology version. + */ + public GridFutureAdapter addDataStreamerFuture(AffinityTopologyVersion topVer) { + final DataStreamerFuture fut = new DataStreamerFuture(topVer); + + boolean add = dataStreamerFuts.add(fut); + + assert add; + + return fut; + } + + /** + /** * Adds future. * @@ -1055,6 +1081,22 @@ public IgniteInternalFuture finishAtomicUpdates(AffinityTopologyVersion topVe return res; } + /** + * + * @return Finish update future. + */ + @SuppressWarnings("unchecked") + public IgniteInternalFuture finishDataStreamerUpdates() { + GridCompoundFuture res = new GridCompoundFuture<>(); + + for (IgniteInternalFuture fut : dataStreamerFuts) + res.add(fut); + + res.markInitialized(); + + return res; + } + /** * @param keys Key for which locks should be released. * @param cacheId Cache ID. @@ -1294,4 +1336,39 @@ private static class FinishAtomicUpdateFuture extends GridCompoundFuture { + /** */ + private static final long serialVersionUID = 0L; + + /** Topology version. Instance field for toString method only. */ + @GridToStringInclude + private final AffinityTopologyVersion topVer; + + /** + * @param topVer Topology version. + */ + DataStreamerFuture(AffinityTopologyVersion topVer) { + this.topVer = topVer; + } + + /** {@inheritDoc} */ + @Override public boolean onDone(@Nullable Void res, @Nullable Throwable err) { + if (super.onDone(res, err)) { + dataStreamerFuts.remove(this); + + return true; + } + + return false; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(DataStreamerFuture.class, this, super.toString()); + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index a901e2aca7825..00d2d1633ce6e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -1309,6 +1309,11 @@ private void dumpPendingObjects(@Nullable AffinityTopologyVersion exchTopVer) { for (GridCacheFuture fut : mvcc.atomicFutures()) U.warn(log, ">>> " + fut); + U.warn(log, "Pending data streamer futures:"); + + for (IgniteInternalFuture fut : mvcc.dataStreamerFutures()) + U.warn(log, ">>> " + fut); + if (tm != null) { U.warn(log, "Pending transaction deadlock detection futures:"); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java index 8f3923515de5b..117a5c33189d7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java @@ -636,6 +636,7 @@ public IgniteInternalFuture partitionReleaseFuture(AffinityTopologyVersion to f.add(mvcc().finishExplicitLocks(topVer)); f.add(tm().finishTxs(topVer)); f.add(mvcc().finishAtomicUpdates(topVer)); + f.add(mvcc().finishDataStreamerUpdates()); f.markInitialized(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java index 7663735fc26c6..32fda87375cb4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java @@ -21,6 +21,7 @@ import java.util.UUID; import java.util.concurrent.DelayQueue; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; @@ -29,13 +30,18 @@ import org.apache.ignite.internal.managers.deployment.GridDeployment; import org.apache.ignite.internal.processors.GridProcessorAdapter; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.GridCacheAdapter; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.GridSpinBusyLock; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.GridWorker; import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.stream.StreamReceiver; import org.apache.ignite.thread.IgniteThread; @@ -288,32 +294,94 @@ private void processRequest(final UUID nodeId, final DataStreamerRequest req) { return; } - Collection col = req.entries(); + localUpdate(nodeId, req, updater, topic); + } + finally { + busyLock.leaveBusy(); + } + } - DataStreamerUpdateJob job = new DataStreamerUpdateJob(ctx, - log, - req.cacheName(), - col, - req.ignoreDeploymentOwnership(), - req.skipStore(), - req.keepBinary(), - updater); + /** + * @param nodeId Node id. + * @param req Request. + * @param updater Updater. + * @param topic Topic. + */ + private void localUpdate(final UUID nodeId, + final DataStreamerRequest req, + final StreamReceiver updater, + final Object topic) { + final boolean allowOverwrite = !(updater instanceof DataStreamerImpl.IsolatedUpdater); - Exception err = null; + try { + GridCacheAdapter cache = ctx.cache().internalCache(req.cacheName()); + + if (cache == null) + throw new IgniteCheckedException("Cache not created or already destroyed."); + + GridCacheContext cctx = cache.context(); + + DataStreamerUpdateJob job = null; + + GridFutureAdapter waitFut = null; + + if (!allowOverwrite) + cctx.topology().readLock(); try { - job.call(); - } - catch (Exception e) { - U.error(log, "Failed to finish update job.", e); + GridDhtTopologyFuture fut = cctx.topologyVersionFuture(); - err = e; + AffinityTopologyVersion topVer = fut.topologyVersion(); + + if (!allowOverwrite && !topVer.equals(req.topologyVersion())) { + Exception err = new IgniteCheckedException( + "DataStreamer will retry data transfer at stable topology " + + "[reqTop=" + req.topologyVersion() + ", topVer=" + topVer + ", node=remote]"); + + sendResponse(nodeId, topic, req.requestId(), err, req.forceLocalDeployment()); + } + else if (allowOverwrite || fut.isDone()) { + job = new DataStreamerUpdateJob(ctx, + log, + req.cacheName(), + req.entries(), + req.ignoreDeploymentOwnership(), + req.skipStore(), + req.keepBinary(), + updater); + + waitFut = allowOverwrite ? null : cctx.mvcc().addDataStreamerFuture(topVer); + } + else { + fut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture e) { + localUpdate(nodeId, req, updater, topic); + } + }); + } + } + finally { + if (!allowOverwrite) + cctx.topology().readUnlock(); } - sendResponse(nodeId, topic, req.requestId(), err, req.forceLocalDeployment()); + if (job != null) { + try { + job.call(); + + sendResponse(nodeId, topic, req.requestId(), null, req.forceLocalDeployment()); + } + finally { + if (waitFut != null) + waitFut.onDone(); + } + } } - finally { - busyLock.leaveBusy(); + catch (Throwable e) { + sendResponse(nodeId, topic, req.requestId(), e, req.forceLocalDeployment()); + + if (e instanceof Error) + throw (Error)e; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java index a6065ddd76118..443783b4ccc2f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java @@ -29,6 +29,7 @@ import java.util.Map.Entry; import java.util.Queue; import java.util.UUID; +import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.DelayQueue; import java.util.concurrent.Delayed; @@ -39,15 +40,15 @@ import java.util.concurrent.atomic.AtomicReference; import javax.cache.CacheException; import javax.cache.expiry.ExpiryPolicy; - import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.IgniteDataStreamerTimeoutException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteInterruptedException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.IgniteDataStreamerTimeoutException; +import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.cluster.ClusterTopologyException; import org.apache.ignite.configuration.CacheConfiguration; @@ -72,11 +73,13 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; +import org.apache.ignite.internal.processors.cache.GridCacheGateway; import org.apache.ignite.internal.processors.cache.GridCacheUtils; import org.apache.ignite.internal.processors.cache.IgniteCacheFutureImpl; import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; import org.apache.ignite.internal.processors.dr.GridDrType; @@ -92,6 +95,8 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.GPC; +import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; @@ -102,6 +107,7 @@ import org.apache.ignite.stream.StreamReceiver; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; +import org.jsr166.LongAdder8; import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; @@ -113,12 +119,15 @@ */ @SuppressWarnings("unchecked") public class DataStreamerImpl implements IgniteDataStreamer, Delayed { - /** Default policy reoslver. */ + /** Default policy resolver. */ private static final DefaultIoPolicyResolver DFLT_IO_PLC_RSLVR = new DefaultIoPolicyResolver(); /** Isolated receiver. */ private static final StreamReceiver ISOLATED_UPDATER = new IsolatedUpdater(); + /** Amount of permissions should be available to continue new data processing. */ + private static final int REMAP_SEMAPHORE_PERMISSIONS_COUNT = Integer.MAX_VALUE; + /** Cache receiver. */ private StreamReceiver rcvr = ISOLATED_UPDATER; @@ -178,6 +187,9 @@ public class DataStreamerImpl implements IgniteDataStreamer, Delayed /** {@code True} if data loader has been cancelled. */ private volatile boolean cancelled; + /** Fail counter. */ + private final LongAdder8 failCntr = new LongAdder8(); + /** Active futures of this data loader. */ @GridToStringInclude private final Collection> activeFuts = new GridConcurrentHashSet<>(); @@ -189,6 +201,16 @@ public class DataStreamerImpl implements IgniteDataStreamer, Delayed boolean rmv = activeFuts.remove(t); assert rmv; + + Throwable err = t.error(); + + if (err != null && !(err instanceof IgniteClientDisconnectedCheckedException)) { + LT.error(log, t.error(), "DataStreamer operation failed.", true); + + failCntr.increment(); + + cancelled = true; + } } }; @@ -231,6 +253,15 @@ public class DataStreamerImpl implements IgniteDataStreamer, Delayed /** Whether a warning at {@link DataStreamerImpl#allowOverwrite()} printed */ private static boolean isWarningPrinted; + /** Allows to pause new data processing while failed data processing in progress. */ + private final Semaphore remapSem = new Semaphore(REMAP_SEMAPHORE_PERMISSIONS_COUNT); + + /** */ + private final ConcurrentLinkedDeque dataToRemap = new ConcurrentLinkedDeque<>(); + + /** */ + private final AtomicBoolean remapOwning = new AtomicBoolean(); + /** * @param ctx Grid kernal context. * @param cacheName Cache name. @@ -301,7 +332,7 @@ public DataStreamerImpl( Buffer buf = bufMappings.get(nodeId); if (buf != null) - buf.onResponse(res); + buf.onResponse(res, nodeId); else if (log.isDebugEnabled()) log.debug("Ignoring response since node has left [nodeId=" + nodeId + ", "); @@ -314,6 +345,17 @@ else if (log.isDebugEnabled()) fut = new DataStreamerFuture(this); publicFut = new IgniteCacheFutureImpl<>(fut); + + GridCacheAdapter cache = ctx.cache().internalCache(cacheName); + + if (cache == null) { // Possible, cache is not configured on node. + assert ccfg != null; + + if (ccfg.getCacheMode() == CacheMode.LOCAL) + throw new CacheException("Impossible to load Local cache configured remotely."); + + ctx.grid().getOrCreateCache(ccfg); + } } /** @@ -356,6 +398,11 @@ private void enterBusy() { if (disconnectErr != null) throw disconnectErr; + throw new IllegalStateException("Data streamer has been closed."); + } + else if (cancelled) { + busyLock.leaveBusy(); + throw new IllegalStateException("Data streamer has been closed."); } } @@ -632,6 +679,37 @@ public void ioPolicyResolver(IgniteClosure ioPlcRslvr) { this.ioPlcRslvr = ioPlcRslvr; } + /** + * + */ + private void acquireRemapSemaphore() throws IgniteInterruptedCheckedException { + try { + if (remapSem.availablePermits() != REMAP_SEMAPHORE_PERMISSIONS_COUNT) { + if (timeout == DFLT_UNLIMIT_TIMEOUT) { + // Wait until failed data being processed. + remapSem.acquire(REMAP_SEMAPHORE_PERMISSIONS_COUNT); + + remapSem.release(REMAP_SEMAPHORE_PERMISSIONS_COUNT); + } + else { + // Wait until failed data being processed. + boolean res = remapSem.tryAcquire(REMAP_SEMAPHORE_PERMISSIONS_COUNT, timeout, TimeUnit.MILLISECONDS); + + if (res) + remapSem.release(REMAP_SEMAPHORE_PERMISSIONS_COUNT); + else + throw new IgniteDataStreamerTimeoutException("Data streamer exceeded timeout " + + "while was waiting for failed data resending finished."); + } + } + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + + throw new IgniteInterruptedCheckedException(e); + } + } + /** * @param entries Entries. * @param resFut Result future. @@ -644,170 +722,266 @@ private void load0( @Nullable final Collection activeKeys, final int remaps ) { - assert entries != null; + try { + assert entries != null; - if (!isWarningPrinted) { - synchronized (this) { - if (!allowOverwrite() && !isWarningPrinted) { - U.warn(log, "Data streamer will not overwrite existing cache entries for better performance " + - "(to change, set allowOverwrite to true)"); - } + final boolean remap = remaps > 0; - isWarningPrinted = true; + if (!remap) { // Failed data should be processed prior to new data. + acquireRemapSemaphore(); } - } - Map> mappings = new HashMap<>(); + if (!isWarningPrinted) { + synchronized (this) { + if (!allowOverwrite() && !isWarningPrinted) { + U.warn(log, "Data streamer will not overwrite existing cache entries for better performance " + + "(to change, set allowOverwrite to true)"); + } - boolean initPda = ctx.deploy().enabled() && jobPda == null; + isWarningPrinted = true; + } + } - AffinityTopologyVersion topVer = ctx.cache().context().exchange().readyAffinityVersion(); + Map> mappings = new HashMap<>(); - for (DataStreamerEntry entry : entries) { - List nodes; + boolean initPda = ctx.deploy().enabled() && jobPda == null; - try { - KeyCacheObject key = entry.getKey(); + GridCacheAdapter cache = ctx.cache().internalCache(cacheName); - assert key != null; + if (cache == null) + throw new IgniteCheckedException("Cache not created or already destroyed."); - if (initPda) { - if (cacheObjCtx.addDeploymentInfo()) - jobPda = new DataStreamerPda(key.value(cacheObjCtx, false), - entry.getValue() != null ? entry.getValue().value(cacheObjCtx, false) : null, - rcvr); - else if (rcvr != null) - jobPda = new DataStreamerPda(rcvr); + GridCacheContext cctx = cache.context(); - initPda = false; - } + GridCacheGateway gate = null; - nodes = nodes(key, topVer); - } - catch (IgniteCheckedException e) { - resFut.onDone(e); + if (!allowOverwrite() && !cctx.isLocal()) { // Cases where cctx required. + gate = cctx.gate(); - return; + gate.enter(); } - if (F.isEmpty(nodes)) { - resFut.onDone(new ClusterTopologyException("Failed to map key to node " + - "(no nodes with cache found in topology) [infos=" + entries.size() + - ", cacheName=" + cacheName + ']')); - - return; - } + try { + AffinityTopologyVersion topVer = allowOverwrite() || cctx.isLocal() ? + ctx.cache().context().exchange().readyAffinityVersion() : + cctx.topology().topologyVersion(); - for (ClusterNode node : nodes) { - Collection col = mappings.get(node); + for (DataStreamerEntry entry : entries) { + List nodes; - if (col == null) - mappings.put(node, col = new ArrayList<>()); + try { + KeyCacheObject key = entry.getKey(); - col.add(entry); - } - } + assert key != null; - for (final Map.Entry> e : mappings.entrySet()) { - final UUID nodeId = e.getKey().id(); + if (initPda) { + if (cacheObjCtx.addDeploymentInfo()) + jobPda = new DataStreamerPda(key.value(cacheObjCtx, false), + entry.getValue() != null ? entry.getValue().value(cacheObjCtx, false) : null, + rcvr); + else if (rcvr != null) + jobPda = new DataStreamerPda(rcvr); - Buffer buf = bufMappings.get(nodeId); + initPda = false; + } - if (buf == null) { - Buffer old = bufMappings.putIfAbsent(nodeId, buf = new Buffer(e.getKey())); + nodes = nodes(key, topVer, cctx); + } + catch (IgniteCheckedException e) { + resFut.onDone(e); - if (old != null) - buf = old; - } + return; + } - final Collection entriesForNode = e.getValue(); + if (F.isEmpty(nodes)) { + resFut.onDone(new ClusterTopologyException("Failed to map key to node " + + "(no nodes with cache found in topology) [infos=" + entries.size() + + ", cacheName=" + cacheName + ']')); - IgniteInClosure> lsnr = new IgniteInClosure>() { - @Override public void apply(IgniteInternalFuture t) { - try { - t.get(); + return; + } - if (activeKeys != null) { - for (DataStreamerEntry e : entriesForNode) - activeKeys.remove(new KeyCacheObjectWrapper(e.getKey())); + for (ClusterNode node : nodes) { + Collection col = mappings.get(node); - if (activeKeys.isEmpty()) - resFut.onDone(); - } - else { - assert entriesForNode.size() == 1; + if (col == null) + mappings.put(node, col = new ArrayList<>()); - // That has been a single key, - // so complete result future right away. - resFut.onDone(); - } + col.add(entry); } - catch (IgniteClientDisconnectedCheckedException e1) { - if (log.isDebugEnabled()) - log.debug("Future finished with disconnect error [nodeId=" + nodeId + ", err=" + e1 + ']'); + } - resFut.onDone(e1); + for (final Map.Entry> e : mappings.entrySet()) { + final UUID nodeId = e.getKey().id(); + + Buffer buf = bufMappings.get(nodeId); + + if (buf == null) { + Buffer old = bufMappings.putIfAbsent(nodeId, buf = new Buffer(e.getKey())); + + if (old != null) + buf = old; } - catch (IgniteCheckedException e1) { - if (log.isDebugEnabled()) - log.debug("Future finished with error [nodeId=" + nodeId + ", err=" + e1 + ']'); - if (cancelled) { - resFut.onDone(new IgniteCheckedException("Data streamer has been cancelled: " + - DataStreamerImpl.this, e1)); + final Collection entriesForNode = e.getValue(); + + IgniteInClosure> lsnr = new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture t) { + try { + t.get(); + + if (activeKeys != null) { + for (DataStreamerEntry e : entriesForNode) + activeKeys.remove(new KeyCacheObjectWrapper(e.getKey())); + + if (activeKeys.isEmpty()) + resFut.onDone(); + } + else { + assert entriesForNode.size() == 1; + + // That has been a single key, + // so complete result future right away. + resFut.onDone(); + } + } + catch (IgniteClientDisconnectedCheckedException e1) { + if (log.isDebugEnabled()) + log.debug("Future finished with disconnect error [nodeId=" + nodeId + ", err=" + e1 + ']'); + + resFut.onDone(e1); + } + catch (IgniteCheckedException e1) { + if (log.isDebugEnabled()) + log.debug("Future finished with error [nodeId=" + nodeId + ", err=" + e1 + ']'); + + if (cancelled) { + resFut.onDone(new IgniteCheckedException("Data streamer has been cancelled: " + + DataStreamerImpl.this, e1)); + } + else if (remaps + 1 > maxRemapCnt) { + resFut.onDone(new IgniteCheckedException("Failed to finish operation (too many remaps): " + + remaps, e1)); + } + else { + try { + remapSem.acquire(); + + final Runnable r = new Runnable() { + @Override public void run() { + try { + if (cancelled) + throw new IllegalStateException("DataStreamer closed."); + + load0(entriesForNode, resFut, activeKeys, remaps + 1); + } + catch (Throwable ex) { + resFut.onDone( + new IgniteCheckedException("DataStreamer remapping failed. ", ex)); + } + finally { + remapSem.release(); + } + } + }; + + dataToRemap.add(r); + + if (!remapOwning.get() && remapOwning.compareAndSet(false, true)) { + ctx.closure().callLocalSafe(new GPC() { + @Override public Boolean call() { + boolean locked = true; + + while (locked || !dataToRemap.isEmpty()) { + if (!locked && !remapOwning.compareAndSet(false, true)) + return false; + + try { + Runnable r = dataToRemap.poll(); + + if (r != null) + r.run(); + } + finally { + if (!dataToRemap.isEmpty()) + locked = true; + else { + remapOwning.set(false); + + locked = false; + } + } + } + + return true; + } + }, true); + } + } + catch (InterruptedException e2) { + resFut.onDone(e2); + } + } + } } - else if (remaps + 1 > maxRemapCnt) { - resFut.onDone(new IgniteCheckedException("Failed to finish operation (too many remaps): " - + remaps), e1); - } - else - load0(entriesForNode, resFut, activeKeys, remaps + 1); - } - } - }; + }; - final GridFutureAdapter f; + final GridFutureAdapter f; - try { - f = buf.update(entriesForNode, topVer, lsnr); - } - catch (IgniteInterruptedCheckedException e1) { - resFut.onDone(e1); + try { + f = buf.update(entriesForNode, topVer, lsnr, remap); + } + catch (IgniteInterruptedCheckedException e1) { + resFut.onDone(e1); - return; - } + return; + } - if (ctx.discovery().node(nodeId) == null) { - if (bufMappings.remove(nodeId, buf)) { - final Buffer buf0 = buf; + if (ctx.discovery().node(nodeId) == null) { + if (bufMappings.remove(nodeId, buf)) { + final Buffer buf0 = buf; - waitAffinityAndRun(new Runnable() { - @Override public void run() { - buf0.onNodeLeft(); + waitAffinityAndRun(new Runnable() { + @Override public void run() { + buf0.onNodeLeft(); - if (f != null) - f.onDone(new ClusterTopologyCheckedException("Failed to wait for request completion " + - "(node has left): " + nodeId)); + if (f != null) + f.onDone(new ClusterTopologyCheckedException("Failed to wait for request completion " + + "(node has left): " + nodeId)); + } + }, ctx.discovery().topologyVersion(), false); } - }, ctx.discovery().topologyVersion(), false); + } } } + finally { + if (gate != null) + gate.leave(); + } + } + catch (Exception ex) { + resFut.onDone(new IgniteCheckedException("DataStreamer data loading failed.", ex)); } } /** * @param key Key to map. * @param topVer Topology version. + * @param cctx Context. * @return Nodes to send requests to. * @throws IgniteCheckedException If failed. */ - private List nodes(KeyCacheObject key, AffinityTopologyVersion topVer) throws IgniteCheckedException { + private List nodes(KeyCacheObject key, + AffinityTopologyVersion topVer, + GridCacheContext cctx) throws IgniteCheckedException { GridAffinityProcessor aff = ctx.affinity(); List res = null; if (!allowOverwrite()) - res = aff.mapKeyToPrimaryAndBackups(cacheName, key, topVer); + res = cctx.isLocal() ? + aff.mapKeyToPrimaryAndBackups(cacheName, key, topVer) : + cctx.topology().nodes(cctx.affinity().partition(key), topVer); else { ClusterNode node = aff.mapKeyToNode(cacheName, key, topVer); @@ -992,7 +1166,10 @@ else if (f.isDone()) { * @throws IgniteCheckedException If failed. */ public void closeEx(boolean cancel) throws IgniteCheckedException { - closeEx(cancel, null); + IgniteCheckedException err = closeEx(cancel, null); + + if (err != null) + throw err; // Throws at close(). } /** @@ -1000,9 +1177,9 @@ public void closeEx(boolean cancel) throws IgniteCheckedException { * @param err Error. * @throws IgniteCheckedException If failed. */ - public void closeEx(boolean cancel, IgniteCheckedException err) throws IgniteCheckedException { + private IgniteCheckedException closeEx(boolean cancel, IgniteCheckedException err) throws IgniteCheckedException { if (!closed.compareAndSet(false, true)) - return; + return null; busyLock.block(); @@ -1029,7 +1206,14 @@ public void closeEx(boolean cancel, IgniteCheckedException err) throws IgniteChe throw e; } + long failed = failCntr.longValue(); + + if (failed > 0 && err == null) + err = new IgniteCheckedException("Some of DataStreamer operations failed [failedCount=" + failed + "]"); + fut.onDone(err); + + return err; } /** @@ -1139,6 +1323,9 @@ private class Buffer { /** */ private final Semaphore sem; + /** Batch topology. */ + private AffinityTopologyVersion batchTopVer; + /** Closure to signal on task finish. */ @GridToStringExclude private final IgniteInClosure> signalC = new IgniteInClosure>() { @@ -1168,38 +1355,65 @@ private class Buffer { sem = new Semaphore(parallelOps); } + /** + * @param remap Remapping flag. + */ + private void renewBatch(boolean remap) { + entries = newEntries(); + curFut = new GridFutureAdapter<>(); + + batchTopVer = null; + + if (!remap) + curFut.listen(signalC); + } + /** * @param newEntries Infos. * @param topVer Topology version. * @param lsnr Listener for the operation future. + * @param remap Remapping flag. * @return Future for operation. * @throws IgniteInterruptedCheckedException If failed. */ @Nullable GridFutureAdapter update(Iterable newEntries, AffinityTopologyVersion topVer, - IgniteInClosure> lsnr) throws IgniteInterruptedCheckedException { + IgniteInClosure> lsnr, + boolean remap) throws IgniteInterruptedCheckedException { List entries0 = null; + GridFutureAdapter curFut0; + AffinityTopologyVersion curBatchTopVer; + synchronized (this) { curFut0 = curFut; curFut0.listen(lsnr); + if (batchTopVer == null) + batchTopVer = topVer; + + curBatchTopVer = batchTopVer; + for (DataStreamerEntry entry : newEntries) entries.add(entry); if (entries.size() >= bufSize) { entries0 = entries; - entries = newEntries(); - curFut = new GridFutureAdapter<>(); - curFut.listen(signalC); + renewBatch(remap); } } - if (entries0 != null) { - submit(entries0, topVer, curFut0); + if (!allowOverwrite() && !topVer.equals(curBatchTopVer)) { + renewBatch(remap); + + curFut0.onDone(null, new IgniteCheckedException("Topology changed during batch preparation." + + "[batchTopVer=" + curBatchTopVer + ", topVer=" + topVer + "]")); + } + else if (entries0 != null) { + submit(entries0, curBatchTopVer, curFut0, remap); if (cancelled) curFut0.onDone(new IgniteCheckedException("Data streamer has been cancelled: " + @@ -1227,6 +1441,8 @@ private List newEntries() { List entries0 = null; GridFutureAdapter curFut0 = null; + acquireRemapSemaphore(); + synchronized (this) { if (!entries.isEmpty()) { entries0 = entries; @@ -1239,7 +1455,7 @@ private List newEntries() { } if (entries0 != null) - submit(entries0, null, curFut0); + submit(entries0, batchTopVer, curFut0, false); // Create compound future for this flush. GridCompoundFuture res = null; @@ -1289,26 +1505,114 @@ private void signalTaskFinished(IgniteInternalFuture f) { sem.release(); } + /** + * @param entries Entries. + * @param reqTopVer Request topology version. + * @param curFut Current future. + */ + private void localUpdate(final Collection entries, + final AffinityTopologyVersion reqTopVer, + final GridFutureAdapter curFut) { + try { + GridCacheContext cctx = ctx.cache().internalCache(cacheName).context(); + + final boolean allowOverwrite = allowOverwrite(); + final boolean loc = cctx.isLocal(); + + if (!loc && !allowOverwrite) + cctx.topology().readLock(); + + try { + GridDhtTopologyFuture fut = loc ? null : cctx.topologyVersionFuture(); + + AffinityTopologyVersion topVer = loc ? reqTopVer : fut.topologyVersion(); + + if (!allowOverwrite && !topVer.equals(reqTopVer)) { + curFut.onDone(new IgniteCheckedException( + "DataStreamer will retry data transfer at stable topology. " + + "[reqTop=" + reqTopVer + " ,topVer=" + topVer + ", node=local]")); + } + else if (loc || allowOverwrite || fut.isDone()) { + IgniteInternalFuture callFut = ctx.closure().callLocalSafe( + new DataStreamerUpdateJob( + ctx, + log, + cacheName, + entries, + false, + skipStore, + keepBinary, + rcvr), + false); + + locFuts.add(callFut); + + final GridFutureAdapter waitFut = (loc || allowOverwrite) ? + null : + cctx.mvcc().addDataStreamerFuture(topVer); + + callFut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture t) { + try { + boolean rmv = locFuts.remove(t); + + assert rmv; + + curFut.onDone(t.get()); + } + catch (IgniteCheckedException e) { + curFut.onDone(e); + } + finally { + if (waitFut != null) + waitFut.onDone(); + } + } + }); + } + else { + fut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture e) { + localUpdate(entries, reqTopVer, curFut); + } + }); + } + } + finally { + if (!loc && !allowOverwrite) + cctx.topology().readUnlock(); + } + } + catch (Throwable ex) { + curFut.onDone(new IgniteCheckedException("DataStreamer data handling failed.", ex)); + } + } + /** * @param entries Entries to submit. * @param topVer Topology version. * @param curFut Current future. + * @param remap Remapping flag. * @throws IgniteInterruptedCheckedException If interrupted. */ private void submit(final Collection entries, @Nullable AffinityTopologyVersion topVer, - final GridFutureAdapter curFut) + final GridFutureAdapter curFut, + boolean remap) throws IgniteInterruptedCheckedException { assert entries != null; assert !entries.isEmpty(); assert curFut != null; - try { - incrementActiveTasks(); - } - catch (IgniteDataStreamerTimeoutException e) { - curFut.onDone(e); - throw e; + if (!remap) { + try { + incrementActiveTasks(); + } + catch (IgniteDataStreamerTimeoutException e) { + curFut.onDone(e); + + throw e; + } } IgniteInternalFuture fut; @@ -1318,27 +1622,8 @@ private void submit(final Collection entries, if (plc == null) plc = PUBLIC_POOL; - if (isLocNode && plc == GridIoPolicy.PUBLIC_POOL) { - fut = ctx.closure().callLocalSafe( - new DataStreamerUpdateJob(ctx, log, cacheName, entries, false, skipStore, keepBinary, rcvr), false); - - locFuts.add(fut); - - fut.listen(new IgniteInClosure>() { - @Override public void apply(IgniteInternalFuture t) { - try { - boolean rmv = locFuts.remove(t); - - assert rmv; - - curFut.onDone(t.get()); - } - catch (IgniteCheckedException e) { - curFut.onDone(e); - } - } - }); - } + if (isLocNode && plc == GridIoPolicy.PUBLIC_POOL) + localUpdate(entries, topVer, curFut); else { try { for (DataStreamerEntry e : entries) { @@ -1466,8 +1751,9 @@ void onNodeLeft() { /** * @param res Response. + * @param nodeId Node id. */ - void onResponse(DataStreamerResponse res) { + void onResponse(DataStreamerResponse res, UUID nodeId) { if (log.isDebugEnabled()) log.debug("Received data load response: " + res); @@ -1488,9 +1774,10 @@ void onResponse(DataStreamerResponse res) { try { GridPeerDeployAware jobPda0 = jobPda; - err = U.unmarshal(ctx, - errBytes, - U.resolveClassLoader(jobPda0 != null ? jobPda0.classLoader() : null, ctx.config())); + err = new IgniteCheckedException("DataStreamer request failed [node=" + nodeId + "]", + (Throwable)U.unmarshal(ctx, + errBytes, + U.resolveClassLoader(jobPda0 != null ? jobPda0.classLoader() : null, ctx.config()))); } catch (IgniteCheckedException e) { f.onDone(null, new IgniteCheckedException("Failed to unmarshal response.", e)); @@ -1613,7 +1900,7 @@ private DataStreamerPda(Object... objs) { /** * Isolated receiver which only loads entry initial value. */ - private static class IsolatedUpdater implements StreamReceiver, + protected static class IsolatedUpdater implements StreamReceiver, DataStreamerCacheUpdaters.InternalUpdater { /** */ private static final long serialVersionUID = 0L; @@ -1630,7 +1917,9 @@ private static class IsolatedUpdater implements StreamReceiver, String> tup = - e != null ? F., String>t(e.getClass(), e.getMessage()) : + e != null && !byMessage ? F., String>t(e.getClass(), e.getMessage()) : F., String>t(null, longMsg); while (true) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDynamicStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDynamicStopSelfTest.java index 5bd60745c7ff9..c92ea9ec45ccf 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDynamicStopSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDynamicStopSelfTest.java @@ -78,27 +78,37 @@ public void checkStopStartCacheWithDataLoader(final boolean allowOverwrite) thro IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { /** {@inheritDoc} */ @Override public Object call() throws Exception { - try (IgniteDataStreamer str = ignite(0).dataStreamer(null)) { - str.allowOverwrite(allowOverwrite); - - int i = 0; - - while (!stop.get()) { - str.addData(i % 10_000, i).listen(new CI1>() { - @Override public void apply(IgniteFuture f) { - try { - f.get(); - } - catch (CacheException ignore) { - // This may be debugged. - } + while (!stop.get()) { + try (IgniteDataStreamer str = ignite(0).dataStreamer(null)) { + str.allowOverwrite(allowOverwrite); + + int i = 0; + + while (!stop.get()) { + try { + str.addData(i % 10_000, i).listen(new CI1>() { + @Override public void apply(IgniteFuture f) { + try { + f.get(); + } + catch (CacheException ignore) { + // This may be debugged. + } + } + }); + } + catch (IllegalStateException ignored) { + break; } - }); - if (i > 0 && i % 10000 == 0) - info("Added: " + i); + if (i > 0 && i % 10000 == 0) + info("Added: " + i); - i++; + i++; + } + } + catch (IllegalStateException | CacheException ignored) { + // This may be debugged. } } @@ -114,6 +124,8 @@ public void checkStopStartCacheWithDataLoader(final boolean allowOverwrite) thro Thread.sleep(500); ignite(0).createCache(ccfg); + + Thread.sleep(1000); } finally { stop.set(true); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java index 9da6cf761b7fe..0801691e78b85 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java @@ -18,6 +18,9 @@ package org.apache.ignite.internal.processors.cache.distributed; import java.io.Serializable; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.Set; import java.util.concurrent.Callable; import javax.cache.Cache; import javax.cache.configuration.FactoryBuilder; @@ -28,32 +31,47 @@ import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.store.CacheStoreAdapter; +import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.internal.util.typedef.P1; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiInClosure; +import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_GRID_NAME; +import static org.apache.ignite.testframework.GridTestUtils.runAsync; /** * Tests for cache data loading during simultaneous grids start. */ -public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractTest { - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-500"); - } - +public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractTest implements Serializable { /** Grids count */ private static int GRIDS_CNT = 5; /** Keys count */ private static int KEYS_CNT = 1_000_000; + /** Client. */ + private volatile boolean client; + + /** Config. */ + private volatile boolean configured; + + /** Allow override. */ + protected volatile boolean allowOverwrite; + + /** Restarts. */ + protected volatile boolean restarts; + /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { @@ -67,7 +85,24 @@ public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractT ccfg.setCacheStoreFactory(new FactoryBuilder.SingletonFactory(new TestCacheStoreAdapter())); - cfg.setCacheConfiguration(ccfg); + if (getTestGridName(0).equals(gridName)) { + if (client) + cfg.setClientMode(true); + + if (configured) + cfg.setCacheConfiguration(ccfg); + } + else + cfg.setCacheConfiguration(ccfg); + + if (!configured) + ccfg.setNodeFilter(new P1() { + @Override public boolean apply(ClusterNode node) { + String name = node.attribute(ATTR_GRID_NAME).toString(); + + return !getTestGridName(0).equals(name); + } + }); return cfg; } @@ -81,22 +116,35 @@ public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractT * @throws Exception if failed */ public void testLoadCacheWithDataStreamer() throws Exception { - IgniteInClosure f = new IgniteInClosure() { - @Override public void apply(Ignite grid) { - try (IgniteDataStreamer dataStreamer = grid.dataStreamer(null)) { - for (int i = 0; i < KEYS_CNT; i++) - dataStreamer.addData(i, Integer.toString(i)); + configured = true; + + try { + IgniteInClosure f = new IgniteInClosure() { + @Override public void apply(Ignite grid) { + try (IgniteDataStreamer dataStreamer = grid.dataStreamer(null)) { + dataStreamer.allowOverwrite(allowOverwrite); + + for (int i = 0; i < KEYS_CNT; i++) + dataStreamer.addData(i, Integer.toString(i)); + } + + log.info("Data loaded."); } - } - }; + }; - loadCache(f); + loadCache(f); + } + finally { + configured = false; + } } /** * @throws Exception if failed */ public void testLoadCacheFromStore() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-4210"); + loadCache(new IgniteInClosure() { @Override public void apply(Ignite grid) { grid.cache(null).loadCache(null); @@ -104,13 +152,178 @@ public void testLoadCacheFromStore() throws Exception { }); } + /** + * @throws Exception if failed + */ + public void testLoadCacheWithDataStreamerSequentialClient() throws Exception { + client = true; + + try { + loadCacheWithDataStreamerSequential(); + } + finally { + client = false; + } + } + + /** + * @throws Exception if failed + */ + public void testLoadCacheWithDataStreamerSequentialClientWithConfig() throws Exception { + client = true; + configured = true; + + try { + loadCacheWithDataStreamerSequential(); + } + finally { + client = false; + configured = false; + } + } + + /** + * @throws Exception if failed + */ + public void testLoadCacheWithDataStreamerSequential() throws Exception { + loadCacheWithDataStreamerSequential(); + } + + /** + * @throws Exception if failed + */ + public void testLoadCacheWithDataStreamerSequentialWithConfigAndRestarts() throws Exception { + restarts = true; + configured = true; + + try { + loadCacheWithDataStreamerSequential(); + } + finally { + restarts = false; + configured = false; + } + } + + /** + * @throws Exception if failed + */ + public void testLoadCacheWithDataStreamerSequentialWithConfig() throws Exception { + configured = true; + + try { + loadCacheWithDataStreamerSequential(); + } + finally { + configured = false; + } + } + + /** + * @throws Exception if failed + */ + private void loadCacheWithDataStreamerSequential() throws Exception { + startGrid(1); + + Ignite g0 = startGrid(0); + + IgniteInternalFuture restartFut = runAsync(new Callable() { + @Override public Object call() throws Exception { + while (restarts) { + stopGrid(1); + + startGrid(1); + + U.sleep(100); + } + + return null; + } + }); + + IgniteInternalFuture fut = runAsync(new Callable() { + @Override public Object call() throws Exception { + for (int i = 2; i < GRIDS_CNT; i++) + startGrid(i); + + return null; + } + }); + + final HashSet set = new HashSet<>(); + + IgniteInClosure f = new IgniteInClosure() { + @Override public void apply(Ignite grid) { + try (IgniteDataStreamer dataStreamer = grid.dataStreamer(null)) { + dataStreamer.allowOverwrite(allowOverwrite); + + for (int i = 0; i < KEYS_CNT; i++) { + set.add(dataStreamer.addData(i, "Data")); + + if (i % 100000 == 0) + log.info("Streaming " + i + "'th entry."); + } + } + } + }; + + f.apply(g0); + + log.info("Data loaded."); + + restarts = false; + + fut.get(); + restartFut.get(); + + for (IgniteFuture res : set) + assertNull(res.get()); + + IgniteCache cache = grid(0).cache(null); + + long size = cache.size(CachePeekMode.PRIMARY); + + if (size != KEYS_CNT) { + Set failedKeys = new LinkedHashSet<>(); + + for (int i = 0; i < KEYS_CNT; i++) + if (!cache.containsKey(i)) { + log.info("Actual cache size: " + size); + + for (Ignite ignite : G.allGrids()) { + IgniteEx igniteEx = (IgniteEx)ignite; + + log.info("Missed key info:" + + igniteEx.localNode().id() + + " primary=" + + ignite.affinity(null).isPrimary(igniteEx.localNode(), i) + + " backup=" + + ignite.affinity(null).isBackup(igniteEx.localNode(), i) + + " local peek=" + + ignite.cache(null).localPeek(i, CachePeekMode.ONHEAP)); + } + + for (int j = i; j < i + 10000; j++) { + if (!cache.containsKey(j)) + failedKeys.add(j); + } + + break; + } + + assert failedKeys.isEmpty() : "Some failed keys: " + failedKeys.toString(); + } + + assertCacheSize(); + } + /** * Loads cache using closure and asserts cache size. * * @param f cache loading closure * @throws Exception if failed */ - private void loadCache(IgniteInClosure f) throws Exception { + protected void loadCache(IgniteInClosure f) throws Exception { Ignite g0 = startGrid(0); IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { @@ -130,17 +343,17 @@ private void loadCache(IgniteInClosure f) throws Exception { } /** Asserts cache size. */ - private void assertCacheSize() { + protected void assertCacheSize() { IgniteCache cache = grid(0).cache(null); - assertEquals(KEYS_CNT, cache.size(CachePeekMode.PRIMARY)); + assertEquals("Data lost.", KEYS_CNT, cache.size(CachePeekMode.PRIMARY)); int total = 0; for (int i = 0; i < GRIDS_CNT; i++) total += grid(i).cache(null).localSize(CachePeekMode.PRIMARY); - assertEquals(KEYS_CNT, total); + assertEquals("Data lost.", KEYS_CNT, total); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTestAllowOverwrite.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTestAllowOverwrite.java new file mode 100644 index 0000000000000..c9cd9faafd8c2 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTestAllowOverwrite.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.distributed; + +/** + * + */ +public class CacheLoadingConcurrentGridStartSelfTestAllowOverwrite extends CacheLoadingConcurrentGridStartSelfTest { + /** + * Default constructor. + */ + public CacheLoadingConcurrentGridStartSelfTestAllowOverwrite() { + allowOverwrite = true; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java index 9fedc35ae4c8e..0f8ae29aca3b6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java @@ -29,9 +29,9 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import javax.cache.Cache; +import javax.cache.CacheException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CachePeekMode; @@ -194,7 +194,7 @@ public void testLocal() throws Exception { assert false; } - catch (IgniteCheckedException e) { + catch (CacheException e) { // Cannot load local cache configured remotely. info("Caught expected exception: " + e); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java index 0c6686f75cc5e..a6a9f5488cefc 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImplSelfTest.java @@ -22,13 +22,17 @@ import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.CyclicBarrier; +import javax.cache.CacheException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.CacheServerNotFoundException; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; @@ -50,6 +54,16 @@ public class DataStreamerImplSelfTest extends GridCommonAbstractTest { /** Started grid counter. */ private static int cnt; + /** No nodes filter. */ + private static volatile boolean noNodesFilter; + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); @@ -72,88 +86,149 @@ public class DataStreamerImplSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testNullPointerExceptionUponDataStreamerClosing() throws Exception { - try { - startGrids(5); + startGrids(5); - final CyclicBarrier barrier = new CyclicBarrier(2); + final CyclicBarrier barrier = new CyclicBarrier(2); - multithreadedAsync(new Callable() { - @Override public Object call() throws Exception { - U.awaitQuiet(barrier); + multithreadedAsync(new Callable() { + @Override public Object call() throws Exception { + U.awaitQuiet(barrier); - G.stopAll(true); + G.stopAll(true); - return null; - } - }, 1); + return null; + } + }, 1); - Ignite g4 = grid(4); + Ignite g4 = grid(4); - IgniteDataStreamer dataLdr = g4.dataStreamer(null); + IgniteDataStreamer dataLdr = g4.dataStreamer(null); - dataLdr.perNodeBufferSize(32); + dataLdr.perNodeBufferSize(32); - for (int i = 0; i < 100000; i += 2) { - dataLdr.addData(i, i); - dataLdr.removeData(i + 1); - } + for (int i = 0; i < 100000; i += 2) { + dataLdr.addData(i, i); + dataLdr.removeData(i + 1); + } - U.awaitQuiet(barrier); + U.awaitQuiet(barrier); - info("Closing data streamer."); + info("Closing data streamer."); - try { - dataLdr.close(true); - } - catch (IllegalStateException ignore) { - // This is ok to ignore this exception as test is racy by it's nature - - // grid is stopping in different thread. - } + try { + dataLdr.close(true); } - finally { - G.stopAll(true); + catch (CacheException | IllegalStateException ignore) { + // This is ok to ignore this exception as test is racy by it's nature - + // grid is stopping in different thread. } } /** * Data streamer should correctly load entries from HashMap in case of grids with more than one node - * and with GridOptimizedMarshaller that requires serializable. + * and with GridOptimizedMarshaller that requires serializable. * * @throws Exception If failed. */ public void testAddDataFromMap() throws Exception { - try { - cnt = 0; + cnt = 0; - startGrids(2); + startGrids(2); - Ignite g0 = grid(0); + Ignite g0 = grid(0); - IgniteDataStreamer dataLdr = g0.dataStreamer(null); + IgniteDataStreamer dataLdr = g0.dataStreamer(null); - Map map = U.newHashMap(KEYS_COUNT); + Map map = U.newHashMap(KEYS_COUNT); - for (int i = 0; i < KEYS_COUNT; i ++) - map.put(i, String.valueOf(i)); + for (int i = 0; i < KEYS_COUNT; i++) + map.put(i, String.valueOf(i)); - dataLdr.addData(map); + dataLdr.addData(map); - dataLdr.close(); + dataLdr.close(); - Random rnd = new Random(); + Random rnd = new Random(); - IgniteCache c = g0.cache(null); + IgniteCache c = g0.cache(null); - for (int i = 0; i < KEYS_COUNT; i ++) { - Integer k = rnd.nextInt(KEYS_COUNT); + for (int i = 0; i < KEYS_COUNT; i++) { + Integer k = rnd.nextInt(KEYS_COUNT); - String v = c.get(k); + String v = c.get(k); + + assertEquals(k.toString(), v); + } + } + + /** + * Test logging on {@code DataStreamer.addData()} method when cache have no data nodes + * + * @throws Exception If fail. + */ + public void testNoDataNodesOnClose() throws Exception { + boolean failed = false; + + cnt = 0; + + noNodesFilter = true; + + try { + Ignite ignite = startGrid(1); - assertEquals(k.toString(), v); + try (IgniteDataStreamer streamer = ignite.dataStreamer(null)) { + streamer.addData(1, "1"); + } + catch (CacheException ex) { + failed = true; } } finally { - G.stopAll(true); + noNodesFilter = false; + + assertTrue(failed); + } + } + + /** + * Test logging on {@code DataStreamer.addData()} method when cache have no data nodes + * + * @throws Exception If fail. + */ + public void testNoDataNodesOnFlush() throws Exception { + boolean failed = false; + + cnt = 0; + + noNodesFilter = true; + + try { + Ignite ignite = startGrid(1); + + IgniteFuture fut = null; + + try (IgniteDataStreamer streamer = ignite.dataStreamer(null)) { + fut = streamer.addData(1, "1"); + + streamer.flush(); + } + catch (IllegalStateException ex) { + try { + fut.get(); + + fail("DataStreamer ignores failed streaming."); + } + catch (CacheServerNotFoundException ignored) { + // No-op. + } + + failed = true; + } + } + finally { + noNodesFilter = false; + + assertTrue(failed); } } @@ -169,6 +244,9 @@ private CacheConfiguration cacheConfiguration() { cacheCfg.setBackups(1); cacheCfg.setWriteSynchronizationMode(FULL_SYNC); + if (noNodesFilter) + cacheCfg.setNodeFilter(F.alwaysFalse()); + return cacheCfg; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java index c1251ae4e80d8..3d3f1462cf61a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java @@ -125,8 +125,6 @@ private void startStopIgnites() throws Exception { ignite.getOrCreateCache(cacheConfiguration()); try (final DataStreamerImpl dataLdr = (DataStreamerImpl)ignite.dataStreamer(null)) { - dataLdr.maxRemapCount(0); - Random rnd = new Random(); long endTime = U.currentTimeMillis() + 15_000; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerTimeoutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerTimeoutTest.java index 4e981b70a54a9..766aa84fdb947 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerTimeoutTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerTimeoutTest.java @@ -17,11 +17,13 @@ package org.apache.ignite.internal.processors.datastreamer; import java.util.Collection; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.CacheException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteDataStreamer; -import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteDataStreamerTimeoutException; +import org.apache.ignite.IgniteException; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInterruptedCheckedException; @@ -36,7 +38,6 @@ * Test timeout for Data streamer. */ public class DataStreamerTimeoutTest extends GridCommonAbstractTest { - /** Cache name. */ public static final String CACHE_NAME = "cacheName"; @@ -46,6 +47,9 @@ public class DataStreamerTimeoutTest extends GridCommonAbstractTest { /** Amount of entries. */ public static final int ENTRY_AMOUNT = 100; + /** Fail on. */ + private static volatile int failOn; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); @@ -76,6 +80,8 @@ private CacheConfiguration cacheConfiguration() { * @throws Exception If fail. */ public void testTimeoutOnCloseMethod() throws Exception { + failOn = 1; + Ignite ignite = startGrid(1); boolean thrown = false; @@ -85,12 +91,10 @@ public void testTimeoutOnCloseMethod() throws Exception { ldr.receiver(new TestDataReceiver()); ldr.perNodeBufferSize(ENTRY_AMOUNT); - for (int i=0; i < ENTRY_AMOUNT; i++) + for (int i = 0; i < ENTRY_AMOUNT; i++) ldr.addData(i, i); - } - catch (IgniteDataStreamerTimeoutException e) { - assertEquals(e.getMessage(), "Data streamer exceeded timeout on flush."); + catch (CacheException | IgniteDataStreamerTimeoutException e) { thrown = true; } finally { @@ -102,40 +106,68 @@ public void testTimeoutOnCloseMethod() throws Exception { /** * Test timeout on {@code DataStreamer.close()} method + * * @throws Exception If fail. */ - public void testTimeoutOnAddDataMethod() throws Exception { - Ignite ignite = startGrid(1); + public void testTimeoutOnAddData() throws Exception { + failOn = 1; - boolean thrown = false; + int processed = timeoutOnAddData(); - IgniteDataStreamer ldr = ignite.dataStreamer(CACHE_NAME); + assertTrue(processed == (failOn + 1) || processed == failOn); - try { - ldr.timeout(TIMEOUT); - ldr.receiver(new TestDataReceiver()); - ldr.perNodeBufferSize(ENTRY_AMOUNT/2); - ldr.perNodeParallelOperations(1); + failOn = ENTRY_AMOUNT / 2; - try { - for (int i=0; i < ENTRY_AMOUNT; i++) - ldr.addData(i, i); - } - catch (IgniteDataStreamerTimeoutException e) { - assertEquals(e.getMessage(), "Data streamer exceeded timeout when starts parallel operation."); + processed = timeoutOnAddData(); + + assertTrue(processed == (failOn + 1) || processed == failOn); + failOn = ENTRY_AMOUNT; + + processed = timeoutOnAddData(); + + assertTrue(processed == (failOn + 1) || processed == failOn); + } + + /** + * + */ + private int timeoutOnAddData() throws Exception { + boolean thrown = false; + int processed = 0; + + try { + Ignite ignite = startGrid(1); + + try (IgniteDataStreamer ldr = ignite.dataStreamer(CACHE_NAME)) { + ldr.timeout(TIMEOUT); + ldr.receiver(new TestDataReceiver()); + ldr.perNodeBufferSize(1); + ldr.perNodeParallelOperations(1); + ((DataStreamerImpl)ldr).maxRemapCount(0); + + try { + for (int i = 0; i < ENTRY_AMOUNT; i++) { + ldr.addData(i, i); + + processed++; + } + } + catch (IllegalStateException e) { + // No-op. + } + } + catch (CacheException | IgniteDataStreamerTimeoutException e) { thrown = true; } - } finally { - if (thrown) - ldr.close(true); - stopAllGrids(); } assertTrue(thrown); + + return processed; } /** @@ -143,16 +175,14 @@ public void testTimeoutOnAddDataMethod() throws Exception { */ private static class TestDataReceiver implements StreamReceiver { - /** Is first. */ - boolean isFirst = true; + /** Count. */ + private final AtomicInteger cnt = new AtomicInteger(); /** {@inheritDoc} */ - @Override public void receive(IgniteCache cache, Collection collection) throws IgniteException { + @Override public void receive(IgniteCache cache, Collection col) throws IgniteException { try { - if (isFirst) + if (cnt.incrementAndGet() == failOn) U.sleep(2 * TIMEOUT); - - isFirst = false; } catch (IgniteInterruptedCheckedException e) { throw new IgniteException(e); diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java index dc412a97b70c0..06639037a625e 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java @@ -39,6 +39,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCachePartitionMapUpdateTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheAndNodeStop; import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTestAllowOverwrite; import org.apache.ignite.internal.processors.cache.distributed.CacheLockReleaseNodeLeaveTest; import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionNotLoadedEventSelfTest; import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionedNearDisabledTxMultiThreadedSelfTest; @@ -214,6 +215,7 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(FairAffinityFunctionBackupFilterSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedPreloadLifecycleSelfTest.class)); suite.addTest(new TestSuite(CacheLoadingConcurrentGridStartSelfTest.class)); + suite.addTest(new TestSuite(CacheLoadingConcurrentGridStartSelfTestAllowOverwrite.class)); suite.addTest(new TestSuite(GridCacheDhtPreloadDelayedSelfTest.class)); suite.addTest(new TestSuite(GridPartitionedBackupLoadSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedLoadCacheSelfTest.class)); From baa752660c6eddf27d15a812252b01b5872385de Mon Sep 17 00:00:00 2001 From: iveselovskiy Date: Thu, 10 Nov 2016 18:47:09 +0300 Subject: [PATCH 336/487] IGNITE-4208: Hadoop: Fixed a bug preventing normal secondary file system start. This closes #1228. --- .../apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java | 2 +- .../apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java | 2 +- .../processors/hadoop/delegate/HadoopDelegateUtils.java | 4 ++-- .../delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java | 3 +-- .../impl/fs/KerberosHadoopFileSystemFactorySelfTest.java | 3 ++- .../hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java | 2 +- .../impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java | 3 +-- .../impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java | 4 ++-- 8 files changed, 11 insertions(+), 12 deletions(-) diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java index 7133c087b9dbe..866fc1a2aad02 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java @@ -336,7 +336,7 @@ public void colocateFileWrites(boolean colocateFileWrites) { HadoopFileSystemFactory factory0 = (HadoopFileSystemFactory)paths.getPayload(getClass().getClassLoader()); - factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory0); + factory = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory0); } catch (IgniteCheckedException e) { throw new IOException("Failed to get secondary file system factory.", e); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java index 18b8bf9fc7e19..82ad68309d547 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java @@ -345,7 +345,7 @@ private void initialize(URI name, Configuration cfg) throws IOException { HadoopFileSystemFactory factory0 = (HadoopFileSystemFactory) paths.getPayload(getClass().getClassLoader()); - factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory0); + factory = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory0); } catch (IgniteCheckedException e) { throw new IOException("Failed to get secondary file system factory.", e); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java index 76d9bff5a712d..2059c8dff913b 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java @@ -81,13 +81,13 @@ public static HadoopIgfsSecondaryFileSystemDelegate secondaryFileSystemDelegate( * @return Delegate. */ @SuppressWarnings("unchecked") - public static HadoopFileSystemFactoryDelegate fileSystemFactoryDelegate(Object proxy) { + public static HadoopFileSystemFactoryDelegate fileSystemFactoryDelegate(ClassLoader ldr, Object proxy) { String clsName = FACTORY_CLS_MAP.get(proxy.getClass().getName()); if (clsName == null) clsName = DFLT_FACTORY_CLS; - return newInstance(clsName, null, proxy); + return newInstance(clsName, ldr, proxy); } /** diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java index 203965cebc137..e336fadb1a19f 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java @@ -56,7 +56,6 @@ import java.net.URI; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -86,7 +85,7 @@ public HadoopIgfsSecondaryFileSystemDelegateImpl(IgniteHadoopIgfsSecondaryFileSy if (factory0 == null) factory0 = new CachingHadoopFileSystemFactory(); - factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory0); + factory = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory0); } /** {@inheritDoc} */ diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java index 8c95a0ea57d51..f35d644361acc 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java @@ -70,7 +70,8 @@ private void checkParameters(String keyTab, String keyTabPrincipal, long relogin GridTestUtils.assertThrows(null, new Callable() { @Override public Object call() throws Exception { - HadoopFileSystemFactoryDelegate delegate = HadoopDelegateUtils.fileSystemFactoryDelegate(fac); + HadoopFileSystemFactoryDelegate delegate = HadoopDelegateUtils.fileSystemFactoryDelegate( + getClass().getClassLoader(), fac); delegate.start(); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java index 7cf7f2dcaa1f2..5d955d449a5be 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java @@ -330,7 +330,7 @@ public TestFactory(CachingHadoopFileSystemFactory factory) { /** {@inheritDoc} */ @Override public void start() throws IgniteException { - delegate = HadoopDelegateUtils.fileSystemFactoryDelegate(factory); + delegate = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory); delegate.start(); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java index adb1330f2a319..453d0c70dedc9 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java @@ -30,7 +30,6 @@ import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory; import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils; import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate; -import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils; import org.apache.ignite.internal.processors.igfs.IgfsEx; import org.apache.ignite.internal.processors.igfs.IgfsUtils; import org.apache.ignite.internal.processors.igfs.IgfsSecondaryFileSystemTestAdapter; @@ -50,7 +49,7 @@ public class HadoopIgfsSecondaryFileSystemTestAdapter implements IgfsSecondaryFi public HadoopIgfsSecondaryFileSystemTestAdapter(HadoopFileSystemFactory factory) { assert factory != null; - this.factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory); + this.factory = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory); this.factory.start(); } diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java index ae03f145dbd05..78f457bd58a20 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java @@ -37,7 +37,6 @@ import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils; import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate; -import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils; import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.U; @@ -185,7 +184,8 @@ private void before() throws Exception { fac.setConfigPaths(primaryConfFullPath); fac.setUri(primaryFsUriStr); - HadoopFileSystemFactoryDelegate facDelegate = HadoopDelegateUtils.fileSystemFactoryDelegate(fac); + HadoopFileSystemFactoryDelegate facDelegate = HadoopDelegateUtils.fileSystemFactoryDelegate( + getClass().getClassLoader(), fac); facDelegate.start(); From ef9d6cf9e334c35b03dfa42e4ce0680c85a693a4 Mon Sep 17 00:00:00 2001 From: iveselovskiy Date: Thu, 10 Nov 2016 18:47:09 +0300 Subject: [PATCH 337/487] IGNITE-4208: Hadoop: Fixed a bug preventing normal secondary file system start. This closes #1228. --- .../apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java | 2 +- .../apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java | 2 +- .../processors/hadoop/delegate/HadoopDelegateUtils.java | 4 ++-- .../delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java | 3 +-- .../impl/fs/KerberosHadoopFileSystemFactorySelfTest.java | 3 ++- .../hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java | 2 +- .../impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java | 3 +-- .../impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java | 4 ++-- 8 files changed, 11 insertions(+), 12 deletions(-) diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java index 7133c087b9dbe..866fc1a2aad02 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java @@ -336,7 +336,7 @@ public void colocateFileWrites(boolean colocateFileWrites) { HadoopFileSystemFactory factory0 = (HadoopFileSystemFactory)paths.getPayload(getClass().getClassLoader()); - factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory0); + factory = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory0); } catch (IgniteCheckedException e) { throw new IOException("Failed to get secondary file system factory.", e); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java index 18b8bf9fc7e19..82ad68309d547 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java @@ -345,7 +345,7 @@ private void initialize(URI name, Configuration cfg) throws IOException { HadoopFileSystemFactory factory0 = (HadoopFileSystemFactory) paths.getPayload(getClass().getClassLoader()); - factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory0); + factory = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory0); } catch (IgniteCheckedException e) { throw new IOException("Failed to get secondary file system factory.", e); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java index 76d9bff5a712d..2059c8dff913b 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopDelegateUtils.java @@ -81,13 +81,13 @@ public static HadoopIgfsSecondaryFileSystemDelegate secondaryFileSystemDelegate( * @return Delegate. */ @SuppressWarnings("unchecked") - public static HadoopFileSystemFactoryDelegate fileSystemFactoryDelegate(Object proxy) { + public static HadoopFileSystemFactoryDelegate fileSystemFactoryDelegate(ClassLoader ldr, Object proxy) { String clsName = FACTORY_CLS_MAP.get(proxy.getClass().getName()); if (clsName == null) clsName = DFLT_FACTORY_CLS; - return newInstance(clsName, null, proxy); + return newInstance(clsName, ldr, proxy); } /** diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java index 203965cebc137..e336fadb1a19f 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopIgfsSecondaryFileSystemDelegateImpl.java @@ -56,7 +56,6 @@ import java.net.URI; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -86,7 +85,7 @@ public HadoopIgfsSecondaryFileSystemDelegateImpl(IgniteHadoopIgfsSecondaryFileSy if (factory0 == null) factory0 = new CachingHadoopFileSystemFactory(); - factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory0); + factory = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory0); } /** {@inheritDoc} */ diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java index 8c95a0ea57d51..f35d644361acc 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/fs/KerberosHadoopFileSystemFactorySelfTest.java @@ -70,7 +70,8 @@ private void checkParameters(String keyTab, String keyTabPrincipal, long relogin GridTestUtils.assertThrows(null, new Callable() { @Override public Object call() throws Exception { - HadoopFileSystemFactoryDelegate delegate = HadoopDelegateUtils.fileSystemFactoryDelegate(fac); + HadoopFileSystemFactoryDelegate delegate = HadoopDelegateUtils.fileSystemFactoryDelegate( + getClass().getClassLoader(), fac); delegate.start(); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java index 7cf7f2dcaa1f2..5d955d449a5be 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopFIleSystemFactorySelfTest.java @@ -330,7 +330,7 @@ public TestFactory(CachingHadoopFileSystemFactory factory) { /** {@inheritDoc} */ @Override public void start() throws IgniteException { - delegate = HadoopDelegateUtils.fileSystemFactoryDelegate(factory); + delegate = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory); delegate.start(); diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java index adb1330f2a319..453d0c70dedc9 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java @@ -30,7 +30,6 @@ import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory; import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils; import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate; -import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils; import org.apache.ignite.internal.processors.igfs.IgfsEx; import org.apache.ignite.internal.processors.igfs.IgfsUtils; import org.apache.ignite.internal.processors.igfs.IgfsSecondaryFileSystemTestAdapter; @@ -50,7 +49,7 @@ public class HadoopIgfsSecondaryFileSystemTestAdapter implements IgfsSecondaryFi public HadoopIgfsSecondaryFileSystemTestAdapter(HadoopFileSystemFactory factory) { assert factory != null; - this.factory = HadoopDelegateUtils.fileSystemFactoryDelegate(factory); + this.factory = HadoopDelegateUtils.fileSystemFactoryDelegate(getClass().getClassLoader(), factory); this.factory.start(); } diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java index ae03f145dbd05..78f457bd58a20 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopSecondaryFileSystemConfigurationTest.java @@ -37,7 +37,6 @@ import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils; import org.apache.ignite.internal.processors.hadoop.delegate.HadoopFileSystemFactoryDelegate; -import org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsUtils; import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.U; @@ -185,7 +184,8 @@ private void before() throws Exception { fac.setConfigPaths(primaryConfFullPath); fac.setUri(primaryFsUriStr); - HadoopFileSystemFactoryDelegate facDelegate = HadoopDelegateUtils.fileSystemFactoryDelegate(fac); + HadoopFileSystemFactoryDelegate facDelegate = HadoopDelegateUtils.fileSystemFactoryDelegate( + getClass().getClassLoader(), fac); facDelegate.start(); From 884b281218d88c028daab25d35c14ee2b41be36e Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Thu, 10 Nov 2016 20:35:15 +0300 Subject: [PATCH 338/487] IGNITE-4186 .NET: Fix "Invalid session release request" exception in IgniteSessionStateStoreProvider.SetAndReleaseItemExclusive This closes #1227 --- .../IgniteSessionStateStoreProviderTest.cs | 49 ++++++++++++++++--- .../IgniteSessionStateStoreProvider.cs | 21 +++++--- 2 files changed, 57 insertions(+), 13 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs index 9c3b07c1a0762..c6e3b30e2db2f 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs @@ -209,6 +209,24 @@ public void TestCaching() Assert.AreEqual(TimeSpan.Zero, lockAge); Assert.AreEqual(SessionStateActions.None, actions); + // SetAndRelease with no lock. This happens with certain versions of ASP.NET. + var item = provider.CreateNewStoreData(HttpContext, 7); + // ReSharper disable once AssignNullToNotNullAttribute (lockId is not supposed to be null, but it can be). + provider.SetAndReleaseItemExclusive(HttpContext, Id, item, null, true); + + // Check added item. + res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions); + Assert.IsNotNull(res); + Assert.IsNull(lockId); + Assert.AreEqual(7, res.Timeout); + Assert.IsFalse(locked); + Assert.AreEqual(TimeSpan.Zero, lockAge); + Assert.AreEqual(SessionStateActions.None, actions); + + // Remove item. + // ReSharper disable once AssignNullToNotNullAttribute (lockId is not supposed to be null, but it can be). + provider.RemoveItem(HttpContext, Id, null, null); + // Add item. provider.CreateUninitializedItem(HttpContext, Id, 7); @@ -228,7 +246,7 @@ public void TestCaching() Assert.IsFalse(locked); Assert.AreEqual(TimeSpan.Zero, lockAge); Assert.AreEqual(SessionStateActions.None, actions); - provider.SetAndReleaseItemExclusive(HttpContext, Id, UpdateStoreData(res), lockId, true); + provider.SetAndReleaseItemExclusive(HttpContext, Id, UpdateStoreData(res), lockId, false); // Not locked, item present. res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions); @@ -331,20 +349,37 @@ public void TestExpiry() Assert.IsFalse(GetProvider().SetItemExpireCallback(null)); // Check there is no item. - var res = provider.GetItem(HttpContext, "myId", out locked, out lockAge, out lockId, out actions); + var res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions); Assert.IsNull(res); - // Put an item. - provider.CreateUninitializedItem(HttpContext, "myId", 1); + // Put an item with CreateUninitializedItem and check. + provider.CreateUninitializedItem(HttpContext, Id, 1); + CheckExpiry(provider); + + // Put an item with SetAndReleaseItemExclusive and check. + var data = provider.CreateNewStoreData(HttpContext, 1); + provider.SetAndReleaseItemExclusive(HttpContext, Id, data, lockId, true); + CheckExpiry(provider); + } - // Check that it is there. - res = provider.GetItem(HttpContext, "myId", out locked, out lockAge, out lockId, out actions); + /// + /// Checks item expiration. + /// + private static void CheckExpiry(SessionStateStoreProviderBase provider) + { + bool locked; + TimeSpan lockAge; + object lockId; + SessionStateActions actions; + + // Check that item is present. + var res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions); Assert.IsNotNull(res); // Wait a minute and check again. Thread.Sleep(TimeSpan.FromMinutes(1.05)); - res = provider.GetItem(HttpContext, "myId", out locked, out lockAge, out lockId, out actions); + res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions); Assert.IsNull(res); } diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs index 86035dd2037bc..a5e81992f227a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs @@ -294,11 +294,20 @@ public override void ReleaseItemExclusive(HttpContext context, string id, object var data = (IgniteSessionStateStoreData) item; - if (!(lockId is long) || data.LockId != (long) lockId) - throw new IgniteException(string.Format(CultureInfo.InvariantCulture, - "Invalid session release request, expected lockId: {0}, actual: {1}", data.LockId, lockId)); + if (newItem) + { + var cache = _expiryCacheHolder.GetCacheWithExpiry(data.Timeout * 60); + + PutItem(key, data, cache); + } + else + { + if (!(lockId is long) || data.LockId != (long) lockId) + throw new IgniteException(string.Format(CultureInfo.InvariantCulture, + "Invalid session release request, expected lockId: {0}, actual: {1}", data.LockId, lockId)); - SetAndUnlockItem(key, data); + SetAndUnlockItem(key, data); + } } /// @@ -333,7 +342,7 @@ public override void ResetItemTimeout(HttpContext context, string id) /// /// The for the current request. /// The session-state - /// value for the new . + /// value for the new , in minutes. /// /// A new for the current request. /// @@ -349,7 +358,7 @@ public override SessionStateStoreData CreateNewStoreData(HttpContext context, in /// The /// for the current request. /// The session - /// for the current request. + /// for the current request, in minutes. public override void CreateUninitializedItem(HttpContext context, string id, int timeout) { var cache = _expiryCacheHolder.GetCacheWithExpiry((long) timeout * 60); From eac5f0c05b85b620f979bd73d5043ae521b6f2f4 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Thu, 10 Nov 2016 21:13:13 +0300 Subject: [PATCH 339/487] IGNITE-4100: Improved README and DEVNOTES for CPP. --- modules/platforms/cpp/DEVNOTES.txt | 10 +++++++--- modules/platforms/cpp/README.txt | 10 +++++++--- modules/platforms/cpp/examples/README.txt | 16 +++++++++------- 3 files changed, 23 insertions(+), 13 deletions(-) diff --git a/modules/platforms/cpp/DEVNOTES.txt b/modules/platforms/cpp/DEVNOTES.txt index 924b0d8281134..5d0b2eb5c8800 100644 --- a/modules/platforms/cpp/DEVNOTES.txt +++ b/modules/platforms/cpp/DEVNOTES.txt @@ -3,7 +3,9 @@ Apache Ignite C++ Build Instructions Here you can find instruction on how to build Apache Ignite C++ core library and stand-alone node binary. To build examples you need to build and install core Apache Ignite library then refer to $IGNITE_HOME/platforms/cpp/example/README.txt for -futher instructions. +further instructions. + +For details on ODBC driver installation and usage please refer to odbc/README.txt. Building on Linux With Autotools ---------------------------------- @@ -15,7 +17,9 @@ Common Requirements: * JAVA_HOME environment variable must be set pointing to Java installation directory. * IGNITE_HOME environment variable must be set to Ignite installation directory. * To build ODBC driver you need to install ODBC Driver Manager on you system. - Apache Ignite has been tested with UnixODBC. + Apache Ignite has been tested with UnixODBC. Make sure that you install development + version of the driver manager which contains header files that are necessary for + building a driver. Building the Apache Ignite C++ components: * Navigate to the directory $IGNITE_HOME/platforms/cpp @@ -37,7 +41,7 @@ use configure script with the following arguments: With the config like that you should not have any ignite dependencies as jvm.dll installed to build ODBC driver. -NOTE: You may want to use "make install" command to install Ignite libraries and heasders +NOTE: You may want to use "make install" command to install Ignite libraries and headers for your operation system. Note however that this action may require superuser privileges. Building on Windows with Visual Studio (tm) diff --git a/modules/platforms/cpp/README.txt b/modules/platforms/cpp/README.txt index dbde9f066e744..b1d7eef307bbd 100644 --- a/modules/platforms/cpp/README.txt +++ b/modules/platforms/cpp/README.txt @@ -2,7 +2,7 @@ Apache Ignite C++ ================================== Apache Ignite C++ provides data grid functionality. -Using Apache Ignite C++ APIs you can execute perform concurrent operations on +Using Apache Ignite C++ APIs you can perform concurrent operations on the data stored in cache. Apache Ignite C++ can access cluster and share data with .Net and @@ -11,14 +11,18 @@ Java applications using binary object format. Support for the following will be added in next releases: * ACID transactions management. * Distributed locks. - * Async operations. + * Asynchronous operations. * Cache SQL continuous queries. * Event listening. * Compute grid functionality. Full source code is provided. Users should build the library for intended platform. + For build instructions please refer to DEVNOTES.txt. +For details on ODBC driver installation and usage please refer to +$IGNITE_HOME/platforms/cpp/odbc/README.txt. + Linux info ============== @@ -49,7 +53,7 @@ Files list: * ignite.exe - executable to start standalone Ignite C++ node. * ignite.core.dll - Ignite C++ API library. - * odbc.dll - Ignite ODBC driver. + * ignite.odbc.dll - Ignite ODBC driver. Development: diff --git a/modules/platforms/cpp/examples/README.txt b/modules/platforms/cpp/examples/README.txt index 29df7322e8eac..91e48696a11bd 100644 --- a/modules/platforms/cpp/examples/README.txt +++ b/modules/platforms/cpp/examples/README.txt @@ -6,8 +6,13 @@ Common requirements * Java Development Kit (JDK) must be installed: https://java.com/en/download/index.jsp * JAVA_HOME environment variable must be set pointing to Java installation directory. * IGNITE_HOME environment variable must be set to Ignite installation directory. - * Ignite must be build and packaged using Maven. You can use the followin Maven command: mvn clean package -DskipTests - * Apache Ignite C++ must be built according to instructions for your platform. + * Ignite must be built and packaged using Maven. You can use the following Maven command: mvn clean package -DskipTests + * Apache Ignite C++ must be built according to instructions for your platform. Refer to + $IGNITE_HOME/platforms/cpp/DEVNOTES.txt for instructions. + * For odbc-example additionally ODBC Driver Manager must be present and installed on your platform and + Apache Ignite ODBC driver must be built and installed according to instructions for your platform. Refer to + $IGNITE_HOME/platforms/cpp/DEVNOTES.txt for build instructions and to $IGNITE_HOME/platforms/cpp/odbc/README.txt. + for installation instructions. Running examples on Linux ---------------------------------- @@ -22,11 +27,8 @@ To build examples execute the following commands one by one from examples root d As a result executables will appear in every example's directory. -Before running examples ensure that: - * LD_LIBRARY_PATH environment variable is set and pointing to a directory with "libjvm.so" library. Typically this - library is located in $JAVA_HOME/jre/lib/amd64/server directory. - * For odbc-example additionaly ODBC Driver Manager must be present and installed on your platform and - Apache Ignite ODBC driver must be built and installed according to instructions for your platform. +Before running examples ensure that LD_LIBRARY_PATH environment variable is set and pointing to a directory with +"libjvm.so" library. Typically this library is located in $JAVA_HOME/jre/lib/amd64/server directory. Running examples on Windows ---------------------------------- From e159fdee0152a6df1bbf4dafc3d18a284e86a57e Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 11 Nov 2016 09:39:32 +0700 Subject: [PATCH 340/487] Fixed classnames.properties generation for ignite-hadoop module. --- modules/hadoop/pom.xml | 49 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml index a3f40e5fe82e2..98f85768c3261 100644 --- a/modules/hadoop/pom.xml +++ b/modules/hadoop/pom.xml @@ -126,6 +126,55 @@ + + org.codehaus.mojo + exec-maven-plugin + 1.3.2 + + + org.apache.ignite + ignite-tools + ${project.version} + + + + + process-classes + + java + + + true + org.apache.ignite.tools.classgen.ClassesGenerator + + ${project.basedir}/target/classes + + + + org.apache.ignite + + + + + + org.apache.maven.plugins maven-jar-plugin From 5a4ebd5de8751dcf32a26c96bf4f39e43bcbb341 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 11 Nov 2016 09:49:41 +0700 Subject: [PATCH 341/487] Fixed classnames.properties generation for ignite-hadoop module. --- modules/hadoop/pom.xml | 49 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml index c02a57405f787..15d6eee3c63ac 100644 --- a/modules/hadoop/pom.xml +++ b/modules/hadoop/pom.xml @@ -126,6 +126,55 @@ + + org.codehaus.mojo + exec-maven-plugin + 1.3.2 + + + org.apache.ignite + ignite-tools + ${project.version} + + + + + process-classes + + java + + + true + org.apache.ignite.tools.classgen.ClassesGenerator + + ${project.basedir}/target/classes + + + + org.apache.ignite + + + + + + org.apache.maven.plugins maven-jar-plugin From 73a8fa8b635cce3b9d8dcad364a32d29f12d4398 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 11 Nov 2016 10:20:32 +0700 Subject: [PATCH 342/487] Fixed classnames.properties generation for ignite-hadoop module. --- modules/hadoop/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml index 15d6eee3c63ac..d0b0481775d42 100644 --- a/modules/hadoop/pom.xml +++ b/modules/hadoop/pom.xml @@ -168,7 +168,7 @@ # ]]> - org.apache.ignite + org.apache.ignite.hadoop:org.apache.ignite.internal.processors.hadoop From 53876d3f1b3e561702fc0cd8c6ddd078d6c7dce6 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 11 Nov 2016 10:26:09 +0700 Subject: [PATCH 343/487] Fixed classnames.properties generation for ignite-hadoop module. --- modules/hadoop/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml index 98f85768c3261..f5bc081cbac2c 100644 --- a/modules/hadoop/pom.xml +++ b/modules/hadoop/pom.xml @@ -168,7 +168,7 @@ # ]]> - org.apache.ignite + org.apache.ignite.hadoop:org.apache.ignite.internal.processors.hadoop From f8aa957327312d76f90231b9bfe6d386d1d4ec37 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 11 Nov 2016 15:56:42 +0700 Subject: [PATCH 344/487] Reverted wrong commit. --- modules/hadoop/pom.xml | 49 ------------------------------------------ 1 file changed, 49 deletions(-) diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml index d0b0481775d42..c02a57405f787 100644 --- a/modules/hadoop/pom.xml +++ b/modules/hadoop/pom.xml @@ -126,55 +126,6 @@ - - org.codehaus.mojo - exec-maven-plugin - 1.3.2 - - - org.apache.ignite - ignite-tools - ${project.version} - - - - - process-classes - - java - - - true - org.apache.ignite.tools.classgen.ClassesGenerator - - ${project.basedir}/target/classes - - - - org.apache.ignite.hadoop:org.apache.ignite.internal.processors.hadoop - - - - - - org.apache.maven.plugins maven-jar-plugin From c6921a311f437504a45a4667ddde85b14269ba57 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 11 Nov 2016 16:01:33 +0700 Subject: [PATCH 345/487] Fixed classnames.properties generation for ignite-hadoop module. --- modules/hadoop/pom.xml | 49 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml index c02a57405f787..d0b0481775d42 100644 --- a/modules/hadoop/pom.xml +++ b/modules/hadoop/pom.xml @@ -126,6 +126,55 @@ + + org.codehaus.mojo + exec-maven-plugin + 1.3.2 + + + org.apache.ignite + ignite-tools + ${project.version} + + + + + process-classes + + java + + + true + org.apache.ignite.tools.classgen.ClassesGenerator + + ${project.basedir}/target/classes + + + + org.apache.ignite.hadoop:org.apache.ignite.internal.processors.hadoop + + + + + + org.apache.maven.plugins maven-jar-plugin From d69e26dd8d4fd9383a149c93c251911a8dd67528 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Fri, 11 Nov 2016 12:22:55 +0300 Subject: [PATCH 346/487] IGNITE-4126 .NET: Add IgniteConfiguration.SwapSpaceSpi --- .../utils/PlatformConfigurationUtils.java | 61 ++++++++- .../Apache.Ignite.Core.Tests.csproj | 1 + .../Cache/CacheSwapSpaceTest.cs | 120 ++++++++++++++++++ .../IgniteConfigurationSerializerTest.cs | 20 ++- .../IgniteConfigurationTest.cs | 22 +++- .../Apache.Ignite.Core.csproj | 3 + .../Cache/Configuration/CacheConfiguration.cs | 2 + .../Apache.Ignite.Core/IgniteConfiguration.cs | 15 ++- .../IgniteConfigurationSection.xsd | 37 ++++++ .../Impl/SwapSpace/SwapSpaceSerializer.cs | 99 +++++++++++++++ .../SwapSpace/File/FileSwapSpaceSpi.cs | 105 +++++++++++++++ .../SwapSpace/ISwapSpaceSpi.cs | 35 +++++ 12 files changed, 509 insertions(+), 11 deletions(-) create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Impl/SwapSpace/SwapSpaceSerializer.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/File/FileSwapSpaceSpi.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/ISwapSpaceSpi.cs diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java index 26374f0c220ba..9480daeb6cb1d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java @@ -51,6 +51,9 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.spi.swapspace.SwapSpaceSpi; +import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi; +import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpiMBean; import org.apache.ignite.transactions.TransactionConcurrency; import org.apache.ignite.transactions.TransactionIsolation; @@ -66,8 +69,14 @@ /** * Configuration utils. */ -@SuppressWarnings("unchecked") +@SuppressWarnings({"unchecked", "TypeMayBeWeakened"}) public class PlatformConfigurationUtils { + /** */ + private static final byte SWAP_TYP_NONE = 0; + + /** */ + private static final byte SWAP_TYP_FILE = 1; + /** * Write .Net configuration to the stream. * @@ -202,7 +211,7 @@ public static NearCacheConfiguration readNearConfiguration(BinaryRawReader in) { * @param in Stream. * @return Eviction policy. */ - public static EvictionPolicy readEvictionPolicy(BinaryRawReader in) { + private static EvictionPolicy readEvictionPolicy(BinaryRawReader in) { byte plcTyp = in.readByte(); switch (plcTyp) { @@ -277,7 +286,7 @@ public static PlatformAffinityFunction readAffinityFunction(BinaryRawReaderEx in * @param cfg NearCacheConfiguration. */ @SuppressWarnings("TypeMayBeWeakened") - public static void writeNearConfiguration(BinaryRawWriter out, NearCacheConfiguration cfg) { + private static void writeNearConfiguration(BinaryRawWriter out, NearCacheConfiguration cfg) { assert cfg != null; out.writeInt(cfg.getNearStartSize()); @@ -371,7 +380,7 @@ else if (p instanceof LruEvictionPolicy) { * @param in Stream. * @return QueryEntity. */ - public static QueryEntity readQueryEntity(BinaryRawReader in) { + private static QueryEntity readQueryEntity(BinaryRawReader in) { QueryEntity res = new QueryEntity(); res.setKeyType(in.readString()); @@ -422,7 +431,7 @@ public static QueryEntity readQueryEntity(BinaryRawReader in) { * @param in Reader. * @return Query index. */ - public static QueryIndex readQueryIndex(BinaryRawReader in) { + private static QueryIndex readQueryIndex(BinaryRawReader in) { QueryIndex res = new QueryIndex(); res.setName(in.readString()); @@ -529,6 +538,27 @@ public static void readIgniteConfiguration(BinaryRawReaderEx in, IgniteConfigura cfg.setTransactionConfiguration(tx); } + + byte swapType = in.readByte(); + + switch (swapType) { + case SWAP_TYP_FILE: { + FileSwapSpaceSpi swap = new FileSwapSpaceSpi(); + + swap.setBaseDirectory(in.readString()); + swap.setMaximumSparsity(in.readFloat()); + swap.setMaxWriteQueueSize(in.readInt()); + swap.setReadStripesNumber(in.readInt()); + swap.setWriteBufferSize(in.readInt()); + + cfg.setSwapSpaceSpi(swap); + + break; + } + + default: + assert swapType == SWAP_TYP_NONE; + } } /** @@ -537,7 +567,7 @@ public static void readIgniteConfiguration(BinaryRawReaderEx in, IgniteConfigura * @param cfg IgniteConfiguration to update. * @param in Reader. */ - public static void readCacheConfigurations(BinaryRawReaderEx in, IgniteConfiguration cfg) { + private static void readCacheConfigurations(BinaryRawReaderEx in, IgniteConfiguration cfg) { int len = in.readInt(); if (len == 0) @@ -569,7 +599,7 @@ public static void readCacheConfigurations(BinaryRawReaderEx in, IgniteConfigura * @param cfg IgniteConfiguration to update. * @param in Reader. */ - public static void readDiscoveryConfiguration(BinaryRawReader in, IgniteConfiguration cfg) { + private static void readDiscoveryConfiguration(BinaryRawReader in, IgniteConfiguration cfg) { boolean hasConfig = in.readBoolean(); if (!hasConfig) @@ -917,6 +947,23 @@ public static void writeIgniteConfiguration(BinaryRawWriter w, IgniteConfigurati else w.writeBoolean(false); + SwapSpaceSpi swap = cfg.getSwapSpaceSpi(); + + if (swap instanceof FileSwapSpaceSpiMBean) { + w.writeByte(SWAP_TYP_FILE); + + FileSwapSpaceSpiMBean fileSwap = (FileSwapSpaceSpiMBean)swap; + + w.writeString(fileSwap.getBaseDirectory()); + w.writeFloat(fileSwap.getMaximumSparsity()); + w.writeInt(fileSwap.getMaxWriteQueueSize()); + w.writeInt(fileSwap.getReadStripesNumber()); + w.writeInt(fileSwap.getWriteBufferSize()); + } + else { + w.writeByte(SWAP_TYP_NONE); + } + w.writeString(cfg.getIgniteHome()); w.writeLong(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getInit()); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj index 34d93d7f37481..a9de399bdc252 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj @@ -67,6 +67,7 @@ + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs new file mode 100644 index 0000000000000..296fd1319552f --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +namespace Apache.Ignite.Core.Tests.Cache +{ + using System.IO; + using System.Linq; + using System.Threading; + using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Cache.Eviction; + using Apache.Ignite.Core.Discovery.Tcp; + using Apache.Ignite.Core.Impl; + using Apache.Ignite.Core.SwapSpace.File; + using NUnit.Framework; + + /// + /// Tests the swap space. + /// + public class CacheSwapSpaceTest + { + /** */ + private readonly string _tempDir = IgniteUtils.GetTempDirectoryName(); + + /// + /// Fixture tear down. + /// + [TestFixtureTearDown] + public void FixtureTearDown() + { + Ignition.StopAll(true); + + Directory.Delete(_tempDir, true); + } + + /// + /// Tests that swap space is disabled by default and cache can't have EnableSwap. + /// + [Test] + public void TestDisabledSwapSpace() + { + var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration()); + + using (var ignite = Ignition.Start(cfg)) + { + // NoopSwapSpaceSpi is used by default. + Assert.IsNull(ignite.GetConfiguration().SwapSpaceSpi); + + var ex = Assert.Throws( + () => ignite.CreateCache(new CacheConfiguration {EnableSwap = true})); + + Assert.IsTrue(ex.Message.EndsWith("has not swap SPI configured")); + } + } + + /// + /// Tests the swap space. + /// + [Test] + public void TestSwapSpace() + { + const int entrySize = 1024; + + var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration()) + { + SwapSpaceSpi = new FileSwapSpaceSpi + { + BaseDirectory = _tempDir, + WriteBufferSize = 64 + } + }; + + using (var ignite = Ignition.Start(cfg)) + { + // Create cache with eviction and swap. + var cache = ignite.CreateCache(new CacheConfiguration("cache") + { + EnableSwap = true, + EvictionPolicy = new LruEvictionPolicy + { + MaxSize = 3 + }, + OffHeapMaxMemory = 5 * entrySize + }); + + // Populate to trigger eviction. + var data = Enumerable.Range(1, entrySize).Select(x => (byte) x).ToArray(); + + for (int i = 0; i < 10; i++) + cache[i] = data; + + // Check that swap files exist. + var files = Directory.GetFiles(_tempDir, "*.*", SearchOption.AllDirectories); + CollectionAssert.IsNotEmpty(files); + + // Wait for metrics update and check metrics. + Thread.Sleep(((TcpDiscoverySpi) ignite.GetConfiguration().DiscoverySpi).HeartbeatFrequency); + + var metrics = cache.GetMetrics(); + + Assert.AreEqual(4, metrics.OffHeapEntriesCount); // Entry takes more space than the value + Assert.AreEqual(3, metrics.OverflowSize / entrySize); // 10 - 3 - 4 = 3 + } + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs index 9fb5bbd96b1c2..ac3e553f45314 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs @@ -45,6 +45,7 @@ namespace Apache.Ignite.Core.Tests using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Lifecycle; using Apache.Ignite.Core.Log; + using Apache.Ignite.Core.SwapSpace.File; using Apache.Ignite.Core.Tests.Binary; using Apache.Ignite.Core.Transactions; using Apache.Ignite.NLog; @@ -113,6 +114,7 @@ public void TestPredefinedXml() + "; var reader = XmlReader.Create(new StringReader(xml)); @@ -199,6 +201,14 @@ public void TestPredefinedXml() Assert.AreEqual(new TimeSpan(0, 1, 2), comm.IdleConnectionTimeout); Assert.IsInstanceOf(cfg.Logger); + + var swap = cfg.SwapSpaceSpi as FileSwapSpaceSpi; + Assert.IsNotNull(swap); + Assert.AreEqual("abcd", swap.BaseDirectory); + Assert.AreEqual(0.7f, swap.MaximumSparsity); + Assert.AreEqual(25, swap.MaximumWriteQueueSize); + Assert.AreEqual(36, swap.ReadStripesNumber); + Assert.AreEqual(47, swap.WriteBufferSize); } /// @@ -599,7 +609,15 @@ private static IgniteConfiguration GetTestConfig() IsLateAffinityAssignment = false, SpringConfigUrl = "test", Logger = new IgniteNLogLogger(), - FailureDetectionTimeout = TimeSpan.FromMinutes(2) + FailureDetectionTimeout = TimeSpan.FromMinutes(2), + SwapSpaceSpi = new FileSwapSpaceSpi + { + MaximumSparsity = 0.1f, + MaximumWriteQueueSize = 55, + WriteBufferSize = 66, + ReadStripesNumber = 77, + BaseDirectory = "test" + } }; } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs index 30bae478574ca..2e39b9b3c368d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs @@ -34,6 +34,7 @@ namespace Apache.Ignite.Core.Tests using Apache.Ignite.Core.Discovery.Tcp.Static; using Apache.Ignite.Core.Events; using Apache.Ignite.Core.Impl; + using Apache.Ignite.Core.SwapSpace.File; using Apache.Ignite.Core.Transactions; using NUnit.Framework; @@ -79,6 +80,7 @@ public void TestDefaultValueAttributes() CheckDefaultValueAttributes(new LruEvictionPolicy()); CheckDefaultValueAttributes(new AtomicConfiguration()); CheckDefaultValueAttributes(new TransactionConfiguration()); + CheckDefaultValueAttributes(new FileSwapSpaceSpi()); } /// @@ -171,6 +173,14 @@ public void TestAllConfigurationProperties() Assert.AreEqual(com.UnacknowledgedMessagesBufferSize, resCom.UnacknowledgedMessagesBufferSize); Assert.AreEqual(cfg.FailureDetectionTimeout, resCfg.FailureDetectionTimeout); + + var swap = (FileSwapSpaceSpi) cfg.SwapSpaceSpi; + var resSwap = (FileSwapSpaceSpi) resCfg.SwapSpaceSpi; + Assert.AreEqual(swap.MaximumSparsity, resSwap.MaximumSparsity); + Assert.AreEqual(swap.BaseDirectory, resSwap.BaseDirectory); + Assert.AreEqual(swap.MaximumWriteQueueSize, resSwap.MaximumWriteQueueSize); + Assert.AreEqual(swap.ReadStripesNumber, resSwap.ReadStripesNumber); + Assert.AreEqual(swap.WriteBufferSize, resSwap.WriteBufferSize); } } @@ -398,7 +408,7 @@ private static void CheckDefaultValueAttributes(object obj) { var props = obj.GetType().GetProperties(); - foreach (var prop in props.Where(p => p.Name != "SelectorsCount")) + foreach (var prop in props.Where(p => p.Name != "SelectorsCount" && p.Name != "ReadStripesNumber")) { var attr = prop.GetCustomAttributes(true).OfType().FirstOrDefault(); var propValue = prop.GetValue(obj, null); @@ -495,7 +505,15 @@ private static IgniteConfiguration GetCustomConfig() SocketSendBufferSize = 2045, UnacknowledgedMessagesBufferSize = 3450 }, - FailureDetectionTimeout = TimeSpan.FromSeconds(3.5) + FailureDetectionTimeout = TimeSpan.FromSeconds(3.5), + SwapSpaceSpi = new FileSwapSpaceSpi + { + ReadStripesNumber = 64, + MaximumWriteQueueSize = 8, + WriteBufferSize = 9, + BaseDirectory = Path.GetTempPath(), + MaximumSparsity = 11.22f + } }; } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj index 66253f6c700bc..64d7388a3dadd 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj @@ -418,6 +418,7 @@ + @@ -471,6 +472,8 @@ + + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs index cb15564dc4750..c0c47229d907b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs @@ -507,6 +507,8 @@ internal void Validate(ILogger log) /// /// Flag indicating whether Ignite should use swap storage by default. + /// + /// Enabling this requires configured . /// [DefaultValue(DefaultEnableSwap)] public bool EnableSwap { get; set; } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs index 68433f74591b1..e7aa64eeb2eaf 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs @@ -39,8 +39,10 @@ namespace Apache.Ignite.Core using Apache.Ignite.Core.Impl; using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Common; + using Apache.Ignite.Core.Impl.SwapSpace; using Apache.Ignite.Core.Lifecycle; using Apache.Ignite.Core.Log; + using Apache.Ignite.Core.SwapSpace; using Apache.Ignite.Core.Transactions; using BinaryReader = Apache.Ignite.Core.Impl.Binary.BinaryReader; using BinaryWriter = Apache.Ignite.Core.Impl.Binary.BinaryWriter; @@ -297,10 +299,13 @@ internal void Write(BinaryWriter writer) writer.WriteInt((int) TransactionConfiguration.DefaultTransactionConcurrency); writer.WriteInt((int) TransactionConfiguration.DefaultTransactionIsolation); writer.WriteLong((long) TransactionConfiguration.DefaultTimeout.TotalMilliseconds); - writer.WriteLong((int) TransactionConfiguration.PessimisticTransactionLogLinger.TotalMilliseconds); + writer.WriteInt((int) TransactionConfiguration.PessimisticTransactionLogLinger.TotalMilliseconds); } else writer.WriteBoolean(false); + + // Swap space + SwapSpaceSerializer.Write(writer, SwapSpaceSpi); } /// @@ -386,6 +391,9 @@ private void ReadCore(BinaryReader r) PessimisticTransactionLogLinger = TimeSpan.FromMilliseconds(r.ReadInt()) }; } + + // Swap + SwapSpaceSpi = SwapSpaceSerializer.Read(r); } /// @@ -790,5 +798,10 @@ public TimeSpan FailureDetectionTimeout get { return _failureDetectionTimeout ?? DefaultFailureDetectionTimeout; } set { _failureDetectionTimeout = value; } } + + /// + /// Gets or sets the swap space SPI. + /// + public ISwapSpaceSpi SwapSpaceSpi { get; set; } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd index e9fb991853de9..06541daeda8e3 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd @@ -1107,6 +1107,43 @@ + + + Swap space SPI. + + + + + Assembly-qualified type name. + + + + + Base directory. + + + + + Maximum sparsity. This property defines maximum acceptable wasted file space to whole file size ratio. When this ratio becomes higher than specified number compacting thread starts working. + + + + + Maximum size of the write queue in bytes. If there are more values are waiting to be written to disk then specified size, SPI will block on write operation. + + + + + Read stripes number. Defines number of file channels to be used concurrently. Defaults to Environment.ProcessorCount. + + + + + Size of the write buffer, in bytes. Write to disk occurs only when this buffer is full. + + + + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/SwapSpace/SwapSpaceSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/SwapSpace/SwapSpaceSerializer.cs new file mode 100644 index 0000000000000..7fdbaadc69430 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/SwapSpace/SwapSpaceSerializer.cs @@ -0,0 +1,99 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.Impl.SwapSpace +{ + using System; + using System.Diagnostics; + using Apache.Ignite.Core.Binary; + using Apache.Ignite.Core.SwapSpace; + using Apache.Ignite.Core.SwapSpace.File; + + /// + /// SwapSpace config serializer. + /// + internal static class SwapSpaceSerializer + { + /// + /// SwapSpace type. + /// + private enum Type : byte + { + None = 0, + File = 1 + } + + /// + /// Writes the configuration to writer. + /// + public static void Write(IBinaryRawWriter writer, ISwapSpaceSpi spi) + { + Debug.Assert(writer != null); + + var fileSwap = spi as FileSwapSpaceSpi; + + if (spi == null) + { + writer.WriteByte((byte) Type.None); + } + else if (fileSwap != null) + { + writer.WriteByte((byte) Type.File); + + writer.WriteString(fileSwap.BaseDirectory); + writer.WriteFloat(fileSwap.MaximumSparsity); + writer.WriteInt(fileSwap.MaximumWriteQueueSize); + writer.WriteInt(fileSwap.ReadStripesNumber); + writer.WriteInt(fileSwap.WriteBufferSize); + + } + else + { + throw new InvalidOperationException("Unsupported swap space SPI: " + spi.GetType()); + } + } + + /// + /// Reads the configuration from reader. + /// + public static ISwapSpaceSpi Read(IBinaryRawReader reader) + { + Debug.Assert(reader != null); + + var type = (Type) reader.ReadByte(); + + switch (type) + { + case Type.None: + return null; + + case Type.File: + return new FileSwapSpaceSpi + { + BaseDirectory = reader.ReadString(), + MaximumSparsity = reader.ReadFloat(), + MaximumWriteQueueSize = reader.ReadInt(), + ReadStripesNumber = reader.ReadInt(), + WriteBufferSize = reader.ReadInt() + }; + + default: + throw new ArgumentOutOfRangeException("Invalid Swap Space SPI type: " + type); + } + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/File/FileSwapSpaceSpi.cs b/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/File/FileSwapSpaceSpi.cs new file mode 100644 index 0000000000000..2140275379960 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/File/FileSwapSpaceSpi.cs @@ -0,0 +1,105 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.SwapSpace.File +{ + using System; + using System.ComponentModel; + + /// + /// File-based swap space SPI implementation which holds keys in memory and values on disk. + /// It is intended for cases when value is bigger than 100 bytes, otherwise it will not + /// have any positive effect. + /// + public class FileSwapSpaceSpi : ISwapSpaceSpi + { + /// + /// Default value for property. + /// + public const float DefaultMaximumSparsity = 0.5f; + + /// + /// Default value for property. + /// + public const int DefaultWriteBufferSize = 64 * 1024; + + /// + /// Default value for property. + /// + public const int DefaultMaximumWriteQueueSize = 1024 * 1024; + + /// + /// Default value for property. + /// + public static readonly int DefaultReadStripesNumber = Environment.ProcessorCount; + + /// + /// Initializes a new instance of the class. + /// + public FileSwapSpaceSpi() + { + MaximumSparsity = DefaultMaximumSparsity; + MaximumWriteQueueSize = DefaultMaximumWriteQueueSize; + ReadStripesNumber = DefaultReadStripesNumber; + WriteBufferSize = DefaultWriteBufferSize; + } + + /// + /// Gets or sets the base directory. + /// + public string BaseDirectory { get; set; } + + /// + /// Gets or sets the maximum sparsity. This property defines maximum acceptable + /// wasted file space to whole file size ratio. + /// When this ratio becomes higher than specified number compacting thread starts working. + /// + /// + /// The maximum sparsity. Must be between 0 and 1. + /// + [DefaultValue(DefaultMaximumSparsity)] + public float MaximumSparsity { get; set; } + + /// + /// Gets or sets the maximum size of the write queue in bytes. If there are more values are waiting + /// to be written to disk then specified size, SPI will block on write operation. + /// + /// + /// The maximum size of the write queue, in bytes. + /// + [DefaultValue(DefaultMaximumWriteQueueSize)] + public int MaximumWriteQueueSize { get; set; } + + /// + /// Gets or sets the read stripes number. Defines number of file channels to be used concurrently. + /// Default is . + /// + /// + /// Number of read stripes. + /// + public int ReadStripesNumber { get; set; } + + /// + /// Gets or sets the size of the write buffer, in bytes. Write to disk occurs only when this buffer is full. + /// + /// + /// The size of the write buffer, in bytes. + /// + [DefaultValue(DefaultWriteBufferSize)] + public int WriteBufferSize { get; set; } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/ISwapSpaceSpi.cs b/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/ISwapSpaceSpi.cs new file mode 100644 index 0000000000000..a348b7bd15d4b --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/ISwapSpaceSpi.cs @@ -0,0 +1,35 @@ +/* + * 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. + */ + +namespace Apache.Ignite.Core.SwapSpace +{ + using System.Diagnostics.CodeAnalysis; + using Apache.Ignite.Core.SwapSpace.File; + + /// + /// Provides a mechanism in grid for storing data on disk. Ignite cache uses swap space to overflow + /// data to disk if it cannot fit in memory. + /// Logically storage is organized into independent 'spaces' in which data is stored. + /// + /// Only predefined implementation is supported now: . + /// + [SuppressMessage("Microsoft.Design", "CA1040:AvoidEmptyInterfaces")] + public interface ISwapSpaceSpi + { + // No-op. + } +} From a70f0bac3ac2487b8ab58598ad921daa952b485f Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 11 Nov 2016 13:03:40 +0300 Subject: [PATCH 347/487] IGNITE-4145: Fixes "No query result found for request" exception when running multiple queries concurrently. This closes #1218. --- .../query/h2/twostep/GridMergeIndex.java | 49 ++++++++++----- ...IgniteCacheQueryMultiThreadedSelfTest.java | 59 +++++++++++++++++++ 2 files changed, 93 insertions(+), 15 deletions(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java index 3914bd791472f..7ac2ee3ab4666 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java @@ -177,36 +177,50 @@ public void fail(UUID nodeId) { public final void addPage(GridResultPage page) { int pageRowsCnt = page.rowsInPage(); - if (pageRowsCnt != 0) - addPage0(page); - Counter cnt = remainingRows.get(page.source()); + // RemainingRowsCount should be updated before page adding to avoid race + // in GridMergeIndexUnsorted cursor iterator + int remainingRowsCount; + int allRows = page.response().allRows(); if (allRows != -1) { // Only the first page contains allRows count and is allowed to init counter. - assert !cnt.initialized : "Counter is already initialized."; + assert cnt.state == State.UNINITIALIZED : "Counter is already initialized."; + + remainingRowsCount = cnt.addAndGet(allRows - pageRowsCnt); - cnt.addAndGet(allRows); expRowsCnt.addAndGet(allRows); + // Add page before setting initialized flag to avoid race condition with adding last page + if (pageRowsCnt > 0) + addPage0(page); + // We need this separate flag to handle case when the first source contains only one page // and it will signal that all remaining counters are zero and fetch is finished. - cnt.initialized = true; + cnt.state = State.INITIALIZED; } + else { + remainingRowsCount = cnt.addAndGet(-pageRowsCnt); - if (cnt.addAndGet(-pageRowsCnt) == 0) { // Result can be negative in case of race between messages, it is ok. - boolean last = true; + if (pageRowsCnt > 0) + addPage0(page); + } - for (Counter c : remainingRows.values()) { // Check all the sources. - if (c.get() != 0 || !c.initialized) { - last = false; + if (remainingRowsCount == 0) { // Result can be negative in case of race between messages, it is ok. + if (cnt.state == State.UNINITIALIZED) + return; - break; - } + // Guarantee that finished state possible only if counter is zero and all pages was added + cnt.state = State.FINISHED; + + for (Counter c : remainingRows.values()) { // Check all the sources. + if (c.state != State.FINISHED) + return; } - if (last && lastSubmitted.compareAndSet(false, true)) { + if (lastSubmitted.compareAndSet(false, true)) { + // Add page-marker that last page was added addPage0(new GridResultPage(null, page.source(), null) { @Override public boolean isLast() { return true; @@ -426,11 +440,16 @@ private class FetchedIterator implements Iterator { } } + /** */ + enum State { + UNINITIALIZED, INITIALIZED, FINISHED + } + /** * Counter with initialization flag. */ private static class Counter extends AtomicInteger { /** */ - volatile boolean initialized; + volatile State state = State.UNINITIALIZED; } } \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java index be644e2ab01d9..efa6bd60affdb 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java @@ -21,6 +21,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashSet; +import java.util.List; import java.util.Random; import java.util.Set; import java.util.UUID; @@ -34,6 +35,7 @@ import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy; import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.SqlQuery; import org.apache.ignite.cache.query.annotations.QuerySqlField; import org.apache.ignite.configuration.CacheConfiguration; @@ -729,6 +731,63 @@ public void testMultiThreadedScanQuery() throws Exception { fut.get(); } + /** + * SqlFieldsQuery paging mechanics stress test + * + * @throws Exception If failed. + */ + @SuppressWarnings({"TooBroadScope"}) + public void testMultiThreadedSqlFieldsQuery() throws Throwable { + int threadCnt = 16; + final int keyCnt = 1100; // set resultSet size bigger than page size + final int logMod = 5000; + + final Ignite g = grid(0); + + // Put test values into cache. + final IgniteCache c = g.cache(null); + + for (int i = 0; i < keyCnt; i++) + c.put(i, new TestValue(i)); + + final AtomicInteger cnt = new AtomicInteger(); + + final AtomicBoolean done = new AtomicBoolean(); + + IgniteInternalFuture fut = multithreadedAsync( + new CAX() { + @Override public void applyx() throws IgniteCheckedException { + int iter = 0; + + while (!done.get() && !Thread.currentThread().isInterrupted()) { + iter++; + + List> entries = + c.query(new SqlFieldsQuery("SELECT * from TestValue").setPageSize(100)).getAll(); + + assert entries != null; + + assertEquals("Entries count is not as expected on iteration: " + iter, keyCnt, entries.size()); + + if (cnt.incrementAndGet() % logMod == 0) { + GridCacheQueryManager qryMgr = + ((IgniteKernal)g).internalCache().context().queries(); + + assert qryMgr != null; + + qryMgr.printMemoryStats(); + } + } + } + }, threadCnt); + + Thread.sleep(DURATION); + + done.set(true); + + fut.get(); + } + /** * Test value. */ From 8bb8bdda2e846dcc92a2fd449e64d7594b2700ed Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Fri, 11 Nov 2016 15:01:14 +0300 Subject: [PATCH 348/487] IGNITE-4053: Moved task error output from console to logger. This closes #1160. --- .../internal/ComputeTaskInternalFuture.java | 11 +++ .../util/future/GridFutureAdapter.java | 12 ++- ...mputeJobExecutionErrorToLogManualTest.java | 88 +++++++++++++++++++ 3 files changed, 109 insertions(+), 2 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/compute/GridComputeJobExecutionErrorToLogManualTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/ComputeTaskInternalFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/ComputeTaskInternalFuture.java index d511480f9822b..d6c54d7ee0b57 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/ComputeTaskInternalFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/ComputeTaskInternalFuture.java @@ -24,6 +24,7 @@ import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; import org.apache.ignite.compute.ComputeJobSibling; import org.apache.ignite.compute.ComputeTask; import org.apache.ignite.compute.ComputeTaskFuture; @@ -59,6 +60,9 @@ public class ComputeTaskInternalFuture extends GridFutureAdapter { @GridToStringExclude private ComputeFuture userFut; + /** */ + private transient IgniteLogger log; + /** * @param ses Task session instance. * @param ctx Kernal context. @@ -71,6 +75,8 @@ public ComputeTaskInternalFuture(ComputeTaskSession ses, GridKernalContext ctx) this.ctx = ctx; userFut = new ComputeFuture<>(this); + + log = ctx.log(ComputeTaskInternalFuture.class); } /** @@ -247,6 +253,11 @@ public ComputeTaskSession getTaskSession() { return S.toString(ComputeTaskInternalFuture.class, this, "super", super.toString()); } + /** {@inheritDoc} */ + @Override public IgniteLogger logger() { + return log; + } + /** * */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java index ea7a2021afe92..2cd534e0905a7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java @@ -21,6 +21,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.AbstractQueuedSynchronizer; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.IgniteFutureCancelledCheckedException; import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; @@ -263,11 +264,11 @@ private void notifyListener(IgniteInClosure> lsn lsnr.apply(this); } catch (IllegalStateException e) { - U.error(null, "Failed to notify listener (is grid stopped?) [fut=" + this + + U.error(logger(), "Failed to notify listener (is grid stopped?) [fut=" + this + ", lsnr=" + lsnr + ", err=" + e.getMessage() + ']', e); } catch (RuntimeException | Error e) { - U.error(null, "Failed to notify listener: " + lsnr, e); + U.error(logger(), "Failed to notify listener: " + lsnr, e); throw e; } @@ -413,6 +414,13 @@ private String state() { return s == INIT ? "INIT" : s == CANCELLED ? "CANCELLED" : "DONE"; } + /** + * @return Logger instance. + */ + @Nullable public IgniteLogger logger() { + return null; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(GridFutureAdapter.class, this, "state", state()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/compute/GridComputeJobExecutionErrorToLogManualTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/compute/GridComputeJobExecutionErrorToLogManualTest.java new file mode 100644 index 0000000000000..691bc9f88996f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/compute/GridComputeJobExecutionErrorToLogManualTest.java @@ -0,0 +1,88 @@ +/* + * 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.compute; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCompute; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Manual test to reproduce IGNITE-4053 + */ +public class GridComputeJobExecutionErrorToLogManualTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int GRID_CNT = 2; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setIpFinder(ipFinder); + + cfg.setDiscoverySpi(disco); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + startGridsMultiThreaded(GRID_CNT, true); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If fails. + */ + public void testRuntimeException() throws Exception { + Ignite ignite = grid(0); + + IgniteCompute async = ignite.compute().withAsync(); + async.run(new IgniteRunnable() { + @Override public void run() { + try { + Thread.sleep(500); + } + catch (InterruptedException e) { + // No-op. + } + } + }); + + async.future().listen(new IgniteInClosure>() { + @Override public void apply(IgniteFuture future) { + throw new RuntimeException(); + } + }); + } +} \ No newline at end of file From 7128a395085b60e86436f807b4bdbca83627d41a Mon Sep 17 00:00:00 2001 From: sboikov Date: Fri, 11 Nov 2016 15:29:38 +0300 Subject: [PATCH 349/487] ignite-4154 Optimize amount of data stored in discovery history Discovery history optimizations: - remove discarded message for discovery pending messages - remove duplicated data from TcpDiscoveryNodeAddedMessage.oldNodesDiscoData - do not store unnecessary data in discovery EnsuredMessageHistory - use special property for EnsuredMessageHistory size instead of IGNITE_DISCOVERY_HISTORY_SIZE Affinity history optimizations: - do not store calculated primary/backup maps in history - try save the same assignments instance for caches with similar affinity Exchange messages optimizations: - do not send duplicated partition state maps for caches with similar affinity - use zip compression for data sent in exchange messages --- .../apache/ignite/IgniteSystemProperties.java | 4 + .../affinity/AffinityAssignment.java | 88 ++++ .../affinity/GridAffinityAssignment.java | 8 +- .../affinity/GridAffinityAssignmentCache.java | 35 +- .../affinity/GridAffinityProcessor.java | 89 +++- .../affinity/GridAffinityUtils.java | 8 +- .../affinity/HistoryAffinityAssignment.java | 169 ++++++++ .../cache/CacheAffinitySharedManager.java | 57 ++- .../cache/DynamicCacheChangeBatch.java | 7 + .../cache/GridCacheAffinityManager.java | 6 +- .../GridCachePartitionExchangeManager.java | 284 +++++++++++-- .../processors/cache/GridCacheProcessor.java | 5 +- .../dht/GridClientPartitionTopology.java | 33 +- .../dht/GridDhtPartitionTopology.java | 3 +- .../dht/GridDhtPartitionTopologyImpl.java | 31 +- .../preloader/GridDhtPartitionFullMap.java | 18 + .../dht/preloader/GridDhtPartitionMap2.java | 53 ++- .../GridDhtPartitionsAbstractMessage.java | 40 +- .../GridDhtPartitionsExchangeFuture.java | 84 +--- .../GridDhtPartitionsFullMessage.java | 150 ++++++- .../GridDhtPartitionsSingleMessage.java | 132 +++++- .../GridDhtPartitionsSingleRequest.java | 4 +- .../dht/preloader/GridDhtPreloader.java | 4 +- .../continuous/GridContinuousProcessor.java | 4 +- .../ignite/internal/util/IgniteUtils.java | 64 +++ .../ignite/spi/discovery/tcp/ClientImpl.java | 26 +- .../ignite/spi/discovery/tcp/ServerImpl.java | 234 ++++++++++- .../TcpDiscoveryNodeAddFinishedMessage.java | 11 + .../TcpDiscoveryNodeAddedMessage.java | 33 +- ...cheExchangeMessageDuplicatedStateTest.java | 393 ++++++++++++++++++ .../IgniteCachePeekModesAbstractTest.java | 2 +- .../IgniteCacheGetRestartTest.java | 3 + ...ngDelayedPartitionMapExchangeSelfTest.java | 8 +- .../GridCacheRebalancingSyncSelfTest.java | 18 +- ...ridCacheSyncReplicatedPreloadSelfTest.java | 3 - .../IgniteCacheSyncRebalanceModeSelfTest.java | 4 +- ...ntinuousQueryFailoverAbstractSelfTest.java | 2 +- .../IgniteNoCustomEventsOnNodeStart.java | 7 + .../discovery/tcp/TcpDiscoverySelfTest.java | 134 +++++- .../junits/common/GridCommonAbstractTest.java | 25 +- .../testsuites/IgniteCacheTestSuite2.java | 3 + .../IgniteCacheOffheapEvictQueryTest.java | 7 + ...entQueryReplicatedNodeRestartSelfTest.java | 7 + ...tedQueryStopOnCancelOrTimeoutSelfTest.java | 7 + .../query/h2/sql/GridQueryParsingTest.java | 11 +- .../src/test/config/incorrect-store-cache.xml | 2 + .../test/config/jdbc-pojo-store-builtin.xml | 3 + .../src/test/config/jdbc-pojo-store-obj.xml | 3 + modules/spring/src/test/config/node.xml | 2 + modules/spring/src/test/config/node1.xml | 2 + .../config/pojo-incorrect-store-cache.xml | 2 + .../spring/src/test/config/store-cache.xml | 2 + .../spring/src/test/config/store-cache1.xml | 2 + 53 files changed, 2061 insertions(+), 275 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheExchangeMessageDuplicatedStateTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index ab6403f1e47a0..a75027b8026a7 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -383,6 +383,10 @@ public final class IgniteSystemProperties { /** Maximum size for discovery messages history. */ public static final String IGNITE_DISCOVERY_HISTORY_SIZE = "IGNITE_DISCOVERY_HISTORY_SIZE"; + /** Maximum number of discovery message history used to support client reconnect. */ + public static final String IGNITE_DISCOVERY_CLIENT_RECONNECT_HISTORY_SIZE = + "IGNITE_DISCOVERY_CLIENT_RECONNECT_HISTORY_SIZE"; + /** Number of cache operation retries in case of topology exceptions. */ public static final String IGNITE_CACHE_RETRIES_COUNT = "IGNITE_CACHE_RETRIES_COUNT"; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java new file mode 100644 index 0000000000000..06207d348d44e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/AffinityAssignment.java @@ -0,0 +1,88 @@ +/* + * 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.affinity; + +import org.apache.ignite.cluster.ClusterNode; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.UUID; + +/** + * Cached affinity calculations. + */ +public interface AffinityAssignment { + /** + * @return {@code True} if related discovery event did not not cause affinity assignment change and + * this assignment is just reference to the previous one. + */ + public boolean clientEventChange(); + + /** + * @return Affinity assignment computed by affinity function. + */ + public List> idealAssignment(); + + /** + * @return Affinity assignment. + */ + public List> assignment(); + + /** + * @return Topology version. + */ + public AffinityTopologyVersion topologyVersion(); + + /** + * Get affinity nodes for partition. + * + * @param part Partition. + * @return Affinity nodes. + */ + public List get(int part); + + /** + * Get affinity node IDs for partition. + * + * @param part Partition. + * @return Affinity nodes IDs. + */ + public HashSet getIds(int part); + + /** + * @return Nodes having primary partitions assignments. + */ + public Set primaryPartitionNodes(); + + /** + * Get primary partitions for specified node ID. + * + * @param nodeId Node ID to get primary partitions for. + * @return Primary partitions for specified node ID. + */ + public Set primaryPartitions(UUID nodeId); + + /** + * Get backup partitions for specified node ID. + * + * @param nodeId Node ID to get backup partitions for. + * @return Backup partitions for specified node ID. + */ + public Set backupPartitions(UUID nodeId); +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java index 568e4e8d62e63..35130a38da45d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java @@ -32,7 +32,7 @@ /** * Cached affinity calculations. */ -public class GridAffinityAssignment implements Serializable { +public class GridAffinityAssignment implements AffinityAssignment, Serializable { /** */ private static final long serialVersionUID = 0L; @@ -86,7 +86,7 @@ public class GridAffinityAssignment implements Serializable { this.topVer = topVer; this.assignment = assignment; - this.idealAssignment = idealAssignment; + this.idealAssignment = idealAssignment.equals(assignment) ? assignment : idealAssignment; primary = new HashMap<>(); backup = new HashMap<>(); @@ -274,10 +274,10 @@ private void initPrimaryBackupMaps() { if (o == this) return true; - if (o == null || getClass() != o.getClass()) + if (o == null || !(o instanceof AffinityAssignment)) return false; - return topVer.equals(((GridAffinityAssignment)o).topVer); + return topVer.equals(((AffinityAssignment)o).topologyVersion()); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java index a81b34dd77e74..a388c7affb65a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java @@ -78,7 +78,7 @@ public class GridAffinityAssignmentCache { private final int partsCnt; /** Affinity calculation results cache: topology version => partition => nodes. */ - private final ConcurrentNavigableMap affCache; + private final ConcurrentNavigableMap affCache; /** */ private List> idealAssignment; @@ -107,6 +107,9 @@ public class GridAffinityAssignmentCache { /** Full history size. */ private final AtomicInteger fullHistSize = new AtomicInteger(); + /** */ + private final Object similarAffKey; + /** * Constructs affinity cached calculations. * @@ -127,6 +130,7 @@ public GridAffinityAssignmentCache(GridKernalContext ctx, { assert ctx != null; assert aff != null; + assert nodeFilter != null; this.ctx = ctx; this.aff = aff; @@ -142,6 +146,17 @@ public GridAffinityAssignmentCache(GridKernalContext ctx, partsCnt = aff.partitions(); affCache = new ConcurrentSkipListMap<>(); head = new AtomicReference<>(new GridAffinityAssignment(AffinityTopologyVersion.NONE)); + + similarAffKey = ctx.affinity().similaryAffinityKey(aff, nodeFilter, backups, partsCnt); + + assert similarAffKey != null; + } + + /** + * @return Key to find caches with similar affinity. + */ + public Object similarAffinityKey() { + return similarAffKey; } /** @@ -170,7 +185,7 @@ public void initialize(AffinityTopologyVersion topVer, List> a GridAffinityAssignment assignment = new GridAffinityAssignment(topVer, affAssignment, idealAssignment); - affCache.put(topVer, assignment); + affCache.put(topVer, new HistoryAffinityAssignment(assignment)); head.set(assignment); for (Map.Entry entry : readyFuts.entrySet()) { @@ -300,7 +315,7 @@ public void clientEventTopologyChange(DiscoveryEvent evt, AffinityTopologyVersio GridAffinityAssignment assignmentCpy = new GridAffinityAssignment(topVer, aff); - affCache.put(topVer, assignmentCpy); + affCache.put(topVer, new HistoryAffinityAssignment(assignmentCpy)); head.set(assignmentCpy); for (Map.Entry entry : readyFuts.entrySet()) { @@ -328,7 +343,7 @@ public AffinityTopologyVersion lastVersion() { * @return Affinity assignment. */ public List> assignments(AffinityTopologyVersion topVer) { - GridAffinityAssignment aff = cachedAffinity(topVer); + AffinityAssignment aff = cachedAffinity(topVer); return aff.assignment(); } @@ -427,7 +442,7 @@ public void dumpDebugInfo() { * @param topVer Topology version. * @return Cached affinity. */ - public GridAffinityAssignment cachedAffinity(AffinityTopologyVersion topVer) { + public AffinityAssignment cachedAffinity(AffinityTopologyVersion topVer) { if (topVer.equals(AffinityTopologyVersion.NONE)) topVer = lastVersion(); else @@ -435,7 +450,7 @@ public GridAffinityAssignment cachedAffinity(AffinityTopologyVersion topVer) { assert topVer.topologyVersion() >= 0 : topVer; - GridAffinityAssignment cache = head.get(); + AffinityAssignment cache = head.get(); if (!cache.topologyVersion().equals(topVer)) { cache = affCache.get(topVer); @@ -463,7 +478,7 @@ public GridAffinityAssignment cachedAffinity(AffinityTopologyVersion topVer) { * @return {@code True} if primary changed or required affinity version not found in history. */ public boolean primaryChanged(int part, AffinityTopologyVersion startVer, AffinityTopologyVersion endVer) { - GridAffinityAssignment aff = affCache.get(startVer); + AffinityAssignment aff = affCache.get(startVer); if (aff == null) return false; @@ -475,7 +490,7 @@ public boolean primaryChanged(int part, AffinityTopologyVersion startVer, Affini ClusterNode primary = nodes.get(0); - for (GridAffinityAssignment assignment : affCache.tailMap(startVer, false).values()) { + for (AffinityAssignment assignment : affCache.tailMap(startVer, false).values()) { List nodes0 = assignment.assignment().get(part); if (nodes0.isEmpty()) @@ -549,10 +564,10 @@ private void onHistoryAdded(GridAffinityAssignment aff) { } if (rmvCnt > 0) { - Iterator it = affCache.values().iterator(); + Iterator it = affCache.values().iterator(); while (it.hasNext() && rmvCnt > 0) { - GridAffinityAssignment aff0 = it.next(); + AffinityAssignment aff0 = it.next(); it.remove(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java index 1726d029dbad3..b9182ae5c246e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java @@ -58,6 +58,7 @@ import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.lang.IgniteUuid; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -385,10 +386,16 @@ private Map> keysToNodes(@Nullable final String c } try { + AffinityAssignment assign0 = cctx.affinity().assignment(topVer); + + GridAffinityAssignment assign = assign0 instanceof GridAffinityAssignment ? + (GridAffinityAssignment)assign0 : + new GridAffinityAssignment(topVer, assign0.assignment(), assign0.idealAssignment()); + AffinityInfo info = new AffinityInfo( cctx.config().getAffinity(), cctx.config().getAffinityMapper(), - new GridAffinityAssignment(topVer, cctx.affinity().assignment(topVer)), + assign, cctx.cacheObjectContext()); IgniteInternalFuture old = affMap.putIfAbsent(key, new GridFinishedFuture<>(info)); @@ -562,6 +569,20 @@ private ClusterNode primary(AffinityInfo aff, K key) throws IgniteCheckedExc return nodes.iterator().next(); } + /** + * @param aff Affinity function. + * @param nodeFilter Node class. + * @param backups Number of backups. + * @param parts Number of partitions. + * @return Key to find caches with similar affinity. + */ + public Object similaryAffinityKey(AffinityFunction aff, + IgnitePredicate nodeFilter, + int backups, + int parts) { + return new SimilarAffinityKey(aff.getClass(), nodeFilter.getClass(), backups, parts); + } + /** {@inheritDoc} */ @Override public void printMemoryStats() { X.println(">>>"); @@ -960,4 +981,70 @@ private AffinityInfo cache() throws IgniteCheckedException { return aff; } } + + /** + * + */ + private static class SimilarAffinityKey { + /** */ + private final int backups; + + /** */ + private final Class affFuncCls; + + /** */ + private final Class filterCls; + + /** */ + private final int partsCnt; + + /** */ + private final int hash; + + /** + * @param affFuncCls Affinity function class. + * @param filterCls Node filter class. + * @param backups Number of backups. + * @param partsCnt Number of partitions. + */ + SimilarAffinityKey(Class affFuncCls, Class filterCls, int backups, int partsCnt) { + this.backups = backups; + this.affFuncCls = affFuncCls; + this.filterCls = filterCls; + this.partsCnt = partsCnt; + + int hash = backups; + hash = 31 * hash + affFuncCls.hashCode(); + hash = 31 * hash + filterCls.hashCode(); + hash= 31 * hash + partsCnt; + + this.hash = hash; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return hash; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (o == this) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + SimilarAffinityKey key = (SimilarAffinityKey)o; + + return backups == key.backups && + affFuncCls == key.affFuncCls && + filterCls == key.filterCls && + partsCnt == key.partsCnt; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SimilarAffinityKey.class, this); + } + } } 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 c24dd2de7d2ee..abd5292799958 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 @@ -180,10 +180,16 @@ public AffinityJob() { cctx.affinity().affinityReadyFuture(topVer).get(); + AffinityAssignment assign0 = cctx.affinity().assignment(topVer); + + GridAffinityAssignment assign = assign0 instanceof GridAffinityAssignment ? + (GridAffinityAssignment)assign0 : + new GridAffinityAssignment(topVer, assign0.assignment(), assign0.idealAssignment()); + return F.t( affinityMessage(ctx, cctx.config().getAffinity()), affinityMessage(ctx, cctx.config().getAffinityMapper()), - new GridAffinityAssignment(topVer, cctx.affinity().assignment(topVer))); + assign); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java new file mode 100644 index 0000000000000..e502dd58f4faa --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/HistoryAffinityAssignment.java @@ -0,0 +1,169 @@ +/* + * 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.affinity; + +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.UUID; + +/** + * + */ +public class HistoryAffinityAssignment implements AffinityAssignment { + /** */ + private final AffinityTopologyVersion topVer; + + /** */ + private final List> assignment; + + /** */ + private final List> idealAssignment; + + /** */ + private final boolean clientEvtChange; + + /** + * @param assign Assignment. + */ + public HistoryAffinityAssignment(GridAffinityAssignment assign) { + this.topVer = assign.topologyVersion(); + this.assignment = assign.assignment(); + this.idealAssignment = assign.idealAssignment(); + this.clientEvtChange = assign.clientEventChange(); + } + + /** {@inheritDoc} */ + @Override public boolean clientEventChange() { + return clientEvtChange; + } + + /** {@inheritDoc} */ + @Override public List> idealAssignment() { + return idealAssignment; + } + + /** {@inheritDoc} */ + @Override public List> assignment() { + return assignment; + } + + /** {@inheritDoc} */ + @Override public AffinityTopologyVersion topologyVersion() { + return topVer; + } + + /** {@inheritDoc} */ + @Override public List get(int part) { + assert part >= 0 && part < assignment.size() : "Affinity partition is out of range" + + " [part=" + part + ", partitions=" + assignment.size() + ']'; + + return assignment.get(part); + } + + /** {@inheritDoc} */ + @Override public HashSet getIds(int part) { + assert part >= 0 && part < assignment.size() : "Affinity partition is out of range" + + " [part=" + part + ", partitions=" + assignment.size() + ']'; + + List nodes = assignment.get(part); + + HashSet ids = U.newHashSet(nodes.size()); + + for (int i = 0; i < nodes.size(); i++) + ids.add(nodes.get(i).id()); + + return ids; + } + + /** {@inheritDoc} */ + @Override public Set primaryPartitionNodes() { + Set res = new HashSet<>(); + + for (int p = 0; p < assignment.size(); p++) { + List nodes = assignment.get(p); + + if (!F.isEmpty(nodes)) + res.add(nodes.get(0)); + } + + return res; + } + + /** {@inheritDoc} */ + @Override public Set primaryPartitions(UUID nodeId) { + Set res = new HashSet<>(); + + for (int p = 0; p < assignment.size(); p++) { + List nodes = assignment.get(p); + + if (!F.isEmpty(nodes) && nodes.get(0).id().equals(nodeId)) + res.add(p); + } + + return res; + } + + /** {@inheritDoc} */ + @Override public Set backupPartitions(UUID nodeId) { + Set res = new HashSet<>(); + + for (int p = 0; p < assignment.size(); p++) { + List nodes = assignment.get(p); + + for (int i = 1; i < nodes.size(); i++) { + ClusterNode node = nodes.get(i); + + if (node.id().equals(nodeId)) { + res.add(p); + + break; + } + } + } + + return res; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return topVer.hashCode(); + } + + /** {@inheritDoc} */ + @SuppressWarnings("SimplifiableIfStatement") + @Override public boolean equals(Object o) { + if (o == this) + return true; + + if (o == null || !(o instanceof AffinityAssignment)) + return false; + + return topVer.equals(((AffinityAssignment)o).topologyVersion()); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(HistoryAffinityAssignment.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java index 1aedf4ef94b38..2890887cca416 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java @@ -127,7 +127,7 @@ public class CacheAffinitySharedManager extends GridCacheSharedManagerAdap * @param node Event node. * @param topVer Topology version. */ - public void onDiscoveryEvent(int type, ClusterNode node, AffinityTopologyVersion topVer) { + void onDiscoveryEvent(int type, ClusterNode node, AffinityTopologyVersion topVer) { if (type == EVT_NODE_JOINED && node.isLocal()) { // Clean-up in case of client reconnect. registeredCaches.clear(); @@ -153,7 +153,7 @@ public void onDiscoveryEvent(int type, ClusterNode node, AffinityTopologyVersion * @param msg Customer message. * @return {@code True} if minor topology version should be increased. */ - public boolean onCustomEvent(CacheAffinityChangeMessage msg) { + boolean onCustomEvent(CacheAffinityChangeMessage msg) { assert lateAffAssign : msg; if (msg.exchangeId() != null) { @@ -219,7 +219,7 @@ private void onCacheStopped(AffinityTopologyVersion topVer) { * @param top Topology. * @param checkCacheId Cache ID. */ - public void checkRebalanceState(GridDhtPartitionTopology top, Integer checkCacheId) { + void checkRebalanceState(GridDhtPartitionTopology top, Integer checkCacheId) { if (!lateAffAssign) return; @@ -508,6 +508,8 @@ public void onExchangeChangeAffinityMessage(GridDhtPartitionsExchangeFuture exch assert assignment != null; + final Map>> affCache = new HashMap<>(); + forAllCaches(crd, new IgniteInClosureX() { @Override public void applyx(GridAffinityAssignmentCache aff) throws IgniteCheckedException { List> idealAssignment = aff.idealAssignment(); @@ -527,7 +529,7 @@ public void onExchangeChangeAffinityMessage(GridDhtPartitionsExchangeFuture exch else newAssignment = idealAssignment; - aff.initialize(topVer, newAssignment); + aff.initialize(topVer, cachedAssignment(aff, newAssignment, affCache)); } }); } @@ -562,6 +564,8 @@ public void onChangeAffinityMessage(final GridDhtPartitionsExchangeFuture exchFu final Map deploymentIds = msg.cacheDeploymentIds(); + final Map>> affCache = new HashMap<>(); + forAllCaches(crd, new IgniteInClosureX() { @Override public void applyx(GridAffinityAssignmentCache aff) throws IgniteCheckedException { AffinityTopologyVersion affTopVer = aff.lastVersion(); @@ -602,7 +606,7 @@ public void onChangeAffinityMessage(final GridDhtPartitionsExchangeFuture exchFu assignment.set(part, nodes); } - aff.initialize(topVer, assignment); + aff.initialize(topVer, cachedAssignment(aff, assignment, affCache)); } else aff.clientEventTopologyChange(exchFut.discoveryEvent(), topVer); @@ -1206,6 +1210,8 @@ private CacheHolder cache(GridDhtPartitionsExchangeFuture fut, DynamicCacheDescr throws IgniteCheckedException { AffinityTopologyVersion topVer = fut.topologyVersion(); + final Map>> affCache = new HashMap<>(); + if (!crd) { for (GridCacheContext cacheCtx : cctx.cacheContexts()) { if (cacheCtx.isLocal()) @@ -1213,7 +1219,7 @@ private CacheHolder cache(GridDhtPartitionsExchangeFuture fut, DynamicCacheDescr boolean latePrimary = cacheCtx.rebalanceEnabled(); - initAffinityOnNodeJoin(fut, cacheCtx.affinity().affinityCache(), null, latePrimary); + initAffinityOnNodeJoin(fut, cacheCtx.affinity().affinityCache(), null, latePrimary, affCache); } return null; @@ -1227,7 +1233,7 @@ private CacheHolder cache(GridDhtPartitionsExchangeFuture fut, DynamicCacheDescr boolean latePrimary = cache.rebalanceEnabled; - initAffinityOnNodeJoin(fut, cache.affinity(), waitRebalanceInfo, latePrimary); + initAffinityOnNodeJoin(fut, cache.affinity(), waitRebalanceInfo, latePrimary, affCache); } }); @@ -1240,12 +1246,14 @@ private CacheHolder cache(GridDhtPartitionsExchangeFuture fut, DynamicCacheDescr * @param aff Affinity. * @param rebalanceInfo Rebalance information. * @param latePrimary If {@code true} delays primary assignment if it is not owner. + * @param affCache Already calculated assignments (to reduce data stored in history). * @throws IgniteCheckedException If failed. */ private void initAffinityOnNodeJoin(GridDhtPartitionsExchangeFuture fut, GridAffinityAssignmentCache aff, WaitRebalanceInfo rebalanceInfo, - boolean latePrimary) + boolean latePrimary, + Map>> affCache) throws IgniteCheckedException { assert lateAffAssign; @@ -1292,7 +1300,26 @@ private void initAffinityOnNodeJoin(GridDhtPartitionsExchangeFuture fut, if (newAssignment == null) newAssignment = idealAssignment; - aff.initialize(fut.topologyVersion(), newAssignment); + aff.initialize(fut.topologyVersion(), cachedAssignment(aff, newAssignment, affCache)); + } + + /** + * @param aff Assignment cache. + * @param assign Assignment. + * @param affCache Assignments already calculated for other caches. + * @return Assignment. + */ + private List> cachedAssignment(GridAffinityAssignmentCache aff, + List> assign, + Map>> affCache) { + List> assign0 = affCache.get(aff.similarAffinityKey()); + + if (assign0 != null && assign0.equals(assign)) + assign = assign0; + else + affCache.put(aff.similarAffinityKey(), assign); + + return assign; } /** @@ -1367,7 +1394,7 @@ public IgniteInternalFuture>>> initAffinity * @return Affinity assignment. * @throws IgniteCheckedException If failed. */ - public Map>> initAffinityOnNodeLeft0(final GridDhtPartitionsExchangeFuture fut) + private Map>> initAffinityOnNodeLeft0(final GridDhtPartitionsExchangeFuture fut) throws IgniteCheckedException { final AffinityTopologyVersion topVer = fut.topologyVersion(); @@ -1554,7 +1581,9 @@ abstract static class CacheHolder { * @param aff Affinity cache. * @param initAff Existing affinity cache. */ - public CacheHolder(boolean rebalanceEnabled, GridAffinityAssignmentCache aff, @Nullable GridAffinityAssignmentCache initAff) { + CacheHolder(boolean rebalanceEnabled, + GridAffinityAssignmentCache aff, + @Nullable GridAffinityAssignmentCache initAff) { this.aff = aff; if (initAff != null) @@ -1606,7 +1635,7 @@ GridAffinityAssignmentCache affinity() { /** * Created cache is started on coordinator. */ - class CacheHolder1 extends CacheHolder { + private class CacheHolder1 extends CacheHolder { /** */ private final GridCacheContext cctx; @@ -1614,7 +1643,7 @@ class CacheHolder1 extends CacheHolder { * @param cctx Cache context. * @param initAff Current affinity. */ - public CacheHolder1(GridCacheContext cctx, @Nullable GridAffinityAssignmentCache initAff) { + CacheHolder1(GridCacheContext cctx, @Nullable GridAffinityAssignmentCache initAff) { super(cctx.rebalanceEnabled(), cctx.affinity().affinityCache(), initAff); assert !cctx.isLocal() : cctx.name(); @@ -1651,7 +1680,7 @@ public CacheHolder1(GridCacheContext cctx, @Nullable GridAffinityAssignmentCache /** * Created if cache is not started on coordinator. */ - static class CacheHolder2 extends CacheHolder { + private static class CacheHolder2 extends CacheHolder { /** */ private final GridCacheSharedContext cctx; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java index e10e5aa8a3868..4dcff9b8bc2f2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java @@ -61,6 +61,13 @@ public DynamicCacheChangeBatch( return id; } + /** + * @param id Message ID. + */ + public void id(IgniteUuid id) { + this.id = id; + } + /** * @return Collection of change requests. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java index 71ae5c9128dbc..c6e7ee65710a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java @@ -25,8 +25,8 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.affinity.AffinityAssignment; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.affinity.GridAffinityAssignment; import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache; import org.apache.ignite.internal.util.GridLeanSet; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -48,7 +48,7 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter { private static final AffinityTopologyVersion LOC_CACHE_TOP_VER = new AffinityTopologyVersion(1); /** */ - public static final String FAILED_TO_FIND_CACHE_ERR_MSG = "Failed to find cache (cache was not started " + + private static final String FAILED_TO_FIND_CACHE_ERR_MSG = "Failed to find cache (cache was not started " + "yet or cache was already stopped): "; /** Affinity cached function. */ @@ -265,7 +265,7 @@ public List nodes(int part, AffinityTopologyVersion topVer) { * @param topVer Topology version. * @return Affinity assignment. */ - public GridAffinityAssignment assignment(AffinityTopologyVersion topVer) { + public AffinityAssignment assignment(AffinityTopologyVersion topVer) { if (cctx.isLocal()) topVer = LOC_CACHE_TOP_VER; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index 00d2d1633ce6e..503b3348e51b4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -44,7 +44,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.cache.affinity.AffinityFunction; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.Event; import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; @@ -56,6 +58,7 @@ import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology; +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.GridDhtTopologyFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage; @@ -64,6 +67,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsAbstractMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; @@ -71,6 +75,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.GridListSet; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -80,6 +85,7 @@ import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.CI2; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.GPC; import org.apache.ignite.internal.util.typedef.internal.S; @@ -531,8 +537,23 @@ public GridDhtPartitionTopology clientTopology(int cacheId, GridDhtPartitionsExc if (top != null) return top; + Object affKey = null; + + DynamicCacheDescriptor desc = cctx.cache().cacheDescriptor(cacheId); + + if (desc != null) { + CacheConfiguration ccfg = desc.cacheConfiguration(); + + AffinityFunction aff = ccfg.getAffinity(); + + affKey = cctx.kernalContext().affinity().similaryAffinityKey(aff, + ccfg.getNodeFilter(), + ccfg.getBackups(), + aff.partitions()); + } + GridClientPartitionTopology old = clientTops.putIfAbsent(cacheId, - top = new GridClientPartitionTopology(cctx, cacheId, exchFut)); + top = new GridClientPartitionTopology(cctx, cacheId, exchFut, affKey)); return old != null ? old : top; } @@ -761,40 +782,16 @@ private void refreshPartitions() { * @param nodes Nodes. * @return {@code True} if message was sent, {@code false} if node left grid. */ - private boolean sendAllPartitions(Collection nodes) { - GridDhtPartitionsFullMessage m = new GridDhtPartitionsFullMessage(null, null, AffinityTopologyVersion.NONE); - - boolean useOldApi = false; - - for (ClusterNode node : nodes) { - if (node.version().compareTo(GridDhtPartitionMap2.SINCE) < 0) - useOldApi = true; - } - - for (GridCacheContext cacheCtx : cctx.cacheContexts()) { - if (!cacheCtx.isLocal() && cacheCtx.started()) { - GridDhtPartitionFullMap locMap = cacheCtx.topology().partitionMap(true); - - if (useOldApi) { - locMap = new GridDhtPartitionFullMap(locMap.nodeId(), - locMap.nodeOrder(), - locMap.updateSequence(), - locMap); - } - - m.addFullPartitionsMap(cacheCtx.cacheId(), locMap); - } - } - - // It is important that client topologies be added after contexts. - for (GridClientPartitionTopology top : cctx.exchange().clientTopologies()) - m.addFullPartitionsMap(top.cacheId(), top.partitionMap(true)); + private boolean sendAllPartitions(Collection nodes) { + GridDhtPartitionsFullMessage m = createPartitionsFullMessage(nodes, null, null, true); if (log.isDebugEnabled()) log.debug("Sending all partitions [nodeIds=" + U.nodeIds(nodes) + ", msg=" + m + ']'); for (ClusterNode node : nodes) { try { + assert !node.equals(cctx.localNode()); + cctx.io().sendNoRetry(node, m, SYSTEM_POOL); } catch (ClusterTopologyCheckedException ignore) { @@ -811,31 +808,140 @@ private boolean sendAllPartitions(Collection nodes) { } /** - * @param node Node. - * @param id ID. + * @param nodes Target nodes. + * @param exchId Non-null exchange ID if message is created for exchange. + * @param lastVer Last version. + * @param compress {@code True} if it is possible to use compression for message. + * @return Message. */ - private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id) { - GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(id, - cctx.kernalContext().clientNode(), - cctx.versions().last()); + public GridDhtPartitionsFullMessage createPartitionsFullMessage(Collection nodes, + @Nullable GridDhtPartitionExchangeId exchId, + @Nullable GridCacheVersion lastVer, + boolean compress) { + GridDhtPartitionsFullMessage m = new GridDhtPartitionsFullMessage(exchId, + lastVer, + exchId != null ? exchId.topologyVersion() : AffinityTopologyVersion.NONE); + + boolean useOldApi = false; + + if (nodes != null) { + for (ClusterNode node : nodes) { + if (node.version().compareTo(GridDhtPartitionMap2.SINCE) < 0) { + useOldApi = true; + compress = false; + + break; + } + else if (node.version().compareToIgnoreTimestamp(GridDhtPartitionsAbstractMessage.PART_MAP_COMPRESS_SINCE) < 0) + compress = false; + } + } + + m.compress(compress); + + Map> dupData = new HashMap<>(); for (GridCacheContext cacheCtx : cctx.cacheContexts()) { if (!cacheCtx.isLocal()) { - GridDhtPartitionMap2 locMap = cacheCtx.topology().localPartitionMap(); + boolean ready; - if (node.version().compareTo(GridDhtPartitionMap2.SINCE) < 0) - locMap = new GridDhtPartitionMap(locMap.nodeId(), locMap.updateSequence(), locMap.map()); + if (exchId != null) { + AffinityTopologyVersion startTopVer = cacheCtx.startTopologyVersion(); + + ready = startTopVer == null || startTopVer.compareTo(exchId.topologyVersion()) <= 0; + } + else + ready = cacheCtx.started(); + + if (ready) { + GridDhtPartitionFullMap locMap = cacheCtx.topology().partitionMap(true); - m.addLocalPartitionMap(cacheCtx.cacheId(), locMap); + if (useOldApi) { + locMap = new GridDhtPartitionFullMap(locMap.nodeId(), + locMap.nodeOrder(), + locMap.updateSequence(), + locMap); + } + + addFullPartitionsMap(m, + dupData, + compress, + cacheCtx.cacheId(), + locMap, + cacheCtx.affinity().affinityCache().similarAffinityKey()); + + if (exchId != null) + m.addPartitionUpdateCounters(cacheCtx.cacheId(), cacheCtx.topology().updateCounters(true)); + } } } - for (GridClientPartitionTopology top : clientTops.values()) { - GridDhtPartitionMap2 locMap = top.localPartitionMap(); + // It is important that client topologies be added after contexts. + for (GridClientPartitionTopology top : cctx.exchange().clientTopologies()) { + GridDhtPartitionFullMap map = top.partitionMap(true); + + addFullPartitionsMap(m, + dupData, + compress, + top.cacheId(), + map, + top.similarAffinityKey()); + + if (exchId != null) + m.addPartitionUpdateCounters(top.cacheId(), top.updateCounters(true)); + } + + return m; + } - m.addLocalPartitionMap(top.cacheId(), locMap); + /** + * @param m Message. + * @param dupData Duplicated data map. + * @param compress {@code True} if need check for duplicated partition state data. + * @param cacheId Cache ID. + * @param map Map to add. + * @param affKey Cache affinity key. + */ + private void addFullPartitionsMap(GridDhtPartitionsFullMessage m, + Map> dupData, + boolean compress, + Integer cacheId, + GridDhtPartitionFullMap map, + Object affKey) { + Integer dupDataCache = null; + + if (compress && affKey != null && !m.containsCache(cacheId)) { + T2 state0 = dupData.get(affKey); + + if (state0 != null && state0.get2().partitionStateEquals(map)) { + GridDhtPartitionFullMap map0 = new GridDhtPartitionFullMap(map.nodeId(), + map.nodeOrder(), + map.updateSequence()); + + for (Map.Entry e : map.entrySet()) + map0.put(e.getKey(), e.getValue().emptyCopy()); + + map = map0; + + dupDataCache = state0.get1(); + } + else + dupData.put(affKey, new T2<>(cacheId, map)); } + m.addFullPartitionsMap(cacheId, map, dupDataCache); + } + + /** + * @param node Node. + * @param id ID. + */ + private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id) { + GridDhtPartitionsSingleMessage m = createPartitionsSingleMessage(node, + id, + cctx.kernalContext().clientNode(), + false); + if (log.isDebugEnabled()) log.debug("Sending local partitions [nodeId=" + node.id() + ", msg=" + m + ']'); @@ -852,6 +958,98 @@ private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExc } } + /** + * @param targetNode Target node. + * @param exchangeId ID. + * @param clientOnlyExchange Client exchange flag. + * @param sndCounters {@code True} if need send partition update counters. + * @return Message. + */ + public GridDhtPartitionsSingleMessage createPartitionsSingleMessage(ClusterNode targetNode, + @Nullable GridDhtPartitionExchangeId exchangeId, + boolean clientOnlyExchange, + boolean sndCounters) + { + boolean compress = + targetNode.version().compareToIgnoreTimestamp(GridDhtPartitionsSingleMessage.PART_MAP_COMPRESS_SINCE) >= 0; + + GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(exchangeId, + clientOnlyExchange, + cctx.versions().last(), + compress); + + Map>> dupData = new HashMap<>(); + + for (GridCacheContext cacheCtx : cctx.cacheContexts()) { + if (!cacheCtx.isLocal()) { + GridDhtPartitionMap2 locMap = cacheCtx.topology().localPartitionMap(); + + if (targetNode.version().compareTo(GridDhtPartitionMap2.SINCE) < 0) + locMap = new GridDhtPartitionMap(locMap.nodeId(), locMap.updateSequence(), locMap.map()); + + addPartitionMap(m, + dupData, + compress, + cacheCtx.cacheId(), + locMap, + cacheCtx.affinity().affinityCache().similarAffinityKey()); + + if (sndCounters) + m.partitionUpdateCounters(cacheCtx.cacheId(), cacheCtx.topology().updateCounters(true)); + } + } + + for (GridClientPartitionTopology top : clientTops.values()) { + if (m.partitions() != null && m.partitions().containsKey(top.cacheId())) + continue; + + GridDhtPartitionMap2 locMap = top.localPartitionMap(); + + addPartitionMap(m, + dupData, + compress, + top.cacheId(), + locMap, + top.similarAffinityKey()); + + if (sndCounters) + m.partitionUpdateCounters(top.cacheId(), top.updateCounters(true)); + } + + return m; + } + + /** + * @param m Message. + * @param dupData Duplicated data map. + * @param compress {@code True} if need check for duplicated partition state data. + * @param cacheId Cache ID. + * @param map Map to add. + * @param affKey Cache affinity key. + */ + private void addPartitionMap(GridDhtPartitionsSingleMessage m, + Map>> dupData, + boolean compress, + Integer cacheId, + GridDhtPartitionMap2 map, + Object affKey) { + Integer dupDataCache = null; + + if (compress) { + T2> state0 = dupData.get(affKey); + + if (state0 != null && state0.get2().equals(map.map())) { + dupDataCache = state0.get1(); + + map = map.emptyCopy(); + } + else + dupData.put(affKey, new T2<>(cacheId, map.map())); + } + + m.addLocalPartitionMap(cacheId, map, dupDataCache); + } + /** * @param nodeId Cause node ID. * @param topVer Topology version. @@ -869,7 +1067,7 @@ private GridDhtPartitionExchangeId exchangeId(UUID nodeId, AffinityTopologyVersi * @param affChangeMsg Affinity change message. * @return Exchange future. */ - GridDhtPartitionsExchangeFuture exchangeFuture(GridDhtPartitionExchangeId exchId, + private GridDhtPartitionsExchangeFuture exchangeFuture(GridDhtPartitionExchangeId exchId, @Nullable DiscoveryEvent discoEvt, @Nullable Collection reqs, @Nullable CacheAffinityChangeMessage affChangeMsg) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index fd6abbd104872..5e777fd8a39c2 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -1958,8 +1958,6 @@ private GridCacheSharedContext createSharedContext(GridKernalContext kernalCtx, req.template(true); - req.deploymentId(desc.deploymentId()); - reqs.add(req); } @@ -1972,6 +1970,9 @@ private GridCacheSharedContext createSharedContext(GridKernalContext kernalCtx, batch.clientReconnect(reconnect); + // Reset random batch ID so that serialized batches with the same descriptors will be exactly the same. + batch.id(null); + return batch; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java index 58933b75f57be..5efb31710a8a5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java @@ -61,6 +61,9 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology { /** Flag to control amount of output for full map. */ private static final boolean FULL_MAP_DEBUG = false; + /** */ + private static final Long ZERO = 0L; + /** Cache shared context. */ private GridCacheSharedContext cctx; @@ -97,18 +100,24 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology { /** Partition update counters. */ private Map cntrMap = new HashMap<>(); + /** */ + private final Object similarAffKey; + /** * @param cctx Context. * @param cacheId Cache ID. * @param exchFut Exchange ID. + * @param similarAffKey Key to find caches with similar affinity. */ public GridClientPartitionTopology( GridCacheSharedContext cctx, int cacheId, - GridDhtPartitionsExchangeFuture exchFut + GridDhtPartitionsExchangeFuture exchFut, + Object similarAffKey ) { this.cctx = cctx; this.cacheId = cacheId; + this.similarAffKey = similarAffKey; topVer = exchFut.topologyVersion(); @@ -124,6 +133,13 @@ public GridClientPartitionTopology( } } + /** + * @return Key to find caches with similar affinity. + */ + @Nullable public Object similarAffinityKey() { + return similarAffKey; + } + /** * @return Full map string representation. */ @@ -873,11 +889,22 @@ private void removeNode(UUID nodeId) { } /** {@inheritDoc} */ - @Override public Map updateCounters() { + @Override public Map updateCounters(boolean skipZeros) { lock.readLock().lock(); try { - return new HashMap<>(cntrMap); + if (skipZeros) { + Map res = U.newHashMap(cntrMap.size()); + + for (Map.Entry e : cntrMap.entrySet()) { + if (!e.getValue().equals(ZERO)) + res.put(e.getKey(), e.getValue()); + } + + return res; + } + else + return new HashMap<>(cntrMap); } finally { lock.readLock().unlock(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java index 6e9b9071bb001..4ae4e4754b6c7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java @@ -220,9 +220,10 @@ public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId, @Nullable Map cntrMap); /** + * @param skipZeros If {@code true} then filters out zero counters. * @return Partition update counters. */ - public Map updateCounters(); + public Map updateCounters(boolean skipZeros); /** * @param part Partition to own. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java index 50f7f0f33d09b..f3751ac40c381 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java @@ -35,8 +35,8 @@ import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.processors.affinity.AffinityAssignment; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.affinity.GridAffinityAssignment; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheMapEntryFactory; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId; @@ -71,6 +71,9 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology { /** Flag to control amount of output for full map. */ private static final boolean FULL_MAP_DEBUG = false; + /** */ + private static final Long ZERO = 0L; + /** Context. */ private final GridCacheContext cctx; @@ -859,7 +862,7 @@ private GridDhtLocalPartition localPartition(int p, /** {@inheritDoc} */ @Override public List nodes(int p, AffinityTopologyVersion topVer) { - GridAffinityAssignment affAssignment = cctx.affinity().assignment(topVer); + AffinityAssignment affAssignment = cctx.affinity().assignment(topVer); List affNodes = affAssignment.get(p); @@ -1500,11 +1503,26 @@ private void removeNode(UUID nodeId) { } /** {@inheritDoc} */ - @Override public Map updateCounters() { + @Override public Map updateCounters(boolean skipZeros) { lock.readLock().lock(); try { - Map res = new HashMap<>(cntrMap); + Map res; + + if (skipZeros) { + res = U.newHashMap(cntrMap.size()); + + for (Map.Entry e : cntrMap.entrySet()) { + Long cntr = e.getValue(); + + if (ZERO.equals(cntr)) + continue; + + res.put(e.getKey(), cntr); + } + } + else + res = new HashMap<>(cntrMap); for (int i = 0; i < locParts.length; i++) { GridDhtLocalPartition part = locParts[i]; @@ -1513,7 +1531,10 @@ private void removeNode(UUID nodeId) { continue; Long cntr0 = res.get(part.id()); - Long cntr1 = part.updateCounter(); + long cntr1 = part.updateCounter(); + + if (skipZeros && cntr1 == 0L) + continue; if (cntr0 == null || cntr1 > cntr0) res.put(part.id(), cntr1); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java index 498d4921b5818..8f5ad1714225c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java @@ -157,6 +157,24 @@ public long updateSequence() { return updateSeq; } + /** + * @param fullMap Map. + * @return {@code True} if this map and given map contain the same data. + */ + public boolean partitionStateEquals(GridDhtPartitionFullMap fullMap) { + if (size() != fullMap.size()) + return false; + + for (Map.Entry e : entrySet()) { + GridDhtPartitionMap2 m = fullMap.get(e.getKey()); + + if (m == null || !m.map().equals(e.getValue().map())) + return false; + } + + return true; + } + /** * @param updateSeq New update sequence value. * @return Old update sequence value. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap2.java index 15b5a2e6b58c7..ce36a117a16d0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap2.java @@ -61,27 +61,24 @@ public class GridDhtPartitionMap2 implements Comparable, E private volatile int moving; /** - * @param nodeId Node ID. - * @param updateSeq Update sequence number. + * Empty constructor required for {@link Externalizable}. */ - public GridDhtPartitionMap2(UUID nodeId, long updateSeq) { - assert nodeId != null; - assert updateSeq > 0; - - this.nodeId = nodeId; - this.updateSeq = updateSeq; - - map = new HashMap<>(); + public GridDhtPartitionMap2() { + // No-op. } /** * @param nodeId Node ID. * @param updateSeq Update sequence number. + * @param top Topology version. * @param m Map to copy. * @param onlyActive If {@code true}, then only active states will be included. */ - public GridDhtPartitionMap2(UUID nodeId, long updateSeq, AffinityTopologyVersion top, - Map m, boolean onlyActive) { + public GridDhtPartitionMap2(UUID nodeId, + long updateSeq, + AffinityTopologyVersion top, + Map m, + boolean onlyActive) { assert nodeId != null; assert updateSeq > 0; @@ -100,10 +97,33 @@ public GridDhtPartitionMap2(UUID nodeId, long updateSeq, AffinityTopologyVersion } /** - * Empty constructor required for {@link Externalizable}. + * @param nodeId Node ID. + * @param updateSeq Update sequence number. + * @param top Topology version. + * @param map Map. + * @param moving Number of moving partitions. */ - public GridDhtPartitionMap2() { - // No-op. + private GridDhtPartitionMap2(UUID nodeId, + long updateSeq, + AffinityTopologyVersion top, + Map map, + int moving) { + this.nodeId = nodeId; + this.updateSeq = updateSeq; + this.top = top; + this.map = map; + this.moving = moving; + } + + /** + * @return Copy with empty partition state map. + */ + public GridDhtPartitionMap2 emptyCopy() { + return new GridDhtPartitionMap2(nodeId, + updateSeq, + top, + U.newHashMap(0), + 0); } /** @@ -277,9 +297,8 @@ public AffinityTopologyVersion topologyVersion() { long ver = in.readLong(); int minorVer = in.readInt(); - if (ver != 0) { + if (ver != 0) top = new AffinityTopologyVersion(ver, minorVer); - } } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java index 4e714ed1819a2..6e69161cb6ff2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java @@ -22,6 +22,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheMessage; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; import org.jetbrains.annotations.Nullable; @@ -29,7 +30,13 @@ /** * Request for single partition info. */ -abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage { +public abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage { + /** */ + public static final IgniteProductVersion PART_MAP_COMPRESS_SINCE = IgniteProductVersion.fromString("1.6.11"); + + /** */ + protected static final byte COMPRESSED_FLAG_MASK = 1; + /** */ private static final long serialVersionUID = 0L; @@ -39,6 +46,9 @@ abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage { /** Last used cache version. */ private GridCacheVersion lastVer; + /** */ + private byte flags; + /** * Required by {@link Externalizable}. */ @@ -79,6 +89,20 @@ public GridDhtPartitionExchangeId exchangeId() { return lastVer; } + /** + * @return {@code True} if message data is compressed. + */ + protected final boolean compressed() { + return (flags & COMPRESSED_FLAG_MASK) != 0; + } + + /** + * @param compressed {@code True} if message data is compressed. + */ + protected final void compressed(boolean compressed) { + flags = compressed ? (byte)(flags | COMPRESSED_FLAG_MASK) : (byte)(flags & ~COMPRESSED_FLAG_MASK); + } + /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { writer.setBuffer(buf); @@ -101,6 +125,12 @@ public GridDhtPartitionExchangeId exchangeId() { writer.incrementState(); case 4: + if (!writer.writeByte("flags", flags)) + return false; + + writer.incrementState(); + + case 5: if (!writer.writeMessage("lastVer", lastVer)) return false; @@ -131,6 +161,14 @@ public GridDhtPartitionExchangeId exchangeId() { reader.incrementState(); case 4: + flags = reader.readByte("flags"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 5: lastVer = reader.readMessage("lastVer"); if (!reader.isLastRead()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index 80b376844b29b..f391265039d99 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java @@ -54,6 +54,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology; +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.GridDhtTopologyFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; @@ -64,6 +65,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; @@ -544,7 +546,7 @@ private void updateTopologies(boolean crd) throws IgniteCheckedException { exchId.topologyVersion().equals(cacheCtx.startTopologyVersion()); if (updateTop && clientTop != null) - cacheCtx.topology().update(exchId, clientTop.partitionMap(true), clientTop.updateCounters()); + cacheCtx.topology().update(exchId, clientTop.partitionMap(true), clientTop.updateCounters(false)); } top.updateTopologyVersion(exchId, this, updSeq, stopping(cacheCtx.cacheId())); @@ -668,7 +670,7 @@ private void clientOnlyExchange() throws IgniteCheckedException { if (top.cacheId() == cacheCtx.cacheId()) { cacheCtx.topology().update(exchId, top.partitionMap(true), - top.updateCounters()); + top.updateCounters(false)); break; } @@ -678,7 +680,7 @@ private void clientOnlyExchange() throws IgniteCheckedException { } else { if (!centralizedAff) - sendLocalPartitions(crd, exchId); + sendLocalPartitions(crd); initDone(); @@ -928,27 +930,14 @@ public boolean stopping(int cacheId) { /** * @param node Node. - * @param id ID. * @throws IgniteCheckedException If failed. */ - private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id) + private void sendLocalPartitions(ClusterNode node) throws IgniteCheckedException { - GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(id, + GridDhtPartitionsSingleMessage m = cctx.exchange().createPartitionsSingleMessage(node, + exchangeId(), clientOnlyExchange, - cctx.versions().last()); - - for (GridCacheContext cacheCtx : cctx.cacheContexts()) { - if (!cacheCtx.isLocal()) { - GridDhtPartitionMap2 locMap = cacheCtx.topology().localPartitionMap(); - - if (node.version().compareTo(GridDhtPartitionMap2.SINCE) < 0) - locMap = new GridDhtPartitionMap(locMap.nodeId(), locMap.updateSequence(), locMap.map()); - - m.addLocalPartitionMap(cacheCtx.cacheId(), locMap); - - m.partitionUpdateCounters(cacheCtx.cacheId(), cacheCtx.topology().updateCounters()); - } - } + true); if (log.isDebugEnabled()) log.debug("Sending local partitions [nodeId=" + node.id() + ", exchId=" + exchId + ", msg=" + m + ']'); @@ -964,51 +953,16 @@ private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExc /** * @param nodes Target nodes. - * @return Message; + * @param compress {@code True} if it is possible to use compression for message. + * @return Message. */ - private GridDhtPartitionsFullMessage createPartitionsMessage(Collection nodes) { + private GridDhtPartitionsFullMessage createPartitionsMessage(Collection nodes, boolean compress) { GridCacheVersion last = lastVer.get(); - GridDhtPartitionsFullMessage m = new GridDhtPartitionsFullMessage(exchangeId(), + return cctx.exchange().createPartitionsFullMessage(nodes, + exchangeId(), last != null ? last : cctx.versions().last(), - topologyVersion()); - - boolean useOldApi = false; - - if (nodes != null) { - for (ClusterNode node : nodes) { - if (node.version().compareTo(GridDhtPartitionMap2.SINCE) < 0) - useOldApi = true; - } - } - - for (GridCacheContext cacheCtx : cctx.cacheContexts()) { - if (!cacheCtx.isLocal()) { - AffinityTopologyVersion startTopVer = cacheCtx.startTopologyVersion(); - - boolean ready = startTopVer == null || startTopVer.compareTo(topologyVersion()) <= 0; - - if (ready) { - GridDhtPartitionFullMap locMap = cacheCtx.topology().partitionMap(true); - - if (useOldApi) - locMap = new GridDhtPartitionFullMap(locMap.nodeId(), locMap.nodeOrder(), locMap.updateSequence(), locMap); - - m.addFullPartitionsMap(cacheCtx.cacheId(), locMap); - - m.addPartitionUpdateCounters(cacheCtx.cacheId(), cacheCtx.topology().updateCounters()); - } - } - } - - // It is important that client topologies be added after contexts. - for (GridClientPartitionTopology top : cctx.exchange().clientTopologies()) { - m.addFullPartitionsMap(top.cacheId(), top.partitionMap(true)); - - m.addPartitionUpdateCounters(top.cacheId(), top.updateCounters()); - } - - return m; + compress); } /** @@ -1016,7 +970,9 @@ private GridDhtPartitionsFullMessage createPartitionsMessage(Collection nodes) throws IgniteCheckedException { - GridDhtPartitionsFullMessage m = createPartitionsMessage(nodes); + GridDhtPartitionsFullMessage m = createPartitionsMessage(nodes, true); + + assert !nodes.contains(cctx.localNode()); if (log.isDebugEnabled()) log.debug("Sending full partition map [nodeIds=" + F.viewReadOnly(nodes, F.node2id()) + @@ -1030,7 +986,7 @@ private void sendAllPartitions(Collection nodes) throws IgniteCheck */ private void sendPartitions(ClusterNode oldestNode) { try { - sendLocalPartitions(oldestNode, exchId); + sendLocalPartitions(oldestNode); } catch (ClusterTopologyCheckedException ignore) { if (log.isDebugEnabled()) @@ -1234,7 +1190,7 @@ private void onAffinityInitialized(IgniteInternalFuture>> assignmentChange = fut.get(); - GridDhtPartitionsFullMessage m = createPartitionsMessage(null); + GridDhtPartitionsFullMessage m = createPartitionsMessage(null, false); CacheAffinityChangeMessage msg = new CacheAffinityChangeMessage(exchId, m, assignmentChange); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java index a4ff04b4ea282..90d6242867135 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java @@ -22,14 +22,19 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.UUID; + import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.GridDirectMap; import org.apache.ignite.internal.GridDirectTransient; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; import org.jetbrains.annotations.NotNull; @@ -47,6 +52,10 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa @GridDirectTransient private Map parts; + /** */ + @GridDirectMap(keyType = Integer.class, valueType = Integer.class) + private Map dupPartsData; + /** */ private byte[] partsBytes; @@ -61,6 +70,10 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa /** Topology version. */ private AffinityTopologyVersion topVer; + /** */ + @GridDirectTransient + private transient boolean compress; + /** * Required by {@link Externalizable}. */ @@ -83,6 +96,13 @@ public GridDhtPartitionsFullMessage(@Nullable GridDhtPartitionExchangeId id, this.topVer = topVer; } + /** + * @param compress {@code True} if it is possible to use compression for message. + */ + public void compress(boolean compress) { + this.compress = compress; + } + /** * @return Local partitions. */ @@ -90,16 +110,36 @@ public Map partitions() { return parts; } + /** + * @param cacheId Cache ID. + * @return {@code True} if message contains full map for given cache. + */ + public boolean containsCache(int cacheId) { + return parts != null && parts.containsKey(cacheId); + } + /** * @param cacheId Cache ID. * @param fullMap Full partitions map. + * @param dupDataCache Optional ID of cache with the same partition state map. */ - public void addFullPartitionsMap(int cacheId, GridDhtPartitionFullMap fullMap) { + public void addFullPartitionsMap(int cacheId, GridDhtPartitionFullMap fullMap, @Nullable Integer dupDataCache) { if (parts == null) parts = new HashMap<>(); - if (!parts.containsKey(cacheId)) + if (!parts.containsKey(cacheId)) { parts.put(cacheId, fullMap); + + if (dupDataCache != null) { + assert compress; + assert parts.containsKey(dupDataCache); + + if (dupPartsData == null) + dupPartsData = new HashMap<>(); + + dupPartsData.put(cacheId, dupDataCache); + } + } } /** @@ -132,11 +172,38 @@ public Map partitionUpdateCounters(int cacheId) { @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { super.prepareMarshal(ctx); - if (parts != null && partsBytes == null) - partsBytes = U.marshal(ctx, parts); + boolean marshal = (parts != null && partsBytes == null) || (partCntrs != null && partCntrsBytes == null); + + if (marshal) { + byte[] partsBytes0 = null; + byte[] partCntrsBytes0 = null; + + if (parts != null && partsBytes == null) + partsBytes0 = U.marshal(ctx, parts); - if (partCntrs != null && partCntrsBytes == null) - partCntrsBytes = U.marshal(ctx, partCntrs); + if (partCntrs != null && partCntrsBytes == null) + partCntrsBytes0 = U.marshal(ctx, partCntrs); + + if (compress) { + assert !compressed(); + + try { + byte[] partsBytesZip = U.zip(partsBytes0); + byte[] partCntrsBytesZip = U.zip(partCntrsBytes0); + + partsBytes0 = partsBytesZip; + partCntrsBytes0 = partCntrsBytesZip; + + compressed(true); + } + catch (IgniteCheckedException e) { + U.error(ctx.logger(getClass()), "Failed to compress partitions data: " + e, e); + } + } + + partsBytes = partsBytes0; + partCntrsBytes = partCntrsBytes0; + } } /** @@ -157,14 +224,49 @@ public void topologyVersion(AffinityTopologyVersion topVer) { @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { super.finishUnmarshal(ctx, ldr); - if (partsBytes != null && parts == null) - parts = U.unmarshal(ctx, partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + if (partsBytes != null && parts == null) { + if (compressed()) + parts = U.unmarshalZip(ctx.marshaller(), partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + else + parts = U.unmarshal(ctx, partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + + if (dupPartsData != null) { + assert parts != null; + + for (Map.Entry e : dupPartsData.entrySet()) { + GridDhtPartitionFullMap map1 = parts.get(e.getKey()); + GridDhtPartitionFullMap map2 = parts.get(e.getValue()); + + assert map1 != null : e.getKey(); + assert map2 != null : e.getValue(); + assert map1.size() == map2.size(); + + for (Map.Entry e0 : map2.entrySet()) { + GridDhtPartitionMap2 partMap1 = map1.get(e0.getKey()); + + assert partMap1 != null && partMap1.map().isEmpty() : partMap1; + assert !partMap1.hasMovingPartitions() : partMap1; + + GridDhtPartitionMap2 partMap2 = e0.getValue(); + + assert partMap2 != null; + + for (Map.Entry stateEntry : partMap2.entrySet()) + partMap1.put(stateEntry.getKey(), stateEntry.getValue()); + } + } + } + } if (parts == null) parts = new HashMap<>(); - if (partCntrsBytes != null && partCntrs == null) - partCntrs = U.unmarshal(ctx, partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + if (partCntrsBytes != null && partCntrs == null) { + if (compressed()) + partCntrs = U.unmarshalZip(ctx.marshaller(), partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + else + partCntrs = U.unmarshal(ctx, partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + } if (partCntrs == null) partCntrs = new HashMap<>(); @@ -185,19 +287,25 @@ public void topologyVersion(AffinityTopologyVersion topVer) { } switch (writer.state()) { - case 5: + case 6: + if (!writer.writeMap("dupPartsData", dupPartsData, MessageCollectionItemType.INT, MessageCollectionItemType.INT)) + return false; + + writer.incrementState(); + + case 7: if (!writer.writeByteArray("partCntrsBytes", partCntrsBytes)) return false; writer.incrementState(); - case 6: + case 8: if (!writer.writeByteArray("partsBytes", partsBytes)) return false; writer.incrementState(); - case 7: + case 9: if (!writer.writeMessage("topVer", topVer)) return false; @@ -219,7 +327,15 @@ public void topologyVersion(AffinityTopologyVersion topVer) { return false; switch (reader.state()) { - case 5: + case 6: + dupPartsData = reader.readMap("dupPartsData", MessageCollectionItemType.INT, MessageCollectionItemType.INT, false); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 7: partCntrsBytes = reader.readByteArray("partCntrsBytes"); if (!reader.isLastRead()) @@ -227,7 +343,7 @@ public void topologyVersion(AffinityTopologyVersion topVer) { reader.incrementState(); - case 6: + case 8: partsBytes = reader.readByteArray("partsBytes"); if (!reader.isLastRead()) @@ -235,7 +351,7 @@ public void topologyVersion(AffinityTopologyVersion topVer) { reader.incrementState(); - case 7: + case 9: topVer = reader.readMessage("topVer"); if (!reader.isLastRead()) @@ -255,7 +371,7 @@ public void topologyVersion(AffinityTopologyVersion topVer) { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 8; + return 10; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java index e4356b17fbbdc..bf08f0abf8832 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java @@ -23,12 +23,16 @@ import java.util.HashMap; import java.util.Map; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.GridDirectMap; import org.apache.ignite.internal.GridDirectTransient; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; import org.jetbrains.annotations.Nullable; @@ -45,6 +49,10 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes @GridDirectTransient private Map parts; + /** */ + @GridDirectMap(keyType = Integer.class, valueType = Integer.class) + private Map dupPartsData; + /** Serialized partitions. */ private byte[] partsBytes; @@ -59,6 +67,10 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes /** */ private boolean client; + /** */ + @GridDirectTransient + private transient boolean compress; + /** * Required by {@link Externalizable}. */ @@ -70,13 +82,16 @@ public GridDhtPartitionsSingleMessage() { * @param exchId Exchange ID. * @param client Client message flag. * @param lastVer Last version. + * @param compress {@code True} if it is possible to use compression for message. */ public GridDhtPartitionsSingleMessage(GridDhtPartitionExchangeId exchId, boolean client, - @Nullable GridCacheVersion lastVer) { + @Nullable GridCacheVersion lastVer, + boolean compress) { super(exchId, lastVer); this.client = client; + this.compress = compress; } /** @@ -87,16 +102,26 @@ public boolean client() { } /** - * Adds partition map to this message. - * * @param cacheId Cache ID to add local partition for. * @param locMap Local partition map. + * @param dupDataCache Optional ID of cache with the same partition state map. */ - public void addLocalPartitionMap(int cacheId, GridDhtPartitionMap2 locMap) { + public void addLocalPartitionMap(int cacheId, GridDhtPartitionMap2 locMap, @Nullable Integer dupDataCache) { if (parts == null) parts = new HashMap<>(); parts.put(cacheId, locMap); + + if (dupDataCache != null) { + assert compress; + assert F.isEmpty(locMap.map()); + assert parts.containsKey(dupDataCache); + + if (dupPartsData == null) + dupPartsData = new HashMap<>(); + + dupPartsData.put(cacheId, dupDataCache); + } } /** @@ -136,22 +161,77 @@ public Map partitions() { @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { super.prepareMarshal(ctx); - if (partsBytes == null && parts != null) - partsBytes = U.marshal(ctx, parts); + boolean marshal = (parts != null && partsBytes == null) || (partCntrs != null && partCntrsBytes == null); + + if (marshal) { + byte[] partsBytes0 = null; + byte[] partCntrsBytes0 = null; + + if (parts != null && partsBytes == null) + partsBytes0 = U.marshal(ctx, parts); - if (partCntrsBytes == null && partCntrs != null) - partCntrsBytes = U.marshal(ctx, partCntrs); + if (partCntrs != null && partCntrsBytes == null) + partCntrsBytes0 = U.marshal(ctx, partCntrs); + + if (compress) { + assert !compressed(); + + try { + byte[] partsBytesZip = U.zip(partsBytes0); + byte[] partCntrsBytesZip = U.zip(partCntrsBytes0); + + partsBytes0 = partsBytesZip; + partCntrsBytes0 = partCntrsBytesZip; + + compressed(true); + } + catch (IgniteCheckedException e) { + U.error(ctx.logger(getClass()), "Failed to compress partitions data: " + e, e); + } + } + + partsBytes = partsBytes0; + partCntrsBytes = partCntrsBytes0; + } } /** {@inheritDoc} */ @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { super.finishUnmarshal(ctx, ldr); - if (partsBytes != null && parts == null) - parts = U.unmarshal(ctx, partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + if (partsBytes != null && parts == null) { + if (compressed()) + parts = U.unmarshalZip(ctx.marshaller(), partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + else + parts = U.unmarshal(ctx, partsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + } + + if (partCntrsBytes != null && partCntrs == null) { + if (compressed()) + partCntrs = U.unmarshalZip(ctx.marshaller(), partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + else + partCntrs = U.unmarshal(ctx, partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + } + + if (dupPartsData != null) { + assert parts != null; + + for (Map.Entry e : dupPartsData.entrySet()) { + GridDhtPartitionMap2 map1 = parts.get(e.getKey()); - if (partCntrsBytes != null && partCntrs == null) - partCntrs = U.unmarshal(ctx, partCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + assert map1 != null : e.getKey(); + assert F.isEmpty(map1.map()); + assert !map1.hasMovingPartitions(); + + GridDhtPartitionMap2 map2 = parts.get(e.getValue()); + + assert map2 != null : e.getValue(); + assert map2.map() != null; + + for (Map.Entry e0 : map2.map().entrySet()) + map1.put(e0.getKey(), e0.getValue()); + } + } } /** {@inheritDoc} */ @@ -169,19 +249,25 @@ public Map partitions() { } switch (writer.state()) { - case 5: + case 6: if (!writer.writeBoolean("client", client)) return false; writer.incrementState(); - case 6: + case 7: + if (!writer.writeMap("dupPartsData", dupPartsData, MessageCollectionItemType.INT, MessageCollectionItemType.INT)) + return false; + + writer.incrementState(); + + case 8: if (!writer.writeByteArray("partCntrsBytes", partCntrsBytes)) return false; writer.incrementState(); - case 7: + case 9: if (!writer.writeByteArray("partsBytes", partsBytes)) return false; @@ -203,7 +289,7 @@ public Map partitions() { return false; switch (reader.state()) { - case 5: + case 6: client = reader.readBoolean("client"); if (!reader.isLastRead()) @@ -211,7 +297,15 @@ public Map partitions() { reader.incrementState(); - case 6: + case 7: + dupPartsData = reader.readMap("dupPartsData", MessageCollectionItemType.INT, MessageCollectionItemType.INT, false); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 8: partCntrsBytes = reader.readByteArray("partCntrsBytes"); if (!reader.isLastRead()) @@ -219,7 +313,7 @@ public Map partitions() { reader.incrementState(); - case 7: + case 9: partsBytes = reader.readByteArray("partsBytes"); if (!reader.isLastRead()) @@ -239,7 +333,7 @@ public Map partitions() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 8; + return 10; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java index a4106afc6f200..850b6d82a9862 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java @@ -81,11 +81,11 @@ public GridDhtPartitionsSingleRequest() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 5; + return 6; } /** {@inheritDoc} */ @Override public String toString() { return S.toString(GridDhtPartitionsSingleRequest.class, this, super.toString()); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java index 09aec815b8ba2..d6865c1125ac5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java @@ -35,8 +35,8 @@ import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; +import org.apache.ignite.internal.processors.affinity.AffinityAssignment; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.affinity.GridAffinityAssignment; import org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; @@ -600,7 +600,7 @@ private void processAffinityAssignmentRequest(final ClusterNode node, log.debug("Affinity is ready for topology version, will send response [topVer=" + topVer + ", node=" + node + ']'); - GridAffinityAssignment assignment = cctx.affinity().assignment(topVer); + AffinityAssignment assignment = cctx.affinity().assignment(topVer); boolean newAffMode = node.version().compareTo(CacheAffinitySharedManager.LATE_AFF_ASSIGN_SINCE) >= 0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java index 3a559e7598754..9fd9b6d44b89e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java @@ -239,7 +239,7 @@ public GridContinuousProcessor(GridKernalContext ctx) { GridCacheContext cctx = interCache != null ? interCache.context() : null; if (cctx != null && cntrsPerNode != null && !cctx.isLocal() && cctx.affinityNode()) - cntrsPerNode.put(ctx.localNodeId(), cctx.topology().updateCounters()); + cntrsPerNode.put(ctx.localNodeId(), cctx.topology().updateCounters(false)); routine.handler().updateCounters(topVer, cntrsPerNode, cntrs); } @@ -1049,7 +1049,7 @@ private void processStartRequest(ClusterNode node, StartRoutineDiscoveryMessage GridCacheAdapter cache = ctx.cache().internalCache(hnd0.cacheName()); if (cache != null && !cache.isLocal() && cache.context().userCache()) - req.addUpdateCounters(ctx.localNodeId(), cache.context().topology().updateCounters()); + req.addUpdateCounters(ctx.localNodeId(), cache.context().topology().updateCounters(false)); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 7b011ddc15629..e0f4a2d7dc197 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -19,6 +19,8 @@ import java.io.BufferedInputStream; import java.io.BufferedOutputStream; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.DataInput; import java.io.DataOutput; import java.io.Externalizable; @@ -128,6 +130,8 @@ import java.util.regex.Pattern; import java.util.zip.ZipEntry; import java.util.zip.ZipFile; +import java.util.zip.ZipInputStream; +import java.util.zip.ZipOutputStream; import javax.management.DynamicMBean; import javax.management.JMException; import javax.management.MBeanServer; @@ -9692,6 +9696,32 @@ public static T unmarshal(Marshaller marsh, InputStream in, @Nullable ClassL } } + /** + * @param marsh Marshaller. + * @param zipBytes Zip-compressed bytes. + * @param clsLdr Class loader to use. + * @return Unmarshalled object. + * @throws IgniteCheckedException + */ + public static T unmarshalZip(Marshaller marsh, byte[] zipBytes, @Nullable ClassLoader clsLdr) throws IgniteCheckedException { + assert marsh != null; + assert zipBytes != null; + + try { + ZipInputStream in = new ZipInputStream(new ByteArrayInputStream(zipBytes)); + + in.getNextEntry(); + + return marsh.unmarshal(in, clsLdr); + } + catch (IgniteCheckedException e) { + throw e; + } + catch (Exception e) { + throw new IgniteCheckedException(e); + } + } + /** * Unmarshals object from the input stream using given class loader. * This method should not close given input stream. @@ -9907,4 +9937,38 @@ public static void restoreOldIgniteName(@Nullable String oldName, @Nullable Stri if (oldName != curName) LOC_IGNITE_NAME.set(oldName); } + + /** + * @param bytes Byte array to compress. + * @return Compressed bytes. + * @throws IgniteCheckedException If failed. + */ + public static byte[] zip(@Nullable byte[] bytes) throws IgniteCheckedException { + try { + if (bytes == null) + return null; + + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + + try (ZipOutputStream zos = new ZipOutputStream(bos)) { + ZipEntry entry = new ZipEntry(""); + + try { + entry.setSize(bytes.length); + + zos.putNextEntry(entry); + + zos.write(bytes); + } + finally { + zos.closeEntry(); + } + } + + return bos.toByteArray(); + } + catch (Exception e) { + throw new IgniteCheckedException(e); + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index f9291219e66d4..733d204bbbf81 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -1049,7 +1049,7 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { /** {@inheritDoc} */ @Override protected void body() throws InterruptedException { - TcpDiscoveryAbstractMessage msg = null; + TcpDiscoveryAbstractMessage msg; while (!Thread.currentThread().isInterrupted()) { Socket sock; @@ -1063,8 +1063,7 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { continue; } - if (msg == null) - msg = queue.poll(); + msg = queue.poll(); if (msg == null) { mux.wait(); @@ -1121,19 +1120,13 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { } } } - catch (IOException e) { + catch (InterruptedException e) { if (log.isDebugEnabled()) - U.error(log, "Failed to send node left message (will stop anyway) " + - "[sock=" + sock + ", msg=" + msg + ']', e); - - U.closeQuiet(sock); + log.debug("Client socket writer interrupted."); - synchronized (mux) { - if (sock == this.sock) - this.sock = null; // Connection has dead. - } + return; } - catch (IgniteCheckedException e) { + catch (Exception e) { if (spi.getSpiContext().isStopping()) { if (log.isDebugEnabled()) log.debug("Failed to send message, node is stopping [msg=" + msg + ", err=" + e + ']'); @@ -1141,7 +1134,12 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { else U.error(log, "Failed to send message: " + msg, e); - msg = null; + U.closeQuiet(sock); + + synchronized (mux) { + if (sock == this.sock) + this.sock = null; // Connection has dead. + } } } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 0de787dbf8111..881474547740f 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -133,7 +133,7 @@ import org.jsr166.ConcurrentHashMap8; import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2; -import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCOVERY_HISTORY_SIZE; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCOVERY_CLIENT_RECONNECT_HISTORY_SIZE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE; import static org.apache.ignite.IgniteSystemProperties.getInteger; @@ -167,7 +167,7 @@ */ class ServerImpl extends TcpDiscoveryImpl { /** */ - private static final int ENSURED_MSG_HIST_SIZE = getInteger(IGNITE_DISCOVERY_HISTORY_SIZE, 1024); + private static final int ENSURED_MSG_HIST_SIZE = getInteger(IGNITE_DISCOVERY_CLIENT_RECONNECT_HISTORY_SIZE, 512); /** */ private static final IgniteProductVersion CUSTOM_MSG_ALLOW_JOINING_FOR_VERIFIED_SINCE = @@ -1479,7 +1479,7 @@ private void printStatistics() { private void prepareNodeAddedMessage( TcpDiscoveryAbstractMessage msg, UUID destNodeId, - @Nullable Collection msgs, + @Nullable Collection msgs, @Nullable IgniteUuid discardMsgId, @Nullable IgniteUuid discardCustomMsgId ) { @@ -1506,7 +1506,19 @@ private void prepareNodeAddedMessage( } nodeAddedMsg.topology(topToSnd); - nodeAddedMsg.messages(msgs, discardMsgId, discardCustomMsgId); + + Collection msgs0 = null; + + if (msgs != null) { + msgs0 = new ArrayList<>(msgs.size()); + + for (PendingMessage pendingMsg : msgs) { + if (pendingMsg.msg != null) + msgs0.add(pendingMsg.msg); + } + } + + nodeAddedMsg.messages(msgs0, discardMsgId, discardCustomMsgId); Map> hist; @@ -1892,7 +1904,10 @@ void add(TcpDiscoveryAbstractMessage msg) { assert spi.ensured(msg) && msg.verified() : msg; if (msg instanceof TcpDiscoveryNodeAddedMessage) { - TcpDiscoveryNodeAddedMessage addedMsg = (TcpDiscoveryNodeAddedMessage)msg; + TcpDiscoveryNodeAddedMessage addedMsg = + new TcpDiscoveryNodeAddedMessage((TcpDiscoveryNodeAddedMessage)msg); + + msg = addedMsg; TcpDiscoveryNode node = addedMsg.node(); @@ -1910,11 +1925,108 @@ void add(TcpDiscoveryAbstractMessage msg) { addedMsg.clientTopology(top); } + + // Do not need this data for client reconnect. + addedMsg.oldNodesDiscoveryData(null); + } + else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage) { + TcpDiscoveryNodeAddFinishedMessage addFinishMsg = (TcpDiscoveryNodeAddFinishedMessage)msg; + + if (addFinishMsg.clientDiscoData() != null) { + addFinishMsg = new TcpDiscoveryNodeAddFinishedMessage(addFinishMsg); + + msg = addFinishMsg; + + Map> discoData = addFinishMsg.clientDiscoData(); + + Set replaced = null; + + for (TcpDiscoveryAbstractMessage msg0 : msgs) { + if (msg0 instanceof TcpDiscoveryNodeAddFinishedMessage) { + Map> existingDiscoData = + ((TcpDiscoveryNodeAddFinishedMessage)msg0).clientDiscoData(); + + // Check if already stored message contains the same data to do not store copies multiple times. + if (existingDiscoData != null) { + for (Map.Entry> e : discoData.entrySet()) { + UUID nodeId = e.getKey(); + + if (F.contains(replaced, nodeId)) + continue; + + Map existingData = existingDiscoData.get(e.getKey()); + + if (existingData != null && mapsEqual(e.getValue(), existingData)) { + e.setValue(existingData); + + if (replaced == null) + replaced = new HashSet<>(); + + boolean add = replaced.add(nodeId); + + assert add; + + if (replaced.size() == discoData.size()) + break; + } + } + + if (replaced != null && replaced.size() == discoData.size()) + break; + } + } + } + } } + else if (msg instanceof TcpDiscoveryNodeLeftMessage) + clearClientAddFinished(msg.creatorNodeId()); + else if (msg instanceof TcpDiscoveryNodeFailedMessage) + clearClientAddFinished(((TcpDiscoveryNodeFailedMessage)msg).failedNodeId()); msgs.add(msg); } + /** + * @param clientId Client node ID. + */ + private void clearClientAddFinished(UUID clientId) { + for (TcpDiscoveryAbstractMessage msg : msgs) { + if (msg instanceof TcpDiscoveryNodeAddFinishedMessage) { + TcpDiscoveryNodeAddFinishedMessage addFinishMsg = (TcpDiscoveryNodeAddFinishedMessage)msg; + + if (addFinishMsg.clientDiscoData() != null && clientId.equals(addFinishMsg.nodeId())) { + addFinishMsg.clientDiscoData(null); + addFinishMsg.clientNodeAttributes(null); + + break; + } + } + } + } + + /** + * @param m1 Map 1. + * @param m2 Map 2. + * @return {@code True} if maps contain the same data. + */ + private boolean mapsEqual(Map m1, Map m2) { + if (m1 == m2) + return true; + + if (m1.size() == m2.size()) { + for (Map.Entry e : m1.entrySet()) { + byte[] data = m2.get(e.getKey()); + + if (!Arrays.equals(e.getValue(), data)) + return false; + } + + return true; + } + + return false; + } + /** * Gets messages starting from provided ID (exclusive). If such * message is not found, {@code null} is returned (this indicates @@ -2008,6 +2120,37 @@ private TcpDiscoveryAbstractMessage prepare(TcpDiscoveryAbstractMessage msg, UUI } } + /** + * + */ + private static class PendingMessage { + /** */ + TcpDiscoveryAbstractMessage msg; + + /** */ + final boolean customMsg; + + /** */ + final IgniteUuid id; + + /** + * @param msg Message. + */ + PendingMessage(TcpDiscoveryAbstractMessage msg) { + assert msg != null && msg.id() != null : msg; + + this.msg = msg; + + id = msg.id(); + customMsg = msg instanceof TcpDiscoveryCustomEventMessage; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(PendingMessage.class, this); + } + } + /** * Pending messages container. */ @@ -2016,7 +2159,7 @@ private static class PendingMessages implements Iterable msgs = new ArrayDeque<>(MAX * 2); + private final Queue msgs = new ArrayDeque<>(MAX * 2); /** Processed custom message IDs. */ private Set procCustomMsgs = new GridBoundedLinkedHashSet<>(MAX * 2); @@ -2024,7 +2167,7 @@ private static class PendingMessages implements Iterable MAX) { - TcpDiscoveryAbstractMessage polled = msgs.poll(); + PendingMessage polled = msgs.poll(); assert polled != null; - if (polled.id().equals(discardId)) + if (polled.id.equals(discardId)) break; } } @@ -2051,6 +2194,7 @@ void add(TcpDiscoveryAbstractMessage msg) { * * @param msgs Message. * @param discardId Discarded message ID. + * @param customDiscardId Discarded custom event message ID. */ void reset( @Nullable Collection msgs, @@ -2059,8 +2203,10 @@ void reset( ) { this.msgs.clear(); - if (msgs != null) - this.msgs.addAll(msgs); + if (msgs != null) { + for (TcpDiscoveryAbstractMessage msg : msgs) + this.msgs.add(new PendingMessage(msg)); + } this.discardId = discardId; this.customDiscardId = customDiscardId; @@ -2070,12 +2216,52 @@ void reset( * Discards message with provided ID and all before it. * * @param id Discarded message ID. + * @param custom {@code True} if discard for {@link TcpDiscoveryCustomEventMessage}. */ void discard(IgniteUuid id, boolean custom) { if (custom) customDiscardId = id; else discardId = id; + + cleanup(); + } + + /** + * + */ + void cleanup() { + Iterator msgIt = msgs.iterator(); + + boolean skipMsg = discardId != null; + boolean skipCustomMsg = customDiscardId != null; + + while (msgIt.hasNext()) { + PendingMessage msg = msgIt.next(); + + if (msg.customMsg) { + if (skipCustomMsg) { + assert customDiscardId != null; + + if (F.eq(customDiscardId, msg.id)) { + msg.msg = null; + + return; + } + } + } + else { + if (skipMsg) { + assert discardId != null; + + if (F.eq(discardId, msg.id)) { + msg.msg = null; + + return; + } + } + } + } } /** @@ -2098,7 +2284,7 @@ private class SkipIterator implements Iterator { private boolean skipCustomMsg = customDiscardId != null; /** Internal iterator. */ - private Iterator msgIt = msgs.iterator(); + private Iterator msgIt = msgs.iterator(); /** Next message. */ private TcpDiscoveryAbstractMessage next; @@ -2136,13 +2322,13 @@ private void advance() { next = null; while (msgIt.hasNext()) { - TcpDiscoveryAbstractMessage msg0 = msgIt.next(); + PendingMessage msg0 = msgIt.next(); - if (msg0 instanceof TcpDiscoveryCustomEventMessage) { + if (msg0.customMsg) { if (skipCustomMsg) { assert customDiscardId != null; - if (F.eq(customDiscardId, msg0.id())) + if (F.eq(customDiscardId, msg0.id)) skipCustomMsg = false; continue; @@ -2152,14 +2338,17 @@ private void advance() { if (skipMsg) { assert discardId != null; - if (F.eq(discardId, msg0.id())) + if (F.eq(discardId, msg0.id)) skipMsg = false; continue; } } - next = msg0; + if (msg0.msg == null) + continue; + + next = msg0.msg; break; } @@ -2985,9 +3174,9 @@ private boolean hasPendingAddMessage(UUID nodeId) { if (pendingMsgs.msgs.isEmpty()) return false; - for (TcpDiscoveryAbstractMessage pendingMsg : pendingMsgs.msgs) { - if (pendingMsg instanceof TcpDiscoveryNodeAddedMessage) { - TcpDiscoveryNodeAddedMessage addMsg = (TcpDiscoveryNodeAddedMessage)pendingMsg; + for (PendingMessage pendingMsg : pendingMsgs.msgs) { + if (pendingMsg.msg instanceof TcpDiscoveryNodeAddedMessage) { + TcpDiscoveryNodeAddedMessage addMsg = (TcpDiscoveryNodeAddedMessage)pendingMsg.msg; if (addMsg.node().id().equals(nodeId) && addMsg.id().compareTo(pendingMsgs.discardId) > 0) return true; @@ -3901,8 +4090,7 @@ else if (!locNodeId.equals(node.id()) && ring.node(node.id()) != null) { Map data = msg.newNodeDiscoveryData(); if (data != null) - spi.onExchange(node.id(), node.id(), data, - U.resolveClassLoader(spi.ignite().configuration())); + spi.onExchange(node.id(), node.id(), data, U.resolveClassLoader(spi.ignite().configuration())); msg.addDiscoveryData(locNodeId, spi.collectExchangeData(node.id())); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java index 1b99a56e7efbe..80f4565099325 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java @@ -58,6 +58,17 @@ public TcpDiscoveryNodeAddFinishedMessage(UUID creatorNodeId, UUID nodeId) { this.nodeId = nodeId; } + /** + * @param msg Message. + */ + public TcpDiscoveryNodeAddFinishedMessage(TcpDiscoveryNodeAddFinishedMessage msg) { + super(msg); + + nodeId = msg.nodeId; + clientDiscoData = msg.clientDiscoData; + clientNodeAttrs = msg.clientNodeAttrs; + } + /** * Gets ID of the node added. * diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java index 6f8e14e90e0b9..bd52c045f5207 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java @@ -17,7 +17,9 @@ package org.apache.ignite.spi.discovery.tcp.messages; +import java.util.Arrays; import java.util.Collection; +import java.util.Iterator; import java.util.LinkedHashMap; import java.util.Map; import java.util.UUID; @@ -233,6 +235,13 @@ public Map> oldNodesDiscoveryData() { return oldNodesDiscoData; } + /** + * @param oldNodesDiscoData Discovery data from old nodes. + */ + public void oldNodesDiscoveryData(Map> oldNodesDiscoData) { + this.oldNodesDiscoData = oldNodesDiscoData; + } + /** * @param nodeId Node ID. * @param discoData Discovery data to add. @@ -240,8 +249,28 @@ public Map> oldNodesDiscoveryData() { public void addDiscoveryData(UUID nodeId, Map discoData) { // Old nodes disco data may be null if message // makes more than 1 pass due to stopping of the nodes in topology. - if (oldNodesDiscoData != null) - oldNodesDiscoData.put(nodeId, discoData); + if (oldNodesDiscoData != null) { + for (Map.Entry> existingDataEntry : oldNodesDiscoData.entrySet()) { + Map existingData = existingDataEntry.getValue(); + + Iterator> it = discoData.entrySet().iterator(); + + while (it.hasNext()) { + Map.Entry discoDataEntry = it.next(); + + byte[] curData = existingData.get(discoDataEntry.getKey()); + + if (Arrays.equals(curData, discoDataEntry.getValue())) + it.remove(); + } + + if (discoData.isEmpty()) + break; + } + + if (!discoData.isEmpty()) + oldNodesDiscoData.put(nodeId, discoData); + } } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheExchangeMessageDuplicatedStateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheExchangeMessageDuplicatedStateTest.java new file mode 100644 index 0000000000000..ed186ac68bc3b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheExchangeMessageDuplicatedStateTest.java @@ -0,0 +1,393 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.ignite.Ignite; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.affinity.fair.FairAffinityFunction; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.TestRecordingCommunicationSpi; +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.GridDhtPartitionsFullMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * + */ +public class CacheExchangeMessageDuplicatedStateTest extends GridCommonAbstractTest { + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final String AFF1_CACHE1 = "a1c1"; + + /** */ + private static final String AFF1_CACHE2 = "a1c2"; + + /** */ + private static final String AFF2_CACHE1 = "a2c1"; + + /** */ + private static final String AFF2_CACHE2 = "a2c2"; + + /** */ + private static final String AFF3_CACHE1 = "a3c1"; + + /** */ + private static final String AFF4_FILTER_CACHE1 = "a4c1"; + + /** */ + private static final String AFF4_FILTER_CACHE2 = "a4c2"; + + /** */ + private static final String AFF5_FILTER_CACHE1 = "a5c1"; + + /** */ + private boolean client; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + + cfg.setClientMode(client); + + TestRecordingCommunicationSpi commSpi = new TestRecordingCommunicationSpi(); + + commSpi.record(GridDhtPartitionsSingleMessage.class, GridDhtPartitionsFullMessage.class); + + cfg.setCommunicationSpi(commSpi); + + List ccfgs = new ArrayList<>(); + + { + CacheConfiguration ccfg = new CacheConfiguration(); + ccfg.setName(AFF1_CACHE1); + ccfg.setAffinity(new RendezvousAffinityFunction()); + ccfgs.add(ccfg); + } + { + CacheConfiguration ccfg = new CacheConfiguration(); + ccfg.setName(AFF1_CACHE2); + ccfg.setAffinity(new RendezvousAffinityFunction()); + ccfgs.add(ccfg); + } + { + CacheConfiguration ccfg = new CacheConfiguration(); + ccfg.setName(AFF2_CACHE1); + ccfg.setAffinity(new FairAffinityFunction()); + ccfgs.add(ccfg); + } + { + CacheConfiguration ccfg = new CacheConfiguration(); + ccfg.setName(AFF2_CACHE2); + ccfg.setAffinity(new FairAffinityFunction()); + ccfgs.add(ccfg); + } + { + CacheConfiguration ccfg = new CacheConfiguration(); + ccfg.setName(AFF3_CACHE1); + ccfg.setBackups(3); + + RendezvousAffinityFunction aff = new RendezvousAffinityFunction(false, 64); + ccfg.setAffinity(aff); + + ccfgs.add(ccfg); + } + { + CacheConfiguration ccfg = new CacheConfiguration(); + ccfg.setName(AFF4_FILTER_CACHE1); + ccfg.setNodeFilter(new TestNodeFilter()); + ccfg.setAffinity(new RendezvousAffinityFunction()); + ccfgs.add(ccfg); + } + { + CacheConfiguration ccfg = new CacheConfiguration(); + ccfg.setName(AFF4_FILTER_CACHE2); + ccfg.setNodeFilter(new TestNodeFilter()); + ccfg.setAffinity(new RendezvousAffinityFunction()); + ccfgs.add(ccfg); + } + { + CacheConfiguration ccfg = new CacheConfiguration(); + ccfg.setName(AFF5_FILTER_CACHE1); + ccfg.setNodeFilter(new TestNodeFilter()); + ccfg.setAffinity(new FairAffinityFunction()); + ccfgs.add(ccfg); + } + + cfg.setCacheConfiguration(ccfgs.toArray(new CacheConfiguration[ccfgs.size()])); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(0); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + + /** + * @throws Exception If failed. + */ + public void testExchangeMessages() throws Exception { + ignite(0); + + startGrid(1); + + awaitPartitionMapExchange(); + + checkMessages(0, true); + + startGrid(2); + + awaitPartitionMapExchange(); + + checkMessages(0, true); + + client = true; + + startGrid(3); + + awaitPartitionMapExchange(); + + checkMessages(0, false); + + stopGrid(0); + + awaitPartitionMapExchange(); + + checkMessages(1, true); + } + + /** + * @param crdIdx Coordinator node index. + * @param checkSingle {@code True} if need check single messages. + */ + private void checkMessages(int crdIdx, boolean checkSingle) { + checkFullMessages(crdIdx); + + if (checkSingle) + checkSingleMessages(crdIdx); + } + + /** + * @param crdIdx Coordinator node index. + */ + private void checkFullMessages(int crdIdx) { + TestRecordingCommunicationSpi commSpi0 = + (TestRecordingCommunicationSpi)ignite(crdIdx).configuration().getCommunicationSpi(); + + List msgs = commSpi0.recordedMessages(false); + + assertTrue(msgs.size() > 0); + + for (Object msg : msgs) { + assertTrue("Unexpected messages: " + msg, msg instanceof GridDhtPartitionsFullMessage); + + checkFullMessage((GridDhtPartitionsFullMessage)msg); + } + } + + /** + * @param crdIdx Coordinator node index. + */ + private void checkSingleMessages(int crdIdx) { + int cnt = 0; + + for (Ignite ignite : Ignition.allGrids()) { + if (getTestGridName(crdIdx).equals(ignite.name()) || ignite.configuration().isClientMode()) + continue; + + TestRecordingCommunicationSpi commSpi0 = + (TestRecordingCommunicationSpi)ignite.configuration().getCommunicationSpi(); + + List msgs = commSpi0.recordedMessages(false); + + assertTrue(msgs.size() > 0); + + for (Object msg : msgs) { + assertTrue("Unexpected messages: " + msg, msg instanceof GridDhtPartitionsSingleMessage); + + checkSingleMessage((GridDhtPartitionsSingleMessage)msg); + } + + cnt++; + } + + assertTrue(cnt > 0); + } + + /** + * @param msg Message. + */ + private void checkFullMessage(GridDhtPartitionsFullMessage msg) { + Map dupPartsData = GridTestUtils.getFieldValue(msg, "dupPartsData"); + + assertNotNull(dupPartsData); + + checkFullMessage(AFF1_CACHE1, AFF1_CACHE2, dupPartsData, msg); + checkFullMessage(AFF2_CACHE1, AFF2_CACHE2, dupPartsData, msg); + checkFullMessage(AFF4_FILTER_CACHE1, AFF4_FILTER_CACHE2, dupPartsData, msg); + + assertFalse(dupPartsData.containsKey(CU.cacheId(AFF3_CACHE1))); + assertFalse(dupPartsData.containsKey(CU.cacheId(AFF5_FILTER_CACHE1))); + + Map> partCntrs = GridTestUtils.getFieldValue(msg, "partCntrs"); + + if (partCntrs != null) { + for (Map cntrs : partCntrs.values()) + assertTrue(cntrs.isEmpty()); + } + } + + /** + * @param msg Message. + */ + private void checkSingleMessage(GridDhtPartitionsSingleMessage msg) { + Map dupPartsData = GridTestUtils.getFieldValue(msg, "dupPartsData"); + + assertNotNull(dupPartsData); + + checkSingleMessage(AFF1_CACHE1, AFF1_CACHE2, dupPartsData, msg); + checkSingleMessage(AFF2_CACHE1, AFF2_CACHE2, dupPartsData, msg); + checkSingleMessage(AFF4_FILTER_CACHE1, AFF4_FILTER_CACHE2, dupPartsData, msg); + + assertFalse(dupPartsData.containsKey(CU.cacheId(AFF3_CACHE1))); + assertFalse(dupPartsData.containsKey(CU.cacheId(AFF5_FILTER_CACHE1))); + + Map> partCntrs = GridTestUtils.getFieldValue(msg, "partCntrs"); + + if (partCntrs != null) { + for (Map cntrs : partCntrs.values()) + assertTrue(cntrs.isEmpty()); + } + } + + /** + * @param cache1 Cache 1. + * @param cache2 Cache 2. + * @param dupPartsData Duplicated data map. + * @param msg Message. + */ + private void checkFullMessage(String cache1, + String cache2, + Map dupPartsData, + GridDhtPartitionsFullMessage msg) + { + Integer cacheId; + Integer dupCacheId; + + if (dupPartsData.containsKey(CU.cacheId(cache1))) { + cacheId = CU.cacheId(cache1); + dupCacheId = CU.cacheId(cache2); + } + else { + cacheId = CU.cacheId(cache2); + dupCacheId = CU.cacheId(cache1); + } + + assertTrue(dupPartsData.containsKey(cacheId)); + assertEquals(dupCacheId, dupPartsData.get(cacheId)); + assertFalse(dupPartsData.containsKey(dupCacheId)); + + Map parts = msg.partitions(); + + GridDhtPartitionFullMap emptyFullMap = parts.get(cacheId); + + for (GridDhtPartitionMap2 map : emptyFullMap.values()) + assertEquals(0, map.map().size()); + + GridDhtPartitionFullMap fullMap = parts.get(dupCacheId); + + for (GridDhtPartitionMap2 map : fullMap.values()) + assertFalse(map.map().isEmpty()); + } + + /** + * @param cache1 Cache 1. + * @param cache2 Cache 2. + * @param dupPartsData Duplicated data map. + * @param msg Message. + */ + private void checkSingleMessage(String cache1, + String cache2, + Map dupPartsData, + GridDhtPartitionsSingleMessage msg) + { + Integer cacheId; + Integer dupCacheId; + + if (dupPartsData.containsKey(CU.cacheId(cache1))) { + cacheId = CU.cacheId(cache1); + dupCacheId = CU.cacheId(cache2); + } + else { + cacheId = CU.cacheId(cache2); + dupCacheId = CU.cacheId(cache1); + } + + assertTrue(dupPartsData.containsKey(cacheId)); + assertEquals(dupCacheId, dupPartsData.get(cacheId)); + assertFalse(dupPartsData.containsKey(dupCacheId)); + + Map parts = msg.partitions(); + + GridDhtPartitionMap2 emptyMap = parts.get(cacheId); + + assertEquals(0, emptyMap.map().size()); + + GridDhtPartitionMap2 map = parts.get(dupCacheId); + + assertFalse(map.map().isEmpty()); + } + + /** + * + */ + private static class TestNodeFilter implements IgnitePredicate { + /** {@inheritDoc} */ + @Override public boolean apply(ClusterNode node) { + // Do not start cache on coordinator. + return node.order() > 1; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java index 5dc059b8bd964..6c577c63ab693 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java @@ -630,7 +630,7 @@ public void testNonLocalPartitionSize() throws Exception { if (cacheMode() == LOCAL) return; - awaitPartitionMapExchange(true, true); + awaitPartitionMapExchange(true, true, null); checkEmpty(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java index 71d118281613e..3b0c2fac32947 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheGetRestartTest.java @@ -33,6 +33,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; @@ -72,6 +73,8 @@ public class IgniteCacheGetRestartTest extends GridCommonAbstractTest { ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1); + Boolean clientMode = client.get(); if (clientMode != null) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.java index 2c47a1c053140..7b57d5fd2a1cd 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRabalancingDelayedPartitionMapExchangeSelfTest.java @@ -115,21 +115,21 @@ public void test() throws Exception { startGrid(2); startGrid(3); - awaitPartitionMapExchange(true, true); + awaitPartitionMapExchange(true, true, null); for (int i = 0; i < 2; i++) { stopGrid(3); - awaitPartitionMapExchange(true, true); + awaitPartitionMapExchange(true, true, null); startGrid(3); - awaitPartitionMapExchange(true, true); + awaitPartitionMapExchange(true, true, null); } startGrid(4); - awaitPartitionMapExchange(true, true); + awaitPartitionMapExchange(true, true, null); assert rs.isEmpty(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java index 5716d5937ae6c..de38952da7fd9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java @@ -240,7 +240,7 @@ public void testSimpleRebalancing() throws Exception { waitForRebalancing(0, new AffinityTopologyVersion(2, waitMinorVer)); waitForRebalancing(1, new AffinityTopologyVersion(2, waitMinorVer)); - awaitPartitionMapExchange(true, true); + awaitPartitionMapExchange(true, true, null); checkPartitionMapExchangeFinished(); @@ -250,7 +250,7 @@ public void testSimpleRebalancing() throws Exception { waitForRebalancing(1, 3); - awaitPartitionMapExchange(true, true); + awaitPartitionMapExchange(true, true, null); checkPartitionMapExchangeFinished(); @@ -261,7 +261,7 @@ public void testSimpleRebalancing() throws Exception { waitForRebalancing(1, new AffinityTopologyVersion(4, waitMinorVer)); waitForRebalancing(2, new AffinityTopologyVersion(4, waitMinorVer)); - awaitPartitionMapExchange(true, true); + awaitPartitionMapExchange(true, true, null); checkPartitionMapExchangeFinished(); @@ -271,7 +271,7 @@ public void testSimpleRebalancing() throws Exception { waitForRebalancing(1, 5); - awaitPartitionMapExchange(true, true); + awaitPartitionMapExchange(true, true, null); checkPartitionMapExchangeFinished(); @@ -339,7 +339,7 @@ public void testLoadRebalancing() throws Exception { concurrentStartFinished = true; - awaitPartitionMapExchange(true, true); + awaitPartitionMapExchange(true, true, null); checkSupplyContextMapIsEmpty(); @@ -607,7 +607,7 @@ public void testComplexRebalancing() throws Exception { waitForRebalancing(3, 5, 1); waitForRebalancing(4, 5, 1); - awaitPartitionMapExchange(true, true); + awaitPartitionMapExchange(true, true, null); checkSupplyContextMapIsEmpty(); @@ -631,7 +631,7 @@ public void testComplexRebalancing() throws Exception { waitForRebalancing(3, 6); waitForRebalancing(4, 6); - awaitPartitionMapExchange(true, true); + awaitPartitionMapExchange(true, true, null); checkSupplyContextMapIsEmpty(); @@ -641,7 +641,7 @@ public void testComplexRebalancing() throws Exception { waitForRebalancing(3, 7); waitForRebalancing(4, 7); - awaitPartitionMapExchange(true, true); + awaitPartitionMapExchange(true, true, null); checkSupplyContextMapIsEmpty(); @@ -650,7 +650,7 @@ public void testComplexRebalancing() throws Exception { waitForRebalancing(3, 8); waitForRebalancing(4, 8); - awaitPartitionMapExchange(true, true); + awaitPartitionMapExchange(true, true, null); checkPartitionMapExchangeFinished(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java index 87d02a50bd097..cde6b8dd71479 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java @@ -41,9 +41,6 @@ public class GridCacheSyncReplicatedPreloadSelfTest extends GridCommonAbstractTe /** */ private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); - /** */ - private static final boolean DISCO_DEBUG_MODE = false; - /** * Constructs test. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheSyncRebalanceModeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheSyncRebalanceModeSelfTest.java index 9b0637e956f86..f3942d54f4b5f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheSyncRebalanceModeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheSyncRebalanceModeSelfTest.java @@ -34,7 +34,9 @@ public class IgniteCacheSyncRebalanceModeSelfTest extends GridCommonAbstractTest { /** Entry count. */ public static final int CNT = 100_000; - public static final String STATIC_CACHE_NAME = "static"; + + /** */ + private static final String STATIC_CACHE_NAME = "static"; /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java index 1b7fe2babdc12..d2cb710edbeb5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java @@ -537,7 +537,7 @@ private boolean checkPartCounter(int nodes, int killedNodeIdx, Map aff = grid(i).affinity(null); - Map act = grid(i).cachex(null).context().topology().updateCounters(); + Map act = grid(i).cachex(null).context().topology().updateCounters(false); for (Map.Entry e : updCntrs.entrySet()) { if (aff.mapPartitionToPrimaryAndBackups(e.getKey()).contains(grid(i).localNode())) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/IgniteNoCustomEventsOnNodeStart.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/IgniteNoCustomEventsOnNodeStart.java index 5ecc27a6e78d4..1259fafec3f85 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/IgniteNoCustomEventsOnNodeStart.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/IgniteNoCustomEventsOnNodeStart.java @@ -68,6 +68,13 @@ public void testNoCustomEventsOnStart() throws Exception { assertFalse(failed); } + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + /** * */ diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java index 1ce98a538657e..043208c955b44 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java @@ -49,6 +49,7 @@ import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.Event; import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.GridComponent; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteKernal; @@ -114,6 +115,12 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest { /** */ private GridStringLogger strLog; + /** */ + private CacheConfiguration[] ccfgs; + + /** */ + private boolean client; + /** * @throws Exception If fails. */ @@ -152,7 +159,10 @@ public TcpDiscoverySelfTest() throws Exception { cfg.setDiscoverySpi(spi); - cfg.setCacheConfiguration(); + if (ccfgs != null) + cfg.setCacheConfiguration(ccfgs); + else + cfg.setCacheConfiguration(); cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED); @@ -194,9 +204,8 @@ else if (gridName.contains("MulticastIpFinder")) { } else if (gridName.contains("testPingInterruptedOnNodeFailedPingingNode")) cfg.setFailureDetectionTimeout(30_000); - else if (gridName.contains("testNoRingMessageWorkerAbnormalFailureNormalNode")) { + else if (gridName.contains("testNoRingMessageWorkerAbnormalFailureNormalNode")) cfg.setFailureDetectionTimeout(3_000); - } else if (gridName.contains("testNoRingMessageWorkerAbnormalFailureSegmentedNode")) { cfg.setFailureDetectionTimeout(6_000); @@ -205,6 +214,8 @@ else if (gridName.contains("testNoRingMessageWorkerAbnormalFailureSegmentedNode" else if (gridName.contains("testNodeShutdownOnRingMessageWorkerFailureFailedNode")) cfg.setGridLogger(strLog = new GridStringLogger()); + cfg.setClientMode(client); + return cfg; } @@ -1960,6 +1971,63 @@ public void testNoExtraNodeFailedMessage() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testDuplicatedDiscoveryDataRemoved() throws Exception { + try { + TestDiscoveryDataDuplicateSpi.checkNodeAdded = false; + TestDiscoveryDataDuplicateSpi.checkClientNodeAddFinished = false; + TestDiscoveryDataDuplicateSpi.fail = false; + + ccfgs = new CacheConfiguration[5]; + + for (int i = 0; i < ccfgs.length; i++) { + CacheConfiguration ccfg = new CacheConfiguration(); + + ccfg.setName(i == 0 ? null : ("static-cache-" + i)); + + ccfgs[i] = ccfg; + } + + TestDiscoveryDataDuplicateSpi spi = new TestDiscoveryDataDuplicateSpi(); + + nodeSpi.set(spi); + + startGrid(0); + + for (int i = 0; i < 5; i++) { + nodeSpi.set(new TestDiscoveryDataDuplicateSpi()); + + startGrid(i + 1); + } + + client = true; + + Ignite clientNode = startGrid(6); + + assertTrue(clientNode.configuration().isClientMode()); + + CacheConfiguration ccfg = new CacheConfiguration(); + ccfg.setName("c1"); + + clientNode.createCache(ccfg); + + client = false; + + nodeSpi.set(new TestDiscoveryDataDuplicateSpi()); + + startGrid(7); + + assertTrue(TestDiscoveryDataDuplicateSpi.checkNodeAdded); + assertTrue(TestDiscoveryDataDuplicateSpi.checkClientNodeAddFinished); + assertFalse(TestDiscoveryDataDuplicateSpi.fail); + } + finally { + stopAllGrids(); + } + } + /** * @param nodeName Node name. * @throws Exception If failed. @@ -2015,6 +2083,66 @@ static class DummyPredicate implements IgniteBiPredicate { } } + /** + * + */ + private static class TestDiscoveryDataDuplicateSpi extends TcpDiscoverySpi { + /** */ + static volatile boolean fail; + + /** */ + static volatile boolean checkNodeAdded; + + /** */ + static volatile boolean checkClientNodeAddFinished; + + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, OutputStream out, + TcpDiscoveryAbstractMessage msg, + long timeout) throws IOException, IgniteCheckedException { + if (msg instanceof TcpDiscoveryNodeAddedMessage) { + Map> discoData = ((TcpDiscoveryNodeAddedMessage)msg).oldNodesDiscoveryData(); + + checkDiscoData(discoData, msg); + } + else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage) { + Map> discoData = ((TcpDiscoveryNodeAddFinishedMessage)msg).clientDiscoData(); + + checkDiscoData(discoData, msg); + } + + super.writeToSocket(sock, out, msg, timeout); + } + + /** + * @param discoData Discovery data. + * @param msg Message. + */ + private void checkDiscoData(Map> discoData, TcpDiscoveryAbstractMessage msg) { + if (discoData != null && discoData.size() > 1) { + int cnt = 0; + + for (Map.Entry> e : discoData.entrySet()) { + Map map = e.getValue(); + + if (map.containsKey(GridComponent.DiscoveryDataExchangeType.CACHE_PROC.ordinal())) + cnt++; + } + + if (cnt > 1) { + fail = true; + + log.error("Expect cache data only from one node, but actually: " + cnt); + } + + if (msg instanceof TcpDiscoveryNodeAddedMessage) + checkNodeAdded = true; + else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage) + checkClientNodeAddFinished = true; + } + } + } + /** * diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java index 87509a4e7d412..22fa36d53d4c2 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java @@ -428,21 +428,42 @@ protected final Ignite startGridsMultiThreaded(int cnt, boolean awaitPartMapExch */ @SuppressWarnings("BusyWait") protected void awaitPartitionMapExchange() throws InterruptedException { - awaitPartitionMapExchange(false, false); + awaitPartitionMapExchange(false, false, null); } /** * @param waitEvicts If {@code true} will wait for evictions finished. * @param waitNode2PartUpdate If {@code true} will wait for nodes node2part info update finished. + * @param nodes Optional nodes. * @throws InterruptedException If interrupted. */ @SuppressWarnings("BusyWait") - protected void awaitPartitionMapExchange(boolean waitEvicts, boolean waitNode2PartUpdate) throws InterruptedException { + protected void awaitPartitionMapExchange(boolean waitEvicts, + boolean waitNode2PartUpdate, + @Nullable Collection nodes) + throws InterruptedException { long timeout = 30_000; + long startTime = -1; + + Set names = new HashSet<>(); + for (Ignite g : G.allGrids()) { + if (nodes != null && !nodes.contains(g.cluster().localNode())) + continue; + IgniteKernal g0 = (IgniteKernal)g; + names.add(g0.configuration().getGridName()); + + if (startTime != -1) { + if (startTime != g0.context().discovery().gridStartTime()) + fail("Found nodes from different clusters, probable some test does not stop nodes " + + "[allNodes=" + names + ']'); + } + else + startTime = g0.context().discovery().gridStartTime(); + for (IgniteCacheProxy c : g0.context().cache().jcaches()) { CacheConfiguration cfg = c.context().config(); diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java index 06639037a625e..6edfd094f7df0 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java @@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.cache.CacheDhtLocalPartitionAfterRemoveSelfTest; import org.apache.ignite.internal.processors.cache.CacheEnumOperationsSingleNodeTest; import org.apache.ignite.internal.processors.cache.CacheEnumOperationsTest; +import org.apache.ignite.internal.processors.cache.CacheExchangeMessageDuplicatedStateTest; import org.apache.ignite.internal.processors.cache.CrossCacheTxRandomOperationsTest; import org.apache.ignite.internal.processors.cache.GridCacheAtomicMessageCountSelfTest; import org.apache.ignite.internal.processors.cache.GridCacheFinishPartitionsSelfTest; @@ -264,6 +265,8 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(IgniteNoCustomEventsOnNodeStart.class)); + suite.addTest(new TestSuite(CacheExchangeMessageDuplicatedStateTest.class)); + return suite; } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java index f21a2791fc819..c24fed4a4bc5e 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java @@ -87,6 +87,13 @@ public class IgniteCacheOffheapEvictQueryTest extends GridCommonAbstractTest { return cfg; } + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + /** * @throws Exception If failed. */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java index 5dbb12cba6e57..742da7cc81861 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java @@ -159,6 +159,13 @@ public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCom return c; } + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + /** * */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java index 0f60db25834fd..072a081a0ee2c 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java @@ -95,6 +95,13 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr g.cache(null).removeAll(); } + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + /** */ public void testRemoteQueryExecutionTimeout() throws Exception { testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 500, TimeUnit.MILLISECONDS, true); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java index d559d2edb9a38..070acb198f130 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java @@ -87,13 +87,22 @@ public class GridQueryParsingTest extends GridCommonAbstractTest { return c; } - /** */ + /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); ignite = startGrid(); } + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + ignite = null; + + super.afterTestsStopped(); + } + /** * @throws Exception If failed. */ diff --git a/modules/spring/src/test/config/incorrect-store-cache.xml b/modules/spring/src/test/config/incorrect-store-cache.xml index 2ccda21a7f78b..b08f84e530812 100644 --- a/modules/spring/src/test/config/incorrect-store-cache.xml +++ b/modules/spring/src/test/config/incorrect-store-cache.xml @@ -23,6 +23,8 @@ http://www.springframework.org/schema/beans/spring-beans.xsd"> + + diff --git a/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml b/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml index d62e76b9edc21..dfaf828fa8d22 100644 --- a/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml +++ b/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml @@ -25,7 +25,10 @@ http://www.springframework.org/schema/util http://www.springframework.org/schema/util/spring-util.xsd"> + + + diff --git a/modules/spring/src/test/config/jdbc-pojo-store-obj.xml b/modules/spring/src/test/config/jdbc-pojo-store-obj.xml index 83e05487aad1d..9bc99776a5bec 100644 --- a/modules/spring/src/test/config/jdbc-pojo-store-obj.xml +++ b/modules/spring/src/test/config/jdbc-pojo-store-obj.xml @@ -25,7 +25,10 @@ http://www.springframework.org/schema/util http://www.springframework.org/schema/util/spring-util.xsd"> + + + diff --git a/modules/spring/src/test/config/node.xml b/modules/spring/src/test/config/node.xml index 6f467ebf0f1b7..e6b0b520f248b 100644 --- a/modules/spring/src/test/config/node.xml +++ b/modules/spring/src/test/config/node.xml @@ -25,6 +25,8 @@ + + diff --git a/modules/spring/src/test/config/node1.xml b/modules/spring/src/test/config/node1.xml index 76539f785a1f0..814736a885f14 100644 --- a/modules/spring/src/test/config/node1.xml +++ b/modules/spring/src/test/config/node1.xml @@ -25,6 +25,8 @@ + + diff --git a/modules/spring/src/test/config/pojo-incorrect-store-cache.xml b/modules/spring/src/test/config/pojo-incorrect-store-cache.xml index 5627bd0fa40a4..f7f9141a841c2 100644 --- a/modules/spring/src/test/config/pojo-incorrect-store-cache.xml +++ b/modules/spring/src/test/config/pojo-incorrect-store-cache.xml @@ -23,6 +23,8 @@ http://www.springframework.org/schema/beans/spring-beans.xsd"> + + diff --git a/modules/spring/src/test/config/store-cache.xml b/modules/spring/src/test/config/store-cache.xml index 07a4ab0ca0287..d7bc014957b7a 100644 --- a/modules/spring/src/test/config/store-cache.xml +++ b/modules/spring/src/test/config/store-cache.xml @@ -25,6 +25,8 @@ + + diff --git a/modules/spring/src/test/config/store-cache1.xml b/modules/spring/src/test/config/store-cache1.xml index a9e58d6d1d113..46eef3f011104 100644 --- a/modules/spring/src/test/config/store-cache1.xml +++ b/modules/spring/src/test/config/store-cache1.xml @@ -26,6 +26,8 @@ + + From c3e8a832098887a0fd09b6e8f63d6a8cbaa20eb9 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Fri, 11 Nov 2016 18:00:36 +0300 Subject: [PATCH 350/487] .NET: Fix DataStreamerTestTopologyChange tests --- .../DataStreamerTestTopologyChange.cs | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTestTopologyChange.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTestTopologyChange.cs index c1f2c538053e5..9e80c087d78df 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTestTopologyChange.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTestTopologyChange.cs @@ -19,7 +19,6 @@ namespace Apache.Ignite.Core.Tests.Dataload { using System; using System.Threading; - using System.Threading.Tasks; using NUnit.Framework; /// @@ -59,7 +58,13 @@ public void TestNoCacheNode() var task = streamer.AddData(2, 3); streamer.Flush(); - AssertThrowsCacheStopped(task); + var ex = Assert.Throws(task.Wait).InnerException; + + Assert.IsNotNull(ex); + + Assert.AreEqual("Java exception occurred [class=org.apache.ignite.cache." + + "CacheServerNotFoundException, message=Failed to find server node for cache " + + "(all affinity nodes have left the grid or cache was stopped): cache]", ex.Message); } } @@ -86,19 +91,13 @@ public void TestDestroyCache() task = streamer.AddData(2, 3); streamer.Flush(); - AssertThrowsCacheStopped(task); - } - } + var ex = Assert.Throws(task.Wait).InnerException; - /// - /// Asserts that cache stopped error is thrown. - /// - private static void AssertThrowsCacheStopped(Task task) - { - var ex = Assert.Throws(task.Wait); - Assert.IsTrue(ex.InnerException.Message.Contains( - "Failed to find server node for cache " + - "(all affinity nodes have left the grid or cache was stopped):")); + Assert.IsNotNull(ex); + + Assert.AreEqual("class org.apache.ignite.IgniteCheckedException: DataStreamer data loading failed.", + ex.Message); + } } } } From a2a3bedce1a232c0c1db6f5e2b737ab47be250b0 Mon Sep 17 00:00:00 2001 From: sboikov Date: Mon, 14 Nov 2016 09:44:48 +0300 Subject: [PATCH 351/487] Fixed IgniteStartFromStreamConfigurationTest to stop started node. --- ...IgniteStartFromStreamConfigurationTest.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/IgniteStartFromStreamConfigurationTest.java b/modules/spring/src/test/java/org/apache/ignite/spring/IgniteStartFromStreamConfigurationTest.java index 0ef08f1b85b77..421011f1baf59 100644 --- a/modules/spring/src/test/java/org/apache/ignite/spring/IgniteStartFromStreamConfigurationTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/spring/IgniteStartFromStreamConfigurationTest.java @@ -30,21 +30,25 @@ * Checks starts from Stream. */ public class IgniteStartFromStreamConfigurationTest extends GridCommonAbstractTest { - /** Tests starts from Stream. */ + /** + * Tests starts from stream. + * + * @throws Exception If failed. + */ public void testStartFromStream() throws Exception { String cfg = "examples/config/example-cache.xml"; URL cfgLocation = U.resolveIgniteUrl(cfg); - Ignite grid = Ignition.start(new FileInputStream(cfgLocation.getFile())); + try (Ignite grid = Ignition.start(new FileInputStream(cfgLocation.getFile()))) { + grid.cache(null).put("1", "1"); - grid.cache(null).put("1", "1"); + assert grid.cache(null).get("1").equals("1"); - assert grid.cache(null).get("1").equals("1"); + IgniteConfiguration icfg = Ignition.loadSpringBean(new FileInputStream(cfgLocation.getFile()), "ignite.cfg"); - IgniteConfiguration icfg = Ignition.loadSpringBean(new FileInputStream(cfgLocation.getFile()), "ignite.cfg"); - - assert icfg.getCacheConfiguration()[0].getAtomicityMode() == CacheAtomicityMode.ATOMIC; + assert icfg.getCacheConfiguration()[0].getAtomicityMode() == CacheAtomicityMode.ATOMIC; + } } } \ No newline at end of file From 85a4b966fdfb7018d1c91b73df1659082128f786 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 14 Nov 2016 13:38:33 +0300 Subject: [PATCH 352/487] IGNITE-4216 .NET: Fix PlatformAffinityFunction to inject resource into baseFunc --- .../affinity/PlatformAffinityFunction.java | 7 +++- .../PlatformDotNetAffinityFunction.java | 9 ++--- .../Cache/Affinity/AffinityFunctionTest.cs | 33 +++++++++++++++++-- .../Apache.Ignite.Core.Tests/TestUtils.cs | 3 +- 4 files changed, 41 insertions(+), 11 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java index 1e844e7b55f04..8076a19b56f2c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java @@ -18,10 +18,12 @@ package org.apache.ignite.internal.processors.platform.cache.affinity; import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.cache.affinity.AffinityFunction; import org.apache.ignite.cache.affinity.AffinityFunctionContext; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.binary.BinaryRawWriterEx; import org.apache.ignite.internal.processors.platform.PlatformContext; import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; @@ -298,7 +300,10 @@ public byte getOverrideFlags() { */ @SuppressWarnings("unused") @IgniteInstanceResource - public void setIgnite(Ignite ignite) { + public void setIgnite(Ignite ignite) throws IgniteCheckedException { this.ignite = ignite; + + if (baseFunc != null && ignite != null) + ((IgniteEx)ignite).context().resource().injectGeneric(baseFunc); } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetAffinityFunction.java index 483fd227bbb10..f9f457dc67240 100644 --- a/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetAffinityFunction.java +++ b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetAffinityFunction.java @@ -18,17 +18,12 @@ package org.apache.ignite.platform.dotnet; import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; -import org.apache.ignite.binary.BinaryRawWriter; import org.apache.ignite.cache.affinity.AffinityFunction; import org.apache.ignite.cache.affinity.AffinityFunctionContext; import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.binary.BinaryRawWriterEx; -import org.apache.ignite.internal.processors.platform.PlatformContext; import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinityFunction; -import org.apache.ignite.internal.processors.platform.memory.PlatformMemory; -import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream; -import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; import org.apache.ignite.lifecycle.LifecycleAware; import org.apache.ignite.resources.IgniteInstanceResource; @@ -178,7 +173,7 @@ public PlatformAffinityFunction getFunc() { */ @SuppressWarnings("unused") @IgniteInstanceResource - private void setIgnite(Ignite ignite) { + private void setIgnite(Ignite ignite) throws IgniteCheckedException { assert func != null; func.setIgnite(ignite); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityFunctionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityFunctionTest.cs index f38cb3e7a9626..d4b66804e6c34 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityFunctionTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityFunctionTest.cs @@ -100,8 +100,8 @@ public void FixtureSetUp() public void FixtureTearDown() { // Check that affinity handles are present - TestUtils.AssertHandleRegistryHasItems(_ignite, _ignite.GetCacheNames().Count - 1, 0); - TestUtils.AssertHandleRegistryHasItems(_ignite2, _ignite.GetCacheNames().Count - 1, 0); + TestUtils.AssertHandleRegistryHasItems(_ignite, _ignite.GetCacheNames().Count - 3, 0); + TestUtils.AssertHandleRegistryHasItems(_ignite2, _ignite.GetCacheNames().Count - 3, 0); // Destroy all caches _ignite.GetCacheNames().ToList().ForEach(_ignite.DestroyCache); @@ -164,6 +164,34 @@ public void TestDynamicCache() CollectionAssert.AreEqual(parts.Select(x => firstTop), parts.Select(x => ctx.GetPreviousAssignment(x))); } + /// + /// Tests the dynamic cache with predefined functions. + /// + [Test] + public void TestDynamicCachePredefined() + { + var caches = new[] + { + new CacheConfiguration("rendezvousPredefined") + { + AffinityFunction = new RendezvousAffinityFunction {Partitions = 1234} + }, + new CacheConfiguration("fairPredefined") + { + AffinityFunction = new FairAffinityFunction {Partitions = 1234} + }, + }.Select(_ignite.CreateCache); + + foreach (var cache in caches) + { + Assert.AreEqual(1234, cache.GetConfiguration().AffinityFunction.Partitions); + + cache[1] = 2; + + Assert.AreEqual(2, cache[1]); + } + } + /// /// Verifies the cache affinity. /// @@ -231,6 +259,7 @@ public void TestExceptionInFunction() }); var ex = Assert.Throws(() => cache.Put(1, 2)); + Assert.IsNotNull(ex.InnerException); Assert.AreEqual("User error", ex.InnerException.Message); } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs index 88a2b522fdaa1..5073b1a9874e8 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs @@ -284,7 +284,8 @@ public static void AssertHandleRegistryHasItems(IIgnite grid, int expectedCount, var items = handleRegistry.GetItems().Where(x => !(x.Value is LifecycleBeanHolder)).ToList(); if (items.Any()) - Assert.Fail("HandleRegistry is not empty in grid '{0}':\n '{1}'", grid.Name, + Assert.Fail("HandleRegistry is not empty in grid '{0}' (expected {1}, actual {2}):\n '{3}'", + grid.Name, expectedCount, handleRegistry.Count, items.Select(x => x.ToString()).Aggregate((x, y) => x + "\n" + y)); } From 6e36a7950db84913ddfd0d98f5a0b50923d2a29c Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Tue, 15 Nov 2016 12:42:29 +0300 Subject: [PATCH 353/487] IGNITE-3191: Fields are now sorted for binary objects which don't implement Binarylizable interface. This closes #1197. --- .../apache/ignite/IgniteSystemProperties.java | 9 +++++ .../binary/BinaryClassDescriptor.java | 37 +++++++++++++++---- .../ignite/internal/binary/BinaryContext.java | 11 ++---- .../ignite/internal/binary/BinaryUtils.java | 4 ++ .../builder/BinaryObjectBuilderImpl.java | 6 ++- 5 files changed, 51 insertions(+), 16 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index a75027b8026a7..22cb9a6fe079f 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -477,6 +477,15 @@ public final class IgniteSystemProperties { @Deprecated public static final String IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES = "IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES"; + /** + * When set to {@code true} fields are written by BinaryMarshaller in sorted order. Otherwise + * the natural order is used. + *

          + * @deprecated Should be removed in Apache Ignite 2.0. + */ + @Deprecated + public static final String IGNITE_BINARY_SORT_OBJECT_FIELDS = "IGNITE_BINARY_SORT_OBJECT_FIELDS"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java index 4c824d4409fb1..d05ce71815666 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java @@ -25,14 +25,15 @@ import java.lang.reflect.Proxy; import java.math.BigDecimal; import java.sql.Timestamp; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.Map; import java.util.Set; +import java.util.TreeMap; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.binary.BinaryObjectException; @@ -120,6 +121,9 @@ public class BinaryClassDescriptor { /** */ private final Class[] intfs; + /** Whether stable schema was published. */ + private volatile boolean stableSchemaPublished; + /** * @param ctx Context. * @param cls Class. @@ -269,10 +273,14 @@ else if (useOptMarshaller) case OBJECT: // Must not use constructor to honor transient fields semantics. ctor = null; - ArrayList fields0 = new ArrayList<>(); stableFieldsMeta = metaDataEnabled ? new HashMap() : null; - BinarySchema.Builder schemaBuilder = BinarySchema.Builder.newBuilder(); + Map fields0; + + if (BinaryUtils.FIELDS_SORTED_ORDER) + fields0 = new TreeMap<>(); + else + fields0 = new LinkedHashMap<>(); Set duplicates = duplicateFields(cls); @@ -300,9 +308,7 @@ else if (useOptMarshaller) BinaryFieldAccessor fieldInfo = BinaryFieldAccessor.create(f, fieldId); - fields0.add(fieldInfo); - - schemaBuilder.addField(fieldId); + fields0.put(name, fieldInfo); if (metaDataEnabled) stableFieldsMeta.put(name, fieldInfo.mode().typeId()); @@ -310,7 +316,12 @@ else if (useOptMarshaller) } } - fields = fields0.toArray(new BinaryFieldAccessor[fields0.size()]); + fields = fields0.values().toArray(new BinaryFieldAccessor[fields0.size()]); + + BinarySchema.Builder schemaBuilder = BinarySchema.Builder.newBuilder(); + + for (BinaryFieldAccessor field : fields) + schemaBuilder.addField(field.id); stableSchema = schemaBuilder.build(); @@ -747,6 +758,18 @@ void write(Object obj, BinaryWriterExImpl writer) throws BinaryObjectException { break; case OBJECT: + if (userType && !stableSchemaPublished) { + // Update meta before write object with new schema + BinaryMetadata meta = new BinaryMetadata(typeId, typeName, stableFieldsMeta, + affKeyFieldName, Collections.singleton(stableSchema), false); + + ctx.updateMetadata(typeId, meta); + + schemaReg.addSchema(stableSchema.schemaId(), stableSchema); + + stableSchemaPublished = true; + } + if (preWrite(writer, obj)) { try { for (BinaryFieldAccessor info : fields) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java index 2ac8b7fc317fb..a72e7ace32d01 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java @@ -785,12 +785,9 @@ private BinaryClassDescriptor registerUserClassDescriptor(Class cls, boolean registered ); - if (!deserialize) { - Collection schemas = desc.schema() != null ? Collections.singleton(desc.schema()) : null; - + if (!deserialize) metaHnd.addMeta(typeId, - new BinaryMetadata(typeId, typeName, desc.fieldsMeta(), affFieldName, schemas, desc.isEnum()).wrap(this)); - } + new BinaryMetadata(typeId, typeName, desc.fieldsMeta(), affFieldName, null, desc.isEnum()).wrap(this)); descByCls.put(cls, desc); @@ -1123,7 +1120,6 @@ public void registerUserType(String clsName, cls2Mappers.put(clsName, mapper); Map fieldsMeta = null; - Collection schemas = null; if (cls != null) { if (serializer == null) { @@ -1148,7 +1144,6 @@ public void registerUserType(String clsName, ); fieldsMeta = desc.fieldsMeta(); - schemas = desc.schema() != null ? Collections.singleton(desc.schema()) : null; descByCls.put(cls, desc); @@ -1157,7 +1152,7 @@ public void registerUserType(String clsName, predefinedTypes.put(id, desc); } - metaHnd.addMeta(id, new BinaryMetadata(id, typeName, fieldsMeta, affKeyFieldName, schemas, isEnum).wrap(this)); + metaHnd.addMeta(id, new BinaryMetadata(id, typeName, fieldsMeta, affKeyFieldName, null, isEnum).wrap(this)); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java index 25d87ffe802be..b3040826fead9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java @@ -128,6 +128,10 @@ public class BinaryUtils { public static final boolean WRAP_TREES = !IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES); + /** Whether to sort field in binary objects (doesn't affect Binarylizable). */ + public static final boolean FIELDS_SORTED_ORDER = + IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_BINARY_SORT_OBJECT_FIELDS); + /** Field type names. */ private static final String[] FIELD_TYPE_NAMES; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java index 2c761925e68af..f0bc8745bf266 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java @@ -42,6 +42,7 @@ import java.util.LinkedHashMap; import java.util.Map; import java.util.Set; +import java.util.TreeMap; /** * @@ -522,7 +523,10 @@ private void ensureReadCacheInit() { Object val = val0 == null ? new BinaryValueWithType(BinaryUtils.typeByClass(Object.class), null) : val0; if (assignedVals == null) - assignedVals = new LinkedHashMap<>(); + if (BinaryUtils.FIELDS_SORTED_ORDER) + assignedVals = new TreeMap<>(); + else + assignedVals = new LinkedHashMap<>(); Object oldVal = assignedVals.put(name, val); From e39888a08da313bec4d30f96488eccb36b4abacc Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Thu, 17 Nov 2016 11:41:05 +0700 Subject: [PATCH 354/487] IGNITE-4163 Fixed load range queries. --- .../ignite/cache/store/jdbc/CacheAbstractJdbcStore.java | 2 +- .../ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java | 4 ++-- .../ignite/cache/store/jdbc/dialect/DB2Dialect.java | 8 ++++++++ .../ignite/cache/store/jdbc/dialect/OracleDialect.java | 8 ++++++++ .../ignite/cache/store/jdbc/dialect/SQLServerDialect.java | 8 ++++++++ 5 files changed, 27 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java index 4eb5b5630635f..4bfd92b836268 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java @@ -1982,7 +1982,7 @@ private LoadCacheCustomQueryWorker(EntryMapping em, String qry, IgniteBiInClosur Map colIdxs = U.newHashMap(meta.getColumnCount()); for (int i = 1; i <= meta.getColumnCount(); i++) - colIdxs.put(meta.getColumnLabel(i), i); + colIdxs.put(meta.getColumnLabel(i).toUpperCase(), i); while (rs.next()) { K1 key = buildObject(em.cacheName, em.keyType(), em.keyKind(), em.keyColumns(), em.keyCols, colIdxs, rs); diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java index 5fa564b082fa5..3ab112af41516 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java @@ -157,8 +157,8 @@ private static String where(Collection keyCols, int keyCnt) { @Override public String loadCacheSelectRangeQuery(String fullTblName, Collection keyCols) { String cols = mkString(keyCols, ","); - return String.format("SELECT %s FROM (SELECT %s, ROWNUM() AS rn FROM %s ORDER BY %s) WHERE mod(rn, ?) = 0", - cols, cols, fullTblName, cols); + return String.format("SELECT %1$s FROM (SELECT %1$s, ROW_NUMBER() OVER() AS rn FROM (SELECT %1$s FROM %2$s ORDER BY %1$s) AS tbl) AS tbl WHERE mod(rn, ?) = 0", + cols, fullTblName); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java index 4bae14a443f5b..551782e9e9200 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java @@ -28,6 +28,14 @@ public class DB2Dialect extends BasicJdbcDialect { /** */ private static final long serialVersionUID = 0L; + /** {@inheritDoc} */ + @Override public String loadCacheSelectRangeQuery(String fullTblName, Collection keyCols) { + String cols = mkString(keyCols, ","); + + return String.format("SELECT %1$s FROM (SELECT %1$s, ROW_NUMBER() OVER(ORDER BY %1$s) AS rn FROM %2$s) WHERE mod(rn, ?) = 0 ORDER BY %1$s", + cols, fullTblName); + } + /** {@inheritDoc} */ @Override public boolean hasMerge() { return true; diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java index a9efe0dbd2c28..e268b24c765de 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java @@ -33,6 +33,14 @@ public class OracleDialect extends BasicJdbcDialect { return true; } + /** {@inheritDoc} */ + @Override public String loadCacheSelectRangeQuery(String fullTblName, Collection keyCols) { + String cols = mkString(keyCols, ","); + + return String.format("SELECT %1$s FROM (SELECT %1$s, ROWNUM AS rn FROM (SELECT %1$s FROM %2$s ORDER BY %1$s)) WHERE mod(rn, ?) = 0", + cols, fullTblName); + } + /** {@inheritDoc} */ @Override public String mergeQuery(String fullTblName, Collection keyCols, Collection uniqCols) { Collection cols = F.concat(false, keyCols, uniqCols); diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java index 883918f9803a4..ace1d3f0bf738 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java @@ -33,6 +33,14 @@ public class SQLServerDialect extends BasicJdbcDialect { return '[' + ident + ']'; } + /** {@inheritDoc} */ + @Override public String loadCacheSelectRangeQuery(String fullTblName, Collection keyCols) { + String cols = mkString(keyCols, ","); + + return String.format("SELECT %1$s FROM (SELECT %1$s, ROW_NUMBER() OVER(ORDER BY %1$s) AS rn FROM %2$s) tbl WHERE rn %% ? = 0 ORDER BY %1$s", + cols, fullTblName); + } + /** {@inheritDoc} */ @Override public boolean hasMerge() { return true; From 0234f67390c88dceefd6e62de98adb922b4ba9ac Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Mon, 21 Nov 2016 17:40:50 +0700 Subject: [PATCH 355/487] IGNITE-3443 Implemented metrics for queries monitoring. --- .../java/org/apache/ignite/IgniteCache.java | 21 +- .../cache/query/QueryDetailMetrics.java | 98 +++ .../ignite/cache/query/QueryMetrics.java | 2 +- .../configuration/CacheConfiguration.java | 30 + .../processors/cache/GridCacheUtilityKey.java | 6 +- .../processors/cache/IgniteCacheProxy.java | 59 +- .../processors/cache/query/CacheQuery.java | 1 - .../cache/query/CacheQueryType.java | 2 +- .../cache/query/GridCacheQueryAdapter.java | 10 - .../GridCacheQueryDetailMetricsAdapter.java | 248 ++++++++ .../query/GridCacheQueryDetailMetricsKey.java | 82 +++ .../query/GridCacheQueryFutureAdapter.java | 2 - .../cache/query/GridCacheQueryManager.java | 149 ++++- .../query/GridCacheQueryMetricsAdapter.java | 27 +- .../processors/query/GridQueryProcessor.java | 102 ++-- .../util/GridBoundedPriorityQueue.java | 68 +++ .../cache/VisorCacheQueryDetailMetrics.java | 167 +++++ ...rCacheQueryDetailMetricsCollectorTask.java | 146 +++++ ...VisorCacheResetQueryDetailMetricsTask.java | 71 +++ .../VisorCacheResetQueryMetricsTask.java | 8 +- .../internal/visor/util/VisorTaskUtils.java | 26 +- .../resources/META-INF/classnames.properties | 75 ++- .../GridBoundedPriorityQueueSelfTest.java | 114 ++++ .../multijvm/IgniteCacheProcessProxy.java | 16 + .../testsuites/IgniteLangSelfTestSuite.java | 4 +- ...cheAbstractQueryDetailMetricsSelfTest.java | 572 ++++++++++++++++++ .../CacheAbstractQueryMetricsSelfTest.java | 305 ++++++---- .../CacheLocalQueryDetailMetricsSelfTest.java | 33 + ...QueryDetailMetricsDistributedSelfTest.java | 33 + ...tionedQueryDetailMetricsLocalSelfTest.java | 33 + ...QueryDetailMetricsDistributedSelfTest.java | 33 + ...icatedQueryDetailMetricsLocalSelfTest.java | 33 + .../IgniteCacheQuerySelfTestSuite2.java | 12 + .../commands/cache/VisorCacheCommand.scala | 7 +- 34 files changed, 2338 insertions(+), 257 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/cache/query/QueryDetailMetrics.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailMetricsAdapter.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailMetricsKey.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedPriorityQueue.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryDetailMetrics.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryDetailMetricsCollectorTask.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetQueryDetailMetricsTask.java create mode 100644 modules/core/src/test/java/org/apache/ignite/lang/utils/GridBoundedPriorityQueueSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryDetailMetricsSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryDetailMetricsSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryDetailMetricsDistributedSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryDetailMetricsLocalSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryDetailMetricsDistributedSelfTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryDetailMetricsLocalSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java index 2290fc561ed90..7eb6e919bcab6 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java @@ -43,6 +43,7 @@ import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.query.Query; import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.QueryDetailMetrics; import org.apache.ignite.cache.query.QueryMetrics; import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.cache.query.SpiQuery; @@ -325,6 +326,24 @@ public interface IgniteCache extends javax.cache.Cache, IgniteAsyncS */ public QueryMetrics queryMetrics(); + /** + * Reset query metrics. + */ + public void resetQueryMetrics(); + + /** + * Gets query detail metrics. + * Query detail metrics could be enabled via {@link CacheConfiguration#setQueryDetailMetricsSize(int)} method. + * + * @return Metrics. + */ + public Collection queryDetailMetrics(); + + /** + * Reset query detail metrics. + */ + public void resetQueryDetailMetrics(); + /** * Attempts to evict all entries associated with keys. Note, * that entry will be evicted only if it's not used (not @@ -335,7 +354,7 @@ public interface IgniteCache extends javax.cache.Cache, IgniteAsyncS public void localEvict(Collection keys); /** - * Peeks at in-memory cached value using default optinal peek mode. + * Peeks at in-memory cached value using default optional peek mode. *

          * This method will not load value from any persistent store or from a remote node. *

          Transactions

          diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryDetailMetrics.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryDetailMetrics.java new file mode 100644 index 0000000000000..0b0188069ea63 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryDetailMetrics.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.query; + +import org.apache.ignite.configuration.CacheConfiguration; + +/** + * Query metrics aggregated by query type and its textual representation. + * + * Query detail metrics could be enabled via {@link CacheConfiguration#setQueryDetailMetricsSize(int)} method. + */ +public interface QueryDetailMetrics { + /** + * @return Query type. + */ + public String queryType(); + + /** + * @return Textual representation of query. + */ + public String query(); + + /** + * @return Cache where query was executed. + */ + public String cache(); + + /** + * Gets total number execution of query. + * + * @return Number of executions. + */ + public int executions(); + + /** + * Gets number of completed execution of query. + * + * @return Number of completed executions. + */ + public int completions(); + + /** + * Gets number of times a query execution failed. + * + * @return Number of times a query execution failed. + */ + public int failures(); + + /** + * Gets minimum execution time of query. + * + * @return Minimum execution time of query. + */ + public long minimumTime(); + + /** + * Gets maximum execution time of query. + * + * @return Maximum execution time of query. + */ + public long maximumTime(); + + /** + * Gets average execution time of query. + * + * @return Average execution time of query. + */ + public double averageTime(); + + /** + * Gets total time of all query executions. + * + * @return Total time of all query executions. + */ + public long totalTime(); + + /** + * Gets latest query start time. + * + * @return Latest time query was stared. + */ + public long lastStartTime(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java index e376561dbbdcc..d0f0a50979690 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java @@ -59,4 +59,4 @@ public interface QueryMetrics { * @return Total number of times a query execution failed. */ public int fails(); -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java index f65bf52104dd7..f9c114b79e277 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java @@ -209,6 +209,9 @@ public class CacheConfiguration extends MutableConfiguration { /** Default timeout after which long query warning will be printed. */ public static final long DFLT_LONG_QRY_WARN_TIMEOUT = 3000; + /** Default number of queries detail metrics to collect. */ + public static final int DFLT_QRY_DETAIL_METRICS_SIZE = 0; + /** Default size for onheap SQL row cache size. */ public static final int DFLT_SQL_ONHEAP_ROW_CACHE_SIZE = 10 * 1024; @@ -361,6 +364,9 @@ public class CacheConfiguration extends MutableConfiguration { /** */ private long longQryWarnTimeout = DFLT_LONG_QRY_WARN_TIMEOUT; + /** */ + private int qryDetailMetricsSz = DFLT_QRY_DETAIL_METRICS_SIZE; + /** * Flag indicating whether data can be read from backup. * If {@code false} always get data from primary node (never from backup). @@ -468,6 +474,7 @@ public CacheConfiguration(CompleteConfiguration cfg) { nodeFilter = cc.getNodeFilter(); pluginCfgs = cc.getPluginConfigurations(); qryEntities = cc.getQueryEntities() == Collections.emptyList() ? null : cc.getQueryEntities(); + qryDetailMetricsSz = cc.getQueryDetailMetricsSize(); readFromBackup = cc.isReadFromBackup(); rebalanceBatchSize = cc.getRebalanceBatchSize(); rebalanceBatchesPrefetchCount = cc.getRebalanceBatchesPrefetchCount(); @@ -1823,6 +1830,29 @@ public CacheConfiguration setLongQueryWarningTimeout(long longQryWarnTimeo return this; } + /** + * Gets size of queries detail metrics that will be stored in memory for monitoring purposes. + * If {@code 0} then history will not be collected. + * Note, larger number may lead to higher memory consumption. + * + * @return Maximum number of query metrics that will be stored in memory. + */ + public int getQueryDetailMetricsSize() { + return qryDetailMetricsSz; + } + + /** + * Sets size of queries detail metrics that will be stored in memory for monitoring purposes. + * + * @param qryDetailMetricsSz Maximum number of latest queries metrics that will be stored in memory. + * @return {@code this} for chaining. + */ + public CacheConfiguration setQueryDetailMetricsSize(int qryDetailMetricsSz) { + this.qryDetailMetricsSz = qryDetailMetricsSz; + + return this; + } + /** * Gets custom name of the sql schema. If custom sql schema is not set then {@code null} will be returned and * quoted case sensitive name will be used as sql schema. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtilityKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtilityKey.java index 48b6b452a9c56..8110170828662 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtilityKey.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtilityKey.java @@ -27,8 +27,8 @@ public abstract class GridCacheUtilityKey impleme private static final long serialVersionUID = 0L; /** {@inheritDoc} */ - @SuppressWarnings("unchecked") @Override - public final boolean equals(Object obj) { + @SuppressWarnings("unchecked") + @Override public final boolean equals(Object obj) { return obj == this || obj != null && obj.getClass() == getClass() && equalsx((K)obj); } @@ -42,4 +42,4 @@ public final boolean equals(Object obj) { /** {@inheritDoc} */ public abstract int hashCode(); -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java index 297ec681ae943..f87fa1d5d815f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java @@ -51,6 +51,7 @@ import org.apache.ignite.cache.query.ContinuousQuery; import org.apache.ignite.cache.query.Query; import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.QueryDetailMetrics; import org.apache.ignite.cache.query.QueryMetrics; import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.cache.query.SpiQuery; @@ -63,10 +64,10 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.binary.BinaryUtils; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.query.CacheQuery; import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture; +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; import org.apache.ignite.internal.util.GridEmptyIterator; @@ -243,6 +244,7 @@ public GridCacheGateway gate() { } } + /** {@inheritDoc} */ @Override public CacheMetrics localMetrics() { GridCacheGateway gate = this.gate; @@ -270,6 +272,7 @@ public GridCacheGateway gate() { } } + /** {@inheritDoc} */ @Override public CacheMetricsMXBean localMxBean() { GridCacheGateway gate = this.gate; @@ -491,8 +494,8 @@ private QueryCursor query( if (grp != null) qry.projection(grp); - final GridCloseableIterator iter = ctx.kernalContext().query().executeQuery(ctx, - new IgniteOutClosureX>() { + final GridCloseableIterator iter = ctx.kernalContext().query().executeQuery(GridCacheQueryType.SCAN, + ctx.name(), ctx, new IgniteOutClosureX>() { @Override public GridCloseableIterator applyx() throws IgniteCheckedException { final GridCloseableIterator iter0 = qry.executeScanQuery(); @@ -520,7 +523,7 @@ private QueryCursor query( } }; } - }, false); + }, true); return new QueryCursorImpl<>(iter); } @@ -547,7 +550,7 @@ private QueryCursor> query(final Query filter, @Nullable Clust if (grp != null) qry.projection(grp); - fut = ctx.kernalContext().query().executeQuery(ctx, + fut = ctx.kernalContext().query().executeQuery(GridCacheQueryType.TEXT, p.getText(), ctx, new IgniteOutClosureX>>() { @Override public CacheQueryFuture> applyx() throws IgniteCheckedException { return qry.execute(); @@ -560,8 +563,8 @@ else if (filter instanceof SpiQuery) { if (grp != null) qry.projection(grp); - fut = ctx.kernalContext().query().executeQuery(ctx, - new IgniteOutClosureX>>() { + fut = ctx.kernalContext().query().executeQuery(GridCacheQueryType.SPI, filter.getClass().getSimpleName(), + ctx, new IgniteOutClosureX>>() { @Override public CacheQueryFuture> applyx() throws IgniteCheckedException { return qry.execute(((SpiQuery)filter).getArgs()); } @@ -886,6 +889,48 @@ private void validate(Query qry) { } } + /** {@inheritDoc} */ + @Override public void resetQueryMetrics() { + GridCacheGateway gate = this.gate; + + CacheOperationContext prev = onEnter(gate, opCtx); + + try { + delegate.context().queries().resetMetrics(); + } + finally { + onLeave(gate, prev); + } + } + + /** {@inheritDoc} */ + @Override public Collection queryDetailMetrics() { + GridCacheGateway gate = this.gate; + + CacheOperationContext prev = onEnter(gate, opCtx); + + try { + return delegate.context().queries().detailMetrics(); + } + finally { + onLeave(gate, prev); + } + } + + /** {@inheritDoc} */ + @Override public void resetQueryDetailMetrics() { + GridCacheGateway gate = this.gate; + + CacheOperationContext prev = onEnter(gate, opCtx); + + try { + delegate.context().queries().resetDetailMetrics(); + } + finally { + onLeave(gate, prev); + } + } + /** {@inheritDoc} */ @Override public void localEvict(Collection keys) { GridCacheGateway gate = this.gate; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQuery.java index 3fa041b205e26..a7dbe2af50399 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQuery.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQuery.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal.processors.cache.query; -import java.util.Map; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.affinity.AffinityKey; import org.apache.ignite.cache.query.Query; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryType.java index bb8bc33cbd3b8..fa0f3df0428f5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryType.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryType.java @@ -44,4 +44,4 @@ public enum CacheQueryType { /** SPI query. */ SPI -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java index f65b733bfe969..2355591def2f1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java @@ -434,16 +434,6 @@ public void validate() throws IgniteCheckedException { throw new IgniteCheckedException("Indexing is disabled for cache: " + cctx.cache().name()); } - /** - * @param res Query result. - * @param err Error or {@code null} if query executed successfully. - * @param startTime Start time. - * @param duration Duration. - */ - public void onCompleted(Object res, Throwable err, long startTime, long duration) { - GridQueryProcessor.onCompleted(cctx, res, err, startTime, duration, log); - } - /** {@inheritDoc} */ @Override public CacheQueryFuture execute(@Nullable Object... args) { return execute0(null, args); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailMetricsAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailMetricsAdapter.java new file mode 100644 index 0000000000000..a477360b46d18 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailMetricsAdapter.java @@ -0,0 +1,248 @@ +/* + * 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.query; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import org.apache.ignite.cache.query.QueryDetailMetrics; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * Adapter for {@link QueryDetailMetrics}. + */ +public class GridCacheQueryDetailMetricsAdapter implements QueryDetailMetrics, Externalizable { + /** */ + private static final long serialVersionUID = 0L; + + /** Query type to track metrics. */ + private GridCacheQueryType qryType; + + /** Textual query representation. */ + private String qry; + + /** Cache name. */ + private String cache; + + /** Number of executions. */ + private int execs; + + /** Number of completions executions. */ + private int completions; + + /** Number of failures. */ + private int failures; + + /** Minimum time of execution. */ + private long minTime = -1; + + /** Maximum time of execution. */ + private long maxTime; + + /** Sum of execution time of completions time. */ + private long totalTime; + + /** Sum of execution time of completions time. */ + private long lastStartTime; + + /** Cached metrics group key.*/ + private GridCacheQueryDetailMetricsKey key; + + /** + * Required by {@link Externalizable}. + */ + public GridCacheQueryDetailMetricsAdapter() { + // No-op. + } + + /** + * Constructor with metrics. + * + * @param qryType Query type. + * @param qry Textual query representation. + * @param cache Cache name where query was executed. + * @param startTime Duration of queue execution. + * @param duration Duration of queue execution. + * @param failed {@code True} query executed unsuccessfully {@code false} otherwise. + */ + public GridCacheQueryDetailMetricsAdapter(GridCacheQueryType qryType, String qry, String cache, long startTime, + long duration, boolean failed) { + this.qryType = qryType; + this.qry = qryType == GridCacheQueryType.SCAN && qry == null ? cache : qry; + this.cache = cache; + + if (failed) { + execs = 1; + failures = 1; + } + else { + execs = 1; + completions = 1; + totalTime = duration; + minTime = duration; + maxTime = duration; + } + + lastStartTime = startTime; + } + + /** + * Copy constructor. + * + * @param qryType Query type. + * @param qry Textual query representation. + * @param cache Cache name where query was executed. + */ + public GridCacheQueryDetailMetricsAdapter(GridCacheQueryType qryType, String qry, String cache, + int execs, int completions, int failures, long minTime, long maxTime, long totalTime, long lastStartTime, + GridCacheQueryDetailMetricsKey key) { + this.qryType = qryType; + this.qry = qry; + this.cache = cache; + this.execs = execs; + this.completions = completions; + this.failures = failures; + this.minTime = minTime; + this.maxTime = maxTime; + this.totalTime = totalTime; + this.lastStartTime = lastStartTime; + this.key = key; + } + + /** + * @return Metrics group key. + */ + public GridCacheQueryDetailMetricsKey key() { + if (key == null) + key = new GridCacheQueryDetailMetricsKey(qryType, qry); + + return key; + } + + /** + * Aggregate metrics. + * + * @param m Other metrics to take into account. + * @return Aggregated metrics. + */ + public GridCacheQueryDetailMetricsAdapter aggregate(QueryDetailMetrics m) { + return new GridCacheQueryDetailMetricsAdapter( + qryType, + qry, + m.cache(), + execs + m.executions(), + completions + m.completions(), + failures + m.failures(), + minTime < 0 || minTime > m.minimumTime() ? m.minimumTime() : minTime, + maxTime < m.maximumTime() ? m.maximumTime() : maxTime, + totalTime + m.totalTime(), + lastStartTime < m.lastStartTime() ? m.lastStartTime() : lastStartTime, + key + ); + } + + /** {@inheritDoc} */ + @Override public String queryType() { + return qryType.name(); + } + + /** {@inheritDoc} */ + @Override public String query() { + return qry; + } + + /** {@inheritDoc} */ + @Override public String cache() { + return cache; + } + + /** {@inheritDoc} */ + @Override public int executions() { + return execs; + } + + /** {@inheritDoc} */ + @Override public int completions() { + return completions; + } + + /** {@inheritDoc} */ + @Override public int failures() { + return failures; + } + + /** {@inheritDoc} */ + @Override public long minimumTime() { + return minTime < 0 ? 0 : minTime; + } + + /** {@inheritDoc} */ + @Override public long maximumTime() { + return maxTime; + } + + /** {@inheritDoc} */ + @Override public double averageTime() { + double val = completions; + + return val > 0 ? totalTime / val : 0; + } + + /** {@inheritDoc} */ + @Override public long totalTime() { + return totalTime; + } + + /** {@inheritDoc} */ + @Override public long lastStartTime() { + return lastStartTime; + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + U.writeEnum(out, qryType); + U.writeString(out, qry); + U.writeString(out, cache); + out.writeInt(execs); + out.writeInt(completions); + out.writeLong(minTime); + out.writeLong(maxTime); + out.writeLong(totalTime); + out.writeLong(lastStartTime); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + qryType = GridCacheQueryType.fromOrdinal(in.readByte()); + qry = U.readString(in); + cache = U.readString(in); + execs = in.readInt(); + completions = in.readInt(); + minTime = in.readLong(); + maxTime = in.readLong(); + totalTime = in.readLong(); + lastStartTime = in.readLong(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridCacheQueryDetailMetricsAdapter.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailMetricsKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailMetricsKey.java new file mode 100644 index 0000000000000..eb70cd1c3dd78 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryDetailMetricsKey.java @@ -0,0 +1,82 @@ +/* + * 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.query; + +import org.apache.ignite.internal.util.typedef.F; + +/** + * Immutable query metrics key used to group metrics. + */ +public class GridCacheQueryDetailMetricsKey { + /** Query type to track metrics. */ + private final GridCacheQueryType qryType; + + /** Textual query representation. */ + private final String qry; + + /** Pre-calculated hash code. */ + private final int hash; + + /** + * Constructor. + * + * @param qryType Query type. + * @param qry Textual query representation. + */ + public GridCacheQueryDetailMetricsKey(GridCacheQueryType qryType, String qry) { + assert qryType != null; + assert qryType != GridCacheQueryType.SQL_FIELDS || qry != null; + + this.qryType = qryType; + this.qry = qry; + + hash = 31 * qryType.hashCode() + (qry != null ? qry.hashCode() : 0); + } + + /** + * @return Query type. + */ + public GridCacheQueryType getQueryType() { + return qryType; + } + + /** + * @return Textual representation of query. + */ + public String getQuery() { + return qry; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return hash; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + GridCacheQueryDetailMetricsKey other = (GridCacheQueryDetailMetricsKey)o; + + return qryType == other.qryType && F.eq(qry, other.qry); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java index db519f58e0954..4b8915cce2d4c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java @@ -155,8 +155,6 @@ boolean fields() { @Override public boolean onDone(Collection res, Throwable err) { cctx.time().removeTimeoutObject(this); - qry.query().onCompleted(res, err, startTime(), duration()); - return super.onDone(res, err); } 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 00a918109cbcc..ab8bd63ee75da 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 @@ -26,6 +26,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -46,6 +47,7 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.cache.query.QueryMetrics; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.events.CacheQueryExecutedEvent; import org.apache.ignite.events.CacheQueryReadEvent; import org.apache.ignite.events.DiscoveryEvent; @@ -80,6 +82,7 @@ import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.GridBoundedPriorityQueue; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; import org.apache.ignite.internal.util.GridEmptyCloseableIterator; import org.apache.ignite.internal.util.GridEmptyIterator; @@ -134,8 +137,37 @@ */ @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") public abstract class GridCacheQueryManager extends GridCacheManagerAdapter { + /** Maximum number of query detail metrics to evict at once. */ + private static final int QRY_DETAIL_METRICS_EVICTION_LIMIT = 10_000; + + /** Comparator for priority queue with query detail metrics with priority to new metrics. */ + private static final Comparator QRY_DETAIL_METRICS_PRIORITY_NEW_CMP = + new Comparator() { + @Override public int compare(GridCacheQueryDetailMetricsAdapter m1, GridCacheQueryDetailMetricsAdapter m2) { + return Long.compare(m1.lastStartTime(), m2.lastStartTime()); + } + }; + + /** Comparator for priority queue with query detail metrics with priority to old metrics. */ + private static final Comparator QRY_DETAIL_METRICS_PRIORITY_OLD_CMP = + new Comparator() { + @Override public int compare(GridCacheQueryDetailMetricsAdapter m1, GridCacheQueryDetailMetricsAdapter m2) { + return Long.compare(m2.lastStartTime(), m1.lastStartTime()); + } + }; + + /** Function to merge query detail metrics. */ + private static final ConcurrentHashMap8.BiFun QRY_DETAIL_METRICS_MERGE_FX = + new ConcurrentHashMap8.BiFun() { + @Override public GridCacheQueryDetailMetricsAdapter apply(GridCacheQueryDetailMetricsAdapter oldVal, + GridCacheQueryDetailMetricsAdapter newVal) { + return oldVal.aggregate(newVal); + } + }; + /** */ - protected GridQueryProcessor qryProc; + private GridQueryProcessor qryProc; /** */ private String space; @@ -147,8 +179,13 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte private volatile GridCacheQueryMetricsAdapter metrics = new GridCacheQueryMetricsAdapter(); /** */ - private final ConcurrentMap qryIters = - new ConcurrentHashMap8<>(); + private int detailMetricsSz; + + /** */ + private ConcurrentHashMap8 detailMetrics; + + /** */ + private final ConcurrentMap qryIters = new ConcurrentHashMap8<>(); /** */ private final ConcurrentMap>> fieldsQryRes = @@ -171,10 +208,18 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte /** {@inheritDoc} */ @Override public void start0() throws IgniteCheckedException { + CacheConfiguration ccfg = cctx.config(); + qryProc = cctx.kernalContext().query(); space = cctx.name(); - maxIterCnt = cctx.config().getMaxQueryIteratorsCount(); + maxIterCnt = ccfg.getMaxQueryIteratorsCount(); + + detailMetricsSz = ccfg.getQueryDetailMetricsSize(); + + if (detailMetricsSz > 0) + detailMetrics = new ConcurrentHashMap8<>(detailMetricsSz); + lsnr = new GridLocalEventListener() { @Override public void onEvent(Event evt) { UUID nodeId = ((DiscoveryEvent)evt).eventNode().id(); @@ -213,7 +258,7 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte cctx.events().addListener(lsnr, EVT_NODE_LEFT, EVT_NODE_FAILED); - enabled = GridQueryProcessor.isEnabled(cctx.config()); + enabled = GridQueryProcessor.isEnabled(ccfg); qryTopVer = cctx.startTopologyVersion(); @@ -1691,6 +1736,8 @@ protected GridCloseableIterator scanQueryLocal(final GridCacheQueryAdapter qry, long startTime = U.currentTimeMillis(); + final String namex = cctx.namex(); + try { assert qry.type() == SCAN; @@ -1699,7 +1746,6 @@ protected GridCloseableIterator scanQueryLocal(final GridCacheQueryAdapter qry, final String taskName = cctx.kernalContext().task().resolveTaskName(qry.taskHash()); final IgniteBiPredicate filter = qry.scanFilter(); - final String namex = cctx.namex(); final ClusterNode locNode = cctx.localNode(); final UUID subjId = qry.subjectId(); @@ -1721,12 +1767,9 @@ protected GridCloseableIterator scanQueryLocal(final GridCacheQueryAdapter qry, final GridCloseableIterator> iter = scanIterator(qry, true); - if (updStatisticsIfNeeded) { + if (updStatisticsIfNeeded) needUpdStatistics = false; - cctx.queries().onCompleted(U.currentTimeMillis() - startTime, false); - } - final boolean readEvt = cctx.gridEvents().isRecordable(EVT_CACHE_QUERY_OBJECT_READ); return new GridCloseableIteratorAdapter() { @@ -1789,7 +1832,8 @@ protected GridCloseableIterator scanQueryLocal(final GridCacheQueryAdapter qry, } catch (Exception e) { if (needUpdStatistics) - cctx.queries().onCompleted(U.currentTimeMillis() - startTime, true); + cctx.queries().collectMetrics(GridCacheQueryType.SCAN, namex, startTime, + U.currentTimeMillis() - startTime, true); throw e; } @@ -2070,6 +2114,52 @@ public QueryMetrics metrics() { return metrics.copy(); } + /** + * Gets cache queries detailed metrics. + * Detail metrics could be enabled by setting non-zero value via {@link CacheConfiguration#setQueryDetailMetricsSize(int)} + * + * @return Cache queries metrics aggregated by query type and query text. + */ + public Collection detailMetrics() { + if (detailMetricsSz > 0) { + // Return no more than latest detailMetricsSz items. + if (detailMetrics.size() > detailMetricsSz) { + GridBoundedPriorityQueue latestMetrics = + new GridBoundedPriorityQueue<>(detailMetricsSz, QRY_DETAIL_METRICS_PRIORITY_NEW_CMP); + + latestMetrics.addAll(detailMetrics.values()); + + return latestMetrics; + } + + return new ArrayList<>(detailMetrics.values()); + } + + return Collections.emptyList(); + } + + /** + * Evict detail metrics. + */ + public void evictDetailMetrics() { + if (detailMetricsSz > 0) { + int sz = detailMetrics.size(); + + if (sz > detailMetricsSz) { + // Limit number of metrics to evict in order make eviction time predictable. + int evictCnt = Math.min(QRY_DETAIL_METRICS_EVICTION_LIMIT, sz - detailMetricsSz); + + Queue metricsToEvict = + new GridBoundedPriorityQueue<>(evictCnt, QRY_DETAIL_METRICS_PRIORITY_OLD_CMP); + + metricsToEvict.addAll(detailMetrics.values()); + + for (GridCacheQueryDetailMetricsAdapter m : metricsToEvict) + detailMetrics.remove(m.key()); + } + } + } + /** * Resets metrics. */ @@ -2078,18 +2168,43 @@ public void resetMetrics() { } /** - * @param fail {@code true} if execution failed. + * Resets detail metrics. */ - public void onExecuted(boolean fail) { - metrics.onQueryExecute(fail); + public void resetDetailMetrics() { + if (detailMetrics != null) + detailMetrics.clear(); } /** + * @param qryType Query type. + * @param qry Query description. + * @param startTime Query start size. * @param duration Execution duration. - * @param fail {@code true} if execution failed. + * @param failed {@code True} if query execution failed. */ - public void onCompleted(long duration, boolean fail) { - metrics.onQueryCompleted(duration, fail); + public void collectMetrics(GridCacheQueryType qryType, String qry, long startTime, long duration, boolean failed) { + metrics.update(duration, failed); + + if (detailMetricsSz > 0) { + // Do not collect metrics for EXPLAIN queries. + if (qryType == SQL_FIELDS && !F.isEmpty(qry)) { + int off = 0; + int len = qry.length(); + + while (off < len && Character.isWhitespace(qry.charAt(off))) + off++; + + if (qry.regionMatches(true, off, "EXPLAIN", 0, 7)) + return; + } + + GridCacheQueryDetailMetricsAdapter m = new GridCacheQueryDetailMetricsAdapter(qryType, qry, + cctx.name(), startTime, duration, failed); + + GridCacheQueryDetailMetricsKey key = m.key(); + + detailMetrics.merge(key, m, QRY_DETAIL_METRICS_MERGE_FX); + } } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java index e70ea9ff46222..0bb578f5deb62 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetricsAdapter.java @@ -100,32 +100,23 @@ public int completedExecutions() { } /** - * Callback for query execution. - * - * @param fail {@code True} query executed unsuccessfully {@code false} otherwise. - */ - public void onQueryExecute(boolean fail) { - execs.increment(); - - if (fail) - fails.increment(); - } - - /** - * Callback for completion of query execution. + * Update metrics. * * @param duration Duration of queue execution. * @param fail {@code True} query executed unsuccessfully {@code false} otherwise. */ - public void onQueryCompleted(long duration, boolean fail) { - minTime.setIfLess(duration); - maxTime.setIfGreater(duration); - - if (fail) + public void update(long duration, boolean fail) { + if (fail) { + execs.increment(); fails.increment(); + } else { + execs.increment(); completed.increment(); + minTime.setIfLess(duration); + maxTime.setIfGreater(duration); + sumTime.add(duration); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index 27c0b71fa5f38..8befa0e784e1e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -70,10 +70,13 @@ import org.apache.ignite.internal.processors.cache.CacheObjectContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheDefaultAffinityKeyMapper; +import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; import org.apache.ignite.internal.processors.cache.QueryCursorImpl; import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture; import org.apache.ignite.internal.processors.cache.query.CacheQueryType; +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType; +import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -111,6 +114,9 @@ public class GridQueryProcessor extends GridProcessorAdapter { /** */ private static final Class GEOMETRY_CLASS = U.classForName("com.vividsolutions.jts.geom.Geometry", null); + /** Queries detail metrics eviction frequency. */ + private static final int QRY_DETAIL_METRICS_EVICTION_FREQ = 3_000; + /** */ private static Set> SQL_TYPES = new HashSet<>(F.>asList( Integer.class, @@ -148,6 +154,9 @@ public class GridQueryProcessor extends GridProcessorAdapter { /** */ private final GridQueryIndexing idx; + /** */ + private GridTimeoutProcessor.CancelableTask qryDetailMetricsEvictTask; + /** */ private static final ThreadLocal requestTopVer = new ThreadLocal<>(); @@ -177,6 +186,14 @@ public GridQueryProcessor(GridKernalContext ctx) throws IgniteCheckedException { idx.start(ctx, busyLock); } + + // Schedule queries detail metrics eviction. + qryDetailMetricsEvictTask = ctx.timeout().schedule(new Runnable() { + @Override public void run() { + for (IgniteCacheProxy cache : ctx.cache().jcaches()) + cache.context().queries().evictDetailMetrics(); + } + }, QRY_DETAIL_METRICS_EVICTION_FREQ, QRY_DETAIL_METRICS_EVICTION_FREQ); } /** @@ -478,6 +495,8 @@ private void addTypeByName(CacheConfiguration ccfg, TypeDescriptor desc) th if (idx != null) idx.stop(); + + U.closeQuiet(qryDetailMetricsEvictTask); } /** {@inheritDoc} */ @@ -754,7 +773,7 @@ public GridCloseableIterator> query(final String spac try { final GridCacheContext cctx = ctx.cache().internalCache(space).context(); - return executeQuery(cctx, new IgniteOutClosureX>>() { + return executeQuery(GridCacheQueryType.SQL_FIELDS, clause, cctx, new IgniteOutClosureX>>() { @Override public GridCloseableIterator> applyx() throws IgniteCheckedException { TypeDescriptor type = typesByName.get(new TypeName(space, resType)); @@ -782,7 +801,7 @@ public QueryCursor> queryTwoStep(final GridCacheContext cctx, final throw new IllegalStateException("Failed to execute query (grid is stopping)."); try { - return executeQuery(cctx, new IgniteOutClosureX>>() { + return executeQuery(GridCacheQueryType.SQL_FIELDS, qry.getSql(), cctx, new IgniteOutClosureX>>() { @Override public QueryCursor> applyx() { return idx.queryTwoStep(cctx, qry); } @@ -808,11 +827,11 @@ public QueryCursor> queryTwoStep(final GridCacheContext>>() { + return executeQuery(GridCacheQueryType.SQL, qry.getSql(), cctx, new IgniteOutClosureX>>() { @Override public QueryCursor> applyx() throws IgniteCheckedException { return idx.queryTwoStep(cctx, qry); } - }, false); + }, true); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -837,8 +856,7 @@ public Iterator> queryLocal( throw new IllegalStateException("Failed to execute query (grid is stopping)."); try { - return executeQuery( - cctx, + return executeQuery(GridCacheQueryType.SQL, qry.getSql(), cctx, new IgniteOutClosureX>>() { @Override public Iterator> applyx() throws IgniteCheckedException { String space = cctx.name(); @@ -887,7 +905,7 @@ public Iterator> queryLocal( } }; } - }, false); + }, true); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -956,7 +974,7 @@ public QueryCursor> queryLocalFields(final GridCacheContext cctx, f try { final boolean keepBinary = cctx.keepBinary(); - return executeQuery(cctx, new IgniteOutClosureX>>() { + return executeQuery(GridCacheQueryType.SQL_FIELDS, qry.getSql(), cctx, new IgniteOutClosureX>>() { @Override public QueryCursor> applyx() throws IgniteCheckedException { final String space = cctx.name(); final String sql = qry.getSql(); @@ -1115,20 +1133,21 @@ public GridCloseableIterator> queryText(final String try { final GridCacheContext cctx = ctx.cache().internalCache(space).context(); - return executeQuery(cctx, new IgniteOutClosureX>>() { - @Override public GridCloseableIterator> applyx() throws IgniteCheckedException { - TypeDescriptor type = typesByName.get(new TypeName(space, resType)); + return executeQuery(GridCacheQueryType.TEXT, clause, cctx, + new IgniteOutClosureX>>() { + @Override public GridCloseableIterator> applyx() throws IgniteCheckedException { + TypeDescriptor type = typesByName.get(new TypeName(space, resType)); - if (type == null || !type.registered()) - throw new CacheException("Failed to find SQL table for type: " + resType); + if (type == null || !type.registered()) + throw new CacheException("Failed to find SQL table for type: " + resType); - return idx.queryLocalText( - space, - clause, - type, - filters); - } - }, false); + return idx.queryLocalText( + space, + clause, + type, + filters); + } + }, true); } finally { busyLock.leaveBusy(); @@ -1744,11 +1763,13 @@ public GridQueryTypeDescriptor type(@Nullable String space, String typeName) thr } /** + * @param qryType Query type. + * @param qry Query description. * @param cctx Cache context. * @param clo Closure. * @param complete Complete. */ - public R executeQuery(GridCacheContext cctx, IgniteOutClosureX clo, boolean complete) + public R executeQuery(GridCacheQueryType qryType, String qry, GridCacheContext cctx, IgniteOutClosureX clo, boolean complete) throws IgniteCheckedException { final long startTime = U.currentTimeMillis(); @@ -1783,30 +1804,18 @@ public R executeQuery(GridCacheContext cctx, IgniteOutClosureX clo, throw new IgniteCheckedException(e); } finally { - cctx.queries().onExecuted(err != null); - - if (complete && err == null) - onCompleted(cctx, res, null, startTime, U.currentTimeMillis() - startTime, log); - } - } + boolean failed = err != null; - /** - * @param cctx Cctx. - * @param res Result. - * @param err Err. - * @param startTime Start time. - * @param duration Duration. - * @param log Logger. - */ - public static void onCompleted(GridCacheContext cctx, Object res, Throwable err, - long startTime, long duration, IgniteLogger log) { - boolean fail = err != null; + long duration = U.currentTimeMillis() - startTime; - cctx.queries().onCompleted(duration, fail); + if (complete || failed) { + cctx.queries().collectMetrics(qryType, qry, startTime, duration, failed); - if (log.isTraceEnabled()) - log.trace("Query execution completed [startTime=" + startTime + - ", duration=" + duration + ", fail=" + fail + ", res=" + res + ']'); + if (log.isTraceEnabled()) + log.trace("Query execution [startTime=" + startTime + ", duration=" + duration + + ", fail=" + failed + ", res=" + res + ']'); + } + } } /** @@ -2485,6 +2494,11 @@ private TypeName(@Nullable String space, String typeName) { * The way to index. */ private enum IndexType { - ASC, DESC, TEXT + /** Ascending index. */ + ASC, + /** Descending index. */ + DESC, + /** Text index. */ + TEXT } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedPriorityQueue.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedPriorityQueue.java new file mode 100644 index 0000000000000..cbc521dc6c81a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridBoundedPriorityQueue.java @@ -0,0 +1,68 @@ +/* + * 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.util; + +import java.util.Comparator; +import java.util.PriorityQueue; +import org.apache.ignite.internal.util.typedef.internal.A; + +/** + * Bounded variant of {@link PriorityQueue}. + * + * @param Type of the queue element. + */ +public class GridBoundedPriorityQueue extends PriorityQueue { + /** */ + private static final long serialVersionUID = 0L; + + /** Queue max capacity. */ + private final int maxCap; + + /** Comparator. */ + private final Comparator cmp; + + /** + * Creates a bounded priority queue with the specified maximum size. + * At most {@code maxCap} elements would be kept in the queue. + * + * @param maxCap Maximum size of the queue. + * @param cmp Comparator that orders the elements. + */ + public GridBoundedPriorityQueue(int maxCap, Comparator cmp) { + super(maxCap, cmp); + + A.notNull(cmp, "comparator not null"); + + this.maxCap = maxCap; + this.cmp = cmp; + } + + /** {@inheritDoc} */ + @Override public boolean offer(E e) { + if (size() >= maxCap) { + E head = peek(); + + if (cmp.compare(e, head) <= 0) + return false; + + poll(); + } + + return super.offer(e); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryDetailMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryDetailMetrics.java new file mode 100644 index 0000000000000..dc469062c6d4d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryDetailMetrics.java @@ -0,0 +1,167 @@ +/* + * 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.visor.cache; + +import java.io.Serializable; +import org.apache.ignite.cache.query.QueryDetailMetrics; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Data transfer object for cache query detail metrics. + */ +public class VisorCacheQueryDetailMetrics implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** Query type. */ + private String qryType; + + /** Textual query representation. */ + private String qry; + + /** Cache name. */ + private String cache; + + /** Number of executions. */ + private int execs; + + /** Number of completions executions. */ + private int completions; + + /** Number of failures. */ + private int failures; + + /** Minimum time of execution. */ + private long minTime; + + /** Maximum time of execution. */ + private long maxTime; + + /** Average time of execution. */ + private double avgTime; + + /** Sum of execution time of completions time. */ + private long totalTime; + + /** Sum of execution time of completions time. */ + private long lastStartTime; + + /** + * @param m Cache query metrics. + * @return Data transfer object for given cache metrics. + */ + public VisorCacheQueryDetailMetrics from(QueryDetailMetrics m) { + qryType = m.queryType(); + qry = m.query(); + cache = m.cache(); + + execs = m.executions(); + completions = m.completions(); + failures = m.failures(); + + minTime = m.minimumTime(); + maxTime = m.maximumTime(); + avgTime = m.averageTime(); + totalTime = m.totalTime(); + lastStartTime = m.lastStartTime(); + + return this; + } + + /** + * @return Query type + */ + public String getQueryType() { + return qryType; + } + + /** + * @return Query type + */ + public String getQuery() { + return qry; + } + + /** + * @return Cache name where query was executed. + */ + public String getCache() { + return cache; + } + + /** + * @return Number of executions. + */ + public int getExecutions() { + return execs; + } + + /** + * @return Number of completed executions. + */ + public int getCompletions() { + return completions; + } + + /** + * @return Total number of times a query execution failed. + */ + public int getFailures() { + return failures; + } + + /** + * @return Minimum execution time of query. + */ + public long getMinimumTime() { + return minTime; + } + + /** + * @return Maximum execution time of query. + */ + public long getMaximumTime() { + return maxTime; + } + + /** + * @return Average execution time of query. + */ + public double getAverageTime() { + return avgTime; + } + + /** + * @return Total time of all query executions. + */ + public long getTotalTime() { + return totalTime; + } + + /** + * @return Latest time query was stared. + */ + public long getLastStartTime() { + return lastStartTime; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorCacheQueryDetailMetrics.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryDetailMetricsCollectorTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryDetailMetricsCollectorTask.java new file mode 100644 index 0000000000000..aee506320471b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryDetailMetricsCollectorTask.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.visor.cache; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.query.QueryDetailMetrics; +import org.apache.ignite.compute.ComputeJobResult; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.processors.cache.GridCacheProcessor; +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryDetailMetricsAdapter; +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryDetailMetricsKey; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorMultiNodeTask; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isIgfsCache; +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isSystemCache; + +/** + * Task to collect cache query metrics. + */ +@GridInternal +public class VisorCacheQueryDetailMetricsCollectorTask extends VisorMultiNodeTask, + Collection> { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorCacheQueryDetailMetricsCollectorJob job(Long arg) { + return new VisorCacheQueryDetailMetricsCollectorJob(arg, debug); + } + + /** {@inheritDoc} */ + @Nullable @Override protected Collection reduce0(List results) + throws IgniteException { + Map taskRes = new HashMap<>(); + + for (ComputeJobResult res : results) { + if (res.getException() != null) + throw res.getException(); + + Collection metrics = res.getData(); + + VisorCacheQueryDetailMetricsCollectorJob.aggregateMetrics(-1, taskRes, metrics); + } + + Collection aggMetrics = taskRes.values(); + + Collection res = new ArrayList<>(aggMetrics.size()); + + for (GridCacheQueryDetailMetricsAdapter m : aggMetrics) + res.add(new VisorCacheQueryDetailMetrics().from(m)); + + return res; + } + + /** + * Job that will actually collect query metrics. + */ + private static class VisorCacheQueryDetailMetricsCollectorJob extends VisorJob> { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Create job with specified argument. + * + * @param arg Last time when metrics were collected. + * @param debug Debug flag. + */ + protected VisorCacheQueryDetailMetricsCollectorJob(@Nullable Long arg, boolean debug) { + super(arg, debug); + } + + /** + * @param since Time when metrics were collected last time. + * @param res Response. + * @param metrics Metrics. + */ + private static void aggregateMetrics(long since, Map res, Collection metrics) { + for (GridCacheQueryDetailMetricsAdapter m : metrics) { + if (m.lastStartTime() > since) { + GridCacheQueryDetailMetricsKey key = m.key(); + + GridCacheQueryDetailMetricsAdapter aggMetrics = res.get(key); + + res.put(key, aggMetrics == null ? m : aggMetrics.aggregate(m)); + } + } + } + + /** {@inheritDoc} */ + @Override protected Collection run(@Nullable Long arg) throws IgniteException { + assert arg != null; + + IgniteConfiguration cfg = ignite.configuration(); + + GridCacheProcessor cacheProc = ignite.context().cache(); + + Collection cacheNames = cacheProc.cacheNames(); + + Map aggMetrics = new HashMap<>(); + + for (String cacheName : cacheNames) { + if (!isSystemCache(cacheName) && !isIgfsCache(cfg, cacheName)) { + IgniteInternalCache cache = cacheProc.cache(cacheName); + + if (cache == null || !cache.context().started()) + continue; + + aggregateMetrics(arg, aggMetrics, cache.context().queries().detailMetrics()); + } + } + + return new ArrayList<>(aggMetrics.values()); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorCacheQueryDetailMetricsCollectorJob.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetQueryDetailMetricsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetQueryDetailMetricsTask.java new file mode 100644 index 0000000000000..84aae9db51bbf --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetQueryDetailMetricsTask.java @@ -0,0 +1,71 @@ +/* + * 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.visor.cache; + +import org.apache.ignite.IgniteCache; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorOneNodeTask; + +/** + * Reset query detail metrics. + */ +@GridInternal +public class VisorCacheResetQueryDetailMetricsTask extends VisorOneNodeTask { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorCacheResetQueryDetailMetricsJob job(Void arg) { + return new VisorCacheResetQueryDetailMetricsJob(arg, debug); + } + + /** + * Job that reset query detail metrics. + */ + private static class VisorCacheResetQueryDetailMetricsJob extends VisorJob { + /** */ + private static final long serialVersionUID = 0L; + + /** + * @param arg Task argument. + * @param debug Debug flag. + */ + private VisorCacheResetQueryDetailMetricsJob(Void arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected Void run(Void arg) { + for (String cacheName : ignite.cacheNames()) { + IgniteCache cache = ignite.cache(cacheName); + + if (cache != null) + cache.resetQueryDetailMetrics(); + } + + return null; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorCacheResetQueryDetailMetricsJob.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetQueryMetricsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetQueryMetricsTask.java index 96d98573e9f8d..5ba6156bc1bbb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetQueryMetricsTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetQueryMetricsTask.java @@ -17,7 +17,7 @@ package org.apache.ignite.internal.visor.cache; -import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.IgniteCache; import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.visor.VisorJob; @@ -53,10 +53,10 @@ private VisorCacheResetQueryMetricsJob(String arg, boolean debug) { /** {@inheritDoc} */ @Override protected Void run(String cacheName) { - IgniteInternalCache cache = ignite.cachex(cacheName); + IgniteCache cache = ignite.cache(cacheName); if (cache != null) - cache.context().queries().resetMetrics(); + cache.resetQueryMetrics(); return null; } @@ -66,4 +66,4 @@ private VisorCacheResetQueryMetricsJob(String arg, boolean debug) { return S.toString(VisorCacheResetQueryMetricsJob.class, this); } } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java index 25aaab50a92ab..1e9346ced5654 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java @@ -158,8 +158,8 @@ public class VisorTaskUtils { * @param name Grid-style nullable name. * @return Name with {@code null} replaced to <default>. */ - public static String escapeName(@Nullable String name) { - return name == null ? DFLT_EMPTY_NAME : name; + public static String escapeName(@Nullable Object name) { + return name == null ? DFLT_EMPTY_NAME : name.toString(); } /** @@ -172,15 +172,6 @@ public static String unescapeName(String name) { return DFLT_EMPTY_NAME.equals(name) ? null : name; } - /** - * @param a First name. - * @param b Second name. - * @return {@code true} if both names equals. - */ - public static boolean safeEquals(@Nullable String a, @Nullable String b) { - return (a != null && b != null) ? a.equals(b) : (a == null && b == null); - } - /** * Concat arrays in one. * @@ -637,12 +628,10 @@ public static VisorFileBlock readBlock(File file, long off, int blockSz, long la else { int toRead = Math.min(blockSz, (int)(fSz - pos)); - byte[] buf = new byte[toRead]; - raf = new RandomAccessFile(file, "r"); - raf.seek(pos); + byte[] buf = new byte[toRead]; int cntRead = raf.read(buf, 0, toRead); if (cntRead != toRead) @@ -887,8 +876,6 @@ public static List startLocalNode(@Nullable IgniteLogger log, String cf try { for (int i = 0; i < nodesToStart; i++) { if (U.isMacOs()) { - StringBuilder envs = new StringBuilder(); - Map macEnv = new HashMap<>(System.getenv()); if (envVars != null) { @@ -905,6 +892,8 @@ public static List startLocalNode(@Nullable IgniteLogger log, String cf macEnv.put(ent.getKey(), ent.getValue()); } + StringBuilder envs = new StringBuilder(); + for (Map.Entry entry : macEnv.entrySet()) { String val = entry.getValue(); @@ -1024,13 +1013,12 @@ public static byte[] zipBytes(byte[] input, int initBufSize) throws IOException ByteArrayOutputStream bos = new ByteArrayOutputStream(initBufSize); try (ZipOutputStream zos = new ZipOutputStream(bos)) { - ZipEntry entry = new ZipEntry(""); - try { + ZipEntry entry = new ZipEntry(""); + entry.setSize(input.length); zos.putNextEntry(entry); - zos.write(input); } finally { diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties index ade21ceb1509f..0d291cca3e6d2 100644 --- a/modules/core/src/main/resources/META-INF/classnames.properties +++ b/modules/core/src/main/resources/META-INF/classnames.properties @@ -75,11 +75,13 @@ org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy$HolderComparator org.apache.ignite.cache.query.CacheQueryEntryEvent org.apache.ignite.cache.query.ContinuousQuery org.apache.ignite.cache.query.Query +org.apache.ignite.cache.query.QueryCancelledException org.apache.ignite.cache.query.ScanQuery org.apache.ignite.cache.query.SpiQuery org.apache.ignite.cache.query.SqlFieldsQuery org.apache.ignite.cache.query.SqlQuery org.apache.ignite.cache.query.TextQuery +org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$2 org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$EntryMapping$1 org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$EntryMapping$2 org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$TypeKind @@ -366,6 +368,7 @@ org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakQu org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakReferenceCloseableIterator org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest +org.apache.ignite.internal.processors.cache.EntryProcessorResourceInjectorProxy org.apache.ignite.internal.processors.cache.GridCacheAdapter org.apache.ignite.internal.processors.cache.GridCacheAdapter$10 org.apache.ignite.internal.processors.cache.GridCacheAdapter$11 @@ -373,9 +376,9 @@ org.apache.ignite.internal.processors.cache.GridCacheAdapter$12 org.apache.ignite.internal.processors.cache.GridCacheAdapter$13 org.apache.ignite.internal.processors.cache.GridCacheAdapter$14 org.apache.ignite.internal.processors.cache.GridCacheAdapter$15 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$16$1 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$16 org.apache.ignite.internal.processors.cache.GridCacheAdapter$17 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$18 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$18$1 org.apache.ignite.internal.processors.cache.GridCacheAdapter$2 org.apache.ignite.internal.processors.cache.GridCacheAdapter$26$1 org.apache.ignite.internal.processors.cache.GridCacheAdapter$28 @@ -533,6 +536,7 @@ org.apache.ignite.internal.processors.cache.GridCacheUtils$8 org.apache.ignite.internal.processors.cache.GridCacheUtils$9 org.apache.ignite.internal.processors.cache.GridCacheValueCollection org.apache.ignite.internal.processors.cache.GridCacheValueCollection$1 +org.apache.ignite.internal.processors.cache.GridDeferredAckMessageSender$DeferredAckMessageBuffer org.apache.ignite.internal.processors.cache.IgniteCacheProxy org.apache.ignite.internal.processors.cache.IgniteCacheProxy$1 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$10 @@ -545,12 +549,13 @@ org.apache.ignite.internal.processors.cache.IgniteCacheProxy$8 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$9 org.apache.ignite.internal.processors.cache.KeyCacheObject org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl +org.apache.ignite.internal.processors.cache.QueryCursorImpl$State org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityProxy org.apache.ignite.internal.processors.cache.binary.BinaryMetadataKey org.apache.ignite.internal.processors.cache.binary.CacheDefaultBinaryAffinityKeyMapper org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$1 -org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$4 org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$5 +org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$6 org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$MetaDataEntryFilter org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$MetaDataPredicate org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$MetadataProcessor @@ -630,6 +635,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocal$2 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter$1 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter$2 +org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxOnePhaseCommitAckRequest org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$1 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$2 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$3 @@ -670,25 +676,23 @@ org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomic org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$27 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$28 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$29 -org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$3 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$30 +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$31 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$4 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$5 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$6 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$7 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$8 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$9 -org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$DeferredResponseBuffer org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateFuture$1 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture$1 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture$2 -org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture$3 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$1 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$2 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$3 -org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$4 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedCache @@ -760,6 +764,7 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$ org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$2 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$3 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$4 +org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$LockTimeoutObject$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$MiniFuture$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockResponse @@ -772,6 +777,7 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticS org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$2 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$3 +org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$4 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$MiniFuture$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFutureAdapter$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearPessimisticTxPrepareFuture$1 @@ -821,22 +827,23 @@ org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManag org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$7 org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$1 org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$ScanQueryFallbackClosableIterator +org.apache.ignite.internal.processors.cache.query.GridCacheQueryDetailMetricsAdapter org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$1 org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$2 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$1$1 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$1$2 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$10 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$11 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$12 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$13 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$14 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$15$1 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$2 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$3 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$15 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$16 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$17$1 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$3$1 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$3$2 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$4 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$5 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$6 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$7 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$8 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$9 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$CacheSqlIndexMetadata org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$CacheSqlMetadata org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$CachedResult$QueueIterator @@ -896,6 +903,8 @@ org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$12 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$13 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$14 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$15 +org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$16 +org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$17 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$2 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$3 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$4 @@ -929,9 +938,9 @@ org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$Po org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostLockClosure1$4 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostLockClosure2 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostMissClosure -org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$2 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$3 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$4 +org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$5 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$CommitListener org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$CommittedVersion org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$NodeFailureTimeoutObject$1 @@ -1005,7 +1014,6 @@ org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$1 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$4 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$5 -org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$6 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$Buffer$1 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$Buffer$2 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$DataStreamerPda @@ -1084,6 +1092,7 @@ org.apache.ignite.internal.processors.datastructures.GridCacheSetProxy org.apache.ignite.internal.processors.datastructures.GridSetQueryPredicate org.apache.ignite.internal.processors.dr.GridDrType org.apache.ignite.internal.processors.dr.IgniteDrDataStreamerCacheUpdater +org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo org.apache.ignite.internal.processors.hadoop.HadoopFileBlock org.apache.ignite.internal.processors.hadoop.HadoopInputSplit org.apache.ignite.internal.processors.hadoop.HadoopJobId @@ -1119,7 +1128,6 @@ org.apache.ignite.internal.processors.igfs.IgfsFragmentizerManager$IdentityHashS org.apache.ignite.internal.processors.igfs.IgfsFragmentizerRequest org.apache.ignite.internal.processors.igfs.IgfsFragmentizerResponse org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse -org.apache.ignite.internal.processors.igfs.IgfsImpl$12$1 org.apache.ignite.internal.processors.igfs.IgfsImpl$IgfsGlobalSpaceTask org.apache.ignite.internal.processors.igfs.IgfsImpl$IgfsGlobalSpaceTask$1 org.apache.ignite.internal.processors.igfs.IgfsInputStreamDescriptor @@ -1155,6 +1163,7 @@ org.apache.ignite.internal.processors.igfs.client.IgfsClientSummaryCallable org.apache.ignite.internal.processors.igfs.client.IgfsClientUpdateCallable org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaIdsForPathCallable org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaInfoForPathCallable +org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaUnlockCallable org.apache.ignite.internal.processors.igfs.data.IgfsDataPutProcessor org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingAddProcessor @@ -1172,6 +1181,8 @@ org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdateTimesProcessor org.apache.ignite.internal.processors.job.GridJobProcessor$5 org.apache.ignite.internal.processors.job.GridJobWorker$3 org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor$SnapshotReducer +org.apache.ignite.internal.processors.odbc.OdbcProtocolVersion +org.apache.ignite.internal.processors.odbc.escape.OdbcEscapeType org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate org.apache.ignite.internal.processors.platform.PlatformEventFilterListener @@ -1180,7 +1191,7 @@ org.apache.ignite.internal.processors.platform.PlatformExtendedException org.apache.ignite.internal.processors.platform.PlatformJavaObjectFactoryProxy org.apache.ignite.internal.processors.platform.PlatformNativeException org.apache.ignite.internal.processors.platform.PlatformNoCallbackException -org.apache.ignite.internal.processors.platform.cache.PlatformCache$1 +org.apache.ignite.internal.processors.platform.cache.PlatformCache$5 org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilterImpl org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessor @@ -1211,6 +1222,8 @@ org.apache.ignite.internal.processors.platform.cpp.PlatformCppConfigurationClosu org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiver org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiverImpl org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$1 +org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$10 +org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$11 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$2 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$3 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$4 @@ -1218,6 +1231,7 @@ org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$5 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$6 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$7 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$8 +org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$9 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetConfigurationClosure org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetService org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetServiceImpl @@ -1232,19 +1246,24 @@ org.apache.ignite.internal.processors.platform.transactions.PlatformTransactions org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$1 org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$FutureListenable$1 org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$InternalFutureListenable$1 +org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionLockProcessor +org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionSetAndUnlockProcessor org.apache.ignite.internal.processors.query.GridQueryFieldMetadata org.apache.ignite.internal.processors.query.GridQueryIndexType -org.apache.ignite.internal.processors.query.GridQueryProcessor$2 org.apache.ignite.internal.processors.query.GridQueryProcessor$3 org.apache.ignite.internal.processors.query.GridQueryProcessor$4 org.apache.ignite.internal.processors.query.GridQueryProcessor$5 org.apache.ignite.internal.processors.query.GridQueryProcessor$6 +org.apache.ignite.internal.processors.query.GridQueryProcessor$7 +org.apache.ignite.internal.processors.query.GridQueryProcessor$8 org.apache.ignite.internal.processors.query.GridQueryProcessor$IndexType org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageResponse org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryRequest +org.apache.ignite.internal.processors.resource.GridResourceIoc$AnnotationSet +org.apache.ignite.internal.processors.resource.GridResourceIoc$ResourceAnnotation org.apache.ignite.internal.processors.rest.GridRestCommand org.apache.ignite.internal.processors.rest.GridRestProcessor$2$1 org.apache.ignite.internal.processors.rest.GridRestProcessor$3 @@ -1332,7 +1351,7 @@ org.apache.ignite.internal.processors.service.ServiceDescriptorImpl org.apache.ignite.internal.processors.task.GridTaskProcessor$1 org.apache.ignite.internal.processors.task.GridTaskThreadContextKey org.apache.ignite.internal.processors.task.GridTaskWorker$3 -org.apache.ignite.internal.processors.task.GridTaskWorker$4 +org.apache.ignite.internal.processors.task.GridTaskWorker$5 org.apache.ignite.internal.processors.task.GridTaskWorker$State org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException @@ -1356,6 +1375,7 @@ org.apache.ignite.internal.util.GridBoundedConcurrentOrderedMap org.apache.ignite.internal.util.GridBoundedConcurrentOrderedSet org.apache.ignite.internal.util.GridBoundedLinkedHashMap org.apache.ignite.internal.util.GridBoundedLinkedHashSet +org.apache.ignite.internal.util.GridBoundedPriorityQueue org.apache.ignite.internal.util.GridByteArrayList org.apache.ignite.internal.util.GridCloseableIteratorAdapter org.apache.ignite.internal.util.GridCloseableIteratorAdapterEx @@ -1407,14 +1427,14 @@ org.apache.ignite.internal.util.IgniteUtils$11 org.apache.ignite.internal.util.IgniteUtils$12 org.apache.ignite.internal.util.IgniteUtils$13 org.apache.ignite.internal.util.IgniteUtils$14 -org.apache.ignite.internal.util.IgniteUtils$16 -org.apache.ignite.internal.util.IgniteUtils$2 -org.apache.ignite.internal.util.IgniteUtils$22 +org.apache.ignite.internal.util.IgniteUtils$15 +org.apache.ignite.internal.util.IgniteUtils$17 org.apache.ignite.internal.util.IgniteUtils$23 org.apache.ignite.internal.util.IgniteUtils$24 org.apache.ignite.internal.util.IgniteUtils$25 org.apache.ignite.internal.util.IgniteUtils$26 org.apache.ignite.internal.util.IgniteUtils$27 +org.apache.ignite.internal.util.IgniteUtils$28 org.apache.ignite.internal.util.IgniteUtils$3 org.apache.ignite.internal.util.IgniteUtils$4 org.apache.ignite.internal.util.IgniteUtils$5 @@ -1647,12 +1667,19 @@ org.apache.ignite.internal.visor.cache.VisorCachePartitionsTask org.apache.ignite.internal.visor.cache.VisorCachePartitionsTask$VisorCachePartitionsJob org.apache.ignite.internal.visor.cache.VisorCacheQueryConfiguration org.apache.ignite.internal.visor.cache.VisorCacheQueryConfigurationV2 +org.apache.ignite.internal.visor.cache.VisorCacheQueryDetailMetrics +org.apache.ignite.internal.visor.cache.VisorCacheQueryDetailMetricsCollectorTask +org.apache.ignite.internal.visor.cache.VisorCacheQueryDetailMetricsCollectorTask$VisorCacheQueryDetailMetricsCollectorJob org.apache.ignite.internal.visor.cache.VisorCacheQueryMetrics org.apache.ignite.internal.visor.cache.VisorCacheRebalanceConfiguration org.apache.ignite.internal.visor.cache.VisorCacheRebalanceTask org.apache.ignite.internal.visor.cache.VisorCacheRebalanceTask$VisorCachesRebalanceJob org.apache.ignite.internal.visor.cache.VisorCacheResetMetricsTask org.apache.ignite.internal.visor.cache.VisorCacheResetMetricsTask$VisorCacheResetMetricsJob +org.apache.ignite.internal.visor.cache.VisorCacheResetQueryDetailMetricsTask +org.apache.ignite.internal.visor.cache.VisorCacheResetQueryDetailMetricsTask$VisorCacheResetQueryDetailMetricsJob +org.apache.ignite.internal.visor.cache.VisorCacheResetQueryMetricsTask +org.apache.ignite.internal.visor.cache.VisorCacheResetQueryMetricsTask$VisorCacheResetQueryMetricsJob org.apache.ignite.internal.visor.cache.VisorCacheStartTask org.apache.ignite.internal.visor.cache.VisorCacheStartTask$VisorCacheStartArg org.apache.ignite.internal.visor.cache.VisorCacheStartTask$VisorCacheStartJob @@ -1682,6 +1709,7 @@ org.apache.ignite.internal.visor.debug.VisorThreadLockInfo org.apache.ignite.internal.visor.debug.VisorThreadMonitorInfo org.apache.ignite.internal.visor.event.VisorGridDeploymentEvent org.apache.ignite.internal.visor.event.VisorGridDiscoveryEvent +org.apache.ignite.internal.visor.event.VisorGridDiscoveryEventV2 org.apache.ignite.internal.visor.event.VisorGridEvent org.apache.ignite.internal.visor.event.VisorGridEventsLost org.apache.ignite.internal.visor.event.VisorGridJobEvent @@ -1769,6 +1797,7 @@ org.apache.ignite.internal.visor.util.VisorClusterGroupEmptyException org.apache.ignite.internal.visor.util.VisorEventMapper org.apache.ignite.internal.visor.util.VisorExceptionWrapper org.apache.ignite.internal.visor.util.VisorTaskUtils$4 +org.apache.ignite.internal.visor.util.VisorTaskUtils$5 org.apache.ignite.internal.websession.WebSessionAttributeProcessor org.apache.ignite.internal.websession.WebSessionEntity org.apache.ignite.lang.IgniteBiClosure diff --git a/modules/core/src/test/java/org/apache/ignite/lang/utils/GridBoundedPriorityQueueSelfTest.java b/modules/core/src/test/java/org/apache/ignite/lang/utils/GridBoundedPriorityQueueSelfTest.java new file mode 100644 index 0000000000000..bcaecc4adcd10 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/lang/utils/GridBoundedPriorityQueueSelfTest.java @@ -0,0 +1,114 @@ +/* + * 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.lang.utils; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Random; + +import org.apache.ignite.internal.util.GridBoundedPriorityQueue; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.testframework.junits.common.GridCommonTest; + +/** + * Test for {@link GridBoundedPriorityQueue}. + */ +@GridCommonTest(group = "Lang") +public class GridBoundedPriorityQueueSelfTest extends GridCommonAbstractTest { + /** Queue items comparator. */ + private static final Comparator CMP = new Comparator() { + @Override public int compare(Integer o1, Integer o2) { + return Integer.compare(o1, o2); + } + }; + + /** Queue items reverse order comparator. */ + private static final Comparator CMP_REVERSE = Collections.reverseOrder(CMP); + + /** + * Test eviction in bounded priority queue. + */ + public void testEviction() { + GridBoundedPriorityQueue queue = new GridBoundedPriorityQueue<>(3, CMP); + + assertTrue(queue.offer(2)); + assertTrue(queue.offer(1)); + assertTrue(queue.offer(3)); + assertTrue(queue.offer(4)); + + assertEquals(3, queue.size()); + + assertFalse(queue.offer(0)); // Item with lower priority should not be put into queue. + + assertEquals(3, queue.size()); + + assertEquals(Integer.valueOf(2), queue.poll()); + assertEquals(Integer.valueOf(3), queue.poll()); + assertEquals(Integer.valueOf(4), queue.poll()); + + assertNull(queue.poll()); + assertTrue(queue.isEmpty()); + + // Test queue with reverse comparator. + GridBoundedPriorityQueue queueReverse = new GridBoundedPriorityQueue<>(3, CMP_REVERSE); + + assertTrue(queueReverse.offer(2)); + assertTrue(queueReverse.offer(1)); + assertTrue(queueReverse.offer(3)); + + assertFalse(queueReverse.offer(4)); // Item with lower priority should not be put into queue. + + assertEquals(Integer.valueOf(3), queueReverse.poll()); + assertEquals(Integer.valueOf(2), queueReverse.poll()); + assertEquals(Integer.valueOf(1), queueReverse.poll()); + + assertNull(queueReverse.poll()); + assertTrue(queueReverse.isEmpty()); + + // Test put random 100 items into GridBoundedPriorityQueue(10) and check result. + queue = new GridBoundedPriorityQueue<>(10, CMP); + queueReverse = new GridBoundedPriorityQueue<>(10, CMP_REVERSE); + + Random rnd = new Random(); + + List items = new ArrayList<>(100); + + for (int i = 0; i < 100; i++) { + Integer item = rnd.nextInt(100); + + items.add(item); + queue.offer(item); + queueReverse.offer(item); + } + + Collections.sort(items); + + for (int i = 0; i < 10; i++) { + assertEquals(items.get(90 + i), queue.poll()); + assertEquals(items.get(9 - i), queueReverse.poll()); + } + + assertNull(queue.poll()); + assertTrue(queue.isEmpty()); + + assertNull(queueReverse.poll()); + assertTrue(queueReverse.isEmpty()); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java index 71dc96453226e..46053b32c2ecc 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java @@ -40,6 +40,7 @@ import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.query.Query; import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.QueryDetailMetrics; import org.apache.ignite.cache.query.QueryMetrics; import org.apache.ignite.cluster.ClusterGroup; import org.apache.ignite.internal.processors.cache.CacheEntryImpl; @@ -187,6 +188,21 @@ private IgniteCacheProcessProxy(String name, boolean async, ExpiryPolicy plc, Ig throw new UnsupportedOperationException("Method should be supported."); } + /** {@inheritDoc} */ + @Override public void resetQueryMetrics() { + throw new UnsupportedOperationException("Method should be supported."); + } + + /** {@inheritDoc} */ + @Override public Collection queryDetailMetrics() { + throw new UnsupportedOperationException("Method should be supported."); + } + + /** {@inheritDoc} */ + @Override public void resetQueryDetailMetrics() { + throw new UnsupportedOperationException("Method should be supported."); + } + /** {@inheritDoc} */ @Override public void localEvict(Collection keys) { compute.call(new LocalEvictTask<>(cacheName, isAsync, keys)); diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteLangSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteLangSelfTestSuite.java index 78c0ab4fe5448..cfec1ec417fb1 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteLangSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteLangSelfTestSuite.java @@ -32,6 +32,7 @@ import org.apache.ignite.lang.IgniteUuidSelfTest; import org.apache.ignite.lang.utils.GridBoundedConcurrentLinkedHashMapSelfTest; import org.apache.ignite.lang.utils.GridBoundedConcurrentOrderedMapSelfTest; +import org.apache.ignite.lang.utils.GridBoundedPriorityQueueSelfTest; import org.apache.ignite.lang.utils.GridCircularBufferSelfTest; import org.apache.ignite.lang.utils.GridConcurrentLinkedHashMapSelfTest; import org.apache.ignite.lang.utils.GridConcurrentWeakHashSetSelfTest; @@ -55,6 +56,7 @@ public static TestSuite suite() throws Exception { TestSuite suite = new TestSuite("Ignite Lang Test Suite"); suite.addTest(new TestSuite(GridTupleSelfTest.class)); + suite.addTest(new TestSuite(GridBoundedPriorityQueueSelfTest.class)); suite.addTest(new TestSuite(GridByteArrayListSelfTest.class)); suite.addTest(new TestSuite(GridLeanMapSelfTest.class)); suite.addTest(new TestSuite(GridLeanIdentitySetSelfTest.class)); @@ -85,4 +87,4 @@ public static TestSuite suite() throws Exception { return suite; } -} \ No newline at end of file +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryDetailMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryDetailMetricsSelfTest.java new file mode 100644 index 0000000000000..1b870e48b8a8e --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryDetailMetricsSelfTest.java @@ -0,0 +1,572 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.ArrayList; +import java.util.Collection; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.query.Query; +import org.apache.ignite.cache.query.QueryDetailMetrics; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.cache.query.TextQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; + +/** + * Tests for cache query details metrics. + */ +public abstract class CacheAbstractQueryDetailMetricsSelfTest extends GridCommonAbstractTest { + /** */ + private static final int QRY_DETAIL_METRICS_SIZE = 3; + + /** Grid count. */ + protected int gridCnt; + + /** Cache mode. */ + protected CacheMode cacheMode; + + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + startGridsMultiThreaded(gridCnt); + + IgniteCache cacheA = grid(0).cache("A"); + IgniteCache cacheB = grid(0).cache("B"); + + for (int i = 0; i < 100; i++) { + cacheA.put(i, String.valueOf(i)); + cacheB.put(i, String.valueOf(i)); + } + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @param cacheName Cache name. + * @return Cache configuration. + */ + private CacheConfiguration configureCache(String cacheName) { + CacheConfiguration ccfg = defaultCacheConfiguration(); + + ccfg.setName(cacheName); + ccfg.setCacheMode(cacheMode); + ccfg.setWriteSynchronizationMode(FULL_SYNC); + ccfg.setIndexedTypes(Integer.class, String.class); + ccfg.setStatisticsEnabled(true); + ccfg.setQueryDetailMetricsSize(QRY_DETAIL_METRICS_SIZE); + + return ccfg; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setIpFinder(ipFinder); + + cfg.setDiscoverySpi(disco); + + cfg.setCacheConfiguration(configureCache("A"), configureCache("B")); + + return cfg; + } + + /** + * Test metrics for SQL fields queries. + * + * @throws Exception In case of error. + */ + public void testSqlFieldsQueryMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + SqlFieldsQuery qry = new SqlFieldsQuery("select * from String"); + + checkQueryMetrics(cache, qry); + } + + /** + * Test metrics for SQL fields queries. + * + * @throws Exception In case of error. + */ + public void testSqlFieldsQueryNotFullyFetchedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + SqlFieldsQuery qry = new SqlFieldsQuery("select * from String"); + qry.setPageSize(10); + + checkQueryNotFullyFetchedMetrics(cache, qry, false); + } + + /** + * Test metrics for failed SQL queries. + * + * @throws Exception In case of error. + */ + public void testSqlFieldsQueryFailedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + SqlFieldsQuery qry = new SqlFieldsQuery("select * from UNKNOWN"); + + checkQueryFailedMetrics(cache, qry); + } + + /** + * Test metrics eviction. + * + * @throws Exception In case of error. + */ + public void testQueryMetricsEviction() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + // Execute several DIFFERENT queries with guaranteed DIFFERENT time of execution. + cache.query(new SqlFieldsQuery("select * from String")).getAll(); + Thread.sleep(100); + + cache.query(new SqlFieldsQuery("select count(*) from String")).getAll(); + Thread.sleep(100); + + cache.query(new SqlFieldsQuery("select * from String limit 1")).getAll(); + Thread.sleep(100); + + cache.query(new SqlFieldsQuery("select * from String limit 2")).getAll(); + Thread.sleep(100); + + cache.query(new ScanQuery()).getAll(); + Thread.sleep(100); + + cache.query(new SqlQuery("String", "from String")).getAll(); + + waitingFor(cache, "size", QRY_DETAIL_METRICS_SIZE); + + for (int i = 0; i < QRY_DETAIL_METRICS_SIZE; i++) + checkMetrics(cache, QRY_DETAIL_METRICS_SIZE, i, 1, 1, 0, false); + + // Check that collected metrics contains correct items: metrics for last 3 queries. + Collection metrics = cache.queryDetailMetrics(); + + String lastMetrics = ""; + + for (QueryDetailMetrics m : metrics) + lastMetrics += m.queryType() + " " + m.query() + ";"; + + assertTrue(lastMetrics.contains("SQL_FIELDS select * from String limit 2;")); + assertTrue(lastMetrics.contains("SCAN A;")); + assertTrue(lastMetrics.contains("SQL from String;")); + + cache = grid(0).context().cache().jcache("B"); + + cache.query(new SqlFieldsQuery("select * from String")).getAll(); + cache.query(new SqlFieldsQuery("select count(*) from String")).getAll(); + cache.query(new SqlFieldsQuery("select * from String limit 1")).getAll(); + cache.query(new SqlFieldsQuery("select * from String limit 2")).getAll(); + cache.query(new ScanQuery()).getAll(); + cache.query(new SqlQuery("String", "from String")).getAll(); + + waitingFor(cache, "size", QRY_DETAIL_METRICS_SIZE); + + for (int i = 0; i < QRY_DETAIL_METRICS_SIZE; i++) + checkMetrics(cache, QRY_DETAIL_METRICS_SIZE, i, 1, 1, 0, false); + + if (gridCnt > 1) { + cache = grid(1).context().cache().jcache("A"); + + cache.query(new SqlFieldsQuery("select * from String")).getAll(); + cache.query(new SqlFieldsQuery("select count(*) from String")).getAll(); + cache.query(new SqlFieldsQuery("select * from String limit 1")).getAll(); + cache.query(new SqlFieldsQuery("select * from String limit 2")).getAll(); + cache.query(new ScanQuery()).getAll(); + cache.query(new SqlQuery("String", "from String")).getAll(); + + waitingFor(cache, "size", QRY_DETAIL_METRICS_SIZE); + + for (int i = 0; i < QRY_DETAIL_METRICS_SIZE; i++) + checkMetrics(cache, QRY_DETAIL_METRICS_SIZE, i, 1, 1, 0, false); + + cache = grid(1).context().cache().jcache("B"); + + cache.query(new SqlFieldsQuery("select * from String")).getAll(); + cache.query(new SqlFieldsQuery("select count(*) from String")).getAll(); + cache.query(new SqlFieldsQuery("select * from String limit 1")).getAll(); + cache.query(new SqlFieldsQuery("select * from String limit 2")).getAll(); + cache.query(new ScanQuery()).getAll(); + cache.query(new SqlQuery("String", "from String")).getAll(); + + waitingFor(cache, "size", QRY_DETAIL_METRICS_SIZE); + + for (int i = 0; i < QRY_DETAIL_METRICS_SIZE; i++) + checkMetrics(cache, QRY_DETAIL_METRICS_SIZE, i, 1, 1, 0, false); + } + } + + /** */ + private static class Worker extends Thread { + /** */ + private final IgniteCache cache; + + /** */ + private final Query qry; + + /** */ + Worker(IgniteCache cache, Query qry) { + this.cache = cache; + this.qry = qry; + } + + /** {@inheritDoc} */ + @Override public void run() { + cache.query(qry).getAll(); + } + } + + /** + * Test metrics if queries executed from several threads. + * + * @throws Exception In case of error. + */ + public void testQueryMetricsMultithreaded() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + Collection workers = new ArrayList<>(); + + int repeat = 10; + + for (int k = 0; k < repeat; k++) { + // Execute as match queries as history size to avoid eviction. + for (int i = 1; i <= QRY_DETAIL_METRICS_SIZE; i++) + workers.add(new Worker(cache, new SqlFieldsQuery("select * from String limit " + i))); + } + + for (Worker worker : workers) + worker.start(); + + for (Worker worker : workers) + worker.join(); + + for (int i = 0; i < QRY_DETAIL_METRICS_SIZE; i++) + checkMetrics(cache, QRY_DETAIL_METRICS_SIZE, i, repeat, repeat, 0, false); + } + + /** + * Test metrics for Scan queries. + * + * @throws Exception In case of error. + */ + public void testScanQueryMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + ScanQuery qry = new ScanQuery<>(); + + checkQueryMetrics(cache, qry); + } + + /** + * Test metrics for Scan queries. + * + * @throws Exception In case of error. + */ + public void testScanQueryNotFullyFetchedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + ScanQuery qry = new ScanQuery<>(); + qry.setPageSize(10); + + checkQueryNotFullyFetchedMetrics(cache, qry, true); + } + + /** + * Test metrics for failed Scan queries. + * + * @throws Exception In case of error. + */ + public void testScanQueryFailedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + ScanQuery qry = new ScanQuery<>(Integer.MAX_VALUE); + + checkQueryFailedMetrics(cache, qry); + } + + /** + * Test metrics for Scan queries. + * + * @throws Exception In case of error. + */ + public void testSqlQueryMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + SqlQuery qry = new SqlQuery<>("String", "from String"); + + checkQueryMetrics(cache, qry); + } + + /** + * Test metrics for Scan queries. + * + * @throws Exception In case of error. + */ + public void testSqlQueryNotFullyFetchedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + SqlQuery qry = new SqlQuery<>("String", "from String"); + qry.setPageSize(10); + + checkQueryNotFullyFetchedMetrics(cache, qry, true); + } + + /** + * Test metrics for failed Scan queries. + * + * @throws Exception In case of error. + */ + public void testSqlQueryFailedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + SqlQuery qry = new SqlQuery<>("Long", "from Long"); + + checkQueryFailedMetrics(cache, qry); + } + + /** + * Test metrics for Sql queries. + * + * @throws Exception In case of error. + */ + public void testTextQueryMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + TextQuery qry = new TextQuery<>("String", "1"); + + checkQueryMetrics(cache, qry); + } + + /** + * Test metrics for Sql queries. + * + * @throws Exception In case of error. + */ + public void testTextQueryNotFullyFetchedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + TextQuery qry = new TextQuery<>("String", "1"); + qry.setPageSize(10); + + checkQueryNotFullyFetchedMetrics(cache, qry, true); + } + + /** + * Test metrics for failed Scan queries. + * + * @throws Exception In case of error. + */ + public void testTextQueryFailedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + TextQuery qry = new TextQuery<>("Unknown", "zzz"); + + checkQueryFailedMetrics(cache, qry); + } + + /** + * Test metrics for SQL cross cache queries. + * + * @throws Exception In case of error. + */ + public void testSqlFieldsCrossCacheQueryMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"B\".String"); + + checkQueryMetrics(cache, qry); + } + + /** + * Test metrics for SQL cross cache queries. + * + * @throws Exception In case of error. + */ + public void testSqlFieldsCrossCacheQueryNotFullyFetchedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"B\".String"); + qry.setPageSize(10); + + checkQueryNotFullyFetchedMetrics(cache, qry, false); + } + + /** + * Test metrics for failed SQL cross cache queries. + * + * @throws Exception In case of error. + */ + public void testSqlFieldsCrossCacheQueryFailedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"G\".String"); + + checkQueryFailedMetrics(cache, qry); + } + + /** + * Check metrics. + * + * @param cache Cache to check metrics. + * @param sz Expected size of metrics. + * @param idx Index of metrics to check. + * @param execs Expected number of executions. + * @param completions Expected number of completions. + * @param failures Expected number of failures. + * @param first {@code true} if metrics checked for first query only. + */ + private void checkMetrics(IgniteCache cache, int sz, int idx, int execs, + int completions, int failures, boolean first) { + Collection metrics = cache.queryDetailMetrics(); + + assertNotNull(metrics); + assertEquals(sz, metrics.size()); + + QueryDetailMetrics m = new ArrayList<>(metrics).get(idx); + + info("Metrics: " + m); + + assertEquals("Executions", execs, m.executions()); + assertEquals("Completions", completions, m.completions()); + assertEquals("Failures", failures, m.failures()); + assertTrue(m.averageTime() >= 0); + assertTrue(m.maximumTime() >= 0); + assertTrue(m.minimumTime() >= 0); + + if (first) + assertTrue("On first execution minTime == maxTime", m.minimumTime() == m.maximumTime()); + } + + /** + * @param cache Cache. + * @param qry Query. + */ + private void checkQueryMetrics(IgniteCache cache, Query qry) { + // Execute query. + cache.query(qry).getAll(); + + checkMetrics(cache, 1, 0, 1, 1, 0, true); + + // Execute again with the same parameters. + cache.query(qry).getAll(); + + checkMetrics(cache, 1, 0, 2, 2, 0, false); + } + + /** + * @param cache Cache. + * @param qry Query. + * @param waitingForCompletion Waiting for query completion. + */ + private void checkQueryNotFullyFetchedMetrics(IgniteCache cache, Query qry, + boolean waitingForCompletion) throws IgniteInterruptedCheckedException { + // Execute query. + cache.query(qry).iterator().next(); + + if (waitingForCompletion) + waitingFor(cache, "completions", 1); + + checkMetrics(cache, 1, 0, 1, 1, 0, true); + + // Execute again with the same parameters. + cache.query(qry).iterator().next(); + + if (waitingForCompletion) + waitingFor(cache, "completions", 2); + + checkMetrics(cache, 1, 0, 2, 2, 0, false); + } + + /** + * @param cache Cache. + * @param qry Query. + */ + private void checkQueryFailedMetrics(IgniteCache cache, Query qry) { + try { + // Execute invalid query. + cache.query(qry).getAll(); + } + catch (Exception ignored) { + // No-op. + } + + checkMetrics(cache, 1, 0, 1, 0, 1, true); + + try { + // Execute invalid query again with the same parameters. + cache.query(qry).getAll(); + } + catch (Exception ignored) { + // No-op. + } + + checkMetrics(cache, 1, 0, 2, 0, 2, true); + } + + /** + * @param cache Cache. + * @param cond Condition to check. + * @param exp Expected value. + */ + private static void waitingFor(final IgniteCache cache, + final String cond, final int exp) throws IgniteInterruptedCheckedException { + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + Collection metrics = cache.queryDetailMetrics(); + + switch (cond) { + case "size": + return metrics.size() == exp; + + case "completions": + int completions = 0; + + for (QueryDetailMetrics m : metrics) + completions += m.completions(); + + return completions == exp; + + default: + return true; + } + } + }, 5000); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java index 10f761237cdb1..e5845d30336fb 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java @@ -17,11 +17,15 @@ package org.apache.ignite.internal.processors.cache; +import java.util.ArrayList; +import java.util.Collection; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.query.Query; import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.cache.query.TextQuery; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInterruptedCheckedException; @@ -52,15 +56,13 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra @Override protected void beforeTest() throws Exception { startGridsMultiThreaded(gridCnt); - IgniteCache cacheA = grid(0).cache("A"); + IgniteCache cacheA = grid(0).cache("A"); + IgniteCache cacheB = grid(0).cache("B"); - for (int i = 0; i < 100; i++) - cacheA.put(String.valueOf(i), i); - - IgniteCache cacheB = grid(0).cache("B"); - - for (int i = 0; i < 100; i++) - cacheB.put(String.valueOf(i), i); + for (int i = 0; i < 100; i++) { + cacheA.put(i, String.valueOf(i)); + cacheB.put(i, String.valueOf(i)); + } } /** {@inheritDoc} */ @@ -78,20 +80,20 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra cfg.setDiscoverySpi(disco); - CacheConfiguration cacheCfg1 = defaultCacheConfiguration(); + CacheConfiguration cacheCfg1 = defaultCacheConfiguration(); cacheCfg1.setName("A"); cacheCfg1.setCacheMode(cacheMode); cacheCfg1.setWriteSynchronizationMode(FULL_SYNC); - cacheCfg1.setIndexedTypes(String.class, Integer.class); + cacheCfg1.setIndexedTypes(Integer.class, String.class); cacheCfg1.setStatisticsEnabled(true); - CacheConfiguration cacheCfg2 = defaultCacheConfiguration(); + CacheConfiguration cacheCfg2 = defaultCacheConfiguration(); cacheCfg2.setName("B"); cacheCfg2.setCacheMode(cacheMode); cacheCfg2.setWriteSynchronizationMode(FULL_SYNC); - cacheCfg2.setIndexedTypes(String.class, Integer.class); + cacheCfg2.setIndexedTypes(Integer.class, String.class); cacheCfg2.setStatisticsEnabled(true); cfg.setCacheConfiguration(cacheCfg1, cacheCfg2); @@ -105,11 +107,11 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra * @throws Exception In case of error. */ public void testSqlFieldsQueryMetrics() throws Exception { - IgniteCache cache = grid(0).context().cache().jcache("A"); + IgniteCache cache = grid(0).context().cache().jcache("A"); - SqlFieldsQuery qry = new SqlFieldsQuery("select * from Integer"); + SqlFieldsQuery qry = new SqlFieldsQuery("select * from String"); - testQueryMetrics(cache, qry); + checkQueryMetrics(cache, qry); } /** @@ -118,12 +120,12 @@ public void testSqlFieldsQueryMetrics() throws Exception { * @throws Exception In case of error. */ public void testSqlFieldsQueryNotFullyFetchedMetrics() throws Exception { - IgniteCache cache = grid(0).context().cache().jcache("A"); + IgniteCache cache = grid(0).context().cache().jcache("A"); - SqlFieldsQuery qry = new SqlFieldsQuery("select * from Integer"); + SqlFieldsQuery qry = new SqlFieldsQuery("select * from String"); qry.setPageSize(10); - testQueryNotFullyFetchedMetrics(cache, qry, false); + checkQueryNotFullyFetchedMetrics(cache, qry, false); } /** @@ -132,11 +134,11 @@ public void testSqlFieldsQueryNotFullyFetchedMetrics() throws Exception { * @throws Exception In case of error. */ public void testSqlFieldsQueryFailedMetrics() throws Exception { - IgniteCache cache = grid(0).context().cache().jcache("A"); + IgniteCache cache = grid(0).context().cache().jcache("A"); SqlFieldsQuery qry = new SqlFieldsQuery("select * from UNKNOWN"); - testQueryFailedMetrics(cache, qry); + checkQueryFailedMetrics(cache, qry); } /** @@ -145,11 +147,11 @@ public void testSqlFieldsQueryFailedMetrics() throws Exception { * @throws Exception In case of error. */ public void testScanQueryMetrics() throws Exception { - IgniteCache cache = grid(0).context().cache().jcache("A"); + IgniteCache cache = grid(0).context().cache().jcache("A"); - ScanQuery qry = new ScanQuery<>(); + ScanQuery qry = new ScanQuery<>(); - testQueryMetrics(cache, qry); + checkQueryMetrics(cache, qry); } /** @@ -158,12 +160,12 @@ public void testScanQueryMetrics() throws Exception { * @throws Exception In case of error. */ public void testScanQueryNotFullyFetchedMetrics() throws Exception { - IgniteCache cache = grid(0).context().cache().jcache("A"); + IgniteCache cache = grid(0).context().cache().jcache("A"); - ScanQuery qry = new ScanQuery<>(); + ScanQuery qry = new ScanQuery<>(); qry.setPageSize(10); - testQueryNotFullyFetchedMetrics(cache, qry, true); + checkQueryNotFullyFetchedMetrics(cache, qry, true); } /** @@ -172,11 +174,91 @@ public void testScanQueryNotFullyFetchedMetrics() throws Exception { * @throws Exception In case of error. */ public void testScanQueryFailedMetrics() throws Exception { - IgniteCache cache = grid(0).context().cache().jcache("A"); + IgniteCache cache = grid(0).context().cache().jcache("A"); + + ScanQuery qry = new ScanQuery<>(Integer.MAX_VALUE); + + checkQueryFailedMetrics(cache, qry); + } + + /** + * Test metrics for Sql queries. + * + * @throws Exception In case of error. + */ + public void testSqlQueryMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + SqlQuery qry = new SqlQuery<>("String", "from String"); + + checkQueryMetrics(cache, qry); + } + + /** + * Test metrics for Sql queries. + * + * @throws Exception In case of error. + */ + public void testSqlQueryNotFullyFetchedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + SqlQuery qry = new SqlQuery<>("String", "from String"); + qry.setPageSize(10); + + checkQueryNotFullyFetchedMetrics(cache, qry, true); + } + + /** + * Test metrics for failed Scan queries. + * + * @throws Exception In case of error. + */ + public void testSqlQueryFailedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); - ScanQuery qry = new ScanQuery<>(Integer.MAX_VALUE); + SqlQuery qry = new SqlQuery<>("Long", "from Long"); - testQueryFailedMetrics(cache, qry); + checkQueryFailedMetrics(cache, qry); + } + + /** + * Test metrics for Sql queries. + * + * @throws Exception In case of error. + */ + public void testTextQueryMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + TextQuery qry = new TextQuery<>("String", "1"); + + checkQueryMetrics(cache, qry); + } + + /** + * Test metrics for Sql queries. + * + * @throws Exception In case of error. + */ + public void testTextQueryNotFullyFetchedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + TextQuery qry = new TextQuery<>("String", "1"); + qry.setPageSize(10); + + checkQueryNotFullyFetchedMetrics(cache, qry, true); + } + + /** + * Test metrics for failed Scan queries. + * + * @throws Exception In case of error. + */ + public void testTextQueryFailedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + TextQuery qry = new TextQuery<>("Unknown", "zzz"); + + checkQueryFailedMetrics(cache, qry); } /** @@ -184,12 +266,12 @@ public void testScanQueryFailedMetrics() throws Exception { * * @throws Exception In case of error. */ - public void testSqlCrossCacheQueryMetrics() throws Exception { - IgniteCache cache = grid(0).context().cache().jcache("A"); + public void testSqlFieldsCrossCacheQueryMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); - SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"B\".Integer"); + SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"B\".String"); - testQueryMetrics(cache, qry); + checkQueryMetrics(cache, qry); } /** @@ -197,13 +279,13 @@ public void testSqlCrossCacheQueryMetrics() throws Exception { * * @throws Exception In case of error. */ - public void testSqlCrossCacheQueryNotFullyFetchedMetrics() throws Exception { - IgniteCache cache = grid(0).context().cache().jcache("A"); + public void testSqlFieldsCrossCacheQueryNotFullyFetchedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); - SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"B\".Integer"); + SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"B\".String"); qry.setPageSize(10); - testQueryNotFullyFetchedMetrics(cache, qry, false); + checkQueryNotFullyFetchedMetrics(cache, qry, false); } /** @@ -211,46 +293,102 @@ public void testSqlCrossCacheQueryNotFullyFetchedMetrics() throws Exception { * * @throws Exception In case of error. */ - public void testSqlCrossCacheQueryFailedMetrics() throws Exception { - IgniteCache cache = grid(0).context().cache().jcache("A"); + public void testSqlFieldsCrossCacheQueryFailedMetrics() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + + SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"G\".String"); + + checkQueryFailedMetrics(cache, qry); + } + + /** */ + private static class Worker extends Thread { + /** */ + private final IgniteCache cache; + + /** */ + private final Query qry; - SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"G\".Integer"); + /** */ + Worker(IgniteCache cache, Query qry) { + this.cache = cache; + this.qry = qry; + } - testQueryFailedMetrics(cache, qry); + /** {@inheritDoc} */ + @Override public void run() { + cache.query(qry).getAll(); + } } /** - * @param cache Cache. - * @param qry Query. + * Test metrics if queries executed from several threads. + * + * @throws Exception In case of error. */ - private void testQueryMetrics(IgniteCache cache, Query qry) { - cache.query(qry).getAll(); + public void testQueryMetricsMultithreaded() throws Exception { + IgniteCache cache = grid(0).context().cache().jcache("A"); + Collection workers = new ArrayList<>(); + + int repeat = 100; + + for (int i = 0; i < repeat; i++) { + workers.add(new CacheAbstractQueryMetricsSelfTest.Worker(cache, new SqlFieldsQuery("select * from String limit " + i))); + workers.add(new CacheAbstractQueryMetricsSelfTest.Worker(cache, new SqlQuery("String", "from String"))); + workers.add(new CacheAbstractQueryMetricsSelfTest.Worker(cache, new ScanQuery())); + workers.add(new CacheAbstractQueryMetricsSelfTest.Worker(cache, new TextQuery("String", "1"))); + } + + for (CacheAbstractQueryMetricsSelfTest.Worker worker : workers) + worker.start(); + + for (CacheAbstractQueryMetricsSelfTest.Worker worker : workers) + worker.join(); + + checkMetrics(cache, repeat * 4, repeat * 4, 0, false); + } + + /** + * Check metrics. + * + * @param cache Cache to check metrics. + * @param execs Expected number of executions. + * @param completions Expected number of completions. + * @param failures Expected number of failures. + * @param first {@code true} if metrics checked for first query only. + */ + private void checkMetrics(IgniteCache cache, int execs, int completions, int failures, boolean first) { GridCacheQueryMetricsAdapter m = (GridCacheQueryMetricsAdapter)cache.queryMetrics(); + assertNotNull(m); + info("Metrics: " + m); - assertEquals(1, m.executions()); - assertEquals(1, m.completedExecutions()); - assertEquals(0, m.fails()); + assertEquals("Executions", execs, m.executions()); + assertEquals("Completions", completions, m.completedExecutions()); + assertEquals("Failures", failures, m.fails()); assertTrue(m.averageTime() >= 0); assertTrue(m.maximumTime() >= 0); assertTrue(m.minimumTime() >= 0); - assertTrue("On first execution minTime == maxTime", m.minimumTime() == m.maximumTime()); - // Execute again with the same parameters. + if (first) + assertTrue("On first execution minTime == maxTime", m.minimumTime() == m.maximumTime()); + } + + /** + * @param cache Cache. + * @param qry Query. + */ + private void checkQueryMetrics(IgniteCache cache, Query qry) { cache.query(qry).getAll(); - m = (GridCacheQueryMetricsAdapter)cache.queryMetrics(); + checkMetrics(cache, 1, 1, 0, true); - info("Metrics: " + m); + // Execute again with the same parameters. + cache.query(qry).getAll(); - assertEquals(2, m.executions()); - assertEquals(2, m.completedExecutions()); - assertEquals(0, m.fails()); - assertTrue(m.averageTime() >= 0); - assertTrue(m.maximumTime() >= 0); - assertTrue(m.minimumTime() >= 0); + checkMetrics(cache, 2, 2, 0, false); } /** @@ -258,24 +396,14 @@ private void testQueryMetrics(IgniteCache cache, Query qry) { * @param qry Query. * @param waitingForCompletion Waiting for query completion. */ - private void testQueryNotFullyFetchedMetrics(IgniteCache cache, Query qry, + private void checkQueryNotFullyFetchedMetrics(IgniteCache cache, Query qry, boolean waitingForCompletion) throws IgniteInterruptedCheckedException { cache.query(qry).iterator().next(); if (waitingForCompletion) waitingForCompletion(cache, 1); - GridCacheQueryMetricsAdapter m = (GridCacheQueryMetricsAdapter)cache.queryMetrics(); - - info("Metrics: " + m); - - assertEquals(1, m.executions()); - assertEquals(1, m.completedExecutions()); - assertEquals(0, m.fails()); - assertTrue(m.averageTime() >= 0); - assertTrue(m.maximumTime() >= 0); - assertTrue(m.minimumTime() >= 0); - assertTrue("On first execution minTime == maxTime", m.minimumTime() == m.maximumTime()); + checkMetrics(cache, 1, 1, 0, true); // Execute again with the same parameters. cache.query(qry).iterator().next(); @@ -283,23 +411,14 @@ private void testQueryNotFullyFetchedMetrics(IgniteCache cache, if (waitingForCompletion) waitingForCompletion(cache, 2); - m = (GridCacheQueryMetricsAdapter)cache.queryMetrics(); - - info("Metrics: " + m); - - assertEquals(2, m.executions()); - assertEquals(2, m.completedExecutions()); - assertEquals(0, m.fails()); - assertTrue(m.averageTime() >= 0); - assertTrue(m.maximumTime() >= 0); - assertTrue(m.minimumTime() >= 0); + checkMetrics(cache, 2, 2, 0, false); } /** * @param cache Cache. * @param qry Query. */ - private void testQueryFailedMetrics(IgniteCache cache, Query qry) { + private void checkQueryFailedMetrics(IgniteCache cache, Query qry) { try { cache.query(qry).getAll(); } @@ -307,16 +426,7 @@ private void testQueryFailedMetrics(IgniteCache cache, Query qr // No-op. } - GridCacheQueryMetricsAdapter m = (GridCacheQueryMetricsAdapter)cache.queryMetrics(); - - info("Metrics: " + m); - - assertEquals(1, m.executions()); - assertEquals(0, m.completedExecutions()); - assertEquals(1, m.fails()); - assertTrue(m.averageTime() == 0); - assertTrue(m.maximumTime() == 0); - assertTrue(m.minimumTime() == 0); + checkMetrics(cache, 1, 0, 1, true); // Execute again with the same parameters. try { @@ -326,23 +436,14 @@ private void testQueryFailedMetrics(IgniteCache cache, Query qr // No-op. } - m = (GridCacheQueryMetricsAdapter)cache.queryMetrics(); - - info("Metrics: " + m); - - assertEquals(2, m.executions()); - assertEquals(0, m.completedExecutions()); - assertEquals(2, m.fails()); - assertTrue(m.averageTime() == 0); - assertTrue(m.maximumTime() == 0); - assertTrue(m.minimumTime() == 0); + checkMetrics(cache, 2, 0, 2, true); } /** * @param cache Cache. * @param exp Expected. */ - private static void waitingForCompletion(final IgniteCache cache, + private static void waitingForCompletion(final IgniteCache cache, final int exp) throws IgniteInterruptedCheckedException { GridTestUtils.waitForCondition(new GridAbsPredicate() { @Override public boolean apply() { diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryDetailMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryDetailMetricsSelfTest.java new file mode 100644 index 0000000000000..77971209418a8 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryDetailMetricsSelfTest.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import static org.apache.ignite.cache.CacheMode.LOCAL; + +/** + * Tests for local cache query metrics. + */ +public class CacheLocalQueryDetailMetricsSelfTest extends CacheAbstractQueryDetailMetricsSelfTest { + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + gridCnt = 1; + cacheMode = LOCAL; + + super.beforeTest(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryDetailMetricsDistributedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryDetailMetricsDistributedSelfTest.java new file mode 100644 index 0000000000000..c9244877ef3a3 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryDetailMetricsDistributedSelfTest.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; + +/** + * Tests for partitioned distributed cache query metrics. + */ +public class CachePartitionedQueryDetailMetricsDistributedSelfTest extends CacheAbstractQueryDetailMetricsSelfTest { + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + gridCnt = 2; + cacheMode = PARTITIONED; + + super.beforeTest(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryDetailMetricsLocalSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryDetailMetricsLocalSelfTest.java new file mode 100644 index 0000000000000..4ffba3e74a2fe --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryDetailMetricsLocalSelfTest.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import static org.apache.ignite.cache.CacheMode.PARTITIONED; + +/** + * Tests for partitioned local cache query metrics. + */ +public class CachePartitionedQueryDetailMetricsLocalSelfTest extends CacheAbstractQueryDetailMetricsSelfTest { + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + gridCnt = 1; + cacheMode = PARTITIONED; + + super.beforeTest(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryDetailMetricsDistributedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryDetailMetricsDistributedSelfTest.java new file mode 100644 index 0000000000000..32edb66e1faa9 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryDetailMetricsDistributedSelfTest.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import static org.apache.ignite.cache.CacheMode.REPLICATED; + +/** + * Tests for replicated distributed cache query metrics. + */ +public class CacheReplicatedQueryDetailMetricsDistributedSelfTest extends CacheAbstractQueryDetailMetricsSelfTest { + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + gridCnt = 2; + cacheMode = REPLICATED; + + super.beforeTest(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryDetailMetricsLocalSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryDetailMetricsLocalSelfTest.java new file mode 100644 index 0000000000000..65c372dc90d55 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryDetailMetricsLocalSelfTest.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import static org.apache.ignite.cache.CacheMode.REPLICATED; + +/** + * Tests for replicated local cache query metrics. + */ +public class CacheReplicatedQueryDetailMetricsLocalSelfTest extends CacheAbstractQueryDetailMetricsSelfTest { + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + gridCnt = 1; + cacheMode = REPLICATED; + + super.beforeTest(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java index be7523fe3a501..21f68649e2f35 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java @@ -18,12 +18,17 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.internal.processors.cache.CacheLocalQueryDetailMetricsSelfTest; import org.apache.ignite.internal.processors.cache.CacheLocalQueryMetricsSelfTest; import org.apache.ignite.internal.processors.cache.CacheOffheapBatchIndexingSingleTypeTest; +import org.apache.ignite.internal.processors.cache.CachePartitionedQueryDetailMetricsDistributedSelfTest; +import org.apache.ignite.internal.processors.cache.CachePartitionedQueryDetailMetricsLocalSelfTest; import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsDistributedSelfTest; import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsLocalSelfTest; import org.apache.ignite.internal.processors.cache.CacheQueryNewClientSelfTest; import org.apache.ignite.internal.processors.cache.CacheQueryOffheapEvictDataLostTest; +import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryDetailMetricsDistributedSelfTest; +import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryDetailMetricsLocalSelfTest; import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest; import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest; import org.apache.ignite.internal.processors.cache.CacheScanPartitionQueryFallbackSelfTest; @@ -93,6 +98,13 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(CacheReplicatedQueryMetricsDistributedSelfTest.class); suite.addTestSuite(CacheReplicatedQueryMetricsLocalSelfTest.class); + // Cache query metrics. + suite.addTestSuite(CacheLocalQueryDetailMetricsSelfTest.class); + suite.addTestSuite(CachePartitionedQueryDetailMetricsDistributedSelfTest.class); + suite.addTestSuite(CachePartitionedQueryDetailMetricsLocalSelfTest.class); + suite.addTestSuite(CacheReplicatedQueryDetailMetricsDistributedSelfTest.class); + suite.addTestSuite(CacheReplicatedQueryDetailMetricsLocalSelfTest.class); + // Unmarshalling query test. suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class); suite.addTestSuite(IgniteCacheNoClassQuerySelfTest.class); 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 1fa185f3ffb5f..04a68aed187b9 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 @@ -22,7 +22,8 @@ import java.util.{Collection => JavaCollection, Collections, UUID} import org.apache.ignite._ import org.apache.ignite.cluster.ClusterNode -import org.apache.ignite.internal.util.typedef._ +import org.apache.ignite.internal.util.lang.{GridFunc => F} +import org.apache.ignite.internal.util.typedef.X import org.apache.ignite.internal.visor.cache._ import org.apache.ignite.internal.visor.util.VisorTaskUtils._ import org.apache.ignite.lang.IgniteBiTuple @@ -287,7 +288,7 @@ class VisorCacheCommand { if (hasArgFlag("scan", argLst)) VisorCacheScanCommand().scan(argLst, node) else { - if (aggrData.nonEmpty && !aggrData.exists(cache => safeEquals(cache.name(), name) && cache.system())) { + if (aggrData.nonEmpty && !aggrData.exists(cache => F.eq(cache.name(), name) && cache.system())) { if (hasArgFlag("clear", argLst)) VisorCacheClearCommand().clear(argLst, node) else if (hasArgFlag("swap", argLst)) @@ -470,7 +471,7 @@ class VisorCacheCommand { println(" Total number of executions: " + ad.execsQuery) println(" Total number of failures: " + ad.failsQuery) - gCfg.foreach(ccfgs => ccfgs.find(ccfg => safeEquals(ccfg.name(), ad.name())) + gCfg.foreach(ccfgs => ccfgs.find(ccfg => F.eq(ccfg.name(), ad.name())) .foreach(ccfg => { nl() From a24a394bb66ba0237a9e9ef940707d422b2980f0 Mon Sep 17 00:00:00 2001 From: Konstantin Dudkov Date: Mon, 21 Nov 2016 13:53:58 +0300 Subject: [PATCH 356/487] IGNITE-2523 "single put" NEAR update request --- .../communication/GridIoMessageFactory.java | 24 +- .../processors/cache/GridCacheIoManager.java | 64 +- .../processors/cache/GridCachePreloader.java | 11 + .../cache/GridCachePreloaderAdapter.java | 7 + .../dht/atomic/GridDhtAtomicCache.java | 96 ++- .../dht/atomic/GridDhtAtomicUpdateFuture.java | 12 +- ...NearAtomicAbstractSingleUpdateRequest.java | 562 ++++++++++++++++++ .../GridNearAtomicAbstractUpdateFuture.java | 15 +- .../GridNearAtomicAbstractUpdateRequest.java | 226 +++++++ ...a => GridNearAtomicFullUpdateRequest.java} | 347 +++++------ ...idNearAtomicSingleUpdateFilterRequest.java | 226 +++++++ .../GridNearAtomicSingleUpdateFuture.java | 137 ++++- ...idNearAtomicSingleUpdateInvokeRequest.java | 303 ++++++++++ .../GridNearAtomicSingleUpdateRequest.java | 359 +++++++++++ .../atomic/GridNearAtomicUpdateFuture.java | 42 +- .../dht/preloader/GridDhtPreloader.java | 20 + .../distributed/near/GridNearAtomicCache.java | 10 +- .../resources/META-INF/classnames.properties | 77 +-- ...CacheAtomicSingleMessageCountSelfTest.java | 259 ++++++++ .../GridCacheAtomicMessageCountSelfTest.java | 18 +- .../IgniteCacheAtomicStopBusySelfTest.java | 10 +- .../IgniteCacheP2pUnmarshallingErrorTest.java | 2 +- ...teCacheClientNodeChangingTopologyTest.java | 22 +- ...tomicInvalidPartitionHandlingSelfTest.java | 2 +- 24 files changed, 2444 insertions(+), 407 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractSingleUpdateRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/{GridNearAtomicUpdateRequest.java => GridNearAtomicFullUpdateRequest.java} (87%) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFilterRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAtomicSingleMessageCountSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java index fd5522413e5da..b20de688ae95d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java @@ -69,7 +69,10 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFilterRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateInvokeRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse; @@ -392,7 +395,7 @@ public GridIoMessageFactory(MessageFactory[] ext) { break; case 40: - msg = new GridNearAtomicUpdateRequest(); + msg = new GridNearAtomicFullUpdateRequest(); break; @@ -756,7 +759,22 @@ public GridIoMessageFactory(MessageFactory[] ext) { break; - // [-3..119] [124] - this + case 125: + msg = new GridNearAtomicSingleUpdateRequest(); + + break; + + case 126: + msg = new GridNearAtomicSingleUpdateInvokeRequest(); + + break; + + case 127: + msg = new GridNearAtomicSingleUpdateFilterRequest(); + + break; + + // [-3..119] [124..127] - this // [120..123] - DR // [-4..-22, -30..-35] - SQL default: diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java index e45028729e95a..c5c1c60847ea1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java @@ -47,7 +47,11 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFilterRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateInvokeRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse; @@ -462,8 +466,8 @@ else if (cacheMsg instanceof GridDhtTxFinishResponse) * @return Atomic future ID if applicable for message. */ @Nullable private GridCacheVersion atomicFututeId(GridCacheMessage cacheMsg) { - if (cacheMsg instanceof GridNearAtomicUpdateRequest) - return ((GridNearAtomicUpdateRequest)cacheMsg).futureVersion(); + if (cacheMsg instanceof GridNearAtomicAbstractUpdateRequest) + return ((GridNearAtomicAbstractUpdateRequest)cacheMsg).futureVersion(); else if (cacheMsg instanceof GridNearAtomicUpdateResponse) return ((GridNearAtomicUpdateResponse) cacheMsg).futureVersion(); else if (cacheMsg instanceof GridDhtAtomicUpdateRequest) @@ -480,8 +484,8 @@ else if (cacheMsg instanceof GridDhtAtomicUpdateResponse) * @return Atomic future ID if applicable for message. */ @Nullable private GridCacheVersion atomicWriteVersion(GridCacheMessage cacheMsg) { - if (cacheMsg instanceof GridNearAtomicUpdateRequest) - return ((GridNearAtomicUpdateRequest)cacheMsg).updateVersion(); + if (cacheMsg instanceof GridNearAtomicAbstractUpdateRequest) + return ((GridNearAtomicAbstractUpdateRequest)cacheMsg).updateVersion(); else if (cacheMsg instanceof GridDhtAtomicUpdateRequest) return ((GridDhtAtomicUpdateRequest)cacheMsg).writeVersion(); @@ -562,7 +566,7 @@ private void processFailedMessage(UUID nodeId, GridCacheMessage msg, IgniteBiInC break; case 40: { - GridNearAtomicUpdateRequest req = (GridNearAtomicUpdateRequest)msg; + GridNearAtomicFullUpdateRequest req = (GridNearAtomicFullUpdateRequest)msg; GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse( ctx.cacheId(), @@ -739,6 +743,54 @@ private void processFailedMessage(UUID nodeId, GridCacheMessage msg, IgniteBiInC break; + case 125: { + GridNearAtomicSingleUpdateRequest req = (GridNearAtomicSingleUpdateRequest)msg; + + GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse( + ctx.cacheId(), + nodeId, + req.futureVersion(), + ctx.deploymentEnabled()); + + res.error(req.classError()); + + sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy()); + } + + break; + + case 126: { + GridNearAtomicSingleUpdateInvokeRequest req = (GridNearAtomicSingleUpdateInvokeRequest)msg; + + GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse( + ctx.cacheId(), + nodeId, + req.futureVersion(), + ctx.deploymentEnabled()); + + res.error(req.classError()); + + sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy()); + } + + break; + + case 127: { + GridNearAtomicSingleUpdateFilterRequest req = (GridNearAtomicSingleUpdateFilterRequest)msg; + + GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse( + ctx.cacheId(), + nodeId, + req.futureVersion(), + ctx.deploymentEnabled()); + + res.error(req.classError()); + + sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy()); + } + + break; + default: throw new IgniteCheckedException("Failed to send response to node. Unsupported direct type [message=" + msg + "]", msg.classError()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java index a49bb04aae363..1d1cfab04ce63 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java @@ -24,6 +24,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; @@ -139,6 +140,16 @@ public Callable addAssignments(GridDhtPreloaderAssignments assignments, */ public IgniteInternalFuture request(Collection keys, AffinityTopologyVersion topVer); + /** + * Requests that preloader sends the request for the key. + * + * @param req Message with keys to request. + * @param topVer Topology version, {@code -1} if not required. + * @return Future to complete when all keys are preloaded. + */ + public IgniteInternalFuture request(GridNearAtomicAbstractUpdateRequest req, + AffinityTopologyVersion topVer); + /** * Force preload process. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java index 58b75df0fa9ce..b15ebc5029b12 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java @@ -26,6 +26,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; @@ -148,6 +149,12 @@ public GridCachePreloaderAdapter(GridCacheContext cctx) { return new GridFinishedFuture<>(); } + /** {@inheritDoc} */ + @Override public IgniteInternalFuture request(GridNearAtomicAbstractUpdateRequest req, + AffinityTopologyVersion topVer) { + return new GridFinishedFuture<>(); + } + /** {@inheritDoc} */ @Override public void onInitialExchangeComplete(@Nullable Throwable err) { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java index 30a3d57ef896b..f7d19731ba591 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java @@ -136,7 +136,7 @@ public class GridDhtAtomicCache extends GridDhtCacheAdapter { /** Update reply closure. */ @GridToStringExclude - private CI2 updateReplyClos; + private CI2 updateReplyClos; /** Pending */ private GridDeferredAckMessageSender deferredUpdateMessageSender; @@ -200,9 +200,9 @@ public GridDhtAtomicCache(GridCacheContext ctx, GridCacheConcurrentMap map @Override protected void init() { super.init(); - updateReplyClos = new CI2() { + updateReplyClos = new CI2() { @SuppressWarnings("ThrowableResultOfMethodCallIgnored") - @Override public void apply(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateResponse res) { + @Override public void apply(GridNearAtomicAbstractUpdateRequest req, GridNearAtomicUpdateResponse res) { if (ctx.config().getAtomicWriteOrderMode() == CLOCK) { assert req.writeSynchronizationMode() != FULL_ASYNC : req; @@ -323,11 +323,11 @@ else if (res.error() != null) { ctx.io().addHandler( ctx.cacheId(), - GridNearAtomicUpdateRequest.class, - new CI2() { + GridNearAtomicAbstractUpdateRequest.class, + new CI2() { @Override public void apply( UUID nodeId, - GridNearAtomicUpdateRequest req + GridNearAtomicAbstractUpdateRequest req ) { processNearAtomicUpdateRequest( nodeId, @@ -335,8 +335,8 @@ else if (res.error() != null) { } @Override public String toString() { - return "GridNearAtomicUpdateRequest handler " + - "[msgIdx=" + GridNearAtomicUpdateRequest.CACHE_MSG_IDX + ']'; + return "GridNearAtomicAbstractUpdateRequest handler " + + "[msgIdx=" + GridNearAtomicAbstractUpdateRequest.CACHE_MSG_IDX + ']'; } }); @@ -1252,7 +1252,9 @@ else if (op == GridCacheOperation.TRANSFORM) { CacheEntryPredicate[] filters = CU.filterArray(filter); - if (conflictPutVal == null && conflictRmvVer == null && !isFastMap(filters, op)) { + if (conflictPutVal == null && + conflictRmvVer == null && + !isFastMap(filters, op)) { return new GridNearAtomicSingleUpdateFuture( ctx, this, @@ -1603,10 +1605,10 @@ else if (!skipVals && ctx.config().isStatisticsEnabled()) */ public void updateAllAsyncInternal( final UUID nodeId, - final GridNearAtomicUpdateRequest req, - final CI2 completionCb + final GridNearAtomicAbstractUpdateRequest req, + final CI2 completionCb ) { - IgniteInternalFuture forceFut = preldr.request(req.keys(), req.topologyVersion()); + IgniteInternalFuture forceFut = preldr.request(req, req.topologyVersion()); if (forceFut == null || forceFut.isDone()) { try { @@ -1652,8 +1654,8 @@ public void updateAllAsyncInternal( * @param e Error. */ private void onForceKeysError(final UUID nodeId, - final GridNearAtomicUpdateRequest req, - final CI2 completionCb, + final GridNearAtomicAbstractUpdateRequest req, + final CI2 completionCb, IgniteCheckedException e ) { GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(ctx.cacheId(), @@ -1673,17 +1675,15 @@ private void onForceKeysError(final UUID nodeId, * @param req Update request. * @param completionCb Completion callback. */ - public void updateAllAsyncInternal0( + private void updateAllAsyncInternal0( UUID nodeId, - GridNearAtomicUpdateRequest req, - CI2 completionCb + GridNearAtomicAbstractUpdateRequest req, + CI2 completionCb ) { GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(ctx.cacheId(), nodeId, req.futureVersion(), ctx.deploymentEnabled()); - List keys = req.keys(); - - assert !req.returnValue() || (req.operation() == TRANSFORM || keys.size() == 1); + assert !req.returnValue() || (req.operation() == TRANSFORM || req.size() == 1); GridDhtAtomicUpdateFuture dhtFut = null; @@ -1696,7 +1696,7 @@ public void updateAllAsyncInternal0( try { // If batch store update is enabled, we need to lock all entries. // First, need to acquire locks on cache entries, then check filter. - List locked = lockEntries(keys, req.topologyVersion()); + List locked = lockEntries(req, req.topologyVersion()); Collection> deleted = null; @@ -1707,7 +1707,7 @@ public void updateAllAsyncInternal0( try { if (top.stopping()) { - res.addFailedKeys(keys, new IgniteCheckedException("Failed to perform cache operation " + + res.addFailedKeys(req.keys(), new IgniteCheckedException("Failed to perform cache operation " + "(cache is stopped): " + name())); completionCb.apply(req, res); @@ -1757,7 +1757,7 @@ public void updateAllAsyncInternal0( GridCacheReturn retVal = null; - if (keys.size() > 1 && // Several keys ... + if (req.size() > 1 && // Several keys ... writeThrough() && !req.skipStore() && // and store is enabled ... !ctx.store().isLocal() && // and this is not local store ... // (conflict resolver should be used for local store) @@ -1853,7 +1853,7 @@ public void updateAllAsyncInternal0( // an attempt to use cleaned resources. U.error(log, "Unexpected exception during cache update", e); - res.addFailedKeys(keys, e); + res.addFailedKeys(req.keys(), e); completionCb.apply(req, res); @@ -1866,7 +1866,7 @@ public void updateAllAsyncInternal0( if (remap) { assert dhtFut == null; - res.remapKeys(keys); + res.remapKeys(req.keys()); completionCb.apply(req, res); } @@ -1904,12 +1904,12 @@ public void updateAllAsyncInternal0( private UpdateBatchResult updateWithBatch( final ClusterNode node, final boolean hasNear, - final GridNearAtomicUpdateRequest req, + final GridNearAtomicAbstractUpdateRequest req, final GridNearAtomicUpdateResponse res, final List locked, final GridCacheVersion ver, @Nullable GridDhtAtomicUpdateFuture dhtFut, - final CI2 completionCb, + final CI2 completionCb, final boolean replicate, final String taskName, @Nullable final IgniteCacheExpiryPolicy expiry, @@ -1929,7 +1929,7 @@ private UpdateBatchResult updateWithBatch( } } - int size = req.keys().size(); + int size = req.size(); Map putMap = null; @@ -2327,12 +2327,12 @@ private void reloadIfNeeded(final List entries) throws Ignite private UpdateSingleResult updateSingle( ClusterNode node, boolean hasNear, - GridNearAtomicUpdateRequest req, + GridNearAtomicAbstractUpdateRequest req, GridNearAtomicUpdateResponse res, List locked, GridCacheVersion ver, @Nullable GridDhtAtomicUpdateFuture dhtFut, - CI2 completionCb, + CI2 completionCb, boolean replicate, String taskName, @Nullable IgniteCacheExpiryPolicy expiry, @@ -2341,8 +2341,6 @@ private UpdateSingleResult updateSingle( GridCacheReturn retVal = null; Collection> deleted = null; - List keys = req.keys(); - AffinityTopologyVersion topVer = req.topologyVersion(); boolean checkReaders = hasNear || ctx.discovery().hasNearCache(name(), topVer); @@ -2352,8 +2350,8 @@ private UpdateSingleResult updateSingle( boolean intercept = ctx.config().getInterceptor() != null; // Avoid iterator creation. - for (int i = 0; i < keys.size(); i++) { - KeyCacheObject k = keys.get(i); + for (int i = 0; i < req.size(); i++) { + KeyCacheObject k = req.key(i); GridCacheOperation op = req.operation(); @@ -2489,7 +2487,7 @@ else if (F.contains(readers, node.id())) // Reader became primary or backup. if (updRes.removeVersion() != null) { if (deleted == null) - deleted = new ArrayList<>(keys.size()); + deleted = new ArrayList<>(req.size()); deleted.add(F.t(entry, updRes.removeVersion())); } @@ -2565,8 +2563,8 @@ else if (F.contains(readers, node.id())) // Reader became primary or backup. @Nullable final Collection rmvKeys, @Nullable final Map> entryProcessorMap, @Nullable GridDhtAtomicUpdateFuture dhtFut, - final CI2 completionCb, - final GridNearAtomicUpdateRequest req, + final CI2 completionCb, + final GridNearAtomicAbstractUpdateRequest req, final GridNearAtomicUpdateResponse res, final boolean replicate, final UpdateBatchResult batchRes, @@ -2803,17 +2801,17 @@ else if (readers.contains(node.id())) // Reader became primary or backup. /** * Acquires java-level locks on cache entries. Returns collection of locked entries. * - * @param keys Keys to lock. + * @param req Request with keys to lock. * @param topVer Topology version to lock on. * @return Collection of locked entries. * @throws GridDhtInvalidPartitionException If entry does not belong to local node. If exception is thrown, * locks are released. */ @SuppressWarnings("ForLoopReplaceableByForEach") - private List lockEntries(List keys, AffinityTopologyVersion topVer) + private List lockEntries(GridNearAtomicAbstractUpdateRequest req, AffinityTopologyVersion topVer) throws GridDhtInvalidPartitionException { - if (keys.size() == 1) { - KeyCacheObject key = keys.get(0); + if (req.size() == 1) { + KeyCacheObject key = req.key(0); while (true) { try { @@ -2836,12 +2834,12 @@ private List lockEntries(List keys, AffinityT } } else { - List locked = new ArrayList<>(keys.size()); + List locked = new ArrayList<>(req.size()); while (true) { - for (KeyCacheObject key : keys) { + for (int i = 0; i < req.size(); i++) { try { - GridDhtCacheEntry entry = entryExx(key, topVer); + GridDhtCacheEntry entry = entryExx(req.key(i), topVer); locked.add(entry); } @@ -2946,7 +2944,7 @@ private void unlockEntries(Collection locked, AffinityTopolog * will return false. * @return {@code True} if filter evaluation succeeded. */ - private boolean checkFilter(GridCacheEntryEx entry, GridNearAtomicUpdateRequest req, + private boolean checkFilter(GridCacheEntryEx entry, GridNearAtomicAbstractUpdateRequest req, GridNearAtomicUpdateResponse res) { try { return ctx.isAllLocked(entry, req.filter()); @@ -2961,7 +2959,7 @@ private boolean checkFilter(GridCacheEntryEx entry, GridNearAtomicUpdateRequest /** * @param req Request to remap. */ - private void remapToNewPrimary(GridNearAtomicUpdateRequest req) { + private void remapToNewPrimary(GridNearAtomicAbstractUpdateRequest req) { assert req.writeSynchronizationMode() == FULL_ASYNC : req; if (log.isDebugEnabled()) @@ -3040,9 +3038,9 @@ else if (req.operation() == UPDATE) { */ @Nullable private GridDhtAtomicUpdateFuture createDhtFuture( GridCacheVersion writeVer, - GridNearAtomicUpdateRequest updateReq, + GridNearAtomicAbstractUpdateRequest updateReq, GridNearAtomicUpdateResponse updateRes, - CI2 completionCb, + CI2 completionCb, boolean force ) { if (!force) { @@ -3073,7 +3071,7 @@ else if (req.operation() == UPDATE) { * @param nodeId Sender node ID. * @param req Near atomic update request. */ - private void processNearAtomicUpdateRequest(UUID nodeId, GridNearAtomicUpdateRequest req) { + private void processNearAtomicUpdateRequest(UUID nodeId, GridNearAtomicAbstractUpdateRequest req) { if (msgLog.isDebugEnabled()) { msgLog.debug("Received near atomic update request [futId=" + req.futureVersion() + ", writeVer=" + req.updateVersion() + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java index 4e59d1164a9a0..c2ad8b8f961b7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java @@ -83,7 +83,7 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter /** Completion callback. */ @GridToStringExclude - private final CI2 completionCb; + private final CI2 completionCb; /** Mappings. */ @GridToStringInclude @@ -93,7 +93,7 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter private Map nearReadersEntries; /** Update request. */ - private final GridNearAtomicUpdateRequest updateReq; + private final GridNearAtomicAbstractUpdateRequest updateReq; /** Update response. */ private final GridNearAtomicUpdateResponse updateRes; @@ -119,9 +119,9 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter */ public GridDhtAtomicUpdateFuture( GridCacheContext cctx, - CI2 completionCb, + CI2 completionCb, GridCacheVersion writeVer, - GridNearAtomicUpdateRequest updateReq, + GridNearAtomicAbstractUpdateRequest updateReq, GridNearAtomicUpdateResponse updateRes ) { this.cctx = cctx; @@ -137,8 +137,8 @@ public GridDhtAtomicUpdateFuture( log = U.logger(cctx.kernalContext(), logRef, GridDhtAtomicUpdateFuture.class); } - keys = new ArrayList<>(updateReq.keys().size()); - mappings = U.newHashMap(updateReq.keys().size()); + keys = new ArrayList<>(updateReq.size()); + mappings = U.newHashMap(updateReq.size()); waitForExchange = !(updateReq.topologyLocked() || (updateReq.fastMap() && !updateReq.clientRequest())); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractSingleUpdateRequest.java new file mode 100644 index 0000000000000..61deeee1e1042 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractSingleUpdateRequest.java @@ -0,0 +1,562 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; + +import java.io.Externalizable; +import java.nio.ByteBuffer; +import java.util.UUID; +import javax.cache.expiry.ExpiryPolicy; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.internal.GridDirectTransient; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; +import org.apache.ignite.internal.processors.cache.GridCacheOperation; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public abstract class GridNearAtomicAbstractSingleUpdateRequest extends GridNearAtomicAbstractUpdateRequest { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private static final CacheEntryPredicate[] NO_FILTER = new CacheEntryPredicate[0]; + + /** Fast map flag mask. */ + private static final int FAST_MAP_FLAG_MASK = 0x1; + + /** Flag indicating whether request contains primary keys. */ + private static final int HAS_PRIMARY_FLAG_MASK = 0x2; + + /** Topology locked flag. Set if atomic update is performed inside TX or explicit lock. */ + private static final int TOP_LOCKED_FLAG_MASK = 0x4; + + /** Skip write-through to a persistent storage. */ + private static final int SKIP_STORE_FLAG_MASK = 0x8; + + /** */ + private static final int CLIENT_REQ_FLAG_MASK = 0x10; + + /** Keep binary flag. */ + private static final int KEEP_BINARY_FLAG_MASK = 0x20; + + /** Return value flag. */ + private static final int RET_VAL_FLAG_MASK = 0x40; + + /** Target node ID. */ + @GridDirectTransient + protected UUID nodeId; + + /** Future version. */ + protected GridCacheVersion futVer; + + /** Update version. Set to non-null if fastMap is {@code true}. */ + private GridCacheVersion updateVer; + + /** Topology version. */ + protected AffinityTopologyVersion topVer; + + /** Write synchronization mode. */ + protected CacheWriteSynchronizationMode syncMode; + + /** Update operation. */ + protected GridCacheOperation op; + + /** Subject ID. */ + protected UUID subjId; + + /** Task name hash. */ + protected int taskNameHash; + + /** */ + @GridDirectTransient + private GridNearAtomicUpdateResponse res; + + /** Compressed boolean flags. */ + protected byte flags; + + /** + * Empty constructor required by {@link Externalizable}. + */ + protected GridNearAtomicAbstractSingleUpdateRequest() { + // No-op. + } + + /** + * Constructor. + * + * @param cacheId Cache ID. + * @param nodeId Node ID. + * @param futVer Future version. + * @param fastMap Fast map scheme flag. + * @param updateVer Update version set if fast map is performed. + * @param topVer Topology version. + * @param topLocked Topology locked flag. + * @param syncMode Synchronization mode. + * @param op Cache update operation. + * @param retval Return value required flag. + * @param subjId Subject ID. + * @param taskNameHash Task name hash code. + * @param skipStore Skip write-through to a persistent storage. + * @param keepBinary Keep binary flag. + * @param clientReq Client node request flag. + * @param addDepInfo Deployment info flag. + */ + protected GridNearAtomicAbstractSingleUpdateRequest( + int cacheId, + UUID nodeId, + GridCacheVersion futVer, + boolean fastMap, + @Nullable GridCacheVersion updateVer, + @NotNull AffinityTopologyVersion topVer, + boolean topLocked, + CacheWriteSynchronizationMode syncMode, + GridCacheOperation op, + boolean retval, + @Nullable UUID subjId, + int taskNameHash, + boolean skipStore, + boolean keepBinary, + boolean clientReq, + boolean addDepInfo + ) { + assert futVer != null; + + this.cacheId = cacheId; + this.nodeId = nodeId; + this.futVer = futVer; + this.updateVer = updateVer; + this.topVer = topVer; + this.syncMode = syncMode; + this.op = op; + this.subjId = subjId; + this.taskNameHash = taskNameHash; + this.addDepInfo = addDepInfo; + + fastMap(fastMap); + topologyLocked(topLocked); + returnValue(retval); + skipStore(skipStore); + keepBinary(keepBinary); + clientRequest(clientReq); + } + + /** {@inheritDoc} */ + @Override public int lookupIndex() { + return CACHE_MSG_IDX; + } + + /** + * @return Mapped node ID. + */ + @Override public UUID nodeId() { + return nodeId; + } + + /** + * @param nodeId Node ID. + */ + @Override public void nodeId(UUID nodeId) { + this.nodeId = nodeId; + } + + /** + * @return Subject ID. + */ + @Override public UUID subjectId() { + return subjId; + } + + /** + * @return Task name hash. + */ + @Override public int taskNameHash() { + return taskNameHash; + } + + /** + * @return Future version. + */ + @Override public GridCacheVersion futureVersion() { + return futVer; + } + + /** + * @return Update version for fast-map request. + */ + @Override public GridCacheVersion updateVersion() { + return updateVer; + } + + /** + * @return Topology version. + */ + @Override public AffinityTopologyVersion topologyVersion() { + return topVer; + } + + /** + * @return Cache write synchronization mode. + */ + @Override public CacheWriteSynchronizationMode writeSynchronizationMode() { + return syncMode; + } + + /** + * @return Expiry policy. + */ + @Override public ExpiryPolicy expiry() { + return null; + } + + /** + * @return Update operation. + */ + @Override public GridCacheOperation operation() { + return op; + } + + /** + * @return Optional arguments for entry processor. + */ + @Override @Nullable public Object[] invokeArguments() { + return null; + } + + /** + * @param res Response. + * @return {@code True} if current response was {@code null}. + */ + @Override public boolean onResponse(GridNearAtomicUpdateResponse res) { + if (this.res == null) { + this.res = res; + + return true; + } + + return false; + } + + /** + * @return Response. + */ + @Override @Nullable public GridNearAtomicUpdateResponse response() { + return res; + } + + /** {@inheritDoc} */ + @Override public boolean addDeploymentInfo() { + return addDepInfo; + } + + /** {@inheritDoc} */ + @Override public IgniteLogger messageLogger(GridCacheSharedContext ctx) { + return ctx.atomicMessageLogger(); + } + + /** + * @return Flag indicating whether this is fast-map udpate. + */ + @Override public boolean fastMap() { + return isFlag(FAST_MAP_FLAG_MASK); + } + + /** + * Sets fastMap flag value. + */ + public void fastMap(boolean val) { + setFlag(val, FAST_MAP_FLAG_MASK); + } + + /** + * @return Topology locked flag. + */ + @Override public boolean topologyLocked() { + return isFlag(TOP_LOCKED_FLAG_MASK); + } + + /** + * Sets topologyLocked flag value. + */ + public void topologyLocked(boolean val) { + setFlag(val, TOP_LOCKED_FLAG_MASK); + } + + /** + * @return {@code True} if request sent from client node. + */ + @Override public boolean clientRequest() { + return isFlag(CLIENT_REQ_FLAG_MASK); + } + + /** + * Sets clientRequest flag value. + */ + public void clientRequest(boolean val) { + setFlag(val, CLIENT_REQ_FLAG_MASK); + } + + /** + * @return Return value flag. + */ + @Override public boolean returnValue() { + return isFlag(RET_VAL_FLAG_MASK); + } + + /** + * Sets returnValue flag value. + */ + public void returnValue(boolean val) { + setFlag(val, RET_VAL_FLAG_MASK); + } + + /** + * @return Skip write-through to a persistent storage. + */ + @Override public boolean skipStore() { + return isFlag(SKIP_STORE_FLAG_MASK); + } + + /** + * Sets skipStore flag value. + */ + public void skipStore(boolean val) { + setFlag(val, SKIP_STORE_FLAG_MASK); + } + + /** + * @return Keep binary flag. + */ + @Override public boolean keepBinary() { + return isFlag(KEEP_BINARY_FLAG_MASK); + } + + /** + * Sets keepBinary flag value. + */ + public void keepBinary(boolean val) { + setFlag(val, KEEP_BINARY_FLAG_MASK); + } + + /** + * @return Flag indicating whether this request contains primary keys. + */ + @Override public boolean hasPrimary() { + return isFlag(HAS_PRIMARY_FLAG_MASK); + } + + /** + * Sets hasPrimary flag value. + */ + public void hasPrimary(boolean val) { + setFlag(val, HAS_PRIMARY_FLAG_MASK); + } + + /** {@inheritDoc} */ + @Nullable @Override public CacheEntryPredicate[] filter() { + return NO_FILTER; + } + + /** + * Sets flag mask. + * + * @param flag Set or clear. + * @param mask Mask. + */ + private void setFlag(boolean flag, int mask) { + flags = flag ? (byte)(flags | mask) : (byte)(flags & ~mask); + } + + /** + * Reads flag mask. + * + * @param mask Mask to read. + * @return Flag value. + */ + private boolean isFlag(int mask) { + return (flags & mask) != 0; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 3: + if (!writer.writeByte("flags", flags)) + return false; + + writer.incrementState(); + + case 4: + if (!writer.writeMessage("futVer", futVer)) + return false; + + writer.incrementState(); + + case 5: + if (!writer.writeByte("op", op != null ? (byte)op.ordinal() : -1)) + return false; + + writer.incrementState(); + + case 6: + if (!writer.writeUuid("subjId", subjId)) + return false; + + writer.incrementState(); + + case 7: + if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1)) + return false; + + writer.incrementState(); + + case 8: + if (!writer.writeInt("taskNameHash", taskNameHash)) + return false; + + writer.incrementState(); + + case 9: + if (!writer.writeMessage("topVer", topVer)) + return false; + + writer.incrementState(); + + case 10: + if (!writer.writeMessage("updateVer", updateVer)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 3: + flags = reader.readByte("flags"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 4: + futVer = reader.readMessage("futVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 5: + byte opOrd; + + opOrd = reader.readByte("op"); + + if (!reader.isLastRead()) + return false; + + op = GridCacheOperation.fromOrdinal(opOrd); + + reader.incrementState(); + + case 6: + subjId = reader.readUuid("subjId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 7: + byte syncModeOrd; + + syncModeOrd = reader.readByte("syncMode"); + + if (!reader.isLastRead()) + return false; + + syncMode = CacheWriteSynchronizationMode.fromOrdinal(syncModeOrd); + + reader.incrementState(); + + case 8: + taskNameHash = reader.readInt("taskNameHash"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 9: + topVer = reader.readMessage("topVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 10: + updateVer = reader.readMessage("updateVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridNearAtomicAbstractSingleUpdateRequest.class); + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 11; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java index 85751bb1ba66f..2fbabaa54fc9c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java @@ -17,6 +17,9 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import javax.cache.expiry.ExpiryPolicy; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.CacheWriteSynchronizationMode; @@ -36,10 +39,6 @@ import org.apache.ignite.lang.IgniteUuid; import org.jetbrains.annotations.Nullable; -import javax.cache.expiry.ExpiryPolicy; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicReference; - import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC; @@ -255,11 +254,11 @@ protected boolean storeFuture() { * @param nodeId Node ID. * @param req Request. */ - protected void mapSingle(UUID nodeId, GridNearAtomicUpdateRequest req) { + protected void mapSingle(UUID nodeId, GridNearAtomicAbstractUpdateRequest req) { if (cctx.localNodeId().equals(nodeId)) { cache.updateAllAsyncInternal(nodeId, req, - new CI2() { - @Override public void apply(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateResponse res) { + new CI2() { + @Override public void apply(GridNearAtomicAbstractUpdateRequest req, GridNearAtomicUpdateResponse res) { onResult(res.nodeId(), res, false); } }); @@ -303,7 +302,7 @@ protected void mapSingle(UUID nodeId, GridNearAtomicUpdateRequest req) { * @param req Request. * @param e Error. */ - protected void onSendError(GridNearAtomicUpdateRequest req, IgniteCheckedException e) { + protected void onSendError(GridNearAtomicAbstractUpdateRequest req, IgniteCheckedException e) { synchronized (mux) { GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(), req.nodeId(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java new file mode 100644 index 0000000000000..bee2ecd61262a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; + +import java.util.List; +import java.util.UUID; +import javax.cache.expiry.ExpiryPolicy; +import javax.cache.processor.EntryProcessor; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.GridCacheDeployable; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheOperation; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public abstract class GridNearAtomicAbstractUpdateRequest extends GridCacheMessage implements GridCacheDeployable { + /** Message index. */ + public static final int CACHE_MSG_IDX = nextIndexId(); + + /** + * @return Mapped node ID. + */ + public abstract UUID nodeId(); + + /** + * @param nodeId Node ID. + */ + public abstract void nodeId(UUID nodeId); + + /** + * @return Subject ID. + */ + public abstract UUID subjectId(); + + /** + * @return Task name hash. + */ + public abstract int taskNameHash(); + + /** + * @return Future version. + */ + public abstract GridCacheVersion futureVersion(); + + /** + * @return Flag indicating whether this is fast-map udpate. + */ + public abstract boolean fastMap(); + + /** + * @return Update version for fast-map request. + */ + public abstract GridCacheVersion updateVersion(); + + /** + * @return Topology locked flag. + */ + public abstract boolean topologyLocked(); + + /** + * @return {@code True} if request sent from client node. + */ + public abstract boolean clientRequest(); + + /** + * @return Cache write synchronization mode. + */ + public abstract CacheWriteSynchronizationMode writeSynchronizationMode(); + + /** + * @return Expiry policy. + */ + public abstract ExpiryPolicy expiry(); + + /** + * @return Return value flag. + */ + public abstract boolean returnValue(); + + /** + * @return Filter. + */ + @Nullable public abstract CacheEntryPredicate[] filter(); + + /** + * @return Skip write-through to a persistent storage. + */ + public abstract boolean skipStore(); + + /** + * @return Keep binary flag. + */ + public abstract boolean keepBinary(); + + /** + * @return Update operation. + */ + public abstract GridCacheOperation operation(); + + /** + * @return Optional arguments for entry processor. + */ + @Nullable public abstract Object[] invokeArguments(); + + /** + * @return Flag indicating whether this request contains primary keys. + */ + public abstract boolean hasPrimary(); + + /** + * @param res Response. + * @return {@code True} if current response was {@code null}. + */ + public abstract boolean onResponse(GridNearAtomicUpdateResponse res); + + /** + * @return Response. + */ + @Nullable public abstract GridNearAtomicUpdateResponse response(); + + /** + * @param key Key to add. + * @param val Optional update value. + * @param conflictTtl Conflict TTL (optional). + * @param conflictExpireTime Conflict expire time (optional). + * @param conflictVer Conflict version (optional). + * @param primary If given key is primary on this mapping. + */ + public abstract void addUpdateEntry(KeyCacheObject key, + @Nullable Object val, + long conflictTtl, + long conflictExpireTime, + @Nullable GridCacheVersion conflictVer, + boolean primary); + + /** + * @return Keys for this update request. + */ + public abstract List keys(); + + /** + * @return Values for this update request. + */ + public abstract List values(); + + /** + * @param idx Key index. + * @return Value. + */ + public abstract CacheObject value(int idx); + + /** + * @param idx Key index. + * @return Entry processor. + */ + public abstract EntryProcessor entryProcessor(int idx); + + /** + * @param idx Index to get. + * @return Write value - either value, or transform closure. + */ + public abstract CacheObject writeValue(int idx); + + + /** + * @return Conflict versions. + */ + @Nullable public abstract List conflictVersions(); + + /** + * @param idx Index. + * @return Conflict version. + */ + @Nullable public abstract GridCacheVersion conflictVersion(int idx); + + /** + * @param idx Index. + * @return Conflict TTL. + */ + public abstract long conflictTtl(int idx); + + /** + * @param idx Index. + * @return Conflict expire time. + */ + public abstract long conflictExpireTime(int idx); + + /** + * Cleanup values. + * + * @param clearKeys If {@code true} clears keys. + */ + public abstract void cleanup(boolean clearKeys); + + /** + * @return Keys size. + */ + public abstract int size(); + + /** + * @param idx Key index. + * @return Key. + */ + public abstract KeyCacheObject key(int idx); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java similarity index 87% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java index 5057fbbb68371..e2314f8fd543e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java @@ -34,8 +34,6 @@ import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.GridCacheDeployable; -import org.apache.ignite.internal.processors.cache.GridCacheMessage; import org.apache.ignite.internal.processors.cache.GridCacheOperation; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; @@ -59,13 +57,10 @@ /** * Lite DHT cache update request sent from near node to primary node. */ -public class GridNearAtomicUpdateRequest extends GridCacheMessage implements GridCacheDeployable { +public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdateRequest { /** */ private static final long serialVersionUID = 0L; - /** Message index. */ - public static final int CACHE_MSG_IDX = nextIndexId(); - /** Target node ID. */ @GridDirectTransient private UUID nodeId; @@ -171,7 +166,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri /** * Empty constructor required by {@link Externalizable}. */ - public GridNearAtomicUpdateRequest() { + public GridNearAtomicFullUpdateRequest() { // No-op. } @@ -199,7 +194,7 @@ public GridNearAtomicUpdateRequest() { * @param addDepInfo Deployment info flag. * @param maxEntryCnt Maximum entries count. */ - public GridNearAtomicUpdateRequest( + GridNearAtomicFullUpdateRequest( int cacheId, UUID nodeId, GridCacheVersion futVer, @@ -260,127 +255,79 @@ public GridNearAtomicUpdateRequest( return CACHE_MSG_IDX; } - /** - * @return Mapped node ID. - */ - public UUID nodeId() { + /** {@inheritDoc} */ + @Override public UUID nodeId() { return nodeId; } - /** - * @param nodeId Node ID. - */ - public void nodeId(UUID nodeId) { + /** {@inheritDoc} */ + @Override public void nodeId(UUID nodeId) { this.nodeId = nodeId; } - /** - * @return Subject ID. - */ - public UUID subjectId() { + /** {@inheritDoc} */ + @Override public UUID subjectId() { return subjId; } - /** - * @return Task name hash. - */ - public int taskNameHash() { + /** {@inheritDoc} */ + @Override public int taskNameHash() { return taskNameHash; } - /** - * @return Future version. - */ - public GridCacheVersion futureVersion() { + /** {@inheritDoc} */ + @Override public GridCacheVersion futureVersion() { return futVer; } - /** - * @return Flag indicating whether this is fast-map udpate. - */ - public boolean fastMap() { - return fastMap; - } - - /** - * @return Update version for fast-map request. - */ - public GridCacheVersion updateVersion() { + /** {@inheritDoc} */ + @Override public GridCacheVersion updateVersion() { return updateVer; } - /** - * @return Topology version. - */ + /** {@inheritDoc} */ @Override public AffinityTopologyVersion topologyVersion() { return topVer; } - /** - * @return Topology locked flag. - */ - public boolean topologyLocked() { - return topLocked; + /** {@inheritDoc} */ + @Override public CacheWriteSynchronizationMode writeSynchronizationMode() { + return syncMode; } - /** - * @return {@code True} if request sent from client node. - */ - public boolean clientRequest() { - return clientReq; + /** {@inheritDoc} */ + @Override public GridCacheOperation operation() { + return op; } - /** - * @return Cache write synchronization mode. - */ - public CacheWriteSynchronizationMode writeSynchronizationMode() { - return syncMode; - } + /** {@inheritDoc} */ + @Override public boolean onResponse(GridNearAtomicUpdateResponse res) { + if (this.res == null) { + this.res = res; - /** - * @return Expiry policy. - */ - public ExpiryPolicy expiry() { - return expiryPlc; - } + return true; + } - /** - * @return Return value flag. - */ - public boolean returnValue() { - return retval; + return false; } - /** - * @return Filter. - */ - @Nullable public CacheEntryPredicate[] filter() { - return filter; + /** {@inheritDoc} */ + @Override @Nullable public GridNearAtomicUpdateResponse response() { + return res; } - /** - * @return Skip write-through to a persistent storage. - */ - public boolean skipStore() { - return skipStore; + /** {@inheritDoc} */ + @Override public boolean addDeploymentInfo() { + return addDepInfo; } - /** - * @return Keep binary flag. - */ - public boolean keepBinary() { - return keepBinary; + /** {@inheritDoc} */ + @Override public IgniteLogger messageLogger(GridCacheSharedContext ctx) { + return ctx.atomicMessageLogger(); } - /** - * @param key Key to add. - * @param val Optional update value. - * @param conflictTtl Conflict TTL (optional). - * @param conflictExpireTime Conflict expire time (optional). - * @param conflictVer Conflict version (optional). - * @param primary If given key is primary on this mapping. - */ - public void addUpdateEntry(KeyCacheObject key, + /** {@inheritDoc} */ + @Override public void addUpdateEntry(KeyCacheObject key, @Nullable Object val, long conflictTtl, long conflictExpireTime, @@ -453,79 +400,57 @@ else if (conflictVers != null) } } - /** - * @return Keys for this update request. - */ - public List keys() { + /** {@inheritDoc} */ + @Override public List keys() { return keys; } - /** - * @return Values for this update request. - */ - public List values() { - return op == TRANSFORM ? entryProcessors : vals; + /** {@inheritDoc} */ + @Override public int size() { + return keys != null ? keys.size() : 0; } - /** - * @return Update operation. - */ - public GridCacheOperation operation() { - return op; + /** {@inheritDoc} */ + @Override public KeyCacheObject key(int idx) { + return keys.get(idx); } - /** - * @return Optional arguments for entry processor. - */ - @Nullable public Object[] invokeArguments() { - return invokeArgs; + /** {@inheritDoc} */ + @Override public List values() { + return op == TRANSFORM ? entryProcessors : vals; } - /** - * @param idx Key index. - * @return Value. - */ + /** {@inheritDoc} */ @SuppressWarnings("unchecked") - public CacheObject value(int idx) { + @Override public CacheObject value(int idx) { assert op == UPDATE : op; return vals.get(idx); } - /** - * @param idx Key index. - * @return Entry processor. - */ + /** {@inheritDoc} */ @SuppressWarnings("unchecked") - public EntryProcessor entryProcessor(int idx) { + @Override public EntryProcessor entryProcessor(int idx) { assert op == TRANSFORM : op; return entryProcessors.get(idx); } - /** - * @param idx Index to get. - * @return Write value - either value, or transform closure. - */ - public CacheObject writeValue(int idx) { + /** {@inheritDoc} */ + @Override public CacheObject writeValue(int idx) { if (vals != null) return vals.get(idx); return null; } - /** - * @return Conflict versions. - */ - @Nullable public List conflictVersions() { + /** {@inheritDoc} */ + @Override @Nullable public List conflictVersions() { return conflictVers; } - /** - * @param idx Index. - * @return Conflict version. - */ - @Nullable public GridCacheVersion conflictVersion(int idx) { + /** {@inheritDoc} */ + @Override @Nullable public GridCacheVersion conflictVersion(int idx) { if (conflictVers != null) { assert idx >= 0 && idx < conflictVers.size(); @@ -535,11 +460,8 @@ public CacheObject writeValue(int idx) { return null; } - /** - * @param idx Index. - * @return Conflict TTL. - */ - public long conflictTtl(int idx) { + /** {@inheritDoc} */ + @Override public long conflictTtl(int idx) { if (conflictTtls != null) { assert idx >= 0 && idx < conflictTtls.size(); @@ -549,11 +471,8 @@ public long conflictTtl(int idx) { return CU.TTL_NOT_CHANGED; } - /** - * @param idx Index. - * @return Conflict expire time. - */ - public long conflictExpireTime(int idx) { + /** {@inheritDoc} */ + @Override public long conflictExpireTime(int idx) { if (conflictExpireTimes != null) { assert idx >= 0 && idx < conflictExpireTimes.size(); @@ -563,36 +482,69 @@ public long conflictExpireTime(int idx) { return CU.EXPIRE_TIME_CALCULATE; } + /** {@inheritDoc} */ + @Override @Nullable public Object[] invokeArguments() { + return invokeArgs; + } + /** - * @return Flag indicating whether this request contains primary keys. + * @return Flag indicating whether this is fast-map udpate. */ - public boolean hasPrimary() { - return hasPrimary; + @Override public boolean fastMap() { + return fastMap; } /** - * @param res Response. - * @return {@code True} if current response was {@code null}. + * @return Topology locked flag. */ - public boolean onResponse(GridNearAtomicUpdateResponse res) { - if (this.res == null) { - this.res = res; + @Override public boolean topologyLocked() { + return topLocked; + } - return true; - } + /** + * @return {@code True} if request sent from client node. + */ + @Override public boolean clientRequest() { + return clientReq; + } - return false; + /** + * @return Return value flag. + */ + @Override public boolean returnValue() { + return retval; } /** - * @return Response. + * @return Skip write-through to a persistent storage. */ - @Nullable public GridNearAtomicUpdateResponse response() { - return res; + @Override public boolean skipStore() { + return skipStore; + } + + /** + * @return Keep binary flag. + */ + @Override public boolean keepBinary() { + return keepBinary; + } + + /** {@inheritDoc} */ + @Override public boolean hasPrimary() { + return hasPrimary; + } + + /** {@inheritDoc} */ + @Nullable @Override public CacheEntryPredicate[] filter() { + return filter; } - /** {@inheritDoc} - * @param ctx*/ + /** {@inheritDoc} */ + @Override public ExpiryPolicy expiry() { + return expiryPlc; + } + + /** {@inheritDoc} */ @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { super.prepareMarshal(ctx); @@ -669,16 +621,6 @@ public boolean onResponse(GridNearAtomicUpdateResponse res) { } } - /** {@inheritDoc} */ - @Override public boolean addDeploymentInfo() { - return addDepInfo; - } - - /** {@inheritDoc} */ - @Override public IgniteLogger messageLogger(GridCacheSharedContext ctx) { - return ctx.atomicMessageLogger(); - } - /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { writer.setBuffer(buf); @@ -779,64 +721,65 @@ public boolean onResponse(GridNearAtomicUpdateResponse res) { writer.incrementState(); case 17: - if (!writer.writeBoolean("retval", retval)) + if (!writer.writeCollection("partIds", partIds, MessageCollectionItemType.INT)) return false; writer.incrementState(); case 18: - if (!writer.writeBoolean("skipStore", skipStore)) + if (!writer.writeBoolean("retval", retval)) return false; writer.incrementState(); case 19: - if (!writer.writeUuid("subjId", subjId)) + if (!writer.writeBoolean("skipStore", skipStore)) return false; writer.incrementState(); case 20: - if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1)) + if (!writer.writeUuid("subjId", subjId)) return false; writer.incrementState(); case 21: - if (!writer.writeInt("taskNameHash", taskNameHash)) + if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1)) return false; writer.incrementState(); case 22: - if (!writer.writeBoolean("topLocked", topLocked)) + if (!writer.writeInt("taskNameHash", taskNameHash)) return false; writer.incrementState(); case 23: - if (!writer.writeMessage("topVer", topVer)) + if (!writer.writeBoolean("topLocked", topLocked)) return false; writer.incrementState(); case 24: - if (!writer.writeMessage("updateVer", updateVer)) + if (!writer.writeMessage("topVer", topVer)) return false; writer.incrementState(); case 25: - if (!writer.writeCollection("vals", vals, MessageCollectionItemType.MSG)) + if (!writer.writeMessage("updateVer", updateVer)) return false; writer.incrementState(); case 26: - if (!writer.writeCollection("partIds", partIds, MessageCollectionItemType.INT)) + if (!writer.writeCollection("vals", vals, MessageCollectionItemType.MSG)) return false; writer.incrementState(); + } return true; @@ -970,7 +913,7 @@ public boolean onResponse(GridNearAtomicUpdateResponse res) { reader.incrementState(); case 17: - retval = reader.readBoolean("retval"); + partIds = reader.readCollection("partIds", MessageCollectionItemType.INT); if (!reader.isLastRead()) return false; @@ -978,7 +921,7 @@ public boolean onResponse(GridNearAtomicUpdateResponse res) { reader.incrementState(); case 18: - skipStore = reader.readBoolean("skipStore"); + retval = reader.readBoolean("retval"); if (!reader.isLastRead()) return false; @@ -986,7 +929,7 @@ public boolean onResponse(GridNearAtomicUpdateResponse res) { reader.incrementState(); case 19: - subjId = reader.readUuid("subjId"); + skipStore = reader.readBoolean("skipStore"); if (!reader.isLastRead()) return false; @@ -994,27 +937,27 @@ public boolean onResponse(GridNearAtomicUpdateResponse res) { reader.incrementState(); case 20: - byte syncModeOrd; - - syncModeOrd = reader.readByte("syncMode"); + subjId = reader.readUuid("subjId"); if (!reader.isLastRead()) return false; - syncMode = CacheWriteSynchronizationMode.fromOrdinal(syncModeOrd); - reader.incrementState(); case 21: - taskNameHash = reader.readInt("taskNameHash"); + byte syncModeOrd; + + syncModeOrd = reader.readByte("syncMode"); if (!reader.isLastRead()) return false; + syncMode = CacheWriteSynchronizationMode.fromOrdinal(syncModeOrd); + reader.incrementState(); case 22: - topLocked = reader.readBoolean("topLocked"); + taskNameHash = reader.readInt("taskNameHash"); if (!reader.isLastRead()) return false; @@ -1022,7 +965,7 @@ public boolean onResponse(GridNearAtomicUpdateResponse res) { reader.incrementState(); case 23: - topVer = reader.readMessage("topVer"); + topLocked = reader.readBoolean("topLocked"); if (!reader.isLastRead()) return false; @@ -1030,7 +973,7 @@ public boolean onResponse(GridNearAtomicUpdateResponse res) { reader.incrementState(); case 24: - updateVer = reader.readMessage("updateVer"); + topVer = reader.readMessage("topVer"); if (!reader.isLastRead()) return false; @@ -1038,7 +981,7 @@ public boolean onResponse(GridNearAtomicUpdateResponse res) { reader.incrementState(); case 25: - vals = reader.readCollection("vals", MessageCollectionItemType.MSG); + updateVer = reader.readMessage("updateVer"); if (!reader.isLastRead()) return false; @@ -1046,7 +989,7 @@ public boolean onResponse(GridNearAtomicUpdateResponse res) { reader.incrementState(); case 26: - partIds = reader.readCollection("partIds", MessageCollectionItemType.INT); + vals = reader.readCollection("vals", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -1055,15 +998,11 @@ public boolean onResponse(GridNearAtomicUpdateResponse res) { } - return reader.afterMessageRead(GridNearAtomicUpdateRequest.class); + return reader.afterMessageRead(GridNearAtomicFullUpdateRequest.class); } - /** - * Cleanup values. - * - * @param clearKeys If {@code true} clears keys. - */ - public void cleanup(boolean clearKeys) { + /** {@inheritDoc} */ + @Override public void cleanup(boolean clearKeys) { vals = null; entryProcessors = null; entryProcessorsBytes = null; @@ -1086,7 +1025,7 @@ public void cleanup(boolean clearKeys) { /** {@inheritDoc} */ @Override public String toString() { - return S.toString(GridNearAtomicUpdateRequest.class, this, "filter", Arrays.toString(filter), + return S.toString(GridNearAtomicFullUpdateRequest.class, this, "filter", Arrays.toString(filter), "parent", super.toString()); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFilterRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFilterRequest.java new file mode 100644 index 0000000000000..e0c24b2c5c217 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFilterRequest.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; + +import java.io.Externalizable; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheEntryPredicate; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheOperation; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public class GridNearAtomicSingleUpdateFilterRequest extends GridNearAtomicSingleUpdateRequest { + /** */ + private static final long serialVersionUID = 0L; + + /** Filter. */ + private CacheEntryPredicate[] filter; + + /** + * Empty constructor required by {@link Externalizable}. + */ + public GridNearAtomicSingleUpdateFilterRequest() { + // No-op. + } + + /** + * Constructor. + * + * @param cacheId Cache ID. + * @param nodeId Node ID. + * @param futVer Future version. + * @param fastMap Fast map scheme flag. + * @param updateVer Update version set if fast map is performed. + * @param topVer Topology version. + * @param topLocked Topology locked flag. + * @param syncMode Synchronization mode. + * @param op Cache update operation. + * @param retval Return value required flag. + * @param filter Optional filter for atomic check. + * @param subjId Subject ID. + * @param taskNameHash Task name hash code. + * @param skipStore Skip write-through to a persistent storage. + * @param keepBinary Keep binary flag. + * @param clientReq Client node request flag. + * @param addDepInfo Deployment info flag. + */ + GridNearAtomicSingleUpdateFilterRequest( + int cacheId, + UUID nodeId, + GridCacheVersion futVer, + boolean fastMap, + @Nullable GridCacheVersion updateVer, + @NotNull AffinityTopologyVersion topVer, + boolean topLocked, + CacheWriteSynchronizationMode syncMode, + GridCacheOperation op, + boolean retval, + @Nullable CacheEntryPredicate[] filter, + @Nullable UUID subjId, + int taskNameHash, + boolean skipStore, + boolean keepBinary, + boolean clientReq, + boolean addDepInfo + ) { + super( + cacheId, + nodeId, + futVer, + fastMap, + updateVer, + topVer, + topLocked, + syncMode, + op, + retval, + subjId, + taskNameHash, + skipStore, + keepBinary, + clientReq, + addDepInfo + ); + + assert filter != null && filter.length > 0; + + this.filter = filter; + } + + /** {@inheritDoc} */ + @Override @Nullable public CacheEntryPredicate[] filter() { + return filter; + } + + /** {@inheritDoc} */ + @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { + super.prepareMarshal(ctx); + + GridCacheContext cctx = ctx.cacheContext(cacheId); + + if (filter != null) { + boolean hasFilter = false; + + for (CacheEntryPredicate p : filter) { + if (p != null) { + hasFilter = true; + + p.prepareMarshal(cctx); + } + } + + if (!hasFilter) + filter = null; + } + } + + /** {@inheritDoc} */ + @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { + super.finishUnmarshal(ctx, ldr); + + if (filter != null) { + GridCacheContext cctx = ctx.cacheContext(cacheId); + + for (CacheEntryPredicate p : filter) { + if (p != null) + p.finishUnmarshal(cctx, ldr); + } + } + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 14: + if (!writer.writeObjectArray("filter", filter, MessageCollectionItemType.MSG)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 14: + filter = reader.readObjectArray("filter", MessageCollectionItemType.MSG, CacheEntryPredicate.class); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridNearAtomicSingleUpdateFilterRequest.class); + } + + /** {@inheritDoc} */ + @Override public byte directType() { + return 127; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 15; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridNearAtomicSingleUpdateFilterRequest.class, this, "filter", Arrays.toString(filter), + "parent", super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java index 256c7ac01417d..eaf2f2c9a6e2d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java @@ -49,6 +49,7 @@ import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteProductVersion; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK; @@ -58,6 +59,9 @@ * DHT atomic cache near update future. */ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpdateFuture { + /** */ + private static final IgniteProductVersion SINGLE_UPDATE_REQUEST = IgniteProductVersion.fromString("1.7.4"); + /** Keys */ private Object key; @@ -66,7 +70,7 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda private Object val; /** Not null is operation is mapped to single node. */ - private GridNearAtomicUpdateRequest req; + private GridNearAtomicAbstractUpdateRequest req; /** * @param cctx Cache context. @@ -126,7 +130,7 @@ public GridNearAtomicSingleUpdateFuture( @Override public boolean onNodeLeft(UUID nodeId) { GridNearAtomicUpdateResponse res = null; - GridNearAtomicUpdateRequest req; + GridNearAtomicAbstractUpdateRequest req; synchronized (mux) { req = this.req != null && this.req.nodeId().equals(nodeId) ? this.req : null; @@ -192,8 +196,8 @@ public GridNearAtomicSingleUpdateFuture( /** {@inheritDoc} */ @SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"}) - @Override public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) { - GridNearAtomicUpdateRequest req; + @Override public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) { + GridNearAtomicAbstractUpdateRequest req; AffinityTopologyVersion remapTopVer = null; @@ -369,7 +373,7 @@ else if (res.error() != null) { * @param req Update request. * @param res Update response. */ - private void updateNear(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateResponse res) { + private void updateNear(GridNearAtomicAbstractUpdateRequest req, GridNearAtomicUpdateResponse res) { assert nearEnabled; if (res.remapKeys() != null || !req.hasPrimary()) @@ -446,7 +450,7 @@ protected void map(AffinityTopologyVersion topVer) { } Exception err = null; - GridNearAtomicUpdateRequest singleReq0 = null; + GridNearAtomicAbstractUpdateRequest singleReq0 = null; GridCacheVersion futVer = cctx.versions().next(topVer); @@ -535,7 +539,7 @@ GridCacheVersion onFutureDone() { * @return Request. * @throws Exception If failed. */ - private GridNearAtomicUpdateRequest mapSingleUpdate(AffinityTopologyVersion topVer, + private GridNearAtomicAbstractUpdateRequest mapSingleUpdate(AffinityTopologyVersion topVer, GridCacheVersion futVer, @Nullable GridCacheVersion updVer) throws Exception { if (key == null) @@ -559,27 +563,94 @@ private GridNearAtomicUpdateRequest mapSingleUpdate(AffinityTopologyVersion topV throw new ClusterTopologyServerNotFoundException("Failed to map keys for cache (all partition nodes " + "left the grid)."); - GridNearAtomicUpdateRequest req = new GridNearAtomicUpdateRequest( - cctx.cacheId(), - primary.id(), - futVer, - false, - updVer, - topVer, - topLocked, - syncMode, - op, - retval, - expiryPlc, - invokeArgs, - filter, - subjId, - taskNameHash, - skipStore, - keepBinary, - cctx.kernalContext().clientNode(), - cctx.deploymentEnabled(), - 1); + GridNearAtomicAbstractUpdateRequest req; + + if (canUseSingleRequest(primary)) { + if (op == TRANSFORM) { + req = new GridNearAtomicSingleUpdateInvokeRequest( + cctx.cacheId(), + primary.id(), + futVer, + false, + updVer, + topVer, + topLocked, + syncMode, + op, + retval, + invokeArgs, + subjId, + taskNameHash, + skipStore, + keepBinary, + cctx.kernalContext().clientNode(), + cctx.deploymentEnabled()); + } + else { + if (filter == null || filter.length == 0) { + req = new GridNearAtomicSingleUpdateRequest( + cctx.cacheId(), + primary.id(), + futVer, + false, + updVer, + topVer, + topLocked, + syncMode, + op, + retval, + subjId, + taskNameHash, + skipStore, + keepBinary, + cctx.kernalContext().clientNode(), + cctx.deploymentEnabled()); + } + else { + req = new GridNearAtomicSingleUpdateFilterRequest( + cctx.cacheId(), + primary.id(), + futVer, + false, + updVer, + topVer, + topLocked, + syncMode, + op, + retval, + filter, + subjId, + taskNameHash, + skipStore, + keepBinary, + cctx.kernalContext().clientNode(), + cctx.deploymentEnabled()); + } + } + } + else { + req = new GridNearAtomicFullUpdateRequest( + cctx.cacheId(), + primary.id(), + futVer, + false, + updVer, + topVer, + topLocked, + syncMode, + op, + retval, + expiryPlc, + invokeArgs, + filter, + subjId, + taskNameHash, + skipStore, + keepBinary, + cctx.kernalContext().clientNode(), + cctx.deploymentEnabled(), + 1); + } req.addUpdateEntry(cacheKey, val, @@ -591,6 +662,16 @@ private GridNearAtomicUpdateRequest mapSingleUpdate(AffinityTopologyVersion topV return req; } + /** + * @param node Target node + * @return {@code True} can use 'single' update requests. + */ + private boolean canUseSingleRequest(ClusterNode node) { + assert node != null; + + return expiryPlc == null && node.version().compareToIgnoreTimestamp(SINGLE_UPDATE_REQUEST) >= 0; + } + /** {@inheritDoc} */ public String toString() { synchronized (mux) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java new file mode 100644 index 0000000000000..42b51d6161ec3 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java @@ -0,0 +1,303 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; + +import java.io.Externalizable; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import javax.cache.processor.EntryProcessor; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.internal.GridDirectTransient; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheOperation; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM; + +/** + * + */ +public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingleUpdateRequest { + /** */ + private static final long serialVersionUID = 0L; + + /** Optional arguments for entry processor. */ + @GridDirectTransient + private Object[] invokeArgs; + + /** Entry processor arguments bytes. */ + private byte[][] invokeArgsBytes; + + /** Entry processors. */ + @GridDirectTransient + private EntryProcessor entryProcessor; + + /** Entry processors bytes. */ + private byte[] entryProcessorBytes; + + /** + * Empty constructor required by {@link Externalizable}. + */ + public GridNearAtomicSingleUpdateInvokeRequest() { + // No-op. + } + + /** + * Constructor. + * + * @param cacheId Cache ID. + * @param nodeId Node ID. + * @param futVer Future version. + * @param fastMap Fast map scheme flag. + * @param updateVer Update version set if fast map is performed. + * @param topVer Topology version. + * @param topLocked Topology locked flag. + * @param syncMode Synchronization mode. + * @param op Cache update operation. + * @param retval Return value required flag. + * @param invokeArgs Optional arguments for entry processor. + * @param subjId Subject ID. + * @param taskNameHash Task name hash code. + * @param skipStore Skip write-through to a persistent storage. + * @param keepBinary Keep binary flag. + * @param clientReq Client node request flag. + * @param addDepInfo Deployment info flag. + */ + GridNearAtomicSingleUpdateInvokeRequest( + int cacheId, + UUID nodeId, + GridCacheVersion futVer, + boolean fastMap, + @Nullable GridCacheVersion updateVer, + @NotNull AffinityTopologyVersion topVer, + boolean topLocked, + CacheWriteSynchronizationMode syncMode, + GridCacheOperation op, + boolean retval, + @Nullable Object[] invokeArgs, + @Nullable UUID subjId, + int taskNameHash, + boolean skipStore, + boolean keepBinary, + boolean clientReq, + boolean addDepInfo + ) { + super( + cacheId, + nodeId, + futVer, + fastMap, + updateVer, + topVer, + topLocked, + syncMode, + op, + retval, + subjId, + taskNameHash, + skipStore, + keepBinary, + clientReq, + addDepInfo + ); + this.invokeArgs = invokeArgs; + + assert op == TRANSFORM : op; + } + + /** + * @param key Key to add. + * @param val Optional update value. + * @param conflictTtl Conflict TTL (optional). + * @param conflictExpireTime Conflict expire time (optional). + * @param conflictVer Conflict version (optional). + * @param primary If given key is primary on this mapping. + */ + @Override public void addUpdateEntry(KeyCacheObject key, + @Nullable Object val, + long conflictTtl, + long conflictExpireTime, + @Nullable GridCacheVersion conflictVer, + boolean primary) { + assert conflictTtl < 0 : conflictTtl; + assert conflictExpireTime < 0 : conflictExpireTime; + assert conflictVer == null : conflictVer; + assert val instanceof EntryProcessor : val; + + entryProcessor = (EntryProcessor)val; + + this.key = key; + partId = key.partition(); + + hasPrimary(hasPrimary() | primary); + } + + /** {@inheritDoc} */ + @Override public List values() { + return Collections.singletonList(entryProcessor); + } + + /** {@inheritDoc} */ + @Override public CacheObject value(int idx) { + assert idx == 0 : idx; + + return null; + } + + /** {@inheritDoc} */ + @Override public EntryProcessor entryProcessor(int idx) { + assert idx == 0 : idx; + + return entryProcessor; + } + + /** {@inheritDoc} */ + @Override public CacheObject writeValue(int idx) { + assert idx == 0 : idx; + + return null; + } + + /** {@inheritDoc} */ + @Nullable @Override public Object[] invokeArguments() { + return invokeArgs; + } + + /** {@inheritDoc} */ + @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { + super.prepareMarshal(ctx); + + GridCacheContext cctx = ctx.cacheContext(cacheId); + + // force addition of deployment info for entry processors if P2P is enabled globally. + if (!addDepInfo && ctx.deploymentEnabled()) + addDepInfo = true; + + if (entryProcessor != null && entryProcessorBytes == null) + entryProcessorBytes = CU.marshal(cctx, entryProcessor); + + if (invokeArgsBytes == null) + invokeArgsBytes = marshalInvokeArguments(invokeArgs, cctx); + } + + /** {@inheritDoc} */ + @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { + super.finishUnmarshal(ctx, ldr); + + if (entryProcessorBytes != null && entryProcessor == null) + entryProcessor = U.unmarshal(ctx, entryProcessorBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + + if (invokeArgs == null) + invokeArgs = unmarshalInvokeArguments(invokeArgsBytes, ctx, ldr); + } + + /** {@inheritDoc} */ + @Override public void cleanup(boolean clearKey) { + super.cleanup(clearKey); + + entryProcessor = null; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 14: + if (!writer.writeByteArray("entryProcessorBytes", entryProcessorBytes)) + return false; + + writer.incrementState(); + + case 15: + if (!writer.writeObjectArray("invokeArgsBytes", invokeArgsBytes, MessageCollectionItemType.BYTE_ARR)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 14: + entryProcessorBytes = reader.readByteArray("entryProcessorBytes"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 15: + invokeArgsBytes = reader.readObjectArray("invokeArgsBytes", MessageCollectionItemType.BYTE_ARR, byte[].class); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridNearAtomicSingleUpdateInvokeRequest.class); + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 16; + } + + /** {@inheritDoc} */ + @Override public byte directType() { + return 126; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java new file mode 100644 index 0000000000000..211b472ad027c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; + +import java.io.Externalizable; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import javax.cache.processor.EntryProcessor; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheOperation; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE; +import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM; + +/** + * + */ +public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSingleUpdateRequest { + /** */ + private static final long serialVersionUID = 0L; + + /** Key to update. */ + @GridToStringInclude + protected KeyCacheObject key; + + /** Value to update. */ + protected CacheObject val; + + /** Partition of key. */ + protected int partId; + + /** + * Empty constructor required by {@link Externalizable}. + */ + public GridNearAtomicSingleUpdateRequest() { + // No-op. + } + + /** + * Constructor. + * + * @param cacheId Cache ID. + * @param nodeId Node ID. + * @param futVer Future version. + * @param fastMap Fast map scheme flag. + * @param updateVer Update version set if fast map is performed. + * @param topVer Topology version. + * @param topLocked Topology locked flag. + * @param syncMode Synchronization mode. + * @param op Cache update operation. + * @param retval Return value required flag. + * @param subjId Subject ID. + * @param taskNameHash Task name hash code. + * @param skipStore Skip write-through to a persistent storage. + * @param keepBinary Keep binary flag. + * @param clientReq Client node request flag. + * @param addDepInfo Deployment info flag. + */ + GridNearAtomicSingleUpdateRequest( + int cacheId, + UUID nodeId, + GridCacheVersion futVer, + boolean fastMap, + @Nullable GridCacheVersion updateVer, + @NotNull AffinityTopologyVersion topVer, + boolean topLocked, + CacheWriteSynchronizationMode syncMode, + GridCacheOperation op, + boolean retval, + @Nullable UUID subjId, + int taskNameHash, + boolean skipStore, + boolean keepBinary, + boolean clientReq, + boolean addDepInfo + ) { + super( + cacheId, + nodeId, + futVer, + fastMap, + updateVer, + topVer, + topLocked, + syncMode, + op, + retval, + subjId, + taskNameHash, + skipStore, + keepBinary, + clientReq, + addDepInfo + ); + } + + /** + * @param key Key to add. + * @param val Optional update value. + * @param conflictTtl Conflict TTL (optional). + * @param conflictExpireTime Conflict expire time (optional). + * @param conflictVer Conflict version (optional). + * @param primary If given key is primary on this mapping. + */ + @Override public void addUpdateEntry(KeyCacheObject key, + @Nullable Object val, + long conflictTtl, + long conflictExpireTime, + @Nullable GridCacheVersion conflictVer, + boolean primary) { + assert op != TRANSFORM; + assert val != null || op == DELETE; + assert conflictTtl < 0 : conflictTtl; + assert conflictExpireTime < 0 : conflictExpireTime; + assert conflictVer == null : conflictVer; + + this.key = key; + partId = key.partition(); + + if (val != null) { + assert val instanceof CacheObject : val; + + this.val = (CacheObject)val; + } + + hasPrimary(hasPrimary() | primary); + } + + /** {@inheritDoc} */ + @Override public int size() { + return key == null ? 0 : 1; + } + + /** {@inheritDoc} */ + @Override public List keys() { + return Collections.singletonList(key); + } + + /** {@inheritDoc} */ + @Override public KeyCacheObject key(int idx) { + assert idx == 0 : idx; + + return key; + } + + /** {@inheritDoc} */ + @Override public List values() { + return Collections.singletonList(val); + } + + /** {@inheritDoc} */ + @Override public CacheObject value(int idx) { + assert idx == 0 : idx; + + return val; + } + + /** {@inheritDoc} */ + @Override public EntryProcessor entryProcessor(int idx) { + assert idx == 0 : idx; + + return null; + } + + /** {@inheritDoc} */ + @Override public CacheObject writeValue(int idx) { + assert idx == 0 : idx; + + return val; + } + + /** {@inheritDoc} */ + @Nullable @Override public List conflictVersions() { + return null; + } + + /** {@inheritDoc} */ + @Nullable @Override public GridCacheVersion conflictVersion(int idx) { + assert idx == 0 : idx; + + return null; + } + + /** {@inheritDoc} */ + @Override public long conflictTtl(int idx) { + assert idx == 0 : idx; + + return CU.TTL_NOT_CHANGED; + } + + /** {@inheritDoc} */ + @Override public long conflictExpireTime(int idx) { + assert idx == 0 : idx; + + return CU.EXPIRE_TIME_CALCULATE; + } + + /** + * {@inheritDoc} + * + * @param ctx + */ + @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { + super.prepareMarshal(ctx); + + GridCacheContext cctx = ctx.cacheContext(cacheId); + + prepareMarshalCacheObject(key, cctx); + + if (val != null) + prepareMarshalCacheObject(val, cctx); + } + + /** {@inheritDoc} */ + @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { + super.finishUnmarshal(ctx, ldr); + + GridCacheContext cctx = ctx.cacheContext(cacheId); + + key.finishUnmarshal(cctx.cacheObjectContext(), ldr); + + if (val != null) + val.finishUnmarshal(cctx.cacheObjectContext(), ldr); + + key.partition(partId); + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 11: + if (!writer.writeMessage("key", key)) + return false; + + writer.incrementState(); + + case 12: + if (!writer.writeInt("partId", partId)) + return false; + + writer.incrementState(); + + case 13: + if (!writer.writeMessage("val", val)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 11: + key = reader.readMessage("key"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 12: + partId = reader.readInt("partId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 13: + val = reader.readMessage("val"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridNearAtomicSingleUpdateRequest.class); + } + + /** {@inheritDoc} */ + @Override public void cleanup(boolean clearKey) { + val = null; + + if (clearKey) + key = null; + } + + /** {@inheritDoc} */ + @Override public byte directType() { + return 125; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 14; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridNearAtomicSingleUpdateRequest.class, this, "parent", super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java index 30a0c3d9156a6..cd6411725748b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java @@ -87,13 +87,13 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu /** Mappings if operations is mapped to more than one node. */ @GridToStringInclude - private Map mappings; + private Map mappings; /** Keys to remap. */ private Collection remapKeys; /** Not null is operation is mapped to single node. */ - private GridNearAtomicUpdateRequest singleReq; + private GridNearAtomicFullUpdateRequest singleReq; /** * @param cctx Cache context. @@ -164,7 +164,7 @@ public GridNearAtomicUpdateFuture( @Override public boolean onNodeLeft(UUID nodeId) { GridNearAtomicUpdateResponse res = null; - GridNearAtomicUpdateRequest req; + GridNearAtomicFullUpdateRequest req; synchronized (mux) { if (singleReq != null) @@ -258,7 +258,7 @@ public GridNearAtomicUpdateFuture( /** {@inheritDoc} */ @SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"}) @Override public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) { - GridNearAtomicUpdateRequest req; + GridNearAtomicFullUpdateRequest req; AffinityTopologyVersion remapTopVer = null; @@ -406,7 +406,7 @@ else if (res.error() != null) { if (rcvAll && nearEnabled) { if (mappings != null) { - for (GridNearAtomicUpdateRequest req0 : mappings.values()) { + for (GridNearAtomicFullUpdateRequest req0 : mappings.values()) { GridNearAtomicUpdateResponse res0 = req0.response(); assert res0 != null : req0; @@ -482,7 +482,7 @@ else if (!nodeErr) * @param req Update request. * @param res Update response. */ - private void updateNear(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateResponse res) { + private void updateNear(GridNearAtomicFullUpdateRequest req, GridNearAtomicUpdateResponse res) { assert nearEnabled; if (res.remapKeys() != null || !req.hasPrimary()) @@ -552,13 +552,13 @@ private void updateNear(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateRes * * @param mappings Mappings to send. */ - private void doUpdate(Map mappings) { + private void doUpdate(Map mappings) { UUID locNodeId = cctx.localNodeId(); - GridNearAtomicUpdateRequest locUpdate = null; + GridNearAtomicFullUpdateRequest locUpdate = null; // Send messages to remote nodes first, then run local update. - for (GridNearAtomicUpdateRequest req : mappings.values()) { + for (GridNearAtomicFullUpdateRequest req : mappings.values()) { if (locNodeId.equals(req.nodeId())) { assert locUpdate == null : "Cannot have more than one local mapping [locUpdate=" + locUpdate + ", req=" + req + ']'; @@ -590,8 +590,8 @@ private void doUpdate(Map mappings) { if (locUpdate != null) { cache.updateAllAsyncInternal(cctx.localNodeId(), locUpdate, - new CI2() { - @Override public void apply(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateResponse res) { + new CI2() { + @Override public void apply(GridNearAtomicFullUpdateRequest req, GridNearAtomicUpdateResponse res) { onResult(res.nodeId(), res, false); } }); @@ -621,8 +621,8 @@ void map(AffinityTopologyVersion topVer, @Nullable Collection re } Exception err = null; - GridNearAtomicUpdateRequest singleReq0 = null; - Map mappings0 = null; + GridNearAtomicFullUpdateRequest singleReq0 = null; + Map mappings0 = null; int size = keys.size(); @@ -651,7 +651,7 @@ void map(AffinityTopologyVersion topVer, @Nullable Collection re singleReq0 = mapSingleUpdate(topVer, futVer, updVer); } else { - Map pendingMappings = mapUpdate(topNodes, + Map pendingMappings = mapUpdate(topNodes, topVer, futVer, updVer, @@ -663,7 +663,7 @@ void map(AffinityTopologyVersion topVer, @Nullable Collection re if (syncMode == PRIMARY_SYNC) { mappings0 = U.newHashMap(pendingMappings.size()); - for (GridNearAtomicUpdateRequest req : pendingMappings.values()) { + for (GridNearAtomicFullUpdateRequest req : pendingMappings.values()) { if (req.hasPrimary()) mappings0.put(req.nodeId(), req); } @@ -756,7 +756,7 @@ GridCacheVersion onFutureDone() { * @throws Exception If failed. */ @SuppressWarnings("ForLoopReplaceableByForEach") - private Map mapUpdate(Collection topNodes, + private Map mapUpdate(Collection topNodes, AffinityTopologyVersion topVer, GridCacheVersion futVer, @Nullable GridCacheVersion updVer, @@ -776,7 +776,7 @@ private Map mapUpdate(Collection if (conflictRmvVals != null) conflictRmvValsIt = conflictRmvVals.iterator(); - Map pendingMappings = U.newHashMap(topNodes.size()); + Map pendingMappings = U.newHashMap(topNodes.size()); // Create mappings first, then send messages. for (Object key : keys) { @@ -848,10 +848,10 @@ else if (conflictRmvVals != null) { UUID nodeId = affNode.id(); - GridNearAtomicUpdateRequest mapped = pendingMappings.get(nodeId); + GridNearAtomicFullUpdateRequest mapped = pendingMappings.get(nodeId); if (mapped == null) { - mapped = new GridNearAtomicUpdateRequest( + mapped = new GridNearAtomicFullUpdateRequest( cctx.cacheId(), nodeId, futVer, @@ -892,7 +892,7 @@ else if (conflictRmvVals != null) { * @return Request. * @throws Exception If failed. */ - private GridNearAtomicUpdateRequest mapSingleUpdate(AffinityTopologyVersion topVer, + private GridNearAtomicFullUpdateRequest mapSingleUpdate(AffinityTopologyVersion topVer, GridCacheVersion futVer, @Nullable GridCacheVersion updVer) throws Exception { Object key = F.first(keys); @@ -953,7 +953,7 @@ else if (conflictRmvVals != null) { throw new ClusterTopologyServerNotFoundException("Failed to map keys for cache (all partition nodes " + "left the grid)."); - GridNearAtomicUpdateRequest req = new GridNearAtomicUpdateRequest( + GridNearAtomicFullUpdateRequest req = new GridNearAtomicFullUpdateRequest( cctx.cacheId(), primary.id(), futVer, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java index 09aec815b8ba2..a9245f0ca42fd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java @@ -50,6 +50,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateRequest; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -661,6 +662,15 @@ public void onPartitionEvicted(GridDhtLocalPartition part, boolean updateSeq) { return true; } + /** {@inheritDoc} */ + @Override public IgniteInternalFuture request(GridNearAtomicAbstractUpdateRequest req, + AffinityTopologyVersion topVer) { + if (!needForceKeys()) + return null; + + return request0(req.keys(), topVer); + } + /** * @param keys Keys to request. * @return Future for request. @@ -670,6 +680,16 @@ public void onPartitionEvicted(GridDhtLocalPartition part, boolean updateSeq) { if (!needForceKeys()) return null; + return request0(keys, topVer); + } + + /** + * @param keys Keys to request. + * @param topVer Topology version. + * @return Future for request. + */ + @SuppressWarnings({"unchecked", "RedundantCast"}) + private GridDhtFuture request0(Collection keys, AffinityTopologyVersion topVer) { final GridDhtForceKeysFuture fut = new GridDhtForceKeysFuture<>(cctx, topVer, keys, this); IgniteInternalFuture topReadyFut = cctx.affinity().affinityReadyFuturex(topVer); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java index 3e0e3924e2a5b..b5b2c728fa4f8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java @@ -44,7 +44,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse; import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx; @@ -126,10 +126,10 @@ public void dht(GridDhtAtomicCache dht) { * @param res Update response. */ public void processNearAtomicUpdateResponse( - GridNearAtomicUpdateRequest req, + GridNearAtomicAbstractUpdateRequest req, GridNearAtomicUpdateResponse res ) { - if (F.size(res.failedKeys()) == req.keys().size()) + if (F.size(res.failedKeys()) == req.size()) return; /* @@ -152,11 +152,11 @@ public void processNearAtomicUpdateResponse( String taskName = ctx.kernalContext().task().resolveTaskName(req.taskNameHash()); - for (int i = 0; i < req.keys().size(); i++) { + for (int i = 0; i < req.size(); i++) { if (F.contains(skipped, i)) continue; - KeyCacheObject key = req.keys().get(i); + KeyCacheObject key = req.key(i); if (F.contains(failed, key)) continue; diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties index 0d291cca3e6d2..71d236fdb8ce4 100644 --- a/modules/core/src/main/resources/META-INF/classnames.properties +++ b/modules/core/src/main/resources/META-INF/classnames.properties @@ -75,13 +75,11 @@ org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy$HolderComparator org.apache.ignite.cache.query.CacheQueryEntryEvent org.apache.ignite.cache.query.ContinuousQuery org.apache.ignite.cache.query.Query -org.apache.ignite.cache.query.QueryCancelledException org.apache.ignite.cache.query.ScanQuery org.apache.ignite.cache.query.SpiQuery org.apache.ignite.cache.query.SqlFieldsQuery org.apache.ignite.cache.query.SqlQuery org.apache.ignite.cache.query.TextQuery -org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$2 org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$EntryMapping$1 org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$EntryMapping$2 org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$TypeKind @@ -368,7 +366,6 @@ org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakQu org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakReferenceCloseableIterator org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest -org.apache.ignite.internal.processors.cache.EntryProcessorResourceInjectorProxy org.apache.ignite.internal.processors.cache.GridCacheAdapter org.apache.ignite.internal.processors.cache.GridCacheAdapter$10 org.apache.ignite.internal.processors.cache.GridCacheAdapter$11 @@ -376,9 +373,9 @@ org.apache.ignite.internal.processors.cache.GridCacheAdapter$12 org.apache.ignite.internal.processors.cache.GridCacheAdapter$13 org.apache.ignite.internal.processors.cache.GridCacheAdapter$14 org.apache.ignite.internal.processors.cache.GridCacheAdapter$15 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$16 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$16$1 org.apache.ignite.internal.processors.cache.GridCacheAdapter$17 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$18$1 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$18 org.apache.ignite.internal.processors.cache.GridCacheAdapter$2 org.apache.ignite.internal.processors.cache.GridCacheAdapter$26$1 org.apache.ignite.internal.processors.cache.GridCacheAdapter$28 @@ -536,7 +533,6 @@ org.apache.ignite.internal.processors.cache.GridCacheUtils$8 org.apache.ignite.internal.processors.cache.GridCacheUtils$9 org.apache.ignite.internal.processors.cache.GridCacheValueCollection org.apache.ignite.internal.processors.cache.GridCacheValueCollection$1 -org.apache.ignite.internal.processors.cache.GridDeferredAckMessageSender$DeferredAckMessageBuffer org.apache.ignite.internal.processors.cache.IgniteCacheProxy org.apache.ignite.internal.processors.cache.IgniteCacheProxy$1 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$10 @@ -549,13 +545,12 @@ org.apache.ignite.internal.processors.cache.IgniteCacheProxy$8 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$9 org.apache.ignite.internal.processors.cache.KeyCacheObject org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl -org.apache.ignite.internal.processors.cache.QueryCursorImpl$State org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityProxy org.apache.ignite.internal.processors.cache.binary.BinaryMetadataKey org.apache.ignite.internal.processors.cache.binary.CacheDefaultBinaryAffinityKeyMapper org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$1 +org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$4 org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$5 -org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$6 org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$MetaDataEntryFilter org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$MetaDataPredicate org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$MetadataProcessor @@ -635,7 +630,6 @@ org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocal$2 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter$1 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter$2 -org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxOnePhaseCommitAckRequest org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$1 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$2 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$3 @@ -676,24 +670,26 @@ org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomic org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$27 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$28 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$29 +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$3 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$30 -org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$31 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$4 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$5 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$6 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$7 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$8 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$9 +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$DeferredResponseBuffer org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse -org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateFuture$1 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture$1 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture$2 +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture$3 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$1 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$2 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$3 -org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$4 +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedCache org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedCache$2 @@ -764,7 +760,6 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$ org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$2 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$3 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$4 -org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$LockTimeoutObject$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$MiniFuture$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockResponse @@ -777,7 +772,6 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticS org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$2 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$3 -org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$4 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$MiniFuture$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFutureAdapter$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearPessimisticTxPrepareFuture$1 @@ -827,23 +821,22 @@ org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManag org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$7 org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$1 org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$ScanQueryFallbackClosableIterator -org.apache.ignite.internal.processors.cache.query.GridCacheQueryDetailMetricsAdapter org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$1 org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$2 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$10 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$1$1 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$1$2 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$11 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$12 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$13 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$14 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$15 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$16 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$17$1 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$3$1 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$3$2 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$15$1 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$2 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$3 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$4 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$5 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$6 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$7 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$8 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$9 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$CacheSqlIndexMetadata org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$CacheSqlMetadata org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$CachedResult$QueueIterator @@ -903,8 +896,6 @@ org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$12 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$13 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$14 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$15 -org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$16 -org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$17 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$2 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$3 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$4 @@ -938,9 +929,9 @@ org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$Po org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostLockClosure1$4 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostLockClosure2 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostMissClosure +org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$2 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$3 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$4 -org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$5 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$CommitListener org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$CommittedVersion org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$NodeFailureTimeoutObject$1 @@ -1014,6 +1005,7 @@ org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$1 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$4 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$5 +org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$6 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$Buffer$1 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$Buffer$2 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$DataStreamerPda @@ -1092,7 +1084,6 @@ org.apache.ignite.internal.processors.datastructures.GridCacheSetProxy org.apache.ignite.internal.processors.datastructures.GridSetQueryPredicate org.apache.ignite.internal.processors.dr.GridDrType org.apache.ignite.internal.processors.dr.IgniteDrDataStreamerCacheUpdater -org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo org.apache.ignite.internal.processors.hadoop.HadoopFileBlock org.apache.ignite.internal.processors.hadoop.HadoopInputSplit org.apache.ignite.internal.processors.hadoop.HadoopJobId @@ -1128,6 +1119,7 @@ org.apache.ignite.internal.processors.igfs.IgfsFragmentizerManager$IdentityHashS org.apache.ignite.internal.processors.igfs.IgfsFragmentizerRequest org.apache.ignite.internal.processors.igfs.IgfsFragmentizerResponse org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse +org.apache.ignite.internal.processors.igfs.IgfsImpl$12$1 org.apache.ignite.internal.processors.igfs.IgfsImpl$IgfsGlobalSpaceTask org.apache.ignite.internal.processors.igfs.IgfsImpl$IgfsGlobalSpaceTask$1 org.apache.ignite.internal.processors.igfs.IgfsInputStreamDescriptor @@ -1163,7 +1155,6 @@ org.apache.ignite.internal.processors.igfs.client.IgfsClientSummaryCallable org.apache.ignite.internal.processors.igfs.client.IgfsClientUpdateCallable org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaIdsForPathCallable org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaInfoForPathCallable -org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaUnlockCallable org.apache.ignite.internal.processors.igfs.data.IgfsDataPutProcessor org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingAddProcessor @@ -1181,8 +1172,6 @@ org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdateTimesProcessor org.apache.ignite.internal.processors.job.GridJobProcessor$5 org.apache.ignite.internal.processors.job.GridJobWorker$3 org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor$SnapshotReducer -org.apache.ignite.internal.processors.odbc.OdbcProtocolVersion -org.apache.ignite.internal.processors.odbc.escape.OdbcEscapeType org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate org.apache.ignite.internal.processors.platform.PlatformEventFilterListener @@ -1191,7 +1180,7 @@ org.apache.ignite.internal.processors.platform.PlatformExtendedException org.apache.ignite.internal.processors.platform.PlatformJavaObjectFactoryProxy org.apache.ignite.internal.processors.platform.PlatformNativeException org.apache.ignite.internal.processors.platform.PlatformNoCallbackException -org.apache.ignite.internal.processors.platform.cache.PlatformCache$5 +org.apache.ignite.internal.processors.platform.cache.PlatformCache$1 org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilterImpl org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessor @@ -1222,8 +1211,6 @@ org.apache.ignite.internal.processors.platform.cpp.PlatformCppConfigurationClosu org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiver org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiverImpl org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$1 -org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$10 -org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$11 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$2 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$3 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$4 @@ -1231,7 +1218,6 @@ org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$5 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$6 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$7 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$8 -org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$9 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetConfigurationClosure org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetService org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetServiceImpl @@ -1246,24 +1232,19 @@ org.apache.ignite.internal.processors.platform.transactions.PlatformTransactions org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$1 org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$FutureListenable$1 org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$InternalFutureListenable$1 -org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionLockProcessor -org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionSetAndUnlockProcessor org.apache.ignite.internal.processors.query.GridQueryFieldMetadata org.apache.ignite.internal.processors.query.GridQueryIndexType +org.apache.ignite.internal.processors.query.GridQueryProcessor$2 org.apache.ignite.internal.processors.query.GridQueryProcessor$3 org.apache.ignite.internal.processors.query.GridQueryProcessor$4 org.apache.ignite.internal.processors.query.GridQueryProcessor$5 org.apache.ignite.internal.processors.query.GridQueryProcessor$6 -org.apache.ignite.internal.processors.query.GridQueryProcessor$7 -org.apache.ignite.internal.processors.query.GridQueryProcessor$8 org.apache.ignite.internal.processors.query.GridQueryProcessor$IndexType org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageResponse org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryRequest -org.apache.ignite.internal.processors.resource.GridResourceIoc$AnnotationSet -org.apache.ignite.internal.processors.resource.GridResourceIoc$ResourceAnnotation org.apache.ignite.internal.processors.rest.GridRestCommand org.apache.ignite.internal.processors.rest.GridRestProcessor$2$1 org.apache.ignite.internal.processors.rest.GridRestProcessor$3 @@ -1351,7 +1332,7 @@ org.apache.ignite.internal.processors.service.ServiceDescriptorImpl org.apache.ignite.internal.processors.task.GridTaskProcessor$1 org.apache.ignite.internal.processors.task.GridTaskThreadContextKey org.apache.ignite.internal.processors.task.GridTaskWorker$3 -org.apache.ignite.internal.processors.task.GridTaskWorker$5 +org.apache.ignite.internal.processors.task.GridTaskWorker$4 org.apache.ignite.internal.processors.task.GridTaskWorker$State org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException @@ -1375,7 +1356,6 @@ org.apache.ignite.internal.util.GridBoundedConcurrentOrderedMap org.apache.ignite.internal.util.GridBoundedConcurrentOrderedSet org.apache.ignite.internal.util.GridBoundedLinkedHashMap org.apache.ignite.internal.util.GridBoundedLinkedHashSet -org.apache.ignite.internal.util.GridBoundedPriorityQueue org.apache.ignite.internal.util.GridByteArrayList org.apache.ignite.internal.util.GridCloseableIteratorAdapter org.apache.ignite.internal.util.GridCloseableIteratorAdapterEx @@ -1427,14 +1407,14 @@ org.apache.ignite.internal.util.IgniteUtils$11 org.apache.ignite.internal.util.IgniteUtils$12 org.apache.ignite.internal.util.IgniteUtils$13 org.apache.ignite.internal.util.IgniteUtils$14 -org.apache.ignite.internal.util.IgniteUtils$15 -org.apache.ignite.internal.util.IgniteUtils$17 +org.apache.ignite.internal.util.IgniteUtils$16 +org.apache.ignite.internal.util.IgniteUtils$2 +org.apache.ignite.internal.util.IgniteUtils$22 org.apache.ignite.internal.util.IgniteUtils$23 org.apache.ignite.internal.util.IgniteUtils$24 org.apache.ignite.internal.util.IgniteUtils$25 org.apache.ignite.internal.util.IgniteUtils$26 org.apache.ignite.internal.util.IgniteUtils$27 -org.apache.ignite.internal.util.IgniteUtils$28 org.apache.ignite.internal.util.IgniteUtils$3 org.apache.ignite.internal.util.IgniteUtils$4 org.apache.ignite.internal.util.IgniteUtils$5 @@ -1667,19 +1647,12 @@ org.apache.ignite.internal.visor.cache.VisorCachePartitionsTask org.apache.ignite.internal.visor.cache.VisorCachePartitionsTask$VisorCachePartitionsJob org.apache.ignite.internal.visor.cache.VisorCacheQueryConfiguration org.apache.ignite.internal.visor.cache.VisorCacheQueryConfigurationV2 -org.apache.ignite.internal.visor.cache.VisorCacheQueryDetailMetrics -org.apache.ignite.internal.visor.cache.VisorCacheQueryDetailMetricsCollectorTask -org.apache.ignite.internal.visor.cache.VisorCacheQueryDetailMetricsCollectorTask$VisorCacheQueryDetailMetricsCollectorJob org.apache.ignite.internal.visor.cache.VisorCacheQueryMetrics org.apache.ignite.internal.visor.cache.VisorCacheRebalanceConfiguration org.apache.ignite.internal.visor.cache.VisorCacheRebalanceTask org.apache.ignite.internal.visor.cache.VisorCacheRebalanceTask$VisorCachesRebalanceJob org.apache.ignite.internal.visor.cache.VisorCacheResetMetricsTask org.apache.ignite.internal.visor.cache.VisorCacheResetMetricsTask$VisorCacheResetMetricsJob -org.apache.ignite.internal.visor.cache.VisorCacheResetQueryDetailMetricsTask -org.apache.ignite.internal.visor.cache.VisorCacheResetQueryDetailMetricsTask$VisorCacheResetQueryDetailMetricsJob -org.apache.ignite.internal.visor.cache.VisorCacheResetQueryMetricsTask -org.apache.ignite.internal.visor.cache.VisorCacheResetQueryMetricsTask$VisorCacheResetQueryMetricsJob org.apache.ignite.internal.visor.cache.VisorCacheStartTask org.apache.ignite.internal.visor.cache.VisorCacheStartTask$VisorCacheStartArg org.apache.ignite.internal.visor.cache.VisorCacheStartTask$VisorCacheStartJob @@ -1709,7 +1682,6 @@ org.apache.ignite.internal.visor.debug.VisorThreadLockInfo org.apache.ignite.internal.visor.debug.VisorThreadMonitorInfo org.apache.ignite.internal.visor.event.VisorGridDeploymentEvent org.apache.ignite.internal.visor.event.VisorGridDiscoveryEvent -org.apache.ignite.internal.visor.event.VisorGridDiscoveryEventV2 org.apache.ignite.internal.visor.event.VisorGridEvent org.apache.ignite.internal.visor.event.VisorGridEventsLost org.apache.ignite.internal.visor.event.VisorGridJobEvent @@ -1797,7 +1769,6 @@ org.apache.ignite.internal.visor.util.VisorClusterGroupEmptyException org.apache.ignite.internal.visor.util.VisorEventMapper org.apache.ignite.internal.visor.util.VisorExceptionWrapper org.apache.ignite.internal.visor.util.VisorTaskUtils$4 -org.apache.ignite.internal.visor.util.VisorTaskUtils$5 org.apache.ignite.internal.websession.WebSessionAttributeProcessor org.apache.ignite.internal.websession.WebSessionEntity org.apache.ignite.lang.IgniteBiClosure diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAtomicSingleMessageCountSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAtomicSingleMessageCountSelfTest.java new file mode 100644 index 0000000000000..88221151d3370 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAtomicSingleMessageCountSelfTest.java @@ -0,0 +1,259 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.CacheEntryProcessor; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFilterRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateInvokeRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; + +/** + * Tests single / transform messages being sent between nodes in ATOMIC mode. + */ +public class CacheAtomicSingleMessageCountSelfTest extends GridCommonAbstractTest { + /** VM ip finder for TCP discovery. */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** Starting grid index. */ + private int idx; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.setForceServerMode(true); + discoSpi.setIpFinder(ipFinder); + + cfg.setDiscoverySpi(discoSpi); + + CacheConfiguration cCfg = new CacheConfiguration(); + + cCfg.setCacheMode(PARTITIONED); + cCfg.setBackups(1); + cCfg.setWriteSynchronizationMode(FULL_SYNC); + cCfg.setAtomicWriteOrderMode(PRIMARY); + + cfg.setCacheConfiguration(cCfg); + + if (idx++ == 0) + cfg.setClientMode(true); + + cfg.setCommunicationSpi(new TestCommunicationSpi()); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testSingleMessage() throws Exception { + startGrids(2); + + try { + awaitPartitionMapExchange(); + + TestCommunicationSpi commSpi = (TestCommunicationSpi)grid(0).configuration().getCommunicationSpi(); + commSpi.resetCount(); + + commSpi.registerMessage(GridNearAtomicFullUpdateRequest.class); + commSpi.registerMessage(GridNearAtomicSingleUpdateRequest.class); + commSpi.registerMessage(GridNearAtomicSingleUpdateInvokeRequest.class); + commSpi.registerMessage(GridNearAtomicSingleUpdateFilterRequest.class); + + int putCnt = 15; + + for (int i = 0; i < putCnt; i++) + jcache(0).put(i, i); + + assertEquals(0, commSpi.messageCount(GridNearAtomicFullUpdateRequest.class)); + assertEquals(putCnt, commSpi.messageCount(GridNearAtomicSingleUpdateRequest.class)); + assertEquals(0, commSpi.messageCount(GridNearAtomicSingleUpdateInvokeRequest.class)); + assertEquals(0, commSpi.messageCount(GridNearAtomicSingleUpdateFilterRequest.class)); + } + finally { + stopAllGrids(); + } + } + + /** + * @throws Exception If failed. + */ + public void testSingleTransformMessage() throws Exception { + startGrids(2); + + int cacheId = ((IgniteKernal)grid(0)).internalCache(null).context().cacheId(); + + try { + awaitPartitionMapExchange(); + + TestCommunicationSpi commSpi = (TestCommunicationSpi)grid(0).configuration().getCommunicationSpi(); + + commSpi.resetCount(); + commSpi.filterCacheId(cacheId); + + commSpi.registerMessage(GridNearAtomicFullUpdateRequest.class); + commSpi.registerMessage(GridNearAtomicSingleUpdateRequest.class); + commSpi.registerMessage(GridNearAtomicSingleUpdateInvokeRequest.class); + commSpi.registerMessage(GridNearAtomicSingleUpdateFilterRequest.class); + + int putCnt = 15; + + for (int i = 0; i < putCnt; i++) { + jcache(0).invoke(i, new CacheEntryProcessor() { + @Override public Object process(MutableEntry entry, + Object... objects) throws EntryProcessorException { + return 2; + } + }); + } + + assertEquals(0, commSpi.messageCount(GridNearAtomicFullUpdateRequest.class)); + assertEquals(0, commSpi.messageCount(GridNearAtomicSingleUpdateRequest.class)); + assertEquals(putCnt, commSpi.messageCount(GridNearAtomicSingleUpdateInvokeRequest.class)); + assertEquals(0, commSpi.messageCount(GridNearAtomicSingleUpdateFilterRequest.class)); + } + finally { + stopAllGrids(); + } + } + + /** + * @throws Exception If failed. + */ + public void testSingleFilterMessage() throws Exception { + startGrids(2); + + try { + awaitPartitionMapExchange(); + + TestCommunicationSpi commSpi = (TestCommunicationSpi)grid(0).configuration().getCommunicationSpi(); + + commSpi.resetCount(); + + commSpi.registerMessage(GridNearAtomicFullUpdateRequest.class); + commSpi.registerMessage(GridNearAtomicSingleUpdateRequest.class); + commSpi.registerMessage(GridNearAtomicSingleUpdateInvokeRequest.class); + commSpi.registerMessage(GridNearAtomicSingleUpdateFilterRequest.class); + + int putCnt = 15; + + for (int i = 0; i < putCnt; i++) + jcache(0).putIfAbsent(i, i); + + assertEquals(0, commSpi.messageCount(GridNearAtomicFullUpdateRequest.class)); + assertEquals(0, commSpi.messageCount(GridNearAtomicSingleUpdateRequest.class)); + assertEquals(0, commSpi.messageCount(GridNearAtomicSingleUpdateInvokeRequest.class)); + assertEquals(putCnt, commSpi.messageCount(GridNearAtomicSingleUpdateFilterRequest.class)); + } + finally { + stopAllGrids(); + } + } + + /** + * Test communication SPI. + */ + private static class TestCommunicationSpi extends TcpCommunicationSpi { + /** Counters map. */ + private Map, AtomicInteger> cntMap = new HashMap<>(); + + /** Cache id to filter */ + private volatile Integer filterCacheId; + + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) + throws IgniteSpiException { + + if (((GridIoMessage)msg).message() instanceof GridCacheMessage) { + int msgCacheId = ((GridCacheMessage)((GridIoMessage)msg).message()).cacheId(); + + if (filterCacheId == null || filterCacheId == msgCacheId) { + AtomicInteger cntr = cntMap.get(((GridIoMessage)msg).message().getClass()); + + if (cntr != null) + cntr.incrementAndGet(); + } + } + + super.sendMessage(node, msg, ackC); + } + + /** + * Registers message for counting. + * + * @param cls Class to count. + */ + void registerMessage(Class cls) { + AtomicInteger cntr = cntMap.get(cls); + + if (cntr == null) + cntMap.put(cls, new AtomicInteger()); + } + + /** + * @param cls Message type to get count. + * @return Number of messages of given class. + */ + int messageCount(Class cls) { + AtomicInteger cntr = cntMap.get(cls); + + return cntr == null ? 0 : cntr.get(); + } + + /** + * Resets counter to zero. + */ + void resetCount() { + cntMap.clear(); + filterCacheId = null; + } + + /** + * @param cacheId Cache ID. + */ + void filterCacheId(int cacheId) { + filterCacheId = cacheId; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java index 0e17102daea78..a6d612a9dbfad 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java @@ -28,7 +28,8 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.managers.communication.GridIoMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiException; @@ -137,11 +138,13 @@ protected void checkMessages(boolean clientMode, TestCommunicationSpi commSpi = (TestCommunicationSpi)grid(0).configuration().getCommunicationSpi(); - commSpi.registerMessage(GridNearAtomicUpdateRequest.class); + commSpi.registerMessage(GridNearAtomicSingleUpdateRequest.class); + commSpi.registerMessage(GridNearAtomicFullUpdateRequest.class); commSpi.registerMessage(GridDhtAtomicUpdateRequest.class); int putCnt = 15; + int expNearSingleCnt = 0; int expNearCnt = 0; int expDhtCnt = 0; @@ -160,20 +163,22 @@ protected void checkMessages(boolean clientMode, if (affinity.isPrimary(locNode, i)) expDhtCnt++; else - expNearCnt ++; + expNearSingleCnt++; } jcache(0).put(i, i); } - assertEquals(expNearCnt, commSpi.messageCount(GridNearAtomicUpdateRequest.class)); + assertEquals(expNearCnt, commSpi.messageCount(GridNearAtomicFullUpdateRequest.class)); + assertEquals(expNearSingleCnt, commSpi.messageCount(GridNearAtomicSingleUpdateRequest.class)); assertEquals(expDhtCnt, commSpi.messageCount(GridDhtAtomicUpdateRequest.class)); if (writeOrderMode == CLOCK) { for (int i = 1; i < 4; i++) { commSpi = (TestCommunicationSpi)grid(i).configuration().getCommunicationSpi(); - assertEquals(0, commSpi.messageCount(GridNearAtomicUpdateRequest.class)); + assertEquals(0, commSpi.messageCount(GridNearAtomicSingleUpdateRequest.class)); + assertEquals(0, commSpi.messageCount(GridNearAtomicFullUpdateRequest.class)); assertEquals(0, commSpi.messageCount(GridDhtAtomicUpdateRequest.class)); } } @@ -181,7 +186,8 @@ protected void checkMessages(boolean clientMode, for (int i = 1; i < 4; i++) { commSpi = (TestCommunicationSpi)grid(i).configuration().getCommunicationSpi(); - assertEquals(0, commSpi.messageCount(GridNearAtomicUpdateRequest.class)); + assertEquals(0, commSpi.messageCount(GridNearAtomicSingleUpdateRequest.class)); + assertEquals(0, commSpi.messageCount(GridNearAtomicFullUpdateRequest.class)); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicStopBusySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicStopBusySelfTest.java index cdb7907330d9f..281397a7c98c8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicStopBusySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicStopBusySelfTest.java @@ -18,7 +18,7 @@ package org.apache.ignite.internal.processors.cache; import org.apache.ignite.cache.CacheAtomicityMode; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest; /** * Stopped node when client operations are executing. @@ -31,28 +31,28 @@ public class IgniteCacheAtomicStopBusySelfTest extends IgniteCacheAbstractStopBu /** {@inheritDoc} */ @Override public void testPut() throws Exception { - bannedMsg.set(GridNearAtomicUpdateRequest.class); + bannedMsg.set(GridNearAtomicSingleUpdateRequest.class); super.testPut(); } /** {@inheritDoc} */ @Override public void testPutBatch() throws Exception { - bannedMsg.set(GridNearAtomicUpdateRequest.class); + bannedMsg.set(GridNearAtomicSingleUpdateRequest.class); super.testPut(); } /** {@inheritDoc} */ @Override public void testPutAsync() throws Exception { - bannedMsg.set(GridNearAtomicUpdateRequest.class); + bannedMsg.set(GridNearAtomicSingleUpdateRequest.class); super.testPut(); } /** {@inheritDoc} */ @Override public void testRemove() throws Exception { - bannedMsg.set(GridNearAtomicUpdateRequest.class); + bannedMsg.set(GridNearAtomicSingleUpdateRequest.class); super.testPut(); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java index f9608e1fb6c8c..fb2d0decf9173 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java @@ -148,7 +148,7 @@ protected void failGet(int k) { * @throws Exception If failed. */ public void testResponseMessageOnUnmarshallingFailed() throws Exception { - // GridNearAtomicUpdateRequest unmarshalling failed test. + // GridNearAtomicFullUpdateRequest unmarshalling failed test. readCnt.set(1); failAtomicPut(++key); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java index 49c328963d1e5..b4ef11a82c0c1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java @@ -61,7 +61,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest; @@ -230,8 +230,8 @@ private void atomicPut(CacheAtomicWriteOrderMode writeOrder, TestCommunicationSpi spi = (TestCommunicationSpi)ignite2.configuration().getCommunicationSpi(); // Block messages requests for both nodes. - spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite0.localNode().id()); - spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite1.localNode().id()); + spi.blockMessages(GridNearAtomicFullUpdateRequest.class, ignite0.localNode().id()); + spi.blockMessages(GridNearAtomicFullUpdateRequest.class, ignite1.localNode().id()); final IgniteCache cache = ignite2.cache(null); @@ -272,7 +272,7 @@ private void atomicPut(CacheAtomicWriteOrderMode writeOrder, map.put(i, i + 1); // Block messages requests for single node. - spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite0.localNode().id()); + spi.blockMessages(GridNearAtomicFullUpdateRequest.class, ignite0.localNode().id()); putFut = GridTestUtils.runAsync(new Callable() { @Override public Object call() throws Exception { @@ -361,11 +361,11 @@ private void atomicNoRemap(CacheAtomicWriteOrderMode writeOrder) throws Exceptio TestCommunicationSpi spi = (TestCommunicationSpi)ignite3.configuration().getCommunicationSpi(); // Block messages requests for both nodes. - spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite0.localNode().id()); - spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite1.localNode().id()); - spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite2.localNode().id()); + spi.blockMessages(GridNearAtomicFullUpdateRequest.class, ignite0.localNode().id()); + spi.blockMessages(GridNearAtomicFullUpdateRequest.class, ignite1.localNode().id()); + spi.blockMessages(GridNearAtomicFullUpdateRequest.class, ignite2.localNode().id()); - spi.record(GridNearAtomicUpdateRequest.class); + spi.record(GridNearAtomicFullUpdateRequest.class); final IgniteCache cache = ignite3.cache(null); @@ -402,7 +402,7 @@ private void atomicNoRemap(CacheAtomicWriteOrderMode writeOrder) throws Exceptio assertEquals(3, msgs.size()); for (Object msg : msgs) - assertTrue(((GridNearAtomicUpdateRequest)msg).clientRequest()); + assertTrue(((GridNearAtomicFullUpdateRequest)msg).clientRequest()); map.put(primaryKey(ignite0.cache(null)), 3); map.put(primaryKey(ignite1.cache(null)), 4); @@ -459,8 +459,8 @@ private void atomicGetAndPut(CacheAtomicWriteOrderMode writeOrder) throws Except TestCommunicationSpi spi = (TestCommunicationSpi)ignite2.configuration().getCommunicationSpi(); // Block messages requests for both nodes. - spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite0.localNode().id()); - spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite1.localNode().id()); + spi.blockMessages(GridNearAtomicFullUpdateRequest.class, ignite0.localNode().id()); + spi.blockMessages(GridNearAtomicFullUpdateRequest.class, ignite1.localNode().id()); final IgniteCache cache = ignite2.cache(null); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java index 74d2d09812e07..0899423599660 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java @@ -478,7 +478,7 @@ private boolean delayMessage(GridIoMessage msg) { Object origMsg = msg.message(); return delay && - ((origMsg instanceof GridNearAtomicUpdateRequest) || (origMsg instanceof GridDhtAtomicUpdateRequest)); + ((origMsg instanceof GridNearAtomicAbstractUpdateRequest) || (origMsg instanceof GridDhtAtomicUpdateRequest)); } } } \ No newline at end of file From 88f38ac6305578946f2881b12d2d557bd561f67d Mon Sep 17 00:00:00 2001 From: Konstantin Dudkov Date: Mon, 21 Nov 2016 15:11:09 +0300 Subject: [PATCH 357/487] IGNITE-3074 Optimize DHT atomic update future --- .../processors/cache/GridCacheEntryEx.java | 4 +- .../processors/cache/GridCacheMapEntry.java | 4 +- .../GridDhtAtomicAbstractUpdateFuture.java | 461 ++++++++++++++++++ .../dht/atomic/GridDhtAtomicCache.java | 33 +- .../GridDhtAtomicSingleUpdateFuture.java | 121 +++++ .../dht/atomic/GridDhtAtomicUpdateFuture.java | 393 +-------------- .../GridNearAtomicFullUpdateRequest.java | 24 +- .../GridNearAtomicSingleUpdateFuture.java | 4 +- .../CacheContinuousQueryHandler.java | 4 +- .../CacheContinuousQueryListener.java | 4 +- .../CacheContinuousQueryManager.java | 6 +- .../cache/GridCacheTestEntryEx.java | 4 +- 12 files changed, 638 insertions(+), 424 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java index ef6a244fa77d0..176fe77aa8d3c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java @@ -27,7 +27,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockCancelledException; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -504,7 +504,7 @@ public GridCacheUpdateAtomicResult innerUpdate( String taskName, @Nullable CacheObject prevVal, @Nullable Long updateCntr, - @Nullable GridDhtAtomicUpdateFuture fut + @Nullable GridDhtAtomicAbstractUpdateFuture fut ) throws IgniteCheckedException, GridCacheEntryRemovedException; /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 59966728eb8d7..2bcf36036a106 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -43,7 +43,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry; import org.apache.ignite.internal.processors.cache.extras.GridCacheEntryExtras; import org.apache.ignite.internal.processors.cache.extras.GridCacheMvccEntryExtras; @@ -1951,7 +1951,7 @@ else if (ttl != CU.TTL_ZERO) final String taskName, @Nullable final CacheObject prevVal, @Nullable final Long updateCntr, - @Nullable GridDhtAtomicUpdateFuture fut + @Nullable GridDhtAtomicAbstractUpdateFuture fut ) throws IgniteCheckedException, GridCacheEntryRemovedException, GridClosureException { assert cctx.atomic(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java new file mode 100644 index 0000000000000..3bbc34823dc5d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java @@ -0,0 +1,461 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import javax.cache.processor.EntryProcessor; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.GridCacheAtomicFuture; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.CI1; +import org.apache.ignite.internal.util.typedef.CI2; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; + +/** + * DHT atomic cache backup update future. + */ +public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapter + implements GridCacheAtomicFuture { + /** */ + private static final long serialVersionUID = 0L; + + /** Logger. */ + protected static IgniteLogger log; + + /** Logger reference. */ + private static final AtomicReference logRef = new AtomicReference<>(); + + /** Logger. */ + protected static IgniteLogger msgLog; + + /** Write version. */ + protected final GridCacheVersion writeVer; + + /** Cache context. */ + protected final GridCacheContext cctx; + + /** Future version. */ + protected final GridCacheVersion futVer; + + /** Completion callback. */ + @GridToStringExclude + private final CI2 completionCb; + + /** Update request. */ + private final GridNearAtomicAbstractUpdateRequest updateReq; + + /** Update response. */ + final GridNearAtomicUpdateResponse updateRes; + + /** Force transform backup flag. */ + private boolean forceTransformBackups; + + /** Mappings. */ + @GridToStringInclude + protected Map mappings; + + /** Continuous query closures. */ + private Collection> cntQryClsrs; + + /** */ + private final boolean waitForExchange; + + /** Response count. */ + private volatile int resCnt; + + /** + * @param cctx Cache context. + * @param completionCb Callback to invoke when future is completed. + * @param writeVer Write version. + * @param updateReq Update request. + * @param updateRes Update response. + */ + protected GridDhtAtomicAbstractUpdateFuture( + GridCacheContext cctx, + CI2 completionCb, + GridCacheVersion writeVer, + GridNearAtomicAbstractUpdateRequest updateReq, + GridNearAtomicUpdateResponse updateRes) { + this.cctx = cctx; + + futVer = cctx.versions().next(updateReq.topologyVersion()); + this.updateReq = updateReq; + this.completionCb = completionCb; + this.updateRes = updateRes; + this.writeVer = writeVer; + + waitForExchange = !(updateReq.topologyLocked() || (updateReq.fastMap() && !updateReq.clientRequest())); + + if (log == null) { + msgLog = cctx.shared().atomicMessageLogger(); + log = U.logger(cctx.kernalContext(), logRef, GridDhtAtomicUpdateFuture.class); + } + } + + /** {@inheritDoc} */ + @Override public final IgniteInternalFuture completeFuture(AffinityTopologyVersion topVer) { + if (waitForExchange && updateReq.topologyVersion().compareTo(topVer) < 0) + return this; + + return null; + } + + /** + * @param clsr Continuous query closure. + */ + public final void addContinuousQueryClosure(CI1 clsr) { + assert !isDone() : this; + + if (cntQryClsrs == null) + cntQryClsrs = new ArrayList<>(10); + + cntQryClsrs.add(clsr); + } + + /** + * @param entry Entry to map. + * @param val Value to write. + * @param entryProcessor Entry processor. + * @param ttl TTL (optional). + * @param conflictExpireTime Conflict expire time (optional). + * @param conflictVer Conflict version (optional). + * @param addPrevVal If {@code true} sends previous value to backups. + * @param prevVal Previous value. + * @param updateCntr Partition update counter. + */ + @SuppressWarnings("ForLoopReplaceableByForEach") + final void addWriteEntry(GridDhtCacheEntry entry, + @Nullable CacheObject val, + EntryProcessor entryProcessor, + long ttl, + long conflictExpireTime, + @Nullable GridCacheVersion conflictVer, + boolean addPrevVal, + @Nullable CacheObject prevVal, + long updateCntr) { + AffinityTopologyVersion topVer = updateReq.topologyVersion(); + + List dhtNodes = cctx.dht().topology().nodes(entry.partition(), topVer); + + if (log.isDebugEnabled()) + log.debug("Mapping entry to DHT nodes [nodes=" + U.nodeIds(dhtNodes) + ", entry=" + entry + ']'); + + CacheWriteSynchronizationMode syncMode = updateReq.writeSynchronizationMode(); + + addDhtKey(entry.key(), dhtNodes); + + for (int i = 0; i < dhtNodes.size(); i++) { + ClusterNode node = dhtNodes.get(i); + + UUID nodeId = node.id(); + + if (!nodeId.equals(cctx.localNodeId())) { + GridDhtAtomicUpdateRequest updateReq = mappings.get(nodeId); + + if (updateReq == null) { + updateReq = new GridDhtAtomicUpdateRequest( + cctx.cacheId(), + nodeId, + futVer, + writeVer, + syncMode, + topVer, + forceTransformBackups, + this.updateReq.subjectId(), + this.updateReq.taskNameHash(), + forceTransformBackups ? this.updateReq.invokeArguments() : null, + cctx.deploymentEnabled(), + this.updateReq.keepBinary(), + this.updateReq.skipStore()); + + mappings.put(nodeId, updateReq); + } + + updateReq.addWriteValue(entry.key(), + val, + entryProcessor, + ttl, + conflictExpireTime, + conflictVer, + addPrevVal, + entry.partition(), + prevVal, + updateCntr); + } + } + } + + /** + * @param key Key. + * @param dhtNodes DHT nodes. + */ + protected abstract void addDhtKey(KeyCacheObject key, List dhtNodes); + + /** + * @param key Key. + * @param readers Near cache readers. + */ + protected abstract void addNearKey(KeyCacheObject key, Collection readers); + + /** + * @param readers Entry readers. + * @param entry Entry. + * @param val Value. + * @param entryProcessor Entry processor.. + * @param ttl TTL for near cache update (optional). + * @param expireTime Expire time for near cache update (optional). + */ + final void addNearWriteEntries(Collection readers, + GridDhtCacheEntry entry, + @Nullable CacheObject val, + EntryProcessor entryProcessor, + long ttl, + long expireTime) { + CacheWriteSynchronizationMode syncMode = updateReq.writeSynchronizationMode(); + + addNearKey(entry.key(), readers); + + AffinityTopologyVersion topVer = updateReq.topologyVersion(); + + for (UUID nodeId : readers) { + GridDhtAtomicUpdateRequest updateReq = mappings.get(nodeId); + + if (updateReq == null) { + ClusterNode node = cctx.discovery().node(nodeId); + + // Node left the grid. + if (node == null) + continue; + + updateReq = new GridDhtAtomicUpdateRequest( + cctx.cacheId(), + nodeId, + futVer, + writeVer, + syncMode, + topVer, + forceTransformBackups, + this.updateReq.subjectId(), + this.updateReq.taskNameHash(), + forceTransformBackups ? this.updateReq.invokeArguments() : null, + cctx.deploymentEnabled(), + this.updateReq.keepBinary(), + this.updateReq.skipStore()); + + mappings.put(nodeId, updateReq); + } + + addNearReaderEntry(entry); + + updateReq.addNearWriteValue(entry.key(), + val, + entryProcessor, + ttl, + expireTime); + } + } + + /** + * adds new nearReader. + * + * @param entry GridDhtCacheEntry. + */ + protected abstract void addNearReaderEntry(GridDhtCacheEntry entry); + + /** + * @return Write version. + */ + final GridCacheVersion writeVersion() { + return writeVer; + } + + /** {@inheritDoc} */ + @Override public final IgniteUuid futureId() { + return futVer.asGridUuid(); + } + + /** {@inheritDoc} */ + @Override public final GridCacheVersion version() { + return futVer; + } + + /** {@inheritDoc} */ + @Override public final boolean onNodeLeft(UUID nodeId) { + boolean res = registerResponse(nodeId); + + if (res && msgLog.isDebugEnabled()) { + msgLog.debug("DTH update fut, node left [futId=" + futVer + ", writeVer=" + writeVer + + ", node=" + nodeId + ']'); + } + + return res; + } + + /** + * @param nodeId Node ID. + * @return {@code True} if request found. + */ + final boolean registerResponse(UUID nodeId) { + int resCnt0; + + GridDhtAtomicUpdateRequest req = mappings != null ? mappings.get(nodeId) : null; + + if (req != null) { + synchronized (this) { + if (req.onResponse()) { + resCnt0 = resCnt; + + resCnt0 += 1; + + resCnt = resCnt0; + } + else + return false; + } + + if (resCnt0 == mappings.size()) + onDone(); + + return true; + } + + return false; + } + + /** + * Sends requests to remote nodes. + */ + final void map() { + if (!F.isEmpty(mappings)) { + for (GridDhtAtomicUpdateRequest req : mappings.values()) { + try { + cctx.io().send(req.nodeId(), req, cctx.ioPolicy()); + + if (msgLog.isDebugEnabled()) { + msgLog.debug("DTH update fut, sent request [futId=" + futVer + + ", writeVer=" + writeVer + ", node=" + req.nodeId() + ']'); + } + } + catch (ClusterTopologyCheckedException ignored) { + if (msgLog.isDebugEnabled()) { + msgLog.debug("DTH update fut, failed to send request, node left [futId=" + futVer + + ", writeVer=" + writeVer + ", node=" + req.nodeId() + ']'); + } + + registerResponse(req.nodeId()); + } + catch (IgniteCheckedException e) { + U.error(msgLog, "Failed to send request [futId=" + futVer + + ", writeVer=" + writeVer + ", node=" + req.nodeId() + ']'); + + registerResponse(req.nodeId()); + } + } + } + else + onDone(); + + // Send response right away if no ACKs from backup is required. + // Backups will send ACKs anyway, future will be completed after all backups have replied. + if (updateReq.writeSynchronizationMode() != FULL_SYNC) + completionCb.apply(updateReq, updateRes); + } + + /** + * Deferred update response. + * + * @param nodeId Backup node ID. + */ + public final void onResult(UUID nodeId) { + if (log.isDebugEnabled()) + log.debug("Received deferred DHT atomic update future result [nodeId=" + nodeId + ']'); + + registerResponse(nodeId); + } + + /** + * Callback for backup update response. + * + * @param nodeId Backup node ID. + * @param updateRes Update response. + */ + public abstract void onResult(UUID nodeId, GridDhtAtomicUpdateResponse updateRes); + + /** + * @param updateRes Response. + * @param err Error. + */ + protected abstract void addFailedKeys(GridNearAtomicUpdateResponse updateRes, Throwable err); + + /** {@inheritDoc} */ + @Override public final boolean onDone(@Nullable Void res, @Nullable Throwable err) { + if (super.onDone(res, err)) { + cctx.mvcc().removeAtomicFuture(version()); + + boolean suc = err == null; + + if (!suc) + addFailedKeys(updateRes, err); + + if (cntQryClsrs != null) { + for (CI1 clsr : cntQryClsrs) + clsr.apply(suc); + } + + if (updateReq.writeSynchronizationMode() == FULL_SYNC) + completionCb.apply(updateReq, updateRes); + + return true; + } + + return false; + } + + /** {@inheritDoc} */ + @Override public boolean trackable() { + return true; + } + + /** {@inheritDoc} */ + @Override public void markNotTrackable() { + // No-op. + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java index f7d19731ba591..d7eb062b3dae9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java @@ -1685,7 +1685,7 @@ private void updateAllAsyncInternal0( assert !req.returnValue() || (req.operation() == TRANSFORM || req.size() == 1); - GridDhtAtomicUpdateFuture dhtFut = null; + GridDhtAtomicAbstractUpdateFuture dhtFut = null; boolean remap = false; @@ -1908,7 +1908,7 @@ private UpdateBatchResult updateWithBatch( final GridNearAtomicUpdateResponse res, final List locked, final GridCacheVersion ver, - @Nullable GridDhtAtomicUpdateFuture dhtFut, + @Nullable GridDhtAtomicAbstractUpdateFuture dhtFut, final CI2 completionCb, final boolean replicate, final String taskName, @@ -2331,7 +2331,7 @@ private UpdateSingleResult updateSingle( GridNearAtomicUpdateResponse res, List locked, GridCacheVersion ver, - @Nullable GridDhtAtomicUpdateFuture dhtFut, + @Nullable GridDhtAtomicAbstractUpdateFuture dhtFut, CI2 completionCb, boolean replicate, String taskName, @@ -2552,7 +2552,7 @@ else if (F.contains(readers, node.id())) // Reader became primary or backup. * @return Deleted entries. */ @SuppressWarnings("ForLoopReplaceableByForEach") - @Nullable private GridDhtAtomicUpdateFuture updatePartialBatch( + @Nullable private GridDhtAtomicAbstractUpdateFuture updatePartialBatch( final boolean hasNear, final int firstEntryIdx, final List entries, @@ -2562,7 +2562,7 @@ else if (F.contains(readers, node.id())) // Reader became primary or backup. @Nullable final Map putMap, @Nullable final Collection rmvKeys, @Nullable final Map> entryProcessorMap, - @Nullable GridDhtAtomicUpdateFuture dhtFut, + @Nullable GridDhtAtomicAbstractUpdateFuture dhtFut, final CI2 completionCb, final GridNearAtomicAbstractUpdateRequest req, final GridNearAtomicUpdateResponse res, @@ -3036,7 +3036,7 @@ else if (req.operation() == UPDATE) { * @param force If {@code true} then creates future without optimizations checks. * @return Backup update future or {@code null} if there are no backups. */ - @Nullable private GridDhtAtomicUpdateFuture createDhtFuture( + @Nullable private GridDhtAtomicAbstractUpdateFuture createDhtFuture( GridCacheVersion writeVer, GridNearAtomicAbstractUpdateRequest updateReq, GridNearAtomicUpdateResponse updateRes, @@ -3064,7 +3064,10 @@ else if (req.operation() == UPDATE) { } } - return new GridDhtAtomicUpdateFuture(ctx, completionCb, writeVer, updateReq, updateRes); + if (updateReq.size() == 1) + return new GridDhtAtomicSingleUpdateFuture(ctx, completionCb, writeVer, updateReq, updateRes); + else + return new GridDhtAtomicUpdateFuture(ctx, completionCb, writeVer, updateReq, updateRes); } /** @@ -3256,7 +3259,7 @@ private void sendDeferredUpdateResponse(UUID nodeId, GridCacheVersion ver) { */ @SuppressWarnings("unchecked") private void processDhtAtomicUpdateResponse(UUID nodeId, GridDhtAtomicUpdateResponse res) { - GridDhtAtomicUpdateFuture updateFut = (GridDhtAtomicUpdateFuture)ctx.mvcc().atomicFuture(res.futureVersion()); + GridDhtAtomicAbstractUpdateFuture updateFut = (GridDhtAtomicAbstractUpdateFuture)ctx.mvcc().atomicFuture(res.futureVersion()); if (updateFut != null) { if (msgLog.isDebugEnabled()) { @@ -3279,7 +3282,7 @@ private void processDhtAtomicUpdateResponse(UUID nodeId, GridDhtAtomicUpdateResp @SuppressWarnings("unchecked") private void processDhtAtomicDeferredUpdateResponse(UUID nodeId, GridDhtAtomicDeferredUpdateResponse res) { for (GridCacheVersion ver : res.futureVersions()) { - GridDhtAtomicUpdateFuture updateFut = (GridDhtAtomicUpdateFuture)ctx.mvcc().atomicFuture(ver); + GridDhtAtomicAbstractUpdateFuture updateFut = (GridDhtAtomicAbstractUpdateFuture)ctx.mvcc().atomicFuture(ver); if (updateFut != null) { if (msgLog.isDebugEnabled()) { @@ -3335,7 +3338,7 @@ private static class UpdateSingleResult { private final Collection> deleted; /** */ - private final GridDhtAtomicUpdateFuture dhtFut; + private final GridDhtAtomicAbstractUpdateFuture dhtFut; /** * @param retVal Return value. @@ -3344,7 +3347,7 @@ private static class UpdateSingleResult { */ private UpdateSingleResult(GridCacheReturn retVal, Collection> deleted, - GridDhtAtomicUpdateFuture dhtFut) { + GridDhtAtomicAbstractUpdateFuture dhtFut) { this.retVal = retVal; this.deleted = deleted; this.dhtFut = dhtFut; @@ -3367,7 +3370,7 @@ private Collection> deleted() /** * @return DHT future. */ - public GridDhtAtomicUpdateFuture dhtFuture() { + public GridDhtAtomicAbstractUpdateFuture dhtFuture() { return dhtFut; } } @@ -3380,7 +3383,7 @@ private static class UpdateBatchResult { private Collection> deleted; /** */ - private GridDhtAtomicUpdateFuture dhtFut; + private GridDhtAtomicAbstractUpdateFuture dhtFut; /** */ private boolean readersOnly; @@ -3414,7 +3417,7 @@ private Collection> deleted() /** * @return DHT future. */ - public GridDhtAtomicUpdateFuture dhtFuture() { + public GridDhtAtomicAbstractUpdateFuture dhtFuture() { return dhtFut; } @@ -3435,7 +3438,7 @@ GridCacheReturn invokeResults() { /** * @param dhtFut DHT future. */ - private void dhtFuture(@Nullable GridDhtAtomicUpdateFuture dhtFut) { + private void dhtFuture(@Nullable GridDhtAtomicAbstractUpdateFuture dhtFut) { this.dhtFut = dhtFut; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java new file mode 100644 index 0000000000000..f83a7b7f96102 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; + +import java.util.Collection; +import java.util.List; +import java.util.UUID; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.typedef.CI2; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * + */ +class GridDhtAtomicSingleUpdateFuture extends GridDhtAtomicAbstractUpdateFuture { + /** */ + private static final long serialVersionUID = 0L; + + /** Future keys. */ + private KeyCacheObject key; + + /** Entries with readers. */ + private GridDhtCacheEntry nearReaderEntry; + + /** + * @param cctx Cache context. + * @param completionCb Callback to invoke when future is completed. + * @param writeVer Write version. + * @param updateReq Update request. + * @param updateRes Update response. + */ + GridDhtAtomicSingleUpdateFuture( + GridCacheContext cctx, + CI2 completionCb, + GridCacheVersion writeVer, + GridNearAtomicAbstractUpdateRequest updateReq, + GridNearAtomicUpdateResponse updateRes + ) { + super(cctx, completionCb, writeVer, updateReq, updateRes); + } + + /** {@inheritDoc} */ + @Override protected void addDhtKey(KeyCacheObject key, List dhtNodes) { + assert this.key == null || this.key.equals(key) : this.key; + + if (mappings == null) + mappings = U.newHashMap(dhtNodes.size()); + + this.key = key; + } + + /** {@inheritDoc} */ + @Override protected void addNearKey(KeyCacheObject key, Collection readers) { + assert this.key == null || this.key.equals(key) : this.key; + + if (mappings == null) + mappings = U.newHashMap(readers.size()); + + this.key = key; + } + + /** {@inheritDoc} */ + @Override protected void addNearReaderEntry(GridDhtCacheEntry entry) { + nearReaderEntry = entry; + } + + /** {@inheritDoc} */ + @Override public void onResult(UUID nodeId, GridDhtAtomicUpdateResponse updateRes) { + if (log.isDebugEnabled()) + log.debug("Received DHT atomic update future result [nodeId=" + nodeId + ", updateRes=" + updateRes + ']'); + + if (updateRes.error() != null) + this.updateRes.addFailedKeys(updateRes.failedKeys(), updateRes.error()); + + if (!F.isEmpty(updateRes.nearEvicted())) { + try { + assert nearReaderEntry != null; + + nearReaderEntry.removeReader(nodeId, updateRes.messageId()); + } + catch (GridCacheEntryRemovedException e) { + if (log.isDebugEnabled()) + log.debug("Entry with evicted reader was removed [entry=" + nearReaderEntry + ", err=" + e + ']'); + } + } + + registerResponse(nodeId); + } + + /** {@inheritDoc} */ + @Override protected void addFailedKeys(GridNearAtomicUpdateResponse updateRes, Throwable err) { + updateRes.addFailedKey(key, err); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridDhtAtomicSingleUpdateFuture.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java index c2ad8b8f961b7..864aadd2447e2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java @@ -23,92 +23,30 @@ import java.util.List; import java.util.Map; import java.util.UUID; -import java.util.concurrent.atomic.AtomicReference; -import javax.cache.processor.EntryProcessor; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteLogger; -import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; -import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.CacheObject; -import org.apache.ignite.internal.processors.cache.GridCacheAtomicFuture; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; -import org.apache.ignite.internal.util.future.GridFutureAdapter; -import org.apache.ignite.internal.util.tostring.GridToStringExclude; -import org.apache.ignite.internal.util.tostring.GridToStringInclude; -import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.CI2; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgniteUuid; -import org.jetbrains.annotations.Nullable; - -import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; /** * DHT atomic cache backup update future. */ -public class GridDhtAtomicUpdateFuture extends GridFutureAdapter - implements GridCacheAtomicFuture { +class GridDhtAtomicUpdateFuture extends GridDhtAtomicAbstractUpdateFuture { /** */ private static final long serialVersionUID = 0L; - /** Logger reference. */ - private static final AtomicReference logRef = new AtomicReference<>(); - - /** Logger. */ - private static IgniteLogger log; - - /** Logger. */ - private static IgniteLogger msgLog; - - /** Cache context. */ - private final GridCacheContext cctx; - - /** Future version. */ - private final GridCacheVersion futVer; - - /** Write version. */ - private final GridCacheVersion writeVer; - - /** Force transform backup flag. */ - private boolean forceTransformBackups; - - /** Completion callback. */ - @GridToStringExclude - private final CI2 completionCb; - - /** Mappings. */ - @GridToStringInclude - private final Map mappings; - - /** Entries with readers. */ - private Map nearReadersEntries; - - /** Update request. */ - private final GridNearAtomicAbstractUpdateRequest updateReq; - - /** Update response. */ - private final GridNearAtomicUpdateResponse updateRes; - /** Future keys. */ private final Collection keys; - /** Continuous query closures. */ - private Collection> cntQryClsrs; - - /** */ - private final boolean waitForExchange; + /** Entries with readers. */ + private Map nearReadersEntries; - /** Response count. */ - private volatile int resCnt; /** * @param cctx Cache context. @@ -117,328 +55,39 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter * @param updateReq Update request. * @param updateRes Update response. */ - public GridDhtAtomicUpdateFuture( + GridDhtAtomicUpdateFuture( GridCacheContext cctx, CI2 completionCb, GridCacheVersion writeVer, GridNearAtomicAbstractUpdateRequest updateReq, GridNearAtomicUpdateResponse updateRes ) { - this.cctx = cctx; - this.writeVer = writeVer; - - futVer = cctx.versions().next(updateReq.topologyVersion()); - this.updateReq = updateReq; - this.completionCb = completionCb; - this.updateRes = updateRes; - - if (log == null) { - msgLog = cctx.shared().atomicMessageLogger(); - log = U.logger(cctx.kernalContext(), logRef, GridDhtAtomicUpdateFuture.class); - } + super(cctx, completionCb, writeVer, updateReq, updateRes); keys = new ArrayList<>(updateReq.size()); mappings = U.newHashMap(updateReq.size()); - - waitForExchange = !(updateReq.topologyLocked() || (updateReq.fastMap() && !updateReq.clientRequest())); - } - - /** - * @return Write version. - */ - GridCacheVersion writeVersion() { - return writeVer; - } - - /** {@inheritDoc} */ - @Override public IgniteUuid futureId() { - return futVer.asGridUuid(); } /** {@inheritDoc} */ - @Override public GridCacheVersion version() { - return futVer; + @Override protected void addDhtKey(KeyCacheObject key, List dhtNodes) { + keys.add(key); } /** {@inheritDoc} */ - @Override public boolean onNodeLeft(UUID nodeId) { - boolean res = registerResponse(nodeId); - - if (res && msgLog.isDebugEnabled()) { - msgLog.debug("DTH update fut, node left [futId=" + futVer + ", writeVer=" + writeVer + - ", node=" + nodeId + ']'); - } - - return res; - } - - /** - * @param nodeId Node ID. - * @return {@code True} if request found. - */ - private boolean registerResponse(UUID nodeId) { - int resCnt0; - - GridDhtAtomicUpdateRequest req = mappings.get(nodeId); - - if (req != null) { - synchronized (this) { - if (req.onResponse()) { - resCnt0 = resCnt; - - resCnt0 += 1; - - resCnt = resCnt0; - } - else - return false; - } - - if (resCnt0 == mappings.size()) - onDone(); - - return true; - } - - return false; + @Override protected void addNearKey(KeyCacheObject key, Collection readers) { + keys.add(key); } /** {@inheritDoc} */ - @Override public boolean trackable() { - return true; - } + @Override protected void addNearReaderEntry(GridDhtCacheEntry entry) { + if (nearReadersEntries == null) + nearReadersEntries = new HashMap<>(); - /** {@inheritDoc} */ - @Override public void markNotTrackable() { - // No-op. + nearReadersEntries.put(entry.key(), entry); } /** {@inheritDoc} */ - @Override public IgniteInternalFuture completeFuture(AffinityTopologyVersion topVer) { - if (waitForExchange && updateReq.topologyVersion().compareTo(topVer) < 0) - return this; - - return null; - } - - /** - * @param entry Entry to map. - * @param val Value to write. - * @param entryProcessor Entry processor. - * @param ttl TTL (optional). - * @param conflictExpireTime Conflict expire time (optional). - * @param conflictVer Conflict version (optional). - * @param addPrevVal If {@code true} sends previous value to backups. - * @param prevVal Previous value. - * @param updateCntr Partition update counter. - */ - @SuppressWarnings("ForLoopReplaceableByForEach") - public void addWriteEntry(GridDhtCacheEntry entry, - @Nullable CacheObject val, - EntryProcessor entryProcessor, - long ttl, - long conflictExpireTime, - @Nullable GridCacheVersion conflictVer, - boolean addPrevVal, - @Nullable CacheObject prevVal, - long updateCntr) { - AffinityTopologyVersion topVer = updateReq.topologyVersion(); - - List dhtNodes = cctx.dht().topology().nodes(entry.partition(), topVer); - - if (log.isDebugEnabled()) - log.debug("Mapping entry to DHT nodes [nodes=" + U.nodeIds(dhtNodes) + ", entry=" + entry + ']'); - - CacheWriteSynchronizationMode syncMode = updateReq.writeSynchronizationMode(); - - keys.add(entry.key()); - - for (int i = 0; i < dhtNodes.size(); i++) { - ClusterNode node = dhtNodes.get(i); - - UUID nodeId = node.id(); - - if (!nodeId.equals(cctx.localNodeId())) { - GridDhtAtomicUpdateRequest updateReq = mappings.get(nodeId); - - if (updateReq == null) { - updateReq = new GridDhtAtomicUpdateRequest( - cctx.cacheId(), - nodeId, - futVer, - writeVer, - syncMode, - topVer, - forceTransformBackups, - this.updateReq.subjectId(), - this.updateReq.taskNameHash(), - forceTransformBackups ? this.updateReq.invokeArguments() : null, - cctx.deploymentEnabled(), - this.updateReq.keepBinary(), - this.updateReq.skipStore()); - - mappings.put(nodeId, updateReq); - } - - updateReq.addWriteValue(entry.key(), - val, - entryProcessor, - ttl, - conflictExpireTime, - conflictVer, - addPrevVal, - entry.partition(), - prevVal, - updateCntr); - } - } - } - - /** - * @param readers Entry readers. - * @param entry Entry. - * @param val Value. - * @param entryProcessor Entry processor.. - * @param ttl TTL for near cache update (optional). - * @param expireTime Expire time for near cache update (optional). - */ - public void addNearWriteEntries(Iterable readers, - GridDhtCacheEntry entry, - @Nullable CacheObject val, - EntryProcessor entryProcessor, - long ttl, - long expireTime) { - CacheWriteSynchronizationMode syncMode = updateReq.writeSynchronizationMode(); - - keys.add(entry.key()); - - AffinityTopologyVersion topVer = updateReq.topologyVersion(); - - for (UUID nodeId : readers) { - GridDhtAtomicUpdateRequest updateReq = mappings.get(nodeId); - - if (updateReq == null) { - ClusterNode node = cctx.discovery().node(nodeId); - - // Node left the grid. - if (node == null) - continue; - - updateReq = new GridDhtAtomicUpdateRequest( - cctx.cacheId(), - nodeId, - futVer, - writeVer, - syncMode, - topVer, - forceTransformBackups, - this.updateReq.subjectId(), - this.updateReq.taskNameHash(), - forceTransformBackups ? this.updateReq.invokeArguments() : null, - cctx.deploymentEnabled(), - this.updateReq.keepBinary(), - this.updateReq.skipStore()); - - mappings.put(nodeId, updateReq); - } - - if (nearReadersEntries == null) - nearReadersEntries = new HashMap<>(); - - nearReadersEntries.put(entry.key(), entry); - - updateReq.addNearWriteValue(entry.key(), - val, - entryProcessor, - ttl, - expireTime); - } - } - - /** - * @param clsr Continuous query closure. - */ - public void addContinuousQueryClosure(CI1 clsr){ - assert !isDone() : this; - - if (cntQryClsrs == null) - cntQryClsrs = new ArrayList<>(10); - - cntQryClsrs.add(clsr); - } - - /** {@inheritDoc} */ - @Override public boolean onDone(@Nullable Void res, @Nullable Throwable err) { - if (super.onDone(res, err)) { - cctx.mvcc().removeAtomicFuture(version()); - - boolean suc = err == null; - - if (!suc) { - for (KeyCacheObject key : keys) - updateRes.addFailedKey(key, err); - } - - if (cntQryClsrs != null) { - for (CI1 clsr : cntQryClsrs) - clsr.apply(suc); - } - - if (updateReq.writeSynchronizationMode() == FULL_SYNC) - completionCb.apply(updateReq, updateRes); - - return true; - } - - return false; - } - - /** - * Sends requests to remote nodes. - */ - public void map() { - if (!mappings.isEmpty()) { - for (GridDhtAtomicUpdateRequest req : mappings.values()) { - try { - cctx.io().send(req.nodeId(), req, cctx.ioPolicy()); - - if (msgLog.isDebugEnabled()) { - msgLog.debug("DTH update fut, sent request [futId=" + futVer + - ", writeVer=" + writeVer + ", node=" + req.nodeId() + ']'); - } - } - catch (ClusterTopologyCheckedException ignored) { - if (msgLog.isDebugEnabled()) { - msgLog.debug("DTH update fut, failed to send request, node left [futId=" + futVer + - ", writeVer=" + writeVer + ", node=" + req.nodeId() + ']'); - } - - registerResponse(req.nodeId()); - } - catch (IgniteCheckedException e) { - U.error(msgLog, "Failed to send request [futId=" + futVer + - ", writeVer=" + writeVer + ", node=" + req.nodeId() + ']'); - - registerResponse(req.nodeId()); - } - } - } - else - onDone(); - - // Send response right away if no ACKs from backup is required. - // Backups will send ACKs anyway, future will be completed after all backups have replied. - if (updateReq.writeSynchronizationMode() != FULL_SYNC) - completionCb.apply(updateReq, updateRes); - } - - /** - * Callback for backup update response. - * - * @param nodeId Backup node ID. - * @param updateRes Update response. - */ - public void onResult(UUID nodeId, GridDhtAtomicUpdateResponse updateRes) { + @Override public void onResult(UUID nodeId, GridDhtAtomicUpdateResponse updateRes) { if (log.isDebugEnabled()) log.debug("Received DHT atomic update future result [nodeId=" + nodeId + ", updateRes=" + updateRes + ']'); @@ -462,16 +111,10 @@ public void onResult(UUID nodeId, GridDhtAtomicUpdateResponse updateRes) { registerResponse(nodeId); } - /** - * Deferred update response. - * - * @param nodeId Backup node ID. - */ - public void onResult(UUID nodeId) { - if (log.isDebugEnabled()) - log.debug("Received deferred DHT atomic update future result [nodeId=" + nodeId + ']'); - - registerResponse(nodeId); + /** {@inheritDoc} */ + @Override protected void addFailedKeys(GridNearAtomicUpdateResponse updateRes, Throwable err) { + for (KeyCacheObject key : keys) + updateRes.addFailedKey(key, err); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java index e2314f8fd543e..b733d7b500f18 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java @@ -487,44 +487,32 @@ else if (conflictVers != null) return invokeArgs; } - /** - * @return Flag indicating whether this is fast-map udpate. - */ + /** {@inheritDoc} */ @Override public boolean fastMap() { return fastMap; } - /** - * @return Topology locked flag. - */ + /** {@inheritDoc} */ @Override public boolean topologyLocked() { return topLocked; } - /** - * @return {@code True} if request sent from client node. - */ + /** {@inheritDoc} */ @Override public boolean clientRequest() { return clientReq; } - /** - * @return Return value flag. - */ + /** {@inheritDoc} */ @Override public boolean returnValue() { return retval; } - /** - * @return Skip write-through to a persistent storage. - */ + /** {@inheritDoc} */ @Override public boolean skipStore() { return skipStore; } - /** - * @return Keep binary flag. - */ + /** {@inheritDoc} */ @Override public boolean keepBinary() { return keepBinary; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java index eaf2f2c9a6e2d..bd231cf0003d8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java @@ -667,9 +667,7 @@ private GridNearAtomicAbstractUpdateRequest mapSingleUpdate(AffinityTopologyVers * @return {@code True} can use 'single' update requests. */ private boolean canUseSingleRequest(ClusterNode node) { - assert node != null; - - return expiryPlc == null && node.version().compareToIgnoreTimestamp(SINGLE_UPDATE_REQUEST) >= 0; + return expiryPlc == null && node != null && node.version().compareToIgnoreTimestamp(SINGLE_UPDATE_REQUEST) >= 0; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 304d031d81fc0..10784fc6b05fe 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -59,7 +59,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheDeploymentManager; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture; import org.apache.ignite.internal.processors.cache.query.CacheQueryType; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager.JCacheQueryLocalListener; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager.JCacheQueryRemoteFilter; @@ -382,7 +382,7 @@ public void keepBinary(boolean keepBinary) { @Override public void onEntryUpdated(final CacheContinuousQueryEvent evt, boolean primary, final boolean recordIgniteEvt, - GridDhtAtomicUpdateFuture fut) { + GridDhtAtomicAbstractUpdateFuture fut) { if (ignoreExpired && evt.getEventType() == EventType.EXPIRED) return ; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java index 8eca81c004079..84b22f933e848 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java @@ -20,7 +20,7 @@ import java.util.Map; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture; import org.jetbrains.annotations.Nullable; /** @@ -41,7 +41,7 @@ public interface CacheContinuousQueryListener { * @param fut Dht atomic future. */ public void onEntryUpdated(CacheContinuousQueryEvent evt, boolean primary, - boolean recordIgniteEvt, @Nullable GridDhtAtomicUpdateFuture fut); + boolean recordIgniteEvt, @Nullable GridDhtAtomicAbstractUpdateFuture fut); /** * Listener unregistered callback. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java index 2863f3d12b2ba..e2fbf52814161 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java @@ -56,7 +56,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheManagerAdapter; import org.apache.ignite.internal.processors.cache.KeyCacheObject; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture; import org.apache.ignite.internal.processors.continuous.GridContinuousHandler; import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -245,7 +245,7 @@ public void onEntryUpdated( boolean primary, boolean preload, long updateCntr, - @Nullable GridDhtAtomicUpdateFuture fut, + @Nullable GridDhtAtomicAbstractUpdateFuture fut, AffinityTopologyVersion topVer ) throws IgniteCheckedException { Map lsnrCol = updateListeners(internal, preload); @@ -290,7 +290,7 @@ public void onEntryUpdated( boolean primary, boolean preload, long updateCntr, - @Nullable GridDhtAtomicUpdateFuture fut, + @Nullable GridDhtAtomicAbstractUpdateFuture fut, AffinityTopologyVersion topVer) throws IgniteCheckedException { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java index bf543cb24ec6f..396629ac4401a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java @@ -26,7 +26,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.eviction.EvictableEntry; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -542,7 +542,7 @@ public void recheckLock() { String taskName, @Nullable CacheObject prevVal, @Nullable Long updateCntr, - @Nullable GridDhtAtomicUpdateFuture fut) throws IgniteCheckedException, + @Nullable GridDhtAtomicAbstractUpdateFuture fut) throws IgniteCheckedException, GridCacheEntryRemovedException { assert false; From 51ca24f2db32dff9c0034603ea3abfe5ef5cd846 Mon Sep 17 00:00:00 2001 From: Konstantin Dudkov Date: Mon, 21 Nov 2016 16:48:44 +0300 Subject: [PATCH 358/487] IGNITE-3075 Implement single key-value pair DHT request/response for ATOMIC cache. --- .../communication/GridIoMessageFactory.java | 10 +- .../processors/cache/GridCacheIoManager.java | 25 +- .../GridDhtAtomicAbstractUpdateFuture.java | 57 +- .../GridDhtAtomicAbstractUpdateRequest.java | 287 ++++++++ .../dht/atomic/GridDhtAtomicCache.java | 17 +- .../GridDhtAtomicSingleUpdateFuture.java | 61 ++ .../GridDhtAtomicSingleUpdateRequest.java | 678 ++++++++++++++++++ .../dht/atomic/GridDhtAtomicUpdateFuture.java | 26 + .../atomic/GridDhtAtomicUpdateRequest.java | 312 +++----- .../GridNearAtomicAbstractUpdateRequest.java | 5 + .../GridNearAtomicFullUpdateRequest.java | 108 +-- .../GridNearAtomicSingleUpdateRequest.java | 5 + .../distributed/near/GridNearAtomicCache.java | 8 +- .../GridCacheAtomicMessageCountSelfTest.java | 6 +- ...tomicInvalidPartitionHandlingSelfTest.java | 2 +- 15 files changed, 1292 insertions(+), 315 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java index b20de688ae95d..f36191cbafcd9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java @@ -67,12 +67,13 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnlockRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicSingleUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFilterRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateInvokeRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse; @@ -774,7 +775,12 @@ public GridIoMessageFactory(MessageFactory[] ext) { break; - // [-3..119] [124..127] - this + case -36: + msg = new GridDhtAtomicSingleUpdateRequest(); + + break; + + // [-3..119] [124..127] [-36]- this // [120..123] - DR // [-4..-22, -30..-35] - SQL default: diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java index c5c1c60847ea1..924ce7917b4eb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java @@ -45,6 +45,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicSingleUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateRequest; @@ -470,8 +472,8 @@ else if (cacheMsg instanceof GridDhtTxFinishResponse) return ((GridNearAtomicAbstractUpdateRequest)cacheMsg).futureVersion(); else if (cacheMsg instanceof GridNearAtomicUpdateResponse) return ((GridNearAtomicUpdateResponse) cacheMsg).futureVersion(); - else if (cacheMsg instanceof GridDhtAtomicUpdateRequest) - return ((GridDhtAtomicUpdateRequest)cacheMsg).futureVersion(); + else if (cacheMsg instanceof GridDhtAtomicAbstractUpdateRequest) + return ((GridDhtAtomicAbstractUpdateRequest)cacheMsg).futureVersion(); else if (cacheMsg instanceof GridDhtAtomicUpdateResponse) return ((GridDhtAtomicUpdateResponse) cacheMsg).futureVersion(); @@ -486,8 +488,8 @@ else if (cacheMsg instanceof GridDhtAtomicUpdateResponse) @Nullable private GridCacheVersion atomicWriteVersion(GridCacheMessage cacheMsg) { if (cacheMsg instanceof GridNearAtomicAbstractUpdateRequest) return ((GridNearAtomicAbstractUpdateRequest)cacheMsg).updateVersion(); - else if (cacheMsg instanceof GridDhtAtomicUpdateRequest) - return ((GridDhtAtomicUpdateRequest)cacheMsg).writeVersion(); + else if (cacheMsg instanceof GridDhtAtomicAbstractUpdateRequest) + return ((GridDhtAtomicAbstractUpdateRequest)cacheMsg).writeVersion(); return null; } @@ -791,6 +793,21 @@ private void processFailedMessage(UUID nodeId, GridCacheMessage msg, IgniteBiInC break; + case -36: { + GridDhtAtomicSingleUpdateRequest req = (GridDhtAtomicSingleUpdateRequest)msg; + + GridDhtAtomicUpdateResponse res = new GridDhtAtomicUpdateResponse( + ctx.cacheId(), + req.futureVersion(), + ctx.deploymentEnabled()); + + res.onError(req.classError()); + + sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy()); + } + + break; + default: throw new IgniteCheckedException("Failed to send response to node. Unsupported direct type [message=" + msg + "]", msg.classError()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java index 3bbc34823dc5d..7e4c4e0b95edb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java @@ -45,6 +45,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; @@ -80,7 +81,7 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte private final CI2 completionCb; /** Update request. */ - private final GridNearAtomicAbstractUpdateRequest updateReq; + protected final GridNearAtomicAbstractUpdateRequest updateReq; /** Update response. */ final GridNearAtomicUpdateResponse updateRes; @@ -90,7 +91,7 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte /** Mappings. */ @GridToStringInclude - protected Map mappings; + protected Map mappings; /** Continuous query closures. */ private Collection> cntQryClsrs; @@ -188,23 +189,16 @@ final void addWriteEntry(GridDhtCacheEntry entry, UUID nodeId = node.id(); if (!nodeId.equals(cctx.localNodeId())) { - GridDhtAtomicUpdateRequest updateReq = mappings.get(nodeId); + GridDhtAtomicAbstractUpdateRequest updateReq = mappings.get(nodeId); if (updateReq == null) { - updateReq = new GridDhtAtomicUpdateRequest( - cctx.cacheId(), - nodeId, + updateReq = createRequest( + node, futVer, writeVer, syncMode, topVer, - forceTransformBackups, - this.updateReq.subjectId(), - this.updateReq.taskNameHash(), - forceTransformBackups ? this.updateReq.invokeArguments() : null, - cctx.deploymentEnabled(), - this.updateReq.keepBinary(), - this.updateReq.skipStore()); + forceTransformBackups); mappings.put(nodeId, updateReq); } @@ -256,7 +250,7 @@ final void addNearWriteEntries(Collection readers, AffinityTopologyVersion topVer = updateReq.topologyVersion(); for (UUID nodeId : readers) { - GridDhtAtomicUpdateRequest updateReq = mappings.get(nodeId); + GridDhtAtomicAbstractUpdateRequest updateReq = mappings.get(nodeId); if (updateReq == null) { ClusterNode node = cctx.discovery().node(nodeId); @@ -265,20 +259,13 @@ final void addNearWriteEntries(Collection readers, if (node == null) continue; - updateReq = new GridDhtAtomicUpdateRequest( - cctx.cacheId(), - nodeId, + updateReq = createRequest( + node, futVer, writeVer, syncMode, topVer, - forceTransformBackups, - this.updateReq.subjectId(), - this.updateReq.taskNameHash(), - forceTransformBackups ? this.updateReq.invokeArguments() : null, - cctx.deploymentEnabled(), - this.updateReq.keepBinary(), - this.updateReq.skipStore()); + forceTransformBackups); mappings.put(nodeId, updateReq); } @@ -336,7 +323,7 @@ final GridCacheVersion writeVersion() { final boolean registerResponse(UUID nodeId) { int resCnt0; - GridDhtAtomicUpdateRequest req = mappings != null ? mappings.get(nodeId) : null; + GridDhtAtomicAbstractUpdateRequest req = mappings != null ? mappings.get(nodeId) : null; if (req != null) { synchronized (this) { @@ -365,7 +352,7 @@ final boolean registerResponse(UUID nodeId) { */ final void map() { if (!F.isEmpty(mappings)) { - for (GridDhtAtomicUpdateRequest req : mappings.values()) { + for (GridDhtAtomicAbstractUpdateRequest req : mappings.values()) { try { cctx.io().send(req.nodeId(), req, cctx.ioPolicy()); @@ -411,6 +398,24 @@ public final void onResult(UUID nodeId) { registerResponse(nodeId); } + /** + * @param node Node. + * @param futVer Future version. + * @param writeVer Update version. + * @param syncMode Write synchronization mode. + * @param topVer Topology version. + * @param forceTransformBackups Force transform backups flag. + * @return Request. + */ + protected abstract GridDhtAtomicAbstractUpdateRequest createRequest( + ClusterNode node, + GridCacheVersion futVer, + GridCacheVersion writeVer, + CacheWriteSynchronizationMode syncMode, + @NotNull AffinityTopologyVersion topVer, + boolean forceTransformBackups + ); + /** * Callback for backup update response. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateRequest.java new file mode 100644 index 0000000000000..f0bea071afe62 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateRequest.java @@ -0,0 +1,287 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; + +import java.io.Externalizable; +import java.util.UUID; +import javax.cache.processor.EntryProcessor; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.internal.GridDirectTransient; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.GridCacheDeployable; +import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public abstract class GridDhtAtomicAbstractUpdateRequest extends GridCacheMessage implements GridCacheDeployable { + /** Message index. */ + public static final int CACHE_MSG_IDX = nextIndexId(); + + /** Node ID. */ + @GridDirectTransient + protected UUID nodeId; + + /** On response flag. Access should be synced on future. */ + @GridDirectTransient + private boolean onRes; + + /** + * Empty constructor required by {@link Externalizable}. + */ + protected GridDhtAtomicAbstractUpdateRequest() { + // N-op. + } + + /** + * Constructor. + * + * @param cacheId Cache ID. + * @param nodeId Node ID. + */ + protected GridDhtAtomicAbstractUpdateRequest(int cacheId, UUID nodeId) { + this.cacheId = cacheId; + this.nodeId = nodeId; + } + + /** {@inheritDoc} */ + @Override public int lookupIndex() { + return CACHE_MSG_IDX; + } + + /** + * @return Node ID. + */ + public UUID nodeId() { + return nodeId; + } + + /** + * @return Keep binary flag. + */ + public abstract boolean keepBinary(); + + /** + * @return Skip write-through to a persistent storage. + */ + public abstract boolean skipStore(); + + /** + * @return {@code True} if on response flag changed. + */ + public boolean onResponse() { + return !onRes && (onRes = true); + } + + /** {@inheritDoc} */ + @Override public boolean addDeploymentInfo() { + return addDepInfo; + } + + /** + * @return Force transform backups flag. + */ + public abstract boolean forceTransformBackups(); + + /** {@inheritDoc} */ + @Override public IgniteLogger messageLogger(GridCacheSharedContext ctx) { + return ctx.atomicMessageLogger(); + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + cleanup(); + } + + /** + * @param key Key to add. + * @param val Value, {@code null} if should be removed. + * @param entryProcessor Entry processor. + * @param ttl TTL (optional). + * @param conflictExpireTime Conflict expire time (optional). + * @param conflictVer Conflict version (optional). + * @param addPrevVal If {@code true} adds previous value. + * @param partId Partition. + * @param prevVal Previous value. + * @param updateCntr Update counter. + */ + public abstract void addWriteValue(KeyCacheObject key, + @Nullable CacheObject val, + EntryProcessor entryProcessor, + long ttl, + long conflictExpireTime, + @Nullable GridCacheVersion conflictVer, + boolean addPrevVal, + int partId, + @Nullable CacheObject prevVal, + @Nullable Long updateCntr + ); + + /** + * @param key Key to add. + * @param val Value, {@code null} if should be removed. + * @param entryProcessor Entry processor. + * @param ttl TTL. + * @param expireTime Expire time. + */ + public abstract void addNearWriteValue(KeyCacheObject key, + @Nullable CacheObject val, + EntryProcessor entryProcessor, + long ttl, + long expireTime); + + /** + * Cleanup values not needed after message was sent. + */ + protected abstract void cleanup(); + + /** + * @return Subject ID. + */ + public abstract UUID subjectId(); + + /** + * @return Task name. + */ + public abstract int taskNameHash(); + + /** + * @return Version assigned on primary node. + */ + public abstract GridCacheVersion futureVersion(); + + /** + * @return Write version. + */ + public abstract GridCacheVersion writeVersion(); + + /** + * @return Cache write synchronization mode. + */ + public abstract CacheWriteSynchronizationMode writeSynchronizationMode(); + + /** + * @return Keys size. + */ + public abstract int size(); + + /** + * @return Keys size. + */ + public abstract int nearSize(); + + /** + * @param key Key to check. + * @return {@code true} if request keys contain key. + */ + public abstract boolean hasKey(KeyCacheObject key); + + /** + * @param idx Key index. + * @return Key. + */ + public abstract KeyCacheObject key(int idx); + + /** + * @param idx Partition index. + * @return Partition id. + */ + public abstract int partitionId(int idx); + + /** + * @param updCntr Update counter. + * @return Update counter. + */ + public abstract Long updateCounter(int updCntr); + + /** + * @param idx Near key index. + * @return Key. + */ + public abstract KeyCacheObject nearKey(int idx); + + /** + * @param idx Key index. + * @return Value. + */ + @Nullable public abstract CacheObject value(int idx); + + /** + * @param idx Key index. + * @return Value. + */ + @Nullable public abstract CacheObject previousValue(int idx); + + /** + * @param idx Key index. + * @return Entry processor. + */ + @Nullable public abstract EntryProcessor entryProcessor(int idx); + + /** + * @param idx Near key index. + * @return Value. + */ + @Nullable public abstract CacheObject nearValue(int idx); + + /** + * @param idx Key index. + * @return Transform closure. + */ + @Nullable public abstract EntryProcessor nearEntryProcessor(int idx); + + /** + * @param idx Index. + * @return Conflict version. + */ + @Nullable public abstract GridCacheVersion conflictVersion(int idx); + + /** + * @param idx Index. + * @return TTL. + */ + public abstract long ttl(int idx); + + /** + * @param idx Index. + * @return TTL for near cache update. + */ + public abstract long nearTtl(int idx); + + /** + * @param idx Index. + * @return Conflict expire time. + */ + public abstract long conflictExpireTime(int idx); + + /** + * @param idx Index. + * @return Expire time for near cache update. + */ + public abstract long nearExpireTime(int idx); + + /** + * @return Optional arguments for entry processor. + */ + @Nullable public abstract Object[] invokeArguments(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java index d7eb062b3dae9..2a7055d551cb9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java @@ -360,11 +360,11 @@ else if (res.error() != null) { ctx.io().addHandler( ctx.cacheId(), - GridDhtAtomicUpdateRequest.class, - new CI2() { + GridDhtAtomicAbstractUpdateRequest.class, + new CI2() { @Override public void apply( UUID nodeId, - GridDhtAtomicUpdateRequest req + GridDhtAtomicAbstractUpdateRequest req ) { processDhtAtomicUpdateRequest( nodeId, @@ -3100,12 +3100,9 @@ private void processNearAtomicUpdateResponse(UUID nodeId, GridNearAtomicUpdateRe GridNearAtomicAbstractUpdateFuture fut = (GridNearAtomicAbstractUpdateFuture)ctx.mvcc().atomicFuture(res.futureVersion()); - if (fut != null) { - if (fut instanceof GridNearAtomicSingleUpdateFuture) - ((GridNearAtomicSingleUpdateFuture)fut).onResult(nodeId, res, false); - else - ((GridNearAtomicUpdateFuture)fut).onResult(nodeId, res, false); - } + if (fut != null) + fut.onResult(nodeId, res, false); + else U.warn(msgLog, "Failed to find near update future for update response (will ignore) " + "[futId" + res.futureVersion() + ", node=" + nodeId + ", res=" + res + ']'); @@ -3115,7 +3112,7 @@ private void processNearAtomicUpdateResponse(UUID nodeId, GridNearAtomicUpdateRe * @param nodeId Sender node ID. * @param req Dht atomic update request. */ - private void processDhtAtomicUpdateRequest(UUID nodeId, GridDhtAtomicUpdateRequest req) { + private void processDhtAtomicUpdateRequest(UUID nodeId, GridDhtAtomicAbstractUpdateRequest req) { if (msgLog.isDebugEnabled()) { msgLog.debug("Received DHT atomic update request [futId=" + req.futureVersion() + ", writeVer=" + req.writeVersion() + ", node=" + nodeId + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java index f83a7b7f96102..656caab261b16 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java @@ -20,7 +20,9 @@ import java.util.Collection; import java.util.List; import java.util.UUID; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.KeyCacheObject; @@ -30,6 +32,8 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteProductVersion; +import org.jetbrains.annotations.NotNull; /** * @@ -38,6 +42,9 @@ class GridDhtAtomicSingleUpdateFuture extends GridDhtAtomicAbstractUpdateFuture /** */ private static final long serialVersionUID = 0L; + /** */ + private static final IgniteProductVersion SINGLE_UPDATE_REQUEST = IgniteProductVersion.fromString("1.7.4"); + /** Future keys. */ private KeyCacheObject key; @@ -86,6 +93,49 @@ class GridDhtAtomicSingleUpdateFuture extends GridDhtAtomicAbstractUpdateFuture nearReaderEntry = entry; } + /** {@inheritDoc} */ + @Override protected GridDhtAtomicAbstractUpdateRequest createRequest( + ClusterNode node, + GridCacheVersion futVer, + GridCacheVersion writeVer, + CacheWriteSynchronizationMode syncMode, + @NotNull AffinityTopologyVersion topVer, + boolean forceTransformBackups + ) { + if (canUseSingleRequest(node)) { + assert !forceTransformBackups; + + return new GridDhtAtomicSingleUpdateRequest( + cctx.cacheId(), + node.id(), + futVer, + writeVer, + syncMode, + topVer, + updateReq.subjectId(), + updateReq.taskNameHash(), + cctx.deploymentEnabled(), + updateReq.keepBinary(), + updateReq.skipStore()); + } + else { + return new GridDhtAtomicUpdateRequest( + cctx.cacheId(), + node.id(), + futVer, + writeVer, + syncMode, + topVer, + forceTransformBackups, + updateReq.subjectId(), + updateReq.taskNameHash(), + forceTransformBackups ? updateReq.invokeArguments() : null, + cctx.deploymentEnabled(), + updateReq.keepBinary(), + updateReq.skipStore()); + } + } + /** {@inheritDoc} */ @Override public void onResult(UUID nodeId, GridDhtAtomicUpdateResponse updateRes) { if (log.isDebugEnabled()) @@ -114,6 +164,17 @@ class GridDhtAtomicSingleUpdateFuture extends GridDhtAtomicAbstractUpdateFuture updateRes.addFailedKey(key, err); } + /** + * @param node Target node + * @return {@code true} if target node supports {@link GridNearAtomicSingleUpdateRequest} + */ + private boolean canUseSingleRequest(ClusterNode node) { + return node.version().compareToIgnoreTimestamp(SINGLE_UPDATE_REQUEST) >= 0 && + cctx.expiry() == null && + updateReq.expiry() == null && + !updateReq.hasConflictData(); + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(GridDhtAtomicSingleUpdateFuture.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java new file mode 100644 index 0000000000000..a03d948017f91 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java @@ -0,0 +1,678 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; + +import java.io.Externalizable; +import java.nio.ByteBuffer; +import java.util.UUID; +import javax.cache.processor.EntryProcessor; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.KEEP_BINARY_FLAG_MASK; +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.SKIP_STORE_FLAG_MASK; + +/** + * + */ +public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdateRequest { + /** */ + private static final long serialVersionUID = 0L; + + /** Near cache key flag. */ + private static final int NEAR_FLAG_MASK = 0x80; + + /** Future version. */ + protected GridCacheVersion futVer; + + /** Write version. */ + protected GridCacheVersion writeVer; + + /** Write synchronization mode. */ + protected CacheWriteSynchronizationMode syncMode; + + /** Topology version. */ + protected AffinityTopologyVersion topVer; + + /** Subject ID. */ + protected UUID subjId; + + /** Task name hash. */ + protected int taskNameHash; + + /** Additional flags. */ + protected byte flags; + + /** Key to update. */ + @GridToStringInclude + protected KeyCacheObject key; + + /** Value to update. */ + @GridToStringInclude + protected CacheObject val; + + /** Previous value. */ + @GridToStringInclude + protected CacheObject prevVal; + + /** Partition. */ + protected long updateCntr; + + /** */ + protected int partId; + + /** + * Empty constructor required by {@link Externalizable}. + */ + public GridDhtAtomicSingleUpdateRequest() { + // No-op. + } + + /** + * Constructor. + * + * @param cacheId Cache ID. + * @param nodeId Node ID. + * @param futVer Future version. + * @param writeVer Write version for cache values. + * @param syncMode Cache write synchronization mode. + * @param topVer Topology version. + * @param subjId Subject ID. + * @param taskNameHash Task name hash code. + * @param addDepInfo Deployment info. + * @param keepBinary Keep binary flag. + * @param skipStore Skip store flag. + */ + GridDhtAtomicSingleUpdateRequest( + int cacheId, + UUID nodeId, + GridCacheVersion futVer, + GridCacheVersion writeVer, + CacheWriteSynchronizationMode syncMode, + @NotNull AffinityTopologyVersion topVer, + UUID subjId, + int taskNameHash, + boolean addDepInfo, + boolean keepBinary, + boolean skipStore + ) { + super(cacheId, nodeId); + this.futVer = futVer; + this.writeVer = writeVer; + this.syncMode = syncMode; + this.topVer = topVer; + this.subjId = subjId; + this.taskNameHash = taskNameHash; + this.addDepInfo = addDepInfo; + + if (skipStore) + setFlag(true, SKIP_STORE_FLAG_MASK); + if (keepBinary) + setFlag(true, KEEP_BINARY_FLAG_MASK); + } + + /** + * @param key Key to add. + * @param val Value, {@code null} if should be removed. + * @param entryProcessor Entry processor. + * @param ttl TTL (optional). + * @param conflictExpireTime Conflict expire time (optional). + * @param conflictVer Conflict version (optional). + * @param addPrevVal If {@code true} adds previous value. + * @param partId Partition. + * @param prevVal Previous value. + * @param updateCntr Update counter. + */ + @Override public void addWriteValue(KeyCacheObject key, + @Nullable CacheObject val, + EntryProcessor entryProcessor, + long ttl, + long conflictExpireTime, + @Nullable GridCacheVersion conflictVer, + boolean addPrevVal, + int partId, + @Nullable CacheObject prevVal, + @Nullable Long updateCntr + ) { + assert entryProcessor == null; + assert ttl <= 0 : ttl; + assert conflictExpireTime <= 0 : conflictExpireTime; + assert conflictVer == null : conflictVer; + + near(false); + + this.key = key; + this.partId = partId; + this.val = val; + + if (addPrevVal) + this.prevVal = prevVal; + + if (updateCntr != null) + this.updateCntr = updateCntr; + } + + /** + * @param key Key to add. + * @param val Value, {@code null} if should be removed. + * @param entryProcessor Entry processor. + * @param ttl TTL. + * @param expireTime Expire time. + */ + @Override public void addNearWriteValue(KeyCacheObject key, + @Nullable CacheObject val, + EntryProcessor entryProcessor, + long ttl, + long expireTime) { + assert entryProcessor == null; + assert ttl <= 0 : ttl; + + near(true); + + this.key = key; + this.val = val; + } + + /** {@inheritDoc} */ + @Override public boolean forceTransformBackups() { + return false; + } + + /** {@inheritDoc} */ + @Override public int size() { + return key != null ? near() ? 0 : 1 : 0; + } + + /** {@inheritDoc} */ + @Override public int nearSize() { + return key != null ? near() ? 1 : 0 : 0; + } + + /** {@inheritDoc} */ + @Override public boolean hasKey(KeyCacheObject key) { + return !near() && F.eq(this.key, key); + } + + /** {@inheritDoc} */ + @Override public boolean skipStore() { + return isFlag(SKIP_STORE_FLAG_MASK); + } + + /** {@inheritDoc} */ + @Override public KeyCacheObject key(int idx) { + assert idx == 0 : idx; + + return near() ? null : key; + } + + /** {@inheritDoc} */ + @Override public int partitionId(int idx) { + assert idx == 0 : idx; + + return partId; + } + + /** {@inheritDoc} */ + @Override public Long updateCounter(int updCntr) { + assert updCntr == 0 : updCntr; + + return updateCntr; + } + + /** {@inheritDoc} */ + @Override public KeyCacheObject nearKey(int idx) { + assert idx == 0 : idx; + + return near() ? key : null; + } + + /** {@inheritDoc} */ + @Override @Nullable public CacheObject value(int idx) { + assert idx == 0 : idx; + + return near() ? null : val; + } + + /** {@inheritDoc} */ + @Override public GridCacheVersion futureVersion() { + return futVer; + } + + /** {@inheritDoc} */ + @Override public GridCacheVersion writeVersion() { + return writeVer; + } + + /** {@inheritDoc} */ + @Override public int taskNameHash() { + return taskNameHash; + } + + /** {@inheritDoc} */ + @Override public UUID subjectId() { + return subjId; + } + + /** {@inheritDoc} */ + @Override public CacheWriteSynchronizationMode writeSynchronizationMode() { + return syncMode; + } + + /** {@inheritDoc} */ + @Override @Nullable public CacheObject previousValue(int idx) { + assert idx == 0 : idx; + + return prevVal; + } + + /** {@inheritDoc} */ + @Override @Nullable public CacheObject nearValue(int idx) { + assert idx == 0 : idx; + + return near() ? val : null; + } + + /** {@inheritDoc} */ + @Override @Nullable public EntryProcessor entryProcessor(int idx) { + assert idx == 0 : idx; + + return null; + } + + /** {@inheritDoc} */ + @Override @Nullable public EntryProcessor nearEntryProcessor(int idx) { + assert idx == 0 : idx; + + return null; + } + + /** {@inheritDoc} */ + @Override @Nullable public GridCacheVersion conflictVersion(int idx) { + assert idx == 0 : idx; + + return null; + } + + /** {@inheritDoc} */ + @Override public long ttl(int idx) { + assert idx == 0 : idx; + + return CU.TTL_NOT_CHANGED; + } + + /** {@inheritDoc} */ + @Override public long nearTtl(int idx) { + assert idx == 0 : idx; + + return CU.TTL_NOT_CHANGED; + } + + /** {@inheritDoc} */ + @Override public long conflictExpireTime(int idx) { + assert idx == 0 : idx; + + return CU.EXPIRE_TIME_CALCULATE; + } + + /** {@inheritDoc} */ + @Override public long nearExpireTime(int idx) { + assert idx == 0 : idx; + + return CU.EXPIRE_TIME_CALCULATE; + } + + /** {@inheritDoc} */ + @Override @Nullable public Object[] invokeArguments() { + return null; + } + + /** {@inheritDoc} */ + @Override public boolean keepBinary() { + return isFlag(KEEP_BINARY_FLAG_MASK); + } + + /** + * + */ + private boolean near() { + return isFlag(NEAR_FLAG_MASK); + } + + /** + * + */ + private void near(boolean near) { + setFlag(near, NEAR_FLAG_MASK); + } + + /** {@inheritDoc} */ + @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { + super.prepareMarshal(ctx); + + GridCacheContext cctx = ctx.cacheContext(cacheId); + + prepareMarshalObject(key, cctx); + + prepareMarshalObject(val, cctx); + + prepareMarshalObject(prevVal, cctx); + + } + + /** {@inheritDoc} */ + @Override public void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { + super.finishUnmarshal(ctx, ldr); + + GridCacheContext cctx = ctx.cacheContext(cacheId); + + finishUnmarshalObject(key, cctx, ldr); + + finishUnmarshalObject(val, cctx, ldr); + + finishUnmarshalObject(prevVal, cctx, ldr); + + key.partition(partId); + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!super.writeTo(buf, writer)) + return false; + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 3: + if (!writer.writeByte("flags", flags)) + return false; + + writer.incrementState(); + + case 4: + if (!writer.writeMessage("futVer", futVer)) + return false; + + writer.incrementState(); + + case 5: + if (!writer.writeMessage("key", key)) + return false; + + writer.incrementState(); + + case 6: + if (!writer.writeInt("partId", partId)) + return false; + + writer.incrementState(); + + case 7: + if (!writer.writeMessage("prevVal", prevVal)) + return false; + + writer.incrementState(); + + case 8: + if (!writer.writeUuid("subjId", subjId)) + return false; + + writer.incrementState(); + + case 9: + if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1)) + return false; + + writer.incrementState(); + + case 10: + if (!writer.writeInt("taskNameHash", taskNameHash)) + return false; + + writer.incrementState(); + + case 11: + if (!writer.writeMessage("topVer", topVer)) + return false; + + writer.incrementState(); + + case 12: + if (!writer.writeLong("updateCntr", updateCntr)) + return false; + + writer.incrementState(); + + case 13: + if (!writer.writeMessage("val", val)) + return false; + + writer.incrementState(); + + case 14: + if (!writer.writeMessage("writeVer", writeVer)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + if (!super.readFrom(buf, reader)) + return false; + + switch (reader.state()) { + case 3: + flags = reader.readByte("flags"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 4: + futVer = reader.readMessage("futVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 5: + key = reader.readMessage("key"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 6: + partId = reader.readInt("partId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 7: + prevVal = reader.readMessage("prevVal"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 8: + subjId = reader.readUuid("subjId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 9: + byte syncModeOrd; + + syncModeOrd = reader.readByte("syncMode"); + + if (!reader.isLastRead()) + return false; + + syncMode = CacheWriteSynchronizationMode.fromOrdinal(syncModeOrd); + + reader.incrementState(); + + case 10: + taskNameHash = reader.readInt("taskNameHash"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 11: + topVer = reader.readMessage("topVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 12: + updateCntr = reader.readLong("updateCntr"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 13: + val = reader.readMessage("val"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 14: + writeVer = reader.readMessage("writeVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(GridDhtAtomicSingleUpdateRequest.class); + } + + /** + * @param obj CacheObject to marshal + * @param ctx context + * @throws IgniteCheckedException if error + */ + private void prepareMarshalObject(CacheObject obj, GridCacheContext ctx) throws IgniteCheckedException { + if (obj != null) + obj.prepareMarshal(ctx.cacheObjectContext()); + } + + /** + * @param obj CacheObject un to marshal + * @param ctx context + * @param ldr class loader + * @throws IgniteCheckedException if error + */ + private void finishUnmarshalObject(@Nullable CacheObject obj, GridCacheContext ctx, + ClassLoader ldr) throws IgniteCheckedException { + if (obj != null) + obj.finishUnmarshal(ctx.cacheObjectContext(), ldr); + } + + /** + * Cleanup values not needed after message was sent. + */ + @Override protected void cleanup() { + val = null; + prevVal = null; + } + + /** {@inheritDoc} */ + @Override public byte directType() { + return -36; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 15; + } + + /** + * Sets flag mask. + * + * @param flag Set or clear. + * @param mask Mask. + */ + private void setFlag(boolean flag, int mask) { + flags = flag ? (byte)(flags | mask) : (byte)(flags & ~mask); + } + + /** + * Reags flag mask. + * + * @param mask Mask to read. + * @return Flag value. + */ + private boolean isFlag(int mask) { + return (flags & mask) != 0; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridDhtAtomicSingleUpdateRequest.class, this, "super", super.toString()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java index 864aadd2447e2..dd1f1c49e9a6a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java @@ -23,7 +23,9 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.KeyCacheObject; @@ -33,6 +35,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.NotNull; /** * DHT atomic cache backup update future. @@ -117,6 +120,29 @@ class GridDhtAtomicUpdateFuture extends GridDhtAtomicAbstractUpdateFuture { updateRes.addFailedKey(key, err); } + /** {@inheritDoc} */ + @Override protected GridDhtAtomicAbstractUpdateRequest createRequest(ClusterNode node, + GridCacheVersion futVer, + GridCacheVersion writeVer, + CacheWriteSynchronizationMode syncMode, + @NotNull AffinityTopologyVersion topVer, + boolean forceTransformBackups) { + return new GridDhtAtomicUpdateRequest( + cctx.cacheId(), + node.id(), + futVer, + writeVer, + syncMode, + topVer, + forceTransformBackups, + updateReq.subjectId(), + updateReq.taskNameHash(), + forceTransformBackups ? updateReq.invokeArguments() : null, + cctx.deploymentEnabled(), + updateReq.keepBinary(), + updateReq.skipStore()); + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(GridDhtAtomicUpdateFuture.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java index 55f7560dbd268..f2fbb0e5a9864 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java @@ -20,25 +20,22 @@ import java.io.Externalizable; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Collection; import java.util.List; import java.util.UUID; import javax.cache.processor.EntryProcessor; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.internal.GridDirectCollection; import org.apache.ignite.internal.GridDirectTransient; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.GridCacheDeployable; -import org.apache.ignite.internal.processors.cache.GridCacheMessage; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType; @@ -52,17 +49,10 @@ /** * Lite dht cache backup update request. */ -public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements GridCacheDeployable { +public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateRequest { /** */ private static final long serialVersionUID = 0L; - /** Message index. */ - public static final int CACHE_MSG_IDX = nextIndexId(); - - /** Node ID. */ - @GridDirectTransient - private UUID nodeId; - /** Future version. */ private GridCacheVersion futVer; @@ -151,10 +141,6 @@ public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements Grid /** Partition. */ private GridLongList updateCntrs; - /** On response flag. Access should be synced on future. */ - @GridDirectTransient - private boolean onRes; - /** */ @GridDirectTransient private List partIds; @@ -162,9 +148,7 @@ public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements Grid /** Keep binary flag. */ private boolean keepBinary; - /** - * Additional flags. - */ + /** Additional flags. */ private byte flags; /** @@ -204,10 +188,8 @@ public GridDhtAtomicUpdateRequest( boolean keepBinary, boolean skipStore ) { - assert invokeArgs == null || forceTransformBackups; + super(cacheId, nodeId); - this.cacheId = cacheId; - this.nodeId = nodeId; this.futVer = futVer; this.writeVer = writeVer; this.syncMode = syncMode; @@ -215,12 +197,14 @@ public GridDhtAtomicUpdateRequest( this.forceTransformBackups = forceTransformBackups; this.subjId = subjId; this.taskNameHash = taskNameHash; + + assert invokeArgs == null || forceTransformBackups; + this.invokeArgs = invokeArgs; this.addDepInfo = addDepInfo; this.keepBinary = keepBinary; - if (skipStore) - flags = (byte)(flags | SKIP_STORE_FLAG_MASK); + setFlag(skipStore, SKIP_STORE_FLAG_MASK); keys = new ArrayList<>(); partIds = new ArrayList<>(); @@ -233,26 +217,8 @@ public GridDhtAtomicUpdateRequest( vals = new ArrayList<>(); } - /** - * @return Force transform backups flag. - */ - public boolean forceTransformBackups() { - return forceTransformBackups; - } - - /** - * @param key Key to add. - * @param val Value, {@code null} if should be removed. - * @param entryProcessor Entry processor. - * @param ttl TTL (optional). - * @param conflictExpireTime Conflict expire time (optional). - * @param conflictVer Conflict version (optional). - * @param addPrevVal If {@code true} adds previous value. - * @param partId Partition. - * @param prevVal Previous value. - * @param updateCntr Update counter. - */ - public void addWriteValue(KeyCacheObject key, + /** {@inheritDoc} */ + @Override public void addWriteValue(KeyCacheObject key, @Nullable CacheObject val, EntryProcessor entryProcessor, long ttl, @@ -328,14 +294,8 @@ else if (conflictVers != null) conflictExpireTimes.add(conflictExpireTime); } - /** - * @param key Key to add. - * @param val Value, {@code null} if should be removed. - * @param entryProcessor Entry processor. - * @param ttl TTL. - * @param expireTime Expire time. - */ - public void addNearWriteValue(KeyCacheObject key, + /** {@inheritDoc} */ + @Override public void addNearWriteValue(KeyCacheObject key, @Nullable CacheObject val, EntryProcessor entryProcessor, long ttl, @@ -387,183 +347,114 @@ public void addNearWriteValue(KeyCacheObject key, } /** {@inheritDoc} */ - @Override public int lookupIndex() { - return CACHE_MSG_IDX; - } - - /** - * @return Node ID. - */ - public UUID nodeId() { - return nodeId; + @Override public boolean forceTransformBackups() { + return forceTransformBackups; } - /** - * @return Subject ID. - */ - public UUID subjectId() { + /** {@inheritDoc} */ + @Override public UUID subjectId() { return subjId; } - /** - * @return Task name. - */ - public int taskNameHash() { + /** {@inheritDoc} */ + @Override public int taskNameHash() { return taskNameHash; } - /** - * @return Keys size. - */ - public int size() { - return keys.size(); - } - - /** - * @return Keys size. - */ - public int nearSize() { - return nearKeys != null ? nearKeys.size() : 0; - } - - /** - * @return Version assigned on primary node. - */ - public GridCacheVersion futureVersion() { + /** {@inheritDoc} */ + @Override public GridCacheVersion futureVersion() { return futVer; } - /** - * @return Write version. - */ - public GridCacheVersion writeVersion() { + /** {@inheritDoc} */ + @Override public GridCacheVersion writeVersion() { return writeVer; } - /** - * @return Cache write synchronization mode. - */ - public CacheWriteSynchronizationMode writeSynchronizationMode() { + /** {@inheritDoc} */ + @Override public CacheWriteSynchronizationMode writeSynchronizationMode() { return syncMode; } - /** - * @return Topology version. - */ + /** {@inheritDoc} */ @Override public AffinityTopologyVersion topologyVersion() { return topVer; } - /** - * @return Keys. - */ - public Collection keys() { - return keys; + /** {@inheritDoc} */ + @Override public int size() { + return keys.size(); } - /** - * @param idx Key index. - * @return Key. - */ - public KeyCacheObject key(int idx) { - return keys.get(idx); + /** {@inheritDoc} */ + @Override public int nearSize() { + return nearKeys != null ? nearKeys.size() : 0; } - /** - * @param idx Partition index. - * @return Partition id. - */ - public int partitionId(int idx) { - return partIds.get(idx); + /** {@inheritDoc} */ + @Override public boolean hasKey(KeyCacheObject key) { + return F.contains(keys, key); } - /** - * @return Skip write-through to a persistent storage. - */ - public boolean skipStore() { - return (flags & SKIP_STORE_FLAG_MASK) == SKIP_STORE_FLAG_MASK; + /** {@inheritDoc} */ + @Override public KeyCacheObject key(int idx) { + return keys.get(idx); } - /** - * @param updCntr Update counter. - * @return Update counter. - */ - public Long updateCounter(int updCntr) { + /** {@inheritDoc} */ + @Override public int partitionId(int idx) { + return partIds.get(idx); + } + + /** {@inheritDoc} */ + @Override public Long updateCounter(int updCntr) { if (updateCntrs != null && updCntr < updateCntrs.size()) return updateCntrs.get(updCntr); return null; } - /** - * @param idx Near key index. - * @return Key. - */ - public KeyCacheObject nearKey(int idx) { + /** {@inheritDoc} */ + @Override public KeyCacheObject nearKey(int idx) { return nearKeys.get(idx); } - /** - * @return Keep binary flag. - */ - public boolean keepBinary() { - return keepBinary; - } - - /** - * @param idx Key index. - * @return Value. - */ - @Nullable public CacheObject value(int idx) { + /** {@inheritDoc} */ + @Override @Nullable public CacheObject value(int idx) { if (vals != null) return vals.get(idx); return null; } - /** - * @param idx Key index. - * @return Value. - */ - @Nullable public CacheObject previousValue(int idx) { + /** {@inheritDoc} */ + @Override @Nullable public CacheObject previousValue(int idx) { if (prevVals != null) return prevVals.get(idx); return null; } - /** - * @param idx Key index. - * @return Entry processor. - */ - @Nullable public EntryProcessor entryProcessor(int idx) { + /** {@inheritDoc} */ + @Override @Nullable public EntryProcessor entryProcessor(int idx) { return entryProcessors == null ? null : entryProcessors.get(idx); } - /** - * @param idx Near key index. - * @return Value. - */ - @Nullable public CacheObject nearValue(int idx) { + /** {@inheritDoc} */ + @Override @Nullable public CacheObject nearValue(int idx) { if (nearVals != null) return nearVals.get(idx); return null; } - /** - * @param idx Key index. - * @return Transform closure. - */ - @Nullable public EntryProcessor nearEntryProcessor(int idx) { + /** {@inheritDoc} */ + @Override @Nullable public EntryProcessor nearEntryProcessor(int idx) { return nearEntryProcessors == null ? null : nearEntryProcessors.get(idx); } - /** - * @param idx Index. - * @return Conflict version. - */ - @Nullable public GridCacheVersion conflictVersion(int idx) { + /** {@inheritDoc} */ + @Override @Nullable public GridCacheVersion conflictVersion(int idx) { if (conflictVers != null) { assert idx >= 0 && idx < conflictVers.size(); @@ -573,11 +464,8 @@ public boolean keepBinary() { return null; } - /** - * @param idx Index. - * @return TTL. - */ - public long ttl(int idx) { + /** {@inheritDoc} */ + @Override public long ttl(int idx) { if (ttls != null) { assert idx >= 0 && idx < ttls.size(); @@ -587,11 +475,8 @@ public long ttl(int idx) { return CU.TTL_NOT_CHANGED; } - /** - * @param idx Index. - * @return TTL for near cache update. - */ - public long nearTtl(int idx) { + /** {@inheritDoc} */ + @Override public long nearTtl(int idx) { if (nearTtls != null) { assert idx >= 0 && idx < nearTtls.size(); @@ -601,11 +486,8 @@ public long nearTtl(int idx) { return CU.TTL_NOT_CHANGED; } - /** - * @param idx Index. - * @return Conflict expire time. - */ - public long conflictExpireTime(int idx) { + /** {@inheritDoc} */ + @Override public long conflictExpireTime(int idx) { if (conflictExpireTimes != null) { assert idx >= 0 && idx < conflictExpireTimes.size(); @@ -615,11 +497,8 @@ public long conflictExpireTime(int idx) { return CU.EXPIRE_TIME_CALCULATE; } - /** - * @param idx Index. - * @return Expire time for near cache update. - */ - public long nearExpireTime(int idx) { + /** {@inheritDoc} */ + @Override public long nearExpireTime(int idx) { if (nearExpireTimes != null) { assert idx >= 0 && idx < nearExpireTimes.size(); @@ -629,17 +508,18 @@ public long nearExpireTime(int idx) { return CU.EXPIRE_TIME_CALCULATE; } - /** - * @return {@code True} if on response flag changed. - */ - public boolean onResponse() { - return !onRes && (onRes = true); + /** {@inheritDoc} */ + @Override public boolean keepBinary() { + return keepBinary; } - /** - * @return Optional arguments for entry processor. - */ - @Nullable public Object[] invokeArguments() { + /** {@inheritDoc} */ + @Override public boolean skipStore() { + return isFlag(SKIP_STORE_FLAG_MASK); + } + + /** {@inheritDoc} */ + @Override @Nullable public Object[] invokeArguments() { return invokeArgs; } @@ -710,16 +590,6 @@ public boolean onResponse() { } } - /** {@inheritDoc} */ - @Override public boolean addDeploymentInfo() { - return addDepInfo; - } - - /** {@inheritDoc} */ - @Override public IgniteLogger messageLogger(GridCacheSharedContext ctx) { - return ctx.atomicMessageLogger(); - } - /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { writer.setBuffer(buf); @@ -1083,14 +953,7 @@ public boolean onResponse() { } /** {@inheritDoc} */ - @Override public void onAckReceived() { - cleanup(); - } - - /** - * Cleanup values not needed after message was sent. - */ - private void cleanup() { + @Override protected void cleanup() { nearVals = null; prevVals = null; } @@ -1105,6 +968,27 @@ private void cleanup() { return 26; } + /** + * Sets flag mask. + * + * @param flag Set or clear. + * @param mask Mask. + */ + private void setFlag(boolean flag, int mask) { + flags = flag ? (byte)(flags | mask) : (byte)(flags & ~mask); + } + + /** + * Reags flag mask. + * + * @param mask Mask to read. + * @return Flag value. + */ + private boolean isFlag(int mask) { + return (flags & mask) != 0; + } + + /** {@inheritDoc} */ @Override public String toString() { return S.toString(GridDhtAtomicUpdateRequest.class, this, "super", super.toString()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java index bee2ecd61262a..bae9e3a1bc31b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java @@ -223,4 +223,9 @@ public abstract void addUpdateEntry(KeyCacheObject key, * @return Key. */ public abstract KeyCacheObject key(int idx); + + /** + * @return {@code True} if request does not have conflict data. + */ + public abstract boolean hasConflictData(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java index b733d7b500f18..c78582893e742 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java @@ -41,6 +41,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -68,24 +69,49 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat /** Future version. */ private GridCacheVersion futVer; - /** Fast map flag. */ - private boolean fastMap; - /** Update version. Set to non-null if fastMap is {@code true}. */ private GridCacheVersion updateVer; /** Topology version. */ private AffinityTopologyVersion topVer; - /** Topology locked flag. Set if atomic update is performed inside TX or explicit lock. */ - private boolean topLocked; - /** Write synchronization mode. */ private CacheWriteSynchronizationMode syncMode; /** Update operation. */ private GridCacheOperation op; + /** Subject ID. */ + protected UUID subjId; + + /** Task name hash. */ + protected int taskNameHash; + + /** */ + @GridDirectTransient + private GridNearAtomicUpdateResponse res; + + /** Fast map flag. */ + protected boolean fastMap; + + /** Topology locked flag. Set if atomic update is performed inside TX or explicit lock. */ + protected boolean topLocked; + + /** Flag indicating whether request contains primary keys. */ + protected boolean hasPrimary; + + /** Skip write-through to a persistent storage. */ + protected boolean skipStore; + + /** */ + protected boolean clientReq; + + /** Keep binary flag. */ + protected boolean keepBinary; + + /** Return value flag. */ + protected boolean retval; + /** Keys to update. */ @GridToStringInclude @GridDirectCollection(KeyCacheObject.class) @@ -107,13 +133,6 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat @GridDirectCollection(byte[].class) private List entryProcessorsBytes; - /** Optional arguments for entry processor. */ - @GridDirectTransient - private Object[] invokeArgs; - - /** Entry processor arguments bytes. */ - private byte[][] invokeArgsBytes; - /** Conflict versions. */ @GridDirectCollection(GridCacheVersion.class) private List conflictVers; @@ -124,8 +143,12 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat /** Conflict expire times. */ private GridLongList conflictExpireTimes; - /** Return value flag. */ - private boolean retval; + /** Optional arguments for entry processor. */ + @GridDirectTransient + private Object[] invokeArgs; + + /** Entry processor arguments bytes. */ + private byte[][] invokeArgsBytes; /** Expiry policy. */ @GridDirectTransient @@ -137,28 +160,6 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat /** Filter. */ private CacheEntryPredicate[] filter; - /** Flag indicating whether request contains primary keys. */ - private boolean hasPrimary; - - /** Subject ID. */ - private UUID subjId; - - /** Task name hash. */ - private int taskNameHash; - - /** Skip write-through to a persistent storage. */ - private boolean skipStore; - - /** */ - private boolean clientReq; - - /** Keep binary flag. */ - private boolean keepBinary; - - /** */ - @GridDirectTransient - private GridNearAtomicUpdateResponse res; - /** Maximum possible size of inner collections. */ @GridDirectTransient private int initSize; @@ -523,7 +524,7 @@ else if (conflictVers != null) } /** {@inheritDoc} */ - @Nullable @Override public CacheEntryPredicate[] filter() { + @Override @Nullable public CacheEntryPredicate[] filter() { return filter; } @@ -532,12 +533,20 @@ else if (conflictVers != null) return expiryPlc; } + /** {@inheritDoc} */ + @Override public boolean hasConflictData() { + return F.size(conflictVers) > 0 || conflictTtls != null || conflictExpireTimes != null; + } + /** {@inheritDoc} */ @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { super.prepareMarshal(ctx); GridCacheContext cctx = ctx.cacheContext(cacheId); + if (expiryPlc != null && expiryPlcBytes == null) + expiryPlcBytes = CU.marshal(cctx, new IgniteExternalizableExpiryPolicy(expiryPlc)); + prepareMarshalCacheObjects(keys, cctx); if (filter != null) { @@ -555,9 +564,6 @@ else if (conflictVers != null) filter = null; } - if (expiryPlc != null && expiryPlcBytes == null) - expiryPlcBytes = CU.marshal(cctx, new IgniteExternalizableExpiryPolicy(expiryPlc)); - if (op == TRANSFORM) { // force addition of deployment info for entry processors if P2P is enabled globally. if (!addDepInfo && ctx.deploymentEnabled()) @@ -579,8 +585,18 @@ else if (conflictVers != null) GridCacheContext cctx = ctx.cacheContext(cacheId); + if (expiryPlcBytes != null && expiryPlc == null) + expiryPlc = U.unmarshal(ctx, expiryPlcBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + finishUnmarshalCacheObjects(keys, cctx, ldr); + if (filter != null) { + for (CacheEntryPredicate p : filter) { + if (p != null) + p.finishUnmarshal(cctx, ldr); + } + } + if (op == TRANSFORM) { if (entryProcessors == null) entryProcessors = unmarshalCollection(entryProcessorsBytes, ctx, ldr); @@ -591,16 +607,6 @@ else if (conflictVers != null) else finishUnmarshalCacheObjects(vals, cctx, ldr); - if (filter != null) { - for (CacheEntryPredicate p : filter) { - if (p != null) - p.finishUnmarshal(cctx, ldr); - } - } - - if (expiryPlcBytes != null && expiryPlc == null) - expiryPlc = U.unmarshal(ctx, expiryPlcBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); - if (partIds != null && !partIds.isEmpty()) { assert partIds.size() == keys.size(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java index 211b472ad027c..f3b972601f192 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java @@ -226,6 +226,11 @@ public GridNearAtomicSingleUpdateRequest() { return CU.EXPIRE_TIME_CALCULATE; } + /** {@inheritDoc} */ + @Override public boolean hasConflictData() { + return false; + } + /** * {@inheritDoc} * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java index b5b2c728fa4f8..a8219b08cefdc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java @@ -41,8 +41,8 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse; @@ -302,15 +302,13 @@ private void processNearAtomicUpdateResponse( */ public void processDhtAtomicUpdateRequest( UUID nodeId, - GridDhtAtomicUpdateRequest req, + GridDhtAtomicAbstractUpdateRequest req, GridDhtAtomicUpdateResponse res ) { GridCacheVersion ver = req.writeVersion(); assert ver != null; - Collection backupKeys = req.keys(); - boolean intercept = req.forceTransformBackups() && ctx.config().getInterceptor() != null; String taskName = ctx.kernalContext().task().resolveTaskName(req.taskNameHash()); @@ -329,7 +327,7 @@ public void processDhtAtomicUpdateRequest( break; } - if (F.contains(backupKeys, key)) { // Reader became backup. + if (req.hasKey(key)) { // Reader became backup. if (entry.markObsolete(ver)) removeEntry(entry); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java index a6d612a9dbfad..e8c5db18dba33 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java @@ -27,6 +27,7 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicSingleUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest; @@ -141,6 +142,7 @@ protected void checkMessages(boolean clientMode, commSpi.registerMessage(GridNearAtomicSingleUpdateRequest.class); commSpi.registerMessage(GridNearAtomicFullUpdateRequest.class); commSpi.registerMessage(GridDhtAtomicUpdateRequest.class); + commSpi.registerMessage(GridDhtAtomicSingleUpdateRequest.class); int putCnt = 15; @@ -171,7 +173,7 @@ protected void checkMessages(boolean clientMode, assertEquals(expNearCnt, commSpi.messageCount(GridNearAtomicFullUpdateRequest.class)); assertEquals(expNearSingleCnt, commSpi.messageCount(GridNearAtomicSingleUpdateRequest.class)); - assertEquals(expDhtCnt, commSpi.messageCount(GridDhtAtomicUpdateRequest.class)); + assertEquals(expDhtCnt, commSpi.messageCount(GridDhtAtomicSingleUpdateRequest.class)); if (writeOrderMode == CLOCK) { for (int i = 1; i < 4; i++) { @@ -179,7 +181,7 @@ protected void checkMessages(boolean clientMode, assertEquals(0, commSpi.messageCount(GridNearAtomicSingleUpdateRequest.class)); assertEquals(0, commSpi.messageCount(GridNearAtomicFullUpdateRequest.class)); - assertEquals(0, commSpi.messageCount(GridDhtAtomicUpdateRequest.class)); + assertEquals(0, commSpi.messageCount(GridDhtAtomicSingleUpdateRequest.class)); } } else { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java index 0899423599660..644e31012f454 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java @@ -478,7 +478,7 @@ private boolean delayMessage(GridIoMessage msg) { Object origMsg = msg.message(); return delay && - ((origMsg instanceof GridNearAtomicAbstractUpdateRequest) || (origMsg instanceof GridDhtAtomicUpdateRequest)); + ((origMsg instanceof GridNearAtomicAbstractUpdateRequest) || (origMsg instanceof GridDhtAtomicAbstractUpdateRequest)); } } } \ No newline at end of file From 6e4a279e34584881469a7d841432e6c38db2f06f Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Mon, 21 Nov 2016 19:15:17 +0500 Subject: [PATCH 359/487] IGNITE-2355: fix test - clear client connections before and after a test. --- ...ClientProtocolMultipleServersSelfTest.java | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java index 0e51938c12f29..cb83a738e02ba 100644 --- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java +++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java @@ -23,6 +23,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -44,6 +45,8 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.hadoop.mapreduce.IgniteHadoopClientProtocolProvider; import org.apache.ignite.igfs.IgfsPath; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.client.GridClient; import org.apache.ignite.internal.client.GridServerUnreachableException; import org.apache.ignite.internal.processors.hadoop.impl.HadoopAbstractSelfTest; import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils; @@ -75,10 +78,19 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS return true; } + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + clearClients(); + } + /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); + clearClients(); + super.afterTest(); } @@ -91,6 +103,18 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS return cfg; } + /** + * + */ + private void clearClients() { + ConcurrentHashMap> cliMap = GridTestUtils.getFieldValue( + IgniteHadoopClientProtocolProvider.class, + IgniteHadoopClientProtocolProvider.class, + "cliMap"); + + cliMap.clear(); + } + /** * @throws Exception If failed. */ From d32fa21b673814b060d2362f06ff44838e9c2cdc Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 22 Nov 2016 11:33:55 +0300 Subject: [PATCH 360/487] IGNITE-3075 Fixed condition for 'single' request creation --- .../GridDhtAtomicAbstractUpdateFuture.java | 19 +++++++----- .../dht/atomic/GridDhtAtomicCache.java | 2 +- .../GridDhtAtomicSingleUpdateFuture.java | 30 ++++++++++++------- .../dht/atomic/GridDhtAtomicUpdateFuture.java | 10 +++++-- .../GridNearAtomicAbstractUpdateRequest.java | 5 ---- .../GridNearAtomicFullUpdateRequest.java | 5 ---- .../GridNearAtomicSingleUpdateRequest.java | 9 ------ ...cheLoadingConcurrentGridStartSelfTest.java | 6 +++- 8 files changed, 44 insertions(+), 42 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java index 7e4c4e0b95edb..361fbe2ef8ebf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java @@ -86,9 +86,6 @@ public abstract class GridDhtAtomicAbstractUpdateFuture extends GridFutureAdapte /** Update response. */ final GridNearAtomicUpdateResponse updateRes; - /** Force transform backup flag. */ - private boolean forceTransformBackups; - /** Mappings. */ @GridToStringInclude protected Map mappings; @@ -198,7 +195,9 @@ final void addWriteEntry(GridDhtCacheEntry entry, writeVer, syncMode, topVer, - forceTransformBackups); + ttl, + conflictExpireTime, + conflictVer); mappings.put(nodeId, updateReq); } @@ -265,7 +264,9 @@ final void addNearWriteEntries(Collection readers, writeVer, syncMode, topVer, - forceTransformBackups); + ttl, + expireTime, + null); mappings.put(nodeId, updateReq); } @@ -404,7 +405,9 @@ public final void onResult(UUID nodeId) { * @param writeVer Update version. * @param syncMode Write synchronization mode. * @param topVer Topology version. - * @param forceTransformBackups Force transform backups flag. + * @param ttl TTL. + * @param conflictExpireTime Conflict expire time. + * @param conflictVer Conflict version. * @return Request. */ protected abstract GridDhtAtomicAbstractUpdateRequest createRequest( @@ -413,7 +416,9 @@ protected abstract GridDhtAtomicAbstractUpdateRequest createRequest( GridCacheVersion writeVer, CacheWriteSynchronizationMode syncMode, @NotNull AffinityTopologyVersion topVer, - boolean forceTransformBackups + long ttl, + long conflictExpireTime, + @Nullable GridCacheVersion conflictVer ); /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java index 2a7055d551cb9..940c74ed863af 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java @@ -2369,7 +2369,7 @@ private UpdateSingleResult updateSingle( assert !(newConflictVer instanceof GridCacheVersionEx) : newConflictVer; - boolean primary = !req.fastMap() || ctx.affinity().primary(ctx.localNode(), entry.key(), + boolean primary = !req.fastMap() || ctx.affinity().primary(ctx.localNode(), entry.partition(), req.topologyVersion()); Object writeVal = op == TRANSFORM ? req.entryProcessor(i) : req.writeValue(i); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java index 656caab261b16..20d6e90971c61 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java @@ -30,10 +30,12 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.typedef.CI2; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteProductVersion; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; /** * @@ -100,11 +102,11 @@ class GridDhtAtomicSingleUpdateFuture extends GridDhtAtomicAbstractUpdateFuture GridCacheVersion writeVer, CacheWriteSynchronizationMode syncMode, @NotNull AffinityTopologyVersion topVer, - boolean forceTransformBackups + long ttl, + long conflictExpireTime, + @Nullable GridCacheVersion conflictVer ) { - if (canUseSingleRequest(node)) { - assert !forceTransformBackups; - + if (canUseSingleRequest(node, ttl, conflictExpireTime, conflictVer)) { return new GridDhtAtomicSingleUpdateRequest( cctx.cacheId(), node.id(), @@ -126,10 +128,10 @@ class GridDhtAtomicSingleUpdateFuture extends GridDhtAtomicAbstractUpdateFuture writeVer, syncMode, topVer, - forceTransformBackups, + false, updateReq.subjectId(), updateReq.taskNameHash(), - forceTransformBackups ? updateReq.invokeArguments() : null, + null, cctx.deploymentEnabled(), updateReq.keepBinary(), updateReq.skipStore()); @@ -166,13 +168,19 @@ class GridDhtAtomicSingleUpdateFuture extends GridDhtAtomicAbstractUpdateFuture /** * @param node Target node - * @return {@code true} if target node supports {@link GridNearAtomicSingleUpdateRequest} + * @param ttl TTL. + * @param conflictExpireTime Conflict expire time. + * @param conflictVer Conflict version. + * @return {@code True} if it is possible to use {@link GridDhtAtomicSingleUpdateRequest}. */ - private boolean canUseSingleRequest(ClusterNode node) { + private boolean canUseSingleRequest(ClusterNode node, + long ttl, + long conflictExpireTime, + @Nullable GridCacheVersion conflictVer) { return node.version().compareToIgnoreTimestamp(SINGLE_UPDATE_REQUEST) >= 0 && - cctx.expiry() == null && - updateReq.expiry() == null && - !updateReq.hasConflictData(); + (ttl == CU.TTL_NOT_CHANGED) && + (conflictExpireTime == CU.EXPIRE_TIME_CALCULATE) && + conflictVer == null; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java index dd1f1c49e9a6a..efb35c4f2cfdb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java @@ -36,6 +36,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; /** * DHT atomic cache backup update future. @@ -126,7 +127,10 @@ class GridDhtAtomicUpdateFuture extends GridDhtAtomicAbstractUpdateFuture { GridCacheVersion writeVer, CacheWriteSynchronizationMode syncMode, @NotNull AffinityTopologyVersion topVer, - boolean forceTransformBackups) { + long ttl, + long conflictExpireTime, + @Nullable GridCacheVersion conflictVer + ) { return new GridDhtAtomicUpdateRequest( cctx.cacheId(), node.id(), @@ -134,10 +138,10 @@ class GridDhtAtomicUpdateFuture extends GridDhtAtomicAbstractUpdateFuture { writeVer, syncMode, topVer, - forceTransformBackups, + false, updateReq.subjectId(), updateReq.taskNameHash(), - forceTransformBackups ? updateReq.invokeArguments() : null, + null, cctx.deploymentEnabled(), updateReq.keepBinary(), updateReq.skipStore()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java index bae9e3a1bc31b..bee2ecd61262a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateRequest.java @@ -223,9 +223,4 @@ public abstract void addUpdateEntry(KeyCacheObject key, * @return Key. */ public abstract KeyCacheObject key(int idx); - - /** - * @return {@code True} if request does not have conflict data. - */ - public abstract boolean hasConflictData(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java index c78582893e742..1b116884c5bc8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java @@ -533,11 +533,6 @@ else if (conflictVers != null) return expiryPlc; } - /** {@inheritDoc} */ - @Override public boolean hasConflictData() { - return F.size(conflictVers) > 0 || conflictTtls != null || conflictExpireTimes != null; - } - /** {@inheritDoc} */ @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { super.prepareMarshal(ctx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java index f3b972601f192..1c1adddb2c713 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java @@ -227,15 +227,6 @@ public GridNearAtomicSingleUpdateRequest() { } /** {@inheritDoc} */ - @Override public boolean hasConflictData() { - return false; - } - - /** - * {@inheritDoc} - * - * @param ctx - */ @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { super.prepareMarshal(ctx); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java index 0801691e78b85..ce64e1d07da1d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java @@ -42,6 +42,7 @@ import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jetbrains.annotations.Nullable; @@ -77,6 +78,8 @@ public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractT @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); + ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1); + CacheConfiguration ccfg = new CacheConfiguration(); ccfg.setCacheMode(PARTITIONED); @@ -95,7 +98,7 @@ public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractT else cfg.setCacheConfiguration(ccfg); - if (!configured) + if (!configured) { ccfg.setNodeFilter(new P1() { @Override public boolean apply(ClusterNode node) { String name = node.attribute(ATTR_GRID_NAME).toString(); @@ -103,6 +106,7 @@ public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractT return !getTestGridName(0).equals(name); } }); + } return cfg; } From d15eba4becf7515b512c1032b193ce75e1589177 Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Tue, 22 Nov 2016 11:56:20 +0300 Subject: [PATCH 361/487] IGNITE-4225 DataStreamer can hang on changing topology --- .../internal/processors/datastreamer/DataStreamerImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java index 443783b4ccc2f..bb9ffdd5795a2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java @@ -1919,7 +1919,7 @@ protected static class IsolatedUpdater implements StreamReceiver Date: Tue, 22 Nov 2016 12:02:57 +0300 Subject: [PATCH 362/487] IGNITE-3748 Data rebalancing of large cache can hang out. --- .../dht/preloader/GridDhtPreloader.java | 18 +++++++++++- .../ignite/internal/util/GridLogThrottle.java | 28 +++++++++++++++---- 2 files changed, 39 insertions(+), 7 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java index 6ed58e7636319..0865d9f709d16 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java @@ -59,6 +59,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.GPC; +import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lang.IgnitePredicate; @@ -788,7 +789,22 @@ void remoteFuture(GridDhtForceKeysFuture fut) { GridDhtLocalPartition part = partsToEvict.poll(); if (part != null) - part.tryEvict(); + try { + part.tryEvict(); + } + catch (Throwable ex) { + if (cctx.kernalContext().isStopping()) { + LT.warn(log, ex, "Partition eviction failed (current node is stopping).", + false, + true); + + partsToEvict.clear(); + + return true; + } + else + LT.error(log, ex, "Partition eviction failed, this can cause grid hang."); + } } finally { if (!partsToEvict.isEmptyx()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java index ce6783ade4c4e..c8ba86502746b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java @@ -81,12 +81,12 @@ public static void error(@Nullable IgniteLogger log, @Nullable Throwable e, Stri * @param log Logger. * @param e Error (optional). * @param msg Message. - * @param byMessage Errors group by message, not by tuple(error, msg). + * @param byMsg Errors group by message, not by tuple(error, msg). */ - public static void error(@Nullable IgniteLogger log, @Nullable Throwable e, String msg, boolean byMessage) { + public static void error(@Nullable IgniteLogger log, @Nullable Throwable e, String msg, boolean byMsg) { assert !F.isEmpty(msg); - log(log, e, msg, null, LogLevel.ERROR, false, byMessage); + log(log, e, msg, null, LogLevel.ERROR, false, byMsg); } /** @@ -102,6 +102,22 @@ public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, Strin log(log, e, msg, null, LogLevel.WARN, false, false); } + /** + * Logs warning if needed. + * + * @param log Logger. + * @param e Error (optional). + * @param msg Message. + * @param quite Print warning anyway. + * @param byMsg Errors group by message, not by tuple(error, msg). + */ + public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, String msg, boolean quite, boolean byMsg) { + assert !F.isEmpty(msg); + + log(log, e, msg, null, LogLevel.WARN, quite, byMsg); + } + + /** * Logs warning if needed. * @@ -168,15 +184,15 @@ public static void clear() { * @param longMsg Long message (or just message). * @param shortMsg Short message for quite logging. * @param level Level where messages should appear. - * @param byMessage Errors group by message, not by tuple(error, msg). + * @param byMsg Errors group by message, not by tuple(error, msg). */ @SuppressWarnings({"RedundantTypeArguments"}) private static void log(@Nullable IgniteLogger log, @Nullable Throwable e, String longMsg, @Nullable String shortMsg, - LogLevel level, boolean quiet, boolean byMessage) { + LogLevel level, boolean quiet, boolean byMsg) { assert !F.isEmpty(longMsg); IgniteBiTuple, String> tup = - e != null && !byMessage ? F., String>t(e.getClass(), e.getMessage()) : + e != null && !byMsg ? F., String>t(e.getClass(), e.getMessage()) : F., String>t(null, longMsg); while (true) { From bc695f8e3306c6d74d4fe53d9a98adedd43ad8f0 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Tue, 22 Nov 2016 12:05:15 +0300 Subject: [PATCH 363/487] IGNITE-4227: ODBC: Implemented SQLError. This closes #1237. --- .../cpp/odbc-test/src/api_robustness_test.cpp | 45 ++++++++++++++ .../platforms/cpp/odbc/include/ignite/odbc.h | 12 +++- .../ignite/odbc/diagnostic/diagnosable.h | 7 +++ .../odbc/diagnostic/diagnosable_adapter.h | 10 ++++ .../odbc/diagnostic/diagnostic_record.h | 19 ++++++ .../diagnostic/diagnostic_record_storage.h | 16 +++++ .../odbc/os/win/src/system/socket_client.cpp | 4 +- .../odbc/src/diagnostic/diagnostic_record.cpp | 16 ++++- .../diagnostic/diagnostic_record_storage.cpp | 18 ++++++ .../platforms/cpp/odbc/src/entry_points.cpp | 26 ++++---- modules/platforms/cpp/odbc/src/odbc.cpp | 59 +++++++++++++++++++ 11 files changed, 214 insertions(+), 18 deletions(-) diff --git a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp index fbd5f12532c28..13a5ea6aeb9c5 100644 --- a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp @@ -1066,4 +1066,49 @@ BOOST_AUTO_TEST_CASE(TestFetchScrollFirst) CheckFetchScrollUnsupportedOrientation(SQL_FETCH_FIRST); } +BOOST_AUTO_TEST_CASE(TestSQLError) +{ + // There are no checks because we do not really care what is the result of these + // calls as long as they do not cause segmentation fault. + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache"); + + SQLCHAR state[6] = { 0 }; + SQLINTEGER nativeCode = 0; + SQLCHAR message[ODBC_BUFFER_SIZE] = { 0 }; + SQLSMALLINT messageLen = 0; + + // Everything is ok. + SQLRETURN ret = SQLError(env, 0, 0, state, &nativeCode, message, sizeof(message), &messageLen); + + if (ret != SQL_SUCCESS && ret != SQL_NO_DATA) + BOOST_FAIL("Unexpected error"); + + ret = SQLError(0, dbc, 0, state, &nativeCode, message, sizeof(message), &messageLen); + + if (ret != SQL_SUCCESS && ret != SQL_NO_DATA) + BOOST_FAIL("Unexpected error"); + + ret = SQLError(0, 0, stmt, state, &nativeCode, message, sizeof(message), &messageLen); + + if (ret != SQL_SUCCESS && ret != SQL_NO_DATA) + BOOST_FAIL("Unexpected error"); + + SQLError(0, 0, 0, state, &nativeCode, message, sizeof(message), &messageLen); + + SQLError(0, 0, stmt, 0, &nativeCode, message, sizeof(message), &messageLen); + + SQLError(0, 0, stmt, state, 0, message, sizeof(message), &messageLen); + + SQLError(0, 0, stmt, state, &nativeCode, 0, sizeof(message), &messageLen); + + SQLError(0, 0, stmt, state, &nativeCode, message, 0, &messageLen); + + SQLError(0, 0, stmt, state, &nativeCode, message, sizeof(message), 0); + + SQLError(0, 0, stmt, 0, 0, 0, 0, 0); + + SQLError(0, 0, 0, 0, 0, 0, 0, 0); +} + BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc.h b/modules/platforms/cpp/odbc/include/ignite/odbc.h index ec0861c565e19..345cdb81b223c 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc.h @@ -255,6 +255,16 @@ namespace ignite SQLRETURN SQLParamData(SQLHSTMT stmt, SQLPOINTER* value); SQLRETURN SQLPutData(SQLHSTMT stmt, SQLPOINTER data, SQLLEN strLengthOrIndicator); + + SQLRETURN SQLError(SQLHENV env, + SQLHDBC conn, + SQLHSTMT stmt, + SQLCHAR* state, + SQLINTEGER* error, + SQLCHAR* msgBuf, + SQLSMALLINT msgBufLen, + SQLSMALLINT* msgResLen); + } // namespace ignite -#endif //_IGNITE_ODBC +#endif //_IGNITE_ODBC \ No newline at end of file diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h index 909fe01d4056a..6937fcc6f3196 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable.h @@ -47,6 +47,13 @@ namespace ignite */ virtual const diagnostic::DiagnosticRecordStorage& GetDiagnosticRecords() const = 0; + /** + * Get diagnostic record. + * + * @return Diagnostic record. + */ + virtual diagnostic::DiagnosticRecordStorage& GetDiagnosticRecords() = 0; + /** * Add new status record. * diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable_adapter.h b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable_adapter.h index 63d26ca7eb9f4..548eecda92074 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable_adapter.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnosable_adapter.h @@ -61,6 +61,16 @@ namespace ignite return diagnosticRecords; } + /** + * Get diagnostic record. + * + * @return Diagnostic record. + */ + virtual diagnostic::DiagnosticRecordStorage& GetDiagnosticRecords() + { + return diagnosticRecords; + } + /** * Add new status record. * diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h index 670e0aa535332..80d5090917579 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h @@ -127,6 +127,19 @@ namespace ignite */ int32_t GetColumnNumber() const; + /** + * Check if the record was retrieved with the SQLError previously. + * + * return True if the record was retrieved with the SQLError + * previously. + */ + bool IsRetrieved() const; + + /** + * Mark record as retrieved with the SQLError. + */ + void MarkRetrieved(); + private: /** SQL state diagnostic code. */ SqlState sqlState; @@ -157,6 +170,12 @@ namespace ignite * result set or the parameter number in the set of parameters. */ int32_t columnNum; + + /** + * Flag that shows if the record was retrieved with the + * SQLError previously. + */ + bool retrieved; }; } } diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record_storage.h b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record_storage.h index 4cc3576f686be..b45bb7dae6440 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record_storage.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record_storage.h @@ -138,6 +138,22 @@ namespace ignite */ const DiagnosticRecord& GetStatusRecord(int32_t idx) const; + /** + * Get specified status record. + * + * @param idx Status record index. + * @return Status record instance reference. + */ + DiagnosticRecord& GetStatusRecord(int32_t idx); + + /** + * Get last non-retrieved status record index. + * + * @return Index of the last non-retrieved status record or zero + * if nothing was found. + */ + int32_t GetLastNonRetrieved() const; + /** * Check if the record is in the success state. * diff --git a/modules/platforms/cpp/odbc/os/win/src/system/socket_client.cpp b/modules/platforms/cpp/odbc/os/win/src/system/socket_client.cpp index bc4cdc067903d..e248323ba5c53 100644 --- a/modules/platforms/cpp/odbc/os/win/src/system/socket_client.cpp +++ b/modules/platforms/cpp/odbc/os/win/src/system/socket_client.cpp @@ -83,8 +83,8 @@ namespace ignite // Attempt to connect to an address until one succeeds for (addrinfo *it = result; it != NULL; it = it->ai_next) { - LOG_MSG("Addr: %u.%u.%u.%u\n", it->ai_addr->sa_data[2], it->ai_addr->sa_data[3], - it->ai_addr->sa_data[4], it->ai_addr->sa_data[5]); + LOG_MSG("Addr: %u.%u.%u.%u\n", it->ai_addr->sa_data[2] & 0xFF, it->ai_addr->sa_data[3] & 0xFF, + it->ai_addr->sa_data[4] & 0xFF, it->ai_addr->sa_data[5] & 0xFF); // Create a SOCKET for connecting to server socketHandle = socket(it->ai_family, it->ai_socktype, it->ai_protocol); diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp index 1b654d26480a3..215d77f5c249f 100644 --- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp +++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp @@ -89,7 +89,8 @@ namespace ignite connectionName(), serverName(), rowNum(0), - columnNum(0) + columnNum(0), + retrieved(false) { // No-op. } @@ -102,7 +103,8 @@ namespace ignite connectionName(connectionName), serverName(serverName), rowNum(rowNum), - columnNum(columnNum) + columnNum(columnNum), + retrieved(false) { // No-op. } @@ -260,6 +262,16 @@ namespace ignite { return columnNum; } + + bool DiagnosticRecord::IsRetrieved() const + { + return retrieved; + } + + void DiagnosticRecord::MarkRetrieved() + { + retrieved = true; + } } } } diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp index 99ef292534447..c075567520b59 100644 --- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp +++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp @@ -102,6 +102,24 @@ namespace ignite return statusRecords[idx - 1]; } + DiagnosticRecord& DiagnosticRecordStorage::GetStatusRecord(int32_t idx) + { + return statusRecords[idx - 1]; + } + + int32_t DiagnosticRecordStorage::GetLastNonRetrieved() const + { + for (size_t i = 0; i < statusRecords.size(); ++i) + { + const DiagnosticRecord& record = statusRecords[i]; + + if (!record.IsRetrieved()) + return static_cast(i + 1); + } + + return 0; + } + bool DiagnosticRecordStorage::IsSuccessful() const { return result == SQL_RESULT_SUCCESS || diff --git a/modules/platforms/cpp/odbc/src/entry_points.cpp b/modules/platforms/cpp/odbc/src/entry_points.cpp index 08016cc678a75..a85b3cf6ba2f8 100644 --- a/modules/platforms/cpp/odbc/src/entry_points.cpp +++ b/modules/platforms/cpp/odbc/src/entry_points.cpp @@ -412,6 +412,19 @@ SQLRETURN SQL_API SQLPutData(SQLHSTMT stmt, return ignite::SQLPutData(stmt, data, strLengthOrIndicator); } +SQLRETURN SQL_API SQLError(SQLHENV env, + SQLHDBC conn, + SQLHSTMT stmt, + SQLCHAR* state, + SQLINTEGER* error, + SQLCHAR* msgBuf, + SQLSMALLINT msgBufLen, + SQLSMALLINT* msgResLen) +{ + return ignite::SQLError(env, conn, stmt, state, + error, msgBuf, msgBufLen, msgResLen); +} + // // ==== Not implemented ==== // @@ -434,19 +447,6 @@ SQLRETURN SQL_API SQLColAttributes(SQLHSTMT stmt, return SQL_SUCCESS; } -SQLRETURN SQL_API SQLError(SQLHENV env, - SQLHDBC conn, - SQLHSTMT stmt, - SQLCHAR* state, - SQLINTEGER* error, - SQLCHAR* msgBuf, - SQLSMALLINT msgBufLen, - SQLSMALLINT* msgResLen) -{ - LOG_MSG("SQLError called\n"); - return SQL_ERROR; -} - SQLRETURN SQL_API SQLGetCursorName(SQLHSTMT stmt, SQLCHAR* nameBuf, SQLSMALLINT nameBufLen, diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp index 79036eb8b2bc3..684ed0868fe8b 100644 --- a/modules/platforms/cpp/odbc/src/odbc.cpp +++ b/modules/platforms/cpp/odbc/src/odbc.cpp @@ -1255,4 +1255,63 @@ namespace ignite return statement->GetDiagnosticRecords().GetReturnCode(); } + SQLRETURN SQLError(SQLHENV env, + SQLHDBC conn, + SQLHSTMT stmt, + SQLCHAR* state, + SQLINTEGER* error, + SQLCHAR* msgBuf, + SQLSMALLINT msgBufLen, + SQLSMALLINT* msgResLen) + { + using namespace ignite::utility; + using namespace ignite::odbc; + using namespace ignite::odbc::diagnostic; + using namespace ignite::odbc::type_traits; + + using ignite::odbc::app::ApplicationDataBuffer; + + LOG_MSG("SQLError called\n"); + + SQLHANDLE handle = 0; + + if (env != 0) + handle = static_cast(env); + else if (conn != 0) + handle = static_cast(conn); + else if (stmt != 0) + handle = static_cast(stmt); + else + return SQL_INVALID_HANDLE; + + Diagnosable *diag = reinterpret_cast(handle); + + DiagnosticRecordStorage& records = diag->GetDiagnosticRecords(); + + int32_t recNum = records.GetLastNonRetrieved(); + + if (recNum < 1 || recNum > records.GetStatusRecordsNumber()) + return SQL_NO_DATA; + + DiagnosticRecord& record = records.GetStatusRecord(recNum); + + record.MarkRetrieved(); + + if (state) + CopyStringToBuffer(record.GetSqlState(), reinterpret_cast(state), 6); + + if (error) + *error = 0; + + SqlLen outResLen; + ApplicationDataBuffer outBuffer(IGNITE_ODBC_C_TYPE_CHAR, msgBuf, msgBufLen, &outResLen); + + outBuffer.PutString(record.GetMessageText()); + + if (msgResLen) + *msgResLen = static_cast(outResLen); + + return SQL_SUCCESS; + } + } // namespace ignite; From 861fab9d0598ca2f06c4a6f293bf2866af31967c Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Tue, 22 Nov 2016 14:52:03 +0500 Subject: [PATCH 364/487] IGNITE-4239: add GridInternal annotaion for tasks instead of jobs. This closes #1250. --- .../processors/cache/GridCacheAdapter.java | 11 +- .../PublicThreadpoolStarvationTest.java | 129 ++++++++++++++++++ .../IgniteComputeGridTestSuite.java | 2 + 3 files changed, 135 insertions(+), 7 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/compute/PublicThreadpoolStarvationTest.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 55400abb72021..2e24e67cae195 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 @@ -5443,7 +5443,6 @@ private IgniteInternalFuture runOp(IgniteTxLocalAdapter tx, /** * Global clear all. */ - @GridInternal private static class GlobalClearAllJob extends TopologyVersionAwareJob { /** */ private static final long serialVersionUID = 0L; @@ -5482,7 +5481,6 @@ protected boolean clearNearCache() { /** * Global clear keys. */ - @GridInternal private static class GlobalClearKeySetJob extends TopologyVersionAwareJob { /** */ private static final long serialVersionUID = 0L; @@ -5527,7 +5525,6 @@ protected boolean clearNearCache() { /** * Global clear all for near cache. */ - @GridInternal private static class GlobalClearAllNearJob extends GlobalClearAllJob { /** */ private static final long serialVersionUID = 0L; @@ -5558,7 +5555,6 @@ private GlobalClearAllNearJob(String cacheName, AffinityTopologyVersion topVer) /** * Global clear keys for near cache. */ - @GridInternal private static class GlobalClearKeySetNearJob extends GlobalClearKeySetJob { /** */ private static final long serialVersionUID = 0L; @@ -5590,7 +5586,6 @@ protected boolean clearNearCache() { /** * Internal callable for partition size calculation. */ - @GridInternal private static class PartitionSizeLongJob extends TopologyVersionAwareJob { /** */ private static final long serialVersionUID = 0L; @@ -5636,7 +5631,6 @@ public String toString() { /** * Internal callable for global size calculation. */ - @GridInternal private static class SizeJob extends TopologyVersionAwareJob { /** */ private static final long serialVersionUID = 0L; @@ -5677,7 +5671,6 @@ public String toString() { /** * Internal callable for global size calculation. */ - @GridInternal private static class SizeLongJob extends TopologyVersionAwareJob { /** */ private static final long serialVersionUID = 0L; @@ -6523,6 +6516,7 @@ private boolean waitAffinityReadyFuture() { /** * Size task. */ + @GridInternal private static class SizeTask extends ComputeTaskAdapter { /** */ private static final long serialVersionUID = 0L; @@ -6588,6 +6582,7 @@ public SizeTask(String cacheName, AffinityTopologyVersion topVer, CachePeekMode[ /** * Size task. */ + @GridInternal private static class SizeLongTask extends ComputeTaskAdapter { /** */ private static final long serialVersionUID = 0L; @@ -6653,6 +6648,7 @@ private SizeLongTask(String cacheName, AffinityTopologyVersion topVer, CachePeek /** * Partition Size Long task. */ + @GridInternal private static class PartitionSizeLongTask extends ComputeTaskAdapter { /** */ private static final long serialVersionUID = 0L; @@ -6737,6 +6733,7 @@ private PartitionSizeLongTask( /** * Clear task. */ + @GridInternal private static class ClearTask extends ComputeTaskAdapter { /** */ private static final long serialVersionUID = 0L; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/compute/PublicThreadpoolStarvationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/compute/PublicThreadpoolStarvationTest.java new file mode 100644 index 0000000000000..e587310a0c146 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/compute/PublicThreadpoolStarvationTest.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.compute; + +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.binary.BinaryMarshaller; +import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; + +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; + +/** + * Test to validate https://issues.apache.org/jira/browse/IGNITE-4239 + * Jobs hang when a lot of jobs calculate cache. + */ +public class PublicThreadpoolStarvationTest extends GridCacheAbstractSelfTest { + /** Cache size. */ + private static final int CACHE_SIZE = 10; + + /** Cache size. */ + private static final String CACHE_NAME = "test"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setPublicThreadPoolSize(1); + + ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1); + + cfg.setMarshaller(new BinaryMarshaller()); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected Class[] indexedTypes() { + return new Class[] { + Integer.class, String.class, + }; + } + + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return ATOMIC; + } + + /** {@inheritDoc} */ + @Override protected int gridCount() { + return 2; + } + + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return PARTITIONED; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + info("Fill caches begin..."); + + fillCaches(); + + info("Caches are filled."); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + grid(0).destroyCache(CACHE_NAME); + + super.afterTestsStopped(); + } + + /** + * @throws Exception If failed. + */ + private void fillCaches() throws Exception { + grid(0).createCache(CACHE_NAME); + + try ( + IgniteDataStreamer streamer = + grid(0).dataStreamer(CACHE_NAME)) { + + for (int i = 0; i < CACHE_SIZE; ++i) + streamer.addData(i, "Data " + i); + } + + awaitPartitionMapExchange(); + } + + /** + * @throws Exception If failed. + */ + public void testCacheSizeOnPublicThreadpoolStarvation() throws Exception { + grid(0).compute().run(new IgniteRunnable() { + @Override public void run() { + try { + Thread.sleep(500); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + grid(0).cache(CACHE_NAME).size(); + } + }); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java index a1a75f8ada2a0..8a501fdb801d6 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java @@ -71,6 +71,7 @@ import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManagerSelfTest; import org.apache.ignite.internal.managers.checkpoint.GridCheckpointTaskSelfTest; import org.apache.ignite.internal.managers.communication.GridCommunicationManagerListenersSelfTest; +import org.apache.ignite.internal.processors.compute.PublicThreadpoolStarvationTest; import org.apache.ignite.p2p.GridMultinodeRedeployContinuousModeSelfTest; import org.apache.ignite.p2p.GridMultinodeRedeployIsolatedModeSelfTest; import org.apache.ignite.p2p.GridMultinodeRedeployPrivateModeSelfTest; @@ -150,6 +151,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridTaskFailoverAffinityRunTest.class); suite.addTestSuite(TaskNodeRestartTest.class); suite.addTestSuite(IgniteRoundRobinErrorAfterClientReconnectTest.class); + suite.addTestSuite(PublicThreadpoolStarvationTest.class); return suite; } From ba99df1554fbd1de2b2367b6ce011a024cd199bd Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Tue, 22 Nov 2016 15:07:20 +0500 Subject: [PATCH 365/487] IGNITE-4239: test cleanup --- .../processors/compute/PublicThreadpoolStarvationTest.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/compute/PublicThreadpoolStarvationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/compute/PublicThreadpoolStarvationTest.java index e587310a0c146..dec32c6a50f29 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/compute/PublicThreadpoolStarvationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/compute/PublicThreadpoolStarvationTest.java @@ -116,12 +116,6 @@ private void fillCaches() throws Exception { public void testCacheSizeOnPublicThreadpoolStarvation() throws Exception { grid(0).compute().run(new IgniteRunnable() { @Override public void run() { - try { - Thread.sleep(500); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } grid(0).cache(CACHE_NAME).size(); } }); From c34d27423a0c45c61341c1fcb3f56727fb91498f Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Tue, 22 Nov 2016 14:13:28 +0300 Subject: [PATCH 366/487] IGNITE-4100: Fix for DEVNOTES paths. --- modules/platforms/cpp/DEVNOTES.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/modules/platforms/cpp/DEVNOTES.txt b/modules/platforms/cpp/DEVNOTES.txt index 5d0b2eb5c8800..473af23633ac4 100644 --- a/modules/platforms/cpp/DEVNOTES.txt +++ b/modules/platforms/cpp/DEVNOTES.txt @@ -5,7 +5,8 @@ stand-alone node binary. To build examples you need to build and install core Ap Ignite library then refer to $IGNITE_HOME/platforms/cpp/example/README.txt for further instructions. -For details on ODBC driver installation and usage please refer to odbc/README.txt. +For details on ODBC driver installation and usage please refer to +$IGNITE_HOME/platforms/cpp/odbc/README.txt. Building on Linux With Autotools ---------------------------------- From 9d82f2ca06fa6069c1976cc75814874256b24f8c Mon Sep 17 00:00:00 2001 From: devozerov Date: Tue, 22 Nov 2016 15:05:29 +0300 Subject: [PATCH 367/487] IGNITE-4259: Fixed a problem with geospatial indexes and BinaryMarshaller. --- .../binary/BinaryClassDescriptor.java | 6 ++-- .../ignite/internal/binary/BinaryContext.java | 4 ++- .../h2/GridBinaryH2IndexingGeoSelfTest.java | 35 +++++++++++++++++++ .../query/h2/GridH2IndexingGeoSelfTest.java | 2 +- .../GeoSpatialIndexingTestSuite.java | 2 ++ 5 files changed, 45 insertions(+), 4 deletions(-) create mode 100644 modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/GridBinaryH2IndexingGeoSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java index d05ce71815666..b121337287d6d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java @@ -41,6 +41,7 @@ import org.apache.ignite.binary.BinarySerializer; import org.apache.ignite.binary.Binarylizable; import org.apache.ignite.internal.processors.cache.CacheObjectImpl; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -156,7 +157,7 @@ public class BinaryClassDescriptor { initialSerializer = serializer; // If serializer is not defined at this point, then we have to use OptimizedMarshaller. - useOptMarshaller = serializer == null; + useOptMarshaller = serializer == null || GridQueryProcessor.isGeometryClass(cls); // Reset reflective serializer so that we rely on existing reflection-based serialization. if (serializer instanceof BinaryReflectiveSerializer) @@ -189,7 +190,8 @@ else if (useOptMarshaller) mode = serializer != null ? BinaryWriteMode.BINARY : BinaryUtils.mode(cls); } - if (useOptMarshaller && userType && !U.isIgnite(cls) && !U.isJdk(cls)) { + if (useOptMarshaller && userType && !U.isIgnite(cls) && !U.isJdk(cls) && + !GridQueryProcessor.isGeometryClass(cls)) { U.warn(ctx.log(), "Class \"" + cls.getName() + "\" cannot be serialized using " + BinaryMarshaller.class.getSimpleName() + " because it either implements Externalizable interface " + "or have writeObject/readObject methods. " + OptimizedMarshaller.class.getSimpleName() + " will be " + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java index a1ea3e98956b8..cc18318b04e61 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java @@ -78,6 +78,7 @@ import org.apache.ignite.internal.processors.platform.PlatformJavaObjectFactoryProxy; import org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionData; import org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionLockResult; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.lang.GridMapEntry; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; @@ -357,7 +358,8 @@ public boolean mustDeserialize(Class cls) { if (BinaryUtils.wrapTrees() && (cls == TreeMap.class || cls == TreeSet.class)) return false; - return marshCtx.isSystemType(cls.getName()) || serializerForClass(cls) == null; + return marshCtx.isSystemType(cls.getName()) || serializerForClass(cls) == null || + GridQueryProcessor.isGeometryClass(cls); } else return desc.useOptimizedMarshaller(); diff --git a/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/GridBinaryH2IndexingGeoSelfTest.java b/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/GridBinaryH2IndexingGeoSelfTest.java new file mode 100644 index 0000000000000..01071f9162e3c --- /dev/null +++ b/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/GridBinaryH2IndexingGeoSelfTest.java @@ -0,0 +1,35 @@ +/* + * 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.query.h2; + +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.binary.BinaryMarshaller; + +/** + * Geo-indexing test for binary mode. + */ +public class GridBinaryH2IndexingGeoSelfTest extends GridH2IndexingGeoSelfTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setMarshaller(new BinaryMarshaller()); + + return cfg; + } +} \ No newline at end of file diff --git a/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexingGeoSelfTest.java b/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexingGeoSelfTest.java index b73a5c0b3a032..28430769329ea 100644 --- a/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexingGeoSelfTest.java +++ b/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexingGeoSelfTest.java @@ -41,7 +41,7 @@ import org.apache.ignite.testframework.GridTestUtils; /** - * + * Geo-indexing test. */ public class GridH2IndexingGeoSelfTest extends GridCacheAbstractSelfTest { /** */ diff --git a/modules/geospatial/src/test/java/org/apache/ignite/testsuites/GeoSpatialIndexingTestSuite.java b/modules/geospatial/src/test/java/org/apache/ignite/testsuites/GeoSpatialIndexingTestSuite.java index d3b8d012cc2d3..1773894f1fb03 100644 --- a/modules/geospatial/src/test/java/org/apache/ignite/testsuites/GeoSpatialIndexingTestSuite.java +++ b/modules/geospatial/src/test/java/org/apache/ignite/testsuites/GeoSpatialIndexingTestSuite.java @@ -18,6 +18,7 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.internal.processors.query.h2.GridBinaryH2IndexingGeoSelfTest; import org.apache.ignite.internal.processors.query.h2.GridH2IndexingGeoSelfTest; /** @@ -33,6 +34,7 @@ public static TestSuite suite() throws Exception { // Geo. suite.addTestSuite(GridH2IndexingGeoSelfTest.class); + suite.addTestSuite(GridBinaryH2IndexingGeoSelfTest.class); return suite; } From b038730ee56a662f73e02bbec83eb1712180fa82 Mon Sep 17 00:00:00 2001 From: isapego Date: Wed, 23 Nov 2016 12:05:54 +0300 Subject: [PATCH 368/487] IGNITE-4249: ODBC: Fixed performance issue caused by ineddicient IO handling on CPP side. This closes #1254. --- .../processors/odbc/OdbcRequestHandler.java | 32 +++++++++++++------ .../src/impl/binary/binary_reader_impl.cpp | 3 +- modules/platforms/cpp/odbc/src/connection.cpp | 21 ++++++------ 3 files changed, 34 insertions(+), 22 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java index 4a31be36c87ef..eef99454db212 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java @@ -192,9 +192,7 @@ private OdbcResponse executeQuery(long reqId, OdbcQueryExecuteRequest req) { QueryCursor qryCur = cache.query(qry); - Iterator iter = qryCur.iterator(); - - qryCursors.put(qryId, new IgniteBiTuple<>(qryCur, iter)); + qryCursors.put(qryId, new IgniteBiTuple(qryCur, null)); List fieldsMeta = ((QueryCursorImpl) qryCur).fieldsMeta(); @@ -220,11 +218,15 @@ private OdbcResponse executeQuery(long reqId, OdbcQueryExecuteRequest req) { */ private OdbcResponse closeQuery(long reqId, OdbcQueryCloseRequest req) { try { - QueryCursor cur = qryCursors.get(req.queryId()).get1(); + IgniteBiTuple tuple = qryCursors.get(req.queryId()); - if (cur == null) + if (tuple == null) return new OdbcResponse(OdbcResponse.STATUS_FAILED, "Failed to find query with ID: " + req.queryId()); + QueryCursor cur = tuple.get1(); + + assert(cur != null); + cur.close(); qryCursors.remove(req.queryId()); @@ -251,17 +253,27 @@ private OdbcResponse closeQuery(long reqId, OdbcQueryCloseRequest req) { */ private OdbcResponse fetchQuery(long reqId, OdbcQueryFetchRequest req) { try { - Iterator cur = qryCursors.get(req.queryId()).get2(); + IgniteBiTuple tuple = qryCursors.get(req.queryId()); - if (cur == null) + if (tuple == null) return new OdbcResponse(OdbcResponse.STATUS_FAILED, "Failed to find query with ID: " + req.queryId()); + Iterator iter = tuple.get2(); + + if (iter == null) { + QueryCursor cur = tuple.get1(); + + iter = cur.iterator(); + + tuple.put(cur, iter); + } + List items = new ArrayList<>(); - for (int i = 0; i < req.pageSize() && cur.hasNext(); ++i) - items.add(cur.next()); + for (int i = 0; i < req.pageSize() && iter.hasNext(); ++i) + items.add(iter.next()); - OdbcQueryFetchResult res = new OdbcQueryFetchResult(req.queryId(), items, !cur.hasNext()); + OdbcQueryFetchResult res = new OdbcQueryFetchResult(req.queryId(), items, !iter.hasNext()); return new OdbcResponse(res); } diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp index c3f4fcc6dc59c..fb75ba58338a5 100644 --- a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp +++ b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp @@ -459,8 +459,7 @@ namespace ignite int32_t realLen = stream->ReadInt32(); if (res && len >= realLen) { - for (int i = 0; i < realLen; i++) - *(res + i) = static_cast(stream->ReadInt8()); + stream->ReadInt8Array(reinterpret_cast(res), realLen); if (len > realLen) *(res + realLen) = 0; // Set NULL terminator if possible. diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp index b8ed9fe64b022..e8db37688b661 100644 --- a/modules/platforms/cpp/odbc/src/connection.cpp +++ b/modules/platforms/cpp/odbc/src/connection.cpp @@ -19,6 +19,8 @@ #include +#include + #include "ignite/odbc/utility.h" #include "ignite/odbc/statement.h" #include "ignite/odbc/connection.h" @@ -178,26 +180,25 @@ namespace ignite if (!connected) IGNITE_ERROR_1(IgniteError::IGNITE_ERR_ILLEGAL_STATE, "Connection is not established"); - OdbcProtocolHeader hdr; + common::FixedSizeArray msg(len + sizeof(OdbcProtocolHeader)); - hdr.len = static_cast(len); + OdbcProtocolHeader *hdr = reinterpret_cast(msg.GetData()); - size_t sent = SendAll(reinterpret_cast(&hdr), sizeof(hdr)); + hdr->len = static_cast(len); - if (sent != sizeof(hdr)) - IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not send message header"); + memcpy(msg.GetData() + sizeof(OdbcProtocolHeader), data, len); - sent = SendAll(data, len); + size_t sent = SendAll(msg.GetData(), msg.GetSize()); - if (sent != len) - IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not send message body"); + if (sent != len + sizeof(OdbcProtocolHeader)) + IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not send message"); } size_t Connection::SendAll(const int8_t* data, size_t len) { int sent = 0; - while (sent != len) + while (sent != static_cast(len)) { int res = socket.Send(data + sent, len - sent); @@ -221,7 +222,7 @@ namespace ignite OdbcProtocolHeader hdr; - size_t received = ReceiveAll(reinterpret_cast(&hdr), sizeof(hdr)); + int64_t received = ReceiveAll(reinterpret_cast(&hdr), sizeof(hdr)); if (received != sizeof(hdr)) IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not receive message header"); From 7a47a0185d308cd3a58c7bfcb4d1cd548bff5b87 Mon Sep 17 00:00:00 2001 From: devozerov Date: Thu, 24 Nov 2016 11:14:08 +0300 Subject: [PATCH 369/487] IGNITE-4270: Allow GridUnsafe.UNALIGNED flag override. --- .../java/org/apache/ignite/IgniteSystemProperties.java | 7 +++++++ .../java/org/apache/ignite/internal/util/GridUnsafe.java | 9 ++++++++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 22cb9a6fe079f..043c95ade0fbc 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -486,6 +486,13 @@ public final class IgniteSystemProperties { @Deprecated public static final String IGNITE_BINARY_SORT_OBJECT_FIELDS = "IGNITE_BINARY_SORT_OBJECT_FIELDS"; + /** + * Whether Ignite can access unaligned memory addresses. + *

          + * Defaults to {@code} false, meaning that unaligned access will be performed only on x86 architecture. + */ + public static final String IGNITE_UNALIGNED_MEMORY_ACCESS = "IGNITE_UNALIGNED_MEMORY_ACCESS"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java index 1f7a53cb6b0a8..1c492efbbf69e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java @@ -22,6 +22,8 @@ import java.security.AccessController; import java.security.PrivilegedActionException; import java.security.PrivilegedExceptionAction; + +import org.apache.ignite.IgniteSystemProperties; import sun.misc.Unsafe; /** @@ -1204,7 +1206,12 @@ public static void putObjectVolatile(Object obj, long off, Object val) { private static boolean unaligned() { String arch = System.getProperty("os.arch"); - return arch.equals("i386") || arch.equals("x86") || arch.equals("amd64") || arch.equals("x86_64"); + boolean res = arch.equals("i386") || arch.equals("x86") || arch.equals("amd64") || arch.equals("x86_64"); + + if (!res) + res = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_UNALIGNED_MEMORY_ACCESS, false); + + return res; } /** From bf330251734018467fa3291fccf0414c9da7dd1b Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Thu, 24 Nov 2016 17:08:08 +0700 Subject: [PATCH 370/487] Web console beta-6. --- modules/web-console/backend/app/agent.js | 32 ++- modules/web-console/backend/app/browser.js | 13 + modules/web-console/backend/app/mongo.js | 6 + .../backend/config/settings.json.sample | 7 - modules/web-console/backend/index.js | 6 +- .../web-console/backend/middlewares/host.js | 7 +- modules/web-console/backend/routes/agent.js | 4 +- .../web-console/backend/services/agents.js | 2 +- .../web-console/backend/services/caches.js | 2 + .../docker/compose/backend/.dockerignore | 2 + .../docker/compose/backend/Dockerfile | 6 +- .../docker/compose/backend/build.sh | 2 +- .../docker/compose/docker-compose.yml | 5 - .../docker/compose/frontend/DockerfileBuild | 4 +- .../docker/compose/frontend/build.sh | 2 +- .../compose/frontend/nginx/web-console.conf | 9 + .../docker/standalone/.dockerignore | 5 + .../web-console/docker/standalone/Dockerfile | 10 +- .../web-console/docker/standalone/build.sh | 2 +- .../docker/standalone/docker-compose.yml | 7 +- .../docker/standalone/nginx/web-console.conf | 9 + .../frontend/app/data/pom-dependencies.json | 2 +- .../directives/ui-ace-pojos/ui-ace-pojos.jade | 2 +- .../frontend/app/filters/duration.filter.js | 3 + .../jade/form/form-field-checkbox.jade | 2 +- .../jade/form/form-field-datalist.jade | 2 +- .../jade/form/form-field-dropdown.jade | 2 +- .../helpers/jade/form/form-field-number.jade | 2 +- .../jade/form/form-field-password.jade | 2 +- .../helpers/jade/form/form-field-text.jade | 2 +- .../frontend/app/helpers/jade/mixins.jade | 24 +- .../modules/configuration/generator/Beans.js | 6 + .../generator/ConfigurationGenerator.js | 165 +++++++------ .../generator/JavaTransformer.service.js | 15 +- .../generator/Properties.service.js | 19 ++ .../generator/SpringTransformer.service.js | 22 +- .../generator/defaults/cache.provider.js | 8 + .../generator/defaults/cluster.provider.js | 2 +- .../modules/form/field/tooltip.directive.js | 2 +- .../modules/form/group/tooltip.directive.js | 2 +- .../app/modules/form/panel/field.directive.js | 4 +- .../app/modules/sql/scan-filter-input.jade | 2 +- .../app/modules/sql/sql.controller.js | 6 +- .../caches/client-near-cache.jade | 2 +- .../configuration/caches/concurrency.jade | 2 +- .../states/configuration/caches/general.jade | 2 +- .../states/configuration/caches/memory.jade | 2 +- .../caches/near-cache-client.jade | 2 +- .../caches/near-cache-server.jade | 2 +- .../configuration/caches/node-filter.jade | 2 +- .../states/configuration/caches/query.jade | 2 +- .../configuration/caches/rebalance.jade | 2 +- .../configuration/caches/statistics.jade | 2 +- .../states/configuration/caches/store.jade | 20 +- .../states/configuration/clusters/atomic.jade | 2 +- .../configuration/clusters/attributes.jade | 2 +- .../states/configuration/clusters/binary.jade | 2 +- .../configuration/clusters/cache-key-cfg.jade | 2 +- .../configuration/clusters/checkpoint.jade | 8 +- .../configuration/clusters/checkpoint/fs.jade | 2 +- .../clusters/checkpoint/jdbc.jade | 41 ++-- .../configuration/clusters/checkpoint/s3.jade | 229 +++++++++--------- .../configuration/clusters/collision.jade | 12 +- .../clusters/collision/custom.jade | 2 +- .../clusters/collision/fifo-queue.jade | 2 +- .../clusters/collision/job-stealing.jade | 2 +- .../clusters/collision/priority-queue.jade | 2 +- .../configuration/clusters/communication.jade | 2 +- .../configuration/clusters/connector.jade | 2 +- .../configuration/clusters/deployment.jade | 2 +- .../configuration/clusters/discovery.jade | 2 +- .../states/configuration/clusters/events.jade | 2 +- .../configuration/clusters/failover.jade | 2 +- .../configuration/clusters/general.jade | 2 +- .../clusters/general/discovery/cloud.jade | 2 +- .../clusters/general/discovery/google.jade | 2 +- .../clusters/general/discovery/jdbc.jade | 3 +- .../clusters/general/discovery/multicast.jade | 2 +- .../clusters/general/discovery/s3.jade | 2 +- .../clusters/general/discovery/shared.jade | 2 +- .../clusters/general/discovery/vm.jade | 2 +- .../clusters/general/discovery/zookeeper.jade | 2 +- .../bounded-exponential-backoff.jade | 2 +- .../zookeeper/retrypolicy/custom.jade | 2 +- .../retrypolicy/exponential-backoff.jade | 2 +- .../zookeeper/retrypolicy/forever.jade | 2 +- .../zookeeper/retrypolicy/n-times.jade | 2 +- .../zookeeper/retrypolicy/one-time.jade | 2 +- .../zookeeper/retrypolicy/until-elapsed.jade | 2 +- .../states/configuration/clusters/igfs.jade | 2 +- .../clusters/load-balancing.jade | 2 +- .../states/configuration/clusters/logger.jade | 2 +- .../configuration/clusters/logger/custom.jade | 2 +- .../configuration/clusters/logger/log4j.jade | 2 +- .../configuration/clusters/logger/log4j2.jade | 2 +- .../configuration/clusters/marshaller.jade | 2 +- .../configuration/clusters/metrics.jade | 2 +- .../states/configuration/clusters/odbc.jade | 2 +- .../states/configuration/clusters/ssl.jade | 2 +- .../states/configuration/clusters/swap.jade | 2 +- .../states/configuration/clusters/thread.jade | 2 +- .../states/configuration/clusters/time.jade | 2 +- .../configuration/clusters/transactions.jade | 2 +- .../states/configuration/domains/general.jade | 2 +- .../states/configuration/domains/query.jade | 16 +- .../states/configuration/domains/store.jade | 12 +- .../states/configuration/igfs/dual.jade | 2 +- .../configuration/igfs/fragmentizer.jade | 2 +- .../states/configuration/igfs/general.jade | 2 +- .../states/configuration/igfs/ipc.jade | 2 +- .../states/configuration/igfs/misc.jade | 6 +- .../states/configuration/igfs/secondary.jade | 2 +- .../summary/summary.controller.js | 12 +- .../controllers/clusters-controller.js | 10 + .../controllers/domains-controller.js | 8 +- .../frontend/gulpfile.babel.js/tasks/jade.js | 4 +- .../frontend/gulpfile.babel.js/tasks/test.js | 92 ------- .../gulpfile.babel.js/webpack/common.js | 5 +- .../webpack/environments/development.js | 5 + modules/web-console/frontend/package.json | 2 +- .../stylesheets/_font-awesome-custom.scss | 18 ++ .../frontend/public/stylesheets/style.scss | 13 +- .../frontend/views/configuration/caches.jade | 22 +- .../views/configuration/clusters.jade | 50 ++-- .../views/configuration/domains-import.jade | 4 +- .../frontend/views/configuration/domains.jade | 8 +- .../frontend/views/configuration/igfs.jade | 14 +- .../frontend/views/configuration/summary.jade | 2 +- .../frontend/views/includes/header.jade | 15 +- .../frontend/views/settings/profile.jade | 2 +- .../web-console/frontend/views/sql/sql.jade | 2 +- .../views/templates/agent-download.jade | 2 +- .../views/templates/batch-confirm.jade | 2 +- .../frontend/views/templates/clone.jade | 2 +- .../frontend/views/templates/confirm.jade | 2 +- modules/web-console/web-agent/README.txt | 4 +- .../web-agent/bin/ignite-web-agent.bat | 7 +- .../web-agent/bin/ignite-web-agent.sh | 6 +- .../console/agent/AgentConfiguration.java | 5 +- .../ignite/console/agent/AgentLauncher.java | 6 +- .../ignite/console/demo/AgentClusterDemo.java | 1 + 141 files changed, 667 insertions(+), 563 deletions(-) delete mode 100644 modules/web-console/frontend/gulpfile.babel.js/tasks/test.js diff --git a/modules/web-console/backend/app/agent.js b/modules/web-console/backend/app/agent.js index 6aa9a1293e3d3..f74a3f2e4c245 100644 --- a/modules/web-console/backend/app/agent.js +++ b/modules/web-console/backend/app/agent.js @@ -295,6 +295,23 @@ module.exports.factory = function(_, fs, path, JSZip, socketio, settings, mongo) return this.executeRest(cmd); } + /** + * @param {Boolean} demo Is need run command on demo node. + * @param {Array.} nids Node ids. + * @param {Number} since Metrics since. + * @returns {Promise} + */ + queryDetailMetrics(demo, nids, since) { + const cmd = new Command(demo, 'exe') + .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask') + .addParam('p1', nids) + .addParam('p2', 'org.apache.ignite.internal.visor.cache.VisorCacheQueryDetailMetricsCollectorTask') + .addParam('p3', 'java.lang.Long') + .addParam('p4', since); + + return this.executeRest(cmd); + } + /** * @param {Boolean} demo Is need run command on demo node. * @param {String} cacheName Cache name. @@ -634,6 +651,19 @@ module.exports.factory = function(_, fs, path, JSZip, socketio, settings, mongo) }); } + attachLegacy(srv) { + /** + * @type {socketIo.Server} + */ + const io = socketio(srv); + + io.on('connection', (socket) => { + socket.on('agent:auth', (data, cb) => { + return cb('You are using an older version of the agent. Please reload agent archive'); + }); + }); + } + /** * @param {http.Server|https.Server} srv Server instance that we want to attach agent handler. */ @@ -646,7 +676,7 @@ module.exports.factory = function(_, fs, path, JSZip, socketio, settings, mongo) /** * @type {socketIo.Server} */ - this._socket = socketio(this._server); + this._socket = socketio(this._server, {path: '/agents'}); this._socket.on('connection', (socket) => { socket.on('agent:auth', (data, cb) => { diff --git a/modules/web-console/backend/app/browser.js b/modules/web-console/backend/app/browser.js index 254851d0bb6b7..271082910e2c1 100644 --- a/modules/web-console/backend/app/browser.js +++ b/modules/web-console/backend/app/browser.js @@ -149,6 +149,19 @@ module.exports.factory = (_, socketio, agentMgr, configure) => { .catch((err) => cb(_errorToJson(err))); }); + // Collect cache query metrics and return result to browser. + socket.on('node:query:metrics', (nids, since, cb) => { + agentMgr.findAgent(accountId()) + .then((agent) => agent.queryDetailMetrics(demo, nids, since)) + .then((data) => { + if (data.finished) + return cb(null, data.result); + + cb(_errorToJson(data.error)); + }) + .catch((err) => cb(_errorToJson(err))); + }); + // Return cache metadata from all nodes in grid. socket.on('node:cache:metadata', (cacheName, cb) => { agentMgr.findAgent(accountId()) diff --git a/modules/web-console/backend/app/mongo.js b/modules/web-console/backend/app/mongo.js index e12af2a6c8dc1..0f38eb2c3c899 100644 --- a/modules/web-console/backend/app/mongo.js +++ b/modules/web-console/backend/app/mongo.js @@ -197,6 +197,12 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose type: String, enum: ['Generic', 'Oracle', 'DB2', 'SQLServer', 'MySQL', 'PostgreSQL', 'H2'] }, + batchSize: Number, + maximumPoolSize: Number, + maximumWriteAttempts: Number, + parallelLoadCacheMinimumThreshold: Number, + hasher: String, + transformer: String, sqlEscapeAll: Boolean }, CacheJdbcBlobStoreFactory: { diff --git a/modules/web-console/backend/config/settings.json.sample b/modules/web-console/backend/config/settings.json.sample index 41f1152a411b7..71a64eabb8614 100644 --- a/modules/web-console/backend/config/settings.json.sample +++ b/modules/web-console/backend/config/settings.json.sample @@ -10,13 +10,6 @@ "mongodb": { "url": "mongodb://localhost/console" }, - "agentServer": { - "port": 3001, - "ssl": false, - "key": "serve/keys/test.key", - "cert": "serve/keys/test.crt", - "keyPassphrase": "password" - }, "mail": { "service": "", "sign": "Kind regards,
          Apache Ignite Team", diff --git a/modules/web-console/backend/index.js b/modules/web-console/backend/index.js index 3a8ada9f2c02a..9eeff3516c2ee 100644 --- a/modules/web-console/backend/index.js +++ b/modules/web-console/backend/index.js @@ -82,9 +82,11 @@ Promise.all([injector('settings'), injector('app'), injector('agent-manager'), i server.on('listening', _onListening.bind(null, server.address())); app.listen(server); + + agentMgr.attach(server); browserMgr.attach(server); - // Start agent server. + // Start legacy agent server. const agentServer = settings.agent.SSLOptions ? https.createServer(settings.agent.SSLOptions) : http.createServer(); @@ -92,7 +94,7 @@ Promise.all([injector('settings'), injector('app'), injector('agent-manager'), i agentServer.on('error', _onError.bind(null, settings.agent.port)); agentServer.on('listening', _onListening.bind(null, agentServer.address())); - agentMgr.attach(agentServer); + agentMgr.attachLegacy(agentServer); // Used for automated test. if (process.send) diff --git a/modules/web-console/backend/middlewares/host.js b/modules/web-console/backend/middlewares/host.js index 5ddd918f5e64d..4c21da2c66b4b 100644 --- a/modules/web-console/backend/middlewares/host.js +++ b/modules/web-console/backend/middlewares/host.js @@ -27,10 +27,11 @@ module.exports = { if (req.headers.origin) return req.headers.origin; - if (req.headers['x-forwarded-server']) - return `${req.headers['x-forwarded-proto'] || 'http'}://${req.headers['x-forwarded-server']}`; + const proto = req.headers['x-forwarded-proto'] || req.protocol; - return `${req.protocol}://${req.get('host')}`; + const host = req.headers['x-forwarded-host'] || req.get('host'); + + return `${proto}://${host}`; }; next(); diff --git a/modules/web-console/backend/routes/agent.js b/modules/web-console/backend/routes/agent.js index 3f90fbd072fc8..477363f9f5266 100644 --- a/modules/web-console/backend/routes/agent.js +++ b/modules/web-console/backend/routes/agent.js @@ -36,9 +36,7 @@ module.exports.factory = function(_, express, agentsService) { /* Get grid topology. */ router.get('/download/zip', (req, res) => { - const host = req.hostname.match(/:/g) ? req.hostname.slice(0, req.hostname.indexOf(':')) : req.hostname; - - agentsService.getArchive(host, req.user.token) + agentsService.getArchive(req.origin(), req.user.token) .then(({fileName, buffer}) => { // Set the archive name. res.attachment(fileName); diff --git a/modules/web-console/backend/services/agents.js b/modules/web-console/backend/services/agents.js index 5c0b6a7800653..4931bf8af3772 100644 --- a/modules/web-console/backend/services/agents.js +++ b/modules/web-console/backend/services/agents.js @@ -63,7 +63,7 @@ module.exports.factory = (_, fs, path, JSZip, settings, agentMgr, errors) => { const prop = []; prop.push('tokens=' + token); - prop.push('server-uri=' + (settings.agent.SSLOptions ? 'https' : 'http') + '://' + host + ':' + settings.agent.port); + prop.push(`server-uri=${host}`); prop.push('#Uncomment following options if needed:'); prop.push('#node-uri=http://localhost:8080'); prop.push('#driver-folder=./jdbc-drivers'); diff --git a/modules/web-console/backend/services/caches.js b/modules/web-console/backend/services/caches.js index e59d51df0281f..646fbea20e264 100644 --- a/modules/web-console/backend/services/caches.js +++ b/modules/web-console/backend/services/caches.js @@ -87,6 +87,7 @@ module.exports.factory = (_, mongo, spaceService, errors) => { */ const removeAllBySpaces = (spaceIds) => { return mongo.Cluster.update({space: {$in: spaceIds}}, {caches: []}, {multi: true}).exec() + .then(() => mongo.Cluster.update({space: {$in: spaceIds}}, {$pull: {checkpointSpi: {kind: 'Cache'}}}, {multi: true}).exec()) .then(() => mongo.DomainModel.update({space: {$in: spaceIds}}, {caches: []}, {multi: true}).exec()) .then(() => mongo.Cache.remove({space: {$in: spaceIds}}).exec()); }; @@ -129,6 +130,7 @@ module.exports.factory = (_, mongo, spaceService, errors) => { return Promise.reject(new errors.IllegalArgumentException('Cache id can not be undefined or null')); return mongo.Cluster.update({caches: {$in: [cacheId]}}, {$pull: {caches: cacheId}}, {multi: true}).exec() + .then(() => mongo.Cluster.update({}, {$pull: {checkpointSpi: {kind: 'Cache', Cache: {cache: cacheId}}}}, {multi: true}).exec()) .then(() => mongo.DomainModel.update({caches: {$in: [cacheId]}}, {$pull: {caches: cacheId}}, {multi: true}).exec()) .then(() => mongo.Cache.remove({_id: cacheId}).exec()) .then(convertRemoveStatus); diff --git a/modules/web-console/docker/compose/backend/.dockerignore b/modules/web-console/docker/compose/backend/.dockerignore index 6fadfa5eb31ae..05df665e2b560 100644 --- a/modules/web-console/docker/compose/backend/.dockerignore +++ b/modules/web-console/docker/compose/backend/.dockerignore @@ -1 +1,3 @@ +build/config/*.json build/node_modules +build/test diff --git a/modules/web-console/docker/compose/backend/Dockerfile b/modules/web-console/docker/compose/backend/Dockerfile index b4f7c9ddf4c39..e391ba4ed9499 100644 --- a/modules/web-console/docker/compose/backend/Dockerfile +++ b/modules/web-console/docker/compose/backend/Dockerfile @@ -15,7 +15,7 @@ # limitations under the License. # -FROM node:4 +FROM node:6 RUN mkdir -p /opt/web-console-backend @@ -23,8 +23,8 @@ WORKDIR /opt/web-console-backend COPY build . -RUN npm -g update npm && npm install --no-optional +RUN npm install --only=production --no-optional -EXPOSE 3000 3001 +EXPOSE 3000 CMD ["npm", "start"] diff --git a/modules/web-console/docker/compose/backend/build.sh b/modules/web-console/docker/compose/backend/build.sh index f925bd795b9a6..d44efbdb4fc51 100755 --- a/modules/web-console/docker/compose/backend/build.sh +++ b/modules/web-console/docker/compose/backend/build.sh @@ -51,7 +51,7 @@ cp -r $IGNITE_WEB_CONSOLE_BACKEND_DIR/. $BUILD_DIR cp $IGNITE_HOME/modules/web-console/web-agent/target/ignite-web-agent*.zip $BUILD_DIR/agent_dists/. echo "Step 4. Build docker image." -docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION . +docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION -t $DOCKER_IMAGE_NAME:latest . echo "Step 5. Cleanup." rm -Rf $BUILD_DIR diff --git a/modules/web-console/docker/compose/docker-compose.yml b/modules/web-console/docker/compose/docker-compose.yml index a2c2f8bd9c691..8b9b86ac4e4d0 100644 --- a/modules/web-console/docker/compose/docker-compose.yml +++ b/modules/web-console/docker/compose/docker-compose.yml @@ -26,9 +26,6 @@ backend: links: # Link mongodb container as with mongodb hostname. - mongodb:mongodb - ports: - # Proxy 3001 port from docker container to 3001 port host machine. (HOST_PORT:DOCKER_PORT) - - 3001:3001 # Restart on crash. restart: always environment: @@ -38,8 +35,6 @@ backend: - server_sessionSecret=CHANGE ME # URL for mongodb connection - mongodb_url=mongodb://mongodb/console - # Port for web-agent. - - agentServer_port=3001 # Mail connection settings. Leave empty if no needed. See also settings, https://github.com/nodemailer/nodemailer - mail_service= - mail_sign= diff --git a/modules/web-console/docker/compose/frontend/DockerfileBuild b/modules/web-console/docker/compose/frontend/DockerfileBuild index 277991f7f9a36..2d61f4ca2c98a 100644 --- a/modules/web-console/docker/compose/frontend/DockerfileBuild +++ b/modules/web-console/docker/compose/frontend/DockerfileBuild @@ -15,7 +15,7 @@ # limitations under the License. # -FROM node:4 +FROM node:6 RUN mkdir -p /opt/web-console-frontend @@ -23,7 +23,7 @@ WORKDIR /opt/web-console-frontend COPY src . -RUN npm update -g npm && npm install --no-optional +RUN npm install --no-optional VOLUME /opt/web-console-frontend/build diff --git a/modules/web-console/docker/compose/frontend/build.sh b/modules/web-console/docker/compose/frontend/build.sh index 4dfa57af5abe0..6d52fb261788d 100755 --- a/modules/web-console/docker/compose/frontend/build.sh +++ b/modules/web-console/docker/compose/frontend/build.sh @@ -50,7 +50,7 @@ docker build -f=./DockerfileBuild -t $DOCKER_BUILD_IMAGE_NAME:latest . docker run -it -v $BUILD_DIR:/opt/web-console-frontend/build --name $DOCKER_BUILD_CONTAINER $DOCKER_BUILD_IMAGE_NAME echo "Step 2. Build NGINX container with SPA and proxy configuration" -docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION . +docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION -t $DOCKER_IMAGE_NAME:latest . echo "Step 3. Cleanup" docker rm -f $DOCKER_BUILD_CONTAINER diff --git a/modules/web-console/docker/compose/frontend/nginx/web-console.conf b/modules/web-console/docker/compose/frontend/nginx/web-console.conf index d80a7f9a46765..323826e230c17 100644 --- a/modules/web-console/docker/compose/frontend/nginx/web-console.conf +++ b/modules/web-console/docker/compose/frontend/nginx/web-console.conf @@ -53,6 +53,15 @@ server { proxy_pass http://backend-api; } + location /agents { + proxy_set_header Upgrade $http_upgrade; + proxy_set_header Connection "upgrade"; + proxy_http_version 1.1; + proxy_set_header X-Forwarded-For $proxy_add_x_forwarded_for; + proxy_set_header Host $host; + proxy_pass http://backend-api; + } + location = /50x.html { root $ignite_console_dir/error_page; } diff --git a/modules/web-console/docker/standalone/.dockerignore b/modules/web-console/docker/standalone/.dockerignore index 35b7244e3f7af..c59189ec92ce3 100644 --- a/modules/web-console/docker/standalone/.dockerignore +++ b/modules/web-console/docker/standalone/.dockerignore @@ -1,2 +1,7 @@ +build/frontend/build build/frontend/node_modules +build/frontend/ignite_modules_temp +build/frontend/test +build/backend/config/*.json build/backend/node_modules +build/backend/test diff --git a/modules/web-console/docker/standalone/Dockerfile b/modules/web-console/docker/standalone/Dockerfile index 785f1099d5256..0bcd07d18f1a7 100644 --- a/modules/web-console/docker/standalone/Dockerfile +++ b/modules/web-console/docker/standalone/Dockerfile @@ -18,7 +18,7 @@ FROM ubuntu:14.04 ENV NPM_CONFIG_LOGLEVEL info -ENV NODE_VERSION 4.4.7 +ENV NODE_VERSION 6.6.0 # Before package list update. RUN set -ex && \ @@ -40,7 +40,7 @@ RUN apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv EA312927 && \ # Update package list & install. RUN apt-get update && \ - apt-get install -y nginx-light mongodb-org-server curl xz-utils + apt-get install -y nginx-light mongodb-org-server curl xz-utils git # Install Node JS. RUN curl -SLO "https://nodejs.org/dist/v$NODE_VERSION/node-v$NODE_VERSION-linux-x64.tar.xz" && \ @@ -51,7 +51,7 @@ RUN curl -SLO "https://nodejs.org/dist/v$NODE_VERSION/node-v$NODE_VERSION-linux- rm "node-v$NODE_VERSION-linux-x64.tar.xz" SHASUMS256.txt.asc SHASUMS256.txt # Install global node packages. -RUN npm upgrade -g npm && npm install -g pm2 +RUN npm install -g pm2 # Install frontend & backend apps. RUN mkdir -p /opt/web-console @@ -62,7 +62,7 @@ COPY build . # Install node modules. RUN cd /opt/web-console/frontend && npm install --no-optional && npm run build -RUN cd /opt/web-console/backend && npm install --no-optional +RUN cd /opt/web-console/backend && npm install --only=production --no-optional # Returns to base path. WORKDIR /opt/web-console @@ -82,6 +82,6 @@ VOLUME ["/etc/nginx"] VOLUME ["/var/lib/mongodb"] VOLUME ["/opt/web-console/serve/agent_dists"] -EXPOSE 80 3001 +EXPOSE 80 ENTRYPOINT ["/opt/web-console/entrypoint.sh"] diff --git a/modules/web-console/docker/standalone/build.sh b/modules/web-console/docker/standalone/build.sh index 4365dec3f7535..5482086937561 100755 --- a/modules/web-console/docker/standalone/build.sh +++ b/modules/web-console/docker/standalone/build.sh @@ -53,7 +53,7 @@ cp -r $IGNITE_WEB_CONSOLE_DIR/backend/. $BUILD_DIR/backend cp $IGNITE_HOME/modules/web-console/web-agent/target/ignite-web-agent*.zip $BUILD_DIR/backend/agent_dists/. echo "Step 4. Build docker image." -docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION . +docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION -t $DOCKER_IMAGE_NAME:latest . echo "Step 5. Cleanup." rm -Rf $BUILD_DIR diff --git a/modules/web-console/docker/standalone/docker-compose.yml b/modules/web-console/docker/standalone/docker-compose.yml index 561c88d6b9c64..eb59e8a880f61 100644 --- a/modules/web-console/docker/standalone/docker-compose.yml +++ b/modules/web-console/docker/standalone/docker-compose.yml @@ -18,10 +18,7 @@ webconsole: image: ignite/web-console-standalone ports: - - 3080:80 - - 3000:3000 - - 3001:3001 - - 27017:27017 + - 80:80 restart: always environment: # Port for serving frontend API @@ -30,8 +27,6 @@ webconsole: - server_sessionSecret="CHANGE ME" # URL for mongodb connection - mongodb_url=mongodb://127.0.0.1/console - # Port for web-agent. - - agentServer_port=3001 # Mail connection settings. Leave empty if no needed. See also settings, https://github.com/nodemailer/nodemailer - mail_service="" - mail_sign="" diff --git a/modules/web-console/docker/standalone/nginx/web-console.conf b/modules/web-console/docker/standalone/nginx/web-console.conf index 3de544f44f422..3d830758f502d 100644 --- a/modules/web-console/docker/standalone/nginx/web-console.conf +++ b/modules/web-console/docker/standalone/nginx/web-console.conf @@ -48,6 +48,15 @@ server { proxy_pass http://backend-api; } + location /agents { + proxy_set_header Upgrade $http_upgrade; + proxy_set_header Connection "upgrade"; + proxy_http_version 1.1; + proxy_set_header X-Forwarded-For $proxy_add_x_forwarded_for; + proxy_set_header Host $host; + proxy_pass http://backend-api; + } + location = /50x.html { root $ignite_console_dir/error_page; } diff --git a/modules/web-console/frontend/app/data/pom-dependencies.json b/modules/web-console/frontend/app/data/pom-dependencies.json index 7ab6c1b9c43e3..acf2bc86d6249 100644 --- a/modules/web-console/frontend/app/data/pom-dependencies.json +++ b/modules/web-console/frontend/app/data/pom-dependencies.json @@ -13,7 +13,7 @@ "Generic": {"groupId": "com.mchange", "artifactId": "c3p0", "version": "0.9.5.1"}, "MySQL": {"groupId": "mysql", "artifactId": "mysql-connector-java", "version": "5.1.37"}, "PostgreSQL": {"groupId": "org.postgresql", "artifactId": "postgresql", "version": "9.4-1204-jdbc42"}, - "H2": {"groupId": "com.h2database", "artifactId": "h2", "version": "1.3.175"}, + "H2": {"groupId": "com.h2database", "artifactId": "h2", "version": "1.4.191"}, "Oracle": {"groupId": "oracle", "artifactId": "jdbc", "version": "11.2", "jar": "ojdbc6.jar"}, "DB2": {"groupId": "ibm", "artifactId": "jdbc", "version": "4.19.26", "jar": "db2jcc4.jar"}, "SQLServer": {"groupId": "microsoft", "artifactId": "jdbc", "version": "4.1", "jar": "sqljdbc41.jar"} diff --git a/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.jade b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.jade index ed1432b330dea..581b8c1451601 100644 --- a/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.jade +++ b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.jade @@ -15,7 +15,7 @@ limitations under the License. mixin check-tooltip(message) - i.tipLabel.fa.fa-question-circle(bs-tooltip='"#{message}"') + i.tipLabel.icon-help(bs-tooltip='"#{message}"') .panel-details-noborder .details-row diff --git a/modules/web-console/frontend/app/filters/duration.filter.js b/modules/web-console/frontend/app/filters/duration.filter.js index deeedd729d459..0d7afe608d1d1 100644 --- a/modules/web-console/frontend/app/filters/duration.filter.js +++ b/modules/web-console/frontend/app/filters/duration.filter.js @@ -20,6 +20,9 @@ export default ['duration', [() => { * @param {Number} t Time in ms. */ return (t) => { + if (t === 9223372036854775807) + return 'Infinite'; + const a = (i, suffix) => i && i !== '00' ? i + suffix + ' ' : ''; const cd = 24 * 60 * 60 * 1000; diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-checkbox.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-checkbox.jade index ef5cb37fa8222..222ecfee76317 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-checkbox.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-checkbox.jade @@ -35,4 +35,4 @@ mixin form-field-checkbox(label, model, name, disabled, required, tip) data-ignite-form-panel-field='' ) span #{label} - i.tipLabel.fa.fa-question-circle(bs-tooltip='' data-title=tip) + i.tipLabel.icon-help(bs-tooltip='' data-title=tip) diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade index 25e58058bc661..4c1970e14bfa0 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade @@ -40,7 +40,7 @@ mixin form-field-datalist(label, model, name, disabled, required, placeholder, o .ignite-form-field +ignite-form-field__label(label, name, required) .ignite-form-field__control - i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip) + i.tipField.fa.icon-help(bs-tooltip='' data-title=tip) +form-field-feedback(name, 'required', errLbl + ' could not be empty!') diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade index f5d035db2e7ae..298db52c89b1b 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade @@ -41,7 +41,7 @@ mixin ignite-form-field-dropdown(label, model, name, disabled, required, multipl .ignite-form-field +ignite-form-field__label(label, name, required) .ignite-form-field__control - i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip) + i.tipField.icon-help(bs-tooltip='' data-title=tip) if block block diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade index 62b3e09b300c2..d48343c36d6f0 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade @@ -38,7 +38,7 @@ mixin ignite-form-field-number(label, model, name, disabled, required, placehold .ignite-form-field +ignite-form-field__label(label, name, required) .ignite-form-field__control - i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip) + i.tipField.icon-help(bs-tooltip='' data-title=tip) +form-field-feedback(name, 'required', 'This field could not be empty') +form-field-feedback(name, 'min', 'Value is less than allowable minimum: '+ min || 0) diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-password.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-password.jade index 51cc1098fa01a..e5e7bc83f3114 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-password.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-password.jade @@ -36,7 +36,7 @@ mixin ignite-form-field-password(label, model, name, disabled, required, placeho .ignite-form-field +ignite-form-field__label(label, name, required) .ignite-form-field__control - i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip) + i.tipField.icon-help(bs-tooltip='' data-title=tip) if block block diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade index 55b850dfe1810..136d23b2340a1 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade @@ -36,7 +36,7 @@ mixin ignite-form-field-text(label, model, name, disabled, required, placeholder .ignite-form-field +ignite-form-field__label(label, name, required) .ignite-form-field__control - i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip) + i.tipField.icon-help(bs-tooltip='' data-title=tip) if block block diff --git a/modules/web-console/frontend/app/helpers/jade/mixins.jade b/modules/web-console/frontend/app/helpers/jade/mixins.jade index 9d46883e2074d..92af1b02a01b3 100644 --- a/modules/web-console/frontend/app/helpers/jade/mixins.jade +++ b/modules/web-console/frontend/app/helpers/jade/mixins.jade @@ -433,10 +433,8 @@ mixin evictionPolicy(model, name, enabled, required, tip) {value: "SORTED", label: "Sorted"},\ {value: undefined, label: "Not set"}\ ]', tip) - span(ng-if=kind ng-init='__ = {};') - a.customize(ng-show='__.expanded' ng-click='__.expanded = false') Hide settings - a.customize(ng-hide='__.expanded' ng-click='__.expanded = true') Show settings - .panel-details(ng-if='__.expanded') + span(ng-show=kind) + +showHideLink('expanded', 'settings') .details-row +number('Batch size', policy + '.batchSize', name + '+ "batchSize"', enabled, '1', '1', 'Number of entries to remove on shrink') @@ -517,9 +515,9 @@ mixin table-pair-edit(tbl, prefix, keyPlaceholder, valPlaceholder, keyJavaBuiltI .fieldSep !{divider} .input-tip if keyJavaBuiltInTypes - input.form-control(id=keyFocusId ignite-on-enter-focus-move=valFocusId type='text' ng-model=keyModel placeholder=keyPlaceholder bs-typeahead container='body' ignite-retain-selection data-min-length='1' bs-options='javaClass for javaClass in javaBuiltInClasses' ignite-on-escape='tableReset()') + input.form-control(id=keyFocusId ignite-on-enter-focus-move=valFocusId type='text' ng-model=keyModel placeholder=keyPlaceholder bs-typeahead container='body' ignite-retain-selection data-min-length='1' bs-options='javaClass for javaClass in javaBuiltInClasses' ignite-on-escape='tableReset(false)') else - input.form-control(id=keyFocusId ignite-on-enter-focus-move=valFocusId type='text' ng-model=keyModel placeholder=keyPlaceholder ignite-on-escape='tableReset()') + input.form-control(id=keyFocusId ignite-on-enter-focus-move=valFocusId type='text' ng-model=keyModel placeholder=keyPlaceholder ignite-on-escape='tableReset(false)') .col-xs-6.col-sm-6.col-md-6 -var btnVisible = 'tablePairSaveVisible(' + tbl + ', ' + index + ')' -var btnSave = 'tablePairSave(tablePairValid, backupItem, ' + tbl + ', ' + index + ')' @@ -528,9 +526,9 @@ mixin table-pair-edit(tbl, prefix, keyPlaceholder, valPlaceholder, keyJavaBuiltI +btn-save(btnVisible, btnSave) .input-tip if valueJavaBuiltInTypes - input.form-control(id=valFocusId type='text' ng-model=valModel placeholder=valPlaceholder bs-typeahead container='body' ignite-retain-selection data-min-length='1' bs-options='javaClass for javaClass in javaBuiltInClasses' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()') + input.form-control(id=valFocusId type='text' ng-model=valModel placeholder=valPlaceholder bs-typeahead container='body' ignite-retain-selection data-min-length='1' bs-options='javaClass for javaClass in javaBuiltInClasses' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)') else - input.form-control(id=valFocusId type='text' ng-model=valModel placeholder=valPlaceholder ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()') + input.form-control(id=valFocusId type='text' ng-model=valModel placeholder=valPlaceholder ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)') //- Mixin for DB dialect. mixin dialect(lbl, model, name, required, tipTitle, genericDialectName, placeholder) @@ -553,3 +551,13 @@ mixin dialect(lbl, model, name, required, tipTitle, genericDialectName, placehol

        • PostgreSQL
        • \
        • H2 database
        • \ ') + +//- Mixin for show/hide links. +mixin showHideLink(name, text) + span(ng-init='__ = {};') + a.customize(ng-show='__.#{name}' ng-click='__.#{name} = false') Hide #{text} + a.customize(ng-hide='__.#{name}' ng-click='__.#{name} = true; ui.loadPanel("#{name}");') Show #{text} + div(ng-if='ui.isPanelLoaded("#{name}")') + .panel-details(ng-show='__.#{name}') + if block + block diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Beans.js b/modules/web-console/frontend/app/modules/configuration/generator/Beans.js index 546f38b086651..2750626f4a74b 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/Beans.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/Beans.js @@ -221,6 +221,12 @@ export class Bean extends EmptyBean { return this; } + propertyInt(name, value, hint) { + this.properties.push({clsName: 'PROPERTY_INT', name, value, hint}); + + return this; + } + stringProperty(model, name = model, mapper) { return this._property(this.properties, 'java.lang.String', model, name, _.nonEmpty, mapper); } diff --git a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js index 869e3dfb52063..5887832ff81df 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js @@ -103,9 +103,9 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig case 'DB2': dsBean = new Bean('com.ibm.db2.jcc.DB2DataSource', id, {}) .property('serverName', `${id}.jdbc.server_name`, 'YOUR_DATABASE_SERVER_NAME') - .property('portNumber', `${id}.jdbc.port_number`, 'YOUR_JDBC_PORT_NUMBER') - .property('databaseName', `${id}.jdbc.database_name`, 'YOUR_JDBC_DRIVER_TYPE') - .property('driverType', `${id}.jdbc.driver_type`, 'YOUR_DATABASE_NAME'); + .propertyInt('portNumber', `${id}.jdbc.port_number`, 'YOUR_JDBC_PORT_NUMBER') + .property('databaseName', `${id}.jdbc.database_name`, 'YOUR_DATABASE_NAME') + .propertyInt('driverType', `${id}.jdbc.driver_type`, 'YOUR_JDBC_DRIVER_TYPE'); break; case 'SQLServer': @@ -234,12 +234,14 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig ipFinder.emptyBeanProperty('curator') .stringProperty('zkConnectionString'); - if (_.get(src, 'retryPolicy.kind')) { + const kind = _.get(src, 'retryPolicy.kind'); + + if (kind) { const policy = src.retryPolicy; let retryPolicyBean; - switch (policy.kind) { + switch (kind) { case 'ExponentialBackoff': retryPolicyBean = new Bean('org.apache.curator.retry.ExponentialBackoffRetry', null, policy.ExponentialBackoff, dflt.ExponentialBackoff) @@ -289,6 +291,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig break; default: + // No-op. } if (retryPolicyBean) @@ -301,6 +304,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig break; default: + // No-op. } if (ipFinder) @@ -425,10 +429,10 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig // Generate checkpoint configurations. static clusterCheckpoint(cluster, caches, cfg = this.igniteConfigurationBean()) { const cfgs = _.filter(_.map(cluster.checkpointSpi, (spi) => { - switch (spi.kind) { + switch (_.get(spi, 'kind')) { case 'FS': const fsBean = new Bean('org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi', - 'checkpointSpi', spi.FS); + 'checkpointSpiFs', spi.FS); fsBean.collectionProperty('directoryPaths', 'directoryPaths', _.get(spi, 'FS.directoryPaths')) .emptyBeanProperty('checkpointListener'); @@ -437,9 +441,11 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig case 'Cache': const cacheBean = new Bean('org.apache.ignite.spi.checkpoint.cache.CacheCheckpointSpi', - 'checkpointSpi', spi.Cache); + 'checkpointSpiCache', spi.Cache); - const cache = _.find(caches, (c) => c._id === _.get(spi, 'Cache.cache') || c.cache._id === _.get(spi, 'Cache.cache')); + const curCache = _.get(spi, 'Cache.cache'); + + const cache = _.find(caches, (c) => curCache && (c._id === curCache || _.get(c, 'cache._id') === curCache)); if (cache) cacheBean.prop('java.lang.String', 'cacheName', cache.name || cache.cache.name); @@ -451,7 +457,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig case 'S3': const s3Bean = new Bean('org.apache.ignite.spi.checkpoint.s3.S3CheckpointSpi', - 'checkpointSpi', spi.S3, clusterDflts.checkpointSpi.S3); + 'checkpointSpiS3', spi.S3, clusterDflts.checkpointSpi.S3); let credentialsBean = null; @@ -459,8 +465,8 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig case 'Basic': credentialsBean = new Bean('com.amazonaws.auth.BasicAWSCredentials', 'awsCredentials', {}); - credentialsBean.constructorArgument('PROPERTY', 'checkpoint.s3.credentials.accessKey') - .constructorArgument('PROPERTY', 'checkpoint.s3.credentials.secretKey'); + credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey', 'YOUR_S3_ACCESS_KEY') + .propertyConstructorArgument('checkpoint.s3.credentials.secretKey', 'YOUR_S3_SECRET_KEY'); break; @@ -645,7 +651,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig case 'JDBC': const jdbcBean = new Bean('org.apache.ignite.spi.checkpoint.jdbc.JdbcCheckpointSpi', - 'checkpointSpi', spi.JDBC, clusterDflts.checkpointSpi.JDBC); + 'checkpointSpiJdbc', spi.JDBC, clusterDflts.checkpointSpi.JDBC); const id = jdbcBean.valueOf('dataSourceBean'); const dialect = _.get(spi.JDBC, 'dialect'); @@ -673,7 +679,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig const clsName = _.get(spi, 'Custom.className'); if (clsName) - return new Bean(clsName, 'checkpointSpi', spi.Cache); + return new Bean(clsName, 'checkpointSpiCustom', spi.Cache); return null; @@ -691,7 +697,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig static clusterCollision(collision, cfg = this.igniteConfigurationBean()) { let colSpi; - switch (collision.kind) { + switch (_.get(collision, 'kind')) { case 'JobStealing': colSpi = new Bean('org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi', 'colSpi', collision.JobStealing, clusterDflts.collision.JobStealing); @@ -727,18 +733,16 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig break; case 'Custom': - colSpi = new Bean(collision.Custom.class, - 'colSpi', collision.PriorityQueue, clusterDflts.collision.PriorityQueue); + if (_.nonNil(_.get(collision, 'Custom.class'))) + colSpi = new EmptyBean(collision.Custom.class); break; default: return cfg; } - if (colSpi.isEmpty()) - return cfg; - - cfg.beanProperty('collisionSpi', colSpi); + if (_.nonNil(colSpi)) + cfg.beanProperty('collisionSpi', colSpi); return cfg; } @@ -907,7 +911,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig _.forEach(cluster.failoverSpi, (spi) => { let failoverSpi; - switch (spi.kind) { + switch (_.get(spi, 'kind')) { case 'JobStealing': failoverSpi = new Bean('org.apache.ignite.spi.failover.jobstealing.JobStealingFailoverSpi', 'failoverSpi', spi.JobStealing, clusterDflts.failoverSpi.JobStealing); @@ -955,26 +959,25 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig _.forEach(cluster.loadBalancingSpi, (spi) => { let loadBalancingSpi; - switch (spi.kind) { + switch (_.get(spi, 'kind')) { case 'RoundRobin': - loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi', 'loadBalancingSpi', spi.RoundRobin, clusterDflts.loadBalancingSpi.RoundRobin); + loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi', 'loadBalancingSpiRR', spi.RoundRobin, clusterDflts.loadBalancingSpi.RoundRobin); loadBalancingSpi.boolProperty('perTask'); break; case 'Adaptive': - loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveLoadBalancingSpi', 'loadBalancingSpi', spi.Adaptive); + loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveLoadBalancingSpi', 'loadBalancingSpiAdaptive', spi.Adaptive); let probeBean; - switch (spi.Adaptive.loadProbe.kind) { + switch (_.get(spi, 'Adaptive.loadProbe.kind')) { case 'Job': probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveJobCountLoadProbe', 'jobProbe', spi.Adaptive.loadProbe.Job, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.Job); probeBean.boolProperty('useAverage'); break; - case 'CPU': probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveCpuLoadProbe', 'cpuProbe', spi.Adaptive.loadProbe.CPU, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.CPU); @@ -983,14 +986,12 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig .intProperty('processorCoefficient'); break; - case 'ProcessingTime': probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveProcessingTimeLoadProbe', 'timeProbe', spi.Adaptive.loadProbe.ProcessingTime, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.ProcessingTime); probeBean.boolProperty('useAverage'); break; - case 'Custom': const className = _.get(spi, 'Adaptive.loadProbe.Custom.className'); @@ -998,7 +999,6 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig probeBean = new Bean(className, 'probe', spi.Adaptive.loadProbe.Job.Custom); break; - default: // No-op. } @@ -1008,7 +1008,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig break; case 'WeightedRandom': - loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.weightedrandom.WeightedRandomLoadBalancingSpi', 'loadBalancingSpi', spi.WeightedRandom, clusterDflts.loadBalancingSpi.WeightedRandom); + loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.weightedrandom.WeightedRandomLoadBalancingSpi', 'loadBalancingSpiRandom', spi.WeightedRandom, clusterDflts.loadBalancingSpi.WeightedRandom); loadBalancingSpi.intProperty('nodeWeight') .boolProperty('useWeights'); @@ -1018,7 +1018,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig const className = _.get(spi, 'Custom.className'); if (className) - loadBalancingSpi = new Bean(className, 'loadBalancingSpi', spi.Custom); + loadBalancingSpi = new Bean(className, 'loadBalancingSpiCustom', spi.Custom); break; default: @@ -1030,19 +1030,16 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig }); if (spis.length) - cfg.arrayProperty('loadBalancingSpi', 'loadBalancingSpi', spis, 'org.apache.ignite.spi.loadbalancing.LoadBalancingSpi'); + cfg.varArgProperty('loadBalancingSpi', 'loadBalancingSpi', spis, 'org.apache.ignite.spi.loadbalancing.LoadBalancingSpi'); return cfg; } // Generate logger group. static clusterLogger(logger, cfg = this.igniteConfigurationBean()) { - if (_.isNil(logger)) - return cfg; - let loggerBean; - switch (logger.kind) { + switch (_.get(logger, 'kind')) { case 'Log4j': if (logger.Log4j && (logger.Log4j.mode === 'Default' || logger.Log4j.mode === 'Path' && _.nonEmpty(logger.Log4j.path))) { loggerBean = new Bean('org.apache.ignite.logger.log4j.Log4JLogger', @@ -1087,6 +1084,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig break; default: + return cfg; } if (loggerBean) @@ -1116,34 +1114,34 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig // Generate marshaller group. static clusterMarshaller(cluster, cfg = this.igniteConfigurationBean(cluster)) { - const marshaller = cluster.marshaller; - - if (marshaller && marshaller.kind) { - let bean; + const kind = _.get(cluster.marshaller, 'kind'); + const settings = _.get(cluster.marshaller, kind); - switch (marshaller.kind) { - case 'OptimizedMarshaller': - bean = new Bean('org.apache.ignite.marshaller.optimized.OptimizedMarshaller', 'marshaller', - marshaller[marshaller.kind]); + if (_.isNil(settings)) + return cfg; - bean.intProperty('poolSize') - .intProperty('requireSerializable'); + let bean; - break; + switch (kind) { + case 'OptimizedMarshaller': + bean = new Bean('org.apache.ignite.marshaller.optimized.OptimizedMarshaller', 'marshaller', settings) + .intProperty('poolSize') + .intProperty('requireSerializable'); - case 'JdkMarshaller': - bean = new Bean('org.apache.ignite.marshaller.jdk.JdkMarshaller', 'marshaller', - marshaller[marshaller.kind]); + break; - break; + case 'JdkMarshaller': + bean = new Bean('org.apache.ignite.marshaller.jdk.JdkMarshaller', 'marshaller', settings); - default: - } + break; - if (bean) - cfg.beanProperty('marshaller', bean); + default: + // No-op. } + if (bean) + cfg.beanProperty('marshaller', bean); + cfg.intProperty('marshalLocalJobs') .intProperty('marshallerCacheKeepAliveTime') .intProperty('marshallerCacheThreadPoolSize', 'marshallerCachePoolSize'); @@ -1214,7 +1212,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig // Generate swap group. static clusterSwap(cluster, cfg = this.igniteConfigurationBean(cluster)) { - if (cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind === 'FileSwapSpaceSpi') { + if (_.get(cluster.swapSpaceSpi, 'kind') === 'FileSwapSpaceSpi') { const bean = new Bean('org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi', 'swapSpaceSpi', cluster.swapSpaceSpi.FileSwapSpaceSpi); @@ -1451,14 +1449,20 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig switch (kind) { case 'CacheJdbcPojoStoreFactory': bean = new Bean('org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory', 'cacheStoreFactory', - storeFactory); + storeFactory, cacheDflts.cacheStoreFactory.CacheJdbcPojoStoreFactory); const jdbcId = bean.valueOf('dataSourceBean'); bean.dataSource(jdbcId, 'dataSourceBean', this.dataSourceBean(jdbcId, storeFactory.dialect)) .beanProperty('dialect', new EmptyBean(this.dialectClsName(storeFactory.dialect))); - bean.boolProperty('sqlEscapeAll'); + bean.intProperty('batchSize') + .intProperty('maximumPoolSize') + .intProperty('maximumWriteAttempts') + .intProperty('parallelLoadCacheMinimumThreshold') + .emptyBeanProperty('hasher') + .emptyBeanProperty('transformer') + .boolProperty('sqlEscapeAll'); const setType = (typeBean, propName) => { if (JavaTypes.nonBuiltInClass(typeBean.valueOf(propName))) @@ -1555,31 +1559,35 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig static cacheNodeFilter(cache, igfss, ccfg = this.cacheConfigurationBean(cache)) { const kind = _.get(cache, 'nodeFilter.kind'); - if (kind && cache.nodeFilter[kind]) { - let bean = null; + const settings = _.get(cache.nodeFilter, kind); - switch (kind) { - case 'IGFS': - const foundIgfs = _.find(igfss, (igfs) => igfs._id === cache.nodeFilter.IGFS.igfs); + if (_.isNil(settings)) + return ccfg; - if (foundIgfs) { - bean = new Bean('org.apache.ignite.internal.processors.igfs.IgfsNodePredicate', 'nodeFilter', foundIgfs) - .stringConstructorArgument('name'); - } + let bean = null; - break; - case 'Custom': - bean = new Bean(cache.nodeFilter.Custom.className, 'nodeFilter'); + switch (kind) { + case 'IGFS': + const foundIgfs = _.find(igfss, {_id: settings.igfs}); - break; - default: - return ccfg; - } + if (foundIgfs) { + bean = new Bean('org.apache.ignite.internal.processors.igfs.IgfsNodePredicate', 'nodeFilter', foundIgfs) + .stringConstructorArgument('name'); + } - if (bean) - ccfg.beanProperty('nodeFilter', bean); + break; + case 'Custom': + if (_.nonEmpty(settings.className)) + bean = new EmptyBean(settings.className); + + break; + default: + // No-op. } + if (bean) + ccfg.beanProperty('nodeFilter', bean); + return ccfg; } @@ -1733,7 +1741,8 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig .pathProperty('tokenDirectoryPath') .intProperty('threadCount'); - cfg.beanProperty('ipcEndpointConfiguration', bean); + if (bean.nonEmpty()) + cfg.beanProperty('ipcEndpointConfiguration', bean); } return cfg; diff --git a/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js index 64d43d8ff8091..b123ab5529f39 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js @@ -395,17 +395,19 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator' return `${item}L`; case 'java.io.Serializable': case 'java.lang.String': - return `"${item}"`; + return `"${item.replace(/\\/g, '\\\\').replace(/"/g, '\\"')}"`; case 'PATH': return `"${item.replace(/\\/g, '\\\\')}"`; case 'java.lang.Class': return `${JavaTypes.shortClassName(item)}.class`; case 'java.util.UUID': return `UUID.fromString("${item}")`; - case 'PROPERTY_CHAR': - return `props.getProperty("${item}").toCharArray()`; case 'PROPERTY': return `props.getProperty("${item}")`; + case 'PROPERTY_CHAR': + return `props.getProperty("${item}").toCharArray()`; + case 'PROPERTY_INT': + return `Integer.parseInt(props.getProperty("${item}"))`; default: if (this._isBean(clsName)) { if (item.isComplex()) @@ -744,6 +746,7 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator' break; case 'PROPERTY': case 'PROPERTY_CHAR': + case 'PROPERTY_INT': imports.push('java.io.InputStream', 'java.util.Properties'); break; @@ -1075,7 +1078,7 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator' sb.startBlock(`public ${clsName}(${arg(_.head(fields))}${fields.length === 1 ? ') {' : ','}`); _.forEach(_.tail(fields), (field, idx) => { - sb.append(`${arg(field)}${idx !== fields.length - 1 ? ',' : ') {'}`); + sb.append(`${arg(field)}${idx !== fields.length - 2 ? ',' : ') {'}`); }); _.forEach(fields, (field) => sb.append(`this.${field.javaFieldName} = ${field.javaFieldName};`)); @@ -1262,8 +1265,8 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator' const valueFields = _.clone(domain.valueFields); if (includeKeyFields) { - _.forEach(domain.keyFields, ({fld}) => { - if (!_.find(valueFields, {name: fld.name})) + _.forEach(domain.keyFields, (fld) => { + if (!_.find(valueFields, {javaFieldName: fld.javaFieldName})) valueFields.push(fld); }); } diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js index 07f8dffe7eb5e..49b4aa69df949 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js @@ -24,6 +24,23 @@ export default class PropertiesGenerator { _collectProperties(bean) { const props = []; + _.forEach(bean.arguments, (arg) => { + switch (arg.clsName) { + case 'BEAN': + props.push(...this._collectProperties(arg.value)); + + break; + case 'PROPERTY': + case 'PROPERTY_CHAR': + case 'PROPERTY_INT': + props.push(`${arg.value}=${arg.hint}`); + + break; + default: + // No-op. + } + }); + _.forEach(bean.properties, (prop) => { switch (prop.clsName) { case 'DATA_SOURCE': @@ -37,6 +54,7 @@ export default class PropertiesGenerator { break; case 'PROPERTY': case 'PROPERTY_CHAR': + case 'PROPERTY_INT': props.push(`${prop.value}=${prop.hint}`); break; @@ -51,6 +69,7 @@ export default class PropertiesGenerator { break; default: + // No-op. } }); diff --git a/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js index 91a85fa65c998..73df25e000e53 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js @@ -20,6 +20,14 @@ import _ from 'lodash'; import AbstractTransformer from './AbstractTransformer'; import StringBuilder from './StringBuilder'; +const escapeXml = (str) => { + return str.replace(/&/g, '&') + .replace(/"/g, '"') + .replace(/'/g, ''') + .replace(/>/g, '>') + .replace(/ { return class SpringTransformer extends AbstractTransformer { static generator = generator; @@ -84,11 +92,14 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator' switch (clsName) { case 'PROPERTY': case 'PROPERTY_CHAR': + case 'PROPERTY_INT': return `\${${item}}`; case 'java.lang.Class': return JavaTypes.fullClassName(item); case 'long': return `${item}L`; + case 'java.lang.String': + return escapeXml(item); default: return item; } @@ -99,9 +110,9 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator' return JavaTypes.nonBuiltInClass(clsName) && JavaTypes.nonEnum(clsName) && _.includes(clsName, '.'); } - static _setCollection(sb, prop, tag) { + static _setCollection(sb, prop) { sb.startBlock(``); - sb.startBlock(`<${tag}>`); + sb.startBlock(''); _.forEach(prop.items, (item, idx) => { if (this._isBean(prop.typeClsName)) { @@ -114,7 +125,7 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator' sb.append(`${item}`); }); - sb.endBlock(``); + sb.endBlock(''); sb.endBlock(''); } @@ -201,11 +212,8 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator' break; case 'ARRAY': - this._setCollection(sb, prop, 'array'); - - break; case 'COLLECTION': - this._setCollection(sb, prop, 'list'); + this._setCollection(sb, prop); break; case 'MAP': diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js index 5ff1e02c63a4d..f50e493268412 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js @@ -37,6 +37,14 @@ const DFLT_CACHE = { sqlEscapeAll: false, storeKeepBinary: false, loadPreviousValue: false, + cacheStoreFactory: { + CacheJdbcPojoStoreFactory: { + batchSize: 512, + maximumWriteAttempts: 2, + parallelLoadCacheMinimumThreshold: 512, + sqlEscapeAll: false + } + }, readThrough: false, writeThrough: false, writeBehindEnabled: false, diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js index 1be4b701bda73..726581de810ea 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js @@ -167,7 +167,7 @@ const DFLT_CLUSTER = { logger: { Log4j: { level: { - clsName: 'org.apache.logging.log4j.Level' + clsName: 'org.apache.log4j.Level' } }, Log4j2: { diff --git a/modules/web-console/frontend/app/modules/form/field/tooltip.directive.js b/modules/web-console/frontend/app/modules/form/field/tooltip.directive.js index 50052805c87fb..4f440a1b9875c 100644 --- a/modules/web-console/frontend/app/modules/form/field/tooltip.directive.js +++ b/modules/web-console/frontend/app/modules/form/field/tooltip.directive.js @@ -15,7 +15,7 @@ * limitations under the License. */ -const template = ''; +const template = ''; export default ['igniteFormFieldTooltip', ['$tooltip', ($tooltip) => { const link = ($scope, $element, $attrs, [form, field], $transclude) => { diff --git a/modules/web-console/frontend/app/modules/form/group/tooltip.directive.js b/modules/web-console/frontend/app/modules/form/group/tooltip.directive.js index 3e470e1c37c81..6027765d22e3c 100644 --- a/modules/web-console/frontend/app/modules/form/group/tooltip.directive.js +++ b/modules/web-console/frontend/app/modules/form/group/tooltip.directive.js @@ -15,7 +15,7 @@ * limitations under the License. */ -const template = ''; +const template = ''; export default ['igniteFormGroupTooltip', ['$tooltip', ($tooltip) => { const link = ($scope, $element, $attrs, $ctrls, $transclude) => { diff --git a/modules/web-console/frontend/app/modules/form/panel/field.directive.js b/modules/web-console/frontend/app/modules/form/panel/field.directive.js index 5dc7b079dc2b8..cf8101ae292ee 100644 --- a/modules/web-console/frontend/app/modules/form/panel/field.directive.js +++ b/modules/web-console/frontend/app/modules/form/panel/field.directive.js @@ -40,8 +40,8 @@ export default ['igniteFormPanelField', ['$parse', 'IgniteLegacyTable', ($parse, else saveDefault(); - scope.tableReset = () => { - if (!LegacyTable.tableSaveAndReset()) + scope.tableReset = (trySave) => { + if (trySave === false || !LegacyTable.tableSaveAndReset()) LegacyTable.tableReset(); }; diff --git a/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade b/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade index addc5f3f06a2a..0396727f95d89 100644 --- a/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade +++ b/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade .modal(tabindex='-1' role='dialog') .modal-dialog diff --git a/modules/web-console/frontend/app/modules/sql/sql.controller.js b/modules/web-console/frontend/app/modules/sql/sql.controller.js index 0c2be01e833b8..4e6e3720bfbef 100644 --- a/modules/web-console/frontend/app/modules/sql/sql.controller.js +++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js @@ -1244,8 +1244,10 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', }; const _closeOldQuery = (paragraph) => { - if (paragraph.queryId) - return agentMonitor.queryClose(paragraph.resNodeId, paragraph.queryId); + const nid = paragraph.resNodeId; + + if (paragraph.queryId && _.find($scope.caches, ({nodes}) => _.includes(nodes, nid))) + return agentMonitor.queryClose(nid, paragraph.queryId); return $q.when(); }; diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/client-near-cache.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/client-near-cache.jade index 9d8ccbea64769..e3d1a81589ced 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/client-near-cache.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/client-near-cache.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'clientNearCache' -var model = 'backupItem.clientNearConfiguration' diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/concurrency.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/concurrency.jade index 37bd88d5b4898..ffcd568852731 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/concurrency.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/concurrency.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'concurrency' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade index e9ff14358db27..14f3ab45e5a63 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'general' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade index 724418f79fb29..f2d3e2bfa656c 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'memory' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.jade index ba538c20007ba..56f7e648c8cc3 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'clientNearCache' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-server.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-server.jade index a96b947971f40..9895281486ef3 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-server.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-server.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'serverNearCache' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade index eb74736738361..b34aba032f6ff 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'nodeFilter' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade index ae13166c3fa94..5062ce1ac9445 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'query' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade index 824442c35d02a..d8ef3adb62239 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'rebalance' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/statistics.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/statistics.jade index 027a2bd82aed1..9f7a3465b29fb 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/statistics.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/statistics.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'statistics' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade index a7d8f1453777b..1cf80b8a193f4 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'store' -var model = 'backupItem' @@ -87,6 +87,24 @@ mixin hibernateField(name, model, items, valid, save, newItem) +dialect('Dialect:', pojoStoreFactory + '.dialect', '"pojoDialect"', required, 'Dialect of SQL implemented by a particular RDBMS:', 'Generic JDBC dialect', 'Choose JDBC dialect') + .details-row + +number('Batch size:', pojoStoreFactory + '.batchSize', '"pojoBatchSize"', true, '512', '1', + 'Maximum batch size for writeAll and deleteAll operations') + .details-row + +number('Thread count:', pojoStoreFactory + '.maximumPoolSize', '"pojoMaximumPoolSize"', true, 'availableProcessors', '1', + 'Maximum workers thread count.
          \ + These threads are responsible for load cache.') + .details-row + +number('Maximum write attempts:', pojoStoreFactory + '.maximumWriteAttempts', '"pojoMaximumWriteAttempts"', true, '2', '0', + 'Maximum write attempts in case of database error') + .details-row + +number('Parallel load threshold:', pojoStoreFactory + '.parallelLoadCacheMinimumThreshold', '"pojoParallelLoadCacheMinimumThreshold"', true, '512', '0', + 'Parallel load cache minimum threshold.
          \ + If 0 then load sequentially.') + .details-row + +java-class('Hasher', pojoStoreFactory + '.hasher', '"pojoHasher"', 'true', 'false', 'Hash calculator') + .details-row + +java-class('Transformer', pojoStoreFactory + '.transformer', '"pojoTransformer"', 'true', 'false', 'Types transformer') .details-row +checkbox('Escape table and filed names', pojoStoreFactory + '.sqlEscapeAll', '"sqlEscapeAll"', 'If enabled than all schema, table and field names will be escaped with double quotes (for example: "tableName"."fieldName").
          \ diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade index ef83356527a31..412ca3ad538b8 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'atomics' -var model = 'backupItem.atomicConfiguration' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade index b41b97c2099da..0366ed87deba4 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'attributes' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade index 9994087dde6dd..f8ec8f9375b76 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'binary' -var model = 'backupItem.binaryConfiguration' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade index 45ccc13b385d5..41a72c1c5a1b2 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'cacheKeyCfg' -var model = 'backupItem.cacheKeyConfiguration' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade index d2552a802634b..5cc996d5ff311 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'checkpoint' -var model = 'backupItem.checkpointSpi' @@ -26,7 +26,7 @@ include ../../../../../app/helpers/jade/mixins.jade label Checkpointing ignite-form-field-tooltip.tipLabel | Checkpointing provides an ability to save an intermediate job state#[br] - | #[a(href="http://apacheignite.gridgain.org/docs/checkpointing" target="_blank") More info] + | #[a(href="http://apacheignite.readme.io/docs/checkpointing" target="_blank") More info] ignite-form-revert .panel-collapse(role='tabpanel' bs-collapse-target id=form) .panel-body(ng-if='ui.isPanelLoaded("#{form}")') @@ -61,10 +61,10 @@ include ../../../../../app/helpers/jade/mixins.jade ') +table-remove-button(model, 'Remove Checkpoint SPI') - div(ng-if='model.kind === "FS"') + div(ng-show='model.kind === "FS"') include ./checkpoint/fs.jade - div(ng-if='model.kind === "Cache"') + div(ng-show='model.kind === "Cache"') .settings-row +dropdown-required-empty('Cache:', 'model.Cache.cache', '"checkpointCacheCache"+ $index', 'true', 'true', 'Choose cache', 'No caches configured for current cluster', 'clusterCaches', 'Cache to use for storing checkpoints') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade index 01f14f4613bcc..efb6ad0699260 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'checkpointFsPaths' -var dirPaths = 'model.FS.directoryPaths' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade index a7b217f14240a..874799c17c0b3 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade .settings-row +text('Data source bean name:', 'model.JDBC.dataSourceBean', '"checkpointJdbcDataSourceBean" + $index', 'model.kind === "JDBC"', 'Input bean name', @@ -22,24 +22,25 @@ include ../../../../../../app/helpers/jade/mixins.jade .settings-row +dialect('Dialect:', 'model.JDBC.dialect', '"checkpointJdbcDialect" + $index', 'model.kind === "JDBC"', 'Dialect of SQL implemented by a particular RDBMS:', 'Generic JDBC dialect', 'Choose JDBC dialect') -.settings-row - +text('User:', 'model.JDBC.user', '"checkpointJdbcUser" + $index', 'false', 'Input user name', 'Checkpoint jdbc user name') -.settings-row - +text('Table name:', 'model.JDBC.checkpointTableName', '"checkpointJdbcCheckpointTableName" + $index', 'false', 'CHECKPOINTS', 'Checkpoint table name') -.settings-row - +text('Key field name:', 'model.JDBC.keyFieldName', '"checkpointJdbcKeyFieldName" + $index', 'false', 'NAME', 'Checkpoint key field name') -.settings-row - +dropdown('Key field type:', 'model.JDBC.keyFieldType', '"checkpointJdbcKeyFieldType" + $index', 'true', 'VARCHAR', 'supportedJdbcTypes', 'Checkpoint key field type') -.settings-row - +text('Value field name:', 'model.JDBC.valueFieldName', '"checkpointJdbcValueFieldName" + $index', 'false', 'VALUE', 'Checkpoint value field name') -.settings-row - +dropdown('Value field type:', 'model.JDBC.valueFieldType', '"checkpointJdbcValueFieldType" + $index', 'true', 'BLOB', 'supportedJdbcTypes', 'Checkpoint value field type') -.settings-row - +text('Expire date field name:', 'model.JDBC.expireDateFieldName', '"checkpointJdbcExpireDateFieldName" + $index', 'false', 'EXPIRE_DATE', 'Checkpoint expire date field name') -.settings-row - +dropdown('Expire date field type:', 'model.JDBC.expireDateFieldType', '"checkpointJdbcExpireDateFieldType"', 'true', 'DATETIME', 'supportedJdbcTypes', 'Checkpoint expire date field type') -.settings-row - +number('Number of retries:', 'model.JDBC.numberOfRetries', '"checkpointJdbcNumberOfRetries"', 'true', '2', '0', 'Number of retries in case of DB failure') .settings-row +java-class('Listener:', 'model.JDBC.checkpointListener', '"checkpointJdbcListener" + $index', 'true', 'false', - 'Checkpoint listener implementation class name') + 'Checkpoint listener implementation class name') ++showHideLink('jdbcExpanded', 'settings') + .details-row + +text('User:', 'model.JDBC.user', '"checkpointJdbcUser" + $index', 'false', 'Input user name', 'Checkpoint jdbc user name') + .details-row + +text('Table name:', 'model.JDBC.checkpointTableName', '"checkpointJdbcCheckpointTableName" + $index', 'false', 'CHECKPOINTS', 'Checkpoint table name') + .details-row + +text('Key field name:', 'model.JDBC.keyFieldName', '"checkpointJdbcKeyFieldName" + $index', 'false', 'NAME', 'Checkpoint key field name') + .details-row + +dropdown('Key field type:', 'model.JDBC.keyFieldType', '"checkpointJdbcKeyFieldType" + $index', 'true', 'VARCHAR', 'supportedJdbcTypes', 'Checkpoint key field type') + .details-row + +text('Value field name:', 'model.JDBC.valueFieldName', '"checkpointJdbcValueFieldName" + $index', 'false', 'VALUE', 'Checkpoint value field name') + .details-row + +dropdown('Value field type:', 'model.JDBC.valueFieldType', '"checkpointJdbcValueFieldType" + $index', 'true', 'BLOB', 'supportedJdbcTypes', 'Checkpoint value field type') + .details-row + +text('Expire date field name:', 'model.JDBC.expireDateFieldName', '"checkpointJdbcExpireDateFieldName" + $index', 'false', 'EXPIRE_DATE', 'Checkpoint expire date field name') + .details-row + +dropdown('Expire date field type:', 'model.JDBC.expireDateFieldType', '"checkpointJdbcExpireDateFieldType"', 'true', 'DATETIME', 'supportedJdbcTypes', 'Checkpoint expire date field type') + .details-row + +number('Number of retries:', 'model.JDBC.numberOfRetries', '"checkpointJdbcNumberOfRetries"', 'true', '2', '0', 'Number of retries in case of DB failure') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade index 8373b03ad2c4a..da28da780a155 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var credentialsModel = 'model.S3.awsCredentials' -var clientCfgModel = 'model.S3.clientConfiguration' @@ -42,6 +42,8 @@ include ../../../../../../app/helpers/jade/mixins.jade
        • Database - Session credentials with keys and session token
        • \
        • Custom - Custom AWS credentials provider
        • \ ') +.settings-row + label Note, AWS credentials will be generated as stub .panel-details(ng-show=checkpointS3Path) .details-row +text('Path:', credentialsModel + '.Properties.path', '"checkpointS3PropertiesPath"', checkpointS3Path, 'Input properties file path', @@ -53,122 +55,123 @@ include ../../../../../../app/helpers/jade/mixins.jade .settings-row +text('Bucket name suffix:', 'model.S3.bucketNameSuffix', '"checkpointS3BucketNameSuffix"', 'false', 'default-bucket', 'Bucket name suffix') .settings-row - +dropdown('Protocol:', clientCfgModel + '.protocol', '"checkpointS3Protocol"', 'true', 'HTTPS', '[\ - {value: "HTTP", label: "HTTP"},\ - {value: "HTTPS", label: "HTTPS"}\ + +java-class('Listener:', 'model.S3.checkpointListener', '"checkpointS3Listener" + $index', 'true', 'false', + 'Checkpoint listener implementation class name') ++showHideLink('s3Expanded', 'client configuration') + .details-row + +dropdown('Protocol:', clientCfgModel + '.protocol', '"checkpointS3Protocol"', 'true', 'HTTPS', '[\ + {value: "HTTP", label: "HTTP"},\ + {value: "HTTPS", label: "HTTPS"}\ + ]', + 'Provides an ability to save an intermediate job state\ +
            \ +
          • HTTP - Using the HTTP protocol is less secure than HTTPS, but can slightly reduce\ + the system resources used when communicating with AWS
          • \ +
          • HTTPS - Using the HTTPS protocol is more secure than using the HTTP protocol, but\ + may use slightly more system resources. AWS recommends using HTTPS for maximize security
          • \ +
          ') + .details-row + +number('Maximum connections:', clientCfgModel + '.maxConnections', '"checkpointS3MaxConnections"', + 'true', '50', '1', 'Maximum number of allowed open HTTP connections') + .details-row + +text('User agent:', clientCfgModel + '.userAgent', '"checkpointS3UserAgent"', 'false', 'System specific header', + 'HTTP user agent header to send with all requests') + .details-row + +text-ip-address('Local address:', clientCfgModel + '.localAddress', '"checkpointS3LocalAddress"', 'true', 'Not specified', + 'Optionally specifies the local address to bind to') + .details-row + +text('Proxy host:', clientCfgModel + '.proxyHost', '"checkpointS3ProxyHost"', 'false', 'Not specified', + 'Optional proxy host the client will connect through') + .details-row + +number('Proxy port:', clientCfgModel + '.proxyPort', '"checkpointS3ProxyPort"', 'true', 'Not specified', '0', + 'Optional proxy port the client will connect through') + .details-row + +text('Proxy user:', clientCfgModel + '.proxyUsername', '"checkpointS3ProxyUsername"', 'false', 'Not specified', + 'Optional proxy user name to use if connecting through a proxy') + .details-row + +text('Proxy domain:', clientCfgModel + '.proxyDomain', '"checkpointS3ProxyDomain"', 'false', 'Not specified', + 'Optional Windows domain name for configuring an NTLM proxy') + .details-row + +text('Proxy workstation:', clientCfgModel + '.proxyWorkstation', '"checkpointS3ProxyWorkstation"', 'false', 'Not specified', + 'Optional Windows workstation name for configuring NTLM proxy support') + .details-row + +dropdown('Retry policy:', clientRetryModel + '.kind', '"checkpointS3RetryPolicy"', 'true', 'Default', '[\ + {value: "Default", label: "Default SDK retry policy"},\ + {value: "DefaultMaxRetries", label: "Default with the specified max retry count"},\ + {value: "DynamoDB", label: "Default for DynamoDB client"},\ + {value: "DynamoDBMaxRetries", label: "DynamoDB with the specified max retry count"},\ + {value: "Custom", label: "Custom configured"}\ ]', - 'Provides an ability to save an intermediate job state\ -
            \ -
          • HTTP - Using the HTTP protocol is less secure than HTTPS, but can slightly reduce\ - the system resources used when communicating with AWS
          • \ -
          • HTTPS - Using the HTTPS protocol is more secure than using the HTTP protocol, but\ - may use slightly more system resources. AWS recommends using HTTPS for maximize security
          • \ -
          ') -.settings-row - +number('Maximum connections:', clientCfgModel + '.maxConnections', '"checkpointS3MaxConnections"', - 'true', '50', '1', 'Maximum number of allowed open HTTP connections') -.settings-row - +text('User agent:', clientCfgModel + '.userAgent', '"checkpointS3UserAgent"', 'false', 'System specific header', - 'HTTP user agent header to send with all requests') -.settings-row - +text-ip-address('Local address:', clientCfgModel + '.localAddress', '"checkpointS3LocalAddress"', 'true', 'Not specified', - 'Optionally specifies the local address to bind to') -.settings-row - +text('Proxy host:', clientCfgModel + '.proxyHost', '"checkpointS3ProxyHost"', 'false', 'Not specified', - 'Optional proxy host the client will connect through') -.settings-row - +number('Proxy port:', clientCfgModel + '.proxyPort', '"checkpointS3ProxyPort"', 'true', 'Not specified', '0', - 'Optional proxy port the client will connect through') -.settings-row - +text('Proxy user:', clientCfgModel + '.proxyUsername', '"checkpointS3ProxyUsername"', 'false', 'Not specified', - 'Optional proxy user name to use if connecting through a proxy') -.settings-row - +text('Proxy domain:', clientCfgModel + '.proxyDomain', '"checkpointS3ProxyDomain"', 'false', 'Not specified', - 'Optional Windows domain name for configuring an NTLM proxy') -.settings-row - +text('Proxy workstation:', clientCfgModel + '.proxyWorkstation', '"checkpointS3ProxyWorkstation"', 'false', 'Not specified', - 'Optional Windows workstation name for configuring NTLM proxy support') -.settings-row - +dropdown('Retry policy:', clientRetryModel + '.kind', '"checkpointS3RetryPolicy"', 'true', 'Default', '[\ - {value: "Default", label: "Default SDK retry policy"},\ - {value: "DefaultMaxRetries", label: "Default with the specified max retry count"},\ - {value: "DynamoDB", label: "Default for DynamoDB client"},\ - {value: "DynamoDBMaxRetries", label: "DynamoDB with the specified max retry count"},\ - {value: "Custom", label: "Custom configured"}\ - ]', - 'Provides an ability to save an intermediate job state\ -
            \ -
          • SDK default retry policy - This policy will honor the maxErrorRetry set in ClientConfiguration
          • \ -
          • Default with the specified max retry count - Default SDK retry policy with the specified max retry count
          • \ -
          • Default for DynamoDB client - This policy will honor the maxErrorRetry set in ClientConfiguration
          • \ -
          • DynamoDB with the specified max retry count - This policy will honor the maxErrorRetry set in ClientConfiguration with the specified max retry count
          • \ -
          • Custom configured - Custom configured SDK retry policy
          • \ -
          ') -.panel-details(ng-show=checkpointS3DefaultMaxRetry) + 'Provides an ability to save an intermediate job state\ +
            \ +
          • SDK default retry policy - This policy will honor the maxErrorRetry set in ClientConfiguration
          • \ +
          • Default with the specified max retry count - Default SDK retry policy with the specified max retry count
          • \ +
          • Default for DynamoDB client - This policy will honor the maxErrorRetry set in ClientConfiguration
          • \ +
          • DynamoDB with the specified max retry count - This policy will honor the maxErrorRetry set in ClientConfiguration with the specified max retry count
          • \ +
          • Custom configured - Custom configured SDK retry policy
          • \ +
          ') + .panel-details(ng-show=checkpointS3DefaultMaxRetry) + .details-row + +number-required('Maximum retry attempts:', clientRetryModel + '.DefaultMaxRetries.maxErrorRetry', '"checkpointS3DefaultMaxErrorRetry"', 'true', checkpointS3DefaultMaxRetry, '-1', '1', + 'Maximum number of retry attempts for failed requests') + .panel-details(ng-show=checkpointS3DynamoDbMaxRetry) + .details-row + +number-required('Maximum retry attempts:', clientRetryModel + '.DynamoDBMaxRetries.maxErrorRetry', '"checkpointS3DynamoDBMaxErrorRetry"', 'true', checkpointS3DynamoDbMaxRetry, '-1', '1', + 'Maximum number of retry attempts for failed requests') + .panel-details(ng-show=checkpointS3CustomRetry) + .details-row + +java-class('Retry condition:', clientRetryModel + '.Custom.retryCondition', '"checkpointS3CustomRetryPolicy" + $index', 'true', checkpointS3CustomRetry, + 'Retry condition on whether a specific request and exception should be retried') + .details-row + +java-class('Backoff strategy:', clientRetryModel + '.Custom.backoffStrategy', '"checkpointS3CustomBackoffStrategy" + $index', 'true', checkpointS3CustomRetry, + 'Back-off strategy for controlling how long the next retry should wait') + .details-row + +number-required('Maximum retry attempts:', clientRetryModel + '.Custom.maxErrorRetry', '"checkpointS3CustomMaxErrorRetry"', 'true', checkpointS3CustomRetry, '-1', '1', + 'Maximum number of retry attempts for failed requests') + .details-row + +checkbox('Honor the max error retry set', clientRetryModel + '.Custom.honorMaxErrorRetryInClientConfig', '"checkpointS3CustomHonorMaxErrorRetryInClientConfig"', + 'Whether this retry policy should honor the max error retry set by ClientConfiguration#setMaxErrorRetry(int)') .details-row - +number-required('Maximum retry attempts:', clientRetryModel + '.DefaultMaxRetries.maxErrorRetry', '"checkpointS3DefaultMaxErrorRetry"', 'true', checkpointS3DefaultMaxRetry, '-1', '1', - 'Maximum number of retry attempts for failed requests') -.panel-details(ng-show=checkpointS3DynamoDbMaxRetry) + +number('Maximum retry attempts:', clientCfgModel + '.maxErrorRetry', '"checkpointS3MaxErrorRetry"', 'true', '-1', '0', + 'Maximum number of retry attempts for failed retryable requests
          \ + If -1 the configured RetryPolicy will be used to control the retry count') .details-row - +number-required('Maximum retry attempts:', clientRetryModel + '.DynamoDBMaxRetries.maxErrorRetry', '"checkpointS3DynamoDBMaxErrorRetry"', 'true', checkpointS3DynamoDbMaxRetry, '-1', '1', - 'Maximum number of retry attempts for failed requests') -.panel-details(ng-show=checkpointS3CustomRetry) + +number('Socket timeout:', clientCfgModel + '.socketTimeout', '"checkpointS3SocketTimeout"', 'true', '50000', '0', + 'Amount of time in milliseconds to wait for data to be transfered over an established, open connection before the connection times out and is closed
          \ + A value of 0 means infinity') .details-row - +java-class('Retry condition:', clientRetryModel + '.Custom.retryCondition', '"checkpointS3CustomRetryPolicy" + $index', 'true', checkpointS3CustomRetry, - 'Retry condition on whether a specific request and exception should be retried') + +number('Connection timeout:', clientCfgModel + '.connectionTimeout', '"checkpointS3ConnectionTimeout"', 'true', '50000', '0', + 'Amount of time in milliseconds to wait when initially establishing a connection before giving up and timing out
          \ + A value of 0 means infinity') .details-row - +java-class('Backoff strategy:', clientRetryModel + '.Custom.backoffStrategy', '"checkpointS3CustomBackoffStrategy" + $index', 'true', checkpointS3CustomRetry, - 'Back-off strategy for controlling how long the next retry should wait') + +number('Request timeout:', clientCfgModel + '.requestTimeout', '"checkpointS3RequestTimeout"', 'true', '0', '-1', + 'Amount of time in milliseconds to wait for the request to complete before giving up and timing out
          \ + A non - positive value means infinity') .details-row - +number-required('Maximum retry attempts:', clientRetryModel + '.Custom.maxErrorRetry', '"checkpointS3CustomMaxErrorRetry"', 'true', checkpointS3CustomRetry, '-1', '1', - 'Maximum number of retry attempts for failed requests') + +text('Signature algorithm:', clientCfgModel + '.signerOverride', '"checkpointS3SignerOverride"', 'false', 'Not specified', + 'Name of the signature algorithm to use for signing requests made by this client') .details-row - +checkbox('Honor the max error retry set', clientRetryModel + '.Custom.honorMaxErrorRetryInClientConfig', '"checkpointS3CustomHonorMaxErrorRetryInClientConfig"', - 'Whether this retry policy should honor the max error retry set by ClientConfiguration#setMaxErrorRetry(int)') -.settings-row - +number('Maximum retry attempts:', clientCfgModel + '.maxErrorRetry', '"checkpointS3MaxErrorRetry"', 'true', '-1', '0', - 'Maximum number of retry attempts for failed retryable requests
          \ - If -1 the configured RetryPolicy will be used to control the retry count') -.settings-row - +number('Socket timeout:', clientCfgModel + '.socketTimeout', '"checkpointS3SocketTimeout"', 'true', '50000', '0', - 'Amount of time in milliseconds to wait for data to be transfered over an established, open connection before the connection times out and is closed
          \ - A value of 0 means infinity') -.settings-row - +number('Connection timeout:', clientCfgModel + '.connectionTimeout', '"checkpointS3ConnectionTimeout"', 'true', '50000', '0', - 'Amount of time in milliseconds to wait when initially establishing a connection before giving up and timing out
          \ - A value of 0 means infinity') -.settings-row - +number('Request timeout:', clientCfgModel + '.requestTimeout', '"checkpointS3RequestTimeout"', 'true', '0', '-1', - 'Amount of time in milliseconds to wait for the request to complete before giving up and timing out
          \ - A non - positive value means infinity') -.settings-row - +text('Signature algorithm:', clientCfgModel + '.signerOverride', '"checkpointS3SignerOverride"', 'false', 'Not specified', - 'Name of the signature algorithm to use for signing requests made by this client') -.settings-row - +number('Connection TTL:', clientCfgModel + '.connectionTTL', '"checkpointS3ConnectionTTL"', 'true', '-1', '-1', - 'Expiration time in milliseconds for a connection in the connection pool
          \ - By default, it is set to -1, i.e. connections do not expire') -.settings-row - +number('Idle timeout:', clientCfgModel + '.connectionMaxIdleMillis', '"checkpointS3ConnectionMaxIdleMillis"', 'true', '60000', '0', - 'Maximum amount of time that an idle connection may sit in the connection pool and still be eligible for reuse') -.settings-row - +java-class('DNS resolver:', clientCfgModel + '.dnsResolver', '"checkpointS3DnsResolver" + $index', 'true', 'false', - 'DNS Resolver that should be used to for resolving AWS IP addresses') -.settings-row - +number('Response metadata cache size:', clientCfgModel + '.responseMetadataCacheSize', '"checkpointS3ResponseMetadataCacheSize"', 'true', '50', '0', - 'Response metadata cache size') -.settings-row - +java-class('SecureRandom class name:', clientCfgModel + '.secureRandom', '"checkpointS3SecureRandom" + $index', 'true', 'false', - 'SecureRandom to be used by the SDK class name') -.settings-row - +checkbox('Use reaper', clientCfgModel + '.useReaper', '"checkpointS3UseReaper"', 'Checks if the IdleConnectionReaper is to be started') -.settings-row - +checkbox('Use GZIP', clientCfgModel + '.useGzip', '"checkpointS3UseGzip"', 'Checks if gzip compression is used') -.settings-row - +checkbox('Preemptively basic authentication', clientCfgModel + '.preemptiveBasicProxyAuth', '"checkpointS3PreemptiveBasicProxyAuth"', - 'Attempt to authenticate preemptively against proxy servers using basic authentication') -.settings-row - +checkbox('TCP KeepAlive', clientCfgModel + '.useTcpKeepAlive', '"checkpointS3UseTcpKeepAlive"', 'TCP KeepAlive support is enabled') -.settings-row - +java-class('Listener:', 'model.S3.checkpointListener', '"checkpointS3Listener" + $index', 'true', 'false', - 'Checkpoint listener implementation class name') + +number('Connection TTL:', clientCfgModel + '.connectionTTL', '"checkpointS3ConnectionTTL"', 'true', '-1', '-1', + 'Expiration time in milliseconds for a connection in the connection pool
          \ + By default, it is set to -1, i.e. connections do not expire') + .details-row + +number('Idle timeout:', clientCfgModel + '.connectionMaxIdleMillis', '"checkpointS3ConnectionMaxIdleMillis"', 'true', '60000', '0', + 'Maximum amount of time that an idle connection may sit in the connection pool and still be eligible for reuse') + .details-row + +java-class('DNS resolver:', clientCfgModel + '.dnsResolver', '"checkpointS3DnsResolver" + $index', 'true', 'false', + 'DNS Resolver that should be used to for resolving AWS IP addresses') + .details-row + +number('Response metadata cache size:', clientCfgModel + '.responseMetadataCacheSize', '"checkpointS3ResponseMetadataCacheSize"', 'true', '50', '0', + 'Response metadata cache size') + .details-row + +java-class('SecureRandom class name:', clientCfgModel + '.secureRandom', '"checkpointS3SecureRandom" + $index', 'true', 'false', + 'SecureRandom to be used by the SDK class name') + .details-row + +checkbox('Use reaper', clientCfgModel + '.useReaper', '"checkpointS3UseReaper"', 'Checks if the IdleConnectionReaper is to be started') + .details-row + +checkbox('Use GZIP', clientCfgModel + '.useGzip', '"checkpointS3UseGzip"', 'Checks if gzip compression is used') + .details-row + +checkbox('Preemptively basic authentication', clientCfgModel + '.preemptiveBasicProxyAuth', '"checkpointS3PreemptiveBasicProxyAuth"', + 'Attempt to authenticate preemptively against proxy servers using basic authentication') + .details-row + +checkbox('TCP KeepAlive', clientCfgModel + '.useTcpKeepAlive', '"checkpointS3UseTcpKeepAlive"', 'TCP KeepAlive support is enabled') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade index 1a5d6d6eb0ee8..91676df7cc1be 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'collision' -var model = 'backupItem.collision' @@ -48,15 +48,15 @@ include ../../../../../app/helpers/jade/mixins.jade
        • Custom - custom CollisionSpi implementation
        • \
        • Default - jobs are activated immediately on arrival to mapped node
        • \ ') - .settings-row(ng-if='#{modelCollisionKind} !== "Noop"') + .settings-row(ng-show='#{modelCollisionKind} !== "Noop"') .panel-details - div(ng-if='#{modelCollisionKind} === "JobStealing"') + div(ng-show='#{modelCollisionKind} === "JobStealing"') include ./collision/job-stealing.jade - div(ng-if='#{modelCollisionKind} === "FifoQueue"') + div(ng-show='#{modelCollisionKind} === "FifoQueue"') include ./collision/fifo-queue.jade - div(ng-if='#{modelCollisionKind} === "PriorityQueue"') + div(ng-show='#{modelCollisionKind} === "PriorityQueue"') include ./collision/priority-queue.jade - div(ng-if='#{modelCollisionKind} === "Custom"') + div(ng-show='#{modelCollisionKind} === "Custom"') include ./collision/custom.jade .col-sm-6 -var model = 'backupItem.collision' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade index 923891764c06f..31a6be7e237a2 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem.collision.Custom' -var required = 'backupItem.collision.kind === "Custom"' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/fifo-queue.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/fifo-queue.jade index f16363d54fa87..cd8b6a3c92e5b 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/fifo-queue.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/fifo-queue.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem.collision.FifoQueue' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade index 365911554d437..d4e537aff6d7f 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem.collision.JobStealing' -var stealingAttributes = model + '.stealingAttributes' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/priority-queue.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/priority-queue.jade index bdd1daca60701..6f52ee0f07f19 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/priority-queue.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/priority-queue.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem.collision.PriorityQueue' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade index 55bf909e5b271..047c9a28f5309 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'communication' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade index 6e7528331e0e1..40d65af250fce 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'connector' -var model = 'backupItem.connector' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade index 67b6b4f368445..4cfd9f52aef7e 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'deployment' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade index c9a2be3168245..3bf0e29440050 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'discovery' -var model = 'backupItem.discovery' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade index 42c319c4aa5ee..3f2d6cb74ff12 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'events' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade index b29701d38820c..aaed8e9a3b07e 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem' -var form = 'failoverSpi' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade index 8ec9b1383f677..b2ce71fa14e53 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'general' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/cloud.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/cloud.jade index 640c78c5aafee..52fb21b619ae7 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/cloud.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/cloud.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var discoveryKind = 'Cloud' -var required = 'backupItem.discovery.kind == "' + discoveryKind + '"' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/google.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/google.jade index b1a595806f934..c12bd04c8f55c 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/google.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/google.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var discoveryKind = 'GoogleStorage' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.jade index 42cf69707ac23..4e4246db22f61 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.jade @@ -14,8 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../app/helpers/jade/mixins.jade -include ../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem.discovery.Jdbc' -var required = 'backupItem.discovery.kind === "Jdbc"' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/multicast.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/multicast.jade index 829fbaa4cf72e..6a7e9faf80a51 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/multicast.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/multicast.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'general' -var model = 'backupItem.discovery.Multicast' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.jade index e255f20547508..e5eaff342adbd 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var discoveryKind = 'S3' -var required = 'backupItem.discovery.kind == "' + discoveryKind + '"' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/shared.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/shared.jade index 2a949a5cbc363..ddd9bfa9b1f05 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/shared.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/shared.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem.discovery.SharedFs' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.jade index 6588e6a9e1b0c..46ebae005833d 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem.discovery.Vm' -var addresses = model + '.addresses' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade index da369937a553f..2e567ed509a67 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'general' -var discoveryKind = 'ZooKeeper' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade index 5e4bbda3e44d7..d3c1f9fc88fbc 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem.discovery.ZooKeeper.retryPolicy.BoundedExponentialBackoff' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade index 3dcf514a1b94c..5a03de84fe98d 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem.discovery.ZooKeeper.retryPolicy' -var retry = model + '.Custom' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade index 618683cee7a86..0a072f73301f3 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem.discovery.ZooKeeper.retryPolicy.ExponentialBackoff' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade index 10e4fb19d62b9..a7bfd20b42171 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem.discovery.ZooKeeper.retryPolicy.Forever' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade index 329f693e01b13..1746232124ee8 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem.discovery.ZooKeeper.retryPolicy.NTimes' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade index 938eceadb1307..5a7e3fe28ccec 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem.discovery.ZooKeeper.retryPolicy.OneTime' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade index 513da6a14476e..548211c4ed2a9 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem.discovery.ZooKeeper.retryPolicy.UntilElapsed' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade index 7eb150558daa2..25bd5b0193168 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'igfs' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade index 916504bc91c95..7fd78bf266771 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var model = 'backupItem' -var form = 'loadBalancing' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger.jade index 31f87a9006dc4..983d81855cba6 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'logger' -var model = 'backupItem.logger' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade index df80af0a98b31..385d647568317 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'logger' -var model = 'backupItem.logger.Custom' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j.jade index cf556ecb0dfe8..1eaa52988955b 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'logger' -var model = 'backupItem.logger.Log4j' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j2.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j2.jade index 8b9d3e197c12b..17df4811ff019 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j2.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j2.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'logger' -var model = 'backupItem.logger.Log4j2' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade index 3f3661cd93b8c..9316e7962fbed 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'marshaller' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/metrics.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/metrics.jade index 2b0dfd6ab6fd2..d5212a94402fe 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/metrics.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/metrics.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'metrics' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade index dd46d2abed19d..101a11c11caac 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'odbcConfiguration' -var model = 'backupItem.odbc' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade index 0a51cf7b57b1f..85ec073bf61fd 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'sslConfiguration' -var cluster = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade index c76391c6ac0e6..8071f3c282a6b 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'swap' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/thread.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/thread.jade index 9669a87b2791a..d2bea86587db1 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/thread.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/thread.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'pools' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/time.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/time.jade index 813948dfb35ac..bcb1cb71f5b53 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/time.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/time.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'time' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade index 294615895e9d9..6a18bb0e6a6f2 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'transactions' -var model = 'backupItem.transactionConfiguration' diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade b/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade index bf21ee11399b8..40759e30c0748 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'general' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade b/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade index 45051b23e8768..8ed7b6839c2c7 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'query' -var model = 'backupItem' @@ -41,15 +41,15 @@ mixin table-index-item-edit(prefix, index, sortAvailable, idAddition) .col-xs-8.col-sm-8.col-md-8 label.fieldSep / .input-tip - button.select-toggle.form-control(id='{{::"#{fieldName}" + #{idAddition}}}' ignite-on-enter-focus-move='{{::"#{direction}S" + #{idAddition}}}' ng-model=fieldNameModel placeholder='{{fields("#{prefix}", #{fieldNameModel}).length > 0 ? "Choose field" : "No fields configured"}}' bs-select bs-options='item.value as item.label for item in fields("#{prefix}", #{fieldNameModel})' ng-disabled='fields("#{prefix}", #{fieldNameModel}).length === 0' ignite-on-escape='tableReset()' tabindex='0') + button.select-toggle.form-control(id='{{::"#{fieldName}" + #{idAddition}}}' ignite-on-enter-focus-move='{{::"#{direction}S" + #{idAddition}}}' ng-model=fieldNameModel placeholder='{{fields("#{prefix}", #{fieldNameModel}).length > 0 ? "Choose field" : "No fields configured"}}' bs-select bs-options='item.value as item.label for item in fields("#{prefix}", #{fieldNameModel})' ng-disabled='fields("#{prefix}", #{fieldNameModel}).length === 0' ignite-on-escape='tableReset(false)' tabindex='0') .col-xs-4.col-sm-4.col-md-4 +btn-save(btnVisible, btnSave) .input-tip - button.select-toggle.form-control(id='{{::"#{direction}" + #{idAddition}}}' ng-model=directionModel bs-select bs-options='item.value as item.label for item in {{sortDirections}}' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()' tabindex='0') + button.select-toggle.form-control(id='{{::"#{direction}" + #{idAddition}}}' ng-model=directionModel bs-select bs-options='item.value as item.label for item in {{sortDirections}}' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)' tabindex='0') .col-xs-12(ng-if='!(#{sortAvailable})') +btn-save(btnVisible, btnSave) .input-tip - button.select-toggle.form-control(id='{{::"#{fieldName}" + #{idAddition}}}' ng-model=fieldNameModel placeholder='{{fields("#{prefix}", #{fieldNameModel}).length > 0 ? "Choose index field" : "No fields configured"}}' bs-select bs-options='item.value as item.label for item in fields("#{prefix}", #{fieldNameModel})' ng-disabled='fields("#{prefix}", #{fieldNameModel}).length === 0' ignite-on-escape='tableReset()' tabindex='0') + button.select-toggle.form-control(id='{{::"#{fieldName}" + #{idAddition}}}' ng-model=fieldNameModel placeholder='{{fields("#{prefix}", #{fieldNameModel}).length > 0 ? "Choose index field" : "No fields configured"}}' bs-select bs-options='item.value as item.label for item in fields("#{prefix}", #{fieldNameModel})' ng-disabled='fields("#{prefix}", #{fieldNameModel}).length === 0' ignite-on-escape='tableReset(false)' tabindex='0') .panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle) @@ -137,11 +137,11 @@ mixin table-index-item-edit(prefix, index, sortAvailable, idAddition) .col-sm-7 label.fieldSep / .input-tip - input.form-control(id='curIndexName{{$index}}' type='text' ignite-on-enter-focus-move='curIndexType{{$index}}' ng-model='indexesTbl.curIndexName' placeholder='Index name' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()') + input.form-control(id='curIndexName{{$index}}' type='text' ignite-on-enter-focus-move='curIndexType{{$index}}' ng-model='indexesTbl.curIndexName' placeholder='Index name' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)') .col-sm-5 +btn-save('tableIndexSaveVisible(indexesTbl, $index)', 'tableIndexSave(indexesTbl, $index)') .input-tip - button.select-toggle.form-control(id='curIndexType{{$index}}' bs-select ng-model='indexesTbl.curIndexType' data-placeholder='Select index type' bs-options='item.value as item.label for item in indexType' tabindex='0' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()') + button.select-toggle.form-control(id='curIndexType{{$index}}' bs-select ng-model='indexesTbl.curIndexType' data-placeholder='Select index type' bs-options='item.value as item.label for item in indexType' tabindex='0' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)') .margin-left-dflt table.links-edit-sub(st-table='item.fields' ng-init='itemIndex = $index') tbody @@ -163,10 +163,10 @@ mixin table-index-item-edit(prefix, index, sortAvailable, idAddition) .col-sm-7 .fieldSep / .input-tip - input#newIndexName.form-control(type='text' ignite-on-enter-focus-move='newIndexType' ng-model='indexesTbl.newIndexName' placeholder='Index name' ignite-on-enter='tableIndexSaveVisible(indexesTbl, -1) && tableIndexSave(indexesTbl, -1)' ignite-on-escape='tableReset()') + input#newIndexName.form-control(type='text' ignite-on-enter-focus-move='newIndexType' ng-model='indexesTbl.newIndexName' placeholder='Index name' ignite-on-enter='tableIndexSaveVisible(indexesTbl, -1) && tableIndexSave(indexesTbl, -1)' ignite-on-escape='tableReset(false)') .col-sm-5 +btn-save('tableIndexSaveVisible(indexesTbl, -1)', 'tableIndexSave(indexesTbl, -1)') .input-tip - button#newIndexType.select-toggle.form-control(bs-select ng-model='indexesTbl.newIndexType' data-placeholder='Select index type' bs-options='item.value as item.label for item in indexType' tabindex='0' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()') + button#newIndexType.select-toggle.form-control(bs-select ng-model='indexesTbl.newIndexType' data-placeholder='Select index type' bs-options='item.value as item.label for item in indexType' tabindex='0' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)') .col-sm-6 +preview-xml-java(model, 'domainModelQuery') diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade b/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade index ab569e706a5e1..f5d6b7d075e5f 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'store' -var model = 'backupItem' @@ -43,15 +43,15 @@ mixin table-db-field-edit(tbl, prefix, focusId, index) .col-xs-3.col-sm-3.col-md-3 .fieldSep / .input-tip - input.form-control(id=databaseNameId ignite-on-enter-focus-move=databaseTypeId type='text' ng-model=databaseNameModel placeholder='DB name' ignite-on-enter='#{javaNameModel} = #{javaNameModel} ? #{javaNameModel} : #{databaseNameModel}' ignite-on-escape='tableReset()') + input.form-control(id=databaseNameId ignite-on-enter-focus-move=databaseTypeId type='text' ng-model=databaseNameModel placeholder='DB name' ignite-on-enter='#{javaNameModel} = #{javaNameModel} ? #{javaNameModel} : #{databaseNameModel}' ignite-on-escape='tableReset(false)') .col-xs-3.col-sm-3.col-md-3 .fieldSep / .input-tip - button.select-toggle.form-control(id=databaseTypeId ignite-on-enter-focus-move=javaNameId ng-model=databaseTypeModel data-placeholder='DB type' ng-class='{placeholder: !#{databaseTypeModel}}' bs-select bs-options='item.value as item.label for item in {{supportedJdbcTypes}}' ignite-on-escape='tableReset()' tabindex='0') + button.select-toggle.form-control(id=databaseTypeId ignite-on-enter-focus-move=javaNameId ng-model=databaseTypeModel data-placeholder='DB type' ng-class='{placeholder: !#{databaseTypeModel}}' bs-select bs-options='item.value as item.label for item in {{supportedJdbcTypes}}' ignite-on-escape='tableReset(false)' tabindex='0') .col-xs-3.col-sm-3.col-md-3 .fieldSep / .input-tip - input.form-control(id=javaNameId ignite-on-enter-focus-move=javaTypeId type='text' ng-model=javaNameModel placeholder='Java name' ignite-on-escape='tableReset()') + input.form-control(id=javaNameId ignite-on-enter-focus-move=javaTypeId type='text' ng-model=javaNameModel placeholder='Java name' ignite-on-escape='tableReset(false)') .col-xs-3.col-sm-3.col-md-3 -var btnVisible = 'tableDbFieldSaveVisible(' + tbl + ', ' + index +')' -var btnSave = 'tableDbFieldSave(' + tbl + ', ' + index +')' @@ -59,12 +59,12 @@ mixin table-db-field-edit(tbl, prefix, focusId, index) +btn-save(btnVisible, btnSave) .input-tip - button.select-toggle.form-control(id=javaTypeId ng-model=javaTypeModel data-placeholder='Java type' ng-class='{placeholder: !#{javaTypeModel}}' bs-select bs-options='item.value as item.label for item in {{supportedJavaTypes}}' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()' tabindex='0') + button.select-toggle.form-control(id=javaTypeId ng-model=javaTypeModel data-placeholder='Java type' ng-class='{placeholder: !#{javaTypeModel}}' bs-select bs-options='item.value as item.label for item in {{supportedJavaTypes}}' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)' tabindex='0') .panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') ignite-form-panel-chevron - label Domain model for cache store + label(id='store-title') Domain model for cache store ignite-form-field-tooltip.tipLabel | Domain model properties for binding database with cache via POJO cache store#[br] | #[a(href="https://apacheignite.readme.io/docs/persistent-store" target="_blank") More info] diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/dual.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/dual.jade index 1b3a00a979eb4..f6ac89f223102 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/igfs/dual.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/dual.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'dualMode' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/fragmentizer.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/fragmentizer.jade index edc235200e98a..16f3749103904 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/igfs/fragmentizer.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/fragmentizer.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'fragmentizer' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade index d12a6e8e17fdd..62cda770ceddc 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'general' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade index f8da2f9d05c8e..83c5de6eae68a 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'ipc' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade index ca22dc5a20fe8..dc0e9fc73991c 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'misc' -var model = 'backupItem' @@ -32,7 +32,7 @@ mixin table-igfs-path-mode-edit(prefix, focusId, index) .col-xs-8.col-sm-8.col-md-8 .fieldSep / .input-tip - input.form-control(id=keyFocusId ignite-on-enter-focus-move=valFocusId type='text' ng-model=keyModel placeholder='Path' ignite-on-escape='tableReset()') + input.form-control(id=keyFocusId ignite-on-enter-focus-move=valFocusId type='text' ng-model=keyModel placeholder='Path' ignite-on-escape='tableReset(false)') .col-xs-4.col-sm-4.col-md-4 -var arg = keyModel + ', ' + valModel -var btnVisible = 'tablePairSaveVisible(tblPathModes, ' + index + ')' @@ -40,7 +40,7 @@ mixin table-igfs-path-mode-edit(prefix, focusId, index) -var btnVisibleAndSave = btnVisible + ' && ' + btnSave +btn-save(btnVisible, btnSave) .input-tip - button.select-toggle.form-control(id=valFocusId bs-select ng-model=valModel data-placeholder='Mode' bs-options='item.value as item.label for item in igfsModes' tabindex='0' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()') + button.select-toggle.form-control(id=valFocusId bs-select ng-model=valModel data-placeholder='Mode' bs-options='item.value as item.label for item in igfsModes' tabindex='0' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)') .panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")') diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade index 114366217659d..b605e7775a632 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../../../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade -var form = 'secondaryFileSystem' -var model = 'backupItem' diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js index 4bfbf486b3763..d739c4349a99b 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js +++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js @@ -223,6 +223,10 @@ export default [ return false; } + function escapeFileName(name) { + return name.replace(/[\\\/*\"\[\],\.:;|=<>?]/g, '-').replace(/ /g, '_'); + } + $scope.selectItem = (cluster) => { delete ctrl.cluster; @@ -318,8 +322,8 @@ export default [ const srcPath = 'src/main/java'; const resourcesPath = 'src/main/resources'; - const serverXml = `${cluster.name}-server.xml`; - const clientXml = `${cluster.name}-client.xml`; + const serverXml = `${escapeFileName(cluster.name)}-server.xml`; + const clientXml = `${escapeFileName(cluster.name)}-client.xml`; const metaPath = `${resourcesPath}/META-INF`; @@ -329,7 +333,7 @@ export default [ const cfgPath = `${srcPath}/config`; zip.file(`${cfgPath}/ServerConfigurationFactory.java`, java.igniteConfiguration(cfg, 'config', 'ServerConfigurationFactory').asString()); - zip.file(`${cfgPath}/ClientConfigurationFactory.java`, java.igniteConfiguration(cfg, 'config', 'ClientConfigurationFactory', clientNearCaches).asString()); + zip.file(`${cfgPath}/ClientConfigurationFactory.java`, java.igniteConfiguration(clientCfg, 'config', 'ClientConfigurationFactory', clientNearCaches).asString()); if (java.isDemoConfigured(cluster, $root.IgniteDemoMode)) { zip.file(`${srcPath}/demo/DemoStartup.java`, java.nodeStartup(cluster, 'demo.DemoStartup', @@ -370,7 +374,7 @@ export default [ $generatorOptional.optionalContent(zip, cluster); zip.generateAsync({type: 'blob', compression: 'DEFLATE', mimeType: 'application/octet-stream'}) - .then((blob) => saver.saveAs(blob, cluster.name + '-project.zip')); + .then((blob) => saver.saveAs(blob, escapeFileName(cluster.name) + '-project.zip')); }; /** diff --git a/modules/web-console/frontend/controllers/clusters-controller.js b/modules/web-console/frontend/controllers/clusters-controller.js index f9096f7f61618..f92a2f1e100fa 100644 --- a/modules/web-console/frontend/controllers/clusters-controller.js +++ b/modules/web-console/frontend/controllers/clusters-controller.js @@ -594,6 +594,13 @@ export default ['clustersController', [ })); } + function checkODBC(item) { + if (_.get(item, 'odbc.odbcEnabled') && _.get(item, 'marshaller.kind')) + return ErrorPopover.show('odbcEnabledInput', 'ODBC can only be used with BinaryMarshaller', $scope.ui, 'odbcConfiguration'); + + return true; + } + function checkSwapConfiguration(item) { const swapKind = item.swapSpaceSpi && item.swapSpaceSpi.kind; @@ -674,6 +681,9 @@ export default ['clustersController', [ if (!checkLoadBalancingConfiguration(item)) return false; + if (!checkODBC(item)) + return false; + if (!checkSwapConfiguration(item)) return false; diff --git a/modules/web-console/frontend/controllers/domains-controller.js b/modules/web-console/frontend/controllers/domains-controller.js index 0a79d82603e69..2d7b875e36f5b 100644 --- a/modules/web-console/frontend/controllers/domains-controller.js +++ b/modules/web-console/frontend/controllers/domains-controller.js @@ -95,7 +95,7 @@ export default ['domainsController', [ break; - case 'indexes': + case 'table-indexes': if ($scope.tableIndexSaveVisible(field, index)) return $scope.tableIndexSave(field, index, stopEdit); @@ -1355,6 +1355,9 @@ export default ['domainsController', [ if (!LegacyUtils.domainForStoreConfigured(item) && !LegacyUtils.domainForQueryConfigured(item) && item.queryMetadata === 'Configuration') return ErrorPopover.show('query-title', 'SQL query domain model should be configured', $scope.ui, 'query'); + if (!LegacyUtils.domainForStoreConfigured(item) && item.generatePojo) + return ErrorPopover.show('store-title', 'Domain model for cache store should be configured when generation of POJO classes is enabled', $scope.ui, 'store'); + return true; } @@ -1437,7 +1440,8 @@ export default ['domainsController', [ } function _newNameIsValidJavaClass(newName) { - return LegacyUtils.isValidJavaClass('New name for value type', newName, false, 'copy-new-nameInput'); + return !$scope.backupItem.generatePojo || + LegacyUtils.isValidJavaClass('New name for value type', newName, false, 'copy-new-nameInput'); } // Save domain model with new name. diff --git a/modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js b/modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js index fe8bb09616996..49359764bdeac 100644 --- a/modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js +++ b/modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js @@ -19,10 +19,10 @@ import gulp from 'gulp'; import ll from 'gulp-ll'; import jade from 'gulp-jade'; -import { jadeViewsPaths, jadeAppModulePaths, jadeModulePaths, destDir } from '../paths'; +import { jadeViewsPaths, jadeAppModulePaths, jadeModulePaths, destDir, rootDir } from '../paths'; const jadeOptions = { - basedir: './', + basedir: rootDir, cache: true }; diff --git a/modules/web-console/frontend/gulpfile.babel.js/tasks/test.js b/modules/web-console/frontend/gulpfile.babel.js/tasks/test.js deleted file mode 100644 index ad4108dae1e21..0000000000000 --- a/modules/web-console/frontend/gulpfile.babel.js/tasks/test.js +++ /dev/null @@ -1,92 +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. - */ - -import gulp from 'gulp'; -import karmaBabelPreprocessor from 'karma-babel-preprocessor'; -import karmaPhantomjsLauncher from 'karma-phantomjs-launcher'; -import karmaWebpack from 'karma-webpack'; -import karmaJasmine from 'karma-jasmine'; - -import {Server} from 'karma'; - -import {rootDir} from '../paths'; - -gulp.task('test', (cb) => { - new Server({ - // Base path that will be used to resolve all patterns (eg. files, exclude). - basePath: rootDir, - - // Frameworks to use available frameworks: https://npmjs.org/browse/keyword/karma-adapter - frameworks: ['jasmine'], - - // List of files / patterns to load in the browser. - files: [ - 'test/**/*.test.js' - ], - - plugins: [ - karmaBabelPreprocessor, - karmaPhantomjsLauncher, - karmaWebpack, - karmaJasmine - ], - - // Preprocess matching files before serving them to the browser - // available preprocessors: https://npmjs.org/browse/keyword/karma-preprocessor. - preprocessors: { - 'test/**/*.js': ['webpack'] - }, - - webpack: { - devtool: 'inline-source-map', - module: { - loaders: [ - {test: /\.js/, loaders: ['babel'], exclude: /node_modules/} - ] - }, - resolve: { - extensions: ['', '.js'] - } - }, - - // Test results reporter to use - // possible values: 'dots', 'progress' - // available reporters: https://npmjs.org/browse/keyword/karma-reporter. - reporters: ['progress'], - - // web server port - port: 9876, - - // enable / disable colors in the output (reporters and logs) - colors: true, - - // enable / disable watching file and executing tests whenever any file changes - autoWatch: true, - - // start these browsers - // available browser launchers: https://npmjs.org/browse/keyword/karma-launcher - browsers: ['PhantomJS'], - - // Continuous Integration mode - // if true, Karma captures browsers, runs the tests and exits - singleRun: true, - - // Concurrency level - // how many browser should be started simultaneous - concurrency: Infinity - }, cb).start(); -}); diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js index f6c2d87c2f4a2..2463d246a4446 100644 --- a/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js +++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js @@ -77,7 +77,10 @@ export default () => { './' ] }, - + jade: { + basedir: rootDir, + locals: {} + }, // Modules resolvers. /* global require */ module: { diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js index 1550dfad54775..cad913354d7f0 100644 --- a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js +++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js @@ -55,6 +55,11 @@ export default () => { changeOrigin: true, ws: true }, + '/agents': { + target: 'http://localhost:3000', + changeOrigin: true, + ws: true + }, '/api/v1/*': { target: 'http://localhost:3000', changeOrigin: true, diff --git a/modules/web-console/frontend/package.json b/modules/web-console/frontend/package.json index fe8c2953e048e..b511ca1e9123f 100644 --- a/modules/web-console/frontend/package.json +++ b/modules/web-console/frontend/package.json @@ -95,7 +95,7 @@ "html-loader": "^0.4.3", "html-webpack-plugin": "^2.21.0", "jade": "^1.11.0", - "jade-html-loader": "0.0.3", + "jade-html-loader": "git://github.com/courcelan/jade-html-loader", "jasmine-core": "^2.4.1", "json-loader": "^0.5.4", "karma": "^0.13.22", diff --git a/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss b/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss index 1c8f325790cd9..15ee60cdb8427 100644 --- a/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss +++ b/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss @@ -30,3 +30,21 @@ $fa-font-path: '~font-awesome/fonts'; @import '~font-awesome/scss/rotated-flipped'; @import '~font-awesome/scss/stacked'; @import '~font-awesome/scss/icons'; + +.fa { + cursor: pointer; +} + +.icon-help { + @extend .fa; + @extend .fa-question-circle-o; + + cursor: default; +} + +.icon-confirm { + @extend .fa; + @extend .fa-question-circle-o; + + cursor: default; +} \ No newline at end of file diff --git a/modules/web-console/frontend/public/stylesheets/style.scss b/modules/web-console/frontend/public/stylesheets/style.scss index 0f8f49a360b32..172abf478f755 100644 --- a/modules/web-console/frontend/public/stylesheets/style.scss +++ b/modules/web-console/frontend/public/stylesheets/style.scss @@ -541,6 +541,12 @@ i.btn { margin-left: 0; } + button.btn[disabled="disabled"] { + i { + cursor: not-allowed; + } + } + > .btn + .dropdown-toggle { margin-right: 0; padding: 3px 6px; @@ -1350,9 +1356,6 @@ label { margin: 0 5px; } -.fa { - cursor: pointer; -} .fa-cursor-default { cursor: default !important; @@ -1372,10 +1375,6 @@ label { margin-right: 5px; } -.fa-question-circle { - cursor: default; -} - label.required:after { color: $brand-primary; content: ' *'; diff --git a/modules/web-console/frontend/views/configuration/caches.jade b/modules/web-console/frontend/views/configuration/caches.jade index 74d5505bc0444..4a4cf2e1a1d24 100644 --- a/modules/web-console/frontend/views/configuration/caches.jade +++ b/modules/web-console/frontend/views/configuration/caches.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade .docs-header h1 Configure Ignite Caches @@ -36,19 +36,19 @@ include ../../app/helpers/jade/mixins.jade div(bs-collapse='' data-allow-multiple='true' ng-model='ui.activePanels') form.form-horizontal(name='ui.inputForm' novalidate ng-if='contentVisible()') .panel-group - include ../../app/modules/states/configuration/caches/general.jade - include ../../app/modules/states/configuration/caches/memory.jade - include ../../app/modules/states/configuration/caches/query.jade - include ../../app/modules/states/configuration/caches/store.jade + include /app/modules/states/configuration/caches/general.jade + include /app/modules/states/configuration/caches/memory.jade + include /app/modules/states/configuration/caches/query.jade + include /app/modules/states/configuration/caches/store.jade +advanced-options-toggle-default div(ng-show='ui.expanded') - include ../../app/modules/states/configuration/caches/concurrency.jade - include ../../app/modules/states/configuration/caches/near-cache-client.jade - include ../../app/modules/states/configuration/caches/near-cache-server.jade - include ../../app/modules/states/configuration/caches/node-filter.jade - include ../../app/modules/states/configuration/caches/rebalance.jade - include ../../app/modules/states/configuration/caches/statistics.jade + include /app/modules/states/configuration/caches/concurrency.jade + include /app/modules/states/configuration/caches/near-cache-client.jade + include /app/modules/states/configuration/caches/near-cache-server.jade + include /app/modules/states/configuration/caches/node-filter.jade + include /app/modules/states/configuration/caches/rebalance.jade + include /app/modules/states/configuration/caches/statistics.jade +advanced-options-toggle-default diff --git a/modules/web-console/frontend/views/configuration/clusters.jade b/modules/web-console/frontend/views/configuration/clusters.jade index 8a3a19408e9f8..78264647b800f 100644 --- a/modules/web-console/frontend/views/configuration/clusters.jade +++ b/modules/web-console/frontend/views/configuration/clusters.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade .docs-header h1 Configure Ignite Clusters @@ -36,33 +36,33 @@ include ../../app/helpers/jade/mixins.jade div(bs-collapse='' data-allow-multiple='true' ng-model='ui.activePanels') form.form-horizontal(name='ui.inputForm' novalidate ng-if='contentVisible()') .panel-group - include ../../app/modules/states/configuration/clusters/general.jade + include /app/modules/states/configuration/clusters/general.jade +advanced-options-toggle-default div(ng-show='ui.expanded') - include ../../app/modules/states/configuration/clusters/atomic.jade - include ../../app/modules/states/configuration/clusters/binary.jade - include ../../app/modules/states/configuration/clusters/cache-key-cfg.jade - include ../../app/modules/states/configuration/clusters/checkpoint.jade - include ../../app/modules/states/configuration/clusters/collision.jade - include ../../app/modules/states/configuration/clusters/communication.jade - include ../../app/modules/states/configuration/clusters/connector.jade - include ../../app/modules/states/configuration/clusters/deployment.jade - include ../../app/modules/states/configuration/clusters/discovery.jade - include ../../app/modules/states/configuration/clusters/events.jade - include ../../app/modules/states/configuration/clusters/failover.jade - include ../../app/modules/states/configuration/clusters/igfs.jade - include ../../app/modules/states/configuration/clusters/load-balancing.jade - include ../../app/modules/states/configuration/clusters/logger.jade - include ../../app/modules/states/configuration/clusters/marshaller.jade - include ../../app/modules/states/configuration/clusters/metrics.jade - include ../../app/modules/states/configuration/clusters/odbc.jade - include ../../app/modules/states/configuration/clusters/ssl.jade - include ../../app/modules/states/configuration/clusters/swap.jade - include ../../app/modules/states/configuration/clusters/thread.jade - include ../../app/modules/states/configuration/clusters/time.jade - include ../../app/modules/states/configuration/clusters/transactions.jade - include ../../app/modules/states/configuration/clusters/attributes.jade + include /app/modules/states/configuration/clusters/atomic.jade + include /app/modules/states/configuration/clusters/binary.jade + include /app/modules/states/configuration/clusters/cache-key-cfg.jade + include /app/modules/states/configuration/clusters/checkpoint.jade + include /app/modules/states/configuration/clusters/collision.jade + include /app/modules/states/configuration/clusters/communication.jade + include /app/modules/states/configuration/clusters/connector.jade + include /app/modules/states/configuration/clusters/deployment.jade + include /app/modules/states/configuration/clusters/discovery.jade + include /app/modules/states/configuration/clusters/events.jade + include /app/modules/states/configuration/clusters/failover.jade + include /app/modules/states/configuration/clusters/igfs.jade + include /app/modules/states/configuration/clusters/load-balancing.jade + include /app/modules/states/configuration/clusters/logger.jade + include /app/modules/states/configuration/clusters/marshaller.jade + include /app/modules/states/configuration/clusters/metrics.jade + include /app/modules/states/configuration/clusters/odbc.jade + include /app/modules/states/configuration/clusters/ssl.jade + include /app/modules/states/configuration/clusters/swap.jade + include /app/modules/states/configuration/clusters/thread.jade + include /app/modules/states/configuration/clusters/time.jade + include /app/modules/states/configuration/clusters/transactions.jade + include /app/modules/states/configuration/clusters/attributes.jade +advanced-options-toggle-default diff --git a/modules/web-console/frontend/views/configuration/domains-import.jade b/modules/web-console/frontend/views/configuration/domains-import.jade index 99014bce6c1d2..e4f95bcaa6e69 100644 --- a/modules/web-console/frontend/views/configuration/domains-import.jade +++ b/modules/web-console/frontend/views/configuration/domains-import.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade mixin chk(mdl, change, tip) input(type='checkbox' ng-model=mdl ng-change=change bs-tooltip='' data-title=tip data-trigger='hover' data-placement='top') @@ -133,7 +133,7 @@ mixin td-ellipses-lbl(w, lbl) td .settings-row label Defaults to be applied for filtered tables - i.tipLabel.fa.fa-question-circle(bs-tooltip='' data-title='Select and apply options for caches generation') + i.tipLabel.icon-help(bs-tooltip='' data-title='Select and apply options for caches generation') .settings-row .col-sm-11 .col-sm-6(style='padding-right: 5px') diff --git a/modules/web-console/frontend/views/configuration/domains.jade b/modules/web-console/frontend/views/configuration/domains.jade index 591234333aa10..14e9ca5240ea0 100644 --- a/modules/web-console/frontend/views/configuration/domains.jade +++ b/modules/web-console/frontend/views/configuration/domains.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade .docs-header h1 Configure Domain Model And SQL Queries @@ -61,6 +61,6 @@ include ../../app/helpers/jade/mixins.jade div(bs-collapse='' data-allow-multiple='true' ng-model='ui.activePanels') form.form-horizontal(name='ui.inputForm' novalidate ng-if='contentVisible()') .panel-group - include ../../app/modules/states/configuration/domains/general.jade - include ../../app/modules/states/configuration/domains/query.jade - include ../../app/modules/states/configuration/domains/store.jade + include /app/modules/states/configuration/domains/general.jade + include /app/modules/states/configuration/domains/query.jade + include /app/modules/states/configuration/domains/store.jade diff --git a/modules/web-console/frontend/views/configuration/igfs.jade b/modules/web-console/frontend/views/configuration/igfs.jade index 16e9f280ac3b8..10c902be3b3be 100644 --- a/modules/web-console/frontend/views/configuration/igfs.jade +++ b/modules/web-console/frontend/views/configuration/igfs.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade .docs-header h1 Configure Ignite In-memory File Systems @@ -37,15 +37,15 @@ include ../../app/helpers/jade/mixins.jade div(bs-collapse='' data-allow-multiple='true' ng-model='ui.activePanels') form.form-horizontal(name='ui.inputForm' novalidate ng-if='contentVisible()') .panel-group - include ../../app/modules/states/configuration/igfs/general.jade + include /app/modules/states/configuration/igfs/general.jade +advanced-options-toggle-default div(ng-show='ui.expanded') - include ../../app/modules/states/configuration/igfs/secondary.jade - include ../../app/modules/states/configuration/igfs/ipc.jade - include ../../app/modules/states/configuration/igfs/fragmentizer.jade - include ../../app/modules/states/configuration/igfs/dual.jade - include ../../app/modules/states/configuration/igfs/misc.jade + include /app/modules/states/configuration/igfs/secondary.jade + include /app/modules/states/configuration/igfs/ipc.jade + include /app/modules/states/configuration/igfs/fragmentizer.jade + include /app/modules/states/configuration/igfs/dual.jade + include /app/modules/states/configuration/igfs/misc.jade +advanced-options-toggle-default diff --git a/modules/web-console/frontend/views/configuration/summary.jade b/modules/web-console/frontend/views/configuration/summary.jade index d041fea0133fb..9a6e553f43103 100644 --- a/modules/web-console/frontend/views/configuration/summary.jade +++ b/modules/web-console/frontend/views/configuration/summary.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade mixin hard-link(ref, txt) a(style='color:#ec1c24' href=ref target='_blank') #{txt} diff --git a/modules/web-console/frontend/views/includes/header.jade b/modules/web-console/frontend/views/includes/header.jade index 9ef09aa71c304..99bcea9a00734 100644 --- a/modules/web-console/frontend/views/includes/header.jade +++ b/modules/web-console/frontend/views/includes/header.jade @@ -24,22 +24,23 @@ header#header.header td(ng-if='$root.user' style='padding-top: 20px') ul.nav.navbar-nav(ignite-sidebar ignite-navbar) li(ng-class='{active: $state.includes("base.configuration")}' bs-dropdown='sidebar.items' data-placement='bottom-right' data-trigger='hover focus' data-container='self' data-animation='' ng-click='$event.stopPropagation()') - a.dropdown-toggle Configuration + a.dropdown-toggle Configure span.caret ul.nav.navbar-nav(ng-controller='notebookController') li.sql-notebooks(ng-if='IgniteDemoMode' ng-class='{active: $state.includes("base.sql")}') - a(ui-sref='base.sql.demo') SQL + a(ui-sref='base.sql.demo') Queries li.sql-notebooks(ng-if='!IgniteDemoMode && !notebooks.length' ng-class='{active: $state.includes("base.sql")}') - a(ng-click='createNotebook()') SQL + a(ng-click='createNotebook()') Queries - li.sql-notebooks(ng-if='!IgniteDemoMode && notebooks.length' ng-class='{active: $state.includes("base.sql")}' bs-dropdown='notebooks' data-placement='bottom-left' data-trigger='hover focus' data-container='self' data-animation='' ng-click='$event.stopPropagation()' aria-haspopup="true" aria-expanded="false") - a.dropdown-toggle SQL + li.sql-notebooks(ng-if='!IgniteDemoMode && notebooks.length' ng-class='{active: $state.includes("base.sql")}' bs-dropdown='notebooks' data-placement='bottom-left' data-trigger='hover focus' data-container='self' data-animation='' ng-click='$event.stopPropagation()' aria-haspopup='true' aria-expanded='false') + a.dropdown-toggle Queries span.caret - li(ui-sref-active='active' ng-repeat='item in navbar.items') - a(ui-sref='{{::item.sref}}') {{::item.text}} + li(ui-sref-active='active' ng-repeat='item in navbar.items' ng-class='{active: $state.includes("base.monitoring")}' bs-dropdown='item.children' data-placement='bottom-left' data-trigger='hover focus' data-container='self' data-animation='' ng-click='$event.stopPropagation()' aria-haspopup='true' aria-expanded='false') + a.dropdown-toggle {{::item.text}} + span.caret a(ng-controller='demoController') button.btn.btn-info(ng-if='IgniteDemoMode' ng-click='closeDemo()') Close demo diff --git a/modules/web-console/frontend/views/settings/profile.jade b/modules/web-console/frontend/views/settings/profile.jade index bc4965a0a2435..6f91f8f45a77d 100644 --- a/modules/web-console/frontend/views/settings/profile.jade +++ b/modules/web-console/frontend/views/settings/profile.jade @@ -56,7 +56,7 @@ mixin lbl(txt) label {{user.token || 'No security token. Regenerate please.'}} i.tipLabel.fa.fa-refresh(ng-click='generateToken()' bs-tooltip='' data-title='Generate random security token') i.tipLabel.fa.fa-clipboard(ignite-copy-to-clipboard='{{user.token}}' bs-tooltip='' data-title='Copy security token to clipboard') - i.tipLabel.fa.fa-question-circle(ng-if=lines bs-tooltip='' data-title='The security token is used for authorization of web agent') + i.tipLabel.icon-help(ng-if=lines bs-tooltip='' data-title='The security token is used for authorization of web agent') .details-row .advanced-options i.fa( diff --git a/modules/web-console/frontend/views/sql/sql.jade b/modules/web-console/frontend/views/sql/sql.jade index 4a596b96e1c1d..e3f64617adb1f 100644 --- a/modules/web-console/frontend/views/sql/sql.jade +++ b/modules/web-console/frontend/views/sql/sql.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../app/directives/ui-grid-settings/ui-grid-settings.jade +include /app/directives/ui-grid-settings/ui-grid-settings.jade mixin btn-toolbar(btn, click, tip, focusId) i.btn.btn-default.fa(class=btn ng-click=click bs-tooltip='' data-title=tip ignite-on-click-focus=focusId data-trigger='hover' data-placement='bottom') diff --git a/modules/web-console/frontend/views/templates/agent-download.jade b/modules/web-console/frontend/views/templates/agent-download.jade index a6da3d10d64f8..20b6b03d058f3 100644 --- a/modules/web-console/frontend/views/templates/agent-download.jade +++ b/modules/web-console/frontend/views/templates/agent-download.jade @@ -36,7 +36,7 @@ .details-row(ng-show='agentLoad.showToken') label.labelField Security token: {{user.token}} i.tipLabel.fa.fa-clipboard(ignite-copy-to-clipboard='{{user.token}}' bs-tooltip='' data-title='Copy security token to clipboard') - i.tipLabel.fa.fa-question-circle(ng-if=lines bs-tooltip='' data-title='The security token is used for authorization of web agent') + i.tipLabel.icon-help(ng-if=lines bs-tooltip='' data-title='The security token is used for authorization of web agent') .agent-download(ng-if='hasAgents') p Connection to Ignite Web Agent is established, but agent failed to connect to Ignite Node p Please check the following: diff --git a/modules/web-console/frontend/views/templates/batch-confirm.jade b/modules/web-console/frontend/views/templates/batch-confirm.jade index c5d377565ce33..0b74a4e3f1013 100644 --- a/modules/web-console/frontend/views/templates/batch-confirm.jade +++ b/modules/web-console/frontend/views/templates/batch-confirm.jade @@ -20,7 +20,7 @@ .modal-header button.close(ng-click='cancel()' aria-hidden='true') × h4.modal-title - i.fa.fa-question + i.icon-confirm | Confirmation .modal-body(ng-show='content') p(ng-bind-html='content' style='text-align: center') diff --git a/modules/web-console/frontend/views/templates/clone.jade b/modules/web-console/frontend/views/templates/clone.jade index 6cf93ca94f20b..99ec58be4697d 100644 --- a/modules/web-console/frontend/views/templates/clone.jade +++ b/modules/web-console/frontend/views/templates/clone.jade @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. -include ../../app/helpers/jade/mixins.jade +include /app/helpers/jade/mixins.jade .modal(tabindex='-1' role='dialog') .modal-dialog diff --git a/modules/web-console/frontend/views/templates/confirm.jade b/modules/web-console/frontend/views/templates/confirm.jade index f9f966befd67c..00314980ae851 100644 --- a/modules/web-console/frontend/views/templates/confirm.jade +++ b/modules/web-console/frontend/views/templates/confirm.jade @@ -20,7 +20,7 @@ .modal-header button.close(ng-click='confirmCancel()' aria-hidden='true') × h4.modal-title - i.fa.fa-question-circle-o + i.icon-confirm | Confirmation .modal-body(ng-show='content') p(ng-bind-html='content' style='text-align: center;') diff --git a/modules/web-console/web-agent/README.txt b/modules/web-console/web-agent/README.txt index c6e625b837520..cc0c80fc99adb 100644 --- a/modules/web-console/web-agent/README.txt +++ b/modules/web-console/web-agent/README.txt @@ -24,7 +24,7 @@ Configuration file: Example configuration file: tokens=1a2b3c4d5f,2j1s134d12 - serverURI=https://console.example.com:3001 + serverURI=https://console.example.com Security tokens: 1) By default security token of current user will be included into "default.properties" inside downloaded "ignite-web-agent-x.x.x.zip". @@ -47,7 +47,7 @@ Options: http://localhost:8080 -s, --server-uri URI for connect to Ignite Web Console via web-socket protocol, default - value: http://localhost:3001 + value: http://localhost:3000 -t, --tokens User's security tokens diff --git a/modules/web-console/web-agent/bin/ignite-web-agent.bat b/modules/web-console/web-agent/bin/ignite-web-agent.bat index f16eb3573c0f2..8291b5531da50 100644 --- a/modules/web-console/web-agent/bin/ignite-web-agent.bat +++ b/modules/web-console/web-agent/bin/ignite-web-agent.bat @@ -53,7 +53,12 @@ goto error_finish :: :: ADD YOUR/CHANGE ADDITIONAL OPTIONS HERE :: -if "%JVM_OPTS%" == "" set JVM_OPTS=-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxPermSize=256m +"%JAVA_HOME%\bin\java.exe" -version 2>&1 | findstr "1\.[7]\." > nul +if %ERRORLEVEL% equ 0 ( + if "%JVM_OPTS%" == "" set JVM_OPTS=-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxPermSize=256m +) else ( + if "%JVM_OPTS%" == "" set JVM_OPTS=-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxMetaspaceSize=256m +) "%JAVA_HOME%\bin\java.exe" %JVM_OPTS% -cp "*" org.apache.ignite.console.agent.AgentLauncher %* diff --git a/modules/web-console/web-agent/bin/ignite-web-agent.sh b/modules/web-console/web-agent/bin/ignite-web-agent.sh index 3f2c2bcefa8fb..2e9f041f8ca7c 100644 --- a/modules/web-console/web-agent/bin/ignite-web-agent.sh +++ b/modules/web-console/web-agent/bin/ignite-web-agent.sh @@ -81,7 +81,11 @@ cd $DIR # ADD YOUR/CHANGE ADDITIONAL OPTIONS HERE # if [ -z "$JVM_OPTS" ] ; then - JVM_OPTS="-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxPermSize=256m" + if [[ `"$JAVA" -version 2>&1 | egrep "1\.[7]\."` ]]; then + JVM_OPTS="-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxPermSize=256m" + else + JVM_OPTS="-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxMetaspaceSize=256m" + fi fi "$JAVA" ${JVM_OPTS} -cp "*" org.apache.ignite.console.agent.AgentLauncher "$@" diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentConfiguration.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentConfiguration.java index d4787ccef8ea4..8f70100a7c3b1 100644 --- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentConfiguration.java +++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentConfiguration.java @@ -31,9 +31,6 @@ * Agent configuration. */ public class AgentConfiguration { - /** Default server port. */ - public static final int DFLT_SERVER_PORT = 3001; - /** Default Ignite node HTTP port. */ public static final int DFLT_NODE_PORT = 8080; @@ -41,7 +38,7 @@ public class AgentConfiguration { public static final String DFLT_CFG_PATH = "default.properties"; /** Default server URI. */ - private static final String DFLT_SERVER_URI = "http://localhost:3001"; + private static final String DFLT_SERVER_URI = "http://localhost:3000"; /** Default Ignite node HTTP URI. */ private static final String DFLT_NODE_URI = "http://localhost:8080"; diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java index 810fad434fdfa..0c03d7787b15d 100644 --- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java +++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java @@ -50,7 +50,6 @@ import static io.socket.client.Socket.EVENT_DISCONNECT; import static io.socket.client.Socket.EVENT_ERROR; import static io.socket.client.Socket.EVENT_RECONNECTING; -import static org.apache.ignite.console.agent.AgentConfiguration.DFLT_SERVER_PORT; /** * Control Center Agent launcher. @@ -227,11 +226,10 @@ public static void main(String[] args) throws Exception { URI uri = URI.create(cfg.serverUri()); - if (uri.getPort() == -1) - uri = URI.create(cfg.serverUri() + ':' + DFLT_SERVER_PORT); - IO.Options opts = new IO.Options(); + opts.path = "/agents"; + opts.reconnectionDelay = RECONNECT_INTERVAL; // Workaround for use self-signed certificate diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java index 2fb9f562bdbe6..489e762eb5362 100644 --- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java +++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java @@ -134,6 +134,7 @@ private static CacheConfiguration cacheConfiguration(String name) { CacheConfiguration ccfg = new CacheConfiguration<>(name); ccfg.setAffinity(new RendezvousAffinityFunction(false, 32)); + ccfg.setQueryDetailMetricsSize(10); ccfg.setStartSize(100); ccfg.setStatisticsEnabled(true); From 7d88c5bfe7d6f130974fab1ed4266fff859afd3d Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Thu, 24 Nov 2016 17:59:33 +0700 Subject: [PATCH 371/487] Web console beta-6. Minor fix. --- modules/web-console/docker/compose/backend/build.sh | 2 +- modules/web-console/docker/compose/frontend/build.sh | 4 ++-- modules/web-console/docker/standalone/build.sh | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/modules/web-console/docker/compose/backend/build.sh b/modules/web-console/docker/compose/backend/build.sh index d44efbdb4fc51..a10b70641bd50 100755 --- a/modules/web-console/docker/compose/backend/build.sh +++ b/modules/web-console/docker/compose/backend/build.sh @@ -27,7 +27,7 @@ WORK_DIR=`cd "$(dirname "$0")"; pwd` BUILD_DIR="$WORK_DIR/build" IGNITE_WEB_CONSOLE_BACKEND_DIR="$IGNITE_HOME/modules/web-console/backend" -DOCKER_IMAGE_NAME="ignite/web-console-backend" +DOCKER_IMAGE_NAME="apacheignite/web-console-backend" echo "Receiving version..." VERSION=`cd $IGNITE_HOME && mvn org.apache.maven.plugins:maven-help-plugin:evaluate -Dexpression=project.version| grep -Ev '(^\[|Download\w+:)'` diff --git a/modules/web-console/docker/compose/frontend/build.sh b/modules/web-console/docker/compose/frontend/build.sh index 6d52fb261788d..c807a86125c6b 100755 --- a/modules/web-console/docker/compose/frontend/build.sh +++ b/modules/web-console/docker/compose/frontend/build.sh @@ -28,8 +28,8 @@ SOURCE_DIR=$WORK_DIR/src BUILD_DIR=$WORK_DIR/build DOCKER_BUILD_CONTAINER=web-console-frontend-builder -DOCKER_BUILD_IMAGE_NAME=ignite/$DOCKER_BUILD_CONTAINER -DOCKER_IMAGE_NAME=ignite/web-console-frontend +DOCKER_BUILD_IMAGE_NAME=apacheignite/$DOCKER_BUILD_CONTAINER +DOCKER_IMAGE_NAME=apacheignite/web-console-frontend echo "Receiving version..." VERSION=`cd $IGNITE_HOME && mvn org.apache.maven.plugins:maven-help-plugin:evaluate -Dexpression=project.version| grep -Ev '(^\[|Download\w+:)'` diff --git a/modules/web-console/docker/standalone/build.sh b/modules/web-console/docker/standalone/build.sh index 5482086937561..c32dc403ba92d 100755 --- a/modules/web-console/docker/standalone/build.sh +++ b/modules/web-console/docker/standalone/build.sh @@ -27,7 +27,7 @@ WORK_DIR=`cd "$(dirname "$0")"; pwd` BUILD_DIR="$WORK_DIR/build" IGNITE_WEB_CONSOLE_DIR="$IGNITE_HOME/modules/web-console" -DOCKER_IMAGE_NAME="ignite/web-console-standalone" +DOCKER_IMAGE_NAME="apacheignite/web-console-standalone" echo "Receiving version..." VERSION=`cd $IGNITE_HOME && mvn org.apache.maven.plugins:maven-help-plugin:evaluate -Dexpression=project.version| grep -Ev '(^\[|Download\w+:)'` From 9c6824b4f33fbdead64299d9e0c34365d5d4a570 Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Thu, 24 Nov 2016 16:27:05 +0300 Subject: [PATCH 372/487] IGNITE-3958 Fixed "Client node should not start rest processor". --- .../rest/RestProcessorMultiStartSelfTest.java | 48 ++++++++++++++++++- .../apache/ignite/IgniteSystemProperties.java | 6 +++ .../apache/ignite/internal/IgniteKernal.java | 12 ++++- .../processors/rest/GridRestProcessor.java | 15 ++++++ 4 files changed, 79 insertions(+), 2 deletions(-) diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestProcessorMultiStartSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestProcessorMultiStartSelfTest.java index 7714dbfb4299b..5b880799c86bc 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestProcessorMultiStartSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestProcessorMultiStartSelfTest.java @@ -17,8 +17,10 @@ package org.apache.ignite.internal.processors.rest; +import java.util.Map; import org.apache.ignite.configuration.ConnectorConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** @@ -28,15 +30,26 @@ public class RestProcessorMultiStartSelfTest extends GridCommonAbstractTest { /** */ private static final int GRID_CNT = 3; + /** */ + private static boolean client = false; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); cfg.setConnectorConfiguration(new ConnectorConfiguration()); + cfg.setClientMode(client); return cfg; } + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + client = false; + } + /** * Test that multiple nodes can start with JETTY enabled. * @@ -53,4 +66,37 @@ public void testMultiStart() throws Exception { stopAllGrids(); } } -} \ No newline at end of file + + /** + * Test that multiple nodes can start with JETTY enabled. + * + * @throws Exception If failed. + */ + public void testMultiStartWithClient() throws Exception { + try { + int clnIdx = GRID_CNT - 1; + + for (int i = 0; i < clnIdx; i++) { + startGrid(i); + + GridRestProcessor rest = grid(i).context().rest(); + + assertNotNull(rest); + assertFalse(((Map)GridTestUtils.getFieldValue(rest, "handlers")).isEmpty()); + } + + client = true; + + startGrid(clnIdx); + + GridRestProcessor rest = grid(GRID_CNT - 1).context().rest(); + + // Check that rest processor doesn't start. + assertNotNull(rest); + assertTrue(((Map)GridTestUtils.getFieldValue(rest, "handlers")).isEmpty()); + } + finally { + stopAllGrids(); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 043c95ade0fbc..de6cbed1fb3a5 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -109,6 +109,12 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_REST_MAX_TASK_RESULTS = "IGNITE_REST_MAX_TASK_RESULTS"; + /** + * This property allows to override default behavior that rest processor + * doesn't start on client node. If set {@code true} than rest processor will be started on client node. + */ + public static final String IGNITE_REST_START_ON_CLIENT = "IGNITE_REST_START_ON_CLIENT"; + /** * This property defines the maximum number of attempts to remap near get to the same * primary node. Remapping may be needed when topology is changed concurrently with diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 1963509d62908..ef9c651ad99a2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -179,6 +179,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_DAEMON; import static org.apache.ignite.IgniteSystemProperties.IGNITE_NO_ASCII; import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_REST_START_ON_CLIENT; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK; import static org.apache.ignite.IgniteSystemProperties.IGNITE_STARVATION_CHECK_INTERVAL; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SUCCESS_FILE; @@ -1632,7 +1633,16 @@ private String onOff(boolean b) { private boolean isRestEnabled() { assert cfg != null; - return cfg.getConnectorConfiguration() != null; + return cfg.getConnectorConfiguration() != null && + // By default rest processor doesn't start on client nodes. + (!isClientNode() || (isClientNode() && IgniteSystemProperties.getBoolean(IGNITE_REST_START_ON_CLIENT))); + } + + /** + * @return {@code True} if node client or daemon otherwise {@code false}. + */ + private boolean isClientNode() { + return cfg.isClientMode() || cfg.isDaemon(); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java index 6d20547e8fa41..3f0478564d4df 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java @@ -75,6 +75,7 @@ import org.apache.ignite.thread.IgniteThread; import org.jsr166.LongAdder8; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_REST_START_ON_CLIENT; import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_AUTH_FAILED; import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_FAILED; import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_SECURITY_CHECK_FAILED; @@ -436,6 +437,13 @@ public GridRestProcessor(GridKernalContext ctx) { /** {@inheritDoc} */ @Override public void start() throws IgniteCheckedException { if (isRestEnabled()) { + if (notStartOnClient()) { + U.quietAndInfo(log, "REST protocols do not start on client node. " + + "To start the protocols on client node set '-DIGNITE_REST_START_ON_CLIENT=true' system property."); + + return; + } + // Register handlers. addHandler(new GridCacheCommandHandler(ctx)); addHandler(new GridTaskCommandHandler(ctx)); @@ -471,6 +479,13 @@ public GridRestProcessor(GridKernalContext ctx) { } } + /** + * @return {@code True} if rest processor should not start on client node. + */ + private boolean notStartOnClient() { + return ctx.clientNode() && !IgniteSystemProperties.getBoolean(IGNITE_REST_START_ON_CLIENT); + } + /** {@inheritDoc} */ @Override public void onKernalStart() throws IgniteCheckedException { if (isRestEnabled()) { From 56998e704e9a67760c70481c10c56e72c0a866bb Mon Sep 17 00:00:00 2001 From: Konstantin Dudkov Date: Fri, 28 Oct 2016 16:27:34 +0300 Subject: [PATCH 373/487] ignite-4088 Added methods to create/destroy multiple caches. This closes #1174. (cherry picked from commit f445e7b) --- .../main/java/org/apache/ignite/Ignite.java | 77 ++++- .../apache/ignite/internal/IgniteKernal.java | 81 +++++ .../processors/cache/GridCacheProcessor.java | 291 +++++++++++++----- .../IgniteDynamicCacheStartSelfTest.java | 217 +++++++++++-- .../processors/igfs/IgfsIgniteMock.java | 19 ++ .../testframework/junits/IgniteMock.java | 15 + .../junits/multijvm/IgniteProcessProxy.java | 15 + .../org/apache/ignite/IgniteSpringBean.java | 21 ++ 8 files changed, 616 insertions(+), 120 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java index bd21468ae73b1..0de08d56584a8 100644 --- a/modules/core/src/main/java/org/apache/ignite/Ignite.java +++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java @@ -20,6 +20,7 @@ import java.util.Collection; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; +import javax.cache.CacheException; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.affinity.Affinity; import org.apache.ignite.cluster.ClusterGroup; @@ -220,8 +221,24 @@ public interface Ignite extends AutoCloseable { * * @param cacheCfg Cache configuration to use. * @return Instance of started cache. + * @throws CacheException If a cache with the same name already exists or other error occurs. */ - public IgniteCache createCache(CacheConfiguration cacheCfg); + public IgniteCache createCache(CacheConfiguration cacheCfg) throws CacheException; + + /** + * Dynamically starts new caches with the given cache configurations. + *

          + * If local node is an affinity node, this method will return the instance of started caches. + * Otherwise, it will create a client caches on local node. + *

          + * If for one of configurations a cache with the same name already exists in the grid, an exception will be thrown regardless + * whether the given configuration matches the configuration of the existing cache or not. + * + * @param cacheCfgs Collection of cache configuration to use. + * @return Collection of instances of started caches. + * @throws CacheException If one of created caches exists or other error occurs. + */ + public Collection createCaches(Collection cacheCfgs) throws CacheException; /** * Dynamically starts new cache using template configuration. @@ -233,8 +250,9 @@ public interface Ignite extends AutoCloseable { * * @param cacheName Cache name. * @return Instance of started cache. + * @throws CacheException If a cache with the same name already exists or other error occurs. */ - public IgniteCache createCache(String cacheName); + public IgniteCache createCache(String cacheName) throws CacheException; /** * Gets existing cache with the given name or creates new one with the given configuration. @@ -245,23 +263,39 @@ public interface Ignite extends AutoCloseable { * * @param cacheCfg Cache configuration to use. * @return Existing or newly created cache. + * @throws CacheException If error occurs. */ - public IgniteCache getOrCreateCache(CacheConfiguration cacheCfg); + public IgniteCache getOrCreateCache(CacheConfiguration cacheCfg) throws CacheException; /** * Gets existing cache with the given name or creates new one using template configuration. * * @param cacheName Cache name. * @return Existing or newly created cache. + * @throws CacheException If error occurs. */ - public IgniteCache getOrCreateCache(String cacheName); + public IgniteCache getOrCreateCache(String cacheName) throws CacheException; + + /** + * Gets existing caches with the given name or created one with the given configuration. + *

          + * If a cache with the same name already exist, this method will not check that the given + * configuration matches the configuration of existing cache and will return an instance + * of the existing cache. + * + * @param cacheCfgs Collection of cache configuration to use. + * @return Collection of existing or newly created caches. + * @throws CacheException If error occurs. + */ + public Collection getOrCreateCaches(Collection cacheCfgs) throws CacheException; /** * Adds cache configuration template. * * @param cacheCfg Cache configuration template. + * @throws CacheException If error occurs. */ - public void addCacheConfiguration(CacheConfiguration cacheCfg); + public void addCacheConfiguration(CacheConfiguration cacheCfg) throws CacheException; /** * Dynamically starts new cache with the given cache configuration. @@ -275,10 +309,11 @@ public interface Ignite extends AutoCloseable { * @param cacheCfg Cache configuration to use. * @param nearCfg Near cache configuration to use on local node in case it is not an * affinity node. + * @throws CacheException If a cache with the same name already exists or other error occurs. * @return Instance of started cache. */ public IgniteCache createCache(CacheConfiguration cacheCfg, - NearCacheConfiguration nearCfg); + NearCacheConfiguration nearCfg) throws CacheException; /** * Gets existing cache with the given cache configuration or creates one if it does not exist. @@ -293,9 +328,10 @@ public IgniteCache createCache(CacheConfiguration cacheCfg, * @param cacheCfg Cache configuration. * @param nearCfg Near cache configuration for client. * @return {@code IgniteCache} instance. + * @throws CacheException If error occurs. */ public IgniteCache getOrCreateCache(CacheConfiguration cacheCfg, - NearCacheConfiguration nearCfg); + NearCacheConfiguration nearCfg) throws CacheException; /** * Starts a near cache on local node if cache was previously started with one of the @@ -305,8 +341,10 @@ public IgniteCache getOrCreateCache(CacheConfiguration cacheC * @param cacheName Cache name. * @param nearCfg Near cache configuration. * @return Cache instance. + * @throws CacheException If error occurs. */ - public IgniteCache createNearCache(@Nullable String cacheName, NearCacheConfiguration nearCfg); + public IgniteCache createNearCache(@Nullable String cacheName, NearCacheConfiguration nearCfg) + throws CacheException; /** * Gets existing near cache with the given name or creates a new one. @@ -314,15 +352,26 @@ public IgniteCache getOrCreateCache(CacheConfiguration cacheC * @param cacheName Cache name. * @param nearCfg Near configuration. * @return {@code IgniteCache} instance. + * @throws CacheException If error occurs. */ - public IgniteCache getOrCreateNearCache(@Nullable String cacheName, NearCacheConfiguration nearCfg); + public IgniteCache getOrCreateNearCache(@Nullable String cacheName, NearCacheConfiguration nearCfg) + throws CacheException; /** * Stops dynamically started cache. * * @param cacheName Cache name to stop. + * @throws CacheException If error occurs. + */ + public void destroyCache(String cacheName) throws CacheException; + + /** + * Stops dynamically started caches. + * + * @param cacheNames Collection of cache names to stop. + * @throws CacheException If error occurs. */ - public void destroyCache(String cacheName); + public void destroyCaches(Collection cacheNames) throws CacheException; /** * Gets an instance of {@link IgniteCache} API. {@code IgniteCache} is a fully-compatible @@ -330,8 +379,9 @@ public IgniteCache getOrCreateCache(CacheConfiguration cacheC * * @param name Cache name. * @return Instance of the cache for the specified name. + * @throws CacheException If error occurs. */ - public IgniteCache cache(@Nullable String name); + public IgniteCache cache(@Nullable String name) throws CacheException; /** * Gets the collection of names of currently available caches. @@ -357,8 +407,9 @@ public IgniteCache getOrCreateCache(CacheConfiguration cacheC * * @param cacheName Cache name ({@code null} for default cache). * @return Data streamer. + * @throws IllegalStateException If node is stopping. */ - public IgniteDataStreamer dataStreamer(@Nullable String cacheName); + public IgniteDataStreamer dataStreamer(@Nullable String cacheName) throws IllegalStateException; /** * Gets an instance of IGFS (Ignite In-Memory File System). If one is not @@ -372,7 +423,7 @@ public IgniteCache getOrCreateCache(CacheConfiguration cacheC * @return IGFS instance. * @throws IllegalArgumentException If IGFS with such name is not configured. */ - public IgniteFileSystem fileSystem(String name); + public IgniteFileSystem fileSystem(String name) throws IllegalArgumentException; /** * Gets all instances of IGFS (Ignite In-Memory File System). diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index ef9c651ad99a2..c5365069b809b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -2541,6 +2541,33 @@ public IgniteInternalCache getCache(@Nullable String name) { } } + + /** {@inheritDoc} */ + @Override public Collection createCaches(Collection cacheCfgs) { + A.notNull(cacheCfgs, "cacheCfgs"); + + guard(); + + try { + ctx.cache().dynamicStartCaches(cacheCfgs, + true, + true).get(); + + List createdCaches = new ArrayList<>(cacheCfgs.size()); + + for (CacheConfiguration cacheCfg : cacheCfgs) + createdCaches.add(ctx.cache().publicJCache(cacheCfg.getName())); + + return createdCaches; + } + catch (IgniteCheckedException e) { + throw CU.convertToCacheException(e); + } + finally { + unguard(); + } + } + /** {@inheritDoc} */ @Override public IgniteCache createCache(String cacheName) { guard(); @@ -2584,6 +2611,32 @@ public IgniteInternalCache getCache(@Nullable String name) { } } + /** {@inheritDoc} */ + @Override public Collection getOrCreateCaches(Collection cacheCfgs) { + A.notNull(cacheCfgs, "cacheCfgs"); + + guard(); + + try { + ctx.cache().dynamicStartCaches(cacheCfgs, + false, + true).get(); + + List createdCaches = new ArrayList<>(cacheCfgs.size()); + + for (CacheConfiguration cacheCfg : cacheCfgs) + createdCaches.add(ctx.cache().publicJCache(cacheCfg.getName())); + + return createdCaches; + } + catch (IgniteCheckedException e) { + throw CU.convertToCacheException(e); + } + finally { + unguard(); + } + } + /** {@inheritDoc} */ @Override public IgniteCache createCache( CacheConfiguration cacheCfg, @@ -2745,6 +2798,18 @@ private void checkNearCacheStarted(IgniteCacheProxy cache) throws IgniteCh } } + /** {@inheritDoc} */ + @Override public void destroyCaches(Collection cacheNames) { + IgniteInternalFuture stopFut = destroyCachesAsync(cacheNames, true); + + try { + stopFut.get(); + } + catch (IgniteCheckedException e) { + throw CU.convertToCacheException(e); + } + } + /** * @param cacheName Cache name. * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. @@ -2761,6 +2826,22 @@ public IgniteInternalFuture destroyCacheAsync(String cacheName, boolean check } } + /** + * @param cacheNames Collection of cache names. + * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. + * @return Ignite future. + */ + public IgniteInternalFuture destroyCachesAsync(Collection cacheNames, boolean checkThreadTx) { + guard(); + + try { + return ctx.cache().dynamicDestroyCaches(cacheNames, checkThreadTx); + } + finally { + unguard(); + } + } + /** {@inheritDoc} */ @Override public IgniteCache getOrCreateCache(String cacheName) { guard(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 5e777fd8a39c2..0e0d76941ae49 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -2284,99 +2284,92 @@ public IgniteInternalFuture dynamicStartCache( if (checkThreadTx) checkEmptyTransactions(); - DynamicCacheDescriptor desc = registeredCaches.get(maskNull(cacheName)); - - DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId()); - - req.failIfExists(failIfExists); - - if (ccfg != null) { - try { - cloneCheckSerializable(ccfg); - } - catch (IgniteCheckedException e) { - return new GridFinishedFuture<>(e); - } - - if (desc != null) { - if (failIfExists) { - return new GridFinishedFuture<>(new CacheExistsException("Failed to start cache " + - "(a cache with the same name is already started): " + cacheName)); - } - else { - CacheConfiguration descCfg = desc.cacheConfiguration(); - - // Check if we were asked to start a near cache. - if (nearCfg != null) { - if (CU.affinityNode(ctx.discovery().localNode(), descCfg.getNodeFilter())) { - // If we are on a data node and near cache was enabled, return success, else - fail. - if (descCfg.getNearConfiguration() != null) - return new GridFinishedFuture<>(); - else - return new GridFinishedFuture<>(new IgniteCheckedException("Failed to start near " + - "cache (local node is an affinity node for cache): " + cacheName)); - } - else - // If local node has near cache, return success. - req.clientStartOnly(true); - } - else - req.clientStartOnly(true); + try { + DynamicCacheChangeRequest req = prepareCacheChangeRequest( + ccfg, + cacheName, + nearCfg, + cacheType, + failIfExists, + failIfNotStarted); - req.deploymentId(desc.deploymentId()); + if (req != null) + return F.first(initiateCacheChanges(F.asList(req), failIfExists)); + else + return new GridFinishedFuture<>(); + } + catch (Exception e) { + return new GridFinishedFuture<>(e); + } + } - req.startCacheConfiguration(descCfg); - } - } - else { - req.deploymentId(IgniteUuid.randomUuid()); + /** + * Dynamically starts multiple caches. + * + * @param ccfgList Collection of cache configuration. + * @param failIfExists Fail if exists flag. + * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. + * @return Future that will be completed when all caches are deployed. + */ + public IgniteInternalFuture dynamicStartCaches( + Collection ccfgList, + boolean failIfExists, + boolean checkThreadTx + ) { + return dynamicStartCaches(ccfgList, CacheType.USER, failIfExists, checkThreadTx); + } - try { - CacheConfiguration cfg = new CacheConfiguration(ccfg); + /** + * Dynamically starts multiple caches. + * + * @param ccfgList Collection of cache configuration. + * @param cacheType Cache type. + * @param failIfExists Fail if exists flag. + * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. + * @return Future that will be completed when all caches are deployed. + */ + private IgniteInternalFuture dynamicStartCaches( + Collection ccfgList, + CacheType cacheType, + boolean failIfExists, + boolean checkThreadTx + ) { + if (checkThreadTx) + checkEmptyTransactions(); - CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg); + List reqList = new ArrayList<>(ccfgList.size()); - initialize(false, cfg, cacheObjCtx); + try { + for (CacheConfiguration ccfg : ccfgList) { + DynamicCacheChangeRequest req = prepareCacheChangeRequest( + ccfg, + ccfg.getName(), + null, + cacheType, + failIfExists, + true + ); - req.startCacheConfiguration(cfg); - } - catch (IgniteCheckedException e) { - return new GridFinishedFuture(e); - } + if (req != null) + reqList.add(req); } } - else { - req.clientStartOnly(true); - - if (desc != null) - ccfg = desc.cacheConfiguration(); - - if (ccfg == null) { - if (failIfNotStarted) - return new GridFinishedFuture<>(new CacheExistsException("Failed to start client cache " + - "(a cache with the given name is not started): " + cacheName)); - else - return new GridFinishedFuture<>(); - } - - req.deploymentId(desc.deploymentId()); - req.startCacheConfiguration(ccfg); + catch (Exception e) { + return new GridFinishedFuture<>(e); } - // Fail cache with swap enabled creation on grid without swap space SPI. - if (ccfg.isSwapEnabled()) - for (ClusterNode n : ctx.discovery().allNodes()) - if (!GridCacheUtils.clientNode(n) && !GridCacheUtils.isSwapEnabled(n)) - return new GridFinishedFuture<>(new IgniteCheckedException("Failed to start cache " + - cacheName + " with swap enabled: Remote Node with ID " + n.id().toString().toUpperCase() + - " has not swap SPI configured")); + if (!reqList.isEmpty()) { + GridCompoundFuture compoundFut = new GridCompoundFuture<>(); - if (nearCfg != null) - req.nearCacheConfiguration(nearCfg); + for (DynamicCacheStartFuture fut : initiateCacheChanges(reqList, failIfExists)) + compoundFut.add((IgniteInternalFuture)fut); - req.cacheType(cacheType); + compoundFut.markInitialized(); - return F.first(initiateCacheChanges(F.asList(req), failIfExists)); + return compoundFut; + } + else + return new GridFinishedFuture<>(); } /** @@ -2395,6 +2388,35 @@ public IgniteInternalFuture dynamicDestroyCache(String cacheName, boolean che return F.first(initiateCacheChanges(F.asList(t), false)); } + /** + * @param cacheNames Collection of cache names to destroy. + * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. + * @return Future that will be completed when cache is destroyed. + */ + public IgniteInternalFuture dynamicDestroyCaches(Collection cacheNames, boolean checkThreadTx) { + if (checkThreadTx) + checkEmptyTransactions(); + + List reqs = new ArrayList<>(cacheNames.size()); + + for (String cacheName : cacheNames) { + DynamicCacheChangeRequest t = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId()); + + t.stop(true); + + reqs.add(t); + } + + GridCompoundFuture compoundFut = new GridCompoundFuture<>(); + + for (DynamicCacheStartFuture fut : initiateCacheChanges(reqs, false)) + compoundFut.add((IgniteInternalFuture)fut); + + compoundFut.markInitialized(); + + return compoundFut; + } + /** * @param cacheName Cache name to close. * @return Future that will be completed when cache is closed. @@ -2416,6 +2438,7 @@ public IgniteInternalFuture dynamicCloseCache(String cacheName) { /** * @param reqs Requests. + * @param failIfExists Fail if exists flag. * @return Collection of futures. */ @SuppressWarnings("TypeMayBeWeakened") @@ -3607,6 +3630,114 @@ private T withBinaryContext(IgniteOutClosureX c) throws IgniteCheckedExce } } + /** + * Prepares DynamicCacheChangeRequest for cache creation. + * + * @param ccfg Cache configuration + * @param cacheName Cache name + * @param nearCfg Near cache configuration + * @param cacheType Cache type + * @param failIfExists Fail if exists flag. + * @param failIfNotStarted If {@code true} fails if cache is not started. + * @return Request or {@code null} if cache already exists. + * @throws IgniteCheckedException if some of pre-checks failed + * @throws CacheExistsException if cache exists and failIfExists flag is {@code true} + */ + private DynamicCacheChangeRequest prepareCacheChangeRequest( + @Nullable CacheConfiguration ccfg, + String cacheName, + @Nullable NearCacheConfiguration nearCfg, + CacheType cacheType, + boolean failIfExists, + boolean failIfNotStarted + ) throws IgniteCheckedException { + DynamicCacheDescriptor desc = registeredCaches.get(maskNull(cacheName)); + + DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId()); + + req.failIfExists(failIfExists); + + if (ccfg != null) { + cloneCheckSerializable(ccfg); + + if (desc != null) { + if (failIfExists) { + throw new CacheExistsException("Failed to start cache " + + "(a cache with the same name is already started): " + cacheName); + } + else { + CacheConfiguration descCfg = desc.cacheConfiguration(); + + // Check if we were asked to start a near cache. + if (nearCfg != null) { + if (CU.affinityNode(ctx.discovery().localNode(), descCfg.getNodeFilter())) { + // If we are on a data node and near cache was enabled, return success, else - fail. + if (descCfg.getNearConfiguration() != null) + return null; + else + throw new IgniteCheckedException("Failed to start near " + + "cache (local node is an affinity node for cache): " + cacheName); + } + else + // If local node has near cache, return success. + req.clientStartOnly(true); + } + else + req.clientStartOnly(true); + + req.deploymentId(desc.deploymentId()); + + req.startCacheConfiguration(descCfg); + } + } + else { + req.deploymentId(IgniteUuid.randomUuid()); + + CacheConfiguration cfg = new CacheConfiguration(ccfg); + + CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg); + + initialize(false, cfg, cacheObjCtx); + + req.startCacheConfiguration(cfg); + } + } + else { + req.clientStartOnly(true); + + if (desc != null) + ccfg = desc.cacheConfiguration(); + + if (ccfg == null) { + if (failIfNotStarted) { + throw new CacheExistsException("Failed to start client cache " + + "(a cache with the given name is not started): " + cacheName); + } + else + return null; + } + + req.deploymentId(desc.deploymentId()); + req.startCacheConfiguration(ccfg); + } + + // Fail cache with swap enabled creation on grid without swap space SPI. + if (ccfg.isSwapEnabled()) + for (ClusterNode n : ctx.discovery().allNodes()) + if (!GridCacheUtils.clientNode(n) && !GridCacheUtils.isSwapEnabled(n)) { + throw new IgniteCheckedException("Failed to start cache " + + cacheName + " with swap enabled: Remote Node with ID " + n.id().toString().toUpperCase() + + " has not swap SPI configured"); + } + + if (nearCfg != null) + req.nearCacheConfiguration(nearCfg); + + req.cacheType(cacheType); + + return req; + } + /** * @param obj Object to clone. * @return Object copy. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java index c9cd750c72f7c..48e06ee7c82a0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java @@ -17,7 +17,9 @@ package org.apache.ignite.internal.processors.cache; +import java.util.ArrayList; import java.util.Collection; +import java.util.List; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.CountDownLatch; @@ -181,7 +183,8 @@ public void testStartStopCacheMultithreadedSameNode() throws Exception { info("Succeeded: " + System.identityHashCode(fut)); succeeded++; - } catch (IgniteCheckedException e) { + } + catch (IgniteCheckedException e) { info(e.getMessage()); failed++; @@ -246,7 +249,8 @@ public void testStartCacheMultithreadedDifferentNodes() throws Exception { info("Succeeded: " + System.identityHashCode(fut)); succeeded++; - } catch (IgniteCheckedException e) { + } + catch (IgniteCheckedException e) { info(e.getMessage()); failed++; @@ -288,6 +292,20 @@ public void testStartStopCacheSimpleAtomic() throws Exception { checkStartStopCacheSimple(CacheAtomicityMode.ATOMIC); } + /** + * @throws Exception If failed. + */ + public void testStartStopCachesSimpleTransactional() throws Exception { + checkStartStopCachesSimple(CacheAtomicityMode.TRANSACTIONAL); + } + + /** + * @throws Exception If failed. + */ + public void testStartStopCachesSimpleAtomic() throws Exception { + checkStartStopCachesSimple(CacheAtomicityMode.ATOMIC); + } + /** * @param mode Cache atomicity mode. * @throws Exception If failed. @@ -325,10 +343,10 @@ private void checkStartStopCacheSimple(CacheAtomicityMode mode) throws Exception for (int g = 0; g < nodeCount(); g++) caches[g] = grid(g).cache(DYNAMIC_CACHE_NAME); - kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get(); + kernal.destroyCache(DYNAMIC_CACHE_NAME); for (int g = 0; g < nodeCount(); g++) { - final IgniteKernal kernal0 = (IgniteKernal) grid(g); + final IgniteKernal kernal0 = (IgniteKernal)grid(g); final int idx = g; @@ -345,6 +363,87 @@ private void checkStartStopCacheSimple(CacheAtomicityMode mode) throws Exception } } + /** + * @param mode Cache atomicity mode. + * @throws Exception If failed. + */ + private void checkStartStopCachesSimple(CacheAtomicityMode mode) throws Exception { + final IgniteEx kernal = grid(0); + final int cacheCnt = 3; + + List ccfgList = new ArrayList<>(); + + for (int i = 0; i < cacheCnt; i++) { + CacheConfiguration ccfg = new CacheConfiguration(); + ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + ccfg.setAtomicityMode(mode); + ccfg.setName(DYNAMIC_CACHE_NAME + Integer.toString(i)); + + ccfgList.add(ccfg); + } + + kernal.createCaches(ccfgList); + + for (int g = 0; g < nodeCount(); g++) { + IgniteEx kernal0 = grid(g); + + for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures()) + f.get(); + + info("Getting cache for node: " + g); + + for (int i = 0; i < cacheCnt; i++) + assertNotNull(grid(g).cache(DYNAMIC_CACHE_NAME + Integer.toString(i))); + } + + for (int i = 0; i < cacheCnt; i++) + grid(0).cache(DYNAMIC_CACHE_NAME + Integer.toString(i)).put(Integer.toString(i), Integer.toString(i)); + + for (int g = 0; g < nodeCount(); g++) { + for (int i = 0; i < cacheCnt; i++) { + assertEquals( + Integer.toString(i), + grid(g).cache(DYNAMIC_CACHE_NAME + Integer.toString(i)).get(Integer.toString(i)) + ); + } + } + + // Grab caches before stop. + final IgniteCache[] caches = new IgniteCache[nodeCount() * cacheCnt]; + + for (int g = 0; g < nodeCount(); g++) { + for (int i = 0; i < cacheCnt; i++) + caches[g * nodeCount() + i] = grid(g).cache(DYNAMIC_CACHE_NAME + Integer.toString(i)); + } + + List namesToDestroy = new ArrayList<>(); + + for (int i = 0; i < cacheCnt; i++) + namesToDestroy.add(DYNAMIC_CACHE_NAME + Integer.toString(i)); + + kernal.destroyCaches(namesToDestroy); + + for (int g = 0; g < nodeCount(); g++) { + final IgniteKernal kernal0 = (IgniteKernal)grid(g); + + for (int i = 0; i < cacheCnt; i++) { + final int idx = g * nodeCount() + i; + final int expVal = i; + + for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures()) + f.get(); + + assertNull(kernal0.cache(DYNAMIC_CACHE_NAME)); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + return caches[idx].get(Integer.toString(expVal)); + } + }, IllegalStateException.class, null); + } + } + } + /** * @throws Exception If failed. */ @@ -378,13 +477,13 @@ public void testStartStopCacheAddNode() throws Exception { } // Undeploy cache. - kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get(); + kernal.destroyCache(DYNAMIC_CACHE_NAME); startGrid(nodeCount() + 1); // Check that cache is not deployed on new node after undeploy. for (int g = 0; g < nodeCount() + 2; g++) { - final IgniteKernal kernal0 = (IgniteKernal) grid(g); + final IgniteKernal kernal0 = (IgniteKernal)grid(g); for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures()) f.get(); @@ -431,16 +530,16 @@ public void testDeployFilter() throws Exception { for (int g = 0; g < nodeCount(); g++) { for (int i = 0; i < 100; i++) { assertFalse(grid(g).affinity(DYNAMIC_CACHE_NAME).mapKeyToPrimaryAndBackups(i) - .contains(grid(nodeCount()).cluster().localNode())); + .contains(grid(nodeCount()).cluster().localNode())); assertFalse(grid(g).affinity(DYNAMIC_CACHE_NAME).mapKeyToPrimaryAndBackups(i) - .contains(grid(nodeCount() + 1).cluster().localNode())); + .contains(grid(nodeCount() + 1).cluster().localNode())); } } // Check that cache is not deployed on new node after undeploy. for (int g = 0; g < nodeCount() + 2; g++) { - final IgniteKernal kernal0 = (IgniteKernal) grid(g); + final IgniteKernal kernal0 = (IgniteKernal)grid(g); for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures()) f.get(); @@ -455,7 +554,7 @@ public void testDeployFilter() throws Exception { }, IllegalArgumentException.class, null); } - kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get(); + kernal.destroyCache(DYNAMIC_CACHE_NAME); stopGrid(nodeCount() + 1); stopGrid(nodeCount()); @@ -486,6 +585,36 @@ public void testFailWhenConfiguredCacheExists() throws Exception { }, CacheExistsException.class, null); } + /** + * @throws Exception If failed. + */ + public void testFailWhenOneOfConfiguredCacheExists() throws Exception { + GridTestUtils.assertThrowsInherited(log, new Callable() { + @Override public Object call() throws Exception { + final Ignite kernal = grid(0); + + CacheConfiguration ccfgDynamic = new CacheConfiguration(); + ccfgDynamic.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + + ccfgDynamic.setName(DYNAMIC_CACHE_NAME); + + ccfgDynamic.setNodeFilter(NODE_FILTER); + + CacheConfiguration ccfgStatic = new CacheConfiguration(); + ccfgStatic.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + + // Cache is already configured, should fail. + ccfgStatic.setName(STATIC_CACHE_NAME); + + ccfgStatic.setNodeFilter(NODE_FILTER); + + return kernal.createCaches(F.asList(ccfgDynamic, ccfgStatic)); + } + }, CacheExistsException.class, null); + + assertNull(grid(0).cache(DYNAMIC_CACHE_NAME)); + } + /** * @throws Exception If failed. */ @@ -522,7 +651,7 @@ public void testClientCache() throws Exception { for (int g = 0; g < nodeCount() + 1; g++) assertEquals("1", ignite(g).cache(DYNAMIC_CACHE_NAME).get("1")); - kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get(); + kernal.destroyCache(DYNAMIC_CACHE_NAME); } finally { stopGrid(nodeCount()); @@ -547,7 +676,7 @@ public void testStartFromClientNode() throws Exception { ccfg.setNodeFilter(NODE_FILTER); - final IgniteKernal started = (IgniteKernal) grid(nodeCount()); + final IgniteKernal started = (IgniteKernal)grid(nodeCount()); started.createCache(ccfg); @@ -564,14 +693,13 @@ public void testStartFromClientNode() throws Exception { for (int g = 0; g < nodeCount() + 1; g++) assertEquals("1", ignite(g).cache(DYNAMIC_CACHE_NAME).get("1")); - kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get(); + kernal.destroyCache(DYNAMIC_CACHE_NAME); } finally { stopGrid(nodeCount()); } } - /** * @throws Exception If failed. */ @@ -610,7 +738,7 @@ public void testStartNearCacheFromClientNode() throws Exception { for (int g = 0; g < nodeCount() + 1; g++) assertEquals("1", ignite(g).cache(DYNAMIC_CACHE_NAME).get("1")); - kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get(); + kernal.destroyCache(DYNAMIC_CACHE_NAME); } finally { stopGrid(nodeCount()); @@ -760,7 +888,7 @@ public void testGetOrCreate() throws Exception { nearGrid.getOrCreateNearCache(DYNAMIC_CACHE_NAME, new NearCacheConfiguration()); GridCacheContext nCtx = ((IgniteKernal)nearGrid) - .internalCache(DYNAMIC_CACHE_NAME).context(); + .internalCache(DYNAMIC_CACHE_NAME).context(); assertTrue(nCtx.isNear()); assertFalse(nCtx.affinityNode()); @@ -771,11 +899,12 @@ public void testGetOrCreate() throws Exception { clientGrid.getOrCreateCache(cfg); GridCacheContext cCtx = ((IgniteKernal)clientGrid) - .internalCache(DYNAMIC_CACHE_NAME).context(); + .internalCache(DYNAMIC_CACHE_NAME).context(); assertFalse(cCtx.isNear()); assertFalse(cCtx.affinityNode()); - } finally { + } + finally { stopGrid(nodeCount() + 1); stopGrid(nodeCount()); } @@ -785,6 +914,40 @@ public void testGetOrCreate() throws Exception { } } + /** {@inheritDoc} */ + public void testGetOrCreateCollection() throws Exception { + final int cacheCnt = 3; + + try { + final Collection ccfgs = new ArrayList<>(); + + for (int i = 0; i < cacheCnt; i++) { + final CacheConfiguration cfg = new CacheConfiguration(); + + cfg.setName(DYNAMIC_CACHE_NAME + Integer.toString(i)); + cfg.setNodeFilter(NODE_FILTER); + + ccfgs.add(cfg); + + grid(0).getOrCreateCaches(ccfgs); + } + + for (int i = 0; i < cacheCnt; i++) { + assertNotNull(grid(0).cache(DYNAMIC_CACHE_NAME + Integer.toString(i))); + + IgniteCache jcache = grid(0).cache(DYNAMIC_CACHE_NAME + Integer.toString(i)); + + jcache.put(Integer.toString(i), Integer.toString(i)); + + assertEquals(jcache.get(Integer.toString(i)), Integer.toString(i)); + } + } + finally { + for (int i = 0; i < cacheCnt; i++) + grid(0).destroyCache(DYNAMIC_CACHE_NAME + Integer.toString(i)); + } + } + /** * @throws Exception If failed. */ @@ -813,7 +976,7 @@ public void testGetOrCreateMultiNode() throws Exception { assertNull(err.get()); for (int i = 0; i < nodeCount(); i++) { - GridCacheContext ctx = ((IgniteKernal) ignite(i)).internalCache(DYNAMIC_CACHE_NAME) + GridCacheContext ctx = ((IgniteKernal)ignite(i)).internalCache(DYNAMIC_CACHE_NAME) .context(); assertTrue(ctx.affinityNode()); @@ -906,7 +1069,7 @@ public void checkGetOrCreateNear(final boolean nearOnly) throws Exception { assertNull(err.get()); for (int i = 0; i < nodeCount(); i++) { - GridCacheContext ctx = ((IgniteKernal) ignite(i)).internalCache(DYNAMIC_CACHE_NAME) + GridCacheContext ctx = ((IgniteKernal)ignite(i)).internalCache(DYNAMIC_CACHE_NAME) .context(); assertTrue(ctx.affinityNode()); @@ -914,7 +1077,7 @@ public void checkGetOrCreateNear(final boolean nearOnly) throws Exception { } for (int i = 0; i < clientCnt; i++) { - GridCacheContext ctx = ((IgniteKernal) ignite(nodeCount() + i)) + GridCacheContext ctx = ((IgniteKernal)ignite(nodeCount() + i)) .internalCache(DYNAMIC_CACHE_NAME).context(); assertFalse(ctx.affinityNode()); @@ -995,12 +1158,12 @@ public void testServerNodesLeftEvent() throws Exception { for (int i = 0; i < nodeCount(); i++) { final int idx = i; - latches[i] = new CountDownLatch(1); - lsnrs[i] = new IgnitePredicate() { - @Override public boolean apply(CacheEvent e) { - switch (e.type()) { - case EventType.EVT_CACHE_NODES_LEFT: - latches[idx].countDown(); + latches[i] = new CountDownLatch(1); + lsnrs[i] = new IgnitePredicate() { + @Override public boolean apply(CacheEvent e) { + switch (e.type()) { + case EventType.EVT_CACHE_NODES_LEFT: + latches[idx].countDown(); break; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java index c9f77cd3fa553..1b779c2801e02 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java @@ -290,6 +290,13 @@ public IgfsIgniteMock(@Nullable String name, IgniteFileSystem igfs) { return null; } + /** {@inheritDoc} */ + @Override public Collection createCaches(Collection cacheCfgs) { + throwUnsupported(); + + return null; + } + /** {@inheritDoc} */ @Override public IgniteCache createCache(String cacheName) { throwUnsupported(); @@ -311,6 +318,13 @@ public IgfsIgniteMock(@Nullable String name, IgniteFileSystem igfs) { return null; } + /** {@inheritDoc} */ + @Override public Collection getOrCreateCaches(Collection cacheCfgs) { + throwUnsupported(); + + return null; + } + /** {@inheritDoc} */ @Override public void addCacheConfiguration(CacheConfiguration cacheCfg) { throwUnsupported(); @@ -353,6 +367,11 @@ public IgfsIgniteMock(@Nullable String name, IgniteFileSystem igfs) { throwUnsupported(); } + /** {@inheritDoc} */ + @Override public void destroyCaches(Collection cacheNames) { + throwUnsupported(); + } + /** {@inheritDoc} */ @Override public IgniteCache cache(@Nullable String name) { throwUnsupported(); diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java index b559897705c5b..5722fa3d6056a 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java @@ -229,6 +229,11 @@ public IgniteMock( return null; } + /** {@inheritDoc} */ + @Override public Collection createCaches(Collection cacheCfgs) { + return null; + } + /** {@inheritDoc} */ @Override public IgniteCache getOrCreateCache(CacheConfiguration cacheCfg) { return null; @@ -260,6 +265,11 @@ public IgniteMock( return null; } + /** {@inheritDoc} */ + @Override public Collection getOrCreateCaches(Collection cacheCfgs) { + return null; + } + /** {@inheritDoc} */ @Override public IgniteCache createCache(String cacheName) { return null; @@ -275,6 +285,11 @@ public IgniteMock( // No-op. } + /** {@inheritDoc} */ + @Override public void destroyCaches(Collection cacheNames) { + // No-op. + } + /** {@inheritDoc} */ @Override public IgniteTransactions transactions() { return null; diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java index 2598bc53afc65..21fc28dae7aa7 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java @@ -425,6 +425,11 @@ public UUID getId() { throw new UnsupportedOperationException("Operation isn't supported yet."); } + /** {@inheritDoc} */ + @Override public Collection createCaches(Collection cacheCfgs) { + throw new UnsupportedOperationException("Operation isn't supported yet."); + } + /** {@inheritDoc} */ @Override public IgniteCache createCache(String cacheName) { throw new UnsupportedOperationException("Operation isn't supported yet."); @@ -440,6 +445,11 @@ public UUID getId() { throw new UnsupportedOperationException("Operation isn't supported yet."); } + /** {@inheritDoc} */ + @Override public Collection getOrCreateCaches(Collection cacheCfgs) { + throw new UnsupportedOperationException("Operation isn't supported yet."); + } + /** {@inheritDoc} */ @Override public void addCacheConfiguration(CacheConfiguration cacheCfg) { throw new UnsupportedOperationException("Operation isn't supported yet."); @@ -476,6 +486,11 @@ public UUID getId() { throw new UnsupportedOperationException("Operation isn't supported yet."); } + /** {@inheritDoc} */ + @Override public void destroyCaches(Collection cacheNames) { + throw new UnsupportedOperationException("Operation isn't supported yet."); + } + /** {@inheritDoc} */ @Override public IgniteCache cache(@Nullable final String name) { return new IgniteCacheProcessProxy<>(name, this); diff --git a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java index 7f79c0e90ad56..0a4bf202420e5 100644 --- a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java +++ b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java @@ -301,6 +301,13 @@ public ApplicationContext getApplicationContext() throws BeansException { return g.createCache(cacheCfg, nearCfg); } + /** {@inheritDoc} */ + @Override public Collection createCaches(Collection cacheCfgs) { + checkIgnite(); + + return g.createCaches(cacheCfgs); + } + /** {@inheritDoc} */ @Override public IgniteCache getOrCreateCache(CacheConfiguration cacheCfg, NearCacheConfiguration nearCfg) { checkIgnite(); @@ -329,6 +336,13 @@ public ApplicationContext getApplicationContext() throws BeansException { return g.getOrCreateCache(cacheName); } + /** {@inheritDoc} */ + @Override public Collection getOrCreateCaches(Collection cacheCfgs) { + checkIgnite(); + + return g.getOrCreateCaches(cacheCfgs); + } + /** {@inheritDoc} */ @Override public IgniteCache createCache(String cacheName) { checkIgnite(); @@ -350,6 +364,13 @@ public ApplicationContext getApplicationContext() throws BeansException { g.destroyCache(cacheName); } + /** {@inheritDoc} */ + @Override public void destroyCaches(Collection cacheNames) { + checkIgnite(); + + g.destroyCaches(cacheNames); + } + /** {@inheritDoc} */ @Override public IgniteTransactions transactions() { checkIgnite(); From 3e2ccfd30427ba0552eea8667c0129ae5ace9c0b Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Fri, 25 Nov 2016 14:26:54 +0300 Subject: [PATCH 374/487] IGNITE-4299: Fixes for examples. --- .../cpp/examples/putget-example/src/putget_example.cpp | 2 +- .../platforms/cpp/examples/query-example/src/query_example.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp b/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp index 0b57886c462dd..8bf9c8c1feffb 100644 --- a/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp +++ b/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp @@ -117,7 +117,7 @@ int main() } std::cout << std::endl; - std::cout << ">>> Example finished, press any key to exit ..." << std::endl; + std::cout << ">>> Example finished, press 'Enter' to exit ..." << std::endl; std::cout << std::endl; std::cin.get(); diff --git a/modules/platforms/cpp/examples/query-example/src/query_example.cpp b/modules/platforms/cpp/examples/query-example/src/query_example.cpp index 9bf3e52ee873b..8c2ca0c5569cc 100644 --- a/modules/platforms/cpp/examples/query-example/src/query_example.cpp +++ b/modules/platforms/cpp/examples/query-example/src/query_example.cpp @@ -450,7 +450,7 @@ int main() } std::cout << std::endl; - std::cout << ">>> Example finished, press any key to exit ..." << std::endl; + std::cout << ">>> Example finished, press 'Enter' to exit ..." << std::endl; std::cout << std::endl; std::cin.get(); From 6fbaef45af8f40062a95058df7ec0984c99035b9 Mon Sep 17 00:00:00 2001 From: Konstantin Dudkov Date: Fri, 25 Nov 2016 13:58:58 +0300 Subject: [PATCH 375/487] IGNITE-4305 marshalling fix in GridNearAtomicSingleUpdateInvokeRequest --- .../dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java index 42b51d6161ec3..238db8bc39035 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java @@ -202,8 +202,10 @@ public GridNearAtomicSingleUpdateInvokeRequest() { if (!addDepInfo && ctx.deploymentEnabled()) addDepInfo = true; - if (entryProcessor != null && entryProcessorBytes == null) + if (entryProcessor != null && entryProcessorBytes == null) { + prepareObject(entryProcessor, cctx); entryProcessorBytes = CU.marshal(cctx, entryProcessor); + } if (invokeArgsBytes == null) invokeArgsBytes = marshalInvokeArguments(invokeArgs, cctx); From 1a2de51f5807a91ce0d5dff28f24ed5bf7abebbc Mon Sep 17 00:00:00 2001 From: Konstantin Dudkov Date: Mon, 28 Nov 2016 12:59:02 +0300 Subject: [PATCH 376/487] IGNITE-4305 marshalling fix --- .../dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java index 238db8bc39035..df9e38441e79f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java @@ -203,7 +203,8 @@ public GridNearAtomicSingleUpdateInvokeRequest() { addDepInfo = true; if (entryProcessor != null && entryProcessorBytes == null) { - prepareObject(entryProcessor, cctx); + if (addDepInfo) + prepareObject(entryProcessor, cctx); entryProcessorBytes = CU.marshal(cctx, entryProcessor); } From c06e4017771603df7118974758d3d6b9cadc41b5 Mon Sep 17 00:00:00 2001 From: Eduard Shangareev Date: Wed, 30 Nov 2016 14:34:47 +0300 Subject: [PATCH 377/487] ignite-4332 Usage of cache.getEntry inside GridCacheQueryManager.runQuery causes to remote operations --- .../cache/query/GridCacheQueryManager.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) 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 ab8bd63ee75da..d4decb47ae489 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 @@ -57,6 +57,7 @@ import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheEntryImpl; import org.apache.ignite.internal.processors.cache.CacheMetricsImpl; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; @@ -1560,9 +1561,12 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { metrics.addGetTimeNanos(System.nanoTime() - start); } + K key0 = null; + V val0 = null; + if (readEvt) { - K key0 = (K)cctx.unwrapBinaryIfNeeded(key, qry.keepBinary()); - V val0 = (V)cctx.unwrapBinaryIfNeeded(val, qry.keepBinary()); + key0 = (K)cctx.unwrapBinaryIfNeeded(key, qry.keepBinary()); + val0 = (V)cctx.unwrapBinaryIfNeeded(val, qry.keepBinary()); switch (type) { case SQL: @@ -1631,12 +1635,12 @@ protected void runQuery(GridCacheQueryInfo qryInfo) { } if (rdc != null || trans != null) { - Cache.Entry entry; + if (key0 == null) + key0 = (K)cctx.unwrapBinaryIfNeeded(key, qry.keepBinary()); + if (val0 == null) + val0 = (V)cctx.unwrapBinaryIfNeeded(val, qry.keepBinary()); - if (qry.keepBinary()) - entry = cache.keepBinary().getEntry(key); - else - entry = cache.getEntry(key); + Cache.Entry entry = new CacheEntryImpl(key0, val0); // Reduce. if (rdc != null) { From 066691098797be8c01daa0e8dc2ba94d4ad73561 Mon Sep 17 00:00:00 2001 From: sboikov Date: Thu, 1 Dec 2016 17:16:28 +0300 Subject: [PATCH 378/487] ignite-4344 Do not create offheap map on client nodes. --- .../processors/cache/GridCacheProcessor.java | 10 +- .../cache/OffheapCacheOnClientsTest.java | 143 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite2.java | 2 + 3 files changed, 150 insertions(+), 5 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/OffheapCacheOnClientsTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 0e0d76941ae49..0be2072e3766d 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -1286,10 +1286,12 @@ private GridCacheContext createCache(CacheConfiguration cfg, U.startLifecycleAware(lifecycleAwares(cfg, cfgStore)); + boolean affNode = CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter()); + GridCacheAffinityManager affMgr = new GridCacheAffinityManager(); GridCacheEventManager evtMgr = new GridCacheEventManager(); - GridCacheSwapManager swapMgr = new GridCacheSwapManager(cfg.getCacheMode() == LOCAL || - !GridCacheUtils.isNearEnabled(cfg)); + GridCacheSwapManager swapMgr = new GridCacheSwapManager( + affNode && (cfg.getCacheMode() == LOCAL || !GridCacheUtils.isNearEnabled(cfg))); GridCacheEvictionManager evictMgr = new GridCacheEvictionManager(); GridCacheQueryManager qryMgr = queryManager(cfg); CacheContinuousQueryManager contQryMgr = new CacheContinuousQueryManager(); @@ -1302,8 +1304,6 @@ private GridCacheContext createCache(CacheConfiguration cfg, storeMgr.initialize(cfgStore, sesHolders); - boolean affNode = CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter()); - GridCacheContext cacheCtx = new GridCacheContext( ctx, sharedCtx, @@ -1427,7 +1427,7 @@ private GridCacheContext createCache(CacheConfiguration cfg, * 7. GridCacheTtlManager. * =============================================== */ - swapMgr = new GridCacheSwapManager(true); + swapMgr = new GridCacheSwapManager(affNode); evictMgr = new GridCacheEvictionManager(); evtMgr = new GridCacheEventManager(); pluginMgr = new CachePluginManager(ctx, cfg); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/OffheapCacheOnClientsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/OffheapCacheOnClientsTest.java new file mode 100644 index 0000000000000..90985b60dce5b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/OffheapCacheOnClientsTest.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.concurrent.ConcurrentMap; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.processors.offheap.GridOffHeapProcessor; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.cache.CacheMemoryMode.*; +import static org.apache.ignite.cache.CacheMode.*; + +/** + * + */ +public class OffheapCacheOnClientsTest extends GridCommonAbstractTest { + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final String CACHE_NAME = "CACHE_NAME"; + + /** */ + private boolean client; + + /** */ + private boolean forceSrvMode; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(0); + + client = true; + + startGrid(1); + + forceSrvMode = true; + + startGrid(2); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + + if (client) { + cfg.setClientMode(true); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(forceSrvMode); + } + + return cfg; + } + /** + * @throws Exception If failed. + */ + public void testOffheapCacheOnClient() throws Exception { + try { + Ignite client = grid(1); + + testStartCacheOnClient(client, OFFHEAP_TIERED); + testStartCacheOnClient(client, OFFHEAP_VALUES); + testStartCacheOnClient(client, ONHEAP_TIERED); + + client = grid(2); + + testStartCacheOnClient(client, OFFHEAP_TIERED); + testStartCacheOnClient(client, OFFHEAP_VALUES); + testStartCacheOnClient(client, ONHEAP_TIERED); + } + finally { + grid(0).destroyCache(CACHE_NAME); + } + } + + /** + * @param client Node. + * @param memMode Memory mode. + * @throws Exception If failed. + */ + private void testStartCacheOnClient(Ignite client, CacheMemoryMode memMode) throws Exception { + assertTrue(client.configuration().isClientMode()); + + try { + client.createCache(new CacheConfiguration(CACHE_NAME) + .setCacheMode(REPLICATED) + .setOffHeapMaxMemory(1024 * 1024) + .setMemoryMode(memMode)); + + IgniteCache cache = client.cache(CACHE_NAME); + + assertNotNull(cache); + + cache.put(1, 1); + assertEquals((Integer)1, cache.get(1)); + + GridOffHeapProcessor offheap = ((IgniteKernal)client).cachex(CACHE_NAME).context().offheap(); + + assertNotNull(offheap); + + ConcurrentMap offheapMaps = GridTestUtils.getFieldValue(offheap, "offheap"); + assertNotNull(offheapMaps); + + assertEquals(0,offheapMaps.size()); + } + finally { + client.destroyCache(CACHE_NAME); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java index 6edfd094f7df0..f632f677ff0a0 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java @@ -39,6 +39,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheIncrementTxTest; import org.apache.ignite.internal.processors.cache.IgniteCachePartitionMapUpdateTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheAndNodeStop; +import org.apache.ignite.internal.processors.cache.OffheapCacheOnClientsTest; import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTest; import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTestAllowOverwrite; import org.apache.ignite.internal.processors.cache.distributed.CacheLockReleaseNodeLeaveTest; @@ -266,6 +267,7 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(IgniteNoCustomEventsOnNodeStart.class)); suite.addTest(new TestSuite(CacheExchangeMessageDuplicatedStateTest.class)); + suite.addTest(new TestSuite(OffheapCacheOnClientsTest.class)); return suite; } From e9ace7730773a6d4a1d30b271854f1fe8a7ba632 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 2 Dec 2016 16:06:41 +0700 Subject: [PATCH 379/487] Improved exception handling. --- .../org/apache/ignite/marshaller/jdk/JdkMarshaller.java | 4 ++-- .../ignite/marshaller/optimized/OptimizedMarshaller.java | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java index 54172dcc5b052..06b7109fdcd22 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java @@ -121,8 +121,8 @@ public class JdkMarshaller extends AbstractNodeNameAwareMarshaller { } catch (ClassNotFoundException e) { throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " + - "(make sure same versions of all classes are available on all nodes or enable peer-class-loading): " + - clsLdr, e); + "(make sure same versions of all classes are available on all nodes or enable peer-class-loading) " + + "[clsLdr=" + clsLdr + ", cls=" + e.getMessage() + "]", e); } catch (Exception e) { throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e); diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java index 467dddf1abd39..6d57864245659 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java @@ -219,8 +219,8 @@ public void setPoolSize(int poolSize) { } catch (ClassNotFoundException e) { throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " + - "(make sure same versions of all classes are available on all nodes or enable peer-class-loading): " + - clsLdr, e); + "(make sure same versions of all classes are available on all nodes or enable peer-class-loading) " + + "[clsLdr=" + clsLdr + ", cls=" + e.getMessage() + "]", e); } catch (Exception e) { throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e); @@ -248,8 +248,8 @@ public void setPoolSize(int poolSize) { } catch (ClassNotFoundException e) { throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " + - "(make sure same version of all classes are available on all nodes or enable peer-class-loading): " + - clsLdr, e); + "(make sure same version of all classes are available on all nodes or enable peer-class-loading)" + + " [clsLdr=" + clsLdr + ", cls=" + e.getMessage() + "]", e); } catch (Exception e) { throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e); From 12bdd6a031a33eda004a66e73cee9628f073ed68 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 2 Dec 2016 16:09:29 +0700 Subject: [PATCH 380/487] Updated classnames.properties for running nodes in IDE. --- .../resources/META-INF/classnames.properties | 86 +++++++++---- .../resources/META-INF/classnames.properties | 114 ++++++++++++++++++ 2 files changed, 176 insertions(+), 24 deletions(-) create mode 100644 modules/hadoop/src/main/resources/META-INF/classnames.properties diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties index 71d236fdb8ce4..4c9596c854ea0 100644 --- a/modules/core/src/main/resources/META-INF/classnames.properties +++ b/modules/core/src/main/resources/META-INF/classnames.properties @@ -75,11 +75,13 @@ org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy$HolderComparator org.apache.ignite.cache.query.CacheQueryEntryEvent org.apache.ignite.cache.query.ContinuousQuery org.apache.ignite.cache.query.Query +org.apache.ignite.cache.query.QueryCancelledException org.apache.ignite.cache.query.ScanQuery org.apache.ignite.cache.query.SpiQuery org.apache.ignite.cache.query.SqlFieldsQuery org.apache.ignite.cache.query.SqlQuery org.apache.ignite.cache.query.TextQuery +org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$2 org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$EntryMapping$1 org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$EntryMapping$2 org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$TypeKind @@ -366,6 +368,7 @@ org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakQu org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakReferenceCloseableIterator org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest +org.apache.ignite.internal.processors.cache.EntryProcessorResourceInjectorProxy org.apache.ignite.internal.processors.cache.GridCacheAdapter org.apache.ignite.internal.processors.cache.GridCacheAdapter$10 org.apache.ignite.internal.processors.cache.GridCacheAdapter$11 @@ -520,10 +523,8 @@ org.apache.ignite.internal.processors.cache.GridCacheUtils$15 org.apache.ignite.internal.processors.cache.GridCacheUtils$16 org.apache.ignite.internal.processors.cache.GridCacheUtils$17 org.apache.ignite.internal.processors.cache.GridCacheUtils$18 -org.apache.ignite.internal.processors.cache.GridCacheUtils$19 org.apache.ignite.internal.processors.cache.GridCacheUtils$2 org.apache.ignite.internal.processors.cache.GridCacheUtils$20 -org.apache.ignite.internal.processors.cache.GridCacheUtils$22 org.apache.ignite.internal.processors.cache.GridCacheUtils$3 org.apache.ignite.internal.processors.cache.GridCacheUtils$4 org.apache.ignite.internal.processors.cache.GridCacheUtils$5 @@ -533,6 +534,7 @@ org.apache.ignite.internal.processors.cache.GridCacheUtils$8 org.apache.ignite.internal.processors.cache.GridCacheUtils$9 org.apache.ignite.internal.processors.cache.GridCacheValueCollection org.apache.ignite.internal.processors.cache.GridCacheValueCollection$1 +org.apache.ignite.internal.processors.cache.GridDeferredAckMessageSender$DeferredAckMessageBuffer org.apache.ignite.internal.processors.cache.IgniteCacheProxy org.apache.ignite.internal.processors.cache.IgniteCacheProxy$1 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$10 @@ -545,12 +547,13 @@ org.apache.ignite.internal.processors.cache.IgniteCacheProxy$8 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$9 org.apache.ignite.internal.processors.cache.KeyCacheObject org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl +org.apache.ignite.internal.processors.cache.QueryCursorImpl$State org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityProxy org.apache.ignite.internal.processors.cache.binary.BinaryMetadataKey org.apache.ignite.internal.processors.cache.binary.CacheDefaultBinaryAffinityKeyMapper org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$1 -org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$4 org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$5 +org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$6 org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$MetaDataEntryFilter org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$MetaDataPredicate org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$MetadataProcessor @@ -630,6 +633,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocal$2 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter$1 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter$2 +org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxOnePhaseCommitAckRequest org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$1 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$2 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$3 @@ -648,6 +652,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFu org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture$1 org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture$2 org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture$3 +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateRequest org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$10 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$11 @@ -670,26 +675,30 @@ org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomic org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$27 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$28 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$29 -org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$3 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$30 +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$31 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$4 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$5 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$6 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$7 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$8 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$9 -org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$DeferredResponseBuffer org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicSingleUpdateRequest org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractSingleUpdateRequest +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateFuture$1 +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateRequest +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFilterRequest org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture$1 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture$2 -org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture$3 +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateInvokeRequest +org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$1 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$2 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$3 -org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$4 -org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedCache org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedCache$2 @@ -760,6 +769,7 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$ org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$2 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$3 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$4 +org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$LockTimeoutObject$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$MiniFuture$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockResponse @@ -772,6 +782,7 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticS org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$2 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$3 +org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$4 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$MiniFuture$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFutureAdapter$1 org.apache.ignite.internal.processors.cache.distributed.near.GridNearPessimisticTxPrepareFuture$1 @@ -821,20 +832,21 @@ org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManag org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$7 org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$1 org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$ScanQueryFallbackClosableIterator +org.apache.ignite.internal.processors.cache.query.GridCacheQueryDetailMetricsAdapter org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$1 org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$2 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$1$1 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$1$2 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$11 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$12 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$13 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$14 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$15$1 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$2 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$3 -org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$4 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$15 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$16 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$17 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$18$1 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$4$1 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$4$2 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$5 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$6 +org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$7 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$8 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$9 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$CacheSqlIndexMetadata @@ -896,6 +908,8 @@ org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$12 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$13 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$14 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$15 +org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$16 +org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$17 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$2 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$3 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$4 @@ -929,9 +943,9 @@ org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$Po org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostLockClosure1$4 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostLockClosure2 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostMissClosure -org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$2 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$3 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$4 +org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$5 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$CommitListener org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$CommittedVersion org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$NodeFailureTimeoutObject$1 @@ -998,6 +1012,7 @@ org.apache.ignite.internal.processors.continuous.StopRoutineAckDiscoveryMessage org.apache.ignite.internal.processors.continuous.StopRoutineDiscoveryMessage org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor$3 org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor$4 +org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor$5 org.apache.ignite.internal.processors.datastreamer.DataStreamerCacheUpdaters$Batched org.apache.ignite.internal.processors.datastreamer.DataStreamerCacheUpdaters$BatchedSorted org.apache.ignite.internal.processors.datastreamer.DataStreamerCacheUpdaters$Individual @@ -1005,9 +1020,9 @@ org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$1 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$4 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$5 -org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$6 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$Buffer$1 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$Buffer$2 +org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$Buffer$3 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$DataStreamerPda org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$DefaultIoPolicyResolver org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$IsolatedUpdater @@ -1084,6 +1099,7 @@ org.apache.ignite.internal.processors.datastructures.GridCacheSetProxy org.apache.ignite.internal.processors.datastructures.GridSetQueryPredicate org.apache.ignite.internal.processors.dr.GridDrType org.apache.ignite.internal.processors.dr.IgniteDrDataStreamerCacheUpdater +org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo org.apache.ignite.internal.processors.hadoop.HadoopFileBlock org.apache.ignite.internal.processors.hadoop.HadoopInputSplit org.apache.ignite.internal.processors.hadoop.HadoopJobId @@ -1119,7 +1135,6 @@ org.apache.ignite.internal.processors.igfs.IgfsFragmentizerManager$IdentityHashS org.apache.ignite.internal.processors.igfs.IgfsFragmentizerRequest org.apache.ignite.internal.processors.igfs.IgfsFragmentizerResponse org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse -org.apache.ignite.internal.processors.igfs.IgfsImpl$12$1 org.apache.ignite.internal.processors.igfs.IgfsImpl$IgfsGlobalSpaceTask org.apache.ignite.internal.processors.igfs.IgfsImpl$IgfsGlobalSpaceTask$1 org.apache.ignite.internal.processors.igfs.IgfsInputStreamDescriptor @@ -1155,6 +1170,7 @@ org.apache.ignite.internal.processors.igfs.client.IgfsClientSummaryCallable org.apache.ignite.internal.processors.igfs.client.IgfsClientUpdateCallable org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaIdsForPathCallable org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaInfoForPathCallable +org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaUnlockCallable org.apache.ignite.internal.processors.igfs.data.IgfsDataPutProcessor org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingAddProcessor @@ -1172,6 +1188,8 @@ org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdateTimesProcessor org.apache.ignite.internal.processors.job.GridJobProcessor$5 org.apache.ignite.internal.processors.job.GridJobWorker$3 org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor$SnapshotReducer +org.apache.ignite.internal.processors.odbc.OdbcProtocolVersion +org.apache.ignite.internal.processors.odbc.escape.OdbcEscapeType org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate org.apache.ignite.internal.processors.platform.PlatformEventFilterListener @@ -1180,7 +1198,8 @@ org.apache.ignite.internal.processors.platform.PlatformExtendedException org.apache.ignite.internal.processors.platform.PlatformJavaObjectFactoryProxy org.apache.ignite.internal.processors.platform.PlatformNativeException org.apache.ignite.internal.processors.platform.PlatformNoCallbackException -org.apache.ignite.internal.processors.platform.cache.PlatformCache$1 +org.apache.ignite.internal.processors.platform.PlatformProcessorImpl$1 +org.apache.ignite.internal.processors.platform.cache.PlatformCache$5 org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilterImpl org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessor @@ -1211,6 +1230,8 @@ org.apache.ignite.internal.processors.platform.cpp.PlatformCppConfigurationClosu org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiver org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiverImpl org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$1 +org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$10 +org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$11 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$2 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$3 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$4 @@ -1218,6 +1239,7 @@ org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$5 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$6 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$7 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$8 +org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$9 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetConfigurationClosure org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetService org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetServiceImpl @@ -1232,19 +1254,24 @@ org.apache.ignite.internal.processors.platform.transactions.PlatformTransactions org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$1 org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$FutureListenable$1 org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$InternalFutureListenable$1 +org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionLockProcessor +org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionSetAndUnlockProcessor org.apache.ignite.internal.processors.query.GridQueryFieldMetadata org.apache.ignite.internal.processors.query.GridQueryIndexType -org.apache.ignite.internal.processors.query.GridQueryProcessor$2 org.apache.ignite.internal.processors.query.GridQueryProcessor$3 org.apache.ignite.internal.processors.query.GridQueryProcessor$4 org.apache.ignite.internal.processors.query.GridQueryProcessor$5 org.apache.ignite.internal.processors.query.GridQueryProcessor$6 +org.apache.ignite.internal.processors.query.GridQueryProcessor$7 +org.apache.ignite.internal.processors.query.GridQueryProcessor$8 org.apache.ignite.internal.processors.query.GridQueryProcessor$IndexType org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageResponse org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryRequest +org.apache.ignite.internal.processors.resource.GridResourceIoc$AnnotationSet +org.apache.ignite.internal.processors.resource.GridResourceIoc$ResourceAnnotation org.apache.ignite.internal.processors.rest.GridRestCommand org.apache.ignite.internal.processors.rest.GridRestProcessor$2$1 org.apache.ignite.internal.processors.rest.GridRestProcessor$3 @@ -1332,7 +1359,7 @@ org.apache.ignite.internal.processors.service.ServiceDescriptorImpl org.apache.ignite.internal.processors.task.GridTaskProcessor$1 org.apache.ignite.internal.processors.task.GridTaskThreadContextKey org.apache.ignite.internal.processors.task.GridTaskWorker$3 -org.apache.ignite.internal.processors.task.GridTaskWorker$4 +org.apache.ignite.internal.processors.task.GridTaskWorker$5 org.apache.ignite.internal.processors.task.GridTaskWorker$State org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException @@ -1356,6 +1383,7 @@ org.apache.ignite.internal.util.GridBoundedConcurrentOrderedMap org.apache.ignite.internal.util.GridBoundedConcurrentOrderedSet org.apache.ignite.internal.util.GridBoundedLinkedHashMap org.apache.ignite.internal.util.GridBoundedLinkedHashSet +org.apache.ignite.internal.util.GridBoundedPriorityQueue org.apache.ignite.internal.util.GridByteArrayList org.apache.ignite.internal.util.GridCloseableIteratorAdapter org.apache.ignite.internal.util.GridCloseableIteratorAdapterEx @@ -1407,14 +1435,14 @@ org.apache.ignite.internal.util.IgniteUtils$11 org.apache.ignite.internal.util.IgniteUtils$12 org.apache.ignite.internal.util.IgniteUtils$13 org.apache.ignite.internal.util.IgniteUtils$14 -org.apache.ignite.internal.util.IgniteUtils$16 -org.apache.ignite.internal.util.IgniteUtils$2 -org.apache.ignite.internal.util.IgniteUtils$22 +org.apache.ignite.internal.util.IgniteUtils$15 +org.apache.ignite.internal.util.IgniteUtils$17 org.apache.ignite.internal.util.IgniteUtils$23 org.apache.ignite.internal.util.IgniteUtils$24 org.apache.ignite.internal.util.IgniteUtils$25 org.apache.ignite.internal.util.IgniteUtils$26 org.apache.ignite.internal.util.IgniteUtils$27 +org.apache.ignite.internal.util.IgniteUtils$28 org.apache.ignite.internal.util.IgniteUtils$3 org.apache.ignite.internal.util.IgniteUtils$4 org.apache.ignite.internal.util.IgniteUtils$5 @@ -1647,12 +1675,19 @@ org.apache.ignite.internal.visor.cache.VisorCachePartitionsTask org.apache.ignite.internal.visor.cache.VisorCachePartitionsTask$VisorCachePartitionsJob org.apache.ignite.internal.visor.cache.VisorCacheQueryConfiguration org.apache.ignite.internal.visor.cache.VisorCacheQueryConfigurationV2 +org.apache.ignite.internal.visor.cache.VisorCacheQueryDetailMetrics +org.apache.ignite.internal.visor.cache.VisorCacheQueryDetailMetricsCollectorTask +org.apache.ignite.internal.visor.cache.VisorCacheQueryDetailMetricsCollectorTask$VisorCacheQueryDetailMetricsCollectorJob org.apache.ignite.internal.visor.cache.VisorCacheQueryMetrics org.apache.ignite.internal.visor.cache.VisorCacheRebalanceConfiguration org.apache.ignite.internal.visor.cache.VisorCacheRebalanceTask org.apache.ignite.internal.visor.cache.VisorCacheRebalanceTask$VisorCachesRebalanceJob org.apache.ignite.internal.visor.cache.VisorCacheResetMetricsTask org.apache.ignite.internal.visor.cache.VisorCacheResetMetricsTask$VisorCacheResetMetricsJob +org.apache.ignite.internal.visor.cache.VisorCacheResetQueryDetailMetricsTask +org.apache.ignite.internal.visor.cache.VisorCacheResetQueryDetailMetricsTask$VisorCacheResetQueryDetailMetricsJob +org.apache.ignite.internal.visor.cache.VisorCacheResetQueryMetricsTask +org.apache.ignite.internal.visor.cache.VisorCacheResetQueryMetricsTask$VisorCacheResetQueryMetricsJob org.apache.ignite.internal.visor.cache.VisorCacheStartTask org.apache.ignite.internal.visor.cache.VisorCacheStartTask$VisorCacheStartArg org.apache.ignite.internal.visor.cache.VisorCacheStartTask$VisorCacheStartJob @@ -1682,6 +1717,7 @@ org.apache.ignite.internal.visor.debug.VisorThreadLockInfo org.apache.ignite.internal.visor.debug.VisorThreadMonitorInfo org.apache.ignite.internal.visor.event.VisorGridDeploymentEvent org.apache.ignite.internal.visor.event.VisorGridDiscoveryEvent +org.apache.ignite.internal.visor.event.VisorGridDiscoveryEventV2 org.apache.ignite.internal.visor.event.VisorGridEvent org.apache.ignite.internal.visor.event.VisorGridEventsLost org.apache.ignite.internal.visor.event.VisorGridJobEvent @@ -1769,6 +1805,7 @@ org.apache.ignite.internal.visor.util.VisorClusterGroupEmptyException org.apache.ignite.internal.visor.util.VisorEventMapper org.apache.ignite.internal.visor.util.VisorExceptionWrapper org.apache.ignite.internal.visor.util.VisorTaskUtils$4 +org.apache.ignite.internal.visor.util.VisorTaskUtils$5 org.apache.ignite.internal.websession.WebSessionAttributeProcessor org.apache.ignite.internal.websession.WebSessionEntity org.apache.ignite.lang.IgniteBiClosure @@ -1798,6 +1835,7 @@ org.apache.ignite.plugin.PluginNotFoundException org.apache.ignite.plugin.PluginValidationException org.apache.ignite.plugin.extensions.communication.Message org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType +org.apache.ignite.plugin.security.SecurityBasicPermissionSet org.apache.ignite.plugin.security.SecurityCredentials org.apache.ignite.plugin.security.SecurityException org.apache.ignite.plugin.security.SecurityPermission diff --git a/modules/hadoop/src/main/resources/META-INF/classnames.properties b/modules/hadoop/src/main/resources/META-INF/classnames.properties new file mode 100644 index 0000000000000..0ac17cf6058dd --- /dev/null +++ b/modules/hadoop/src/main/resources/META-INF/classnames.properties @@ -0,0 +1,114 @@ +# +# 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. +# + +org.apache.ignite.hadoop.fs.BasicHadoopFileSystemFactory +org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory +org.apache.ignite.hadoop.fs.HadoopFileSystemFactory +org.apache.ignite.hadoop.fs.KerberosHadoopFileSystemFactory +org.apache.ignite.hadoop.mapreduce.IgniteHadoopWeightedMapReducePlanner$MapperPriority +org.apache.ignite.hadoop.util.BasicUserNameMapper +org.apache.ignite.hadoop.util.ChainedUserNameMapper +org.apache.ignite.hadoop.util.KerberosUserNameMapper +org.apache.ignite.hadoop.util.KerberosUserNameMapper$State +org.apache.ignite.hadoop.util.UserNameMapper +org.apache.ignite.internal.processors.hadoop.HadoopAttributes +org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo +org.apache.ignite.internal.processors.hadoop.HadoopExternalSplit +org.apache.ignite.internal.processors.hadoop.HadoopFileBlock +org.apache.ignite.internal.processors.hadoop.HadoopInputSplit +org.apache.ignite.internal.processors.hadoop.HadoopJobId +org.apache.ignite.internal.processors.hadoop.HadoopJobInfo +org.apache.ignite.internal.processors.hadoop.HadoopJobPhase +org.apache.ignite.internal.processors.hadoop.HadoopJobProperty +org.apache.ignite.internal.processors.hadoop.HadoopJobStatus +org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan +org.apache.ignite.internal.processors.hadoop.HadoopSplitWrapper +org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException +org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo +org.apache.ignite.internal.processors.hadoop.HadoopTaskType +org.apache.ignite.internal.processors.hadoop.counter.HadoopCounterAdapter +org.apache.ignite.internal.processors.hadoop.counter.HadoopCountersImpl +org.apache.ignite.internal.processors.hadoop.counter.HadoopCountersImpl$CounterKey +org.apache.ignite.internal.processors.hadoop.counter.HadoopLongCounter +org.apache.ignite.internal.processors.hadoop.counter.HadoopPerformanceCounter +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsCommunicationException +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$1 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$10 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$11 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$12 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$13 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$14 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$15 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$16 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$2 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$3 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$4 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$5 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$6 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$8 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$9 +org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsOutProc$1 +org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobMetadata +org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$1 +org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$CancelJobProcessor +org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$IncrementCountersProcessor +org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$InitializeReducersProcessor +org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$JobLocalState$1 +org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$JobLocalState$2 +org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$RemoveMappersProcessor +org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$RemoveReducerProcessor +org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$StackedProcessor +org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$UpdatePhaseProcessor +org.apache.ignite.internal.processors.hadoop.message.HadoopMessage +org.apache.ignite.internal.processors.hadoop.planner.HadoopDefaultMapReducePlan +org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolJobCountersTask +org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolJobStatusTask +org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolJobStatusTask$1 +org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolKillJobTask +org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolNextTaskIdTask +org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolSubmitJobTask +org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolTaskAdapter +org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolTaskAdapter$Job +org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolTaskArguments +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffle$1 +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffle$2 +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleAck +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleJob$4 +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleMessage +org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopConcurrentHashMultimap$State +org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskState +org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskStatus +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopExternalTaskExecutor$1 +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopExternalTaskExecutor$2 +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopExternalTaskExecutor$4 +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopExternalTaskExecutor$HadoopProcess +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopExternalTaskExecutor$HadoopProcess$1 +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopJobInfoUpdateRequest +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopPrepareForJobRequest +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopProcessDescriptor +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopProcessStartedAck +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopTaskExecutionRequest +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopTaskFinishedMessage +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner$1 +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner$2 +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner$2$1 +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner$3 +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner$MessageListener$1 +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopExternalProcessStarter$1 +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopExternalCommunication$HandshakeAndBackpressureFilter$1 +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopExternalCommunication$ProcessHandshakeMessage +org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopHandshakeTimeoutException From 33dda46061aae73e5c138851cfdd5f49a1c254cb Mon Sep 17 00:00:00 2001 From: sboikov Date: Fri, 2 Dec 2016 12:13:38 +0300 Subject: [PATCH 381/487] ignite-4285 For serializable txs allow multiple threads to get read lock for the same key --- .../processors/cache/CacheLockCandidates.java | 42 ++ .../cache/CacheLockCandidatesList.java | 71 ++ .../processors/cache/GridCacheEntryEx.java | 3 +- .../processors/cache/GridCacheMapEntry.java | 117 +++- .../processors/cache/GridCacheMvcc.java | 376 +++++++---- .../cache/GridCacheMvccCallback.java | 4 +- .../cache/GridCacheMvccCandidate.java | 80 +-- .../cache/GridCacheMvccManager.java | 19 +- .../GridDistributedCacheEntry.java | 303 +++------ .../distributed/dht/GridDhtCacheEntry.java | 32 +- .../distributed/dht/GridDhtLockFuture.java | 34 +- .../dht/GridDhtTransactionalCacheAdapter.java | 1 - .../dht/GridDhtTxPrepareFuture.java | 5 +- .../colocated/GridDhtColocatedLockFuture.java | 8 +- .../distributed/near/GridNearCacheEntry.java | 44 +- .../distributed/near/GridNearLockFuture.java | 3 +- .../near/GridNearTransactionalCache.java | 5 +- .../cache/local/GridLocalCacheEntry.java | 173 ++--- .../cache/local/GridLocalLockFuture.java | 2 +- .../cache/transactions/IgniteTxManager.java | 6 +- .../CacheSerializableTransactionsTest.java | 604 +++++++++++++++++- .../cache/GridCacheMvccFlagsTest.java | 8 +- .../GridCacheMvccPartitionedSelfTest.java | 334 ++++++++-- .../cache/GridCacheMvccSelfTest.java | 212 +++--- .../cache/GridCacheTestEntryEx.java | 77 +-- .../hashmap/GridHashMapLoadTest.java | 7 +- 26 files changed, 1721 insertions(+), 849 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidates.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidatesList.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidates.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidates.java new file mode 100644 index 0000000000000..9cf16f4eac55c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidates.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; + +/** + * + */ +public interface CacheLockCandidates { + /** + * @param idx Candidate index. + * @return Candidate. + */ + public GridCacheMvccCandidate candidate(int idx); + + /** + * @return Number of candidates. + */ + public int size(); + + /** + * @param ver Candidate version. + * @return {@code True} if contains candidate with given version. + */ + public boolean hasCandidate(GridCacheVersion ver); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidatesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidatesList.java new file mode 100644 index 0000000000000..e026bceb97e58 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidatesList.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.ArrayList; +import java.util.List; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * + */ +class CacheLockCandidatesList implements CacheLockCandidates { + /** */ + @GridToStringInclude + private List list = new ArrayList<>(); + + /** + * @param cand Candidate to add. + */ + void add(GridCacheMvccCandidate cand) { + assert !hasCandidate(cand.version()) : cand; + + list.add(cand); + } + + /** {@inheritDoc} */ + @Override public GridCacheMvccCandidate candidate(int idx) { + assert idx < list.size() : idx; + + return list.get(idx); + } + + /** {@inheritDoc} */ + @Override public int size() { + return list.size(); + } + + /** {@inheritDoc} */ + @Override public boolean hasCandidate(GridCacheVersion ver) { + for (int i = 0; i < list.size(); i++) { + GridCacheMvccCandidate cand = list.get(i); + + if (cand.version().equals(ver)) + return true; + } + + return false; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(CacheLockCandidatesList.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java index 176fe77aa8d3c..d8194fcb30de7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java @@ -565,6 +565,7 @@ public GridTuple3> innerUpdateLoca * @param timeout Timeout for lock acquisition. * @param serOrder Version for serializable transactions ordering. * @param serReadVer Optional read entry version for optimistic serializable transaction. + * @param read Read lock flag. * @return {@code True} if lock was acquired, {@code false} otherwise. * @throws GridCacheEntryRemovedException If this entry is obsolete. * @throws GridDistributedLockCancelledException If lock has been cancelled. @@ -573,7 +574,7 @@ public boolean tmLock(IgniteInternalTx tx, long timeout, @Nullable GridCacheVersion serOrder, @Nullable GridCacheVersion serReadVer, - boolean keepBinary + boolean read ) throws GridCacheEntryRemovedException, GridDistributedLockCancelledException; /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index aec28bb862c74..31baedacc7956 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -79,9 +79,11 @@ import org.jetbrains.annotations.Nullable; import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_EXPIRED; +import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_LOCKED; import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_PUT; import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ; import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_REMOVED; +import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_UNLOCKED; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE; import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE; @@ -827,8 +829,6 @@ private Object innerGet0( if (readThrough && !cctx.readThrough()) readThrough = false; - GridCacheMvccCandidate owner; - CacheObject ret; GridCacheVersion startVer; @@ -841,10 +841,6 @@ private Object innerGet0( synchronized (this) { checkObsolete(); - GridCacheMvcc mvcc = mvccExtras(); - - owner = mvcc == null ? null : mvcc.anyOwner(); - boolean valid = valid(tx != null ? tx.topologyVersion() : cctx.affinity().affinityTopologyVersion()); CacheObject val; @@ -899,11 +895,13 @@ private Object innerGet0( if (evt && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) { transformClo = EntryProcessorResourceInjectorProxy.unwrap(transformClo); + GridCacheMvcc mvcc = mvccExtras(); + cctx.events().addEvent( partition(), key, tx, - owner, + mvcc != null ? mvcc.anyOwner() : null, EVT_CACHE_OBJECT_READ, ret, ret != null, @@ -1010,11 +1008,13 @@ else if (tx.dht()) { if (evt && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) { transformClo = EntryProcessorResourceInjectorProxy.unwrap(transformClo); + GridCacheMvcc mvcc = mvccExtras(); + cctx.events().addEvent( partition(), key, tx, - owner, + mvcc != null ? mvcc.anyOwner() : null, EVT_CACHE_OBJECT_READ, ret, ret != null, @@ -3391,14 +3391,14 @@ private boolean checkExpired() throws IgniteCheckedException { } /** {@inheritDoc} */ - @Override public synchronized boolean hasValue() { + @Override public final synchronized boolean hasValue() { return hasValueUnlocked(); } /** * @return {@code True} if this entry has value. */ - protected boolean hasValueUnlocked() { + protected final boolean hasValueUnlocked() { assert Thread.holdsLock(this); return val != null || hasOffHeapPointer(); @@ -4318,7 +4318,7 @@ private void evictFailed(@Nullable CacheObject prevVal) throws IgniteCheckedExce } /** {@inheritDoc} */ - @Override public GridCacheBatchSwapEntry evictInBatchInternal(GridCacheVersion obsoleteVer) + @Override public final GridCacheBatchSwapEntry evictInBatchInternal(GridCacheVersion obsoleteVer) throws IgniteCheckedException { assert Thread.holdsLock(this); assert cctx.isSwapOrOffheapEnabled(); @@ -4385,7 +4385,7 @@ private void evictFailed(@Nullable CacheObject prevVal) throws IgniteCheckedExce * @param filter Entry filter. * @return {@code True} if entry is visitable. */ - public boolean visitable(CacheEntryPredicate[] filter) { + public final boolean visitable(CacheEntryPredicate[] filter) { boolean rmv = false; try { @@ -4440,7 +4440,7 @@ public boolean visitable(CacheEntryPredicate[] filter) { } /** {@inheritDoc} */ - @Override public boolean deleted() { + @Override public final boolean deleted() { if (!cctx.deferredDelete()) return false; @@ -4450,7 +4450,7 @@ public boolean visitable(CacheEntryPredicate[] filter) { } /** {@inheritDoc} */ - @Override public synchronized boolean obsoleteOrDeleted() { + @Override public final synchronized boolean obsoleteOrDeleted() { return obsoleteVersionExtras() != null || (cctx.deferredDelete() && (deletedUnlocked() || !hasValueUnlocked())); } @@ -4459,7 +4459,7 @@ public boolean visitable(CacheEntryPredicate[] filter) { * @return {@code True} if deleted. */ @SuppressWarnings("SimplifiableIfStatement") - protected boolean deletedUnlocked() { + protected final boolean deletedUnlocked() { assert Thread.holdsLock(this); if (!cctx.deferredDelete()) @@ -4471,7 +4471,7 @@ protected boolean deletedUnlocked() { /** * @param deleted {@code True} if deleted. */ - protected void deletedUnlocked(boolean deleted) { + protected final void deletedUnlocked(boolean deleted) { assert Thread.holdsLock(this); assert cctx.deferredDelete(); @@ -4508,7 +4508,7 @@ protected void decrementMapPublicSize() { /** * @return MVCC. */ - @Nullable protected GridCacheMvcc mvccExtras() { + @Nullable protected final GridCacheMvcc mvccExtras() { return extras != null ? extras.mvcc() : null; } @@ -4516,7 +4516,7 @@ protected void decrementMapPublicSize() { * @return All MVCC local and non near candidates. */ @SuppressWarnings("ForLoopReplaceableByForEach") - @Nullable public synchronized List mvccAllLocal() { + @Nullable public final synchronized List mvccAllLocal() { GridCacheMvcc mvcc = extras != null ? extras.mvcc() : null; if (mvcc == null) @@ -4542,21 +4542,22 @@ protected void decrementMapPublicSize() { /** * @param mvcc MVCC. */ - protected void mvccExtras(@Nullable GridCacheMvcc mvcc) { + protected final void mvccExtras(@Nullable GridCacheMvcc mvcc) { extras = (extras != null) ? extras.mvcc(mvcc) : mvcc != null ? new GridCacheMvccEntryExtras(mvcc) : null; } /** * @return Obsolete version. */ - @Nullable protected GridCacheVersion obsoleteVersionExtras() { + @Nullable protected final GridCacheVersion obsoleteVersionExtras() { return extras != null ? extras.obsoleteVersion() : null; } /** * @param obsoleteVer Obsolete version. + * @param ext Extras. */ - protected void obsoleteVersionExtras(@Nullable GridCacheVersion obsoleteVer, GridCacheObsoleteEntryExtras ext) { + private void obsoleteVersionExtras(@Nullable GridCacheVersion obsoleteVer, GridCacheObsoleteEntryExtras ext) { extras = (extras != null) ? extras.obsoleteVersion(obsoleteVer) : obsoleteVer != null ? @@ -4564,6 +4565,80 @@ protected void obsoleteVersionExtras(@Nullable GridCacheVersion obsoleteVer, Gri null; } + /** + * @param prevOwners Previous owners. + * @param owners Current owners. + * @param val Entry value. + */ + protected final void checkOwnerChanged(@Nullable CacheLockCandidates prevOwners, + @Nullable CacheLockCandidates owners, + CacheObject val) { + assert !Thread.holdsLock(this); + + if (prevOwners != null && owners == null) { + cctx.mvcc().callback().onOwnerChanged(this, null); + + if (cctx.events().isRecordable(EVT_CACHE_OBJECT_UNLOCKED)) { + boolean hasVal = hasValue(); + + GridCacheMvccCandidate cand = prevOwners.candidate(0); + + cctx.events().addEvent(partition(), + key, + cand.nodeId(), + cand, + EVT_CACHE_OBJECT_UNLOCKED, + val, + hasVal, + val, + hasVal, + null, + null, + null, + true); + } + } + + if (owners != null) { + for (int i = 0; i < owners.size(); i++) { + GridCacheMvccCandidate owner = owners.candidate(i); + + boolean locked = prevOwners == null || !prevOwners.hasCandidate(owner.version()); + + if (locked) { + cctx.mvcc().callback().onOwnerChanged(this, owner); + + if (owner.local()) + checkThreadChain(owner); + + if (cctx.events().isRecordable(EVT_CACHE_OBJECT_LOCKED)) { + boolean hasVal = hasValue(); + + // Event notification. + cctx.events().addEvent(partition(), + key, + owner.nodeId(), + owner, + EVT_CACHE_OBJECT_LOCKED, + val, + hasVal, + val, + hasVal, + null, + null, + null, + true); + } + } + } + } + } + + /** + * @param owner Starting candidate in the chain. + */ + protected abstract void checkThreadChain(GridCacheMvccCandidate owner); + /** * Updates metrics. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvcc.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvcc.java index 507a2c901538f..498584cc36303 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvcc.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvcc.java @@ -107,7 +107,7 @@ public GridCacheMvcc(GridCacheContext cctx) { /** * @return Any owner. */ - @Nullable public GridCacheMvccCandidate anyOwner() { + @Nullable GridCacheMvccCandidate anyOwner() { GridCacheMvccCandidate owner = localOwner(); if (owner == null) @@ -116,11 +116,23 @@ public GridCacheMvcc(GridCacheContext cctx) { return owner; } + /** + * @return All owners. + */ + @Nullable public CacheLockCandidates allOwners() { + CacheLockCandidates owners = localOwners(); + + if (owners == null) + owners = remoteOwner(); + + return owners; + } + /** * @return Remote candidate only if it's first in the list and is marked * as 'used'. */ - @Nullable public GridCacheMvccCandidate remoteOwner() { + @Nullable private GridCacheMvccCandidate remoteOwner() { if (rmts != null) { assert !rmts.isEmpty(); @@ -132,11 +144,59 @@ public GridCacheMvcc(GridCacheContext cctx) { return null; } + /** + * @return All local owners. + */ + @Nullable public CacheLockCandidates localOwners() { + if (locs != null) { + assert !locs.isEmpty(); + + CacheLockCandidates owners = null; + + GridCacheMvccCandidate first = locs.getFirst(); + + if (first.read()) { + for (GridCacheMvccCandidate cand : locs) { + if (cand.owner()) { + assert cand.read() : this; + + if (owners != null) { + CacheLockCandidatesList list; + + if (owners.size() == 1) { + GridCacheMvccCandidate owner = owners.candidate(0); + + owners = list = new CacheLockCandidatesList(); + + ((CacheLockCandidatesList)owners).add(owner); + } + else + list = ((CacheLockCandidatesList)owners); + + list.add(cand); + } + else + owners = cand; + } + + if (!cand.read()) + break; + } + } + else if (first.owner()) + owners = first; + + return owners; + } + + return null; + } + /** * @return Local candidate only if it's first in the list and is marked * as 'owner'. */ - @Nullable public GridCacheMvccCandidate localOwner() { + @Nullable GridCacheMvccCandidate localOwner() { if (locs != null) { assert !locs.isEmpty(); @@ -185,6 +245,29 @@ public GridCacheMvcc(GridCacheContext cctx) { return null; } + /** + * @param cand Existing candidate. + * @param newCand New candidate. + * @return {@code False} if new candidate can not be added. + */ + private boolean compareSerializableVersion(GridCacheMvccCandidate cand, GridCacheMvccCandidate newCand) { + assert cand.serializable() && newCand.serializable(); + + GridCacheVersion candOrder = cand.serializableOrder(); + + assert candOrder != null : cand; + + GridCacheVersion newCandOrder = newCand.serializableOrder(); + + assert newCandOrder != null : newCand; + + int cmp = SER_VER_COMPARATOR.compare(candOrder, newCandOrder); + + assert cmp != 0; + + return cmp < 0; + } + /** * @param cand Candidate to add. * @return {@code False} if failed to add candidate and transaction should be cancelled. @@ -200,25 +283,34 @@ private boolean add0(GridCacheMvccCandidate cand) { if (!cand.nearLocal()) { if (!locs.isEmpty()) { if (cand.serializable()) { - GridCacheMvccCandidate last = locs.getLast(); - - if (!last.serializable()) - return false; - - GridCacheVersion lastOrder = last.serializableOrder(); + Iterator it = locs.descendingIterator(); - assert lastOrder != null : last; + if (cand.read()) { + while (it.hasNext()) { + GridCacheMvccCandidate c = it.next(); - GridCacheVersion candOrder = cand.serializableOrder(); + if (!c.serializable()) + return false; - assert candOrder != null : cand; - - int cmp = SER_VER_COMPARATOR.compare(lastOrder, candOrder); + if (!c.read()) { + if (compareSerializableVersion(c, cand)) + break; + else + return false; + } + } + } + else { + while (it.hasNext()) { + GridCacheMvccCandidate c = it.next(); - assert cmp != 0; + if (!c.serializable() || !compareSerializableVersion(c, cand)) + return false; - if (cmp > 0) - return false; + if (!c.read()) + break; + } + } locs.addLast(cand); @@ -284,12 +376,12 @@ private boolean add0(GridCacheMvccCandidate cand) { } // Remote. else { - assert !cand.serializable() : cand; + assert !cand.serializable() && !cand.read() : cand; if (rmts == null) rmts = new LinkedList<>(); - assert !cand.owner() || localOwner() == null : "Cannot have local and remote owners " + + assert !cand.owner() || localOwners() == null : "Cannot have local and remote owners " + "at the same time [cand=" + cand + ", locs=" + locs + ", rmts=" + rmts + ']'; GridCacheMvccCandidate cur = candidate(rmts, cand.version()); @@ -398,9 +490,8 @@ public boolean isEmpty(GridCacheVersion... exclude) { * @param baseVer Base version. * @param committedVers Committed versions relative to base. * @param rolledbackVers Rolled back versions relative to base. - * @return Lock owner. */ - @Nullable public GridCacheMvccCandidate orderCompleted(GridCacheVersion baseVer, + public void orderCompleted(GridCacheVersion baseVer, Collection committedVers, Collection rolledbackVers) { assert baseVer != null; @@ -415,10 +506,13 @@ public boolean isEmpty(GridCacheVersion... exclude) { if (!cur.version().equals(baseVer) && committedVers.contains(cur.version())) { cur.setOwner(); - assert localOwner() == null || localOwner().nearLocal(): "Cannot not have local owner and " + + assert localOwners() == null || localOwner().nearLocal(): "Cannot not have local owner and " + "remote completed transactions at the same time [baseVer=" + baseVer + - ", committedVers=" + committedVers + ", rolledbackVers=" + rolledbackVers + - ", localOwner=" + localOwner() + ", locs=" + locs + ", rmts=" + rmts + ']'; + ", committedVers=" + committedVers + + ", rolledbackVers=" + rolledbackVers + + ", localOwner=" + localOwner() + + ", locs=" + locs + + ", rmts=" + rmts + ']'; if (maxIdx < 0) maxIdx = it.nextIndex(); @@ -462,8 +556,6 @@ else if (maxIdx >= 0 && cur.version().isGreaterEqual(baseVer)) { rmts = null; } } - - return anyOwner(); } /** @@ -471,11 +563,10 @@ else if (maxIdx >= 0 && cur.version().isGreaterEqual(baseVer)) { * * @param baseVer Base version. * @param owned Owned list. - * @return Current owner. */ - @Nullable public GridCacheMvccCandidate markOwned(GridCacheVersion baseVer, GridCacheVersion owned) { + public void markOwned(GridCacheVersion baseVer, GridCacheVersion owned) { if (owned == null) - return anyOwner(); + return; if (rmts != null) { GridCacheMvccCandidate baseCand = candidate(rmts, baseVer); @@ -483,8 +574,6 @@ else if (maxIdx >= 0 && cur.version().isGreaterEqual(baseVer)) { if (baseCand != null) baseCand.ownerVersion(owned); } - - return anyOwner(); } /** @@ -495,6 +584,7 @@ else if (maxIdx >= 0 && cur.version().isGreaterEqual(baseVer)) { * @param reenter Reentry flag ({@code true} if reentry is allowed). * @param tx Transaction flag. * @param implicitSingle Implicit transaction flag. + * @param read Read lock flag. * @return New lock candidate if lock was added, or current owner if lock was reentered, * or null if lock was owned by another thread and timeout is negative. */ @@ -505,7 +595,8 @@ else if (maxIdx >= 0 && cur.version().isGreaterEqual(baseVer)) { long timeout, boolean reenter, boolean tx, - boolean implicitSingle) { + boolean implicitSingle, + boolean read) { return addLocal( parent, /*nearNodeId*/null, @@ -517,7 +608,8 @@ else if (maxIdx >= 0 && cur.version().isGreaterEqual(baseVer)) { reenter, tx, implicitSingle, - /*dht-local*/false + /*dht-local*/false, + /*read*/read ); } @@ -533,6 +625,7 @@ else if (maxIdx >= 0 && cur.version().isGreaterEqual(baseVer)) { * @param tx Transaction flag. * @param implicitSingle Implicit flag. * @param dhtLoc DHT local flag. + * @param read Read lock flag. * @return New lock candidate if lock was added, or current owner if lock was reentered, * or null if lock was owned by another thread and timeout is negative. */ @@ -547,7 +640,8 @@ else if (maxIdx >= 0 && cur.version().isGreaterEqual(baseVer)) { boolean reenter, boolean tx, boolean implicitSingle, - boolean dhtLoc) { + boolean dhtLoc, + boolean read) { if (log.isDebugEnabled()) log.debug("Adding local candidate [mvcc=" + this + ", parent=" + parent + ", threadId=" + threadId + ", ver=" + ver + ", timeout=" + timeout + ", reenter=" + reenter + ", tx=" + tx + "]"); @@ -582,14 +676,14 @@ else if (maxIdx >= 0 && cur.version().isGreaterEqual(baseVer)) { nearVer, threadId, ver, - timeout, /*local*/true, /*reenter*/false, tx, implicitSingle, /*near-local*/false, dhtLoc, - serOrder + serOrder, + read ); if (serOrder == null) { @@ -613,7 +707,6 @@ else if (maxIdx >= 0 && cur.version().isGreaterEqual(baseVer)) { * @param otherNodeId Other node ID. * @param threadId Thread ID. * @param ver Lock version. - * @param timeout Lock acquire timeout. * @param tx Transaction flag. * @param implicitSingle Implicit flag. * @param nearLoc Near local flag. @@ -625,7 +718,6 @@ public GridCacheMvccCandidate addRemote( @Nullable UUID otherNodeId, long threadId, GridCacheVersion ver, - long timeout, boolean tx, boolean implicitSingle, boolean nearLoc) { @@ -636,14 +728,14 @@ public GridCacheMvccCandidate addRemote( null, threadId, ver, - timeout, /*local*/false, /*reentry*/false, tx, implicitSingle, nearLoc, false, - null + null, + /*read*/false ); addRemote(cand); @@ -659,9 +751,9 @@ public GridCacheMvccCandidate addRemote( * @param otherNodeId Other node ID. * @param threadId Thread ID. * @param ver Lock version. - * @param timeout Lock acquire timeout. * @param tx Transaction flag. * @param implicitSingle Implicit flag. + * @param read Read lock flag. * @return Add remote candidate. */ public GridCacheMvccCandidate addNearLocal(GridCacheEntryEx parent, @@ -669,23 +761,23 @@ public GridCacheMvccCandidate addNearLocal(GridCacheEntryEx parent, @Nullable UUID otherNodeId, long threadId, GridCacheVersion ver, - long timeout, boolean tx, - boolean implicitSingle) { + boolean implicitSingle, + boolean read) { GridCacheMvccCandidate cand = new GridCacheMvccCandidate(parent, nodeId, otherNodeId, null, threadId, ver, - timeout, /*local*/true, /*reentry*/false, tx, implicitSingle, /*near loc*/true, /*dht loc*/false, - null); + null, + /*read*/read); add0(cand); @@ -695,7 +787,7 @@ public GridCacheMvccCandidate addNearLocal(GridCacheEntryEx parent, /** * @param cand Remote candidate. */ - public void addRemote(GridCacheMvccCandidate cand) { + private void addRemote(GridCacheMvccCandidate cand) { assert !cand.local(); if (log.isDebugEnabled()) @@ -710,11 +802,11 @@ public void addRemote(GridCacheMvccCandidate cand) { * @param ver Lock version to acquire or set to ready. * @return Current owner. */ - @Nullable public GridCacheMvccCandidate readyLocal(GridCacheVersion ver) { + @Nullable public CacheLockCandidates readyLocal(GridCacheVersion ver) { GridCacheMvccCandidate cand = candidate(ver); if (cand == null) - return anyOwner(); + return allOwners(); assert cand.local(); @@ -725,14 +817,14 @@ public void addRemote(GridCacheMvccCandidate cand) { * @param cand Local candidate added in any of the {@code addLocal(..)} methods. * @return Current lock owner. */ - @Nullable public GridCacheMvccCandidate readyLocal(GridCacheMvccCandidate cand) { + @Nullable public CacheLockCandidates readyLocal(GridCacheMvccCandidate cand) { assert cand.local(); cand.setReady(); reassign(); - return anyOwner(); + return allOwners(); } /** @@ -751,9 +843,12 @@ public void addRemote(GridCacheMvccCandidate cand) { * @param pending Pending dht versions that are not owned and which version is less then mapped. * @return Lock owner after reassignment. */ - @Nullable public GridCacheMvccCandidate readyNearLocal(GridCacheVersion ver, GridCacheVersion mappedVer, - Collection committedVers, Collection rolledBackVers, - Collection pending) { + @Nullable public CacheLockCandidates readyNearLocal(GridCacheVersion ver, + GridCacheVersion mappedVer, + Collection committedVers, + Collection rolledBackVers, + Collection pending) + { GridCacheMvccCandidate cand = candidate(locs, ver); if (cand != null) { @@ -785,7 +880,7 @@ public void addRemote(GridCacheMvccCandidate cand) { if (c.owner()) continue; - assert !c.ready() : + assert !c.ready() || (c.read() && cand.read()): "Cannot have more then one ready near-local candidate [c=" + c + ", cand=" + cand + ", mvcc=" + this + ']'; @@ -819,7 +914,7 @@ public void addRemote(GridCacheMvccCandidate cand) { reassign(); } - return anyOwner(); + return allOwners(); } /** @@ -831,7 +926,7 @@ public void addRemote(GridCacheMvccCandidate cand) { * @param rolledback Rolledback versions. * @return Lock owner. */ - @Nullable public GridCacheMvccCandidate doneRemote( + @Nullable public CacheLockCandidates doneRemote( GridCacheVersion ver, Collection pending, Collection committed, @@ -879,7 +974,7 @@ else if (!committed.contains(c.version()) && !rolledback.contains(c.version()) & } } - return anyOwner(); + return allOwners(); } /** @@ -942,19 +1037,39 @@ private void reassign() { if (locs != null) { boolean first = true; - for (ListIterator it = locs.listIterator(); it.hasNext(); ) { + ListIterator it = locs.listIterator(); + + while (it.hasNext()) { GridCacheMvccCandidate cand = it.next(); - if (first && cand.serializable()) { - if (cand.owner() || !cand.ready()) + if (first) { + if (cand.read()) { + if (cand.ready() && !cand.owner()) + cand.setOwner(); + + while (it.hasNext()) { + cand = it.next(); + + if (!cand.read()) + break; + + if (cand.ready() && !cand.owner()) + cand.setOwner(); + } + return; + } + else if (cand.serializable()) { + if (cand.owner() || !cand.ready()) + return; - cand.setOwner(); + cand.setOwner(); - return; - } + return; + } - first = false; + first = false; + } if (cand.owner()) return; @@ -1036,6 +1151,8 @@ private void reassign() { } if (assigned) { + assert !cand.serializable() : cand; + it.remove(); // Owner must be first in the list. @@ -1066,15 +1183,16 @@ private void reassign() { * * @return Owner. */ - @Nullable public GridCacheMvccCandidate recheck() { + @Nullable public CacheLockCandidates recheck() { reassign(); - return anyOwner(); + return allOwners(); } /** * Local local release. - * @return Removed lock candidate or null if candidate was not removed. + * + * @return Removed candidate. */ @Nullable public GridCacheMvccCandidate releaseLocal() { return releaseLocal(Thread.currentThread().getId()); @@ -1084,32 +1202,45 @@ private void reassign() { * Local release. * * @param threadId ID of the thread. - * @return Current owner. + * @return Removed candidate. */ @Nullable public GridCacheMvccCandidate releaseLocal(long threadId) { - GridCacheMvccCandidate owner = localOwner(); + CacheLockCandidates owners = localOwners(); - if (owner == null || owner.threadId() != threadId) - // Release had no effect. - return owner; + // Release had no effect. + if (owners == null) + return null; - owner.setUsed(); + GridCacheMvccCandidate owner = null; - remove0(owner.version(), true); + for (int i = 0; i < owners.size(); i++) { + GridCacheMvccCandidate owner0 = owners.candidate(i); - return anyOwner(); + if (owner0.threadId() == threadId) { + owner = owner0; + + break; + } + } + + if (owner != null) { + owner.setUsed(); + + remove0(owner.version(), true); + + return owner; + } + else + return null; } /** * Removes lock even if it is not owner. * * @param ver Lock version. - * @return Current owner. */ - @Nullable public GridCacheMvccCandidate remove(GridCacheVersion ver) { + public void remove(GridCacheVersion ver) { remove0(ver, false); - - return anyOwner(); } /** @@ -1118,7 +1249,7 @@ private void reassign() { * @param nodeId Node ID. * @return Current owner. */ - @Nullable public GridCacheMvccCandidate removeExplicitNodeCandidates(UUID nodeId) { + @Nullable public CacheLockCandidates removeExplicitNodeCandidates(UUID nodeId) { if (rmts != null) { for (Iterator it = rmts.iterator(); it.hasNext(); ) { GridCacheMvccCandidate cand = it.next(); @@ -1153,7 +1284,7 @@ private void reassign() { reassign(); - return anyOwner(); + return allOwners(); } /** @@ -1177,7 +1308,7 @@ private void reassign() { * @param threadId Thread ID. * @return Candidate or null if there is no candidate for given ID. */ - @Nullable public GridCacheMvccCandidate localCandidate(long threadId) { + @Nullable GridCacheMvccCandidate localCandidate(long threadId) { // Don't return reentries. return localCandidate(threadId, false); } @@ -1187,7 +1318,7 @@ private void reassign() { * @param threadId Thread ID. * @return Remote candidate. */ - @Nullable public GridCacheMvccCandidate remoteCandidate(UUID nodeId, long threadId) { + @Nullable GridCacheMvccCandidate remoteCandidate(UUID nodeId, long threadId) { if (rmts != null) for (GridCacheMvccCandidate c : rmts) if (c.nodeId().equals(nodeId) && c.threadId() == threadId) @@ -1217,7 +1348,7 @@ private void reassign() { * @param ver Version. * @return {@code True} if candidate with given version exists. */ - public boolean hasCandidate(GridCacheVersion ver) { + boolean hasCandidate(GridCacheVersion ver) { return candidate(ver) != null; } @@ -1283,41 +1414,25 @@ private List candidates(List col return cands; } - /** - * @return {@code True} if lock is owner by current thread. - */ - public boolean isLocallyOwnedByCurrentThread() { - return isLocallyOwnedByThread(Thread.currentThread().getId(), true); - } - /** * @param threadId Thread ID to check. * @param exclude Versions to ignore. * @return {@code True} if lock is owned by the thread with given ID. */ - public boolean isLocallyOwnedByThread(long threadId, boolean allowDhtLoc, GridCacheVersion... exclude) { - GridCacheMvccCandidate owner = localOwner(); - - return owner != null && owner.threadId() == threadId && owner.nodeId().equals(cctx.nodeId()) && - (allowDhtLoc || !owner.dhtLocal()) && !U.containsObjectArray(exclude, owner.version()); - } + boolean isLocallyOwnedByThread(long threadId, boolean allowDhtLoc, GridCacheVersion... exclude) { + CacheLockCandidates owners = localOwners(); - /** - * @param threadId Thread ID. - * @param nodeId Node ID. - * @return {@code True} if lock is held by given thread and node IDs. - */ - public boolean isLockedByThread(long threadId, UUID nodeId) { - GridCacheMvccCandidate owner = anyOwner(); + if (owners != null) { + for (int i = 0; i < owners.size(); i++) { + GridCacheMvccCandidate owner = owners.candidate(i); - return owner != null && owner.threadId() == threadId && owner.nodeId().equals(nodeId); - } + if (owner.threadId() == threadId && owner.nodeId().equals(cctx.nodeId()) && + (allowDhtLoc || !owner.dhtLocal()) && !U.containsObjectArray(exclude, owner.version())) + return true; + } + } - /** - * @return {@code True} if lock is owned by any thread or node. - */ - public boolean isOwnedByAny() { - return anyOwner() != null; + return false; } /** @@ -1325,10 +1440,10 @@ public boolean isOwnedByAny() { * @param lockVer ID of lock candidate. * @return {@code True} if candidate is owner. */ - public boolean isLocallyOwned(GridCacheVersion lockVer) { - GridCacheMvccCandidate owner = localOwner(); + boolean isLocallyOwned(GridCacheVersion lockVer) { + CacheLockCandidates owners = localOwners(); - return owner != null && owner.version().equals(lockVer); + return owners != null && owners.hasCandidate(lockVer); } /** @@ -1336,30 +1451,25 @@ public boolean isLocallyOwned(GridCacheVersion lockVer) { * @param threadId Thread ID. * @return {@code True} if locked by lock ID or thread ID. */ - public boolean isLocallyOwnedByIdOrThread(GridCacheVersion lockVer, long threadId) { - GridCacheMvccCandidate owner = localOwner(); + boolean isLocallyOwnedByIdOrThread(GridCacheVersion lockVer, long threadId) { + CacheLockCandidates owners = localOwners(); - return owner != null && (owner.version().equals(lockVer) || owner.threadId() == threadId); - } + if (owners != null) { + for (int i = 0; i < owners.size(); i++) { + GridCacheMvccCandidate owner = owners.candidate(i); - /** - * @return First remote entry or null. - */ - @Nullable public GridCacheMvccCandidate firstRemote() { - return rmts == null ? null : rmts.getFirst(); - } + if ((owner.version().equals(lockVer) || owner.threadId() == threadId)) + return true; + } + } - /** - * @return First local entry or null. - */ - @Nullable public GridCacheMvccCandidate firstLocal() { - return locs == null ? null : locs.getFirst(); + return false; } /** * @return Local MVCC candidates. */ - @Nullable public List allLocal() { + @Nullable List allLocal() { return locs; } @@ -1367,10 +1477,10 @@ public boolean isLocallyOwnedByIdOrThread(GridCacheVersion lockVer, long threadI * @param ver Version to check for ownership. * @return {@code True} if lock is owned by the specified version. */ - public boolean isOwnedBy(GridCacheVersion ver) { - GridCacheMvccCandidate cand = anyOwner(); + boolean isOwnedBy(GridCacheVersion ver) { + CacheLockCandidates owners = allOwners(); - return cand != null && cand.version().equals(ver); + return owners != null && owners.hasCandidate(ver); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCallback.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCallback.java index fc1faf7566732..2ba41f73244c7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCallback.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCallback.java @@ -37,11 +37,9 @@ public interface GridCacheMvccCallback { * can be made from this call. * * @param entry Entry. - * @param prev Previous candidate. * @param owner Current owner. */ - public void onOwnerChanged(GridCacheEntryEx entry, GridCacheMvccCandidate prev, - GridCacheMvccCandidate owner); + public void onOwnerChanged(GridCacheEntryEx entry, GridCacheMvccCandidate owner); /** * Called when entry has no more candidates. This call happens diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCandidate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCandidate.java index f1c1b83a51d66..e9dd455901e44 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCandidate.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCandidate.java @@ -43,6 +43,7 @@ import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.LOCAL; import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.NEAR_LOCAL; import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.OWNER; +import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.READ; import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.READY; import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.REENTRY; import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.REMOVED; @@ -54,7 +55,7 @@ * Lock candidate. */ public class GridCacheMvccCandidate implements Externalizable, - Comparable { + Comparable, CacheLockCandidates { /** */ private static final long serialVersionUID = 0L; @@ -69,14 +70,6 @@ public class GridCacheMvccCandidate implements Externalizable, @GridToStringInclude private GridCacheVersion ver; - /** Maximum wait time. */ - @GridToStringInclude - private long timeout; - - /** Candidate timestamp. */ - @GridToStringInclude - private long ts; - /** Thread ID. */ @GridToStringInclude private long threadId; @@ -143,7 +136,6 @@ public GridCacheMvccCandidate() { * @param otherVer Other version. * @param threadId Requesting thread ID. * @param ver Cache version. - * @param timeout Maximum wait time. * @param loc {@code True} if the lock is local. * @param reentry {@code True} if candidate is for reentry. * @param tx Transaction flag. @@ -151,6 +143,7 @@ public GridCacheMvccCandidate() { * @param nearLoc Near-local flag. * @param dhtLoc DHT local flag. * @param serOrder Version for serializable transactions ordering. + * @param read Read lock flag. */ public GridCacheMvccCandidate( GridCacheEntryEx parent, @@ -159,14 +152,14 @@ public GridCacheMvccCandidate( @Nullable GridCacheVersion otherVer, long threadId, GridCacheVersion ver, - long timeout, boolean loc, boolean reentry, boolean tx, boolean singleImplicit, boolean nearLoc, boolean dhtLoc, - @Nullable GridCacheVersion serOrder + @Nullable GridCacheVersion serOrder, + boolean read ) { assert nodeId != null; assert ver != null; @@ -178,7 +171,6 @@ public GridCacheMvccCandidate( this.otherVer = otherVer; this.threadId = threadId; this.ver = ver; - this.timeout = timeout; this.serOrder = serOrder; mask(LOCAL, loc); @@ -187,8 +179,7 @@ public GridCacheMvccCandidate( mask(SINGLE_IMPLICIT, singleImplicit); mask(NEAR_LOCAL, nearLoc); mask(DHT_LOCAL, dhtLoc); - - ts = U.currentTimeMillis(); + mask(READ, read); id = IDGEN.incrementAndGet(); } @@ -245,14 +236,14 @@ public GridCacheMvccCandidate reenter() { otherVer, threadId, ver, - timeout, local(), /*reentry*/true, tx(), singleImplicit(), nearLocal(), dhtLocal(), - serializableOrder()); + serializableOrder(), + read()); reentry.topVer = topVer; @@ -410,20 +401,6 @@ public GridCacheVersion version() { return ver; } - /** - * @return Maximum wait time. - */ - public long timeout() { - return timeout; - } - - /** - * @return Timestamp at the time of entering pending set. - */ - public long timestamp() { - return ts; - } - /** * @return {@code True} if lock is local. */ @@ -473,6 +450,13 @@ public boolean serializable() { return serOrder; } + /** + * @return Read lock flag. + */ + public boolean read() { + return READ.get(flags()); + } + /** * @return {@code True} if this candidate is a reentry. */ @@ -586,16 +570,21 @@ public IgniteTxKey key() { return parent0.txKey(); } - /** - * Checks if this candidate matches version or thread-nodeId combination. - * - * @param nodeId Node ID to check. - * @param ver Version to check. - * @param threadId Thread ID to check. - * @return {@code True} if matched. - */ - public boolean matches(GridCacheVersion ver, UUID nodeId, long threadId) { - return ver.equals(this.ver) || (nodeId.equals(this.nodeId) && threadId == this.threadId); + /** {@inheritDoc} */ + @Override public GridCacheMvccCandidate candidate(int idx) { + assert idx == 0 : idx; + + return this; + } + + /** {@inheritDoc} */ + @Override public int size() { + return 1; + } + + /** {@inheritDoc} */ + @Override public boolean hasCandidate(GridCacheVersion ver) { + return this.ver.equals(ver); } /** {@inheritDoc} */ @@ -610,7 +599,6 @@ public boolean matches(GridCacheVersion ver, UUID nodeId, long threadId) { ver.writeExternal(out); } - out.writeLong(timeout); out.writeLong(threadId); out.writeLong(id); out.writeShort(flags()); @@ -626,7 +614,6 @@ public boolean matches(GridCacheVersion ver, UUID nodeId, long threadId) { ver.readExternal(in); } - timeout = in.readLong(); threadId = in.readLong(); id = in.readLong(); @@ -635,8 +622,6 @@ public boolean matches(GridCacheVersion ver, UUID nodeId, long threadId) { mask(OWNER, OWNER.get(flags)); mask(USED, USED.get(flags)); mask(TX, TX.get(flags)); - - ts = U.currentTimeMillis(); } /** {@inheritDoc} */ @@ -719,7 +704,10 @@ enum Mask { NEAR_LOCAL(0x200), /** */ - REMOVED(0x400); + REMOVED(0x400), + + /** */ + READ(0x800); /** All mask values. */ private static final Mask[] MASKS = values(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java index c57e17cf8846f..0d0e9ee319d4d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java @@ -140,15 +140,14 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter { private final GridCacheMvccCallback cb = new GridCacheMvccCallback() { /** {@inheritDoc} */ @SuppressWarnings({"unchecked"}) - @Override public void onOwnerChanged(final GridCacheEntryEx entry, final GridCacheMvccCandidate prev, - final GridCacheMvccCandidate owner) { + @Override public void onOwnerChanged(final GridCacheEntryEx entry, final GridCacheMvccCandidate owner) { int nested = nestedLsnrCalls.get(); if (nested < MAX_NESTED_LSNR_CALLS) { nestedLsnrCalls.set(nested + 1); try { - notifyOwnerChanged(entry, prev, owner); + notifyOwnerChanged(entry, owner); } finally { nestedLsnrCalls.set(nested); @@ -157,7 +156,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter { else { cctx.kernalContext().closure().runLocalSafe(new GridPlainRunnable() { @Override public void run() { - notifyOwnerChanged(entry, prev, owner); + notifyOwnerChanged(entry, owner); } }, true); } @@ -182,19 +181,13 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter { /** * @param entry Entry to notify callback for. - * @param prev Previous lock owner. * @param owner Current lock owner. */ - private void notifyOwnerChanged(final GridCacheEntryEx entry, final GridCacheMvccCandidate prev, - final GridCacheMvccCandidate owner) { + private void notifyOwnerChanged(final GridCacheEntryEx entry, final GridCacheMvccCandidate owner) { assert entry != null; - assert owner != prev : "New and previous owner are identical instances: " + owner; - assert owner == null || prev == null || !owner.version().equals(prev.version()) : - "New and previous owners have identical versions [owner=" + owner + ", prev=" + prev + ']'; if (log.isDebugEnabled()) - log.debug("Received owner changed callback [" + entry.key() + ", owner=" + owner + ", prev=" + - prev + ']'); + log.debug("Received owner changed callback [" + entry.key() + ", owner=" + owner + ']'); if (owner != null && (owner.local() || owner.nearLocal())) { Collection> futCol = mvccFuts.get(owner.version()); @@ -226,7 +219,7 @@ private void notifyOwnerChanged(final GridCacheEntryEx entry, final GridCacheMvc if (log.isDebugEnabled()) log.debug("Lock future not found for owner change callback (will try transaction futures) [owner=" + - owner + ", prev=" + prev + ", entry=" + entry + ']'); + owner + ", entry=" + entry + ']'); // If no future was found, delegate to transaction manager. if (cctx.tm().onOwnerChanged(entry, owner)) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java index 2d1b02ed4f94b..3d55f31c7aa34 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.UUID; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheLockCandidates; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; @@ -66,7 +67,7 @@ public GridDistributedCacheEntry( /** * */ - protected void refreshRemotes() { + private void refreshRemotes() { GridCacheMvcc mvcc = mvccExtras(); rmts = mvcc == null ? Collections.emptyList() : mvcc.remoteCandidates(); @@ -82,6 +83,7 @@ protected void refreshRemotes() { * @param reenter Reentry flag. * @param tx Transaction flag. * @param implicitSingle Implicit flag. + * @param read Read lock flag. * @return New candidate. * @throws GridCacheEntryRemovedException If entry has been removed. */ @@ -92,10 +94,11 @@ protected void refreshRemotes() { long timeout, boolean reenter, boolean tx, - boolean implicitSingle) throws GridCacheEntryRemovedException { + boolean implicitSingle, + boolean read) throws GridCacheEntryRemovedException { GridCacheMvccCandidate cand; - GridCacheMvccCandidate prev; - GridCacheMvccCandidate owner; + CacheLockCandidates prev; + CacheLockCandidates owner; CacheObject val; @@ -110,16 +113,23 @@ protected void refreshRemotes() { mvccExtras(mvcc); } - prev = mvcc.anyOwner(); + prev = mvcc.allOwners(); boolean emptyBefore = mvcc.isEmpty(); - cand = mvcc.addLocal(this, threadId, ver, timeout, reenter, tx, implicitSingle); + cand = mvcc.addLocal(this, + threadId, + ver, + timeout, + reenter, + tx, + implicitSingle, + read); if (cand != null) cand.topologyVersion(topVer); - owner = mvcc.anyOwner(); + owner = mvcc.allOwners(); boolean emptyAfter = mvcc.isEmpty(); @@ -168,7 +178,6 @@ protected void refreshRemotes() { * @param otherNodeId Other node ID. * @param threadId Thread ID. * @param ver Lock version. - * @param timeout Lock acquire timeout. * @param tx Transaction flag. * @param implicitSingle Implicit flag. * @param owned Owned candidate version. @@ -180,13 +189,12 @@ public void addRemote( @Nullable UUID otherNodeId, long threadId, GridCacheVersion ver, - long timeout, boolean tx, boolean implicitSingle, @Nullable GridCacheVersion owned ) throws GridDistributedLockCancelledException, GridCacheEntryRemovedException { - GridCacheMvccCandidate prev; - GridCacheMvccCandidate owner; + CacheLockCandidates prev; + CacheLockCandidates owner; CacheObject val; @@ -204,7 +212,7 @@ public void addRemote( mvccExtras(mvcc); } - prev = mvcc.anyOwner(); + prev = mvcc.allOwners(); boolean emptyBefore = mvcc.isEmpty(); @@ -214,7 +222,6 @@ public void addRemote( otherNodeId, threadId, ver, - timeout, tx, implicitSingle, /*near-local*/false @@ -223,62 +230,7 @@ public void addRemote( if (owned != null) mvcc.markOwned(ver, owned); - owner = mvcc.anyOwner(); - - boolean emptyAfter = mvcc.isEmpty(); - - checkCallbacks(emptyBefore, emptyAfter); - - val = this.val; - - refreshRemotes(); - - if (emptyAfter) - mvccExtras(null); - } - - // This call must be outside of synchronization. - checkOwnerChanged(prev, owner, val); - } - - /** - * Adds new lock candidate. - * - * @param cand Remote lock candidate. - * @throws GridDistributedLockCancelledException If lock has been canceled. - * @throws GridCacheEntryRemovedException If this entry is obsolete. - */ - public void addRemote(GridCacheMvccCandidate cand) throws GridDistributedLockCancelledException, - GridCacheEntryRemovedException { - - CacheObject val; - - GridCacheMvccCandidate prev; - GridCacheMvccCandidate owner; - - synchronized (this) { - cand.parent(this); - - // Check removed locks prior to obsolete flag. - checkRemoved(cand.version()); - - checkObsolete(); - - GridCacheMvcc mvcc = mvccExtras(); - - if (mvcc == null) { - mvcc = new GridCacheMvcc(cctx); - - mvccExtras(mvcc); - } - - boolean emptyBefore = mvcc.isEmpty(); - - prev = mvcc.anyOwner(); - - mvcc.addRemote(cand); - - owner = mvcc.anyOwner(); + owner = mvcc.allOwners(); boolean emptyAfter = mvcc.isEmpty(); @@ -303,8 +255,8 @@ public void addRemote(GridCacheMvccCandidate cand) throws GridDistributedLockCan * @throws GridCacheEntryRemovedException If entry was removed. */ public void removeExplicitNodeLocks(UUID nodeId) throws GridCacheEntryRemovedException { - GridCacheMvccCandidate prev = null; - GridCacheMvccCandidate owner = null; + CacheLockCandidates prev = null; + CacheLockCandidates owner = null; CacheObject val = null; @@ -314,7 +266,7 @@ public void removeExplicitNodeLocks(UUID nodeId) throws GridCacheEntryRemovedExc GridCacheMvcc mvcc = mvccExtras(); if (mvcc != null) { - prev = mvcc.anyOwner(); + prev = mvcc.allOwners(); boolean emptyBefore = mvcc.isEmpty(); @@ -346,8 +298,9 @@ public void removeExplicitNodeLocks(UUID nodeId) throws GridCacheEntryRemovedExc * @return Removed candidate, or null if thread still holds the lock. */ @Nullable public GridCacheMvccCandidate removeLock() { - GridCacheMvccCandidate prev = null; - GridCacheMvccCandidate owner = null; + GridCacheMvccCandidate rmvd = null; + CacheLockCandidates prev = null; + CacheLockCandidates owner = null; CacheObject val; @@ -355,11 +308,11 @@ public void removeExplicitNodeLocks(UUID nodeId) throws GridCacheEntryRemovedExc GridCacheMvcc mvcc = mvccExtras(); if (mvcc != null) { - prev = mvcc.anyOwner(); + prev = mvcc.allOwners(); boolean emptyBefore = mvcc.isEmpty(); - owner = mvcc.releaseLocal(); + rmvd = mvcc.releaseLocal(); boolean emptyAfter = mvcc.isEmpty(); @@ -367,28 +320,38 @@ public void removeExplicitNodeLocks(UUID nodeId) throws GridCacheEntryRemovedExc if (emptyAfter) mvccExtras(null); + else + owner = mvcc.allOwners(); } val = this.val; } - if (log.isDebugEnabled()) - log.debug("Released local candidate from entry [owner=" + owner + ", prev=" + prev + + if (log.isDebugEnabled()) { + log.debug("Released local candidate from entry [owner=" + owner + + ", prev=" + prev + + ", rmvd=" + rmvd + ", entry=" + this + ']'); + } + + if (prev != null) { + for (int i = 0; i < prev.size(); i++) { + GridCacheMvccCandidate cand = prev.candidate(i); - if (prev != null && owner != prev) - checkThreadChain(prev); + checkThreadChain(cand); + } + } // This call must be outside of synchronization. checkOwnerChanged(prev, owner, val); - return owner != prev ? prev : null; + return rmvd; } /** {@inheritDoc} */ @Override public boolean removeLock(GridCacheVersion ver) throws GridCacheEntryRemovedException { - GridCacheMvccCandidate prev = null; - GridCacheMvccCandidate owner = null; + CacheLockCandidates prev = null; + CacheLockCandidates owner = null; GridCacheMvccCandidate doomed; @@ -408,13 +371,11 @@ public void removeExplicitNodeLocks(UUID nodeId) throws GridCacheEntryRemovedExc checkObsolete(); if (doomed != null) { - assert mvcc != null; - - prev = mvcc.anyOwner(); + prev = mvcc.allOwners(); boolean emptyBefore = mvcc.isEmpty(); - owner = mvcc.remove(doomed.version()); + mvcc.remove(doomed.version()); boolean emptyAfter = mvcc.isEmpty(); @@ -425,6 +386,8 @@ public void removeExplicitNodeLocks(UUID nodeId) throws GridCacheEntryRemovedExc if (emptyAfter) mvccExtras(null); + else + owner = mvcc.allOwners(); } val = this.val; @@ -477,10 +440,10 @@ public boolean addRemoved(GridCacheVersion ver) { * @return Owner. * @throws GridCacheEntryRemovedException If entry is removed. */ - @Nullable public GridCacheMvccCandidate readyLock(GridCacheVersion ver) + @Nullable public CacheLockCandidates readyLock(GridCacheVersion ver) throws GridCacheEntryRemovedException { - GridCacheMvccCandidate prev = null; - GridCacheMvccCandidate owner = null; + CacheLockCandidates prev = null; + CacheLockCandidates owner = null; CacheObject val; @@ -490,13 +453,13 @@ public boolean addRemoved(GridCacheVersion ver) { GridCacheMvcc mvcc = mvccExtras(); if (mvcc != null) { - prev = mvcc.anyOwner(); + prev = mvcc.allOwners(); boolean emptyBefore = mvcc.isEmpty(); owner = mvcc.readyLocal(ver); - assert owner == null || owner.owner() : "Owner flag not set for owner: " + owner; + assert owner == null || owner.candidate(0).owner() : "Owner flag not set for owner: " + owner; boolean emptyAfter = mvcc.isEmpty(); @@ -523,16 +486,16 @@ public boolean addRemoved(GridCacheVersion ver) { * @param committed Committed versions. * @param rolledBack Rolled back versions. * @param pending Pending locks on dht node with version less then mapped. - * @return Current lock owner. * * @throws GridCacheEntryRemovedException If entry is removed. */ - @Nullable public GridCacheMvccCandidate readyNearLock(GridCacheVersion ver, GridCacheVersion mapped, + public void readyNearLock(GridCacheVersion ver, GridCacheVersion mapped, Collection committed, Collection rolledBack, - Collection pending) throws GridCacheEntryRemovedException { - GridCacheMvccCandidate prev = null; - GridCacheMvccCandidate owner = null; + Collection pending) throws GridCacheEntryRemovedException + { + CacheLockCandidates prev = null; + CacheLockCandidates owner = null; CacheObject val; @@ -542,13 +505,13 @@ public boolean addRemoved(GridCacheVersion ver) { GridCacheMvcc mvcc = mvccExtras(); if (mvcc != null) { - prev = mvcc.anyOwner(); + prev = mvcc.allOwners(); boolean emptyBefore = mvcc.isEmpty(); owner = mvcc.readyNearLocal(ver, mapped, committed, rolledBack, pending); - assert owner == null || owner.owner() : "Owner flag is not set for owner: " + owner; + assert owner == null || owner.candidate(0).owner() : "Owner flag is not set for owner: " + owner; boolean emptyAfter = mvcc.isEmpty(); @@ -563,75 +526,6 @@ public boolean addRemoved(GridCacheVersion ver) { // This call must be made outside of synchronization. checkOwnerChanged(prev, owner, val); - - return owner; - } - - /** - * Reorders completed versions. - * - * @param baseVer Base version for reordering. - * @param committedVers Completed versions. - * @param rolledbackVers Rolled back versions. - * @throws GridCacheEntryRemovedException If entry has been removed. - */ - public void orderCompleted(GridCacheVersion baseVer, Collection committedVers, - Collection rolledbackVers) - throws GridCacheEntryRemovedException { - if (!F.isEmpty(committedVers) || !F.isEmpty(rolledbackVers)) { - GridCacheMvccCandidate prev = null; - GridCacheMvccCandidate owner = null; - - CacheObject val; - - synchronized (this) { - checkObsolete(); - - GridCacheMvcc mvcc = mvccExtras(); - - if (mvcc != null) { - prev = mvcc.anyOwner(); - - boolean emptyBefore = mvcc.isEmpty(); - - owner = mvcc.orderCompleted(baseVer, committedVers, rolledbackVers); - - boolean emptyAfter = mvcc.isEmpty(); - - checkCallbacks(emptyBefore, emptyAfter); - - if (emptyAfter) - mvccExtras(null); - } - - val = this.val; - } - - // This call must be made outside of synchronization. - checkOwnerChanged(prev, owner, val); - } - } - - /** - * - * @param lockVer Done version. - * @param baseVer Base version. - * @param committedVers Completed versions for reordering. - * @param rolledbackVers Rolled back versions for reordering. - * @param sysInvalidate Flag indicating if this entry is done from invalidated transaction (in case of tx - * salvage). In this case all locks before salvaged lock will marked as used and corresponding - * transactions will be invalidated. - * @throws GridCacheEntryRemovedException If entry has been removed. - * @return Owner. - */ - @Nullable public GridCacheMvccCandidate doneRemote( - GridCacheVersion lockVer, - GridCacheVersion baseVer, - Collection committedVers, - Collection rolledbackVers, - boolean sysInvalidate) throws GridCacheEntryRemovedException { - return doneRemote(lockVer, baseVer, Collections.emptySet(), committedVers, - rolledbackVers, sysInvalidate); } /** @@ -645,17 +539,16 @@ public void orderCompleted(GridCacheVersion baseVer, Collection pendingVers, Collection committedVers, Collection rolledbackVers, boolean sysInvalidate) throws GridCacheEntryRemovedException { - GridCacheMvccCandidate prev = null; - GridCacheMvccCandidate owner = null; + CacheLockCandidates prev = null; + CacheLockCandidates owner = null; CacheObject val; @@ -665,7 +558,7 @@ public void orderCompleted(GridCacheVersion baseVer, Collection readCache = ignite.cache(readCacheName); + final IgniteCache writeCache = ignite.cache(writeCacheName); + + List readKeys = testKeys(readCache); + + for (final Integer readKey : readKeys) { + final CyclicBarrier barrier = new CyclicBarrier(THREADS); + + readCache.put(readKey, Integer.MIN_VALUE); + + GridTestUtils.runMultiThreaded(new Callable() { + @Override public Void call() throws Exception { + try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) { + if (entry) + readCache.get(readKey); + else + readCache.getEntry(readKey); + + barrier.await(); + + writeCache.put(putKey.incrementAndGet(), 0); + + tx.commit(); + } + + return null; + } + }, THREADS, "test-thread"); + + assertEquals((Integer)Integer.MIN_VALUE, readCache.get(readKey)); + + readCache.put(readKey, readKey); + + assertEquals(readKey, readCache.get(readKey)); + } + } + + /** + * @throws Exception If failed. + */ + public void testNoReadLockConflictMultiNode() throws Exception { + Ignite ignite0 = ignite(0); + + for (final CacheConfiguration ccfg : cacheConfigurations()) { + logCacheInfo(ccfg); + + final AtomicInteger putKey = new AtomicInteger(1_000_000); + + ignite0.createCache(ccfg); + + try { + final int THREADS = 64; + + IgniteCache cache0 = ignite0.cache(ccfg.getName()); + + List readKeys = testKeys(cache0); + + for (final Integer readKey : readKeys) { + final CyclicBarrier barrier = new CyclicBarrier(THREADS); + + cache0.put(readKey, Integer.MIN_VALUE); + + final AtomicInteger idx = new AtomicInteger(); + + GridTestUtils.runMultiThreaded(new Callable() { + @Override public Void call() throws Exception { + Ignite ignite = ignite(idx.incrementAndGet() % (CLIENTS + SRVS)); + + IgniteCache cache = ignite.cache(ccfg.getName()); + + try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) { + cache.get(readKey); + + barrier.await(); + + cache.put(putKey.incrementAndGet(), 0); + + tx.commit(); + } + + return null; + } + }, THREADS, "test-thread"); + + assertEquals((Integer)Integer.MIN_VALUE, cache0.get(readKey)); + + cache0.put(readKey, readKey); + + assertEquals(readKey, cache0.get(readKey)); + } + } + finally { + destroyCache(ccfg.getName()); + } + } + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings("UnnecessaryLocalVariable") + public void testReadLockPessimisticTxConflict() throws Exception { + Ignite ignite0 = ignite(0); + + for (CacheConfiguration ccfg : cacheConfigurations()) { + logCacheInfo(ccfg); + + ignite0.createCache(ccfg); + + try { + Ignite ignite = ignite0; + + IgniteCache cache = ignite.cache(ccfg.getName()); + + Integer writeKey = Integer.MAX_VALUE; + + List readKeys = testKeys(cache); + + for (Integer readKey : readKeys) { + CountDownLatch latch = new CountDownLatch(1); + + IgniteInternalFuture fut = lockKey(latch, cache, readKey); + + try { + // No conflict for write, conflict with pessimistic tx for read. + try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) { + cache.put(writeKey, writeKey); + + cache.get(readKey); + + tx.commit(); + } + + fail(); + } + catch (TransactionOptimisticException e) { + log.info("Expected exception: " + e); + } + finally { + latch.countDown(); + } + + fut.get(); + } + } + finally { + destroyCache(ccfg.getName()); + } + } + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings("UnnecessaryLocalVariable") + public void testReadWriteTxConflict() throws Exception { + Ignite ignite0 = ignite(0); + + for (CacheConfiguration ccfg : cacheConfigurations()) { + logCacheInfo(ccfg); + + ignite0.createCache(ccfg); + + try { + Ignite ignite = ignite0; + + IgniteCache cache = ignite.cache(ccfg.getName()); + + Integer writeKey = Integer.MAX_VALUE; + + List readKeys = testKeys(cache); + + for (Integer readKey : readKeys) { + try { + // No conflict for read, conflict for write. + try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) { + cache.getAndPut(writeKey, writeKey); + + cache.get(readKey); + + updateKey(cache, writeKey, writeKey + readKey); + + tx.commit(); + } + + fail(); + } + catch (TransactionOptimisticException e) { + log.info("Expected exception: " + e); + } + + assertEquals((Integer)(writeKey + readKey), cache.get(writeKey)); + assertNull(cache.get(readKey)); + + cache.put(readKey, readKey); + + assertEquals(readKey, cache.get(readKey)); + } + } + finally { + destroyCache(ccfg.getName()); + } + } + } + + /** + * @throws Exception If failed. + */ + public void testReadWriteTransactionsNoDeadlock() throws Exception { + checkReadWriteTransactionsNoDeadlock(false); + } + + /** + * @throws Exception If failed. + */ + public void testReadWriteTransactionsNoDeadlockMultinode() throws Exception { + checkReadWriteTransactionsNoDeadlock(true); + } + + /** + * @param multiNode Multi-node test flag. + * @throws Exception If failed. + */ + private void checkReadWriteTransactionsNoDeadlock(final boolean multiNode) throws Exception { + final Ignite ignite0 = ignite(0); + + for (final CacheConfiguration ccfg : cacheConfigurations()) { + logCacheInfo(ccfg); + + ignite0.createCache(ccfg); + + try { + final long stopTime = U.currentTimeMillis() + 10_000; + + final AtomicInteger idx = new AtomicInteger(); + + GridTestUtils.runMultiThreaded(new Callable() { + @Override public Void call() throws Exception { + Ignite ignite = multiNode ? ignite(idx.incrementAndGet() % (SRVS + CLIENTS)) : ignite0; + + IgniteCache cache = ignite.cache(ccfg.getName()); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + while (U.currentTimeMillis() < stopTime) { + try { + try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) { + for (int i = 0; i < 10; i++) { + Integer key = rnd.nextInt(30); + + if (rnd.nextBoolean()) + cache.get(key); + else + cache.put(key, key); + } + + tx.commit(); + } + } + catch (TransactionOptimisticException ignore) { + // No-op. + } + } + + return null; + } + }, 32, "test-thread"); + } + finally { + destroyCache(ccfg.getName()); + } + } + } + + /** + * @throws Exception If failed. + */ + public void testReadWriteAccountTx() throws Exception { + final CacheConfiguration ccfg = cacheConfiguration(PARTITIONED, + FULL_SYNC, + 1, + false, + false); + + ignite(0).createCache(ccfg); + + try { + final int ACCOUNTS = 50; + final int VAL_PER_ACCOUNT = 1000; + + IgniteCache cache0 = ignite(0).cache(ccfg.getName()); + + final Set keys = new HashSet<>(); + + for (int i = 0; i < ACCOUNTS; i++) { + cache0.put(i, new Account(VAL_PER_ACCOUNT)); + + keys.add(i); + } + + final List clients = clients(); + + final AtomicBoolean stop = new AtomicBoolean(); + + final AtomicInteger idx = new AtomicInteger(); + + IgniteInternalFuture readFut = GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Void call() throws Exception { + try { + int threadIdx = idx.getAndIncrement(); + + int nodeIdx = threadIdx % (SRVS + CLIENTS); + + Ignite node = ignite(nodeIdx); + + IgniteCache cache = node.cache(ccfg.getName()); + + IgniteTransactions txs = node.transactions(); + + Integer putKey = ACCOUNTS + threadIdx; + + while (!stop.get()) { + int sum; + + while (true) { + sum = 0; + + try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) { + Map data = cache.getAll(keys); + + for (int i = 0; i < ACCOUNTS; i++) { + Account account = data.get(i); + + assertNotNull(account); + + sum += account.value(); + } + + cache.put(putKey, new Account(sum)); + + tx.commit(); + } + catch (TransactionOptimisticException e) { + continue; + } + + break; + } + + assertEquals(ACCOUNTS * VAL_PER_ACCOUNT, sum); + } + + return null; + } + catch (Throwable e) { + stop.set(true); + + log.error("Unexpected error: " + e); + + throw e; + } + } + }, (SRVS + CLIENTS) * 2, "update-thread"); + + IgniteInternalFuture updateFut = GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Void call() throws Exception { + try { + int nodeIdx = idx.getAndIncrement() % clients.size(); + + Ignite node = clients.get(nodeIdx); + + IgniteCache cache = node.cache(ccfg.getName()); + + IgniteTransactions txs = node.transactions(); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + while (!stop.get()) { + int id1 = rnd.nextInt(ACCOUNTS); + + int id2 = rnd.nextInt(ACCOUNTS); + + while (id2 == id1) + id2 = rnd.nextInt(ACCOUNTS); + + while (true) { + try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) { + Account a1 = cache.get(id1); + Account a2 = cache.get(id2); + + assertNotNull(a1); + assertNotNull(a2); + + if (a1.value() > 0) { + a1 = new Account(a1.value() - 1); + a2 = new Account(a2.value() + 1); + } + + cache.put(id1, a1); + cache.put(id2, a2); + + tx.commit(); + } + catch (TransactionOptimisticException e) { + continue; + } + + break; + } + } + + return null; + } + catch (Throwable e) { + stop.set(true); + + log.error("Unexpected error: " + e); + + throw e; + } + } + }, 2, "update-thread"); + + try { + U.sleep(15_000); + } + finally { + stop.set(true); + } + + readFut.get(); + updateFut.get(); + int sum = 0; + + for (int i = 0; i < ACCOUNTS; i++) { + Account a = cache0.get(i); + + assertNotNull(a); + assertTrue(a.value() >= 0); + + log.info("Account: " + a.value()); + + sum += a.value(); + } + + assertEquals(ACCOUNTS * VAL_PER_ACCOUNT, sum); + } + finally { + ignite(0).destroyCache(ccfg.getName()); + } + } + /** * @throws Exception If failed. */ @@ -4189,13 +4777,17 @@ private List testKeys(IgniteCache cache) throws Excep List keys = new ArrayList<>(); - if (ccfg.getCacheMode() == PARTITIONED) - keys.add(nearKey(cache)); + if (!cache.unwrap(Ignite.class).configuration().isClientMode()) { + if (ccfg.getCacheMode() == PARTITIONED) + keys.add(nearKey(cache)); - keys.add(primaryKey(cache)); + keys.add(primaryKey(cache)); - if (ccfg.getBackups() != 0) - keys.add(backupKey(cache)); + if (ccfg.getBackups() != 0) + keys.add(backupKey(cache)); + } + else + keys.add(nearKey(cache)); return keys; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java index 234f3627990f6..ff2d62dbf591b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java @@ -82,14 +82,14 @@ public void testAllTrueFlags() { ver, 1, ver, - 0, true, true, true, true, true, true, - null + null, + false ); c.setOwner(); @@ -123,14 +123,14 @@ public void testAllFalseFlags() { ver, 1, ver, - 0, false, false, false, false, false, false, - null + null, + false ); short flags = c.flags(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java index 1b97663941b00..11a91b5fe8eb8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java @@ -29,6 +29,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.Nullable; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheMode.PARTITIONED; @@ -37,6 +38,9 @@ * Test cases for multi-threaded tests in partitioned cache. */ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest { + /** */ + private static final UUID nodeId = UUID.randomUUID(); + /** Grid. */ private IgniteKernal grid; @@ -94,8 +98,8 @@ public void testNearLocalsWithPending() { GridCacheVersion ver1 = version(1); GridCacheVersion ver2 = version(2); - GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, 0, false, true); - GridCacheMvccCandidate c2 = entry.addNearLocal(node1, 1, ver2, 0, true); + GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, false, true); + GridCacheMvccCandidate c2 = entry.addNearLocal(node1, 1, ver2, true); Collection rmtCands = entry.remoteMvccSnapshot(); Collection nearLocCands = entry.localCandidates(); @@ -128,8 +132,8 @@ public void testNearLocalsWithCommitted() { GridCacheVersion ver1 = version(1); GridCacheVersion ver2 = version(2); - GridCacheMvccCandidate c1 = entry.addNearLocal(node1, 1, ver1, 0, true); - GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, 0, false, true); + GridCacheMvccCandidate c1 = entry.addNearLocal(node1, 1, ver1, true); + GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, false, true); Collection rmtCands = entry.remoteMvccSnapshot(); Collection nearLocCands = entry.localCandidates(); @@ -161,8 +165,8 @@ public void testNearLocalsWithRolledback() { GridCacheVersion ver1 = version(1); GridCacheVersion ver2 = version(2); - GridCacheMvccCandidate c1 = entry.addNearLocal(node1, 1, ver1, 0, true); - GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, 0, false, true); + GridCacheMvccCandidate c1 = entry.addNearLocal(node1, 1, ver1, true); + GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, false, true); Collection rmtCands = entry.remoteMvccSnapshot(); Collection nearLocCands = entry.localCandidates(); @@ -194,8 +198,8 @@ public void testNearLocals() { GridCacheVersion ver1 = version(1); GridCacheVersion ver2 = version(2); - GridCacheMvccCandidate c1 = entry.addNearLocal(node1, 1, ver1, 0, true); - GridCacheMvccCandidate c2 = entry.addNearLocal(node1, 1, ver2, 0, true); + GridCacheMvccCandidate c1 = entry.addNearLocal(node1, 1, ver1, true); + GridCacheMvccCandidate c2 = entry.addNearLocal(node1, 1, ver2, true); entry.readyNearLocal(ver2, ver2, empty(), empty(), empty()); @@ -224,8 +228,8 @@ public void testNearLocalsWithOwned() { GridCacheVersion ver1 = version(1); GridCacheVersion ver2 = version(2); - GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, 0, false, true); - GridCacheMvccCandidate c2 = entry.addNearLocal(node1, 1, ver2, 0, true); + GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, false, true); + GridCacheMvccCandidate c2 = entry.addNearLocal(node1, 1, ver2, true); Collection rmtCands = entry.remoteMvccSnapshot(); Collection nearLocCands = entry.localCandidates(); @@ -263,11 +267,11 @@ public void testAddPendingRemote0() throws Exception { GridCacheVersion ver0 = version(0); GridCacheVersion ver1 = version(1); - entry.addNearLocal(node1, 1, ver1, 0, true); + entry.addNearLocal(node1, 1, ver1, true); entry.readyNearLocal(ver1, ver1, empty(), empty(), Collections.singletonList(ver0)); - entry.addRemote(node1, 1, ver0, 0, false, true); + entry.addRemote(node1, 1, ver0, false, true); Collection rmtCands = entry.remoteMvccSnapshot(); Collection nearLocCands = entry.localCandidates(); @@ -297,13 +301,13 @@ public void testAddPendingRemote1() throws Exception { GridCacheVersion ver2 = version(2); GridCacheVersion ver3 = version(3); - GridCacheMvccCandidate c3 = entry.addNearLocal(node1, 1, ver3, 0, true); + GridCacheMvccCandidate c3 = entry.addNearLocal(node1, 1, ver3, true); entry.readyNearLocal(ver3, ver3, empty(), empty(), Arrays.asList(ver0, ver1, ver2)); - GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, 0, false, true); - GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, 0, false, true); - GridCacheMvccCandidate c0 = entry.addRemote(node1, 1, ver0, 0, false, true); + GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, false, true); + GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, false, true); + GridCacheMvccCandidate c0 = entry.addRemote(node1, 1, ver0, false, true); Collection rmtCands = entry.remoteMvccSnapshot(); @@ -340,13 +344,13 @@ public void testAddPendingRemote2() throws Exception { GridCacheVersion ver2 = version(2); GridCacheVersion ver3 = version(3); - GridCacheMvccCandidate c3 = entry.addNearLocal(node1, 1, ver3, 0, true); - entry.addNearLocal(node1, 1, ver2, 0, true); + GridCacheMvccCandidate c3 = entry.addNearLocal(node1, 1, ver3, true); + entry.addNearLocal(node1, 1, ver2, true); entry.readyNearLocal(ver3, ver3, empty(), empty(), Arrays.asList(ver0, ver1, ver2)); - GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, 0, false, true); - GridCacheMvccCandidate c0 = entry.addRemote(node1, 1, ver0, 0, false, true); + GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, false, true); + GridCacheMvccCandidate c0 = entry.addRemote(node1, 1, ver0, false, true); Collection rmtCands = entry.remoteMvccSnapshot(); @@ -386,12 +390,12 @@ public void testSalvageRemote() { GridCacheVersion ver5 = version(5); GridCacheVersion ver6 = version(6); - entry.addRemote(node1, 1, ver1, 0, false, true); - entry.addRemote(node1, 1, ver2, 0, false, true); - GridCacheMvccCandidate c3 = entry.addNearLocal(node1, 1, ver3, 0, true); - GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, 0, false, true); - entry.addRemote(node1, 1, ver5, 0, false, true); - entry.addRemote(node1, 1, ver6, 0, false, true); + entry.addRemote(node1, 1, ver1, false, true); + entry.addRemote(node1, 1, ver2, false, true); + GridCacheMvccCandidate c3 = entry.addNearLocal(node1, 1, ver3, true); + GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, false, true); + entry.addRemote(node1, 1, ver5, false, true); + entry.addRemote(node1, 1, ver6, false, true); Collection rmtCands = entry.remoteMvccSnapshot(); @@ -442,9 +446,9 @@ public void testNearRemoteConsistentOrdering0() throws Exception { GridCacheVersion ver2 = version(20); GridCacheVersion ver3 = version(30); - entry.addRemote(node1, 1, ver1, 0, false, true); - entry.addNearLocal(node1, 1, nearVer2, 0, true); - entry.addRemote(node1, 1, ver3, 0, false, true); + entry.addRemote(node1, 1, ver1, false, true); + entry.addNearLocal(node1, 1, nearVer2, true); + entry.addRemote(node1, 1, ver3, false, true); Collection rmtCands = entry.remoteMvccSnapshot(); Collection nearLocCands = entry.localCandidates(); @@ -480,9 +484,9 @@ public void testNearRemoteConsistentOrdering1() throws Exception { GridCacheVersion ver2 = version(20); GridCacheVersion ver3 = version(30); - entry.addRemote(node1, 1, ver1, 0, false, true); - entry.addNearLocal(node1, 1, nearVer2, 0, true); - entry.addRemote(node1, 1, ver3, 0, false, true); + entry.addRemote(node1, 1, ver1, false, true); + entry.addNearLocal(node1, 1, nearVer2, true); + entry.addRemote(node1, 1, ver3, false, true); Collection rmtCands = entry.remoteMvccSnapshot(); Collection nearLocCands = entry.localCandidates(); @@ -525,9 +529,9 @@ public void testNearRemoteConsistentOrdering2() throws Exception { GridCacheVersion ver2 = version(20); GridCacheVersion ver3 = version(30); - entry.addRemote(node1, 1, ver1, 0, false, true); - entry.addNearLocal(node1, 1, nearVer2, 0, true); - entry.addRemote(node1, 1, ver3, 0, false, true); + entry.addRemote(node1, 1, ver1, false, true); + entry.addNearLocal(node1, 1, nearVer2, true); + entry.addRemote(node1, 1, ver3, false, true); Collection rmtCands = entry.remoteMvccSnapshot(); Collection nearLocCands = entry.localCandidates(); @@ -570,9 +574,9 @@ public void testNearRemoteConsistentOrdering3() throws Exception { GridCacheVersion ver2 = version(20); GridCacheVersion ver3 = version(30); - entry.addRemote(node1, 1, ver1, 0, false, true); - entry.addNearLocal(node1, 1, nearVer2, 0, true); - entry.addRemote(node1, 1, ver3, 0, false, true); + entry.addRemote(node1, 1, ver1, false, true); + entry.addNearLocal(node1, 1, nearVer2, true); + entry.addRemote(node1, 1, ver3, false, true); Collection rmtCands = entry.remoteMvccSnapshot(); Collection nearLocCands = entry.localCandidates(); @@ -594,6 +598,222 @@ public void testNearRemoteConsistentOrdering3() throws Exception { assertEquals(ver1, rmtCands.iterator().next().version()); } + /** + * @throws Exception If failed. + */ + public void testSerializableReadLocksAdd() throws Exception { + GridCacheAdapter cache = grid.internalCache(); + + GridCacheVersion serOrder1 = new GridCacheVersion(0, 0, 10, 1); + GridCacheVersion serOrder2 = new GridCacheVersion(0, 0, 20, 1); + GridCacheVersion serOrder3 = new GridCacheVersion(0, 0, 15, 1); + + { + GridCacheMvcc mvcc = new GridCacheMvcc(cache.context()); + + GridCacheTestEntryEx e = new GridCacheTestEntryEx(cache.context(), "1"); + + GridCacheMvccCandidate cand1 = addLocal(mvcc, e, version(1), serOrder1, true); + + assertNotNull(cand1); + + GridCacheMvccCandidate cand2 = addLocal(mvcc, e, version(2), serOrder2, true); + + assertNotNull(cand2); + + GridCacheMvccCandidate cand3 = addLocal(mvcc, e, version(3), serOrder3, false); + + assertNull(cand3); + + cand3 = addLocal(mvcc, e, version(3), serOrder3, true); + + assertNotNull(cand3); + } + + { + GridCacheMvcc mvcc = new GridCacheMvcc(cache.context()); + + GridCacheTestEntryEx e = new GridCacheTestEntryEx(cache.context(), "1"); + + GridCacheMvccCandidate cand1 = addLocal(mvcc, e, version(1), serOrder2, true); + + assertNotNull(cand1); + + GridCacheMvccCandidate cand2 = addLocal(mvcc, e, version(2), serOrder1, true); + + assertNotNull(cand2); + + GridCacheMvccCandidate cand3 = addLocal(mvcc, e, version(3), serOrder3, false); + + assertNull(cand3); + + cand3 = addLocal(mvcc, e, version(3), serOrder3, true); + + assertNotNull(cand3); + } + + { + GridCacheMvcc mvcc = new GridCacheMvcc(cache.context()); + + GridCacheTestEntryEx e = new GridCacheTestEntryEx(cache.context(), "1"); + + GridCacheMvccCandidate cand1 = addLocal(mvcc, e, version(1), serOrder3, false); + + assertNotNull(cand1); + + GridCacheMvccCandidate cand2 = addLocal(mvcc, e, version(2), serOrder2, true); + + assertNotNull(cand2); + + GridCacheMvccCandidate cand3 = addLocal(mvcc, e, version(3), serOrder1, true); + + assertNull(cand3); + + cand3 = addLocal(mvcc, e, version(3), serOrder1, false); + + assertNull(cand3); + } + } + + /** + * @throws Exception If failed. + */ + public void testSerializableReadLocksAssign() throws Exception { + GridCacheAdapter cache = grid.internalCache(); + + GridCacheVersion serOrder1 = new GridCacheVersion(0, 0, 10, 1); + GridCacheVersion serOrder2 = new GridCacheVersion(0, 0, 20, 1); + GridCacheVersion serOrder3 = new GridCacheVersion(0, 0, 15, 1); + + { + GridCacheMvcc mvcc = new GridCacheMvcc(cache.context()); + + GridCacheTestEntryEx e = new GridCacheTestEntryEx(cache.context(), "1"); + + GridCacheMvccCandidate cand1 = addLocal(mvcc, e, version(1), serOrder1, true); + + assertNotNull(cand1); + + GridCacheMvccCandidate cand2 = addLocal(mvcc, e, version(2), serOrder2, true); + + assertNotNull(cand2); + + GridCacheMvccCandidate cand3 = addLocal(mvcc, e, version(3), serOrder3, false); + + assertNull(cand3); + + cand3 = addLocal(mvcc, e, version(3), serOrder3, true); + + assertNotNull(cand3); + + CacheLockCandidates owners = mvcc.recheck(); + + assertNull(owners); + + cand1.setReady(); + + owners = mvcc.recheck(); + + assertSame(cand1, owners); + checkCandidates(owners, cand1.version()); + + cand2.setReady(); + + owners = mvcc.recheck(); + checkCandidates(owners, cand1.version(), cand2.version()); + + mvcc.remove(cand1.version()); + + owners = mvcc.recheck(); + assertSame(cand2, owners); + checkCandidates(owners, cand2.version()); + } + + { + GridCacheMvcc mvcc = new GridCacheMvcc(cache.context()); + + GridCacheTestEntryEx e = new GridCacheTestEntryEx(cache.context(), "1"); + + GridCacheMvccCandidate cand1 = addLocal(mvcc, e, version(1), serOrder1, true); + + assertNotNull(cand1); + + GridCacheMvccCandidate cand2 = addLocal(mvcc, e, version(2), serOrder2, true); + + assertNotNull(cand2); + + GridCacheMvccCandidate cand3 = addLocal(mvcc, e, version(3), serOrder3, false); + + assertNull(cand3); + + cand3 = addLocal(mvcc, e, version(3), serOrder3, true); + + assertNotNull(cand3); + + CacheLockCandidates owners = mvcc.recheck(); + + assertNull(owners); + + cand2.setReady(); + + owners = mvcc.recheck(); + + assertSame(cand2, owners); + checkCandidates(owners, cand2.version()); + + cand1.setReady(); + + owners = mvcc.recheck(); + checkCandidates(owners, cand1.version(), cand2.version()); + + mvcc.remove(cand2.version()); + + owners = mvcc.recheck(); + assertSame(cand1, owners); + checkCandidates(owners, cand1.version()); + } + } + + /** + * @param all Candidates list. + * @param vers Expected candidates. + */ + private void checkCandidates(CacheLockCandidates all, GridCacheVersion...vers) { + assertNotNull(all); + assertEquals(vers.length, all.size()); + + for (GridCacheVersion ver : vers) + assertTrue(all.hasCandidate(ver)); + } + + /** + * @param mvcc Mvcc. + * @param e Entry. + * @param ver Version. + * @param serOrder Serializable tx version. + * @param read Read lock flag. + * @return Candidate. + */ + @Nullable private GridCacheMvccCandidate addLocal(GridCacheMvcc mvcc, + GridCacheEntryEx e, + GridCacheVersion ver, + GridCacheVersion serOrder, + boolean read) { + return mvcc.addLocal(e, + nodeId, + null, + 1, + ver, + 0, + serOrder, + false, + true, + false, + true, + read + ); + } + /** * @throws Exception If failed. */ @@ -627,7 +847,8 @@ private void checkNonSerializableConflict() throws Exception { false, true, false, - true + true, + false ); assertNotNull(cand1); @@ -642,7 +863,8 @@ private void checkNonSerializableConflict() throws Exception { false, true, false, - true + true, + false ); assertNull(cand2); @@ -681,7 +903,8 @@ private void checkSerializableAdd(boolean incVer) throws Exception { false, true, false, - true + true, + false ); assertNotNull(cand1); @@ -696,7 +919,8 @@ private void checkSerializableAdd(boolean incVer) throws Exception { false, true, false, - true + true, + false ); assertNotNull(cand2); @@ -711,7 +935,8 @@ private void checkSerializableAdd(boolean incVer) throws Exception { false, true, false, - true + true, + false ); assertNull(cand3); @@ -726,36 +951,37 @@ private void checkSerializableAdd(boolean incVer) throws Exception { false, true, false, - true + true, + false ); assertNotNull(cand4); - GridCacheMvccCandidate owner = mvcc.recheck(); + CacheLockCandidates owners = mvcc.recheck(); - assertNull(owner); + assertNull(owners); cand2.setReady(); - owner = mvcc.recheck(); + owners = mvcc.recheck(); - assertNull(owner); + assertNull(owners); cand1.setReady(); - owner = mvcc.recheck(); + owners = mvcc.recheck(); - assertSame(cand1, owner); + assertSame(cand1, owners); - owner = mvcc.recheck(); + owners = mvcc.recheck(); - assertSame(cand1, owner); + assertSame(cand1, owners); mvcc.remove(cand1.version()); - owner = mvcc.recheck(); + owners = mvcc.recheck(); - assertSame(cand2, owner); + assertSame(cand2, owners); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java index 59f9a9d0d7154..f46b2904785b5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java @@ -95,14 +95,14 @@ public void testMarshalUnmarshalCandidate() throws Exception { version(1), 123, version(2), - 123, /*local*/false, /*reentry*/false, true, false, false, false, - null + null, + false ); Marshaller marshaller = getTestResources().getMarshaller(); @@ -129,14 +129,14 @@ public void testRemotes() { GridCacheVersion ver4 = version(4); GridCacheVersion ver5 = version(5); - entry.addRemote(node1, 1, ver1, 0, false, true); + entry.addRemote(node1, 1, ver1, false, true); Collection cands = entry.remoteMvccSnapshot(); assert cands.size() == 1; assert cands.iterator().next().version().equals(ver1); - entry.addRemote(node2, 5, ver5, 0, false, true); + entry.addRemote(node2, 5, ver5, false, true); cands = entry.remoteMvccSnapshot(); @@ -147,7 +147,7 @@ public void testRemotes() { // Check order. checkOrder(cands, ver1, ver5); - entry.addRemote(node1, 3, ver3, 0, false, true); + entry.addRemote(node1, 3, ver3, false, true); cands = entry.remoteMvccSnapshot(); @@ -162,7 +162,7 @@ public void testRemotes() { checkDone(entry.candidate(ver3)); - entry.addRemote(node1, 2, ver2, 0, false, true); + entry.addRemote(node1, 2, ver2, false, true); cands = entry.remoteMvccSnapshot(); @@ -197,7 +197,7 @@ public void testRemotes() { checkDone(entry.candidate(ver5)); - entry.addRemote(node1, 4, ver4, 0, false, true); + entry.addRemote(node1, 4, ver4, false, true); cands = entry.remoteMvccSnapshot(); @@ -278,10 +278,10 @@ public void testNearRemoteWithOwned() { GridCacheVersion ver3 = version(3); GridCacheVersion ver4 = version(4); - GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, 0, false, true); - GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, 0, false, true); - GridCacheMvccCandidate c3 = entry.addRemote(node1, 1, ver3, 0, false, true); - GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, 0, false, true); + GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, false, true); + GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, false, true); + GridCacheMvccCandidate c3 = entry.addRemote(node1, 1, ver3, false, true); + GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, false, true); GridCacheMvccCandidate[] candArr = new GridCacheMvccCandidate[] {c1, c2, c3, c4}; @@ -322,12 +322,12 @@ public void testNearRemoteWithOwned1() { GridCacheVersion ver5 = version(5); GridCacheVersion ver6 = version(6); - GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, 0, false, true); - GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, 0, false, true); - GridCacheMvccCandidate c3 = entry.addRemote(node1, 1, ver3, 0, false, true); - GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, 0, false, true); - GridCacheMvccCandidate c5 = entry.addRemote(node1, 1, ver5, 0, false, true); - GridCacheMvccCandidate c6 = entry.addRemote(node1, 1, ver6, 0, false, true); + GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, false, true); + GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, false, true); + GridCacheMvccCandidate c3 = entry.addRemote(node1, 1, ver3, false, true); + GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, false, true); + GridCacheMvccCandidate c5 = entry.addRemote(node1, 1, ver5, false, true); + GridCacheMvccCandidate c6 = entry.addRemote(node1, 1, ver6, false, true); GridCacheMvccCandidate[] candArr = new GridCacheMvccCandidate[] {c1, c2, c3, c4, c5, c6}; @@ -369,13 +369,13 @@ public void testNearRemoteWithOwned2() { GridCacheVersion ver5 = version(5); GridCacheVersion ver6 = version(6); - GridCacheMvccCandidate c0 = entry.addRemote(node1, 1, ver0, 0, false, true); - GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, 0, false, true); - GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, 0, false, true); - GridCacheMvccCandidate c3 = entry.addRemote(node1, 1, ver3, 0, false, true); - GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, 0, false, true); - GridCacheMvccCandidate c5 = entry.addRemote(node1, 1, ver5, 0, false, true); - GridCacheMvccCandidate c6 = entry.addRemote(node1, 1, ver6, 0, false, true); + GridCacheMvccCandidate c0 = entry.addRemote(node1, 1, ver0, false, true); + GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, false, true); + GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, false, true); + GridCacheMvccCandidate c3 = entry.addRemote(node1, 1, ver3, false, true); + GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, false, true); + GridCacheMvccCandidate c5 = entry.addRemote(node1, 1, ver5, false, true); + GridCacheMvccCandidate c6 = entry.addRemote(node1, 1, ver6, false, true); GridCacheMvccCandidate[] candArr = new GridCacheMvccCandidate[] {c0, c1, c2, c3, c4, c5, c6}; @@ -486,7 +486,7 @@ public void testLocalWithRemote() { GridCacheVersion ver2 = version(2); GridCacheVersion ver3 = version(3); - entry.addRemote(nodeId, 1, ver2, 0, false, true); + entry.addRemote(nodeId, 1, ver2, false, true); entry.addLocal(3, ver3, 0, false, true); @@ -529,15 +529,15 @@ public void testCompletedWithBaseInTheMiddle() { GridCacheVersion ver7 = version(7); GridCacheVersion ver8 = version(8); - entry.addRemote(node1, 1, ver1, 0, false, true); - entry.addRemote(node2, 2, ver2, 0, false, true); - entry.addRemote(node1, 3, ver3, 0, false, true); - entry.addRemote(node2, 4, ver4, 0, false, true); - entry.addRemote(node1, 5, ver5, 0, false, true); - entry.addRemote(node2, 7, ver7, 0, false, true); - entry.addRemote(node2, 8, ver8, 0, false, true); + entry.addRemote(node1, 1, ver1, false, true); + entry.addRemote(node2, 2, ver2, false, true); + entry.addRemote(node1, 3, ver3, false, true); + entry.addRemote(node2, 4, ver4, false, true); + entry.addRemote(node1, 5, ver5, false, true); + entry.addRemote(node2, 7, ver7, false, true); + entry.addRemote(node2, 8, ver8, false, true); - GridCacheMvccCandidate doomed = entry.addRemote(node2, 6, ver6, 0, false, true); + GridCacheMvccCandidate doomed = entry.addRemote(node2, 6, ver6, false, true); // No reordering happens. checkOrder(entry.remoteMvccSnapshot(), ver1, ver2, ver3, ver4, ver5, ver7, ver8, ver6); @@ -581,13 +581,13 @@ public void testCompletedWithCompletedBaseInTheMiddle() { GridCacheVersion ver6 = version(6); GridCacheVersion ver7 = version(7); - entry.addRemote(node1, 1, ver1, 0, false, true); - entry.addRemote(node2, 2, ver2, 0, false, true); - entry.addRemote(node1, 3, ver3, 0, false, true); - entry.addRemote(node2, 4, ver4, 0, false, true); - entry.addRemote(node1, 5, ver5, 0, false, true); - entry.addRemote(node2, 6, ver6, 0, false, true); - entry.addRemote(node2, 7, ver7, 0, false, true); + entry.addRemote(node1, 1, ver1, false, true); + entry.addRemote(node2, 2, ver2, false, true); + entry.addRemote(node1, 3, ver3, false, true); + entry.addRemote(node2, 4, ver4, false, true); + entry.addRemote(node1, 5, ver5, false, true); + entry.addRemote(node2, 6, ver6, false, true); + entry.addRemote(node2, 7, ver7, false, true); List committed = Arrays.asList(ver4, ver6, ver2); @@ -623,13 +623,13 @@ public void testCompletedTwiceWithBaseInTheMiddle() { GridCacheVersion ver6 = version(6); GridCacheVersion ver7 = version(7); - entry.addRemote(node1, 1, ver1, 0, false, true); - entry.addRemote(node2, 2, ver2, 0, false, true); - entry.addRemote(node1, 3, ver3, 0, false, true); - entry.addRemote(node2, 4, ver4, 0, false, true); - entry.addRemote(node1, 5, ver5, 0, false, true); - entry.addRemote(node2, 6, ver6, 0, false, true); - entry.addRemote(node2, 7, ver7, 0, false, true); + entry.addRemote(node1, 1, ver1, false, true); + entry.addRemote(node2, 2, ver2, false, true); + entry.addRemote(node1, 3, ver3, false, true); + entry.addRemote(node2, 4, ver4, false, true); + entry.addRemote(node1, 5, ver5, false, true); + entry.addRemote(node2, 6, ver6, false, true); + entry.addRemote(node2, 7, ver7, false, true); List completed = Arrays.asList(ver4, ver6); @@ -669,11 +669,11 @@ public void testCompletedWithBaseInTheMiddleNoChange() { GridCacheVersion ver6 = version(6); GridCacheVersion ver7 = version(7); - entry.addRemote(node1, 1, ver1, 0, false, false); - entry.addRemote(node2, 2, ver2, 0, false, false); - entry.addRemote(node1, 3, ver3, 0, false, false); - entry.addRemote(node2, 4, ver4, 0, false, false); - entry.addRemote(node1, 5, ver5, 0, false, false); + entry.addRemote(node1, 1, ver1, false, false); + entry.addRemote(node2, 2, ver2, false, false); + entry.addRemote(node1, 3, ver3, false, false); + entry.addRemote(node2, 4, ver4, false, false); + entry.addRemote(node1, 5, ver5, false, false); List committed = Arrays.asList(ver6, ver7); @@ -708,13 +708,13 @@ public void testCompletedWithBaseInTheBeginning() { GridCacheVersion ver6 = version(6); GridCacheVersion ver7 = version(7); - entry.addRemote(node1, 1, ver1, 0, false, false); - entry.addRemote(node2, 2, ver2, 0, false, false); - entry.addRemote(node1, 3, ver3, 0, false, false); - entry.addRemote(node2, 4, ver4, 0, false, false); - entry.addRemote(node1, 5, ver5, 0, false, false); - entry.addRemote(node2, 6, ver6, 0, false, false); - entry.addRemote(node2, 7, ver7, 0, false, false); + entry.addRemote(node1, 1, ver1, false, false); + entry.addRemote(node2, 2, ver2, false, false); + entry.addRemote(node1, 3, ver3, false, false); + entry.addRemote(node2, 4, ver4, false, false); + entry.addRemote(node1, 5, ver5, false, false); + entry.addRemote(node2, 6, ver6, false, false); + entry.addRemote(node2, 7, ver7, false, false); List committed = Arrays.asList(ver4, ver6, ver3); @@ -750,11 +750,11 @@ public void testCompletedWithBaseInTheBeginningNoChange() { GridCacheVersion ver6 = version(6); GridCacheVersion ver7 = version(7); - entry.addRemote(node1, 1, ver1, 0, false, false); - entry.addRemote(node2, 2, ver2, 0, false, false); - entry.addRemote(node1, 3, ver3, 0, false, false); - entry.addRemote(node2, 4, ver4, 0, false, false); - entry.addRemote(node1, 5, ver5, 0, false, false); + entry.addRemote(node1, 1, ver1, false, false); + entry.addRemote(node2, 2, ver2, false, false); + entry.addRemote(node1, 3, ver3, false, false); + entry.addRemote(node2, 4, ver4, false, false); + entry.addRemote(node1, 5, ver5, false, false); List committed = Arrays.asList(ver6, ver7); @@ -789,11 +789,11 @@ public void testCompletedWithBaseInTheEndNoChange() { GridCacheVersion ver6 = version(6); GridCacheVersion ver7 = version(7); - entry.addRemote(node1, 1, ver1, 0, false, false); - entry.addRemote(node2, 2, ver2, 0, false, false); - entry.addRemote(node1, 3, ver3, 0, false, false); - entry.addRemote(node2, 4, ver4, 0, false, false); - entry.addRemote(node1, 5, ver5, 0, false, false); + entry.addRemote(node1, 1, ver1, false, false); + entry.addRemote(node2, 2, ver2, false, false); + entry.addRemote(node1, 3, ver3, false, false); + entry.addRemote(node2, 4, ver4, false, false); + entry.addRemote(node1, 5, ver5, false, false); List committed = Arrays.asList(ver6, ver7); @@ -829,12 +829,12 @@ public void testCompletedWithBaseNotPresentInTheMiddle() { GridCacheVersion ver7 = version(7); // Don't add version 2. - entry.addRemote(node1, 1, ver1, 0, false, true); - entry.addRemote(node1, 3, ver3, 0, false, true); - entry.addRemote(node2, 4, ver4, 0, false, true); - entry.addRemote(node1, 5, ver5, 0, false, true); - entry.addRemote(node2, 6, ver6, 0, false, true); - entry.addRemote(node2, 7, ver7, 0, false, true); + entry.addRemote(node1, 1, ver1, false, true); + entry.addRemote(node1, 3, ver3, false, true); + entry.addRemote(node2, 4, ver4, false, true); + entry.addRemote(node1, 5, ver5, false, true); + entry.addRemote(node2, 6, ver6, false, true); + entry.addRemote(node2, 7, ver7, false, true); List committed = Arrays.asList(ver6, ver4); @@ -870,9 +870,9 @@ public void testCompletedWithBaseNotPresentInTheMiddleNoChange() { GridCacheVersion ver7 = version(7); // Don't add versions 2, 5, 6, 7. - entry.addRemote(node1, 1, ver1, 0, false, true); - entry.addRemote(node1, 3, ver3, 0, false, true); - entry.addRemote(node2, 4, ver4, 0, false, true); + entry.addRemote(node1, 1, ver1, false, true); + entry.addRemote(node1, 3, ver3, false, true); + entry.addRemote(node2, 4, ver4, false, true); List committed = Arrays.asList(ver6, ver5, ver7); @@ -905,12 +905,12 @@ public void testCompletedWithBaseNotPresentInTheBeginning() { GridCacheVersion ver7 = version(7); // Don't add version 1. - entry.addRemote(node1, 2, ver2, 0, false, true); - entry.addRemote(node1, 3, ver3, 0, false, true); - entry.addRemote(node2, 4, ver4, 0, false, true); - entry.addRemote(node1, 5, ver5, 0, false, true); - entry.addRemote(node2, 6, ver6, 0, false, true); - entry.addRemote(node2, 7, ver7, 0, false, true); + entry.addRemote(node1, 2, ver2, false, true); + entry.addRemote(node1, 3, ver3, false, true); + entry.addRemote(node2, 4, ver4, false, true); + entry.addRemote(node1, 5, ver5, false, true); + entry.addRemote(node2, 6, ver6, false, true); + entry.addRemote(node2, 7, ver7, false, true); List committed = Arrays.asList(ver4, ver6, ver3); @@ -946,12 +946,12 @@ public void testCompletedWithBaseNotPresentInTheBeginningNoChange() { GridCacheVersion ver7 = version(7); // Don't add version 6, 7 - entry.addRemote(node1, 2, ver2, 0, false, true); - entry.addRemote(node1, 3, ver3, 0, false, true); - entry.addRemote(node2, 4, ver4, 0, false, true); - entry.addRemote(node1, 5, ver5, 0, false, true); - entry.addRemote(node1, 6, ver6, 0, false, true); - entry.addRemote(node1, 7, ver7, 0, false, true); + entry.addRemote(node1, 2, ver2, false, true); + entry.addRemote(node1, 3, ver3, false, true); + entry.addRemote(node2, 4, ver4, false, true); + entry.addRemote(node1, 5, ver5, false, true); + entry.addRemote(node1, 6, ver6, false, true); + entry.addRemote(node1, 7, ver7, false, true); List committed = Arrays.asList(ver2, ver3); @@ -987,10 +987,10 @@ public void testCompletedWithBaseNotPresentInTheEndNoChange() { GridCacheVersion ver7 = version(7); // Don't add version 5, 6, 7 - entry.addRemote(node1, 1, ver1, 0, false, true); - entry.addRemote(node1, 2, ver2, 0, false, true); - entry.addRemote(node1, 3, ver3, 0, false, true); - entry.addRemote(node2, 4, ver4, 0, false, true); + entry.addRemote(node1, 1, ver1, false, true); + entry.addRemote(node1, 2, ver2, false, true); + entry.addRemote(node1, 3, ver3, false, true); + entry.addRemote(node2, 4, ver4, false, true); List committed = Arrays.asList(ver6, ver7); @@ -1021,7 +1021,7 @@ public void testLocalAndRemote() { GridCacheVersion ver4 = version(4); GridCacheVersion ver5 = version(5); - entry.addRemote(node1, 1, ver1, 0, false, false); + entry.addRemote(node1, 1, ver1, false, false); entry.addLocal(2, ver2, 0, true, true); Collection cands = entry.remoteMvccSnapshot(); @@ -1029,7 +1029,7 @@ public void testLocalAndRemote() { assert cands.size() == 1; assert cands.iterator().next().version().equals(ver1); - entry.addRemote(node2, 5, ver5, 0, false, false); + entry.addRemote(node2, 5, ver5, false, false); cands = entry.remoteMvccSnapshot(); @@ -1040,7 +1040,7 @@ public void testLocalAndRemote() { checkOrder(cands, ver1, ver5); checkOrder(entry.localCandidates(true), ver2); - entry.addRemote(node1, 3, ver3, 0, false, true); + entry.addRemote(node1, 3, ver3, false, true); entry.addLocal(4, ver4, 0, /*reenter*/true, false); cands = entry.remoteMvccSnapshot(); @@ -1174,11 +1174,11 @@ public void testMultipleLocalAndRemoteLocks1() throws Exception { linkCandidates(ctx, c13, c33); - entry2.addRemote(nodeId, 3, ver2, 0, false, true); + entry2.addRemote(nodeId, 3, ver2, false, true); checkLocal(entry2.candidate(ver1), ver1, true, false, false); - entry3.addRemote(nodeId, 3, ver2, 0, false, false); + entry3.addRemote(nodeId, 3, ver2, false, false); entry3.readyLocal(ver3); @@ -1245,11 +1245,11 @@ public void testMultipleLocalAndRemoteLocks2() throws Exception { linkCandidates(ctx, c13, c33); - entry2.addRemote(UUID.randomUUID(), 3, ver1, 0, false, true); + entry2.addRemote(UUID.randomUUID(), 3, ver1, false, true); checkLocal(entry2.candidate(ver2), ver2, true, false, false); - entry3.addRemote(UUID.randomUUID(), 3, ver1, 0, false, true); + entry3.addRemote(UUID.randomUUID(), 3, ver1, false, true); entry3.readyLocal(ver3); @@ -1421,7 +1421,7 @@ public void testReverseOrder1() { checkLocal(c1k1, ver2, true, true, false); - GridCacheMvccCandidate c2k1 = entry1.addRemote(id, 2, ver1, 0, false, true); + GridCacheMvccCandidate c2k1 = entry1.addRemote(id, 2, ver1, false, true); // Force recheck of assignments. entry1.recheckLock(); @@ -1437,7 +1437,7 @@ public void testReverseOrder1() { assert c1k2.previous() == c1k1; - GridCacheMvccCandidate c2k2 = entry2.addRemote(id, 3, ver1, 0, false, true); + GridCacheMvccCandidate c2k2 = entry2.addRemote(id, 3, ver1, false, true); entry2.readyLocal(c1k2); @@ -1479,8 +1479,8 @@ public void testReverseOrder2() throws Exception { checkLocal(v3k2, ver3, false, false, false); // Remote locks. - GridCacheMvccCandidate v2k1 = entry1.addRemote(id, 3, ver2, 0, false, false); - GridCacheMvccCandidate v2k2 = entry2.addRemote(id, 3, ver2, 0, false, false); + GridCacheMvccCandidate v2k1 = entry1.addRemote(id, 3, ver2, false, false); + GridCacheMvccCandidate v2k2 = entry2.addRemote(id, 3, ver2, false, false); checkRemote(v2k1, ver2, false, false); checkRemote(v2k2, ver2, false, false); @@ -1602,7 +1602,7 @@ public void testReverseOrder4() throws Exception { checkLocal(v1k1, ver1, true, false, false); checkLocal(v1k2, ver1, true, false, false); - GridCacheMvccCandidate v2k2 = entry2.addRemote(id, 5, ver2, 0, false, false); + GridCacheMvccCandidate v2k2 = entry2.addRemote(id, 5, ver2, false, false); checkRemote(v2k2, ver2, false, false); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java index 396629ac4401a..48621afc1a8b8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java @@ -64,24 +64,12 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr * @param ctx Context. * @param key Key. */ - public GridCacheTestEntryEx(GridCacheContext ctx, Object key) { + GridCacheTestEntryEx(GridCacheContext ctx, Object key) { mvcc = new GridCacheMvcc(ctx); this.key = ctx.toCacheKeyObject(key); } - /** - * @param ctx Context. - * @param key Key. - * @param val Value. - */ - public GridCacheTestEntryEx(GridCacheContext ctx, Object key, Object val) { - mvcc = new GridCacheMvcc(ctx); - - this.key = ctx.toCacheKeyObject(key); - this.val = ctx.toCacheObject(val); - } - /** {@inheritDoc} */ @Override public int memorySize() throws IgniteCheckedException { return 1024; @@ -146,7 +134,7 @@ public GridCacheTestEntryEx(GridCacheContext ctx, Object key, Object val) { * @return New lock candidate if lock was added, or current owner if lock was reentered, * or null if lock was owned by another thread and timeout is negative. */ - @Nullable public GridCacheMvccCandidate addLocal( + @Nullable GridCacheMvccCandidate addLocal( long threadId, GridCacheVersion ver, long timeout, @@ -159,6 +147,7 @@ public GridCacheTestEntryEx(GridCacheContext ctx, Object key, Object val) { timeout, reenter, tx, + false, false ); } @@ -169,14 +158,13 @@ public GridCacheTestEntryEx(GridCacheContext ctx, Object key, Object val) { * @param nodeId Node ID. * @param threadId Thread ID. * @param ver Lock version. - * @param timeout Lock acquire timeout. * @param ec Not used. * @param tx Transaction flag. * @return Remote candidate. */ - public GridCacheMvccCandidate addRemote(UUID nodeId, long threadId, GridCacheVersion ver, long timeout, + GridCacheMvccCandidate addRemote(UUID nodeId, long threadId, GridCacheVersion ver, boolean ec, boolean tx) { - return mvcc.addRemote(this, nodeId, null, threadId, ver, timeout, tx, true, false); + return mvcc.addRemote(this, nodeId, null, threadId, ver, tx, true, false); } /** @@ -185,20 +173,19 @@ public GridCacheMvccCandidate addRemote(UUID nodeId, long threadId, GridCacheVer * @param nodeId Node ID. * @param threadId Thread ID. * @param ver Lock version. - * @param timeout Lock acquire timeout. * @param tx Transaction flag. * @return Remote candidate. */ - public GridCacheMvccCandidate addNearLocal(UUID nodeId, long threadId, GridCacheVersion ver, long timeout, + GridCacheMvccCandidate addNearLocal(UUID nodeId, long threadId, GridCacheVersion ver, boolean tx) { - return mvcc.addNearLocal(this, nodeId, null, threadId, ver, timeout, tx, true); + return mvcc.addNearLocal(this, nodeId, null, threadId, ver, tx, true, false); } /** * * @param baseVer Base version. */ - public void salvageRemote(GridCacheVersion baseVer) { + void salvageRemote(GridCacheVersion baseVer) { mvcc.salvageRemote(baseVer); } @@ -210,17 +197,16 @@ public void salvageRemote(GridCacheVersion baseVer) { * @param baseVer Base version. * @param committedVers Committed versions relative to base. * @param rolledbackVers Rolled back versions relative to base. - * @return Lock owner. */ - @Nullable public GridCacheMvccCandidate orderCompleted(GridCacheVersion baseVer, + void orderCompleted(GridCacheVersion baseVer, Collection committedVers, Collection rolledbackVers) { - return mvcc.orderCompleted(baseVer, committedVers, rolledbackVers); + mvcc.orderCompleted(baseVer, committedVers, rolledbackVers); } /** * @param ver Version. */ - public void doneRemote(GridCacheVersion ver) { + void doneRemote(GridCacheVersion ver) { mvcc.doneRemote(ver, Collections.emptyList(), Collections.emptyList(), Collections.emptyList()); } @@ -229,16 +215,15 @@ public void doneRemote(GridCacheVersion ver) { * @param baseVer Base version. * @param owned Owned. */ - public void orderOwned(GridCacheVersion baseVer, GridCacheVersion owned) { + void orderOwned(GridCacheVersion baseVer, GridCacheVersion owned) { mvcc.markOwned(baseVer, owned); } /** * @param ver Lock version to acquire or set to ready. - * @return Current owner. */ - @Nullable public GridCacheMvccCandidate readyLocal(GridCacheVersion ver) { - return mvcc.readyLocal(ver); + void readyLocal(GridCacheVersion ver) { + mvcc.readyLocal(ver); } /** @@ -247,44 +232,33 @@ public void orderOwned(GridCacheVersion baseVer, GridCacheVersion owned) { * @param committedVers Committed versions. * @param rolledbackVers Rolled back versions. * @param pending Pending versions. - * @return Lock owner. */ - @Nullable public GridCacheMvccCandidate readyNearLocal(GridCacheVersion ver, GridCacheVersion mapped, + void readyNearLocal(GridCacheVersion ver, GridCacheVersion mapped, Collection committedVers, Collection rolledbackVers, Collection pending) { - return mvcc.readyNearLocal(ver, mapped, committedVers, rolledbackVers, pending); + mvcc.readyNearLocal(ver, mapped, committedVers, rolledbackVers, pending); } /** * @param cand Candidate to set to ready. - * @return Current owner. */ - @Nullable public GridCacheMvccCandidate readyLocal(GridCacheMvccCandidate cand) { - return mvcc.readyLocal(cand); - } - - /** - * Local local release. - * @return Removed lock candidate or null if candidate was not removed. - */ - @Nullable public GridCacheMvccCandidate releaseLocal() { - return releaseLocal(Thread.currentThread().getId()); + void readyLocal(GridCacheMvccCandidate cand) { + mvcc.readyLocal(cand); } /** * Local release. * * @param threadId ID of the thread. - * @return Current owner. */ - @Nullable public GridCacheMvccCandidate releaseLocal(long threadId) { - return mvcc.releaseLocal(threadId); + void releaseLocal(long threadId) { + mvcc.releaseLocal(threadId); } /** * */ - public void recheckLock() { + void recheckLock() { mvcc.recheck(); } @@ -594,8 +568,9 @@ public void recheckLock() { long timeout, @Nullable GridCacheVersion serOrder, GridCacheVersion serReadVer, - boolean keepBinary) { - assert false; return false; + boolean read) { + assert false; + return false; } /** @inheritDoc */ @@ -765,7 +740,7 @@ public void recheckLock() { } /** @inheritDoc */ - public Collection localCandidates(boolean reentries, GridCacheVersion... exclude) { + Collection localCandidates(boolean reentries, GridCacheVersion... exclude) { return mvcc.localCandidates(reentries, exclude); } @@ -793,7 +768,7 @@ public Collection localCandidates(boolean reentries, Gri /** * @return Any MVCC owner. */ - public GridCacheMvccCandidate anyOwner() { + GridCacheMvccCandidate anyOwner() { return mvcc.anyOwner(); } diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java index b459dde735b0d..5c12f8496ee99 100644 --- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java +++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java @@ -21,6 +21,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; +import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.testframework.junits.GridTestKernalContext; @@ -86,10 +87,14 @@ public void testMapEntry() throws Exception { long timeout, @Nullable GridCacheVersion serOrder, GridCacheVersion serReadVer, - boolean keepBinary) { + boolean read) { return false; } + @Override protected void checkThreadChain(GridCacheMvccCandidate owner) { + // No-op. + } + @Override public void txUnlock(IgniteInternalTx tx) { // No-op. } From cc13d9d155f70e22e08ef203ac64e5cc0aa0a50f Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 28 Nov 2016 11:30:14 +0300 Subject: [PATCH 382/487] IGNITE-4026: Fixed BinaryObjectBuilder.build() can fail if one of the fields is Externalizable, enum from binary object. This closes #1281. This closes #1289. (cherry picked from commit 0b7c62d) --- .../internal/binary/GridBinaryMarshaller.java | 2 +- .../binary/builder/BinaryBuilderReader.java | 11 ++ .../builder/BinaryBuilderSerializer.java | 4 + .../builder/BinaryObjectBuilderImpl.java | 10 ++ ...BinaryObjectBuilderAdditionalSelfTest.java | 157 +++++++++++++++++- 5 files changed, 181 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java index ad635214898d0..624fa33fd0f5d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java @@ -43,7 +43,7 @@ public class GridBinaryMarshaller { private static final ThreadLocal BINARY_CTX = new ThreadLocal<>(); /** */ - static final byte OPTM_MARSH = -2; + public static final byte OPTM_MARSH = -2; /** */ public static final byte BYTE = 1; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java index 347fb2bb9d99e..baaabd6a1a0bd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java @@ -33,6 +33,7 @@ import java.util.Date; import java.util.HashMap; import java.util.Map; +import org.apache.ignite.internal.util.typedef.internal.U; import static java.nio.charset.StandardCharsets.UTF_8; @@ -757,6 +758,16 @@ else if (flag == GridBinaryMarshaller.DECIMAL) { return new BinaryPlainBinaryObject(binaryObj); } + case GridBinaryMarshaller.OPTM_MARSH: { + final BinaryHeapInputStream bin = BinaryHeapInputStream.create(arr, pos); + + final Object obj = BinaryUtils.doReadOptimized(bin, ctx, U.resolveClassLoader(ctx.configuration())); + + pos = bin.position(); + + return obj; + } + default: throw new BinaryObjectException("Invalid flag value: " + type); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java index b296437226426..6974176f90053 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java @@ -116,8 +116,12 @@ public void writeValue(BinaryWriterExImpl writer, Object val, boolean forceCol, String typeName = writer.context().userTypeName(clsName); BinaryMetadata meta = new BinaryMetadata(typeId, typeName, null, null, null, true); + writer.context().updateMetadata(typeId, meta); + // Need register class for marshaller to be able to deserialize enum value. + writer.context().descriptorForClass(val.getClass(), false); + writer.writeByte(GridBinaryMarshaller.ENUM); writer.writeInt(typeId); writer.writeInt(((Enum)val).ordinal()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java index f0bc8745bf266..6ea9e7ec3a728 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java @@ -22,6 +22,7 @@ import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.binary.BinaryType; +import org.apache.ignite.internal.binary.BinaryEnumObjectImpl; import org.apache.ignite.internal.binary.BinaryMetadata; import org.apache.ignite.internal.binary.BinaryObjectImpl; import org.apache.ignite.internal.binary.BinaryWriterExImpl; @@ -385,6 +386,15 @@ private Map checkMetadata(BinaryType meta, Map if (((BinaryValueWithType)newVal).value() == null) nullFieldVal = true; } + // Detect Enum and Enum array type. + else if (newVal instanceof BinaryEnumObjectImpl) + newFldTypeId = GridBinaryMarshaller.ENUM; + else if (newVal.getClass().isArray() && newVal.getClass().getComponentType() == BinaryObject.class) { + BinaryObject[] arr = (BinaryObject[])newVal; + + newFldTypeId = arr.length > 0 && arr[0] instanceof BinaryEnumObjectImpl ? + GridBinaryMarshaller.ENUM_ARR : GridBinaryMarshaller.OBJ_ARR; + } else newFldTypeId = BinaryUtils.typeByClass(newVal.getClass()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java index 24806cbf2bf01..507aa6bf239ea 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java @@ -21,6 +21,10 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; import java.lang.reflect.Field; import java.math.BigDecimal; import java.math.BigInteger; @@ -1423,11 +1427,19 @@ private BinaryObjectBuilderImpl wrap(Object obj) { * @return Wrapper. */ private BinaryObjectBuilderImpl newWrapper(Class aCls) { + return newWrapper(aCls.getName()); + } + + /** + * @param typeName Type name. + * @return Wrapper. + */ + private BinaryObjectBuilderImpl newWrapper(String typeName) { CacheObjectBinaryProcessorImpl processor = (CacheObjectBinaryProcessorImpl)( (IgniteBinaryImpl)binaries()).processor(); - return new BinaryObjectBuilderImpl(processor.binaryContext(), processor.typeId(aCls.getName()), - processor.binaryContext().userTypeName(aCls.getName())); + return new BinaryObjectBuilderImpl(processor.binaryContext(), processor.typeId(typeName), + processor.binaryContext().userTypeName(typeName)); } /** @@ -1508,4 +1520,145 @@ public void testCollectionsSerialization() { assert OBJ.equals(binaryObj.type().fieldTypeName("asSetHint")); assert OBJ.equals(binaryObj.type().fieldTypeName("asMapHint")); } + + /** + * Checks that externalizable value is correctly serialized/deserialized. + * + * @throws Exception If failed. + */ + public void testBuilderExternalizable() throws Exception { + BinaryObjectBuilder builder = newWrapper("TestType"); + + final TestObjectExternalizable exp = new TestObjectExternalizable("test"); + final TestObjectExternalizable[] expArr = new TestObjectExternalizable[]{ + new TestObjectExternalizable("test1"), new TestObjectExternalizable("test2")}; + + BinaryObject extObj = builder.setField("extVal", exp).setField("extArr", expArr).build(); + + assertEquals(exp, extObj.field("extVal")); + Assert.assertArrayEquals(expArr, (Object[])extObj.field("extArr")); + + builder = extObj.toBuilder(); + + extObj = builder.setField("intVal", 10).build(); + + assertEquals(exp, extObj.field("extVal")); + Assert.assertArrayEquals(expArr, (Object[])extObj.field("extArr")); + assertEquals(Integer.valueOf(10), extObj.field("intVal")); + + builder = extObj.toBuilder(); + + extObj = builder.setField("strVal", "some string").build(); + + assertEquals(exp, extObj.field("extVal")); + Assert.assertArrayEquals(expArr, (Object[])extObj.field("extArr")); + assertEquals(Integer.valueOf(10), extObj.field("intVal")); + assertEquals("some string", extObj.field("strVal")); + } + + /** + * Checks correct serialization/deserialization of enums in builder. + * + * @throws Exception If failed. + */ + public void testEnum() throws Exception { + BinaryObjectBuilder builder = newWrapper("TestType"); + + final TestEnum exp = TestEnum.A; + final TestEnum[] expArr = {TestEnum.A, TestEnum.B}; + + BinaryObject enumObj = builder.setField("testEnum", exp).setField("testEnumArr", expArr).build(); + + assertEquals(exp, ((BinaryObject)enumObj.field("testEnum")).deserialize()); + Assert.assertArrayEquals(expArr, (Object[])deserializeEnumBinaryArray(enumObj.field("testEnumArr"))); + + builder = newWrapper(enumObj.type().typeName()); + + enumObj = builder.setField("testEnum", (Object)enumObj.field("testEnum")) + .setField("testEnumArr", (Object)enumObj.field("testEnumArr")).build(); + + assertEquals(exp, ((BinaryObject)enumObj.field("testEnum")).deserialize()); + Assert.assertArrayEquals(expArr, (Object[])deserializeEnumBinaryArray(enumObj.field("testEnumArr"))); + } + + /** + * @param obj BinaryObject array. + * @return Deserialized enums. + */ + private TestEnum[] deserializeEnumBinaryArray(Object obj) { + Object[] arr = (Object[])obj; + + final TestEnum[] res = new TestEnum[arr.length]; + + for (int i = 0; i < arr.length; i++) + res[i] = ((BinaryObject)arr[i]).deserialize(); + + return res; + } + + /** + * + */ + private static class TestObjectExternalizable implements Externalizable { + /** */ + private String val; + + /** + * + */ + public TestObjectExternalizable() { + } + + /** + * @param val Value. + */ + public TestObjectExternalizable(final String val) { + this.val = val; + } + + /** {@inheritDoc} */ + @Override public void writeExternal(final ObjectOutput out) throws IOException { + out.writeUTF(val); + } + + /** {@inheritDoc} */ + @Override public void readExternal(final ObjectInput in) throws IOException, ClassNotFoundException { + val = in.readUTF(); + } + + /** {@inheritDoc} */ + @Override public boolean equals(final Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + final TestObjectExternalizable that = (TestObjectExternalizable)o; + + return val != null ? val.equals(that.val) : that.val == null; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return val != null ? val.hashCode() : 0; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "TestObjectExternalizable{" + + "val='" + val + '\'' + + '}'; + } + } + + /** + * + */ + private enum TestEnum { + /** */ + A, + + /** */ + B + } } From b4aedfd5350b4a318f1608596a171789513835a4 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Fri, 2 Dec 2016 20:10:09 +0300 Subject: [PATCH 383/487] IGNITE-4347: Fixed NPE. --- .../internal/processors/odbc/OdbcRequestHandler.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java index eef99454db212..0a70a68856672 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java @@ -184,11 +184,17 @@ private OdbcResponse executeQuery(long reqId, OdbcQueryExecuteRequest req) { qry.setDistributedJoins(distributedJoins); qry.setEnforceJoinOrder(enforceJoinOrder); - IgniteCache cache = ctx.grid().cache(req.cacheName()); + IgniteCache cache0 = ctx.grid().cache(req.cacheName()); + + if (cache0 == null) + return new OdbcResponse(OdbcResponse.STATUS_FAILED, + "Cache doesn't exist (did you configure it?): " + req.cacheName()); + + IgniteCache cache = cache0.withKeepBinary(); if (cache == null) return new OdbcResponse(OdbcResponse.STATUS_FAILED, - "Cache doesn't exist (did you configure it?): " + req.cacheName()); + "Can not get cache with keep binary: " + req.cacheName()); QueryCursor qryCur = cache.query(qry); From acf20b32d8fb68e42b904b091fb2b943f4558cef Mon Sep 17 00:00:00 2001 From: sboikov Date: Mon, 5 Dec 2016 14:01:28 +0300 Subject: [PATCH 384/487] ignite-4296 Optimize GridDhtPartitionsSingleMessage processing - optimized processing of GridDhtPartitionsSingleMessage - minor optimizations for RendezvousAffinityFunction - fixed heartbeats sending in tcp discovery --- .../RendezvousAffinityFunction.java | 80 +++-- .../discovery/GridDiscoveryManager.java | 118 +------- .../GridCachePartitionExchangeManager.java | 6 +- .../processors/cache/GridCacheUtils.java | 17 -- .../CacheObjectBinaryProcessorImpl.java | 3 +- .../dht/GridClientPartitionTopology.java | 120 ++++---- .../dht/GridDhtLocalPartition.java | 1 - .../dht/GridDhtPartitionTopology.java | 28 +- .../dht/GridDhtPartitionTopologyImpl.java | 284 +++++++++++------- .../preloader/GridDhtPartitionFullMap.java | 18 +- .../GridDhtPartitionsExchangeFuture.java | 56 +++- .../cache/transactions/IgniteTxHandler.java | 4 +- .../service/GridServiceProcessor.java | 4 +- .../ignite/spi/discovery/tcp/ServerImpl.java | 40 ++- .../tcp/internal/TcpDiscoveryStatistics.java | 4 + .../AbstractAffinityFunctionSelfTest.java | 2 +- ...ridDiscoveryManagerAliveCacheSelfTest.java | 2 +- ...ridCachePartitionedAffinitySpreadTest.java | 7 +- .../dht/GridCacheDhtTestUtils.java | 232 -------------- .../h2/twostep/GridReduceQueryExecutor.java | 14 +- 20 files changed, 437 insertions(+), 603 deletions(-) delete mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java index ec12973ceefd7..75e7c92b4f24c 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java @@ -17,7 +17,6 @@ package org.apache.ignite.cache.affinity.rendezvous; -import java.io.ByteArrayOutputStream; import java.io.Externalizable; import java.io.IOException; import java.io.ObjectInput; @@ -354,46 +353,69 @@ public Object resolveNodeHash(ClusterNode node) { /** * Returns collection of nodes (primary first) for specified partition. + * + * @param d Message digest. + * @param part Partition. + * @param nodes Nodes. + * @param nodesHash Serialized nodes hashes. + * @param backups Number of backups. + * @param neighborhoodCache Neighborhood. + * @return Assignment. */ - public List assignPartition(int part, List nodes, int backups, + public List assignPartition(MessageDigest d, + int part, + List nodes, + Map nodesHash, + int backups, @Nullable Map> neighborhoodCache) { if (nodes.size() <= 1) return nodes; - List> lst = new ArrayList<>(); + if (d == null) + d = digest.get(); - MessageDigest d = digest.get(); + List> lst = new ArrayList<>(nodes.size()); - for (ClusterNode node : nodes) { - Object nodeHash = resolveNodeHash(node); + try { + for (int i = 0; i < nodes.size(); i++) { + ClusterNode node = nodes.get(i); - try { - ByteArrayOutputStream out = new ByteArrayOutputStream(); + byte[] nodeHashBytes = nodesHash.get(node); - byte[] nodeHashBytes = U.marshal(ignite.configuration().getMarshaller(), nodeHash); + if (nodeHashBytes == null) { + Object nodeHash = resolveNodeHash(node); - out.write(U.intToBytes(part), 0, 4); // Avoid IOException. - out.write(nodeHashBytes, 0, nodeHashBytes.length); // Avoid IOException. + byte[] nodeHashBytes0 = U.marshal(ignite.configuration().getMarshaller(), nodeHash); + + // Add 4 bytes for partition bytes. + nodeHashBytes = new byte[nodeHashBytes0.length + 4]; + + System.arraycopy(nodeHashBytes0, 0, nodeHashBytes, 4, nodeHashBytes0.length); + + nodesHash.put(node, nodeHashBytes); + } + + U.intToBytes(part, nodeHashBytes, 0); d.reset(); - byte[] bytes = d.digest(out.toByteArray()); + byte[] bytes = d.digest(nodeHashBytes); long hash = - (bytes[0] & 0xFFL) - | ((bytes[1] & 0xFFL) << 8) - | ((bytes[2] & 0xFFL) << 16) - | ((bytes[3] & 0xFFL) << 24) - | ((bytes[4] & 0xFFL) << 32) - | ((bytes[5] & 0xFFL) << 40) - | ((bytes[6] & 0xFFL) << 48) - | ((bytes[7] & 0xFFL) << 56); + (bytes[0] & 0xFFL) + | ((bytes[1] & 0xFFL) << 8) + | ((bytes[2] & 0xFFL) << 16) + | ((bytes[3] & 0xFFL) << 24) + | ((bytes[4] & 0xFFL) << 32) + | ((bytes[5] & 0xFFL) << 40) + | ((bytes[6] & 0xFFL) << 48) + | ((bytes[7] & 0xFFL) << 56); lst.add(F.t(hash, node)); } - catch (IgniteCheckedException e) { - throw new IgniteException(e); - } + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); } Collections.sort(lst, COMPARATOR); @@ -474,8 +496,18 @@ else if (affinityBackupFilter == null && backupFilter == null) Map> neighborhoodCache = exclNeighbors ? GridCacheUtils.neighbors(affCtx.currentTopologySnapshot()) : null; + MessageDigest d = digest.get(); + + List nodes = affCtx.currentTopologySnapshot(); + + Map nodesHash = U.newHashMap(nodes.size()); + for (int i = 0; i < parts; i++) { - List partAssignment = assignPartition(i, affCtx.currentTopologySnapshot(), affCtx.backups(), + List partAssignment = assignPartition(d, + i, + nodes, + nodesHash, + affCtx.backups(), neighborhoodCache); assignments.add(partAssignment); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java index d24f900023a43..ddd4ee3e07b4c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java @@ -42,7 +42,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingQueue; @@ -1620,17 +1620,6 @@ public Collection cacheNodes(AffinityTopologyVersion topVer) { return resolveDiscoCache(null, topVer).allNodesWithCaches(topVer.topologyVersion()); } - /** - * Gets cache remote nodes for cache with given name. - * - * @param cacheName Cache name. - * @param topVer Topology version. - * @return Collection of cache nodes. - */ - public Collection remoteCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) { - return resolveDiscoCache(cacheName, topVer).remoteCacheNodes(cacheName, topVer.topologyVersion()); - } - /** * Gets cache remote nodes for cache with given name. * @@ -1648,7 +1637,7 @@ public Collection remoteCacheNodes(AffinityTopologyVersion topVer) * @param topVer Topology version. * @return Collection of cache nodes. */ - public Collection aliveCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) { + Collection aliveCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) { return resolveDiscoCache(cacheName, topVer).aliveCacheNodes(cacheName, topVer.topologyVersion()); } @@ -1659,38 +1648,20 @@ public Collection aliveCacheNodes(@Nullable String cacheName, Affin * @param topVer Topology version. * @return Collection of cache nodes. */ - public Collection aliveRemoteCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) { + Collection aliveRemoteCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) { return resolveDiscoCache(cacheName, topVer).aliveRemoteCacheNodes(cacheName, topVer.topologyVersion()); } /** - * Gets alive remote server nodes with at least one cache configured. - * * @param topVer Topology version (maximum allowed node order). - * @return Collection of alive cache nodes. + * @return Oldest alive server nodes with at least one cache configured. */ - public Collection aliveRemoteServerNodesWithCaches(AffinityTopologyVersion topVer) { - return resolveDiscoCache(null, topVer).aliveRemoteServerNodesWithCaches(topVer.topologyVersion()); - } + @Nullable public ClusterNode oldestAliveCacheServerNode(AffinityTopologyVersion topVer) { + DiscoCache cache = resolveDiscoCache(null, topVer); - /** - * Gets alive server nodes with at least one cache configured. - * - * @param topVer Topology version (maximum allowed node order). - * @return Collection of alive cache nodes. - */ - public Collection aliveServerNodesWithCaches(AffinityTopologyVersion topVer) { - return resolveDiscoCache(null, topVer).aliveServerNodesWithCaches(topVer.topologyVersion()); - } + Map.Entry e = cache.aliveSrvNodesWithCaches.firstEntry(); - /** - * Gets alive nodes with at least one cache configured. - * - * @param topVer Topology version (maximum allowed node order). - * @return Collection of alive cache nodes. - */ - public Collection aliveNodesWithCaches(AffinityTopologyVersion topVer) { - return resolveDiscoCache(null, topVer).aliveNodesWithCaches(topVer.topologyVersion()); + return e != null ? e.getKey() : null; } /** @@ -2579,20 +2550,10 @@ private class DiscoCache { */ private final ConcurrentMap> aliveRmtCacheNodes; - /** - * Cached alive remote nodes with caches. - */ - private final Collection aliveNodesWithCaches; - /** * Cached alive server remote nodes with caches. */ - private final Collection aliveSrvNodesWithCaches; - - /** - * Cached alive remote server nodes with caches. - */ - private final Collection aliveRmtSrvNodesWithCaches; + private final ConcurrentSkipListMap aliveSrvNodesWithCaches; /** * @param loc Local node. @@ -2625,9 +2586,7 @@ private DiscoCache(ClusterNode loc, Collection rmts) { aliveCacheNodes = new ConcurrentHashMap8<>(allNodes.size(), 1.0f); aliveRmtCacheNodes = new ConcurrentHashMap8<>(allNodes.size(), 1.0f); - aliveNodesWithCaches = new ConcurrentSkipListSet<>(); - aliveSrvNodesWithCaches = new ConcurrentSkipListSet<>(); - aliveRmtSrvNodesWithCaches = new ConcurrentSkipListSet<>(); + aliveSrvNodesWithCaches = new ConcurrentSkipListMap<>(GridNodeOrderComparator.INSTANCE); nodesByVer = new TreeMap<>(); long maxOrder0 = 0; @@ -2681,18 +2640,8 @@ private DiscoCache(ClusterNode loc, Collection rmts) { } } - if (hasCaches) { - if (alive(node.id())) { - aliveNodesWithCaches.add(node); - - if (!CU.clientNode(node)) { - aliveSrvNodesWithCaches.add(node); - - if (!loc.id().equals(node.id())) - aliveRmtSrvNodesWithCaches.add(node); - } - } - } + if (hasCaches && alive(node.id()) && !CU.clientNode(node)) + aliveSrvNodesWithCaches.put(node, Boolean.TRUE); IgniteProductVersion nodeVer = U.productVersion(node); @@ -2820,17 +2769,6 @@ Collection cacheNodes(@Nullable String cacheName, final long topVer return filter(topVer, allCacheNodes.get(cacheName)); } - /** - * Gets all remote nodes that have cache with given name. - * - * @param cacheName Cache name. - * @param topVer Topology version. - * @return Collection of nodes. - */ - Collection remoteCacheNodes(@Nullable String cacheName, final long topVer) { - return filter(topVer, rmtCacheNodes.get(cacheName)); - } - /** * Gets all remote nodes that have at least one cache configured. * @@ -2875,36 +2813,6 @@ Collection aliveRemoteCacheNodes(@Nullable String cacheName, final return filter(topVer, aliveRmtCacheNodes.get(maskNull(cacheName))); } - /** - * Gets all alive remote server nodes with at least one cache configured. - * - * @param topVer Topology version. - * @return Collection of nodes. - */ - Collection aliveRemoteServerNodesWithCaches(final long topVer) { - return filter(topVer, aliveRmtSrvNodesWithCaches); - } - - /** - * Gets all alive server nodes with at least one cache configured. - * - * @param topVer Topology version. - * @return Collection of nodes. - */ - Collection aliveServerNodesWithCaches(final long topVer) { - return filter(topVer, aliveSrvNodesWithCaches); - } - - /** - * Gets all alive remote nodes with at least one cache configured. - * - * @param topVer Topology version. - * @return Collection of nodes. - */ - Collection aliveNodesWithCaches(final long topVer) { - return filter(topVer, aliveNodesWithCaches); - } - /** * Checks if cache with given name has at least one node with near cache enabled. * @@ -2928,9 +2836,7 @@ void updateAlives(ClusterNode leftNode) { filterNodeMap(aliveRmtCacheNodes, leftNode); - aliveNodesWithCaches.remove(leftNode); aliveSrvNodesWithCaches.remove(leftNode); - aliveRmtSrvNodesWithCaches.remove(leftNode); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index 503b3348e51b4..7a24aa14ded91 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -740,7 +740,7 @@ public void scheduleResendPartitions() { * Partition refresh callback. */ private void refreshPartitions() { - ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, AffinityTopologyVersion.NONE); + ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(AffinityTopologyVersion.NONE); if (oldest == null) { if (log.isDebugEnabled()) @@ -1224,7 +1224,7 @@ else if (!cacheCtx.isLocal()) top = cacheCtx.topology(); if (top != null) - updated |= top.update(null, entry.getValue(), null) != null; + updated |= top.update(null, entry.getValue(), null); } if (!cctx.kernalContext().clientNode() && updated) @@ -1273,7 +1273,7 @@ else if (!cacheCtx.isLocal()) top = cacheCtx.topology(); if (top != null) { - updated |= top.update(null, entry.getValue(), null) != null; + updated |= top.update(null, entry.getValue(), null, true); cctx.affinity().checkRebalanceState(top, cacheId); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 90e428ca10b80..d32f4c1867bcf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -489,23 +489,6 @@ public static boolean isNearEnabled(CacheConfiguration cfg) { return cfg.getNearConfiguration() != null; } - /** - * Gets oldest alive server node with at least one cache configured for specified topology version. - * - * @param ctx Context. - * @param topVer Maximum allowed topology version. - * @return Oldest alive cache server node. - */ - @Nullable public static ClusterNode oldestAliveCacheServerNode(GridCacheSharedContext ctx, - AffinityTopologyVersion topVer) { - Collection nodes = ctx.discovery().aliveServerNodesWithCaches(topVer); - - if (nodes.isEmpty()) - return null; - - return oldest(nodes); - } - /** * @param nodes Nodes. * @return Oldest node for the given topology version. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java index 568a4dae683cc..1d60c422a6417 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java @@ -304,8 +304,7 @@ public CacheObjectBinaryProcessorImpl(GridKernalContext ctx) { assert !metaDataCache.context().affinityNode(); while (true) { - ClusterNode oldestSrvNode = - CU.oldestAliveCacheServerNode(ctx.cache().context(), AffinityTopologyVersion.NONE); + ClusterNode oldestSrvNode = ctx.discovery().oldestAliveCacheServerNode(AffinityTopologyVersion.NONE); if (oldestSrvNode == null) break; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java index 5efb31710a8a5..816132d351139 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java @@ -271,7 +271,7 @@ private void beforeExchange0(ClusterNode loc, GridDhtPartitionsExchangeFuture ex removeNode(exchId.nodeId()); // In case if node joins, get topology at the time of joining node. - ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, topVer); + ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(topVer); assert oldest != null; @@ -536,7 +536,8 @@ public long lastUpdateSequence() { try { assert node2part != null && node2part.valid() : "Invalid node2part [node2part: " + node2part + - ", locNodeId=" + cctx.localNodeId() + ", gridName=" + cctx.gridName() + ']'; + ", locNodeId=" + cctx.localNodeId() + + ", gridName=" + cctx.gridName() + ']'; GridDhtPartitionFullMap m = node2part; @@ -549,7 +550,7 @@ public long lastUpdateSequence() { /** {@inheritDoc} */ @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"}) - @Nullable @Override public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId, + @Nullable @Override public boolean update(@Nullable GridDhtPartitionExchangeId exchId, GridDhtPartitionFullMap partMap, Map cntrMap) { if (log.isDebugEnabled()) @@ -563,7 +564,7 @@ public long lastUpdateSequence() { log.debug("Stale exchange id for full partition map update (will ignore) [lastExchId=" + lastExchangeId + ", exchId=" + exchId + ']'); - return null; + return false; } if (node2part != null && node2part.compareTo(partMap) >= 0) { @@ -571,7 +572,7 @@ public long lastUpdateSequence() { log.debug("Stale partition map for full partition map update (will ignore) [lastExchId=" + lastExchangeId + ", exchId=" + exchId + ", curMap=" + node2part + ", newMap=" + partMap + ']'); - return null; + return false; } updateSeq.incrementAndGet(); @@ -634,7 +635,7 @@ public long lastUpdateSequence() { if (log.isDebugEnabled()) log.debug("Partition map after full update: " + fullMapString()); - return null; + return false; } finally { lock.writeLock().unlock(); @@ -642,10 +643,10 @@ public long lastUpdateSequence() { } /** {@inheritDoc} */ - @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"}) - @Nullable @Override public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId, + @Nullable @Override public boolean update(@Nullable GridDhtPartitionExchangeId exchId, GridDhtPartitionMap2 parts, - Map cntrMap) { + Map cntrMap, + boolean checkEvictions) { if (log.isDebugEnabled()) log.debug("Updating single partition map [exchId=" + exchId + ", parts=" + mapString(parts) + ']'); @@ -654,29 +655,27 @@ public long lastUpdateSequence() { log.debug("Received partition update for non-existing node (will ignore) [exchId=" + exchId + ", parts=" + parts + ']'); - return null; + return false; } lock.writeLock().lock(); try { if (stopping) - return null; + return false; if (lastExchangeId != null && exchId != null && lastExchangeId.compareTo(exchId) > 0) { if (log.isDebugEnabled()) log.debug("Stale exchange id for single partition map update (will ignore) [lastExchId=" + lastExchangeId + ", exchId=" + exchId + ']'); - return null; + return false; } if (exchId != null) lastExchangeId = exchId; if (node2part == null) { - U.dumpStack(log, "Created invalid: " + node2part); - // Create invalid partition map. node2part = new GridDhtPartitionFullMap(); } @@ -688,43 +687,45 @@ public long lastUpdateSequence() { log.debug("Stale update sequence for single partition map update (will ignore) [exchId=" + exchId + ", curSeq=" + cur.updateSequence() + ", newSeq=" + parts.updateSequence() + ']'); - return null; + return false; } long updateSeq = this.updateSeq.incrementAndGet(); - node2part = new GridDhtPartitionFullMap(node2part, updateSeq); - - boolean changed = false; + node2part.updateSequence(updateSeq); - if (cur == null || !cur.equals(parts)) - changed = true; + boolean changed = cur == null || !cur.equals(parts); - node2part.put(parts.nodeId(), parts); + if (changed) { + node2part.put(parts.nodeId(), parts); - part2node = new HashMap<>(part2node); + // Add new mappings. + for (Integer p : parts.keySet()) { + Set ids = part2node.get(p); - // Add new mappings. - for (Integer p : parts.keySet()) { - Set ids = part2node.get(p); + if (ids == null) + // Initialize HashSet to size 3 in anticipation that there won't be + // more than 3 nodes per partition. + part2node.put(p, ids = U.newHashSet(3)); - if (ids == null) - // Initialize HashSet to size 3 in anticipation that there won't be - // more than 3 nodes per partition. - part2node.put(p, ids = U.newHashSet(3)); + ids.add(parts.nodeId()); + } - changed |= ids.add(parts.nodeId()); - } + // Remove obsolete mappings. + if (cur != null) { + for (Integer p : cur.keySet()) { + if (parts.containsKey(p)) + continue; - // Remove obsolete mappings. - if (cur != null) { - for (Integer p : F.view(cur.keySet(), F0.notIn(parts.keySet()))) { - Set ids = part2node.get(p); + Set ids = part2node.get(p); - if (ids != null) - changed |= ids.remove(parts.nodeId()); + if (ids != null) + ids.remove(parts.nodeId()); + } } } + else + cur.updateSequence(parts.updateSequence(), parts.topologyVersion()); if (cntrMap != null) { for (Map.Entry e : cntrMap.entrySet()) { @@ -740,13 +741,18 @@ public long lastUpdateSequence() { if (log.isDebugEnabled()) log.debug("Partition map after single update: " + fullMapString()); - return changed ? localPartitionMap() : null; + return changed; } finally { lock.writeLock().unlock(); } } + /** {@inheritDoc} */ + @Override public void checkEvictions() { + // No-op. + } + /** * Updates value for single partition. * @@ -755,13 +761,12 @@ public long lastUpdateSequence() { * @param state State. * @param updateSeq Update sequence. */ - @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"}) private void updateLocal(int p, UUID nodeId, GridDhtPartitionState state, long updateSeq) { assert lock.isWriteLockedByCurrentThread(); assert nodeId.equals(cctx.localNodeId()); // In case if node joins, get topology at the time of joining node. - ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, topVer); + ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(topVer); // If this node became the oldest node. if (oldest.id().equals(cctx.localNodeId())) { @@ -811,7 +816,7 @@ private void removeNode(UUID nodeId) { assert nodeId != null; assert lock.writeLock().isHeldByCurrentThread(); - ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, topVer); + ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(topVer); ClusterNode loc = cctx.localNode(); @@ -876,18 +881,6 @@ private void removeNode(UUID nodeId) { } } - /** {@inheritDoc} */ - @Nullable @Override public GridDhtPartitionMap2 partitions(UUID nodeId) { - lock.readLock().lock(); - - try { - return node2part.get(nodeId); - } - finally { - lock.readLock().unlock(); - } - } - /** {@inheritDoc} */ @Override public Map updateCounters(boolean skipZeros) { lock.readLock().lock(); @@ -918,6 +911,27 @@ private void removeNode(UUID nodeId) { return false; } + /** {@inheritDoc} */ + @Override public boolean hasMovingPartitions() { + lock.readLock().lock(); + + try { + assert node2part != null && node2part.valid() : "Invalid node2part [node2part: " + node2part + + ", locNodeId=" + cctx.localNodeId() + + ", gridName=" + cctx.gridName() + ']'; + + for (GridDhtPartitionMap2 map : node2part.values()) { + if (map.hasMovingPartitions()) + return true; + } + + return false; + } + finally { + lock.readLock().unlock(); + } + } + /** {@inheritDoc} */ @Override public void printMemoryStats(int threshold) { X.println(">>> Cache partition topology stats [grid=" + cctx.gridName() + ", cacheId=" + cacheId + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java index 39a3e08cc5c51..668a1cdc006d9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java @@ -479,7 +479,6 @@ IgniteInternalFuture rent(boolean updateSeq) { if ((reservations & 0xFFFF) == 0 && casState(reservations, RENTING)) { shouldBeRenting = false; - if (log.isDebugEnabled()) log.debug("Moved partition to RENTING state: " + this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java index 0f75a5db47f3a..14ce1f9589ff1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java @@ -194,6 +194,11 @@ public void beforeExchange(GridDhtPartitionsExchangeFuture exchFut, boolean affR */ public GridDhtPartitionFullMap partitionMap(boolean onlyActive); + /** + * @return {@code True} If one of cache nodes has partitions in {@link GridDhtPartitionState#MOVING} state. + */ + public boolean hasMovingPartitions(); + /** * @param e Entry removed from cache. */ @@ -203,9 +208,9 @@ public void beforeExchange(GridDhtPartitionsExchangeFuture exchFut, boolean affR * @param exchId Exchange ID. * @param partMap Update partition map. * @param cntrMap Partition update counters. - * @return Local partition map if there were evictions or {@code null} otherwise. + * @return {@code True} if topology state changed. */ - public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId, + public boolean update(@Nullable GridDhtPartitionExchangeId exchId, GridDhtPartitionFullMap partMap, @Nullable Map cntrMap); @@ -213,11 +218,18 @@ public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId, * @param exchId Exchange ID. * @param parts Partitions. * @param cntrMap Partition update counters. - * @return Local partition map if there were evictions or {@code null} otherwise. + * @param checkEvictions Check evictions flag. + * @return {@code True} if topology state changed. */ - @Nullable public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId, + @Nullable public boolean update(@Nullable GridDhtPartitionExchangeId exchId, GridDhtPartitionMap2 parts, - @Nullable Map cntrMap); + @Nullable Map cntrMap, + boolean checkEvictions); + + /** + * + */ + public void checkEvictions(); /** * @param skipZeros If {@code true} then filters out zero counters. @@ -237,12 +249,6 @@ public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId, */ public void onEvicted(GridDhtLocalPartition part, boolean updateSeq); - /** - * @param nodeId Node to get partitions for. - * @return Partitions for node. - */ - @Nullable public GridDhtPartitionMap2 partitions(UUID nodeId); - /** * Prints memory stats. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java index ab573bd23c152..1b4dcc9293dee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java @@ -44,7 +44,6 @@ 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.GridDhtPartitionsExchangeFuture; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridAtomicLong; import org.apache.ignite.internal.util.StripedCompositeReadWriteLock; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -330,8 +329,9 @@ private boolean waitForRent() throws IgniteCheckedException { } /** {@inheritDoc} */ - @Override public void initPartitions( - GridDhtPartitionsExchangeFuture exchFut) throws IgniteInterruptedCheckedException { + @Override public void initPartitions(GridDhtPartitionsExchangeFuture exchFut) + throws IgniteInterruptedCheckedException + { U.writeLock(lock); try { @@ -356,9 +356,7 @@ private boolean waitForRent() throws IgniteCheckedException { private void initPartitions0(GridDhtPartitionsExchangeFuture exchFut, long updateSeq) { ClusterNode loc = cctx.localNode(); - ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx.shared(), topVer); - - assert oldest != null || cctx.kernalContext().clientNode(); + ClusterNode oldest = currentCoordinator(); GridDhtPartitionExchangeId exchId = exchFut.exchangeId(); @@ -397,7 +395,7 @@ private void initPartitions0(GridDhtPartitionsExchangeFuture exchFut, long updat if (log.isDebugEnabled()) log.debug("Owned partition for oldest node: " + locPart); - updateLocal(p, loc.id(), locPart.state(), updateSeq); + updateSeq = updateLocal(p, locPart.state(), updateSeq); } } } @@ -419,7 +417,7 @@ private void initPartitions0(GridDhtPartitionsExchangeFuture exchFut, long updat if (state.active()) { locPart.rent(false); - updateLocal(p, loc.id(), locPart.state(), updateSeq); + updateSeq = updateLocal(p, locPart.state(), updateSeq); if (log.isDebugEnabled()) log.debug("Evicting partition with rebalancing disabled " + @@ -443,8 +441,6 @@ else if (belongs) * @param updateSeq Update sequence. */ private void createPartitions(List> aff, long updateSeq) { - ClusterNode loc = cctx.localNode(); - int num = cctx.affinity().partitions(); for (int p = 0; p < num; p++) { @@ -454,7 +450,7 @@ private void createPartitions(List> aff, long updateSeq) { // will be created in MOVING state. GridDhtLocalPartition locPart = createPartition(p); - updateLocal(p, loc.id(), locPart.state(), updateSeq); + updateSeq = updateLocal(p, locPart.state(), updateSeq); } } // If this node's map is empty, we pre-create local partitions, @@ -485,10 +481,7 @@ else if (localNode(p, aff)) if (exchId.isLeft()) removeNode(exchId.nodeId()); - // In case if node joins, get topology at the time of joining node. - ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx.shared(), topVer); - - assert oldest != null || cctx.kernalContext().clientNode(); + ClusterNode oldest = currentCoordinator(); if (log.isDebugEnabled()) log.debug("Partition map beforeExchange [exchId=" + exchId + ", fullMap=" + fullMapString() + ']'); @@ -548,8 +541,6 @@ else if (!node2part.nodeId().equals(loc.id())) { @Override public boolean afterExchange(GridDhtPartitionsExchangeFuture exchFut) throws IgniteCheckedException { boolean changed = waitForRent(); - ClusterNode loc = cctx.localNode(); - int num = cctx.affinity().partitions(); AffinityTopologyVersion topVer = exchFut.topologyVersion(); @@ -600,7 +591,7 @@ else if (!node2part.nodeId().equals(loc.id())) { assert owned : "Failed to own partition [cacheName" + cctx.name() + ", locPart=" + locPart + ']'; - updateLocal(p, loc.id(), locPart.state(), updateSeq); + updateSeq = updateLocal(p, locPart.state(), updateSeq); changed = true; @@ -620,7 +611,7 @@ else if (log.isDebugEnabled()) locPart + ", owners = " + owners + ']'); } else - updateLocal(p, loc.id(), locPart.state(), updateSeq); + updateSeq = updateLocal(p, locPart.state(), updateSeq); } } else { @@ -630,7 +621,7 @@ else if (log.isDebugEnabled()) if (state == MOVING) { locPart.rent(false); - updateLocal(p, loc.id(), locPart.state(), updateSeq); + updateSeq = updateLocal(p, locPart.state(), updateSeq); changed = true; @@ -803,8 +794,11 @@ private GridDhtLocalPartition localPartition(int p, map.put(i, part.state()); } - return new GridDhtPartitionMap2(cctx.nodeId(), updateSeq.get(), topVer, - Collections.unmodifiableMap(map), true); + return new GridDhtPartitionMap2(cctx.nodeId(), + updateSeq.get(), + topVer, + Collections.unmodifiableMap(map), + true); } finally { lock.readLock().unlock(); @@ -985,7 +979,7 @@ private List ownersAndMoving(int p, AffinityTopologyVersion topVer) /** {@inheritDoc} */ @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"}) - @Nullable @Override public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId, + @Nullable @Override public boolean update(@Nullable GridDhtPartitionExchangeId exchId, GridDhtPartitionFullMap partMap, @Nullable Map cntrMap) { if (log.isDebugEnabled()) @@ -997,7 +991,7 @@ private List ownersAndMoving(int p, AffinityTopologyVersion topVer) try { if (stopping) - return null; + return false; if (cntrMap != null) { for (Map.Entry e : cntrMap.entrySet()) { @@ -1025,7 +1019,7 @@ private List ownersAndMoving(int p, AffinityTopologyVersion topVer) log.debug("Stale exchange id for full partition map update (will ignore) [lastExchId=" + lastExchangeId + ", exchId=" + exchId + ']'); - return null; + return false; } if (node2part != null && node2part.compareTo(partMap) >= 0) { @@ -1033,7 +1027,7 @@ private List ownersAndMoving(int p, AffinityTopologyVersion topVer) log.debug("Stale partition map for full partition map update (will ignore) [lastExchId=" + lastExchangeId + ", exchId=" + exchId + ", curMap=" + node2part + ", newMap=" + partMap + ']'); - return null; + return false; } long updateSeq = this.updateSeq.incrementAndGet(); @@ -1076,7 +1070,7 @@ private List ownersAndMoving(int p, AffinityTopologyVersion topVer) node2part = partMap; - Map> p2n = new HashMap<>(cctx.affinity().partitions(), 1.0f); + Map> p2n = U.newHashMap(cctx.affinity().partitions()); for (Map.Entry e : partMap.entrySet()) { for (Integer p : e.getValue().keySet()) { @@ -1110,7 +1104,7 @@ private List ownersAndMoving(int p, AffinityTopologyVersion topVer) if (log.isDebugEnabled()) log.debug("Partition map after full update: " + fullMapString()); - return changed ? localPartitionMap() : null; + return changed; } finally { lock.writeLock().unlock(); @@ -1118,10 +1112,10 @@ private List ownersAndMoving(int p, AffinityTopologyVersion topVer) } /** {@inheritDoc} */ - @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"}) - @Nullable @Override public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId, + @Nullable @Override public boolean update(@Nullable GridDhtPartitionExchangeId exchId, GridDhtPartitionMap2 parts, - @Nullable Map cntrMap) { + @Nullable Map cntrMap, + boolean checkEvictions) { if (log.isDebugEnabled()) log.debug("Updating single partition map [exchId=" + exchId + ", parts=" + mapString(parts) + ']'); @@ -1130,33 +1124,28 @@ private List ownersAndMoving(int p, AffinityTopologyVersion topVer) log.debug("Received partition update for non-existing node (will ignore) [exchId=" + exchId + ", parts=" + parts + ']'); - return null; + return false; } lock.writeLock().lock(); try { if (stopping) - return null; + return false; if (cntrMap != null) { for (Map.Entry e : cntrMap.entrySet()) { - Long cntr = this.cntrMap.get(e.getKey()); + Integer p = e.getKey(); - if (cntr == null || cntr < e.getValue()) - this.cntrMap.put(e.getKey(), e.getValue()); - } - - for (int i = 0; i < locParts.length(); i++) { - GridDhtLocalPartition part = locParts.get(i); + Long cntr = this.cntrMap.get(p); - if (part == null) - continue; + if (cntr == null || cntr < e.getValue()) + this.cntrMap.put(p, e.getValue()); - Long cntr = cntrMap.get(part.id()); + GridDhtLocalPartition part = locParts.get(p); - if (cntr != null) - part.updateCounter(cntr); + if (part != null) + part.updateCounter(e.getValue()); } } @@ -1165,7 +1154,7 @@ private List ownersAndMoving(int p, AffinityTopologyVersion topVer) log.debug("Stale exchange id for single partition map update (will ignore) [lastExchId=" + lastExchangeId + ", exchId=" + exchId + ']'); - return null; + return false; } if (exchId != null) @@ -1182,60 +1171,91 @@ private List ownersAndMoving(int p, AffinityTopologyVersion topVer) log.debug("Stale update sequence for single partition map update (will ignore) [exchId=" + exchId + ", curSeq=" + cur.updateSequence() + ", newSeq=" + parts.updateSequence() + ']'); - return null; + return false; } long updateSeq = this.updateSeq.incrementAndGet(); - node2part = new GridDhtPartitionFullMap(node2part, updateSeq); - - boolean changed = false; + node2part.newUpdateSequence(updateSeq); - if (cur == null || !cur.equals(parts)) - changed = true; + boolean changed = cur == null || !cur.equals(parts); - node2part.put(parts.nodeId(), parts); + if (changed) { + node2part.put(parts.nodeId(), parts); - part2node = new HashMap<>(part2node); + // Add new mappings. + for (Integer p : parts.keySet()) { + Set ids = part2node.get(p); - // Add new mappings. - for (Integer p : parts.keySet()) { - Set ids = part2node.get(p); + if (ids == null) + // Initialize HashSet to size 3 in anticipation that there won't be + // more than 3 nodes per partition. + part2node.put(p, ids = U.newHashSet(3)); - if (ids == null) - // Initialize HashSet to size 3 in anticipation that there won't be - // more than 3 nodes per partition. - part2node.put(p, ids = U.newHashSet(3)); + ids.add(parts.nodeId()); + } - changed |= ids.add(parts.nodeId()); - } + // Remove obsolete mappings. + if (cur != null) { + for (Integer p : cur.keySet()) { + if (parts.containsKey(p)) + continue; - // Remove obsolete mappings. - if (cur != null) { - for (Integer p : F.view(cur.keySet(), F0.notIn(parts.keySet()))) { - Set ids = part2node.get(p); + Set ids = part2node.get(p); - if (ids != null) - changed |= ids.remove(parts.nodeId()); + if (ids != null) + ids.remove(parts.nodeId()); + } } } + else + cur.updateSequence(parts.updateSequence(), parts.topologyVersion()); - AffinityTopologyVersion affVer = cctx.affinity().affinityTopologyVersion(); - - if (!affVer.equals(AffinityTopologyVersion.NONE) && affVer.compareTo(topVer) >= 0) { - List> aff = cctx.affinity().assignments(topVer); - - changed |= checkEvictions(updateSeq, aff); - - updateRebalanceVersion(aff); - } + if (checkEvictions) + changed |= checkEvictions(updateSeq); consistencyCheck(); if (log.isDebugEnabled()) log.debug("Partition map after single update: " + fullMapString()); - return changed ? localPartitionMap() : null; + return changed; + } + finally { + lock.writeLock().unlock(); + } + } + + /** + * @param updateSeq Update sequence. + * @return {@code True} if state changed. + */ + private boolean checkEvictions(long updateSeq) { + AffinityTopologyVersion affVer = cctx.affinity().affinityTopologyVersion(); + + boolean changed = false; + + if (!affVer.equals(AffinityTopologyVersion.NONE) && affVer.compareTo(topVer) >= 0) { + List> aff = cctx.affinity().assignments(topVer); + + changed = checkEvictions(updateSeq, aff); + + updateRebalanceVersion(aff); + } + + return changed; + } + + /** {@inheritDoc} */ + @Override public void checkEvictions() { + lock.writeLock().lock(); + + try { + long updateSeq = this.updateSeq.incrementAndGet(); + + node2part.newUpdateSequence(updateSeq); + + checkEvictions(updateSeq); } finally { lock.writeLock().unlock(); @@ -1270,7 +1290,7 @@ private boolean checkEvictions(long updateSeq, List> aff) { if (nodeIds.containsAll(F.nodeIds(affNodes))) { part.rent(false); - updateLocal(part.id(), locId, part.state(), updateSeq); + updateSeq = updateLocal(part.id(), part.state(), updateSeq); changed = true; @@ -1295,7 +1315,7 @@ private boolean checkEvictions(long updateSeq, List> aff) { if (locId.equals(n.id())) { part.rent(false); - updateLocal(part.id(), locId, part.state(), updateSeq); + updateSeq = updateLocal(part.id(), part.state(), updateSeq); changed = true; @@ -1315,20 +1335,28 @@ private boolean checkEvictions(long updateSeq, List> aff) { return changed; } + /** + * @return Current coordinator node. + */ + @Nullable private ClusterNode currentCoordinator() { + ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(topVer); + + assert oldest != null || cctx.kernalContext().clientNode(); + + return oldest; + } + /** * Updates value for single partition. * * @param p Partition. - * @param nodeId Node ID. * @param state State. * @param updateSeq Update sequence. + * @return Update sequence. */ @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"}) - private void updateLocal(int p, UUID nodeId, GridDhtPartitionState state, long updateSeq) { - assert nodeId.equals(cctx.nodeId()); - - // In case if node joins, get topology at the time of joining node. - ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx.shared(), topVer); + private long updateLocal(int p, GridDhtPartitionState state, long updateSeq) { + ClusterNode oldest = currentCoordinator(); assert oldest != null || cctx.kernalContext().clientNode(); @@ -1338,12 +1366,16 @@ private void updateLocal(int p, UUID nodeId, GridDhtPartitionState state, long u if (seq != updateSeq) { if (seq > updateSeq) { - if (this.updateSeq.get() < seq) { + long seq0 = this.updateSeq.get(); + + if (seq0 < seq) { // Update global counter if necessary. - boolean b = this.updateSeq.compareAndSet(this.updateSeq.get(), seq + 1); + boolean b = this.updateSeq.compareAndSet(seq0, seq + 1); - assert b : "Invalid update sequence [updateSeq=" + updateSeq + ", seq=" + seq + - ", curUpdateSeq=" + this.updateSeq.get() + ", node2part=" + node2part.toFullString() + ']'; + assert b : "Invalid update sequence [updateSeq=" + updateSeq + + ", seq=" + seq + + ", curUpdateSeq=" + this.updateSeq.get() + + ", node2part=" + node2part.toFullString() + ']'; updateSeq = seq + 1; } @@ -1355,11 +1387,19 @@ private void updateLocal(int p, UUID nodeId, GridDhtPartitionState state, long u } } - GridDhtPartitionMap2 map = node2part.get(nodeId); + UUID locNodeId = cctx.localNodeId(); + + GridDhtPartitionMap2 map = node2part.get(locNodeId); - if (map == null) - node2part.put(nodeId, map = new GridDhtPartitionMap2(nodeId, updateSeq, topVer, - Collections.emptyMap(), false)); + if (map == null) { + map = new GridDhtPartitionMap2(locNodeId, + updateSeq, + topVer, + Collections.emptyMap(), + false); + + node2part.put(locNodeId, map); + } map.updateSequence(updateSeq, topVer); @@ -1370,7 +1410,9 @@ private void updateLocal(int p, UUID nodeId, GridDhtPartitionState state, long u if (ids == null) part2node.put(p, ids = U.newHashSet(3)); - ids.add(nodeId); + ids.add(locNodeId); + + return updateSeq; } /** @@ -1395,8 +1437,6 @@ private void removeNode(UUID nodeId) { else node2part = new GridDhtPartitionFullMap(node2part, node2part.updateSequence()); - part2node = new HashMap<>(part2node); - GridDhtPartitionMap2 parts = node2part.remove(nodeId); if (parts != null) { @@ -1418,13 +1458,11 @@ private void removeNode(UUID nodeId) { /** {@inheritDoc} */ @Override public boolean own(GridDhtLocalPartition part) { - ClusterNode loc = cctx.localNode(); - lock.writeLock().lock(); try { if (part.own()) { - updateLocal(part.id(), loc.id(), part.state(), updateSeq.incrementAndGet()); + updateLocal(part.id(), part.state(), updateSeq.incrementAndGet()); consistencyCheck(); @@ -1452,7 +1490,7 @@ private void removeNode(UUID nodeId) { long seq = updateSeq ? this.updateSeq.incrementAndGet() : this.updateSeq.get(); - updateLocal(part.id(), cctx.localNodeId(), part.state(), seq); + updateLocal(part.id(), part.state(), seq); consistencyCheck(); } @@ -1461,18 +1499,6 @@ private void removeNode(UUID nodeId) { } } - /** {@inheritDoc} */ - @Nullable @Override public GridDhtPartitionMap2 partitions(UUID nodeId) { - lock.readLock().lock(); - - try { - return node2part.get(nodeId); - } - finally { - lock.readLock().unlock(); - } - } - /** {@inheritDoc} */ @Override public Map updateCounters(boolean skipZeros) { lock.readLock().lock(); @@ -1525,6 +1551,30 @@ private void removeNode(UUID nodeId) { return curTopVer.equals(topVer) && curTopVer.equals(rebalancedTopVer); } + /** {@inheritDoc} */ + @Override public boolean hasMovingPartitions() { + lock.readLock().lock(); + + try { + assert node2part != null && node2part.valid() : "Invalid node2part [node2part: " + node2part + + ", cache=" + cctx.name() + + ", started=" + cctx.started() + + ", stopping=" + stopping + + ", locNodeId=" + cctx.localNode().id() + + ", locName=" + cctx.gridName() + ']'; + + for (GridDhtPartitionMap2 map : node2part.values()) { + if (map.hasMovingPartitions()) + return true; + } + + return false; + } + finally { + lock.readLock().unlock(); + } + } + /** {@inheritDoc} */ @Override public void printMemoryStats(int threshold) { X.println(">>> Cache partition topology stats [grid=" + cctx.gridName() + ", cache=" + cctx.name() + ']'); @@ -1607,10 +1657,12 @@ private boolean hasState(final int p, @Nullable UUID nodeId, final GridDhtPartit if (state == match) return true; - if (matches != null && matches.length > 0) - for (GridDhtPartitionState s : matches) + if (matches != null && matches.length > 0) { + for (GridDhtPartitionState s : matches) { if (state == s) return true; + } + } } return false; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java index 8f5ad1714225c..e8860f25b2ab8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java @@ -103,10 +103,13 @@ public GridDhtPartitionFullMap(UUID nodeId, long nodeOrder, long updateSeq, Map< for (Map.Entry e : m.entrySet()) { GridDhtPartitionMap2 part = e.getValue(); - if (onlyActive) - put(e.getKey(), new GridDhtPartitionMap2(part.nodeId(), part.updateSequence(), part.topologyVersion(), part.map(), true)); - else - put(e.getKey(), part); + GridDhtPartitionMap2 cpy = new GridDhtPartitionMap2(part.nodeId(), + part.updateSequence(), + part.topologyVersion(), + part.map(), + onlyActive); + + put(e.getKey(), cpy); } } @@ -175,6 +178,13 @@ public boolean partitionStateEquals(GridDhtPartitionFullMap fullMap) { return true; } + /** + * @param updateSeq New update sequence value. + */ + public void newUpdateSequence(long updateSeq) { + this.updateSeq = updateSeq; + } + /** * @param updateSeq New update sequence value. * @return Old update sequence value. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index f391265039d99..e945de958a339 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java @@ -110,6 +110,10 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter remaining = new HashSet<>(); + /** */ + @GridToStringExclude + private int pendingSingleUpdates; + /** */ @GridToStringExclude private List srvNodes; @@ -1162,13 +1166,16 @@ public void onReceive(final ClusterNode node, final GridDhtPartitionsSingleMessa */ private void processMessage(ClusterNode node, GridDhtPartitionsSingleMessage msg) { boolean allReceived = false; + boolean updateSingleMap = false; synchronized (mux) { assert crd != null; if (crd.isLocal()) { if (remaining.remove(node.id())) { - updatePartitionSingleMap(msg); + updateSingleMap = true; + + pendingSingleUpdates++; allReceived = remaining.isEmpty(); } @@ -1177,8 +1184,42 @@ private void processMessage(ClusterNode node, GridDhtPartitionsSingleMessage msg singleMsgs.put(node, msg); } - if (allReceived) + if (updateSingleMap) { + try { + updatePartitionSingleMap(msg); + } + finally { + synchronized (mux) { + assert pendingSingleUpdates > 0; + + pendingSingleUpdates--; + + if (pendingSingleUpdates == 0) + mux.notifyAll(); + } + } + } + + if (allReceived) { + awaitSingleMapUpdates(); + onAllReceived(false); + } + } + + /** + * + */ + private void awaitSingleMapUpdates() { + synchronized (mux) { + try { + while (pendingSingleUpdates > 0) + U.wait(mux); + } + catch (IgniteInterruptedCheckedException e) { + U.warn(log, "Failed to wait for partition map updates, thread was interrupted: " + e); + } + } } /** @@ -1218,6 +1259,11 @@ private void onAllReceived(boolean discoThread) { } } + for (GridCacheContext cacheCtx : cctx.cacheContexts()) { + if (!cacheCtx.isLocal()) + cacheCtx.topology().checkEvictions(); + } + updateLastVersion(cctx.versions().last()); cctx.versions().onExchange(lastVer.get().order()); @@ -1374,7 +1420,7 @@ private void updatePartitionFullMap(GridDhtPartitionsFullMessage msg) { if (cacheCtx != null) cacheCtx.topology().update(exchId, entry.getValue(), cntrMap); else { - ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, AffinityTopologyVersion.NONE); + ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(AffinityTopologyVersion.NONE); if (oldest != null && oldest.isLocal()) cctx.exchange().clientTopology(cacheId, this).update(exchId, entry.getValue(), cntrMap); @@ -1395,7 +1441,7 @@ private void updatePartitionSingleMap(GridDhtPartitionsSingleMessage msg) { GridDhtPartitionTopology top = cacheCtx != null ? cacheCtx.topology() : cctx.exchange().clientTopology(cacheId, this); - top.update(exchId, entry.getValue(), msg.partitionUpdateCounters(cacheId)); + top.update(exchId, entry.getValue(), msg.partitionUpdateCounters(cacheId), false); } } @@ -1557,6 +1603,8 @@ public void onNodeLeft(final ClusterNode node) { if (crd0.isLocal()) { if (allReceived) { + awaitSingleMapUpdates(); + onAllReceived(true); return; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index fbd8ce50d034f..cf692640c2365 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -679,14 +679,14 @@ else if (txFinishMsgLog.isDebugEnabled()) { * @param req Request. * @return Future. */ - @Nullable public IgniteInternalFuture processNearTxFinishRequest(UUID nodeId, + @Nullable private IgniteInternalFuture processNearTxFinishRequest(UUID nodeId, GridNearTxFinishRequest req) { if (txFinishMsgLog.isDebugEnabled()) txFinishMsgLog.debug("Received near finish request [txId=" + req.version() + ", node=" + nodeId + ']'); IgniteInternalFuture fut = finish(nodeId, null, req); - assert req.txState() != null || fut.error() != null || + assert req.txState() != null || (fut != null && fut.error() != null) || (ctx.tm().tx(req.version()) == null && ctx.tm().nearTx(req.version()) == null); return fut; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 6c263638de7c5..b9b92b856b6c2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -1265,7 +1265,7 @@ private Iterator> serviceEntries(IgniteBiPredicate(); @@ -1589,7 +1589,7 @@ private class TopologyListener implements GridLocalEventListener { depExe.submit(new BusyRunnable() { @Override public void run0() { - ClusterNode oldest = CU.oldestAliveCacheServerNode(cache.context().shared(), topVer); + ClusterNode oldest = ctx.discovery().oldestAliveCacheServerNode(topVer); if (oldest != null && oldest.isLocal()) { final Collection retries = new ConcurrentLinkedQueue<>(); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 881474547740f..a660ec8bb09ae 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -129,6 +129,7 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingResponse; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryRedirectToClient; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessage; +import org.apache.ignite.thread.IgniteThreadPoolExecutor; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -174,8 +175,7 @@ class ServerImpl extends TcpDiscoveryImpl { IgniteProductVersion.fromString("1.5.0"); /** */ - private final ThreadPoolExecutor utilityPool = new ThreadPoolExecutor(0, 1, 2000, TimeUnit.MILLISECONDS, - new LinkedBlockingQueue()); + private IgniteThreadPoolExecutor utilityPool; /** Nodes ring. */ @GridToStringExclude @@ -297,6 +297,13 @@ class ServerImpl extends TcpDiscoveryImpl { spiState = DISCONNECTED; } + utilityPool = new IgniteThreadPoolExecutor("disco-pool", + spi.ignite().name(), + 0, + 1, + 2000, + new LinkedBlockingQueue()); + if (debugMode) { if (!log.isInfoEnabled()) throw new IgniteSpiException("Info log level should be enabled for TCP discovery to work " + @@ -2403,9 +2410,12 @@ private class RingMessageWorker extends MessageWorkerAdapter 0) + if (elapsed > 0 || !isLocalNodeCoordinator()) return; TcpDiscoveryHeartbeatMessage msg = new TcpDiscoveryHeartbeatMessage(getConfiguredNodeId()); @@ -5361,7 +5375,9 @@ private void checkHeartbeatsReceiving() { if (lastTimeStatusMsgSent < locNode.lastUpdateTime()) lastTimeStatusMsgSent = locNode.lastUpdateTime(); - long elapsed = (lastTimeStatusMsgSent + hbCheckFreq) - U.currentTimeMillis(); + long updateTime = Math.max(lastTimeStatusMsgSent, lastRingMsgTime); + + long elapsed = (updateTime + hbCheckFreq) - U.currentTimeMillis(); if (elapsed > 0) return; @@ -6062,11 +6078,11 @@ private boolean processJoinRequestMessage(TcpDiscoveryJoinRequestMessage msg, TcpDiscoverySpiState state = spiStateCopy(); - long socketTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() : + long sockTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() : spi.getSocketTimeout(); if (state == CONNECTED) { - spi.writeToSocket(msg, sock, RES_OK, socketTimeout); + spi.writeToSocket(msg, sock, RES_OK, sockTimeout); if (log.isDebugEnabled()) log.debug("Responded to join request message [msg=" + msg + ", res=" + RES_OK + ']'); @@ -6103,7 +6119,7 @@ private boolean processJoinRequestMessage(TcpDiscoveryJoinRequestMessage msg, // Local node is stopping. Remote node should try next one. res = RES_CONTINUE_JOIN; - spi.writeToSocket(msg, sock, res, socketTimeout); + spi.writeToSocket(msg, sock, res, sockTimeout); if (log.isDebugEnabled()) log.debug("Responded to join request message [msg=" + msg + ", res=" + res + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java index 9e73632feae6b..c79064491bb43 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java @@ -28,6 +28,8 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCustomEventMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddFinishedMessage; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddedMessage; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeFailedMessage; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeLeftMessage; @@ -317,7 +319,9 @@ public synchronized void onMessageSent(TcpDiscoveryAbstractMessage msg, long tim assert time >= 0 : time; if (crdSinceTs.get() > 0 && + (msg instanceof TcpDiscoveryCustomEventMessage) || (msg instanceof TcpDiscoveryNodeAddedMessage) || + (msg instanceof TcpDiscoveryNodeAddFinishedMessage) || (msg instanceof TcpDiscoveryNodeLeftMessage) || (msg instanceof TcpDiscoveryNodeFailedMessage)) { ringMsgsSndTs.put(msg.id(), U.currentTimeMillis()); diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java index 878d7d1a6a6a2..43017db856389 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java @@ -104,6 +104,7 @@ public void testRandomReassignmentThreeBackups() throws Exception { } /** + * @param backups Number of backups. * @throws Exception If failed. */ protected void checkNodeRemoved(int backups) throws Exception { @@ -247,7 +248,6 @@ else if (nodes.size() == maxNodes) { } } - /** * @param assignment Assignment to verify. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java index 390c83e46bba1..31b4bc7ca6682 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java @@ -239,7 +239,7 @@ private void validateAlives() { GridCacheSharedContext ctx = k.context().cache().context(); ClusterNode oldest = - GridCacheUtils.oldestAliveCacheServerNode(ctx, new AffinityTopologyVersion(currVer)); + ctx.discovery().oldestAliveCacheServerNode(new AffinityTopologyVersion(currVer)); assertNotNull(oldest); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java index a59ca8b4b30db..2d46cf4bc3e91 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java @@ -76,7 +76,12 @@ private void checkDistribution(RendezvousAffinityFunction aff, Collection parts = new HashMap<>(nodes.size()); for (int part = 0; part < aff.getPartitions(); part++) { - Collection affNodes = aff.assignPartition(part, new ArrayList(nodes), 0, null); + Collection affNodes = aff.assignPartition(null, + part, + new ArrayList<>(nodes), + new HashMap(), + 0, + null); assertEquals(1, affNodes.size()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java deleted file mode 100644 index 377a55f57d269..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java +++ /dev/null @@ -1,232 +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.distributed.dht; - -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteLogger; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.cache.affinity.AffinityFunction; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.GridCacheAdapter; -import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap; -import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; -import org.apache.ignite.internal.processors.cache.KeyCacheObject; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.P1; -import org.apache.ignite.internal.util.typedef.internal.CU; - -import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING; - -/** - * Utility methods for dht preloader testing. - */ -public class GridCacheDhtTestUtils { - /** - * Ensure singleton. - */ - private GridCacheDhtTestUtils() { - // No-op. - } - - /** - * @param dht Cache. - * @param keyCnt Number of test keys to put into cache. - * @throws IgniteCheckedException If failed to prepare. - */ - @SuppressWarnings({"UnusedAssignment", "unchecked"}) - static void prepareKeys(GridDhtCache dht, int keyCnt) throws IgniteCheckedException { - AffinityFunction aff = dht.context().config().getAffinity(); - - GridCacheConcurrentMap cacheMap; - - try { - Field field = GridCacheAdapter.class.getDeclaredField("map"); - - field.setAccessible(true); - - cacheMap = (GridCacheConcurrentMap)field.get(dht); - } - catch (Exception e) { - throw new IgniteCheckedException("Failed to get cache map.", e); - } - - GridDhtPartitionTopology top = dht.topology(); - - GridCacheContext ctx = dht.context(); - - for (int i = 0; i < keyCnt; i++) { - KeyCacheObject cacheKey = ctx.toCacheKeyObject(i); - - cacheMap.putEntryIfObsoleteOrAbsent( - AffinityTopologyVersion.NONE, - cacheKey, - ctx.toCacheKeyObject("value" + i), - false, - false); - - dht.preloader().request(Collections.singleton(cacheKey), AffinityTopologyVersion.NONE); - - GridDhtLocalPartition part = top.localPartition(aff.partition(i), false); - - assert part != null; - - part.own(); - } - } - - /** - * @param dht Dht cache. - * @param idx Cache index - */ - static void printDhtTopology(GridDhtCache dht, int idx) { - final Affinity aff = dht.affinity(); - - Ignite ignite = dht.context().grid(); - ClusterNode locNode = ignite.cluster().localNode(); - - GridDhtPartitionTopology top = dht.topology(); - - System.out.println("\nTopology of cache #" + idx + " (" + locNode.id() + ")" + ":"); - System.out.println("----------------------------------"); - - List affParts = new LinkedList<>(); - - GridDhtPartitionMap2 map = dht.topology().partitions(locNode.id()); - - if (map != null) - for (int p : map.keySet()) - affParts.add(p); - - Collections.sort(affParts); - - System.out.println("Affinity partitions: " + affParts + "\n"); - - List locals = new ArrayList(top.localPartitions()); - - Collections.sort(locals); - - for (final GridDhtLocalPartition part : locals) { - Collection partNodes = aff.mapKeyToPrimaryAndBackups(part.id()); - - String ownStr = !partNodes.contains(dht.context().localNode()) ? "NOT AN OWNER" : - F.eqNodes(CU.primary(partNodes), locNode) ? "PRIMARY" : "BACKUP"; - - Collection keys = F.viewReadOnly(dht.keySet(), F.identity(), new P1() { - @Override public boolean apply(Integer k) { - return aff.partition(k) == part.id(); - } - }); - - System.out.println("Local partition: [" + part + "], [owning=" + ownStr + ", keyCnt=" + keys.size() + - ", keys=" + keys + "]"); - } - - System.out.println("\nNode map:"); - - for (Map.Entry e : top.partitionMap(false).entrySet()) { - List list = new ArrayList<>(e.getValue().keySet()); - - Collections.sort(list); - - System.out.println("[node=" + e.getKey() + ", parts=" + list + "]"); - } - - System.out.println(""); - } - - /** - * Checks consistency of partitioned cache. - * Any preload processes must be finished before this method call(). - * - * @param dht Dht cache. - * @param idx Cache index. - * @param log Logger. - */ - @SuppressWarnings("unchecked") - static void checkDhtTopology(GridDhtCache dht, int idx, IgniteLogger log) { - assert dht != null; - assert idx >= 0; - assert log != null; - - log.info("Checking balanced state of cache #" + idx); - - Affinity aff = (Affinity)dht.affinity(); - - Ignite ignite = dht.context().grid(); - ClusterNode locNode = ignite.cluster().localNode(); - - GridDhtPartitionTopology top = dht.topology(); - - // Expected partitions calculated with affinity function. - // They should be in topology in OWNING state. - Collection affParts = new HashSet<>(); - - GridDhtPartitionMap2 map = dht.topology().partitions(locNode.id()); - - if (map != null) - for (int p : map.keySet()) - affParts.add(p); - - if (F.isEmpty(affParts)) - return; - - for (int p : affParts) - assert top.localPartition(p, false) != null : - "Partition does not exist in topology: [cache=" + idx + ", part=" + p + "]"; - - for (GridDhtLocalPartition p : top.localPartitions()) { - assert affParts.contains(p.id()) : - "Invalid local partition: [cache=" + idx + ", part=" + p + ", node partitions=" + affParts + "]"; - - assert p.state() == OWNING : "Invalid partition state [cache=" + idx + ", part=" + p + "]"; - - Collection partNodes = aff.mapPartitionToPrimaryAndBackups(p.id()); - - assert partNodes.contains(locNode) : - "Partition affinity nodes does not contain local node: [cache=" + idx + "]"; - } - - // Check keys. - for (GridCacheEntryEx e : dht.entries()) { - GridDhtCacheEntry entry = (GridDhtCacheEntry)e; - - if (!affParts.contains(entry.partition())) - log.warning("Partition of stored entry is obsolete for node: [cache=" + idx + ", entry=" + entry + - ", node partitions=" + affParts + "]"); - - int p = aff.partition(entry.key()); - - if (!affParts.contains(p)) - log.warning("Calculated entry partition is not in node partitions: [cache=" + idx + ", part=" + p + - ", entry=" + entry + ", node partitions=" + affParts + "]"); - } - } -} 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 48567dabb018e..40b1197317acd 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 @@ -58,8 +58,7 @@ import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheContext; -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.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable; import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery; import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery; @@ -383,17 +382,10 @@ private boolean isPreloadingActive(final GridCacheContext cctx, List cctx) { - GridDhtPartitionFullMap fullMap = cctx.topology().partitionMap(false); - - for (GridDhtPartitionMap2 map : fullMap.values()) { - if (map.hasMovingPartitions()) - return true; - } - - return false; + return cctx.topology().hasMovingPartitions(); } /** From 6ba1711a1fa10d8276974227491136070c3ed43a Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Tue, 6 Dec 2016 12:55:41 +0300 Subject: [PATCH 385/487] IGNITE-4242 ExchangeManager should wait for cache rebalancing in async way --- .../GridCachePartitionExchangeManager.java | 128 +++------- .../processors/cache/GridCachePreloader.java | 11 +- .../cache/GridCachePreloaderAdapter.java | 5 +- .../preloader/GridDhtPartitionDemander.java | 230 +++++++++++------- .../dht/preloader/GridDhtPreloader.java | 9 +- .../GridCacheRebalancingSyncSelfTest.java | 2 + 6 files changed, 183 insertions(+), 202 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index 7a24aa14ded91..f04a6ce2fa025 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -21,18 +21,18 @@ import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.Comparator; import java.util.Date; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.NavigableMap; -import java.util.Queue; import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.LinkedBlockingDeque; @@ -87,7 +87,6 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; -import org.apache.ignite.internal.util.typedef.internal.GPC; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.GridWorker; @@ -97,13 +96,11 @@ import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; -import org.jsr166.ConcurrentLinkedDeque8; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PRELOAD_RESEND_TIMEOUT; import static org.apache.ignite.IgniteSystemProperties.IGNITE_THREAD_DUMP_ON_EXCHANGE_TIMEOUT; import static org.apache.ignite.IgniteSystemProperties.getLong; -import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STARTED; import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_JOINED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; @@ -156,9 +153,6 @@ public class GridCachePartitionExchangeManager extends GridCacheSharedMana /** */ private GridFutureAdapter reconnectExchangeFut; - /** */ - private final Queue> rebalanceQ = new ConcurrentLinkedDeque8<>(); - /** * Partition map futures. * This set also contains already completed exchange futures to address race conditions when coordinator @@ -1596,12 +1590,8 @@ void addFuture(GridDhtPartitionsExchangeFuture exchFut) { @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException { long timeout = cctx.gridConfig().getNetworkTimeout(); - boolean startEvtFired = false; - int cnt = 0; - IgniteInternalFuture asyncStartFut = null; - while (!isCancelled()) { GridDhtPartitionsExchangeFuture exchFut = null; @@ -1703,20 +1693,8 @@ void addFuture(GridDhtPartitionsExchangeFuture exchFut) { continue; changed |= cacheCtx.topology().afterExchange(exchFut); - - // Preload event notification. - if (!exchFut.skipPreload() && cacheCtx.events().isRecordable(EVT_CACHE_REBALANCE_STARTED)) { - if (!cacheCtx.isReplicated() || !startEvtFired) { - DiscoveryEvent discoEvt = exchFut.discoveryEvent(); - - cacheCtx.events().addPreloadEvent(-1, EVT_CACHE_REBALANCE_STARTED, - discoEvt.eventNode(), discoEvt.type(), discoEvt.timestamp()); - } - } } - startEvtFired = true; - if (!cctx.kernalContext().clientNode() && changed && futQ.isEmpty()) refreshPartitions(); } @@ -1755,8 +1733,6 @@ void addFuture(GridDhtPartitionsExchangeFuture exchFut) { if (assignsMap != null) { int size = assignsMap.size(); - rebalanceQ.clear(); - NavigableMap> orderMap = new TreeMap<>(); for (Map.Entry e : assignsMap.entrySet()) { @@ -1772,101 +1748,65 @@ void addFuture(GridDhtPartitionsExchangeFuture exchFut) { orderMap.get(order).add(cacheId); } - Callable marshR = null; - List> orderedRs = new ArrayList<>(size); + Runnable r = null; + + List rebList = new LinkedList<>(); + + boolean assignsCancelled = false; - //Ordered rebalance scheduling. - for (Integer order : orderMap.keySet()) { + for (Integer order : orderMap.descendingKeySet()) { for (Integer cacheId : orderMap.get(order)) { GridCacheContext cacheCtx = cctx.cacheContext(cacheId); - List waitList = new ArrayList<>(size - 1); + GridDhtPreloaderAssignments assigns = assignsMap.get(cacheId); - for (List cIds : orderMap.headMap(order).values()) { - for (Integer cId : cIds) - waitList.add(cctx.cacheContext(cId).name()); - } + if (assigns != null) + assignsCancelled |= assigns.cancelled(); - Callable r = cacheCtx.preloader().addAssignments(assignsMap.get(cacheId), + // Cancels previous rebalance future (in case it's not done yet). + // Sends previous rebalance stopped event (if necessary). + // Creates new rebalance future. + // Sends current rebalance started event (if necessary). + // Finishes cache sync future (on empty assignments). + Runnable cur = cacheCtx.preloader().addAssignments(assigns, forcePreload, - waitList, - cnt); + cnt, + r); - if (r != null) { - U.log(log, "Cache rebalancing scheduled: [cache=" + cacheCtx.name() + - ", waitList=" + waitList.toString() + "]"); + if (cur != null) { + rebList.add(U.maskName(cacheCtx.name())); - if (cacheId == CU.cacheId(GridCacheUtils.MARSH_CACHE_NAME)) - marshR = r; - else - orderedRs.add(r); + r = cur; } } } - if (asyncStartFut != null) - asyncStartFut.get(); // Wait for thread stop. + if (assignsCancelled) { // Pending exchange. + U.log(log, "Skipping rebalancing (obsolete exchange ID) " + + "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() + + ", node=" + exchFut.discoveryEvent().eventNode().id() + ']'); + } + else if (r != null) { + Collections.reverse(rebList); - rebalanceQ.addAll(orderedRs); + U.log(log, "Rebalancing scheduled [order=" + rebList + "]"); - if (marshR != null || !rebalanceQ.isEmpty()) { if (futQ.isEmpty()) { - U.log(log, "Rebalancing required " + + U.log(log, "Rebalancing started " + "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() + ", node=" + exchFut.discoveryEvent().eventNode().id() + ']'); - if (marshR != null) { - try { - marshR.call(); //Marshaller cache rebalancing launches in sync way. - } - catch (Exception ex) { - if (log.isDebugEnabled()) - log.debug("Failed to send initial demand request to node"); - - continue; - } - } - - final GridFutureAdapter fut = new GridFutureAdapter(); - - asyncStartFut = fut; - - cctx.kernalContext().closure().callLocalSafe(new GPC() { - @Override public Boolean call() { - try { - while (true) { - Callable r = rebalanceQ.poll(); - - if (r == null) - return false; - - if (!r.call()) - return false; - } - } - catch (Exception ex) { - if (log.isDebugEnabled()) - log.debug("Failed to send initial demand request to node"); - - return false; - } - finally { - fut.onDone(); - } - } - }, /*system pool*/true); + r.run(); // Starts rebalancing routine. } - else { + else U.log(log, "Skipping rebalancing (obsolete exchange ID) " + "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() + ", node=" + exchFut.discoveryEvent().eventNode().id() + ']'); - } } - else { + else U.log(log, "Skipping rebalancing (nothing scheduled) " + "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() + ", node=" + exchFut.discoveryEvent().eventNode().id() + ']'); - } } } catch (IgniteInterruptedCheckedException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java index 1d1cfab04ce63..3c4456d5984d9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java @@ -19,7 +19,6 @@ import java.util.Collection; import java.util.UUID; -import java.util.concurrent.Callable; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -84,14 +83,14 @@ public interface GridCachePreloader { * * @param assignments Assignments to add. * @param forcePreload Force preload flag. - * @param caches Rebalancing of these caches will be finished before this started. * @param cnt Counter. - * @return Rebalancing closure. + * @param next Runnable responsible for cache rebalancing start. + * @return Rebalancing runnable. */ - public Callable addAssignments(GridDhtPreloaderAssignments assignments, + public Runnable addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload, - Collection caches, - int cnt); + int cnt, + Runnable next); /** * @param p Preload predicate. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java index b15ebc5029b12..656a960b56b9f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java @@ -19,7 +19,6 @@ import java.util.Collection; import java.util.UUID; -import java.util.concurrent.Callable; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.affinity.AffinityFunction; @@ -166,8 +165,8 @@ public GridCachePreloaderAdapter(GridCacheContext cctx) { } /** {@inheritDoc} */ - @Override public Callable addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload, - Collection caches, int cnt) { + @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload, + int cnt, Runnable next) { return null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index 57d522922e801..a6808c73577e1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java @@ -28,8 +28,8 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.Callable; import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReadWriteLock; @@ -72,6 +72,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_OBJECT_LOADED; import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_LOADED; +import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STARTED; import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STOPPED; import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE; import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING; @@ -121,6 +122,18 @@ public class GridDhtPartitionDemander { /** Cached rebalance topics. */ private final Map rebalanceTopics; + /** + * Started event sent. + * Make sense for replicated cache only. + */ + private final AtomicBoolean startedEvtSent = new AtomicBoolean(); + + /** + * Stopped event sent. + * Make sense for replicated cache only. + */ + private final AtomicBoolean stoppedEvtSent = new AtomicBoolean(); + /** * @param cctx Cctx. * @param demandLock Demand lock. @@ -249,45 +262,25 @@ private void preloadEvent(int part, int type, DiscoveryEvent discoEvt) { } /** - * @param name Cache name. - * @param fut Future. - * @throws IgniteCheckedException If failed. + * Sets last exchange future. + * + * @param lastFut Last future to set. */ - private boolean waitForCacheRebalancing(String name, RebalanceFuture fut) throws IgniteCheckedException { - if (log.isDebugEnabled()) - log.debug("Waiting for another cache to start rebalancing [cacheName=" + cctx.name() + - ", waitCache=" + name + ']'); - - RebalanceFuture wFut = (RebalanceFuture)cctx.kernalContext().cache().internalCache(name) - .preloader().rebalanceFuture(); - - if (!topologyChanged(fut) && wFut.updateSeq == fut.updateSeq) { - if (!wFut.get()) { - U.log(log, "Skipping waiting of " + name + " cache [top=" + fut.topologyVersion() + - "] (cache rebalanced with missed partitions)"); - - return false; - } - - return true; - } - else { - U.log(log, "Skipping waiting of " + name + " cache [top=" + fut.topologyVersion() + - "] (topology already changed)"); - - return false; - } + void onTopologyChanged(GridDhtPartitionsExchangeFuture lastFut) { + lastExchangeFut = lastFut; } /** * @param assigns Assignments. * @param force {@code True} if dummy reassign. - * @param caches Rebalancing of these caches will be finished before this started. * @param cnt Counter. - * @return Rebalancing closure. + * @param next Runnable responsible for cache rebalancing start. + * @return Rebalancing runnable. */ - Callable addAssignments(final GridDhtPreloaderAssignments assigns, boolean force, - final Collection caches, int cnt) { + Runnable addAssignments(final GridDhtPreloaderAssignments assigns, + boolean force, + int cnt, + final Runnable next) { if (log.isDebugEnabled()) log.debug("Adding partition assignments: " + assigns); @@ -296,7 +289,7 @@ Callable addAssignments(final GridDhtPreloaderAssignments assigns, bool if (delay == 0 || force) { final RebalanceFuture oldFut = rebalanceFut; - final RebalanceFuture fut = new RebalanceFuture(assigns, cctx, log, oldFut.isInitial(), cnt); + final RebalanceFuture fut = new RebalanceFuture(assigns, cctx, log, startedEvtSent, stoppedEvtSent, cnt); if (!oldFut.isInitial()) oldFut.cancel(); @@ -310,20 +303,69 @@ Callable addAssignments(final GridDhtPreloaderAssignments assigns, bool rebalanceFut = fut; - if (assigns.isEmpty()) { - fut.doneIfEmpty(assigns.cancelled()); + fut.sendRebalanceStartedEvent(); + + if (assigns.cancelled()) { // Pending exchange. + if (log.isDebugEnabled()) + log.debug("Rebalancing skipped due to cancelled assignments."); + + fut.onDone(false); + + fut.sendRebalanceFinishedEvent(); + + return null; + } + + if (assigns.isEmpty()) { // Nothing to rebalance. + if (log.isDebugEnabled()) + log.debug("Rebalancing skipped due to empty assignments."); + + fut.onDone(true); + + ((GridFutureAdapter)cctx.preloader().syncFuture()).onDone(); + + fut.sendRebalanceFinishedEvent(); return null; } - return new Callable() { - @Override public Boolean call() throws Exception { - for (String c : caches) { - if (!waitForCacheRebalancing(c, fut)) - return false; + return new Runnable() { + @Override public void run() { + try { + if (next != null) + fut.listen(new CI1>() { + @Override public void apply(IgniteInternalFuture f) { + try { + if (f.get()) // Not cancelled. + next.run(); // Starts next cache rebalancing (according to the order). + } + catch (IgniteCheckedException ignored) { + if (log.isDebugEnabled()) + log.debug(ignored.getMessage()); + } + } + }); + + requestPartitions(fut, assigns); } + catch (IgniteCheckedException e) { + ClusterTopologyCheckedException cause = e.getCause(ClusterTopologyCheckedException.class); - return requestPartitions(fut, assigns); + if (cause != null) + log.warning("Failed to send initial demand request to node. " + e.getMessage()); + else + log.error("Failed to send initial demand request to node.", e); + + fut.cancel(); + } + catch (Throwable th) { + log.error("Runtime error caught during initial demand request sending.", th); + + fut.cancel(); + + if (th instanceof Error) + throw th; + } } }; } @@ -361,14 +403,17 @@ else if (delay > 0) { * @throws IgniteCheckedException If failed. * @return Partitions were requested. */ - private boolean requestPartitions( + private void requestPartitions( RebalanceFuture fut, GridDhtPreloaderAssignments assigns ) throws IgniteCheckedException { - for (Map.Entry e : assigns.entrySet()) { - if (topologyChanged(fut)) - return false; + if (topologyChanged(fut)) { + fut.cancel(); + + return; + } + for (Map.Entry e : assigns.entrySet()) { final ClusterNode node = e.getKey(); GridDhtPartitionDemandMessage d = e.getValue(); @@ -387,7 +432,7 @@ private boolean requestPartitions( //Check remote node rebalancing API version. if (node.version().compareTo(GridDhtPreloader.REBALANCING_VER_2_SINCE) >= 0) { - U.log(log, "Starting rebalancing [cache=" + cctx.name() + ", mode=" + cfg.getRebalanceMode() + + U.log(log, "Starting rebalancing [mode=" + cfg.getRebalanceMode() + ", fromNode=" + node.id() + ", partitionsCount=" + parts.size() + ", topology=" + fut.topologyVersion() + ", updateSeq=" + fut.updateSeq + "]"); @@ -446,8 +491,6 @@ private boolean requestPartitions( worker.run(node, d); } } - - return true; } /** @@ -738,15 +781,6 @@ else if (log.isDebugEnabled()) return S.toString(GridDhtPartitionDemander.class, this); } - /** - * Sets last exchange future. - * - * @param lastFut Last future to set. - */ - void updateLastExchangeFuture(GridDhtPartitionsExchangeFuture lastFut) { - lastExchangeFut = lastFut; - } - /** * */ @@ -754,8 +788,11 @@ public static class RebalanceFuture extends GridFutureAdapter { /** */ private static final long serialVersionUID = 1L; - /** Should EVT_CACHE_REBALANCE_STOPPED event be sent of not. */ - private final boolean sndStoppedEvnt; + /** Should EVT_CACHE_REBALANCE_STARTED event be sent or not. */ + private final AtomicBoolean startedEvtSent; + + /** Should EVT_CACHE_REBALANCE_STOPPED event be sent or not. */ + private final AtomicBoolean stoppedEvtSent; /** */ private final GridCacheContext cctx; @@ -783,13 +820,15 @@ public static class RebalanceFuture extends GridFutureAdapter { * @param assigns Assigns. * @param cctx Context. * @param log Logger. - * @param sentStopEvnt Stop event flag. + * @param startedEvtSent Start event sent flag. + * @param stoppedEvtSent Stop event sent flag. * @param updateSeq Update sequence. */ RebalanceFuture(GridDhtPreloaderAssignments assigns, GridCacheContext cctx, IgniteLogger log, - boolean sentStopEvnt, + AtomicBoolean startedEvtSent, + AtomicBoolean stoppedEvtSent, long updateSeq) { assert assigns != null; @@ -797,7 +836,8 @@ public static class RebalanceFuture extends GridFutureAdapter { this.topVer = assigns.topologyVersion(); this.cctx = cctx; this.log = log; - this.sndStoppedEvnt = sentStopEvnt; + this.startedEvtSent = startedEvtSent; + this.stoppedEvtSent = stoppedEvtSent; this.updateSeq = updateSeq; } @@ -809,7 +849,8 @@ public RebalanceFuture() { this.topVer = null; this.cctx = null; this.log = null; - this.sndStoppedEvnt = false; + this.startedEvtSent = null; + this.stoppedEvtSent = null; this.updateSeq = -1; } @@ -847,24 +888,6 @@ private void appendPartitions(UUID nodeId, Collection parts) { } } - /** - * @param cancelled Is cancelled. - */ - private void doneIfEmpty(boolean cancelled) { - synchronized (this) { - if (isDone()) - return; - - assert remaining.isEmpty(); - - if (log.isDebugEnabled()) - log.debug("Rebalancing is not required [cache=" + cctx.name() + - ", topology=" + topVer + "]"); - - checkIsDone(cancelled, true); - } - } - /** * Cancels this future. * @@ -875,8 +898,7 @@ private void doneIfEmpty(boolean cancelled) { if (isDone()) return true; - U.log(log, "Cancelled rebalancing from all nodes [cache=" + cctx.name() - + ", topology=" + topologyVersion() + ']'); + U.log(log, "Cancelled rebalancing from all nodes [topology=" + topologyVersion() + ']'); if (!cctx.kernalContext().isStopping()) { for (UUID nodeId : remaining.keySet()) @@ -885,7 +907,7 @@ private void doneIfEmpty(boolean cancelled) { remaining.clear(); - checkIsDone(true /* cancelled */, false); + checkIsDone(true /* cancelled */); } return true; @@ -907,7 +929,7 @@ private void cancel(UUID nodeId) { remaining.remove(nodeId); - onDone(false); // Finishing rebalance future a non completed. + onDone(false); // Finishing rebalance future as non completed. checkIsDone(); // But will finish syncFuture only when other nodes are preloaded or rebalancing cancelled. } @@ -988,8 +1010,7 @@ private void partitionDone(UUID nodeId, int p) { if (parts.isEmpty()) { U.log(log, "Completed " + ((remaining.size() == 1 ? "(final) " : "") + - "rebalancing [cache=" + cctx.name() + - ", fromNode=" + nodeId + ", topology=" + topologyVersion() + + "rebalancing [fromNode=" + nodeId + ", topology=" + topologyVersion() + ", time=" + (U.currentTimeMillis() - t.get1()) + " ms]")); remaining.remove(nodeId); @@ -1022,23 +1043,20 @@ private void preloadEvent(int type, DiscoveryEvent discoEvt) { * */ private void checkIsDone() { - checkIsDone(false, false); + checkIsDone(false); } /** * @param cancelled Is cancelled. - * @param wasEmpty {@code True} if future was created without assignments. */ - private void checkIsDone(boolean cancelled, boolean wasEmpty) { + private void checkIsDone(boolean cancelled) { if (remaining.isEmpty()) { - if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_STOPPED) && (!cctx.isReplicated() || sndStoppedEvnt)) - preloadEvent(EVT_CACHE_REBALANCE_STOPPED, exchFut.discoveryEvent()); + sendRebalanceFinishedEvent(); if (log.isDebugEnabled()) log.debug("Completed rebalance future: " + this); - if (!wasEmpty) - cctx.shared().exchange().scheduleResendPartitions(); + cctx.shared().exchange().scheduleResendPartitions(); Collection m = new HashSet<>(); @@ -1064,6 +1082,30 @@ private void checkIsDone(boolean cancelled, boolean wasEmpty) { } } + /** + * + */ + private void sendRebalanceStartedEvent() { + if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_STARTED) && + (!cctx.isReplicated() || !startedEvtSent.get())) { + preloadEvent(EVT_CACHE_REBALANCE_STARTED, exchFut.discoveryEvent()); + + startedEvtSent.set(true); + } + } + + /** + * + */ + private void sendRebalanceFinishedEvent() { + if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_STOPPED) && + (!cctx.isReplicated() || !stoppedEvtSent.get())) { + preloadEvent(EVT_CACHE_REBALANCE_STOPPED, exchFut.discoveryEvent()); + + stoppedEvtSent.set(true); + } + } + /** {@inheritDoc} */ public String toString() { return S.toString(RebalanceFuture.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java index 0865d9f709d16..692e7c0de45a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java @@ -22,7 +22,6 @@ import java.util.Collections; import java.util.List; import java.util.UUID; -import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReadWriteLock; @@ -255,7 +254,7 @@ private IgniteCheckedException stopError() { @Override public void onTopologyChanged(GridDhtPartitionsExchangeFuture lastFut) { supplier.onTopologyChanged(lastFut.topologyVersion()); - demander.updateLastExchangeFuture(lastFut); + demander.onTopologyChanged(lastFut); } /** {@inheritDoc} */ @@ -413,9 +412,9 @@ public void handleDemandMessage(int idx, UUID id, GridDhtPartitionDemandMessage } /** {@inheritDoc} */ - @Override public Callable addAssignments(GridDhtPreloaderAssignments assignments, - boolean forcePreload, Collection caches, int cnt) { - return demander.addAssignments(assignments, forcePreload, caches, cnt); + @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments, + boolean forcePreload, int cnt, Runnable next) { + return demander.addAssignments(assignments, forcePreload, cnt, next); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java index de38952da7fd9..3dfcd85ea46b9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java @@ -501,6 +501,8 @@ protected void checkPartitionMapMessagesAbsent() throws Exception { record = true; + log.info("Checking GridDhtPartitions*Message absent (it will take 30 SECONDS) ... "); + U.sleep(30_000); record = false; From 0d4a1b7381fece47ee480f3a06bff7c51a7fead4 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Wed, 7 Dec 2016 18:02:49 +0700 Subject: [PATCH 386/487] Improved exception handling for failed queries. --- .../org/apache/ignite/internal/visor/query/VisorQueryJob.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java index 6d1de6adb1a13..c66b2dda8d22c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java @@ -197,7 +197,7 @@ private QueryCursor> near(IgniteCache(new VisorExceptionWrapper(e), null); } } From ac8602dbdf2bbf5b16a611eaf6d520a0a7b0010b Mon Sep 17 00:00:00 2001 From: Sergi Vladykin Date: Mon, 15 Aug 2016 16:46:54 +0300 Subject: [PATCH 387/487] ignite-3685 - fixed --- .../processors/query/h2/opt/GridH2Row.java | 2 +- .../IgniteCacheAbstractFieldsQuerySelfTest.java | 2 +- .../IgniteCacheLocalFieldsQuerySelfTest.java | 16 ++++++++++++++++ 3 files changed, 18 insertions(+), 2 deletions(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java index 9486a2ec6d432..8e7b1612c1ed3 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java @@ -98,7 +98,7 @@ public abstract class GridH2Row extends Row implements GridSearchRowPointer { /** {@inheritDoc} */ @Override public void setKey(long key) { - throw new UnsupportedOperationException(); + // No-op, may be set in H2 INFORMATION_SCHEMA. } /** {@inheritDoc} */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java index 926d294ec1d25..d5f02ebdf980d 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java @@ -71,7 +71,7 @@ public abstract class IgniteCacheAbstractFieldsQuerySelfTest extends GridCommonA private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); /** Cache name. */ - private static final String CACHE = "cache"; + protected static final String CACHE = "cache"; /** Empty cache name. */ private static final String EMPTY_CACHE = "emptyCache"; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalFieldsQuerySelfTest.java index be1f196ab5075..462118f1f8b9a 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalFieldsQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalFieldsQuerySelfTest.java @@ -18,6 +18,8 @@ package org.apache.ignite.internal.processors.cache.local; import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractFieldsQuerySelfTest; import static org.apache.ignite.cache.CacheMode.LOCAL; @@ -26,6 +28,10 @@ * Tests for fields queries. */ public class IgniteCacheLocalFieldsQuerySelfTest extends IgniteCacheAbstractFieldsQuerySelfTest { +// static { +// System.setProperty(IgniteSystemProperties.IGNITE_H2_DEBUG_CONSOLE, "1"); +// } + /** {@inheritDoc} */ @Override protected CacheMode cacheMode() { return LOCAL; @@ -35,4 +41,14 @@ public class IgniteCacheLocalFieldsQuerySelfTest extends IgniteCacheAbstractFiel @Override protected int gridCount() { return 1; } + + /** + * @throws Exception If failed. + */ + public void testInformationSchema() throws Exception { + IgniteEx ignite = grid(0); + + ignite.cache(CACHE).query( + new SqlFieldsQuery("SELECT VALUE FROM INFORMATION_SCHEMA.SETTINGS").setLocal(true)).getAll(); + } } \ No newline at end of file From bbaa79af8ef526b5d2684db0e3d71d60a8f1ebe7 Mon Sep 17 00:00:00 2001 From: agura Date: Wed, 7 Dec 2016 19:36:11 +0300 Subject: [PATCH 388/487] IGNITE-3770 GridLogThrottle.warn ignores the exception --- .../affinity/fair/FairAffinityFunction.java | 2 +- .../RendezvousAffinityFunction.java | 2 +- .../apache/ignite/internal/IgniteKernal.java | 2 +- .../GridDeploymentPerVersionStore.java | 2 +- .../discovery/GridDiscoveryManager.java | 2 +- .../eventstorage/GridEventStorageManager.java | 2 +- .../cache/GridCacheDeploymentManager.java | 4 +- .../cache/GridCacheEventManager.java | 10 ++--- .../store/GridCacheStoreManagerAdapter.java | 2 +- .../store/GridCacheWriteBehindStore.java | 2 +- .../clock/GridClockSyncProcessor.java | 2 +- .../igfs/IgfsFragmentizerManager.java | 29 +++++++------ .../internal/processors/igfs/IgfsImpl.java | 35 ++++++++-------- .../OsDiscoveryNodeValidationProcessor.java | 2 +- .../processors/task/GridTaskWorker.java | 2 +- .../ignite/internal/util/GridLogThrottle.java | 35 ++++++++-------- .../ignite/internal/util/IgniteUtils.java | 6 +-- .../shmem/IpcSharedMemoryNativeLoader.java | 2 +- .../shmem/IpcSharedMemoryServerEndpoint.java | 4 +- .../nio/GridConnectionBytesVerifyFilter.java | 2 +- .../internal/util/nio/GridNioCodecFilter.java | 2 +- .../internal/util/nio/GridNioFilterChain.java | 2 +- .../internal/util/nio/GridNioServer.java | 2 +- .../util/nio/GridSelectorNioSessionImpl.java | 2 +- .../tcp/TcpCommunicationSpi.java | 12 +++--- .../ignite/spi/discovery/tcp/ClientImpl.java | 7 ++-- .../ignite/spi/discovery/tcp/ServerImpl.java | 41 ++++++++----------- .../spi/discovery/tcp/TcpDiscoveryImpl.java | 2 +- .../spi/discovery/tcp/TcpDiscoverySpi.java | 8 ++-- .../TcpDiscoveryMulticastIpFinder.java | 4 +- .../ignite/testframework/GridTestUtils.java | 2 +- .../junits/common/GridCommonAbstractTest.java | 6 +-- .../ignite/util/GridLogThrottleTest.java | 27 ++++-------- .../HadoopExternalCommunication.java | 6 +-- .../processors/query/h2/IgniteH2Indexing.java | 2 +- .../IgniteCacheOffheapEvictQueryTest.java | 2 +- .../http/UriDeploymentHttpScanner.java | 8 ++-- 37 files changed, 133 insertions(+), 151 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java index 105efabe529fc..cffcf108c18df 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java @@ -331,7 +331,7 @@ public void setExcludeNeighbors(boolean exclNeighbors) { balance(tier, pendingParts, fullMap, topSnapshot, true); if (!exclNeighborsWarn) { - LT.warn(log, null, "Affinity function excludeNeighbors property is ignored " + + LT.warn(log, "Affinity function excludeNeighbors property is ignored " + "because topology has no enough nodes to assign backups."); exclNeighborsWarn = true; diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java index 75e7c92b4f24c..cbd0136576b4a 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java @@ -462,7 +462,7 @@ else if (affinityBackupFilter == null && backupFilter == null) } if (!exclNeighborsWarn) { - LT.warn(log, null, "Affinity function excludeNeighbors property is ignored " + + LT.warn(log, "Affinity function excludeNeighbors property is ignored " + "because topology has no enough nodes to assign backups."); exclNeighborsWarn = true; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index c5365069b809b..8fda72fc18419 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -1001,7 +1001,7 @@ else if (e instanceof IgniteCheckedException) // at least one waiting request, then it is possible starvation. if (exec.getPoolSize() == exec.getActiveCount() && completedCnt == lastCompletedCnt && !exec.getQueue().isEmpty()) - LT.warn(log, null, "Possible thread pool starvation detected (no task completed in last " + + LT.warn(log, "Possible thread pool starvation detected (no task completed in last " + interval + "ms, is executorService pool size large enough?)"); lastCompletedCnt = completedCnt; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentPerVersionStore.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentPerVersionStore.java index 5e30bf6457707..0bf8328472237 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentPerVersionStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentPerVersionStore.java @@ -317,7 +317,7 @@ else if (log.isDebugEnabled()) if (ctx.localNodeId().equals(e.getKey())) { // Warn only if mode is not CONTINUOUS. if (meta.deploymentMode() != CONTINUOUS) - LT.warn(log, null, "Local node is in participants (most probably, " + + LT.warn(log, "Local node is in participants (most probably, " + "IgniteConfiguration.getPeerClassLoadingLocalClassPathExclude() " + "is not used properly " + "[locNodeId=" + ctx.localNodeId() + ", meta=" + meta + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java index ddd4ee3e07b4c..9aa4db1e0042f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java @@ -995,7 +995,7 @@ private void checkSegmentOnStart() throws IgniteCheckedException { break; if (ctx.config().isWaitForSegmentOnStart()) { - LT.warn(log, null, "Failed to check network segment (retrying every 2000 ms)."); + LT.warn(log, "Failed to check network segment (retrying every 2000 ms)."); // Wait and check again. U.sleep(2000); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java index 5b451a17a0c3f..607bb9688aaf1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java @@ -278,7 +278,7 @@ public void record(Event evt) { int type = evt.type(); if (!isRecordable(type)) { - LT.warn(log, null, "Trying to record event without checking if it is recordable: " + + LT.warn(log, "Trying to record event without checking if it is recordable: " + U.gridEventName(type)); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java index 8e662333f4ec5..ad4892bcea09e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java @@ -399,7 +399,7 @@ public void p2pContext(UUID sndId, IgniteUuid ldrId, String userVer, DeploymentM ", daemon=" + daemon + ']'); if (!daemon) { - LT.warn(log, null, "Ignoring deployment in PRIVATE or ISOLATED mode " + + LT.warn(log, "Ignoring deployment in PRIVATE or ISOLATED mode " + "[sndId=" + sndId + ", ldrId=" + ldrId + ", userVer=" + userVer + ", mode=" + mode + ", participants=" + participants + ", daemon=" + daemon + ']'); } @@ -408,7 +408,7 @@ public void p2pContext(UUID sndId, IgniteUuid ldrId, String userVer, DeploymentM } if (mode != cctx.gridConfig().getDeploymentMode()) { - LT.warn(log, null, "Local and remote deployment mode mismatch (please fix configuration and restart) " + + LT.warn(log, "Local and remote deployment mode mismatch (please fix configuration and restart) " + "[locDepMode=" + cctx.gridConfig().getDeploymentMode() + ", rmtDepMode=" + mode + ", rmtNodeId=" + sndId + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java index ec8b8cc453e7b..1c1873814e675 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java @@ -251,7 +251,7 @@ public void addEvent( assert key != null || type == EVT_CACHE_STARTED || type == EVT_CACHE_STOPPED; if (!cctx.events().isRecordable(type)) - LT.warn(log, null, "Added event without checking if event is recordable: " + U.gridEventName(type)); + LT.warn(log, "Added event without checking if event is recordable: " + U.gridEventName(type)); // Events are not fired for internal entry. if (key == null || !key.internal()) { @@ -261,7 +261,7 @@ public void addEvent( evtNode = findNodeInHistory(evtNodeId); if (evtNode == null) - LT.warn(log, null, "Failed to find event node in grid topology history " + + LT.warn(log, "Failed to find event node in grid topology history " + "(try to increase topology history size configuration property of configured " + "discovery SPI): " + evtNodeId); @@ -284,7 +284,7 @@ public void addEvent( log.debug("Failed to unmarshall cache object value for the event notification: " + e); if (!forceKeepBinary) - LT.warn(log, null, "Failed to unmarshall cache object value for the event notification " + + LT.warn(log, "Failed to unmarshall cache object value for the event notification " + "(all further notifications will keep binary object format)."); forceKeepBinary = true; @@ -351,7 +351,7 @@ public void addPreloadEvent(int part, int type, ClusterNode discoNode, int disco assert discoTs > 0; if (!cctx.events().isRecordable(type)) - LT.warn(log, null, "Added event without checking if event is recordable: " + U.gridEventName(type)); + LT.warn(log, "Added event without checking if event is recordable: " + U.gridEventName(type)); cctx.gridEvents().record(new CacheRebalancingEvent(cctx.name(), cctx.localNode(), "Cache rebalancing event.", type, part, discoNode, discoType, discoTs)); @@ -364,7 +364,7 @@ public void addPreloadEvent(int part, int type, ClusterNode discoNode, int disco */ public void addUnloadEvent(int part) { if (!cctx.events().isRecordable(EVT_CACHE_REBALANCE_PART_UNLOADED)) - LT.warn(log, null, "Added event without checking if event is recordable: " + + LT.warn(log, "Added event without checking if event is recordable: " + U.gridEventName(EVT_CACHE_REBALANCE_PART_UNLOADED)); cctx.gridEvents().record(new CacheRebalancingEvent(cctx.name(), cctx.localNode(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java index cd0c50f57a1f8..024375e75c477 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java @@ -547,7 +547,7 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx, return true; } - LT.warn(log, null, "Calling Cache.loadCache() method will have no effect, " + + LT.warn(log, "Calling Cache.loadCache() method will have no effect, " + "CacheConfiguration.getStore() is not defined for cache: " + cctx.namexx()); return false; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java index 468945b97ec6c..858d9a7498510 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java @@ -705,7 +705,7 @@ private boolean updateStore(StoreOperation operation, } } catch (Exception e) { - LT.warn(log, e, "Unable to update underlying store: " + store); + LT.error(log, e, "Unable to update underlying store: " + store); if (writeCache.sizex() > cacheCriticalSize || stopping.get()) { for (Map.Entry> entry : vals.entrySet()) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java index b5c89cf5834f2..07643164368ce 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java @@ -458,7 +458,7 @@ private void requestTime(UUID rmtNodeId) { srv.sendPacket(req, addr, port); } catch (IgniteCheckedException e) { - LT.warn(log, e, "Failed to send time request to remote node [rmtNodeId=" + rmtNodeId + + LT.error(log, e, "Failed to send time request to remote node [rmtNodeId=" + rmtNodeId + ", addr=" + addr + ", port=" + port + ']'); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java index d64c64ad1e02f..2e82f33024b57 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java @@ -17,6 +17,19 @@ package org.apache.ignite.internal.processors.igfs; +import java.util.Collection; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; @@ -41,20 +54,6 @@ import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.Nullable; -import java.util.Collection; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.LinkedBlockingDeque; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; - import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; @@ -383,7 +382,7 @@ protected FragmentizerCoordinator() { } catch (IgniteCheckedException | IgniteException e) { if (!X.hasCause(e, InterruptedException.class) && !X.hasCause(e, IgniteInterruptedCheckedException.class)) - LT.warn(log, e, "Failed to get fragmentizer file info (will retry)."); + LT.error(log, e, "Failed to get fragmentizer file info (will retry)."); else { if (log.isDebugEnabled()) log.debug("Got interrupted exception in fragmentizer coordinator (grid is stopping)."); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index 1c985c0d6ed5c..ab4ee8533a051 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -17,6 +17,21 @@ package org.apache.ignite.internal.processors.igfs; +import java.io.OutputStream; +import java.net.URI; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -81,22 +96,6 @@ import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; -import java.io.OutputStream; -import java.net.URI; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; - import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_DELETED; import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_DELETED; import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_READ; @@ -1308,7 +1307,7 @@ private IgfsOutputStream create0( secondarySpaceSize = secondaryFs.usedSpaceSize(); } catch (IgniteException e) { - LT.warn(log, e, "Failed to get secondary file system consumed space size."); + LT.error(log, e, "Failed to get secondary file system consumed space size."); secondarySpaceSize = -1; } @@ -1841,4 +1840,4 @@ public FormatRunnable(GridFutureAdapter fut) { } } } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/nodevalidation/OsDiscoveryNodeValidationProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/nodevalidation/OsDiscoveryNodeValidationProcessor.java index a7e06e9beb918..37e59bc20262a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/nodevalidation/OsDiscoveryNodeValidationProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/nodevalidation/OsDiscoveryNodeValidationProcessor.java @@ -58,7 +58,7 @@ public OsDiscoveryNodeValidationProcessor(GridKernalContext ctx) { ", rmtNodeAddrs=" + U.addressesAsString(node) + ", locNodeId=" + locNode.id() + ", rmtNodeId=" + node.id() + ']'; - LT.warn(log, null, errMsg); + LT.warn(log, errMsg); // Always output in debug. if (log.isDebugEnabled()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java index 0be69d126bb36..3478c70a86e11 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java @@ -626,7 +626,7 @@ private void processMappedJobs(Map jobs) thro res.setOccupied(true); if (resCache && jobRes.size() > ctx.discovery().size() && jobRes.size() % SPLIT_WARN_THRESHOLD == 0) - LT.warn(log, null, "Number of jobs in task is too large for task: " + ses.getTaskName() + + LT.warn(log, "Number of jobs in task is too large for task: " + ses.getTaskName() + ". Consider reducing number of jobs or disabling job result cache with " + "@ComputeTaskNoResultCache annotation."); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java index c8ba86502746b..7f30dd71e0f31 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java @@ -93,13 +93,12 @@ public static void error(@Nullable IgniteLogger log, @Nullable Throwable e, Stri * Logs warning if needed. * * @param log Logger. - * @param e Error (optional). * @param msg Message. */ - public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, String msg) { + public static void warn(@Nullable IgniteLogger log, String msg) { assert !F.isEmpty(msg); - log(log, e, msg, null, LogLevel.WARN, false, false); + log(log, null, msg, null, LogLevel.WARN, false, false); } /** @@ -122,28 +121,26 @@ public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, Strin * Logs warning if needed. * * @param log Logger. - * @param e Error (optional). * @param msg Message. - * @param quite Print warning anyway. + * @param quiet Print warning anyway. */ - public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, String msg, boolean quite) { + public static void warn(@Nullable IgniteLogger log, String msg, boolean quiet) { assert !F.isEmpty(msg); - log(log, e, msg, null, LogLevel.WARN, quite, false); + log(log, null, msg, null, LogLevel.WARN, quiet, false); } /** * Logs warning if needed. * * @param log Logger. - * @param e Error (optional). * @param longMsg Long message (or just message). - * @param shortMsg Short message for quite logging. + * @param shortMsg Short message for quiet logging. */ - public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, String longMsg, @Nullable String shortMsg) { + public static void warn(@Nullable IgniteLogger log, String longMsg, @Nullable String shortMsg) { assert !F.isEmpty(longMsg); - log(log, e, longMsg, shortMsg, LogLevel.WARN, false, false); + log(log, null, longMsg, shortMsg, LogLevel.WARN, false, false); } /** @@ -151,12 +148,12 @@ public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, Strin * * @param log Logger. * @param msg Message. - * @param quite Print info anyway. + * @param quiet Print info anyway. */ - public static void info(@Nullable IgniteLogger log, String msg, boolean quite) { + public static void info(@Nullable IgniteLogger log, String msg, boolean quiet) { assert !F.isEmpty(msg); - log(log, null, msg, null, LogLevel.INFO, quite, false); + log(log, null, msg, null, LogLevel.INFO, quiet, false); } /** @@ -166,6 +163,8 @@ public static void info(@Nullable IgniteLogger log, String msg, boolean quite) { * @param msg Message. */ public static void info(@Nullable IgniteLogger log, String msg) { + assert !F.isEmpty(msg); + info(log, msg, false); } @@ -182,13 +181,13 @@ public static void clear() { * @param log Logger. * @param e Error (optional). * @param longMsg Long message (or just message). - * @param shortMsg Short message for quite logging. + * @param shortMsg Short message for quiet logging. * @param level Level where messages should appear. * @param byMsg Errors group by message, not by tuple(error, msg). */ @SuppressWarnings({"RedundantTypeArguments"}) - private static void log(@Nullable IgniteLogger log, @Nullable Throwable e, String longMsg, @Nullable String shortMsg, - LogLevel level, boolean quiet, boolean byMsg) { + private static void log(@Nullable IgniteLogger log, @Nullable Throwable e, String longMsg, + @Nullable String shortMsg, LogLevel level, boolean quiet, boolean byMsg) { assert !F.isEmpty(longMsg); IgniteBiTuple, String> tup = @@ -283,4 +282,4 @@ private enum LogLevel { */ public abstract void doLog(IgniteLogger log, String longMsg, String shortMsg, Throwable e, boolean quiet); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 914b3ec32f3e9..3fa3f7b92eccb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -4105,7 +4105,7 @@ public static void warn(@Nullable IgniteLogger log, Object msg) { } /** - * Logs warning message in both verbose and quite modes. + * Logs warning message in both verbose and quiet modes. * * @param log Logger to use. * @param msg Message to log. @@ -4115,7 +4115,7 @@ public static void quietAndWarn(IgniteLogger log, Object msg) { } /** - * Logs warning message in both verbose and quite modes. + * Logs warning message in both verbose and quiet modes. * * @param log Logger to use. * @param shortMsg Short message. @@ -4285,7 +4285,7 @@ public static void quiet(boolean err, Object... objs) { } /** - * Prints out the message in quite and info modes. + * Prints out the message in quiet and info modes. * * @param log Logger. * @param msg Message to print. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java index 2771d28fc5421..02c4de58719e7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java @@ -150,7 +150,7 @@ private static void doLoad(IgniteLogger log) throws IgniteCheckedException { try { if (log != null) - LT.warn(log, null, "Failed to load 'igniteshmem' library from classpath. Will try to load it from IGNITE_HOME."); + LT.warn(log, "Failed to load 'igniteshmem' library from classpath. Will try to load it from IGNITE_HOME."); String igniteHome = X.resolveIgniteHome(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java index 94c3820a9004b..178e6081d9403 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java @@ -304,13 +304,13 @@ public void omitOutOfResourcesWarning(boolean omitOutOfResourcesWarn) { String msg = "Failed to process incoming connection (most probably, shared memory " + "rest endpoint has been configured by mistake)."; - LT.warn(log, null, msg); + LT.warn(log, msg); sendErrorResponse(out, e); } catch (IpcOutOfSystemResourcesException e) { if (!omitOutOfResourcesWarn) - LT.warn(log, null, OUT_OF_RESOURCES_MSG); + LT.warn(log, OUT_OF_RESOURCES_MSG); sendErrorResponse(out, e); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java index 13d7ca7cc2e96..213fd8dd040a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java @@ -115,7 +115,7 @@ else if (U.bytesEqual(magicBuf, 0, U.IGNITE_HEADER, 0, U.IGNITE_HEADER.length)) else { ses.close(); - LT.warn(log, null, "Unknown connection detected (is some other software connecting to this " + + LT.warn(log, "Unknown connection detected (is some other software connecting to this " + "Ignite port?) [rmtAddr=" + ses.remoteAddress() + ", locAddr=" + ses.localAddress() + ']'); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java index a2f543d5338a6..7083ccf492f17 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java @@ -110,7 +110,7 @@ public GridNioCodecFilter(GridNioParser parser, IgniteLogger log, boolean direct if (directMode) return; - LT.warn(log, null, "Parser returned null but there are still unread data in input buffer (bug in " + + LT.warn(log, "Parser returned null but there are still unread data in input buffer (bug in " + "parser code?) [parser=" + parser + ", ses=" + ses + ']'); input.position(input.limit()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java index 8a43e2938832f..a3a74e3c235a9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java @@ -158,7 +158,7 @@ public String toString() { head.onExceptionCaught(ses, e); } catch (Exception ex) { - LT.warn(log, ex, "Failed to forward GridNioException to filter chain [ses=" + ses + ", e=" + e + ']'); + LT.error(log, ex, "Failed to forward GridNioException to filter chain [ses=" + ses + ", e=" + e + ']'); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java index 24b8fad0c8d7f..c8e2e0be067a8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java @@ -768,7 +768,7 @@ else if (cnt == 0) filterChain.onMessageReceived(ses, readBuf); if (readBuf.remaining() > 0) { - LT.warn(log, null, "Read buffer contains data after filter chain processing (will discard " + + LT.warn(log, "Read buffer contains data after filter chain processing (will discard " + "remaining bytes) [ses=" + ses + ", remainingCnt=" + readBuf.remaining() + ']'); readBuf.clear(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java index 0ba6af2bd1c58..63c9845ca3fc3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java @@ -227,7 +227,7 @@ void resend(Collection> futs) { if (recovery != null) { if (!recovery.add(last)) { - LT.warn(log, null, "Unacknowledged messages queue size overflow, will attempt to reconnect " + + LT.warn(log, "Unacknowledged messages queue size overflow, will attempt to reconnect " + "[remoteAddr=" + remoteAddress() + ", queueLimit=" + recovery.queueLimit() + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 767490fb7e90f..1fe437cc710c0 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -331,7 +331,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter private final GridNioServerListener srvLsnr = new GridNioServerListenerAdapter() { @Override public void onSessionWriteTimeout(GridNioSession ses) { - LT.warn(log, null, "Communication SPI Session write timed out (consider increasing " + + LT.warn(log, "Communication SPI Session write timed out (consider increasing " + "'socketWriteTimeout' " + "configuration property) [remoteAddr=" + ses.remoteAddress() + ", writeTimeout=" + sockWriteTimeout + ']'); @@ -2146,9 +2146,9 @@ private GridCommunicationClient reserveClient(ClusterNode node) throws IgniteChe catch (IgniteCheckedException e) { if (e.hasCause(IpcOutOfSystemResourcesException.class)) // Has cause or is itself the IpcOutOfSystemResourcesException. - LT.warn(log, null, OUT_OF_RESOURCES_TCP_MSG); + LT.warn(log, OUT_OF_RESOURCES_TCP_MSG); else if (getSpiContext().node(node.id()) != null) - LT.warn(log, null, e.getMessage()); + LT.warn(log, e.getMessage()); else if (log.isDebugEnabled()) log.debug("Failed to establish shared memory connection with local node (node has left): " + node.id()); @@ -2510,11 +2510,11 @@ protected GridCommunicationClient createTcpClient(ClusterNode node) throws Ignit boolean failureDetThrReached = timeoutHelper.checkFailureTimeoutReached(e); if (failureDetThrReached) - LT.warn(log, null, "Connect timed out (consider increasing 'failureDetectionTimeout' " + + LT.warn(log, "Connect timed out (consider increasing 'failureDetectionTimeout' " + "configuration property) [addr=" + addr + ", failureDetectionTimeout=" + failureDetectionTimeout() + ']'); else if (X.hasCause(e, SocketTimeoutException.class)) - LT.warn(log, null, "Connect timed out (consider increasing 'connTimeout' " + + LT.warn(log, "Connect timed out (consider increasing 'connTimeout' " + "configuration property) [addr=" + addr + ", connTimeout=" + connTimeout + ']'); if (errs == null) @@ -2545,7 +2545,7 @@ else if (X.hasCause(e, SocketTimeoutException.class)) assert errs != null; if (X.hasCause(errs, ConnectException.class)) - LT.warn(log, null, "Failed to connect to a remote node " + + LT.warn(log, "Failed to connect to a remote node " + "(make sure that destination node is alive and " + "operating system firewall is disabled on local and remote hosts) " + "[addrs=" + addrs + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 7b8c029f03d38..8928f28d4fd27 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -64,7 +64,6 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteUuid; -import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.spi.IgniteSpiContext; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.IgniteSpiOperationTimeoutHelper; @@ -483,7 +482,7 @@ else if (state == DISCONNECTED) { if (timeout > 0 && (U.currentTimeMillis() - startTime) > timeout) return null; - LT.warn(log, null, "IP finder returned empty addresses list. " + + LT.warn(log, "IP finder returned empty addresses list. " + "Please check IP finder configuration" + (spi.ipFinder instanceof TcpDiscoveryMulticastIpFinder ? " and make sure multicast works on your network. " : ". ") + @@ -553,7 +552,7 @@ else if (addrs.isEmpty()) { if (timeout > 0 && (U.currentTimeMillis() - startTime) > timeout) return null; - LT.warn(log, null, "Failed to connect to any address from IP finder (will retry to join topology " + + LT.warn(log, "Failed to connect to any address from IP finder (will retry to join topology " + "every 2 secs): " + toOrderedList(addrs0), true); Thread.sleep(2000); @@ -917,7 +916,7 @@ public void setSocket(SocketStream sockStream, UUID rmtNodeId) { ClassNotFoundException clsNotFoundEx = X.cause(e, ClassNotFoundException.class); if (clsNotFoundEx != null) - LT.warn(log, null, "Failed to read message due to ClassNotFoundException " + + LT.warn(log, "Failed to read message due to ClassNotFoundException " + "(make sure same versions of all classes are available on all nodes) " + "[rmtNodeId=" + rmtNodeId + ", err=" + clsNotFoundEx.getMessage() + ']'); else diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index a660ec8bb09ae..204b685984be3 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -54,7 +54,6 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import javax.net.ssl.SSLException; @@ -529,7 +528,7 @@ else if (log.isInfoEnabled()) { boolean res = pingNode(node); if (!res && !node.isClient() && nodeAlive(nodeId)) { - LT.warn(log, null, "Failed to ping node (status check will be initiated): " + nodeId); + LT.warn(log, "Failed to ping node (status check will be initiated): " + nodeId); msgWorker.addMessage(new TcpDiscoveryStatusCheckMessage(locNode, node.id())); } @@ -910,7 +909,7 @@ else if (spiState == LOOPBACK_PROBLEM) { U.addressesAsString(msg.addresses(), msg.hostNames()) + ']'); } else - LT.warn(log, null, "Node has not been connected to topology and will repeat join process. " + + LT.warn(log, "Node has not been connected to topology and will repeat join process. " + "Check remote nodes logs for possible error messages. " + "Note that large topology may require significant time to start. " + "Increase 'TcpDiscoverySpi.networkTimeout' configuration property " + @@ -1059,7 +1058,7 @@ else if (!spi.ipFinder.isShared() && !ipFinderHasLocAddr) { } if (e != null && X.hasCause(e, ConnectException.class)) { - LT.warn(log, null, "Failed to connect to any address from IP finder " + + LT.warn(log, "Failed to connect to any address from IP finder " + "(make sure IP finder addresses are correct and firewalls are disabled on all host machines): " + toOrderedList(addrs), true); } @@ -3148,7 +3147,7 @@ else if (log.isDebugEnabled()) } } - LT.warn(log, null, "Local node has detected failed nodes and started cluster-wide procedure. " + + LT.warn(log, "Local node has detected failed nodes and started cluster-wide procedure. " + "To speed up failure detection please see 'Failure Detection' section under javadoc" + " for 'TcpDiscoverySpi'"); } @@ -3233,7 +3232,7 @@ private void processJoinRequestMessage(final TcpDiscoveryJoinRequestMessage msg) "[locNodeAddrs=" + U.addressesAsString(locNode) + ", rmtNodeAddrs=" + U.addressesAsString(node) + ']'; - LT.warn(log, null, errMsg); + LT.warn(log, errMsg); // Always output in debug. if (log.isDebugEnabled()) @@ -3286,7 +3285,7 @@ private void processJoinRequestMessage(final TcpDiscoveryJoinRequestMessage msg) } // Output warning. - LT.warn(log, null, "Ignoring join request from node (duplicate ID) [node=" + node + + LT.warn(log, "Ignoring join request from node (duplicate ID) [node=" + node + ", existingNode=" + existingNode + ']'); // Ignore join request. @@ -3341,8 +3340,7 @@ else if (log.isDebugEnabled()) if (subj == null) { // Node has not pass authentication. - LT.warn(log, null, - "Authentication failed [nodeId=" + node.id() + + LT.warn(log, "Authentication failed [nodeId=" + node.id() + ", addrs=" + U.addressesAsString(node) + ']', "Authentication failed [nodeId=" + U.id8(node.id()) + ", addrs=" + U.addressesAsString(node) + ']'); @@ -3371,8 +3369,7 @@ else if (log.isDebugEnabled()) else { if (!(subj instanceof Serializable)) { // Node has not pass authentication. - LT.warn(log, null, - "Authentication subject is not Serializable [nodeId=" + node.id() + + LT.warn(log, "Authentication subject is not Serializable [nodeId=" + node.id() + ", addrs=" + U.addressesAsString(node) + ']', "Authentication subject is not Serializable [nodeId=" + U.id8(node.id()) + ", addrs=" + @@ -3442,7 +3439,7 @@ else if (log.isDebugEnabled()) return; } - LT.warn(log, null, err.message()); + LT.warn(log, err.message()); // Always output in debug. if (log.isDebugEnabled()) @@ -3483,7 +3480,7 @@ else if (log.isDebugEnabled()) ", rmtNodeAddrs=" + U.addressesAsString(node) + ", locNodeId=" + locNode.id() + ", rmtNodeId=" + msg.creatorNodeId() + ']'; - LT.warn(log, null, errMsg); + LT.warn(log, errMsg); // Always output in debug. if (log.isDebugEnabled()) @@ -3771,7 +3768,7 @@ else if (sendMessageToRemotes(msg)) * @param sndMsg Message to send. */ private void nodeCheckError(TcpDiscoveryNode node, String errMsg, String sndMsg) { - LT.warn(log, null, errMsg); + LT.warn(log, errMsg); // Always output in debug. if (log.isDebugEnabled()) @@ -4056,8 +4053,7 @@ else if (!locNodeId.equals(node.id()) && ring.node(node.id()) != null) { if (!permissionsEqual(coordSubj.subject().permissions(), subj.subject().permissions())) { // Node has not pass authentication. - LT.warn(log, null, - "Authentication failed [nodeId=" + node.id() + + LT.warn(log, "Authentication failed [nodeId=" + node.id() + ", addrs=" + U.addressesAsString(node) + ']', "Authentication failed [nodeId=" + U.id8(node.id()) + ", addrs=" + U.addressesAsString(node) + ']'); @@ -4148,7 +4144,6 @@ else if (!locNodeId.equals(node.id()) && ring.node(node.id()) != null) { rmCrd.subject().permissions())) { // Node has not pass authentication. LT.warn(log, - null, "Failed to authenticate local node " + "(local authentication result is different from rest of topology) " + "[nodeId=" + node.id() + ", addrs=" + U.addressesAsString(node) + ']', @@ -5593,7 +5588,7 @@ private class SocketReader extends IgniteSpiThread { "[rmtAddr=" + sock.getRemoteSocketAddress() + ", locAddr=" + sock.getLocalSocketAddress() + ']'); - LT.warn(log, null, "Failed to read magic header (too few bytes received) [rmtAddr=" + + LT.warn(log, "Failed to read magic header (too few bytes received) [rmtAddr=" + sock.getRemoteSocketAddress() + ", locAddr=" + sock.getLocalSocketAddress() + ']'); return; @@ -5609,7 +5604,7 @@ private class SocketReader extends IgniteSpiThread { "[rmtAddr=" + sock.getRemoteSocketAddress() + ", locAddr=" + sock.getLocalSocketAddress() + ']'); - LT.warn(log, null, "Unknown connection detected (is some other software connecting to " + + LT.warn(log, "Unknown connection detected (is some other software connecting to " + "this Ignite port?" + (!spi.isSslEnabled() ? " missing SSL configuration on remote node?" : "" ) + ") [rmtAddr=" + sock.getInetAddress() + ']', true); @@ -5729,7 +5724,7 @@ else if (log.isDebugEnabled()) U.error(log, "Caught exception on handshake [err=" + e +", sock=" + sock + ']', e); if (X.hasCause(e, SSLException.class) && spi.isSslEnabled() && !spi.isNodeStopping0()) - LT.warn(log, null, "Failed to initialize connection " + + LT.warn(log, "Failed to initialize connection " + "(missing SSL configuration on remote node?) " + "[rmtAddr=" + sock.getInetAddress() + ']', true); else if ((X.hasCause(e, ObjectStreamException.class) || !sock.isClosed()) @@ -5758,12 +5753,12 @@ else if ((X.hasCause(e, ObjectStreamException.class) || !sock.isClosed()) onException("Caught exception on handshake [err=" + e +", sock=" + sock + ']', e); if (e.hasCause(SocketTimeoutException.class)) - LT.warn(log, null, "Socket operation timed out on handshake " + + LT.warn(log, "Socket operation timed out on handshake " + "(consider increasing 'networkTimeout' configuration property) " + "[netTimeout=" + spi.netTimeout + ']'); else if (e.hasCause(ClassNotFoundException.class)) - LT.warn(log, null, "Failed to read message due to ClassNotFoundException " + + LT.warn(log, "Failed to read message due to ClassNotFoundException " + "(make sure same versions of all classes are available on all nodes) " + "[rmtAddr=" + sock.getRemoteSocketAddress() + ", err=" + X.cause(e, ClassNotFoundException.class).getMessage() + ']'); @@ -5995,7 +5990,7 @@ else if (msg instanceof TcpDiscoveryLoopbackProblemMessage) { return; if (e.hasCause(ClassNotFoundException.class)) - LT.warn(log, null, "Failed to read message due to ClassNotFoundException " + + LT.warn(log, "Failed to read message due to ClassNotFoundException " + "(make sure same versions of all classes are available on all nodes) " + "[rmtNodeId=" + nodeId + ", err=" + X.cause(e, ClassNotFoundException.class).getMessage() + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java index 0816cbca120eb..f199c20e17edb 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java @@ -329,7 +329,7 @@ protected final void registerLocalNodeAddress() throws IgniteSpiException { */ protected boolean checkAckTimeout(long ackTimeout) { if (ackTimeout > spi.getMaxAckTimeout()) { - LT.warn(log, null, "Acknowledgement timeout is greater than maximum acknowledgement timeout " + + LT.warn(log, "Acknowledgement timeout is greater than maximum acknowledgement timeout " + "(consider increasing 'maxAckTimeout' configuration property) " + "[ackTimeout=" + ackTimeout + ", maxAckTimeout=" + spi.getMaxAckTimeout() + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java index a8704e7ab66f5..45933e112f78d 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java @@ -1471,7 +1471,7 @@ protected T readMessage(Socket sock, @Nullable InputStream in, long timeout) } catch (IOException | IgniteCheckedException e) { if (X.hasCause(e, SocketTimeoutException.class)) - LT.warn(log, null, "Timed out waiting for message to be read (most probably, the reason is " + + LT.warn(log, "Timed out waiting for message to be read (most probably, the reason is " + "in long GC pauses on remote node) [curTimeout=" + timeout + ']'); throw e; @@ -1511,7 +1511,7 @@ protected int readReceipt(Socket sock, long timeout) throws IOException { return res; } catch (SocketTimeoutException e) { - LT.warn(log, null, "Timed out waiting for message delivery receipt (most probably, the reason is " + + LT.warn(log, "Timed out waiting for message delivery receipt (most probably, the reason is " + "in long GC pauses on remote node; consider tuning GC and increasing 'ackTimeout' " + "configuration property). Will retry to send message with increased timeout. " + "Current timeout: " + timeout + '.'); @@ -1575,7 +1575,7 @@ protected Collection resolvedAddresses() throws IgniteSpiExce res.add(resolved); } catch (UnknownHostException ignored) { - LT.warn(log, null, "Failed to resolve address from IP finder (host is unknown): " + addr); + LT.warn(log, "Failed to resolve address from IP finder (host is unknown): " + addr); // Add address in any case. res.add(addr); @@ -2045,7 +2045,7 @@ boolean cancel() { // Close socket - timeout occurred. U.closeQuiet(sock); - LT.warn(log, null, "Socket write has timed out (consider increasing " + + LT.warn(log, "Socket write has timed out (consider increasing " + (failureDetectionTimeoutEnabled() ? "'IgniteConfiguration.failureDetectionTimeout' configuration property) [" + "failureDetectionTimeout=" + failureDetectionTimeout() + ']' : diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java index e96abe96550f6..8fe8a65719c16 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java @@ -598,7 +598,7 @@ private T2, Boolean> requestAddresses(InetAddress addrRes = new AddressResponse(data); } catch (IgniteCheckedException e) { - LT.warn(log, e, "Failed to deserialize multicast response."); + LT.error(log, e, "Failed to deserialize multicast response."); continue; } @@ -876,7 +876,7 @@ private MulticastSocket createSocket() throws IOException { } catch (IOException e) { if (!isInterrupted()) { - LT.warn(log, e, "Failed to send/receive address message (will try to reconnect)."); + LT.error(log, e, "Failed to send/receive address message (will try to reconnect)."); synchronized (this) { U.close(sock); diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java index b3ce46b9ed1d4..0ae6575acc7c1 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java @@ -1061,7 +1061,7 @@ public static void waitTopologyUpdate(@Nullable String cacheName, int bac Collection nodes = top.nodes(p, AffinityTopologyVersion.NONE); if (nodes.size() > backups + 1) { - LT.warn(log, null, "Partition map was not updated yet (will wait) [grid=" + g.name() + + LT.warn(log, "Partition map was not updated yet (will wait) [grid=" + g.name() + ", p=" + p + ", nodes=" + F.nodeIds(nodes) + ']'); wait = true; diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java index d5f6b1c3bea66..90fabd95a1b47 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java @@ -507,7 +507,7 @@ protected void awaitPartitionMapExchange(boolean waitEvicts, if (affNodes.size() != owners.size() || !affNodes.containsAll(owners) || (waitEvicts && loc != null && loc.state() != GridDhtPartitionState.OWNING)) { - LT.warn(log(), null, "Waiting for topology map update [" + + LT.warn(log(), "Waiting for topology map update [" + "grid=" + g.name() + ", cache=" + cfg.getName() + ", cacheId=" + dht.context().cacheId() + @@ -524,7 +524,7 @@ protected void awaitPartitionMapExchange(boolean waitEvicts, match = true; } else { - LT.warn(log(), null, "Waiting for topology map update [" + + LT.warn(log(), "Waiting for topology map update [" + "grid=" + g.name() + ", cache=" + cfg.getName() + ", cacheId=" + dht.context().cacheId() + @@ -590,7 +590,7 @@ protected void awaitPartitionMapExchange(boolean waitEvicts, } if (entry.getValue() != GridDhtPartitionState.OWNING) { - LT.warn(log(), null, + LT.warn(log(), "Waiting for correct partition state, should be OWNING [state=" + entry.getValue() + "]"); diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridLogThrottleTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridLogThrottleTest.java index d9540a803bef6..9eac0ccdcb30a 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridLogThrottleTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridLogThrottleTest.java @@ -53,26 +53,20 @@ public void testThrottle() throws Exception { // LOGGED. LT.error(log, new RuntimeException("Test exception 2."), "Test"); - // OMITTED. - LT.warn(log, new RuntimeException("Test exception 1."), "Test"); - - // OMITTED. - LT.warn(log, new RuntimeException("Test exception 2."), "Test1"); - - // OMITTED. - LT.warn(log, new RuntimeException("Test exception 2."), "Test3"); - // LOGGED. LT.error(log, null, "Test - without throwable."); // OMITTED. LT.error(log, null, "Test - without throwable."); + // OMITTED. + LT.warn(log, "Test - without throwable."); + // LOGGED. - LT.warn(log, null, "Test - without throwable1."); + LT.warn(log, "Test - without throwable1."); // OMITTED. - LT.warn(log, null, "Test - without throwable1."); + LT.warn(log, "Test - without throwable1."); Thread.sleep(LT.throttleTimeout()); @@ -90,14 +84,11 @@ public void testThrottle() throws Exception { // LOGGED. LT.error(log, new RuntimeException("Test exception 2."), "Test"); - // OMITTED. - LT.warn(log, new RuntimeException("Test exception 1."), "Test"); - - // OMITTED. - LT.warn(log, new RuntimeException("Test exception 2."), "Test1"); + // LOGGED. + LT.warn(log, "Test - without throwable."); // OMITTED. - LT.warn(log, new RuntimeException("Test exception 2."), "Test3"); + LT.warn(log, "Test - without throwable."); Thread.sleep(LT.throttleTimeout()); @@ -121,4 +112,4 @@ public void testThrottle() throws Exception { //OMMITED. LT.info(log(), "Test info message."); } -} \ No newline at end of file +} diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java index 6d903d8b821a2..bc047e7e2fb3d 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java @@ -856,7 +856,7 @@ private HadoopCommunicationClient reserveClient(HadoopProcessDescriptor desc) th catch (IgniteCheckedException e) { if (e.hasCause(IpcOutOfSystemResourcesException.class)) // Has cause or is itself the IpcOutOfSystemResourcesException. - LT.warn(log, null, OUT_OF_RESOURCES_TCP_MSG); + LT.warn(log, OUT_OF_RESOURCES_TCP_MSG); else if (log.isDebugEnabled()) log.debug("Failed to establish shared memory connection with local hadoop process: " + desc); @@ -1059,7 +1059,7 @@ protected HadoopCommunicationClient createTcpClient(HadoopProcessDescriptor desc ", err=" + e + ']'); if (X.hasCause(e, SocketTimeoutException.class)) - LT.warn(log, null, "Connect timed out (consider increasing 'connTimeout' " + + LT.warn(log, "Connect timed out (consider increasing 'connTimeout' " + "configuration property) [addr=" + addr + ", port=" + port + ']'); if (errs == null) @@ -1084,7 +1084,7 @@ protected HadoopCommunicationClient createTcpClient(HadoopProcessDescriptor desc assert errs != null; if (X.hasCause(errs, ConnectException.class)) - LT.warn(log, null, "Failed to connect to a remote Hadoop process (is process still running?). " + + LT.warn(log, "Failed to connect to a remote Hadoop process (is process still running?). " + "Make sure operating system firewall is disabled on local and remote host) " + "[addrs=" + addr + ", port=" + port + ']'); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java index c3f701a97108e..6da8758b5e728 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java @@ -996,7 +996,7 @@ private ResultSet executeSqlQueryWithTimer(String space, PreparedStatement stmt, String longMsg = "Query execution is too long [time=" + time + " ms, sql='" + sql + '\'' + ", plan=" + U.nl() + plan.getString(1) + U.nl() + ", parameters=" + params + "]"; - LT.warn(log, null, longMsg, msg); + LT.warn(log, longMsg, msg); } return rs; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java index c24fed4a4bc5e..bddef66091243 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java @@ -181,7 +181,7 @@ public void testEvictAndRemove() throws Exception { } } - LT.warn(log, null, e.getMessage()); + LT.warn(log, e.getMessage()); return; } diff --git a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/scanners/http/UriDeploymentHttpScanner.java b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/scanners/http/UriDeploymentHttpScanner.java index 48bfd7f4be7cf..bb7260d0ef989 100644 --- a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/scanners/http/UriDeploymentHttpScanner.java +++ b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/scanners/http/UriDeploymentHttpScanner.java @@ -343,11 +343,11 @@ private void processHttp(Collection files, UriDeploymentScannerContext s catch (IOException e) { if (!scanCtx.isCancelled()) { if (X.hasCause(e, ConnectException.class)) { - LT.warn(scanCtx.getLogger(), e, "Failed to connect to HTTP server " + + LT.error(scanCtx.getLogger(), e, "Failed to connect to HTTP server " + "(connection refused): " + U.hidePassword(url)); } else if (X.hasCause(e, UnknownHostException.class)) { - LT.warn(scanCtx.getLogger(), e, "Failed to connect to HTTP server " + + LT.error(scanCtx.getLogger(), e, "Failed to connect to HTTP server " + "(host is unknown): " + U.hidePassword(url)); } else @@ -404,11 +404,11 @@ private Set getUrls(URL url, UriDeploymentScannerContext scanCtx) { catch (IOException e) { if (!scanCtx.isCancelled()) { if (X.hasCause(e, ConnectException.class)) { - LT.warn(scanCtx.getLogger(), e, "Failed to connect to HTTP server (connection refused): " + + LT.error(scanCtx.getLogger(), e, "Failed to connect to HTTP server (connection refused): " + U.hidePassword(url.toString())); } else if (X.hasCause(e, UnknownHostException.class)) { - LT.warn(scanCtx.getLogger(), e, "Failed to connect to HTTP server (host is unknown): " + + LT.error(scanCtx.getLogger(), e, "Failed to connect to HTTP server (host is unknown): " + U.hidePassword(url.toString())); } else From 18598574bb2992aa193eed1d72ca333a1e21ad72 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 8 Dec 2016 12:36:07 +0300 Subject: [PATCH 389/487] GG-11746: Backport of IGNITE-4379: Fixed broken local SqlFieldsQuery. --- .../processors/query/h2/IgniteH2Indexing.java | 36 +++++++++---------- ...teCachePartitionedFieldsQuerySelfTest.java | 25 +++++++++++++ 2 files changed, 43 insertions(+), 18 deletions(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java index 5c2fab564e421..3ea238bf705b6 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java @@ -737,33 +737,33 @@ private void removeTable(TableDescriptor tbl) throws IgniteCheckedException { @Nullable final Collection params, final IndexingQueryFilter filters, final int timeout, final GridQueryCancel cancel) throws IgniteCheckedException { - setFilters(filters); + final Connection conn = connectionForThread(schema(spaceName)); - try { - final Connection conn = connectionForThread(schema(spaceName)); + final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true); - final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true); + List meta; - List meta; + try { + meta = meta(stmt.getMetaData()); + } + catch (SQLException e) { + throw new IgniteCheckedException("Cannot prepare query metadata", e); + } - try { - meta = meta(stmt.getMetaData()); - } - catch (SQLException e) { - throw new IgniteCheckedException("Cannot prepare query metadata", e); - } + return new GridQueryFieldsResultAdapter(meta, null) { + @Override public GridCloseableIterator> iterator() throws IgniteCheckedException { + setFilters(filters); - return new GridQueryFieldsResultAdapter(meta, null) { - @Override public GridCloseableIterator> iterator() throws IgniteCheckedException{ + try { ResultSet rs = executeSqlQueryWithTimer(spaceName, stmt, conn, qry, params, timeout, cancel); return new FieldsIterator(rs); } - }; - } - finally { - setFilters(null); - } + finally { + setFilters(null); + } + } + }; } /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedFieldsQuerySelfTest.java index 653947e391e03..af5845d708196 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedFieldsQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedFieldsQuerySelfTest.java @@ -17,7 +17,13 @@ package org.apache.ignite.internal.processors.cache.distributed.near; +import java.util.List; +import javax.cache.Cache; +import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CachePeekMode; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractFieldsQuerySelfTest; @@ -54,4 +60,23 @@ protected NearCacheConfiguration nearConfiguration() { return cc; } + + /** @throws Exception If failed. */ + public void testLocalQuery() throws Exception { + IgniteCache cache = grid(0).cache( null); + + awaitPartitionMapExchange(true, true, null); + + int expected = 0; + + for(Cache.Entry e: cache.localEntries(CachePeekMode.PRIMARY)){ + if(e.getValue() instanceof Integer) + expected++; + } + + QueryCursor> qry = cache + .query(new SqlFieldsQuery("select _key, _val from Integer").setLocal(true)); + + assertEquals(expected, qry.getAll().size()); + } } \ No newline at end of file From 671a77a2d81cac401765dddf25f30fba4e4ab17f Mon Sep 17 00:00:00 2001 From: sboikov Date: Thu, 8 Dec 2016 12:56:46 +0300 Subject: [PATCH 390/487] ignite-4154 Fixed node version check for compression feature usage --- .../GridCachePartitionExchangeManager.java | 23 ++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index f04a6ce2fa025..7f11dc45aacb7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -826,7 +826,7 @@ public GridDhtPartitionsFullMessage createPartitionsFullMessage(Collection= 0; + boolean compress = canUsePartitionMapCompression(targetNode); GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(exchangeId, clientOnlyExchange, @@ -1554,6 +1553,24 @@ private void dumpPendingObjects(@Nullable AffinityTopologyVersion exchTopVer) { return deque.poll(time, MILLISECONDS); } + /** + * @param node Target node. + * @return {@code True} if can use compression for partition map messages. + */ + @SuppressWarnings("SimplifiableIfStatement") + private boolean canUsePartitionMapCompression(ClusterNode node) { + IgniteProductVersion ver = node.version(); + + if (ver.compareToIgnoreTimestamp(GridDhtPartitionsAbstractMessage.PART_MAP_COMPRESS_SINCE) >= 0) { + if (ver.minor() == 7 && ver.maintenance() < 4) + return false; + + return true; + } + + return false; + } + /** * Exchange future thread. All exchanges happen only by one thread and next * exchange will not start until previous one completes. From 391f4be4c687a7f325aeec8b727c9c85ca003454 Mon Sep 17 00:00:00 2001 From: agura Date: Wed, 7 Dec 2016 20:11:50 +0300 Subject: [PATCH 391/487] ignite-2358 toString() method for cache store implementations --- .../cache/store/cassandra/CassandraCacheStore.java | 6 ++++++ .../cache/store/cassandra/datasource/DataSource.java | 9 +++++++++ .../apache/ignite/cache/store/CacheStoreAdapter.java | 6 ++++++ .../ignite/cache/store/jdbc/CacheJdbcPojoStore.java | 6 ++++++ .../processors/cache/CacheStoreBalancingWrapper.java | 6 ++++++ .../processors/cache/GridCacheLoaderWriterStore.java | 6 ++++++ .../platform/dotnet/PlatformDotNetCacheStore.java | 11 +++++++++++ 7 files changed, 50 insertions(+) diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java index f7e7917a3db37..e8da3a7c8e511 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java @@ -41,6 +41,7 @@ import org.apache.ignite.cache.store.cassandra.session.ExecutionAssistant; import org.apache.ignite.cache.store.cassandra.session.GenericBatchExecutionAssistant; import org.apache.ignite.cache.store.cassandra.session.LoadCacheCustomQueryWorker; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.logger.NullLogger; @@ -406,4 +407,9 @@ private void closeCassandraSession(CassandraSession ses) { if (ses != null && (storeSes == null || storeSes.transaction() == null)) U.closeQuiet(ses); } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(CassandraCacheStore.class, this); + } } diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java index 1ecb28f04d3f2..915eebde73f01 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java @@ -39,6 +39,8 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.store.cassandra.session.CassandraSession; import org.apache.ignite.cache.store.cassandra.session.CassandraSessionImpl; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.S; /** * Data source abstraction to specify configuration of the Cassandra session to be used. @@ -54,9 +56,11 @@ public class DataSource { private ConsistencyLevel writeConsistency; /** Username to use for authentication. */ + @GridToStringExclude private String user; /** Password to use for authentication. */ + @GridToStringExclude private String pwd; /** Port to use for Cassandra connection. */ @@ -547,4 +551,9 @@ private ConsistencyLevel parseConsistencyLevel(String level) { private synchronized void invalidate() { ses = null; } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(DataSource.class, this); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreAdapter.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreAdapter.java index eaf522590e403..f3436f6befd80 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreAdapter.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; import javax.cache.Cache; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lang.IgniteBiPredicate; @@ -93,4 +94,9 @@ public abstract class CacheStoreAdapter implements CacheStore { @Override public void sessionEnd(boolean commit) { // No-op. } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(CacheStoreAdapter.class, this); + } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java index b348020a4777b..209dd117b964a 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java @@ -35,6 +35,7 @@ import org.apache.ignite.cache.store.CacheStore; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.binary.BinaryObjectEx; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; @@ -335,6 +336,11 @@ protected Object buildBinaryObject(String typeName, JdbcTypeField[] fields, } } + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(CacheJdbcPojoStore.class, this); + } + /** * Description of type property. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStoreBalancingWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStoreBalancingWrapper.java index 8992326f19373..ce2330c879160 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStoreBalancingWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStoreBalancingWrapper.java @@ -31,6 +31,7 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteBiInClosure; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -227,6 +228,11 @@ public void loadAll(Collection keys, final IgniteBiInClosure delegate.sessionEnd(commit); } + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(CacheStoreBalancingWrapper.class, this); + } + /** * */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStore.java index c497ac0a429e0..03beaf0973314 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStore.java @@ -25,6 +25,7 @@ import javax.cache.integration.CacheLoader; import javax.cache.integration.CacheWriter; import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lifecycle.LifecycleAware; import org.jetbrains.annotations.Nullable; @@ -142,4 +143,9 @@ CacheWriter writer() { @Override public void sessionEnd(boolean commit) { // No-op. } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(GridCacheLoaderWriterStore.class, this); + } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java index 3563dd668cadd..7505d4820a20f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java @@ -31,7 +31,10 @@ import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; import org.apache.ignite.internal.util.lang.GridTuple; import org.apache.ignite.internal.util.lang.IgniteInClosureX; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lifecycle.LifecycleAware; @@ -98,12 +101,15 @@ public class PlatformDotNetCacheStore implements CacheStore, Platfor private Map props; /** Native factory. */ + @GridToStringInclude private final Object nativeFactory; /** Interop processor. */ + @GridToStringExclude protected PlatformContext platformCtx; /** Pointer to native store. */ + @GridToStringExclude protected long ptr; /** @@ -441,4 +447,9 @@ protected int doInvoke(IgniteInClosure task, IgniteInClosure< return res; } } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(PlatformDotNetCacheStore.class, this); + } } From bc977d3211906ef94e1f7d3f0f988efbed65034f Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 9 Dec 2016 16:11:31 +0700 Subject: [PATCH 392/487] IGNITE-4350 Reworked JdbcTypesDefaultTransformed logic. Improved improved error messages in CacheJdbcPojoStore. --- .../cache/store/jdbc/CacheJdbcPojoStore.java | 13 +- .../jdbc/JdbcTypesDefaultTransformer.java | 112 +++---- .../jdbc/JdbcTypesDefaultTransformerTest.java | 283 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite.java | 2 + 4 files changed, 349 insertions(+), 61 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/JdbcTypesDefaultTransformerTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java index 209dd117b964a..b9f6e8ac6ffe8 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java @@ -150,15 +150,15 @@ private Object extractBinaryParameter(String fieldName, Object obj) throws Cache */ private Object buildBuiltinObject(String typeName, JdbcTypeField[] fields, Map loadColIdxs, ResultSet rs) throws CacheLoaderException { - try { - JdbcTypeField field = fields[0]; + JdbcTypeField field = fields[0]; + try { Integer colIdx = columnIndex(loadColIdxs, field.getDatabaseFieldName()); return transformer.getColumnValue(rs, colIdx, field.getJavaFieldType()); } catch (SQLException e) { - throw new CacheLoaderException("Failed to read object of class: " + typeName, e); + throw new CacheLoaderException("Failed to read object: [cls=" + typeName + ", prop=" + field + "]", e); } } @@ -211,12 +211,13 @@ private Object buildPojoObject(@Nullable String cacheName, String typeName, } catch (Exception e) { throw new CacheLoaderException("Failed to set property in POJO class [type=" + typeName + - ", prop=" + fldJavaName + ", col=" + colIdx + ", dbName=" + dbName + "]", e); + ", colIdx=" + colIdx + ", prop=" + fld + + ", dbValCls=" + colVal.getClass().getName() + ", dbVal=" + colVal + "]", e); } } catch (SQLException e) { - throw new CacheLoaderException("Failed to read object property [type= " + typeName + - ", prop=" + fldJavaName + ", col=" + colIdx + ", dbName=" + dbName + "]", e); + throw new CacheLoaderException("Failed to read object property [type=" + typeName + + ", colIdx=" + colIdx + ", prop=" + fld + "]", e); } } diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcTypesDefaultTransformer.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcTypesDefaultTransformer.java index e91c2d3a5fce8..c32eaa227dfd0 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcTypesDefaultTransformer.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcTypesDefaultTransformer.java @@ -17,9 +17,12 @@ package org.apache.ignite.cache.store.jdbc; +import java.math.BigDecimal; import java.nio.ByteBuffer; +import java.sql.Date; import java.sql.ResultSet; import java.sql.SQLException; +import java.sql.Time; import java.sql.Timestamp; import java.util.UUID; @@ -35,56 +38,71 @@ public class JdbcTypesDefaultTransformer implements JdbcTypesTransformer { /** {@inheritDoc} */ @Override public Object getColumnValue(ResultSet rs, int colIdx, Class type) throws SQLException { - Object val = rs.getObject(colIdx); + if (type == String.class) + return rs.getString(colIdx); - if (val == null) - return null; + if (type == int.class || type == Integer.class) { + int res = rs.getInt(colIdx); - if (type == int.class) - return rs.getInt(colIdx); + return rs.wasNull() && type == Integer.class ? null : res; + } + + if (type == long.class || type == Long.class) { + long res = rs.getLong(colIdx); + + return rs.wasNull() && type == Long.class ? null : res; + } + + if (type == double.class || type == Double.class) { + double res = rs.getDouble(colIdx); + + return rs.wasNull() && type == Double.class ? null : res; + } + + if (type == Date.class || type == java.util.Date.class) + return rs.getDate(colIdx); + + if (type == Timestamp.class) + return rs.getTimestamp(colIdx); + + if (type == Time.class) + return rs.getTime(colIdx); - if (type == long.class) - return rs.getLong(colIdx); + if (type == boolean.class || type == Boolean.class) { + boolean res = rs.getBoolean(colIdx); - if (type == double.class) - return rs.getDouble(colIdx); + return rs.wasNull() && type == Boolean.class ? null : res; + } - if (type == boolean.class || type == Boolean.class) - return rs.getBoolean(colIdx); + if (type == byte.class || type == Byte.class) { + byte res = rs.getByte(colIdx); - if (type == byte.class) - return rs.getByte(colIdx); + return rs.wasNull() && type == Byte.class ? null : res; + } - if (type == short.class) - return rs.getShort(colIdx); + if (type == short.class || type == Short.class) { + short res = rs.getShort(colIdx); - if (type == float.class) - return rs.getFloat(colIdx); + return rs.wasNull() && type == Short.class ? null : res; + } - if (type == Integer.class || type == Long.class || type == Double.class || - type == Byte.class || type == Short.class || type == Float.class) { - Number num = (Number)val; + if (type == float.class || type == Float.class) { + float res = rs.getFloat(colIdx); - if (type == Integer.class) - return num.intValue(); - else if (type == Long.class) - return num.longValue(); - else if (type == Double.class) - return num.doubleValue(); - else if (type == Byte.class) - return num.byteValue(); - else if (type == Short.class) - return num.shortValue(); - else if (type == Float.class) - return num.floatValue(); + return rs.wasNull() && type == Float.class ? null : res; } + if (type == BigDecimal.class) + return rs.getBigDecimal(colIdx); + if (type == UUID.class) { - if (val instanceof UUID) - return val; + Object res = rs.getObject(colIdx); + + if (res instanceof UUID) + return res; - if (val instanceof byte[]) { - ByteBuffer bb = ByteBuffer.wrap((byte[])val); + if (res instanceof byte[]) { + ByteBuffer bb = ByteBuffer.wrap((byte[])res); long most = bb.getLong(); long least = bb.getLong(); @@ -92,26 +110,10 @@ else if (type == Float.class) return new UUID(most, least); } - if (val instanceof String) - return UUID.fromString((String)val); - } - - // Workaround for known issue with Oracle JDBC driver https://community.oracle.com/thread/2355464?tstart=0 - if (type == java.sql.Date.class && val instanceof java.util.Date) - return new java.sql.Date(((java.util.Date)val).getTime()); - - // Workaround for known issue with Oracle JDBC driver and timestamp. - // http://stackoverflow.com/questions/13269564/java-lang-classcastexception-oracle-sql-timestamp-cannot-be-cast-to-java-sql-ti - if (type == Timestamp.class && !(val instanceof Timestamp) && - val.getClass().getName().startsWith("oracle.sql.TIMESTAMP")) { - try { - return val.getClass().getMethod("timestampValue").invoke(val); - } - catch (Exception e) { - throw new SQLException("Failed to read data of oracle.sql.TIMESTAMP type.", e); - } + if (res instanceof String) + return UUID.fromString((String)res); } - return val; + return rs.getObject(colIdx); } } diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/JdbcTypesDefaultTransformerTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/JdbcTypesDefaultTransformerTest.java new file mode 100644 index 0000000000000..5e490f7368053 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/JdbcTypesDefaultTransformerTest.java @@ -0,0 +1,283 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.store.jdbc; + +import java.math.BigDecimal; +import java.sql.Connection; +import java.sql.Date; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.UUID; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Test for {@link JdbcTypesDefaultTransformer}. + */ +public class JdbcTypesDefaultTransformerTest extends GridCommonAbstractTest { + /** + * @throws Exception If failed. + */ + public void testTransformer() throws Exception { + // Connection to H2. + String jdbcUrl = "jdbc:h2:mem:JdbcTypesDefaultTransformerTest"; + String usr = "sa"; + String pwd = ""; + + // Connection to Oracle. + // -Duser.region=us -Duser.language=en +// Class.forName("oracle.jdbc.OracleDriver"); +// String jdbcUrl = "jdbc:oracle:thin:@localhost:1521:XE"; +// String usr = "test"; +// String pwd = "test"; + + // Connection to MS SQL. +// Class.forName("com.microsoft.sqlserver.jdbc.SQLServerDriver"); +// String jdbcUrl = "jdbc:sqlserver://localhost;databaseName=master"; +// String usr = "test"; +// String pwd = "test"; + + // Connection to DB2. +// Class.forName("com.ibm.db2.jcc.DB2Driver"); +// String jdbcUrl = "jdbc:db2://localhost:50000/sample"; +// String usr = "test"; +// String pwd = "test"; + + // Connection to Postgre SQL. +// Class.forName("org.postgresql.Driver"); +// String jdbcUrl = "jdbc:postgresql://localhost:5433/postgres"; +// String usr = "test"; +// String pwd = "test"; + + // Connection to My SQL. +// Class.forName("com.mysql.jdbc.Driver"); +// String jdbcUrl = "jdbc:mysql://localhost:3306/test"; +// String usr = "test"; +// String pwd = "test"; + + try (Connection conn = DriverManager.getConnection(jdbcUrl, usr, pwd)) { + Statement stmt = conn.createStatement(); + + try { + stmt.executeUpdate("DROP TABLE TEST_TRANSFORMER"); + } + catch (SQLException ignored) { + // No-op. + } + + // Create table in H2. + stmt.executeUpdate("CREATE TABLE TEST_TRANSFORMER(id INTEGER, " + + "c1 BOOLEAN, c2 INTEGER, c3 TINYINT, c4 SMALLINT, c5 BIGINT, c6 DECIMAL(20, 2), c7 DOUBLE PRECISION, c8 REAL, " + + "c9 TIME, c10 DATE, c11 TIMESTAMP, c12 VARCHAR(100), c13 UUID)"); + + // Create table in ORACLE. +// stmt.executeUpdate("CREATE TABLE TEST_TRANSFORMER(id INTEGER, " + +// "c1 NUMBER(1), c2 INTEGER, c3 NUMBER(3), c4 NUMBER(4), c5 NUMBER(20), c6 NUMBER(20, 2), c7 NUMBER(20, 2), c8 NUMBER(10, 2), " + +// "c9 TIMESTAMP, c10 DATE, c11 TIMESTAMP, c12 VARCHAR(100), c13 VARCHAR(36))"); + + // Create table in MS SQL. +// stmt.executeUpdate("CREATE TABLE TEST_TRANSFORMER(id INTEGER, " + +// "c1 BIT, c2 INTEGER, c3 TINYINT, c4 SMALLINT, c5 BIGINT, c6 DECIMAL(20, 2), c7 DOUBLE PRECISION, c8 REAL, " + +// "c9 TIME, c10 DATE, c11 DATETIME, c12 VARCHAR(100), c13 VARCHAR(36))"); + + // Create table in DB2. +// stmt.executeUpdate("CREATE TABLE TEST_TRANSFORMER(id INTEGER, " + +// "c1 SMALLINT , c2 INTEGER, c3 SMALLINT , c4 SMALLINT, c5 BIGINT, c6 DECIMAL(20, 2), c7 DOUBLE PRECISION, c8 REAL, " + +// "c9 TIME, c10 DATE, c11 TIMESTAMP, c12 VARCHAR(100), c13 VARCHAR(36))"); + + // Create table in Postgre SQL. +// stmt.executeUpdate("CREATE TABLE TEST_TRANSFORMER(id INTEGER, " + +// "c1 BOOLEAN, c2 INTEGER, c3 SMALLINT, c4 SMALLINT, c5 BIGINT, c6 DECIMAL(20, 2), c7 DOUBLE PRECISION, c8 REAL, " + +// "c9 TIME, c10 DATE, c11 TIMESTAMP, c12 VARCHAR(100), c13 UUID)"); + + // Create table in MySQL. +// stmt.executeUpdate("CREATE TABLE TEST_TRANSFORMER(id INTEGER, " + +// "c1 BOOLEAN, c2 INTEGER, c3 TINYINT, c4 SMALLINT, c5 BIGINT, c6 DECIMAL(20, 2), c7 DOUBLE PRECISION, c8 REAL, " + +// "c9 TIME, c10 DATE, c11 TIMESTAMP(3), c12 VARCHAR(100), c13 VARCHAR(36))"); + + // Add data to H2, Postgre SQL and MySQL. + stmt.executeUpdate("INSERT INTO TEST_TRANSFORMER(id, c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13) " + + "VALUES (1, true, 1, 2, 3, 4, 5.35, 6.15, 7.32, '00:01:08', '2016-01-01', '2016-01-01 00:01:08.296', " + + "'100', '736bc956-090c-40d2-94da-916f2161f8a2')"); + stmt.executeUpdate("INSERT INTO TEST_TRANSFORMER(id, c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13) " + + "VALUES (2, false, 10, 20, 30, 40, 50, 60, 70, current_time, current_date, current_timestamp, " + + "'100.55', '736bc956-090c-40d2-94da-916f2161cdea')"); + + // Add data to Oracle. +// stmt.executeUpdate("INSERT INTO TEST_TRANSFORMER(id, c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13) " + +// "VALUES (1, 1, 1, 2, 3, 4, 5.35, 6.15, 7.32, " + +// "TO_TIMESTAMP('2016-01-01 00:01:08', 'YYYY-MM-DD HH24:MI:SS'), " + +// "TO_DATE('2016-01-01', 'YYYY-MM-DD')," + +// "TO_TIMESTAMP('2016-01-01 00:01:08.296', 'YYYY-MM-DD HH24:MI:SS.FF3'), " + +// "'100', '736bc956-090c-40d2-94da-916f2161f8a2')"); +// stmt.executeUpdate("INSERT INTO TEST_TRANSFORMER(id, c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13) " + +// "VALUES (2, 0, 10, 20, 30, 40, 50, 60, 70," + +// "TO_TIMESTAMP('2016-01-01 00:01:08', 'YYYY-MM-DD HH24:MI:SS'), " + +// "TO_DATE('2016-01-01', 'YYYY-MM-DD')," + +// "TO_TIMESTAMP('2016-01-01 00:01:08.296', 'YYYY-MM-DD HH24:MI:SS.FF3'), " + +// "'100.55', '736bc956-090c-40d2-94da-916f2161cdea')"); + + // Add data to MS SQL or IBM DB2. +// stmt.executeUpdate("INSERT INTO TEST_TRANSFORMER(id, c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13) " + +// "VALUES (1, 1, 1, 2, 3, 4, 5.35, 6.15, 7.32, '00:01:08', '2016-01-01', '2016-01-01 00:01:08.296', " + +// "'100', '736bc956-090c-40d2-94da-916f2161f8a2')"); +// stmt.executeUpdate("INSERT INTO TEST_TRANSFORMER(id, c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13) " + +// "VALUES (2, 0, 10, 20, 30, 40, 50, 60, 70, '00:01:08', '2016-01-01', '2016-01-01 00:01:08.296', " + +// "'100.55', '736bc956-090c-40d2-94da-916f2161cdea')"); + + stmt.executeUpdate("INSERT INTO TEST_TRANSFORMER(id, c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13) " + + "VALUES (3, null, null, null, null, null, null, null, null, null, null, null, null, null)"); + + ResultSet rs = stmt.executeQuery("select c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13 from TEST_TRANSFORMER order by id"); + + assertTrue(rs.next()); + + JdbcTypesDefaultTransformer transformer = JdbcTypesDefaultTransformer.INSTANCE; + + // c1: Test BOOLEAN column. + assertTrue((Boolean)transformer.getColumnValue(rs, 1, boolean.class)); + assertTrue((Boolean)transformer.getColumnValue(rs, 1, Boolean.class)); + assertEquals(1, transformer.getColumnValue(rs, 1, int.class)); + assertEquals(1.0, transformer.getColumnValue(rs, 1, double.class)); + + String s = (String)transformer.getColumnValue(rs, 1, String.class); + assertTrue("true".equalsIgnoreCase(s) || "1".equals(s) || "t".equalsIgnoreCase(s)); + + // c2: Test INTEGER column. + assertEquals(1, transformer.getColumnValue(rs, 2, int.class)); + assertEquals(1, transformer.getColumnValue(rs, 2, Integer.class)); + assertEquals(1L, transformer.getColumnValue(rs, 2, Long.class)); + assertEquals(1.0, transformer.getColumnValue(rs, 2, double.class)); + assertEquals("1", transformer.getColumnValue(rs, 2, String.class)); + + // c3: Test TINYINT column. + byte b = 2; + assertEquals(b, transformer.getColumnValue(rs, 3, byte.class)); + assertEquals(b, transformer.getColumnValue(rs, 3, Byte.class)); + assertEquals(2.0, transformer.getColumnValue(rs, 3, double.class)); + assertEquals("2", transformer.getColumnValue(rs, 3, String.class)); + + // c4: Test SMALLINT column. + short shrt = 3; + assertEquals(shrt, transformer.getColumnValue(rs, 4, short.class)); + assertEquals(shrt, transformer.getColumnValue(rs, 4, Short.class)); + assertEquals(3.0, transformer.getColumnValue(rs, 4, double.class)); + assertEquals("3", transformer.getColumnValue(rs, 4, String.class)); + + // c5: Test BIGINT column. + assertEquals(4L, transformer.getColumnValue(rs, 5, long.class)); + assertEquals(4L, transformer.getColumnValue(rs, 5, Long.class)); + assertEquals(4, transformer.getColumnValue(rs, 5, int.class)); + assertEquals(4, transformer.getColumnValue(rs, 5, Integer.class)); + assertEquals(4.0, transformer.getColumnValue(rs, 5, double.class)); + assertEquals("4", transformer.getColumnValue(rs, 5, String.class)); + assertEquals(new BigDecimal("4"), transformer.getColumnValue(rs, 5, BigDecimal.class)); + + // c6: Test DECIMAL column. + assertEquals(new BigDecimal("5.35"), transformer.getColumnValue(rs, 6, BigDecimal.class)); + assertEquals(5L, transformer.getColumnValue(rs, 6, long.class)); + assertEquals("5.35", transformer.getColumnValue(rs, 6, String.class)); + + // c7: Test DOUBLE column. + assertEquals(6.15, transformer.getColumnValue(rs, 7, double.class)); + assertEquals(6.15, transformer.getColumnValue(rs, 7, Double.class)); + assertEquals(6, transformer.getColumnValue(rs, 7, int.class)); + assertEquals(6, transformer.getColumnValue(rs, 7, Integer.class)); + assertTrue(transformer.getColumnValue(rs, 7, String.class).toString().startsWith("6.15")); + + // c8: Test REAL column. + assertTrue((7.32f - (Float)transformer.getColumnValue(rs, 8, float.class)) < 0.01); + assertTrue((7.32f - (Float)transformer.getColumnValue(rs, 8, Float.class)) < 0.01); + assertTrue((7.32 - (Double)transformer.getColumnValue(rs, 8, double.class)) < 0.01); + assertTrue((7.32 - (Double)transformer.getColumnValue(rs, 8, Double.class)) < 0.01); + assertEquals(7, transformer.getColumnValue(rs, 8, int.class)); + assertEquals(7, transformer.getColumnValue(rs, 8, Integer.class)); + assertTrue(transformer.getColumnValue(rs, 8, String.class).toString().startsWith("7.32")); + + // c9: Test TIME column. + assertTrue(transformer.getColumnValue(rs, 9, Time.class) instanceof Time); + assertTrue(transformer.getColumnValue(rs, 9, String.class).toString().contains("00:01:08")); + + // c10: Test DATE column. + assertTrue(transformer.getColumnValue(rs, 10, Date.class) instanceof Date); + assertTrue(transformer.getColumnValue(rs, 10, String.class).toString().startsWith("2016-01-01")); + + // c11: Test TIMESTAMP column. + transformer.getColumnValue(rs, 11, Timestamp.class); + assertTrue(transformer.getColumnValue(rs, 11, String.class).toString().startsWith("2016-01-01 00:01:08.29")); + + // c12: Test VARCHAR column. + assertEquals("100", transformer.getColumnValue(rs, 12, String.class)); + assertEquals(100, transformer.getColumnValue(rs, 12, int.class)); + + // c13: Test UUID column. + transformer.getColumnValue(rs, 13, UUID.class); + assertEquals("736bc956-090c-40d2-94da-916f2161f8a2", transformer.getColumnValue(rs, 13, String.class)); + + assertTrue(rs.next()); + + // Test BOOLEAN column. + assertFalse((Boolean)transformer.getColumnValue(rs, 1, boolean.class)); + assertFalse((Boolean)transformer.getColumnValue(rs, 1, Boolean.class)); + assertEquals(0, transformer.getColumnValue(rs, 1, int.class)); + assertEquals(0.0, transformer.getColumnValue(rs, 1, double.class)); + + s = (String)transformer.getColumnValue(rs, 1, String.class); + assertTrue("false".equalsIgnoreCase(s) || "0".equals(s) || "f".equalsIgnoreCase(s)); + + assertTrue(rs.next()); + + // Check how null values will be transformed. + assertNotNull(transformer.getColumnValue(rs, 1, boolean.class)); + assertNull(transformer.getColumnValue(rs, 1, Boolean.class)); + + assertEquals(0, transformer.getColumnValue(rs, 2, int.class)); + assertNull(transformer.getColumnValue(rs, 2, Integer.class)); + + assertEquals((byte)0, transformer.getColumnValue(rs, 3, byte.class)); + assertNull(transformer.getColumnValue(rs, 3, Byte.class)); + + assertEquals((short)0, transformer.getColumnValue(rs, 4, short.class)); + assertNull(transformer.getColumnValue(rs, 4, Short.class)); + + assertEquals(0L, transformer.getColumnValue(rs, 5, long.class)); + assertNull(transformer.getColumnValue(rs, 5, Long.class)); + + assertNull(transformer.getColumnValue(rs, 6, BigDecimal.class)); + + assertEquals(0d, transformer.getColumnValue(rs, 7, double.class)); + assertNull(transformer.getColumnValue(rs, 7, Double.class)); + + assertEquals(0f, transformer.getColumnValue(rs, 8, float.class)); + assertNull(transformer.getColumnValue(rs, 8, Float.class)); + + assertNull(transformer.getColumnValue(rs, 9, Time.class)); + + assertNull(transformer.getColumnValue(rs, 10, Date.class)); + + assertNull(transformer.getColumnValue(rs, 11, Timestamp.class)); + + assertNull(transformer.getColumnValue(rs, 12, String.class)); + + assertNull(transformer.getColumnValue(rs, 13, UUID.class)); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java index fd13e98d838f9..deec72af95cd8 100755 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java @@ -39,6 +39,7 @@ import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreTest; import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest; import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest; +import org.apache.ignite.cache.store.jdbc.JdbcTypesDefaultTransformerTest; import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest; import org.apache.ignite.internal.processors.cache.CacheDeferredDeleteSanitySelfTest; import org.apache.ignite.internal.processors.cache.CacheEntryProcessorCopySelfTest; @@ -246,6 +247,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(GridCacheConfigurationConsistencySelfTest.class); suite.addTestSuite(GridCacheJdbcBlobStoreSelfTest.class); suite.addTestSuite(GridCacheJdbcBlobStoreMultithreadedSelfTest.class); + suite.addTestSuite(JdbcTypesDefaultTransformerTest.class); suite.addTestSuite(CacheJdbcPojoStoreTest.class); suite.addTestSuite(CacheJdbcPojoStoreOptimizedMarshallerSelfTest.class); suite.addTestSuite(CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest.class); From b83ec8e57c7c48f2baa4780cf3b2e46df075f3df Mon Sep 17 00:00:00 2001 From: sboikov Date: Fri, 9 Dec 2016 14:32:42 +0300 Subject: [PATCH 393/487] IGNITE-500 CacheLoadingConcurrentGridStartSelfTest fails: prevent 'localUpdate' execution while top read lock is held. --- .../datastreamer/DataStreamProcessor.java | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java index 32fda87375cb4..fee4dd6616b28 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java @@ -328,6 +328,8 @@ private void localUpdate(final UUID nodeId, if (!allowOverwrite) cctx.topology().readLock(); + GridDhtTopologyFuture topWaitFut = null; + try { GridDhtTopologyFuture fut = cctx.topologyVersionFuture(); @@ -352,19 +354,25 @@ else if (allowOverwrite || fut.isDone()) { waitFut = allowOverwrite ? null : cctx.mvcc().addDataStreamerFuture(topVer); } - else { - fut.listen(new IgniteInClosure>() { - @Override public void apply(IgniteInternalFuture e) { - localUpdate(nodeId, req, updater, topic); - } - }); - } + else + topWaitFut = fut; } finally { if (!allowOverwrite) cctx.topology().readUnlock(); } + if (topWaitFut != null) { + // Need call 'listen' after topology read lock is released. + topWaitFut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture e) { + localUpdate(nodeId, req, updater, topic); + } + }); + + return; + } + if (job != null) { try { job.call(); From 6e485637e2738a7e809eac1a802f0964dc12383d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 9 Dec 2016 15:42:40 +0300 Subject: [PATCH 394/487] IGNITE-4379: Fixed local query execution. This closes #1323. --- .../processors/query/h2/IgniteH2Indexing.java | 54 +++++++++---------- ...gniteCacheAbstractFieldsQuerySelfTest.java | 19 +++++++ 2 files changed, 45 insertions(+), 28 deletions(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java index 6da8758b5e728..362ddd853443c 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java @@ -774,31 +774,38 @@ private void removeTable(TableDescriptor tbl) throws IgniteCheckedException { throws IgniteCheckedException { final Connection conn = connectionForSpace(spaceName); - initLocalQueryContext(conn, enforceJoinOrder, filters); + setupConnection(conn, false, enforceJoinOrder); + + final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true); + + List meta; try { - final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true); + meta = meta(stmt.getMetaData()); + } + catch (SQLException e) { + throw new IgniteCheckedException("Cannot prepare query metadata", e); + } - List meta; + final GridH2QueryContext ctx = new GridH2QueryContext(nodeId, nodeId, 0, LOCAL) + .filter(filters).distributedJoins(false); - try { - meta = meta(stmt.getMetaData()); - } - catch (SQLException e) { - throw new IgniteCheckedException("Cannot prepare query metadata", e); - } + return new GridQueryFieldsResultAdapter(meta, null) { + @Override public GridCloseableIterator> iterator() throws IgniteCheckedException { + assert GridH2QueryContext.get() == null; + + GridH2QueryContext.set(ctx); - return new GridQueryFieldsResultAdapter(meta, null) { - @Override public GridCloseableIterator> iterator() throws IgniteCheckedException{ + try { ResultSet rs = executeSqlQueryWithTimer(spaceName, stmt, conn, qry, params, timeout, cancel); return new FieldsIterator(rs); } - }; - } - finally { - GridH2QueryContext.clearThreadLocal(); - } + finally { + GridH2QueryContext.clearThreadLocal(); + } + } + }; } /** @@ -1024,17 +1031,6 @@ public void bindParameters(PreparedStatement stmt, @Nullable Collection } } - /** - * @param conn Connection. - * @param enforceJoinOrder Enforce join order of tables. - * @param filter Filter. - */ - private void initLocalQueryContext(Connection conn, boolean enforceJoinOrder, IndexingQueryFilter filter) { - setupConnection(conn, false, enforceJoinOrder); - - GridH2QueryContext.set(new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter).distributedJoins(false)); - } - /** * @param conn Connection to use. * @param distributedJoins If distributed joins are enabled. @@ -1061,7 +1057,9 @@ public void setupConnection(Connection conn, boolean distributedJoins, boolean e Connection conn = connectionForThread(tbl.schemaName()); - initLocalQueryContext(conn, false, filter); + setupConnection(conn, false, false); + + GridH2QueryContext.set(new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter).distributedJoins(false)); try { ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, sql, params, true, 0, null); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java index d5f02ebdf980d..2c355011114ed 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java @@ -28,10 +28,12 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.Callable; +import javax.cache.Cache; import javax.cache.CacheException; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.CacheRebalanceMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cache.affinity.AffinityKey; @@ -649,6 +651,23 @@ public void testQueryString() throws Exception { } } + /** @throws Exception If failed. */ + public void testLocalQuery() throws Exception { + IgniteCache cache = grid(0).cache( null); + + int expected = 0; + + for(Cache.Entry e: cache.localEntries(CachePeekMode.PRIMARY)){ + if(e.getValue() instanceof Integer) + expected++; + } + + QueryCursor> qry = cache + .query(new SqlFieldsQuery("select _key, _val from Integer").setLocal(true)); + + assertEquals(expected, qry.getAll().size()); + } + /** @throws Exception If failed. */ public void testQueryIntegersWithJoin() throws Exception { QueryCursor> qry = grid(0).cache(null).query(new SqlFieldsQuery( From 6fd8bf6338470275e687a686044c7d935d3714ff Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 7 Dec 2016 18:49:06 +0300 Subject: [PATCH 395/487] Fixed tests for IGNITE-4379 commit. --- ...gniteCacheAbstractFieldsQuerySelfTest.java | 19 -------------- ...teCachePartitionedFieldsQuerySelfTest.java | 25 +++++++++++++++++++ 2 files changed, 25 insertions(+), 19 deletions(-) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java index 2c355011114ed..d5f02ebdf980d 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java @@ -28,12 +28,10 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.Callable; -import javax.cache.Cache; import javax.cache.CacheException; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.CacheRebalanceMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cache.affinity.AffinityKey; @@ -651,23 +649,6 @@ public void testQueryString() throws Exception { } } - /** @throws Exception If failed. */ - public void testLocalQuery() throws Exception { - IgniteCache cache = grid(0).cache( null); - - int expected = 0; - - for(Cache.Entry e: cache.localEntries(CachePeekMode.PRIMARY)){ - if(e.getValue() instanceof Integer) - expected++; - } - - QueryCursor> qry = cache - .query(new SqlFieldsQuery("select _key, _val from Integer").setLocal(true)); - - assertEquals(expected, qry.getAll().size()); - } - /** @throws Exception If failed. */ public void testQueryIntegersWithJoin() throws Exception { QueryCursor> qry = grid(0).cache(null).query(new SqlFieldsQuery( diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedFieldsQuerySelfTest.java index 653947e391e03..af5845d708196 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedFieldsQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedFieldsQuerySelfTest.java @@ -17,7 +17,13 @@ package org.apache.ignite.internal.processors.cache.distributed.near; +import java.util.List; +import javax.cache.Cache; +import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CachePeekMode; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractFieldsQuerySelfTest; @@ -54,4 +60,23 @@ protected NearCacheConfiguration nearConfiguration() { return cc; } + + /** @throws Exception If failed. */ + public void testLocalQuery() throws Exception { + IgniteCache cache = grid(0).cache( null); + + awaitPartitionMapExchange(true, true, null); + + int expected = 0; + + for(Cache.Entry e: cache.localEntries(CachePeekMode.PRIMARY)){ + if(e.getValue() instanceof Integer) + expected++; + } + + QueryCursor> qry = cache + .query(new SqlFieldsQuery("select _key, _val from Integer").setLocal(true)); + + assertEquals(expected, qry.getAll().size()); + } } \ No newline at end of file From c143bc1a77baa13f61d6ba00509fa1fcb33757b1 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Fri, 9 Dec 2016 16:05:03 +0300 Subject: [PATCH 396/487] IGNITE-4063: Preserved order of fields in the metadata according with schema. This closes #1270. --- .../binary/BinaryClassDescriptor.java | 12 ++-- .../ignite/internal/binary/BinaryUtils.java | 10 ++- .../builder/BinaryObjectBuilderImpl.java | 11 +++- .../platform/PlatformContextImpl.java | 2 +- .../platform/utils/PlatformUtils.java | 28 ++++++++ .../binary/BinaryMarshallerSelfTest.java | 66 +++++++++++++++++++ 6 files changed, 119 insertions(+), 10 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java index b121337287d6d..5ec519abca641 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java @@ -28,7 +28,6 @@ import java.util.Collection; import java.util.Collections; import java.util.Date; -import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; import java.util.Map; @@ -275,15 +274,20 @@ else if (useOptMarshaller) case OBJECT: // Must not use constructor to honor transient fields semantics. ctor = null; - stableFieldsMeta = metaDataEnabled ? new HashMap() : null; Map fields0; - if (BinaryUtils.FIELDS_SORTED_ORDER) + if (BinaryUtils.FIELDS_SORTED_ORDER) { fields0 = new TreeMap<>(); - else + + stableFieldsMeta = metaDataEnabled ? new TreeMap() : null; + } + else { fields0 = new LinkedHashMap<>(); + stableFieldsMeta = metaDataEnabled ? new LinkedHashMap() : null; + } + Set duplicates = duplicateFields(cls); Collection names = new HashSet<>(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java index b3040826fead9..bbf50213fdd6d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java @@ -947,11 +947,17 @@ public static BinaryMetadata mergeMetadata(@Nullable BinaryMetadata oldMeta, Bin } // Check and merge fields. - boolean changed = false; + Map mergedFields; + + if (FIELDS_SORTED_ORDER) + mergedFields = new TreeMap<>(oldMeta.fieldsMap()); + else + mergedFields = new LinkedHashMap<>(oldMeta.fieldsMap()); - Map mergedFields = new HashMap<>(oldMeta.fieldsMap()); Map newFields = newMeta.fieldsMap(); + boolean changed = false; + for (Map.Entry newField : newFields.entrySet()) { Integer oldFieldType = mergedFields.put(newField.getKey(), newField.getValue()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java index 6ea9e7ec3a728..68a0ff3e35986 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java @@ -400,8 +400,12 @@ else if (newVal.getClass().isArray() && newVal.getClass().getComponentType() == if (oldFldTypeName == null) { // It's a new field, we have to add it to metadata. - if (fieldsMeta == null) - fieldsMeta = new HashMap<>(); + if (fieldsMeta == null) { + if (BinaryUtils.FIELDS_SORTED_ORDER) + fieldsMeta = new TreeMap<>(); + else + fieldsMeta = new LinkedHashMap<>(); + } fieldsMeta.put(name, newFldTypeId); } @@ -532,11 +536,12 @@ private void ensureReadCacheInit() { @Override public BinaryObjectBuilder setField(String name, Object val0) { Object val = val0 == null ? new BinaryValueWithType(BinaryUtils.typeByClass(Object.class), null) : val0; - if (assignedVals == null) + if (assignedVals == null) { if (BinaryUtils.FIELDS_SORTED_ORDER) assignedVals = new TreeMap<>(); else assignedVals = new LinkedHashMap<>(); + } Object oldVal = assignedVals.put(name, val); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java index e7fdb0a2dcc15..6cec7a1a204ef 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java @@ -361,7 +361,7 @@ public PlatformContextImpl(GridKernalContext ctx, PlatformCallbackGateway gate, String typeName = reader.readString(); String affKey = reader.readString(); - Map fields = PlatformUtils.readMap(reader, + Map fields = PlatformUtils.readLinkedMap(reader, new PlatformReaderBiClosure() { @Override public IgniteBiTuple read(BinaryRawReaderEx reader) { return F.t(reader.readString(), reader.readInt()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java index 0d30ad95582a9..959ff68a4e19f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java @@ -387,6 +387,34 @@ public static Map readMap(BinaryRawReaderEx reader, return map; } + /** + * Read linked map. + * + * @param reader Reader. + * @param readClo Reader closure. + * @return Map. + */ + public static Map readLinkedMap(BinaryRawReaderEx reader, + @Nullable PlatformReaderBiClosure readClo) { + int cnt = reader.readInt(); + + Map map = U.newLinkedHashMap(cnt); + + if (readClo == null) { + for (int i = 0; i < cnt; i++) + map.put((K)reader.readObjectDetached(), (V)reader.readObjectDetached()); + } + else { + for (int i = 0; i < cnt; i++) { + IgniteBiTuple entry = readClo.read(reader); + + map.put(entry.getKey(), entry.getValue()); + } + } + + return map; + } + /** * Read nullable map. * diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java index 39a4d32383f7a..6d07c9ba42770 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java @@ -53,6 +53,7 @@ import java.util.concurrent.ConcurrentSkipListSet; import junit.framework.Assert; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.binary.BinaryBasicIdMapper; import org.apache.ignite.binary.BinaryBasicNameMapper; import org.apache.ignite.binary.BinaryCollectionFactory; @@ -3108,6 +3109,71 @@ public void testBinaryEqualsComplexObject() throws Exception { assertNotEquals(binObj02, binObj11); } + + /** + * The test must be refactored after {@link IgniteSystemProperties#IGNITE_BINARY_SORT_OBJECT_FIELDS} + * is removed. + * + * @throws Exception If failed. + */ + public void testFieldOrder() throws Exception { + if (BinaryUtils.FIELDS_SORTED_ORDER) + return; + + BinaryMarshaller m = binaryMarshaller(); + + BinaryObjectImpl binObj = marshal(simpleObject(), m); + + Collection fieldsBin = binObj.type().fieldNames(); + + Field[] fields = SimpleObject.class.getDeclaredFields(); + + assertEquals(fields.length, fieldsBin.size()); + + int i = 0; + + for (String fieldName : fieldsBin) { + assertEquals(fields[i].getName(), fieldName); + + ++i; + } + } + + /** + * The test must be refactored after {@link IgniteSystemProperties#IGNITE_BINARY_SORT_OBJECT_FIELDS} + * is removed. + * + * @throws Exception If failed. + */ + public void testFieldOrderByBuilder() throws Exception { + if (BinaryUtils.FIELDS_SORTED_ORDER) + return; + + BinaryMarshaller m = binaryMarshaller(); + + BinaryObjectBuilder builder = new BinaryObjectBuilderImpl(binaryContext(m), "MyFakeClass"); + + String[] fieldNames = {"field9", "field8", "field0", "field1", "field2"}; + + for (String fieldName : fieldNames) + builder.setField(fieldName, 0); + + BinaryObject binObj = builder.build(); + + + Collection fieldsBin = binObj.type().fieldNames(); + + assertEquals(fieldNames.length, fieldsBin.size()); + + int i = 0; + + for (String fieldName : fieldsBin) { + assertEquals(fieldNames[i], fieldName); + + ++i; + } + } + /** * @param obj Instance of the BinaryObjectImpl to offheap marshalling. * @param marsh Binary marshaller. From f087b661c466c9fd27d049efa8e55a2d7bdf9834 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 9 Dec 2016 17:45:33 +0300 Subject: [PATCH 397/487] IGNITE-4264: Fixed cluster wide cache metrics. This closes #1336. --- .../processors/cache/GridCacheAdapter.java | 2 +- ...apCacheMetricsForClusterGroupSelfTest.java | 141 ++++++++++++++++++ ...alCacheOffHeapAndSwapMetricsSelfTest.java} | 2 +- .../IgniteCacheMetricsSelfTestSuite.java | 6 +- 4 files changed, 147 insertions(+), 4 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/OffheapCacheMetricsForClusterGroupSelfTest.java rename modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/{CacheOffHeapAndSwapMetricsSelfTest.java => LocalCacheOffHeapAndSwapMetricsSelfTest.java} (99%) 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 2e24e67cae195..b30ec709870d9 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 @@ -3234,7 +3234,7 @@ public IgniteInternalFuture removeAsync(final K key, @Nullable final Ca /** {@inheritDoc} */ @Override public CacheMetrics clusterMetrics() { - return clusterMetrics(ctx.grid().cluster().forCacheNodes(ctx.name())); + return clusterMetrics(ctx.grid().cluster().forDataNodes(ctx.name())); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/OffheapCacheMetricsForClusterGroupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/OffheapCacheMetricsForClusterGroupSelfTest.java new file mode 100644 index 0000000000000..53a20fcd83861 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/OffheapCacheMetricsForClusterGroupSelfTest.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.Event; +import org.apache.ignite.events.EventType; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import java.util.concurrent.CountDownLatch; + +import static org.apache.ignite.events.EventType.EVT_NODE_METRICS_UPDATED; + +/** + * Test for cluster wide offheap cache metrics. + */ +public class OffheapCacheMetricsForClusterGroupSelfTest extends GridCommonAbstractTest { + /** Grid count. */ + private static final int GRID_CNT = 3; + + /** Client count */ + private static final int CLIENT_CNT = 3; + + /** Grid client mode */ + private boolean clientMode; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setClientMode(clientMode); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + // start grids + clientMode = false; + for (int i = 0; i < GRID_CNT; i++) + startGrid("server-" + i); + + // start clients + clientMode = true; + for (int i = 0; i < CLIENT_CNT; i++) + startGrid("client-" + i); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + public void testGetOffHeapPrimaryEntriesCount() throws Exception { + String cacheName = "testGetOffHeapPrimaryEntriesCount"; + IgniteCache cache = grid("client-0").createCache(cacheConfiguration(cacheName)); + + for (int i = 0; i < 1000; i++) + cache.put(i, i); + + awaitMetricsUpdate(); + + assertGetOffHeapPrimaryEntriesCount(cacheName, 1000); + + for (int j = 0; j < 1000; j++) + cache.get(j); + + awaitMetricsUpdate(); + + assertGetOffHeapPrimaryEntriesCount(cacheName, 1000); + + cache = grid("client-1").cache(cacheName); + + for (int j = 0; j < 1000; j++) + cache.get(j); + + awaitMetricsUpdate(); + + assertGetOffHeapPrimaryEntriesCount(cacheName, 1000); + } + + /** + * Wait for {@link EventType#EVT_NODE_METRICS_UPDATED} event will be receieved. + */ + private void awaitMetricsUpdate() throws InterruptedException { + final CountDownLatch latch = new CountDownLatch((GRID_CNT + 1) * 2); + + IgnitePredicate lsnr = new IgnitePredicate() { + @Override public boolean apply(Event ignore) { + latch.countDown(); + + return true; + } + }; + + for (int i = 0; i < GRID_CNT; i++) + grid("server-" + i).events().localListen(lsnr, EVT_NODE_METRICS_UPDATED); + + latch.await(); + } + + private void assertGetOffHeapPrimaryEntriesCount(String cacheName, int count) throws Exception { + for (int i = 0; i < GRID_CNT; i++) { + IgniteCache cache = grid("server-" + i).cache(cacheName); + assertEquals(count, cache.metrics().getOffHeapPrimaryEntriesCount()); + } + + for (int i = 0; i < CLIENT_CNT; i++) { + IgniteCache cache = grid("client-" + i).cache(cacheName); + assertEquals(count, cache.metrics().getOffHeapPrimaryEntriesCount()); + } + } + + private static CacheConfiguration cacheConfiguration(String cacheName) { + CacheConfiguration cfg = new CacheConfiguration<>(cacheName); + + cfg.setBackups(1); + cfg.setStatisticsEnabled(true); + cfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED); + cfg.setOffHeapMaxMemory(1024 * 1024 * 1024); + return cfg; + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/CacheOffHeapAndSwapMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/LocalCacheOffHeapAndSwapMetricsSelfTest.java similarity index 99% rename from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/CacheOffHeapAndSwapMetricsSelfTest.java rename to modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/LocalCacheOffHeapAndSwapMetricsSelfTest.java index 6837a272b0d0d..ae8807f706be2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/CacheOffHeapAndSwapMetricsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/LocalCacheOffHeapAndSwapMetricsSelfTest.java @@ -30,7 +30,7 @@ /** * */ -public class CacheOffHeapAndSwapMetricsSelfTest extends GridCommonAbstractTest { +public class LocalCacheOffHeapAndSwapMetricsSelfTest extends GridCommonAbstractTest { /** Grid count. */ private static final int GRID_CNT = 1; diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java index 90ecbdaff4ded..f3dd1c83d5924 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java @@ -19,6 +19,7 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.processors.cache.CacheMetricsForClusterGroupSelfTest; +import org.apache.ignite.internal.processors.cache.OffheapCacheMetricsForClusterGroupSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPartitionedMetricsSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPartitionedTckMetricsSelfTestImpl; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearMetricsSelfTest; @@ -26,7 +27,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMetricsSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheAtomicReplicatedMetricsSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedMetricsSelfTest; -import org.apache.ignite.internal.processors.cache.local.CacheOffHeapAndSwapMetricsSelfTest; +import org.apache.ignite.internal.processors.cache.local.LocalCacheOffHeapAndSwapMetricsSelfTest; import org.apache.ignite.internal.processors.cache.local.GridCacheAtomicLocalMetricsNoStoreSelfTest; import org.apache.ignite.internal.processors.cache.local.GridCacheAtomicLocalMetricsSelfTest; import org.apache.ignite.internal.processors.cache.local.GridCacheAtomicLocalTckMetricsSelfTestImpl; @@ -48,7 +49,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheReplicatedMetricsSelfTest.class); suite.addTestSuite(GridCachePartitionedMetricsSelfTest.class); suite.addTestSuite(GridCachePartitionedHitsAndMissesSelfTest.class); - suite.addTestSuite(CacheOffHeapAndSwapMetricsSelfTest.class); + suite.addTestSuite(LocalCacheOffHeapAndSwapMetricsSelfTest.class); // Atomic cache. suite.addTestSuite(GridCacheAtomicLocalMetricsSelfTest.class); @@ -60,6 +61,7 @@ public static TestSuite suite() throws Exception { // Cluster wide metrics. suite.addTestSuite(CacheMetricsForClusterGroupSelfTest.class); + suite.addTestSuite(OffheapCacheMetricsForClusterGroupSelfTest.class); return suite; } From 6d744db27111c68e13b06ec99428a4c4148d97b6 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 12 Dec 2016 11:44:57 +0300 Subject: [PATCH 398/487] IGNITE-4231 - Hangs on compute result serialization error. Fix --- .../processors/job/GridJobWorker.java | 69 ++++++- .../closure/GridClosureSerializationTest.java | 177 ++++++++++++++++++ .../testsuites/IgniteBasicTestSuite.java | 2 + 3 files changed, 241 insertions(+), 7 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureSerializationTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java index 5f38b29025584..f5c6a2738e115 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.Map; import java.util.UUID; import java.util.concurrent.Callable; @@ -790,6 +791,64 @@ void finishJob(@Nullable Object res, } else { try { + byte[] resBytes = null; + byte[] exBytes = null; + byte[] attrBytes = null; + + boolean loc = ctx.localNodeId().equals(sndNode.id()) && !ctx.config().isMarshalLocalJobs(); + + Map attrs = jobCtx.getAttributes(); + + // Try serialize response, and if exception - return to client. + if (!loc) { + try { + resBytes = U.marshal(marsh, res); + } + catch (IgniteCheckedException e) { + resBytes = U.marshal(marsh, null); + + if (ex != null) + ex.addSuppressed(e); + else + ex = U.convertException(e); + + U.error(log, "Failed to serialize job response [nodeId=" + taskNode.id() + + ", ses=" + ses + ", jobId=" + ses.getJobId() + ", job=" + job + + ", resCls=" + (res == null ? null : res.getClass()) + ']', e); + } + + try { + attrBytes = U.marshal(marsh, attrs); + } + catch (IgniteCheckedException e) { + attrBytes = U.marshal(marsh, Collections.emptyMap()); + + if (ex != null) + ex.addSuppressed(e); + else + ex = U.convertException(e); + + U.error(log, "Failed to serialize job attributes [nodeId=" + taskNode.id() + + ", ses=" + ses + ", jobId=" + ses.getJobId() + ", job=" + job + + ", attrs=" + attrs + ']', e); + } + + try { + exBytes = U.marshal(marsh, ex); + } + catch (IgniteCheckedException e) { + String msg = "Failed to serialize job exception [nodeId=" + taskNode.id() + + ", ses=" + ses + ", jobId=" + ses.getJobId() + ", job=" + job + + ", msg=\"" + e.getMessage() + "\"]"; + + ex = new IgniteException(msg); + + U.error(log, msg, e); + + exBytes = U.marshal(marsh, ex); + } + } + if (ex != null) { if (isStarted) { // Job failed. @@ -804,19 +863,15 @@ else if (!internal && ctx.event().isRecordable(EVT_JOB_REJECTED)) else if (!internal && ctx.event().isRecordable(EVT_JOB_FINISHED)) evts = addEvent(evts, EVT_JOB_FINISHED, /*no message for success. */null); - boolean loc = ctx.localNodeId().equals(sndNode.id()) && !ctx.config().isMarshalLocalJobs(); - - Map attrs = jobCtx.getAttributes(); - GridJobExecuteResponse jobRes = new GridJobExecuteResponse( ctx.localNodeId(), ses.getId(), ses.getJobId(), - loc ? null : U.marshal(marsh, ex), + exBytes, loc ? ex : null, - loc ? null: U.marshal(marsh, res), + resBytes, loc ? res : null, - loc ? null : U.marshal(marsh, attrs), + attrBytes, loc ? attrs : null, isCancelled(), retry ? ctx.cache().context().exchange().readyAffinityVersion() : null); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureSerializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureSerializationTest.java new file mode 100644 index 0000000000000..2426dd7bdbc4e --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureSerializationTest.java @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.closure; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.concurrent.Callable; +import org.apache.ignite.IgniteException; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.compute.ComputeJobContext; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.resources.JobContextResource; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests handling of job result serialization error. + */ +public class GridClosureSerializationTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception { + final IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setMarshaller(null); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(0); + startGrid(1); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "Convert2Lambda"}) + public void testSerializationFailure() throws Exception { + final IgniteEx ignite0 = grid(0); + final IgniteEx ignite1 = grid(1); + + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + ignite1.compute(ignite1.cluster().forNode(ignite0.localNode())).call(new IgniteCallable() { + @Override public Object call() throws Exception { + return new CaseClass.CaseClass2(); + } + }); + + return null; + } + }, BinaryObjectException.class, null); + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "Convert2Lambda"}) + public void testExceptionSerializationFailure() throws Exception { + final IgniteEx ignite0 = grid(0); + final IgniteEx ignite1 = grid(1); + + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + ignite1.compute(ignite1.cluster().forNode(ignite0.localNode())).call(new IgniteCallable() { + @Override public Object call() throws Exception { + throw new BrokenException(); + } + }); + + return null; + } + }, IgniteException.class, null); + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "Convert2Lambda"}) + public void testAttributesSerializationFailure() throws Exception { + final IgniteEx ignite0 = grid(0); + final IgniteEx ignite1 = grid(1); + + GridTestUtils.assertThrows(null, new Callable() { + @JobContextResource + private ComputeJobContext jobCtx; + + @Override public Object call() throws Exception { + ignite1.compute(ignite1.cluster().forNode(ignite0.localNode())).call(new IgniteCallable() { + @Override public Object call() throws Exception { + jobCtx.setAttribute("test-attr", new BrokenAttribute()); + + return null; + } + }); + + return null; + } + }, IgniteException.class, null); + } + + /** + * Binary marshaller will fail because subclass defines other field with different case. + */ + @SuppressWarnings("unused") + private static class CaseClass { + /** */ + private String val; + + /** + * + */ + private static class CaseClass2 extends CaseClass { + /** */ + private String vAl; + } + } + + /** + * + */ + private static class BrokenAttribute implements Externalizable { + /** {@inheritDoc} */ + @Override public void writeExternal(final ObjectOutput out) throws IOException { + throw new IOException("Test exception"); + } + + /** {@inheritDoc} */ + @Override public void readExternal(final ObjectInput in) throws IOException, ClassNotFoundException { + throw new IOException("Test exception"); + } + } + + /** + * + */ + private static class BrokenException extends Exception implements Externalizable { + /** {@inheritDoc} */ + @Override public void writeExternal(final ObjectOutput out) throws IOException { + throw new IOException("Test exception"); + } + + /** {@inheritDoc} */ + @Override public void readExternal(final ObjectInput in) throws IOException, ClassNotFoundException { + throw new IOException("Test exception"); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java index 6ab0885976f10..1c1fcf7b494a9 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java @@ -41,6 +41,7 @@ import org.apache.ignite.internal.processors.cache.IgniteMarshallerCacheConcurrentReadWriteTest; import org.apache.ignite.internal.processors.cache.OffHeapTieredTransactionSelfTest; import org.apache.ignite.internal.processors.closure.GridClosureProcessorSelfTest; +import org.apache.ignite.internal.processors.closure.GridClosureSerializationTest; import org.apache.ignite.internal.processors.continuous.GridEventConsumeSelfTest; import org.apache.ignite.internal.processors.continuous.GridMessageListenSelfTest; import org.apache.ignite.internal.processors.odbc.OdbcProcessorValidationSelfTest; @@ -109,6 +110,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(GridProductVersionSelfTest.class); suite.addTestSuite(GridAffinityProcessorRendezvousSelfTest.class); suite.addTestSuite(GridClosureProcessorSelfTest.class); + suite.addTestSuite(GridClosureSerializationTest.class); suite.addTestSuite(ClosureServiceClientsNodesTest.class); suite.addTestSuite(GridStartStopSelfTest.class); suite.addTestSuite(GridProjectionForCachesSelfTest.class); From eeb2f2a38fb792b711eb665e380d14bc00f6e078 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 12 Dec 2016 12:14:01 +0300 Subject: [PATCH 399/487] IGNITE-3862 - GridServiceProxy invocation never times out. Fix --- .../org/apache/ignite/IgniteServices.java | 18 +- .../ignite/internal/IgniteServicesImpl.java | 11 +- .../affinity/GridAffinityProcessor.java | 2 +- .../processors/cache/GridCacheAdapter.java | 6 +- .../CacheDataStructuresManager.java | 6 +- .../cache/query/GridCacheQueryManager.java | 4 +- .../closure/GridClosureProcessor.java | 32 +- .../processors/job/GridJobWorker.java | 7 + .../platform/services/PlatformServices.java | 2 +- .../service/GridServiceProcessor.java | 11 +- .../processors/service/GridServiceProxy.java | 18 +- .../processors/task/GridTaskWorker.java | 7 + .../IgniteComputeTopologyExceptionTest.java | 5 +- ...iteServiceProxyTimeoutInitializedTest.java | 284 ++++++++++++++++++ .../testsuites/IgniteKernalSelfTestSuite.java | 2 + 15 files changed, 390 insertions(+), 25 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/service/IgniteServiceProxyTimeoutInitializedTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteServices.java b/modules/core/src/main/java/org/apache/ignite/IgniteServices.java index 08577c55118db..83fd487c88f1b 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteServices.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteServices.java @@ -350,6 +350,22 @@ public void deployKeyAffinitySingleton(String name, Service svc, @Nullable Strin */ public T serviceProxy(String name, Class svcItf, boolean sticky) throws IgniteException; + /** + * Gets a remote handle on the service with timeout. If service is available locally, + * then local instance is returned and timeout ignored, otherwise, a remote proxy is dynamically + * created and provided for the specified service. + * + * @param name Service name. + * @param svcItf Interface for the service. + * @param sticky Whether or not Ignite should always contact the same remote + * service or try to load-balance between services. + * @param timeout If greater than 0 created proxy will wait for service availability only specified time, + * and will limit remote service invocation time. + * @return Either proxy over remote service or local service if it is deployed locally. + * @throws IgniteException If failed to create service proxy. + */ + public T serviceProxy(String name, Class svcItf, boolean sticky, long timeout) throws IgniteException; + /** {@inheritDoc} */ @Override public IgniteServices withAsync(); -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java index b8042c349995f..400f28d1926a3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java @@ -222,14 +222,21 @@ public IgniteServicesImpl(GridKernalContext ctx, ClusterGroupAdapter prj, boolea /** {@inheritDoc} */ @Override public T serviceProxy(String name, Class svcItf, boolean sticky) throws IgniteException { + return (T) serviceProxy(name, svcItf, sticky, 0); + } + + /** {@inheritDoc} */ + @Override public T serviceProxy(final String name, final Class svcItf, final boolean sticky, + final long timeout) throws IgniteException { A.notNull(name, "name"); A.notNull(svcItf, "svcItf"); A.ensure(svcItf.isInterface(), "Service class must be an interface: " + svcItf); + A.ensure(timeout >= 0, "Timeout cannot be negative: " + timeout); guard(); try { - return (T)ctx.service().serviceProxy(prj, name, svcItf, sticky); + return (T)ctx.service().serviceProxy(prj, name, svcItf, sticky, timeout); } finally { unguard(); @@ -289,4 +296,4 @@ private void unguard() { protected Object readResolve() throws ObjectStreamException { return prj.services(); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java index b9182ae5c246e..b6efafbab394c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java @@ -497,7 +497,7 @@ private Map> keysToNodes(@Nullable final String c private AffinityInfo affinityInfoFromNode(@Nullable String cacheName, AffinityTopologyVersion topVer, ClusterNode n) throws IgniteCheckedException { GridTuple3 t = ctx.closure() - .callAsyncNoFailover(BROADCAST, affinityJob(cacheName, topVer), F.asList(n), true/*system pool*/).get(); + .callAsyncNoFailover(BROADCAST, affinityJob(cacheName, topVer), F.asList(n), true/*system pool*/, 0).get(); AffinityFunction f = (AffinityFunction)unmarshall(ctx, n.id(), t.get1()); AffinityKeyMapper m = (AffinityKeyMapper)unmarshall(ctx, n.id(), t.get2()); 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 b30ec709870d9..4d59d50ace4d3 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 @@ -3628,13 +3628,15 @@ private IgniteInternalFuture runLoadKeysCallable(final Set keys, return ctx.closures().callAsyncNoFailover(BROADCAST, new LoadKeysCallableV2<>(ctx.name(), keys, update, plc, keepBinary), nodes, - true); + true, + 0); } else { return ctx.closures().callAsyncNoFailover(BROADCAST, new LoadKeysCallable<>(ctx.name(), keys, update, plc), nodes, - true); + true, + 0); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java index c018f71b3e183..c1983df4f4d86 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java @@ -491,7 +491,8 @@ public void removeSetData(IgniteUuid id) throws IgniteCheckedException { cctx.closures().callAsyncNoFailover(BROADCAST, new BlockSetCallable(cctx.name(), id), nodes, - true).get(); + true, + 0).get(); } catch (IgniteCheckedException e) { if (e.hasCause(ClusterTopologyCheckedException.class)) { @@ -514,7 +515,8 @@ else if (!pingNodes(nodes)) { cctx.closures().callAsyncNoFailover(BROADCAST, new RemoveSetDataCallable(cctx.name(), id, topVer), nodes, - true).get(); + true, + 0).get(); } catch (IgniteCheckedException e) { if (e.hasCause(ClusterTopologyCheckedException.class)) { 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 d4decb47ae489..1165157784b5e 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 @@ -220,7 +220,7 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte if (detailMetricsSz > 0) detailMetrics = new ConcurrentHashMap8<>(detailMetricsSz); - + lsnr = new GridLocalEventListener() { @Override public void onEvent(Event evt) { UUID nodeId = ((DiscoveryEvent)evt).eventNode().id(); @@ -2237,7 +2237,7 @@ public Collection sqlMetadata() throws IgniteCheckedExcept // Get metadata from remote nodes. if (!nodes.isEmpty()) - rmtFut = cctx.closures().callAsyncNoFailover(BROADCAST, Collections.singleton(job), nodes, true); + rmtFut = cctx.closures().callAsyncNoFailover(BROADCAST, Collections.singleton(job), nodes, true, 0); // Get local metadata. IgniteInternalFuture> locFut = cctx.closures().callLocalSafe(job, true); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java index 9d295d3ced3b8..3ed985ef00972 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java @@ -81,6 +81,7 @@ import static org.apache.ignite.compute.ComputeJobResultPolicy.REDUCE; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_NO_FAILOVER; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SUBGRID; +import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_TIMEOUT; /** * @@ -520,9 +521,15 @@ public ComputeTaskInternalFuture affinityRun(@NotNull Collection cach * @param Type. * @return Grid future for collection of closure results. */ - public IgniteInternalFuture callAsyncNoFailover(GridClosureCallMode mode, @Nullable Callable job, - @Nullable Collection nodes, boolean sys) { + public IgniteInternalFuture callAsyncNoFailover( + GridClosureCallMode mode, + @Nullable Callable job, + @Nullable Collection nodes, + boolean sys, + long timeout + ) { assert mode != null; + assert timeout >= 0 : timeout; busyLock.readLock(); @@ -536,6 +543,9 @@ public IgniteInternalFuture callAsyncNoFailover(GridClosureCallMode mode, ctx.task().setThreadContext(TC_NO_FAILOVER, true); ctx.task().setThreadContext(TC_SUBGRID, nodes); + if (timeout > 0) + ctx.task().setThreadContext(TC_TIMEOUT, timeout); + return ctx.task().execute(new T7<>(mode, job), null, sys); } finally { @@ -548,13 +558,19 @@ public IgniteInternalFuture callAsyncNoFailover(GridClosureCallMode mode, * @param jobs Closures to execute. * @param nodes Grid nodes. * @param sys If {@code true}, then system pool will be used. + * @param timeout If greater than 0 limits task execution. Cannot be negative. * @param Type. * @return Grid future for collection of closure results. */ - public IgniteInternalFuture> callAsyncNoFailover(GridClosureCallMode mode, - @Nullable Collection> jobs, @Nullable Collection nodes, - boolean sys) { + public IgniteInternalFuture> callAsyncNoFailover( + GridClosureCallMode mode, + @Nullable Collection> jobs, + @Nullable Collection nodes, + boolean sys, + long timeout + ) { assert mode != null; + assert timeout >= 0 : timeout; busyLock.readLock(); @@ -568,6 +584,9 @@ public IgniteInternalFuture> callAsyncNoFailover(GridClosureCa ctx.task().setThreadContext(TC_NO_FAILOVER, true); ctx.task().setThreadContext(TC_SUBGRID, nodes); + if (timeout > 0) + ctx.task().setThreadContext(TC_TIMEOUT, timeout); + return ctx.task().execute(new T6<>(mode, jobs), null, sys); } finally { @@ -580,6 +599,7 @@ public IgniteInternalFuture> callAsyncNoFailover(GridClosureCa * @param job Closure to execute. * @param nodes Grid nodes. * @param sys If {@code true}, then system pool will be used. + * @param timeout If greater than 0 limits task execution. Cannot be negative. * @param Type. * @return Grid future for collection of closure results. */ @@ -2304,4 +2324,4 @@ private C4MLAV2(Runnable r) { return S.toString(C4MLAV2.class, this, super.toString()); } } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java index f5c6a2738e115..9bee8490fca4f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java @@ -43,14 +43,17 @@ import org.apache.ignite.internal.GridJobSessionImpl; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.deployment.GridDeployment; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.GridReservable; import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.processors.service.GridServiceNotFoundException; import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.GridWorker; @@ -587,6 +590,10 @@ else if (X.hasCause(e, InterruptedException.class)) { else U.warn(log, msg); } + else if (X.hasCause(e, GridServiceNotFoundException.class) || + X.hasCause(e, ClusterTopologyCheckedException.class)) + // Should be throttled, because GridServiceProxy continuously retry getting service. + LT.error(log, e, "Failed to execute job [jobId=" + ses.getJobId() + ", ses=" + ses + ']'); else U.error(log, "Failed to execute job [jobId=" + ses.getJobId() + ", ses=" + ses + ']', e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java index 962a4c0af00c3..c266986eb6e76 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java @@ -340,7 +340,7 @@ private ServiceDescriptor findDescriptor(String name) { Object proxy = PlatformService.class.isAssignableFrom(d.serviceClass()) ? services.serviceProxy(name, PlatformService.class, sticky) - : new GridServiceProxy<>(services.clusterGroup(), name, Service.class, sticky, + : new GridServiceProxy<>(services.clusterGroup(), name, Service.class, sticky, 0, platformCtx.kernalContext()); return new ServiceProxyHolder(proxy, d.serviceClass()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index b9b92b856b6c2..3690f357f1e5e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -694,9 +694,10 @@ public IgniteInternalFuture cancelAll() { /** * @param name Service name. + * @param timeout If greater than 0 limits task execution time. Cannot be negative. * @return Service topology. */ - public Map serviceTopology(String name) throws IgniteCheckedException { + public Map serviceTopology(String name, long timeout) throws IgniteCheckedException { ClusterNode node = cache.affinity().mapKeyToNode(name); if (node.version().compareTo(ServiceTopologyCallable.SINCE_VER) >= 0) { @@ -708,7 +709,8 @@ public Map serviceTopology(String name) throws IgniteCheckedExcep GridClosureCallMode.BROADCAST, call, Collections.singletonList(node), - false + false, + timeout ).get(); } else @@ -828,12 +830,13 @@ public ServiceContextImpl serviceContext(String name) { * @param name Service name. * @param svcItf Service class. * @param sticky Whether multi-node request should be done. + * @param timeout If greater than 0 limits service acquire time. Cannot be negative. * @param Service interface type. * @return The proxy of a service by its name and class. * @throws IgniteException If failed to create proxy. */ @SuppressWarnings("unchecked") - public T serviceProxy(ClusterGroup prj, String name, Class svcItf, boolean sticky) + public T serviceProxy(ClusterGroup prj, String name, Class svcItf, boolean sticky, long timeout) throws IgniteException { if (hasLocalNode(prj)) { ServiceContextImpl ctx = serviceContext(name); @@ -851,7 +854,7 @@ public T serviceProxy(ClusterGroup prj, String name, Class svcItf } } - return new GridServiceProxy(prj, name, svcItf, sticky, ctx).proxy(); + return new GridServiceProxy(prj, name, svcItf, sticky, timeout, ctx).proxy(); } /** 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 564a13ae7e6c7..aa609340b3bfb 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 @@ -84,11 +84,15 @@ public class GridServiceProxy implements Serializable { /** Whether multi-node request should be done. */ private final boolean sticky; + /** Service availability wait timeout. */ + private final long waitTimeout; + /** * @param prj Grid projection. * @param name Service name. * @param svc Service type class. * @param sticky Whether multi-node request should be done. + * @param timeout Service availability wait timeout. Cannot be negative. * @param ctx Context. */ @SuppressWarnings("unchecked") @@ -96,12 +100,16 @@ public GridServiceProxy(ClusterGroup prj, String name, Class svc, boolean sticky, + long timeout, GridKernalContext ctx) { + assert timeout >= 0 : timeout; + this.prj = prj; this.ctx = ctx; this.name = name; this.sticky = sticky; + this.waitTimeout = timeout; hasLocNode = hasLocalNode(prj); log = ctx.log(getClass()); @@ -145,6 +153,8 @@ else if (U.isToStringMethod(mtd)) ctx.gateway().readLock(); try { + final long startTime = U.currentTimeMillis(); + while (true) { ClusterNode node = null; @@ -171,7 +181,8 @@ else if (U.isToStringMethod(mtd)) GridClosureCallMode.BROADCAST, new ServiceProxyCallable(mtd.getName(), name, mtd.getParameterTypes(), args), Collections.singleton(node), - false + false, + waitTimeout ).get(); } } @@ -203,6 +214,9 @@ else if (U.isToStringMethod(mtd)) throw new IgniteException(e); } + + if (waitTimeout > 0 && U.currentTimeMillis() - startTime >= waitTimeout) + throw new IgniteException("Service acquire timeout was reached, stopping. [timeout=" + waitTimeout + "]"); } } finally { @@ -246,7 +260,7 @@ private ClusterNode randomNodeForService(String name) throws IgniteCheckedExcept if (hasLocNode && ctx.service().service(name) != null) return ctx.discovery().localNode(); - Map snapshot = ctx.service().serviceTopology(name); + Map snapshot = ctx.service().serviceTopology(name, waitTimeout); if (snapshot == null || snapshot.isEmpty()) return null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java index 3478c70a86e11..d89e80bc99825 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java @@ -68,6 +68,7 @@ import org.apache.ignite.internal.managers.deployment.GridDeployment; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.closure.AffinityTask; +import org.apache.ignite.internal.processors.service.GridServiceNotFoundException; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.typedef.CO; import org.apache.ignite.internal.util.typedef.F; @@ -1065,6 +1066,12 @@ else if (X.hasCause(e, ComputeJobFailoverException.class)) { return null; } + else if (X.hasCause(e, GridServiceNotFoundException.class) || + X.hasCause(e, ClusterTopologyCheckedException.class)) { + // Should be throttled, because GridServiceProxy continuously retry getting service. + LT.error(log, e, "Failed to obtain remote job result policy for result from " + + "ComputeTask.result(..) method (will fail the whole task): " + jobRes); + } else U.error(log, "Failed to obtain remote job result policy for result from " + "ComputeTask.result(..) method (will fail the whole task): " + jobRes, e); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java index c74dacacede51..3ed91e805d656 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java @@ -88,7 +88,8 @@ public void testCorrectCheckedException() throws Exception { } }, nodes, - false); + false, + 0); try { fut.get(); @@ -99,4 +100,4 @@ public void testCorrectCheckedException() throws Exception { log.info("Expected exception: " + e); } } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/IgniteServiceProxyTimeoutInitializedTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/IgniteServiceProxyTimeoutInitializedTest.java new file mode 100644 index 0000000000000..41eef310b2dac --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/IgniteServiceProxyTimeoutInitializedTest.java @@ -0,0 +1,284 @@ +/* + * 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.service; + +import org.apache.ignite.IgniteException; +import org.apache.ignite.binary.BinaryBasicIdMapper; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryReader; +import org.apache.ignite.binary.BinaryWriter; +import org.apache.ignite.binary.Binarylizable; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.compute.ComputeTaskTimeoutException; +import org.apache.ignite.configuration.BinaryConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.services.Service; +import org.apache.ignite.services.ServiceConfiguration; +import org.apache.ignite.services.ServiceContext; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +/** + * Tests service proxy timeouts. + */ +public class IgniteServiceProxyTimeoutInitializedTest extends GridCommonAbstractTest { + /** */ + private static Service srvc; + + /** */ + private static CountDownLatch latch1; + + /** */ + private static CountDownLatch latch2; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception { + final IgniteConfiguration cfg = super.getConfiguration(gridName); + + final ServiceConfiguration scfg = new ServiceConfiguration(); + + if (gridName.endsWith("0")) { + scfg.setName("testService"); + scfg.setService(srvc); + scfg.setMaxPerNodeCount(1); + scfg.setTotalCount(1); + scfg.setNodeFilter(new NodeFilter()); + + final Map attrs = new HashMap<>(); + + attrs.put("clusterGroup", "0"); + + cfg.setUserAttributes(attrs); + + cfg.setServiceConfiguration(scfg); + } + + cfg.setMarshaller(null); + + final BinaryConfiguration binCfg = new BinaryConfiguration(); + + // Despite defaults explicitly set to lower case. + binCfg.setIdMapper(new BinaryBasicIdMapper(true)); + + cfg.setBinaryConfiguration(binCfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + + /** + * Checks that we limit retries to get not available service by timeout. + * + * @throws Exception If fail. + */ + @SuppressWarnings({"Convert2Lambda", "ThrowableResultOfMethodCallIgnored"}) + public void testUnavailableService() throws Exception { + srvc = new TestWaitServiceImpl(); + + latch1 = new CountDownLatch(1); + latch2 = new CountDownLatch(1); + + try { + GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + startGrid(0); + + return null; + } + }); + + assert latch1.await(1, TimeUnit.MINUTES); + + final IgniteEx ignite1 = startGrid(1); + + final TestService testSrvc = ignite1.services().serviceProxy("testService", TestService.class, false, 500); + + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + testSrvc.test(); + + return null; + } + }, IgniteException.class, null); + } + finally { + latch2.countDown(); + } + } + + /** + * Checks that service not hangs if timeout set. Here we get hang with marshalling exception. + * + * @throws Exception If fail. + */ + @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "Convert2Lambda"}) + public void testServiceException() throws Exception { + srvc = new HangServiceImpl(); + + // Start service grid. + startGrid(0); + final IgniteEx ignite1 = startGrid(1); + + final HangService testSrvc = ignite1.services().serviceProxy("testService", HangService.class, false, 1_000); + + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + testSrvc.hang(); + + return null; + } + }, ComputeTaskTimeoutException.class, null); + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 60_000; + } + + /** + * + */ + private static class NodeFilter implements IgnitePredicate { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public boolean apply(final ClusterNode clusterNode) { + return "0".equals(clusterNode.attribute("clusterGroup")); + } + } + + /** + * + */ + private interface TestService { + /** */ + void test(); + } + + /** + * + */ + private static class TestWaitServiceImpl implements Service, TestService { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public void test() { + // No-op + } + + /** {@inheritDoc} */ + @Override public void cancel(final ServiceContext ctx) { + // No-op + } + + /** {@inheritDoc} */ + @Override public void init(final ServiceContext ctx) throws Exception { + latch1.countDown(); + + // Simulate long initialization. + latch2.await(1, TimeUnit.MINUTES); + } + + /** {@inheritDoc} */ + @Override public void execute(final ServiceContext ctx) throws Exception { + // No-op + } + } + + /** + * + */ + private static class HangClass implements Binarylizable { + + /** {@inheritDoc} */ + @Override public void writeBinary(final BinaryWriter writer) throws BinaryObjectException { + try { + U.sleep(10_000); + } + catch (IgniteInterruptedCheckedException e) { + throw new BinaryObjectException(e); + } + } + + /** {@inheritDoc} */ + @Override public void readBinary(final BinaryReader reader) throws BinaryObjectException { + try { + U.sleep(10_000); + } + catch (IgniteInterruptedCheckedException e) { + throw new BinaryObjectException(e); + } + } + } + + /** + * + */ + private interface HangService { + /** + * @return Hangs deserialization. + */ + HangClass hang(); + } + + /** + * + */ + private static class HangServiceImpl implements HangService, Service { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public HangClass hang() { + return new HangClass(); + } + + /** {@inheritDoc} */ + @Override public void cancel(final ServiceContext ctx) { + + } + + /** {@inheritDoc} */ + @Override public void init(final ServiceContext ctx) throws Exception { + + } + + /** {@inheritDoc} */ + @Override public void execute(final ServiceContext ctx) throws Exception { + + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java index d9cc8c0473522..350b7152c36d9 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java @@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.service.IgniteServiceDeploymentClassLoadingDefaultMarshallerTest; import org.apache.ignite.internal.processors.service.IgniteServiceDeploymentClassLoadingJdkMarshallerTest; import org.apache.ignite.internal.processors.service.IgniteServiceDeploymentClassLoadingOptimizedMarshallerTest; +import org.apache.ignite.internal.processors.service.IgniteServiceProxyTimeoutInitializedTest; import org.apache.ignite.internal.processors.service.IgniteServiceReassignmentTest; import org.apache.ignite.internal.processors.service.ServicePredicateAccessCacheTest; import org.apache.ignite.internal.util.GridStartupWithUndefinedIgniteHomeSelfTest; @@ -141,6 +142,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(GridServiceProxyNodeStopSelfTest.class); suite.addTestSuite(GridServiceProxyClientReconnectSelfTest.class); suite.addTestSuite(IgniteServiceReassignmentTest.class); + suite.addTestSuite(IgniteServiceProxyTimeoutInitializedTest.class); suite.addTestSuite(IgniteServiceDeploymentClassLoadingDefaultMarshallerTest.class); suite.addTestSuite(IgniteServiceDeploymentClassLoadingOptimizedMarshallerTest.class); From 8dd4ada6287af49a243aba1e58daffa7e44f39a9 Mon Sep 17 00:00:00 2001 From: sboikov Date: Mon, 12 Dec 2016 14:56:44 +0300 Subject: [PATCH 400/487] GridClosureProcessor: fixed javadoc. --- .../ignite/internal/processors/closure/GridClosureProcessor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java index 3ed985ef00972..a07dbf8e897c7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java @@ -599,7 +599,6 @@ public IgniteInternalFuture> callAsyncNoFailover( * @param job Closure to execute. * @param nodes Grid nodes. * @param sys If {@code true}, then system pool will be used. - * @param timeout If greater than 0 limits task execution. Cannot be negative. * @param Type. * @return Grid future for collection of closure results. */ From 83d961cff88cf2ead0bbc4ded3285f4faf9157fc Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 12 Dec 2016 17:52:22 +0300 Subject: [PATCH 401/487] IGNITE-4413 .NET: Fix DateTime argument handling in SqlQuery This closes #1341 # Conflicts: # modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs --- .../CacheQueriesCodeConfigurationTest.cs | 24 +++++++++++++-- .../Cache/Query/CacheQueriesTest.cs | 8 +++++ .../Cache/Query/QueryBase.cs | 15 ++++++++-- .../Impl/Cache/CacheImpl.cs | 29 +------------------ 4 files changed, 43 insertions(+), 33 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs index d5f98ac97e4ce..92e2891e76878 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs @@ -52,7 +52,8 @@ public void TestQueryEntityConfiguration() Fields = new[] { new QueryField("Name", typeof (string)), - new QueryField("Age", typeof (int)) + new QueryField("Age", typeof (int)), + new QueryField("Birthday", typeof(DateTime)), }, Indexes = new[] { @@ -71,7 +72,8 @@ public void TestQueryEntityConfiguration() cache[1] = new QueryPerson("Arnold", 10); cache[2] = new QueryPerson("John", 20); - using (var cursor = cache.Query(new SqlQuery(typeof (QueryPerson), "age > 10"))) + using (var cursor = cache.Query(new SqlQuery(typeof (QueryPerson), "age > ? and birthday < ?", + 10, DateTime.UtcNow))) { Assert.AreEqual(2, cursor.GetAll().Single().Key); } @@ -145,7 +147,9 @@ public void TestAttributeConfigurationQuery() cache[2] = new AttributeQueryPerson("John", 20); - using (var cursor = cache.Query(new SqlQuery(typeof(AttributeQueryPerson), "age > ?", 10))) + using (var cursor = cache.Query(new SqlQuery(typeof(AttributeQueryPerson), + "age > ? and age < ? and birthday > ? and birthday < ?", 10, 30, + DateTime.UtcNow.AddYears(-21), DateTime.UtcNow.AddYears(-19)))) { Assert.AreEqual(2, cursor.GetAll().Single().Key); } @@ -186,6 +190,8 @@ public AttributeQueryPerson(string name, int age) { Name = name; Age = age; + Salary = age; + Birthday = DateTime.UtcNow.AddYears(-age); } /// @@ -214,6 +220,18 @@ public AttributeQueryPerson(string name, int age) /// [QuerySqlField] public AttributeQueryAddress Address { get; set; } + + /// + /// Gets or sets the salary. + /// + [QuerySqlField] + public decimal? Salary { get; set; } + + /// + /// Gets or sets the birthday. + /// + [QuerySqlField] + public DateTime Birthday { get; set; } } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs index a8ffe13d5f9da..af79db99d4e9e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs @@ -15,6 +15,7 @@ * limitations under the License. */ +// ReSharper disable UnusedAutoPropertyAccessor.Global namespace Apache.Ignite.Core.Tests.Cache.Query { using System; @@ -758,6 +759,7 @@ public QueryPerson(string name, int age) { Name = name; Age = age; + Birthday = DateTime.UtcNow.AddYears(-age); } /// @@ -769,6 +771,12 @@ public QueryPerson(string name, int age) /// Age. /// public int Age { get; set; } + + /// + /// Gets or sets the birthday. + /// + [QuerySqlField] // Enforce Timestamp serialization + public DateTime Birthday { get; set; } } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Query/QueryBase.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Query/QueryBase.cs index cf1f637ec1389..d992845d304fa 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Query/QueryBase.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Query/QueryBase.cs @@ -17,6 +17,8 @@ namespace Apache.Ignite.Core.Cache.Query { + using System; + using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Cache; @@ -66,7 +68,7 @@ protected internal QueryBase() /// /// Writer. /// Arguments. - internal static void WriteQueryArgs(BinaryWriter writer, object[] args) + internal static void WriteQueryArgs(IBinaryRawWriter writer, object[] args) { if (args == null) writer.WriteInt(0); @@ -75,7 +77,16 @@ internal static void WriteQueryArgs(BinaryWriter writer, object[] args) writer.WriteInt(args.Length); foreach (var arg in args) - writer.WriteObject(arg); + { + // Write DateTime as TimeStamp always, otherwise it does not make sense + // Wrapped DateTime comparison does not work in SQL + var dt = arg as DateTime?; // Works with DateTime also + + if (dt != null) + writer.WriteTimestamp(dt); + else + writer.WriteObject(arg); + } } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs index 359611d960a90..57e70e18de0e2 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs @@ -991,7 +991,7 @@ public IQueryCursor QueryFields(SqlFieldsQuery qry, Func QueryFields(SqlFieldsQuery qry, Func(cursor, Marshaller, _flagKeepBinary); } - /// - /// Write query arguments. - /// - /// Writer. - /// Arguments. - private static void WriteQueryArgs(BinaryWriter writer, object[] args) - { - if (args == null) - writer.WriteInt(0); - else - { - writer.WriteInt(args.Length); - - foreach (var arg in args) - { - // Write DateTime as TimeStamp always, otherwise it does not make sense - // Wrapped DateTime comparison does not work in SQL - var dt = arg as DateTime?; // Works with DateTime also - - if (dt != null) - writer.WriteTimestamp(dt); - else - writer.WriteObject(arg); - } - } - } - /** */ public IContinuousQueryHandle QueryContinuous(ContinuousQuery qry) { From c103ac33d590492577bb687b29969550cdb532cf Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 21 Dec 2016 16:53:18 +0300 Subject: [PATCH 402/487] IGNITE-4248: Fixed: Some loggers ignore IGNITE_QUIET system property. This closes #1367. --- .../src/main/java/org/apache/ignite/IgniteLogger.java | 4 ++-- .../hadoop/impl/igfs/HadoopIgfsJclLogger.java | 9 ++++++++- .../java/org/apache/ignite/logger/jcl/JclLogger.java | 9 ++++++++- .../org/apache/ignite/logger/slf4j/Slf4jLogger.java | 11 +++++++++-- 4 files changed, 27 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java b/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java index f3afa995f0ca4..8d814fd78b39e 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java @@ -146,9 +146,9 @@ public interface IgniteLogger { public boolean isInfoEnabled(); /** - * Tests whether {@code info} and {@code debug} levels are turned off. + * Tests whether Logger is in "Quiet mode". * - * @return Whether {@code info} and {@code debug} levels are turned off. + * @return {@code true} "Quiet mode" is enabled, {@code false} otherwise */ public boolean isQuiet(); diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsJclLogger.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsJclLogger.java index 0ae8a9fe1a421..64752040fe446 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsJclLogger.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/igfs/HadoopIgfsJclLogger.java @@ -24,6 +24,8 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_QUIET; + /** * JCL logger wrapper for Hadoop. */ @@ -32,6 +34,9 @@ public class HadoopIgfsJclLogger implements IgniteLogger { @GridToStringInclude private Log impl; + /** Quiet flag. */ + private final boolean quiet; + /** * Constructor. * @@ -41,6 +46,8 @@ public class HadoopIgfsJclLogger implements IgniteLogger { assert impl != null; this.impl = impl; + + quiet = Boolean.valueOf(System.getProperty(IGNITE_QUIET, "true")); } /** {@inheritDoc} */ @@ -81,7 +88,7 @@ public class HadoopIgfsJclLogger implements IgniteLogger { /** {@inheritDoc} */ @Override public boolean isQuiet() { - return !isInfoEnabled() && !isDebugEnabled(); + return quiet; } /** {@inheritDoc} */ diff --git a/modules/jcl/src/main/java/org/apache/ignite/logger/jcl/JclLogger.java b/modules/jcl/src/main/java/org/apache/ignite/logger/jcl/JclLogger.java index a13caa104c559..c75197a02977f 100644 --- a/modules/jcl/src/main/java/org/apache/ignite/logger/jcl/JclLogger.java +++ b/modules/jcl/src/main/java/org/apache/ignite/logger/jcl/JclLogger.java @@ -22,6 +22,8 @@ import org.apache.ignite.IgniteLogger; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_QUIET; + /** * This logger wraps any JCL (Jakarta Commons Logging) * loggers. Implementation simply delegates to underlying JCL logger. This logger @@ -77,6 +79,9 @@ public class JclLogger implements IgniteLogger { /** JCL implementation proxy. */ private Log impl; + /** Quiet flag. */ + private final boolean quiet; + /** * Creates new logger. */ @@ -93,6 +98,8 @@ public JclLogger(Log impl) { assert impl != null; this.impl = impl; + + quiet = Boolean.valueOf(System.getProperty(IGNITE_QUIET, "true")); } /** {@inheritDoc} */ @@ -133,7 +140,7 @@ public JclLogger(Log impl) { /** {@inheritDoc} */ @Override public boolean isQuiet() { - return !isInfoEnabled() && !isDebugEnabled(); + return quiet; } /** {@inheritDoc} */ diff --git a/modules/slf4j/src/main/java/org/apache/ignite/logger/slf4j/Slf4jLogger.java b/modules/slf4j/src/main/java/org/apache/ignite/logger/slf4j/Slf4jLogger.java index 51e56693c10c5..2b0e98041db39 100644 --- a/modules/slf4j/src/main/java/org/apache/ignite/logger/slf4j/Slf4jLogger.java +++ b/modules/slf4j/src/main/java/org/apache/ignite/logger/slf4j/Slf4jLogger.java @@ -22,6 +22,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_QUIET; + /** * SLF4J-based implementation for logging. This logger should be used * by loaders that have prefer slf4j-based logging. @@ -41,11 +43,14 @@ public class Slf4jLogger implements IgniteLogger { /** SLF4J implementation proxy. */ private final Logger impl; + /** Quiet flag. */ + private final boolean quiet; + /** * Creates new logger. */ public Slf4jLogger() { - impl = LoggerFactory.getLogger(Logger.ROOT_LOGGER_NAME); + this(LoggerFactory.getLogger(Logger.ROOT_LOGGER_NAME)); } /** @@ -57,6 +62,8 @@ public Slf4jLogger(Logger impl) { assert impl != null; this.impl = impl; + + quiet = Boolean.valueOf(System.getProperty(IGNITE_QUIET, "true")); } /** {@inheritDoc} */ @@ -129,7 +136,7 @@ public Slf4jLogger(Logger impl) { /** {@inheritDoc} */ @Override public boolean isQuiet() { - return !isInfoEnabled() && !isDebugEnabled(); + return quiet; } /** {@inheritDoc} */ From 708cc8c6849b21063a555895671f6f820d92184a Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 22 Dec 2016 12:48:58 +0300 Subject: [PATCH 403/487] IGNITE-4408: Allow BinaryObjects pass to IndexingSpi. This closes #1353. --- .../apache/ignite/IgniteSystemProperties.java | 8 + .../processors/cache/IgniteCacheProxy.java | 2 +- .../cache/query/GridCacheQueryAdapter.java | 2 +- .../processors/query/GridQueryProcessor.java | 36 +++- .../ignite/spi/indexing/IndexingSpi.java | 3 + .../cache/query/IndexingSpiQuerySelfTest.java | 199 +++++++++++++++++- 6 files changed, 229 insertions(+), 21 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index de6cbed1fb3a5..fe78d889b178f 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -499,6 +499,14 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_UNALIGNED_MEMORY_ACCESS = "IGNITE_UNALIGNED_MEMORY_ACCESS"; + /** + * When set to {@code true} BinaryObject will be unwrapped before passing to IndexingSpi to preserve + * old behavior query processor with IndexingSpi. + *

          + * @deprecated Should be removed in Apache Ignite 2.0. + */ + public static final String IGNITE_UNWRAP_BINARY_FOR_INDEXING_SPI = "IGNITE_UNWRAP_BINARY_FOR_INDEXING_SPI"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java index f87fa1d5d815f..b9737c62dcbf3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java @@ -848,7 +848,7 @@ private boolean isReplicatedDataNode() { */ private void validate(Query qry) { if (!GridQueryProcessor.isEnabled(ctx.config()) && !(qry instanceof ScanQuery) && - !(qry instanceof ContinuousQuery)) + !(qry instanceof ContinuousQuery) && !(qry instanceof SpiQuery)) throw new CacheException("Indexing is disabled for cache: " + ctx.cache().name() + ". Use setIndexedTypes or setTypeMetadata methods on CacheConfiguration to enable."); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java index 2355591def2f1..b29e5e7a62ebd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java @@ -430,7 +430,7 @@ public ClusterGroup projection() { * @throws IgniteCheckedException If query is invalid. */ public void validate() throws IgniteCheckedException { - if ((type != SCAN && type != SET) && !GridQueryProcessor.isEnabled(cctx.config())) + if ((type != SCAN && type != SET && type != SPI) && !GridQueryProcessor.isEnabled(cctx.config())) throw new IgniteCheckedException("Indexing is disabled for cache: " + cctx.cache().name()); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index 8befa0e784e1e..6c093ee7b541c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -44,7 +44,7 @@ import javax.cache.CacheException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; -import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.binary.BinaryField; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryType; @@ -160,6 +160,9 @@ public class GridQueryProcessor extends GridProcessorAdapter { /** */ private static final ThreadLocal requestTopVer = new ThreadLocal<>(); + /** Default is @{true} */ + private final boolean isIndexingSpiAllowsBinary = !IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_UNWRAP_BINARY_FOR_INDEXING_SPI); + /** * @param ctx Kernal context. */ @@ -680,7 +683,11 @@ public void store(final String space, final CacheObject key, final CacheObject v if (ctx.indexing().enabled()) { coctx = cacheObjectContext(space); - ctx.indexing().store(space, key.value(coctx, false), val.value(coctx, false), expirationTime); + Object key0 = unwrap(key, coctx); + + Object val0 = unwrap(val, coctx); + + ctx.indexing().store(space, key0, val0, expirationTime); } if (idx == null) @@ -735,6 +742,13 @@ public void store(final String space, final CacheObject key, final CacheObject v } } + /** + * Unwrap CacheObject if needed. + */ + private Object unwrap(CacheObject obj, CacheObjectContext coctx) { + return isIndexingSpiAllowsBinary && ctx.cacheObjects().isBinaryObject(obj) ? obj : obj.value(coctx, false); + } + /** * @throws IgniteCheckedException If failed. */ @@ -1025,7 +1039,9 @@ public void remove(String space, CacheObject key, CacheObject val) throws Ignite if (ctx.indexing().enabled()) { CacheObjectContext coctx = cacheObjectContext(space); - ctx.indexing().remove(space, key.value(coctx, false)); + Object key0 = unwrap(key, coctx); + + ctx.indexing().remove(space, key0); } if (idx == null) @@ -1168,11 +1184,9 @@ public void onSwap(String spaceName, CacheObject key) throws IgniteCheckedExcept if (ctx.indexing().enabled()) { CacheObjectContext coctx = cacheObjectContext(spaceName); - ctx.indexing().onSwap( - spaceName, - key.value( - coctx, - false)); + Object key0 = unwrap(key, coctx); + + ctx.indexing().onSwap(spaceName, key0); } if (idx == null) @@ -1207,7 +1221,11 @@ public void onUnswap(String spaceName, CacheObject key, CacheObject val) if (ctx.indexing().enabled()) { CacheObjectContext coctx = cacheObjectContext(spaceName); - ctx.indexing().onUnswap(spaceName, key.value(coctx, false), val.value(coctx, false)); + Object key0 = unwrap(key, coctx); + + Object val0 = unwrap(val, coctx); + + ctx.indexing().onUnswap(spaceName, key0, val0); } if (idx == null) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java index a3ea33ed26ff2..bbe27c0719b2a 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java @@ -35,6 +35,9 @@ * methods. Note again that calling methods from this interface on the obtained instance can lead * to undefined behavior and explicitly not supported. * + * NOTE: Key and value arguments of IgniteSpi methods can be {@link org.apache.ignite.binary.BinaryObject} instances. + * BinaryObjects can be deserialized manually if original objects needed. + * * Here is a Java example on how to configure SPI. *

            * IndexingSpi spi = new MyIndexingSpi();
          diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
          index 94b0c8aedbb2b..f66b99ef5b1c7 100644
          --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
          +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
          @@ -17,11 +17,22 @@
           
           package org.apache.ignite.internal.processors.cache.query;
           
          +import java.io.Serializable;
          +import java.util.ArrayList;
          +import java.util.Collection;
          +import java.util.Iterator;
          +import java.util.Map;
          +import java.util.SortedMap;
          +import java.util.TreeMap;
          +import java.util.concurrent.Callable;
          +import javax.cache.Cache;
           import junit.framework.TestCase;
           import org.apache.ignite.Ignite;
           import org.apache.ignite.IgniteCache;
          +import org.apache.ignite.IgniteSystemProperties;
           import org.apache.ignite.IgniteTransactions;
           import org.apache.ignite.Ignition;
          +import org.apache.ignite.binary.BinaryObject;
           import org.apache.ignite.cache.CacheAtomicityMode;
           import org.apache.ignite.cache.query.QueryCursor;
           import org.apache.ignite.cache.query.SpiQuery;
          @@ -40,17 +51,9 @@
           import org.apache.ignite.transactions.TransactionConcurrency;
           import org.apache.ignite.transactions.TransactionIsolation;
           import org.apache.ignite.transactions.TransactionState;
          +import org.jetbrains.annotations.NotNull;
           import org.jetbrains.annotations.Nullable;
           
          -import java.util.ArrayList;
          -import java.util.Collection;
          -import java.util.Iterator;
          -import java.util.Map;
          -import java.util.SortedMap;
          -import java.util.TreeMap;
          -import java.util.concurrent.Callable;
          -import javax.cache.Cache;
          -
           /**
            * Indexing Spi query test
            */
          @@ -85,6 +88,94 @@ public void testSimpleIndexingSpi() throws Exception {
                       System.out.println(entry);
               }
           
          +    /**
          +     * @throws Exception If failed.
          +     */
          +    public void testIndexingSpiWithDisabledQueryProcessor() throws Exception {
          +        IgniteConfiguration cfg = configuration();
          +
          +        cfg.setIndexingSpi(new MyIndexingSpi());
          +
          +        Ignite ignite = Ignition.start(cfg);
          +
          +        CacheConfiguration ccfg = new CacheConfiguration<>("test-cache");
          +
          +        IgniteCache cache = ignite.createCache(ccfg);
          +
          +        for (int i = 0; i < 10; i++)
          +            cache.put(i, i);
          +
          +        QueryCursor> cursor = cache.query(new SpiQuery().setArgs(2, 5));
          +
          +        for (Cache.Entry entry : cursor)
          +            System.out.println(entry);
          +    }
          +
          +    /**
          +     * @throws Exception If failed.
          +     */
          +    public void testBinaryIndexingSpi() throws Exception {
          +        IgniteConfiguration cfg = configuration();
          +
          +        cfg.setIndexingSpi(new MyBinaryIndexingSpi());
          +
          +        Ignite ignite = Ignition.start(cfg);
          +
          +        CacheConfiguration ccfg = new CacheConfiguration<>("test-binary-cache");
          +
          +        ccfg.setIndexedTypes(PersonKey.class, Person.class);
          +
          +        IgniteCache cache = ignite.createCache(ccfg);
          +
          +        for (int i = 0; i < 10; i++) {
          +            PersonKey key = new PersonKey(i);
          +
          +            cache.put(key, new Person("John Doe " + i));
          +        }
          +
          +        QueryCursor> cursor = cache.query(
          +            new SpiQuery().setArgs(new PersonKey(2), new PersonKey(5)));
          +
          +        for (Cache.Entry entry : cursor)
          +            System.out.println(entry);
          +
          +        cache.remove(new PersonKey(9));
          +    }
          +
          +
          +    /**
          +     * @throws Exception If failed.
          +     */
          +    public void testNonBinaryIndexingSpi() throws Exception {
          +        System.setProperty(IgniteSystemProperties.IGNITE_UNWRAP_BINARY_FOR_INDEXING_SPI, "true");
          +
          +        IgniteConfiguration cfg = configuration();
          +
          +        cfg.setIndexingSpi(new MyIndexingSpi());
          +
          +        Ignite ignite = Ignition.start(cfg);
          +
          +        CacheConfiguration ccfg = new CacheConfiguration<>("test-binary-cache");
          +
          +        ccfg.setIndexedTypes(PersonKey.class, Person.class);
          +
          +        IgniteCache cache = ignite.createCache(ccfg);
          +
          +        for (int i = 0; i < 10; i++) {
          +            PersonKey key = new PersonKey(i);
          +
          +            cache.put(key, new Person("John Doe " + i));
          +        }
          +
          +        QueryCursor> cursor = cache.query(
          +            new SpiQuery().setArgs(new PersonKey(2), new PersonKey(5)));
          +
          +        for (Cache.Entry entry : cursor)
          +            System.out.println(entry);
          +
          +        cache.remove(new PersonKey(9));
          +    }
          +
               /**
                * @throws Exception If failed.
                */
          @@ -173,6 +264,9 @@ private static class MyIndexingSpi extends IgniteSpiAdapter implements IndexingS
                       Object from = paramsIt.next();
                       Object to = paramsIt.next();
           
          +            from = from instanceof BinaryObject ? ((BinaryObject)from).deserialize() : from;
          +            to = to instanceof BinaryObject ? ((BinaryObject)to).deserialize() : to;
          +
                       SortedMap map = idx.subMap(from, to);
           
                       Collection> res = new ArrayList<>(map.size());
          @@ -186,6 +280,9 @@ private static class MyIndexingSpi extends IgniteSpiAdapter implements IndexingS
                   /** {@inheritDoc} */
                   @Override public void store(@Nullable String spaceName, Object key, Object val, long expirationTime)
                       throws IgniteSpiException {
          +            assertFalse(key instanceof BinaryObject);
          +            assertFalse(val instanceof BinaryObject);
          +
                       idx.put(key, val);
                   }
           
          @@ -205,14 +302,96 @@ private static class MyIndexingSpi extends IgniteSpiAdapter implements IndexingS
                   }
               }
           
          +    /**
          +     * Indexing Spi implementation for test. Accepts binary objects only
          +     */
          +    private static class MyBinaryIndexingSpi extends MyIndexingSpi {
          +
          +        /** {@inheritDoc} */
          +        @Override public void store(@Nullable String spaceName, Object key, Object val,
          +            long expirationTime) throws IgniteSpiException {
          +            assertTrue(key instanceof BinaryObject);
          +
          +            assertTrue(val instanceof BinaryObject);
          +
          +            super.store(spaceName, ((BinaryObject)key).deserialize(), ((BinaryObject)val).deserialize(), expirationTime);
          +        }
          +
          +        /** {@inheritDoc} */
          +        @Override public void remove(@Nullable String spaceName, Object key) throws IgniteSpiException {
          +            assertTrue(key instanceof BinaryObject);
          +        }
          +
          +        /** {@inheritDoc} */
          +        @Override public void onSwap(@Nullable String spaceName, Object key) throws IgniteSpiException {
          +            assertTrue(key instanceof BinaryObject);
          +        }
          +
          +        /** {@inheritDoc} */
          +        @Override
          +        public void onUnswap(@Nullable String spaceName, Object key, Object val) throws IgniteSpiException {
          +            assertTrue(key instanceof BinaryObject);
          +
          +            assertTrue(val instanceof BinaryObject);
          +        }
          +    }
          +
               /**
                * Broken Indexing Spi implementation for test
                */
          -    private class MyBrokenIndexingSpi extends MyIndexingSpi {
          +    private static class MyBrokenIndexingSpi extends MyIndexingSpi {
                   /** {@inheritDoc} */
                   @Override public void store(@Nullable String spaceName, Object key, Object val,
                       long expirationTime) throws IgniteSpiException {
                       throw new IgniteSpiException("Test exception");
                   }
               }
          +
          +    /**
          +     *
          +     */
          +    private static class PersonKey implements Serializable, Comparable {
          +        /** */
          +        private int id;
          +
          +        /** */
          +        public PersonKey(int id) {
          +            this.id = id;
          +        }
          +
          +        /** {@inheritDoc} */
          +        @Override public int compareTo(@NotNull PersonKey o) {
          +            return Integer.compare(id, o.id);
          +        }
          +
          +        /** {@inheritDoc} */
          +        @Override public boolean equals(Object o) {
          +            if (this == o)
          +                return true;
          +            if (o == null || getClass() != o.getClass())
          +                return false;
          +
          +            PersonKey key = (PersonKey)o;
          +
          +            return id == key.id;
          +        }
          +
          +        /** {@inheritDoc} */
          +        @Override public int hashCode() {
          +            return id;
          +        }
          +    }
          +
          +    /**
          +     *
          +     */
          +    private static class Person implements Serializable {
          +        /** */
          +        private String name;
          +
          +        /** */
          +        Person(String name) {
          +            this.name = name;
          +        }
          +    }
           }
          \ No newline at end of file
          
          From 6e71ef26d8e3c6f86d1f0b9f4bec97b7e33d0b2e Mon Sep 17 00:00:00 2001
          From: Valentin Kulichenko 
          Date: Thu, 22 Dec 2016 13:05:35 -0800
          Subject: [PATCH 404/487] IGNITE-4439 - Attribute based node filter
          
          ---
           .../ignite/util/AttributeNodeFilter.java      | 105 ++++++++++
           .../testsuites/IgniteBasicTestSuite.java      |   3 +
           .../util/AttributeNodeFilterSelfTest.java     | 184 ++++++++++++++++++
           3 files changed, 292 insertions(+)
           create mode 100644 modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
           create mode 100644 modules/core/src/test/java/org/apache/ignite/util/AttributeNodeFilterSelfTest.java
          
          diff --git a/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java b/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
          new file mode 100644
          index 0000000000000..e2b972be462fb
          --- /dev/null
          +++ b/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
          @@ -0,0 +1,105 @@
          +/*
          + * 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.util;
          +
          +import java.util.Collections;
          +import java.util.Map;
          +import org.apache.ignite.cluster.ClusterGroup;
          +import org.apache.ignite.cluster.ClusterNode;
          +import org.apache.ignite.configuration.CacheConfiguration;
          +import org.apache.ignite.configuration.IgniteConfiguration;
          +import org.apache.ignite.internal.util.typedef.F;
          +import org.apache.ignite.internal.util.typedef.internal.A;
          +import org.apache.ignite.lang.IgnitePredicate;
          +import org.apache.ignite.services.ServiceConfiguration;
          +import org.jetbrains.annotations.Nullable;
          +
          +/**
          + * Implementation of {@code IgnitePredicate} based on
          + * {@link IgniteConfiguration#getUserAttributes() user attributes}.
          + * This filter can be used in methods like {@link ClusterGroup#forPredicate(IgnitePredicate)},
          + * {@link CacheConfiguration#setNodeFilter(IgnitePredicate)},
          + * {@link ServiceConfiguration#setNodeFilter(IgnitePredicate)}, etc.
          + * 

          + * The filter will evaluate to true if a node has all provided attributes set to + * corresponding values. Here is an example of how you can configure node filter for a + * cache or a service so that it's deployed only on nodes that have {@code group} + * attribute set to value {@code data}: + *

          + * <property name="nodeFilter">
          + *     <bean class="org.apache.ignite.util.ClusterAttributeNodeFilter">
          + *         <constructor-arg value="group"/>
          + *         <constructor-arg value="data"/>
          + *     </bean>
          + * </property>
          + * 
          + * You can also specify multiple attributes for the filter: + *
          + * <property name="nodeFilter">
          + *     <bean class="org.apache.ignite.util.ClusterAttributeNodeFilter">
          + *         <constructor-arg>
          + *             <map>
          + *                 <entry key="cpu-group" value="high"/>
          + *                 <entry key="memory-group" value="high"/>
          + *             </map>
          + *         </constructor-arg>
          + *     </bean>
          + * </property>
          + * 
          + * With this configuration a cache or a service will deploy only on nodes that have both + * {@code cpu-group} and {@code memory-group} attributes set to value {@code high}. + */ +public class AttributeNodeFilter implements IgnitePredicate { + /** Attributes. */ + private final Map attrs; + + /** + * Creates new node filter with a single attribute value. + * + * @param attrName Attribute name. + * @param attrVal Attribute value. + */ + public AttributeNodeFilter(String attrName, @Nullable Object attrVal) { + A.notNull(attrName, "attrName"); + + attrs = Collections.singletonMap(attrName, attrVal); + } + + /** + * Creates new node filter with a set of attributes. + * + * @param attrs Attributes. + */ + public AttributeNodeFilter(Map attrs) { + A.notNull(attrs, "attrs"); + + this.attrs = attrs; + } + + /** {@inheritDoc} */ + @Override public boolean apply(ClusterNode node) { + Map nodeAttrs = node.attributes(); + + for (Map.Entry attr : attrs.entrySet()) { + if (!F.eq(nodeAttrs.get(attr.getKey()), attr.getValue())) + return false; + } + + return true; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java index 1c1fcf7b494a9..b2fafe20a56c7 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java @@ -20,6 +20,7 @@ import java.util.Set; import junit.framework.TestSuite; import org.apache.ignite.GridSuppressedExceptionSelfTest; +import org.apache.ignite.util.AttributeNodeFilterSelfTest; import org.apache.ignite.internal.ClusterGroupHostsSelfTest; import org.apache.ignite.internal.ClusterGroupSelfTest; import org.apache.ignite.internal.GridFailFastNodeFailureDetectionSelfTest; @@ -148,6 +149,8 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(SecurityPermissionSetBuilderTest.class); + suite.addTestSuite(AttributeNodeFilterSelfTest.class); + return suite; } } diff --git a/modules/core/src/test/java/org/apache/ignite/util/AttributeNodeFilterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/util/AttributeNodeFilterSelfTest.java new file mode 100644 index 0000000000000..ac3800f19545c --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/util/AttributeNodeFilterSelfTest.java @@ -0,0 +1,184 @@ +/* + * 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.util; + +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; +import java.util.Collections; +import java.util.Map; +import org.apache.ignite.Ignite; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests for {@link AttributeNodeFilter}. + */ +public class AttributeNodeFilterSelfTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private Map attrs; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); + + if (attrs != null) + cfg.setUserAttributes(attrs); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + attrs = null; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testSingleAttribute() throws Exception { + IgnitePredicate filter = new AttributeNodeFilter("attr", "value"); + + assertTrue(filter.apply(nodeProxy(F.asMap("attr", "value")))); + assertFalse(filter.apply(nodeProxy(F.asMap("attr", "wrong")))); + assertFalse(filter.apply(nodeProxy(F.asMap("attr", null)))); + assertFalse(filter.apply(nodeProxy(Collections.emptyMap()))); + assertFalse(filter.apply(nodeProxy(F.asMap("wrong", "value")))); + assertFalse(filter.apply(nodeProxy(F.asMap("null", "value")))); + } + + /** + * @throws Exception If failed. + */ + public void testSingleAttributeNullValue() throws Exception { + IgnitePredicate filter = new AttributeNodeFilter("attr", null); + + assertTrue(filter.apply(nodeProxy(F.asMap("attr", null)))); + assertTrue(filter.apply(nodeProxy(Collections.emptyMap()))); + assertTrue(filter.apply(nodeProxy(F.asMap("wrong", "value")))); + assertTrue(filter.apply(nodeProxy(F.asMap("wrong", null)))); + assertFalse(filter.apply(nodeProxy(F.asMap("attr", "value")))); + } + + /** + * @throws Exception If failed. + */ + public void testMultipleAttributes() throws Exception { + IgnitePredicate filter = + new AttributeNodeFilter(F.asMap("attr1", "value1", "attr2", "value2")); + + assertTrue(filter.apply(nodeProxy(F.asMap("attr1", "value1", "attr2", "value2")))); + assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "wrong", "attr2", "value2")))); + assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "value1", "attr2", "wrong")))); + assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "wrong", "attr2", "wrong")))); + assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "value1")))); + assertFalse(filter.apply(nodeProxy(F.asMap("attr2", "value2")))); + assertFalse(filter.apply(nodeProxy(Collections.emptyMap()))); + } + + /** + * @throws Exception If failed. + */ + public void testMultipleAttributesNullValues() throws Exception { + IgnitePredicate filter = new AttributeNodeFilter(F.asMap("attr1", null, "attr2", null)); + + assertTrue(filter.apply(nodeProxy(F.asMap("attr1", null, "attr2", null)))); + assertTrue(filter.apply(nodeProxy(F.asMap("attr1", null)))); + assertTrue(filter.apply(nodeProxy(F.asMap("attr2", null)))); + assertTrue(filter.apply(nodeProxy(Collections.emptyMap()))); + assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "value1")))); + assertFalse(filter.apply(nodeProxy(F.asMap("attr2", "value2")))); + assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "value1", "attr2", "value2")))); + } + + /** + * @throws Exception If failed. + */ + public void testClusterGroup() throws Exception { + Ignite group1 = startGridsMultiThreaded(3); + + attrs = F.asMap("group", "data"); + + Ignite group2 = startGridsMultiThreaded(3, 2); + + assertEquals(2, group1.cluster().forPredicate(new AttributeNodeFilter("group", "data")).nodes().size()); + assertEquals(2, group2.cluster().forPredicate(new AttributeNodeFilter("group", "data")).nodes().size()); + + assertEquals(3, group1.cluster().forPredicate(new AttributeNodeFilter("group", null)).nodes().size()); + assertEquals(3, group2.cluster().forPredicate(new AttributeNodeFilter("group", null)).nodes().size()); + + assertEquals(0, group1.cluster().forPredicate(new AttributeNodeFilter("group", "wrong")).nodes().size()); + assertEquals(0, group2.cluster().forPredicate(new AttributeNodeFilter("group", "wrong")).nodes().size()); + } + + /** + * @throws Exception If failed. + */ + public void testCacheFilter() throws Exception { + Ignite group1 = startGridsMultiThreaded(3); + + attrs = F.asMap("group", "data"); + + Ignite group2 = startGridsMultiThreaded(3, 2); + + group1.createCache(new CacheConfiguration<>("test-cache"). + setNodeFilter(new AttributeNodeFilter("group", "data"))); + + assertEquals(2, group1.cluster().forDataNodes("test-cache").nodes().size()); + assertEquals(2, group2.cluster().forDataNodes("test-cache").nodes().size()); + + assertEquals(0, group1.cluster().forDataNodes("wrong").nodes().size()); + assertEquals(0, group2.cluster().forDataNodes("wrong").nodes().size()); + } + + /** + * @param attrs Attributes. + * @return Node proxy. + */ + private static ClusterNode nodeProxy(final Map attrs) { + return (ClusterNode)Proxy.newProxyInstance( + ClusterNode.class.getClassLoader(), + new Class[] { ClusterNode.class }, + new InvocationHandler() { + @SuppressWarnings("SuspiciousMethodCalls") + @Override public Object invoke(Object proxy, Method mtd, Object[] args) throws Throwable { + if ("attributes".equals(mtd.getName())) + return attrs; + + throw new UnsupportedOperationException(); + } + }); + } +} From babfc2f051f8471f541bd054650a47cceb3cc09e Mon Sep 17 00:00:00 2001 From: sboikov Date: Fri, 23 Dec 2016 12:02:24 +0300 Subject: [PATCH 405/487] AttributeNodeFilter: added serialVersionUID. --- .../main/java/org/apache/ignite/util/AttributeNodeFilter.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java b/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java index e2b972be462fb..fed0d43f26019 100644 --- a/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java +++ b/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java @@ -65,6 +65,9 @@ * {@code cpu-group} and {@code memory-group} attributes set to value {@code high}. */ public class AttributeNodeFilter implements IgnitePredicate { + /** */ + private static final long serialVersionUID = 0L; + /** Attributes. */ private final Map attrs; From 2b3a180ff7692c0253da3ff7c32d65c09f9488d2 Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Fri, 23 Dec 2016 16:34:10 +0700 Subject: [PATCH 406/487] Web console beta-7. (cherry picked from commit 8e7c852) --- modules/web-console/backend/app/agent.js | 15 + modules/web-console/backend/app/browser.js | 13 + modules/web-console/backend/app/mongo.js | 24 +- modules/web-console/backend/routes/demo.js | 17 +- modules/web-console/backend/routes/profile.js | 3 +- .../web-console/backend/services/notebooks.js | 14 +- .../web-console/backend/services/sessions.js | 6 +- .../web-console/backend/services/spaces.js | 15 + modules/web-console/frontend/app/app.js | 5 - .../controllers/reset-password.controller.js | 14 +- .../{event-types.json => event-groups.json} | 0 .../frontend/app/data/pom-dependencies.json | 12 +- .../ui-ace-docker/ui-ace-docker.controller.js | 2 +- .../ui-ace-docker/ui-ace-docker.jade | 2 +- .../ui-ace-pojos/ui-ace-pojos.controller.js | 12 +- .../ui-ace-pom/ui-ace-pom.controller.js | 4 +- .../jade/form/form-field-dropdown.jade | 5 +- .../helpers/jade/form/form-field-number.jade | 3 +- .../helpers/jade/form/form-field-text.jade | 19 +- .../frontend/app/helpers/jade/mixins.jade | 52 +- .../frontend/app/modules/Demo/Demo.module.js | 6 +- .../modules/configuration/Version.service.js | 6 +- .../configuration/configuration.module.js | 63 +- .../generator/AbstractTransformer.js | 17 + .../modules/configuration/generator/Beans.js | 5 + .../generator/ConfigurationGenerator.js | 2795 ++++++------- ...enerator-optional.js => Custom.service.js} | 12 +- .../configuration/generator/Docker.service.js | 4 +- .../generator/JavaTransformer.service.js | 2318 +++++------ .../{Pom.service.js => Maven.service.js} | 7 +- .../generator/Properties.service.js | 2 +- .../configuration/generator/Readme.service.js | 2 +- .../generator/SharpTransformer.service.js | 437 +- .../generator/SpringTransformer.service.js | 497 ++- ....provider.js => Cache.platform.service.js} | 12 +- .../{cache.provider.js => Cache.service.js} | 14 +- ...rovider.js => Cluster.platform.service.js} | 14 +- ...cluster.provider.js => Cluster.service.js} | 12 +- .../defaults/Event-groups.service.js} | 19 +- .../{igfs.provider.js => IGFS.service.js} | 12 +- .../generator/generator-common.js | 625 --- .../configuration/generator/generator-java.js | 3617 ----------------- .../generator/generator-spring.js | 2111 ---------- .../frontend/app/modules/sql/Notebook.data.js | 11 +- .../app/modules/sql/Notebook.service.js | 2 +- .../app/modules/sql/scan-filter-input.jade | 39 - .../app/modules/sql/sql.controller.js | 211 +- .../frontend/app/modules/sql/sql.module.js | 2 - .../app/modules/states/configuration.state.js | 2 + .../configuration/caches/node-filter.jade | 2 +- .../states/configuration/caches/query.jade | 3 + .../states/configuration/caches/store.jade | 4 +- .../configuration/clusters/checkpoint.jade | 11 +- .../configuration/clusters/checkpoint/fs.jade | 8 +- .../clusters/checkpoint/jdbc.jade | 8 +- .../configuration/clusters/checkpoint/s3.jade | 25 +- .../clusters/collision/custom.jade | 2 +- .../clusters/collision/job-stealing.jade | 2 +- .../configuration/clusters/deployment.jade | 129 +- .../states/configuration/clusters/events.jade | 4 +- .../configuration/clusters/failover.jade | 4 +- .../clusters/general/discovery/zookeeper.jade | 2 +- .../zookeeper/retrypolicy/custom.jade | 2 +- .../clusters/load-balancing.jade | 23 +- .../configuration/clusters/logger/custom.jade | 2 +- .../states/configuration/clusters/ssl.jade | 2 +- .../summary/summary-zipper.service.js} | 40 +- .../summary/summary.controller.js | 103 +- .../configuration/summary/summary.worker.js | 123 + .../frontend/app/modules/user/Auth.service.js | 11 +- .../app/services/JavaTypes.service.js | 13 +- .../frontend/app/services/Messages.service.js | 17 +- .../frontend/controllers/admin-controller.js | 211 +- .../frontend/controllers/caches-controller.js | 22 +- .../controllers/clusters-controller.js | 42 +- .../controllers/domains-controller.js | 32 +- .../frontend/controllers/igfs-controller.js | 20 +- .../controllers/profile-controller.js | 3 +- .../gulpfile.babel.js/webpack/common.js | 17 +- .../webpack/environments/development.js | 14 +- .../webpack/environments/production.js | 3 +- .../webpack/plugins/progress.js | 82 - modules/web-console/frontend/package.json | 178 +- .../frontend/public/images/cache.png | Bin 23700 -> 24791 bytes .../frontend/public/images/domains.png | Bin 23828 -> 22131 bytes .../frontend/public/images/igfs.png | Bin 14307 -> 14139 bytes .../frontend/public/images/query-chart.png | Bin 16637 -> 17142 bytes .../frontend/public/images/query-metadata.png | Bin 32298 -> 39361 bytes .../frontend/public/images/query-table.png | Bin 29189 -> 28065 bytes .../frontend/public/images/summary.png | Bin 31997 -> 33650 bytes .../stylesheets/_font-awesome-custom.scss | 23 +- .../public/stylesheets/form-field.scss | 37 + .../frontend/public/stylesheets/style.scss | 111 +- .../frontend/test/unit/JavaTypes.test.js | 17 +- .../frontend/test/unit/Version.test.js | 8 +- .../views/configuration/domains-import.jade | 5 +- .../frontend/views/configuration/summary.jade | 25 +- .../frontend/views/settings/admin.jade | 85 +- .../frontend/views/sql/notebook-new.jade | 2 +- .../web-console/frontend/views/sql/sql.jade | 235 +- .../frontend/views/templates/alert.jade | 2 +- .../frontend/views/templates/select.jade | 2 +- 102 files changed, 4554 insertions(+), 10273 deletions(-) rename modules/web-console/frontend/app/data/{event-types.json => event-groups.json} (100%) rename modules/web-console/frontend/app/modules/configuration/generator/{generator-optional.js => Custom.service.js} (82%) rename modules/web-console/frontend/app/modules/configuration/generator/{Pom.service.js => Maven.service.js} (97%) rename modules/web-console/frontend/app/modules/configuration/generator/defaults/{cache.platform.provider.js => Cache.platform.service.js} (88%) rename modules/web-console/frontend/app/modules/configuration/generator/defaults/{cache.provider.js => Cache.service.js} (95%) rename modules/web-console/frontend/app/modules/configuration/generator/defaults/{cluster.platform.provider.js => Cluster.platform.service.js} (84%) rename modules/web-console/frontend/app/modules/configuration/generator/defaults/{cluster.provider.js => Cluster.service.js} (98%) rename modules/web-console/frontend/app/modules/configuration/{EventGroups.provider.js => generator/defaults/Event-groups.service.js} (78%) rename modules/web-console/frontend/app/modules/configuration/generator/defaults/{igfs.provider.js => IGFS.service.js} (92%) delete mode 100644 modules/web-console/frontend/app/modules/configuration/generator/generator-common.js delete mode 100644 modules/web-console/frontend/app/modules/configuration/generator/generator-java.js delete mode 100644 modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js delete mode 100644 modules/web-console/frontend/app/modules/sql/scan-filter-input.jade rename modules/web-console/frontend/app/modules/{sql/scan-filter-input.service.js => states/configuration/summary/summary-zipper.service.js} (51%) create mode 100644 modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js delete mode 100644 modules/web-console/frontend/gulpfile.babel.js/webpack/plugins/progress.js diff --git a/modules/web-console/backend/app/agent.js b/modules/web-console/backend/app/agent.js index f74a3f2e4c245..791ea50ed58b9 100644 --- a/modules/web-console/backend/app/agent.js +++ b/modules/web-console/backend/app/agent.js @@ -312,6 +312,21 @@ module.exports.factory = function(_, fs, path, JSZip, socketio, settings, mongo) return this.executeRest(cmd); } + /** + * @param {Boolean} demo Is need run command on demo node. + * @param {Array.} nids Node ids. + * @returns {Promise} + */ + queryResetDetailMetrics(demo, nids) { + const cmd = new Command(demo, 'exe') + .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask') + .addParam('p1', nids) + .addParam('p2', 'org.apache.ignite.internal.visor.cache.VisorCacheResetQueryDetailMetricsTask') + .addParam('p3', 'java.lang.Void'); + + return this.executeRest(cmd); + } + /** * @param {Boolean} demo Is need run command on demo node. * @param {String} cacheName Cache name. diff --git a/modules/web-console/backend/app/browser.js b/modules/web-console/backend/app/browser.js index 271082910e2c1..499d84d8e990a 100644 --- a/modules/web-console/backend/app/browser.js +++ b/modules/web-console/backend/app/browser.js @@ -162,6 +162,19 @@ module.exports.factory = (_, socketio, agentMgr, configure) => { .catch((err) => cb(_errorToJson(err))); }); + // Collect cache query metrics and return result to browser. + socket.on('node:query:reset:metrics', (nids, cb) => { + agentMgr.findAgent(accountId()) + .then((agent) => agent.queryResetDetailMetrics(demo, nids)) + .then((data) => { + if (data.finished) + return cb(null, data.result); + + cb(_errorToJson(data.error)); + }) + .catch((err) => cb(_errorToJson(err))); + }); + // Return cache metadata from all nodes in grid. socket.on('node:cache:metadata', (cacheName, cb) => { agentMgr.findAgent(accountId()) diff --git a/modules/web-console/backend/app/mongo.js b/modules/web-console/backend/app/mongo.js index 0f38eb2c3c899..58ab11983aca1 100644 --- a/modules/web-console/backend/app/mongo.js +++ b/modules/web-console/backend/app/mongo.js @@ -247,6 +247,7 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose longQueryWarningTimeout: Number, sqlFunctionClasses: [String], snapshotableIndex: Boolean, + queryDetailMetricsSize: Number, statisticsEnabled: Boolean, managementEnabled: Boolean, readFromBackup: Boolean, @@ -823,7 +824,24 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose Custom: { className: String } - }] + }], + deploymentSpi: { + kind: {type: String, enum: ['URI', 'Local', 'Custom']}, + URI: { + uriList: [String], + temporaryDirectoryPath: String, + scanners: [String], + listener: String, + checkMd5: Boolean, + encodeUri: Boolean + }, + Local: { + listener: String + }, + Custom: { + className: String + } + } }); ClusterSchema.index({name: 1, space: 1}, {unique: true}); @@ -843,13 +861,15 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose result: {type: String, enum: ['none', 'table', 'bar', 'pie', 'line', 'area']}, pageSize: Number, timeLineSpan: String, + maxPages: Number, hideSystemColumns: Boolean, cacheName: String, chartsOptions: {barChart: {stacked: Boolean}, areaChart: {style: String}}, rate: { value: Number, unit: Number - } + }, + qryType: String }] }); diff --git a/modules/web-console/backend/routes/demo.js b/modules/web-console/backend/routes/demo.js index ad4be6e6a4e15..3f4166d883763 100644 --- a/modules/web-console/backend/routes/demo.js +++ b/modules/web-console/backend/routes/demo.js @@ -39,20 +39,17 @@ module.exports.factory = (_, express, settings, mongo, spacesService, errors) => router.post('/reset', (req, res) => { spacesService.spaces(req.user._id, true) .then((spaces) => { - if (spaces.length) { - const spaceIds = spaces.map((space) => space._id); - - return Promise.all([ - mongo.Cluster.remove({space: {$in: spaceIds}}).exec(), - mongo.Cache.remove({space: {$in: spaceIds}}).exec(), - mongo.DomainModel.remove({space: {$in: spaceIds}}).exec(), - mongo.Igfs.remove({space: {$in: spaceIds}}).exec() - ]).then(() => spaces[0]); - } + const spaceIds = _.map(spaces, '_id'); + + return spacesService.cleanUp(spaceIds) + .then(() => mongo.Space.remove({_id: {$in: _.tail(spaceIds)}}).exec()) + .then(() => _.head(spaces)); }) .catch((err) => { if (err instanceof errors.MissingResourceException) return spacesService.createDemoSpace(req.user._id); + + throw err; }) .then((space) => { return Promise.all(_.map(clusters, (cluster) => { diff --git a/modules/web-console/backend/routes/profile.js b/modules/web-console/backend/routes/profile.js index 4d01cda1e44ae..1d6fccb8213f3 100644 --- a/modules/web-console/backend/routes/profile.js +++ b/modules/web-console/backend/routes/profile.js @@ -45,7 +45,7 @@ module.exports.factory = function(_, express, mongo, usersService) { usersService.save(req.body) .then((user) => { - const becomeUsed = req.session.viewedUser && user.admin; + const becomeUsed = req.session.viewedUser && req.user.admin; if (becomeUsed) { req.session.viewedUser = user; @@ -64,6 +64,7 @@ module.exports.factory = function(_, express, mongo, usersService) { }); }); }) + .then(() => usersService.get(req.user, req.session.viewedUser)) .then(res.api.ok) .catch(res.api.error); }); diff --git a/modules/web-console/backend/services/notebooks.js b/modules/web-console/backend/services/notebooks.js index 8846d8e365dd7..9aa2c386d8abe 100644 --- a/modules/web-console/backend/services/notebooks.js +++ b/modules/web-console/backend/services/notebooks.js @@ -34,12 +34,14 @@ module.exports = { module.exports.factory = (_, mongo, spacesService, errors) => { /** * Convert remove status operation to own presentation. + * * @param {RemoveResult} result - The results of remove operation. */ const convertRemoveStatus = ({result}) => ({rowsAffected: result.n}); /** - * Update existing notebook + * Update existing notebook. + * * @param {Object} notebook - The notebook for updating * @returns {Promise.} that resolves cache id */ @@ -53,6 +55,7 @@ module.exports.factory = (_, mongo, spacesService, errors) => { /** * Create new notebook. + * * @param {Object} notebook - The notebook for creation. * @returns {Promise.} that resolves cache id. */ @@ -67,6 +70,7 @@ module.exports.factory = (_, mongo, spacesService, errors) => { class NotebooksService { /** * Create or update Notebook. + * * @param {Object} notebook - The Notebook * @returns {Promise.} that resolves Notebook id of merge operation. */ @@ -78,16 +82,18 @@ module.exports.factory = (_, mongo, spacesService, errors) => { } /** - * Get caches by spaces. + * Get notebooks by spaces. + * * @param {mongo.ObjectId|String} spaceIds - The spaces ids that own caches. - * @returns {Promise.} - contains requested caches. + * @returns {Promise.} - contains requested caches. */ static listBySpaces(spaceIds) { return mongo.Notebook.find({space: {$in: spaceIds}}).sort('name').lean().exec(); } /** - * Remove Notebook. + * Remove notebook. + * * @param {mongo.ObjectId|String} notebookId - The Notebook id for remove. * @returns {Promise.<{rowsAffected}>} - The number of affected rows. */ diff --git a/modules/web-console/backend/services/sessions.js b/modules/web-console/backend/services/sessions.js index ff0e303068080..7f62a606c9369 100644 --- a/modules/web-console/backend/services/sessions.js +++ b/modules/web-console/backend/services/sessions.js @@ -38,11 +38,11 @@ module.exports.factory = (_, mongo, errors) => { * @param {mongo.ObjectId|String} viewedUserId - id of user to become. */ static become(session, viewedUserId) { + if (!session.req.user.admin) + return Promise.reject(new errors.IllegalAccessError('Became this user is not permitted. Only administrators can perform this actions.')); + return mongo.Account.findById(viewedUserId).lean().exec() .then((viewedUser) => { - if (!session.req.user.admin) - throw new errors.IllegalAccessError('Became this user is not permitted. Only administrators can perform this actions.'); - viewedUser.token = session.req.user.token; session.viewedUser = viewedUser; diff --git a/modules/web-console/backend/services/spaces.js b/modules/web-console/backend/services/spaces.js index 863d57cc9c74e..85f346e7bc510 100644 --- a/modules/web-console/backend/services/spaces.js +++ b/modules/web-console/backend/services/spaces.js @@ -68,6 +68,21 @@ module.exports.factory = (mongo, errors) => { static createDemoSpace(userId) { return new mongo.Space({name: 'Demo space', owner: userId, demo: true}).save(); } + + /** + * Clean up spaces. + * + * @param {mongo.ObjectId|String} spaceIds - The space ids for clean up. + * @returns {Promise.<>} + */ + static cleanUp(spaceIds) { + return Promise.all([ + mongo.Cluster.remove({space: {$in: spaceIds}}).exec(), + mongo.Cache.remove({space: {$in: spaceIds}}).exec(), + mongo.DomainModel.remove({space: {$in: spaceIds}}).exec(), + mongo.Igfs.remove({space: {$in: spaceIds}}).exec() + ]); + } } return SpacesService; diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js index 3510743e129c3..4ecd9b5394483 100644 --- a/modules/web-console/frontend/app/app.js +++ b/modules/web-console/frontend/app/app.js @@ -99,11 +99,6 @@ import domainsValidation from './filters/domainsValidation.filter'; import duration from './filters/duration.filter'; import hasPojo from './filters/hasPojo.filter'; -// Generators -import $generatorOptional from './modules/configuration/generator/generator-optional'; - -window.$generatorOptional = $generatorOptional; - // Controllers import admin from 'controllers/admin-controller'; import caches from 'controllers/caches-controller'; diff --git a/modules/web-console/frontend/app/controllers/reset-password.controller.js b/modules/web-console/frontend/app/controllers/reset-password.controller.js index da0c37b05c428..f84a8762af49c 100644 --- a/modules/web-console/frontend/app/controllers/reset-password.controller.js +++ b/modules/web-console/frontend/app/controllers/reset-password.controller.js @@ -21,10 +21,10 @@ export default ['resetPassword', [ ($scope, $modal, $http, $state, Messages, Focus) => { if ($state.params.token) { $http.post('/api/v1/password/validate/token', {token: $state.params.token}) - .success((res) => { - $scope.email = res.email; - $scope.token = res.token; - $scope.error = res.error; + .then(({data}) => { + $scope.email = data.email; + $scope.token = data.token; + $scope.error = data.error; if ($scope.token && !$scope.error) Focus.move('user_password'); @@ -34,16 +34,16 @@ export default ['resetPassword', [ // Try to reset user password for provided token. $scope.resetPassword = (reset_info) => { $http.post('/api/v1/password/reset', reset_info) - .success(() => { + .then(() => { $state.go('signin'); Messages.showInfo('Password successfully changed'); }) - .error((err, state) => { + .catch(({data, state}) => { if (state === 503) $state.go('signin'); - Messages.showError(err); + Messages.showError(data); }); }; } diff --git a/modules/web-console/frontend/app/data/event-types.json b/modules/web-console/frontend/app/data/event-groups.json similarity index 100% rename from modules/web-console/frontend/app/data/event-types.json rename to modules/web-console/frontend/app/data/event-groups.json diff --git a/modules/web-console/frontend/app/data/pom-dependencies.json b/modules/web-console/frontend/app/data/pom-dependencies.json index acf2bc86d6249..7d2bed0de8172 100644 --- a/modules/web-console/frontend/app/data/pom-dependencies.json +++ b/modules/web-console/frontend/app/data/pom-dependencies.json @@ -10,11 +10,11 @@ "HadoopIgfsJcl": {"artifactId": "ignite-hadoop"}, "SLF4J": {"artifactId": "ignite-slf4j"}, - "Generic": {"groupId": "com.mchange", "artifactId": "c3p0", "version": "0.9.5.1"}, - "MySQL": {"groupId": "mysql", "artifactId": "mysql-connector-java", "version": "5.1.37"}, - "PostgreSQL": {"groupId": "org.postgresql", "artifactId": "postgresql", "version": "9.4-1204-jdbc42"}, + "Generic": {"groupId": "com.mchange", "artifactId": "c3p0", "version": "0.9.5.2"}, + "MySQL": {"groupId": "mysql", "artifactId": "mysql-connector-java", "version": "5.1.40"}, + "PostgreSQL": {"groupId": "org.postgresql", "artifactId": "postgresql", "version": "9.4.1212.jre7"}, "H2": {"groupId": "com.h2database", "artifactId": "h2", "version": "1.4.191"}, - "Oracle": {"groupId": "oracle", "artifactId": "jdbc", "version": "11.2", "jar": "ojdbc6.jar"}, - "DB2": {"groupId": "ibm", "artifactId": "jdbc", "version": "4.19.26", "jar": "db2jcc4.jar"}, - "SQLServer": {"groupId": "microsoft", "artifactId": "jdbc", "version": "4.1", "jar": "sqljdbc41.jar"} + "Oracle": {"groupId": "com.oracle.jdbc", "artifactId": "ojdbc7", "version": "12.1.0.2", "jar": "ojdbc7.jar"}, + "DB2": {"groupId": "ibm", "artifactId": "jdbc", "version": "4.21.29", "jar": "db2jcc4.jar"}, + "SQLServer": {"groupId": "microsoft", "artifactId": "jdbc", "version": "4.2", "jar": "sqljdbc41.jar"} } diff --git a/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js index 32feaf357c3a3..de335aefdcacb 100644 --- a/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js +++ b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js @@ -15,7 +15,7 @@ * limitations under the License. */ -export default ['$scope', 'GeneratorDocker', function($scope, docker) { +export default ['$scope', 'IgniteDockerGenerator', function($scope, docker) { const ctrl = this; // Watchers definition. diff --git a/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade index 3b0e7b8e9f492..3a24cfbd956fd 100644 --- a/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade +++ b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade @@ -20,7 +20,7 @@ mixin hard-link(ref, txt) .panel-details-noborder .details-row p - +hard-link('https://docs.docker.com/reference/builder', 'Docker') + +hard-link('https://docs.docker.com/engine/reference/builder/', 'Docker') |  file is a text file with instructions to create Docker image.
          | To build image you have to store following Docker file with your Ignite XML configuration to the same directory.
          | Also you could use predefined  diff --git a/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js index 4e1187447096c..61bf086bc741e 100644 --- a/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js +++ b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js @@ -30,7 +30,7 @@ export default ['$scope', 'JavaTypes', 'JavaTransformer', function($scope, JavaT const updatePojos = () => { delete ctrl.pojos; - if (!ctrl.cluster || !ctrl.cluster.caches) + if (_.isNil(ctrl.cluster) || _.isEmpty(ctrl.cluster.caches)) return; ctrl.pojos = generator.pojos(ctrl.cluster.caches, ctrl.useConstructor, ctrl.includeKeyFields); @@ -46,7 +46,7 @@ export default ['$scope', 'JavaTypes', 'JavaTransformer', function($scope, JavaT const classes = ctrl.classes = []; _.forEach(ctrl.pojos, (pojo) => { - if (pojo.keyType && JavaTypes.nonBuiltInClass(pojo.keyType)) + if (_.nonNil(pojo.keyClass)) classes.push(pojo.keyType); classes.push(pojo.valueType); @@ -55,17 +55,17 @@ export default ['$scope', 'JavaTypes', 'JavaTransformer', function($scope, JavaT // Update pojos class. const updateClass = (value) => { - if (!value || !ctrl.pojos.length) + if (_.isEmpty(value)) return; - const keyType = ctrl.pojos[0].keyType; + const pojo = value[0]; - ctrl.class = ctrl.class || (JavaTypes.nonBuiltInClass(keyType) ? keyType : null) || ctrl.pojos[0].valueType; + ctrl.class = ctrl.class || (pojo.keyClass ? pojo.keyType : pojo.valueType); }; // Update pojos data. const updatePojosData = (value) => { - if (!value) + if (_.isNil(value)) return; _.forEach(ctrl.pojos, (pojo) => { diff --git a/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js index 2bf78c359230d..477cf20999271 100644 --- a/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js +++ b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js @@ -15,7 +15,7 @@ * limitations under the License. */ -export default ['$scope', 'GeneratorPom', 'IgniteVersion', function($scope, pom, Version) { +export default ['$scope', 'IgniteMavenGenerator', 'IgniteVersion', function($scope, maven, Version) { const ctrl = this; // Watchers definition. @@ -25,7 +25,7 @@ export default ['$scope', 'GeneratorPom', 'IgniteVersion', function($scope, pom, if (!value) return; - ctrl.data = pom.generate($scope.cluster, Version.productVersion().ignite).asString(); + ctrl.data = maven.generate($scope.cluster, Version.productVersion().ignite).asString(); }; // Setup watchers. diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade index 298db52c89b1b..33af6d1d8785a 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade @@ -28,7 +28,7 @@ mixin ignite-form-field-dropdown(label, model, name, disabled, required, multipl data-ng-disabled=disabled && '#{disabled}' || '!#{options}.length' bs-select - bs-options='item.value as item.label for item in #{options}' + bs-options='item.value as item.label for item in #{options}' data-multiple=multiple ? '1' : false data-container='body > .wrapper' @@ -41,7 +41,8 @@ mixin ignite-form-field-dropdown(label, model, name, disabled, required, multipl .ignite-form-field +ignite-form-field__label(label, name, required) .ignite-form-field__control - i.tipField.icon-help(bs-tooltip='' data-title=tip) + if tip + i.tipField.icon-help(bs-tooltip='' data-title=tip) if block block diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade index d48343c36d6f0..58b0dcd654291 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade @@ -38,7 +38,8 @@ mixin ignite-form-field-number(label, model, name, disabled, required, placehold .ignite-form-field +ignite-form-field__label(label, name, required) .ignite-form-field__control - i.tipField.icon-help(bs-tooltip='' data-title=tip) + if tip + i.tipField.icon-help(bs-tooltip='' data-title=tip) +form-field-feedback(name, 'required', 'This field could not be empty') +form-field-feedback(name, 'min', 'Value is less than allowable minimum: '+ min || 0) diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade index 136d23b2340a1..1f93d3b8ede50 100644 --- a/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade +++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade @@ -30,13 +30,30 @@ mixin ignite-form-field-input(name, model, disabled, required, placeholder) data-ignite-form-panel-field='' )&attributes(attributes ? attributes.attributes ? attributes.attributes : attributes: {}) +mixin ignite-form-field-url-input(name, model, disabled, required, placeholder) + input.form-control( + id='{{ #{name} }}Input' + name='{{ #{name} }}' + placeholder=placeholder + type='url' + + data-ng-model=model + + data-ng-required=required && '#{required}' + data-ng-disabled=disabled && '#{disabled}' + data-ng-focus='tableReset()' + + data-ignite-form-panel-field='' + )&attributes(attributes ? attributes.attributes ? attributes.attributes : attributes: {}) + mixin ignite-form-field-text(label, model, name, disabled, required, placeholder, tip) -var errLbl = label.substring(0, label.length - 1) .ignite-form-field +ignite-form-field__label(label, name, required) .ignite-form-field__control - i.tipField.icon-help(bs-tooltip='' data-title=tip) + if tip + i.tipField.icon-help(bs-tooltip='' data-title=tip) if block block diff --git a/modules/web-console/frontend/app/helpers/jade/mixins.jade b/modules/web-console/frontend/app/helpers/jade/mixins.jade index 92af1b02a01b3..6ca41f645c68c 100644 --- a/modules/web-console/frontend/app/helpers/jade/mixins.jade +++ b/modules/web-console/frontend/app/helpers/jade/mixins.jade @@ -183,6 +183,14 @@ mixin text-enabled(lbl, model, name, enabled, required, placeholder, tip) if block block +//- Mixin for text field with autofocus. +mixin text-enabled-autofocus(lbl, model, name, enabled, required, placeholder, tip) + +ignite-form-field-text(lbl, model, name, enabledToDisabled(enabled), required, placeholder, tip)( + data-ignite-form-field-input-autofocus='true' + ) + if block + block + //- Mixin for text field. mixin text(lbl, model, name, required, placeholder, tip) +ignite-form-field-text(lbl, model, name, false, required, placeholder, tip) @@ -221,12 +229,28 @@ mixin dropdown-required-empty(lbl, model, name, enabled, required, placeholder, if block block +//- Mixin for required dropdown field with autofocus. +mixin dropdown-required-empty-autofocus(lbl, model, name, enabled, required, placeholder, placeholderEmpty, options, tip) + +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), required, false, placeholder, placeholderEmpty, options, tip)( + data-ignite-form-field-input-autofocus='true' + ) + if block + block + //- Mixin for required dropdown field. mixin dropdown-required(lbl, model, name, enabled, required, placeholder, options, tip) +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), required, false, placeholder, '', options, tip) if block block +//- Mixin for required dropdown field with autofocus. +mixin dropdown-required-autofocus(lbl, model, name, enabled, required, placeholder, options, tip) + +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), required, false, placeholder, '', options, tip)( + data-ignite-form-field-input-autofocus='true' + ) + if block + block + //- Mixin for dropdown field. mixin dropdown(lbl, model, name, enabled, placeholder, options, tip) +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), false, false, placeholder, '', options, tip) @@ -324,6 +348,28 @@ mixin table-java-package-field(name, model, items, valid, save, newItem) ignite-on-escape=onEscape ) +//- Mixin for table java package field. +mixin table-url-field(name, model, items, valid, save, newItem) + -var resetOnEnter = newItem ? '(stopblur = true) && (group.add = [{}])' : '(field.edit = false)' + -var onEnter = valid + ' && (' + save + '); ' + valid + ' && ' + resetOnEnter + ';' + + -var onEscape = newItem ? 'group.add = []' : 'field.edit = false' + + -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false' + -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';' + + div(ignite-on-focus-out=onBlur) + if block + block + + .input-tip + +ignite-form-field-url-input(name, model, false, 'true', 'Enter URL')( + data-ignite-unique=items + data-ignite-form-field-input-autofocus='true' + + ignite-on-enter=onEnter + ignite-on-escape=onEscape + ) //- Mixin for table address field. mixin table-address-field(name, model, items, valid, save, newItem, portRange) @@ -393,17 +439,17 @@ mixin table-save-button(valid, save, newItem) ) //- Mixin for table remove button. -mixin table-remove-conditional-button(items, show, tip) +mixin table-remove-conditional-button(items, show, tip, row) i.tipField.fa.fa-remove( ng-hide='!#{show} || field.edit' bs-tooltip data-title=tip - ng-click='#{items}.splice(#{items}.indexOf(model), 1)' + ng-click='#{items}.splice(#{items}.indexOf(#{row}), 1)' ) //- Mixin for table remove button. mixin table-remove-button(items, tip) - +table-remove-conditional-button(items, 'true', tip) + +table-remove-conditional-button(items, 'true', tip, 'model') //- Mixin for cache mode. mixin cacheMode(lbl, model, name, placeholder) diff --git a/modules/web-console/frontend/app/modules/Demo/Demo.module.js b/modules/web-console/frontend/app/modules/Demo/Demo.module.js index 83d55ed596da7..a3700ca3ed141 100644 --- a/modules/web-console/frontend/app/modules/Demo/Demo.module.js +++ b/modules/web-console/frontend/app/modules/Demo/Demo.module.js @@ -41,11 +41,11 @@ angular url: '/demo/reset', controller: ['$state', '$http', 'IgniteMessages', ($state, $http, Messages) => { $http.post('/api/v1/demo/reset') - .success(() => $state.go('base.configuration.clusters')) - .error((err) => { + .then(() => $state.go('base.configuration.clusters')) + .catch((res) => { $state.go('base.configuration.clusters'); - Messages.showError(err); + Messages.showError(res); }); }], metaTags: {} diff --git a/modules/web-console/frontend/app/modules/configuration/Version.service.js b/modules/web-console/frontend/app/modules/configuration/Version.service.js index 06efdda42b83b..f0e9c4c2a2a89 100644 --- a/modules/web-console/frontend/app/modules/configuration/Version.service.js +++ b/modules/web-console/frontend/app/modules/configuration/Version.service.js @@ -22,7 +22,7 @@ const VERSION_MATCHER = /(\d+)\.(\d+)\.(\d+)([-.]([^0123456789][^-]+)(-SNAPSHOT) const numberComparator = (a, b) => a > b ? 1 : a < b ? -1 : 0; -export default class Version { +export default class IgniteVersion { /** * Tries to parse product version from it's string representation. * @@ -70,7 +70,7 @@ export default class Version { if (res !== 0) return res; - return numberComparator(pa.revTs, pb.maintenance); + return numberComparator(pa.revTs, pb.revTs); } /** @@ -79,7 +79,7 @@ export default class Version { */ productVersion() { return { - ignite: '1.7.0' + ignite: '1.8.0' }; } diff --git a/modules/web-console/frontend/app/modules/configuration/configuration.module.js b/modules/web-console/frontend/app/modules/configuration/configuration.module.js index 27f7befbeb4f5..4288ff79ea182 100644 --- a/modules/web-console/frontend/app/modules/configuration/configuration.module.js +++ b/modules/web-console/frontend/app/modules/configuration/configuration.module.js @@ -17,26 +17,28 @@ import angular from 'angular'; -import igniteEventGroups from './EventGroups.provider'; + import igniteSidebar from './Sidebar.provider'; -import Version from './Version.service'; +import IgniteVersion from './Version.service'; -import clusterDefaults from './generator/defaults/cluster.provider'; -import clusterPlatformDefaults from './generator/defaults/cluster.platform.provider'; -import cacheDefaults from './generator/defaults/cache.provider'; -import cachePlatformDefaults from './generator/defaults/cache.platform.provider'; -import igfsDefaults from './generator/defaults/igfs.provider'; +import IgniteClusterDefaults from './generator/defaults/Cluster.service'; +import IgniteClusterPlatformDefaults from './generator/defaults/Cluster.platform.service'; +import IgniteCacheDefaults from './generator/defaults/Cache.service'; +import IgniteCachePlatformDefaults from './generator/defaults/Cache.platform.service'; +import IgniteIGFSDefaults from './generator/defaults/IGFS.service'; +import IgniteEventGroups from './generator/defaults/Event-groups.service'; -import ConfigurationGenerator from './generator/ConfigurationGenerator'; -import PlatformGenerator from './generator/PlatformGenerator'; +import IgniteConfigurationGenerator from './generator/ConfigurationGenerator'; +import IgnitePlatformGenerator from './generator/PlatformGenerator'; -import SpringTransformer from './generator/SpringTransformer.service'; -import JavaTransformer from './generator/JavaTransformer.service'; +import IgniteSpringTransformer from './generator/SpringTransformer.service'; +import IgniteJavaTransformer from './generator/JavaTransformer.service'; import SharpTransformer from './generator/SharpTransformer.service'; -import GeneratorDocker from './generator/Docker.service'; -import GeneratorPom from './generator/Pom.service'; -import GeneratorProperties from './generator/Properties.service'; -import GeneratorReadme from './generator/Readme.service'; +import IgniteDockerGenerator from './generator/Docker.service'; +import IgniteMavenGenerator from './generator/Maven.service'; +import IgniteGeneratorProperties from './generator/Properties.service'; +import IgniteReadmeGenerator from './generator/Readme.service'; +import IgniteCustomGenerator from './generator/Custom.service'; import igniteSidebarDirective from './sidebar.directive'; @@ -45,21 +47,22 @@ angular .module('ignite-console.configuration', [ ]) -.provider('igniteClusterDefaults', clusterDefaults) -.provider('igniteClusterPlatformDefaults', clusterPlatformDefaults) -.provider('igniteCacheDefaults', cacheDefaults) -.provider('igniteCachePlatformDefaults', cachePlatformDefaults) -.provider('igniteIgfsDefaults', igfsDefaults) -.provider(...igniteEventGroups) .provider(...igniteSidebar) .directive(...igniteSidebarDirective) -.service('IgniteVersion', Version) -.service('IgniteConfigurationGenerator', ConfigurationGenerator) -.service('IgnitePlatformGenerator', PlatformGenerator) -.service('SpringTransformer', SpringTransformer) -.service('JavaTransformer', JavaTransformer) +.service('IgniteConfigurationGenerator', IgniteConfigurationGenerator) +.service('IgnitePlatformGenerator', IgnitePlatformGenerator) +.service('SpringTransformer', IgniteSpringTransformer) +.service('JavaTransformer', IgniteJavaTransformer) .service('IgniteSharpTransformer', SharpTransformer) -.service('IgnitePropertiesGenerator', GeneratorProperties) -.service('IgniteReadmeGenerator', GeneratorReadme) -.service(...GeneratorDocker) -.service(...GeneratorPom); +.service('IgniteVersion', IgniteVersion) +.service('IgniteEventGroups', IgniteEventGroups) +.service('IgniteClusterDefaults', IgniteClusterDefaults) +.service('IgniteClusterPlatformDefaults', IgniteClusterPlatformDefaults) +.service('IgniteCacheDefaults', IgniteCacheDefaults) +.service('IgniteCachePlatformDefaults', IgniteCachePlatformDefaults) +.service('IgniteIGFSDefaults', IgniteIGFSDefaults) +.service('IgnitePropertiesGenerator', IgniteGeneratorProperties) +.service('IgniteReadmeGenerator', IgniteReadmeGenerator) +.service('IgniteDockerGenerator', IgniteDockerGenerator) +.service('IgniteMavenGenerator', IgniteMavenGenerator) +.service('IgniteCustomGenerator', IgniteCustomGenerator); diff --git a/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js b/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js index 6244a531ea85b..f5afe59805114 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js @@ -17,7 +17,24 @@ import StringBuilder from './StringBuilder'; +import IgniteConfigurationGenerator from './ConfigurationGenerator'; +import IgniteEventGroups from './defaults/Event-groups.service'; + +import IgniteClusterDefaults from './defaults/Cluster.service'; +import IgniteCacheDefaults from './defaults/Cache.service'; +import IgniteIGFSDefaults from './defaults/IGFS.service'; + +import JavaTypes from '../../../services/JavaTypes.service'; + +const clusterDflts = new IgniteClusterDefaults(); +const cacheDflts = new IgniteCacheDefaults(); +const igfsDflts = new IgniteIGFSDefaults(); + export default class AbstractTransformer { + static generator = IgniteConfigurationGenerator; + static javaTypes = new JavaTypes(clusterDflts, cacheDflts, igfsDflts); + static eventGroups = new IgniteEventGroups(); + // Append comment with time stamp. static mainComment(sb, ...lines) { lines.push(sb.generatedBy()); diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Beans.js b/modules/web-console/frontend/app/modules/configuration/generator/Beans.js index 2750626f4a74b..ca1934282510e 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/Beans.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/Beans.js @@ -17,6 +17,11 @@ import _ from 'lodash'; +_.mixin({ + nonNil: _.negate(_.isNil), + nonEmpty: _.negate(_.isEmpty) +}); + export class EmptyBean { /** * @param {String} clsName diff --git a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js index 5887832ff81df..8770bf6063c73 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js @@ -19,1776 +19,1825 @@ import DFLT_DIALECTS from 'app/data/dialects.json'; import { EmptyBean, Bean } from './Beans'; -export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'igniteIgfsDefaults', (JavaTypes, clusterDflts, cacheDflts, igfsDflts) => { - class ConfigurationGenerator { - static igniteConfigurationBean(cluster) { - return new Bean('org.apache.ignite.configuration.IgniteConfiguration', 'cfg', cluster, clusterDflts); - } +import IgniteClusterDefaults from './defaults/Cluster.service'; +import IgniteCacheDefaults from './defaults/Cache.service'; +import IgniteIGFSDefaults from './defaults/IGFS.service'; - static igfsConfigurationBean(igfs) { - return new Bean('org.apache.ignite.configuration.FileSystemConfiguration', 'igfs', igfs, igfsDflts); - } +import JavaTypes from '../../../services/JavaTypes.service'; - static cacheConfigurationBean(cache) { - return new Bean('org.apache.ignite.configuration.CacheConfiguration', 'ccfg', cache, cacheDflts); - } +const clusterDflts = new IgniteClusterDefaults(); +const cacheDflts = new IgniteCacheDefaults(); +const igfsDflts = new IgniteIGFSDefaults(); - static domainConfigurationBean(domain) { - return new Bean('org.apache.ignite.cache.QueryEntity', 'qryEntity', domain, cacheDflts); - } +const javaTypes = new JavaTypes(clusterDflts, cacheDflts, igfsDflts); - static discoveryConfigurationBean(discovery) { - return new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery', discovery, clusterDflts.discovery); - } +export default class IgniteConfigurationGenerator { + static igniteConfigurationBean(cluster) { + return new Bean('org.apache.ignite.configuration.IgniteConfiguration', 'cfg', cluster, clusterDflts); + } - /** - * Function to generate ignite configuration. - * - * @param {Object} cluster Cluster to process. - * @param {Boolean} client - * @return {Bean} Generated ignite configuration. - */ - static igniteConfiguration(cluster, client) { - const cfg = this.igniteConfigurationBean(cluster); - - this.clusterGeneral(cluster, cfg, client); - this.clusterAtomics(cluster.atomicConfiguration, cfg); - this.clusterBinary(cluster.binaryConfiguration, cfg); - this.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, cfg); - this.clusterCheckpoint(cluster, cluster.caches, cfg); - this.clusterCollision(cluster.collision, cfg); - this.clusterCommunication(cluster, cfg); - this.clusterConnector(cluster.connector, cfg); - this.clusterDeployment(cluster, cfg); - this.clusterEvents(cluster, cfg); - this.clusterFailover(cluster, cfg); - this.clusterLoadBalancing(cluster, cfg); - this.clusterLogger(cluster.logger, cfg); - this.clusterODBC(cluster.odbc, cfg); - this.clusterMarshaller(cluster, cfg); - this.clusterMetrics(cluster, cfg); - this.clusterSwap(cluster, cfg); - this.clusterTime(cluster, cfg); - this.clusterPools(cluster, cfg); - this.clusterTransactions(cluster.transactionConfiguration, cfg); - this.clusterSsl(cluster, cfg); - this.clusterUserAttributes(cluster, cfg); - - this.clusterCaches(cluster, cluster.caches, cluster.igfss, client, cfg); - - if (!client) - this.clusterIgfss(cluster.igfss, cfg); + static igfsConfigurationBean(igfs) { + return new Bean('org.apache.ignite.configuration.FileSystemConfiguration', 'igfs', igfs, igfsDflts); + } - return cfg; - } + static cacheConfigurationBean(cache) { + return new Bean('org.apache.ignite.configuration.CacheConfiguration', 'ccfg', cache, cacheDflts); + } - static dialectClsName(dialect) { - return DFLT_DIALECTS[dialect] || 'Unknown database: ' + (dialect || 'Choose JDBC dialect'); - } + static domainConfigurationBean(domain) { + return new Bean('org.apache.ignite.cache.QueryEntity', 'qryEntity', domain, cacheDflts); + } - static dataSourceBean(id, dialect) { - let dsBean; + static discoveryConfigurationBean(discovery) { + return new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery', discovery, clusterDflts.discovery); + } - switch (dialect) { - case 'Generic': - dsBean = new Bean('com.mchange.v2.c3p0.ComboPooledDataSource', id, {}) - .property('jdbcUrl', `${id}.jdbc.url`, 'jdbc:your_database'); + /** + * Function to generate ignite configuration. + * + * @param {Object} cluster Cluster to process. + * @param {Boolean} client + * @return {Bean} Generated ignite configuration. + */ + static igniteConfiguration(cluster, client) { + const cfg = this.igniteConfigurationBean(cluster); + + this.clusterGeneral(cluster, cfg, client); + this.clusterAtomics(cluster.atomicConfiguration, cfg); + this.clusterBinary(cluster.binaryConfiguration, cfg); + this.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, cfg); + this.clusterCheckpoint(cluster, cluster.caches, cfg); + this.clusterCollision(cluster.collision, cfg); + this.clusterCommunication(cluster, cfg); + this.clusterConnector(cluster.connector, cfg); + this.clusterDeployment(cluster, cfg); + this.clusterEvents(cluster, cfg); + this.clusterFailover(cluster, cfg); + this.clusterLoadBalancing(cluster, cfg); + this.clusterLogger(cluster.logger, cfg); + this.clusterODBC(cluster.odbc, cfg); + this.clusterMarshaller(cluster, cfg); + this.clusterMetrics(cluster, cfg); + this.clusterSwap(cluster, cfg); + this.clusterTime(cluster, cfg); + this.clusterPools(cluster, cfg); + this.clusterTransactions(cluster.transactionConfiguration, cfg); + this.clusterSsl(cluster, cfg); + this.clusterUserAttributes(cluster, cfg); + + this.clusterCaches(cluster, cluster.caches, cluster.igfss, client, cfg); + + if (!client) + this.clusterIgfss(cluster.igfss, cfg); + + return cfg; + } - break; - case 'Oracle': - dsBean = new Bean('oracle.jdbc.pool.OracleDataSource', id, {}) - .property('URL', `${id}.jdbc.url`, 'jdbc:oracle:thin:@[host]:[port]:[database]'); + static dialectClsName(dialect) { + return DFLT_DIALECTS[dialect] || 'Unknown database: ' + (dialect || 'Choose JDBC dialect'); + } - break; - case 'DB2': - dsBean = new Bean('com.ibm.db2.jcc.DB2DataSource', id, {}) - .property('serverName', `${id}.jdbc.server_name`, 'YOUR_DATABASE_SERVER_NAME') - .propertyInt('portNumber', `${id}.jdbc.port_number`, 'YOUR_JDBC_PORT_NUMBER') - .property('databaseName', `${id}.jdbc.database_name`, 'YOUR_DATABASE_NAME') - .propertyInt('driverType', `${id}.jdbc.driver_type`, 'YOUR_JDBC_DRIVER_TYPE'); + static dataSourceBean(id, dialect) { + let dsBean; + + switch (dialect) { + case 'Generic': + dsBean = new Bean('com.mchange.v2.c3p0.ComboPooledDataSource', id, {}) + .property('jdbcUrl', `${id}.jdbc.url`, 'jdbc:your_database'); + + break; + case 'Oracle': + dsBean = new Bean('oracle.jdbc.pool.OracleDataSource', id, {}) + .property('URL', `${id}.jdbc.url`, 'jdbc:oracle:thin:@[host]:[port]:[database]'); + + break; + case 'DB2': + dsBean = new Bean('com.ibm.db2.jcc.DB2DataSource', id, {}) + .property('serverName', `${id}.jdbc.server_name`, 'YOUR_DATABASE_SERVER_NAME') + .propertyInt('portNumber', `${id}.jdbc.port_number`, 'YOUR_JDBC_PORT_NUMBER') + .property('databaseName', `${id}.jdbc.database_name`, 'YOUR_DATABASE_NAME') + .propertyInt('driverType', `${id}.jdbc.driver_type`, 'YOUR_JDBC_DRIVER_TYPE'); + + break; + case 'SQLServer': + dsBean = new Bean('com.microsoft.sqlserver.jdbc.SQLServerDataSource', id, {}) + .property('URL', `${id}.jdbc.url`, 'jdbc:sqlserver://[host]:[port][;databaseName=database]'); + + break; + case 'MySQL': + dsBean = new Bean('com.mysql.jdbc.jdbc2.optional.MysqlDataSource', id, {}) + .property('URL', `${id}.jdbc.url`, 'jdbc:mysql://[host]:[port]/[database]'); + + break; + case 'PostgreSQL': + dsBean = new Bean('org.postgresql.ds.PGPoolingDataSource', id, {}) + .property('url', `${id}.jdbc.url`, 'jdbc:postgresql://[host]:[port]/[database]'); + + break; + case 'H2': + dsBean = new Bean('org.h2.jdbcx.JdbcDataSource', id, {}) + .property('URL', `${id}.jdbc.url`, 'jdbc:h2:tcp://[host]/[database]'); + + break; + default: + } - break; - case 'SQLServer': - dsBean = new Bean('com.microsoft.sqlserver.jdbc.SQLServerDataSource', id, {}) - .property('URL', `${id}.jdbc.url`, 'jdbc:sqlserver://[host]:[port][;databaseName=database]'); + if (dsBean) { + dsBean.property('user', `${id}.jdbc.username`, 'YOUR_USER_NAME') + .property('password', `${id}.jdbc.password`, 'YOUR_PASSWORD'); + } - break; - case 'MySQL': - dsBean = new Bean('com.mysql.jdbc.jdbc2.optional.MysqlDataSource', id, {}) - .property('URL', `${id}.jdbc.url`, 'jdbc:mysql://[host]:[port]/[database]'); + return dsBean; + } - break; - case 'PostgreSQL': - dsBean = new Bean('org.postgresql.ds.PGPoolingDataSource', id, {}) - .property('url', `${id}.jdbc.url`, 'jdbc:postgresql://[host]:[port]/[database]'); + // Generate general section. + static clusterGeneral(cluster, cfg = this.igniteConfigurationBean(cluster), client = false) { + if (client) + cfg.prop('boolean', 'clientMode', true); - break; - case 'H2': - dsBean = new Bean('org.h2.jdbcx.JdbcDataSource', id, {}) - .property('URL', `${id}.jdbc.url`, 'jdbc:h2:tcp://[host]/[database]'); + cfg.stringProperty('name', 'gridName') + .stringProperty('localHost'); - break; - default: - } - - if (dsBean) { - dsBean.property('user', `${id}.jdbc.username`, 'YOUR_USER_NAME') - .property('password', `${id}.jdbc.password`, 'YOUR_PASSWORD'); - } + if (_.isNil(cluster.discovery)) + return cfg; - return dsBean; - } + const discovery = new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery', + cluster.discovery, clusterDflts.discovery); - // Generate general section. - static clusterGeneral(cluster, cfg = this.igniteConfigurationBean(cluster), client = false) { - if (client) - cfg.prop('boolean', 'clientMode', true); + let ipFinder; - cfg.stringProperty('name', 'gridName') - .stringProperty('localHost'); + switch (discovery.valueOf('kind')) { + case 'Vm': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder', + 'ipFinder', cluster.discovery.Vm, clusterDflts.discovery.Vm); - if (_.isNil(cluster.discovery)) - return cfg; + ipFinder.collectionProperty('addrs', 'addresses', cluster.discovery.Vm.addresses); - const discovery = new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery', - cluster.discovery, clusterDflts.discovery); + break; + case 'Multicast': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder', + 'ipFinder', cluster.discovery.Multicast, clusterDflts.discovery.Multicast); - let ipFinder; + ipFinder.stringProperty('multicastGroup') + .intProperty('multicastPort') + .intProperty('responseWaitTime') + .intProperty('addressRequestAttempts') + .stringProperty('localAddress') + .collectionProperty('addrs', 'addresses', cluster.discovery.Multicast.addresses); - switch (discovery.valueOf('kind')) { - case 'Vm': - ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder', - 'ipFinder', cluster.discovery.Vm, clusterDflts.discovery.Vm); + break; + case 'S3': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinder', + 'ipFinder', cluster.discovery.S3, clusterDflts.discovery.S3); - ipFinder.collectionProperty('addrs', 'addresses', cluster.discovery.Vm.addresses); + ipFinder.stringProperty('bucketName'); - break; - case 'Multicast': - ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder', - 'ipFinder', cluster.discovery.Multicast, clusterDflts.discovery.Multicast); + break; + case 'Cloud': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinder', + 'ipFinder', cluster.discovery.Cloud, clusterDflts.discovery.Cloud); - ipFinder.stringProperty('multicastGroup') - .intProperty('multicastPort') - .intProperty('responseWaitTime') - .intProperty('addressRequestAttempts') - .stringProperty('localAddress') - .collectionProperty('addrs', 'addresses', cluster.discovery.Multicast.addresses); + ipFinder.stringProperty('credential') + .pathProperty('credentialPath') + .stringProperty('identity') + .stringProperty('provider') + .collectionProperty('regions', 'regions', cluster.discovery.Cloud.regions) + .collectionProperty('zones', 'zones', cluster.discovery.Cloud.zones); - break; - case 'S3': - ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinder', - 'ipFinder', cluster.discovery.S3, clusterDflts.discovery.S3); + break; + case 'GoogleStorage': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.gce.TcpDiscoveryGoogleStorageIpFinder', + 'ipFinder', cluster.discovery.GoogleStorage, clusterDflts.discovery.GoogleStorage); - ipFinder.stringProperty('bucketName'); + ipFinder.stringProperty('projectName') + .stringProperty('bucketName') + .pathProperty('serviceAccountP12FilePath') + .stringProperty('serviceAccountId'); - break; - case 'Cloud': - ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinder', - 'ipFinder', cluster.discovery.Cloud, clusterDflts.discovery.Cloud); + break; + case 'Jdbc': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder', + 'ipFinder', cluster.discovery.Jdbc, clusterDflts.discovery.Jdbc); - ipFinder.stringProperty('credential') - .pathProperty('credentialPath') - .stringProperty('identity') - .stringProperty('provider') - .collectionProperty('regions', 'regions', cluster.discovery.Cloud.regions) - .collectionProperty('zones', 'zones', cluster.discovery.Cloud.zones); + ipFinder.intProperty('initSchema'); - break; - case 'GoogleStorage': - ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.gce.TcpDiscoveryGoogleStorageIpFinder', - 'ipFinder', cluster.discovery.GoogleStorage, clusterDflts.discovery.GoogleStorage); + if (ipFinder.includes('dataSourceBean', 'dialect')) { + const id = ipFinder.valueOf('dataSourceBean'); - ipFinder.stringProperty('projectName') - .stringProperty('bucketName') - .pathProperty('serviceAccountP12FilePath') - .stringProperty('serviceAccountId'); + ipFinder.dataSource(id, 'dataSource', this.dataSourceBean(id, ipFinder.valueOf('dialect'))); + } - break; - case 'Jdbc': - ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder', - 'ipFinder', cluster.discovery.Jdbc, clusterDflts.discovery.Jdbc); + break; + case 'SharedFs': + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder', + 'ipFinder', cluster.discovery.SharedFs, clusterDflts.discovery.SharedFs); + + ipFinder.pathProperty('path'); + + break; + case 'ZooKeeper': + const src = cluster.discovery.ZooKeeper; + const dflt = clusterDflts.discovery.ZooKeeper; + + ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.zk.TcpDiscoveryZookeeperIpFinder', + 'ipFinder', src, dflt); + + ipFinder.emptyBeanProperty('curator') + .stringProperty('zkConnectionString'); + + const kind = _.get(src, 'retryPolicy.kind'); + + if (kind) { + const policy = src.retryPolicy; + + let retryPolicyBean; + + switch (kind) { + case 'ExponentialBackoff': + retryPolicyBean = new Bean('org.apache.curator.retry.ExponentialBackoffRetry', null, + policy.ExponentialBackoff, dflt.ExponentialBackoff) + .intConstructorArgument('baseSleepTimeMs') + .intConstructorArgument('maxRetries') + .intConstructorArgument('maxSleepMs'); + + break; + case 'BoundedExponentialBackoff': + retryPolicyBean = new Bean('org.apache.curator.retry.BoundedExponentialBackoffRetry', + null, policy.BoundedExponentialBackoff, dflt.BoundedExponentialBackoffRetry) + .intConstructorArgument('baseSleepTimeMs') + .intConstructorArgument('maxSleepTimeMs') + .intConstructorArgument('maxRetries'); + + break; + case 'UntilElapsed': + retryPolicyBean = new Bean('org.apache.curator.retry.RetryUntilElapsed', null, + policy.UntilElapsed, dflt.UntilElapsed) + .intConstructorArgument('maxElapsedTimeMs') + .intConstructorArgument('sleepMsBetweenRetries'); + + break; + + case 'NTimes': + retryPolicyBean = new Bean('org.apache.curator.retry.RetryNTimes', null, + policy.NTimes, dflt.NTimes) + .intConstructorArgument('n') + .intConstructorArgument('sleepMsBetweenRetries'); + + break; + case 'OneTime': + retryPolicyBean = new Bean('org.apache.curator.retry.RetryOneTime', null, + policy.OneTime, dflt.OneTime) + .intConstructorArgument('sleepMsBetweenRetry'); + + break; + case 'Forever': + retryPolicyBean = new Bean('org.apache.curator.retry.RetryForever', null, + policy.Forever, dflt.Forever) + .intConstructorArgument('retryIntervalMs'); + + break; + case 'Custom': + const className = _.get(policy, 'Custom.className'); + + if (_.nonEmpty(className)) + retryPolicyBean = new EmptyBean(className); + + break; + default: + // No-op. + } - ipFinder.intProperty('initSchema'); + if (retryPolicyBean) + ipFinder.beanProperty('retryPolicy', retryPolicyBean); + } - if (ipFinder.includes('dataSourceBean', 'dialect')) { - const id = ipFinder.valueOf('dataSourceBean'); + ipFinder.pathProperty('basePath', '/services') + .stringProperty('serviceName') + .boolProperty('allowDuplicateRegistrations'); - ipFinder.dataSource(id, 'dataSource', this.dataSourceBean(id, ipFinder.valueOf('dialect'))); - } + break; + default: + // No-op. + } - break; - case 'SharedFs': - ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder', - 'ipFinder', cluster.discovery.SharedFs, clusterDflts.discovery.SharedFs); + if (ipFinder) + discovery.beanProperty('ipFinder', ipFinder); - ipFinder.pathProperty('path'); + this.clusterDiscovery(cluster.discovery, cfg, discovery); - break; - case 'ZooKeeper': - const src = cluster.discovery.ZooKeeper; - const dflt = clusterDflts.discovery.ZooKeeper; + return cfg; + } - ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.zk.TcpDiscoveryZookeeperIpFinder', - 'ipFinder', src, dflt); + static igfsDataCache(igfs) { + return this.cacheConfiguration({ + name: igfs.name + '-data', + cacheMode: 'PARTITIONED', + atomicityMode: 'TRANSACTIONAL', + writeSynchronizationMode: 'FULL_SYNC', + backups: 0, + igfsAffinnityGroupSize: igfs.affinnityGroupSize || 512 + }); + } - ipFinder.emptyBeanProperty('curator') - .stringProperty('zkConnectionString'); + static igfsMetaCache(igfs) { + return this.cacheConfiguration({ + name: igfs.name + '-meta', + cacheMode: 'REPLICATED', + atomicityMode: 'TRANSACTIONAL', + writeSynchronizationMode: 'FULL_SYNC' + }); + } - const kind = _.get(src, 'retryPolicy.kind'); + static clusterCaches(cluster, caches, igfss, client, cfg = this.igniteConfigurationBean(cluster)) { + const ccfgs = _.map(caches, (cache) => this.cacheConfiguration(cache)); - if (kind) { - const policy = src.retryPolicy; + if (!client) { + _.forEach(igfss, (igfs) => { + ccfgs.push(this.igfsDataCache(igfs)); + ccfgs.push(this.igfsMetaCache(igfs)); + }); + } - let retryPolicyBean; + cfg.varArgProperty('ccfgs', 'cacheConfiguration', ccfgs, 'org.apache.ignite.configuration.CacheConfiguration'); - switch (kind) { - case 'ExponentialBackoff': - retryPolicyBean = new Bean('org.apache.curator.retry.ExponentialBackoffRetry', null, - policy.ExponentialBackoff, dflt.ExponentialBackoff) - .intConstructorArgument('baseSleepTimeMs') - .intConstructorArgument('maxRetries') - .intConstructorArgument('maxSleepMs'); + return cfg; + } - break; - case 'BoundedExponentialBackoff': - retryPolicyBean = new Bean('org.apache.curator.retry.BoundedExponentialBackoffRetry', - null, policy.BoundedExponentialBackoffRetry, dflt.BoundedExponentialBackoffRetry) - .intConstructorArgument('baseSleepTimeMs') - .intConstructorArgument('maxSleepTimeMs') - .intConstructorArgument('maxRetries'); + // Generate atomics group. + static clusterAtomics(atomics, cfg = this.igniteConfigurationBean()) { + const acfg = new Bean('org.apache.ignite.configuration.AtomicConfiguration', 'atomicCfg', + atomics, clusterDflts.atomics); - break; - case 'UntilElapsed': - retryPolicyBean = new Bean('org.apache.curator.retry.RetryUntilElapsed', null, - policy.UntilElapsed, dflt.UntilElapsed) - .intConstructorArgument('maxElapsedTimeMs') - .intConstructorArgument('sleepMsBetweenRetries'); + acfg.enumProperty('cacheMode') + .intProperty('atomicSequenceReserveSize'); - break; + if (acfg.valueOf('cacheMode') === 'PARTITIONED') + acfg.intProperty('backups'); - case 'NTimes': - retryPolicyBean = new Bean('org.apache.curator.retry.RetryNTimes', null, - policy.NTimes, dflt.NTimes) - .intConstructorArgument('n') - .intConstructorArgument('sleepMsBetweenRetries'); + if (acfg.isEmpty()) + return cfg; - break; - case 'OneTime': - retryPolicyBean = new Bean('org.apache.curator.retry.RetryOneTime', null, - policy.OneTime, dflt.OneTime) - .intConstructorArgument('sleepMsBetweenRetry'); + cfg.beanProperty('atomicConfiguration', acfg); - break; - case 'Forever': - retryPolicyBean = new Bean('org.apache.curator.retry.RetryForever', null, - policy.Forever, dflt.Forever) - .intConstructorArgument('retryIntervalMs'); + return cfg; + } - break; - case 'Custom': - if (_.nonEmpty(policy.Custom.className)) - retryPolicyBean = new EmptyBean(policy.Custom.className); + // Generate binary group. + static clusterBinary(binary, cfg = this.igniteConfigurationBean()) { + const binaryCfg = new Bean('org.apache.ignite.configuration.BinaryConfiguration', 'binaryCfg', + binary, clusterDflts.binary); - break; - default: - // No-op. - } + binaryCfg.emptyBeanProperty('idMapper') + .emptyBeanProperty('nameMapper') + .emptyBeanProperty('serializer'); - if (retryPolicyBean) - ipFinder.beanProperty('retryPolicy', retryPolicyBean); - } + const typeCfgs = []; - ipFinder.pathProperty('basePath', '/services') - .stringProperty('serviceName') - .boolProperty('allowDuplicateRegistrations'); + _.forEach(binary.typeConfigurations, (type) => { + const typeCfg = new Bean('org.apache.ignite.binary.BinaryTypeConfiguration', + javaTypes.toJavaName('binaryType', type.typeName), type, clusterDflts.binary.typeConfigurations); - break; - default: - // No-op. - } + typeCfg.stringProperty('typeName') + .emptyBeanProperty('idMapper') + .emptyBeanProperty('nameMapper') + .emptyBeanProperty('serializer') + .intProperty('enum'); - if (ipFinder) - discovery.beanProperty('ipFinder', ipFinder); + if (typeCfg.nonEmpty()) + typeCfgs.push(typeCfg); + }); - this.clusterDiscovery(cluster.discovery, cfg, discovery); + binaryCfg.collectionProperty('types', 'typeConfigurations', typeCfgs, 'org.apache.ignite.binary.BinaryTypeConfiguration') + .boolProperty('compactFooter'); + if (binaryCfg.isEmpty()) return cfg; - } - - static igfsDataCache(igfs) { - return this.cacheConfiguration({ - name: igfs.name + '-data', - cacheMode: 'PARTITIONED', - atomicityMode: 'TRANSACTIONAL', - writeSynchronizationMode: 'FULL_SYNC', - backups: 0, - igfsAffinnityGroupSize: igfs.affinnityGroupSize || 512 - }); - } - static igfsMetaCache(igfs) { - return this.cacheConfiguration({ - name: igfs.name + '-meta', - cacheMode: 'REPLICATED', - atomicityMode: 'TRANSACTIONAL', - writeSynchronizationMode: 'FULL_SYNC' - }); - } + cfg.beanProperty('binaryConfiguration', binaryCfg); - static clusterCaches(cluster, caches, igfss, client, cfg = this.igniteConfigurationBean(cluster)) { - const ccfgs = _.map(caches, (cache) => this.cacheConfiguration(cache)); + return cfg; + } - if (!client) { - _.forEach(igfss, (igfs) => { - ccfgs.push(this.igfsDataCache(igfs)); - ccfgs.push(this.igfsMetaCache(igfs)); - }); + // Generate cache key configurations. + static clusterCacheKeyConfiguration(keyCfgs, cfg = this.igniteConfigurationBean()) { + const items = _.reduce(keyCfgs, (acc, keyCfg) => { + if (keyCfg.typeName && keyCfg.affinityKeyFieldName) { + acc.push(new Bean('org.apache.ignite.cache.CacheKeyConfiguration', null, keyCfg) + .stringConstructorArgument('typeName') + .stringConstructorArgument('affinityKeyFieldName')); } - cfg.varArgProperty('ccfgs', 'cacheConfiguration', ccfgs, 'org.apache.ignite.configuration.CacheConfiguration'); + return acc; + }, []); + if (_.isEmpty(items)) return cfg; - } - // Generate atomics group. - static clusterAtomics(atomics, cfg = this.igniteConfigurationBean()) { - const acfg = new Bean('org.apache.ignite.configuration.AtomicConfiguration', 'atomicCfg', - atomics, clusterDflts.atomics); + cfg.arrayProperty('cacheKeyConfiguration', 'cacheKeyConfiguration', items, + 'org.apache.ignite.cache.CacheKeyConfiguration'); - acfg.enumProperty('cacheMode') - .intProperty('atomicSequenceReserveSize'); + return cfg; + } - if (acfg.valueOf('cacheMode') === 'PARTITIONED') - acfg.intProperty('backups'); + // Generate checkpoint configurations. + static clusterCheckpoint(cluster, caches, cfg = this.igniteConfigurationBean()) { + const cfgs = _.filter(_.map(cluster.checkpointSpi, (spi) => { + switch (_.get(spi, 'kind')) { + case 'FS': + const fsBean = new Bean('org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi', + 'checkpointSpiFs', spi.FS); - if (acfg.isEmpty()) - return cfg; + fsBean.collectionProperty('directoryPaths', 'directoryPaths', _.get(spi, 'FS.directoryPaths')) + .emptyBeanProperty('checkpointListener'); - cfg.beanProperty('atomicConfiguration', acfg); + return fsBean; - return cfg; - } + case 'Cache': + const cacheBean = new Bean('org.apache.ignite.spi.checkpoint.cache.CacheCheckpointSpi', + 'checkpointSpiCache', spi.Cache); - // Generate binary group. - static clusterBinary(binary, cfg = this.igniteConfigurationBean()) { - const binaryCfg = new Bean('org.apache.ignite.configuration.BinaryConfiguration', 'binaryCfg', - binary, clusterDflts.binary); + const curCache = _.get(spi, 'Cache.cache'); - binaryCfg.emptyBeanProperty('idMapper') - .emptyBeanProperty('nameMapper') - .emptyBeanProperty('serializer'); + const cache = _.find(caches, (c) => curCache && (c._id === curCache || _.get(c, 'cache._id') === curCache)); - const typeCfgs = []; + if (cache) + cacheBean.prop('java.lang.String', 'cacheName', cache.name || cache.cache.name); - _.forEach(binary.typeConfigurations, (type) => { - const typeCfg = new Bean('org.apache.ignite.binary.BinaryTypeConfiguration', - JavaTypes.toJavaName('binaryType', type.typeName), type, clusterDflts.binary.typeConfigurations); + cacheBean.stringProperty('cacheName') + .emptyBeanProperty('checkpointListener'); - typeCfg.stringProperty('typeName') - .emptyBeanProperty('idMapper') - .emptyBeanProperty('nameMapper') - .emptyBeanProperty('serializer') - .intProperty('enum'); + return cacheBean; - if (typeCfg.nonEmpty()) - typeCfgs.push(typeCfg); - }); + case 'S3': + const s3Bean = new Bean('org.apache.ignite.spi.checkpoint.s3.S3CheckpointSpi', + 'checkpointSpiS3', spi.S3, clusterDflts.checkpointSpi.S3); - binaryCfg.collectionProperty('types', 'typeConfigurations', typeCfgs, 'org.apache.ignite.binary.BinaryTypeConfiguration') - .boolProperty('compactFooter'); + let credentialsBean = null; - if (binaryCfg.isEmpty()) - return cfg; + switch (_.get(spi.S3, 'awsCredentials.kind')) { + case 'Basic': + credentialsBean = new Bean('com.amazonaws.auth.BasicAWSCredentials', 'awsCredentials', {}); - cfg.beanProperty('binaryConfiguration', binaryCfg); + credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey', 'YOUR_S3_ACCESS_KEY') + .propertyConstructorArgument('checkpoint.s3.credentials.secretKey', 'YOUR_S3_SECRET_KEY'); - return cfg; - } + break; - // Generate cache key configurations. - static clusterCacheKeyConfiguration(keyCfgs, cfg = this.igniteConfigurationBean()) { - const items = _.reduce(keyCfgs, (acc, keyCfg) => { - if (keyCfg.typeName && keyCfg.affinityKeyFieldName) { - acc.push(new Bean('org.apache.ignite.cache.CacheKeyConfiguration', null, keyCfg) - .stringConstructorArgument('typeName') - .stringConstructorArgument('affinityKeyFieldName')); - } + case 'Properties': + credentialsBean = new Bean('com.amazonaws.auth.PropertiesCredentials', 'awsCredentials', {}); - return acc; - }, []); + const fileBean = new Bean('java.io.File', '', spi.S3.awsCredentials.Properties) + .pathConstructorArgument('path'); - if (_.isEmpty(items)) - return cfg; + if (fileBean.nonEmpty()) + credentialsBean.beanConstructorArgument('file', fileBean); - cfg.arrayProperty('cacheKeyConfiguration', 'cacheKeyConfiguration', items, - 'org.apache.ignite.cache.CacheKeyConfiguration'); + break; - return cfg; - } + case 'Anonymous': + credentialsBean = new Bean('com.amazonaws.auth.AnonymousAWSCredentials', 'awsCredentials', {}); - // Generate checkpoint configurations. - static clusterCheckpoint(cluster, caches, cfg = this.igniteConfigurationBean()) { - const cfgs = _.filter(_.map(cluster.checkpointSpi, (spi) => { - switch (_.get(spi, 'kind')) { - case 'FS': - const fsBean = new Bean('org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi', - 'checkpointSpiFs', spi.FS); + break; - fsBean.collectionProperty('directoryPaths', 'directoryPaths', _.get(spi, 'FS.directoryPaths')) - .emptyBeanProperty('checkpointListener'); + case 'BasicSession': + credentialsBean = new Bean('com.amazonaws.auth.BasicSessionCredentials', 'awsCredentials', {}); - return fsBean; + // TODO 2054 Arguments in one line is very long string. + credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey') + .propertyConstructorArgument('checkpoint.s3.credentials.secretKey') + .propertyConstructorArgument('checkpoint.s3.credentials.sessionToken'); - case 'Cache': - const cacheBean = new Bean('org.apache.ignite.spi.checkpoint.cache.CacheCheckpointSpi', - 'checkpointSpiCache', spi.Cache); + break; - const curCache = _.get(spi, 'Cache.cache'); + case 'Custom': + const className = _.get(spi.S3.awsCredentials, 'Custom.className'); - const cache = _.find(caches, (c) => curCache && (c._id === curCache || _.get(c, 'cache._id') === curCache)); + if (className) + credentialsBean = new Bean(className, 'awsCredentials', {}); - if (cache) - cacheBean.prop('java.lang.String', 'cacheName', cache.name || cache.cache.name); + break; - cacheBean.stringProperty('cacheName') - .emptyBeanProperty('checkpointListener'); + default: + break; + } - return cacheBean; + if (credentialsBean) + s3Bean.beanProperty('awsCredentials', credentialsBean); - case 'S3': - const s3Bean = new Bean('org.apache.ignite.spi.checkpoint.s3.S3CheckpointSpi', - 'checkpointSpiS3', spi.S3, clusterDflts.checkpointSpi.S3); + s3Bean.stringProperty('bucketNameSuffix'); - let credentialsBean = null; + const clientBean = new Bean('com.amazonaws.ClientConfiguration', 'clientCfg', spi.S3.clientConfiguration, + clusterDflts.checkpointSpi.S3.clientConfiguration); - switch (_.get(spi.S3, 'awsCredentials.kind')) { - case 'Basic': - credentialsBean = new Bean('com.amazonaws.auth.BasicAWSCredentials', 'awsCredentials', {}); + clientBean.enumProperty('protocol') + .intProperty('maxConnections') + .stringProperty('userAgent'); - credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey', 'YOUR_S3_ACCESS_KEY') - .propertyConstructorArgument('checkpoint.s3.credentials.secretKey', 'YOUR_S3_SECRET_KEY'); + const locAddr = new Bean('java.net.InetAddress', '', spi.S3.clientConfiguration) + .factoryMethod('getByName') + .stringConstructorArgument('localAddress'); - break; + if (locAddr.nonEmpty()) + clientBean.beanProperty('localAddress', locAddr); - case 'Properties': - credentialsBean = new Bean('com.amazonaws.auth.PropertiesCredentials', 'awsCredentials', {}); + clientBean.stringProperty('proxyHost') + .intProperty('proxyPort') + .stringProperty('proxyUsername'); - const fileBean = new Bean('java.io.File', '', spi.S3.awsCredentials.Properties) - .pathConstructorArgument('path'); + const userName = clientBean.valueOf('proxyUsername'); - if (fileBean.nonEmpty()) - credentialsBean.beanConstructorArgument('file', fileBean); + if (userName) + clientBean.property('proxyPassword', `checkpoint.s3.proxy.${userName}.password`); - break; + clientBean.stringProperty('proxyDomain') + .stringProperty('proxyWorkstation'); - case 'Anonymous': - credentialsBean = new Bean('com.amazonaws.auth.AnonymousAWSCredentials', 'awsCredentials', {}); + const retryPolicy = spi.S3.clientConfiguration.retryPolicy; - break; + if (retryPolicy) { + const kind = retryPolicy.kind; - case 'BasicSession': - credentialsBean = new Bean('com.amazonaws.auth.BasicSessionCredentials', 'awsCredentials', {}); + const policy = retryPolicy[kind]; - // TODO 2054 Arguments in one line is very long string. - credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey') - .propertyConstructorArgument('checkpoint.s3.credentials.secretKey') - .propertyConstructorArgument('checkpoint.s3.credentials.sessionToken'); + let retryBean; + + switch (kind) { + case 'Default': + retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', { + retryCondition: 'DEFAULT_RETRY_CONDITION', + backoffStrategy: 'DEFAULT_BACKOFF_STRATEGY', + maxErrorRetry: 'DEFAULT_MAX_ERROR_RETRY', + honorMaxErrorRetryInClientConfig: true + }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy); + + retryBean.constantConstructorArgument('retryCondition') + .constantConstructorArgument('backoffStrategy') + .constantConstructorArgument('maxErrorRetry') + .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); break; - case 'Custom': - const className = _.get(spi.S3.awsCredentials, 'Custom.className'); + case 'DefaultMaxRetries': + retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', { + retryCondition: 'DEFAULT_RETRY_CONDITION', + backoffStrategy: 'DEFAULT_BACKOFF_STRATEGY', + maxErrorRetry: _.get(policy, 'maxErrorRetry') || -1, + honorMaxErrorRetryInClientConfig: false + }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy); - credentialsBean = new Bean(className, 'awsCredentials', {}); + retryBean.constantConstructorArgument('retryCondition') + .constantConstructorArgument('backoffStrategy') + .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry')) + .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); break; - default: - break; - } + case 'DynamoDB': + retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', { + retryCondition: 'DEFAULT_RETRY_CONDITION', + backoffStrategy: 'DYNAMODB_DEFAULT_BACKOFF_STRATEGY', + maxErrorRetry: 'DYNAMODB_DEFAULT_MAX_ERROR_RETRY', + honorMaxErrorRetryInClientConfig: true + }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy); - if (credentialsBean) - s3Bean.beanProperty('awsCredentials', credentialsBean); + retryBean.constantConstructorArgument('retryCondition') + .constantConstructorArgument('backoffStrategy') + .constantConstructorArgument('maxErrorRetry') + .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); - s3Bean.stringProperty('bucketNameSuffix'); + break; - const clientBean = new Bean('com.amazonaws.ClientConfiguration', 'clientCfg', spi.S3.clientConfiguration, - clusterDflts.checkpointSpi.S3.clientConfiguration); + case 'DynamoDBMaxRetries': + retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', { + retryCondition: 'DEFAULT_RETRY_CONDITION', + backoffStrategy: 'DYNAMODB_DEFAULT_BACKOFF_STRATEGY', + maxErrorRetry: _.get(policy, 'maxErrorRetry') || -1, + honorMaxErrorRetryInClientConfig: false + }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy); - clientBean.enumProperty('protocol') - .intProperty('maxConnections') - .stringProperty('userAgent'); + retryBean.constantConstructorArgument('retryCondition') + .constantConstructorArgument('backoffStrategy') + .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry')) + .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); - const locAddr = new Bean('java.net.InetAddress', '', spi.S3.clientConfiguration) - .factoryMethod('getByName') - .stringConstructorArgument('localAddress'); + break; - if (locAddr.nonEmpty()) - clientBean.beanProperty('localAddress', locAddr); + case 'Custom': + retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', policy); - clientBean.stringProperty('proxyHost') - .intProperty('proxyPort') - .stringProperty('proxyUsername'); + retryBean.beanConstructorArgument('retryCondition', retryBean.valueOf('retryCondition') ? new EmptyBean(retryBean.valueOf('retryCondition')) : null) + .beanConstructorArgument('backoffStrategy', retryBean.valueOf('backoffStrategy') ? new EmptyBean(retryBean.valueOf('backoffStrategy')) : null) + .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry')) + .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); - const userName = clientBean.valueOf('proxyUsername'); + break; - if (userName) - clientBean.property('proxyPassword', `checkpoint.s3.proxy.${userName}.password`); + default: + break; + } - clientBean.stringProperty('proxyDomain') - .stringProperty('proxyWorkstation'); + if (retryBean) + clientBean.beanProperty('retryPolicy', retryBean); + } - const retryPolicy = spi.S3.clientConfiguration.retryPolicy; + clientBean.intProperty('maxErrorRetry') + .intProperty('socketTimeout') + .intProperty('connectionTimeout') + .intProperty('requestTimeout') + .intProperty('socketSendBufferSizeHints') + .stringProperty('signerOverride') + .intProperty('connectionTTL') + .intProperty('connectionMaxIdleMillis') + .emptyBeanProperty('dnsResolver') + .intProperty('responseMetadataCacheSize') + .emptyBeanProperty('secureRandom') + .boolProperty('useReaper') + .boolProperty('useGzip') + .boolProperty('preemptiveBasicProxyAuth') + .boolProperty('useTcpKeepAlive'); + + if (clientBean.nonEmpty()) + s3Bean.beanProperty('clientConfiguration', clientBean); + + s3Bean.emptyBeanProperty('checkpointListener'); + + return s3Bean; + + case 'JDBC': + const jdbcBean = new Bean('org.apache.ignite.spi.checkpoint.jdbc.JdbcCheckpointSpi', + 'checkpointSpiJdbc', spi.JDBC, clusterDflts.checkpointSpi.JDBC); + + const id = jdbcBean.valueOf('dataSourceBean'); + const dialect = _.get(spi.JDBC, 'dialect'); + + jdbcBean.dataSource(id, 'dataSource', this.dataSourceBean(id, dialect)); + + if (!_.isEmpty(jdbcBean.valueOf('user'))) { + jdbcBean.stringProperty('user') + .property('pwd', `checkpoint.${jdbcBean.valueOf('dataSourceBean')}.${jdbcBean.valueOf('user')}.jdbc.password`, 'YOUR_PASSWORD'); + } - if (retryPolicy) { - const kind = retryPolicy.kind; + jdbcBean.stringProperty('checkpointTableName') + .stringProperty('keyFieldName') + .stringProperty('keyFieldType') + .stringProperty('valueFieldName') + .stringProperty('valueFieldType') + .stringProperty('expireDateFieldName') + .stringProperty('expireDateFieldType') + .intProperty('numberOfRetries') + .emptyBeanProperty('checkpointListener'); - const policy = retryPolicy[kind]; + return jdbcBean; - let retryBean; + case 'Custom': + const clsName = _.get(spi, 'Custom.className'); - switch (kind) { - case 'Default': - retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', { - retryCondition: 'DEFAULT_RETRY_CONDITION', - backoffStrategy: 'DEFAULT_BACKOFF_STRATEGY', - maxErrorRetry: 'DEFAULT_MAX_ERROR_RETRY', - honorMaxErrorRetryInClientConfig: true - }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy); + if (clsName) + return new Bean(clsName, 'checkpointSpiCustom', spi.Cache); - retryBean.constantConstructorArgument('retryCondition') - .constantConstructorArgument('backoffStrategy') - .constantConstructorArgument('maxErrorRetry') - .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); + return null; - break; + default: + return null; + } + }), (checkpointBean) => _.nonNil(checkpointBean)); - case 'DefaultMaxRetries': - retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', { - retryCondition: 'DEFAULT_RETRY_CONDITION', - backoffStrategy: 'DEFAULT_BACKOFF_STRATEGY', - maxErrorRetry: _.get(policy, 'maxErrorRetry') || -1, - honorMaxErrorRetryInClientConfig: false - }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy); + cfg.arrayProperty('checkpointSpi', 'checkpointSpi', cfgs, 'org.apache.ignite.spi.checkpoint.CheckpointSpi'); - retryBean.constantConstructorArgument('retryCondition') - .constantConstructorArgument('backoffStrategy') - .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry')) - .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); + return cfg; + } - break; + // Generate collision group. + static clusterCollision(collision, cfg = this.igniteConfigurationBean()) { + let colSpi; + + switch (_.get(collision, 'kind')) { + case 'JobStealing': + colSpi = new Bean('org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi', + 'colSpi', collision.JobStealing, clusterDflts.collision.JobStealing); + + colSpi.intProperty('activeJobsThreshold') + .intProperty('waitJobsThreshold') + .intProperty('messageExpireTime') + .intProperty('maximumStealingAttempts') + .boolProperty('stealingEnabled') + .emptyBeanProperty('externalCollisionListener') + .mapProperty('stealingAttrs', 'stealingAttributes'); + + break; + case 'FifoQueue': + colSpi = new Bean('org.apache.ignite.spi.collision.fifoqueue.FifoQueueCollisionSpi', + 'colSpi', collision.FifoQueue, clusterDflts.collision.FifoQueue); + + colSpi.intProperty('parallelJobsNumber') + .intProperty('waitingJobsNumber'); + + break; + case 'PriorityQueue': + colSpi = new Bean('org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi', + 'colSpi', collision.PriorityQueue, clusterDflts.collision.PriorityQueue); + + colSpi.intProperty('parallelJobsNumber') + .intProperty('waitingJobsNumber') + .intProperty('priorityAttributeKey') + .intProperty('jobPriorityAttributeKey') + .intProperty('defaultPriority') + .intProperty('starvationIncrement') + .boolProperty('starvationPreventionEnabled'); + + break; + case 'Custom': + if (_.nonNil(_.get(collision, 'Custom.class'))) + colSpi = new EmptyBean(collision.Custom.class); + + break; + default: + return cfg; + } - case 'DynamoDB': - retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', { - retryCondition: 'DEFAULT_RETRY_CONDITION', - backoffStrategy: 'DYNAMODB_DEFAULT_BACKOFF_STRATEGY', - maxErrorRetry: 'DYNAMODB_DEFAULT_MAX_ERROR_RETRY', - honorMaxErrorRetryInClientConfig: true - }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy); + if (_.nonNil(colSpi)) + cfg.beanProperty('collisionSpi', colSpi); - retryBean.constantConstructorArgument('retryCondition') - .constantConstructorArgument('backoffStrategy') - .constantConstructorArgument('maxErrorRetry') - .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); + return cfg; + } - break; + // Generate communication group. + static clusterCommunication(cluster, cfg = this.igniteConfigurationBean(cluster)) { + const commSpi = new Bean('org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi', 'communicationSpi', + cluster.communication, clusterDflts.communication); + + commSpi.emptyBeanProperty('listener') + .stringProperty('localAddress') + .intProperty('localPort') + .intProperty('localPortRange') + .intProperty('sharedMemoryPort') + .intProperty('directBuffer') + .intProperty('directSendBuffer') + .intProperty('idleConnectionTimeout') + .intProperty('connectTimeout') + .intProperty('maxConnectTimeout') + .intProperty('reconnectCount') + .intProperty('socketSendBuffer') + .intProperty('socketReceiveBuffer') + .intProperty('messageQueueLimit') + .intProperty('slowClientQueueLimit') + .intProperty('tcpNoDelay') + .intProperty('ackSendThreshold') + .intProperty('unacknowledgedMessagesBufferSize') + .intProperty('socketWriteTimeout') + .intProperty('selectorsCount') + .emptyBeanProperty('addressResolver'); + + if (commSpi.nonEmpty()) + cfg.beanProperty('communicationSpi', commSpi); + + cfg.intProperty('networkTimeout') + .intProperty('networkSendRetryDelay') + .intProperty('networkSendRetryCount') + .intProperty('discoveryStartupDelay'); + + return cfg; + } - case 'DynamoDBMaxRetries': - retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', { - retryCondition: 'DEFAULT_RETRY_CONDITION', - backoffStrategy: 'DYNAMODB_DEFAULT_BACKOFF_STRATEGY', - maxErrorRetry: _.get(policy, 'maxErrorRetry') || -1, - honorMaxErrorRetryInClientConfig: false - }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy); + // Generate REST access configuration. + static clusterConnector(connector, cfg = this.igniteConfigurationBean()) { + const connCfg = new Bean('org.apache.ignite.configuration.ConnectorConfiguration', + 'connectorConfiguration', connector, clusterDflts.connector); + + if (connCfg.valueOf('enabled')) { + connCfg.pathProperty('jettyPath') + .stringProperty('host') + .intProperty('port') + .intProperty('portRange') + .intProperty('idleTimeout') + .intProperty('idleQueryCursorTimeout') + .intProperty('idleQueryCursorCheckFrequency') + .intProperty('receiveBufferSize') + .intProperty('sendBufferSize') + .intProperty('sendQueueLimit') + .intProperty('directBuffer') + .intProperty('noDelay') + .intProperty('selectorCount') + .intProperty('threadPoolSize') + .emptyBeanProperty('messageInterceptor') + .stringProperty('secretKey'); + + if (connCfg.valueOf('sslEnabled')) { + connCfg.intProperty('sslClientAuth') + .emptyBeanProperty('sslFactory'); + } - retryBean.constantConstructorArgument('retryCondition') - .constantConstructorArgument('backoffStrategy') - .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry')) - .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); + if (connCfg.nonEmpty()) + cfg.beanProperty('connectorConfiguration', connCfg); + } - break; + return cfg; + } - case 'Custom': - retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', policy); + // Generate deployment group. + static clusterDeployment(cluster, cfg = this.igniteConfigurationBean(cluster)) { + cfg.enumProperty('deploymentMode') + .boolProperty('peerClassLoadingEnabled'); - retryBean.beanConstructorArgument('retryCondition', retryBean.valueOf('retryCondition') ? new EmptyBean(retryBean.valueOf('retryCondition')) : null) - .beanConstructorArgument('backoffStrategy', retryBean.valueOf('backoffStrategy') ? new EmptyBean(retryBean.valueOf('backoffStrategy')) : null) - .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry')) - .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig')); + if (cfg.valueOf('peerClassLoadingEnabled')) { + cfg.intProperty('peerClassLoadingMissedResourcesCacheSize') + .intProperty('peerClassLoadingThreadPoolSize') + .varArgProperty('p2pLocClsPathExcl', 'peerClassLoadingLocalClassPathExclude', + cluster.peerClassLoadingLocalClassPathExclude); + } - break; + let deploymentBean = null; - default: - break; - } + switch (_.get(cluster, 'deploymentSpi.kind')) { + case 'URI': + const uriDeployment = cluster.deploymentSpi.URI; - if (retryBean) - clientBean.beanProperty('retryPolicy', retryBean); - } + deploymentBean = new Bean('org.apache.ignite.spi.deployment.uri.UriDeploymentSpi', 'deploymentSpi', uriDeployment); - clientBean.intProperty('maxErrorRetry') - .intProperty('socketTimeout') - .intProperty('connectionTimeout') - .intProperty('requestTimeout') - .intProperty('socketSendBufferSizeHints') - .stringProperty('signerOverride') - .intProperty('connectionTTL') - .intProperty('connectionMaxIdleMillis') - .emptyBeanProperty('dnsResolver') - .intProperty('responseMetadataCacheSize') - .emptyBeanProperty('secureRandom') - .boolProperty('useReaper') - .boolProperty('useGzip') - .boolProperty('preemptiveBasicProxyAuth') - .boolProperty('useTcpKeepAlive'); - - if (clientBean.nonEmpty()) - s3Bean.beanProperty('clientConfiguration', clientBean); - - s3Bean.emptyBeanProperty('checkpointListener'); - - return s3Bean; - - case 'JDBC': - const jdbcBean = new Bean('org.apache.ignite.spi.checkpoint.jdbc.JdbcCheckpointSpi', - 'checkpointSpiJdbc', spi.JDBC, clusterDflts.checkpointSpi.JDBC); - - const id = jdbcBean.valueOf('dataSourceBean'); - const dialect = _.get(spi.JDBC, 'dialect'); - - jdbcBean.dataSource(id, 'dataSource', this.dataSourceBean(id, dialect)); - - if (!_.isEmpty(jdbcBean.valueOf('user'))) { - jdbcBean.stringProperty('user') - .property('pwd', `checkpoint.${jdbcBean.valueOf('dataSourceBean')}.${jdbcBean.valueOf('user')}.jdbc.password`, 'YOUR_PASSWORD'); - } + const scanners = _.map(uriDeployment.scanners, (scanner) => new EmptyBean(scanner)); - jdbcBean.stringProperty('checkpointTableName') - .stringProperty('keyFieldName') - .stringProperty('keyFieldType') - .stringProperty('valueFieldName') - .stringProperty('valueFieldType') - .stringProperty('expireDateFieldName') - .stringProperty('expireDateFieldType') - .intProperty('numberOfRetries') - .emptyBeanProperty('checkpointListener'); + deploymentBean.collectionProperty('uriList', 'uriList', uriDeployment.uriList) + .stringProperty('temporaryDirectoryPath') + .varArgProperty('scanners', 'scanners', scanners, + 'org.apache.ignite.spi.deployment.uri.scanners.UriDeploymentScanner') + .emptyBeanProperty('listener') + .boolProperty('checkMd5') + .boolProperty('encodeUri'); - return jdbcBean; + cfg.beanProperty('deploymentSpi', deploymentBean); - case 'Custom': - const clsName = _.get(spi, 'Custom.className'); + break; - if (clsName) - return new Bean(clsName, 'checkpointSpiCustom', spi.Cache); + case 'Local': + deploymentBean = new Bean('org.apache.ignite.spi.deployment.local.LocalDeploymentSpi', 'deploymentSpi', cluster.deploymentSpi.Local); - return null; + deploymentBean.emptyBeanProperty('listener'); - default: - return null; - } - }), (checkpointBean) => _.nonNil(checkpointBean)); + cfg.beanProperty('deploymentSpi', deploymentBean); - cfg.arrayProperty('checkpointSpi', 'checkpointSpi', cfgs, 'org.apache.ignite.spi.checkpoint.CheckpointSpi'); + break; - return cfg; - } + case 'Custom': + cfg.emptyBeanProperty('deploymentSpi.Custom.className'); - // Generate collision group. - static clusterCollision(collision, cfg = this.igniteConfigurationBean()) { - let colSpi; + break; - switch (_.get(collision, 'kind')) { - case 'JobStealing': - colSpi = new Bean('org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi', - 'colSpi', collision.JobStealing, clusterDflts.collision.JobStealing); + default: + // No-op. + } - colSpi.intProperty('activeJobsThreshold') - .intProperty('waitJobsThreshold') - .intProperty('messageExpireTime') - .intProperty('maximumStealingAttempts') - .boolProperty('stealingEnabled') - .emptyBeanProperty('externalCollisionListener') - .mapProperty('stealingAttrs', 'stealingAttributes'); + return cfg; + } - break; - case 'FifoQueue': - colSpi = new Bean('org.apache.ignite.spi.collision.fifoqueue.FifoQueueCollisionSpi', - 'colSpi', collision.FifoQueue, clusterDflts.collision.FifoQueue); + // Generate discovery group. + static clusterDiscovery(discovery, cfg = this.igniteConfigurationBean(), discoSpi = this.discoveryConfigurationBean(discovery)) { + discoSpi.stringProperty('localAddress') + .intProperty('localPort') + .intProperty('localPortRange') + .emptyBeanProperty('addressResolver') + .intProperty('socketTimeout') + .intProperty('ackTimeout') + .intProperty('maxAckTimeout') + .intProperty('networkTimeout') + .intProperty('joinTimeout') + .intProperty('threadPriority') + .intProperty('heartbeatFrequency') + .intProperty('maxMissedHeartbeats') + .intProperty('maxMissedClientHeartbeats') + .intProperty('topHistorySize') + .emptyBeanProperty('listener') + .emptyBeanProperty('dataExchange') + .emptyBeanProperty('metricsProvider') + .intProperty('reconnectCount') + .intProperty('statisticsPrintFrequency') + .intProperty('ipFinderCleanFrequency') + .emptyBeanProperty('authenticator') + .intProperty('forceServerMode') + .intProperty('clientReconnectDisabled'); + + if (discoSpi.nonEmpty()) + cfg.beanProperty('discoverySpi', discoSpi); + + return discoSpi; + } - colSpi.intProperty('parallelJobsNumber') - .intProperty('waitingJobsNumber'); + // Generate events group. + static clusterEvents(cluster, cfg = this.igniteConfigurationBean(cluster)) { + const eventStorage = cluster.eventStorage; - break; - case 'PriorityQueue': - colSpi = new Bean('org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi', - 'colSpi', collision.PriorityQueue, clusterDflts.collision.PriorityQueue); - - colSpi.intProperty('parallelJobsNumber') - .intProperty('waitingJobsNumber') - .intProperty('priorityAttributeKey') - .intProperty('jobPriorityAttributeKey') - .intProperty('defaultPriority') - .intProperty('starvationIncrement') - .boolProperty('starvationPreventionEnabled'); + let eventStorageBean = null; - break; - case 'Custom': - if (_.nonNil(_.get(collision, 'Custom.class'))) - colSpi = new EmptyBean(collision.Custom.class); + switch (_.get(eventStorage, 'kind')) { + case 'Memory': + eventStorageBean = new Bean('org.apache.ignite.spi.eventstorage.memory.MemoryEventStorageSpi', 'eventStorage', eventStorage.Memory, clusterDflts.eventStorage.Memory); - break; - default: - return cfg; - } + eventStorageBean.intProperty('expireAgeMs') + .intProperty('expireCount') + .emptyBeanProperty('filter'); - if (_.nonNil(colSpi)) - cfg.beanProperty('collisionSpi', colSpi); + break; - return cfg; - } + case 'Custom': + const className = _.get(eventStorage, 'Custom.className'); - // Generate communication group. - static clusterCommunication(cluster, cfg = this.igniteConfigurationBean(cluster)) { - const commSpi = new Bean('org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi', 'communicationSpi', - cluster.communication, clusterDflts.communication); + if (className) + eventStorageBean = new EmptyBean(className); - commSpi.emptyBeanProperty('listener') - .stringProperty('localAddress') - .intProperty('localPort') - .intProperty('localPortRange') - .intProperty('sharedMemoryPort') - .intProperty('directBuffer') - .intProperty('directSendBuffer') - .intProperty('idleConnectionTimeout') - .intProperty('connectTimeout') - .intProperty('maxConnectTimeout') - .intProperty('reconnectCount') - .intProperty('socketSendBuffer') - .intProperty('socketReceiveBuffer') - .intProperty('messageQueueLimit') - .intProperty('slowClientQueueLimit') - .intProperty('tcpNoDelay') - .intProperty('ackSendThreshold') - .intProperty('unacknowledgedMessagesBufferSize') - .intProperty('socketWriteTimeout') - .intProperty('selectorsCount') - .emptyBeanProperty('addressResolver'); - - if (commSpi.nonEmpty()) - cfg.beanProperty('communicationSpi', commSpi); - - cfg.intProperty('networkTimeout') - .intProperty('networkSendRetryDelay') - .intProperty('networkSendRetryCount') - .intProperty('discoveryStartupDelay'); + break; - return cfg; + default: + // No-op. } - // Generate REST access configuration. - static clusterConnector(connector, cfg = this.igniteConfigurationBean()) { - const connCfg = new Bean('org.apache.ignite.configuration.ConnectorConfiguration', - 'connectorConfiguration', connector, clusterDflts.connector); - - if (connCfg.valueOf('enabled')) { - connCfg.pathProperty('jettyPath') - .stringProperty('host') - .intProperty('port') - .intProperty('portRange') - .intProperty('idleTimeout') - .intProperty('idleQueryCursorTimeout') - .intProperty('idleQueryCursorCheckFrequency') - .intProperty('receiveBufferSize') - .intProperty('sendBufferSize') - .intProperty('sendQueueLimit') - .intProperty('directBuffer') - .intProperty('noDelay') - .intProperty('selectorCount') - .intProperty('threadPoolSize') - .emptyBeanProperty('messageInterceptor') - .stringProperty('secretKey'); - - if (connCfg.valueOf('sslEnabled')) { - connCfg.intProperty('sslClientAuth') - .emptyBeanProperty('sslFactory'); - } - - if (connCfg.nonEmpty()) - cfg.beanProperty('connectorConfiguration', connCfg); - } + if (eventStorageBean && eventStorageBean.nonEmpty()) + cfg.beanProperty('eventStorageSpi', eventStorageBean); - return cfg; - } + if (_.nonEmpty(cluster.includeEventTypes)) + cfg.eventTypes('evts', 'includeEventTypes', cluster.includeEventTypes); - // Generate deployment group. - static clusterDeployment(cluster, cfg = this.igniteConfigurationBean(cluster)) { - cfg.enumProperty('deploymentMode') - .boolProperty('peerClassLoadingEnabled'); + return cfg; + } - if (cfg.valueOf('peerClassLoadingEnabled')) { - cfg.intProperty('peerClassLoadingMissedResourcesCacheSize') - .intProperty('peerClassLoadingThreadPoolSize') - .varArgProperty('p2pLocClsPathExcl', 'peerClassLoadingLocalClassPathExclude', - cluster.peerClassLoadingLocalClassPathExclude); - } + // Generate failover group. + static clusterFailover(cluster, cfg = this.igniteConfigurationBean(cluster)) { + const spis = []; - return cfg; - } + _.forEach(cluster.failoverSpi, (spi) => { + let failoverSpi; - // Generate discovery group. - static clusterDiscovery(discovery, cfg = this.igniteConfigurationBean(), discoSpi = this.discoveryConfigurationBean(discovery)) { - discoSpi.stringProperty('localAddress') - .intProperty('localPort') - .intProperty('localPortRange') - .emptyBeanProperty('addressResolver') - .intProperty('socketTimeout') - .intProperty('ackTimeout') - .intProperty('maxAckTimeout') - .intProperty('networkTimeout') - .intProperty('joinTimeout') - .intProperty('threadPriority') - .intProperty('heartbeatFrequency') - .intProperty('maxMissedHeartbeats') - .intProperty('maxMissedClientHeartbeats') - .intProperty('topHistorySize') - .emptyBeanProperty('listener') - .emptyBeanProperty('dataExchange') - .emptyBeanProperty('metricsProvider') - .intProperty('reconnectCount') - .intProperty('statisticsPrintFrequency') - .intProperty('ipFinderCleanFrequency') - .emptyBeanProperty('authenticator') - .intProperty('forceServerMode') - .intProperty('clientReconnectDisabled'); - - if (discoSpi.nonEmpty()) - cfg.beanProperty('discoverySpi', discoSpi); - - return discoSpi; - } + switch (_.get(spi, 'kind')) { + case 'JobStealing': + failoverSpi = new Bean('org.apache.ignite.spi.failover.jobstealing.JobStealingFailoverSpi', + 'failoverSpi', spi.JobStealing, clusterDflts.failoverSpi.JobStealing); - // Generate events group. - static clusterEvents(cluster, cfg = this.igniteConfigurationBean(cluster)) { - const eventStorage = cluster.eventStorage; + failoverSpi.intProperty('maximumFailoverAttempts'); - let eventStorageBean = null; + break; + case 'Never': + failoverSpi = new Bean('org.apache.ignite.spi.failover.never.NeverFailoverSpi', + 'failoverSpi', spi.Never); - switch (_.get(eventStorage, 'kind')) { - case 'Memory': - eventStorageBean = new Bean('org.apache.ignite.spi.eventstorage.memory.MemoryEventStorageSpi', 'eventStorage', eventStorage.Memory, clusterDflts.eventStorage.Memory); + break; + case 'Always': + failoverSpi = new Bean('org.apache.ignite.spi.failover.always.AlwaysFailoverSpi', + 'failoverSpi', spi.Always, clusterDflts.failoverSpi.Always); - eventStorageBean.intProperty('expireAgeMs') - .intProperty('expireCount') - .emptyBeanProperty('filter'); + failoverSpi.intProperty('maximumFailoverAttempts'); break; - case 'Custom': - const className = _.get(eventStorage, 'Custom.className'); + const className = _.get(spi, 'Custom.class'); if (className) - eventStorageBean = new EmptyBean(className); + failoverSpi = new EmptyBean(className); break; - default: // No-op. } - if (eventStorageBean && eventStorageBean.nonEmpty()) - cfg.beanProperty('eventStorageSpi', eventStorageBean); - - if (_.nonEmpty(cluster.includeEventTypes)) - cfg.eventTypes('evts', 'includeEventTypes', cluster.includeEventTypes); - - return cfg; - } - - // Generate failover group. - static clusterFailover(cluster, cfg = this.igniteConfigurationBean(cluster)) { - const spis = []; - - _.forEach(cluster.failoverSpi, (spi) => { - let failoverSpi; + if (failoverSpi) + spis.push(failoverSpi); + }); - switch (_.get(spi, 'kind')) { - case 'JobStealing': - failoverSpi = new Bean('org.apache.ignite.spi.failover.jobstealing.JobStealingFailoverSpi', - 'failoverSpi', spi.JobStealing, clusterDflts.failoverSpi.JobStealing); + if (spis.length) + cfg.arrayProperty('failoverSpi', 'failoverSpi', spis, 'org.apache.ignite.spi.failover.FailoverSpi'); - failoverSpi.intProperty('maximumFailoverAttempts'); - - break; - case 'Never': - failoverSpi = new Bean('org.apache.ignite.spi.failover.never.NeverFailoverSpi', - 'failoverSpi', spi.Never); - - break; - case 'Always': - failoverSpi = new Bean('org.apache.ignite.spi.failover.always.AlwaysFailoverSpi', - 'failoverSpi', spi.Always, clusterDflts.failoverSpi.Always); + return cfg; + } - failoverSpi.intProperty('maximumFailoverAttempts'); + // Generate load balancing configuration group. + static clusterLoadBalancing(cluster, cfg = this.igniteConfigurationBean(cluster)) { + const spis = []; - break; - case 'Custom': - const className = _.get(spi, 'Custom.class'); + _.forEach(cluster.loadBalancingSpi, (spi) => { + let loadBalancingSpi; - if (className) - failoverSpi = new EmptyBean(className); + switch (_.get(spi, 'kind')) { + case 'RoundRobin': + loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi', 'loadBalancingSpiRR', spi.RoundRobin, clusterDflts.loadBalancingSpi.RoundRobin); - break; - default: - // No-op. - } + loadBalancingSpi.boolProperty('perTask'); - if (failoverSpi) - spis.push(failoverSpi); - }); + break; + case 'Adaptive': + loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveLoadBalancingSpi', 'loadBalancingSpiAdaptive', spi.Adaptive); - if (spis.length) - cfg.arrayProperty('failoverSpi', 'failoverSpi', spis, 'org.apache.ignite.spi.failover.FailoverSpi'); + let probeBean; - return cfg; - } + switch (_.get(spi, 'Adaptive.loadProbe.kind')) { + case 'Job': + probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveJobCountLoadProbe', 'jobProbe', spi.Adaptive.loadProbe.Job, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.Job); - // Generate load balancing configuration group. - static clusterLoadBalancing(cluster, cfg = this.igniteConfigurationBean(cluster)) { - const spis = []; + probeBean.boolProperty('useAverage'); - _.forEach(cluster.loadBalancingSpi, (spi) => { - let loadBalancingSpi; + break; + case 'CPU': + probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveCpuLoadProbe', 'cpuProbe', spi.Adaptive.loadProbe.CPU, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.CPU); - switch (_.get(spi, 'kind')) { - case 'RoundRobin': - loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi', 'loadBalancingSpiRR', spi.RoundRobin, clusterDflts.loadBalancingSpi.RoundRobin); + probeBean.boolProperty('useAverage') + .boolProperty('useProcessors') + .intProperty('processorCoefficient'); - loadBalancingSpi.boolProperty('perTask'); + break; + case 'ProcessingTime': + probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveProcessingTimeLoadProbe', 'timeProbe', spi.Adaptive.loadProbe.ProcessingTime, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.ProcessingTime); - break; - case 'Adaptive': - loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveLoadBalancingSpi', 'loadBalancingSpiAdaptive', spi.Adaptive); + probeBean.boolProperty('useAverage'); - let probeBean; + break; + case 'Custom': + const className = _.get(spi, 'Adaptive.loadProbe.Custom.className'); - switch (_.get(spi, 'Adaptive.loadProbe.kind')) { - case 'Job': - probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveJobCountLoadProbe', 'jobProbe', spi.Adaptive.loadProbe.Job, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.Job); + if (className) + probeBean = new Bean(className, 'probe', spi.Adaptive.loadProbe.Job.Custom); - probeBean.boolProperty('useAverage'); + break; + default: + // No-op. + } - break; - case 'CPU': - probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveCpuLoadProbe', 'cpuProbe', spi.Adaptive.loadProbe.CPU, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.CPU); + if (probeBean) + loadBalancingSpi.beanProperty('loadProbe', probeBean); - probeBean.boolProperty('useAverage') - .boolProperty('useProcessors') - .intProperty('processorCoefficient'); + break; + case 'WeightedRandom': + loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.weightedrandom.WeightedRandomLoadBalancingSpi', 'loadBalancingSpiRandom', spi.WeightedRandom, clusterDflts.loadBalancingSpi.WeightedRandom); - break; - case 'ProcessingTime': - probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveProcessingTimeLoadProbe', 'timeProbe', spi.Adaptive.loadProbe.ProcessingTime, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.ProcessingTime); + loadBalancingSpi.intProperty('nodeWeight') + .boolProperty('useWeights'); - probeBean.boolProperty('useAverage'); + break; + case 'Custom': + const className = _.get(spi, 'Custom.className'); - break; - case 'Custom': - const className = _.get(spi, 'Adaptive.loadProbe.Custom.className'); + if (className) + loadBalancingSpi = new Bean(className, 'loadBalancingSpiCustom', spi.Custom); - if (className) - probeBean = new Bean(className, 'probe', spi.Adaptive.loadProbe.Job.Custom); + break; + default: + // No-op. + } - break; - default: - // No-op. - } + if (loadBalancingSpi) + spis.push(loadBalancingSpi); + }); - if (probeBean) - loadBalancingSpi.beanProperty('loadProbe', probeBean); + if (spis.length) + cfg.varArgProperty('loadBalancingSpi', 'loadBalancingSpi', spis, 'org.apache.ignite.spi.loadbalancing.LoadBalancingSpi'); - break; - case 'WeightedRandom': - loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.weightedrandom.WeightedRandomLoadBalancingSpi', 'loadBalancingSpiRandom', spi.WeightedRandom, clusterDflts.loadBalancingSpi.WeightedRandom); + return cfg; + } - loadBalancingSpi.intProperty('nodeWeight') - .boolProperty('useWeights'); + // Generate logger group. + static clusterLogger(logger, cfg = this.igniteConfigurationBean()) { + let loggerBean; - break; - case 'Custom': - const className = _.get(spi, 'Custom.className'); + switch (_.get(logger, 'kind')) { + case 'Log4j': + if (logger.Log4j && (logger.Log4j.mode === 'Default' || logger.Log4j.mode === 'Path' && _.nonEmpty(logger.Log4j.path))) { + loggerBean = new Bean('org.apache.ignite.logger.log4j.Log4JLogger', + 'logger', logger.Log4j, clusterDflts.logger.Log4j); - if (className) - loadBalancingSpi = new Bean(className, 'loadBalancingSpiCustom', spi.Custom); + if (loggerBean.valueOf('mode') === 'Path') + loggerBean.pathConstructorArgument('path'); - break; - default: - // No-op. + loggerBean.enumProperty('level'); } - if (loadBalancingSpi) - spis.push(loadBalancingSpi); - }); - - if (spis.length) - cfg.varArgProperty('loadBalancingSpi', 'loadBalancingSpi', spis, 'org.apache.ignite.spi.loadbalancing.LoadBalancingSpi'); + break; + case 'Log4j2': + if (logger.Log4j2 && _.nonEmpty(logger.Log4j2.path)) { + loggerBean = new Bean('org.apache.ignite.logger.log4j2.Log4J2Logger', + 'logger', logger.Log4j2, clusterDflts.logger.Log4j2); - return cfg; - } - - // Generate logger group. - static clusterLogger(logger, cfg = this.igniteConfigurationBean()) { - let loggerBean; - - switch (_.get(logger, 'kind')) { - case 'Log4j': - if (logger.Log4j && (logger.Log4j.mode === 'Default' || logger.Log4j.mode === 'Path' && _.nonEmpty(logger.Log4j.path))) { - loggerBean = new Bean('org.apache.ignite.logger.log4j.Log4JLogger', - 'logger', logger.Log4j, clusterDflts.logger.Log4j); - - if (loggerBean.valueOf('mode') === 'Path') - loggerBean.pathConstructorArgument('path'); - - loggerBean.enumProperty('level'); - } - - break; - case 'Log4j2': - if (logger.Log4j2 && _.nonEmpty(logger.Log4j2.path)) { - loggerBean = new Bean('org.apache.ignite.logger.log4j2.Log4J2Logger', - 'logger', logger.Log4j2, clusterDflts.logger.Log4j2); - - loggerBean.pathConstructorArgument('path') - .enumProperty('level'); - } - - break; - case 'Null': - loggerBean = new EmptyBean('org.apache.ignite.logger.NullLogger'); - - break; - case 'Java': - loggerBean = new EmptyBean('org.apache.ignite.logger.java.JavaLogger'); + loggerBean.pathConstructorArgument('path') + .enumProperty('level'); + } - break; - case 'JCL': - loggerBean = new EmptyBean('org.apache.ignite.logger.jcl.JclLogger'); + break; + case 'Null': + loggerBean = new EmptyBean('org.apache.ignite.logger.NullLogger'); - break; - case 'SLF4J': - loggerBean = new EmptyBean('org.apache.ignite.logger.slf4j.Slf4jLogger'); + break; + case 'Java': + loggerBean = new EmptyBean('org.apache.ignite.logger.java.JavaLogger'); - break; - case 'Custom': - if (logger.Custom && _.nonEmpty(logger.Custom.class)) - loggerBean = new EmptyBean(logger.Custom.class); + break; + case 'JCL': + loggerBean = new EmptyBean('org.apache.ignite.logger.jcl.JclLogger'); - break; - default: - return cfg; - } + break; + case 'SLF4J': + loggerBean = new EmptyBean('org.apache.ignite.logger.slf4j.Slf4jLogger'); - if (loggerBean) - cfg.beanProperty('gridLogger', loggerBean); + break; + case 'Custom': + if (logger.Custom && _.nonEmpty(logger.Custom.class)) + loggerBean = new EmptyBean(logger.Custom.class); - return cfg; + break; + default: + return cfg; } - // Generate IGFSs configs. - static clusterIgfss(igfss, cfg = this.igniteConfigurationBean()) { - const igfsCfgs = _.map(igfss, (igfs) => { - const igfsCfg = this.igfsGeneral(igfs); + if (loggerBean) + cfg.beanProperty('gridLogger', loggerBean); - this.igfsIPC(igfs, igfsCfg); - this.igfsFragmentizer(igfs, igfsCfg); - this.igfsDualMode(igfs, igfsCfg); - this.igfsSecondFS(igfs, igfsCfg); - this.igfsMisc(igfs, igfsCfg); - - return igfsCfg; - }); - - cfg.varArgProperty('igfsCfgs', 'fileSystemConfiguration', igfsCfgs, 'org.apache.ignite.configuration.FileSystemConfiguration'); + return cfg; + } - return cfg; - } + // Generate IGFSs configs. + static clusterIgfss(igfss, cfg = this.igniteConfigurationBean()) { + const igfsCfgs = _.map(igfss, (igfs) => { + const igfsCfg = this.igfsGeneral(igfs); - // Generate marshaller group. - static clusterMarshaller(cluster, cfg = this.igniteConfigurationBean(cluster)) { - const kind = _.get(cluster.marshaller, 'kind'); - const settings = _.get(cluster.marshaller, kind); + this.igfsIPC(igfs, igfsCfg); + this.igfsFragmentizer(igfs, igfsCfg); + this.igfsDualMode(igfs, igfsCfg); + this.igfsSecondFS(igfs, igfsCfg); + this.igfsMisc(igfs, igfsCfg); - if (_.isNil(settings)) - return cfg; + return igfsCfg; + }); - let bean; + cfg.varArgProperty('igfsCfgs', 'fileSystemConfiguration', igfsCfgs, 'org.apache.ignite.configuration.FileSystemConfiguration'); - switch (kind) { - case 'OptimizedMarshaller': - bean = new Bean('org.apache.ignite.marshaller.optimized.OptimizedMarshaller', 'marshaller', settings) - .intProperty('poolSize') - .intProperty('requireSerializable'); + return cfg; + } - break; + // Generate marshaller group. + static clusterMarshaller(cluster, cfg = this.igniteConfigurationBean(cluster)) { + const kind = _.get(cluster.marshaller, 'kind'); + const settings = _.get(cluster.marshaller, kind); - case 'JdkMarshaller': - bean = new Bean('org.apache.ignite.marshaller.jdk.JdkMarshaller', 'marshaller', settings); + let bean; - break; + switch (kind) { + case 'OptimizedMarshaller': + bean = new Bean('org.apache.ignite.marshaller.optimized.OptimizedMarshaller', 'marshaller', settings) + .intProperty('poolSize') + .intProperty('requireSerializable'); - default: - // No-op. - } + break; - if (bean) - cfg.beanProperty('marshaller', bean); + case 'JdkMarshaller': + bean = new Bean('org.apache.ignite.marshaller.jdk.JdkMarshaller', 'marshaller', settings); - cfg.intProperty('marshalLocalJobs') - .intProperty('marshallerCacheKeepAliveTime') - .intProperty('marshallerCacheThreadPoolSize', 'marshallerCachePoolSize'); + break; - return cfg; + default: + // No-op. } - // Generate metrics group. - static clusterMetrics(cluster, cfg = this.igniteConfigurationBean(cluster)) { - cfg.intProperty('metricsExpireTime') - .intProperty('metricsHistorySize') - .intProperty('metricsLogFrequency') - .intProperty('metricsUpdateFrequency'); - - return cfg; - } + if (bean) + cfg.beanProperty('marshaller', bean); - // Generate ODBC group. - static clusterODBC(odbc, cfg = this.igniteConfigurationBean()) { - if (_.get(odbc, 'odbcEnabled') !== true) - return cfg; + cfg.intProperty('marshalLocalJobs') + .intProperty('marshallerCacheKeepAliveTime') + .intProperty('marshallerCacheThreadPoolSize', 'marshallerCachePoolSize'); - const bean = new Bean('org.apache.ignite.configuration.OdbcConfiguration', 'odbcConfiguration', - odbc, clusterDflts.odbcConfiguration); + return cfg; + } - bean.stringProperty('endpointAddress') - .intProperty('maxOpenCursors'); + // Generate metrics group. + static clusterMetrics(cluster, cfg = this.igniteConfigurationBean(cluster)) { + cfg.intProperty('metricsExpireTime') + .intProperty('metricsHistorySize') + .intProperty('metricsLogFrequency') + .intProperty('metricsUpdateFrequency'); - cfg.beanProperty('odbcConfiguration', bean); + return cfg; + } + // Generate ODBC group. + static clusterODBC(odbc, cfg = this.igniteConfigurationBean()) { + if (_.get(odbc, 'odbcEnabled') !== true) return cfg; - } - - // Java code generator for cluster's SSL configuration. - static clusterSsl(cluster, cfg = this.igniteConfigurationBean(cluster)) { - if (cluster.sslEnabled && _.nonNil(cluster.sslContextFactory)) { - const bean = new Bean('org.apache.ignite.ssl.SslContextFactory', 'sslCtxFactory', - cluster.sslContextFactory); - bean.intProperty('keyAlgorithm') - .pathProperty('keyStoreFilePath'); + const bean = new Bean('org.apache.ignite.configuration.OdbcConfiguration', 'odbcConfiguration', + odbc, clusterDflts.odbcConfiguration); - if (_.nonEmpty(bean.valueOf('keyStoreFilePath'))) - bean.propertyChar('keyStorePassword', 'ssl.key.storage.password', 'YOUR_SSL_KEY_STORAGE_PASSWORD'); + bean.stringProperty('endpointAddress') + .intProperty('maxOpenCursors'); - bean.intProperty('keyStoreType') - .intProperty('protocol'); + cfg.beanProperty('odbcConfiguration', bean); - if (_.nonEmpty(cluster.sslContextFactory.trustManagers)) { - bean.arrayProperty('trustManagers', 'trustManagers', - _.map(cluster.sslContextFactory.trustManagers, (clsName) => new EmptyBean(clsName)), - 'javax.net.ssl.TrustManager'); - } - else { - bean.pathProperty('trustStoreFilePath'); + return cfg; + } - if (_.nonEmpty(bean.valueOf('trustStoreFilePath'))) - bean.propertyChar('trustStorePassword', 'ssl.trust.storage.password', 'YOUR_SSL_TRUST_STORAGE_PASSWORD'); + // Java code generator for cluster's SSL configuration. + static clusterSsl(cluster, cfg = this.igniteConfigurationBean(cluster)) { + if (cluster.sslEnabled && _.nonNil(cluster.sslContextFactory)) { + const bean = new Bean('org.apache.ignite.ssl.SslContextFactory', 'sslCtxFactory', + cluster.sslContextFactory); - bean.intProperty('trustStoreType'); - } + bean.intProperty('keyAlgorithm') + .pathProperty('keyStoreFilePath'); - cfg.beanProperty('sslContextFactory', bean); - } + if (_.nonEmpty(bean.valueOf('keyStoreFilePath'))) + bean.propertyChar('keyStorePassword', 'ssl.key.storage.password', 'YOUR_SSL_KEY_STORAGE_PASSWORD'); - return cfg; - } + bean.intProperty('keyStoreType') + .intProperty('protocol'); - // Generate swap group. - static clusterSwap(cluster, cfg = this.igniteConfigurationBean(cluster)) { - if (_.get(cluster.swapSpaceSpi, 'kind') === 'FileSwapSpaceSpi') { - const bean = new Bean('org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi', 'swapSpaceSpi', - cluster.swapSpaceSpi.FileSwapSpaceSpi); + if (_.nonEmpty(cluster.sslContextFactory.trustManagers)) { + bean.arrayProperty('trustManagers', 'trustManagers', + _.map(cluster.sslContextFactory.trustManagers, (clsName) => new EmptyBean(clsName)), + 'javax.net.ssl.TrustManager'); + } + else { + bean.pathProperty('trustStoreFilePath'); - bean.pathProperty('baseDirectory') - .intProperty('readStripesNumber') - .floatProperty('maximumSparsity') - .intProperty('maxWriteQueueSize') - .intProperty('writeBufferSize'); + if (_.nonEmpty(bean.valueOf('trustStoreFilePath'))) + bean.propertyChar('trustStorePassword', 'ssl.trust.storage.password', 'YOUR_SSL_TRUST_STORAGE_PASSWORD'); - cfg.beanProperty('swapSpaceSpi', bean); + bean.intProperty('trustStoreType'); } - return cfg; + cfg.beanProperty('sslContextFactory', bean); } - // Generate time group. - static clusterTime(cluster, cfg = this.igniteConfigurationBean(cluster)) { - cfg.intProperty('clockSyncSamples') - .intProperty('clockSyncFrequency') - .intProperty('timeServerPortBase') - .intProperty('timeServerPortRange'); + return cfg; + } - return cfg; - } + // Generate swap group. + static clusterSwap(cluster, cfg = this.igniteConfigurationBean(cluster)) { + if (_.get(cluster.swapSpaceSpi, 'kind') === 'FileSwapSpaceSpi') { + const bean = new Bean('org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi', 'swapSpaceSpi', + cluster.swapSpaceSpi.FileSwapSpaceSpi); - // Generate thread pools group. - static clusterPools(cluster, cfg = this.igniteConfigurationBean(cluster)) { - cfg.intProperty('publicThreadPoolSize') - .intProperty('systemThreadPoolSize') - .intProperty('managementThreadPoolSize') - .intProperty('igfsThreadPoolSize') - .intProperty('rebalanceThreadPoolSize'); + bean.pathProperty('baseDirectory') + .intProperty('readStripesNumber') + .floatProperty('maximumSparsity') + .intProperty('maxWriteQueueSize') + .intProperty('writeBufferSize'); - return cfg; + cfg.beanProperty('swapSpaceSpi', bean); } - // Generate transactions group. - static clusterTransactions(transactionConfiguration, cfg = this.igniteConfigurationBean()) { - const bean = new Bean('org.apache.ignite.configuration.TransactionConfiguration', 'transactionConfiguration', - transactionConfiguration, clusterDflts.transactionConfiguration); - - bean.enumProperty('defaultTxConcurrency') - .enumProperty('defaultTxIsolation') - .intProperty('defaultTxTimeout') - .intProperty('pessimisticTxLogLinger') - .intProperty('pessimisticTxLogSize') - .boolProperty('txSerializableEnabled') - .emptyBeanProperty('txManagerFactory'); + return cfg; + } - if (bean.nonEmpty()) - cfg.beanProperty('transactionConfiguration', bean); + // Generate time group. + static clusterTime(cluster, cfg = this.igniteConfigurationBean(cluster)) { + cfg.intProperty('clockSyncSamples') + .intProperty('clockSyncFrequency') + .intProperty('timeServerPortBase') + .intProperty('timeServerPortRange'); - return cfg; - } + return cfg; + } - // Generate user attributes group. - static clusterUserAttributes(cluster, cfg = this.igniteConfigurationBean(cluster)) { - cfg.mapProperty('attrs', 'attributes', 'userAttributes'); + // Generate thread pools group. + static clusterPools(cluster, cfg = this.igniteConfigurationBean(cluster)) { + cfg.intProperty('publicThreadPoolSize') + .intProperty('systemThreadPoolSize') + .intProperty('managementThreadPoolSize') + .intProperty('igfsThreadPoolSize') + .intProperty('rebalanceThreadPoolSize'); - return cfg; - } + return cfg; + } - // Generate domain model for general group. - static domainModelGeneral(domain, cfg = this.domainConfigurationBean(domain)) { - switch (cfg.valueOf('queryMetadata')) { - case 'Annotations': - if (_.nonNil(domain.keyType) && _.nonNil(domain.valueType)) - cfg.varArgProperty('indexedTypes', 'indexedTypes', [domain.keyType, domain.valueType], 'java.lang.Class'); + // Generate transactions group. + static clusterTransactions(transactionConfiguration, cfg = this.igniteConfigurationBean()) { + const bean = new Bean('org.apache.ignite.configuration.TransactionConfiguration', 'transactionConfiguration', + transactionConfiguration, clusterDflts.transactionConfiguration); - break; - case 'Configuration': - cfg.stringProperty('keyType', 'keyType', (val) => JavaTypes.fullClassName(val)) - .stringProperty('valueType', 'valueType', (val) => JavaTypes.fullClassName(val)); + bean.enumProperty('defaultTxConcurrency') + .enumProperty('defaultTxIsolation') + .intProperty('defaultTxTimeout') + .intProperty('pessimisticTxLogLinger') + .intProperty('pessimisticTxLogSize') + .boolProperty('txSerializableEnabled') + .emptyBeanProperty('txManagerFactory'); - break; - default: - } + if (bean.nonEmpty()) + cfg.beanProperty('transactionConfiguration', bean); - return cfg; - } + return cfg; + } - // Generate domain model for query group. - static domainModelQuery(domain, cfg = this.domainConfigurationBean(domain)) { - if (cfg.valueOf('queryMetadata') === 'Configuration') { - const fields = _.map(domain.fields, - (e) => ({name: e.name, className: JavaTypes.fullClassName(e.className)})); + // Generate user attributes group. + static clusterUserAttributes(cluster, cfg = this.igniteConfigurationBean(cluster)) { + cfg.mapProperty('attrs', 'attributes', 'userAttributes'); - cfg.mapProperty('fields', fields, 'fields', true) - .mapProperty('aliases', 'aliases'); + return cfg; + } - const indexes = _.map(domain.indexes, (index) => - new Bean('org.apache.ignite.cache.QueryIndex', 'index', index, cacheDflts.indexes) - .stringProperty('name') - .enumProperty('indexType') - .mapProperty('indFlds', 'fields', 'fields', true) - ); + // Generate domain model for general group. + static domainModelGeneral(domain, cfg = this.domainConfigurationBean(domain)) { + switch (cfg.valueOf('queryMetadata')) { + case 'Annotations': + if (_.nonNil(domain.keyType) && _.nonNil(domain.valueType)) + cfg.varArgProperty('indexedTypes', 'indexedTypes', [domain.keyType, domain.valueType], 'java.lang.Class'); - cfg.collectionProperty('indexes', 'indexes', indexes, 'org.apache.ignite.cache.QueryIndex'); - } + break; + case 'Configuration': + cfg.stringProperty('keyType', 'keyType', (val) => javaTypes.fullClassName(val)) + .stringProperty('valueType', 'valueType', (val) => javaTypes.fullClassName(val)); - return cfg; + break; + default: } - // Generate domain model db fields. - static _domainModelDatabaseFields(cfg, propName, domain) { - const fields = _.map(domain[propName], (field) => { - return new Bean('org.apache.ignite.cache.store.jdbc.JdbcTypeField', 'typeField', field, cacheDflts.typeField) - .constantConstructorArgument('databaseFieldType') - .stringConstructorArgument('databaseFieldName') - .classConstructorArgument('javaFieldType') - .stringConstructorArgument('javaFieldName'); - }); - - cfg.varArgProperty(propName, propName, fields, 'org.apache.ignite.cache.store.jdbc.JdbcTypeField'); + return cfg; + } - return cfg; - } + // Generate domain model for query group. + static domainModelQuery(domain, cfg = this.domainConfigurationBean(domain)) { + if (cfg.valueOf('queryMetadata') === 'Configuration') { + const fields = _.map(domain.fields, + (e) => ({name: e.name, className: javaTypes.fullClassName(e.className)})); - // Generate domain model for store group. - static domainStore(domain, cfg = this.domainConfigurationBean(domain)) { - cfg.stringProperty('databaseSchema') - .stringProperty('databaseTable'); + cfg.mapProperty('fields', fields, 'fields', true) + .mapProperty('aliases', 'aliases'); - this._domainModelDatabaseFields(cfg, 'keyFields', domain); - this._domainModelDatabaseFields(cfg, 'valueFields', domain); + const indexes = _.map(domain.indexes, (index) => + new Bean('org.apache.ignite.cache.QueryIndex', 'index', index, cacheDflts.indexes) + .stringProperty('name') + .enumProperty('indexType') + .mapProperty('indFlds', 'fields', 'fields', true) + ); - return cfg; + cfg.collectionProperty('indexes', 'indexes', indexes, 'org.apache.ignite.cache.QueryIndex'); } - /** - * Generate eviction policy object. - * @param {Object} ccfg Parent configuration. - * @param {String} name Property name. - * @param {Object} src Source. - * @param {Object} dflt Default. - * @returns {Object} Parent configuration. - * @private - */ - static _evictionPolicy(ccfg, name, src, dflt) { - let bean; - - switch (_.get(src, 'kind')) { - case 'LRU': - bean = new Bean('org.apache.ignite.cache.eviction.lru.LruEvictionPolicy', 'evictionPlc', - src.LRU, dflt.LRU); - - break; - case 'FIFO': - bean = new Bean('org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy', 'evictionPlc', - src.FIFO, dflt.FIFO); - - break; - case 'SORTED': - bean = new Bean('org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy', 'evictionPlc', - src.SORTED, dflt.SORTED); - - break; - default: - return ccfg; - } + return cfg; + } - bean.intProperty('batchSize') - .intProperty('maxMemorySize') - .intProperty('maxSize'); + // Generate domain model db fields. + static _domainModelDatabaseFields(cfg, propName, domain) { + const fields = _.map(domain[propName], (field) => { + return new Bean('org.apache.ignite.cache.store.jdbc.JdbcTypeField', 'typeField', field, cacheDflts.typeField) + .constantConstructorArgument('databaseFieldType') + .stringConstructorArgument('databaseFieldName') + .classConstructorArgument('javaFieldType') + .stringConstructorArgument('javaFieldName'); + }); - ccfg.beanProperty(name, bean); + cfg.varArgProperty(propName, propName, fields, 'org.apache.ignite.cache.store.jdbc.JdbcTypeField'); - return ccfg; - } - - // Generate cache general group. - static cacheGeneral(cache, ccfg = this.cacheConfigurationBean(cache)) { - ccfg.stringProperty('name') - .enumProperty('cacheMode') - .enumProperty('atomicityMode'); + return cfg; + } - if (ccfg.valueOf('cacheMode') === 'PARTITIONED' && ccfg.valueOf('backups')) { - ccfg.intProperty('backups') - .intProperty('readFromBackup'); - } + // Generate domain model for store group. + static domainStore(domain, cfg = this.domainConfigurationBean(domain)) { + cfg.stringProperty('databaseSchema') + .stringProperty('databaseTable'); - ccfg.intProperty('copyOnRead'); + this._domainModelDatabaseFields(cfg, 'keyFields', domain); + this._domainModelDatabaseFields(cfg, 'valueFields', domain); - if (ccfg.valueOf('cacheMode') === 'PARTITIONED' && ccfg.valueOf('atomicityMode') === 'TRANSACTIONAL') - ccfg.intProperty('invalidate'); + return cfg; + } - return ccfg; + /** + * Generate eviction policy object. + * @param {Object} ccfg Parent configuration. + * @param {String} name Property name. + * @param {Object} src Source. + * @param {Object} dflt Default. + * @returns {Object} Parent configuration. + * @private + */ + static _evictionPolicy(ccfg, name, src, dflt) { + let bean; + + switch (_.get(src, 'kind')) { + case 'LRU': + bean = new Bean('org.apache.ignite.cache.eviction.lru.LruEvictionPolicy', 'evictionPlc', + src.LRU, dflt.LRU); + + break; + case 'FIFO': + bean = new Bean('org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy', 'evictionPlc', + src.FIFO, dflt.FIFO); + + break; + case 'SORTED': + bean = new Bean('org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy', 'evictionPlc', + src.SORTED, dflt.SORTED); + + break; + default: + return ccfg; } - // Generate cache memory group. - static cacheMemory(cache, ccfg = this.cacheConfigurationBean(cache)) { - ccfg.enumProperty('memoryMode'); + bean.intProperty('batchSize') + .intProperty('maxMemorySize') + .intProperty('maxSize'); - if (ccfg.valueOf('memoryMode') !== 'OFFHEAP_VALUES') - ccfg.intProperty('offHeapMaxMemory'); + ccfg.beanProperty(name, bean); - this._evictionPolicy(ccfg, 'evictionPolicy', cache.evictionPolicy, cacheDflts.evictionPolicy); + return ccfg; + } - ccfg.intProperty('startSize') - .boolProperty('swapEnabled'); + // Generate cache general group. + static cacheGeneral(cache, ccfg = this.cacheConfigurationBean(cache)) { + ccfg.stringProperty('name') + .enumProperty('cacheMode') + .enumProperty('atomicityMode'); - return ccfg; + if (ccfg.valueOf('cacheMode') === 'PARTITIONED' && ccfg.valueOf('backups')) { + ccfg.intProperty('backups') + .intProperty('readFromBackup'); } - // Generate cache queries & Indexing group. - static cacheQuery(cache, domains, ccfg = this.cacheConfigurationBean(cache)) { - const indexedTypes = _.reduce(domains, (acc, domain) => { - if (domain.queryMetadata === 'Annotations') - acc.push(domain.keyType, domain.valueType); - - return acc; - }, []); + ccfg.intProperty('copyOnRead'); - ccfg.stringProperty('sqlSchema') - .intProperty('sqlOnheapRowCacheSize') - .intProperty('longQueryWarningTimeout') - .arrayProperty('indexedTypes', 'indexedTypes', indexedTypes, 'java.lang.Class') - .arrayProperty('sqlFunctionClasses', 'sqlFunctionClasses', cache.sqlFunctionClasses, 'java.lang.Class') - .intProperty('snapshotableIndex') - .intProperty('sqlEscapeAll'); - - return ccfg; - } + if (ccfg.valueOf('cacheMode') === 'PARTITIONED' && ccfg.valueOf('atomicityMode') === 'TRANSACTIONAL') + ccfg.intProperty('invalidate'); - // Generate cache store group. - static cacheStore(cache, domains, ccfg = this.cacheConfigurationBean(cache)) { - const kind = _.get(cache, 'cacheStoreFactory.kind'); + return ccfg; + } - if (kind && cache.cacheStoreFactory[kind]) { - let bean = null; + // Generate cache memory group. + static cacheMemory(cache, ccfg = this.cacheConfigurationBean(cache)) { + ccfg.enumProperty('memoryMode'); - const storeFactory = cache.cacheStoreFactory[kind]; + if (ccfg.valueOf('memoryMode') !== 'OFFHEAP_VALUES') + ccfg.intProperty('offHeapMaxMemory'); - switch (kind) { - case 'CacheJdbcPojoStoreFactory': - bean = new Bean('org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory', 'cacheStoreFactory', - storeFactory, cacheDflts.cacheStoreFactory.CacheJdbcPojoStoreFactory); + this._evictionPolicy(ccfg, 'evictionPolicy', cache.evictionPolicy, cacheDflts.evictionPolicy); - const jdbcId = bean.valueOf('dataSourceBean'); + ccfg.intProperty('startSize') + .boolProperty('swapEnabled'); - bean.dataSource(jdbcId, 'dataSourceBean', this.dataSourceBean(jdbcId, storeFactory.dialect)) - .beanProperty('dialect', new EmptyBean(this.dialectClsName(storeFactory.dialect))); + return ccfg; + } - bean.intProperty('batchSize') - .intProperty('maximumPoolSize') - .intProperty('maximumWriteAttempts') - .intProperty('parallelLoadCacheMinimumThreshold') - .emptyBeanProperty('hasher') - .emptyBeanProperty('transformer') - .boolProperty('sqlEscapeAll'); + // Generate cache queries & Indexing group. + static cacheQuery(cache, domains, ccfg = this.cacheConfigurationBean(cache)) { + const indexedTypes = _.reduce(domains, (acc, domain) => { + if (domain.queryMetadata === 'Annotations') + acc.push(domain.keyType, domain.valueType); + + return acc; + }, []); + + ccfg.stringProperty('sqlSchema') + .intProperty('sqlOnheapRowCacheSize') + .intProperty('longQueryWarningTimeout') + .arrayProperty('indexedTypes', 'indexedTypes', indexedTypes, 'java.lang.Class') + .intProperty('queryDetailMetricsSize') + .arrayProperty('sqlFunctionClasses', 'sqlFunctionClasses', cache.sqlFunctionClasses, 'java.lang.Class') + .intProperty('snapshotableIndex') + .intProperty('sqlEscapeAll'); + + return ccfg; + } - const setType = (typeBean, propName) => { - if (JavaTypes.nonBuiltInClass(typeBean.valueOf(propName))) - typeBean.stringProperty(propName); - else - typeBean.classProperty(propName); - }; + // Generate cache store group. + static cacheStore(cache, domains, ccfg = this.cacheConfigurationBean(cache)) { + const kind = _.get(cache, 'cacheStoreFactory.kind'); - const types = _.reduce(domains, (acc, domain) => { - if (_.isNil(domain.databaseTable)) - return acc; + if (kind && cache.cacheStoreFactory[kind]) { + let bean = null; - const typeBean = new Bean('org.apache.ignite.cache.store.jdbc.JdbcType', 'type', - _.merge({}, domain, {cacheName: cache.name})) - .stringProperty('cacheName'); + const storeFactory = cache.cacheStoreFactory[kind]; - setType(typeBean, 'keyType'); - setType(typeBean, 'valueType'); + switch (kind) { + case 'CacheJdbcPojoStoreFactory': + bean = new Bean('org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory', 'cacheStoreFactory', + storeFactory, cacheDflts.cacheStoreFactory.CacheJdbcPojoStoreFactory); + + const jdbcId = bean.valueOf('dataSourceBean'); + + bean.dataSource(jdbcId, 'dataSourceBean', this.dataSourceBean(jdbcId, storeFactory.dialect)) + .beanProperty('dialect', new EmptyBean(this.dialectClsName(storeFactory.dialect))); + + bean.intProperty('batchSize') + .intProperty('maximumPoolSize') + .intProperty('maximumWriteAttempts') + .intProperty('parallelLoadCacheMinimumThreshold') + .emptyBeanProperty('hasher') + .emptyBeanProperty('transformer') + .boolProperty('sqlEscapeAll'); + + const setType = (typeBean, propName) => { + if (javaTypes.nonBuiltInClass(typeBean.valueOf(propName))) + typeBean.stringProperty(propName); + else + typeBean.classProperty(propName); + }; + + const types = _.reduce(domains, (acc, domain) => { + if (_.isNil(domain.databaseTable)) + return acc; - this.domainStore(domain, typeBean); + const typeBean = new Bean('org.apache.ignite.cache.store.jdbc.JdbcType', 'type', + _.merge({}, domain, {cacheName: cache.name})) + .stringProperty('cacheName'); - acc.push(typeBean); + setType(typeBean, 'keyType'); + setType(typeBean, 'valueType'); - return acc; - }, []); + this.domainStore(domain, typeBean); - bean.varArgProperty('types', 'types', types, 'org.apache.ignite.cache.store.jdbc.JdbcType'); + acc.push(typeBean); - break; - case 'CacheJdbcBlobStoreFactory': - bean = new Bean('org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory', 'cacheStoreFactory', - storeFactory); + return acc; + }, []); - if (bean.valueOf('connectVia') === 'DataSource') { - const blobId = bean.valueOf('dataSourceBean'); + bean.varArgProperty('types', 'types', types, 'org.apache.ignite.cache.store.jdbc.JdbcType'); - bean.dataSource(blobId, 'dataSourceBean', this.dataSourceBean(blobId, storeFactory.dialect)); - } - else { - ccfg.stringProperty('connectionUrl') - .stringProperty('user') - .property('password', `ds.${storeFactory.user}.password`, 'YOUR_PASSWORD'); - } + break; + case 'CacheJdbcBlobStoreFactory': + bean = new Bean('org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory', 'cacheStoreFactory', + storeFactory); - bean.boolProperty('initSchema') - .stringProperty('createTableQuery') - .stringProperty('loadQuery') - .stringProperty('insertQuery') - .stringProperty('updateQuery') - .stringProperty('deleteQuery'); + if (bean.valueOf('connectVia') === 'DataSource') { + const blobId = bean.valueOf('dataSourceBean'); - break; - case 'CacheHibernateBlobStoreFactory': - bean = new Bean('org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory', - 'cacheStoreFactory', storeFactory); + bean.dataSource(blobId, 'dataSourceBean', this.dataSourceBean(blobId, storeFactory.dialect)); + } + else { + ccfg.stringProperty('connectionUrl') + .stringProperty('user') + .property('password', `ds.${storeFactory.user}.password`, 'YOUR_PASSWORD'); + } - bean.propsProperty('props', 'hibernateProperties'); + bean.boolProperty('initSchema') + .stringProperty('createTableQuery') + .stringProperty('loadQuery') + .stringProperty('insertQuery') + .stringProperty('updateQuery') + .stringProperty('deleteQuery'); - break; - default: - } + break; + case 'CacheHibernateBlobStoreFactory': + bean = new Bean('org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory', + 'cacheStoreFactory', storeFactory); - if (bean) - ccfg.beanProperty('cacheStoreFactory', bean); - } + bean.propsProperty('props', 'hibernateProperties'); - ccfg.boolProperty('storeKeepBinary') - .boolProperty('loadPreviousValue') - .boolProperty('readThrough') - .boolProperty('writeThrough'); - - if (ccfg.valueOf('writeBehindEnabled')) { - ccfg.boolProperty('writeBehindEnabled') - .intProperty('writeBehindBatchSize') - .intProperty('writeBehindFlushSize') - .intProperty('writeBehindFlushFrequency') - .intProperty('writeBehindFlushThreadCount'); + break; + default: } - return ccfg; + if (bean) + ccfg.beanProperty('cacheStoreFactory', bean); } - // Generate cache concurrency control group. - static cacheConcurrency(cache, ccfg = this.cacheConfigurationBean(cache)) { - ccfg.intProperty('maxConcurrentAsyncOperations') - .intProperty('defaultLockTimeout') - .enumProperty('atomicWriteOrderMode') - .enumProperty('writeSynchronizationMode'); - - return ccfg; + ccfg.boolProperty('storeKeepBinary') + .boolProperty('loadPreviousValue') + .boolProperty('readThrough') + .boolProperty('writeThrough'); + + if (ccfg.valueOf('writeBehindEnabled')) { + ccfg.boolProperty('writeBehindEnabled') + .intProperty('writeBehindBatchSize') + .intProperty('writeBehindFlushSize') + .intProperty('writeBehindFlushFrequency') + .intProperty('writeBehindFlushThreadCount'); } - // Generate cache node filter group. - static cacheNodeFilter(cache, igfss, ccfg = this.cacheConfigurationBean(cache)) { - const kind = _.get(cache, 'nodeFilter.kind'); + return ccfg; + } - const settings = _.get(cache.nodeFilter, kind); + // Generate cache concurrency control group. + static cacheConcurrency(cache, ccfg = this.cacheConfigurationBean(cache)) { + ccfg.intProperty('maxConcurrentAsyncOperations') + .intProperty('defaultLockTimeout') + .enumProperty('atomicWriteOrderMode') + .enumProperty('writeSynchronizationMode'); - if (_.isNil(settings)) - return ccfg; + return ccfg; + } - let bean = null; + // Generate cache node filter group. + static cacheNodeFilter(cache, igfss, ccfg = this.cacheConfigurationBean(cache)) { + const kind = _.get(cache, 'nodeFilter.kind'); - switch (kind) { - case 'IGFS': - const foundIgfs = _.find(igfss, {_id: settings.igfs}); + const settings = _.get(cache.nodeFilter, kind); - if (foundIgfs) { - bean = new Bean('org.apache.ignite.internal.processors.igfs.IgfsNodePredicate', 'nodeFilter', foundIgfs) - .stringConstructorArgument('name'); - } + if (_.isNil(settings)) + return ccfg; - break; - case 'Custom': - if (_.nonEmpty(settings.className)) - bean = new EmptyBean(settings.className); + let bean = null; - break; - default: - // No-op. - } + switch (kind) { + case 'IGFS': + const foundIgfs = _.find(igfss, {_id: settings.igfs}); - if (bean) - ccfg.beanProperty('nodeFilter', bean); + if (foundIgfs) { + bean = new Bean('org.apache.ignite.internal.processors.igfs.IgfsNodePredicate', 'nodeFilter', foundIgfs) + .stringConstructorArgument('name'); + } - return ccfg; - } + break; + case 'Custom': + if (_.nonEmpty(settings.className)) + bean = new EmptyBean(settings.className); - // Generate cache rebalance group. - static cacheRebalance(cache, ccfg = this.cacheConfigurationBean(cache)) { - if (ccfg.valueOf('cacheMode') !== 'LOCAL') { - ccfg.enumProperty('rebalanceMode') - .intProperty('rebalanceThreadPoolSize') - .intProperty('rebalanceBatchSize') - .intProperty('rebalanceBatchesPrefetchCount') - .intProperty('rebalanceOrder') - .intProperty('rebalanceDelay') - .intProperty('rebalanceTimeout') - .intProperty('rebalanceThrottle'); - } + break; + default: + // No-op. + } - if (ccfg.includes('igfsAffinnityGroupSize')) { - const bean = new Bean('org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper', 'affinityMapper', cache) - .intConstructorArgument('igfsAffinnityGroupSize'); + if (bean) + ccfg.beanProperty('nodeFilter', bean); - ccfg.beanProperty('affinityMapper', bean); - } + return ccfg; + } - return ccfg; + // Generate cache rebalance group. + static cacheRebalance(cache, ccfg = this.cacheConfigurationBean(cache)) { + if (ccfg.valueOf('cacheMode') !== 'LOCAL') { + ccfg.enumProperty('rebalanceMode') + .intProperty('rebalanceThreadPoolSize') + .intProperty('rebalanceBatchSize') + .intProperty('rebalanceBatchesPrefetchCount') + .intProperty('rebalanceOrder') + .intProperty('rebalanceDelay') + .intProperty('rebalanceTimeout') + .intProperty('rebalanceThrottle'); } - // Generate server near cache group. - static cacheNearServer(cache, ccfg = this.cacheConfigurationBean(cache)) { - if (ccfg.valueOf('cacheMode') === 'PARTITIONED' && _.get(cache, 'nearConfiguration.enabled')) { - const bean = new Bean('org.apache.ignite.configuration.NearCacheConfiguration', 'nearConfiguration', - cache.nearConfiguration, cacheDflts.nearConfiguration); + if (ccfg.includes('igfsAffinnityGroupSize')) { + const bean = new Bean('org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper', 'affinityMapper', cache) + .intConstructorArgument('igfsAffinnityGroupSize'); - bean.intProperty('nearStartSize'); + ccfg.beanProperty('affinityMapper', bean); + } - this._evictionPolicy(bean, 'nearEvictionPolicy', - bean.valueOf('nearEvictionPolicy'), cacheDflts.evictionPolicy); + return ccfg; + } - ccfg.beanProperty('nearConfiguration', bean); - } + // Generate server near cache group. + static cacheNearServer(cache, ccfg = this.cacheConfigurationBean(cache)) { + if (ccfg.valueOf('cacheMode') === 'PARTITIONED' && _.get(cache, 'nearConfiguration.enabled')) { + const bean = new Bean('org.apache.ignite.configuration.NearCacheConfiguration', 'nearConfiguration', + cache.nearConfiguration, cacheDflts.nearConfiguration); - return ccfg; - } + bean.intProperty('nearStartSize'); - // Generate client near cache group. - static cacheNearClient(cache, ccfg = this.cacheConfigurationBean(cache)) { - if (ccfg.valueOf('cacheMode') === 'PARTITIONED' && _.get(cache, 'clientNearConfiguration.enabled')) { - const bean = new Bean('org.apache.ignite.configuration.NearCacheConfiguration', - JavaTypes.toJavaName('nearConfiguration', ccfg.valueOf('name')), - cache.clientNearConfiguration, cacheDflts.clientNearConfiguration); + this._evictionPolicy(bean, 'nearEvictionPolicy', + bean.valueOf('nearEvictionPolicy'), cacheDflts.evictionPolicy); - bean.intProperty('nearStartSize'); + ccfg.beanProperty('nearConfiguration', bean); + } - this._evictionPolicy(bean, 'nearEvictionPolicy', - bean.valueOf('nearEvictionPolicy'), cacheDflts.evictionPolicy); + return ccfg; + } - return bean; - } + // Generate client near cache group. + static cacheNearClient(cache, ccfg = this.cacheConfigurationBean(cache)) { + if (ccfg.valueOf('cacheMode') === 'PARTITIONED' && _.get(cache, 'clientNearConfiguration.enabled')) { + const bean = new Bean('org.apache.ignite.configuration.NearCacheConfiguration', + javaTypes.toJavaName('nearConfiguration', ccfg.valueOf('name')), + cache.clientNearConfiguration, cacheDflts.clientNearConfiguration); - return ccfg; - } + bean.intProperty('nearStartSize'); - // Generate cache statistics group. - static cacheStatistics(cache, ccfg = this.cacheConfigurationBean(cache)) { - ccfg.boolProperty('statisticsEnabled') - .boolProperty('managementEnabled'); + this._evictionPolicy(bean, 'nearEvictionPolicy', + bean.valueOf('nearEvictionPolicy'), cacheDflts.evictionPolicy); - return ccfg; + return bean; } - // Generate domain models configs. - static cacheDomains(domains, ccfg) { - const qryEntities = _.reduce(domains, (acc, domain) => { - if (_.isNil(domain.queryMetadata) || domain.queryMetadata === 'Configuration') { - const qryEntity = this.domainModelGeneral(domain); - - this.domainModelQuery(domain, qryEntity); + return ccfg; + } - acc.push(qryEntity); - } + // Generate cache statistics group. + static cacheStatistics(cache, ccfg = this.cacheConfigurationBean(cache)) { + ccfg.boolProperty('statisticsEnabled') + .boolProperty('managementEnabled'); - return acc; - }, []); + return ccfg; + } - ccfg.collectionProperty('qryEntities', 'queryEntities', qryEntities, 'org.apache.ignite.cache.QueryEntity'); - } + // Generate domain models configs. + static cacheDomains(domains, ccfg) { + const qryEntities = _.reduce(domains, (acc, domain) => { + if (_.isNil(domain.queryMetadata) || domain.queryMetadata === 'Configuration') { + const qryEntity = this.domainModelGeneral(domain); - static cacheConfiguration(cache, ccfg = this.cacheConfigurationBean(cache)) { - this.cacheGeneral(cache, ccfg); - this.cacheMemory(cache, ccfg); - this.cacheQuery(cache, cache.domains, ccfg); - this.cacheStore(cache, cache.domains, ccfg); + this.domainModelQuery(domain, qryEntity); - const igfs = _.get(cache, 'nodeFilter.IGFS.instance'); - this.cacheNodeFilter(cache, igfs ? [igfs] : [], ccfg); - this.cacheConcurrency(cache, ccfg); - this.cacheRebalance(cache, ccfg); - this.cacheNearServer(cache, ccfg); - this.cacheStatistics(cache, ccfg); - this.cacheDomains(cache.domains, ccfg); + acc.push(qryEntity); + } - return ccfg; - } + return acc; + }, []); - // Generate IGFS general group. - static igfsGeneral(igfs, cfg = this.igfsConfigurationBean(igfs)) { - if (_.isEmpty(igfs.name)) - return cfg; + ccfg.collectionProperty('qryEntities', 'queryEntities', qryEntities, 'org.apache.ignite.cache.QueryEntity'); + } - cfg.stringProperty('name') - .stringProperty('name', 'dataCacheName', (name) => name + '-data') - .stringProperty('name', 'metaCacheName', (name) => name + '-meta') - .enumProperty('defaultMode'); + static cacheConfiguration(cache, ccfg = this.cacheConfigurationBean(cache)) { + this.cacheGeneral(cache, ccfg); + this.cacheMemory(cache, ccfg); + this.cacheQuery(cache, cache.domains, ccfg); + this.cacheStore(cache, cache.domains, ccfg); + + const igfs = _.get(cache, 'nodeFilter.IGFS.instance'); + this.cacheNodeFilter(cache, igfs ? [igfs] : [], ccfg); + this.cacheConcurrency(cache, ccfg); + this.cacheRebalance(cache, ccfg); + this.cacheNearServer(cache, ccfg); + this.cacheStatistics(cache, ccfg); + this.cacheDomains(cache.domains, ccfg); + + return ccfg; + } + // Generate IGFS general group. + static igfsGeneral(igfs, cfg = this.igfsConfigurationBean(igfs)) { + if (_.isEmpty(igfs.name)) return cfg; - } - // Generate IGFS secondary file system group. - static igfsSecondFS(igfs, cfg = this.igfsConfigurationBean(igfs)) { - if (igfs.secondaryFileSystemEnabled) { - const secondFs = igfs.secondaryFileSystem || {}; + cfg.stringProperty('name') + .stringProperty('name', 'dataCacheName', (name) => name + '-data') + .stringProperty('name', 'metaCacheName', (name) => name + '-meta') + .enumProperty('defaultMode'); - const bean = new Bean('org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem', - 'secondaryFileSystem', secondFs, igfsDflts.secondaryFileSystem); + return cfg; + } - bean.stringProperty('userName', 'defaultUserName'); + // Generate IGFS secondary file system group. + static igfsSecondFS(igfs, cfg = this.igfsConfigurationBean(igfs)) { + if (igfs.secondaryFileSystemEnabled) { + const secondFs = igfs.secondaryFileSystem || {}; - const factoryBean = new Bean('org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory', - 'fac', secondFs); + const bean = new Bean('org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem', + 'secondaryFileSystem', secondFs, igfsDflts.secondaryFileSystem); - factoryBean.stringProperty('uri') - .pathProperty('cfgPath', 'configPaths'); + bean.stringProperty('userName', 'defaultUserName'); - bean.beanProperty('fileSystemFactory', factoryBean); + const factoryBean = new Bean('org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory', + 'fac', secondFs); - cfg.beanProperty('secondaryFileSystem', bean); - } + factoryBean.stringProperty('uri') + .pathProperty('cfgPath', 'configPaths'); - return cfg; + bean.beanProperty('fileSystemFactory', factoryBean); + + cfg.beanProperty('secondaryFileSystem', bean); } - // Generate IGFS IPC group. - static igfsIPC(igfs, cfg = this.igfsConfigurationBean(igfs)) { - if (igfs.ipcEndpointEnabled) { - const bean = new Bean('org.apache.ignite.igfs.IgfsIpcEndpointConfiguration', 'ipcEndpointConfiguration', - igfs.ipcEndpointConfiguration, igfsDflts.ipcEndpointConfiguration); - - bean.enumProperty('type') - .stringProperty('host') - .intProperty('port') - .intProperty('memorySize') - .pathProperty('tokenDirectoryPath') - .intProperty('threadCount'); - - if (bean.nonEmpty()) - cfg.beanProperty('ipcEndpointConfiguration', bean); - } + return cfg; + } - return cfg; - } + // Generate IGFS IPC group. + static igfsIPC(igfs, cfg = this.igfsConfigurationBean(igfs)) { + if (igfs.ipcEndpointEnabled) { + const bean = new Bean('org.apache.ignite.igfs.IgfsIpcEndpointConfiguration', 'ipcEndpointConfiguration', + igfs.ipcEndpointConfiguration, igfsDflts.ipcEndpointConfiguration); - // Generate IGFS fragmentizer group. - static igfsFragmentizer(igfs, cfg = this.igfsConfigurationBean(igfs)) { - if (igfs.fragmentizerEnabled) { - cfg.intProperty('fragmentizerConcurrentFiles') - .intProperty('fragmentizerThrottlingBlockLength') - .intProperty('fragmentizerThrottlingDelay'); - } - else - cfg.boolProperty('fragmentizerEnabled'); + bean.enumProperty('type') + .stringProperty('host') + .intProperty('port') + .intProperty('memorySize') + .pathProperty('tokenDirectoryPath') + .intProperty('threadCount'); - return cfg; + if (bean.nonEmpty()) + cfg.beanProperty('ipcEndpointConfiguration', bean); } - // Generate IGFS Dual mode group. - static igfsDualMode(igfs, cfg = this.igfsConfigurationBean(igfs)) { - cfg.intProperty('dualModeMaxPendingPutsSize') - .emptyBeanProperty('dualModePutExecutorService') - .intProperty('dualModePutExecutorServiceShutdown'); + return cfg; + } - return cfg; + // Generate IGFS fragmentizer group. + static igfsFragmentizer(igfs, cfg = this.igfsConfigurationBean(igfs)) { + if (igfs.fragmentizerEnabled) { + cfg.intProperty('fragmentizerConcurrentFiles') + .intProperty('fragmentizerThrottlingBlockLength') + .intProperty('fragmentizerThrottlingDelay'); } + else + cfg.boolProperty('fragmentizerEnabled'); - // Generate IGFS miscellaneous group. - static igfsMisc(igfs, cfg = this.igfsConfigurationBean(igfs)) { - cfg.intProperty('blockSize') - .intProperty('streamBufferSize') - .intProperty('maxSpaceSize') - .intProperty('maximumTaskRangeLength') - .intProperty('managementPort') - .intProperty('perNodeBatchSize') - .intProperty('perNodeParallelBatchCount') - .intProperty('prefetchBlocks') - .intProperty('sequentialReadsBeforePrefetch') - .intProperty('trashPurgeTimeout') - .intProperty('colocateMetadata') - .intProperty('relaxedConsistency') - .mapProperty('pathModes', 'pathModes'); + return cfg; + } - return cfg; - } + // Generate IGFS Dual mode group. + static igfsDualMode(igfs, cfg = this.igfsConfigurationBean(igfs)) { + cfg.intProperty('dualModeMaxPendingPutsSize') + .emptyBeanProperty('dualModePutExecutorService') + .intProperty('dualModePutExecutorServiceShutdown'); + + return cfg; } - return ConfigurationGenerator; -}]; + // Generate IGFS miscellaneous group. + static igfsMisc(igfs, cfg = this.igfsConfigurationBean(igfs)) { + cfg.intProperty('blockSize') + .intProperty('streamBufferSize') + .intProperty('maxSpaceSize') + .intProperty('maximumTaskRangeLength') + .intProperty('managementPort') + .intProperty('perNodeBatchSize') + .intProperty('perNodeParallelBatchCount') + .intProperty('prefetchBlocks') + .intProperty('sequentialReadsBeforePrefetch') + .intProperty('trashPurgeTimeout') + .intProperty('colocateMetadata') + .intProperty('relaxedConsistency') + .mapProperty('pathModes', 'pathModes'); + + return cfg; + } +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/generator-optional.js b/modules/web-console/frontend/app/modules/configuration/generator/Custom.service.js similarity index 82% rename from modules/web-console/frontend/app/modules/configuration/generator/generator-optional.js rename to modules/web-console/frontend/app/modules/configuration/generator/Custom.service.js index 61de1a2d3e7f0..a185485c16fa9 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/generator-optional.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/Custom.service.js @@ -16,10 +16,8 @@ */ // Optional content generation entry point. -const $generatorOptional = {}; - -$generatorOptional.optionalContent = function(zip, cluster) { // eslint-disable-line no-unused-vars - // No-op. -}; - -export default $generatorOptional; +export default class IgniteCustomGenerator { + optionalContent(zip, cluster) { // eslint-disable-line no-unused-vars + // No-op. + } +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js index f9776a29f792e..bcfa2e24a50eb 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js @@ -18,7 +18,7 @@ /** * Docker file generation entry point. */ -class GeneratorDocker { +export default class IgniteDockerGenerator { /** * Generate from section. * @@ -74,5 +74,3 @@ class GeneratorDocker { ].join('\n'); } } - -export default ['GeneratorDocker', GeneratorDocker]; diff --git a/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js index b123ab5529f39..959077906f055 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js @@ -158,7 +158,7 @@ const PREDEFINED_QUERIES = [ ]; // Var name generator function. -const beenNameSeed = () => { +const beanNameSeed = () => { let idx = ''; const names = []; @@ -174,1551 +174,1577 @@ const beenNameSeed = () => { }; }; -export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator', (JavaTypes, eventGroups, generator) => { - class JavaTransformer extends AbstractTransformer { - static generator = generator; - - // Mapping for objects to method call. - static METHOD_MAPPING = { - 'org.apache.ignite.configuration.CacheConfiguration': { - id: (ccfg) => JavaTypes.toJavaName('cache', ccfg.findProperty('name').value), - args: '', - generator: (sb, id, ccfg) => { - const cacheName = ccfg.findProperty('name').value; - const dataSources = JavaTransformer.collectDataSources(ccfg); - - const javadoc = [ - `Create configuration for cache "${cacheName}".`, - '', - '@return Configured cache.' - ]; +export default class IgniteJavaTransformer extends AbstractTransformer { + // Mapping for objects to method call. + static METHOD_MAPPING = { + 'org.apache.ignite.configuration.CacheConfiguration': { + prefix: 'cache', + name: 'name', + args: '', + generator: (sb, id, ccfg) => { + const cacheName = ccfg.findProperty('name').value; + const dataSources = IgniteJavaTransformer.collectDataSources(ccfg); + + const javadoc = [ + `Create configuration for cache "${cacheName}".`, + '', + '@return Configured cache.' + ]; - if (dataSources.length) - javadoc.push('@throws Exception if failed to create cache configuration.'); + if (dataSources.length) + javadoc.push('@throws Exception if failed to create cache configuration.'); - JavaTransformer.commentBlock(sb, ...javadoc); - sb.startBlock(`public static CacheConfiguration ${id}()${dataSources.length ? ' throws Exception' : ''} {`); + IgniteJavaTransformer.commentBlock(sb, ...javadoc); + sb.startBlock(`public static CacheConfiguration ${id}()${dataSources.length ? ' throws Exception' : ''} {`); - JavaTransformer.constructBean(sb, ccfg, [], true); + IgniteJavaTransformer.constructBean(sb, ccfg, [], true); - sb.emptyLine(); - sb.append(`return ${ccfg.id};`); + sb.emptyLine(); + sb.append(`return ${ccfg.id};`); - sb.endBlock('}'); + sb.endBlock('}'); - return sb; - } - }, - 'org.apache.ignite.cache.store.jdbc.JdbcType': { - id: (type) => JavaTypes.toJavaName('jdbcType', JavaTypes.shortClassName(type.findProperty('valueType').value)), - args: 'ccfg.getName()', - generator: (sb, name, jdbcType) => { - const javadoc = [ - `Create JDBC type for "${name}".`, - '', - '@param cacheName Cache name.', - '@return Configured JDBC type.' - ]; + return sb; + } + }, + 'org.apache.ignite.cache.store.jdbc.JdbcType': { + prefix: 'jdbcType', + name: 'valueType', + args: 'ccfg.getName()', + generator: (sb, name, jdbcType) => { + const javadoc = [ + `Create JDBC type for "${name}".`, + '', + '@param cacheName Cache name.', + '@return Configured JDBC type.' + ]; - JavaTransformer.commentBlock(sb, ...javadoc); - sb.startBlock(`private static JdbcType ${name}(String cacheName) {`); + IgniteJavaTransformer.commentBlock(sb, ...javadoc); + sb.startBlock(`private static JdbcType ${name}(String cacheName) {`); - const cacheName = jdbcType.findProperty('cacheName'); + const cacheName = jdbcType.findProperty('cacheName'); - cacheName.clsName = 'var'; - cacheName.value = 'cacheName'; + cacheName.clsName = 'var'; + cacheName.value = 'cacheName'; - JavaTransformer.constructBean(sb, jdbcType); + IgniteJavaTransformer.constructBean(sb, jdbcType); - sb.emptyLine(); - sb.append(`return ${jdbcType.id};`); + sb.emptyLine(); + sb.append(`return ${jdbcType.id};`); - sb.endBlock('}'); + sb.endBlock('}'); - return sb; - } + return sb; } - }; - - // Append comment line. - static comment(sb, ...lines) { - _.forEach(lines, (line) => sb.append(`// ${line}`)); } + }; - // Append comment block. - static commentBlock(sb, ...lines) { - if (lines.length === 1) - sb.append(`/** ${_.head(lines)} **/`); - else { - sb.append('/**'); + // Append comment line. + static comment(sb, ...lines) { + _.forEach(lines, (line) => sb.append(`// ${line}`)); + } - _.forEach(lines, (line) => sb.append(` * ${line}`)); + // Append comment block. + static commentBlock(sb, ...lines) { + if (lines.length === 1) + sb.append(`/** ${_.head(lines)} **/`); + else { + sb.append('/**'); - sb.append(' **/'); - } + _.forEach(lines, (line) => sb.append(` * ${line}`)); + + sb.append(' **/'); } + } - /** - * @param {Bean} bean - */ - static _newBean(bean) { - const shortClsName = JavaTypes.shortClassName(bean.clsName); - - if (_.isEmpty(bean.arguments)) - return `new ${shortClsName}()`; - - const args = _.map(bean.arguments, (arg) => { - switch (arg.clsName) { - case 'MAP': - return arg.id; - case 'BEAN': - return this._newBean(arg.value); - default: - return this._toObject(arg.clsName, arg.value); - } - }); + /** + * @param {Bean} bean + */ + static _newBean(bean) { + const shortClsName = this.javaTypes.shortClassName(bean.clsName); + + if (_.isEmpty(bean.arguments)) + return `new ${shortClsName}()`; + + const args = _.map(bean.arguments, (arg) => { + switch (arg.clsName) { + case 'MAP': + return arg.id; + case 'BEAN': + return this._newBean(arg.value); + default: + return this._toObject(arg.clsName, arg.value); + } + }); - if (bean.factoryMtd) - return `${shortClsName}.${bean.factoryMtd}(${args.join(', ')})`; + if (bean.factoryMtd) + return `${shortClsName}.${bean.factoryMtd}(${args.join(', ')})`; - return `new ${shortClsName}(${args.join(', ')})`; - } + return `new ${shortClsName}(${args.join(', ')})`; + } - /** - * @param {StringBuilder} sb - * @param {String} parentId - * @param {String} propertyName - * @param {String} value - * @private - */ - static _setProperty(sb, parentId, propertyName, value) { - sb.append(`${parentId}.set${_.upperFirst(propertyName)}(${value});`); - } + /** + * @param {StringBuilder} sb + * @param {String} parentId + * @param {String} propertyName + * @param {String} value + * @private + */ + static _setProperty(sb, parentId, propertyName, value) { + sb.append(`${parentId}.set${_.upperFirst(propertyName)}(${value});`); + } - /** - * @param {StringBuilder} sb - * @param {Array.} vars - * @param {Boolean} limitLines - * @param {Bean} bean - * @param {String} id - - * @private - */ - static constructBean(sb, bean, vars = [], limitLines = false, id = bean.id) { - _.forEach(bean.arguments, (arg) => { - switch (arg.clsName) { - case 'MAP': - this._constructMap(sb, arg, vars); + /** + * @param {StringBuilder} sb + * @param {Array.} vars + * @param {Boolean} limitLines + * @param {Bean} bean + * @param {String} id + + * @private + */ + static constructBean(sb, bean, vars = [], limitLines = false, id = bean.id) { + _.forEach(bean.arguments, (arg) => { + switch (arg.clsName) { + case 'MAP': + this._constructMap(sb, arg, vars); - sb.emptyLine(); + sb.emptyLine(); - break; - default: - if (this._isBean(arg.clsName) && arg.value.isComplex()) { - this.constructBean(sb, arg.value, vars, limitLines); + break; + default: + if (this._isBean(arg.clsName) && arg.value.isComplex()) { + this.constructBean(sb, arg.value, vars, limitLines); - sb.emptyLine(); - } - } - }); + sb.emptyLine(); + } + } + }); - const clsName = JavaTypes.shortClassName(bean.clsName); + const clsName = this.javaTypes.shortClassName(bean.clsName); - sb.append(`${this.varInit(clsName, id, vars)} = ${this._newBean(bean)};`); + sb.append(`${this.varInit(clsName, id, vars)} = ${this._newBean(bean)};`); - if (_.nonEmpty(bean.properties)) { - sb.emptyLine(); + if (_.nonEmpty(bean.properties)) { + sb.emptyLine(); - this._setProperties(sb, bean, vars, limitLines, id); - } + this._setProperties(sb, bean, vars, limitLines, id); } + } - /** - * @param {StringBuilder} sb - * @param {Bean} bean - * @param {Array.} vars - * @param {Boolean} limitLines - * @private - */ - static constructStoreFactory(sb, bean, vars, limitLines = false) { - const shortClsName = JavaTypes.shortClassName(bean.clsName); - - if (_.includes(vars, bean.id)) - sb.append(`${bean.id} = ${this._newBean(bean)};`); - else { - vars.push(bean.id); - - sb.append(`${shortClsName} ${bean.id} = ${this._newBean(bean)};`); - } - - sb.emptyLine(); + /** + * @param {StringBuilder} sb + * @param {Bean} bean + * @param {Array.} vars + * @param {Boolean} limitLines + * @private + */ + static constructStoreFactory(sb, bean, vars, limitLines = false) { + const shortClsName = this.javaTypes.shortClassName(bean.clsName); + + if (_.includes(vars, bean.id)) + sb.append(`${bean.id} = ${this._newBean(bean)};`); + else { + vars.push(bean.id); + + sb.append(`${shortClsName} ${bean.id} = ${this._newBean(bean)};`); + } - sb.startBlock(`${bean.id}.setDataSourceFactory(new Factory() {`); - this.commentBlock(sb, '{@inheritDoc}'); - sb.startBlock('@Override public DataSource create() {'); + sb.emptyLine(); - sb.append(`return DataSources.INSTANCE_${bean.findProperty('dataSourceBean').id};`); + sb.startBlock(`${bean.id}.setDataSourceFactory(new Factory() {`); + this.commentBlock(sb, '{@inheritDoc}'); + sb.startBlock('@Override public DataSource create() {'); - sb.endBlock('};'); - sb.endBlock('});'); + sb.append(`return DataSources.INSTANCE_${bean.findProperty('dataSourceBean').id};`); - const storeFactory = _.cloneDeep(bean); + sb.endBlock('};'); + sb.endBlock('});'); - _.remove(storeFactory.properties, (p) => _.includes(['dataSourceBean'], p.name)); + const storeFactory = _.cloneDeep(bean); - if (storeFactory.properties.length) { - sb.emptyLine(); + _.remove(storeFactory.properties, (p) => _.includes(['dataSourceBean'], p.name)); - this._setProperties(sb, storeFactory, vars, limitLines); - } - } + if (storeFactory.properties.length) { + sb.emptyLine(); - static _isBean(clsName) { - return JavaTypes.nonBuiltInClass(clsName) && JavaTypes.nonEnum(clsName) && _.includes(clsName, '.'); + this._setProperties(sb, storeFactory, vars, limitLines); } + } - static _toObject(clsName, val) { - const items = _.isArray(val) ? val : [val]; + static _isBean(clsName) { + return this.javaTypes.nonBuiltInClass(clsName) && this.javaTypes.nonEnum(clsName) && _.includes(clsName, '.'); + } - return _.map(items, (item) => { - if (_.isNil(item)) - return 'null'; + static _toObject(clsName, val) { + const items = _.isArray(val) ? val : [val]; + + return _.map(items, (item) => { + if (_.isNil(item)) + return 'null'; + + switch (clsName) { + case 'var': + return item; + case 'byte': + return `(byte) ${item}`; + case 'float': + return `${item}f`; + case 'long': + return `${item}L`; + case 'java.io.Serializable': + case 'java.lang.String': + return `"${item.replace(/\\/g, '\\\\').replace(/"/g, '\\"')}"`; + case 'PATH': + return `"${item.replace(/\\/g, '\\\\')}"`; + case 'java.lang.Class': + return `${this.javaTypes.shortClassName(item)}.class`; + case 'java.util.UUID': + return `UUID.fromString("${item}")`; + case 'PROPERTY': + return `props.getProperty("${item}")`; + case 'PROPERTY_CHAR': + return `props.getProperty("${item}").toCharArray()`; + case 'PROPERTY_INT': + return `Integer.parseInt(props.getProperty("${item}"))`; + default: + if (this._isBean(clsName)) { + if (item.isComplex()) + return item.id; + + return this._newBean(item); + } - switch (clsName) { - case 'var': + if (this.javaTypes.nonEnum(clsName)) return item; - case 'byte': - return `(byte) ${item}`; - case 'float': - return `${item}f`; - case 'long': - return `${item}L`; - case 'java.io.Serializable': - case 'java.lang.String': - return `"${item.replace(/\\/g, '\\\\').replace(/"/g, '\\"')}"`; - case 'PATH': - return `"${item.replace(/\\/g, '\\\\')}"`; - case 'java.lang.Class': - return `${JavaTypes.shortClassName(item)}.class`; - case 'java.util.UUID': - return `UUID.fromString("${item}")`; - case 'PROPERTY': - return `props.getProperty("${item}")`; - case 'PROPERTY_CHAR': - return `props.getProperty("${item}").toCharArray()`; - case 'PROPERTY_INT': - return `Integer.parseInt(props.getProperty("${item}"))`; - default: - if (this._isBean(clsName)) { - if (item.isComplex()) - return item.id; - - return this._newBean(item); - } - if (JavaTypes.nonEnum(clsName)) - return item; - - return `${JavaTypes.shortClassName(clsName)}.${item}`; - } - }); - } + return `${this.javaTypes.shortClassName(clsName)}.${item}`; + } + }); + } - static _constructBeans(sb, type, items, vars, limitLines) { - if (this._isBean(type)) { - // Construct objects inline for preview or simple objects. - const mapper = this.METHOD_MAPPING[type]; + static _mapperId(mapper) { + return (item) => this.javaTypes.toJavaName(mapper.prefix, item.findProperty(mapper.name).value); + } - const nextId = mapper ? mapper.id : beenNameSeed(); + static _constructBeans(sb, type, items, vars, limitLines) { + if (this._isBean(type)) { + // Construct objects inline for preview or simple objects. + const mapper = this.METHOD_MAPPING[type]; - // Prepare objects refs. - return _.map(items, (item) => { - if (limitLines && mapper) - return mapper.id(item) + (limitLines ? `(${mapper.args})` : ''); + const nextId = mapper ? this._mapperId(mapper) : beanNameSeed(); - if (item.isComplex()) { - const id = nextId(item); + // Prepare objects refs. + return _.map(items, (item) => { + if (limitLines && mapper) + return nextId(item) + (limitLines ? `(${mapper.args})` : ''); - this.constructBean(sb, item, vars, limitLines, id); + if (item.isComplex()) { + const id = nextId(item); - sb.emptyLine(); + this.constructBean(sb, item, vars, limitLines, id); - return id; - } + sb.emptyLine(); - return this._newBean(item); - }); - } + return id; + } - return this._toObject(type, items); + return this._newBean(item); + }); } - /** - * - * @param sb - * @param parentId - * @param arrProp - * @param vars - * @param limitLines - * @private - */ - static _setVarArg(sb, parentId, arrProp, vars, limitLines) { - const refs = this._constructBeans(sb, arrProp.typeClsName, arrProp.items, vars, limitLines); - - // Set refs to property. - if (refs.length === 1) - this._setProperty(sb, parentId, arrProp.name, _.head(refs)); - else { - sb.startBlock(`${parentId}.set${_.upperFirst(arrProp.name)}(`); - - const lastIdx = refs.length - 1; - - _.forEach(refs, (ref, idx) => { - sb.append(ref + (lastIdx !== idx ? ',' : '')); - }); + return this._toObject(type, items); + } - sb.endBlock(');'); - } + /** + * + * @param sb + * @param parentId + * @param arrProp + * @param vars + * @param limitLines + * @private + */ + static _setVarArg(sb, parentId, arrProp, vars, limitLines) { + const refs = this._constructBeans(sb, arrProp.typeClsName, arrProp.items, vars, limitLines); + + // Set refs to property. + if (refs.length === 1) + this._setProperty(sb, parentId, arrProp.name, _.head(refs)); + else { + sb.startBlock(`${parentId}.set${_.upperFirst(arrProp.name)}(`); + + const lastIdx = refs.length - 1; + + _.forEach(refs, (ref, idx) => { + sb.append(ref + (lastIdx !== idx ? ',' : '')); + }); + + sb.endBlock(');'); } + } - /** - * - * @param sb - * @param parentId - * @param arrProp - * @param vars - * @param limitLines - * @private - */ - static _setArray(sb, parentId, arrProp, vars, limitLines) { - const refs = this._constructBeans(sb, arrProp.typeClsName, arrProp.items, vars, limitLines); + /** + * + * @param sb + * @param parentId + * @param arrProp + * @param vars + * @param limitLines + * @private + */ + static _setArray(sb, parentId, arrProp, vars, limitLines) { + const refs = this._constructBeans(sb, arrProp.typeClsName, arrProp.items, vars, limitLines); - const arrType = JavaTypes.shortClassName(arrProp.typeClsName); + const arrType = this.javaTypes.shortClassName(arrProp.typeClsName); - // Set refs to property. - sb.startBlock(`${parentId}.set${_.upperFirst(arrProp.name)}(new ${arrType}[] {`); + // Set refs to property. + sb.startBlock(`${parentId}.set${_.upperFirst(arrProp.name)}(new ${arrType}[] {`); - const lastIdx = refs.length - 1; + const lastIdx = refs.length - 1; - _.forEach(refs, (ref, idx) => sb.append(ref + (lastIdx !== idx ? ',' : ''))); + _.forEach(refs, (ref, idx) => sb.append(ref + (lastIdx !== idx ? ',' : ''))); - sb.endBlock('});'); - } + sb.endBlock('});'); + } - static _constructMap(sb, map, vars = []) { - const keyClsName = JavaTypes.shortClassName(map.keyClsName); - const valClsName = JavaTypes.shortClassName(map.valClsName); + static _constructMap(sb, map, vars = []) { + const keyClsName = this.javaTypes.shortClassName(map.keyClsName); + const valClsName = this.javaTypes.shortClassName(map.valClsName); - const mapClsName = map.ordered ? 'LinkedHashMap' : 'HashMap'; + const mapClsName = map.ordered ? 'LinkedHashMap' : 'HashMap'; - const type = `${mapClsName}<${keyClsName}, ${valClsName}>`; + const type = `${mapClsName}<${keyClsName}, ${valClsName}>`; - sb.append(`${this.varInit(type, map.id, vars)} = new ${mapClsName}<>();`); + sb.append(`${this.varInit(type, map.id, vars)} = new ${mapClsName}<>();`); - sb.emptyLine(); + sb.emptyLine(); - _.forEach(map.entries, (entry) => { - const key = this._toObject(map.keyClsName, entry[map.keyField]); - const val = entry[map.valField]; + _.forEach(map.entries, (entry) => { + const key = this._toObject(map.keyClsName, entry[map.keyField]); + const val = entry[map.valField]; - if (_.isArray(val) && map.valClsName === 'java.lang.String') { - if (val.length > 1) { - sb.startBlock(`${map.id}.put(${key},`); + if (_.isArray(val) && map.valClsName === 'java.lang.String') { + if (val.length > 1) { + sb.startBlock(`${map.id}.put(${key},`); - _.forEach(val, (line, idx) => { - sb.append(`"${line}"${idx !== val.length - 1 ? ' +' : ''}`); - }); + _.forEach(val, (line, idx) => { + sb.append(`"${line}"${idx !== val.length - 1 ? ' +' : ''}`); + }); - sb.endBlock(');'); - } - else - sb.append(`${map.id}.put(${key}, ${this._toObject(map.valClsName, _.head(val))});`); + sb.endBlock(');'); } else - sb.append(`${map.id}.put(${key}, ${this._toObject(map.valClsName, val)});`); - }); - } + sb.append(`${map.id}.put(${key}, ${this._toObject(map.valClsName, _.head(val))});`); + } + else + sb.append(`${map.id}.put(${key}, ${this._toObject(map.valClsName, val)});`); + }); + } - static varInit(type, id, vars) { - if (_.includes(vars, id)) - return id; + static varInit(type, id, vars) { + if (_.includes(vars, id)) + return id; - vars.push(id); + vars.push(id); - return `${type} ${id}`; - } + return `${type} ${id}`; + } - /** - * - * @param {StringBuilder} sb - * @param {Bean} bean - * @param {String} id - * @param {Array.} vars - * @param {Boolean} limitLines - * @returns {StringBuilder} - */ - static _setProperties(sb = new StringBuilder(), bean, vars = [], limitLines = false, id = bean.id) { - _.forEach(bean.properties, (prop, idx) => { - switch (prop.clsName) { - case 'DATA_SOURCE': - this._setProperty(sb, id, 'dataSource', `DataSources.INSTANCE_${prop.id}`); - - break; - case 'EVENT_TYPES': - if (prop.eventTypes.length === 1) - this._setProperty(sb, id, prop.name, _.head(prop.eventTypes)); - else { - sb.append(`int[] ${prop.id} = new int[${_.head(prop.eventTypes)}.length`); + /** + * + * @param {StringBuilder} sb + * @param {Bean} bean + * @param {String} id + * @param {Array.} vars + * @param {Boolean} limitLines + * @returns {StringBuilder} + */ + static _setProperties(sb = new StringBuilder(), bean, vars = [], limitLines = false, id = bean.id) { + _.forEach(bean.properties, (prop, idx) => { + switch (prop.clsName) { + case 'DATA_SOURCE': + this._setProperty(sb, id, 'dataSource', `DataSources.INSTANCE_${prop.id}`); + + break; + case 'EVENT_TYPES': + if (prop.eventTypes.length === 1) + this._setProperty(sb, id, prop.name, _.head(prop.eventTypes)); + else { + sb.append(`int[] ${prop.id} = new int[${_.head(prop.eventTypes)}.length`); + + _.forEach(_.tail(prop.eventTypes), (evtGrp) => { + sb.append(` + ${evtGrp}.length`); + }); - _.forEach(_.tail(prop.eventTypes), (evtGrp) => { - sb.append(` + ${evtGrp}.length`); - }); + sb.append('];'); - sb.append('];'); + sb.emptyLine(); + sb.append('int k = 0;'); + + _.forEach(prop.eventTypes, (evtGrp, evtIdx) => { sb.emptyLine(); - sb.append('int k = 0;'); + sb.append(`System.arraycopy(${evtGrp}, 0, ${prop.id}, k, ${evtGrp}.length);`); - _.forEach(prop.eventTypes, (evtGrp, evtIdx) => { - sb.emptyLine(); + if (evtIdx < prop.eventTypes.length - 1) + sb.append(`k += ${evtGrp}.length;`); + }); - sb.append(`System.arraycopy(${evtGrp}, 0, ${prop.id}, k, ${evtGrp}.length);`); + sb.emptyLine(); - if (evtIdx < prop.eventTypes.length - 1) - sb.append(`k += ${evtGrp}.length;`); - }); + sb.append(`cfg.setIncludeEventTypes(${prop.id});`); + } - sb.emptyLine(); + break; + case 'ARRAY': + if (prop.varArg) + this._setVarArg(sb, id, prop, vars, limitLines); + else + this._setArray(sb, id, prop, vars, limitLines); - sb.append(`cfg.setIncludeEventTypes(${prop.id});`); - } + break; + case 'COLLECTION': + const nonBean = !this._isBean(prop.typeClsName); - break; - case 'ARRAY': - if (prop.varArg) - this._setVarArg(sb, id, prop, vars, limitLines); - else - this._setArray(sb, id, prop, vars, limitLines); + if (nonBean && prop.implClsName === 'java.util.ArrayList') { + const items = _.map(prop.items, (item) => this._toObject(prop.typeClsName, item)); - break; - case 'COLLECTION': - const nonBean = !this._isBean(prop.typeClsName); + if (items.length > 1) { + sb.startBlock(`${id}.set${_.upperFirst(prop.name)}(Arrays.asList(`); - if (nonBean && prop.implClsName === 'java.util.ArrayList') { - const items = _.map(prop.items, (item) => this._toObject(prop.typeClsName, item)); + _.forEach(items, (item, i) => sb.append(item + (i !== items.length - 1 ? ',' : ''))); - if (items.length > 1) { - sb.startBlock(`${id}.set${_.upperFirst(prop.name)}(Arrays.asList(`); + sb.endBlock('));'); + } + else + this._setProperty(sb, id, prop.name, `Arrays.asList(${items})`); + } + else { + const colTypeClsName = this.javaTypes.shortClassName(prop.typeClsName); + const implClsName = this.javaTypes.shortClassName(prop.implClsName); - _.forEach(items, (item, i) => sb.append(item + (i !== items.length - 1 ? ',' : ''))); + sb.append(`${this.varInit(`${implClsName}<${colTypeClsName}>`, prop.id, vars)} = new ${implClsName}<>();`); - sb.endBlock('));'); - } - else - this._setProperty(sb, id, prop.name, `Arrays.asList(${items})`); + sb.emptyLine(); + + if (nonBean) { + _.forEach(this._toObject(colTypeClsName, prop.items), (item) => { + sb.append(`${prop.id}.add("${item}");`); + + sb.emptyLine(); + }); } else { - const colTypeClsName = JavaTypes.shortClassName(prop.typeClsName); - const implClsName = JavaTypes.shortClassName(prop.implClsName); + _.forEach(prop.items, (item) => { + this.constructBean(sb, item, vars, limitLines); - sb.append(`${this.varInit(`${implClsName}<${colTypeClsName}>`, prop.id, vars)} = new ${implClsName}<>();`); + sb.append(`${prop.id}.add(${item.id});`); - sb.emptyLine(); + sb.emptyLine(); + }); + } - if (nonBean) { - _.forEach(this._toObject(colTypeClsName, prop.items), (item) => { - sb.append(`${prop.id}.add("${item}");`); + this._setProperty(sb, id, prop.name, prop.id); + } - sb.emptyLine(); - }); - } - else { - _.forEach(prop.items, (item) => { - this.constructBean(sb, item, vars, limitLines); + break; + case 'MAP': + this._constructMap(sb, prop, vars); - sb.append(`${prop.id}.add(${item.id});`); + if (_.nonEmpty(prop.entries)) + sb.emptyLine(); - sb.emptyLine(); - }); - } + this._setProperty(sb, id, prop.name, prop.id); - this._setProperty(sb, id, prop.name, prop.id); - } + break; + case 'java.util.Properties': + sb.append(`${this.varInit('Properties', prop.id, vars)} = new Properties();`); - break; - case 'MAP': - this._constructMap(sb, prop, vars); + if (_.nonEmpty(prop.entries)) + sb.emptyLine(); - if (_.nonEmpty(prop.entries)) - sb.emptyLine(); + _.forEach(prop.entries, (entry) => { + const key = this._toObject('java.lang.String', entry.name); + const val = this._toObject('java.lang.String', entry.value); - this._setProperty(sb, id, prop.name, prop.id); + sb.append(`${prop.id}.setProperty(${key}, ${val});`); + }); - break; - case 'java.util.Properties': - sb.append(`${this.varInit('Properties', prop.id, vars)} = new Properties();`); + sb.emptyLine(); - if (_.nonEmpty(prop.entries)) - sb.emptyLine(); + this._setProperty(sb, id, prop.name, prop.id); - _.forEach(prop.entries, (entry) => { - const key = this._toObject('java.lang.String', entry.name); - const val = this._toObject('java.lang.String', entry.value); + break; + case 'BEAN': + const embedded = prop.value; - sb.append(`${prop.id}.setProperty(${key}, ${val});`); - }); + if (_.includes(STORE_FACTORY, embedded.clsName)) { + this.constructStoreFactory(sb, embedded, vars, limitLines); sb.emptyLine(); - this._setProperty(sb, id, prop.name, prop.id); + this._setProperty(sb, id, prop.name, embedded.id); + } + else if (embedded.isComplex()) { + this.constructBean(sb, embedded, vars, limitLines); - break; - case 'BEAN': - const embedded = prop.value; + sb.emptyLine(); - if (_.includes(STORE_FACTORY, embedded.clsName)) { - this.constructStoreFactory(sb, embedded, vars, limitLines); + this._setProperty(sb, id, prop.name, embedded.id); + } + else + this._setProperty(sb, id, prop.name, this._newBean(embedded)); - sb.emptyLine(); + break; + default: + this._setProperty(sb, id, prop.name, this._toObject(prop.clsName, prop.value)); + } - this._setProperty(sb, id, prop.name, embedded.id); - } - else if (embedded.isComplex()) { - this.constructBean(sb, embedded, vars, limitLines); + this._emptyLineIfNeeded(sb, bean.properties, idx); + }); - sb.emptyLine(); + return sb; + } - this._setProperty(sb, id, prop.name, embedded.id); - } - else - this._setProperty(sb, id, prop.name, this._newBean(embedded)); + static _collectMapImports(prop) { + const imports = []; - break; - default: - this._setProperty(sb, id, prop.name, this._toObject(prop.clsName, prop.value)); - } + imports.push(prop.ordered ? 'java.util.LinkedHashMap' : 'java.util.HashMap'); + imports.push(prop.keyClsName); + imports.push(prop.valClsName); - this._emptyLineIfNeeded(sb, bean.properties, idx); - }); + return imports; + } - return sb; - } + static collectBeanImports(bean) { + const imports = [bean.clsName]; - static collectBeanImports(bean) { - const imports = [bean.clsName]; + _.forEach(bean.arguments, (arg) => { + switch (arg.clsName) { + case 'BEAN': + imports.push(...this.collectPropertiesImports(arg.value.properties)); - _.forEach(bean.arguments, (arg) => { - switch (arg.clsName) { - case 'BEAN': - imports.push(...this.collectPropertiesImports(arg.value.properties)); + break; + case 'java.lang.Class': + imports.push(this.javaTypes.fullClassName(arg.value)); - break; - case 'java.lang.Class': - imports.push(JavaTypes.fullClassName(arg.value)); + break; - break; - default: - imports.push(arg.clsName); - } - }); + case 'MAP': + imports.push(...this._collectMapImports(arg)); - imports.push(...this.collectPropertiesImports(bean.properties)); + break; + default: + imports.push(arg.clsName); + } + }); - if (_.includes(STORE_FACTORY, bean.clsName)) - imports.push('javax.sql.DataSource', 'javax.cache.configuration.Factory'); + imports.push(...this.collectPropertiesImports(bean.properties)); - return imports; - } + if (_.includes(STORE_FACTORY, bean.clsName)) + imports.push('javax.sql.DataSource', 'javax.cache.configuration.Factory'); - /** - * @param {Array.} props - * @returns {Array.} - */ - static collectPropertiesImports(props) { - const imports = []; + return imports; + } - _.forEach(props, (prop) => { - switch (prop.clsName) { - case 'DATA_SOURCE': - imports.push(prop.value.clsName); + /** + * @param {Array.} props + * @returns {Array.} + */ + static collectPropertiesImports(props) { + const imports = []; - break; - case 'PROPERTY': - case 'PROPERTY_CHAR': - case 'PROPERTY_INT': - imports.push('java.io.InputStream', 'java.util.Properties'); + _.forEach(props, (prop) => { + switch (prop.clsName) { + case 'DATA_SOURCE': + imports.push(prop.value.clsName); - break; - case 'BEAN': - imports.push(...this.collectBeanImports(prop.value)); + break; + case 'PROPERTY': + case 'PROPERTY_CHAR': + case 'PROPERTY_INT': + imports.push('java.io.InputStream', 'java.util.Properties'); - break; - case 'ARRAY': - imports.push(prop.typeClsName); + break; + case 'BEAN': + imports.push(...this.collectBeanImports(prop.value)); - if (this._isBean(prop.typeClsName)) - _.forEach(prop.items, (item) => imports.push(...this.collectBeanImports(item))); + break; + case 'ARRAY': + imports.push(prop.typeClsName); - break; - case 'COLLECTION': - imports.push(prop.typeClsName); + if (this._isBean(prop.typeClsName)) + _.forEach(prop.items, (item) => imports.push(...this.collectBeanImports(item))); - if (this._isBean(prop.typeClsName)) { - _.forEach(prop.items, (item) => imports.push(...this.collectBeanImports(item))); + break; + case 'COLLECTION': + imports.push(prop.typeClsName); - imports.push(prop.implClsName); - } - else if (prop.implClsName === 'java.util.ArrayList') - imports.push('java.util.Arrays'); - else - imports.push(prop.implClsName); - - break; - case 'MAP': - imports.push(prop.ordered ? 'java.util.LinkedHashMap' : 'java.util.HashMap'); - imports.push(prop.keyClsName); - imports.push(prop.valClsName); - - break; - default: - if (!JavaTypes.nonEnum(prop.clsName)) - imports.push(prop.clsName); - } - }); + if (this._isBean(prop.typeClsName)) { + _.forEach(prop.items, (item) => imports.push(...this.collectBeanImports(item))); - return imports; - } + imports.push(prop.implClsName); + } + else if (prop.implClsName === 'java.util.ArrayList') + imports.push('java.util.Arrays'); + else + imports.push(prop.implClsName); - static _prepareImports(imports) { - return _.sortedUniq(_.sortBy(_.filter(imports, (cls) => !cls.startsWith('java.lang.') && _.includes(cls, '.')))); - } + break; + case 'MAP': + imports.push(...this._collectMapImports(prop)); - /** - * @param {Bean} bean - * @returns {Array.} - */ - static collectStaticImports(bean) { - const imports = []; + break; + default: + if (!this.javaTypes.nonEnum(prop.clsName)) + imports.push(prop.clsName); + } + }); - _.forEach(bean.properties, (prop) => { - switch (prop.clsName) { - case 'EVENT_TYPES': - _.forEach(prop.eventTypes, (value) => { - const evtGrp = _.find(eventGroups, {value}); + return imports; + } - imports.push(`${evtGrp.class}.${evtGrp.value}`); - }); + static _prepareImports(imports) { + return _.sortedUniq(_.sortBy(_.filter(imports, (cls) => !cls.startsWith('java.lang.') && _.includes(cls, '.')))); + } - break; - default: - // No-op. - } - }); + /** + * @param {Bean} bean + * @returns {Array.} + */ + static collectStaticImports(bean) { + const imports = []; - return imports; - } + _.forEach(bean.properties, (prop) => { + switch (prop.clsName) { + case 'EVENT_TYPES': + _.forEach(prop.eventTypes, (value) => { + const evtGrp = _.find(this.eventGroups, {value}); - /** - * @param {Bean} bean - * @returns {Object} - */ - static collectBeansWithMapping(bean) { - const beans = {}; + imports.push(`${evtGrp.class}.${evtGrp.value}`); + }); - _.forEach(bean.properties, (prop) => { - switch (prop.clsName) { - case 'BEAN': - _.merge(beans, this.collectBeansWithMapping(prop.value)); + break; + default: + // No-op. + } + }); - break; - case 'ARRAY': - if (this._isBean(prop.typeClsName)) { - const mapping = this.METHOD_MAPPING[prop.typeClsName]; + return imports; + } - _.reduce(prop.items, (acc, item) => { - if (mapping) { - acc[mapping.id(item)] = item; + /** + * @param {Bean} bean + * @returns {Object} + */ + static collectBeansWithMapping(bean) { + const beans = {}; - _.merge(acc, this.collectBeansWithMapping(item)); - } - return acc; - }, beans); - } + _.forEach(bean.properties, (prop) => { + switch (prop.clsName) { + case 'BEAN': + _.merge(beans, this.collectBeansWithMapping(prop.value)); - break; - default: - // No-op. - } - }); + break; + case 'ARRAY': + if (this._isBean(prop.typeClsName)) { + const mapper = this.METHOD_MAPPING[prop.typeClsName]; - return beans; - } + const mapperId = mapper ? this._mapperId(mapper) : null; - /** - * Build Java startup class with configuration. - * - * @param {Bean} cfg - * @param pkg Package name. - * @param {String} clsName Class name for generate factory class otherwise generate code snippet. - * @param {Array.} clientNearCaches Is client node. - * @returns {StringBuilder} - */ - static igniteConfiguration(cfg, pkg, clsName, clientNearCaches) { - const sb = new StringBuilder(); - - sb.append(`package ${pkg};`); - sb.emptyLine(); + _.reduce(prop.items, (acc, item) => { + if (mapperId) + acc[mapperId(item)] = item; - const imports = this.collectBeanImports(cfg); + _.merge(acc, this.collectBeansWithMapping(item)); - if (_.nonEmpty(clientNearCaches)) - imports.push('org.apache.ignite.configuration.NearCacheConfiguration'); + return acc; + }, beans); + } - if (_.includes(imports, 'oracle.jdbc.pool.OracleDataSource')) - imports.push('java.sql.SQLException'); + break; + default: + // No-op. + } + }); - const hasProps = this.hasProperties(cfg); + return beans; + } - if (hasProps) - imports.push('java.util.Properties', 'java.io.InputStream'); + /** + * Build Java startup class with configuration. + * + * @param {Bean} cfg + * @param pkg Package name. + * @param {String} clsName Class name for generate factory class otherwise generate code snippet. + * @param {Array.} clientNearCaches Is client node. + * @returns {StringBuilder} + */ + static igniteConfiguration(cfg, pkg, clsName, clientNearCaches) { + const sb = new StringBuilder(); - _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`)); + sb.append(`package ${pkg};`); + sb.emptyLine(); - sb.emptyLine(); + const imports = this.collectBeanImports(cfg); - const staticImports = this._prepareImports(this.collectStaticImports(cfg)); + const nearCacheBeans = []; - if (staticImports.length) { - _.forEach(this._prepareImports(staticImports), (cls) => sb.append(`import static ${cls};`)); + if (_.nonEmpty(clientNearCaches)) { + imports.push('org.apache.ignite.configuration.NearCacheConfiguration'); - sb.emptyLine(); - } + _.forEach(clientNearCaches, (cache) => { + const nearCacheBean = this.generator.cacheNearClient(cache); - this.mainComment(sb); - sb.startBlock(`public class ${clsName} {`); + nearCacheBean.cacheName = cache.name; - // 2. Add external property file - if (hasProps) { - this.commentBlock(sb, 'Secret properties loading.'); - sb.append('private static final Properties props = new Properties();'); - sb.emptyLine(); - sb.startBlock('static {'); - sb.startBlock('try (InputStream in = IgniteConfiguration.class.getClassLoader().getResourceAsStream("secret.properties")) {'); - sb.append('props.load(in);'); - sb.endBlock('}'); - sb.startBlock('catch (Exception ignored) {'); - sb.append('// No-op.'); - sb.endBlock('}'); - sb.endBlock('}'); - sb.emptyLine(); - } + imports.push(...this.collectBeanImports(nearCacheBean)); - // 3. Add data sources. - const dataSources = this.collectDataSources(cfg); + nearCacheBeans.push(nearCacheBean); + }); + } - if (dataSources.length) { - this.commentBlock(sb, 'Helper class for datasource creation.'); - sb.startBlock('public static class DataSources {'); + if (_.includes(imports, 'oracle.jdbc.pool.OracleDataSource')) + imports.push('java.sql.SQLException'); - _.forEach(dataSources, (ds, idx) => { - const dsClsName = JavaTypes.shortClassName(ds.clsName); + const hasProps = this.hasProperties(cfg); - if (idx !== 0) - sb.emptyLine(); + if (hasProps) + imports.push('java.util.Properties', 'java.io.InputStream'); - sb.append(`public static final ${dsClsName} INSTANCE_${ds.id} = create${ds.id}();`); - sb.emptyLine(); + _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`)); - sb.startBlock(`private static ${dsClsName} create${ds.id}() {`); + sb.emptyLine(); - if (dsClsName === 'OracleDataSource') - sb.startBlock('try {'); + const staticImports = this._prepareImports(this.collectStaticImports(cfg)); - this.constructBean(sb, ds); + if (staticImports.length) { + _.forEach(this._prepareImports(staticImports), (cls) => sb.append(`import static ${cls};`)); - sb.emptyLine(); - sb.append(`return ${ds.id};`); + sb.emptyLine(); + } - if (dsClsName === 'OracleDataSource') { - sb.endBlock('}'); - sb.startBlock('catch (SQLException ex) {'); - sb.append('throw new Error(ex);'); - sb.endBlock('}'); - } + this.mainComment(sb); + sb.startBlock(`public class ${clsName} {`); - sb.endBlock('}'); - }); + // 2. Add external property file + if (hasProps) { + this.commentBlock(sb, 'Secret properties loading.'); + sb.append('private static final Properties props = new Properties();'); + sb.emptyLine(); + sb.startBlock('static {'); + sb.startBlock('try (InputStream in = IgniteConfiguration.class.getClassLoader().getResourceAsStream("secret.properties")) {'); + sb.append('props.load(in);'); + sb.endBlock('}'); + sb.startBlock('catch (Exception ignored) {'); + sb.append('// No-op.'); + sb.endBlock('}'); + sb.endBlock('}'); + sb.emptyLine(); + } - sb.endBlock('}'); + // 3. Add data sources. + const dataSources = this.collectDataSources(cfg); + + if (dataSources.length) { + this.commentBlock(sb, 'Helper class for datasource creation.'); + sb.startBlock('public static class DataSources {'); + + _.forEach(dataSources, (ds, idx) => { + const dsClsName = this.javaTypes.shortClassName(ds.clsName); + if (idx !== 0) + sb.emptyLine(); + + sb.append(`public static final ${dsClsName} INSTANCE_${ds.id} = create${ds.id}();`); sb.emptyLine(); - } - _.forEach(clientNearCaches, (cache) => { - this.commentBlock(sb, `Configuration of near cache for cache: ${cache.name}.`, - '', - '@return Near cache configuration.', - '@throws Exception If failed to construct near cache configuration instance.' - ); + sb.startBlock(`private static ${dsClsName} create${ds.id}() {`); - const nearCacheBean = generator.cacheNearClient(cache); + if (dsClsName === 'OracleDataSource') + sb.startBlock('try {'); - sb.startBlock(`public static NearCacheConfiguration ${nearCacheBean.id}() throws Exception {`); + this.constructBean(sb, ds); - this.constructBean(sb, nearCacheBean); sb.emptyLine(); + sb.append(`return ${ds.id};`); - sb.append(`return ${nearCacheBean.id};`); - sb.endBlock('}'); + if (dsClsName === 'OracleDataSource') { + sb.endBlock('}'); + sb.startBlock('catch (SQLException ex) {'); + sb.append('throw new Error(ex);'); + sb.endBlock('}'); + } - sb.emptyLine(); + sb.endBlock('}'); }); - this.commentBlock(sb, 'Configure grid.', + sb.endBlock('}'); + + sb.emptyLine(); + } + + _.forEach(nearCacheBeans, (nearCacheBean) => { + this.commentBlock(sb, `Configuration of near cache for cache: ${nearCacheBean.cacheName}.`, '', - '@return Ignite configuration.', - '@throws Exception If failed to construct Ignite configuration instance.' + '@return Near cache configuration.', + '@throws Exception If failed to construct near cache configuration instance.' ); - sb.startBlock('public static IgniteConfiguration createConfiguration() throws Exception {'); - this.constructBean(sb, cfg, [], true); + sb.startBlock(`public static NearCacheConfiguration ${nearCacheBean.id}() throws Exception {`); + this.constructBean(sb, nearCacheBean); sb.emptyLine(); - sb.append(`return ${cfg.id};`); - + sb.append(`return ${nearCacheBean.id};`); sb.endBlock('}'); - const beans = this.collectBeansWithMapping(cfg); + sb.emptyLine(); + }); - _.forEach(beans, (bean, id) => { - sb.emptyLine(); + this.commentBlock(sb, 'Configure grid.', + '', + '@return Ignite configuration.', + '@throws Exception If failed to construct Ignite configuration instance.' + ); + sb.startBlock('public static IgniteConfiguration createConfiguration() throws Exception {'); - this.METHOD_MAPPING[bean.clsName].generator(sb, id, bean); - }); + this.constructBean(sb, cfg, [], true); - sb.endBlock('}'); + sb.emptyLine(); - return sb; - } + sb.append(`return ${cfg.id};`); - static cluster(cluster, pkg, clsName, client) { - const cfg = this.generator.igniteConfiguration(cluster, client); + sb.endBlock('}'); - const clientNearCaches = client ? _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')) : []; + const beans = this.collectBeansWithMapping(cfg); - return this.igniteConfiguration(cfg, pkg, clsName, clientNearCaches); - } - - /** - * Generate source code for type by its domain model. - * - * @param fullClsName Full class name. - * @param fields Fields. - * @param addConstructor If 'true' then empty and full constructors should be generated. - * @returns {StringBuilder} - */ - static pojo(fullClsName, fields, addConstructor) { - const dotIdx = fullClsName.lastIndexOf('.'); + _.forEach(beans, (bean, id) => { + sb.emptyLine(); - const pkg = fullClsName.substring(0, dotIdx); - const clsName = fullClsName.substring(dotIdx + 1); + this.METHOD_MAPPING[bean.clsName].generator(sb, id, bean); + }); - const sb = new StringBuilder(); + sb.endBlock('}'); - sb.append(`package ${pkg};`); - sb.emptyLine(); - - const imports = ['java.io.Serializable']; + return sb; + } - _.forEach(fields, (field) => imports.push(JavaTypes.fullClassName(field.javaFieldType))); + static cluster(cluster, pkg, clsName, client) { + const cfg = this.generator.igniteConfiguration(cluster, client); - _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`)); + const clientNearCaches = client ? _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')) : []; - sb.emptyLine(); + return this.igniteConfiguration(cfg, pkg, clsName, clientNearCaches); + } - this.mainComment(sb, - `${clsName} definition.`, - '' - ); - sb.startBlock(`public class ${clsName} implements Serializable {`); - sb.append('/** */'); - sb.append('private static final long serialVersionUID = 0L;'); - sb.emptyLine(); + /** + * Generate source code for type by its domain model. + * + * @param fullClsName Full class name. + * @param fields Fields. + * @param addConstructor If 'true' then empty and full constructors should be generated. + * @returns {StringBuilder} + */ + static pojo(fullClsName, fields, addConstructor) { + const dotIdx = fullClsName.lastIndexOf('.'); - // Generate fields declaration. - _.forEach(fields, (field) => { - const fldName = field.javaFieldName; - const fldType = JavaTypes.shortClassName(field.javaFieldType); + const pkg = fullClsName.substring(0, dotIdx); + const clsName = fullClsName.substring(dotIdx + 1); - sb.append(`/** Value for ${fldName}. */`); - sb.append(`private ${fldType} ${fldName};`); + const sb = new StringBuilder(); - sb.emptyLine(); - }); + sb.append(`package ${pkg};`); + sb.emptyLine(); - // Generate constructors. - if (addConstructor) { - this.commentBlock(sb, 'Empty constructor.'); - sb.startBlock(`public ${clsName}() {`); - this.comment(sb, 'No-op.'); - sb.endBlock('}'); + const imports = ['java.io.Serializable']; - sb.emptyLine(); + _.forEach(fields, (field) => imports.push(this.javaTypes.fullClassName(field.javaFieldType))); - this.commentBlock(sb, 'Full constructor.'); + _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`)); - const arg = (field) => { - const fldType = JavaTypes.shortClassName(field.javaFieldType); + sb.emptyLine(); - return `${fldType} ${field.javaFieldName}`; - }; + this.mainComment(sb, + `${clsName} definition.`, + '' + ); + sb.startBlock(`public class ${clsName} implements Serializable {`); + sb.append('/** */'); + sb.append('private static final long serialVersionUID = 0L;'); + sb.emptyLine(); - sb.startBlock(`public ${clsName}(${arg(_.head(fields))}${fields.length === 1 ? ') {' : ','}`); + // Generate fields declaration. + _.forEach(fields, (field) => { + const fldName = field.javaFieldName; + const fldType = this.javaTypes.shortClassName(field.javaFieldType); - _.forEach(_.tail(fields), (field, idx) => { - sb.append(`${arg(field)}${idx !== fields.length - 2 ? ',' : ') {'}`); - }); + sb.append(`/** Value for ${fldName}. */`); + sb.append(`private ${fldType} ${fldName};`); - _.forEach(fields, (field) => sb.append(`this.${field.javaFieldName} = ${field.javaFieldName};`)); + sb.emptyLine(); + }); - sb.endBlock('}'); + // Generate constructors. + if (addConstructor) { + this.commentBlock(sb, 'Empty constructor.'); + sb.startBlock(`public ${clsName}() {`); + this.comment(sb, 'No-op.'); + sb.endBlock('}'); - sb.emptyLine(); - } + sb.emptyLine(); - // Generate getters and setters methods. - _.forEach(fields, (field) => { - const fldType = JavaTypes.shortClassName(field.javaFieldType); - const fldName = field.javaFieldName; + this.commentBlock(sb, 'Full constructor.'); - this.commentBlock(sb, - `Gets ${fldName}`, - '', - `@return Value for ${fldName}.` - ); - sb.startBlock(`public ${fldType} ${JavaTypes.toJavaName('get', fldName)}() {`); - sb.append('return ' + fldName + ';'); - sb.endBlock('}'); + const arg = (field) => { + const fldType = this.javaTypes.shortClassName(field.javaFieldType); - sb.emptyLine(); + return `${fldType} ${field.javaFieldName}`; + }; - this.commentBlock(sb, - `Sets ${fldName}`, - '', - `@param ${fldName} New value for ${fldName}.` - ); - sb.startBlock(`public void ${JavaTypes.toJavaName('set', fldName)}(${fldType} ${fldName}) {`); - sb.append(`this.${fldName} = ${fldName};`); - sb.endBlock('}'); + sb.startBlock(`public ${clsName}(${arg(_.head(fields))}${fields.length === 1 ? ') {' : ','}`); - sb.emptyLine(); + _.forEach(_.tail(fields), (field, idx) => { + sb.append(`${arg(field)}${idx !== fields.length - 2 ? ',' : ') {'}`); }); - // Generate equals() method. - this.commentBlock(sb, '{@inheritDoc}'); - sb.startBlock('@Override public boolean equals(Object o) {'); - sb.startBlock('if (this == o)'); - sb.append('return true;'); + _.forEach(fields, (field) => sb.append(`this.${field.javaFieldName} = ${field.javaFieldName};`)); - sb.endBlock(''); + sb.endBlock('}'); - sb.startBlock(`if (!(o instanceof ${clsName}))`); - sb.append('return false;'); + sb.emptyLine(); + } - sb.endBlock(''); + // Generate getters and setters methods. + _.forEach(fields, (field) => { + const fldType = this.javaTypes.shortClassName(field.javaFieldType); + const fldName = field.javaFieldName; - sb.append(`${clsName} that = (${clsName})o;`); + this.commentBlock(sb, + `Gets ${fldName}`, + '', + `@return Value for ${fldName}.` + ); + sb.startBlock(`public ${fldType} ${this.javaTypes.toJavaName('get', fldName)}() {`); + sb.append('return ' + fldName + ';'); + sb.endBlock('}'); - _.forEach(fields, (field) => { - sb.emptyLine(); + sb.emptyLine(); - const javaName = field.javaFieldName; - const javaType = field.javaFieldType; + this.commentBlock(sb, + `Sets ${fldName}`, + '', + `@param ${fldName} New value for ${fldName}.` + ); + sb.startBlock(`public void ${this.javaTypes.toJavaName('set', fldName)}(${fldType} ${fldName}) {`); + sb.append(`this.${fldName} = ${fldName};`); + sb.endBlock('}'); - switch (javaType) { - case 'float': - sb.startBlock(`if (Float.compare(${javaName}, that.${javaName}) != 0)`); + sb.emptyLine(); + }); - break; - case 'double': - sb.startBlock(`if (Double.compare(${javaName}, that.${javaName}) != 0)`); + // Generate equals() method. + this.commentBlock(sb, '{@inheritDoc}'); + sb.startBlock('@Override public boolean equals(Object o) {'); + sb.startBlock('if (this == o)'); + sb.append('return true;'); - break; - default: - if (JavaTypes.isJavaPrimitive(javaType)) - sb.startBlock('if (' + javaName + ' != that.' + javaName + ')'); - else - sb.startBlock('if (' + javaName + ' != null ? !' + javaName + '.equals(that.' + javaName + ') : that.' + javaName + ' != null)'); - } + sb.endBlock(''); - sb.append('return false;'); + sb.startBlock(`if (!(o instanceof ${clsName}))`); + sb.append('return false;'); - sb.endBlock(''); - }); + sb.endBlock(''); - sb.append('return true;'); - sb.endBlock('}'); + sb.append(`${clsName} that = (${clsName})o;`); + _.forEach(fields, (field) => { sb.emptyLine(); - // Generate hashCode() method. - this.commentBlock(sb, '{@inheritDoc}'); - sb.startBlock('@Override public int hashCode() {'); + const javaName = field.javaFieldName; + const javaType = field.javaFieldType; - let first = true; - let tempVar = false; + switch (javaType) { + case 'float': + sb.startBlock(`if (Float.compare(${javaName}, that.${javaName}) != 0)`); - _.forEach(fields, (field) => { - const javaName = field.javaFieldName; - const javaType = field.javaFieldType; + break; + case 'double': + sb.startBlock(`if (Double.compare(${javaName}, that.${javaName}) != 0)`); - let fldHashCode; + break; + default: + if (this.javaTypes.isPrimitive(javaType)) + sb.startBlock('if (' + javaName + ' != that.' + javaName + ')'); + else + sb.startBlock('if (' + javaName + ' != null ? !' + javaName + '.equals(that.' + javaName + ') : that.' + javaName + ' != null)'); + } - switch (javaType) { - case 'boolean': - fldHashCode = `${javaName} ? 1 : 0`; + sb.append('return false;'); - break; - case 'byte': - case 'short': - fldHashCode = `(int)${javaName}`; + sb.endBlock(''); + }); - break; - case 'int': - fldHashCode = `${javaName}`; + sb.append('return true;'); + sb.endBlock('}'); - break; - case 'long': - fldHashCode = `(int)(${javaName} ^ (${javaName} >>> 32))`; + sb.emptyLine(); - break; - case 'float': - fldHashCode = `${javaName} != +0.0f ? Float.floatToIntBits(${javaName}) : 0`; + // Generate hashCode() method. + this.commentBlock(sb, '{@inheritDoc}'); + sb.startBlock('@Override public int hashCode() {'); - break; - case 'double': - sb.append(`${tempVar ? 'ig_hash_temp' : 'long ig_hash_temp'} = Double.doubleToLongBits(${javaName});`); + let first = true; + let tempVar = false; - tempVar = true; + _.forEach(fields, (field) => { + const javaName = field.javaFieldName; + const javaType = field.javaFieldType; - fldHashCode = `${javaName} != +0.0f ? Float.floatToIntBits(${javaName}) : 0`; + let fldHashCode; - break; - default: - fldHashCode = `${javaName} != null ? ${javaName}.hashCode() : 0`; - } + switch (javaType) { + case 'boolean': + fldHashCode = `${javaName} ? 1 : 0`; - sb.append(first ? `int res = ${fldHashCode};` : `res = 31 * res + ${fldHashCode.startsWith('(') ? fldHashCode : `(${fldHashCode})`};`); + break; + case 'byte': + case 'short': + fldHashCode = `(int)${javaName}`; - first = false; + break; + case 'int': + fldHashCode = `${javaName}`; - sb.emptyLine(); - }); + break; + case 'long': + fldHashCode = `(int)(${javaName} ^ (${javaName} >>> 32))`; - sb.append('return res;'); - sb.endBlock('}'); + break; + case 'float': + fldHashCode = `${javaName} != +0.0f ? Float.floatToIntBits(${javaName}) : 0`; - sb.emptyLine(); + break; + case 'double': + sb.append(`${tempVar ? 'ig_hash_temp' : 'long ig_hash_temp'} = Double.doubleToLongBits(${javaName});`); - this.commentBlock(sb, '{@inheritDoc}'); - sb.startBlock('@Override public String toString() {'); - sb.startBlock(`return "${clsName} [" + `); + tempVar = true; - _.forEach(fields, (field, idx) => { - sb.append(`"${field.javaFieldName}=" + ${field.javaFieldName}${idx < fields.length - 1 ? ' + ", " + ' : ' +'}`); - }); + fldHashCode = `${javaName} != +0.0f ? Float.floatToIntBits(${javaName}) : 0`; - sb.endBlock('"]";'); - sb.endBlock('}'); + break; + default: + fldHashCode = `${javaName} != null ? ${javaName}.hashCode() : 0`; + } - sb.endBlock('}'); + sb.append(first ? `int res = ${fldHashCode};` : `res = 31 * res + ${fldHashCode.startsWith('(') ? fldHashCode : `(${fldHashCode})`};`); - return sb.asString(); - } + first = false; - /** - * Generate source code for type by its domain models. - * - * @param caches List of caches to generate POJOs for. - * @param addConstructor If 'true' then generate constructors. - * @param includeKeyFields If 'true' then include key fields into value POJO. - */ - static pojos(caches, addConstructor, includeKeyFields) { - const pojos = []; - - _.forEach(caches, (cache) => { - _.forEach(cache.domains, (domain) => { - // Process only domains with 'generatePojo' flag and skip already generated classes. - if (domain.generatePojo && !_.find(pojos, {valueType: domain.valueType}) && - // Skip domain models without value fields. - _.nonEmpty(domain.valueFields)) { - const pojo = {}; - - // Key class generation only if key is not build in java class. - if (_.nonNil(domain.keyFields) && domain.keyFields.length > 0) { - pojo.keyType = domain.keyType; - pojo.keyClass = this.pojo(domain.keyType, domain.keyFields, addConstructor); - } + sb.emptyLine(); + }); - const valueFields = _.clone(domain.valueFields); + sb.append('return res;'); + sb.endBlock('}'); - if (includeKeyFields) { - _.forEach(domain.keyFields, (fld) => { - if (!_.find(valueFields, {javaFieldName: fld.javaFieldName})) - valueFields.push(fld); - }); - } + sb.emptyLine(); - pojo.valueType = domain.valueType; - pojo.valueClass = this.pojo(domain.valueType, valueFields, addConstructor); + this.commentBlock(sb, '{@inheritDoc}'); + sb.startBlock('@Override public String toString() {'); + sb.startBlock(`return "${clsName} [" + `); - pojos.push(pojo); - } - }); - }); + _.forEach(fields, (field, idx) => { + sb.append(`"${field.javaFieldName}=" + ${field.javaFieldName}${idx < fields.length - 1 ? ' + ", " + ' : ' +'}`); + }); - return pojos; - } + sb.endBlock('"]";'); + sb.endBlock('}'); - // Generate creation and execution of cache query. - static _multilineQuery(sb, query, prefix, postfix) { - if (_.isEmpty(query)) - return; + sb.endBlock('}'); - _.forEach(query, (line, ix) => { - if (ix === 0) { - if (query.length === 1) - sb.append(`${prefix}"${line}"${postfix}`); - else - sb.startBlock(`${prefix}"${line}" +`); + return sb.asString(); + } + + /** + * Generate source code for type by its domain models. + * + * @param caches List of caches to generate POJOs for. + * @param addConstructor If 'true' then generate constructors. + * @param includeKeyFields If 'true' then include key fields into value POJO. + */ + static pojos(caches, addConstructor, includeKeyFields) { + const pojos = []; + + _.forEach(caches, (cache) => { + _.forEach(cache.domains, (domain) => { + // Process only domains with 'generatePojo' flag and skip already generated classes. + if (domain.generatePojo && !_.find(pojos, {valueType: domain.valueType}) && + // Skip domain models without value fields. + _.nonEmpty(domain.valueFields)) { + const pojo = { + keyType: domain.keyType, + valueType: domain.valueType + }; + + // Key class generation only if key is not build in java class. + if (this.javaTypes.nonBuiltInClass(domain.keyType) && _.nonEmpty(domain.keyFields)) + pojo.keyClass = this.pojo(domain.keyType, domain.keyFields, addConstructor); + + const valueFields = _.clone(domain.valueFields); + + if (includeKeyFields) { + _.forEach(domain.keyFields, (fld) => { + if (!_.find(valueFields, {javaFieldName: fld.javaFieldName})) + valueFields.push(fld); + }); + } + + pojo.valueClass = this.pojo(domain.valueType, valueFields, addConstructor); + + pojos.push(pojo); } - else - sb.append(`"${line}"${ix === query.length - 1 ? postfix : ' +'}`); }); + }); - if (query.length > 1) - sb.endBlock(''); - else - sb.emptyLine(); - } + return pojos; + } - // Generate creation and execution of prepared statement. - static _prepareStatement(sb, conVar, query) { - this._multilineQuery(sb, query, `${conVar}.prepareStatement(`, ').executeUpdate();'); - } + // Generate creation and execution of cache query. + static _multilineQuery(sb, query, prefix, postfix) { + if (_.isEmpty(query)) + return; - static demoStartup(sb, cluster, shortFactoryCls) { - const cachesWithDataSource = _.filter(cluster.caches, (cache) => { - const kind = _.get(cache, 'cacheStoreFactory.kind'); + _.forEach(query, (line, ix) => { + if (ix === 0) { + if (query.length === 1) + sb.append(`${prefix}"${line}"${postfix}`); + else + sb.startBlock(`${prefix}"${line}" +`); + } + else + sb.append(`"${line}"${ix === query.length - 1 ? postfix : ' +'}`); + }); - if (kind) { - const store = cache.cacheStoreFactory[kind]; + if (query.length > 1) + sb.endBlock(''); + else + sb.emptyLine(); + } - return (store.connectVia === 'DataSource' || _.isNil(store.connectVia)) && store.dialect; - } + // Generate creation and execution of prepared statement. + static _prepareStatement(sb, conVar, query) { + this._multilineQuery(sb, query, `${conVar}.prepareStatement(`, ').executeUpdate();'); + } - return false; - }); + static demoStartup(sb, cluster, shortFactoryCls) { + const cachesWithDataSource = _.filter(cluster.caches, (cache) => { + const kind = _.get(cache, 'cacheStoreFactory.kind'); - const uniqDomains = []; + if (kind) { + const store = cache.cacheStoreFactory[kind]; - // Prepare array of cache and his demo domain model list. Every domain is contained only in first cache. - const demoTypes = _.reduce(cachesWithDataSource, (acc, cache) => { - const domains = _.filter(cache.domains, (domain) => _.nonEmpty(domain.valueFields) && - !_.includes(uniqDomains, domain)); + return (store.connectVia === 'DataSource' || _.isNil(store.connectVia)) && store.dialect; + } - if (_.nonEmpty(domains)) { - uniqDomains.push(...domains); + return false; + }); - acc.push({ - cache, - domains - }); - } + const uniqDomains = []; - return acc; - }, []); + // Prepare array of cache and his demo domain model list. Every domain is contained only in first cache. + const demoTypes = _.reduce(cachesWithDataSource, (acc, cache) => { + const domains = _.filter(cache.domains, (domain) => _.nonEmpty(domain.valueFields) && + !_.includes(uniqDomains, domain)); - if (_.nonEmpty(demoTypes)) { - // Group domain modes by data source - const typeByDs = _.groupBy(demoTypes, ({cache}) => cache.cacheStoreFactory[cache.cacheStoreFactory.kind].dataSourceBean); + if (_.nonEmpty(domains)) { + uniqDomains.push(...domains); - let rndNonDefined = true; + acc.push({ + cache, + domains + }); + } - const generatedConsts = []; + return acc; + }, []); - _.forEach(typeByDs, (types) => { - _.forEach(types, (type) => { - _.forEach(type.domains, (domain) => { - const valType = domain.valueType.toUpperCase(); + if (_.nonEmpty(demoTypes)) { + // Group domain modes by data source + const typeByDs = _.groupBy(demoTypes, ({cache}) => cache.cacheStoreFactory[cache.cacheStoreFactory.kind].dataSourceBean); - const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type)); + let rndNonDefined = true; - if (desc) { - if (rndNonDefined && desc.rndRequired) { - this.commentBlock(sb, 'Random generator for demo data.'); - sb.append('private static final Random rnd = new Random();'); + const generatedConsts = []; - sb.emptyLine(); + _.forEach(typeByDs, (types) => { + _.forEach(types, (type) => { + _.forEach(type.domains, (domain) => { + const valType = domain.valueType.toUpperCase(); - rndNonDefined = false; - } + const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type)); - _.forEach(desc.insertCntConsts, (cnt) => { - if (!_.includes(generatedConsts, cnt.name)) { - this.commentBlock(sb, cnt.comment); - sb.append(`private static final int ${cnt.name} = ${cnt.val};`); + if (desc) { + if (rndNonDefined && desc.rndRequired) { + this.commentBlock(sb, 'Random generator for demo data.'); + sb.append('private static final Random rnd = new Random();'); - sb.emptyLine(); + sb.emptyLine(); - generatedConsts.push(cnt.name); - } - }); + rndNonDefined = false; } - }); - }); - }); - // Generation of fill database method - this.commentBlock(sb, 'Fill data for Demo.'); - sb.startBlock('private static void prepareDemoData() throws SQLException {'); + _.forEach(desc.insertCntConsts, (cnt) => { + if (!_.includes(generatedConsts, cnt.name)) { + this.commentBlock(sb, cnt.comment); + sb.append(`private static final int ${cnt.name} = ${cnt.val};`); - let firstDs = true; + sb.emptyLine(); - _.forEach(typeByDs, (types, ds) => { - const conVar = ds + 'Con'; + generatedConsts.push(cnt.name); + } + }); + } + }); + }); + }); - if (firstDs) - firstDs = false; - else - sb.emptyLine(); + // Generation of fill database method + this.commentBlock(sb, 'Fill data for Demo.'); + sb.startBlock('private static void prepareDemoData() throws SQLException {'); - sb.startBlock(`try (Connection ${conVar} = ${shortFactoryCls}.DataSources.INSTANCE_${ds}.getConnection()) {`); + let firstDs = true; - let first = true; - let stmtFirst = true; + _.forEach(typeByDs, (types, ds) => { + const conVar = ds + 'Con'; - _.forEach(types, (type) => { - _.forEach(type.domains, (domain) => { - const valType = domain.valueType.toUpperCase(); + if (firstDs) + firstDs = false; + else + sb.emptyLine(); - const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type)); + sb.startBlock(`try (Connection ${conVar} = ${shortFactoryCls}.DataSources.INSTANCE_${ds}.getConnection()) {`); - if (desc) { - if (first) - first = false; - else - sb.emptyLine(); + let first = true; + let stmtFirst = true; - this.comment(sb, `Generate ${desc.type}.`); + _.forEach(types, (type) => { + _.forEach(type.domains, (domain) => { + const valType = domain.valueType.toUpperCase(); - if (desc.schema) - this._prepareStatement(sb, conVar, [`CREATE SCHEMA IF NOT EXISTS ${desc.schema}`]); + const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type)); - this._prepareStatement(sb, conVar, desc.create); + if (desc) { + if (first) + first = false; + else + sb.emptyLine(); - this._prepareStatement(sb, conVar, desc.clearQuery); + this.comment(sb, `Generate ${desc.type}.`); - let stmtVar = 'stmt'; + if (desc.schema) + this._prepareStatement(sb, conVar, [`CREATE SCHEMA IF NOT EXISTS ${desc.schema}`]); - if (stmtFirst) { - stmtFirst = false; + this._prepareStatement(sb, conVar, desc.create); - stmtVar = 'PreparedStatement stmt'; - } + this._prepareStatement(sb, conVar, desc.clearQuery); - if (_.isFunction(desc.customGeneration)) - desc.customGeneration(sb, conVar, stmtVar); - else { - sb.append(`${stmtVar} = ${conVar}.prepareStatement("${desc.insertPattern}");`); + let stmtVar = 'stmt'; - sb.emptyLine(); + if (stmtFirst) { + stmtFirst = false; - sb.startBlock(`for (int id = 0; id < ${desc.insertCntConsts[0].name}; id ++) {`); + stmtVar = 'PreparedStatement stmt'; + } - desc.fillInsertParameters(sb); + if (_.isFunction(desc.customGeneration)) + desc.customGeneration(sb, conVar, stmtVar); + else { + sb.append(`${stmtVar} = ${conVar}.prepareStatement("${desc.insertPattern}");`); - sb.emptyLine(); + sb.emptyLine(); - sb.append('stmt.executeUpdate();'); + sb.startBlock(`for (int id = 0; id < ${desc.insertCntConsts[0].name}; id ++) {`); - sb.endBlock('}'); - } + desc.fillInsertParameters(sb); sb.emptyLine(); - sb.append(`${conVar}.commit();`); + sb.append('stmt.executeUpdate();'); + + sb.endBlock('}'); } - }); - }); - sb.endBlock('}'); + sb.emptyLine(); + + sb.append(`${conVar}.commit();`); + } + }); }); sb.endBlock('}'); + }); - sb.emptyLine(); + sb.endBlock('}'); - this.commentBlock(sb, 'Print result table to console.'); - sb.startBlock('private static void printResult(List> rows) {'); - sb.append('for (Cache.Entry row: rows)'); - sb.append(' System.out.println(row);'); - sb.endBlock('}'); + sb.emptyLine(); - sb.emptyLine(); + this.commentBlock(sb, 'Print result table to console.'); + sb.startBlock('private static void printResult(List> rows) {'); + sb.append('for (Cache.Entry row: rows)'); + sb.append(' System.out.println(row);'); + sb.endBlock('}'); - // Generation of execute queries method. - this.commentBlock(sb, 'Run demo.'); - sb.startBlock('private static void runDemo(Ignite ignite) throws SQLException {'); + sb.emptyLine(); - const getType = (fullType) => fullType.substr(fullType.lastIndexOf('.') + 1); + // Generation of execute queries method. + this.commentBlock(sb, 'Run demo.'); + sb.startBlock('private static void runDemo(Ignite ignite) throws SQLException {'); - const cacheLoaded = []; - let rowVariableDeclared = false; - firstDs = true; + const getType = (fullType) => fullType.substr(fullType.lastIndexOf('.') + 1); - _.forEach(typeByDs, (types, ds) => { - const conVar = ds + 'Con'; + const cacheLoaded = []; + let rowVariableDeclared = false; + firstDs = true; - if (firstDs) - firstDs = false; - else - sb.emptyLine(); + _.forEach(typeByDs, (types, ds) => { + const conVar = ds + 'Con'; - sb.startBlock(`try (Connection ${conVar} = ${shortFactoryCls}.DataSources.INSTANCE_${ds}.getConnection()) {`); + if (firstDs) + firstDs = false; + else + sb.emptyLine(); - let first = true; + sb.startBlock(`try (Connection ${conVar} = ${shortFactoryCls}.DataSources.INSTANCE_${ds}.getConnection()) {`); - _.forEach(types, (type) => { - _.forEach(type.domains, (domain) => { - const valType = domain.valueType.toUpperCase(); + let first = true; - const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type)); + _.forEach(types, (type) => { + _.forEach(type.domains, (domain) => { + const valType = domain.valueType.toUpperCase(); - if (desc) { - if (_.isEmpty(desc.selectQuery)) - return; + const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type)); - if (first) - first = false; - else - sb.emptyLine(); + if (desc) { + if (_.isEmpty(desc.selectQuery)) + return; - const cacheName = type.cache.name; + if (first) + first = false; + else + sb.emptyLine(); - if (!_.includes(cacheLoaded, cacheName)) { - sb.append(`ignite.cache("${cacheName}").loadCache(null);`); + const cacheName = type.cache.name; - sb.emptyLine(); + if (!_.includes(cacheLoaded, cacheName)) { + sb.append(`ignite.cache("${cacheName}").loadCache(null);`); - cacheLoaded.push(cacheName); - } + sb.emptyLine(); - const varRows = rowVariableDeclared ? 'rows' : 'List> rows'; + cacheLoaded.push(cacheName); + } - this._multilineQuery(sb, desc.selectQuery, `${varRows} = ignite.cache("${cacheName}").query(new SqlQuery<>("${getType(domain.valueType)}", `, ')).getAll();'); + const varRows = rowVariableDeclared ? 'rows' : 'List> rows'; - sb.append('printResult(rows);'); + this._multilineQuery(sb, desc.selectQuery, `${varRows} = ignite.cache("${cacheName}").query(new SqlQuery<>("${getType(domain.valueType)}", `, ')).getAll();'); - rowVariableDeclared = true; - } - }); - }); + sb.append('printResult(rows);'); - sb.endBlock('}'); + rowVariableDeclared = true; + } + }); }); sb.endBlock('}'); - } - } - - /** - * Function to generate java class for node startup with cluster configuration. - * - * @param {Object} cluster Cluster to process. - * @param {String} fullClsName Full class name. - * @param {String} cfgRef Config. - * @param {String} [factoryCls] fully qualified class name of configuration factory. - * @param {Array.} [clientNearCaches] Is client node. - */ - static nodeStartup(cluster, fullClsName, cfgRef, factoryCls, clientNearCaches) { - const dotIdx = fullClsName.lastIndexOf('.'); - - const pkg = fullClsName.substring(0, dotIdx); - const clsName = fullClsName.substring(dotIdx + 1); + }); - const demo = clsName === 'DemoStartup'; + sb.endBlock('}'); + } + } - const sb = new StringBuilder(); + /** + * Function to generate java class for node startup with cluster configuration. + * + * @param {Object} cluster Cluster to process. + * @param {String} fullClsName Full class name. + * @param {String} cfgRef Config. + * @param {String} [factoryCls] fully qualified class name of configuration factory. + * @param {Array.} [clientNearCaches] Is client node. + */ + static nodeStartup(cluster, fullClsName, cfgRef, factoryCls, clientNearCaches) { + const dotIdx = fullClsName.lastIndexOf('.'); - const imports = ['org.apache.ignite.Ignition', 'org.apache.ignite.Ignite']; + const pkg = fullClsName.substring(0, dotIdx); + const clsName = fullClsName.substring(dotIdx + 1); - if (demo) { - imports.push('org.h2.tools.Server', 'java.sql.Connection', 'java.sql.PreparedStatement', - 'java.sql.SQLException', 'java.util.Random', 'java.util.List', 'javax.cache.Cache', - 'org.apache.ignite.cache.query.SqlQuery'); - } + const demo = clsName === 'DemoStartup'; - let shortFactoryCls; + const sb = new StringBuilder(); - if (factoryCls) { - imports.push(factoryCls); + const imports = ['org.apache.ignite.Ignition', 'org.apache.ignite.Ignite']; - shortFactoryCls = JavaTypes.shortClassName(factoryCls); - } + if (demo) { + imports.push('org.h2.tools.Server', 'java.sql.Connection', 'java.sql.PreparedStatement', + 'java.sql.SQLException', 'java.util.Random', 'java.util.List', 'javax.cache.Cache', + 'org.apache.ignite.cache.query.SqlQuery'); + } - sb.append(`package ${pkg};`) - .emptyLine(); + let shortFactoryCls; - _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`)); - sb.emptyLine(); + if (factoryCls) { + imports.push(factoryCls); - if (demo) { - this.mainComment(sb, - 'To start demo configure data sources in secret.properties file.', - 'For H2 database it should be like following:', - 'dsH2.jdbc.url=jdbc:h2:tcp://localhost/mem:DemoDB;DB_CLOSE_DELAY=-1', - 'dsH2.jdbc.username=sa', - 'dsH2.jdbc.password=', - '' - ); - } - else - this.mainComment(sb); + shortFactoryCls = this.javaTypes.shortClassName(factoryCls); + } - sb.startBlock(`public class ${clsName} {`); + sb.append(`package ${pkg};`) + .emptyLine(); - if (demo && shortFactoryCls) - this.demoStartup(sb, cluster, shortFactoryCls); + _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`)); + sb.emptyLine(); - this.commentBlock(sb, - 'Start up node with specified configuration.', - '', - '@param args Command line arguments, none required.', - '@throws Exception If failed.' + if (demo) { + this.mainComment(sb, + 'To start demo configure data sources in secret.properties file.', + 'For H2 database it should be like following:', + 'dsH2.jdbc.url=jdbc:h2:tcp://localhost/mem:DemoDB;DB_CLOSE_DELAY=-1', + 'dsH2.jdbc.username=sa', + 'dsH2.jdbc.password=', + '' ); - sb.startBlock('public static void main(String[] args) throws Exception {'); - - if (demo) { - sb.startBlock('try {'); - sb.append('// Start H2 database server.'); - sb.append('Server.createTcpServer("-tcpDaemon").start();'); - sb.endBlock('}'); - sb.startBlock('catch (SQLException ignore) {'); - sb.append('// No-op.'); - sb.endBlock('}'); - - sb.emptyLine(); - } - - if ((_.nonEmpty(clientNearCaches) || demo) && shortFactoryCls) { - sb.append(`Ignite ignite = Ignition.start(${cfgRef});`); + } + else + this.mainComment(sb); - _.forEach(clientNearCaches, (cache, idx) => { - sb.emptyLine(); + sb.startBlock(`public class ${clsName} {`); - if (idx === 0) - sb.append('// Demo of near cache creation on client node.'); + if (demo && shortFactoryCls) + this.demoStartup(sb, cluster, shortFactoryCls); - const nearCacheMtd = JavaTypes.toJavaName('nearConfiguration', cache.name); + this.commentBlock(sb, + 'Start up node with specified configuration.', + '', + '@param args Command line arguments, none required.', + '@throws Exception If failed.' + ); + sb.startBlock('public static void main(String[] args) throws Exception {'); - sb.append(`ignite.getOrCreateCache(${shortFactoryCls}.${cache.name}(), ${shortFactoryCls}.${nearCacheMtd}());`); - }); - } - else - sb.append(`Ignition.start(${cfgRef});`); + if (demo) { + sb.startBlock('try {'); + sb.append('// Start H2 database server.'); + sb.append('Server.createTcpServer("-tcpDaemon").start();'); + sb.endBlock('}'); + sb.startBlock('catch (SQLException ignore) {'); + sb.append('// No-op.'); + sb.endBlock('}'); - if (demo) { - sb.emptyLine(); + sb.emptyLine(); + } - sb.append('prepareDemoData();'); + if ((_.nonEmpty(clientNearCaches) || demo) && shortFactoryCls) { + sb.append(`Ignite ignite = Ignition.start(${cfgRef});`); + _.forEach(clientNearCaches, (cache, idx) => { sb.emptyLine(); - sb.append('runDemo(ignite);'); - } + if (idx === 0) + sb.append('// Demo of near cache creation on client node.'); - sb.endBlock('}'); - - sb.endBlock('}'); + const nearCacheMtd = this.javaTypes.toJavaName('nearConfiguration', cache.name); - return sb.asString(); + sb.append(`ignite.getOrCreateCache(${shortFactoryCls}.${this.javaTypes.toJavaName('cache', cache.name)}(), ${shortFactoryCls}.${nearCacheMtd}());`); + }); } + else + sb.append(`Ignition.start(${cfgRef});`); - /** - * Function to generate java class for load caches. - * - * @param caches Caches to load. - * @param pkg Class package name. - * @param clsName Class name. - * @param {String} cfgRef Config. - */ - static loadCaches(caches, pkg, clsName, cfgRef) { - const sb = new StringBuilder(); - - sb.append(`package ${pkg};`) - .emptyLine(); - - const imports = ['org.apache.ignite.Ignition', 'org.apache.ignite.Ignite']; - - _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`)); + if (demo) { sb.emptyLine(); - this.mainComment(sb); - sb.startBlock(`public class ${clsName} {`); + sb.append('prepareDemoData();'); - this.commentBlock(sb, - '

          ', - 'Utility to load caches from database.', - '

          ', - 'How to use:', - '

            ', - '
          • Start cluster.
          • ', - '
          • Start this utility and wait while load complete.
          • ', - '
          ', - '', - '@param args Command line arguments, none required.', - '@throws Exception If failed.' - ); - sb.startBlock('public static void main(String[] args) throws Exception {'); + sb.emptyLine(); - sb.startBlock(`try (Ignite ignite = Ignition.start(${cfgRef})) {`); + sb.append('runDemo(ignite);'); + } - sb.append('System.out.println(">>> Loading caches...");'); + sb.endBlock('}'); - sb.emptyLine(); + sb.endBlock('}'); - _.forEach(caches, (cache) => { - sb.append('System.out.println(">>> Loading cache: ' + cache.name + '");'); - sb.append('ignite.cache("' + cache.name + '").loadCache(null);'); + return sb.asString(); + } - sb.emptyLine(); - }); + /** + * Function to generate java class for load caches. + * + * @param caches Caches to load. + * @param pkg Class package name. + * @param clsName Class name. + * @param {String} cfgRef Config. + */ + static loadCaches(caches, pkg, clsName, cfgRef) { + const sb = new StringBuilder(); + + sb.append(`package ${pkg};`) + .emptyLine(); + + const imports = ['org.apache.ignite.Ignition', 'org.apache.ignite.Ignite']; + + _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`)); + sb.emptyLine(); + + this.mainComment(sb); + sb.startBlock(`public class ${clsName} {`); + + this.commentBlock(sb, + '

          ', + 'Utility to load caches from database.', + '

          ', + 'How to use:', + '

            ', + '
          • Start cluster.
          • ', + '
          • Start this utility and wait while load complete.
          • ', + '
          ', + '', + '@param args Command line arguments, none required.', + '@throws Exception If failed.' + ); + sb.startBlock('public static void main(String[] args) throws Exception {'); + + sb.startBlock(`try (Ignite ignite = Ignition.start(${cfgRef})) {`); + + sb.append('System.out.println(">>> Loading caches...");'); + + sb.emptyLine(); + + _.forEach(caches, (cache) => { + sb.append('System.out.println(">>> Loading cache: ' + cache.name + '");'); + sb.append('ignite.cache("' + cache.name + '").loadCache(null);'); - sb.append('System.out.println(">>> All caches loaded!");'); + sb.emptyLine(); + }); - sb.endBlock('}'); + sb.append('System.out.println(">>> All caches loaded!");'); - sb.endBlock('}'); + sb.endBlock('}'); - sb.endBlock('}'); + sb.endBlock('}'); - return sb.asString(); - } + sb.endBlock('}'); - /** - * Checks if cluster has demo types. - * - * @param cluster Cluster to check. - * @param demo Is demo enabled. - * @returns {boolean} True if cluster has caches with demo types. - */ - static isDemoConfigured(cluster, demo) { - return demo && _.find(cluster.caches, (cache) => _.find(cache.domains, (domain) => _.find(PREDEFINED_QUERIES, (desc) => domain.valueType.toUpperCase().endsWith(desc.type)))); - } + return sb.asString(); } - return JavaTransformer; -}]; + /** + * Checks if cluster has demo types. + * + * @param cluster Cluster to check. + * @param demo Is demo enabled. + * @returns {boolean} True if cluster has caches with demo types. + */ + static isDemoConfigured(cluster, demo) { + return demo && _.find(cluster.caches, (cache) => _.find(cache.domains, (domain) => _.find(PREDEFINED_QUERIES, (desc) => domain.valueType.toUpperCase().endsWith(desc.type)))); + } +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js similarity index 97% rename from modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js rename to modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js index db58532187780..2e017610e0e81 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js @@ -23,7 +23,7 @@ import POM_DEPENDENCIES from 'app/data/pom-dependencies.json'; /** * Pom file generation entry point. */ -class GeneratorPom { +export default class IgniteMavenGenerator { escapeId(s) { if (typeof (s) !== 'string') return s; @@ -184,6 +184,9 @@ class GeneratorPom { this.addDependency(deps, 'org.apache.ignite', 'ignite-indexing', version); this.addDependency(deps, 'org.apache.ignite', 'ignite-rest-http', version); + if (_.get(cluster, 'deploymentSpi.kind') === 'URI') + this.addDependency(deps, 'org.apache.ignite', 'ignite-urideploy', version); + let dep = POM_DEPENDENCIES[cluster.discovery.kind]; if (dep) @@ -229,5 +232,3 @@ class GeneratorPom { return sb; } } - -export default ['GeneratorPom', GeneratorPom]; diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js index 49b4aa69df949..8a6a4715db439 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js @@ -20,7 +20,7 @@ import StringBuilder from './StringBuilder'; /** * Properties generation entry point. */ -export default class PropertiesGenerator { +export default class IgnitePropertiesGenerator { _collectProperties(bean) { const props = []; diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js index 7043807fb818c..0aa34ee2878b9 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js @@ -20,7 +20,7 @@ import StringBuilder from './StringBuilder'; /** * Properties generation entry point. */ -export default class ReadmeGenerator { +export default class IgniteReadmeGenerator { header(sb) { sb.append('Content of this folder was generated by Apache Ignite Web Console'); sb.append('================================================================='); diff --git a/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js index 19043f6f8a57a..6e6bffe92298c 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js @@ -19,225 +19,238 @@ import _ from 'lodash'; import AbstractTransformer from './AbstractTransformer'; import StringBuilder from './StringBuilder'; -export default ['JavaTypes', 'IgnitePlatformGenerator', (JavaTypes, generator) => { - return class SharpTransformer extends AbstractTransformer { - static generator = generator; - - static commentBlock(sb, ...lines) { - _.forEach(lines, (line) => sb.append(`// ${line}`)); - } - - static doc(sb, ...lines) { - sb.append('/// '); - _.forEach(lines, (line) => sb.append(`/// ${line}`)); - sb.append('/// '); - } - - static mainComment(sb) { - return this.doc(sb, sb.generatedBy()); - } - - /** - * - * @param {Array.} sb - * @param {Bean} bean - */ - static _defineBean(sb, bean) { - const shortClsName = JavaTypes.shortClassName(bean.clsName); - - sb.append(`var ${bean.id} = new ${shortClsName}();`); - } - - /** - * @param {StringBuilder} sb - * @param {Bean} parent - * @param {Bean} propertyName - * @param {String|Bean} value - * @private - */ - static _setProperty(sb, parent, propertyName, value) { - sb.append(`${parent.id}.${_.upperFirst(propertyName)} = ${value};`); - } - - /** - * - * @param {StringBuilder} sb - * @param {Bean} parent - * @param {String} propertyName - * @param {Bean} bean - * @private - */ - static _setBeanProperty(sb, parent, propertyName, bean) { - sb.append(`${parent.id}.${_.upperFirst(propertyName)} = ${bean.id};`); - } - - static _toObject(clsName, val) { - const items = _.isArray(val) ? val : [val]; - - return _.map(items, (item, idx) => { - if (_.isNil(item)) - return 'null'; - - const shortClsName = JavaTypes.shortClassName(clsName); - - switch (shortClsName) { - // case 'byte': - // return `(byte) ${item}`; - // case 'Serializable': - case 'String': - if (items.length > 1) - return `"${item}"${idx !== items.length - 1 ? ' +' : ''}`; - - return `"${item}"`; - // case 'Path': - // return `"${item.replace(/\\/g, '\\\\')}"`; - // case 'Class': - // return `${this.shortClassName(item)}.class`; - // case 'UUID': - // return `UUID.fromString("${item}")`; - // case 'PropertyChar': - // return `props.getProperty("${item}").toCharArray()`; - // case 'Property': - // return `props.getProperty("${item}")`; - // case 'Bean': - // if (item.isComplex()) - // return item.id; +import ConfigurationGenerator from './ConfigurationGenerator'; + +import ClusterDefaults from './defaults/Cluster.service'; +import CacheDefaults from './defaults/Cache.service'; +import IGFSDefaults from './defaults/IGFS.service'; + +import JavaTypes from '../../../services/JavaTypes.service'; + +const generator = new ConfigurationGenerator(); + +const clusterDflts = new ClusterDefaults(); +const cacheDflts = new CacheDefaults(); +const igfsDflts = new IGFSDefaults(); + +const javaTypes = new JavaTypes(clusterDflts, cacheDflts, igfsDflts); + +export default class SharpTransformer extends AbstractTransformer { + static generator = generator; + + static commentBlock(sb, ...lines) { + _.forEach(lines, (line) => sb.append(`// ${line}`)); + } + + static doc(sb, ...lines) { + sb.append('/// '); + _.forEach(lines, (line) => sb.append(`/// ${line}`)); + sb.append('/// '); + } + + static mainComment(sb) { + return this.doc(sb, sb.generatedBy()); + } + + /** + * + * @param {Array.} sb + * @param {Bean} bean + */ + static _defineBean(sb, bean) { + const shortClsName = javaTypes.shortClassName(bean.clsName); + + sb.append(`var ${bean.id} = new ${shortClsName}();`); + } + + /** + * @param {StringBuilder} sb + * @param {Bean} parent + * @param {Bean} propertyName + * @param {String|Bean} value + * @private + */ + static _setProperty(sb, parent, propertyName, value) { + sb.append(`${parent.id}.${_.upperFirst(propertyName)} = ${value};`); + } + + /** + * + * @param {StringBuilder} sb + * @param {Bean} parent + * @param {String} propertyName + * @param {Bean} bean + * @private + */ + static _setBeanProperty(sb, parent, propertyName, bean) { + sb.append(`${parent.id}.${_.upperFirst(propertyName)} = ${bean.id};`); + } + + static _toObject(clsName, val) { + const items = _.isArray(val) ? val : [val]; + + return _.map(items, (item, idx) => { + if (_.isNil(item)) + return 'null'; + + const shortClsName = javaTypes.shortClassName(clsName); + + switch (shortClsName) { + // case 'byte': + // return `(byte) ${item}`; + // case 'Serializable': + case 'String': + if (items.length > 1) + return `"${item}"${idx !== items.length - 1 ? ' +' : ''}`; + + return `"${item}"`; + // case 'Path': + // return `"${item.replace(/\\/g, '\\\\')}"`; + // case 'Class': + // return `${this.shortClassName(item)}.class`; + // case 'UUID': + // return `UUID.fromString("${item}")`; + // case 'PropertyChar': + // return `props.getProperty("${item}").toCharArray()`; + // case 'Property': + // return `props.getProperty("${item}")`; + // case 'Bean': + // if (item.isComplex()) + // return item.id; + // + // return this._newBean(item); + default: + if (javaTypes.nonEnum(shortClsName)) + return item; + + return `${shortClsName}.${item}`; + } + }); + } + + /** + * + * @param {StringBuilder} sb + * @param {Bean} bean + * @returns {Array} + */ + static _setProperties(sb = new StringBuilder(), bean) { + _.forEach(bean.properties, (prop) => { + switch (prop.clsName) { + case 'ICollection': + // const implClsName = JavaTypes.shortClassName(prop.implClsName); + + const colTypeClsName = javaTypes.shortClassName(prop.typeClsName); + + if (colTypeClsName === 'String') { + const items = this._toObject(colTypeClsName, prop.items); + + sb.append(`${bean.id}.${_.upperFirst(prop.name)} = new {${items.join(', ')}};`); + } + // else { + // if (_.includes(vars, prop.id)) + // sb.append(`${prop.id} = new ${implClsName}<>();`); + // else { + // vars.push(prop.id); // - // return this._newBean(item); - default: - if (JavaTypes.nonEnum(shortClsName)) - return item; - - return `${shortClsName}.${item}`; - } - }); - } - - /** - * - * @param {StringBuilder} sb - * @param {Bean} bean - * @returns {Array} - */ - static _setProperties(sb = new StringBuilder(), bean) { - _.forEach(bean.properties, (prop) => { - switch (prop.clsName) { - case 'ICollection': - // const implClsName = JavaTypes.shortClassName(prop.implClsName); - - const colTypeClsName = JavaTypes.shortClassName(prop.typeClsName); - - if (colTypeClsName === 'String') { - const items = this._toObject(colTypeClsName, prop.items); - - sb.append(`${bean.id}.${_.upperFirst(prop.name)} = new {${items.join(', ')}};`); - } - // else { - // if (_.includes(vars, prop.id)) - // sb.append(`${prop.id} = new ${implClsName}<>();`); - // else { - // vars.push(prop.id); - // - // sb.append(`${clsName}<${colTypeClsName}> ${prop.id} = new ${implClsName}<>();`); - // } - // - // sb.emptyLine(); - // - // if (nonBean) { - // const items = this._toObject(colTypeClsName, prop.items); - // - // _.forEach(items, (item) => { - // sb.append(`${prop.id}.add("${item}");`); - // - // sb.emptyLine(); - // }); - // } - // else { - // _.forEach(prop.items, (item) => { - // this.constructBean(sb, item, vars, limitLines); - // - // sb.append(`${prop.id}.add(${item.id});`); - // - // sb.emptyLine(); - // }); - // - // this._setProperty(sb, bean.id, prop.name, prop.id); - // } - // } - - break; - - case 'Bean': - const nestedBean = prop.value; - - this._defineBean(sb, nestedBean); - - sb.emptyLine(); - - this._setProperties(sb, nestedBean); - - sb.emptyLine(); - - this._setBeanProperty(sb, bean, prop.name, nestedBean); - - break; - default: - this._setProperty(sb, bean, prop.name, this._toObject(prop.clsName, prop.value)); - } - }); - - return sb; - } - - /** - * Build Java startup class with configuration. - * - * @param {Bean} cfg - * @param pkg Package name. - * @param clsName Class name for generate factory class otherwise generate code snippet. - * @param clientNearCfg Optional near cache configuration for client node. - * @returns {String} - */ - static toClassFile(cfg, pkg, clsName) { - const sb = new StringBuilder(); - - sb.startBlock(`namespace ${pkg}`, '{'); - - _.forEach(_.sortBy(cfg.collectClasses()), (cls) => sb.append(`using ${cls};`)); - sb.emptyLine(); - - - this.mainComment(sb); - sb.startBlock(`public class ${clsName}`, '{'); - - this.doc(sb, 'Configure grid.'); - sb.startBlock('public static IgniteConfiguration CreateConfiguration()', '{'); - - this._defineBean(sb, cfg); - - sb.emptyLine(); - - this._setProperties(sb, cfg); + // sb.append(`${clsName}<${colTypeClsName}> ${prop.id} = new ${implClsName}<>();`); + // } + // + // sb.emptyLine(); + // + // if (nonBean) { + // const items = this._toObject(colTypeClsName, prop.items); + // + // _.forEach(items, (item) => { + // sb.append(`${prop.id}.add("${item}");`); + // + // sb.emptyLine(); + // }); + // } + // else { + // _.forEach(prop.items, (item) => { + // this.constructBean(sb, item, vars, limitLines); + // + // sb.append(`${prop.id}.add(${item.id});`); + // + // sb.emptyLine(); + // }); + // + // this._setProperty(sb, bean.id, prop.name, prop.id); + // } + // } + + break; + + case 'Bean': + const nestedBean = prop.value; + + this._defineBean(sb, nestedBean); + + sb.emptyLine(); + + this._setProperties(sb, nestedBean); + + sb.emptyLine(); + + this._setBeanProperty(sb, bean, prop.name, nestedBean); + + break; + default: + this._setProperty(sb, bean, prop.name, this._toObject(prop.clsName, prop.value)); + } + }); + + return sb; + } + + /** + * Build Java startup class with configuration. + * + * @param {Bean} cfg + * @param pkg Package name. + * @param clsName Class name for generate factory class otherwise generate code snippet. + * @returns {String} + */ + static toClassFile(cfg, pkg, clsName) { + const sb = new StringBuilder(); + + sb.startBlock(`namespace ${pkg}`, '{'); + + _.forEach(_.sortBy(cfg.collectClasses()), (cls) => sb.append(`using ${cls};`)); + sb.emptyLine(); + + + this.mainComment(sb); + sb.startBlock(`public class ${clsName}`, '{'); + + this.doc(sb, 'Configure grid.'); + sb.startBlock('public static IgniteConfiguration CreateConfiguration()', '{'); + + this._defineBean(sb, cfg); + + sb.emptyLine(); + + this._setProperties(sb, cfg); - sb.emptyLine(); + sb.emptyLine(); - sb.append(`return ${cfg.id};`); + sb.append(`return ${cfg.id};`); - sb.endBlock('}'); + sb.endBlock('}'); - sb.endBlock('}'); + sb.endBlock('}'); - sb.endBlock('}'); + sb.endBlock('}'); - return sb.asString(); - } + return sb.asString(); + } - static generateSection(bean) { - const sb = new StringBuilder(); + static generateSection(bean) { + const sb = new StringBuilder(); - this._setProperties(sb, bean); + this._setProperties(sb, bean); - return sb.asString(); - } - }; -}]; + return sb.asString(); + } +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js index 73df25e000e53..b2345759886f8 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js @@ -20,314 +20,311 @@ import _ from 'lodash'; import AbstractTransformer from './AbstractTransformer'; import StringBuilder from './StringBuilder'; -const escapeXml = (str) => { - return str.replace(/&/g, '&') - .replace(/"/g, '"') - .replace(/'/g, ''') - .replace(/>/g, '>') - .replace(/ { - return class SpringTransformer extends AbstractTransformer { - static generator = generator; - - static commentBlock(sb, ...lines) { - if (lines.length > 1) { - sb.append(''); - } - else - sb.append(``); +export default class IgniteSpringTransformer extends AbstractTransformer { + static escapeXml(str) { + return str.replace(/&/g, '&') + .replace(/"/g, '"') + .replace(/'/g, ''') + .replace(/>/g, '>') + .replace(/ 1) { + sb.append(''); } + else + sb.append(``); + } - static appendBean(sb, bean, appendId) { - const beanTags = []; - - if (appendId) - beanTags.push(`id="${bean.id}"`); - - beanTags.push(`class="${bean.clsName}"`); - - if (bean.factoryMtd) - beanTags.push(`factory-method="${bean.factoryMtd}"`); - - sb.startBlock(``); - - _.forEach(bean.arguments, (arg) => { - if (arg.clsName === 'MAP') { - sb.startBlock(''); - this._constructMap(sb, arg); - sb.endBlock(''); - } - else if (_.isNil(arg.value)) { - sb.startBlock(''); - sb.append(''); - sb.endBlock(''); - } - else if (arg.constant) { - sb.startBlock(''); - sb.append(``); - sb.endBlock(''); - } - else if (arg.clsName === 'BEAN') { - sb.startBlock(''); - this.appendBean(sb, arg.value); - sb.endBlock(''); - } - else - sb.append(``); - }); + static appendBean(sb, bean, appendId) { + const beanTags = []; - this._setProperties(sb, bean); + if (appendId) + beanTags.push(`id="${bean.id}"`); - sb.endBlock(''); - } + beanTags.push(`class="${bean.clsName}"`); - static _toObject(clsName, items) { - return _.map(_.isArray(items) ? items : [items], (item) => { - switch (clsName) { - case 'PROPERTY': - case 'PROPERTY_CHAR': - case 'PROPERTY_INT': - return `\${${item}}`; - case 'java.lang.Class': - return JavaTypes.fullClassName(item); - case 'long': - return `${item}L`; - case 'java.lang.String': - return escapeXml(item); - default: - return item; - } - }); - } + if (bean.factoryMtd) + beanTags.push(`factory-method="${bean.factoryMtd}"`); - static _isBean(clsName) { - return JavaTypes.nonBuiltInClass(clsName) && JavaTypes.nonEnum(clsName) && _.includes(clsName, '.'); - } + sb.startBlock(``); - static _setCollection(sb, prop) { - sb.startBlock(``); - sb.startBlock(''); + _.forEach(bean.arguments, (arg) => { + if (arg.clsName === 'MAP') { + sb.startBlock(''); + this._constructMap(sb, arg); + sb.endBlock(''); + } + else if (_.isNil(arg.value)) { + sb.startBlock(''); + sb.append(''); + sb.endBlock(''); + } + else if (arg.constant) { + sb.startBlock(''); + sb.append(``); + sb.endBlock(''); + } + else if (arg.clsName === 'BEAN') { + sb.startBlock(''); + this.appendBean(sb, arg.value); + sb.endBlock(''); + } + else + sb.append(``); + }); + + this._setProperties(sb, bean); + + sb.endBlock(''); + } + + static _toObject(clsName, items) { + return _.map(_.isArray(items) ? items : [items], (item) => { + switch (clsName) { + case 'PROPERTY': + case 'PROPERTY_CHAR': + case 'PROPERTY_INT': + return `\${${item}}`; + case 'java.lang.Class': + return this.javaTypes.fullClassName(item); + case 'long': + return `${item}L`; + case 'java.lang.String': + case 'PATH': + return this.escapeXml(item); + default: + return item; + } + }); + } - _.forEach(prop.items, (item, idx) => { - if (this._isBean(prop.typeClsName)) { - if (idx !== 0) - sb.emptyLine(); + static _isBean(clsName) { + return this.javaTypes.nonBuiltInClass(clsName) && this.javaTypes.nonEnum(clsName) && _.includes(clsName, '.'); + } - this.appendBean(sb, item); - } - else - sb.append(`${item}`); - }); + static _setCollection(sb, prop) { + sb.startBlock(``); + sb.startBlock(''); - sb.endBlock(''); - sb.endBlock(''); - } + _.forEach(prop.items, (item, idx) => { + if (this._isBean(prop.typeClsName)) { + if (idx !== 0) + sb.emptyLine(); - static _constructMap(sb, map) { - sb.startBlock(''); + this.appendBean(sb, item); + } + else + sb.append(`${item}`); + }); - _.forEach(map.entries, (entry) => { - const key = entry[map.keyField]; - const val = entry[map.valField]; + sb.endBlock(''); + sb.endBlock(''); + } - const isKeyBean = this._isBean(map.keyClsName); - const isValBean = this._isBean(map.valClsName); + static _constructMap(sb, map) { + sb.startBlock(''); + _.forEach(map.entries, (entry) => { + const key = entry[map.keyField]; + const val = entry[map.valField]; - if (isKeyBean || isValBean) { - sb.startBlock(''); + const isKeyBean = this._isBean(map.keyClsName); + const isValBean = this._isBean(map.valClsName); - sb.startBlock(''); - if (isKeyBean) - this.appendBean(sb, key); - else - sb.append(this._toObject(map.keyClsName, key)); - sb.endBlock(''); - sb.startBlock(''); - if (isValBean) - this.appendBean(sb, val); - else - sb.append(this._toObject(map.valClsName, val)); - sb.endBlock(''); + if (isKeyBean || isValBean) { + sb.startBlock(''); - sb.endBlock(''); - } + sb.startBlock(''); + if (isKeyBean) + this.appendBean(sb, key); else - sb.append(``); - }); + sb.append(this._toObject(map.keyClsName, key)); + sb.endBlock(''); - sb.endBlock(''); - } + sb.startBlock(''); + if (isValBean) + this.appendBean(sb, val); + else + sb.append(this._toObject(map.valClsName, val)); + sb.endBlock(''); - /** - * - * @param {StringBuilder} sb - * @param {Bean} bean - * @returns {StringBuilder} - */ - static _setProperties(sb, bean) { - _.forEach(bean.properties, (prop, idx) => { - switch (prop.clsName) { - case 'DATA_SOURCE': - const valAttr = prop.name === 'dataSource' ? 'ref' : 'value'; + sb.endBlock(''); + } + else + sb.append(``); + }); - sb.append(``); + sb.endBlock(''); + } - break; - case 'EVENT_TYPES': - sb.startBlock(``); + /** + * + * @param {StringBuilder} sb + * @param {Bean} bean + * @returns {StringBuilder} + */ + static _setProperties(sb, bean) { + _.forEach(bean.properties, (prop, idx) => { + switch (prop.clsName) { + case 'DATA_SOURCE': + const valAttr = prop.name === 'dataSource' ? 'ref' : 'value'; - if (prop.eventTypes.length === 1) { - const evtGrp = _.find(eventGroups, {value: _.head(prop.eventTypes)}); + sb.append(``); - evtGrp && sb.append(``); - } - else { - sb.startBlock(''); + break; + case 'EVENT_TYPES': + sb.startBlock(``); - _.forEach(prop.eventTypes, (item, ix) => { - ix > 0 && sb.emptyLine(); + if (prop.eventTypes.length === 1) { + const evtGrp = _.find(this.eventGroups, {value: _.head(prop.eventTypes)}); - const evtGrp = _.find(eventGroups, {value: item}); + evtGrp && sb.append(``); + } + else { + sb.startBlock(''); - if (evtGrp) { - sb.append(``); + _.forEach(prop.eventTypes, (item, ix) => { + ix > 0 && sb.emptyLine(); - _.forEach(evtGrp.events, (event) => - sb.append(``)); - } - }); + const evtGrp = _.find(this.eventGroups, {value: item}); - sb.endBlock(''); - } + if (evtGrp) { + sb.append(``); - sb.endBlock(''); + _.forEach(evtGrp.events, (event) => + sb.append(``)); + } + }); - break; - case 'ARRAY': - case 'COLLECTION': - this._setCollection(sb, prop); + sb.endBlock(''); + } - break; - case 'MAP': - sb.startBlock(``); + sb.endBlock(''); - this._constructMap(sb, prop); + break; + case 'ARRAY': + case 'COLLECTION': + this._setCollection(sb, prop); - sb.endBlock(''); + break; + case 'MAP': + sb.startBlock(``); - break; - case 'java.util.Properties': - sb.startBlock(``); - sb.startBlock(''); + this._constructMap(sb, prop); - _.forEach(prop.entries, (entry) => { - sb.append(`${entry.value}`); - }); + sb.endBlock(''); - sb.endBlock(''); - sb.endBlock(''); + break; + case 'java.util.Properties': + sb.startBlock(``); + sb.startBlock(''); - break; - case 'BEAN': - sb.startBlock(``); + _.forEach(prop.entries, (entry) => { + sb.append(`${entry.value}`); + }); - this.appendBean(sb, prop.value); + sb.endBlock(''); + sb.endBlock(''); - sb.endBlock(''); + break; + case 'BEAN': + sb.startBlock(``); - break; - default: - sb.append(``); - } + this.appendBean(sb, prop.value); - this._emptyLineIfNeeded(sb, bean.properties, idx); - }); + sb.endBlock(''); - return sb; - } + break; + default: + sb.append(``); + } - /** - * Build final XML. - * - * @param {Bean} cfg Ignite configuration. - * @param {Boolean} clientNearCaches - * @returns {StringBuilder} - */ - static igniteConfiguration(cfg, clientNearCaches) { - const sb = new StringBuilder(); - - // 0. Add header. - sb.append(''); - sb.emptyLine(); + this._emptyLineIfNeeded(sb, bean.properties, idx); + }); + + return sb; + } + + /** + * Build final XML. + * + * @param {Bean} cfg Ignite configuration. + * @param {Boolean} clientNearCaches + * @returns {StringBuilder} + */ + static igniteConfiguration(cfg, clientNearCaches) { + const sb = new StringBuilder(); + + // 0. Add header. + sb.append(''); + sb.emptyLine(); + + this.mainComment(sb); + sb.emptyLine(); + + // 1. Start beans section. + sb.startBlock([ + '']); + + // 2. Add external property file + if (this.hasProperties(cfg)) { + this.commentBlock(sb, 'Load external properties file.'); + + sb.startBlock(''); + sb.append(''); + sb.endBlock(''); - this.mainComment(sb); sb.emptyLine(); + } - // 1. Start beans section. - sb.startBlock([ - '']); + // 3. Add data sources. + const dataSources = this.collectDataSources(cfg); - // 2. Add external property file - if (this.hasProperties(cfg)) { - this.commentBlock(sb, 'Load external properties file.'); + if (dataSources.length) { + this.commentBlock(sb, 'Data source beans will be initialized from external properties file.'); - sb.startBlock(''); - sb.append(''); - sb.endBlock(''); + _.forEach(dataSources, (ds) => { + this.appendBean(sb, ds, true); sb.emptyLine(); - } - - // 3. Add data sources. - const dataSources = this.collectDataSources(cfg); - - if (dataSources.length) { - this.commentBlock(sb, 'Data source beans will be initialized from external properties file.'); - - _.forEach(dataSources, (ds) => { - this.appendBean(sb, ds, true); - - sb.emptyLine(); - }); - } + }); + } - _.forEach(clientNearCaches, (cache) => { - this.commentBlock(sb, 'Configuration of near cache for cache "' + cache.name + '"'); + _.forEach(clientNearCaches, (cache) => { + this.commentBlock(sb, `Configuration of near cache for cache "${cache.name}"`); - this.appendBean(sb, generator.cacheNearClient(cache), true); + this.appendBean(sb, this.generator.cacheNearClient(cache), true); - sb.emptyLine(); - }); + sb.emptyLine(); + }); - // 3. Add main content. - this.appendBean(sb, cfg); + // 3. Add main content. + this.appendBean(sb, cfg); - // 4. Close beans section. - sb.endBlock(''); + // 4. Close beans section. + sb.endBlock(''); - return sb; - } + return sb; + } - static cluster(cluster, client) { - const cfg = generator.igniteConfiguration(cluster, client); + static cluster(cluster, client) { + const cfg = this.generator.igniteConfiguration(cluster, client); - const clientNearCaches = client ? _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')) : []; + const clientNearCaches = client ? _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')) : []; - return this.igniteConfiguration(cfg, clientNearCaches); - } - }; -}]; + return this.igniteConfiguration(cfg, clientNearCaches); + } +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.platform.service.js similarity index 88% rename from modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js rename to modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.platform.service.js index f06e11b5f57be..eeac3a03f7901 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.platform.service.js @@ -49,12 +49,8 @@ const DFLT_CACHE = { } }; -export default function() { - this.append = (dflts) => { - _.merge(DFLT_CACHE, dflts); - }; - - this.$get = ['igniteCacheDefaults', (cacheDefaults) => { - return _.merge({}, cacheDefaults, DFLT_CACHE); - }]; +export default class IgniteCachePlatformDefaults { + constructor() { + Object.assign(this, DFLT_CACHE); + } } diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.service.js similarity index 95% rename from modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js rename to modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.service.js index f50e493268412..14b315f4fdde1 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.service.js @@ -15,8 +15,6 @@ * limitations under the License. */ -import _ from 'lodash'; - const DFLT_CACHE = { cacheMode: { clsName: 'org.apache.ignite.cache.CacheMode' @@ -126,12 +124,8 @@ const DFLT_CACHE = { } }; -export default function() { - this.append = (dflts) => { - _.merge(DFLT_CACHE, dflts); - }; - - this.$get = [() => { - return DFLT_CACHE; - }]; +export default class IgniteCacheDefaults { + constructor() { + Object.assign(this, DFLT_CACHE); + } } diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.platform.service.js similarity index 84% rename from modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js rename to modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.platform.service.js index 582426ec8e7b4..b7019511355e9 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.platform.service.js @@ -15,8 +15,6 @@ * limitations under the License. */ -import _ from 'lodash'; - const enumValueMapper = (val) => _.capitalize(val); const DFLT_CLUSTER = { @@ -38,12 +36,8 @@ const DFLT_CLUSTER = { } }; -export default function() { - this.append = (dflts) => { - _.merge(DFLT_CLUSTER, dflts); - }; - - this.$get = ['igniteClusterDefaults', (clusterDefaults) => { - return _.merge({}, clusterDefaults, DFLT_CLUSTER); - }]; +export default class IgniteClusterPlatformDefaults { + constructor() { + Object.assign(this, DFLT_CLUSTER); + } } diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js similarity index 98% rename from modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js rename to modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js index 726581de810ea..6333ef96afcd1 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js @@ -282,12 +282,8 @@ const DFLT_CLUSTER = { } }; -export default function() { - this.append = (dflts) => { - _.merge(DFLT_CLUSTER, dflts); - }; - - this.$get = [() => { - return DFLT_CLUSTER; - }]; +export default class IgniteClusterDefaults { + constructor() { + Object.assign(this, DFLT_CLUSTER); + } } diff --git a/modules/web-console/frontend/app/modules/configuration/EventGroups.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Event-groups.service.js similarity index 78% rename from modules/web-console/frontend/app/modules/configuration/EventGroups.provider.js rename to modules/web-console/frontend/app/modules/configuration/generator/defaults/Event-groups.service.js index 61f31885a9691..315da1f41fb1b 100644 --- a/modules/web-console/frontend/app/modules/configuration/EventGroups.provider.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Event-groups.service.js @@ -15,16 +15,13 @@ * limitations under the License. */ -// Events groups. -import GROUPS from 'app/data/event-types.json'; - -export default ['igniteEventGroups', function() { - const groups = GROUPS; +import _ from 'lodash'; - this.push = (data) => groups.push(data); - - this.$get = [() => { - return groups; - }]; -}]; +// Events groups. +import EVENT_GROUPS from 'app/data/event-groups.json'; +export default class IgniteEventGroups { + constructor() { + return _.clone(EVENT_GROUPS); + } +} diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/IGFS.service.js similarity index 92% rename from modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js rename to modules/web-console/frontend/app/modules/configuration/generator/defaults/IGFS.service.js index c556336a3953b..985a56e10761f 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/IGFS.service.js @@ -57,12 +57,8 @@ const DFLT_IGFS = { } }; -export default function() { - this.append = (dflts) => { - _.merge(DFLT_IGFS, dflts); - }; - - this.$get = [() => { - return DFLT_IGFS; - }]; +export default class IgniteIGFSDefaults { + constructor() { + Object.assign(this, DFLT_IGFS); + } } diff --git a/modules/web-console/frontend/app/modules/configuration/generator/generator-common.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-common.js deleted file mode 100644 index d502c8a5c4605..0000000000000 --- a/modules/web-console/frontend/app/modules/configuration/generator/generator-common.js +++ /dev/null @@ -1,625 +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. - */ - -// Entry point for common functions for code generation. -const $generatorCommon = {}; - -// Add leading zero. -$generatorCommon.addLeadingZero = function(numberStr, minSize) { - if (typeof (numberStr) !== 'string') - numberStr = String(numberStr); - - while (numberStr.length < minSize) - numberStr = '0' + numberStr; - - return numberStr; -}; - -// Format date to string. -$generatorCommon.formatDate = function(date) { - const dd = $generatorCommon.addLeadingZero(date.getDate(), 2); - const mm = $generatorCommon.addLeadingZero(date.getMonth() + 1, 2); - - const yyyy = date.getFullYear(); - - return mm + '/' + dd + '/' + yyyy + ' ' + $generatorCommon.addLeadingZero(date.getHours(), 2) + ':' + $generatorCommon.addLeadingZero(date.getMinutes(), 2); -}; - -/** - * Generate title comment for XML, Java, ... files. - * - * @param sbj {string} What is generated. - * @returns {string} Text to add as title comment in generated java class. - */ -$generatorCommon.mainComment = function mainComment(sbj) { - return 'This ' + sbj + ' was generated by Ignite Web Console (' + $generatorCommon.formatDate(new Date()) + ')'; -}; - -// Create result holder with service functions and properties for XML and java code generation. -$generatorCommon.builder = function(deep) { - if (_.isNil($generatorCommon.JavaTypes)) - $generatorCommon.JavaTypes = angular.element(document.getElementById('app')).injector().get('JavaTypes'); - - const res = []; - - res.deep = deep || 0; - res.needEmptyLine = false; - res.lineStart = true; - res.datasources = []; - res.imports = {}; - res.staticImports = {}; - res.vars = {}; - - res.safeDeep = 0; - res.safeNeedEmptyLine = false; - res.safeImports = {}; - res.safeDatasources = []; - res.safePoint = -1; - - res.mergeProps = function(fromRes) { - if ($generatorCommon.isDefinedAndNotEmpty(fromRes)) { - res.datasources = fromRes.datasources; - - angular.extend(res.imports, fromRes.imports); - angular.extend(res.staticImports, fromRes.staticImports); - angular.extend(res.vars, fromRes.vars); - } - }; - - res.mergeLines = function(fromRes) { - if ($generatorCommon.isDefinedAndNotEmpty(fromRes)) { - if (res.needEmptyLine) - res.push(''); - - _.forEach(fromRes, function(line) { - res.append(line); - }); - } - }; - - res.startSafeBlock = function() { - res.safeDeep = this.deep; - this.safeNeedEmptyLine = this.needEmptyLine; - this.safeImports = _.cloneDeep(this.imports); - this.safeStaticImports = _.cloneDeep(this.staticImports); - this.safeDatasources = this.datasources.slice(); - this.safePoint = this.length; - }; - - res.rollbackSafeBlock = function() { - if (this.safePoint >= 0) { - this.splice(this.safePoint, this.length - this.safePoint); - - this.deep = res.safeDeep; - this.needEmptyLine = this.safeNeedEmptyLine; - this.datasources = this.safeDatasources; - this.imports = this.safeImports; - this.staticImports = this.safeStaticImports; - this.safePoint = -1; - } - }; - - res.asString = function() { - return this.join('\n'); - }; - - res.append = function(s) { - this.push((this.lineStart ? _.repeat(' ', this.deep) : '') + s); - - return this; - }; - - res.line = function(s) { - if (s) { - if (res.needEmptyLine) - res.push(''); - - res.append(s); - } - - res.needEmptyLine = false; - - res.lineStart = true; - - return res; - }; - - res.startBlock = function(s) { - if (s) { - if (this.needEmptyLine) - this.push(''); - - this.append(s); - } - - this.needEmptyLine = false; - - this.lineStart = true; - - this.deep++; - - return this; - }; - - res.endBlock = function(s) { - this.deep--; - - if (s) - this.append(s); - - this.lineStart = true; - - return this; - }; - - res.softEmptyLine = function() { - this.needEmptyLine = this.length > 0; - }; - - res.emptyLineIfNeeded = function() { - if (this.needEmptyLine) { - this.push(''); - this.lineStart = true; - - this.needEmptyLine = false; - } - }; - - /** - * Add class to imports. - * - * @param clsName Full class name. - * @returns {String} Short class name or full class name in case of names conflict. - */ - res.importClass = function(clsName) { - if ($generatorCommon.JavaTypes.isJavaPrimitive(clsName)) - return clsName; - - const fullClassName = $generatorCommon.JavaTypes.fullClassName(clsName); - - const dotIdx = fullClassName.lastIndexOf('.'); - - const shortName = dotIdx > 0 ? fullClassName.substr(dotIdx + 1) : fullClassName; - - if (this.imports[shortName]) { - if (this.imports[shortName] !== fullClassName) - return fullClassName; // Short class names conflict. Return full name. - } - else - this.imports[shortName] = fullClassName; - - return shortName; - }; - - /** - * Add class to imports. - * - * @param member Static member. - * @returns {String} Short class name or full class name in case of names conflict. - */ - res.importStatic = function(member) { - const dotIdx = member.lastIndexOf('.'); - - const shortName = dotIdx > 0 ? member.substr(dotIdx + 1) : member; - - if (this.staticImports[shortName]) { - if (this.staticImports[shortName] !== member) - return member; // Short class names conflict. Return full name. - } - else - this.staticImports[shortName] = member; - - return shortName; - }; - - /** - * @returns String with "java imports" section. - */ - res.generateImports = function() { - const genImports = []; - - for (const clsName in this.imports) { - if (this.imports.hasOwnProperty(clsName) && this.imports[clsName].lastIndexOf('java.lang.', 0) !== 0) - genImports.push('import ' + this.imports[clsName] + ';'); - } - - genImports.sort(); - - return genImports.join('\n'); - }; - - /** - * @returns String with "java imports" section. - */ - res.generateStaticImports = function() { - const statImports = []; - - for (const clsName in this.staticImports) { - if (this.staticImports.hasOwnProperty(clsName) && this.staticImports[clsName].lastIndexOf('java.lang.', 0) !== 0) - statImports.push('import static ' + this.staticImports[clsName] + ';'); - } - - statImports.sort(); - - return statImports.join('\n'); - }; - - return res; -}; - -// Eviction policies code generation descriptors. -$generatorCommon.EVICTION_POLICIES = { - LRU: { - className: 'org.apache.ignite.cache.eviction.lru.LruEvictionPolicy', - fields: {batchSize: {dflt: 1}, maxMemorySize: null, maxSize: {dflt: 100000}} - }, - FIFO: { - className: 'org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy', - fields: {batchSize: {dflt: 1}, maxMemorySize: null, maxSize: {dflt: 100000}} - }, - SORTED: { - className: 'org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy', - fields: {batchSize: {dflt: 1}, maxMemorySize: null, maxSize: {dflt: 100000}} - } -}; - -// Marshaller code generation descriptors. -$generatorCommon.MARSHALLERS = { - OptimizedMarshaller: { - className: 'org.apache.ignite.marshaller.optimized.OptimizedMarshaller', - fields: {poolSize: null, requireSerializable: null } - }, - JdkMarshaller: { - className: 'org.apache.ignite.marshaller.jdk.JdkMarshaller', - fields: {} - } -}; - -// Pairs of supported databases and their JDBC dialects. -$generatorCommon.JDBC_DIALECTS = { - Generic: 'org.apache.ignite.cache.store.jdbc.dialect.BasicJdbcDialect', - Oracle: 'org.apache.ignite.cache.store.jdbc.dialect.OracleDialect', - DB2: 'org.apache.ignite.cache.store.jdbc.dialect.DB2Dialect', - SQLServer: 'org.apache.ignite.cache.store.jdbc.dialect.SQLServerDialect', - MySQL: 'org.apache.ignite.cache.store.jdbc.dialect.MySQLDialect', - PostgreSQL: 'org.apache.ignite.cache.store.jdbc.dialect.BasicJdbcDialect', - H2: 'org.apache.ignite.cache.store.jdbc.dialect.H2Dialect' -}; - -// Return JDBC dialect full class name for specified database. -$generatorCommon.jdbcDialectClassName = function(db) { - const dialectClsName = $generatorCommon.JDBC_DIALECTS[db]; - - return dialectClsName ? dialectClsName : 'Unknown database: ' + db; -}; - -// Generate default data cache for specified igfs instance. -$generatorCommon.igfsDataCache = function(igfs) { - return { - name: igfs.name + '-data', - cacheMode: 'PARTITIONED', - atomicityMode: 'TRANSACTIONAL', - writeSynchronizationMode: 'FULL_SYNC', - backups: 0, - igfsAffinnityGroupSize: igfs.affinnityGroupSize || 512 - }; -}; - -// Generate default meta cache for specified igfs instance. -$generatorCommon.igfsMetaCache = function(igfs) { - return { - name: igfs.name + '-meta', - cacheMode: 'REPLICATED', - atomicityMode: 'TRANSACTIONAL', - writeSynchronizationMode: 'FULL_SYNC' - }; -}; - -// Pairs of supported databases and their data sources. -$generatorCommon.DATA_SOURCES = { - Generic: 'com.mchange.v2.c3p0.ComboPooledDataSource', - Oracle: 'oracle.jdbc.pool.OracleDataSource', - DB2: 'com.ibm.db2.jcc.DB2DataSource', - SQLServer: 'com.microsoft.sqlserver.jdbc.SQLServerDataSource', - MySQL: 'com.mysql.jdbc.jdbc2.optional.MysqlDataSource', - PostgreSQL: 'org.postgresql.ds.PGPoolingDataSource', - H2: 'org.h2.jdbcx.JdbcDataSource' -}; - -// Return data source full class name for specified database. -$generatorCommon.dataSourceClassName = function(db) { - const dsClsName = $generatorCommon.DATA_SOURCES[db]; - - return dsClsName ? dsClsName : 'Unknown database: ' + db; -}; - -// Store factories code generation descriptors. -$generatorCommon.STORE_FACTORIES = { - CacheJdbcPojoStoreFactory: { - className: 'org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory', - suffix: 'JdbcPojo', - fields: { - configuration: {type: 'bean'} - } - }, - CacheJdbcBlobStoreFactory: { - className: 'org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory', - suffix: 'JdbcBlob', - fields: { - initSchema: null, - createTableQuery: null, - loadQuery: null, - insertQuery: null, - updateQuery: null, - deleteQuery: null - } - }, - CacheHibernateBlobStoreFactory: { - className: 'org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory', - suffix: 'Hibernate', - fields: {hibernateProperties: {type: 'propertiesAsList', propVarName: 'props'}} - } -}; - -// Swap space SPI code generation descriptor. -$generatorCommon.SWAP_SPACE_SPI = { - className: 'org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi', - fields: { - baseDirectory: {type: 'path'}, - readStripesNumber: null, - maximumSparsity: {type: 'float'}, - maxWriteQueueSize: null, - writeBufferSize: null - } -}; - -// Transaction configuration code generation descriptor. -$generatorCommon.TRANSACTION_CONFIGURATION = { - className: 'org.apache.ignite.configuration.TransactionConfiguration', - fields: { - defaultTxConcurrency: {type: 'enum', enumClass: 'org.apache.ignite.transactions.TransactionConcurrency', dflt: 'PESSIMISTIC'}, - defaultTxIsolation: {type: 'enum', enumClass: 'org.apache.ignite.transactions.TransactionIsolation', dflt: 'REPEATABLE_READ'}, - defaultTxTimeout: {dflt: 0}, - pessimisticTxLogLinger: {dflt: 10000}, - pessimisticTxLogSize: null, - txSerializableEnabled: null, - txManagerFactory: {type: 'bean'} - } -}; - -// SSL configuration code generation descriptor. -$generatorCommon.SSL_CONFIGURATION_TRUST_FILE_FACTORY = { - className: 'org.apache.ignite.ssl.SslContextFactory', - fields: { - keyAlgorithm: null, - keyStoreFilePath: {type: 'path'}, - keyStorePassword: {type: 'raw'}, - keyStoreType: null, - protocol: null, - trustStoreFilePath: {type: 'path'}, - trustStorePassword: {type: 'raw'}, - trustStoreType: null - } -}; - -// SSL configuration code generation descriptor. -$generatorCommon.SSL_CONFIGURATION_TRUST_MANAGER_FACTORY = { - className: 'org.apache.ignite.ssl.SslContextFactory', - fields: { - keyAlgorithm: null, - keyStoreFilePath: {type: 'path'}, - keyStorePassword: {type: 'raw'}, - keyStoreType: null, - protocol: null, - trustManagers: {type: 'array'} - } -}; - -// Communication configuration code generation descriptor. -$generatorCommon.CONNECTOR_CONFIGURATION = { - className: 'org.apache.ignite.configuration.ConnectorConfiguration', - fields: { - jettyPath: null, - host: null, - port: {dflt: 11211}, - portRange: {dflt: 100}, - idleTimeout: {dflt: 7000}, - idleQueryCursorTimeout: {dflt: 600000}, - idleQueryCursorCheckFrequency: {dflt: 60000}, - receiveBufferSize: {dflt: 32768}, - sendBufferSize: {dflt: 32768}, - sendQueueLimit: {dflt: 0}, - directBuffer: {dflt: false}, - noDelay: {dflt: true}, - selectorCount: null, - threadPoolSize: null, - messageInterceptor: {type: 'bean'}, - secretKey: null, - sslEnabled: {dflt: false} - } -}; - -// Communication configuration code generation descriptor. -$generatorCommon.COMMUNICATION_CONFIGURATION = { - className: 'org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi', - fields: { - listener: {type: 'bean'}, - localAddress: null, - localPort: {dflt: 47100}, - localPortRange: {dflt: 100}, - sharedMemoryPort: {dflt: 48100}, - directBuffer: {dflt: false}, - directSendBuffer: {dflt: false}, - idleConnectionTimeout: {dflt: 30000}, - connectTimeout: {dflt: 5000}, - maxConnectTimeout: {dflt: 600000}, - reconnectCount: {dflt: 10}, - socketSendBuffer: {dflt: 32768}, - socketReceiveBuffer: {dflt: 32768}, - messageQueueLimit: {dflt: 1024}, - slowClientQueueLimit: null, - tcpNoDelay: {dflt: true}, - ackSendThreshold: {dflt: 16}, - unacknowledgedMessagesBufferSize: {dflt: 0}, - socketWriteTimeout: {dflt: 2000}, - selectorsCount: null, - addressResolver: {type: 'bean'} - } -}; - -// Communication configuration code generation descriptor. -$generatorCommon.IGFS_IPC_CONFIGURATION = { - className: 'org.apache.ignite.igfs.IgfsIpcEndpointConfiguration', - fields: { - type: {type: 'enum', enumClass: 'org.apache.ignite.igfs.IgfsIpcEndpointType'}, - host: {dflt: '127.0.0.1'}, - port: {dflt: 10500}, - memorySize: {dflt: 262144}, - tokenDirectoryPath: {dflt: 'ipc/shmem'}, - threadCount: null - } -}; - -$generatorCommon.ODBC_CONFIGURATION = { - className: 'org.apache.ignite.configuration.OdbcConfiguration', - fields: { - endpointAddress: {dflt: '0.0.0.0:10800..10810'}, - maxOpenCursors: {dflt: 128} - } -}; - -// Check that cache has datasource. -$generatorCommon.cacheHasDatasource = function(cache) { - if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) { - const storeFactory = cache.cacheStoreFactory[cache.cacheStoreFactory.kind]; - - return !!(storeFactory && (storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : false) : storeFactory.dialect)); // eslint-disable-line no-nested-ternary - } - - return false; -}; - -$generatorCommon.secretPropertiesNeeded = function(cluster) { - return !_.isNil(_.find(cluster.caches, $generatorCommon.cacheHasDatasource)) || cluster.sslEnabled; -}; - -// Check that binary is configured. -$generatorCommon.binaryIsDefined = function(binary) { - return binary && ($generatorCommon.isDefinedAndNotEmpty(binary.idMapper) || $generatorCommon.isDefinedAndNotEmpty(binary.nameMapper) || - $generatorCommon.isDefinedAndNotEmpty(binary.serializer) || $generatorCommon.isDefinedAndNotEmpty(binary.typeConfigurations) || - (!_.isNil(binary.compactFooter) && !binary.compactFooter)); -}; - -// Extract domain model metadata location. -$generatorCommon.domainQueryMetadata = function(domain) { - return domain.queryMetadata ? domain.queryMetadata : 'Configuration'; -}; - -/** - * @param {Object} obj Object to check. - * @param {Array} props Array of properties names. - * @returns {boolean} 'true' if - */ -$generatorCommon.hasAtLeastOneProperty = function(obj, props) { - return obj && props && _.findIndex(props, (prop) => !_.isNil(obj[prop])) >= 0; -}; - -/** - * Convert some name to valid java name. - * - * @param prefix To append to java name. - * @param name to convert. - * @returns {string} Valid java name. - */ -$generatorCommon.toJavaName = function(prefix, name) { - const javaName = name ? name.replace(/[^A-Za-z_0-9]+/g, '_') : 'dflt'; - - return prefix + javaName.charAt(0).toLocaleUpperCase() + javaName.slice(1); -}; - -/** - * @param v Value to check. - * @returns {boolean} 'true' if value defined and not empty string. - */ -$generatorCommon.isDefinedAndNotEmpty = function(v) { - let defined = !_.isNil(v); - - if (defined && (_.isString(v) || _.isArray(v))) - defined = v.length > 0; - - return defined; -}; - -/** - * @param {Object} obj Object to check. - * @param {Array} props Properties names. - * @returns {boolean} 'true' if object contains at least one from specified properties. - */ -$generatorCommon.hasProperty = function(obj, props) { - for (const propName in props) { - if (props.hasOwnProperty(propName)) { - if (obj[propName]) - return true; - } - } - - return false; -}; - -/** - * Get class for selected implementation of Failover SPI. - * - * @param spi Failover SPI configuration. - * @returns {*} Class for selected implementation of Failover SPI. - */ -$generatorCommon.failoverSpiClass = function(spi) { - switch (spi.kind) { - case 'JobStealing': return 'org.apache.ignite.spi.failover.jobstealing.JobStealingFailoverSpi'; - case 'Never': return 'org.apache.ignite.spi.failover.never.NeverFailoverSpi'; - case 'Always': return 'org.apache.ignite.spi.failover.always.AlwaysFailoverSpi'; - case 'Custom': return _.get(spi, 'Custom.class'); - default: return 'Unknown'; - } -}; - -$generatorCommon.loggerConfigured = function(logger) { - if (logger && logger.kind) { - const log = logger[logger.kind]; - - switch (logger.kind) { - case 'Log4j2': return log && $generatorCommon.isDefinedAndNotEmpty(log.path); - - case 'Log4j': - if (!log || !log.mode) - return false; - - if (log.mode === 'Path') - return $generatorCommon.isDefinedAndNotEmpty(log.path); - - return true; - - case 'Custom': return log && $generatorCommon.isDefinedAndNotEmpty(log.class); - - default: - return true; - } - } - - return false; -}; - -export default $generatorCommon; diff --git a/modules/web-console/frontend/app/modules/configuration/generator/generator-java.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-java.js deleted file mode 100644 index 296b942c42ec0..0000000000000 --- a/modules/web-console/frontend/app/modules/configuration/generator/generator-java.js +++ /dev/null @@ -1,3617 +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. - */ - -// Java generation entry point. -const $generatorJava = {}; - -/** - * Translate some value to valid java code. - * - * @param val Value to convert. - * @param type Value type. - * @returns {*} String with value that will be valid for java. - */ -$generatorJava.toJavaCode = function(val, type) { - if (val === null) - return 'null'; - - if (type === 'raw') - return val; - - if (type === 'class') - return val + '.class'; - - if (type === 'float') - return val + 'f'; - - if (type === 'path') - return '"' + val.replace(/\\/g, '\\\\') + '"'; - - if (type) - return type + '.' + val; - - if (typeof (val) === 'string') - return '"' + val.replace('"', '\\"') + '"'; - - if (typeof (val) === 'number' || typeof (val) === 'boolean') - return String(val); - - return 'Unknown type: ' + typeof (val) + ' (' + val + ')'; -}; - -/** - * @param propName Property name. - * @param setterName Optional concrete setter name. - * @returns Property setter with name by java conventions. - */ -$generatorJava.setterName = function(propName, setterName) { - return setterName ? setterName : $generatorCommon.toJavaName('set', propName); -}; - -// Add constructor argument -$generatorJava.constructorArg = function(obj, propName, dflt, notFirst, opt) { - const v = (obj ? obj[propName] : null) || dflt; - - if ($generatorCommon.isDefinedAndNotEmpty(v)) - return (notFirst ? ', ' : '') + $generatorJava.toJavaCode(v); - else if (!opt) - return notFirst ? ', null' : 'null'; - - return ''; -}; - -/** - * Add variable declaration. - * - * @param res Resulting output with generated code. - * @param varName Variable name. - * @param varFullType Variable full class name to be added to imports. - * @param varFullActualType Variable actual full class name to be added to imports. - * @param varFullGenericType1 Optional full class name of first generic. - * @param varFullGenericType2 Optional full class name of second generic. - * @param subClass If 'true' then variable will be declared as anonymous subclass. - */ -$generatorJava.declareVariable = function(res, varName, varFullType, varFullActualType, varFullGenericType1, varFullGenericType2, subClass) { - res.emptyLineIfNeeded(); - - const varType = res.importClass(varFullType); - - const varNew = !res.vars[varName]; - - if (varNew) - res.vars[varName] = true; - - if (varFullActualType && varFullGenericType1) { - const varActualType = res.importClass(varFullActualType); - const varGenericType1 = res.importClass(varFullGenericType1); - let varGenericType2 = null; - - if (varFullGenericType2) - varGenericType2 = res.importClass(varFullGenericType2); - - res.line((varNew ? (varType + '<' + varGenericType1 + (varGenericType2 ? ', ' + varGenericType2 : '') + '> ') : '') + - varName + ' = new ' + varActualType + '<>();'); - } - else - res.line((varNew ? (varType + ' ') : '') + varName + ' = new ' + varType + '()' + (subClass ? ' {' : ';')); - - if (!subClass) - res.needEmptyLine = true; - - return varName; -}; - -/** - * Add local variable declaration. - * - * @param res Resulting output with generated code. - * @param varName Variable name. - * @param varFullType Variable full class name to be added to imports. - */ -$generatorJava.declareVariableLocal = function(res, varName, varFullType) { - const varType = res.importClass(varFullType); - - res.line(varType + ' ' + varName + ' = new ' + varType + '();'); - - res.needEmptyLine = true; -}; - -/** - * Add custom variable declaration. - * - * @param res Resulting output with generated code. - * @param varName Variable name. - * @param varFullType Variable full class name to be added to imports. - * @param varExpr Custom variable creation expression. - * @param modifier Additional variable modifier. - */ -$generatorJava.declareVariableCustom = function(res, varName, varFullType, varExpr, modifier) { - const varType = res.importClass(varFullType); - - const varNew = !res.vars[varName]; - - if (varNew) - res.vars[varName] = true; - - res.line((varNew ? ((modifier ? modifier + ' ' : '') + varType + ' ') : '') + varName + ' = ' + varExpr + ';'); - - res.needEmptyLine = true; -}; - -/** - * Add array variable declaration. - * - * @param res Resulting output with generated code. - * @param varName Variable name. - * @param varFullType Variable full class name to be added to imports. - * @param length Array length. - */ -$generatorJava.declareVariableArray = function(res, varName, varFullType, length) { - const varType = res.importClass(varFullType); - - const varNew = !res.vars[varName]; - - if (varNew) - res.vars[varName] = true; - - res.line((varNew ? (varType + '[] ') : '') + varName + ' = new ' + varType + '[' + length + '];'); - - res.needEmptyLine = true; -}; - -/** - * Clear list of declared variables. - * - * @param res - */ -$generatorJava.resetVariables = function(res) { - res.vars = {}; -}; - -/** - * Add property via setter / property name. - * - * @param res Resulting output with generated code. - * @param varName Variable name. - * @param obj Source object with data. - * @param propName Property name to take from source object. - * @param dataType Optional info about property data type. - * @param setterName Optional special setter name. - * @param dflt Optional default value. - */ -$generatorJava.property = function(res, varName, obj, propName, dataType, setterName, dflt) { - if (!_.isNil(obj)) { - const val = obj[propName]; - - if ($generatorCommon.isDefinedAndNotEmpty(val)) { - const missDflt = _.isNil(dflt); - - // Add to result if no default provided or value not equals to default. - if (missDflt || (!missDflt && val !== dflt)) { - res.line(varName + '.' + $generatorJava.setterName(propName, setterName) + - '(' + $generatorJava.toJavaCode(val, dataType) + ');'); - - return true; - } - } - } - - return false; -}; - -/** - * Add enum property via setter / property name. - * - * @param res Resulting output with generated code. - * @param varName Variable name. - * @param obj Source object with data. - * @param propName Property name to take from source object. - * @param dataType Name of enum class - * @param setterName Optional special setter name. - * @param dflt Optional default value. - */ -$generatorJava.enumProperty = function(res, varName, obj, propName, dataType, setterName, dflt) { - const val = obj[propName]; - - if ($generatorCommon.isDefinedAndNotEmpty(val)) { - const missDflt = _.isNil(dflt); - - // Add to result if no default provided or value not equals to default. - if (missDflt || (!missDflt && val !== dflt)) { - res.line(varName + '.' + $generatorJava.setterName(propName, setterName) + - '(' + $generatorJava.toJavaCode(val, dataType ? res.importClass(dataType) : null) + ');'); - - return true; - } - } - - return false; -}; - -// Add property for class name. -$generatorJava.classNameProperty = function(res, varName, obj, propName) { - const val = obj[propName]; - - if (!_.isNil(val)) { - res.line(varName + '.' + $generatorJava.setterName(propName) + - '("' + $generatorCommon.JavaTypes.fullClassName(val) + '");'); - } -}; - -/** - * Add list property. - * - * @param res Resulting output with generated code. - * @param varName Variable name. - * @param obj Source object with data. - * @param propName Property name to take from source object. - * @param dataType Optional data type. - * @param setterName Optional setter name. - */ -$generatorJava.listProperty = function(res, varName, obj, propName, dataType, setterName) { - const val = obj[propName]; - - if (val && val.length > 0) { - res.emptyLineIfNeeded(); - - res.importClass('java.util.Arrays'); - - $generatorJava.fxVarArgs(res, varName + '.' + $generatorJava.setterName(propName, setterName), false, - _.map(val, (v) => $generatorJava.toJavaCode(v, dataType)), '(Arrays.asList(', '))'); - - res.needEmptyLine = true; - } -}; - -/** - * Add function with varargs arguments. - * - * @param res Resulting output with generated code. - * @param fx Function name. - * @param quote Whether to quote arguments. - * @param args Array with arguments. - * @param startBlock Optional start block string. - * @param endBlock Optional end block string. - * @param startQuote Start quote string. - * @param endQuote End quote string. - */ -$generatorJava.fxVarArgs = function(res, fx, quote, args, startBlock = '(', endBlock = ')', startQuote = '"', endQuote = '"') { - const quoteArg = (arg) => quote ? startQuote + arg + endQuote : arg; - - if (args.length === 1) - res.append(fx + startBlock + quoteArg(args[0]) + endBlock + ';'); - else { - res.startBlock(fx + startBlock); - - const len = args.length - 1; - - _.forEach(args, (arg, ix) => res.line(quoteArg(arg) + (ix < len ? ', ' : ''))); - - res.endBlock(endBlock + ';'); - } -}; - -/** - * Add array property. - * - * @param res Resulting output with generated code. - * @param varName Variable name. - * @param obj Source object with data. - * @param propName Property name to take from source object. - * @param setterName Optional setter name. - */ -$generatorJava.arrayProperty = function(res, varName, obj, propName, setterName) { - const val = obj[propName]; - - if (val && val.length > 0) { - res.emptyLineIfNeeded(); - - $generatorJava.fxVarArgs(res, varName + '.' + $generatorJava.setterName(propName, setterName), false, - _.map(val, (v) => 'new ' + res.importClass(v) + '()'), '({ ', ' });'); - - res.needEmptyLine = true; - } -}; - -/** - * Add multi-param property (setter with several arguments). - * - * @param res Resulting output with generated code. - * @param varName Variable name. - * @param obj Source object with data. - * @param propName Property name to take from source object. - * @param dataType Optional data type. - * @param setterName Optional setter name. - */ -$generatorJava.multiparamProperty = function(res, varName, obj, propName, dataType, setterName) { - const val = obj[propName]; - - if (val && val.length > 0) { - $generatorJava.fxVarArgs(res, varName + '.' + $generatorJava.setterName(propName, setterName), false, - _.map(val, (v) => $generatorJava.toJavaCode(dataType === 'class' ? res.importClass(v) : v, dataType))); - } -}; - -/** - * Add complex bean. - * - * @param res Resulting output with generated code. - * @param varName Variable name. - * @param bean - * @param beanPropName Bean property name. - * @param beanVarName - * @param beanClass Bean class. - * @param props - * @param createBeanAlthoughNoProps If 'true' then create empty bean. - */ -$generatorJava.beanProperty = function(res, varName, bean, beanPropName, beanVarName, beanClass, props, createBeanAlthoughNoProps) { - if (bean && $generatorCommon.hasProperty(bean, props)) { - res.emptyLineIfNeeded(); - - $generatorJava.declareVariable(res, beanVarName, beanClass); - - _.forIn(props, function(descr, propName) { - if (props.hasOwnProperty(propName)) { - if (descr) { - switch (descr.type) { - case 'list': - $generatorJava.listProperty(res, beanVarName, bean, propName, descr.elementsType, descr.setterName); - break; - - case 'array': - $generatorJava.arrayProperty(res, beanVarName, bean, propName, descr.setterName); - break; - - case 'enum': - $generatorJava.enumProperty(res, beanVarName, bean, propName, descr.enumClass, descr.setterName, descr.dflt); - break; - - case 'float': - $generatorJava.property(res, beanVarName, bean, propName, 'float', descr.setterName); - break; - - case 'path': - $generatorJava.property(res, beanVarName, bean, propName, 'path', descr.setterName); - break; - - case 'raw': - $generatorJava.property(res, beanVarName, bean, propName, 'raw', descr.setterName); - break; - - case 'propertiesAsList': - const val = bean[propName]; - - if (val && val.length > 0) { - $generatorJava.declareVariable(res, descr.propVarName, 'java.util.Properties'); - - _.forEach(val, function(nameAndValue) { - const eqIndex = nameAndValue.indexOf('='); - - if (eqIndex >= 0) { - res.line(descr.propVarName + '.setProperty(' + - '"' + nameAndValue.substring(0, eqIndex) + '", ' + - '"' + nameAndValue.substr(eqIndex + 1) + '");'); - } - }); - - res.needEmptyLine = true; - - res.line(beanVarName + '.' + $generatorJava.setterName(propName) + '(' + descr.propVarName + ');'); - } - break; - - case 'bean': - if ($generatorCommon.isDefinedAndNotEmpty(bean[propName])) - res.line(beanVarName + '.' + $generatorJava.setterName(propName) + '(new ' + res.importClass(bean[propName]) + '());'); - - break; - - default: - $generatorJava.property(res, beanVarName, bean, propName, null, descr.setterName, descr.dflt); - } - } - else - $generatorJava.property(res, beanVarName, bean, propName); - } - }); - - res.needEmptyLine = true; - - res.line(varName + '.' + $generatorJava.setterName(beanPropName) + '(' + beanVarName + ');'); - - res.needEmptyLine = true; - } - else if (createBeanAlthoughNoProps) { - res.emptyLineIfNeeded(); - res.line(varName + '.' + $generatorJava.setterName(beanPropName) + '(new ' + res.importClass(beanClass) + '());'); - - res.needEmptyLine = true; - } -}; - -/** - * Add eviction policy. - * - * @param res Resulting output with generated code. - * @param varName Current using variable name. - * @param evtPlc Data to add. - * @param propName Name in source data. - */ -$generatorJava.evictionPolicy = function(res, varName, evtPlc, propName) { - if (evtPlc && evtPlc.kind) { - const evictionPolicyDesc = $generatorCommon.EVICTION_POLICIES[evtPlc.kind]; - - const obj = evtPlc[evtPlc.kind.toUpperCase()]; - - $generatorJava.beanProperty(res, varName, obj, propName, propName, - evictionPolicyDesc.className, evictionPolicyDesc.fields, true); - } -}; - -// Generate cluster general group. -$generatorJava.clusterGeneral = function(cluster, clientNearCfg, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorJava.declareVariable(res, 'cfg', 'org.apache.ignite.configuration.IgniteConfiguration'); - - $generatorJava.property(res, 'cfg', cluster, 'name', null, 'setGridName'); - res.needEmptyLine = true; - - $generatorJava.property(res, 'cfg', cluster, 'localHost'); - res.needEmptyLine = true; - - if (clientNearCfg) { - res.line('cfg.setClientMode(true);'); - - res.needEmptyLine = true; - } - - if (cluster.discovery) { - const d = cluster.discovery; - - $generatorJava.declareVariable(res, 'discovery', 'org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi'); - - switch (d.kind) { - case 'Multicast': - $generatorJava.beanProperty(res, 'discovery', d.Multicast, 'ipFinder', 'ipFinder', - 'org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder', - { - multicastGroup: null, - multicastPort: null, - responseWaitTime: null, - addressRequestAttempts: null, - localAddress: null, - addresses: {type: 'list'} - }, true); - - break; - - case 'Vm': - $generatorJava.beanProperty(res, 'discovery', d.Vm, 'ipFinder', 'ipFinder', - 'org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder', - {addresses: {type: 'list'}}, true); - - break; - - case 'S3': - $generatorJava.beanProperty(res, 'discovery', d.S3, 'ipFinder', 'ipFinder', - 'org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinder', {bucketName: null}, true); - - break; - - case 'Cloud': - $generatorJava.beanProperty(res, 'discovery', d.Cloud, 'ipFinder', 'ipFinder', - 'org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinder', - { - credential: null, - credentialPath: null, - identity: null, - provider: null, - regions: {type: 'list'}, - zones: {type: 'list'} - }, true); - - break; - - case 'GoogleStorage': - $generatorJava.beanProperty(res, 'discovery', d.GoogleStorage, 'ipFinder', 'ipFinder', - 'org.apache.ignite.spi.discovery.tcp.ipfinder.gce.TcpDiscoveryGoogleStorageIpFinder', - { - projectName: null, - bucketName: null, - serviceAccountP12FilePath: null, - serviceAccountId: null - }, true); - - break; - - case 'Jdbc': - $generatorJava.declareVariable(res, 'ipFinder', - 'org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder'); - $generatorJava.property(res, 'ipFinder', d.Jdbc, 'initSchema'); - - const datasource = d.Jdbc; - if (datasource.dataSourceBean && datasource.dialect) { - res.needEmptyLine = !datasource.initSchema; - - res.line('ipFinder.setDataSource(DataSources.INSTANCE_' + datasource.dataSourceBean + ');'); - } - - res.needEmptyLine = true; - - res.line('discovery.setIpFinder(ipFinder);'); - - break; - - case 'SharedFs': - $generatorJava.beanProperty(res, 'discovery', d.SharedFs, 'ipFinder', 'ipFinder', - 'org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder', {path: null}, true); - - break; - - case 'ZooKeeper': - const finderVar = 'ipFinder'; - - $generatorJava.declareVariable(res, 'ipFinder', 'org.apache.ignite.spi.discovery.tcp.ipfinder.zk.TcpDiscoveryZookeeperIpFinder'); - - if (d.ZooKeeper) { - if ($generatorCommon.isDefinedAndNotEmpty(d.ZooKeeper.curator)) - res.line(finderVar + '.setCurator(new ' + res.importClass(d.ZooKeeper.curator) + '());'); - - $generatorJava.property(res, finderVar, d.ZooKeeper, 'zkConnectionString'); - - if (d.ZooKeeper.retryPolicy && d.ZooKeeper.retryPolicy.kind) { - const kind = d.ZooKeeper.retryPolicy.kind; - const retryPolicy = d.ZooKeeper.retryPolicy[kind]; - - switch (kind) { - case 'ExponentialBackoff': - res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.ExponentialBackoffRetry') + '(' + - $generatorJava.constructorArg(retryPolicy, 'baseSleepTimeMs', 1000) + - $generatorJava.constructorArg(retryPolicy, 'maxRetries', 10, true) + - $generatorJava.constructorArg(retryPolicy, 'maxSleepMs', null, true, true) + '));'); - - break; - - case 'BoundedExponentialBackoff': - res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.BoundedExponentialBackoffRetry') + '(' + - $generatorJava.constructorArg(retryPolicy, 'baseSleepTimeMs', 1000) + - $generatorJava.constructorArg(retryPolicy, 'maxSleepTimeMs', 2147483647, true) + - $generatorJava.constructorArg(retryPolicy, 'maxRetries', 10, true) + '));'); - - break; - - case 'UntilElapsed': - res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.RetryUntilElapsed') + '(' + - $generatorJava.constructorArg(retryPolicy, 'maxElapsedTimeMs', 60000) + - $generatorJava.constructorArg(retryPolicy, 'sleepMsBetweenRetries', 1000, true) + '));'); - - break; - - case 'NTimes': - res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.RetryNTimes') + '(' + - $generatorJava.constructorArg(retryPolicy, 'n', 10) + - $generatorJava.constructorArg(retryPolicy, 'sleepMsBetweenRetries', 1000, true) + '));'); - - break; - - case 'OneTime': - res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.RetryOneTime') + '(' + - $generatorJava.constructorArg(retryPolicy, 'sleepMsBetweenRetry', 1000) + '));'); - - break; - - case 'Forever': - res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.RetryForever') + '(' + - $generatorJava.constructorArg(retryPolicy, 'retryIntervalMs', 1000) + '));'); - - break; - - case 'Custom': - if (retryPolicy && $generatorCommon.isDefinedAndNotEmpty(retryPolicy.className)) - res.line(finderVar + '.setRetryPolicy(new ' + res.importClass(retryPolicy.className) + '());'); - - break; - - default: - } - } - - $generatorJava.property(res, finderVar, d.ZooKeeper, 'basePath', null, null, '/services'); - $generatorJava.property(res, finderVar, d.ZooKeeper, 'serviceName', null, null, 'ignite'); - $generatorJava.property(res, finderVar, d.ZooKeeper, 'allowDuplicateRegistrations', null, null, false); - } - - res.line('discovery.setIpFinder(ipFinder);'); - - break; - - default: - res.line('Unknown discovery kind: ' + d.kind); - } - - res.needEmptyLine = false; - - $generatorJava.clusterDiscovery(d, res); - - res.emptyLineIfNeeded(); - - res.line('cfg.setDiscoverySpi(discovery);'); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate atomics group. -$generatorJava.clusterAtomics = function(atomics, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.hasAtLeastOneProperty(atomics, ['cacheMode', 'atomicSequenceReserveSize', 'backups'])) { - res.startSafeBlock(); - - $generatorJava.declareVariable(res, 'atomicCfg', 'org.apache.ignite.configuration.AtomicConfiguration'); - - $generatorJava.enumProperty(res, 'atomicCfg', atomics, 'cacheMode', 'org.apache.ignite.cache.CacheMode', null, 'PARTITIONED'); - - const cacheMode = atomics.cacheMode ? atomics.cacheMode : 'PARTITIONED'; - - let hasData = cacheMode !== 'PARTITIONED'; - - hasData = $generatorJava.property(res, 'atomicCfg', atomics, 'atomicSequenceReserveSize', null, null, 1000) || hasData; - - if (cacheMode === 'PARTITIONED') - hasData = $generatorJava.property(res, 'atomicCfg', atomics, 'backups', null, null, 0) || hasData; - - res.needEmptyLine = true; - - res.line('cfg.setAtomicConfiguration(atomicCfg);'); - - res.needEmptyLine = true; - - if (!hasData) - res.rollbackSafeBlock(); - } - - return res; -}; - -// Generate binary group. -$generatorJava.clusterBinary = function(binary, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.binaryIsDefined(binary)) { - const varName = 'binary'; - - $generatorJava.declareVariable(res, varName, 'org.apache.ignite.configuration.BinaryConfiguration'); - - if ($generatorCommon.isDefinedAndNotEmpty(binary.idMapper)) - res.line(varName + '.setIdMapper(new ' + res.importClass(binary.idMapper) + '());'); - - if ($generatorCommon.isDefinedAndNotEmpty(binary.nameMapper)) - res.line(varName + '.setNameMapper(new ' + res.importClass(binary.nameMapper) + '());'); - - if ($generatorCommon.isDefinedAndNotEmpty(binary.serializer)) - res.line(varName + '.setSerializer(new ' + res.importClass(binary.serializer) + '());'); - - res.needEmptyLine = $generatorCommon.isDefinedAndNotEmpty(binary.idMapper) || $generatorCommon.isDefinedAndNotEmpty(binary.serializer); - - if ($generatorCommon.isDefinedAndNotEmpty(binary.typeConfigurations)) { - const arrVar = 'types'; - - $generatorJava.declareVariable(res, arrVar, 'java.util.Collection', 'java.util.ArrayList', 'org.apache.ignite.binary.BinaryTypeConfiguration'); - - _.forEach(binary.typeConfigurations, function(type) { - if ($generatorCommon.isDefinedAndNotEmpty(type.typeName)) - res.line(arrVar + '.add(' + $generatorJava.binaryTypeFunctionName(type.typeName) + '());'); // TODO IGNITE-2269 Replace using of separated methods for binary type configurations to extended constructors. - }); - - res.needEmptyLine = true; - - res.line(varName + '.setTypeConfigurations(' + arrVar + ');'); - - res.needEmptyLine = true; - } - - $generatorJava.property(res, varName, binary, 'compactFooter', null, null, true); - - res.needEmptyLine = true; - - res.line('cfg.setBinaryConfiguration(' + varName + ');'); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate cache key configurations. -$generatorJava.clusterCacheKeyConfiguration = function(keyCfgs, res) { - if (!res) - res = $generatorCommon.builder(); - - keyCfgs = _.filter(keyCfgs, (cfg) => cfg.typeName && cfg.affinityKeyFieldName); - - if (_.isEmpty(keyCfgs)) - return res; - - $generatorJava.declareVariableArray(res, 'keyConfigurations', 'org.apache.ignite.cache.CacheKeyConfiguration', keyCfgs.length); - - const cacheKeyCfg = res.importClass('org.apache.ignite.cache.CacheKeyConfiguration'); - - _.forEach(keyCfgs, (cfg, idx) => { - res.needEmptyLine = true; - - res.line(`keyConfigurations[${idx}] = new ${cacheKeyCfg}("${cfg.typeName}", "${cfg.affinityKeyFieldName}");`); - - res.needEmptyLine = true; - }); - - res.line('cfg.setCacheKeyConfiguration(keyConfigurations);'); - - res.needEmptyLine = true; - - return res; -}; - -// TODO IGNITE-2269 Remove specified methods after implamentation of extended constructors. -// Construct binary type configuration factory method name. -$generatorJava.binaryTypeFunctionName = function(typeName) { - const dotIdx = typeName.lastIndexOf('.'); - - const shortName = dotIdx > 0 ? typeName.substr(dotIdx + 1) : typeName; - - return $generatorCommon.toJavaName('binaryType', shortName); -}; - -// TODO IGNITE-2269 Remove specified methods after implamentation of extended constructors. -// Generate factory method for specified BinaryTypeConfiguration. -$generatorJava.binaryTypeConfiguration = function(type, res) { - const typeName = type.typeName; - - res.line('/**'); - res.line(' * Create binary type configuration for ' + typeName + '.'); - res.line(' *'); - res.line(' * @return Configured binary type.'); - res.line(' */'); - res.startBlock('private static BinaryTypeConfiguration ' + $generatorJava.binaryTypeFunctionName(typeName) + '() {'); - - $generatorJava.resetVariables(res); - - const typeVar = 'typeCfg'; - - $generatorJava.declareVariable(res, typeVar, 'org.apache.ignite.binary.BinaryTypeConfiguration'); - - $generatorJava.property(res, typeVar, type, 'typeName'); - - if ($generatorCommon.isDefinedAndNotEmpty(type.idMapper)) - res.line(typeVar + '.setIdMapper(new ' + res.importClass(type.idMapper) + '());'); - - if ($generatorCommon.isDefinedAndNotEmpty(type.nameMapper)) - res.line(typeVar + '.setNameMapper(new ' + res.importClass(type.nameMapper) + '());'); - - if ($generatorCommon.isDefinedAndNotEmpty(type.serializer)) - res.line(typeVar + '.setSerializer(new ' + res.importClass(type.serializer) + '());'); - - $generatorJava.property(res, typeVar, type, 'enum', null, null, false); - - res.needEmptyLine = true; - - res.line('return ' + typeVar + ';'); - res.endBlock('}'); - - res.needEmptyLine = true; -}; - -// TODO IGNITE-2269 Remove specified methods after implamentation of extended constructors. -// Generates binary type configuration factory methods. -$generatorJava.binaryTypeConfigurations = function(binary, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!_.isNil(binary)) { - _.forEach(binary.typeConfigurations, function(type) { - $generatorJava.binaryTypeConfiguration(type, res); - }); - } - - return res; -}; - -// Generate collision group. -$generatorJava.clusterCollision = function(collision, res) { - if (!res) - res = $generatorCommon.builder(); - - if (collision && collision.kind && collision.kind !== 'Noop') { - const spi = collision[collision.kind]; - - if (collision.kind !== 'Custom' || (spi && $generatorCommon.isDefinedAndNotEmpty(spi.class))) { - const varName = 'collisionSpi'; - - switch (collision.kind) { - case 'JobStealing': - $generatorJava.declareVariable(res, varName, 'org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi'); - - $generatorJava.property(res, varName, spi, 'activeJobsThreshold', null, null, 95); - $generatorJava.property(res, varName, spi, 'waitJobsThreshold', null, null, 0); - $generatorJava.property(res, varName, spi, 'messageExpireTime', null, null, 1000); - $generatorJava.property(res, varName, spi, 'maximumStealingAttempts', null, null, 5); - $generatorJava.property(res, varName, spi, 'stealingEnabled', null, null, true); - - if ($generatorCommon.isDefinedAndNotEmpty(spi.externalCollisionListener)) { - res.line(varName + '.' + $generatorJava.setterName('externalCollisionListener') + - '(new ' + res.importClass(spi.externalCollisionListener) + '());'); - } - - if ($generatorCommon.isDefinedAndNotEmpty(spi.stealingAttributes)) { - const stealingAttrsVar = 'stealingAttrs'; - - res.needEmptyLine = true; - - $generatorJava.declareVariable(res, stealingAttrsVar, 'java.util.Map', 'java.util.HashMap', 'String', 'java.io.Serializable'); - - _.forEach(spi.stealingAttributes, function(attr) { - res.line(stealingAttrsVar + '.put("' + attr.name + '", "' + attr.value + '");'); - }); - - res.needEmptyLine = true; - - res.line(varName + '.setStealingAttributes(' + stealingAttrsVar + ');'); - } - - break; - - case 'FifoQueue': - $generatorJava.declareVariable(res, varName, 'org.apache.ignite.spi.collision.fifoqueue.FifoQueueCollisionSpi'); - - $generatorJava.property(res, varName, spi, 'parallelJobsNumber'); - $generatorJava.property(res, varName, spi, 'waitingJobsNumber'); - - break; - - case 'PriorityQueue': - $generatorJava.declareVariable(res, varName, 'org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi'); - - $generatorJava.property(res, varName, spi, 'parallelJobsNumber'); - $generatorJava.property(res, varName, spi, 'waitingJobsNumber'); - $generatorJava.property(res, varName, spi, 'priorityAttributeKey', null, null, 'grid.task.priority'); - $generatorJava.property(res, varName, spi, 'jobPriorityAttributeKey', null, null, 'grid.job.priority'); - $generatorJava.property(res, varName, spi, 'defaultPriority', null, null, 0); - $generatorJava.property(res, varName, spi, 'starvationIncrement', null, null, 1); - $generatorJava.property(res, varName, spi, 'starvationPreventionEnabled', null, null, true); - - break; - - case 'Custom': - $generatorJava.declareVariable(res, varName, spi.class); - - break; - - default: - } - - res.needEmptyLine = true; - - res.line('cfg.setCollisionSpi(' + varName + ');'); - - res.needEmptyLine = true; - } - } - - return res; -}; - -// Generate communication group. -$generatorJava.clusterCommunication = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - const cfg = $generatorCommon.COMMUNICATION_CONFIGURATION; - - $generatorJava.beanProperty(res, 'cfg', cluster.communication, 'communicationSpi', 'commSpi', cfg.className, cfg.fields); - - res.needEmptyLine = false; - - $generatorJava.property(res, 'cfg', cluster, 'networkTimeout', null, null, 5000); - $generatorJava.property(res, 'cfg', cluster, 'networkSendRetryDelay', null, null, 1000); - $generatorJava.property(res, 'cfg', cluster, 'networkSendRetryCount', null, null, 3); - $generatorJava.property(res, 'cfg', cluster, 'segmentCheckFrequency'); - $generatorJava.property(res, 'cfg', cluster, 'waitForSegmentOnStart', null, null, false); - $generatorJava.property(res, 'cfg', cluster, 'discoveryStartupDelay', null, null, 60000); - - res.needEmptyLine = true; - - return res; -}; - -// Generate REST access group. -$generatorJava.clusterConnector = function(connector, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!_.isNil(connector) && connector.enabled) { - const cfg = _.cloneDeep($generatorCommon.CONNECTOR_CONFIGURATION); - - if (connector.sslEnabled) { - cfg.fields.sslClientAuth = {dflt: false}; - cfg.fields.sslFactory = {type: 'bean'}; - } - - $generatorJava.beanProperty(res, 'cfg', connector, 'connectorConfiguration', 'clientCfg', - cfg.className, cfg.fields, true); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate deployment group. -$generatorJava.clusterDeployment = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorJava.enumProperty(res, 'cfg', cluster, 'deploymentMode', 'org.apache.ignite.configuration.DeploymentMode', null, 'SHARED'); - - res.softEmptyLine(); - - const p2pEnabled = cluster.peerClassLoadingEnabled; - - if (!_.isNil(p2pEnabled)) { - $generatorJava.property(res, 'cfg', cluster, 'peerClassLoadingEnabled', null, null, false); - - if (p2pEnabled) { - $generatorJava.property(res, 'cfg', cluster, 'peerClassLoadingMissedResourcesCacheSize', null, null, 100); - $generatorJava.property(res, 'cfg', cluster, 'peerClassLoadingThreadPoolSize', null, null, 2); - $generatorJava.multiparamProperty(res, 'cfg', cluster, 'peerClassLoadingLocalClassPathExclude'); - } - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate discovery group. -$generatorJava.clusterDiscovery = function(disco, res) { - if (!res) - res = $generatorCommon.builder(); - - if (disco) { - $generatorJava.property(res, 'discovery', disco, 'localAddress'); - $generatorJava.property(res, 'discovery', disco, 'localPort', null, null, 47500); - $generatorJava.property(res, 'discovery', disco, 'localPortRange', null, null, 100); - - if ($generatorCommon.isDefinedAndNotEmpty(disco.addressResolver)) { - $generatorJava.beanProperty(res, 'discovery', disco, 'addressResolver', 'addressResolver', disco.addressResolver, {}, true); - res.needEmptyLine = false; - } - - $generatorJava.property(res, 'discovery', disco, 'socketTimeout', null, null, 5000); - $generatorJava.property(res, 'discovery', disco, 'ackTimeout', null, null, 5000); - $generatorJava.property(res, 'discovery', disco, 'maxAckTimeout', null, null, 600000); - $generatorJava.property(res, 'discovery', disco, 'networkTimeout', null, null, 5000); - $generatorJava.property(res, 'discovery', disco, 'joinTimeout', null, null, 0); - $generatorJava.property(res, 'discovery', disco, 'threadPriority', null, null, 10); - $generatorJava.property(res, 'discovery', disco, 'heartbeatFrequency', null, null, 2000); - $generatorJava.property(res, 'discovery', disco, 'maxMissedHeartbeats', null, null, 1); - $generatorJava.property(res, 'discovery', disco, 'maxMissedClientHeartbeats', null, null, 5); - $generatorJava.property(res, 'discovery', disco, 'topHistorySize', null, null, 1000); - - if ($generatorCommon.isDefinedAndNotEmpty(disco.listener)) { - $generatorJava.beanProperty(res, 'discovery', disco, 'listener', 'listener', disco.listener, {}, true); - res.needEmptyLine = false; - } - - if ($generatorCommon.isDefinedAndNotEmpty(disco.dataExchange)) { - $generatorJava.beanProperty(res, 'discovery', disco, 'dataExchange', 'dataExchange', disco.dataExchange, {}, true); - res.needEmptyLine = false; - } - - if ($generatorCommon.isDefinedAndNotEmpty(disco.metricsProvider)) { - $generatorJava.beanProperty(res, 'discovery', disco, 'metricsProvider', 'metricsProvider', disco.metricsProvider, {}, true); - res.needEmptyLine = false; - } - - $generatorJava.property(res, 'discovery', disco, 'reconnectCount', null, null, 10); - $generatorJava.property(res, 'discovery', disco, 'statisticsPrintFrequency', null, null, 0); - $generatorJava.property(res, 'discovery', disco, 'ipFinderCleanFrequency', null, null, 60000); - - if ($generatorCommon.isDefinedAndNotEmpty(disco.authenticator)) { - $generatorJava.beanProperty(res, 'discovery', disco, 'authenticator', 'authenticator', disco.authenticator, {}, true); - res.needEmptyLine = false; - } - - $generatorJava.property(res, 'discovery', disco, 'forceServerMode', null, null, false); - $generatorJava.property(res, 'discovery', disco, 'clientReconnectDisabled', null, null, false); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate events group. -$generatorJava.clusterEvents = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - if (cluster.includeEventTypes && cluster.includeEventTypes.length > 0) { - res.emptyLineIfNeeded(); - - const evtGrps = angular.element(document.getElementById('app')).injector().get('igniteEventGroups'); - - if (cluster.includeEventTypes.length === 1) { - const evtGrp = _.find(evtGrps, {value: cluster.includeEventTypes[0]}); - const evts = res.importStatic(evtGrp.class + '.' + evtGrp.value); - - res.line('cfg.setIncludeEventTypes(' + evts + ');'); - } - else { - _.forEach(cluster.includeEventTypes, function(value, ix) { - const evtGrp = _.find(evtGrps, {value}); - const evts = res.importStatic(evtGrp.class + '.' + evtGrp.value); - - if (ix === 0) - res.line('int[] events = new int[' + evts + '.length'); - else - res.line(' + ' + evts + '.length'); - }); - - res.line('];'); - - res.needEmptyLine = true; - - res.line('int k = 0;'); - - _.forEach(cluster.includeEventTypes, function(value, idx) { - res.needEmptyLine = true; - - const evtGrp = _.find(evtGrps, {value}); - const evts = res.importStatic(evtGrp.class + '.' + value); - - res.line('System.arraycopy(' + evts + ', 0, events, k, ' + evts + '.length);'); - - if (idx < cluster.includeEventTypes.length - 1) - res.line('k += ' + evts + '.length;'); - }); - - res.needEmptyLine = true; - - res.line('cfg.setIncludeEventTypes(events);'); - } - - res.needEmptyLine = true; - } - - res.needEmptyLine = true; - - return res; -}; - -// Generate failover group. -$generatorJava.clusterFailover = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.isDefinedAndNotEmpty(cluster.failoverSpi) && _.findIndex(cluster.failoverSpi, function(spi) { - return $generatorCommon.isDefinedAndNotEmpty(spi.kind) && (spi.kind !== 'Custom' || $generatorCommon.isDefinedAndNotEmpty(_.get(spi, spi.kind + '.class'))); - }) >= 0) { - const arrayVarName = 'failoverSpiList'; - - $generatorJava.declareVariable(res, arrayVarName, 'java.util.List', 'java.util.ArrayList', 'org.apache.ignite.spi.failover.FailoverSpi'); - - _.forEach(cluster.failoverSpi, function(spi) { - if (spi.kind && (spi.kind !== 'Custom' || $generatorCommon.isDefinedAndNotEmpty(_.get(spi, spi.kind + '.class')))) { - const varName = 'failoverSpi'; - - const maxAttempts = _.get(spi, spi.kind + '.maximumFailoverAttempts'); - - if ((spi.kind === 'JobStealing' || spi.kind === 'Always') && $generatorCommon.isDefinedAndNotEmpty(maxAttempts) && maxAttempts !== 5) { - const spiCls = res.importClass($generatorCommon.failoverSpiClass(spi)); - - $generatorJava.declareVariableCustom(res, varName, 'org.apache.ignite.spi.failover.FailoverSpi', 'new ' + spiCls + '()'); - - if ($generatorCommon.isDefinedAndNotEmpty(spi[spi.kind].maximumFailoverAttempts)) - res.line('((' + spiCls + ') ' + varName + ').setMaximumFailoverAttempts(' + spi[spi.kind].maximumFailoverAttempts + ');'); - - res.needEmptyLine = true; - - res.line(arrayVarName + '.add(' + varName + ');'); - } - else - res.line(arrayVarName + '.add(new ' + res.importClass($generatorCommon.failoverSpiClass(spi)) + '());'); - - res.needEmptyLine = true; - } - }); - - res.line('cfg.setFailoverSpi(' + arrayVarName + '.toArray(new FailoverSpi[' + arrayVarName + '.size()]));'); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate marshaller group. -$generatorJava.clusterLogger = function(logger, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.loggerConfigured(logger)) { - const varName = 'logger'; - - const log = logger[logger.kind]; - - switch (logger.kind) { - case 'Log4j2': - $generatorJava.declareVariableCustom(res, varName, 'org.apache.ignite.logger.log4j2.Log4J2Logger', - 'new Log4J2Logger(' + $generatorJava.toJavaCode(log.path, 'path') + ')'); - - res.needEmptyLine = true; - - if ($generatorCommon.isDefinedAndNotEmpty(log.level)) - res.line(varName + '.setLevel(' + res.importClass('org.apache.logging.log4j.Level') + '.' + log.level + ');'); - - break; - - case 'Null': - $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.NullLogger'); - - break; - - case 'Java': - $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.java.JavaLogger'); - - break; - - case 'JCL': - $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.jcl.JclLogger'); - - break; - - case 'SLF4J': - $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.slf4j.Slf4jLogger'); - - break; - - case 'Log4j': - if (log.mode === 'Default') - $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.log4j.Log4JLogger'); - else { - $generatorJava.declareVariableCustom(res, varName, 'org.apache.ignite.logger.log4j.Log4JLogger', - 'new Log4JLogger(' + $generatorJava.toJavaCode(log.path, 'path') + ')'); - } - - if ($generatorCommon.isDefinedAndNotEmpty(log.level)) - res.line(varName + '.setLevel(' + res.importClass('org.apache.log4j.Level') + '.' + log.level + ');'); - - break; - - case 'Custom': - $generatorJava.declareVariable(res, varName, log.class); - - break; - - default: - } - - res.needEmptyLine = true; - - res.line('cfg.setGridLogger(' + varName + ');'); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate marshaller group. -$generatorJava.clusterMarshaller = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - const marshaller = cluster.marshaller; - - if (marshaller && marshaller.kind) { - const marshallerDesc = $generatorCommon.MARSHALLERS[marshaller.kind]; - - $generatorJava.beanProperty(res, 'cfg', marshaller[marshaller.kind], 'marshaller', 'marshaller', - marshallerDesc.className, marshallerDesc.fields, true); - - $generatorJava.beanProperty(res, 'marshaller', marshaller[marshaller.kind], marshallerDesc.className, marshallerDesc.fields, true); - } - - $generatorJava.property(res, 'cfg', cluster, 'marshalLocalJobs', null, null, false); - $generatorJava.property(res, 'cfg', cluster, 'marshallerCacheKeepAliveTime', null, null, 10000); - $generatorJava.property(res, 'cfg', cluster, 'marshallerCacheThreadPoolSize', null, 'setMarshallerCachePoolSize'); - - res.needEmptyLine = true; - - return res; -}; - -// Generate metrics group. -$generatorJava.clusterMetrics = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorJava.property(res, 'cfg', cluster, 'metricsExpireTime'); - $generatorJava.property(res, 'cfg', cluster, 'metricsHistorySize', null, null, 10000); - $generatorJava.property(res, 'cfg', cluster, 'metricsLogFrequency', null, null, 60000); - $generatorJava.property(res, 'cfg', cluster, 'metricsUpdateFrequency', null, null, 2000); - - res.needEmptyLine = true; - - return res; -}; - -// Generate swap group. -$generatorJava.clusterSwap = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - if (cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind === 'FileSwapSpaceSpi') { - $generatorJava.beanProperty(res, 'cfg', cluster.swapSpaceSpi.FileSwapSpaceSpi, 'swapSpaceSpi', 'swapSpi', - $generatorCommon.SWAP_SPACE_SPI.className, $generatorCommon.SWAP_SPACE_SPI.fields, true); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate time group. -$generatorJava.clusterTime = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorJava.property(res, 'cfg', cluster, 'clockSyncSamples', null, null, 8); - $generatorJava.property(res, 'cfg', cluster, 'clockSyncFrequency', null, null, 120000); - $generatorJava.property(res, 'cfg', cluster, 'timeServerPortBase', null, null, 31100); - $generatorJava.property(res, 'cfg', cluster, 'timeServerPortRange', null, null, 100); - - res.needEmptyLine = true; - - return res; -}; - -// Generate ODBC configuration group. -$generatorJava.clusterODBC = function(odbc, res) { - if (!res) - res = $generatorCommon.builder(); - - if (odbc && odbc.odbcEnabled) { - $generatorJava.beanProperty(res, 'cfg', odbc, 'odbcConfiguration', 'odbcConfiguration', - $generatorCommon.ODBC_CONFIGURATION.className, $generatorCommon.ODBC_CONFIGURATION.fields, true); - } - - res.needEmptyLine = true; - - return res; -}; - -// Generate thread pools group. -$generatorJava.clusterPools = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorJava.property(res, 'cfg', cluster, 'publicThreadPoolSize'); - $generatorJava.property(res, 'cfg', cluster, 'systemThreadPoolSize'); - $generatorJava.property(res, 'cfg', cluster, 'managementThreadPoolSize'); - $generatorJava.property(res, 'cfg', cluster, 'igfsThreadPoolSize'); - $generatorJava.property(res, 'cfg', cluster, 'rebalanceThreadPoolSize'); - - res.needEmptyLine = true; - - return res; -}; - -// Generate transactions group. -$generatorJava.clusterTransactions = function(transactionConfiguration, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorJava.beanProperty(res, 'cfg', transactionConfiguration, 'transactionConfiguration', - 'transactionConfiguration', $generatorCommon.TRANSACTION_CONFIGURATION.className, - $generatorCommon.TRANSACTION_CONFIGURATION.fields, false); - - return res; -}; - -// Generate user attributes group. -$generatorJava.clusterUserAttributes = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.isDefinedAndNotEmpty(cluster.attributes)) { - $generatorJava.declareVariable(res, 'attributes', 'java.util.Map', 'java.util.HashMap', 'java.lang.String', 'java.lang.String'); - - _.forEach(cluster.attributes, function(attr) { - res.line('attributes.put("' + attr.name + '", "' + attr.value + '");'); - }); - - res.needEmptyLine = true; - - res.line('cfg.setUserAttributes(attributes);'); - - res.needEmptyLine = true; - } - - res.needEmptyLine = true; - - return res; -}; - - -// Generate cache general group. -$generatorJava.cacheGeneral = function(cache, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!varName) - varName = $generatorJava.nextVariableName('cache', cache); - - $generatorJava.property(res, varName, cache, 'name'); - - $generatorJava.enumProperty(res, varName, cache, 'cacheMode', 'org.apache.ignite.cache.CacheMode'); - $generatorJava.enumProperty(res, varName, cache, 'atomicityMode', 'org.apache.ignite.cache.CacheAtomicityMode'); - - if (cache.cacheMode === 'PARTITIONED' && $generatorJava.property(res, varName, cache, 'backups')) - $generatorJava.property(res, varName, cache, 'readFromBackup'); - - $generatorJava.property(res, varName, cache, 'copyOnRead'); - - if (cache.cacheMode === 'PARTITIONED' && cache.atomicityMode === 'TRANSACTIONAL') - $generatorJava.property(res, varName, cache, 'invalidate'); - - res.needEmptyLine = true; - - return res; -}; - -// Generate cache memory group. -$generatorJava.cacheMemory = function(cache, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!varName) - varName = $generatorJava.nextVariableName('cache', cache); - - $generatorJava.enumProperty(res, varName, cache, 'memoryMode', 'org.apache.ignite.cache.CacheMemoryMode', null, 'ONHEAP_TIERED'); - - if (cache.memoryMode !== 'OFFHEAP_VALUES') - $generatorJava.property(res, varName, cache, 'offHeapMaxMemory', null, null, -1); - - res.softEmptyLine(); - - $generatorJava.evictionPolicy(res, varName, cache.evictionPolicy, 'evictionPolicy'); - - $generatorJava.property(res, varName, cache, 'startSize', null, null, 1500000); - $generatorJava.property(res, varName, cache, 'swapEnabled', null, null, false); - - res.needEmptyLine = true; - - return res; -}; - -// Generate cache query & indexing group. -$generatorJava.cacheQuery = function(cache, domains, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!varName) - varName = $generatorJava.nextVariableName('cache', cache); - - $generatorJava.property(res, varName, cache, 'sqlSchema'); - $generatorJava.property(res, varName, cache, 'sqlOnheapRowCacheSize', null, null, 10240); - $generatorJava.property(res, varName, cache, 'longQueryWarningTimeout', null, null, 3000); - - const indexedTypes = _.reduce(domains, (acc, domain) => { - if (domain.queryMetadata === 'Annotations') { - acc.push(domain.keyType); - acc.push(domain.valueType); - } - - return acc; - }, []); - - if (indexedTypes.length > 0) { - res.softEmptyLine(); - - $generatorJava.multiparamProperty(res, varName, {indexedTypes}, 'indexedTypes', 'class'); - } - - res.softEmptyLine(); - - $generatorJava.multiparamProperty(res, varName, cache, 'sqlFunctionClasses', 'class'); - - res.softEmptyLine(); - - $generatorJava.property(res, varName, cache, 'snapshotableIndex', null, null, false); - $generatorJava.property(res, varName, cache, 'sqlEscapeAll', null, null, false); - - res.needEmptyLine = true; - - return res; -}; - -/** - * Generate cache store datasource. - * - * @param storeFactory Factory to generate data source for. - * @param res Resulting output with generated code. - */ -$generatorJava.cacheStoreDataSource = function(storeFactory, res) { - const dialect = storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : null) : storeFactory.dialect; - - if (dialect) { - const varName = 'dataSource'; - - const dataSourceBean = storeFactory.dataSourceBean; - - const varType = res.importClass($generatorCommon.dataSourceClassName(dialect)); - - res.line('public static final ' + varType + ' INSTANCE_' + dataSourceBean + ' = create' + dataSourceBean + '();'); - - res.needEmptyLine = true; - - res.startBlock('private static ' + varType + ' create' + dataSourceBean + '() {'); - if (dialect === 'Oracle') - res.startBlock('try {'); - - $generatorJava.resetVariables(res); - - $generatorJava.declareVariable(res, varName, varType); - - switch (dialect) { - case 'Generic': - res.line(varName + '.setJdbcUrl(props.getProperty("' + dataSourceBean + '.jdbc.url"));'); - - break; - - case 'DB2': - res.line(varName + '.setServerName(props.getProperty("' + dataSourceBean + '.jdbc.server_name"));'); - res.line(varName + '.setPortNumber(Integer.valueOf(props.getProperty("' + dataSourceBean + '.jdbc.port_number")));'); - res.line(varName + '.setDatabaseName(props.getProperty("' + dataSourceBean + '.jdbc.database_name"));'); - res.line(varName + '.setDriverType(Integer.valueOf(props.getProperty("' + dataSourceBean + '.jdbc.driver_type")));'); - - break; - - case 'PostgreSQL': - res.line(varName + '.setUrl(props.getProperty("' + dataSourceBean + '.jdbc.url"));'); - - break; - - default: - res.line(varName + '.setURL(props.getProperty("' + dataSourceBean + '.jdbc.url"));'); - } - - res.line(varName + '.setUser(props.getProperty("' + dataSourceBean + '.jdbc.username"));'); - res.line(varName + '.setPassword(props.getProperty("' + dataSourceBean + '.jdbc.password"));'); - - res.needEmptyLine = true; - - res.line('return dataSource;'); - - if (dialect === 'Oracle') { - res.endBlock('}'); - res.startBlock('catch (' + res.importClass('java.sql.SQLException') + ' ex) {'); - res.line('throw new Error(ex);'); - res.endBlock('}'); - } - - res.endBlock('}'); - - res.needEmptyLine = true; - - return dataSourceBean; - } - - return null; -}; - -$generatorJava.clusterDataSources = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - const datasources = []; - - let storeFound = false; - - function startSourcesFunction() { - if (!storeFound) { - res.line('/** Helper class for datasource creation. */'); - res.startBlock('public static class DataSources {'); - - storeFound = true; - } - } - - _.forEach(cluster.caches, function(cache) { - const factoryKind = cache.cacheStoreFactory.kind; - - const storeFactory = cache.cacheStoreFactory[factoryKind]; - - if (storeFactory) { - const beanClassName = $generatorJava.dataSourceClassName(res, storeFactory); - - if (beanClassName && !_.includes(datasources, beanClassName)) { - datasources.push(beanClassName); - - if (factoryKind === 'CacheJdbcPojoStoreFactory' || factoryKind === 'CacheJdbcBlobStoreFactory') { - startSourcesFunction(); - - $generatorJava.cacheStoreDataSource(storeFactory, res); - } - } - } - }); - - if (cluster.discovery.kind === 'Jdbc') { - const datasource = cluster.discovery.Jdbc; - - if (datasource.dataSourceBean && datasource.dialect) { - const beanClassName = $generatorJava.dataSourceClassName(res, datasource); - - if (beanClassName && !_.includes(datasources, beanClassName)) { - startSourcesFunction(); - - $generatorJava.cacheStoreDataSource(datasource, res); - } - } - } - - if (storeFound) - res.endBlock('}'); - - return res; -}; - -/** - * Generate cache store group. - * - * @param cache Cache descriptor. - * @param domains Domain model descriptors. - * @param cacheVarName Cache variable name. - * @param res Resulting output with generated code. - * @returns {*} Java code for cache store configuration. - */ -$generatorJava.cacheStore = function(cache, domains, cacheVarName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!cacheVarName) - cacheVarName = $generatorJava.nextVariableName('cache', cache); - - if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) { - const factoryKind = cache.cacheStoreFactory.kind; - - const storeFactory = cache.cacheStoreFactory[factoryKind]; - - if (storeFactory) { - const storeFactoryDesc = $generatorCommon.STORE_FACTORIES[factoryKind]; - - const varName = 'storeFactory' + storeFactoryDesc.suffix; - - if (factoryKind === 'CacheJdbcPojoStoreFactory') { - // Generate POJO store factory. - $generatorJava.declareVariable(res, varName, 'org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory', null, null, null, true); - res.deep++; - - res.line('/** {@inheritDoc} */'); - res.startBlock('@Override public ' + res.importClass('org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStore') + ' create() {'); - - res.line('setDataSource(DataSources.INSTANCE_' + storeFactory.dataSourceBean + ');'); - - res.needEmptyLine = true; - - res.line('return super.create();'); - res.endBlock('}'); - res.endBlock('};'); - - res.needEmptyLine = true; - - res.line(varName + '.setDialect(new ' + - res.importClass($generatorCommon.jdbcDialectClassName(storeFactory.dialect)) + '());'); - - res.needEmptyLine = true; - - if (storeFactory.sqlEscapeAll) { - res.line(varName + '.setSqlEscapeAll(true);'); - - res.needEmptyLine = true; - } - - const domainConfigs = _.filter(domains, function(domain) { - return $generatorCommon.domainQueryMetadata(domain) === 'Configuration' && - $generatorCommon.isDefinedAndNotEmpty(domain.databaseTable); - }); - - if ($generatorCommon.isDefinedAndNotEmpty(domainConfigs)) { - $generatorJava.declareVariable(res, 'jdbcTypes', 'java.util.Collection', 'java.util.ArrayList', 'org.apache.ignite.cache.store.jdbc.JdbcType'); - - res.needEmptyLine = true; - - _.forEach(domainConfigs, function(domain) { - if ($generatorCommon.isDefinedAndNotEmpty(domain.databaseTable)) - res.line('jdbcTypes.add(jdbcType' + $generatorJava.extractType(domain.valueType) + '(' + cacheVarName + '.getName()));'); - }); - - res.needEmptyLine = true; - - res.line(varName + '.setTypes(jdbcTypes.toArray(new JdbcType[jdbcTypes.size()]));'); - - res.needEmptyLine = true; - } - - res.line(cacheVarName + '.setCacheStoreFactory(' + varName + ');'); - } - else if (factoryKind === 'CacheJdbcBlobStoreFactory') { - // Generate POJO store factory. - $generatorJava.declareVariable(res, varName, 'org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory', null, null, null, storeFactory.connectVia === 'DataSource'); - - if (storeFactory.connectVia === 'DataSource') { - res.deep++; - - res.line('/** {@inheritDoc} */'); - res.startBlock('@Override public ' + res.importClass('org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStore') + ' create() {'); - - res.line('setDataSource(DataSources.INSTANCE_' + storeFactory.dataSourceBean + ');'); - - res.needEmptyLine = true; - - res.line('return super.create();'); - res.endBlock('}'); - res.endBlock('};'); - - res.needEmptyLine = true; - - $generatorJava.property(res, varName, storeFactory, 'initSchema'); - $generatorJava.property(res, varName, storeFactory, 'createTableQuery'); - $generatorJava.property(res, varName, storeFactory, 'loadQuery'); - $generatorJava.property(res, varName, storeFactory, 'insertQuery'); - $generatorJava.property(res, varName, storeFactory, 'updateQuery'); - $generatorJava.property(res, varName, storeFactory, 'deleteQuery'); - } - else { - $generatorJava.property(res, varName, storeFactory, 'connectionUrl'); - - if (storeFactory.user) { - $generatorJava.property(res, varName, storeFactory, 'user'); - res.line(varName + '.setPassword(props.getProperty("ds.' + storeFactory.user + '.password"));'); - } - } - - res.needEmptyLine = true; - - res.line(cacheVarName + '.setCacheStoreFactory(' + varName + ');'); - } - else - $generatorJava.beanProperty(res, cacheVarName, storeFactory, 'cacheStoreFactory', varName, storeFactoryDesc.className, storeFactoryDesc.fields, true); - - res.needEmptyLine = true; - } - } - - res.softEmptyLine(); - - $generatorJava.property(res, cacheVarName, cache, 'storeKeepBinary', null, null, false); - $generatorJava.property(res, cacheVarName, cache, 'loadPreviousValue', null, null, false); - $generatorJava.property(res, cacheVarName, cache, 'readThrough', null, null, false); - $generatorJava.property(res, cacheVarName, cache, 'writeThrough', null, null, false); - - res.softEmptyLine(); - - if (cache.writeBehindEnabled) { - $generatorJava.property(res, cacheVarName, cache, 'writeBehindEnabled', null, null, false); - $generatorJava.property(res, cacheVarName, cache, 'writeBehindBatchSize', null, null, 512); - $generatorJava.property(res, cacheVarName, cache, 'writeBehindFlushSize', null, null, 10240); - $generatorJava.property(res, cacheVarName, cache, 'writeBehindFlushFrequency', null, null, 5000); - $generatorJava.property(res, cacheVarName, cache, 'writeBehindFlushThreadCount', null, null, 1); - } - - res.needEmptyLine = true; - - return res; -}; - -// Generate cache node filter group. -$generatorJava.cacheNodeFilter = function(cache, igfss, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!varName) - varName = $generatorJava.nextVariableName('cache', cache); - - switch (_.get(cache, 'nodeFilter.kind')) { - case 'IGFS': - const foundIgfs = _.find(igfss, (igfs) => igfs._id === cache.nodeFilter.IGFS.igfs); - - if (foundIgfs) { - const predClsName = res.importClass('org.apache.ignite.internal.processors.igfs.IgfsNodePredicate'); - - res.line(`${varName}.setNodeFilter(new ${predClsName}("${foundIgfs.name}"));`); - } - - break; - - case 'OnNodes': - const nodes = cache.nodeFilter.OnNodes.nodeIds; - - if ($generatorCommon.isDefinedAndNotEmpty(nodes)) { - const startQuote = res.importClass('java.util.UUID') + '.fromString("'; - - $generatorJava.fxVarArgs(res, varName + '.setNodeFilter(new ' + - res.importClass('org.apache.ignite.internal.util.lang.GridNodePredicate'), true, nodes, '(', '))', - startQuote, '")'); - } - - break; - - case 'Custom': - res.line(varName + '.setNodeFilter(new ' + res.importClass(cache.nodeFilter.Custom.className) + '());'); - - break; - - default: break; - } - - res.needEmptyLine = true; - - return res; -}; - -// Generate cache concurrency group. -$generatorJava.cacheConcurrency = function(cache, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!varName) - varName = $generatorJava.nextVariableName('cache', cache); - - $generatorJava.property(res, varName, cache, 'maxConcurrentAsyncOperations', null, null, 500); - $generatorJava.property(res, varName, cache, 'defaultLockTimeout', null, null, 0); - $generatorJava.enumProperty(res, varName, cache, 'atomicWriteOrderMode', 'org.apache.ignite.cache.CacheAtomicWriteOrderMode'); - $generatorJava.enumProperty(res, varName, cache, 'writeSynchronizationMode', 'org.apache.ignite.cache.CacheWriteSynchronizationMode', null, 'PRIMARY_SYNC'); - - res.needEmptyLine = true; - - return res; -}; - -// Generate cache rebalance group. -$generatorJava.cacheRebalance = function(cache, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!varName) - varName = $generatorJava.nextVariableName('cache', cache); - - if (cache.cacheMode !== 'LOCAL') { - $generatorJava.enumProperty(res, varName, cache, 'rebalanceMode', 'org.apache.ignite.cache.CacheRebalanceMode', null, 'ASYNC'); - $generatorJava.property(res, varName, cache, 'rebalanceThreadPoolSize', null, null, 1); - $generatorJava.property(res, varName, cache, 'rebalanceBatchSize', null, null, 524288); - $generatorJava.property(res, varName, cache, 'rebalanceBatchesPrefetchCount', null, null, 2); - $generatorJava.property(res, varName, cache, 'rebalanceOrder', null, null, 0); - $generatorJava.property(res, varName, cache, 'rebalanceDelay', null, null, 0); - $generatorJava.property(res, varName, cache, 'rebalanceTimeout', null, null, 10000); - $generatorJava.property(res, varName, cache, 'rebalanceThrottle', null, null, 0); - } - - res.softEmptyLine(); - - if (cache.igfsAffinnityGroupSize) { - res.line(varName + '.setAffinityMapper(new ' + res.importClass('org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper') + '(' + cache.igfsAffinnityGroupSize + '));'); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate cache server near cache group. -$generatorJava.cacheServerNearCache = function(cache, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!varName) - varName = $generatorJava.nextVariableName('cache', cache); - - if (cache.cacheMode === 'PARTITIONED' && cache.nearCacheEnabled) { - res.needEmptyLine = true; - - if (cache.nearConfiguration) { - $generatorJava.declareVariable(res, 'nearCfg', 'org.apache.ignite.configuration.NearCacheConfiguration'); - - res.needEmptyLine = true; - - if (cache.nearConfiguration.nearStartSize) { - $generatorJava.property(res, 'nearCfg', cache.nearConfiguration, 'nearStartSize', null, null, 375000); - - res.needEmptyLine = true; - } - - if (cache.nearConfiguration.nearEvictionPolicy && cache.nearConfiguration.nearEvictionPolicy.kind) { - $generatorJava.evictionPolicy(res, 'nearCfg', cache.nearConfiguration.nearEvictionPolicy, 'nearEvictionPolicy'); - - res.needEmptyLine = true; - } - - res.line(varName + '.setNearConfiguration(nearCfg);'); - - res.needEmptyLine = true; - } - } - - return res; -}; - -// Generate cache statistics group. -$generatorJava.cacheStatistics = function(cache, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!varName) - varName = $generatorJava.nextVariableName('cache', cache); - - $generatorJava.property(res, varName, cache, 'statisticsEnabled', null, null, false); - $generatorJava.property(res, varName, cache, 'managementEnabled', null, null, false); - - res.needEmptyLine = true; - - return res; -}; - -// Generate domain model query fields. -$generatorJava.domainModelQueryFields = function(res, domain) { - const fields = domain.fields; - - if (fields && fields.length > 0) { - $generatorJava.declareVariable(res, 'fields', 'java.util.LinkedHashMap', 'java.util.LinkedHashMap', 'java.lang.String', 'java.lang.String'); - - _.forEach(fields, function(field) { - res.line('fields.put("' + field.name + '", "' + $generatorCommon.JavaTypes.fullClassName(field.className) + '");'); - }); - - res.needEmptyLine = true; - - res.line('qryMeta.setFields(fields);'); - - res.needEmptyLine = true; - } -}; - -// Generate domain model query aliases. -$generatorJava.domainModelQueryAliases = function(res, domain) { - const aliases = domain.aliases; - - if (aliases && aliases.length > 0) { - $generatorJava.declareVariable(res, 'aliases', 'java.util.Map', 'java.util.HashMap', 'java.lang.String', 'java.lang.String'); - - _.forEach(aliases, function(alias) { - res.line('aliases.put("' + alias.field + '", "' + alias.alias + '");'); - }); - - res.needEmptyLine = true; - - res.line('qryMeta.setAliases(aliases);'); - - res.needEmptyLine = true; - } -}; - -// Generate domain model indexes. -$generatorJava.domainModelQueryIndexes = function(res, domain) { - const indexes = domain.indexes; - - if (indexes && indexes.length > 0) { - res.needEmptyLine = true; - - $generatorJava.declareVariable(res, 'indexes', 'java.util.List', 'java.util.ArrayList', 'org.apache.ignite.cache.QueryIndex'); - - _.forEach(indexes, function(index) { - const fields = index.fields; - - // One row generation for 1 field index. - if (fields && fields.length === 1) { - const field = index.fields[0]; - - res.line('indexes.add(new ' + res.importClass('org.apache.ignite.cache.QueryIndex') + - '("' + field.name + '", ' + - res.importClass('org.apache.ignite.cache.QueryIndexType') + '.' + index.indexType + ', ' + - field.direction + ', "' + index.name + '"));'); - } - else { - res.needEmptyLine = true; - - $generatorJava.declareVariable(res, 'index', 'org.apache.ignite.cache.QueryIndex'); - - $generatorJava.property(res, 'index', index, 'name'); - $generatorJava.enumProperty(res, 'index', index, 'indexType', 'org.apache.ignite.cache.QueryIndexType'); - - res.needEmptyLine = true; - - if (fields && fields.length > 0) { - $generatorJava.declareVariable(res, 'indFlds', 'java.util.LinkedHashMap', 'java.util.LinkedHashMap', 'String', 'Boolean'); - - _.forEach(fields, function(field) { - res.line('indFlds.put("' + field.name + '", ' + field.direction + ');'); - }); - - res.needEmptyLine = true; - - res.line('index.setFields(indFlds);'); - - res.needEmptyLine = true; - } - - res.line('indexes.add(index);'); - } - }); - - res.needEmptyLine = true; - - res.line('qryMeta.setIndexes(indexes);'); - - res.needEmptyLine = true; - } -}; - -// Generate domain model db fields. -$generatorJava.domainModelDatabaseFields = function(res, domain, fieldProperty) { - const dbFields = domain[fieldProperty]; - - if (dbFields && dbFields.length > 0) { - res.needEmptyLine = true; - - res.importClass('java.sql.Types'); - - res.startBlock('jdbcType.' + $generatorCommon.toJavaName('set', fieldProperty) + '('); - - const lastIx = dbFields.length - 1; - - res.importClass('org.apache.ignite.cache.store.jdbc.JdbcTypeField'); - - _.forEach(dbFields, function(field, ix) { - res.line('new JdbcTypeField(' + - 'Types.' + field.databaseFieldType + ', ' + '"' + field.databaseFieldName + '", ' + - res.importClass(field.javaFieldType) + '.class, ' + '"' + field.javaFieldName + '"' + ')' + (ix < lastIx ? ',' : '')); - }); - - res.endBlock(');'); - - res.needEmptyLine = true; - } -}; - -// Generate domain model general group. -$generatorJava.domainModelGeneral = function(domain, res) { - if (!res) - res = $generatorCommon.builder(); - - switch ($generatorCommon.domainQueryMetadata(domain)) { - case 'Annotations': - if ($generatorCommon.isDefinedAndNotEmpty(domain.keyType) || $generatorCommon.isDefinedAndNotEmpty(domain.valueType)) { - const types = []; - - if ($generatorCommon.isDefinedAndNotEmpty(domain.keyType)) - types.push($generatorJava.toJavaCode(res.importClass(domain.keyType), 'class')); - else - types.push('???'); - - if ($generatorCommon.isDefinedAndNotEmpty(domain.valueType)) - types.push($generatorJava.toJavaCode(res.importClass(domain.valueType), 'class')); - else - types.push('???'); - - if ($generatorCommon.isDefinedAndNotEmpty(types)) - $generatorJava.fxVarArgs(res, 'cache.setIndexedTypes', false, types); - } - - break; - - case 'Configuration': - $generatorJava.classNameProperty(res, 'jdbcTypes', domain, 'keyType'); - $generatorJava.property(res, 'jdbcTypes', domain, 'valueType'); - - if ($generatorCommon.isDefinedAndNotEmpty(domain.fields)) { - res.needEmptyLine = true; - - $generatorJava.classNameProperty(res, 'qryMeta', domain, 'keyType'); - $generatorJava.property(res, 'qryMeta', domain, 'valueType'); - } - - break; - - default: - } - - res.needEmptyLine = true; - - return res; -}; - -// Generate domain model for query group. -$generatorJava.domainModelQuery = function(domain, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.domainQueryMetadata(domain) === 'Configuration') { - $generatorJava.domainModelQueryFields(res, domain); - $generatorJava.domainModelQueryAliases(res, domain); - $generatorJava.domainModelQueryIndexes(res, domain); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate domain model for store group. -$generatorJava.domainStore = function(domain, withTypes, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorJava.property(res, 'jdbcType', domain, 'databaseSchema'); - $generatorJava.property(res, 'jdbcType', domain, 'databaseTable'); - - if (withTypes) { - $generatorJava.classNameProperty(res, 'jdbcType', domain, 'keyType'); - $generatorJava.property(res, 'jdbcType', domain, 'valueType'); - } - - $generatorJava.domainModelDatabaseFields(res, domain, 'keyFields'); - $generatorJava.domainModelDatabaseFields(res, domain, 'valueFields'); - - res.needEmptyLine = true; - - return res; -}; - -// Generate domain model configs. -$generatorJava.cacheDomains = function(domains, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - const domainConfigs = _.filter(domains, function(domain) { - return $generatorCommon.domainQueryMetadata(domain) === 'Configuration' && - $generatorCommon.isDefinedAndNotEmpty(domain.fields); - }); - - // Generate domain model configs. - if ($generatorCommon.isDefinedAndNotEmpty(domainConfigs)) { - $generatorJava.declareVariable(res, 'queryEntities', 'java.util.Collection', 'java.util.ArrayList', 'org.apache.ignite.cache.QueryEntity'); - - _.forEach(domainConfigs, function(domain) { - if ($generatorCommon.isDefinedAndNotEmpty(domain.fields)) - res.line('queryEntities.add(queryEntity' + $generatorJava.extractType(domain.valueType) + '());'); - }); - - res.needEmptyLine = true; - - res.line(varName + '.setQueryEntities(queryEntities);'); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate cache configs. -$generatorJava.cache = function(cache, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorJava.cacheGeneral(cache, varName, res); - $generatorJava.cacheMemory(cache, varName, res); - $generatorJava.cacheQuery(cache, cache.domains, varName, res); - $generatorJava.cacheStore(cache, cache.domains, varName, res); - - const igfs = _.get(cache, 'nodeFilter.IGFS.instance'); - - $generatorJava.cacheNodeFilter(cache, igfs ? [igfs] : [], varName, res); - $generatorJava.cacheConcurrency(cache, varName, res); - $generatorJava.cacheRebalance(cache, varName, res); - $generatorJava.cacheServerNearCache(cache, varName, res); - $generatorJava.cacheStatistics(cache, varName, res); - $generatorJava.cacheDomains(cache.domains, varName, res); -}; - -// Generation of cache domain model in separate methods. -$generatorJava.clusterDomains = function(caches, res) { - const domains = []; - - const typeVarName = 'jdbcType'; - const metaVarName = 'qryMeta'; - - _.forEach(caches, function(cache) { - _.forEach(cache.domains, function(domain) { - if (_.isNil(_.find(domains, function(m) { - return m === domain.valueType; - }))) { - $generatorJava.resetVariables(res); - - const type = $generatorJava.extractType(domain.valueType); - - if ($generatorCommon.isDefinedAndNotEmpty(domain.databaseTable)) { - res.line('/**'); - res.line(' * Create JDBC type for ' + type + '.'); - res.line(' *'); - res.line(' * @param cacheName Cache name.'); - res.line(' * @return Configured JDBC type.'); - res.line(' */'); - res.startBlock('private static JdbcType jdbcType' + type + '(String cacheName) {'); - - $generatorJava.declareVariable(res, typeVarName, 'org.apache.ignite.cache.store.jdbc.JdbcType'); - - res.needEmptyLine = true; - - res.line(typeVarName + '.setCacheName(cacheName);'); - - $generatorJava.domainStore(domain, true, res); - - res.needEmptyLine = true; - - res.line('return ' + typeVarName + ';'); - res.endBlock('}'); - - res.needEmptyLine = true; - } - - if ($generatorCommon.domainQueryMetadata(domain) === 'Configuration' && - $generatorCommon.isDefinedAndNotEmpty(domain.fields)) { - res.line('/**'); - res.line(' * Create SQL Query descriptor for ' + type + '.'); - res.line(' *'); - res.line(' * @return Configured query entity.'); - res.line(' */'); - res.startBlock('private static QueryEntity queryEntity' + type + '() {'); - - $generatorJava.declareVariable(res, metaVarName, 'org.apache.ignite.cache.QueryEntity'); - - $generatorJava.classNameProperty(res, metaVarName, domain, 'keyType'); - $generatorJava.property(res, metaVarName, domain, 'valueType'); - - res.needEmptyLine = true; - - $generatorJava.domainModelQuery(domain, res); - - res.emptyLineIfNeeded(); - res.line('return ' + metaVarName + ';'); - - res.needEmptyLine = true; - - res.endBlock('}'); - } - - domains.push(domain.valueType); - } - }); - }); -}; - -/** - * @param prefix Variable prefix. - * @param obj Object to process. - * @param names Known names to generate next unique name. - */ -$generatorJava.nextVariableName = function(prefix, obj, names) { - let nextName = $generatorCommon.toJavaName(prefix, obj.name); - - let ix = 0; - - const checkNextName = (name) => name === nextName + (ix === 0 ? '' : '_' + ix); - - while (_.find(names, (name) => checkNextName(name))) - ix++; - - if (ix > 0) - nextName = nextName + '_' + ix; - - return nextName; -}; - -// Generate cluster caches. -$generatorJava.clusterCaches = function(caches, igfss, isSrvCfg, res) { - function clusterCache(cache, names) { - res.emptyLineIfNeeded(); - - const cacheName = $generatorJava.nextVariableName('cache', cache, names); - - $generatorJava.resetVariables(res); - - const hasDatasource = $generatorCommon.cacheHasDatasource(cache); - - res.line('/**'); - res.line(' * Create configuration for cache "' + cache.name + '".'); - res.line(' *'); - res.line(' * @return Configured cache.'); - - if (hasDatasource) - res.line(' * @throws Exception if failed to create cache configuration.'); - - res.line(' */'); - res.startBlock('public static CacheConfiguration ' + cacheName + '()' + (hasDatasource ? ' throws Exception' : '') + ' {'); - - $generatorJava.declareVariable(res, cacheName, 'org.apache.ignite.configuration.CacheConfiguration'); - - $generatorJava.cache(cache, cacheName, res); - - res.line('return ' + cacheName + ';'); - res.endBlock('}'); - - names.push(cacheName); - - res.needEmptyLine = true; - } - - if (!res) - res = $generatorCommon.builder(); - - const names = []; - - if ($generatorCommon.isDefinedAndNotEmpty(caches)) { - res.emptyLineIfNeeded(); - - _.forEach(caches, function(cache) { - clusterCache(cache, names); - }); - - res.needEmptyLine = true; - } - - if (isSrvCfg && $generatorCommon.isDefinedAndNotEmpty(igfss)) { - res.emptyLineIfNeeded(); - - _.forEach(igfss, function(igfs) { - clusterCache($generatorCommon.igfsDataCache(igfs), names); - clusterCache($generatorCommon.igfsMetaCache(igfs), names); - }); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate cluster caches. -$generatorJava.clusterCacheUse = function(caches, igfss, res) { - function clusterCacheInvoke(cache, names) { - names.push($generatorJava.nextVariableName('cache', cache, names)); - } - - if (!res) - res = $generatorCommon.builder(); - - const cacheNames = []; - - _.forEach(caches, function(cache) { - clusterCacheInvoke(cache, cacheNames); - }); - - const igfsNames = []; - - _.forEach(igfss, function(igfs) { - clusterCacheInvoke($generatorCommon.igfsDataCache(igfs), igfsNames); - clusterCacheInvoke($generatorCommon.igfsMetaCache(igfs), igfsNames); - }); - - const allCacheNames = cacheNames.concat(igfsNames); - - if (allCacheNames.length) { - res.line('cfg.setCacheConfiguration(' + allCacheNames.join('(), ') + '());'); - - res.needEmptyLine = true; - } - - return res; -}; - -// Get class name from fully specified class path. -$generatorJava.extractType = function(fullType) { - return fullType.substring(fullType.lastIndexOf('.') + 1); -}; - -/** - * Generate java class code. - * - * @param domain Domain model object. - * @param key If 'true' then key class should be generated. - * @param pkg Package name. - * @param useConstructor If 'true' then empty and full constructors should be generated. - * @param includeKeyFields If 'true' then include key fields into value POJO. - * @param res Resulting output with generated code. - */ -$generatorJava.javaClassCode = function(domain, key, pkg, useConstructor, includeKeyFields, res) { - if (!res) - res = $generatorCommon.builder(); - - const type = $generatorJava.extractType(key ? domain.keyType : domain.valueType); - - // Class comment. - res.line('/**'); - res.line(' * ' + type + ' definition.'); - res.line(' *'); - res.line(' * ' + $generatorCommon.mainComment('POJO')); - res.line(' */'); - - res.startBlock('public class ' + type + ' implements ' + res.importClass('java.io.Serializable') + ' {'); - - res.line('/** */'); - res.line('private static final long serialVersionUID = 0L;'); - res.needEmptyLine = true; - - const allFields = (key || includeKeyFields) ? domain.keyFields.slice() : []; - - if (!key) { - _.forEach(domain.valueFields, (valFld) => { - if (_.findIndex(allFields, (fld) => fld.javaFieldName === valFld.javaFieldName) < 0) - allFields.push(valFld); - }); - } - - // Generate allFields declaration. - _.forEach(allFields, function(field) { - const fldName = field.javaFieldName; - - res.line('/** Value for ' + fldName + '. */'); - - res.line('private ' + res.importClass(field.javaFieldType) + ' ' + fldName + ';'); - - res.needEmptyLine = true; - }); - - // Generate constructors. - if (useConstructor) { - res.line('/**'); - res.line(' * Empty constructor.'); - res.line(' */'); - res.startBlock('public ' + type + '() {'); - res.line('// No-op.'); - res.endBlock('}'); - - res.needEmptyLine = true; - - res.line('/**'); - res.line(' * Full constructor.'); - res.line(' */'); - res.startBlock('public ' + type + '('); - - _.forEach(allFields, function(field, idx) { - res.line(res.importClass(field.javaFieldType) + ' ' + field.javaFieldName + (idx < allFields.length - 1 ? ',' : '')); - }); - - res.endBlock(') {'); - - res.startBlock(); - - _.forEach(allFields, (field) => res.line('this.' + field.javaFieldName + ' = ' + field.javaFieldName + ';')); - - res.endBlock('}'); - - res.needEmptyLine = true; - } - - // Generate getters and setters methods. - _.forEach(allFields, function(field) { - const fldName = field.javaFieldName; - - const fldType = res.importClass(field.javaFieldType); - - res.line('/**'); - res.line(' * Gets ' + fldName + '.'); - res.line(' *'); - res.line(' * @return Value for ' + fldName + '.'); - res.line(' */'); - res.startBlock('public ' + fldType + ' ' + $generatorCommon.toJavaName('get', fldName) + '() {'); - res.line('return ' + fldName + ';'); - res.endBlock('}'); - - res.needEmptyLine = true; - - res.line('/**'); - res.line(' * Sets ' + fldName + '.'); - res.line(' *'); - res.line(' * @param ' + fldName + ' New value for ' + fldName + '.'); - res.line(' */'); - res.startBlock('public void ' + $generatorCommon.toJavaName('set', fldName) + '(' + fldType + ' ' + fldName + ') {'); - res.line('this.' + fldName + ' = ' + fldName + ';'); - res.endBlock('}'); - - res.needEmptyLine = true; - }); - - // Generate equals() method. - res.line('/** {@inheritDoc} */'); - res.startBlock('@Override public boolean equals(Object o) {'); - res.startBlock('if (this == o)'); - res.line('return true;'); - res.endBlock(); - res.append(''); - - res.startBlock('if (!(o instanceof ' + type + '))'); - res.line('return false;'); - res.endBlock(); - - res.needEmptyLine = true; - - res.line(type + ' that = (' + type + ')o;'); - - _.forEach(allFields, function(field) { - res.needEmptyLine = true; - - const javaName = field.javaFieldName; - const javaType = field.javaFieldType; - - if ($generatorCommon.JavaTypes.isJavaPrimitive(javaType)) { - if (javaType === 'float') - res.startBlock('if (Float.compare(' + javaName + ', that.' + javaName + ') != 0)'); - else if (javaType === 'double') - res.startBlock('if (Double.compare(' + javaName + ', that.' + javaName + ') != 0)'); - else - res.startBlock('if (' + javaName + ' != that.' + javaName + ')'); - } - else - res.startBlock('if (' + javaName + ' != null ? !' + javaName + '.equals(that.' + javaName + ') : that.' + javaName + ' != null)'); - - res.line('return false;'); - res.endBlock(); - }); - - res.needEmptyLine = true; - - res.line('return true;'); - res.endBlock('}'); - - res.needEmptyLine = true; - - // Generate hashCode() method. - res.line('/** {@inheritDoc} */'); - res.startBlock('@Override public int hashCode() {'); - - let first = true; - let tempVar = false; - - _.forEach(allFields, function(field) { - const javaName = field.javaFieldName; - const javaType = field.javaFieldType; - - if (!first) - res.needEmptyLine = true; - - if ($generatorCommon.JavaTypes.isJavaPrimitive(javaType)) { - if (javaType === 'boolean') - res.line(first ? 'int res = ' + javaName + ' ? 1 : 0;' : 'res = 31 * res + (' + javaName + ' ? 1 : 0);'); - else if (javaType === 'byte' || javaType === 'short') - res.line(first ? 'int res = (int)' + javaName + ';' : 'res = 31 * res + (int)' + javaName + ';'); - else if (javaType === 'int') - res.line(first ? 'int res = ' + javaName + ';' : 'res = 31 * res + ' + javaName + ';'); - else if (javaType === 'long') { - res.line(first - ? 'int res = (int)(' + javaName + ' ^ (' + javaName + ' >>> 32));' - : 'res = 31 * res + (int)(' + javaName + ' ^ (' + javaName + ' >>> 32));'); - } - else if (javaType === 'float') { - res.line(first - ? 'int res = ' + javaName + ' != +0.0f ? Float.floatToIntBits(' + javaName + ') : 0;' - : 'res = 31 * res + (' + javaName + ' != +0.0f ? Float.floatToIntBits(' + javaName + ') : 0);'); - } - else if (javaType === 'double') { - res.line((tempVar ? 'ig_hash_temp' : 'long ig_hash_temp') + ' = Double.doubleToLongBits(' + javaName + ');'); - - res.needEmptyLine = true; - - res.line(first - ? 'int res = (int)(ig_hash_temp ^ (ig_hash_temp >>> 32));' - : 'res = 31 * res + (int)(ig_hash_temp ^ (ig_hash_temp >>> 32));'); - - tempVar = true; - } - } - else { - res.line(first - ? 'int res = ' + javaName + ' != null ? ' + javaName + '.hashCode() : 0;' - : 'res = 31 * res + (' + javaName + ' != null ? ' + javaName + '.hashCode() : 0);'); - } - - first = false; - }); - - res.needEmptyLine = true; - res.line('return res;'); - res.endBlock('}'); - res.needEmptyLine = true; - - // Generate toString() method. - res.line('/** {@inheritDoc} */'); - res.startBlock('@Override public String toString() {'); - - res.startBlock('return \"' + type + ' [" + '); - - _.forEach(allFields, function(field, idx) { - res.line('\"' + field.javaFieldName + '=\" + ' + field.javaFieldName + (idx < allFields.length - 1 ? ' + ", " + ' : ' +')); - }); - - res.endBlock('"]";'); - res.endBlock('}'); - - res.endBlock('}'); - - return 'package ' + pkg + ';' + '\n\n' + res.generateImports() + '\n' + res.generateStaticImports() + '\n' + res.asString(); -}; - -/** - * Generate source code for type by its domain models. - * - * @param caches List of caches to generate POJOs for. - * @param useConstructor If 'true' then generate constructors. - * @param includeKeyFields If 'true' then include key fields into value POJO. - */ -$generatorJava.pojos = function(caches, useConstructor, includeKeyFields) { - const pojos = []; - - _.forEach(caches, (cache) => { - _.forEach(cache.domains, (domain) => { - // Process only domains with 'generatePojo' flag and skip already generated classes. - if (domain.generatePojo && !_.find(pojos, {valueType: domain.valueType}) && - // Skip domain models without value fields. - $generatorCommon.isDefinedAndNotEmpty(domain.valueFields)) { - const pojo = {}; - - // Key class generation only if key is not build in java class. - if (!_.isNil(domain.keyFields) && domain.keyFields.length > 0) { - pojo.keyType = domain.keyType; - pojo.keyClass = $generatorJava.javaClassCode(domain, true, - domain.keyType.substring(0, domain.keyType.lastIndexOf('.')), useConstructor, includeKeyFields); - } - - pojo.valueType = domain.valueType; - pojo.valueClass = $generatorJava.javaClassCode(domain, false, - domain.valueType.substring(0, domain.valueType.lastIndexOf('.')), useConstructor, includeKeyFields); - - pojos.push(pojo); - } - }); - }); - - return pojos; -}; - -/** - * @param type Full type name. - * @returns Field java type name. - */ -$generatorJava.javaTypeName = function(type) { - const ix = $generatorJava.javaBuiltInClasses.indexOf(type); - - const resType = ix >= 0 ? $generatorJava.javaBuiltInFullNameClasses[ix] : type; - - return resType.indexOf('java.lang.') >= 0 ? resType.substring(10) : resType; -}; - -/** - * Java code generator for cluster's SSL configuration. - * - * @param cluster Cluster to get SSL configuration. - * @param res Optional configuration presentation builder object. - * @returns Configuration presentation builder object - */ -$generatorJava.clusterSsl = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - if (cluster.sslEnabled && !_.isNil(cluster.sslContextFactory)) { - - cluster.sslContextFactory.keyStorePassword = $generatorCommon.isDefinedAndNotEmpty(cluster.sslContextFactory.keyStoreFilePath) ? - 'props.getProperty("ssl.key.storage.password").toCharArray()' : null; - - cluster.sslContextFactory.trustStorePassword = $generatorCommon.isDefinedAndNotEmpty(cluster.sslContextFactory.trustStoreFilePath) ? - 'props.getProperty("ssl.trust.storage.password").toCharArray()' : null; - - const propsDesc = $generatorCommon.isDefinedAndNotEmpty(cluster.sslContextFactory.trustManagers) ? - $generatorCommon.SSL_CONFIGURATION_TRUST_MANAGER_FACTORY.fields : - $generatorCommon.SSL_CONFIGURATION_TRUST_FILE_FACTORY.fields; - - $generatorJava.beanProperty(res, 'cfg', cluster.sslContextFactory, 'sslContextFactory', 'sslContextFactory', - 'org.apache.ignite.ssl.SslContextFactory', propsDesc, true); - - res.needEmptyLine = true; - } - - return res; -}; - -/** - * Java code generator for cluster's IGFS configurations. - * - * @param igfss List of configured IGFS. - * @param varName Name of IGFS configuration variable. - * @param res Optional configuration presentation builder object. - * @returns Configuration presentation builder object. - */ -$generatorJava.igfss = function(igfss, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.isDefinedAndNotEmpty(igfss)) { - res.emptyLineIfNeeded(); - - const arrayName = 'fileSystems'; - const igfsInst = 'igfs'; - - res.line(res.importClass('org.apache.ignite.configuration.FileSystemConfiguration') + '[] ' + arrayName + ' = new FileSystemConfiguration[' + igfss.length + '];'); - - _.forEach(igfss, function(igfs, ix) { - $generatorJava.declareVariable(res, igfsInst, 'org.apache.ignite.configuration.FileSystemConfiguration'); - - $generatorJava.igfsGeneral(igfs, igfsInst, res); - $generatorJava.igfsIPC(igfs, igfsInst, res); - $generatorJava.igfsFragmentizer(igfs, igfsInst, res); - $generatorJava.igfsDualMode(igfs, igfsInst, res); - $generatorJava.igfsSecondFS(igfs, igfsInst, res); - $generatorJava.igfsMisc(igfs, igfsInst, res); - - res.line(arrayName + '[' + ix + '] = ' + igfsInst + ';'); - - res.needEmptyLine = true; - }); - - res.line(varName + '.' + 'setFileSystemConfiguration(' + arrayName + ');'); - - res.needEmptyLine = true; - } - - return res; -}; - -/** - * Java code generator for IGFS IPC configuration. - * - * @param igfs Configured IGFS. - * @param varName Name of IGFS configuration variable. - * @param res Optional configuration presentation builder object. - * @returns Configuration presentation builder object. - */ -$generatorJava.igfsIPC = function(igfs, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!varName) - varName = $generatorJava.nextVariableName('igfs', igfs); - - if (igfs.ipcEndpointEnabled) { - const desc = $generatorCommon.IGFS_IPC_CONFIGURATION; - - $generatorJava.beanProperty(res, varName, igfs.ipcEndpointConfiguration, 'ipcEndpointConfiguration', 'ipcEndpointCfg', - desc.className, desc.fields, true); - - res.needEmptyLine = true; - } - - return res; -}; - -/** - * Java code generator for IGFS fragmentizer configuration. - * - * @param igfs Configured IGFS. - * @param varName Name of IGFS configuration variable. - * @param res Optional configuration presentation builder object. - * @returns Configuration presentation builder object. - */ -$generatorJava.igfsFragmentizer = function(igfs, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!varName) - varName = $generatorJava.nextVariableName('igfs', igfs); - - if (igfs.fragmentizerEnabled) { - $generatorJava.property(res, varName, igfs, 'fragmentizerConcurrentFiles', null, null, 0); - $generatorJava.property(res, varName, igfs, 'fragmentizerThrottlingBlockLength', null, null, 16777216); - $generatorJava.property(res, varName, igfs, 'fragmentizerThrottlingDelay', null, null, 200); - - res.needEmptyLine = true; - } - else - $generatorJava.property(res, varName, igfs, 'fragmentizerEnabled'); - - return res; -}; - -/** - * Java code generator for IGFS dual mode configuration. - * - * @param igfs Configured IGFS. - * @param varName Name of IGFS configuration variable. - * @param res Optional configuration presentation builder object. - * @returns Configuration presentation builder object. - */ -$generatorJava.igfsDualMode = function(igfs, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!varName) - varName = $generatorJava.nextVariableName('igfs', igfs); - - $generatorJava.property(res, varName, igfs, 'dualModeMaxPendingPutsSize', null, null, 0); - - if ($generatorCommon.isDefinedAndNotEmpty(igfs.dualModePutExecutorService)) - res.line(varName + '.' + $generatorJava.setterName('dualModePutExecutorService') + '(new ' + res.importClass(igfs.dualModePutExecutorService) + '());'); - - $generatorJava.property(res, varName, igfs, 'dualModePutExecutorServiceShutdown', null, null, false); - - res.needEmptyLine = true; - - return res; -}; - -$generatorJava.igfsSecondFS = function(igfs, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!varName) - varName = $generatorJava.nextVariableName('igfs', igfs); - - if (igfs.secondaryFileSystemEnabled) { - const secondFs = igfs.secondaryFileSystem || {}; - - const nameDefined = $generatorCommon.isDefinedAndNotEmpty(secondFs.userName); - const cfgDefined = $generatorCommon.isDefinedAndNotEmpty(secondFs.cfgPath); - - res.line(varName + '.setSecondaryFileSystem(new ' + - res.importClass('org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem') + '(' + - $generatorJava.constructorArg(secondFs, 'uri', null) + - (cfgDefined || nameDefined ? $generatorJava.constructorArg(secondFs, 'cfgPath', null, true) : '') + - $generatorJava.constructorArg(secondFs, 'userName', null, true, true) + - '));'); - - res.needEmptyLine = true; - } - - return res; -}; - -/** - * Java code generator for IGFS general configuration. - * - * @param igfs Configured IGFS. - * @param varName Name of IGFS configuration variable. - * @param res Optional configuration presentation builder object. - * @returns Configuration presentation builder object. - */ -$generatorJava.igfsGeneral = function(igfs, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!varName) - varName = $generatorJava.nextVariableName('igfs', igfs); - - if ($generatorCommon.isDefinedAndNotEmpty(igfs.name)) { - igfs.dataCacheName = $generatorCommon.igfsDataCache(igfs).name; - igfs.metaCacheName = $generatorCommon.igfsMetaCache(igfs).name; - - $generatorJava.property(res, varName, igfs, 'name'); - $generatorJava.property(res, varName, igfs, 'dataCacheName'); - $generatorJava.property(res, varName, igfs, 'metaCacheName'); - $generatorJava.enumProperty(res, varName, igfs, 'defaultMode', 'org.apache.ignite.igfs.IgfsMode', null, 'DUAL_ASYNC'); - - res.needEmptyLine = true; - } - - return res; -}; - -/** - * Java code generator for IGFS misc configuration. - * - * @param igfs Configured IGFS. - * @param varName Name of IGFS configuration variable. - * @param res Optional configuration presentation builder object. - * @returns Configuration presentation builder object. - */ -$generatorJava.igfsMisc = function(igfs, varName, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!varName) - varName = $generatorJava.nextVariableName('igfs', igfs); - - $generatorJava.property(res, varName, igfs, 'blockSize', null, null, 65536); - $generatorJava.property(res, varName, igfs, 'streamBufferSize', null, null, 65536); - $generatorJava.property(res, varName, igfs, 'maxSpaceSize', null, null, 0); - $generatorJava.property(res, varName, igfs, 'maximumTaskRangeLength', null, null, 0); - $generatorJava.property(res, varName, igfs, 'managementPort', null, null, 11400); - $generatorJava.property(res, varName, igfs, 'perNodeBatchSize', null, null, 100); - $generatorJava.property(res, varName, igfs, 'perNodeParallelBatchCount', null, null, 8); - $generatorJava.property(res, varName, igfs, 'prefetchBlocks', null, null, 0); - $generatorJava.property(res, varName, igfs, 'sequentialReadsBeforePrefetch', null, null, 0); - $generatorJava.property(res, varName, igfs, 'trashPurgeTimeout', null, null, 1000); - $generatorJava.property(res, varName, igfs, 'colocateMetadata', null, null, true); - $generatorJava.property(res, varName, igfs, 'relaxedConsistency', null, null, true); - - if (igfs.pathModes && igfs.pathModes.length > 0) { - res.needEmptyLine = true; - - $generatorJava.declareVariable(res, 'pathModes', 'java.util.Map', 'java.util.HashMap', 'String', 'org.apache.ignite.igfs.IgfsMode'); - - _.forEach(igfs.pathModes, function(pair) { - res.line('pathModes.put("' + pair.path + '", IgfsMode.' + pair.mode + ');'); - }); - - res.needEmptyLine = true; - - res.line(varName + '.setPathModes(pathModes);'); - } - - res.needEmptyLine = true; - - return res; -}; - -$generatorJava.clusterConfiguration = function(cluster, clientNearCfg, res) { - $generatorJava.clusterGeneral(cluster, clientNearCfg, res); - - $generatorJava.clusterAtomics(cluster.atomicConfiguration, res); - - $generatorJava.clusterBinary(cluster.binaryConfiguration, res); - - $generatorJava.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, res); - - $generatorJava.clusterCollision(cluster.collision, res); - - $generatorJava.clusterCommunication(cluster, res); - - $generatorJava.clusterConnector(cluster.connector, res); - - $generatorJava.clusterDeployment(cluster, res); - - $generatorJava.clusterEvents(cluster, res); - - $generatorJava.clusterFailover(cluster, res); - - $generatorJava.clusterLogger(cluster.logger, res); - - $generatorJava.clusterODBC(cluster.odbc, res); - - $generatorJava.clusterMarshaller(cluster, res); - - $generatorJava.clusterMetrics(cluster, res); - - $generatorJava.clusterSwap(cluster, res); - - $generatorJava.clusterTime(cluster, res); - - $generatorJava.clusterPools(cluster, res); - - $generatorJava.clusterTransactions(cluster.transactionConfiguration, res); - - const isSrvCfg = _.isNil(clientNearCfg); - - if (isSrvCfg) - $generatorJava.clusterCacheUse(cluster.caches, cluster.igfss, res); - - $generatorJava.clusterSsl(cluster, res); - - if (isSrvCfg) - $generatorJava.igfss(cluster.igfss, 'cfg', res); - - $generatorJava.clusterUserAttributes(cluster, res); - - return res; -}; - -// Generate loading of secret properties file. -$generatorJava.tryLoadSecretProperties = function(cluster, res) { - if ($generatorCommon.secretPropertiesNeeded(cluster)) { - res.importClass('org.apache.ignite.configuration.IgniteConfiguration'); - - $generatorJava.declareVariableCustom(res, 'props', 'java.util.Properties', 'new Properties()', 'private static final'); - - res.startBlock('static {'); - res.startBlock('try (' + res.importClass('java.io.InputStream') + ' in = IgniteConfiguration.class.getClassLoader().getResourceAsStream("secret.properties")) {'); - res.line('props.load(in);'); - res.endBlock('}'); - res.startBlock('catch (Exception ignored) {'); - res.line('// No-op.'); - res.endBlock('}'); - res.endBlock('}'); - - res.needEmptyLine = true; - } -}; - -/** - * Function to generate java code for cluster configuration. - * - * @param cluster Cluster to process. - * @param pkg Package name. - * @param javaClass Class name for generate factory class otherwise generate code snippet. - * @param clientNearCfg Optional near cache configuration for client node. - */ -$generatorJava.cluster = function(cluster, pkg, javaClass, clientNearCfg) { - const res = $generatorCommon.builder(); - - const isSrvCfg = _.isNil(clientNearCfg); - - if (cluster) { - const resCfg = $generatorJava.clusterConfiguration(cluster, clientNearCfg, $generatorCommon.builder()); - - res.mergeProps(resCfg); - - res.line('/**'); - res.line(' * ' + $generatorCommon.mainComment('configuration')); - res.line(' */'); - res.startBlock('public class ' + javaClass + ' {'); - - $generatorJava.tryLoadSecretProperties(cluster, res); - - $generatorJava.clusterDataSources(cluster, res); - - res.line('/**'); - res.line(' * Configure grid.'); - res.line(' *'); - res.line(' * @return Ignite configuration.'); - res.line(' * @throws Exception If failed to construct Ignite configuration instance.'); - res.line(' */'); - res.startBlock('public static IgniteConfiguration createConfiguration() throws Exception {'); - - res.mergeLines(resCfg); - - res.needEmptyLine = true; - - res.line('return cfg;'); - res.endBlock('}'); - - res.needEmptyLine = true; - - $generatorJava.clusterDomains(cluster.caches, res); - - $generatorJava.clusterCaches(cluster.caches, cluster.igfss, isSrvCfg, res); - - // TODO IGNITE-2269 Remove specified methods after implamentation of extended constructors. - $generatorJava.binaryTypeConfigurations(cluster.binaryConfiguration, res); - - res.needEmptyLine = true; - - if (clientNearCfg) { - res.line('/**'); - res.line(' * Configure client near cache configuration.'); - res.line(' *'); - res.line(' * @return Near cache configuration.'); - res.line(' * @throws Exception If failed to construct near cache configuration instance.'); - res.line(' */'); - res.startBlock('public static NearCacheConfiguration createNearCacheConfiguration() throws Exception {'); - - $generatorJava.resetVariables(res); - - $generatorJava.declareVariable(res, 'clientNearCfg', 'org.apache.ignite.configuration.NearCacheConfiguration'); - - if (clientNearCfg.nearStartSize) { - $generatorJava.property(res, 'clientNearCfg', clientNearCfg, 'nearStartSize'); - - res.needEmptyLine = true; - } - - if (clientNearCfg.nearEvictionPolicy && clientNearCfg.nearEvictionPolicy.kind) - $generatorJava.evictionPolicy(res, 'clientNearCfg', clientNearCfg.nearEvictionPolicy, 'nearEvictionPolicy'); - - res.line('return clientNearCfg;'); - res.endBlock('}'); - - res.needEmptyLine = true; - } - - res.endBlock('}'); - - return 'package ' + pkg + ';\n\n' + res.generateImports() + '\n' + res.generateStaticImports() + '\n' + res.asString(); - } - - return res.asString(); -}; - -/** Generate data source class name for specified store factory. - * - * @param res Optional configuration presentation builder object. - * @param storeFactory Store factory for data source class name generation. - * @returns {*} Data source class name. - */ -$generatorJava.dataSourceClassName = function(res, storeFactory) { - const dialect = storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : null) : storeFactory.dialect; - - if (dialect) { - const dataSourceBean = storeFactory.dataSourceBean; - - const dsClsName = $generatorCommon.dataSourceClassName(dialect); - - const varType = res.importClass(dsClsName); - - return $generatorCommon.toJavaName(varType, dataSourceBean); - } - - return null; -}; - -// Descriptors for generation of demo data. -const PREDEFINED_QUERIES = [ - { - schema: 'CARS', - type: 'PARKING', - create: 'CREATE TABLE IF NOT EXISTS CARS.PARKING (\n' + - 'ID INTEGER NOT NULL PRIMARY KEY,\n' + - 'NAME VARCHAR(50) NOT NULL,\n' + - 'CAPACITY INTEGER NOT NULL)', - clearQuery: 'DELETE FROM CARS.PARKING', - insertCntConsts: [{name: 'DEMO_MAX_PARKING_CNT', val: 5, comment: 'How many parkings to generate.'}], - insertPattern: ['INSERT INTO CARS.PARKING(ID, NAME, CAPACITY) VALUES(?, ?, ?)'], - fillInsertParameters(res) { - res.line('stmt.setInt(1, id);'); - res.line('stmt.setString(2, "Parking #" + (id + 1));'); - res.line('stmt.setInt(3, 10 + rnd.nextInt(20));'); - }, - selectQuery: ['SELECT * FROM PARKING WHERE CAPACITY >= 20'] - }, - { - schema: 'CARS', - type: 'CAR', - create: 'CREATE TABLE IF NOT EXISTS CARS.CAR (\n' + - 'ID INTEGER NOT NULL PRIMARY KEY,\n' + - 'PARKING_ID INTEGER NOT NULL,\n' + - 'NAME VARCHAR(50) NOT NULL);', - clearQuery: 'DELETE FROM CARS.CAR', - rndRequired: true, - insertCntConsts: [ - {name: 'DEMO_MAX_CAR_CNT', val: 10, comment: 'How many cars to generate.'}, - {name: 'DEMO_MAX_PARKING_CNT', val: 5, comment: 'How many parkings to generate.'} - ], - insertPattern: ['INSERT INTO CARS.CAR(ID, PARKING_ID, NAME) VALUES(?, ?, ?)'], - fillInsertParameters(res) { - res.line('stmt.setInt(1, id);'); - res.line('stmt.setInt(2, rnd.nextInt(DEMO_MAX_PARKING_CNT));'); - res.line('stmt.setString(3, "Car #" + (id + 1));'); - }, - selectQuery: ['SELECT * FROM CAR WHERE PARKINGID = 2'] - }, - { - type: 'COUNTRY', - create: 'CREATE TABLE IF NOT EXISTS COUNTRY (\n' + - 'ID INTEGER NOT NULL PRIMARY KEY,\n' + - 'NAME VARCHAR(50),\n' + - 'POPULATION INTEGER NOT NULL);', - clearQuery: 'DELETE FROM COUNTRY', - insertCntConsts: [{name: 'DEMO_MAX_COUNTRY_CNT', val: 5, comment: 'How many countries to generate.'}], - insertPattern: ['INSERT INTO COUNTRY(ID, NAME, POPULATION) VALUES(?, ?, ?)'], - fillInsertParameters(res) { - res.line('stmt.setInt(1, id);'); - res.line('stmt.setString(2, "Country #" + (id + 1));'); - res.line('stmt.setInt(3, 10000000 + rnd.nextInt(100000000));'); - }, - selectQuery: ['SELECT * FROM COUNTRY WHERE POPULATION BETWEEN 15000000 AND 25000000'] - }, - { - type: 'DEPARTMENT', - create: 'CREATE TABLE IF NOT EXISTS DEPARTMENT (\n' + - 'ID INTEGER NOT NULL PRIMARY KEY,\n' + - 'COUNTRY_ID INTEGER NOT NULL,\n' + - 'NAME VARCHAR(50) NOT NULL);', - clearQuery: 'DELETE FROM DEPARTMENT', - rndRequired: true, - insertCntConsts: [ - {name: 'DEMO_MAX_DEPARTMENT_CNT', val: 5, comment: 'How many departments to generate.'}, - {name: 'DEMO_MAX_COUNTRY_CNT', val: 5, comment: 'How many countries to generate.'} - ], - insertPattern: ['INSERT INTO DEPARTMENT(ID, COUNTRY_ID, NAME) VALUES(?, ?, ?)'], - fillInsertParameters(res) { - res.line('stmt.setInt(1, id);'); - res.line('stmt.setInt(2, rnd.nextInt(DEMO_MAX_COUNTRY_CNT));'); - res.line('stmt.setString(3, "Department #" + (id + 1));'); - }, - selectQuery: ['SELECT * FROM DEPARTMENT'] - }, - { - type: 'EMPLOYEE', - create: 'CREATE TABLE IF NOT EXISTS EMPLOYEE (\n' + - 'ID INTEGER NOT NULL PRIMARY KEY,\n' + - 'DEPARTMENT_ID INTEGER NOT NULL,\n' + - 'MANAGER_ID INTEGER,\n' + - 'FIRST_NAME VARCHAR(50) NOT NULL,\n' + - 'LAST_NAME VARCHAR(50) NOT NULL,\n' + - 'EMAIL VARCHAR(50) NOT NULL,\n' + - 'PHONE_NUMBER VARCHAR(50),\n' + - 'HIRE_DATE DATE NOT NULL,\n' + - 'JOB VARCHAR(50) NOT NULL,\n' + - 'SALARY DOUBLE);', - clearQuery: 'DELETE FROM EMPLOYEE', - rndRequired: true, - insertCntConsts: [ - {name: 'DEMO_MAX_EMPLOYEE_CNT', val: 10, comment: 'How many employees to generate.'}, - {name: 'DEMO_MAX_DEPARTMENT_CNT', val: 5, comment: 'How many departments to generate.'} - ], - specialGeneration(res, conVar) { - // $generatorJava.declareVariableCustom(res, 'stmt', 'java.sql.PreparedStatement', conVar + - // '.prepareStatement("INSERT INTO EMPLOYEE(ID, DEPARTMENT_ID, FIRST_NAME, LAST_NAME, EMAIL, PHONE_NUMBER, HIRE_DATE, JOB, SALARY) VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?)")'); - // - // res.startBlock('for (int id = 0; id < DEMO_MAX_DEPARTMENT_CNT; id ++) {'); - // res.line('stmt.setInt(1, id);'); - // res.line('stmt.setInt(2, id);'); - // res.line('stmt.setString(3, "First name manager #" + (id + 1));'); - // res.line('stmt.setString(4, "Last name manager#" + (id + 1));'); - // res.line('stmt.setString(5, "Email manager#" + (id + 1));'); - // res.line('stmt.setString(6, "Phone number manager#" + (id + 1));'); - // res.line('stmt.setString(7, "2014-01-01");'); - // res.line('stmt.setString(8, "Job manager #" + (id + 1));'); - // res.line('stmt.setDouble(9, 1000.0 + rnd.nextInt(500));'); - // - // res.needEmptyLine = true; - // - // res.line('stmt.executeUpdate();'); - // - // res.endBlock('}'); - // - // res.needEmptyLine = true; - - $generatorJava.declareVariableCustom(res, 'stmt', 'java.sql.PreparedStatement', conVar + - '.prepareStatement("INSERT INTO EMPLOYEE(ID, DEPARTMENT_ID, MANAGER_ID, FIRST_NAME, LAST_NAME, EMAIL, PHONE_NUMBER, HIRE_DATE, JOB, SALARY) VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?)")'); - - res.startBlock('for (int id = 0; id < DEMO_MAX_EMPLOYEE_CNT; id ++) {'); - - res.needEmptyLine = true; - - res.line('int depId = rnd.nextInt(DEMO_MAX_DEPARTMENT_CNT);'); - - res.line('stmt.setInt(1, DEMO_MAX_DEPARTMENT_CNT + id);'); - res.line('stmt.setInt(2, depId);'); - res.line('stmt.setInt(3, depId);'); - res.line('stmt.setString(4, "First name manager #" + (id + 1));'); - res.line('stmt.setString(5, "Last name manager#" + (id + 1));'); - res.line('stmt.setString(6, "Email manager#" + (id + 1));'); - res.line('stmt.setString(7, "Phone number manager#" + (id + 1));'); - res.line('stmt.setString(8, "2014-01-01");'); - res.line('stmt.setString(9, "Job manager #" + (id + 1));'); - res.line('stmt.setDouble(10, 600.0 + rnd.nextInt(300));'); - - res.needEmptyLine = true; - - res.line('stmt.executeUpdate();'); - - res.endBlock('}'); - - res.needEmptyLine = true; - }, - selectQuery: ['SELECT * FROM EMPLOYEE WHERE SALARY > 700'] - } -]; - -// Generate creation and execution of prepared statement. -function _prepareStatement(res, conVar, query, select) { - if (query) { - const lines = query.split('\n'); - - _.forEach(lines, function(line, ix) { - if (ix === 0) { - if (lines.length === 1) - res.line(conVar + '.prepareStatement("' + line + '").execute' + (select ? 'Query' : 'Update') + '();'); - else - res.startBlock(conVar + '.prepareStatement("' + line + '" +'); - } - else - res.line('"' + line + '"' + (ix === lines.length - 1 ? ').execute' + (select ? 'Query' : 'Update') + '();' : ' +')); - }); - - if (lines.length > 0) - res.needEmptyLine = true; - - if (lines.length > 1) - res.endBlock(); - } -} - -// Generate creation and execution of cache query. -function _multilineQuery(res, query, prefix, postfix) { - if (query) { - const lines = query.split('\n'); - - _.forEach(lines, function(line, ix) { - if (ix === 0) { - if (lines.length === 1) - res.line(prefix + '"' + line + '"' + postfix); - else - res.startBlock(prefix + '"' + line + '" +'); - } - else - res.line('"' + line + '"' + (ix === lines.length - 1 ? postfix : ' +')); - }); - - if (lines.length > 1) - res.endBlock(); - } -} - -/** - * Checks if cluster has demo types. - * - * @param cluster Cluster to check. - * @param demo Is demo enabled. - * @returns {boolean} True if cluster has caches with demo types. - */ -$generatorJava.isDemoConfigured = function(cluster, demo) { - return demo && - _.find(cluster.caches, (cache) => _.find(cache.domains, (domain) => _.find(PREDEFINED_QUERIES, - (desc) => domain.valueType.toUpperCase().endsWith(desc.type)))); -}; - -$generatorJava.generateDemo = function(cluster, res, factoryCls) { - const cachesWithDataSource = _.filter(cluster.caches, (cache) => { - if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) { - const storeFactory = cache.cacheStoreFactory[cache.cacheStoreFactory.kind]; - - return storeFactory.connectVia === 'DataSource' && storeFactory.dialect || - !storeFactory.connectVia && storeFactory.dialect; - } - - return false; - }); - - // Prepare array of cache and his demo domain model list. Every domain is contained only in first cache. - const demoTypes = _.filter(_.map(cachesWithDataSource, (cache, idx) => { - return { - cache, - domains: _.filter(cache.domains, (domain) => - $generatorCommon.isDefinedAndNotEmpty(domain.valueFields) && - !_.find(cachesWithDataSource, (checkCache, checkIx) => checkIx < idx && _.find(checkCache.domains, domain)) - ) - }; - }), (cache) => $generatorCommon.isDefinedAndNotEmpty(cache.domains)); - - if ($generatorCommon.isDefinedAndNotEmpty(demoTypes)) { - const typeByDs = {}; - - // Group domain modes by data source - _.forEach(demoTypes, function(type) { - const ds = type.cache.cacheStoreFactory[type.cache.cacheStoreFactory.kind].dataSourceBean; - - if (!typeByDs[ds]) - typeByDs[ds] = [type]; - else - typeByDs[ds].push(type); - }); - - let rndDefined = false; - - const generatedConsts = []; - - _.forEach(typeByDs, function(types) { - _.forEach(types, function(type) { - _.forEach(type.domains, function(domain) { - const desc = _.find(PREDEFINED_QUERIES, (d) => domain.valueType.toUpperCase().endsWith(d.type)); - - if (desc) { - if (!rndDefined && desc.rndRequired) { - res.line('/** Random generator for demo data. */'); - $generatorJava.declareVariableCustom(res, 'rnd', 'java.util.Random', 'new Random()', 'private static final'); - - rndDefined = true; - } - - _.forEach(desc.insertCntConsts, function(cnt) { - if (!_.includes(generatedConsts, cnt.name)) { - res.line('/** ' + cnt.comment + ' */'); - res.line('private static final int ' + cnt.name + ' = ' + cnt.val + ';'); - res.needEmptyLine = true; - - generatedConsts.push(cnt.name); - } - }); - } - }); - }); - }); - - res.needEmptyLine = true; - - // Generation of fill database method - res.line('/** Fill data for Demo. */'); - res.startBlock('private static void prepareDemoData() throws ' + res.importClass('java.sql.SQLException') + ' {'); - - _.forEach(typeByDs, function(types, ds) { - const conVar = ds + 'Con'; - - res.startBlock('try (' + res.importClass('java.sql.Connection') + ' ' + conVar + ' = ' + res.importClass(factoryCls) + '.DataSources.INSTANCE_' + ds + '.getConnection()) {'); - - _.forEach(types, function(type) { - _.forEach(type.domains, function(domain) { - const desc = _.find(PREDEFINED_QUERIES, (d) => domain.valueType.toUpperCase().endsWith(d.type)); - - if (desc) { - res.line('// Generate ' + desc.type + '.'); - - if (desc.schema) - _prepareStatement(res, conVar, 'CREATE SCHEMA IF NOT EXISTS ' + desc.schema); - - _prepareStatement(res, conVar, desc.create); - - _prepareStatement(res, conVar, desc.clearQuery); - - res.needEmptyLine = true; - - if (!desc.specialGeneration) { - $generatorJava.declareVariableCustom(res, 'stmt', 'java.sql.PreparedStatement', conVar + '.prepareStatement("' + desc.insertPattern + '")'); - - res.startBlock('for (int id = 0; id < ' + desc.insertCntConsts[0].name + '; id ++) {'); - - desc.fillInsertParameters(res); - - res.needEmptyLine = true; - - res.line('stmt.executeUpdate();'); - - res.endBlock('}'); - - res.needEmptyLine = true; - } - else - desc.specialGeneration(res, conVar); - - res.line(conVar + '.commit();'); - - res.needEmptyLine = true; - } - }); - }); - - res.endBlock('}'); - - res.needEmptyLine = true; - }); - - res.endBlock('}'); - - res.needEmptyLine = true; - - res.line('/** Print result table to console */'); - res.startBlock('private static void printResult(' + res.importClass('java.util.List') + '<' + res.importClass('javax.cache.Cache') + '.Entry> rows) {'); - res.startBlock('for (Cache.Entry row: rows) {'); - res.line('System.out.println(row);'); - res.endBlock('}'); - res.endBlock('}'); - res.needEmptyLine = true; - - // Generation of execute queries method. - res.line('/** Run demo. */'); - res.startBlock('private static void runDemo(Ignite ignite) throws SQLException {'); - - const getType = function(fullType) { - return fullType.substr(fullType.lastIndexOf('.') + 1); - }; - - const cacheLoaded = []; - let rowVariableDeclared = false; - - _.forEach(typeByDs, function(types, ds) { - const conVar = ds + 'Con'; - - res.startBlock('try (Connection ' + conVar + ' = ' + factoryCls + '.DataSources.INSTANCE_' + ds + '.getConnection()) {'); - - _.forEach(types, function(type) { - _.forEach(type.domains, function(domain) { - const desc = _.find(PREDEFINED_QUERIES, (d) => domain.valueType.toUpperCase().endsWith(d.type)); - - if (desc) { - _.forEach(desc.selectQuery, function(query) { - const cacheName = type.cache.name; - - if (!_.includes(cacheLoaded, cacheName)) { - res.line('ignite.cache("' + cacheName + '").loadCache(null);'); - - cacheLoaded.push(cacheName); - } - - _multilineQuery(res, query, (rowVariableDeclared ? 'rows' : 'List> rows') + - ' = ignite.cache("' + cacheName + '").query(new ' + res.importClass('org.apache.ignite.cache.query.SqlQuery') + - '<>("' + getType(domain.valueType) + '", ', ')).getAll();'); - - res.line('printResult(rows);'); - - rowVariableDeclared = true; - - res.needEmptyLine = true; - }); - } - }); - }); - - res.endBlock('}'); - - res.needEmptyLine = true; - }); - - res.endBlock('}'); - - res.needEmptyLine = true; - } - - return demoTypes; -}; - -/** - * Function to generate java class for node startup with cluster configuration. - * - * @param cluster Cluster to process. - * @param pkg Class package name. - * @param cls Class name. - * @param cfg Config. - * @param factoryCls Optional fully qualified class name of configuration factory. - * @param clientNearCfg Optional near cache configuration for client node. - */ -$generatorJava.nodeStartup = function(cluster, pkg, cls, cfg, factoryCls, clientNearCfg) { - const demo = cls === 'DemoStartup'; - - const res = $generatorCommon.builder(); - - res.line('/**'); - res.line(' * ' + $generatorCommon.mainComment('node start up')); - - if (demo) { - res.line(' *'); - res.line(' * To start demo configure data sources in secret.properties file.'); - res.line(' * For H2 database it should be like following:'); - res.line(' * dsH2.jdbc.url=jdbc:h2:tcp://localhost/mem:DemoDB;DB_CLOSE_DELAY=-1'); - res.line(' * dsH2.jdbc.username=sa'); - res.line(' * dsH2.jdbc.password='); - } - - res.line(' */'); - res.startBlock('public class ' + cls + ' {'); - - const demoTypes = demo ? $generatorJava.generateDemo(cluster, res, factoryCls) : null; - - res.line('/**'); - res.line(' * Start up node with specified configuration.'); - res.line(' *'); - res.line(' * @param args Command line arguments, none required.'); - res.line(' * @throws Exception If failed.'); - res.line(' */'); - - res.startBlock('public static void main(String[] args) throws Exception {'); - - if (demo) { - res.startBlock('try {'); - res.line('// Start H2 database server.'); - res.line(res.importClass('org.h2.tools.Server') + '.createTcpServer("-tcpDaemon").start();'); - res.endBlock('}'); - res.startBlock('catch (' + res.importClass('java.sql.SQLException') + ' ignore) {'); - res.line('// No-op.'); - res.endBlock('}'); - - res.needEmptyLine = true; - } - - if (factoryCls) - res.importClass(factoryCls); - - if (clientNearCfg || $generatorCommon.isDefinedAndNotEmpty(demoTypes)) { - res.line(res.importClass('org.apache.ignite.Ignite') + ' ignite = ' + - res.importClass('org.apache.ignite.Ignition') + '.start(' + cfg + ');'); - } - else - res.line(res.importClass('org.apache.ignite.Ignition') + '.start(' + cfg + ');'); - - if (clientNearCfg) { - res.needEmptyLine = true; - - if ($generatorCommon.isDefinedAndNotEmpty(cluster.caches)) { - res.line('// Demo of near cache creation on client node.'); - - const names = []; - - _.forEach(cluster.caches, function(cache) { - $generatorJava.nextVariableName('cache', cache, names); - - res.line('ignite.getOrCreateCache(' + res.importClass(factoryCls) + '.' + - $generatorJava.nextVariableName('cache', cache, names[names.length - 1]) + '(), ' + - res.importClass(factoryCls) + '.createNearCacheConfiguration());'); - }); - - res.needEmptyLine = true; - } - } - - if (demo) { - res.needEmptyLine = true; - - res.line('prepareDemoData();'); - - res.needEmptyLine = true; - - res.line('runDemo(ignite);'); - } - - res.endBlock('}'); - - res.endBlock('}'); - - return 'package ' + pkg + ';\n\n' + res.generateImports() + '\n' + res.generateStaticImports() + '\n' + res.asString(); -}; - -/** - * Function to generate java class for load caches. - * - * @param caches Caches to load. - * @param pkg Class package name. - * @param cls Class name. - * @param cfg Config. - */ -$generatorJava.loadCaches = function(caches, pkg, cls, cfg) { - const res = $generatorCommon.builder(); - - res.line('/**'); - res.line(' * ' + $generatorCommon.mainComment('utility')); - res.line(' */'); - res.startBlock('public class ' + cls + ' {'); - - res.line('/**'); - res.line(' *

          '); - res.line(' * Utility to load caches from database.'); - res.line(' *

          '); - res.line(' * How to use:'); - res.line(' *

            '); - res.line(' *
          • Start cluster.
          • '); - res.line(' *
          • Start this utility and wait while load complete.
          • '); - res.line(' *
          '); - res.line(' *'); - res.line(' * @param args Command line arguments, none required.'); - res.line(' * @throws Exception If failed.'); - res.line(' */'); - - res.startBlock('public static void main(String[] args) throws Exception {'); - - res.startBlock('try (' + res.importClass('org.apache.ignite.Ignite') + ' ignite = ' + - res.importClass('org.apache.ignite.Ignition') + '.start(' + cfg + ')) {'); - - res.line('System.out.println(">>> Loading caches...");'); - - res.needEmptyLine = true; - - _.forEach(caches, (cache) => { - res.line('System.out.println(">>> Loading cache: ' + cache.name + '");'); - res.line('ignite.cache("' + cache.name + '").loadCache(null);'); - - res.needEmptyLine = true; - }); - - res.needEmptyLine = true; - - res.line('System.out.println(">>> All caches loaded!");'); - - res.endBlock('}'); - - res.endBlock('}'); - - res.endBlock('}'); - - return 'package ' + pkg + ';\n\n' + res.generateImports() + '\n' + res.generateStaticImports() + '\n' + res.asString(); -}; - - -export default $generatorJava; diff --git a/modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js deleted file mode 100644 index f70c66f5c91e1..0000000000000 --- a/modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js +++ /dev/null @@ -1,2111 +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. - */ - -// XML generation entry point. -const $generatorSpring = {}; - -// Do XML escape. -$generatorSpring.escape = function(s) { - if (typeof (s) !== 'string') - return s; - - return s.replace(/&/g, '&').replace(//g, '>').replace(/"/g, '"'); -}; - -// Add constructor argument -$generatorSpring.constructorArg = function(res, ix, obj, propName, dflt, opt) { - const v = (obj ? obj[propName] : null) || dflt; - - if ($generatorCommon.isDefinedAndNotEmpty(v)) - res.line('= 0 ? 'index="' + ix + '" ' : '') + 'value="' + v + '"/>'); - else if (!opt) { - res.startBlock('= 0 ? 'index="' + ix + '"' : '') + '>'); - res.line(''); - res.endBlock(''); - } -}; - -// Add XML element. -$generatorSpring.element = function(res, tag, attr1, val1, attr2, val2) { - let elem = '<' + tag; - - if (attr1) - elem += ' ' + attr1 + '="' + val1 + '"'; - - if (attr2) - elem += ' ' + attr2 + '="' + val2 + '"'; - - elem += '/>'; - - res.emptyLineIfNeeded(); - res.line(elem); -}; - -// Add property. -$generatorSpring.property = function(res, obj, propName, setterName, dflt) { - if (!_.isNil(obj)) { - const val = obj[propName]; - - if ($generatorCommon.isDefinedAndNotEmpty(val)) { - const missDflt = _.isNil(dflt); - - // Add to result if no default provided or value not equals to default. - if (missDflt || (!missDflt && val !== dflt)) { - $generatorSpring.element(res, 'property', 'name', setterName ? setterName : propName, 'value', $generatorSpring.escape(val)); - - return true; - } - } - } - - return false; -}; - -// Add property for class name. -$generatorSpring.classNameProperty = function(res, obj, propName) { - const val = obj[propName]; - - if (!_.isNil(val)) - $generatorSpring.element(res, 'property', 'name', propName, 'value', $generatorCommon.JavaTypes.fullClassName(val)); -}; - -// Add list property. -$generatorSpring.listProperty = function(res, obj, propName, listType, rowFactory) { - const val = obj[propName]; - - if (val && val.length > 0) { - res.emptyLineIfNeeded(); - - if (!listType) - listType = 'list'; - - if (!rowFactory) - rowFactory = (v) => '' + $generatorSpring.escape(v) + ''; - - res.startBlock(''); - res.startBlock('<' + listType + '>'); - - _.forEach(val, (v) => res.line(rowFactory(v))); - - res.endBlock(''); - res.endBlock(''); - - res.needEmptyLine = true; - } -}; - -// Add array property -$generatorSpring.arrayProperty = function(res, obj, propName, descr, rowFactory) { - const val = obj[propName]; - - if (val && val.length > 0) { - res.emptyLineIfNeeded(); - - if (!rowFactory) - rowFactory = (v) => ''; - - res.startBlock(''); - res.startBlock(''); - - _.forEach(val, (v) => res.append(rowFactory(v))); - - res.endBlock(''); - res.endBlock(''); - } -}; - -/** - * Add bean property with internal content. - * - * @param res Optional configuration presentation builder object. - * @param bean Bean object for code generation. - * @param beanPropName Name of property to set generated bean as value. - * @param desc Bean metadata object. - * @param createBeanAlthoughNoProps Always generate bean even it has no properties defined. - */ -$generatorSpring.beanProperty = function(res, bean, beanPropName, desc, createBeanAlthoughNoProps) { - const props = desc.fields; - - if (bean && $generatorCommon.hasProperty(bean, props)) { - if (!createBeanAlthoughNoProps) - res.startSafeBlock(); - - res.emptyLineIfNeeded(); - res.startBlock(''); - - if (createBeanAlthoughNoProps) - res.startSafeBlock(); - - res.startBlock(''); - - let hasData = false; - - _.forIn(props, function(descr, propName) { - if (props.hasOwnProperty(propName)) { - if (descr) { - switch (descr.type) { - case 'list': - $generatorSpring.listProperty(res, bean, propName, descr.setterName); - - break; - - case 'array': - $generatorSpring.arrayProperty(res, bean, propName, descr); - - break; - - case 'propertiesAsList': - const val = bean[propName]; - - if (val && val.length > 0) { - res.startBlock(''); - res.startBlock(''); - - _.forEach(val, function(nameAndValue) { - const eqIndex = nameAndValue.indexOf('='); - if (eqIndex >= 0) { - res.line('' + - $generatorSpring.escape(nameAndValue.substr(eqIndex + 1)) + ''); - } - }); - - res.endBlock(''); - res.endBlock(''); - - hasData = true; - } - - break; - - case 'bean': - if ($generatorCommon.isDefinedAndNotEmpty(bean[propName])) { - res.startBlock(''); - res.line(''); - res.endBlock(''); - - hasData = true; - } - - break; - - default: - if ($generatorSpring.property(res, bean, propName, descr.setterName, descr.dflt)) - hasData = true; - } - } - else - if ($generatorSpring.property(res, bean, propName)) - hasData = true; - } - }); - - res.endBlock(''); - - if (createBeanAlthoughNoProps && !hasData) { - res.rollbackSafeBlock(); - - res.line(''); - } - - res.endBlock(''); - - if (!createBeanAlthoughNoProps && !hasData) - res.rollbackSafeBlock(); - } - else if (createBeanAlthoughNoProps) { - res.emptyLineIfNeeded(); - res.startBlock(''); - res.line(''); - res.endBlock(''); - } -}; - -/** - * Add bean property without internal content. - * - * @param res Optional configuration presentation builder object. - * @param obj Object to take bean class name. - * @param propName Property name. - */ -$generatorSpring.simpleBeanProperty = function(res, obj, propName) { - if (!_.isNil(obj)) { - const val = obj[propName]; - - if ($generatorCommon.isDefinedAndNotEmpty(val)) { - res.startBlock(''); - res.line(''); - res.endBlock(''); - } - } - - return false; -}; - -// Generate eviction policy. -$generatorSpring.evictionPolicy = function(res, evtPlc, propName) { - if (evtPlc && evtPlc.kind) { - $generatorSpring.beanProperty(res, evtPlc[evtPlc.kind.toUpperCase()], propName, - $generatorCommon.EVICTION_POLICIES[evtPlc.kind], true); - } -}; - -// Generate discovery. -$generatorSpring.clusterGeneral = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.property(res, cluster, 'name', 'gridName'); - $generatorSpring.property(res, cluster, 'localHost'); - - if (cluster.discovery) { - res.startBlock(''); - res.startBlock(''); - res.startBlock(''); - - const d = cluster.discovery; - - switch (d.kind) { - case 'Multicast': - res.startBlock(''); - - if (d.Multicast) { - $generatorSpring.property(res, d.Multicast, 'multicastGroup'); - $generatorSpring.property(res, d.Multicast, 'multicastPort'); - $generatorSpring.property(res, d.Multicast, 'responseWaitTime'); - $generatorSpring.property(res, d.Multicast, 'addressRequestAttempts'); - $generatorSpring.property(res, d.Multicast, 'localAddress'); - $generatorSpring.listProperty(res, d.Multicast, 'addresses'); - } - - res.endBlock(''); - - break; - - case 'Vm': - res.startBlock(''); - - if (d.Vm) - $generatorSpring.listProperty(res, d.Vm, 'addresses'); - - res.endBlock(''); - - break; - - case 'S3': - res.startBlock(''); - - if (d.S3) { - if (d.S3.bucketName) - res.line(''); - } - - res.endBlock(''); - - break; - - case 'Cloud': - res.startBlock(''); - - if (d.Cloud) { - $generatorSpring.property(res, d.Cloud, 'credential'); - $generatorSpring.property(res, d.Cloud, 'credentialPath'); - $generatorSpring.property(res, d.Cloud, 'identity'); - $generatorSpring.property(res, d.Cloud, 'provider'); - $generatorSpring.listProperty(res, d.Cloud, 'regions'); - $generatorSpring.listProperty(res, d.Cloud, 'zones'); - } - - res.endBlock(''); - - break; - - case 'GoogleStorage': - res.startBlock(''); - - if (d.GoogleStorage) { - $generatorSpring.property(res, d.GoogleStorage, 'projectName'); - $generatorSpring.property(res, d.GoogleStorage, 'bucketName'); - $generatorSpring.property(res, d.GoogleStorage, 'serviceAccountP12FilePath'); - $generatorSpring.property(res, d.GoogleStorage, 'serviceAccountId'); - } - - res.endBlock(''); - - break; - - case 'Jdbc': - res.startBlock(''); - - if (d.Jdbc) { - const datasource = d.Jdbc; - - res.line(''); - - if (datasource.dataSourceBean && datasource.dialect) { - res.line(''); - - if (!_.find(res.datasources, { dataSourceBean: datasource.dataSourceBean })) { - res.datasources.push({ - dataSourceBean: datasource.dataSourceBean, - dialect: datasource.dialect - }); - } - } - } - - res.endBlock(''); - - break; - - case 'SharedFs': - res.startBlock(''); - - if (d.SharedFs) - $generatorSpring.property(res, d.SharedFs, 'path'); - - res.endBlock(''); - - break; - - case 'ZooKeeper': - res.startBlock(''); - - if (d.ZooKeeper) { - if ($generatorCommon.isDefinedAndNotEmpty(d.ZooKeeper.curator)) { - res.startBlock(''); - res.line(''); - res.endBlock(''); - } - - $generatorSpring.property(res, d.ZooKeeper, 'zkConnectionString'); - - if (d.ZooKeeper.retryPolicy && d.ZooKeeper.retryPolicy.kind) { - const kind = d.ZooKeeper.retryPolicy.kind; - const retryPolicy = d.ZooKeeper.retryPolicy[kind]; - const customClassDefined = retryPolicy && $generatorCommon.isDefinedAndNotEmpty(retryPolicy.className); - - if (kind !== 'Custom' || customClassDefined) - res.startBlock(''); - - switch (kind) { - case 'ExponentialBackoff': - res.startBlock(''); - $generatorSpring.constructorArg(res, 0, retryPolicy, 'baseSleepTimeMs', 1000); - $generatorSpring.constructorArg(res, 1, retryPolicy, 'maxRetries', 10); - $generatorSpring.constructorArg(res, 2, retryPolicy, 'maxSleepMs', null, true); - res.endBlock(''); - - break; - - case 'BoundedExponentialBackoff': - res.startBlock(''); - $generatorSpring.constructorArg(res, 0, retryPolicy, 'baseSleepTimeMs', 1000); - $generatorSpring.constructorArg(res, 1, retryPolicy, 'maxSleepTimeMs', 2147483647); - $generatorSpring.constructorArg(res, 2, retryPolicy, 'maxRetries', 10); - res.endBlock(''); - - break; - - case 'UntilElapsed': - res.startBlock(''); - $generatorSpring.constructorArg(res, 0, retryPolicy, 'maxElapsedTimeMs', 60000); - $generatorSpring.constructorArg(res, 1, retryPolicy, 'sleepMsBetweenRetries', 1000); - res.endBlock(''); - - break; - - case 'NTimes': - res.startBlock(''); - $generatorSpring.constructorArg(res, 0, retryPolicy, 'n', 10); - $generatorSpring.constructorArg(res, 1, retryPolicy, 'sleepMsBetweenRetries', 1000); - res.endBlock(''); - - break; - - case 'OneTime': - res.startBlock(''); - $generatorSpring.constructorArg(res, 0, retryPolicy, 'sleepMsBetweenRetry', 1000); - res.endBlock(''); - - break; - - case 'Forever': - res.startBlock(''); - $generatorSpring.constructorArg(res, 0, retryPolicy, 'retryIntervalMs', 1000); - res.endBlock(''); - - break; - - case 'Custom': - if (customClassDefined) - res.line(''); - - break; - - default: - } - - if (kind !== 'Custom' || customClassDefined) - res.endBlock(''); - } - - $generatorSpring.property(res, d.ZooKeeper, 'basePath', null, '/services'); - $generatorSpring.property(res, d.ZooKeeper, 'serviceName', null, 'ignite'); - $generatorSpring.property(res, d.ZooKeeper, 'allowDuplicateRegistrations', null, false); - } - - res.endBlock(''); - - break; - - default: - res.line('Unknown discovery kind: ' + d.kind); - } - - res.endBlock(''); - - $generatorSpring.clusterDiscovery(d, res); - - res.endBlock(''); - res.endBlock(''); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate atomics group. -$generatorSpring.clusterAtomics = function(atomics, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.hasAtLeastOneProperty(atomics, ['cacheMode', 'atomicSequenceReserveSize', 'backups'])) { - res.startSafeBlock(); - - res.emptyLineIfNeeded(); - - res.startBlock(''); - res.startBlock(''); - - const cacheMode = atomics.cacheMode ? atomics.cacheMode : 'PARTITIONED'; - - let hasData = cacheMode !== 'PARTITIONED'; - - $generatorSpring.property(res, atomics, 'cacheMode', null, 'PARTITIONED'); - - hasData = $generatorSpring.property(res, atomics, 'atomicSequenceReserveSize', null, 1000) || hasData; - - if (cacheMode === 'PARTITIONED') - hasData = $generatorSpring.property(res, atomics, 'backups', null, 0) || hasData; - - res.endBlock(''); - res.endBlock(''); - - res.needEmptyLine = true; - - if (!hasData) - res.rollbackSafeBlock(); - } - - return res; -}; - -// Generate binary group. -$generatorSpring.clusterBinary = function(binary, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.binaryIsDefined(binary)) { - res.startBlock(''); - res.startBlock(''); - - $generatorSpring.simpleBeanProperty(res, binary, 'idMapper'); - $generatorSpring.simpleBeanProperty(res, binary, 'nameMapper'); - $generatorSpring.simpleBeanProperty(res, binary, 'serializer'); - - if ($generatorCommon.isDefinedAndNotEmpty(binary.typeConfigurations)) { - res.startBlock(''); - res.startBlock(''); - - _.forEach(binary.typeConfigurations, function(type) { - res.startBlock(''); - - $generatorSpring.property(res, type, 'typeName'); - $generatorSpring.simpleBeanProperty(res, type, 'idMapper'); - $generatorSpring.simpleBeanProperty(res, type, 'nameMapper'); - $generatorSpring.simpleBeanProperty(res, type, 'serializer'); - $generatorSpring.property(res, type, 'enum', null, false); - - res.endBlock(''); - }); - - res.endBlock(''); - res.endBlock(''); - } - - $generatorSpring.property(res, binary, 'compactFooter', null, true); - - res.endBlock(''); - res.endBlock(''); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate cache key configurations. -$generatorSpring.clusterCacheKeyConfiguration = function(keyCfgs, res) { - if (!res) - res = $generatorCommon.builder(); - - keyCfgs = _.filter(keyCfgs, (cfg) => cfg.typeName && cfg.affinityKeyFieldName); - - if (_.isEmpty(keyCfgs)) - return res; - - res.startBlock(''); - res.startBlock(''); - - _.forEach(keyCfgs, (cfg) => { - res.startBlock(''); - - $generatorSpring.constructorArg(res, -1, cfg, 'typeName'); - $generatorSpring.constructorArg(res, -1, cfg, 'affinityKeyFieldName'); - - res.endBlock(''); - }); - - res.endBlock(''); - res.endBlock(''); - - return res; -}; - -// Generate collision group. -$generatorSpring.clusterCollision = function(collision, res) { - if (!res) - res = $generatorCommon.builder(); - - if (collision && collision.kind && collision.kind !== 'Noop') { - const spi = collision[collision.kind]; - - if (collision.kind !== 'Custom' || (spi && $generatorCommon.isDefinedAndNotEmpty(spi.class))) { - res.startBlock(''); - - switch (collision.kind) { - case 'JobStealing': - res.startBlock(''); - $generatorSpring.property(res, spi, 'activeJobsThreshold', null, 95); - $generatorSpring.property(res, spi, 'waitJobsThreshold', null, 0); - $generatorSpring.property(res, spi, 'messageExpireTime', null, 1000); - $generatorSpring.property(res, spi, 'maximumStealingAttempts', null, 5); - $generatorSpring.property(res, spi, 'stealingEnabled', null, true); - - if ($generatorCommon.isDefinedAndNotEmpty(spi.externalCollisionListener)) { - res.needEmptyLine = true; - - res.startBlock(''); - res.line(''); - res.endBlock(''); - } - - if ($generatorCommon.isDefinedAndNotEmpty(spi.stealingAttributes)) { - res.needEmptyLine = true; - - res.startBlock(''); - res.startBlock(''); - - _.forEach(spi.stealingAttributes, function(attr) { - $generatorSpring.element(res, 'entry', 'key', attr.name, 'value', attr.value); - }); - - res.endBlock(''); - res.endBlock(''); - } - - res.endBlock(''); - - break; - - case 'FifoQueue': - res.startBlock(''); - $generatorSpring.property(res, spi, 'parallelJobsNumber'); - $generatorSpring.property(res, spi, 'waitingJobsNumber'); - res.endBlock(''); - - break; - - case 'PriorityQueue': - res.startBlock(''); - $generatorSpring.property(res, spi, 'parallelJobsNumber'); - $generatorSpring.property(res, spi, 'waitingJobsNumber'); - $generatorSpring.property(res, spi, 'priorityAttributeKey', null, 'grid.task.priority'); - $generatorSpring.property(res, spi, 'jobPriorityAttributeKey', null, 'grid.job.priority'); - $generatorSpring.property(res, spi, 'defaultPriority', null, 0); - $generatorSpring.property(res, spi, 'starvationIncrement', null, 1); - $generatorSpring.property(res, spi, 'starvationPreventionEnabled', null, true); - res.endBlock(''); - - break; - - case 'Custom': - res.line(''); - - break; - - default: - } - - res.endBlock(''); - } - } - - return res; -}; - -// Generate communication group. -$generatorSpring.clusterCommunication = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.beanProperty(res, cluster.communication, 'communicationSpi', $generatorCommon.COMMUNICATION_CONFIGURATION); - - $generatorSpring.property(res, cluster, 'networkTimeout', null, 5000); - $generatorSpring.property(res, cluster, 'networkSendRetryDelay', null, 1000); - $generatorSpring.property(res, cluster, 'networkSendRetryCount', null, 3); - $generatorSpring.property(res, cluster, 'segmentCheckFrequency'); - $generatorSpring.property(res, cluster, 'waitForSegmentOnStart', null, false); - $generatorSpring.property(res, cluster, 'discoveryStartupDelay', null, 60000); - - res.needEmptyLine = true; - - return res; -}; - -/** - * XML generator for cluster's REST access configuration. - * - * @param connector Cluster REST connector configuration. - * @param res Optional configuration presentation builder object. - * @returns Configuration presentation builder object - */ -$generatorSpring.clusterConnector = function(connector, res) { - if (!res) - res = $generatorCommon.builder(); - - if (!_.isNil(connector) && connector.enabled) { - const cfg = _.cloneDeep($generatorCommon.CONNECTOR_CONFIGURATION); - - if (connector.sslEnabled) { - cfg.fields.sslClientAuth = {dflt: false}; - cfg.fields.sslFactory = {type: 'bean'}; - } - - $generatorSpring.beanProperty(res, connector, 'connectorConfiguration', cfg, true); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate deployment group. -$generatorSpring.clusterDeployment = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorSpring.property(res, cluster, 'deploymentMode', null, 'SHARED')) - res.needEmptyLine = true; - - const p2pEnabled = cluster.peerClassLoadingEnabled; - - if (!_.isNil(p2pEnabled)) { - $generatorSpring.property(res, cluster, 'peerClassLoadingEnabled', null, false); - - if (p2pEnabled) { - $generatorSpring.property(res, cluster, 'peerClassLoadingMissedResourcesCacheSize', null, 100); - $generatorSpring.property(res, cluster, 'peerClassLoadingThreadPoolSize', null, 2); - $generatorSpring.listProperty(res, cluster, 'peerClassLoadingLocalClassPathExclude'); - } - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate discovery group. -$generatorSpring.clusterDiscovery = function(disco, res) { - if (!res) - res = $generatorCommon.builder(); - - if (disco) { - $generatorSpring.property(res, disco, 'localAddress'); - $generatorSpring.property(res, disco, 'localPort', null, 47500); - $generatorSpring.property(res, disco, 'localPortRange', null, 100); - if ($generatorCommon.isDefinedAndNotEmpty(disco.addressResolver)) - $generatorSpring.beanProperty(res, disco, 'addressResolver', {className: disco.addressResolver}, true); - $generatorSpring.property(res, disco, 'socketTimeout', null, 5000); - $generatorSpring.property(res, disco, 'ackTimeout', null, 5000); - $generatorSpring.property(res, disco, 'maxAckTimeout', null, 600000); - $generatorSpring.property(res, disco, 'networkTimeout', null, 5000); - $generatorSpring.property(res, disco, 'joinTimeout', null, 0); - $generatorSpring.property(res, disco, 'threadPriority', null, 10); - $generatorSpring.property(res, disco, 'heartbeatFrequency', null, 2000); - $generatorSpring.property(res, disco, 'maxMissedHeartbeats', null, 1); - $generatorSpring.property(res, disco, 'maxMissedClientHeartbeats', null, 5); - $generatorSpring.property(res, disco, 'topHistorySize', null, 1000); - if ($generatorCommon.isDefinedAndNotEmpty(disco.listener)) - $generatorSpring.beanProperty(res, disco, 'listener', {className: disco.listener}, true); - if ($generatorCommon.isDefinedAndNotEmpty(disco.dataExchange)) - $generatorSpring.beanProperty(res, disco, 'dataExchange', {className: disco.dataExchange}, true); - if ($generatorCommon.isDefinedAndNotEmpty(disco.metricsProvider)) - $generatorSpring.beanProperty(res, disco, 'metricsProvider', {className: disco.metricsProvider}, true); - $generatorSpring.property(res, disco, 'reconnectCount', null, 10); - $generatorSpring.property(res, disco, 'statisticsPrintFrequency', null, 0); - $generatorSpring.property(res, disco, 'ipFinderCleanFrequency', null, 60000); - if ($generatorCommon.isDefinedAndNotEmpty(disco.authenticator)) - $generatorSpring.beanProperty(res, disco, 'authenticator', {className: disco.authenticator}, true); - $generatorSpring.property(res, disco, 'forceServerMode', null, false); - $generatorSpring.property(res, disco, 'clientReconnectDisabled', null, false); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate events group. -$generatorSpring.clusterEvents = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - if (cluster.includeEventTypes && cluster.includeEventTypes.length > 0) { - res.emptyLineIfNeeded(); - - res.startBlock(''); - - const evtGrps = angular.element(document.getElementById('app')).injector().get('igniteEventGroups'); - - if (cluster.includeEventTypes.length === 1) { - const evtGrp = _.find(evtGrps, {value: cluster.includeEventTypes[0]}); - - if (evtGrp) - res.line(''); - } - else { - res.startBlock(''); - - _.forEach(cluster.includeEventTypes, (item, ix) => { - ix > 0 && res.line(); - - const evtGrp = _.find(evtGrps, {value: item}); - - if (evtGrp) { - res.line(''); - - _.forEach(evtGrp.events, (event) => res.line('')); - } - }); - - res.endBlock(''); - } - - res.endBlock(''); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate failover group. -$generatorSpring.clusterFailover = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.isDefinedAndNotEmpty(cluster.failoverSpi) && _.findIndex(cluster.failoverSpi, function(spi) { - return $generatorCommon.isDefinedAndNotEmpty(spi.kind) && (spi.kind !== 'Custom' || $generatorCommon.isDefinedAndNotEmpty(_.get(spi, spi.kind + '.class'))); - }) >= 0) { - res.startBlock(''); - res.startBlock(''); - - _.forEach(cluster.failoverSpi, function(spi) { - if (spi.kind && (spi.kind !== 'Custom' || $generatorCommon.isDefinedAndNotEmpty(_.get(spi, spi.kind + '.class')))) { - const maxAttempts = _.get(spi, spi.kind + '.maximumFailoverAttempts'); - - if ((spi.kind === 'JobStealing' || spi.kind === 'Always') && $generatorCommon.isDefinedAndNotEmpty(maxAttempts) && maxAttempts !== 5) { - res.startBlock(''); - - $generatorSpring.property(res, spi[spi.kind], 'maximumFailoverAttempts', null, 5); - - res.endBlock(''); - } - else - res.line(''); - - res.needEmptyLine = true; - } - }); - - res.needEmptyLine = true; - - res.endBlock(''); - res.endBlock(''); - } - - return res; -}; - -// Generate marshaller group. -$generatorSpring.clusterLogger = function(logger, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.loggerConfigured(logger)) { - res.startBlock(''); - - const log = logger[logger.kind]; - - switch (logger.kind) { - case 'Log4j2': - res.startBlock(''); - res.line(''); - $generatorSpring.property(res, log, 'level'); - res.endBlock(''); - - break; - - case 'Null': - res.line(''); - - break; - - case 'Java': - res.line(''); - - break; - - case 'JCL': - res.line(''); - - break; - - case 'SLF4J': - res.line(''); - - break; - - case 'Log4j': - if (log.mode === 'Default' && !$generatorCommon.isDefinedAndNotEmpty(log.level)) - res.line(''); - else { - res.startBlock(''); - - if (log.mode === 'Path') - res.line(''); - - $generatorSpring.property(res, log, 'level'); - res.endBlock(''); - } - - break; - - case 'Custom': - res.line(''); - - break; - - default: - } - - res.endBlock(''); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate marshaller group. -$generatorSpring.clusterMarshaller = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - const marshaller = cluster.marshaller; - - if (marshaller && marshaller.kind) - $generatorSpring.beanProperty(res, marshaller[marshaller.kind], 'marshaller', $generatorCommon.MARSHALLERS[marshaller.kind], true); - - res.softEmptyLine(); - - $generatorSpring.property(res, cluster, 'marshalLocalJobs', null, false); - $generatorSpring.property(res, cluster, 'marshallerCacheKeepAliveTime', null, 10000); - $generatorSpring.property(res, cluster, 'marshallerCacheThreadPoolSize', 'marshallerCachePoolSize'); - - res.needEmptyLine = true; - - return res; -}; - -// Generate metrics group. -$generatorSpring.clusterMetrics = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.property(res, cluster, 'metricsExpireTime'); - $generatorSpring.property(res, cluster, 'metricsHistorySize', null, 10000); - $generatorSpring.property(res, cluster, 'metricsLogFrequency', null, 60000); - $generatorSpring.property(res, cluster, 'metricsUpdateFrequency', null, 2000); - - res.needEmptyLine = true; - - return res; -}; - -// Generate swap group. -$generatorSpring.clusterSwap = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - if (cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind === 'FileSwapSpaceSpi') { - $generatorSpring.beanProperty(res, cluster.swapSpaceSpi.FileSwapSpaceSpi, 'swapSpaceSpi', - $generatorCommon.SWAP_SPACE_SPI, true); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate time group. -$generatorSpring.clusterTime = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.property(res, cluster, 'clockSyncSamples', null, 8); - $generatorSpring.property(res, cluster, 'clockSyncFrequency', null, 120000); - $generatorSpring.property(res, cluster, 'timeServerPortBase', null, 31100); - $generatorSpring.property(res, cluster, 'timeServerPortRange', null, 100); - - res.needEmptyLine = true; - - return res; -}; - -// Generate OBC configuration group. -$generatorSpring.clusterODBC = function(odbc, res) { - if (!res) - res = $generatorCommon.builder(); - - if (odbc && odbc.odbcEnabled) - $generatorSpring.beanProperty(res, odbc, 'odbcConfiguration', $generatorCommon.ODBC_CONFIGURATION, true); - - res.needEmptyLine = true; - - return res; -}; - -// Generate thread pools group. -$generatorSpring.clusterPools = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.property(res, cluster, 'publicThreadPoolSize'); - $generatorSpring.property(res, cluster, 'systemThreadPoolSize'); - $generatorSpring.property(res, cluster, 'managementThreadPoolSize'); - $generatorSpring.property(res, cluster, 'igfsThreadPoolSize'); - $generatorSpring.property(res, cluster, 'rebalanceThreadPoolSize'); - - res.needEmptyLine = true; - - return res; -}; - -// Generate transactions group. -$generatorSpring.clusterTransactions = function(transactionConfiguration, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.beanProperty(res, transactionConfiguration, 'transactionConfiguration', $generatorCommon.TRANSACTION_CONFIGURATION, false); - - res.needEmptyLine = true; - - return res; -}; - -// Generate user attributes group. -$generatorSpring.clusterUserAttributes = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.isDefinedAndNotEmpty(cluster.attributes)) { - res.startBlock(''); - res.startBlock(''); - - _.forEach(cluster.attributes, function(attr) { - $generatorSpring.element(res, 'entry', 'key', attr.name, 'value', attr.value); - }); - - res.endBlock(''); - res.endBlock(''); - } - - res.needEmptyLine = true; - - return res; -}; - -/** - * XML generator for cluster's SSL configuration. - * - * @param cluster Cluster to get SSL configuration. - * @param res Optional configuration presentation builder object. - * @returns Configuration presentation builder object - */ -$generatorSpring.clusterSsl = function(cluster, res) { - if (!res) - res = $generatorCommon.builder(); - - if (cluster.sslEnabled && !_.isNil(cluster.sslContextFactory)) { - let sslFactory; - - if (_.isEmpty(cluster.sslContextFactory.keyStoreFilePath) && _.isEmpty(cluster.sslContextFactory.trustStoreFilePath)) - sslFactory = cluster.sslContextFactory; - else { - sslFactory = _.clone(cluster.sslContextFactory); - - sslFactory.keyStorePassword = _.isEmpty(cluster.sslContextFactory.keyStoreFilePath) ? null : '${ssl.key.storage.password}'; - sslFactory.trustStorePassword = _.isEmpty(cluster.sslContextFactory.trustStoreFilePath) ? null : '${ssl.trust.storage.password}'; - } - - const propsDesc = $generatorCommon.isDefinedAndNotEmpty(cluster.sslContextFactory.trustManagers) ? - $generatorCommon.SSL_CONFIGURATION_TRUST_MANAGER_FACTORY : - $generatorCommon.SSL_CONFIGURATION_TRUST_FILE_FACTORY; - - $generatorSpring.beanProperty(res, sslFactory, 'sslContextFactory', propsDesc, true); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate cache general group. -$generatorSpring.cacheGeneral = function(cache, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.property(res, cache, 'name'); - - $generatorSpring.property(res, cache, 'cacheMode'); - $generatorSpring.property(res, cache, 'atomicityMode'); - - if (cache.cacheMode === 'PARTITIONED' && $generatorSpring.property(res, cache, 'backups')) - $generatorSpring.property(res, cache, 'readFromBackup'); - - $generatorSpring.property(res, cache, 'copyOnRead'); - - if (cache.cacheMode === 'PARTITIONED' && cache.atomicityMode === 'TRANSACTIONAL') - $generatorSpring.property(res, cache, 'invalidate'); - - res.needEmptyLine = true; - - return res; -}; - -// Generate cache memory group. -$generatorSpring.cacheMemory = function(cache, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.property(res, cache, 'memoryMode', null, 'ONHEAP_TIERED'); - - if (cache.memoryMode !== 'OFFHEAP_VALUES') - $generatorSpring.property(res, cache, 'offHeapMaxMemory', null, -1); - - res.softEmptyLine(); - - $generatorSpring.evictionPolicy(res, cache.evictionPolicy, 'evictionPolicy'); - - res.softEmptyLine(); - - $generatorSpring.property(res, cache, 'startSize', null, 1500000); - $generatorSpring.property(res, cache, 'swapEnabled', null, false); - - res.needEmptyLine = true; - - return res; -}; - -// Generate cache query & indexing group. -$generatorSpring.cacheQuery = function(cache, domains, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.property(res, cache, 'sqlSchema'); - $generatorSpring.property(res, cache, 'sqlOnheapRowCacheSize', null, 10240); - $generatorSpring.property(res, cache, 'longQueryWarningTimeout', null, 3000); - - const indexedTypes = _.filter(domains, (domain) => domain.queryMetadata === 'Annotations'); - - if (indexedTypes.length > 0) { - res.softEmptyLine(); - - res.startBlock(''); - res.startBlock(''); - - _.forEach(indexedTypes, function(domain) { - res.line('' + $generatorCommon.JavaTypes.fullClassName(domain.keyType) + ''); - res.line('' + $generatorCommon.JavaTypes.fullClassName(domain.valueType) + ''); - }); - - res.endBlock(''); - res.endBlock(''); - } - - res.softEmptyLine(); - - $generatorSpring.listProperty(res, cache, 'sqlFunctionClasses'); - - res.softEmptyLine(); - - $generatorSpring.property(res, cache, 'snapshotableIndex', null, false); - $generatorSpring.property(res, cache, 'sqlEscapeAll', null, false); - - res.needEmptyLine = true; - - return res; -}; - -// Generate cache store group. -$generatorSpring.cacheStore = function(cache, domains, res) { - if (!res) - res = $generatorCommon.builder(); - - if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) { - const factoryKind = cache.cacheStoreFactory.kind; - - const storeFactory = cache.cacheStoreFactory[factoryKind]; - - if (storeFactory) { - if (factoryKind === 'CacheJdbcPojoStoreFactory') { - res.startBlock(''); - res.startBlock(''); - - $generatorSpring.property(res, storeFactory, 'dataSourceBean'); - - res.startBlock(''); - res.line(''); - res.endBlock(''); - - if (storeFactory.sqlEscapeAll) - $generatorSpring.property(res, storeFactory, 'sqlEscapeAll'); - - const domainConfigs = _.filter(domains, function(domain) { - return $generatorCommon.isDefinedAndNotEmpty(domain.databaseTable); - }); - - if ($generatorCommon.isDefinedAndNotEmpty(domainConfigs)) { - res.startBlock(''); - res.startBlock(''); - - _.forEach(domainConfigs, function(domain) { - res.startBlock(''); - - $generatorSpring.property(res, cache, 'name', 'cacheName'); - - $generatorSpring.classNameProperty(res, domain, 'keyType'); - $generatorSpring.property(res, domain, 'valueType'); - - $generatorSpring.domainStore(domain, res); - - res.endBlock(''); - }); - - res.endBlock(''); - res.endBlock(''); - } - - res.endBlock(''); - res.endBlock(''); - } - else if (factoryKind === 'CacheJdbcBlobStoreFactory') { - res.startBlock(''); - res.startBlock(''); - - if (storeFactory.connectVia === 'DataSource') - $generatorSpring.property(res, storeFactory, 'dataSourceBean'); - else { - $generatorSpring.property(res, storeFactory, 'connectionUrl'); - $generatorSpring.property(res, storeFactory, 'user'); - res.line(''); - } - - $generatorSpring.property(res, storeFactory, 'initSchema'); - $generatorSpring.property(res, storeFactory, 'createTableQuery'); - $generatorSpring.property(res, storeFactory, 'loadQuery'); - $generatorSpring.property(res, storeFactory, 'insertQuery'); - $generatorSpring.property(res, storeFactory, 'updateQuery'); - $generatorSpring.property(res, storeFactory, 'deleteQuery'); - - res.endBlock(''); - res.endBlock(''); - } - else - $generatorSpring.beanProperty(res, storeFactory, 'cacheStoreFactory', $generatorCommon.STORE_FACTORIES[factoryKind], true); - - if (storeFactory.dataSourceBean && (storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : null) : storeFactory.dialect)) { - if (!_.find(res.datasources, { dataSourceBean: storeFactory.dataSourceBean})) { - res.datasources.push({ - dataSourceBean: storeFactory.dataSourceBean, - dialect: storeFactory.dialect - }); - } - } - } - } - - res.softEmptyLine(); - - $generatorSpring.property(res, cache, 'storeKeepBinary', null, false); - $generatorSpring.property(res, cache, 'loadPreviousValue', null, false); - $generatorSpring.property(res, cache, 'readThrough', null, false); - $generatorSpring.property(res, cache, 'writeThrough', null, false); - - res.softEmptyLine(); - - if (cache.writeBehindEnabled) { - $generatorSpring.property(res, cache, 'writeBehindEnabled', null, false); - $generatorSpring.property(res, cache, 'writeBehindBatchSize', null, 512); - $generatorSpring.property(res, cache, 'writeBehindFlushSize', null, 10240); - $generatorSpring.property(res, cache, 'writeBehindFlushFrequency', null, 5000); - $generatorSpring.property(res, cache, 'writeBehindFlushThreadCount', null, 1); - } - - res.needEmptyLine = true; - - return res; -}; - -// Generate cache node filter group. -$generatorSpring.cacheNodeFilter = function(cache, igfss, res) { - if (!res) - res = $generatorCommon.builder(); - - const kind = _.get(cache, 'nodeFilter.kind'); - - if (_.isNil(kind) || _.isNil(cache.nodeFilter[kind])) - return res; - - switch (kind) { - case 'IGFS': - const foundIgfs = _.find(igfss, (igfs) => igfs._id === cache.nodeFilter.IGFS.igfs); - - if (foundIgfs) { - res.startBlock(''); - res.startBlock(''); - res.line(''); - res.endBlock(''); - res.endBlock(''); - } - - break; - - case 'OnNodes': - const nodes = cache.nodeFilter.OnNodes.nodeIds; - - if ($generatorCommon.isDefinedAndNotEmpty(nodes)) { - res.startBlock(''); - res.startBlock(''); - res.startBlock(''); - res.startBlock(''); - - _.forEach(nodes, (nodeId) => { - res.startBlock(''); - res.line(''); - res.endBlock(''); - }); - - res.endBlock(''); - res.endBlock(''); - res.endBlock(''); - res.endBlock(''); - } - - break; - - case 'Custom': - res.startBlock(''); - res.line(''); - res.endBlock(''); - - break; - - default: break; - } - - res.needEmptyLine = true; - - return res; -}; - -// Generate cache concurrency group. -$generatorSpring.cacheConcurrency = function(cache, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.property(res, cache, 'maxConcurrentAsyncOperations', null, 500); - $generatorSpring.property(res, cache, 'defaultLockTimeout', null, 0); - $generatorSpring.property(res, cache, 'atomicWriteOrderMode'); - $generatorSpring.property(res, cache, 'writeSynchronizationMode', null, 'PRIMARY_SYNC'); - - res.needEmptyLine = true; - - return res; -}; - -// Generate cache rebalance group. -$generatorSpring.cacheRebalance = function(cache, res) { - if (!res) - res = $generatorCommon.builder(); - - if (cache.cacheMode !== 'LOCAL') { - $generatorSpring.property(res, cache, 'rebalanceMode', null, 'ASYNC'); - $generatorSpring.property(res, cache, 'rebalanceThreadPoolSize', null, 1); - $generatorSpring.property(res, cache, 'rebalanceBatchSize', null, 524288); - $generatorSpring.property(res, cache, 'rebalanceBatchesPrefetchCount', null, 2); - $generatorSpring.property(res, cache, 'rebalanceOrder', null, 0); - $generatorSpring.property(res, cache, 'rebalanceDelay', null, 0); - $generatorSpring.property(res, cache, 'rebalanceTimeout', null, 10000); - $generatorSpring.property(res, cache, 'rebalanceThrottle', null, 0); - } - - res.softEmptyLine(); - - if (cache.igfsAffinnityGroupSize) { - res.startBlock(''); - res.startBlock(''); - $generatorSpring.constructorArg(res, -1, cache, 'igfsAffinnityGroupSize'); - res.endBlock(''); - res.endBlock(''); - } - - return res; -}; - -// Generate cache server near cache group. -$generatorSpring.cacheServerNearCache = function(cache, res) { - if (!res) - res = $generatorCommon.builder(); - - if (cache.cacheMode === 'PARTITIONED' && cache.nearCacheEnabled) { - res.emptyLineIfNeeded(); - - res.startBlock(''); - res.startBlock(''); - - if (cache.nearConfiguration) { - if (cache.nearConfiguration.nearStartSize) - $generatorSpring.property(res, cache.nearConfiguration, 'nearStartSize', null, 375000); - - $generatorSpring.evictionPolicy(res, cache.nearConfiguration.nearEvictionPolicy, 'nearEvictionPolicy'); - } - - res.endBlock(''); - res.endBlock(''); - } - - res.needEmptyLine = true; - - return res; -}; - -// Generate cache statistics group. -$generatorSpring.cacheStatistics = function(cache, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.property(res, cache, 'statisticsEnabled', null, false); - $generatorSpring.property(res, cache, 'managementEnabled', null, false); - - res.needEmptyLine = true; - - return res; -}; - -// Generate domain model query fields. -$generatorSpring.domainModelQueryFields = function(res, domain) { - const fields = domain.fields; - - if (fields && fields.length > 0) { - res.emptyLineIfNeeded(); - - res.startBlock(''); - res.startBlock(''); - - _.forEach(fields, function(field) { - $generatorSpring.element(res, 'entry', 'key', field.name, 'value', $generatorCommon.JavaTypes.fullClassName(field.className)); - }); - - res.endBlock(''); - res.endBlock(''); - - res.needEmptyLine = true; - } -}; - -// Generate domain model query fields. -$generatorSpring.domainModelQueryAliases = function(res, domain) { - const aliases = domain.aliases; - - if (aliases && aliases.length > 0) { - res.emptyLineIfNeeded(); - - res.startBlock(''); - res.startBlock(''); - - _.forEach(aliases, function(alias) { - $generatorSpring.element(res, 'entry', 'key', alias.field, 'value', alias.alias); - }); - - res.endBlock(''); - res.endBlock(''); - - res.needEmptyLine = true; - } -}; - -// Generate domain model indexes. -$generatorSpring.domainModelQueryIndexes = function(res, domain) { - const indexes = domain.indexes; - - if (indexes && indexes.length > 0) { - res.emptyLineIfNeeded(); - - res.startBlock(''); - res.startBlock(''); - - _.forEach(indexes, function(index) { - res.startBlock(''); - - $generatorSpring.property(res, index, 'name'); - $generatorSpring.property(res, index, 'indexType'); - - const fields = index.fields; - - if (fields && fields.length > 0) { - res.startBlock(''); - res.startBlock(''); - - _.forEach(fields, function(field) { - $generatorSpring.element(res, 'entry', 'key', field.name, 'value', field.direction); - }); - - res.endBlock(''); - res.endBlock(''); - } - - res.endBlock(''); - }); - - res.endBlock(''); - res.endBlock(''); - - res.needEmptyLine = true; - } -}; - -// Generate domain model db fields. -$generatorSpring.domainModelDatabaseFields = function(res, domain, fieldProp) { - const fields = domain[fieldProp]; - - if (fields && fields.length > 0) { - res.emptyLineIfNeeded(); - - res.startBlock(''); - - res.startBlock(''); - - _.forEach(fields, function(field) { - res.startBlock(''); - - $generatorSpring.property(res, field, 'databaseFieldName'); - - res.startBlock(''); - res.line(''); - res.endBlock(''); - - $generatorSpring.property(res, field, 'javaFieldName'); - - $generatorSpring.classNameProperty(res, field, 'javaFieldType'); - - res.endBlock(''); - }); - - res.endBlock(''); - res.endBlock(''); - - res.needEmptyLine = true; - } -}; - -// Generate domain model general group. -$generatorSpring.domainModelGeneral = function(domain, res) { - if (!res) - res = $generatorCommon.builder(); - - switch ($generatorCommon.domainQueryMetadata(domain)) { - case 'Annotations': - if ($generatorCommon.isDefinedAndNotEmpty(domain.keyType) || $generatorCommon.isDefinedAndNotEmpty(domain.valueType)) { - res.startBlock(''); - res.startBlock(''); - - if ($generatorCommon.isDefinedAndNotEmpty(domain.keyType)) - res.line('' + $generatorCommon.JavaTypes.fullClassName(domain.keyType) + ''); - else - res.line('???'); - - if ($generatorCommon.isDefinedAndNotEmpty(domain.valueType)) - res.line('' + $generatorCommon.JavaTypes.fullClassName(domain.valueType) + ''); - else - res.line('>???'); - - res.endBlock(''); - res.endBlock(''); - } - - break; - - case 'Configuration': - $generatorSpring.classNameProperty(res, domain, 'keyType'); - $generatorSpring.property(res, domain, 'valueType'); - - break; - - default: - } - - res.needEmptyLine = true; - - return res; -}; - -// Generate domain model for query group. -$generatorSpring.domainModelQuery = function(domain, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.domainQueryMetadata(domain) === 'Configuration') { - $generatorSpring.domainModelQueryFields(res, domain); - $generatorSpring.domainModelQueryAliases(res, domain); - $generatorSpring.domainModelQueryIndexes(res, domain); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate domain model for store group. -$generatorSpring.domainStore = function(domain, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.property(res, domain, 'databaseSchema'); - $generatorSpring.property(res, domain, 'databaseTable'); - - res.softEmptyLine(); - - $generatorSpring.domainModelDatabaseFields(res, domain, 'keyFields'); - $generatorSpring.domainModelDatabaseFields(res, domain, 'valueFields'); - - res.needEmptyLine = true; - - return res; -}; - -$generatorSpring.cacheQueryMetadata = function(domain, res) { - if (!res) - res = $generatorCommon.builder(); - - res.startBlock(''); - - $generatorSpring.classNameProperty(res, domain, 'keyType'); - $generatorSpring.property(res, domain, 'valueType'); - - $generatorSpring.domainModelQuery(domain, res); - - res.endBlock(''); - - res.needEmptyLine = true; - - return res; -}; - -// Generate domain models configs. -$generatorSpring.cacheDomains = function(domains, res) { - if (!res) - res = $generatorCommon.builder(); - - const domainConfigs = _.filter(domains, function(domain) { - return $generatorCommon.domainQueryMetadata(domain) === 'Configuration' && - $generatorCommon.isDefinedAndNotEmpty(domain.fields); - }); - - if ($generatorCommon.isDefinedAndNotEmpty(domainConfigs)) { - res.emptyLineIfNeeded(); - - res.startBlock(''); - res.startBlock(''); - - _.forEach(domainConfigs, function(domain) { - $generatorSpring.cacheQueryMetadata(domain, res); - }); - - res.endBlock(''); - res.endBlock(''); - } - - return res; -}; - -// Generate cache configs. -$generatorSpring.cache = function(cache, res) { - if (!res) - res = $generatorCommon.builder(); - - res.startBlock(''); - - $generatorSpring.cacheConfiguration(cache, res); - - res.endBlock(''); - - return res; -}; - -// Generate cache configs. -$generatorSpring.cacheConfiguration = function(cache, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.cacheGeneral(cache, res); - $generatorSpring.cacheMemory(cache, res); - $generatorSpring.cacheQuery(cache, cache.domains, res); - $generatorSpring.cacheStore(cache, cache.domains, res); - - const igfs = _.get(cache, 'nodeFilter.IGFS.instance'); - - $generatorSpring.cacheNodeFilter(cache, igfs ? [igfs] : [], res); - $generatorSpring.cacheConcurrency(cache, res); - $generatorSpring.cacheRebalance(cache, res); - $generatorSpring.cacheServerNearCache(cache, res); - $generatorSpring.cacheStatistics(cache, res); - $generatorSpring.cacheDomains(cache.domains, res); - - return res; -}; - -// Generate caches configs. -$generatorSpring.clusterCaches = function(caches, igfss, isSrvCfg, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.isDefinedAndNotEmpty(caches) || (isSrvCfg && $generatorCommon.isDefinedAndNotEmpty(igfss))) { - res.emptyLineIfNeeded(); - - res.startBlock(''); - res.startBlock(''); - - _.forEach(caches, function(cache) { - $generatorSpring.cache(cache, res); - - res.needEmptyLine = true; - }); - - if (isSrvCfg) { - _.forEach(igfss, (igfs) => { - $generatorSpring.cache($generatorCommon.igfsDataCache(igfs), res); - - res.needEmptyLine = true; - - $generatorSpring.cache($generatorCommon.igfsMetaCache(igfs), res); - - res.needEmptyLine = true; - }); - } - - res.endBlock(''); - res.endBlock(''); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate IGFSs configs. -$generatorSpring.igfss = function(igfss, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.isDefinedAndNotEmpty(igfss)) { - res.emptyLineIfNeeded(); - - res.startBlock(''); - res.startBlock(''); - - _.forEach(igfss, function(igfs) { - res.startBlock(''); - - $generatorSpring.igfsGeneral(igfs, res); - $generatorSpring.igfsIPC(igfs, res); - $generatorSpring.igfsFragmentizer(igfs, res); - $generatorSpring.igfsDualMode(igfs, res); - $generatorSpring.igfsSecondFS(igfs, res); - $generatorSpring.igfsMisc(igfs, res); - - res.endBlock(''); - - res.needEmptyLine = true; - }); - - res.endBlock(''); - res.endBlock(''); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate IGFS IPC configuration. -$generatorSpring.igfsIPC = function(igfs, res) { - if (!res) - res = $generatorCommon.builder(); - - if (igfs.ipcEndpointEnabled) { - $generatorSpring.beanProperty(res, igfs.ipcEndpointConfiguration, 'ipcEndpointConfiguration', $generatorCommon.IGFS_IPC_CONFIGURATION, true); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate IGFS fragmentizer configuration. -$generatorSpring.igfsFragmentizer = function(igfs, res) { - if (!res) - res = $generatorCommon.builder(); - - if (igfs.fragmentizerEnabled) { - $generatorSpring.property(res, igfs, 'fragmentizerConcurrentFiles', null, 0); - $generatorSpring.property(res, igfs, 'fragmentizerThrottlingBlockLength', null, 16777216); - $generatorSpring.property(res, igfs, 'fragmentizerThrottlingDelay', null, 200); - - res.needEmptyLine = true; - } - else - $generatorSpring.property(res, igfs, 'fragmentizerEnabled'); - - return res; -}; - -// Generate IGFS dual mode configuration. -$generatorSpring.igfsDualMode = function(igfs, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.property(res, igfs, 'dualModeMaxPendingPutsSize', null, 0); - - if ($generatorCommon.isDefinedAndNotEmpty(igfs.dualModePutExecutorService)) { - res.startBlock(''); - res.line(''); - res.endBlock(''); - } - - $generatorSpring.property(res, igfs, 'dualModePutExecutorServiceShutdown', null, false); - - res.needEmptyLine = true; - - return res; -}; - -$generatorSpring.igfsSecondFS = function(igfs, res) { - if (!res) - res = $generatorCommon.builder(); - - if (igfs.secondaryFileSystemEnabled) { - const secondFs = igfs.secondaryFileSystem || {}; - - res.startBlock(''); - - res.startBlock(''); - - const nameDefined = $generatorCommon.isDefinedAndNotEmpty(secondFs.userName); - const cfgDefined = $generatorCommon.isDefinedAndNotEmpty(secondFs.cfgPath); - - $generatorSpring.constructorArg(res, 0, secondFs, 'uri'); - - if (cfgDefined || nameDefined) - $generatorSpring.constructorArg(res, 1, secondFs, 'cfgPath'); - - $generatorSpring.constructorArg(res, 2, secondFs, 'userName', null, true); - - res.endBlock(''); - res.endBlock(''); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate IGFS general configuration. -$generatorSpring.igfsGeneral = function(igfs, res) { - if (!res) - res = $generatorCommon.builder(); - - if ($generatorCommon.isDefinedAndNotEmpty(igfs.name)) { - igfs.dataCacheName = $generatorCommon.igfsDataCache(igfs).name; - igfs.metaCacheName = $generatorCommon.igfsMetaCache(igfs).name; - - $generatorSpring.property(res, igfs, 'name'); - $generatorSpring.property(res, igfs, 'dataCacheName'); - $generatorSpring.property(res, igfs, 'metaCacheName'); - $generatorSpring.property(res, igfs, 'defaultMode', null, 'DUAL_ASYNC'); - - res.needEmptyLine = true; - } - - return res; -}; - -// Generate IGFS misc configuration. -$generatorSpring.igfsMisc = function(igfs, res) { - if (!res) - res = $generatorCommon.builder(); - - $generatorSpring.property(res, igfs, 'blockSize', null, 65536); - $generatorSpring.property(res, igfs, 'streamBufferSize', null, 65536); - $generatorSpring.property(res, igfs, 'maxSpaceSize', null, 0); - $generatorSpring.property(res, igfs, 'maximumTaskRangeLength', null, 0); - $generatorSpring.property(res, igfs, 'managementPort', null, 11400); - $generatorSpring.property(res, igfs, 'perNodeBatchSize', null, 100); - $generatorSpring.property(res, igfs, 'perNodeParallelBatchCount', null, 8); - $generatorSpring.property(res, igfs, 'prefetchBlocks', null, 0); - $generatorSpring.property(res, igfs, 'sequentialReadsBeforePrefetch', null, 0); - $generatorSpring.property(res, igfs, 'trashPurgeTimeout', null, 1000); - $generatorSpring.property(res, igfs, 'colocateMetadata', null, true); - $generatorSpring.property(res, igfs, 'relaxedConsistency', null, true); - - res.softEmptyLine(); - - if (igfs.pathModes && igfs.pathModes.length > 0) { - res.startBlock(''); - res.startBlock(''); - - _.forEach(igfs.pathModes, function(pair) { - res.line(''); - }); - - res.endBlock(''); - res.endBlock(''); - } - - return res; -}; - -// Generate DataSource beans. -$generatorSpring.generateDataSources = function(datasources, res) { - if (!res) - res = $generatorCommon.builder(); - - if (datasources.length > 0) { - res.line(''); - - _.forEach(datasources, (datasource) => $generatorSpring.generateDataSource(datasource, res)); - - res.needEmptyLine = true; - - res.emptyLineIfNeeded(); - } - - return res; -}; - -$generatorSpring.generateDataSource = function(datasource, res) { - const beanId = datasource.dataSourceBean; - - res.startBlock(''); - - switch (datasource.dialect) { - case 'Generic': - res.line(''); - - break; - - case 'DB2': - res.line(''); - res.line(''); - res.line(''); - res.line(''); - - break; - - case 'PostgreSQL': - res.line(''); - - break; - - default: - res.line(''); - } - - res.line(''); - res.line(''); - - res.endBlock(''); - - res.needEmptyLine = true; - - res.emptyLineIfNeeded(); -}; - -$generatorSpring.clusterConfiguration = function(cluster, clientNearCfg, res) { - const isSrvCfg = _.isNil(clientNearCfg); - - if (!isSrvCfg) { - res.line(''); - - res.needEmptyLine = true; - } - - $generatorSpring.clusterGeneral(cluster, res); - - $generatorSpring.clusterAtomics(cluster.atomicConfiguration, res); - - $generatorSpring.clusterBinary(cluster.binaryConfiguration, res); - - $generatorSpring.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, res); - - $generatorSpring.clusterCollision(cluster.collision, res); - - $generatorSpring.clusterCommunication(cluster, res); - - $generatorSpring.clusterConnector(cluster.connector, res); - - $generatorSpring.clusterDeployment(cluster, res); - - $generatorSpring.clusterEvents(cluster, res); - - $generatorSpring.clusterFailover(cluster, res); - - $generatorSpring.clusterLogger(cluster.logger, res); - - $generatorSpring.clusterODBC(cluster.odbc, res); - - $generatorSpring.clusterMarshaller(cluster, res); - - $generatorSpring.clusterMetrics(cluster, res); - - $generatorSpring.clusterSwap(cluster, res); - - $generatorSpring.clusterTime(cluster, res); - - $generatorSpring.clusterPools(cluster, res); - - $generatorSpring.clusterTransactions(cluster.transactionConfiguration, res); - - $generatorSpring.clusterCaches(cluster.caches, cluster.igfss, isSrvCfg, res); - - $generatorSpring.clusterSsl(cluster, res); - - if (isSrvCfg) - $generatorSpring.igfss(cluster.igfss, res); - - $generatorSpring.clusterUserAttributes(cluster, res); - - return res; -}; - -$generatorSpring.cluster = function(cluster, clientNearCfg) { - if (cluster) { - const res = $generatorCommon.builder(1); - - if (clientNearCfg) { - res.startBlock(''); - - if (clientNearCfg.nearStartSize) - $generatorSpring.property(res, clientNearCfg, 'nearStartSize'); - - if (clientNearCfg.nearEvictionPolicy && clientNearCfg.nearEvictionPolicy.kind) - $generatorSpring.evictionPolicy(res, clientNearCfg.nearEvictionPolicy, 'nearEvictionPolicy'); - - res.endBlock(''); - - res.needEmptyLine = true; - - res.emptyLineIfNeeded(); - } - - // Generate Ignite Configuration. - res.startBlock(''); - - $generatorSpring.clusterConfiguration(cluster, clientNearCfg, res); - - res.endBlock(''); - - // Build final XML: - // 1. Add header. - let xml = '\n\n'; - - xml += '\n\n'; - xml += ' this.NotebookData.findIndex(notebook)) .then((idx) => { this.NotebookData.remove(notebook) diff --git a/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade b/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade deleted file mode 100644 index 0396727f95d89..0000000000000 --- a/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade +++ /dev/null @@ -1,39 +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. - -include /app/helpers/jade/mixins.jade - -.modal(tabindex='-1' role='dialog') - .modal-dialog - .modal-content - .modal-header - button.close(ng-click='$hide()') × - h4.modal-title Scan filter - form.form-horizontal.modal-body.row(name='ui.inputForm' novalidate) - .settings-row - .col-sm-2 - label.required.labelFormField Filter:  - .col-sm-10 - .input-tip - +ignite-form-field-input('"filter"', 'ui.filter', false, 'true', 'Enter filter')( - data-ignite-form-field-input-autofocus='true' - ignite-on-enter='form.$valid && ok()' - ) - .settings-row - +checkbox('Case sensitive search', 'ui.caseSensitive', '"caseSensitive"', 'Select this checkbox for case sensitive search') - .modal-footer - button.btn.btn-default(id='btn-cancel' ng-click='$hide()') Cancel - button.btn.btn-primary(id='btn-scan' ng-disabled='ui.inputForm.$invalid' ng-click='ok()') Scan diff --git a/modules/web-console/frontend/app/modules/sql/sql.controller.js b/modules/web-console/frontend/app/modules/sql/sql.controller.js index 4e6e3720bfbef..0d0b171df5595 100644 --- a/modules/web-console/frontend/app/modules/sql/sql.controller.js +++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js @@ -50,6 +50,9 @@ class Paragraph { const self = this; self.id = 'paragraph-' + paragraphId++; + self.qryType = paragraph.qryType || 'query'; + self.maxPages = 0; + self.filter = ''; _.assign(this, paragraph); @@ -77,27 +80,28 @@ class Paragraph { enableColumnMenus: false, flatEntityAccess: true, fastWatch: true, + categories: [], rebuildColumns() { if (_.isNil(this.api)) return; - this.categories = []; + this.categories.length = 0; + this.columnDefs = _.reduce(self.meta, (cols, col, idx) => { - if (self.columnFilter(col)) { - cols.push({ - displayName: col.fieldName, - headerTooltip: _fullColName(col), - field: idx.toString(), - minWidth: 50, - cellClass: 'cell-left' - }); + cols.push({ + displayName: col.fieldName, + headerTooltip: _fullColName(col), + field: idx.toString(), + minWidth: 50, + cellClass: 'cell-left', + visible: self.columnFilter(col) + }); - this.categories.push({ - name: col.fieldName, - visible: true, - selectable: true - }); - } + this.categories.push({ + name: col.fieldName, + visible: self.columnFilter(col), + selectable: true + }); return cols; }, []); @@ -182,8 +186,8 @@ class Paragraph { } // Controller for SQL notebook screen. -export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', '$animate', '$location', '$anchorScroll', '$state', '$filter', '$modal', '$popover', 'IgniteLoading', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteAgentMonitor', 'IgniteChartColors', 'IgniteNotebook', 'IgniteScanFilterInput', 'IgniteNodes', 'uiGridExporterConstants', 'IgniteVersion', - function($root, $scope, $http, $q, $timeout, $interval, $animate, $location, $anchorScroll, $state, $filter, $modal, $popover, Loading, LegacyUtils, Messages, Confirm, agentMonitor, IgniteChartColors, Notebook, ScanFilterInput, Nodes, uiGridExporterConstants, Version) { +export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', '$animate', '$location', '$anchorScroll', '$state', '$filter', '$modal', '$popover', 'IgniteLoading', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteAgentMonitor', 'IgniteChartColors', 'IgniteNotebook', 'IgniteNodes', 'uiGridExporterConstants', 'IgniteVersion', + function($root, $scope, $http, $q, $timeout, $interval, $animate, $location, $anchorScroll, $state, $filter, $modal, $popover, Loading, LegacyUtils, Messages, Confirm, agentMonitor, IgniteChartColors, Notebook, Nodes, uiGridExporterConstants, Version) { let stopTopology = null; const _tryStopRefresh = function(paragraph) { @@ -206,6 +210,15 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', $scope.caches = []; $scope.pageSizes = [50, 100, 200, 400, 800, 1000]; + $scope.maxPages = [ + {label: 'Unlimited', value: 0}, + {label: '1', value: 1}, + {label: '5', value: 5}, + {label: '10', value: 10}, + {label: '20', value: 20}, + {label: '50', value: 50}, + {label: '100', value: 100} + ]; $scope.timeLineSpans = ['1', '5', '10', '15', '30']; @@ -213,7 +226,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', $scope.modes = LegacyUtils.mkOptions(['PARTITIONED', 'REPLICATED', 'LOCAL']); - $scope.loadingText = $root.IgniteDemoMode ? 'Demo grid is starting. Please wait...' : 'Loading notebook screen...'; + $scope.loadingText = $root.IgniteDemoMode ? 'Demo grid is starting. Please wait...' : 'Loading query notebook screen...'; $scope.timeUnit = [ {value: 1000, label: 'seconds', short: 's'}, @@ -768,11 +781,10 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', if (idx >= 0) { if (!_.includes($scope.notebook.expandedParagraphs, idx)) - $scope.notebook.expandedParagraphs.push(idx); + $scope.notebook.expandedParagraphs = $scope.notebook.expandedParagraphs.concat([idx]); - setTimeout(function() { - $scope.notebook.paragraphs[idx].ace.focus(); - }); + if ($scope.notebook.paragraphs[idx].ace) + setTimeout(() => $scope.notebook.paragraphs[idx].ace.focus()); } $location.hash(id); @@ -816,7 +828,8 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', let item = _.find(cachesAcc, {name: cache.name}); if (_.isNil(item)) { - cache.label = maskCacheName(cache.name); + cache.label = maskCacheName(cache.name, true); + cache.value = cache.name; cache.nodes = []; @@ -839,7 +852,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', return; // Reset to first cache in case of stopped selected. - const cacheNames = _.map($scope.caches, (cache) => cache.name); + const cacheNames = _.map($scope.caches, (cache) => cache.value); _.forEach($scope.notebook.paragraphs, (paragraph) => { if (!_.includes(cacheNames, paragraph.cacheName)) @@ -885,7 +898,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', (paragraph) => new Paragraph($animate, $timeout, paragraph)); if (_.isEmpty($scope.notebook.paragraphs)) - $scope.addParagraph(); + $scope.addQuery(); else $scope.rebuildScrollParagraphs(); }) @@ -936,32 +949,37 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', paragraph.edit = false; }; - $scope.addParagraph = function() { + $scope.addParagraph = (paragraph, sz) => { + if ($scope.caches && $scope.caches.length > 0) + paragraph.cacheName = _.head($scope.caches).value; + + $scope.notebook.paragraphs.push(paragraph); + + $scope.notebook.expandedParagraphs.push(sz); + + $scope.rebuildScrollParagraphs(); + + $location.hash(paragraph.id); + }; + + $scope.addQuery = function() { const sz = $scope.notebook.paragraphs.length; const paragraph = new Paragraph($animate, $timeout, { name: 'Query' + (sz === 0 ? '' : sz), query: '', - pageSize: $scope.pageSizes[0], + pageSize: $scope.pageSizes[1], timeLineSpan: $scope.timeLineSpans[0], result: 'none', rate: { value: 1, unit: 60000, installed: false - } + }, + qryType: 'query' }); - if ($scope.caches && $scope.caches.length > 0) - paragraph.cacheName = $scope.caches[0].name; - - $scope.notebook.paragraphs.push(paragraph); - - $scope.notebook.expandedParagraphs.push(sz); - - $scope.rebuildScrollParagraphs(); - - $location.hash(paragraph.id); + $scope.addParagraph(paragraph, sz); $timeout(() => { $anchorScroll(); @@ -970,6 +988,26 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', }); }; + $scope.addScan = function() { + const sz = $scope.notebook.paragraphs.length; + + const paragraph = new Paragraph($animate, $timeout, { + name: 'Scan' + (sz === 0 ? '' : sz), + query: '', + pageSize: $scope.pageSizes[1], + timeLineSpan: $scope.timeLineSpans[0], + result: 'none', + rate: { + value: 1, + unit: 60000, + installed: false + }, + qryType: 'scan' + }); + + $scope.addParagraph(paragraph, sz); + }; + function _saveChartSettings(paragraph) { if (!_.isEmpty(paragraph.charts)) { const chart = paragraph.charts[0].api.getScope().chart; @@ -1010,7 +1048,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', }; $scope.removeParagraph = function(paragraph) { - Confirm.confirm('Are you sure you want to remove: "' + paragraph.name + '"?') + Confirm.confirm('Are you sure you want to remove query: "' + paragraph.name + '"?') .then(function() { $scope.stopRefresh(paragraph); @@ -1315,8 +1353,8 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', return false; }; - $scope.execute = (paragraph, nonCollocatedJoins = false) => { - const local = !!paragraph.localQry; + $scope.execute = (paragraph, local = false) => { + const nonCollocatedJoins = !!paragraph.nonCollocatedJoins; $scope.actionAvailable(paragraph, true) && _chooseNode(paragraph.cacheName, local) .then((nid) => { @@ -1330,16 +1368,16 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', return _closeOldQuery(paragraph) .then(() => { const args = paragraph.queryArgs = { + type: 'QUERY', cacheName: paragraph.cacheName, - pageSize: paragraph.pageSize, query: paragraph.query, - firstPageOnly: paragraph.firstPageOnly, + pageSize: paragraph.pageSize, + maxPages: paragraph.maxPages, nonCollocatedJoins, - type: 'QUERY', localNid: local ? nid : null }; - const qry = args.firstPageOnly ? addLimit(args.query, args.pageSize) : paragraph.query; + const qry = args.maxPages ? addLimit(args.query, args.pageSize * args.maxPages) : paragraph.query; return agentMonitor.query(nid, args.cacheName, qry, nonCollocatedJoins, local, args.pageSize); }) @@ -1386,10 +1424,10 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', .then(() => _chooseNode(paragraph.cacheName, false)) .then((nid) => { const args = paragraph.queryArgs = { + type: 'EXPLAIN', cacheName: paragraph.cacheName, - pageSize: paragraph.pageSize, query: 'EXPLAIN ' + paragraph.query, - type: 'EXPLAIN' + pageSize: paragraph.pageSize }; return agentMonitor.query(nid, args.cacheName, args.query, false, false, args.pageSize); @@ -1403,8 +1441,10 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', .then(() => paragraph.ace.focus()); }; - $scope.scan = (paragraph, query = null) => { - const local = !!paragraph.localQry; + $scope.scan = (paragraph, local = false) => { + const {filter, caseSensitive} = paragraph; + const prefix = caseSensitive ? SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE : SCAN_CACHE_WITH_FILTER; + const query = `${prefix}${filter}`; $scope.actionAvailable(paragraph, false) && _chooseNode(paragraph.cacheName, local) .then((nid) => { @@ -1418,45 +1458,22 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', _closeOldQuery(paragraph) .then(() => { const args = paragraph.queryArgs = { + type: 'SCAN', cacheName: paragraph.cacheName, - pageSize: paragraph.pageSize, - firstPageOnly: paragraph.firstPageOnly, query, - type: 'SCAN', + filter, + pageSize: paragraph.pageSize, localNid: local ? nid : null }; return agentMonitor.query(nid, args.cacheName, query, false, local, args.pageSize); }) - .then((res) => { - if (paragraph.firstPageOnly) { - res.hasMore = false; - - _processQueryResult(paragraph, true, res); - - _closeOldQuery(paragraph); - } - else - _processQueryResult(paragraph, true, res); - }) + .then((res) => _processQueryResult(paragraph, true, res)) .catch((err) => { paragraph.errMsg = err.message; _showLoading(paragraph, false); - }) - .then(() => paragraph.ace.focus()); - }); - }; - - $scope.scanWithFilter = (paragraph) => { - if (!$scope.actionAvailable(paragraph, false)) - return; - - ScanFilterInput.open() - .then(({filter, caseSensitive}) => { - const prefix = caseSensitive ? SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE : SCAN_CACHE_WITH_FILTER; - - $scope.scan(paragraph, `${prefix}${filter}`); + }); }); }; @@ -1511,25 +1528,23 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', _showLoading(paragraph, false); }) - .then(() => paragraph.ace.focus()); + .then(() => paragraph.ace && paragraph.ace.focus()); }; - const _export = (fileName, columnFilter, meta, rows) => { + const _export = (fileName, columnDefs, meta, rows) => { let csvContent = ''; const cols = []; const excludedCols = []; - if (meta) { - _.forEach(meta, (col, idx) => { - if (columnFilter(col)) - cols.push(_fullColName(col)); - else - excludedCols.push(idx); - }); + _.forEach(meta, (col, idx) => { + if (columnDefs[idx].visible) + cols.push(_fullColName(col)); + else + excludedCols.push(idx); + }); - csvContent += cols.join(';') + '\n'; - } + csvContent += cols.join(';') + '\n'; _.forEach(rows, (row) => { cols.length = 0; @@ -1543,8 +1558,8 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', }); } else { - _.forEach(meta, (col) => { - if (columnFilter(col)) { + _.forEach(columnDefs, (col) => { + if (col.visible) { const elem = row[col.fieldName]; cols.push(_.isUndefined(elem) ? '' : JSON.stringify(elem)); @@ -1559,7 +1574,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', }; $scope.exportCsv = function(paragraph) { - _export(paragraph.name + '.csv', paragraph.columnFilter, paragraph.meta, paragraph.rows); + _export(paragraph.name + '.csv', paragraph.gridOptions.columnDefs, paragraph.meta, paragraph.rows); // paragraph.gridOptions.api.exporter.csvExport(uiGridExporterConstants.ALL, uiGridExporterConstants.VISIBLE); }; @@ -1573,17 +1588,17 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', return Promise.resolve(args.localNid || _chooseNode(args.cacheName, false)) .then((nid) => agentMonitor.queryGetAll(nid, args.cacheName, args.query, !!args.nonCollocatedJoins, !!args.localNid)) - .then((res) => _export(paragraph.name + '-all.csv', paragraph.columnFilter, res.columns, res.rows)) + .then((res) => _export(paragraph.name + '-all.csv', paragraph.gridOptions.columnDefs, res.columns, res.rows)) .catch(Messages.showError) - .then(() => paragraph.ace.focus()); + .then(() => paragraph.ace && paragraph.ace.focus()); }; // $scope.exportPdfAll = function(paragraph) { // $http.post('/api/v1/agent/query/getAll', {query: paragraph.query, cacheName: paragraph.cacheName}) - // .success(function(item) { - // _export(paragraph.name + '-all.csv', item.meta, item.rows); + // .then(({data}) { + // _export(paragraph.name + '-all.csv', data.meta, data.rows); // }) - // .error(Messages.showError); + // .catch(Messages.showError); // }; $scope.rateAsString = function(paragraph) { @@ -1652,9 +1667,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', $scope.dblclickMetadata = function(paragraph, node) { paragraph.ace.insert(node.name); - setTimeout(function() { - paragraph.ace.focus(); - }, 1); + setTimeout(() => paragraph.ace.focus(), 1); }; $scope.importMetadata = function() { diff --git a/modules/web-console/frontend/app/modules/sql/sql.module.js b/modules/web-console/frontend/app/modules/sql/sql.module.js index d615d28e108c2..a1ffde9646f99 100644 --- a/modules/web-console/frontend/app/modules/sql/sql.module.js +++ b/modules/web-console/frontend/app/modules/sql/sql.module.js @@ -19,7 +19,6 @@ import angular from 'angular'; import NotebookData from './Notebook.data'; import Notebook from './Notebook.service'; -import ScanFilterInput from './scan-filter-input.service'; import notebook from './notebook.controller'; import sql from './sql.controller'; @@ -55,6 +54,5 @@ angular.module('ignite-console.sql', [ ) .service('IgniteNotebookData', NotebookData) .service('IgniteNotebook', Notebook) - .service('IgniteScanFilterInput', ScanFilterInput) .controller('notebookController', notebook) .controller('sqlController', sql); diff --git a/modules/web-console/frontend/app/modules/states/configuration.state.js b/modules/web-console/frontend/app/modules/states/configuration.state.js index 888c804b37af4..61dca13f9633b 100644 --- a/modules/web-console/frontend/app/modules/states/configuration.state.js +++ b/modules/web-console/frontend/app/modules/states/configuration.state.js @@ -24,12 +24,14 @@ import previewPanel from './configuration/preview-panel.directive.js'; import ConfigurationSummaryCtrl from './configuration/summary/summary.controller'; import ConfigurationResource from './configuration/Configuration.resource'; import summaryTabs from './configuration/summary/summary-tabs.directive'; +import IgniteSummaryZipper from './configuration/summary/summary-zipper.service'; angular.module('ignite-console.states.configuration', ['ui.router']) .directive(...previewPanel) // Summary screen .directive(...summaryTabs) // Services. + .service('IgniteSummaryZipper', IgniteSummaryZipper) .service('IgniteConfigurationResource', ConfigurationResource) // Configure state provider. .config(['$stateProvider', 'AclRouteProvider', ($stateProvider, AclRoute) => { diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade index b34aba032f6ff..bcac5adcaf995 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade @@ -54,6 +54,6 @@ include /app/helpers/jade/mixins.jade -var required = nodeFilterKind + ' === "Custom"' +java-class('Class name:', customNodeFilter + '.className', '"customNodeFilter"', - 'true', required, 'Class name of custom node filter implementation') + 'true', required, 'Class name of custom node filter implementation', required) .col-sm-6 +preview-xml-java(model, 'cacheNodeFilter', 'igfss') diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade index 5062ce1ac9445..cfbaf12c57a8a 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade @@ -52,6 +52,9 @@ include /app/helpers/jade/mixins.jade .settings-row +number('Long query timeout:', model + '.longQueryWarningTimeout', '"longQueryWarningTimeout"', 'true', '3000', '0', 'Timeout in milliseconds after which long query warning will be printed') + .settings-row + +number('History size:', model + '.queryDetailMetricsSize', '"queryDetailMetricsSize"', 'true', '0', '0', + 'Size of queries detail metrics that will be stored in memory for monitoring purposes') .settings-row -var form = 'querySqlFunctionClasses'; -var sqlFunctionClasses = model + '.sqlFunctionClasses'; diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade index 1cf80b8a193f4..ea350f2394ef7 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade @@ -102,9 +102,9 @@ mixin hibernateField(name, model, items, valid, save, newItem) 'Parallel load cache minimum threshold.
          \ If 0 then load sequentially.') .details-row - +java-class('Hasher', pojoStoreFactory + '.hasher', '"pojoHasher"', 'true', 'false', 'Hash calculator') + +java-class('Hasher', pojoStoreFactory + '.hasher', '"pojoHasher"', 'true', 'false', 'Hash calculator', required) .details-row - +java-class('Transformer', pojoStoreFactory + '.transformer', '"pojoTransformer"', 'true', 'false', 'Types transformer') + +java-class('Transformer', pojoStoreFactory + '.transformer', '"pojoTransformer"', 'true', 'false', 'Types transformer', required) .details-row +checkbox('Escape table and filed names', pojoStoreFactory + '.sqlEscapeAll', '"sqlEscapeAll"', 'If enabled than all schema, table and field names will be escaped with double quotes (for example: "tableName"."fieldName").
          \ diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade index 5cc996d5ff311..259909eac9ed9 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade @@ -19,6 +19,7 @@ include /app/helpers/jade/mixins.jade -var form = 'checkpoint' -var model = 'backupItem.checkpointSpi' -var CustomCheckpoint = 'model.kind === "Custom"' +-var CacheCheckpoint = 'model.kind === "Cache"' .panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') @@ -44,7 +45,7 @@ include /app/helpers/jade/mixins.jade .group-content(ng-show='#{model} && #{model}.length > 0' ng-repeat='model in #{model} track by $index') hr(ng-if='$index != 0') .settings-row - +dropdown-required('Checkpoint SPI:', 'model.kind', '"checkpointKind" + $index', 'true', 'true', 'Choose checkpoint configuration variant', '[\ + +dropdown-required-autofocus('Checkpoint SPI:', 'model.kind', '"checkpointKind" + $index', 'true', 'true', 'Choose checkpoint configuration variant', '[\ {value: "FS", label: "File System"},\ {value: "Cache", label: "Cache"},\ {value: "S3", label: "Amazon S3"},\ @@ -64,13 +65,13 @@ include /app/helpers/jade/mixins.jade div(ng-show='model.kind === "FS"') include ./checkpoint/fs.jade - div(ng-show='model.kind === "Cache"') + div(ng-show=CacheCheckpoint) .settings-row - +dropdown-required-empty('Cache:', 'model.Cache.cache', '"checkpointCacheCache"+ $index', 'true', 'true', + +dropdown-required-empty('Cache:', 'model.Cache.cache', '"checkpointCacheCache"+ $index', 'true', CacheCheckpoint, 'Choose cache', 'No caches configured for current cluster', 'clusterCaches', 'Cache to use for storing checkpoints') .settings-row +java-class('Listener:', 'model.Cache.checkpointListener', '"checkpointCacheListener" + $index', 'true', 'false', - 'Checkpoint listener implementation class name') + 'Checkpoint listener implementation class name', CacheCheckpoint) div(ng-show='model.kind === "S3"') include ./checkpoint/s3.jade @@ -80,6 +81,6 @@ include /app/helpers/jade/mixins.jade .settings-row(ng-show=CustomCheckpoint) +java-class('Class name:', 'model.Custom.className', '"checkpointCustomClassName" + $index', 'true', CustomCheckpoint, - 'Custom CheckpointSpi implementation class') + 'Custom CheckpointSpi implementation class', CustomCheckpoint) .col-sm-6 +preview-xml-java('backupItem', 'clusterCheckpoint', 'caches') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade index efb6ad0699260..6ec4535dbdcf5 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade @@ -36,13 +36,13 @@ include /app/helpers/jade/mixins.jade -var valid = form + '[' + name + '].$valid' -var save = dirPaths + '[$index] = ' + model - div(ng-repeat='model in #{dirPaths} track by $index' ng-init='obj = {}') + div(ng-repeat='item in #{dirPaths} track by $index' ng-init='obj = {}') label.col-xs-12.col-sm-12.col-md-12 .indexField | {{ $index+1 }}) - +table-remove-conditional-button(dirPaths, 'true', 'Remove path') + +table-remove-conditional-button(dirPaths, 'true', 'Remove path', 'item') span(ng-hide='field.edit') - a.labelFormField(ng-click='(field.edit = true) && (#{model} = model)') {{ model }} + a.labelFormField(ng-click='(field.edit = true) && (#{model} = item)') {{ item }} span(ng-if='field.edit') +table-text-field(name, model, dirPaths, valid, save, 'Input directory path', false) +table-save-button(valid, save, false) @@ -63,4 +63,4 @@ include /app/helpers/jade/mixins.jade .settings-row +java-class('Listener:', 'model.FS.checkpointListener', '"checkpointFsListener" + $index', 'true', 'false', - 'Checkpoint listener implementation class name') + 'Checkpoint listener implementation class name', 'model.kind === "FS"') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade index 874799c17c0b3..5a13337f564f1 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade @@ -16,15 +16,17 @@ include /app/helpers/jade/mixins.jade +-var jdbcCheckpoint = 'model.kind === "JDBC"' + .settings-row - +text('Data source bean name:', 'model.JDBC.dataSourceBean', '"checkpointJdbcDataSourceBean" + $index', 'model.kind === "JDBC"', 'Input bean name', + +text('Data source bean name:', 'model.JDBC.dataSourceBean', '"checkpointJdbcDataSourceBean" + $index', jdbcCheckpoint, 'Input bean name', 'Name of the data source bean in Spring context') .settings-row - +dialect('Dialect:', 'model.JDBC.dialect', '"checkpointJdbcDialect" + $index', 'model.kind === "JDBC"', + +dialect('Dialect:', 'model.JDBC.dialect', '"checkpointJdbcDialect" + $index', jdbcCheckpoint, 'Dialect of SQL implemented by a particular RDBMS:', 'Generic JDBC dialect', 'Choose JDBC dialect') .settings-row +java-class('Listener:', 'model.JDBC.checkpointListener', '"checkpointJdbcListener" + $index', 'true', 'false', - 'Checkpoint listener implementation class name') + 'Checkpoint listener implementation class name', jdbcCheckpoint) +showHideLink('jdbcExpanded', 'settings') .details-row +text('User:', 'model.JDBC.user', '"checkpointJdbcUser" + $index', 'false', 'Input user name', 'Checkpoint jdbc user name') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade index da28da780a155..6531897130051 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade @@ -18,16 +18,17 @@ include /app/helpers/jade/mixins.jade -var credentialsModel = 'model.S3.awsCredentials' -var clientCfgModel = 'model.S3.clientConfiguration' --var checkpointS3Path = 'model.S3.awsCredentials.kind === "Properties"' --var checkpointS3Custom = 'model.S3.awsCredentials.kind === "Custom"' +-var checkpointS3 = 'model.kind === "S3"' +-var checkpointS3Path = checkpointS3 + ' && model.S3.awsCredentials.kind === "Properties"' +-var checkpointS3Custom = checkpointS3 + ' && model.S3.awsCredentials.kind === "Custom"' -var clientRetryModel = clientCfgModel + '.retryPolicy' --var checkpointS3DefaultMaxRetry = clientRetryModel + '.kind === "DefaultMaxRetries"' --var checkpointS3DynamoDbMaxRetry = clientRetryModel + '.kind === "DynamoDBMaxRetries"' --var checkpointS3CustomRetry = clientRetryModel + '.kind === "Custom"' +-var checkpointS3DefaultMaxRetry = checkpointS3 + ' && ' + clientRetryModel + '.kind === "DefaultMaxRetries"' +-var checkpointS3DynamoDbMaxRetry = checkpointS3 + ' && ' + clientRetryModel + '.kind === "DynamoDBMaxRetries"' +-var checkpointS3CustomRetry = checkpointS3 + ' && ' + clientRetryModel + '.kind === "Custom"' .settings-row - +dropdown-required('AWS credentials:', 'model.S3.awsCredentials.kind', '"checkpointS3AwsCredentials"', 'true', 'model.kind === "S3"', 'Custom', '[\ + +dropdown-required('AWS credentials:', 'model.S3.awsCredentials.kind', '"checkpointS3AwsCredentials"', 'true', checkpointS3, 'Custom', '[\ {value: "Basic", label: "Basic"},\ {value: "Properties", label: "Properties"},\ {value: "Anonymous", label: "Anonymous"},\ @@ -51,12 +52,12 @@ include /app/helpers/jade/mixins.jade .panel-details(ng-show=checkpointS3Custom) .details-row +java-class('Class name:', credentialsModel + '.Custom.className', '"checkpointS3CustomClassName" + $index', 'true', checkpointS3Custom, - 'Custom AWS credentials provider implementation class') + 'Custom AWS credentials provider implementation class', checkpointS3Custom) .settings-row +text('Bucket name suffix:', 'model.S3.bucketNameSuffix', '"checkpointS3BucketNameSuffix"', 'false', 'default-bucket', 'Bucket name suffix') .settings-row +java-class('Listener:', 'model.S3.checkpointListener', '"checkpointS3Listener" + $index', 'true', 'false', - 'Checkpoint listener implementation class name') + 'Checkpoint listener implementation class name', checkpointS3) +showHideLink('s3Expanded', 'client configuration') .details-row +dropdown('Protocol:', clientCfgModel + '.protocol', '"checkpointS3Protocol"', 'true', 'HTTPS', '[\ @@ -121,10 +122,10 @@ include /app/helpers/jade/mixins.jade .panel-details(ng-show=checkpointS3CustomRetry) .details-row +java-class('Retry condition:', clientRetryModel + '.Custom.retryCondition', '"checkpointS3CustomRetryPolicy" + $index', 'true', checkpointS3CustomRetry, - 'Retry condition on whether a specific request and exception should be retried') + 'Retry condition on whether a specific request and exception should be retried', checkpointS3CustomRetry) .details-row +java-class('Backoff strategy:', clientRetryModel + '.Custom.backoffStrategy', '"checkpointS3CustomBackoffStrategy" + $index', 'true', checkpointS3CustomRetry, - 'Back-off strategy for controlling how long the next retry should wait') + 'Back-off strategy for controlling how long the next retry should wait', checkpointS3CustomRetry) .details-row +number-required('Maximum retry attempts:', clientRetryModel + '.Custom.maxErrorRetry', '"checkpointS3CustomMaxErrorRetry"', 'true', checkpointS3CustomRetry, '-1', '1', 'Maximum number of retry attempts for failed requests') @@ -159,13 +160,13 @@ include /app/helpers/jade/mixins.jade 'Maximum amount of time that an idle connection may sit in the connection pool and still be eligible for reuse') .details-row +java-class('DNS resolver:', clientCfgModel + '.dnsResolver', '"checkpointS3DnsResolver" + $index', 'true', 'false', - 'DNS Resolver that should be used to for resolving AWS IP addresses') + 'DNS Resolver that should be used to for resolving AWS IP addresses', checkpointS3) .details-row +number('Response metadata cache size:', clientCfgModel + '.responseMetadataCacheSize', '"checkpointS3ResponseMetadataCacheSize"', 'true', '50', '0', 'Response metadata cache size') .details-row +java-class('SecureRandom class name:', clientCfgModel + '.secureRandom', '"checkpointS3SecureRandom" + $index', 'true', 'false', - 'SecureRandom to be used by the SDK class name') + 'SecureRandom to be used by the SDK class name', checkpointS3) .details-row +checkbox('Use reaper', clientCfgModel + '.useReaper', '"checkpointS3UseReaper"', 'Checks if the IdleConnectionReaper is to be started') .details-row diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade index 31a6be7e237a2..8e77ac42922c6 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade @@ -21,4 +21,4 @@ include /app/helpers/jade/mixins.jade div .details-row - +java-class('Class:', model + '.class', '"collisionCustom"', 'true', required, 'CollisionSpi implementation class') + +java-class('Class:', model + '.class', '"collisionCustom"', 'true', required, 'CollisionSpi implementation class', required) diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade index d4e537aff6d7f..dbe0478344492 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade @@ -37,7 +37,7 @@ div 'Node should attempt to steal jobs from other nodes') .details-row +java-class('External listener:', model + '.externalCollisionListener', '"jsExternalCollisionListener"', 'true', 'false', - 'Listener to be set for notification of external collision events') + 'Listener to be set for notification of external collision events', 'backupItem.collision.kind === "JobStealing"') .details-row +ignite-form-group ignite-form-field-label diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade index 4cfd9f52aef7e..aa99b491c0f1c 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade @@ -18,8 +18,14 @@ include /app/helpers/jade/mixins.jade -var form = 'deployment' -var model = 'backupItem' +-var modelDeployment = 'backupItem.deploymentSpi' -var exclude = model + '.peerClassLoadingLocalClassPathExclude' -var enabled = 'backupItem.peerClassLoadingEnabled' +-var uriListModel = modelDeployment + '.URI.uriList' +-var scannerModel = modelDeployment + '.URI.scanners' +-var uriDeployment = modelDeployment + '.kind === "URI"' +-var localDeployment = modelDeployment + '.kind === "Local"' +-var customDeployment = modelDeployment + '.kind === "Custom"' .panel.panel-default(ng-form=form novalidate) .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")') @@ -57,7 +63,7 @@ include /app/helpers/jade/mixins.jade .settings-row +number('Pool size:', model + '.peerClassLoadingThreadPoolSize', '"peerClassLoadingThreadPoolSize"', enabled, '2', '1', 'Thread pool size to use for peer class loading') .settings-row - +ignite-form-group(ng-model=exclude ng-form=form) + +ignite-form-group -var uniqueTip = 'Such package already exists' ignite-form-field-label @@ -81,7 +87,7 @@ include /app/helpers/jade/mixins.jade | {{ $index+1 }}) +table-remove-button(exclude, 'Remove package name') span(ng-hide='field.edit') - a.labelFormField(ng-click='#{enabled} && (field.edit = true) && (#{model} = model)') {{ model }} + a.labelFormField(ng-click='(field.edit = true) && (#{model} = model)') {{ model }} span(ng-if='field.edit') +table-java-package-field(name, model, exclude, valid, save, false) +table-save-button(valid, save, false) @@ -107,8 +113,125 @@ include /app/helpers/jade/mixins.jade +table-save-button(valid, save, true) +unique-feedback(name, uniqueTip) - .group-content-empty(ng-if='!(#{exclude}.length) && !group.add.length') | Not defined + .settings-row + +dropdown('Deployment variant:', modelDeployment + '.kind', '"deploymentKind"', 'true', 'Default', + '[\ + {value: "URI", label: "URI"},\ + {value: "Local", label: "Local"}, \ + {value: "Custom", label: "Custom"},\ + {value: undefined, label: "Default"}\ + ]', + 'Grid deployment SPI is in charge of deploying tasks and classes from different sources:\ +
            \ +
          • URI - Deploy tasks from different sources like file system folders, email and HTTP
          • \ +
          • Local - Only within VM deployment on local node
          • \ +
          • Custom - Custom implementation of DeploymentSpi
          • \ +
          • Default - Default configuration of LocalDeploymentSpi will be used
          • \ +
          ') + .panel-details(ng-show=uriDeployment) + .details-row + +ignite-form-group() + -var uniqueTip = 'Such URI already configured' + + ignite-form-field-label + | URI list + ignite-form-group-tooltip + | List of URI which point to GAR file and which should be scanned by SPI for the new tasks + ignite-form-group-add(ng-click='(group.add = [{}])') + | Add URI. + + .group-content(ng-if=uriListModel + '.length') + -var model = 'obj.model'; + -var name = '"edit" + $index' + -var valid = form + '[' + name + '].$valid' + -var save = uriListModel + '[$index] = ' + model + + div(ng-repeat='model in #{uriListModel} track by $index' ng-init='obj = {}') + label.col-xs-12.col-sm-12.col-md-12 + .indexField + | {{ $index+1 }}) + +table-remove-button(uriListModel, 'Remove URI') + span(ng-hide='field.edit') + a.labelFormField(ng-click='(field.edit = true) && (#{model} = model)') {{ model }} + span(ng-if='field.edit') + +table-url-field(name, model, uriListModel, valid, save, false) + +table-save-button(valid, save, false) + +unique-feedback(name, uniqueTip) + + .group-content(ng-repeat='field in group.add') + -var model = 'new'; + -var name = '"new"' + -var valid = form + '[' + name + '].$valid' + -var save = uriListModel + '.push(' + model + ')' + + div(type='internal' name='URI') + label.col-xs-12.col-sm-12.col-md-12 + +table-url-field(name, model, uriListModel, valid, save, true) + +table-save-button(valid, save, true) + +unique-feedback(name, uniqueTip) + + .group-content-empty(ng-if='!(#{uriListModel}.length) && !group.add.length') + | Not defined + .details-row + +text('Temporary directory path:', modelDeployment + '.URI.temporaryDirectoryPath', '"DeploymentURITemporaryDirectoryPath"', 'false', 'Temporary directory path', + 'Absolute path to temporary directory which will be used by deployment SPI to keep all deployed classes in') + .details-row + +ignite-form-group() + -var uniqueTip = 'Such scanner already configured' + + ignite-form-field-label + | Scanner list + ignite-form-group-tooltip + | List of URI deployment scanners + ignite-form-group-add(ng-click='(group.add = [{}])') + | Add scanner + + .group-content(ng-if=scannerModel + '.length') + -var model = 'obj.model'; + -var name = '"edit" + $index' + -var valid = form + '[' + name + '].$valid' + -var save = scannerModel + '[$index] = ' + model + + div(ng-repeat='model in #{scannerModel} track by $index' ng-init='obj = {}') + label.col-xs-12.col-sm-12.col-md-12 + .indexField + | {{ $index+1 }}) + +table-remove-button(scannerModel, 'Remove scanner') + span(ng-hide='field.edit') + a.labelFormField(ng-click='(field.edit = true) && (#{model} = model)') {{ model }} + span(ng-if='field.edit') + +table-java-class-field('Scanner:', name, model, scannerModel, valid, save, false) + +table-save-button(valid, save, false) + +unique-feedback(name, uniqueTip) + + .group-content(ng-repeat='field in group.add') + -var model = 'new'; + -var name = '"new"' + -var valid = form + '[' + name + '].$valid' + -var save = scannerModel + '.push(' + model + ')' + + div(type='internal' name='Scanner') + label.col-xs-12.col-sm-12.col-md-12 + // (lbl, name, model, items, valid, save, newItem) + +table-java-class-field('Scanner:', name, model, scannerModel, valid, save, true) + +table-save-button(valid, save, true) + +unique-feedback(name, uniqueTip) + + .group-content-empty(ng-if='!(#{scannerModel}.length) && !group.add.length') + | Not defined + .details-row + +java-class('Listener:', modelDeployment + '.URI.listener', '"DeploymentURIListener"', 'true', 'false', 'Deployment event listener', uriDeployment) + .details-row + +checkbox('Check MD5', modelDeployment + '.URI.checkMd5', '"DeploymentURICheckMd5"', 'Exclude files with same md5s from deployment') + .details-row + +checkbox('Encode URI', modelDeployment + '.URI.encodeUri', '"DeploymentURIEncodeUri"', 'URI must be encoded before usage') + .panel-details(ng-show=localDeployment) + .details-row + +java-class('Listener:', modelDeployment + '.Local.listener', '"DeploymentLocalListener"', 'true', 'false', 'Deployment event listener', localDeployment) + .panel-details(ng-show=customDeployment) + .details-row + +java-class('Class:', modelDeployment + '.Custom.className', '"DeploymentCustom"', 'true', customDeployment, 'DeploymentSpi implementation class', customDeployment) .col-sm-6 +preview-xml-java(model, 'clusterDeployment') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade index 3f2d6cb74ff12..643ea9703d230 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade @@ -59,10 +59,10 @@ include /app/helpers/jade/mixins.jade .settings-row +java-class('Filter:', modelEventStorage + '.Memory.filter', '"EventStorageFilter"', 'true', 'false', 'Filter for events to be recorded
          \ - Should be implementation of o.a.i.lang.IgnitePredicate<o.a.i.events.Event>') + Should be implementation of o.a.i.lang.IgnitePredicate<o.a.i.events.Event>', eventStorageMemory) .settings-row(ng-show=eventStorageCustom) - +java-class('Class:', modelEventStorage + '.Custom.className', '"EventStorageCustom"', 'true', eventStorageCustom, 'Event storage implementation class name') + +java-class('Class:', modelEventStorage + '.Custom.className', '"EventStorageCustom"', 'true', eventStorageCustom, 'Event storage implementation class name', eventStorageCustom) .col-sm-6 +preview-xml-java(model, 'clusterEvents') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade index aaed8e9a3b07e..16656598c2f99 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade @@ -45,7 +45,7 @@ include /app/helpers/jade/mixins.jade .group-content(ng-show='#{failoverSpi} && #{failoverSpi}.length > 0' ng-repeat='model in #{failoverSpi} track by $index') hr(ng-if='$index != 0') .settings-row - +dropdown-required('Failover SPI:', 'model.kind', '"failoverKind" + $index', 'true', 'true', 'Choose Failover SPI', '[\ + +dropdown-required-autofocus('Failover SPI:', 'model.kind', '"failoverKind" + $index', 'true', 'true', 'Choose Failover SPI', '[\ {value: "JobStealing", label: "Job stealing"},\ {value: "Never", label: "Never"},\ {value: "Always", label: "Always"},\ @@ -68,6 +68,6 @@ include /app/helpers/jade/mixins.jade 'Maximum number of attempts to execute a failed job on another node') .settings-row(ng-show=failoverCustom) +java-class('SPI implementation', 'model.Custom.class', '"failoverSpiClass" + $index', 'true', failoverCustom, - 'Custom FailoverSpi implementation class name.') + 'Custom FailoverSpi implementation class name.', failoverCustom) .col-sm-6 +preview-xml-java(model, 'clusterFailover') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade index 2e567ed509a67..48b1776db2c35 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade @@ -27,7 +27,7 @@ div +java-class('Curator:', model + '.curator', '"curator"', 'true', 'false', 'The Curator framework in use
          \ By default generates curator of org.apache.curator. framework.imps.CuratorFrameworkImpl\ - class with configured connect string, retry policy, and default session and connection timeouts') + class with configured connect string, retry policy, and default session and connection timeouts', required) .details-row +text('Connect string:', model + '.zkConnectionString', '"' + discoveryKind + 'ConnectionString"', required, 'host:port[chroot][,host:port[chroot]]', 'When "IGNITE_ZK_CONNECTION_STRING" system property is not configured this property will be used') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade index 5a03de84fe98d..5db89f5162e55 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade @@ -21,4 +21,4 @@ include /app/helpers/jade/mixins.jade -var required = 'backupItem.discovery.kind === "ZooKeeper" && backupItem.discovery.ZooKeeper.retryPolicy.kind === "Custom"' .details-row - +java-class('Class name:', retry + '.className', '"customClassName"', 'true', required, 'Custom retry policy implementation class name') + +java-class('Class name:', retry + '.className', '"customClassName"', 'true', required, 'Custom retry policy implementation class name', required) diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade index 7fd78bf266771..9fa9fc9300cf5 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade @@ -46,7 +46,7 @@ include /app/helpers/jade/mixins.jade .group-content(ng-show='#{loadBalancingSpi} && #{loadBalancingSpi}.length > 0' ng-repeat='model in #{loadBalancingSpi} track by $index') hr(ng-if='$index != 0') .settings-row - +dropdown-required('Load balancing:', 'model.kind', '"loadBalancingKind" + $index', 'true', 'true', 'Choose load balancing SPI', '[\ + +dropdown-required-autofocus('Load balancing:', 'model.kind', '"loadBalancingKind" + $index', 'true', 'true', 'Choose load balancing SPI', '[\ {value: "RoundRobin", label: "Round-robin"},\ {value: "Adaptive", label: "Adaptive"},\ {value: "WeightedRandom", label: "Random"},\ @@ -78,27 +78,30 @@ include /app/helpers/jade/mixins.jade
        • Default - Default load probing implementation
        • \ ') .settings-row(ng-show='model.kind === "Adaptive" && model.Adaptive.loadProbe.kind') - .panel-details - .details-row(ng-show='model.Adaptive.loadProbe.kind === "Job"') + .panel-details(ng-show='model.Adaptive.loadProbe.kind === "Job"') + .details-row +checkbox('Use average', 'model.Adaptive.loadProbe.Job.useAverage', '"loadBalancingAdaptiveJobUseAverage" + $index', 'Use average CPU load vs. current') - .details-row(ng-show='model.Adaptive.loadProbe.kind === "CPU"') + .panel-details(ng-show='model.Adaptive.loadProbe.kind === "CPU"') + .details-row +checkbox('Use average', 'model.Adaptive.loadProbe.CPU.useAverage', '"loadBalancingAdaptiveCPUUseAverage" + $index', 'Use average CPU load vs. current') - .details-row(ng-show='model.Adaptive.loadProbe.kind === "CPU"') + .details-row +checkbox('Use processors', 'model.Adaptive.loadProbe.CPU.useProcessors', '"loadBalancingAdaptiveCPUUseProcessors" + $index', "divide each node's CPU load by the number of processors on that node") - .details-row(ng-show='model.Adaptive.loadProbe.kind === "CPU"') + .details-row +number-min-max-step('Processor coefficient:', 'model.Adaptive.loadProbe.CPU.processorCoefficient', '"loadBalancingAdaptiveCPUProcessorCoefficient" + $index', 'true', '1', '0.001', '1', '0.05', 'Coefficient of every CPU') - .details-row(ng-show='model.Adaptive.loadProbe.kind === "ProcessingTime"') + .panel-details(ng-show='model.Adaptive.loadProbe.kind === "ProcessingTime"') + .details-row +checkbox('Use average', 'model.Adaptive.loadProbe.ProcessingTime.useAverage', '"loadBalancingAdaptiveJobUseAverage" + $index', 'Use average execution time vs. current') - .details-row(ng-show=loadProbeCustom) + .panel-details(ng-show=loadProbeCustom) + .details-row +java-class('Load brobe implementation:', 'model.Adaptive.loadProbe.Custom.className', '"loadBalancingAdaptiveJobUseClass" + $index', 'true', loadProbeCustom, - 'Custom load balancing SPI implementation class name.') + 'Custom load balancing SPI implementation class name.', loadProbeCustom) .settings-row(ng-show='model.kind === "WeightedRandom"') +number('Node weight:', 'model.WeightedRandom.nodeWeight', '"loadBalancingWRNodeWeight" + $index', 'true', 10, '1', 'Weight of node') .settings-row(ng-show='model.kind === "WeightedRandom"') +checkbox('Use weights', 'model.WeightedRandom.useWeights', '"loadBalancingWRUseWeights" + $index', 'Node weights should be checked when doing random load balancing') .settings-row(ng-show=loadBalancingCustom) +java-class('Load balancing SPI implementation:', 'model.Custom.className', '"loadBalancingClass" + $index', 'true', loadBalancingCustom, - 'Custom load balancing SPI implementation class name.') + 'Custom load balancing SPI implementation class name.', loadBalancingCustom) .col-sm-6 +preview-xml-java(model, 'clusterLoadBalancing') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade index 385d647568317..87d2b7d7e56b9 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade @@ -22,4 +22,4 @@ include /app/helpers/jade/mixins.jade div .details-row - +java-class('Class:', model + '.class', '"customLogger"', 'true', required, 'Logger implementation class name') + +java-class('Class:', model + '.class', '"customLogger"', 'true', required, 'Logger implementation class name', required) diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade index 85ec073bf61fd..fbd979c954c05 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade @@ -72,7 +72,7 @@ include /app/helpers/jade/mixins.jade label.col-xs-12.col-sm-12.col-md-12 .indexField | {{ $index+1 }}) - +table-remove-conditional-button(trust, enabled, 'Remove trust manager') + +table-remove-conditional-button(trust, enabled, 'Remove trust manager', 'model') span(ng-hide='field.edit') a.labelFormField(ng-click='#{enabled} && (field.edit = true) && (#{model} = model)') {{ model }} span(ng-if='field.edit') diff --git a/modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js similarity index 51% rename from modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js rename to modules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js index 18ba3baa2107f..08cfa71c43292 100644 --- a/modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js +++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js @@ -15,37 +15,23 @@ * limitations under the License. */ -export default class ScanFilter { - static $inject = ['$rootScope', '$q', '$modal']; +import Worker from 'worker?inline=true!./summary.worker'; - constructor($root, $q, $modal) { - this.deferred = null; - this.$q = $q; +export default ['$q', function($q) { + return function({ cluster, data }) { + const defer = $q.defer(); + const worker = new Worker(); - const scope = $root.$new(); + worker.postMessage({ cluster, data }); - scope.ui = {}; - - scope.ok = () => { - this.deferred.resolve({filter: scope.ui.filter, caseSensitive: !!scope.ui.caseSensitive}); - - this.modal.hide(); + worker.onmessage = (e) => { + defer.resolve(e.data); }; - scope.$hide = () => { - this.modal.hide(); - - this.deferred.reject(); + worker.onerror = (err) => { + defer.reject(err); }; - this.modal = $modal({templateUrl: '/scan-filter-input.html', scope, placement: 'center', show: false}); - } - - open() { - this.deferred = this.$q.defer(); - - this.modal.$promise.then(this.modal.show); - - return this.deferred.promise; - } -} + return defer.promise; + }; +}]; diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js index d739c4349a99b..cfc6df9c89538 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js +++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js @@ -16,15 +16,19 @@ */ import _ from 'lodash'; -import JSZip from 'jszip'; import saver from 'file-saver'; +const escapeFileName = (name) => name.replace(/[\\\/*\"\[\],\.:;|=<>?]/g, '-').replace(/ /g, '_'); + export default [ - '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteLoading', '$filter', 'IgniteConfigurationResource', 'JavaTypes', 'IgniteVersion', 'IgniteConfigurationGenerator', 'SpringTransformer', 'JavaTransformer', 'GeneratorDocker', 'GeneratorPom', 'IgnitePropertiesGenerator', 'IgniteReadmeGenerator', 'IgniteFormUtils', - function($root, $scope, $http, LegacyUtils, Messages, Loading, $filter, Resource, JavaTypes, Version, generator, spring, java, docker, pom, propsGenerator, readme, FormUtils) { + '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteLoading', '$filter', 'IgniteConfigurationResource', 'JavaTypes', 'IgniteVersion', 'IgniteConfigurationGenerator', 'SpringTransformer', 'JavaTransformer', 'IgniteDockerGenerator', 'IgniteMavenGenerator', 'IgnitePropertiesGenerator', 'IgniteReadmeGenerator', 'IgniteFormUtils', 'IgniteSummaryZipper', + function($root, $scope, $http, LegacyUtils, Messages, Loading, $filter, Resource, JavaTypes, Version, generator, spring, java, docker, pom, propsGenerator, readme, FormUtils, SummaryZipper) { const ctrl = this; - $scope.ui = { ready: false }; + $scope.ui = { + isSafari: !!(/constructor/i.test(window.HTMLElement) || window.safari), + ready: false + }; Loading.start('summaryPage'); @@ -223,10 +227,6 @@ export default [ return false; } - function escapeFileName(name) { - return name.replace(/[\\\/*\"\[\],\.:;|=<>?]/g, '-').replace(/ /g, '_'); - } - $scope.selectItem = (cluster) => { delete ctrl.cluster; @@ -297,84 +297,19 @@ export default [ // TODO IGNITE-2114: implemented as independent logic for download. $scope.downloadConfiguration = function() { - const cluster = $scope.cluster; - - const zip = new JSZip(); - - if (!ctrl.data) - ctrl.data = {}; - - if (!ctrl.data.docker) - ctrl.data.docker = docker.generate(cluster, 'latest'); - - zip.file('Dockerfile', ctrl.data.docker); - zip.file('.dockerignore', docker.ignoreFile()); - - const cfg = generator.igniteConfiguration(cluster, false); - const clientCfg = generator.igniteConfiguration(cluster, true); - const clientNearCaches = _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')); - - const secProps = propsGenerator.generate(cfg); - - if (secProps) - zip.file('src/main/resources/secret.properties', secProps); - - const srcPath = 'src/main/java'; - const resourcesPath = 'src/main/resources'; - - const serverXml = `${escapeFileName(cluster.name)}-server.xml`; - const clientXml = `${escapeFileName(cluster.name)}-client.xml`; - - const metaPath = `${resourcesPath}/META-INF`; - - zip.file(`${metaPath}/${serverXml}`, spring.igniteConfiguration(cfg).asString()); - zip.file(`${metaPath}/${clientXml}`, spring.igniteConfiguration(clientCfg, clientNearCaches).asString()); - - const cfgPath = `${srcPath}/config`; - - zip.file(`${cfgPath}/ServerConfigurationFactory.java`, java.igniteConfiguration(cfg, 'config', 'ServerConfigurationFactory').asString()); - zip.file(`${cfgPath}/ClientConfigurationFactory.java`, java.igniteConfiguration(clientCfg, 'config', 'ClientConfigurationFactory', clientNearCaches).asString()); - - if (java.isDemoConfigured(cluster, $root.IgniteDemoMode)) { - zip.file(`${srcPath}/demo/DemoStartup.java`, java.nodeStartup(cluster, 'demo.DemoStartup', - 'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory')); - } - - // Generate loader for caches with configured store. - const cachesToLoad = _.filter(cluster.caches, (cache) => _.nonNil(cache.cacheStoreFactory)); - - if (_.nonEmpty(cachesToLoad)) - zip.file(`${srcPath}/load/LoadCaches.java`, java.loadCaches(cachesToLoad, 'load', 'LoadCaches', `"${clientXml}"`)); - - const startupPath = `${srcPath}/startup`; - - zip.file(`${startupPath}/ServerNodeSpringStartup.java`, java.nodeStartup(cluster, 'startup.ServerNodeSpringStartup', `"${serverXml}"`)); - zip.file(`${startupPath}/ClientNodeSpringStartup.java`, java.nodeStartup(cluster, 'startup.ClientNodeSpringStartup', `"${clientXml}"`)); - - zip.file(`${startupPath}/ServerNodeCodeStartup.java`, java.nodeStartup(cluster, 'startup.ServerNodeCodeStartup', - 'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory')); - zip.file(`${startupPath}/ClientNodeCodeStartup.java`, java.nodeStartup(cluster, 'startup.ClientNodeCodeStartup', - 'ClientConfigurationFactory.createConfiguration()', 'config.ClientConfigurationFactory', clientNearCaches)); - - zip.file('pom.xml', pom.generate(cluster, Version.productVersion().ignite).asString()); - - zip.file('README.txt', readme.generate()); - zip.file('jdbc-drivers/README.txt', readme.generateJDBC()); - - if (_.isEmpty(ctrl.data.pojos)) - ctrl.data.pojos = java.pojos(cluster.caches); - - for (const pojo of ctrl.data.pojos) { - if (pojo.keyClass && JavaTypes.nonBuiltInClass(pojo.keyType)) - zip.file(`${srcPath}/${pojo.keyType.replace(/\./g, '/')}.java`, pojo.keyClass); + if ($scope.isPrepareDownloading) + return; - zip.file(`${srcPath}/${pojo.valueType.replace(/\./g, '/')}.java`, pojo.valueClass); - } + const cluster = $scope.cluster; - $generatorOptional.optionalContent(zip, cluster); + $scope.isPrepareDownloading = true; - zip.generateAsync({type: 'blob', compression: 'DEFLATE', mimeType: 'application/octet-stream'}) - .then((blob) => saver.saveAs(blob, escapeFileName(cluster.name) + '-project.zip')); + return new SummaryZipper({ cluster, data: ctrl.data || {}, IgniteDemoMode: $root.IgniteDemoMode }) + .then((data) => { + saver.saveAs(data, escapeFileName(cluster.name) + '-project.zip'); + }) + .catch((err) => Messages.showError('Failed to generate project files. ' + err.message)) + .then(() => $scope.isPrepareDownloading = false); }; /** @@ -393,7 +328,7 @@ export default [ const dialects = $scope.dialects; if (dialects.Oracle) - window.open('http://www.oracle.com/technetwork/apps-tech/jdbc-112010-090769.html'); + window.open('http://www.oracle.com/technetwork/database/features/jdbc/default-2280470.html'); if (dialects.DB2) window.open('http://www-01.ibm.com/support/docview.wss?uid=swg21363866'); diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js new file mode 100644 index 0000000000000..6b240017ab017 --- /dev/null +++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js @@ -0,0 +1,123 @@ +/* + * 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. + */ + +import JSZip from 'jszip'; + +import IgniteVersion from 'app/modules/configuration/Version.service'; + +import MavenGenerator from 'app/modules/configuration/generator/Maven.service'; +import DockerGenerator from 'app/modules/configuration/generator/Docker.service'; +import ReadmeGenerator from 'app/modules/configuration/generator/Readme.service'; +import PropertiesGenerator from 'app/modules/configuration/generator/Properties.service'; +import ConfigurationGenerator from 'app/modules/configuration/generator/ConfigurationGenerator'; + +import JavaTransformer from 'app/modules/configuration/generator/JavaTransformer.service'; +import SpringTransformer from 'app/modules/configuration/generator/SpringTransformer.service'; + +const Version = new IgniteVersion(); + +const maven = new MavenGenerator(); +const docker = new DockerGenerator(); +const readme = new ReadmeGenerator(); +const properties = new PropertiesGenerator(); + +const java = new JavaTransformer[0](); +const spring = new SpringTransformer[0](); + +const generator = new ConfigurationGenerator[0](); + +const escapeFileName = (name) => name.replace(/[\\\/*\"\[\],\.:;|=<>?]/g, '-').replace(/ /g, '_'); + +// eslint-disable-next-line no-undef +onmessage = function(e) { + const {cluster, data, demo} = e.data; + + const zip = new JSZip(); + + if (!data.docker) + data.docker = docker.generate(cluster, 'latest'); + + zip.file('Dockerfile', data.docker); + zip.file('.dockerignore', docker.ignoreFile()); + + const cfg = generator.igniteConfiguration(cluster, false); + const clientCfg = generator.igniteConfiguration(cluster, true); + const clientNearCaches = _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')); + + const secProps = properties.generate(cfg); + + if (secProps) + zip.file('src/main/resources/secret.properties', secProps); + + const srcPath = 'src/main/java'; + const resourcesPath = 'src/main/resources'; + + const serverXml = `${escapeFileName(cluster.name)}-server.xml`; + const clientXml = `${escapeFileName(cluster.name)}-client.xml`; + + const metaPath = `${resourcesPath}/META-INF`; + + zip.file(`${metaPath}/${serverXml}`, spring.igniteConfiguration(cfg).asString()); + zip.file(`${metaPath}/${clientXml}`, spring.igniteConfiguration(clientCfg, clientNearCaches).asString()); + + const cfgPath = `${srcPath}/config`; + + zip.file(`${cfgPath}/ServerConfigurationFactory.java`, java.igniteConfiguration(cfg, 'config', 'ServerConfigurationFactory').asString()); + zip.file(`${cfgPath}/ClientConfigurationFactory.java`, java.igniteConfiguration(clientCfg, 'config', 'ClientConfigurationFactory', clientNearCaches).asString()); + + if (java.isDemoConfigured(cluster, demo)) { + zip.file(`${srcPath}/demo/DemoStartup.java`, java.nodeStartup(cluster, 'demo.DemoStartup', + 'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory')); + } + + // Generate loader for caches with configured store. + const cachesToLoad = _.filter(cluster.caches, (cache) => _.nonNil(cache.cacheStoreFactory)); + + if (_.nonEmpty(cachesToLoad)) + zip.file(`${srcPath}/load/LoadCaches.java`, java.loadCaches(cachesToLoad, 'load', 'LoadCaches', `"${clientXml}"`)); + + const startupPath = `${srcPath}/startup`; + + zip.file(`${startupPath}/ServerNodeSpringStartup.java`, java.nodeStartup(cluster, 'startup.ServerNodeSpringStartup', `"${serverXml}"`)); + zip.file(`${startupPath}/ClientNodeSpringStartup.java`, java.nodeStartup(cluster, 'startup.ClientNodeSpringStartup', `"${clientXml}"`)); + + zip.file(`${startupPath}/ServerNodeCodeStartup.java`, java.nodeStartup(cluster, 'startup.ServerNodeCodeStartup', + 'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory')); + zip.file(`${startupPath}/ClientNodeCodeStartup.java`, java.nodeStartup(cluster, 'startup.ClientNodeCodeStartup', + 'ClientConfigurationFactory.createConfiguration()', 'config.ClientConfigurationFactory', clientNearCaches)); + + zip.file('pom.xml', maven.generate(cluster, Version.productVersion().ignite).asString()); + + zip.file('README.txt', readme.generate()); + zip.file('jdbc-drivers/README.txt', readme.generateJDBC()); + + if (_.isEmpty(data.pojos)) + data.pojos = java.pojos(cluster.caches); + + for (const pojo of data.pojos) { + if (pojo.keyClass) + zip.file(`${srcPath}/${pojo.keyType.replace(/\./g, '/')}.java`, pojo.keyClass); + + zip.file(`${srcPath}/${pojo.valueType.replace(/\./g, '/')}.java`, pojo.valueClass); + } + + zip.generateAsync({ + type: 'blob', + compression: 'DEFLATE', + mimeType: 'application/octet-stream' + }).then((blob) => postMessage(blob)); +}; diff --git a/modules/web-console/frontend/app/modules/user/Auth.service.js b/modules/web-console/frontend/app/modules/user/Auth.service.js index 43e2f9216071e..e0f905da8bf8b 100644 --- a/modules/web-console/frontend/app/modules/user/Auth.service.js +++ b/modules/web-console/frontend/app/modules/user/Auth.service.js @@ -20,12 +20,11 @@ export default ['Auth', ['$http', '$rootScope', '$state', '$window', 'IgniteErro return { forgotPassword(userInfo) { $http.post('/api/v1/password/forgot', userInfo) - .success(() => $state.go('password.send')) - .error((err) => ErrorPopover.show('forgot_email', Messages.errorMessage(null, err))); + .then(() => $state.go('password.send')) + .cacth(({data}) => ErrorPopover.show('forgot_email', Messages.errorMessage(null, data))); }, auth(action, userInfo) { $http.post('/api/v1/' + action, userInfo) - .catch(({data}) => Promise.reject(data)) .then(() => { if (action === 'password/forgot') return; @@ -41,16 +40,16 @@ export default ['Auth', ['$http', '$rootScope', '$state', '$window', 'IgniteErro $root.gettingStarted.tryShow(); }); }) - .catch((err) => ErrorPopover.show(action + '_email', Messages.errorMessage(null, err))); + .catch((res) => ErrorPopover.show(action + '_email', Messages.errorMessage(null, res))); }, logout() { $http.post('/api/v1/logout') - .success(() => { + .then(() => { User.clean(); $window.open($state.href('signin'), '_self'); }) - .error(Messages.showError); + .catch(Messages.showError); } }; }]]; diff --git a/modules/web-console/frontend/app/services/JavaTypes.service.js b/modules/web-console/frontend/app/services/JavaTypes.service.js index 679914f0c5b49..944fea5aa893a 100644 --- a/modules/web-console/frontend/app/services/JavaTypes.service.js +++ b/modules/web-console/frontend/app/services/JavaTypes.service.js @@ -40,7 +40,7 @@ const VALID_UUID = /^[0-9a-f]{8}-[0-9a-f]{4}-[1-5][0-9a-f]{3}-[89ab][0-9a-f]{3}- * Utility service for various check on java types. */ export default class JavaTypes { - static $inject = ['igniteClusterDefaults', 'igniteCacheDefaults', 'igniteIgfsDefaults']; + static $inject = ['IgniteClusterDefaults', 'IgniteCacheDefaults', 'IgniteIGFSDefaults']; constructor(clusterDflts, cacheDflts, igfsDflts) { this.enumClasses = _.uniq(this._enumClassesAcc(_.merge(clusterDflts, cacheDflts, igfsDflts), [])); @@ -101,14 +101,9 @@ export default class JavaTypes { * @return {String} Class name. */ shortClassName(clsName) { - if (this.isJavaPrimitive(clsName)) - return clsName; + const dotIdx = clsName.lastIndexOf('.'); - const fullClsName = this.fullClassName(clsName); - - const dotIdx = fullClsName.lastIndexOf('.'); - - return dotIdx > 0 ? fullClsName.substr(dotIdx + 1) : fullClsName; + return dotIdx > 0 ? clsName.substr(dotIdx + 1) : clsName; } /** @@ -163,7 +158,7 @@ export default class JavaTypes { * @param {String} clsName Class name to check. * @returns {boolean} 'true' if given class name is java primitive. */ - isJavaPrimitive(clsName) { + isPrimitive(clsName) { return _.includes(JAVA_PRIMITIVES, clsName); } diff --git a/modules/web-console/frontend/app/services/Messages.service.js b/modules/web-console/frontend/app/services/Messages.service.js index e679488e8066b..fefdae9a4378c 100644 --- a/modules/web-console/frontend/app/services/Messages.service.js +++ b/modules/web-console/frontend/app/services/Messages.service.js @@ -24,6 +24,9 @@ export default ['IgniteMessages', ['$alert', ($alert) => { prefix = prefix || ''; if (err) { + if (err.hasOwnProperty('data')) + err = err.data; + if (err.hasOwnProperty('message')) return prefix + err.message; @@ -38,26 +41,26 @@ export default ['IgniteMessages', ['$alert', ($alert) => { msgModal.hide(); }; - const _showMessage = (err, type, duration, icon) => { + const _showMessage = (message, err, type, duration) => { hideAlert(); - const title = errorMessage(null, err); + const title = err ? errorMessage(message, err) : errorMessage(null, message); msgModal = $alert({type, title, duration}); - msgModal.$scope.icon = icon; + msgModal.$scope.icon = `icon-${type}`; }; return { errorMessage, hideAlert, - showError(err) { - _showMessage(err, 'danger', 10, 'fa-exclamation-triangle'); + showError(message, err) { + _showMessage(message, err, 'danger', 10); return false; }, - showInfo(err) { - _showMessage(err, 'success', 3, 'fa-check-circle-o'); + showInfo(message) { + _showMessage(message, null, 'success', 3); } }; }]]; diff --git a/modules/web-console/frontend/controllers/admin-controller.js b/modules/web-console/frontend/controllers/admin-controller.js index 70043016bff1d..cf7fd71adeec0 100644 --- a/modules/web-console/frontend/controllers/admin-controller.js +++ b/modules/web-console/frontend/controllers/admin-controller.js @@ -15,79 +15,220 @@ * limitations under the License. */ +const ICON_SORT = ''; + +const CLUSTER_HEADER_TEMPLATE = `
          ${ICON_SORT}
          `; +const MODEL_HEADER_TEMPLATE = `
          ${ICON_SORT}
          `; +const CACHE_HEADER_TEMPLATE = `
          ${ICON_SORT}
          `; +const IGFS_HEADER_TEMPLATE = `
          ${ICON_SORT}
          `; + +const ACTIONS_TEMPLATE = ` +`; + +const EMAIL_TEMPLATE = ''; + // Controller for Admin screen. export default ['adminController', [ - '$rootScope', '$scope', '$http', '$q', '$state', 'IgniteMessages', 'IgniteConfirm', 'User', 'IgniteNotebookData', 'IgniteCountries', - ($rootScope, $scope, $http, $q, $state, Messages, Confirm, User, Notebook, Countries) => { + '$rootScope', '$scope', '$http', '$q', '$state', '$filter', 'uiGridConstants', 'IgniteMessages', 'IgniteConfirm', 'User', 'IgniteNotebookData', 'IgniteCountries', + ($rootScope, $scope, $http, $q, $state, $filter, uiGridConstants, Messages, Confirm, User, Notebook, Countries) => { $scope.users = null; - const _reloadUsers = () => { - $http.post('/api/v1/admin/list') - .success((users) => { - $scope.users = users; + const companySelectOptions = []; + const countrySelectOptions = []; - _.forEach($scope.users, (user) => { - user.userName = user.firstName + ' ' + user.lastName; - user.countryCode = Countries.getByName(user.country).code; - user.label = user.userName + ' ' + user.email + ' ' + - (user.company || '') + ' ' + (user.countryCode || ''); - }); - }) - .error(Messages.showError); - }; + const COLUMNS_DEFS = [ + {displayName: 'Actions', cellTemplate: ACTIONS_TEMPLATE, field: 'test', minWidth: 80, width: 80, enableFiltering: false, enableSorting: false}, + {displayName: 'User', field: 'userName', minWidth: 65, enableFiltering: true, filter: { placeholder: 'Filter by name...' }}, + {displayName: 'Email', field: 'email', cellTemplate: EMAIL_TEMPLATE, minWidth: 160, enableFiltering: true, filter: { placeholder: 'Filter by email...' }}, + {displayName: 'Company', field: 'company', minWidth: 160, filter: { + selectOptions: companySelectOptions, type: uiGridConstants.filter.SELECT, condition: uiGridConstants.filter.EXACT } + }, + {displayName: 'Country', field: 'countryCode', minWidth: 80, filter: { + selectOptions: countrySelectOptions, type: uiGridConstants.filter.SELECT, condition: uiGridConstants.filter.EXACT } + }, + {displayName: 'Last login', field: 'lastLogin', cellFilter: 'date:"medium"', minWidth: 175, width: 175, enableFiltering: false, sort: { direction: 'desc', priority: 0 }}, + {displayName: 'Clusters count', headerCellTemplate: CLUSTER_HEADER_TEMPLATE, field: '_clusters', type: 'number', headerTooltip: 'Clusters count', minWidth: 50, width: 50, enableFiltering: false}, + {displayName: 'Models count', headerCellTemplate: MODEL_HEADER_TEMPLATE, field: '_models', type: 'number', headerTooltip: 'Models count', minWidth: 50, width: 50, enableFiltering: false}, + {displayName: 'Caches count', headerCellTemplate: CACHE_HEADER_TEMPLATE, field: '_caches', type: 'number', headerTooltip: 'Caches count', minWidth: 50, width: 50, enableFiltering: false}, + {displayName: 'IGFS count', headerCellTemplate: IGFS_HEADER_TEMPLATE, field: '_igfs', type: 'number', headerTooltip: 'IGFS count', minWidth: 50, width: 50, enableFiltering: false} + ]; - _reloadUsers(); + const ctrl = $scope.ctrl = {}; - $scope.becomeUser = function(user) { + const becomeUser = function(user) { $http.get('/api/v1/admin/become', { params: {viewedUserId: user._id}}) - .catch(({data}) => Promise.reject(data)) .then(() => User.load()) - .then((becomeUser) => { - $rootScope.$broadcast('user', becomeUser); - - $state.go('base.configuration.clusters'); - }) + .then(() => $state.go('base.configuration.clusters')) .then(() => Notebook.load()) .catch(Messages.showError); }; - $scope.removeUser = (user) => { - Confirm.confirm('Are you sure you want to remove user: "' + user.userName + '"?') + const removeUser = (user) => { + Confirm.confirm(`Are you sure you want to remove user: "${user.userName}"?`) .then(() => { $http.post('/api/v1/admin/remove', {userId: user._id}) - .success(() => { + .then(() => { const i = _.findIndex($scope.users, (u) => u._id === user._id); if (i >= 0) $scope.users.splice(i, 1); - Messages.showInfo('User has been removed: "' + user.userName + '"'); + Messages.showInfo(`User has been removed: "${user.userName}"`); }) - .error((err, status) => { + .catch(({data, status}) => { if (status === 503) - Messages.showInfo(err); + Messages.showInfo(data); else - Messages.showError(Messages.errorMessage('Failed to remove user: ', err)); + Messages.showError('Failed to remove user: ', data); }); }); }; - $scope.toggleAdmin = (user) => { + const toggleAdmin = (user) => { if (user.adminChanging) return; user.adminChanging = true; $http.post('/api/v1/admin/save', {userId: user._id, adminFlag: !user.admin}) - .success(() => { + .then(() => { user.admin = !user.admin; - Messages.showInfo('Admin right was successfully toggled for user: "' + user.userName + '"'); + Messages.showInfo(`Admin right was successfully toggled for user: "${user.userName}"`); }) - .error((err) => { - Messages.showError(Messages.errorMessage('Failed to toggle admin right for user: ', err)); + .catch((res) => { + Messages.showError('Failed to toggle admin right for user: ', res); }) .finally(() => user.adminChanging = false); }; + + + ctrl.gridOptions = { + data: [], + columnVirtualizationThreshold: 30, + columnDefs: COLUMNS_DEFS, + categories: [ + {name: 'Actions', visible: true, selectable: true}, + {name: 'User', visible: true, selectable: true}, + {name: 'Email', visible: true, selectable: true}, + {name: 'Company', visible: true, selectable: true}, + {name: 'Country', visible: true, selectable: true}, + {name: 'Last login', visible: true, selectable: true}, + + {name: 'Clusters count', visible: true, selectable: true}, + {name: 'Models count', visible: true, selectable: true}, + {name: 'Caches count', visible: true, selectable: true}, + {name: 'IGFS count', visible: true, selectable: true} + ], + enableFiltering: true, + enableRowSelection: false, + enableRowHeaderSelection: false, + enableColumnMenus: false, + multiSelect: false, + modifierKeysToMultiSelect: true, + noUnselect: true, + flatEntityAccess: true, + fastWatch: true, + onRegisterApi: (api) => { + ctrl.gridApi = api; + + api.becomeUser = becomeUser; + api.removeUser = removeUser; + api.toggleAdmin = toggleAdmin; + } + }; + + /** + * Set grid height. + * + * @param {Number} rows Rows count. + * @private + */ + const adjustHeight = (rows) => { + const height = Math.min(rows, 20) * 30 + 75; + + // Remove header height. + ctrl.gridApi.grid.element.css('height', height + 'px'); + + ctrl.gridApi.core.handleWindowResize(); + }; + + const usersToFilterOptions = (column) => { + return _.sortBy( + _.map( + _.groupBy($scope.users, (usr) => { + const fld = usr[column]; + + return _.isNil(fld) ? fld : fld.toUpperCase(); + }), + (arr, value) => ({label: `${_.head(arr)[column] || 'Not set'} (${arr.length})`, value}) + ), + 'value'); + }; + + const _reloadUsers = () => { + $http.post('/api/v1/admin/list') + .then(({ data }) => { + $scope.users = data; + + companySelectOptions.length = 0; + countrySelectOptions.length = 0; + + _.forEach($scope.users, (user) => { + user.userName = user.firstName + ' ' + user.lastName; + user.countryCode = Countries.getByName(user.country).code; + + user._clusters = user.counters.clusters; + user._models = user.counters.models; + user._caches = user.counters.caches; + user._igfs = user.counters.igfs; + }); + + companySelectOptions.push(...usersToFilterOptions('company')); + countrySelectOptions.push(...usersToFilterOptions('countryCode')); + + $scope.ctrl.gridOptions.data = data; + + adjustHeight(data.length); + }) + .catch(Messages.showError); + }; + + _reloadUsers(); + + const _enableColumns = (categories, visible) => { + _.forEach(categories, (cat) => { + cat.visible = visible; + + _.forEach(ctrl.gridOptions.columnDefs, (col) => { + if (col.displayName === cat.name) + col.visible = visible; + }); + }); + + ctrl.gridApi.grid.refresh(); + }; + + const _selectableColumns = () => _.filter(ctrl.gridOptions.categories, (cat) => cat.selectable); + + ctrl.toggleColumns = (category, visible) => _enableColumns([category], visible); + ctrl.selectAllColumns = () => _enableColumns(_selectableColumns(), true); + ctrl.clearAllColumns = () => _enableColumns(_selectableColumns(), false); } ]]; diff --git a/modules/web-console/frontend/controllers/caches-controller.js b/modules/web-console/frontend/controllers/caches-controller.js index 8c01173ae509c..e7521b55ad403 100644 --- a/modules/web-console/frontend/controllers/caches-controller.js +++ b/modules/web-console/frontend/controllers/caches-controller.js @@ -467,14 +467,14 @@ export default ['cachesController', [ // Save cache in database. function save(item) { $http.post('/api/v1/configuration/caches/save', item) - .success(function(_id) { + .then(({data}) => { + const _id = data; + item.label = _cacheLbl(item); $scope.ui.inputForm.$setPristine(); - const idx = _.findIndex($scope.caches, function(cache) { - return cache._id === _id; - }); + const idx = _.findIndex($scope.caches, {_id}); if (idx >= 0) _.assign($scope.caches[idx], item); @@ -487,21 +487,21 @@ export default ['cachesController', [ if (_.includes(item.clusters, cluster.value)) cluster.caches = _.union(cluster.caches, [_id]); else - _.remove(cluster.caches, (id) => id === _id); + _.pull(cluster.caches, _id); }); _.forEach($scope.domains, (domain) => { if (_.includes(item.domains, domain.value)) domain.meta.caches = _.union(domain.meta.caches, [_id]); else - _.remove(domain.meta.caches, (id) => id === _id); + _.pull(domain.meta.caches, _id); }); $scope.selectItem(item); Messages.showInfo('Cache "' + item.name + '" saved.'); }) - .error(Messages.showError); + .catch(Messages.showError); } // Save cache. @@ -559,7 +559,7 @@ export default ['cachesController', [ const _id = selectedItem._id; $http.post('/api/v1/configuration/caches/remove', {_id}) - .success(function() { + .then(() => { Messages.showInfo('Cache has been removed: ' + selectedItem.name); const caches = $scope.caches; @@ -582,7 +582,7 @@ export default ['cachesController', [ _.forEach($scope.domains, (domain) => _.remove(domain.meta.caches, (id) => id === _id)); } }) - .error(Messages.showError); + .catch(Messages.showError); }); }; @@ -591,7 +591,7 @@ export default ['cachesController', [ Confirm.confirm('Are you sure you want to remove all caches?') .then(function() { $http.post('/api/v1/configuration/caches/remove/all') - .success(function() { + .then(() => { Messages.showInfo('All caches have been removed'); $scope.caches = []; @@ -603,7 +603,7 @@ export default ['cachesController', [ $scope.ui.inputForm.$error = {}; $scope.ui.inputForm.$setPristine(); }) - .error(Messages.showError); + .catch(Messages.showError); }); }; diff --git a/modules/web-console/frontend/controllers/clusters-controller.js b/modules/web-console/frontend/controllers/clusters-controller.js index f92a2f1e100fa..7f90b90e198fb 100644 --- a/modules/web-console/frontend/controllers/clusters-controller.js +++ b/modules/web-console/frontend/controllers/clusters-controller.js @@ -17,7 +17,7 @@ // Controller for Clusters screen. export default ['clustersController', [ - '$rootScope', '$scope', '$http', '$state', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'igniteEventGroups', 'DemoInfo', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', + '$rootScope', '$scope', '$http', '$state', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteEventGroups', 'DemoInfo', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', function($root, $scope, $http, $state, $timeout, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, igniteEventGroups, DemoInfo, LegacyTable, Resource, ErrorPopover, FormUtils) { UnsavedChangesGuard.install($scope); @@ -31,6 +31,12 @@ export default ['clustersController', [ cacheKeyConfiguration: [], communication: {}, connector: {}, + deploymentSpi: { + URI: { + uriList: [], + scanners: [] + } + }, discovery: { Cloud: { regions: [], @@ -38,6 +44,7 @@ export default ['clustersController', [ } }, marshaller: {}, + peerClassLoadingLocalClassPathExclude: [], sslContextFactory: { trustManagers: [] }, @@ -276,6 +283,16 @@ export default ['clustersController', [ if (!cluster.eventStorage) cluster.eventStorage = { kind: 'Memory' }; + + if (!cluster.peerClassLoadingLocalClassPathExclude) + cluster.peerClassLoadingLocalClassPathExclude = []; + + if (!cluster.deploymentSpi) { + cluster.deploymentSpi = {URI: { + uriList: [], + scanners: [] + }}; + } }); if ($state.params.linkId) @@ -699,17 +716,20 @@ export default ['clustersController', [ // Save cluster in database. function save(item) { $http.post('/api/v1/configuration/clusters/save', item) - .success(function(_id) { + .then(({data}) => { + const _id = data; + item.label = _clusterLbl(item); $scope.ui.inputForm.$setPristine(); - const idx = _.findIndex($scope.clusters, (cluster) => cluster._id === _id); + const idx = _.findIndex($scope.clusters, {_id}); if (idx >= 0) _.assign($scope.clusters[idx], item); else { item._id = _id; + $scope.clusters.push(item); } @@ -717,21 +737,21 @@ export default ['clustersController', [ if (_.includes(item.caches, cache.value)) cache.cache.clusters = _.union(cache.cache.clusters, [_id]); else - _.remove(cache.cache.clusters, (id) => id === _id); + _.pull(cache.cache.clusters, _id); }); _.forEach($scope.igfss, (igfs) => { if (_.includes(item.igfss, igfs.value)) igfs.igfs.clusters = _.union(igfs.igfs.clusters, [_id]); else - _.remove(igfs.igfs.clusters, (id) => id === _id); + _.pull(igfs.igfs.clusters, _id); }); $scope.selectItem(item); - Messages.showInfo('Cluster "' + item.name + '" saved.'); + Messages.showInfo(`Cluster "${item.name}" saved.`); }) - .error(Messages.showError); + .catch(Messages.showError); } // Save cluster. @@ -774,7 +794,7 @@ export default ['clustersController', [ const _id = selectedItem._id; $http.post('/api/v1/configuration/clusters/remove', {_id}) - .success(function() { + .then(() => { Messages.showInfo('Cluster has been removed: ' + selectedItem.name); const clusters = $scope.clusters; @@ -795,7 +815,7 @@ export default ['clustersController', [ _.forEach($scope.igfss, (igfs) => _.remove(igfs.igfs.clusters, (id) => id === _id)); } }) - .error(Messages.showError); + .catch(Messages.showError); }); }; @@ -804,7 +824,7 @@ export default ['clustersController', [ Confirm.confirm('Are you sure you want to remove all clusters?') .then(function() { $http.post('/api/v1/configuration/clusters/remove/all') - .success(() => { + .then(() => { Messages.showInfo('All clusters have been removed'); $scope.clusters = []; @@ -816,7 +836,7 @@ export default ['clustersController', [ $scope.ui.inputForm.$error = {}; $scope.ui.inputForm.$setPristine(); }) - .error(Messages.showError); + .catch(Messages.showError); }); }; diff --git a/modules/web-console/frontend/controllers/domains-controller.js b/modules/web-console/frontend/controllers/domains-controller.js index 2d7b875e36f5b..303110e086696 100644 --- a/modules/web-console/frontend/controllers/domains-controller.js +++ b/modules/web-console/frontend/controllers/domains-controller.js @@ -756,15 +756,15 @@ export default ['domainsController', [ Loading.start('importDomainFromDb'); $http.post('/api/v1/configuration/domains/save/batch', batch) - .success(function(savedBatch) { + .then(({data}) => { let lastItem; const newItems = []; - _.forEach(_mapCaches(savedBatch.generatedCaches), function(cache) { + _.forEach(_mapCaches(data.generatedCaches), function(cache) { $scope.caches.push(cache); }); - _.forEach(savedBatch.savedDomains, function(savedItem) { + _.forEach(data.savedDomains, function(savedItem) { const idx = _.findIndex($scope.domains, function(domain) { return domain._id === savedItem._id; }); @@ -792,7 +792,7 @@ export default ['domainsController', [ $scope.ui.showValid = true; }) - .error(Messages.showError) + .catch(Messages.showError) .finally(() => { Loading.finish('importDomainFromDb'); @@ -1382,10 +1382,10 @@ export default ['domainsController', [ item.kind = 'store'; $http.post('/api/v1/configuration/domains/save', item) - .success(function(res) { + .then(({data}) => { $scope.ui.inputForm.$setPristine(); - const savedMeta = res.savedDomains[0]; + const savedMeta = data.savedDomains[0]; const idx = _.findIndex($scope.domains, function(domain) { return domain._id === savedMeta._id; @@ -1400,16 +1400,16 @@ export default ['domainsController', [ if (_.includes(item.caches, cache.value)) cache.cache.domains = _.union(cache.cache.domains, [savedMeta._id]); else - _.remove(cache.cache.domains, (id) => id === savedMeta._id); + _.pull(cache.cache.domains, savedMeta._id); }); $scope.selectItem(savedMeta); - Messages.showInfo('Domain model "' + item.valueType + '" saved.'); + Messages.showInfo(`Domain model "${item.valueType}" saved.`); _checkShowValidPresentation(); }) - .error(Messages.showError); + .catch(Messages.showError); } // Save domain model. @@ -1469,14 +1469,12 @@ export default ['domainsController', [ const _id = selectedItem._id; $http.post('/api/v1/configuration/domains/remove', {_id}) - .success(function() { + .then(() => { Messages.showInfo('Domain model has been removed: ' + selectedItem.valueType); const domains = $scope.domains; - const idx = _.findIndex(domains, function(domain) { - return domain._id === _id; - }); + const idx = _.findIndex(domains, {_id}); if (idx >= 0) { domains.splice(idx, 1); @@ -1488,12 +1486,12 @@ export default ['domainsController', [ else $scope.backupItem = emptyDomain; - _.forEach($scope.caches, (cache) => _.remove(cache.cache.domains, (id) => id === _id)); + _.forEach($scope.caches, (cache) => _.pull(cache.cache.domains, _id)); } _checkShowValidPresentation(); }) - .error(Messages.showError); + .catch(Messages.showError); }); }; @@ -1504,7 +1502,7 @@ export default ['domainsController', [ Confirm.confirm('Are you sure you want to remove all domain models?') .then(function() { $http.post('/api/v1/configuration/domains/remove/all') - .success(function() { + .then(() => { Messages.showInfo('All domain models have been removed'); $scope.domains = []; @@ -1516,7 +1514,7 @@ export default ['domainsController', [ $scope.ui.inputForm.$error = {}; $scope.ui.inputForm.$setPristine(); }) - .error(Messages.showError); + .catch(Messages.showError); }); }; diff --git a/modules/web-console/frontend/controllers/igfs-controller.js b/modules/web-console/frontend/controllers/igfs-controller.js index e505f1c450948..b3c6043ce32b6 100644 --- a/modules/web-console/frontend/controllers/igfs-controller.js +++ b/modules/web-console/frontend/controllers/igfs-controller.js @@ -296,12 +296,12 @@ export default ['igfsController', [ // Save IGFS in database. function save(item) { $http.post('/api/v1/configuration/igfs/save', item) - .success(function(_id) { + .then(({data}) => { + const _id = data; + $scope.ui.inputForm.$setPristine(); - const idx = _.findIndex($scope.igfss, function(igfs) { - return igfs._id === _id; - }); + const idx = _.findIndex($scope.igfss, {_id}); if (idx >= 0) _.assign($scope.igfss[idx], item); @@ -312,9 +312,9 @@ export default ['igfsController', [ $scope.selectItem(item); - Messages.showInfo('IGFS "' + item.name + '" saved.'); + Messages.showInfo(`IGFS "${item.name}" saved.`); }) - .error(Messages.showError); + .catch(Messages.showError); } // Save IGFS. @@ -359,7 +359,7 @@ export default ['igfsController', [ const _id = selectedItem._id; $http.post('/api/v1/configuration/igfs/remove', {_id}) - .success(function() { + .then(() => { Messages.showInfo('IGFS has been removed: ' + selectedItem.name); const igfss = $scope.igfss; @@ -379,7 +379,7 @@ export default ['igfsController', [ $scope.backupItem = emptyIgfs; } }) - .error(Messages.showError); + .catch(Messages.showError); }); }; @@ -390,7 +390,7 @@ export default ['igfsController', [ Confirm.confirm('Are you sure you want to remove all IGFS?') .then(function() { $http.post('/api/v1/configuration/igfs/remove/all') - .success(function() { + .then(() => { Messages.showInfo('All IGFS have been removed'); $scope.igfss = []; @@ -398,7 +398,7 @@ export default ['igfsController', [ $scope.ui.inputForm.$error = {}; $scope.ui.inputForm.$setPristine(); }) - .error(Messages.showError); + .catch(Messages.showError); }); }; diff --git a/modules/web-console/frontend/controllers/profile-controller.js b/modules/web-console/frontend/controllers/profile-controller.js index fd595d9df8c66..87a880596c4bd 100644 --- a/modules/web-console/frontend/controllers/profile-controller.js +++ b/modules/web-console/frontend/controllers/profile-controller.js @@ -74,7 +74,6 @@ export default ['profileController', [ $scope.saveUser = () => { $http.post('/api/v1/profile/save', $scope.user) - .catch(({data}) => Promise.reject(data)) .then(User.load) .then(() => { if ($scope.expandedPassword) @@ -89,7 +88,7 @@ export default ['profileController', [ $root.$broadcast('user', $scope.user); }) - .catch((err) => Messages.showError(Messages.errorMessage('Failed to save profile: ', err))); + .catch((res) => Messages.showError('Failed to save profile: ', res)); }; } ]]; diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js index 2463d246a4446..7360ac4c0da3c 100644 --- a/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js +++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js @@ -20,7 +20,7 @@ import fs from 'fs'; import webpack from 'webpack'; import autoprefixer from 'autoprefixer-core'; import jade from 'jade'; -import progressPlugin from './plugins/progress'; +import ProgressBarPlugin from 'progress-bar-webpack-plugin'; import eslintFormatter from 'eslint-friendly-formatter'; import ExtractTextPlugin from 'extract-text-webpack-plugin'; @@ -61,7 +61,6 @@ export default () => { // Output system. output: { path: destDir, - publicPath: './', filename: '[name].js' }, @@ -111,8 +110,10 @@ export default () => { loader: 'babel-loader', query: { cacheDirectory: true, - plugins: ['transform-runtime', - 'add-module-exports'], + plugins: [ + 'transform-runtime', + 'add-module-exports' + ], presets: ['angular'] } @@ -126,10 +127,8 @@ export default () => { loader: development ? `style-loader!${stylesLoader}` : ExtractTextPlugin.extract('style-loader', stylesLoader) }, { - test: /\.(woff2|woff|ttf|eot|svg)?(\?v=[0-9]\.[0-9]\.[0-9])?$/, - loaders: [ - `${assetsLoader}?name=assets/fonts/[name].[ext]` - ] + test: /\.(ttf|eot|svg|woff(2)?)(\?v=[\d.]+)?(\?[a-z0-9#-]+)?$/, + loaders: [`${assetsLoader}?name=assets/fonts/[name].[ext]`] }, { test: /\.(jpe?g|png|gif)$/i, @@ -186,7 +185,7 @@ export default () => { }, favicon }), - progressPlugin + new ProgressBarPlugin() ] }; }; diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js index cad913354d7f0..34e1f6ae858b0 100644 --- a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js +++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js @@ -20,9 +20,8 @@ import webpack from 'webpack'; import {destDir, rootDir, srcDir} from '../../paths'; -const devServerHost = 'localhost'; +const backendPort = 3000; const devServerPort = 9000; -const devServerUrl = `http://${devServerHost}:${devServerPort}/`; export default () => { const plugins = [ @@ -31,11 +30,10 @@ export default () => { return { entry: { - webpack: `webpack-dev-server/client?${devServerUrl}`, app: [path.join(srcDir, 'app.js'), 'webpack/hot/only-dev-server'] }, output: { - publicPath: devServerUrl + publicPath: `http://localhost:${devServerPort}/` }, context: rootDir, debug: true, @@ -44,24 +42,22 @@ export default () => { devServer: { compress: true, historyApiFallback: true, - publicPath: '/', contentBase: destDir, - info: true, hot: true, inline: true, proxy: { '/socket.io': { - target: 'http://localhost:3000', + target: `http://localhost:${backendPort}`, changeOrigin: true, ws: true }, '/agents': { - target: 'http://localhost:3000', + target: `http://localhost:${backendPort}`, changeOrigin: true, ws: true }, '/api/v1/*': { - target: 'http://localhost:3000', + target: `http://localhost:${backendPort}`, changeOrigin: true, pathRewrite: { '^/api/v1': '' diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js index db667203ac722..1194568f79217 100644 --- a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js +++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js @@ -37,8 +37,7 @@ export default () => { devtool: 'cheap-source-map', output: { publicPath: '/', - filename: '[name].[chunkhash].js', - path: destDir + filename: '[name].[chunkhash].js' }, plugins }; diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/plugins/progress.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/plugins/progress.js deleted file mode 100644 index 5f753c722c592..0000000000000 --- a/modules/web-console/frontend/gulpfile.babel.js/webpack/plugins/progress.js +++ /dev/null @@ -1,82 +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. - */ - -import ProgressPlugin from 'webpack/lib/ProgressPlugin'; - -let chars = 0; -let lastState = 0; -let lastStateTime = 0; - -const outputStream = process.stdout; - -const _goToLineStart = (nextMessage) => { - let str = ''; - - for (; chars > nextMessage.length; chars--) - str += '\b \b'; - - chars = nextMessage.length; - - for (let i = 0; i < chars; i++) - str += '\b'; - - if (str) - outputStream.write(str); -}; - -export default new ProgressPlugin((percentage, msg) => { - let state = msg; - - if (percentage < 1) { - percentage = Math.floor(percentage * 100); - - msg = percentage + '% ' + msg; - - if (percentage < 100) - msg = ' ' + msg; - - if (percentage < 10) - msg = ' ' + msg; - } - - state = state.replace(/^\d+\/\d+\s+/, ''); - - if (percentage === 0) { - lastState = null; - lastStateTime = (new Date()).getTime(); - } - else if (state !== lastState || percentage === 1) { - const now = (new Date()).getTime(); - - if (lastState) { - const stateMsg = (now - lastStateTime) + 'ms ' + lastState; - - _goToLineStart(stateMsg); - - outputStream.write(stateMsg + '\n'); - - chars = 0; - } - - lastState = state; - lastStateTime = now; - } - - _goToLineStart(msg); - - outputStream.write(msg); -}); diff --git a/modules/web-console/frontend/package.json b/modules/web-console/frontend/package.json index b511ca1e9123f..fd50d5b6e7552 100644 --- a/modules/web-console/frontend/package.json +++ b/modules/web-console/frontend/package.json @@ -29,97 +29,99 @@ "win32" ], "dependencies": { - "angular": "^1.5.5", - "angular-acl": "^0.1.7", - "angular-animate": "^1.5.5", - "angular-aria": "^1.5.5", - "angular-cookies": "^1.5.5", - "angular-drag-and-drop-lists": "^1.4.0", - "angular-gridster": "^0.13.3", - "angular-motion": "^0.4.4", - "angular-nvd3": "^1.0.7", - "angular-retina": "^0.3.13", - "angular-sanitize": "^1.5.5", - "angular-smart-table": "^2.1.8", - "angular-socket-io": "^0.7.0", - "angular-strap": "^2.3.8", - "angular-touch": "^1.5.5", - "angular-tree-control": "^0.2.26", - "angular-ui-grid": "^3.1.1", - "angular-ui-router": "^0.3.1", - "bootstrap-sass": "^3.3.6", - "brace": "^0.8.0", - "es6-promise": "^3.0.2", - "file-saver": "^1.3.2", - "font-awesome": "^4.6.3", - "glob": "^7.0.3", - "jquery": "^3.0.0", - "jszip": "^3.0.0", - "lodash": "^4.8.2", - "nvd3": "^1.8.3", - "raleway-webfont": "^3.0.1", - "roboto-font": "^0.1.0", - "socket.io-client": "^1.4.6", - "ui-router-metatags": "^1.0.3" + "angular": "~1.5.9", + "angular-acl": "~0.1.7", + "angular-animate": "~1.5.9", + "angular-aria": "~1.5.9", + "angular-cookies": "~1.5.9", + "angular-drag-and-drop-lists": "~1.4.0", + "angular-gridster": "~0.13.3", + "angular-motion": "~0.4.4", + "angular-nvd3": "~1.0.9", + "angular-retina": "~0.3.13", + "angular-sanitize": "~1.5.9", + "angular-smart-table": "~2.1.8", + "angular-socket-io": "~0.7.0", + "angular-strap": "~2.3.8", + "angular-touch": "~1.5.9", + "angular-tree-control": "~0.2.26", + "angular-ui-grid": "~3.2.9", + "angular-ui-router": "~0.3.1", + "bootstrap-sass": "~3.3.6", + "brace": "~0.8.0", + "es6-promise": "~3.3.1", + "file-saver": "~1.3.2", + "font-awesome": "~4.7.0", + "glob": "~7.1.1", + "jquery": "~3.1.1", + "jszip": "~3.1.3", + "lodash": "~4.17.2", + "nvd3": "1.8.4", + "raleway-webfont": "~3.0.1", + "roboto-font": "~0.1.0", + "socket.io-client": "~1.7.2", + "ui-router-metatags": "~1.0.3" }, "devDependencies": { - "assets-webpack-plugin": "^3.2.0", - "autoprefixer-core": "^6.0.1", - "babel-core": "^6.7.6", - "babel-eslint": "^7.0.0", - "babel-loader": "^6.2.4", - "babel-plugin-add-module-exports": "^0.2.1", - "babel-plugin-transform-builtin-extend": "^1.1.0", - "babel-plugin-transform-runtime": "^6.7.5", - "babel-polyfill": "^6.7.4", - "babel-preset-angular": "^6.0.15", - "babel-preset-es2015": "^6.9.0", - "babel-runtime": "^6.6.1", - "chai": "^3.5.0", - "cross-env": "^1.0.7", - "css-loader": "^0.23.0", - "eslint": "^3.0.0", - "eslint-friendly-formatter": "^2.0.5", - "eslint-loader": "^1.0.0", - "expose-loader": "^0.7.1", - "extract-text-webpack-plugin": "^1.0.1", - "file-loader": "^0.9.0", - "gulp": "^3.9.1", - "gulp-eslint": "^3.0.0", - "gulp-inject": "^4.0.0", - "gulp-jade": "^1.1.0", - "gulp-ll": "^1.0.4", - "gulp-rimraf": "^0.2.0", - "gulp-sequence": "^0.4.1", - "gulp-util": "^3.0.7", - "html-loader": "^0.4.3", - "html-webpack-plugin": "^2.21.0", - "jade": "^1.11.0", + "assets-webpack-plugin": "~3.5.0", + "autoprefixer-core": "~6.0.1", + "babel-core": "~6.20.0", + "babel-eslint": "~7.0.0", + "babel-loader": "~6.2.4", + "babel-plugin-add-module-exports": "~0.2.1", + "babel-plugin-transform-builtin-extend": "~1.1.0", + "babel-plugin-transform-runtime": "~6.15.0", + "babel-polyfill": "~6.20.0", + "babel-preset-angular": "~6.0.15", + "babel-preset-es2015": "~6.18.0", + "babel-runtime": "~6.20.0", + "chai": "~3.5.0", + "cross-env": "~1.0.7", + "css-loader": "~0.23.0", + "eslint": "~3.12.2", + "eslint-friendly-formatter": "~2.0.5", + "eslint-loader": "~1.6.1", + "expose-loader": "~0.7.1", + "extract-text-webpack-plugin": "~1.0.1", + "file-loader": "~0.9.0", + "gulp": "~3.9.1", + "gulp-eslint": "~3.0.0", + "gulp-inject": "~4.1.0", + "gulp-jade": "~1.1.0", + "gulp-ll": "~1.0.4", + "gulp-rimraf": "~0.2.0", + "gulp-sequence": "~0.4.1", + "gulp-util": "~3.0.7", + "html-loader": "~0.4.3", + "html-webpack-plugin": "~2.24.1", + "jade": "~1.11.0", "jade-html-loader": "git://github.com/courcelan/jade-html-loader", - "jasmine-core": "^2.4.1", - "json-loader": "^0.5.4", - "karma": "^0.13.22", - "karma-babel-preprocessor": "^6.0.1", - "karma-jasmine": "^1.0.2", - "karma-mocha": "^1.0.1", - "karma-mocha-reporter": "^2.2.0", - "karma-phantomjs-launcher": "^1.0.0", - "karma-teamcity-reporter": "^1.0.0", - "karma-webpack": "^1.7.0", + "jasmine-core": "~2.5.2", + "json-loader": "~0.5.4", + "karma": "~0.13.22", + "karma-babel-preprocessor": "~6.0.1", + "karma-jasmine": "~1.1.0", + "karma-mocha": "~1.3.0", + "karma-mocha-reporter": "~2.2.0", + "karma-phantomjs-launcher": "~1.0.0", + "karma-teamcity-reporter": "~1.0.0", + "karma-webpack": "~1.8.0", "mocha": "~2.5.3", - "mocha-teamcity-reporter": "^1.0.0", - "morgan": "^1.7.0", - "ngtemplate-loader": "^1.3.1", - "node-sass": "^3.4.2", - "phantomjs-prebuilt": "^2.1.7", - "postcss-loader": "^0.9.1", - "require-dir": "^0.3.0", - "resolve-url-loader": "^1.4.3", - "sass-loader": "^3.1.1", - "style-loader": "^0.13.1", - "url": "^0.11.0", - "url-loader": "^0.5.6", - "webpack": "^1.13.1", - "webpack-dev-server": "^1.15.0" + "mocha-teamcity-reporter": "~1.1.1", + "morgan": "~1.7.0", + "ngtemplate-loader": "~1.3.1", + "node-sass": "~3.13.1", + "phantomjs-prebuilt": "~2.1.7", + "postcss-loader": "~0.9.1", + "progress-bar-webpack-plugin": "~1.9.0", + "require-dir": "~0.3.0", + "resolve-url-loader": "~1.6.1", + "sass-loader": "~3.1.1", + "style-loader": "~0.13.1", + "url": "~0.11.0", + "url-loader": "~0.5.6", + "webpack": "~1.14.0", + "webpack-dev-server": "~1.16.2", + "worker-loader": "~0.7.1" } } diff --git a/modules/web-console/frontend/public/images/cache.png b/modules/web-console/frontend/public/images/cache.png index 83fd98790df1d91b2c41bb39e5c97fa277e96725..3ff310379c05abc75e687ab0e08d9a2026a3b083 100644 GIT binary patch literal 24791 zcmZ5{bzGC*`@VpLw1g7UH4vDLQa~ml4bsg3VIU!>G{}%f83HnxqZ=tfQb|E_bThiU zyZ4*#PrN?A?;m?TuRYIs&V9~v;=b?ey3R*!EoBN)CemBCZc(VJDC*w2MF72Z3!jUG z0C%TiMj;#b=UQ7s|2YI2f zoHUf|o?)G3>tTwb-=F1+DZy4MTVKeRb=WH(-fTn4*PRSa?d|QiHn;Mu1~0l?O$}bF zh>Isn##N)wCc~BARP199e@>pPi)tbCV5_O|sPpmYv)%ygWPgrEo2lck7>J>s?bK3< z%-&eyk$tF_i`c`aG!J+crW*0Yb@ z>!0rn`03cZ7c;PZHdb{!o|cR`>z=!|Lu~cMHPlob02F&`q8zJj9SpLTvvjJ05-N5& zLM)9czI&A`X<1dfYikG>T6#t|hzDyWP0NXE3P@E#)XKnqJvDW8^~)!tT}Sf)DG9&E zUqIg13@jWRw>AgsZ`(9lgjcj~pB!uOIVH91ttJ0?nY>UHpkmsPtC3KmP}`YqS3G_6 zOrz>}{-!E@Vw+1NptxsYxF$%cV)L8&M$yz{B{HvjJl@O7PDf6Rt7s6NlWT5n@O3vg zeBIyRO?&xD;pdJs?CE3Yl`~hVj_sw4_RY1qzQM({_MfQksoy^;$_CX@+ak`}ERP-a z!_urj2DlhpCTmZBPHN2!{tA^2v;H;Qu#(3bnLri1FUXyR3XH2hOlOTe%ktY#FH~3C zeTQ2QciyO&xZS!%M*H`NcPlZO0e6$wT~+fr@fZkL`(28F&jit$Bndau3?Q2U$q)*`D>opX+fhuMj*;Q29x?qVTymk81ZZK^DUDwk}m6Zqg8L+&}##b zY9Db=gbi=$hRJh40d%-aMhu9KlwaYW%W^CN>SnGadlRVh4Z`1b{q5`GNrRO31u73_ z7KnS%f?pb|Lx`+soc3_^g{xo((9b z(iRA0b9}q|VL+##;(2`9!}!(6^|ve9m4**U717!NX#j1V;2XT@TFv)}MW!gkz~m^Y zwS5~2fxeBWMHmaN-jSh*{h!t?fZi?_*6;QRj|W6i<=I6C>f;!f4+5*Q-*?{hGqT`D z=D76Y?bOc;4zu4q^cF76(7g}d$&Ap6mt3m&`tBr8@Ri{JeJ{k1_$zg1o`AT(+F@oQ zbcKnthk}C~%8se_%gM`zk09Tqaw=Rw#A|eod3@Mb`%RDfj{fgV&Y~skc#N-ScX3g6 zZ{U$crCWMY&SMYs7wMYUWl6J!2nafF{OxNmi&3|DYDm1Gmyt+&j;HqS z7sAw_vtsNS^eCb}Y>VR*Pj!(G6$djdRkDS@({$dWU1 z9W-ow24!NdD1C9moR^Q8Y;N12-R;Z9UZ_SRH0vvEJM}(I@u+`(wT=z-lT|>OGm8BV zpwC83UWb#C+m`U&*z|$jyU!G6E_d%B>aR6tCIrX1*3 zP4w3N zcu02jTtc5a`+Tjqyv4huYm?_t2~E^-eYMTbX0>|7#3R?kH|J~8Mdf4ZuOc~{_AMt% zdHc9-5-WLbiKyNj3wWfHPVG0so-|TKfwxRB*4U zjID5fyL}Jf zImgjJf=Et~f*cJrJ&wJJEi|G$hPcUZ_!z z6&y&&x2!L3`1GTlim}78E-N(v6c@$5+Q_8DKlxsHS}s66l~Qbe7Pbm~-{TNUfY@hK z!g!cB15-a%od%v254B-rT@Fsm^OEXS61>jk<_-B?!xZNS(n!Amq_y=J$PoF4ql76y zlbks|9{QyRt7_2|bS&%zKL<}DTisN5R_;2Rp4=Q)TRa9vx*7v!3NeE)x3L83PnZOC z0;j47pwfGVyaP28JXNR1=jZWdq3$mY$6z_m!RC-5&-M4mzM)`kJJn99N(BVrEUn$x z+vw|^({)eqZ>(o>&*Bo-DouI*hY<@=bW7aLl9B8Zv{qa*%I$ZT(m#|Mc* zz2?S+kC74=eboZHL(($sfeyckh;TKe+CH<}IA%e5Am|8TQ;HmEC*ob`2pPyMi4R zq+#4$a=y5!%ScAr+Z#_rI)7s)unBnGWK`!YsrtnG9-ymu^lh>Vs>++IKi<46aE)?k zbK@S6ESF13oCSanZCQh6FMl>Ih6KExE(0vqX*7HnMo|jpukc;W*A)??!>)YW*w)NE zHv0a+4)mmo>+R>hrj>lCeevhsWP2Ul^7_@z;?}OJNIsL&i~Ay5FG|K_E9W9tiG_Wa z*Ln1E>zFtbdu)c*+qZ}NKP;E;u>87i8t;VD%4P9w!>Lo5q1jtsm+$a~le%W*XLP=p za)Noa60a|;yLD@kHtIL01>9JsO3BBrK@~kLm4*ab*Bs)34E5bACL1{)KHSP-hB-v~ zU@ARc+pES0f)?wXb^dfd+JH&`9%}2Jw?>HkhM>i;`$;G(lB;QgK)^-rGIsyT6vagV z_5f+G%)rk@C3Db)<>rJ&hCR>Os@+Y4mpvTX*uC@)Igj}qB(hmK2C+l1_}g5a&D*6p z-M{gj=VwiI%G`R9z(-w2u0>6NKafR9hGKA!2(-i-MIqs6+}^slq5 zOmqwl?r!H=nws0dp-ynO6LGX^VIWBJn8-TnZd*)&FM~)J^?56^5_+Lf@_duYh6>X5 z;_BG--45&H71lVUP+0AY^8E}I%roz$zC{g0Af65|Nz}gSehK_TM-d817%toZmhgV{ z`v4`X?84^w#680M=KSvdp1B$(OObHYvwP#V=$Z;tF!_UyT}`d=5y;nfNcB8mi~EX= z3oA8re+Bx>B+C7}=tr}h2w*2eXVtnNK|(@)iRM@G@518;M-CB5hw48i7_sEwa* z>bE9pgtn`hR(9EkbF9(<5a3Wk%7aq%dRc+$dE4KuWv|pQlv3bAvbZ!k>Js{}i6&iz>S1mmuDk~cNL&&No=bHpM|WaBslJ#3{C6huq8RVM_9J z!Q9VZ57$EkZ5`{iUMRaRs5KPw)0VSYbDwe@-Qf(gc!beUdXraNDG( zegi_Qt%}X4usY=1oP*?55XgZ*P3q;={P%B`H^T(j2Md5v$x()8 zzph@3pYgT(oNb1p-XDP^>$;rn#FroqM4^x{go#b|M9xg@G#>gv2kh&2&O^UU+FU;7 z70ALx9YUL=adm1gRPkmNnH9t)?Ku|Cd#=6g)KsX0tu$VpcrCe@;xw84gWC_$*&*_0Pv_&pwZ6*GjWR3H zq(*YmTEA^s;61w;?^FQ$?F~IaltsOvXSeNeLtQ7`-B$yWeK~&u1zi?w8aIA29B>?i zf{o?5FiM78-fIWDyJ?S{B#yH_{vmFB!1aVqo?nmR6U#v-Ei?bqEcv+C+9%^j09e~e z#%!9nZtewpRe&fbWF4xOlcqOX4gPTA=C+DFw;%TuPtr08^3vEW)hX(mEY%l zzYd-__;Njix->_pi(lD3JglxE)?FIN%yih2M1-bEr5$~IDa5r94+-NOtk)2Kz5C}< zp{82O-VSUfA@Q^Y_x11qk#>h5-$V1VC2f){XA=Kj^~tq5=T{^)k;Q_ERdzk8^^d$? zbq~)!dMV?-6zrDQicf!?a94*DN|F(gF=hweGH8S<_IF(obG7pyc5j8(Z#2S5d>Q4} zOhjEQyMxn_=VVG(QpYnal-j&cFPpGNW#GFoi!x6C@X}uS=h(A8m z5kV~s^;>{s;M5W{=%P}L#23Y;dg7EE~XT6g@}3pY0MG8}mG+G`?ol9pytOf(z!poJ5!Z>8gNISqf7qqh)5g#A-kN z8jeFvaJslyCU=)fuGYz9e4JaO^>K|eRb~119G(6abJhE;0#sO7JPkOz(rZv+3{YxM zY+IKc_o%0@?s9Tklz&Orpm%m;1E8ZBz+T3wH{Zpy6pel z-hO}Va0W~CTkV{~Ra`rPC|WJEyB~hYpAH9qpGoge$RApNq+2`_knovvfN^y+4T!n(yXY4Etm2q^1${qRY^^)&}oaysJqQnA)q zSZ`N01tTLg(NSTJsZQfiBqdQ%Cb@U4qLO&a?4Ee8Z!kRD!0Wboa$}gFz+`|!I+d@9 zL=wUyLNbPo zoa%g!D4%EwN|aj@`+ic1ExuKqEn1^oBwYiVzI9dx7}U6?P=brU=HNbC)Lh7bXko0y zhiIJ8hWzi69#8&_1k_Ipq(PTjD1xV!-M1}_Lx8`2)Uf4}vga1CdnUKK)^$I_MNc!A z8;`AVfzV5~@8NWC&`3k}wX){Ct`chHg38BdxAA^#b%IWtrKQk>G14SS5h$vyWQv}U z*ri*6x-;AfT8@3i_7%w@A#W$~GJElepQjZ4vv_Q^Z*nOH`K14_HMuOR?Xao(u_pa9 z4FkEY$B~WQT^fkkIWsT!DAK?(bRRxG{sAG7Yw0T5B=kpEo{AuNNNH4yWK=xZYOF0m09RpdTXbq}y;Pkyg3NKnGymbHQ0coVICG?qnL9XqnV)+gSUsLya? zK|RWV$s|fa-u>!n0Z68{ly4WrEJXGI)F(A&XHDATm4%LhMEcv8yF%zo?d+|Gze@Ek z>bl=W=~R~J9F>(jX9GSngpvx$$}>KxYk3DDdU%cx^_3R3lE8YSEDZe^Y@egzHet8H z46;uUI=uS(A1Jqq+LS@=*49E+AVOl%QQi+iC#@TIMB$!et{zes16>iM-;ow{CCH|N z_IM)yrb^ZL?bXIYIO8;8;CsK_h(GM+`H#+`_{uFqm<7#MPqI#?xu%?9ozxBB?j5v+!PQV<8j^SV~Fyn0YSSrvu{IQ!R(A7KO}Dj9l(I)<=Tl< zjgJ231li6$8B~xe-v^~oxIy@wC$`xd$Q_YTN|%0r4k#8Q6!bj2o z_8itu3imRLP(0X$L&6|LE+_j;m&)*IACqR$hb=e+MpE9NIyWKyZNE*5E)k)e%Ihir z1&X)~_6~VNB5p>tV3MpFj>Qb}j=6b4=Sc24aqsg__81EQL31j)AgJ^ur?#kV(GPPK z_&)VX#nO(}-aiq!r)sFmc$Ss`I0&)=E--DI89A{7JF6B^;ag^g1+i<_S@lj6BvDj8 z^GvK0d^w)1N*x%&+y&GjgFXvlBbW4lV&}N_Np8N|d2N(QK?pJT19)jY)TMDO7ZSuy z8*q)3LzIc^g3$NsX?S(p6y##QMHH88ac{MK)4@gVsc*cahlB{A{{9g2yYJc9Y1|{Yvpp`zmaXc_3u~qCx2Gl zmA_Hpw-zgv>ar(#@k5yc%e)7=Sb6rfsY7-OXP~KD7ze?)j}P%suYiVeCCIzN7VQ6&e@bz?F})hq#uMPW6xj9x?+X z8>N~Kp6^aFElxannbFoUYkp~CyR^uQR-sQphQ|5~g)QUy_@pr^^0JWBr8~7}1-kWA z_O4oRhcb+bC*W`5YbbNMvKQ+75T#X^3$)wyGZei|$x$RgBBPMqZ7kg4P9h_eJxNRM z;oJYSq8k_^&j@SmzF$|wd|kck&titIgI9XYmy_9~ z9RKM7aA9Zwam0iO;)@bA!Nq}v!(c(x2Du=n>tf4Q_oCJK*TlaP+oGt&qS@ps5MT4# zIJOEOsL4+fd-cYE#h%$aTvaZIJPWSmj%FsJg;B% z436n2CNjHU2z|^>P>&k9w`uu$PR)eFt}rVkkLH!|SKs?NaIirO(!%JcGLY zTqid#m$iOSaTQuj&lkL*&UXvi78abOd~~HwOnBjy|JcWh&CxhN75L56E}&WM6Y@!= zCAqnTSt|LRs2_qbJXarUr!bM;rkC955k_{OVXxTL6K_GY*qH8$2%-zH-7(cOKan|V zN}-?A`RwFfEtzpN5nR=WKI`&b&+mP8~b+xe&heP6J?$R3^f9BoLb-9u*x#?2xX z)WbT#n~mE;9uD@tsJP2k?Jqko?t}i?1q2JFwkm{~Rw=;$;&SqXw?TXYrByZwt2T%q z+A7`OG8Ak=g>2GWsAcZtDnUuI7yz_~>|0*eO|<1x5qp+5(J3c93fX=po{r8kM>mmx z8M`K`ij)<;V4F^1NEH3D7|xYGFD#KzsOk%5&c?4w`?M+DUNWc9H9Km#_JnmrC#UeH#QnOs;w> zGJJeceR50s`|2yzG3C~~rFvrwJ!NbK7WC%KMN4iu)dG)QcGNx>k~;_uj-||dj=eJ< zTzUD~sXLe1X{4kka;=z^&Popw=kVyj>sxfH; z(=R-ojk#oi()hhq^D&+{2Cbe%-i3a$EG?GR?W!#xqSqr167ZPB^3R%1Su{r+ZYf%$ryuNP=GS7T~rtcmdL&6)gc zrl3&dL(Qcw`a}nF>z5@Al?AJ=Uh&A|65Md2+tDM^qvI(E?;Y>+a(%;!C`S zch2@nNIrE~r(`=W!)3q->5c2OZQbf=t3%r3_fiSVjY3IaLP7O%^{0)Z=x3-`4tJD| z^VUD#SNT-4Z3gs|h%KXI$=m(5x^h7F*uO+CdSK*-K?%=kG*Yz__7!Vy8Y=ogE6_cs z`$+UJS>iKhKaSejGr$439^}q0UuRKnc3ot2S)coWy3VE}DRQ-I<{Fi*0hjc?mf1zn zhQ~vTrylQN4Zih3tm7h9A=P>bJ-kpB>*21S1ob{96Js2@h6k58pfREU7QTM&OP0^M z&y?!L{jQFvPbegzSO!9g0AZ z%@0atJ$86mmbMRWJL$AuhRQ1``qun8hwz?Sp(XA-WaU4cj8}vDAbn5;4KD1*pptCz zM`kbUFPh#)jos^?l$XA7I};_d_|b;L6NyFCU%{tTihGQU{tVD{3@?k?Gt~)BdFvX+ zydA|-!`kG3yiw6$8)VTsEkley(wX2yRFO0KX`OzgTdgy@dne)gWI@q*1T zv8z_vZ5bn$pRD`phH~Ju7i6tHxZKTAwg~1-qPIOc z2s@l)cE`%oTrJ=AtQVX!GzrYDvH4x?_{nzj>diuZ*`^_#dniD;|5wz(3*@I9*zmXkjMgOZ`?-$x;Y*>|@c!+IWFsZ^GsU%HnEDe;Vo5!y6kCV=C(SSe<$zAk?D451#0oq%9iB~CM_=s3ERMgneW?^f;kxOf%c>x3A z?XF8m?daIqvh-t{CA#hVYLoDq}O2CI(Zs@^= zn=RhMvWC<5Z(p4JQNaLg9xJ?}K3q7>>-*IHIJs>v$cOQkI1}JftruX1uxAjdv4`+d z>0Xp4bt2?+!@Cinp}OwCRG`Ex4bDD^IJL5in3#K@(2oZFAZ>s29S8Eh6OgWYX93!8 zA#l^jbtnKUGM`j8vU{hHts3k;#>8c~SGC(``=0)Vg;qgievPxS&6ZQ&{Zm`;ZM5;$ zABJA)$v3?>AyKp`F>QO!K(0ubvHF?ofeZNU!inEiqZLouLV_pf?sdR2u5Y=UP(WUX zbU7R3$wiDBBQwY|FK4F{mYy6GD`^L=b7vKb2W^;5htW^R+bH@Z;Qv&hb-atSHsp~)WbNeuM!AhAA z?@w1c!JXXLbYeypZx|TQALHt<_*=*FFkVh1RGKshM=zb@DY?`_R!$KaxSW8MOEv@f zF{%0<_8IVy7|wo@F_Gi>d^%coRnB=BpYb||qR%E&wgtqVC2x~0?t{8Y?>QheoVM-0 z{rWNCmvgLImA?vGViOKE3=A1>kNlL=wYBTL`401q@MRpxa7{swK%PHtbg5NewgpY6 z>}izQK5nFRhEKV^a{tRHd7E@Sd7~w6m9&g2qkUIT;}a5y?}#eN>1&g5lNJPhrN8a- z%4&O)6hJ^jHS#lb_k}_RRg<6bJ{7+vq~o9Hw(_lr6Sqw!`ObkT5!HP)3VSAgn@i*K zyBR{{eu&q{m5lJeG|<1rF8#<|HQLO2rkS)uP)9IBL68`jwY84CVbK9Xz0|!aI+RYX za^uPt#gJfgaZ{0h7T$U0>v|xc{#H6rLinLZlx=rCT=o&*>c{$%1*0Dnh^7XOy)r93 z9e=~%?2;ZU96N;gZ|cC^Q1e^U3I23RF8v>RI~r0iMTWQK$T3pWV6U=M>?)siM+Pup zlYCzvGy`ZSets4If;tA{_#F#1lSkqLD_^$o0?X$5aj;XTK(vNkPp>h>mj;R8{icS& zn-8cKUpV!pvB*C})rD8o+hTmjKoZ0cQ+9a+>hd7=yCraH^at}Z``p$ydV)8x&J8OA zW^wo1Li=UghE>9jy{uP*YvE-K5{oueznK;vR7KYv z{@8-M1!wU*75KqO?u!%8;8@VDa5?Ps(;!wxDqO1Vg5KJ+B*IX^QhawQsSZG@7^=d& zvKweq!Y@jZ%K6xR>N}7G!C1H1% z^_k1ougw-xB*r6n4V-fH9iyI1d z*0D?3q}FA|H6?ZdN2ud#@|#;1@dTLcbIwD1{rQ@UmK%-Z=Z$qC-W)Y+R zQhAqdkdkr=Hx@~C! z3#6{UFY7oR0z`6({*o}--*Q=Wf5w8IkrKg?<1AcIcV+;=EY#}*$h$-*EJS&Evns@) z;Dzp}2IB5MiMLCUk#<7n;t|X7lsOD9Z+3L?AZ!jQWcKo+uj@f4tY1?6I*oea7CQBM zQt4%SH^aYMan#f$A-2t%U3Ea7#fb2qkXx{c8{sl+5c=@7|eN1{1 z9-X)JSk^|m*763%IIh>A zl(yhG&8J7A_ok`qwg#`)1sj`^o=bj3`k2dPQ$ZvorMXP#z0#}R1Ud4s$T%i4t^RHt z`n2+eEQfaOn!trsL;|Vv{zFE=`TR3RL5Q<;n*Ovl?>GyG!J*Ci9kF9~%635TNbDV= zh6*U_xc+fzseE}$Vwu6GDc)WQXaXksFFTnw#&DTkS@?~K2?>$bWt*Mh%$@3Ghav07(=IcuR~_y zuBq?Q`)T(LM5Qi=c}IxhGDuU5%62!r%!Y?J;+e^^Jsuw3`OZ-pg&L1}E5g2<%16r9 zR*9#t$`4ZYQ8p`U#%To_P7#ADhP^RgtRy@dUD*ilvjzL+7Ok!+0c~8XTNJo`8S{m@ zGqV(iB6P%G$Kx~s<$fm1?*=xACm*Yx%Zdj?{k(`BX85rnuevJ5|1b;IFN zXqK=MH0AmOZMJw3lDUj6sZQu}K)o4#7G{Ur7^hk8Aby$Sre#gQZ=btVFsk=BLXpL= z4{3Ns;0B)o#3Hj-UorE$@OHjl4pQww+HzeJdEp z@V9oYy~g^YY%tMt{jJn3B*wCCD;?VkLSyAP12ME4$e11K#RrC}v?op&v#UQir>U~C zis;;`<%D7mpEE&IYi z;pK;m;D$Z7+=uGm#ndviMCvDcTwGkI-D145Tdr}qxa~j!584BU%*=gfGK@L28i<%u zhMf1SFN-9ad{4(!aX`|*0n|fDv~evZ0X9__{|`1 za*)S~how4e1oho4q&PZ z0XhXC@8F0o;afo-;5wRe+2QMmMy4+-L0jP_{}g^k$MX@}zWPRzgw!97QINYw4~jeVMfn(z=bFHCnqE$t(hLG^8aJ^&9VpK07HgAgG;leGmJj-|Y|468C& z#0BsChp*7SMRVJqx56Jgh9rlxATR~{fe<^P*Kb{Wp0DJzO-y+bLMK2B_;O8txvyxh zUZZNZzu|lS5T+{?!IEXL!w!{awDByNp!M!qZ*VMD@){IlPDCfWQf=pcTDIasSw6lq zuN}@`sG|^#tO(midzi^KVyKIyW<15#?r}c=SU8XP^7ki~!;a$aUi`R|gFK-KbCqd>jMnI|7YVR0TWXw&gD4W)DRE#wmgL)rY$WTQ#<@ zj^Bo_{L~9pc0ULM`$@vUH+u=xzQl>~a!#U}$CQ@;3ztPtFNSOpbMSA{tDsi>OceKX zzD$R}T^4?3<7ht!a`^5&%jRv@InwI1e3i@uovwjC^&5SW>S{f_$*q3%=jQVvst+W+ zSZ#()5Ds0!AD?!A0R8Zh{yhd5(|>_?Yv}t8K)3Bei&iZYsPo^TA#nS1b%F=DV=a1Y zE@FVJPOuhaE63vxjzR<_XAC1Z=`NcFt?eiqoHxHaBm4=K`brUi-$sa51{+5ug zD0s>I5%?s#V_K&9BVxq)j+qhCIa6bc8IUI}K9X|m-kYH7rIxMn%6-$Okgdp1dhD)J zeYf4IXL+=8dXb|EE=@d#NYv!({Gi*4pdcu`Rbc99)Xz|w;LGgZ5=u`sN+H=Gl~N2V zRZxCm_B@SYyU5ZsoqS@P6o-)Av#i(tN$@o5{^&4I;mK>YyMSPS$c-Ayy#((BW4$W1 zSoL-?3I6he%h;#)jlgxky9xv{`PGOO(T;lS=Ijer70(z6KY6(4**d7TAg5Fqe`%lf z2BE1}b?@mUXf;g?Mf%hR`GAmN+7vqOpiFBtm{zs zGpYjkdMfC-B*Gz5I*BQz_5yF(d8DYdxW@;>dvg|+2t|p@is5Sb(CvJzur=f2cl-x* zc{jiV|8M7W`Az8Q==9wlvGjLVFo`7UC@I5_;tt!(iO_4#&nn;THo33r=H|;XQd7VA zROxabz=iYVNh(g0DfqfJ<~CDv*G}+oq;YST)n7l3gZuIn&@uyE2_S8&j*b@sbz;pG z7fk>%K8I%}q$%k+54Omqw7hU5M@J+%jQ>&82yDt=*$jxaY$pmSSbU9a*@V&+RD@M2 z`cU9J_zXLrP=j}U5wdU)PX#rGV{;-1%nAajdJ_bt?tV2b9x9K2?_wZPG&Z!~mBCaK zOZo6TM_NV5nJoNE1?QUfYD@h4*NnBC37LlXsn6e5CR*DiTx+}jta7;)mnt|~-crUP zS-Yp8UxR!>tvhHDVGl~nl|nX{AkG;Zo@N;oC+rDnwO@Wg^{s6Sp2zM*+Gncn1?T<> z&F^#@TYT>h8THypL0)=xSBDBGHb?D0b_#`@dHmF2GAFgOd9>#N;m1mh(vM&O(X~0u!ACPnG$MSaUXO>mXZJWJIC~OYed|ZYI zrgv^Rd(`m}LgWG75`1d<@W}*^In3L2E4#@d+**_Rpxoenh>~%kmlT> zwgDTrJWxn?9w_~W@ISqd)A59wUi-5p730ozX;S)2iTUWS4Y|VEz%sFE`|l0_RmcBe zS;W3Cj;V33U79TC{5v-m&)6g7&Vq*n>K00L6Sg#Na_~nHBeG#N+_lneve)|X=y6nM zS4T(GGm}S^Ki+IW73O^}W0p($C|Cu|eO{12F$S&$tT?Ubf1of=!>5(qu;Fhb3s7-X z+$z*|`+>AN>?{{@{w|E3~DH=SJ?ZE=X&&HAI>al?;nA%cg^F!Q{D&s#iSBKWFO5z-~0iee{1Tx z`aVtOi<vJ@P7(SqYBrhCP;xrdRtfLSwfo zX-Qs^>4@~*_0%<}8q>s3)gFE}{71bZC8ta3zZcvrw39S;S7d4#`w(;USHIEldnNE6 z-kEW{>|2V}^`>tQl92G?6q&OeNYn0slU55S9quQRNZNz5b(PL$Jrz(0dA8A*uYYTy zXb1?J{G^{q9Wd^bLL`q&x%|}v%A0GL*xJ#A zQBv@e&VLmiv~(Bj<+x)WDpAnA1KYC&)T07s+#wNp0FCbNDwSrlU}TAR!gMGn$5y_! z^6k{iDN+D0X9E5CL|!j10au9cLICDOxdM1A@@X_d$}W? zHAk(h56e$S^X-HdzlrnEps)}DB%>&%FW6HU9K(J$HM2R{R_wv)_nz@iK_T~?WfYI) zYUcGNv|{#>gJkVveh}TcB07kj%+-2fhroQ(=FD{Z^b3b41|wUI?#EP=fNMMJx{oo@ zkd0sxLOI@>3t%t+YBK{x2>J48WW&RQ*mdrjSz=VjqcC3uQM0s^yZbuY)QaiOqfRCD z8tWOm9^VMtA)%?OV2oA7>Xvpwx}`!)O0O5=)5s~_eh51Kb3O6`lRpp4Nvt!I7bkHtVcKVqKl9Z=8;JZmTZn4zzZ~h3XejD|6VkZ1FT;D>X zNTf#UazMeK`iHj7lI!hjv;%or0Mxa{FxxGzH?~2KePdfNpe6Y-WNU=_gekTdU z)D1QAZ{{$EUPp34^8Sz5wyDm>y=j=Hs~+zrUf{i`|;knISB*n zB_y=Q^Bydd_J~bd;QNnv)v1Fz5yj>?Br^uxIrji0?%F|jxpRsV7C|@qv56h7%b@R? z3u-u>$bb$!7t&riJ(n`RHL8ed%{sdO(|b6LUwz>1#b;j^;l|&d?PgtD#1C%KL;`7 z+Wg(WpO_=&^TffKZ(6!cBgr}jgOv$-xpRh$eu>U?H@!;NFdGIO{a!Y*1DrWuhg~sE zOZ-O)Ec-@mXdSW72_*&R!aZD9rJBDXF}To)X&;avTuX$@JNvu1k0XDa;+b*KKK?vQ z&=s$sKm!2M?(*1x&z3$S`7~HbyQ1A^ljWRR9+KGX2H3w4Gb`3-JdH*64%mMLZ}T;7 z)FPWAYXnem1A(~=CEkxET6u`4)3+`EKoN3m>tyaEs+z97hBZc$>+aMsAkqIS zd2PXT2N6)#HK^A1eXiRg>@6Z2#hAss@X{ipuBZ>35Y&^4j6gwCd_)xNm10XNppbU! zjuC~Fw3w|vpmynjjQjcN=?jLxheU!g4IdlN>SOYmE}Kr%01VKUpV1w8!qGGViMR~Y zzX#TWETSU$sryc)DZos_a_m=|@b=M&R_L8?gi!Zn20eqi(~-Bm0dXAAwtvyKRNy2o zrML>CKdnFwzXaryUzX%#FKy)`xU9~L{FGfJ37_4T_fjJ1ivsEXJs#yZ9$FH^kp==L z)mA87%iCGh~A=$FhaD9Iz$i=5+zZiM+t^8LG(mtMhK$UWJV8?32A!oB+76$ zxu55K-s?K&!#SVknmuc;{a+AwudI@eriJ^g*nhjbXV#q2nCZ-Prcb#%G;W`>P${@;^veO;k zdFI$%a&#rpq{Roc9bc_d2Y>tpKEBr06f^mU zjaq8kquYR#$J!@11xw4KKEMje`kW3u0V^M66T?;zI0<2{4|N7Pw+tJ;#v`BRX!e#BvaF{h!W=D*;txS|a}tS8_CpJjaHatelNWa7s&J@GrglX!Bmi z_*X)`A2A|0*MXDC2R|8A_P!eTVZnz@91h`Xb+twJ$tq@1>pw(C+zA}kcB9-6%B<}D z!T$O_`I||X-GfEk-G_a=+-}#V4q?H1vG(Q@MxA9Gf$zBs83Cq9%-dia=Qs8oc8}5A zz6s>n;VCh8(mU7?S?ykPw%vy0(9vb*Az}2BklXeJH2UaTu^k3!Ujo=*Np@wZY7;6S zX?zW*(u=p+*QN;z$V2Vz-%Cv;%2aPT!i5~hJiMC&l_n;xNRE3!DB?qpwPbPpGYh)taa9vzmsXkFKCc2g_b(5EunW zrp@4I$Y?l}C;e&|2r|*3o$}l$Ra@XkQ@&(J5qYbg-T}--`BVL ztvx0%?)SdE9Yd0nhNx^kpl8(4irod8>D#3sb}p~7G7hH!292KDRezg)T`N(w=C63u z!N{82x!%>jiW{qv7e+;`Mb(D!OhTss=IX>(Rajv(eFm}WJybtbr37}bj62ZS;LVk> z$Nm8Ys6AKa)a8Hp10=Dgg5UjuZ44JUUxR?OP`4BKfx)d}Rozr9*!0S0Ns`X8hgp!c zQAMg2df`C*_ax=+yIZT&D|6TgMNT7s8Jrc!MABt;P(4_{dZ+pj3J}htNFP*3#dC9)wBjpxM3+YoLnL2Fvd% z1#!PM%+9@Q{fjP1B2mYImjxfPxw5XGM?Zc8F7KexSVG#ne7SH#LWTRj@XGTY_Ir7n z+>^21)4QV7^V#C_;@X>05AN${V&P^FvAsskfJQqjB_-vts}@P5j^?3o&BzV-KBCdg zo_l}%RaN*TJ%?$=fX}-RvoedGSe`f^<1ghu&+^*%lcbl(b2j;`+BQD7>KpB+(uEoq z{ZI<|!w-0QiNr6S8iGjzDM!c%fKkf!?INI(Iz?1T$tQlxl8Kv{2kk_Fh#5iv50!|C zo$Oqu+5!~&hedt?mP^*1xlR4T4==Hd+7x}a7c|w4NwWKy+*8cL&gPRYOA{{y2e3>p zLKn*D=R1p+X~ab}#!d*2!q%ryuD<~m!J*Ju8|oF?A>@cH>u6UYqlcnTlgyos@Zzee zlQ+4{Eij4QsG5{ZZ5Q_z4>lHSyi2R@IrwHUSMouYFJ!|c2el(4-2nCkAg9CEYgY((bIi_+37 zhSoMC@?BS@h0^Z4;ATsBS4{m8NKzGB1@~s_wX#O%OR`i>u-`>r=VRKD3VH= zCujBJX|lS+t(TC1Es}e3oiYZgIFAs=%*N=W`v7cW=;^Fl!Nu~v^L=gx{SvA@8ps;I zDq@Zf=R$i934*>yb^3S1rAavV>d2{ZsfkGUNQ>^oqCM#Dy%wruF33}6q1WF}nWO0S z5~|?5TXuw`h~Fp_*8(B~8QycApEB@9Rz}ngjImcT7W&;T9C~X-`~hRJRu{D+-lx=s z`?U0nLVSnu`hR&qf(4wDdSWqUf-gp~zZ{zG1iT}y^SUioD5U$yjXyYyLAF{djQ^!i zbo%iHazsPwMjPDqm*-`GU|}1|etLiSk5;B%sYsvsmilz9p>&RS#0f%7$tTavj;Npi zdCQj4^|wL#YGxYgtyhscto~2Th7c~67VXP=(Z83X0D+>MGyr@QKOW~p^KT#?&HbCB zab<)2ium8Uurv-!EsAxnvKTA1FJ7S>dM&G)j^r*ej6D8SwX1rZjuHox9S!^T(LI5j z^6T=gxIdK2!^KME3#R&1Mi1`IilQ|1aZ`Lz+Am?9o!9YuM{P6EZUu6YitI}n#LTxJ z+?xA!rI@mi=wIRaNCWq6K_nBi-F+87l-|3Ijp-7U7^64)!-k`KG_D>;)SrGFO`1nX z0twtYKIIk%_(U-OrG8FIGfk=IkP?s+z*fi)ud9{-XcsCsXV_cSy;jMESWll%%+7f) zWVkVKsXxiLQl$XJWY)zm)wew>go`L3)SINoss@xU+A%7BN2bJi`!O<)5$~LAZBt2C zIQCEkGHF{ppYJQoM0NP94A4bn2L`)xBS}A;t`#zOJ&-(Vr|D5Q?sd~kIJ?{fDJ z)We4~N;_h}rpIscdykoP?aG|w{MAPUYum_=V#!-j%z9rzwQ9LZYqtu`7+H7_<;Dxh z6$u(_s!5HY71#5?&}w3Nqax?C(9C_p$ZV9E{FzHYk|UfdN?RBGudt)HExDFMJma=t z($0ID4tMma?{epP=KWSvkf7P_(ViL2qo;`r4H-2lSCfgJ22%kwQ9=rRYk%a+R=lNS zWw7?V%l0uil*2!O3rF5!TmpbHGlzyo_@|IO_E-(Ba!SFQl(G^u7q)OTfaoJC)TZBp zzDGTE;FC&_2SEHtYx_GuC73%qj1AyE#4okB(EAt_8~eXes}^=t6V9MCAFpmO=b+Bi(DWLg_&DFazls^_uQ%83Rz8&R=dVJ@_IQ z3E+?arOYK`YxRYhJKmY7@Cz?#p9&w9s+k`HXpVGI3pc-gK}bV8Pc!`keo5P(%IBb7 z2mT_E|CPHmK!*jN!&dt_=h)!I_aPT}vg?WC0peAsOx#WL>Su$P=aZWE}75XBnY%FY#*&x=zPbB=qGb zsrZx+c77?iap+b{2=f)KfVd!ABqed?f15s;Zj5FyEX>LqqHw?GpvSDW@o9V|m3`F9$KLm;NtQ5_Mxp~`DWL0fM4Y-s|JvG3Q`{>T4E*|VbLl!|mqz*|*66LFEaQMNYxz+=038HH zpt$~f4DB(3qD&QllW_7b>=t+dc>i3Xd%P!rfJ>t z2>Pay8w?mZB*ctdPxK=}ald1wiv8Xxm>UOOgb7SBD?wy&iyAE5y?zN$uPxn~RzM|D zzAJUv6iOB~r~QcR`Ug?~Va~nD#nNHPmROGV^TqMa5SR-An$`S2$l_0LcrWB5a`ol4 z+u}O*c{0DCi)G#Gv3$sDIC1qtH=HOtlJ-dVM)?!XU8oVRo8@z!=jCG?kk8)J7go@m z^n;`2nAbssXD9oF1v_aWI>Xv}&5uP)xCBD14|f!eZh{lFaGqqE3x~*y3`$BI{ZZa2 zm->eUsJv5u&UQpVgyL|Q`QD7R5Fpy7Hg0n>!h~ZjK;5yN%OP*d#xJz?EOlo0@!H=p zn75REx`H^47`=AkJ4In_EU3LmzI4Zs_N{G46{`4BrGT7)DD{|cVlRW?aZ#s{&>QQA zxw&o62Hr)yHd{XPGx4#1V$d6Qn{Bs3Y}&%jig@*uFCgQ7#Q8aODf@aR$|720=T(^Z zjd;_a7E71sF@DyAeHeZ-+ux#Xume5>3|I->gx`no7NnJSN*iYSmb@bMcoB5_uA9M6 zj@=IOI=`umDkXTzm{}*EjGw8Xl=Y!k5g;seD&Ke^a#yXAO(vb?R~`G(%HaKXgfs#p z=$i$p(UGfEX2U3-%S+5HmcPBlC1tEQE>-M$=@EmuzGK(>b|@|Iht<(+`u%~N>`Xm8 z-qOoIU8HNvYcJqfC<(~`)IzlR)upmP%F%W4vjF?vo&8bc7>VZoszE~tun&J9a;Hk< z?h$4^=HZ|=%_HEv*W1i9*P^4Ey^4T%v$?2GSDVx{U0$2#AOLj;^e{_~cEkq_^K3|+ zJw`jQ6h%u&=B^9UJ{`n&XU7QK2Ong4RFqJA%wQ)_q9cod4TZ9B<*hT4!KCamAdJ26 zxEMHD0Bgb`A~wYlF5;>_|E`qdwb?-O>=dzWtOYTB`@^kFnnNLO@D1rT@loir~nPREX*e1V+R ziMZvjr<-n<08GNTH=CMr)vW7WDGCbM{8PEPG|L}6;`0r3+5`SbmCagJ$NOog0M9oR zOU8fTL0rRMIBR|RY_8(@c_$b?4=fC&5e3luX{R0s)`L8mXv6pb3_~bt&cCZ;zvfVz?I}3l}qPU)ZR(R^4 zuz)dSj!Ihwh{Dg#LmA{VHM}qZyhLrnGc#~C%2D;K4Ghz!9XordN9Zp9)5c zh)UP)96Pn4{*))4K}}k~{pH~E8|L1SBHngppPf~i6IKIKWETK__JhXrLHx)O*d1tX zCBGEwI&WBOJVJf8#Xi0F&o@R|>;0!+{j~o?rU>kpnaD2I|0xTJX`x>Zp;4&kKQ9dF zqkO0y$&(lV_6& za!Uhpw0#*Vv0F^vLf-usmJ8%I=j_?6>o=woJmjEJ9K2SAK`7Yq?dEWcLZv-UoDFrazPJYwq zG}_PUqZuWbPAT|hUp)s%Oe(OHnYhxy2I5tyL2uIv_Y?=f)=7bhB2c;CBE8-Q< z&bN!VR-|ojBLl2a#ZsWGxE_$kKfaW}$C)eYf0(1F zJ}aeiYb`0Z)gS4mvb~3wChyf?5KEN>owjU#vFNt2mtH)|7DlqDFQv_mMtZ-dzw^yi zBj;R6ITTl}e)N=7cK7uW;+uAVmCjF%OOjjqb()!e^J3wf#g*2nVxy+1ErhB+gb8Ap zv+c|re5YS(`SFo`@L>YLM1HiRT=i5olci7C3njKAmYeiK5H#0XlBC$%Z#P%eW z)9Pbq>Q_e*vmbLG(7bkqgceI?UC)BJ`V9wYOchJ(k9ZZ5caPQ5K@dpvqvCB>mg2Hv zPD|d}#;;?qEMcjfL9EJ3?PJD;$#}rS6;e*GYbueoZNdK`-+6$Z7g}%zWz_RYSf__d zJ{zooAo{S8lnG-(f(W@93a$pZrHYEW2u0T5h;ZoLu=4HN9y!Bs?5Kb5m@hC;^%6+X zKHd5X>_smtM>r@lkKJkUI0;Z~=_Qb3AS`Ry@ zf<!|-i;+-JiR;ri4z1J?pc(aQ$@ z8oiJ-sUHrBp<}UCNcU-&f0#37oG}IJl>)cFc*V&*kN(!SS%@)u&tj>4=a&y(uK91+ z5Y@|51}wNy_>h>TF0RHJu#-@=3p+Uos;W|5Fr-y+X-&-?6RY@EJ$vn&Mt}u(6f#_r z#qzoHEOa|+k1=VE)AwgslhA#O7e-#*5oSNqogB-?otnhG-Xq2r`(HaeuNi&Irz5I; zpnx#3lsD-B?1W(%Sr!LmCvj%_fPLl1Cy%YwIZitvNIEh6fT2>HuM8`wO(O9>XUl(Q z=8A$@CWTV4;)=yZe=BI@+er#CN-}EXn^wUZjZI>C)$anv#0Ct{%|2{_kEu_62COt8t4u{nV8}qAIYBXeenS$JNg^-L^s;##%ml{_O!4B(bZ1I$V5-B;oL`?seTDqn zkM=(&yh<3h-N0ZZ>1Fr~!$aS2J6mK*vNRjcO^m0uJoDpQzBM^8_TI++I=ElayubJn z_uvSu!)Ge|GWA!L!j&ZKR5U`L3+D#;mrwGnps?NKZNScWg)a#8>3*^K<(0QwneSk! zV~w>dW>Rc1hur(1=Fg@bYipn5szq@#VtB>E8I8jHDi0J(xhW<6B8o;54(4g&QDOr*@e~)-5;SBv6e-48KVg--$QW zgSil%egex58qo(x7Po;b48Hp=p0xVmEf} zcYJ|L7U0!<*gH66ttTtt5=L?F`i1ZHghMZLe^Pq||E0w>j0o`rT$>7VF>k9Je*Nv? z-wQZ@4YbcM&L+$(gPT=t$w=j4eZw{%9q!-O&OcJaR~kCS{QYVUQ)-hvvPLyRVS|+g zV?5Z5f}%lYg|^D~?-^B2`TT2U08I2g%qt`0^fs$mICcVbnD&6oWk83wDm70f8_UWm zUFDWZn+6$pGbR)r!(Dn5_6#%WDWhD!{aQ+#GAL3`cV>Me-r|7u6%I&Aipph$L+Nmk zxKiaX6pm@@l3dMU$4f<5Zqtx5r}63F!|Tv3QA(51q2i&$!d8eaiD#1g1&DL>ue%1Y zH?G{&j0L9r#()>olxcs`GmkLm7a$$-yTuflkixJx66p#*&3saF%v`^G?aW?reRp+8 z$)Z18Y(Dlohny*uDEQ+wQlk+*rUe9RFtV zg!lN1LC(P#e)sAX?LaQXJp566KYo$dz{=~YlP22T{|8r2>zYMZs=bHK5=tk^4M8w) z^D@=-u5I)ZL?}u`nakLeP=dOQjd1lJ!Pov8j>k4)Cj=S9v;*&?+^;1p2Nbl9uP$zWVAV3dTupq4n_hI#DXxW#+o;GRUjqSlCaDoI0t}C=;hBw z`s1q8wZNX`KA)Lo>Izw`CVz!2%t*bGF2yCd71!WUthl$uiaUq* z`~LBtG0x4ocrtRa_gZt$J)f+#_Fh@&K~8Q(t_7h$4YZZzq;EUOQnmx5j2VvqSG~c z#irtujM9C={x>a_W3=y{(rerjN4yl*+yjy{E3WcuTkBZKGh9KWYn)9n+-v+{wG_A8 z#9+rmiZ|3$Ayozo6D`u*HM zzomQ}iU{zU(YT=7mo7BeJ1ND7DqC-_rMU^eMR^A}v8Y18w-0CQMjy^T3$3Er zW{*Bt{QVQwRse^|&Sy6mO}j!G6lg4lFf|8E?Dy+|ad>{!EM7xD5}&Ii)6R)|p`Shr zPEq)_2Y%n~(eUHq>6YZYpz;G2A0{VU^3gE~zu^N)*&wAzTF8qWFxM0L@@j;iRgg1X zoK`(_tNlysoZ+}bx$v#Ol=|7$83 zoxjF@xj4?-kZ-w*#zwx?M+@?eM%_lMa}KKSkEz2#E7dNY%Mx7PLw;_6gAKFtDz2iI zlRK?eQnH6r?_yun{(;Bc54Jp79G00~|DI8~f3ySMPmvB~^dT3TzoFQ+U*S3`ck`tBT%*ki%$UF@cx4%If{i(DS$FEZzAIGnnbz zHywAOx`lV`iqM?nRO%az&fW8<-BFR~{J8L#arICG@0l!9&bGQpVcQBO&Gdj$u)c%( zJcfis#$P6=8=EVI>G_qdq@!-SmG(kRyKZK67dKgE6|bIz5n8>PTpGkuODjEq2hks7 zz)U{|uAAdQT)976pBQ!*l=dC~)Q=8(C4%I0&In=@#YmM1aeYG)BcwjcNHM`8LQCTH z^eMefn64J#(ql+4P&#E9-nxG>bRSu}n!UFUJB+66OQa{`jC%nFJxt5*& zYwz&)iqHO7E4)W*qe+*UG7FX2T)UkN9ATS&_Fcwfo8~h?-czk%=Wh>!JY5ra$wfRy z&U(&>X*gjS8wc_VSe@QLgwI~$HBRTNQSf`&ZMuO3J{)r7Y-}S9|N83T6G?tIDVVaM z4HJ}rj~geQHnRlzXXp7&iBPY`=k?CcRjIA0Nq8DPKwHn_5$sfdiURte3aYmvVo&TY z*?vw?;_($3O z;Qsn`XSHT$8L@llAKOXY6z$_W0N7w;}bJhhNBrk`un9IqURf(2I6gM+&>18 z&Z^NS+_P6qSd<-Xb@O(qf8j&SDsFjqq&h}D_GzRERD|=_>ebr|Y*vI!c~qb*)cez{ z;NOn|O7+XK)w}xA@!LLc6tVftH9Zg`450%;ycKpL+AZ-Gqm+Q7>=$th7Jq1nHpx3C zL?rt)m}A?B-Rc&MU@Q)8N9PRTvLd1567VNLf;4Kp2r9|98V<>uiv@1w5WNjZvMe7L zwPt;L++2}s5$Fh2GY!-)xkvkE}x@b(5cP zE#7-I?<3cTK*$s!L%q3;_AtnL|6*|09KoZCAe&5Itp!~KCU~y9Ud}qN{xlWzS>N5g z)Rp8 z7P41FK~a6CBdWvms2xNPJmN^GIe&-bS2k3ti$PH>`CYb>C)L!f!HLLr7zFYAW_rDX zvS$CdGwX*Xg-}F1%G*}nw{GFp8nX7+;aZwHF@Y#~IZT;n>(x>l(`q?BHbo;z@&q9N zBdCF>Q1O;WiLoNO^B~aLKhcI&)gn#iC4#gUpo)mzK&Ap#Do66YaYUPiLAR{%=Z$wB z7njQ%)u_&Yw<-J{sb)F=4i|hHh{fCH*p@y@0;QxGf7iqM_4dihAGV{-acvqV(DkDQ z)B8#V?zepKBqW7)(bQd2fjp-UGm*29Ah3H3i)U4HAmp>h5iN9el~i>`dM^8ehttGT zYUzo;*LRJPRBXSKJMJov5PEHdlz`*ujQ3f@m_5RxG#-Ki*o4htA3!koAwwkmO)~Zv z2;6gc*-uWxrXDf`_zk8*C$KXQ!dyFcR6I0DC~eN|DX)+KvIB`Oj=1E@@1@-j=JOWR;!kHG% zKt9b57+J6cd|xcYjZI{`!uq3Mwn(uV&()B8wRtu{1w=E0Sn{2O-JuF2EC5oR$XoQ$iCDHjM);-?8VI4Qr$Nsgl3xIS8J3O1>sk?L z1Jpp6ioAi;z&{3iDzP1;w&vu-v~%!>2e$Rfw;;!9+(>GRZ=a?8CM zLrd!^n${60eDnlprm@8`F>P%f3!E~u-od>cluk# z%;Ct9Ld|XJYu@AA9Ofg9u4&0j?w^*s@S-hc0E`nU) zxizRYYCOth7k2`CzYiax=T056Za@k4jyaa!^g1o>n&FQ5ICQ3Ldmb55U#&zLeYq3A z+=8*(W{3%L)5!`;BphXmnXAeejQlVWeH=TFP%_+`dif2?iGP>N)?9lnI}Bg+Q#>)w z9IIDrU<+B@2z)m(6FmI2kl8&_5VsXO&mq499axl21{DJuG?5`q;^-$@$3hkRdm6_E zu&5|CxV9Rj>-CHV@1?0Jv=llhM)Tlq3yk}F`80S~S7v`-PzHf?E-!I}zV2IEzf1E; zCWMU-U)i|*37Tl6&@Wlux2;y}4{TJ+M;Z$Ah9b9di-C3?HIG#JkADhjT|NBEdfo6= zLTs#4u7M5JfMl=QAc5@bFR1fe>Gog)$UQQPJf6^fY>MW~;7P-Lfxt>{h^LF{Q-HG>D>xpWZ?1XN5DA(tsr3JOi3i!%)7 zpX3fZpFm9U!;6C<`KpOyXAwe8Fl8jZc``n;7MG+!@EU*?A1{WmMuF8L9gv-BQOBtA z!ZsX)RAW#_v=5P{80Bn(4==c$pM9Io5D-a=CiDCe>kwTNOin`>z5vEbyzYaE(I>)| zc&YmDEgqpXgvtmi(s{3>qgepGeL8iYLA=sw@5iXGl8fAhd9SBpdln9+qjoUz^#~^J zr5%CBQ1>u}ZJA113b`3@I5G11w zA$}RG$q`W_jc7`I67%A$53(b8VFa`7M1bVXf2toJB-()6CrM<{Z8`Dgm44wqL3y_7 zVI45UP+AQ4985a1{Oc0rg0K5!qr#RY%m-kOc*!hU!c6N2q@!-`?__g~Uc>W|@%r{PW+T z33W2ZXafBD2@b7)y*=!#&3f2(pYQ{)TS>VXc+~f$acmbnE_y&eBroCzfmA9W&38N?@xP%#>1f~8|qLg&G-1pTWndxL(! zW<3)uG9cz-x{(nRnPUG#+{^R<_+4KTBq4K4Z3=9%8I!Aa5wFio1X}<%dl23LnP|#tL+A5JAy34@sbQ&i`TfmIp=h zUF%zq9N%pg_^?e3`}up};>{C`)2*h4hAS1vhT{4cGfjcgpC5s>ZU+V=uenw2b2vE6zYlBcQMAVqAu3Vm8$kVe=5h!b z*!*z*1q2HGGMXmq1EtC0x0H!_9j#R5e9HqSbT@cyCR#@M>+oUL8b~78%Xlglb13rb zsXw~uOW3Dh)XuA4tkZ3mAExTz}8{28ZG*eoF~4$@wm-fAu!)J7lOumvfr#Pxy;|ty?nzB?+E8 za6g<1X0um{d^wFaTwMVc0H76=KsMclO%?yD)!8w*@Du_OwS|0sw?DfpfHzFmOJV)7 zZvfWhssE;Xz{R5KXvoWN=aRj^(xpCcV0T&gJ^^Tc(IQ#VXil37CYrd1_dq8DP~{*%gqsylZ2( z^dQt*<${h9v0AhK1sY`KK)zkzY%SoX-NIv2$?Lzz@TH&ZE`Y?KgtP(L(NV3d^)gZ) z+!kKwke;SF?+)0 zIY4hcu1!btCSEUC6mIQ~ql{pT2>;j#gsW{l9-z+w!1&O;!Vnj~y{WjJjs1DGpOZh5&QAU9;X5Ol?DOamA>iv)ojrmrx%7pAuK4YY+| zRupqsb(nxyt`}%f9v3Fo5HcM(OfjLm_gL3cv8a6FOkVY*OO%fy(Z%Khd6xzt7oHfCS zN}2?U$2AO|`oP0zP59Ox7Zd8qrjC|xLC?s?I8Q*j0d@_=5q*8m|VD^LVCc5jr4qN+b}2+>?Rv z>^f!_&3!6Wm*41DO2bcKEhJ7(z+V$!${TJIMizvd9HyhY0~RvF$cL)#c*UgA5*@Pw zC127%)T9PSJLG`wdpd1Ii!Xr$f%UNL$@RG6jGvUjTaQk?NiYmG$jgVzs0-b0TA1L! zEWgV|`>(mdvD;*o{ME+i>^YSc0b~9M3BLYr@YeyV406j2nY7JhcUN+Vv@Gs4tkXFpzd=?c%I0*cM8jyw#zGlLWqxVGN|bjl)3qucSvi z%{vF*Q38XeX<1cjipKXo@@x{?&0}*+!B|bbB&cBTR{$48ihf@D8Dhq z0$NKVH(U3U{ik+6%13DjEp~Z#v1M%!^y%{9xk75AN?f7$f3;2u*Q}~+#RshvY_tCP ztgD`xXA-?-auHm5o*VzgSGc!+Lf!Q!p`!qi1iM@XPhx#+k!=Utns7O&%JohtNWS}l z`PJ2p)z)lK4fUm!;Ow61_^;bSzn?DN9e} z|91IM(cdK;;`z>rh1{q(UdPMV@WQ>Ki*39{1-#Bpd3_JzB}821cUhizIO5es6TeDU z8I=uZKjLCwqA+IGf9(K>Hv7qJ80wtnE6;`_2`h_}uQZKjFIV?_D6=)yz;QKU-Ghdh z3F7q-HZIZ|Yl=V2@c~A%FvwA?4SdSPrt5G`6kKUBepK1!v^%`k#WudNaMJsAXX&X! z=DYFOGbB;Gg4(onn1=crz$ya2$l7T#=%=m)Q; zoS(myb$Vd@-BAgzZ~pc^v(46twy9WszL_I+q(3DUU9WQ4{b^cr377v5yT8NQUUr4K zag+aAZDVVBL>oxc`Lx;{TvCZK=vgkbDdj-sEqTmUVFY@o4~o+7Bx`eQf-+_jQGBQM zmxi8I3Wx@{(JIKKoGGDM3zx*nq-MLjd>ySTE6m046!Q?#X3_zuhlkISc%}oz0)}p#Ha*=-yNv zc7yh?Lmm<6HOw^;%+n{4)=QH@;k<-Xm>XVUg^TUoaQR>m+K@7C62?OCMw%?N9oIV^ z$qbG&;p!BEh4{h)W+bY^H4bu;ectJgb*H&0Y>@cyMzIAqnBoNp^gkpfHZ^a`^a>>F zHuBOh`N6HBN4cVI{u&vz`XqHNzO>XCbAv;x7*p=Ax?!S*CSqecEl+8=l7{YCqU}uQYOF z@^)z-4vEae?tmr{Q}J6Fpjwi1sr2PyMg6yM6b+H@Na^_f+-V3qRT(kVch1Etex4!> z>a1>-h51KlPaOdW0_=3~Wt!EDJ_k}V&nc=jp=5Lr*R~ku856G9<_c!g#ddHOZu+9W zUScXtYelu)uR`$_V^pKhmNoVEb10GB6OC09GOAFcz-XC1m&f|)wIJ7js)+o8 zo2_I=>mSHf$Qijshx#r$>-;8B=|T$((V(g(SCphX-$yeZEG5d5vYy|P!dit$DI+QL z2oJV;MV~Wy!Q2>1&&@dHUTFUd=>+HY>Zi%IaR&$w`H+Z!*(cNH_-aO#jSZ>@PMwXz z>Y?(5)z~Xv-`(zIrx-xs9G*fh`|jbo+V2@}vGIEk!+(Q6T_Os)Oi4z*@_u}HWz^zK zokOEvg{&~`O!Bom!AjCu#B@c*_H3a1WG~Y4Qd`S7t--&f-M)SC7wb0zPwDjTSR%Uy zqJp+>tJv@EWydo81AV|_JF|vCn%A8j5yVic{`7}a6WDrn^3kfqEl~dVGqc`GStAL{ zLdXDIX+BD8n{*L-0n%GVP&bHEXeYw4>h%%i7Hu^IkG=vs37d&_{eX6#nW&RT;>DD5 zm_R78 zYJqD4=;o5T&V#7hw14hCmY<|u?NgCl%(*LO?LFpksg`M#JeOab{LyOi4c>k|fa7QI z`&qmOQxbZYE%)sn9h=sPqLWBi^F{(aY|M0lFs=n`52wwjIXfmkjfE)$Z zrB&5H%~=lbjpvU(F|n3?KK<4F_rtLB=0=gbrS7e<9=$f62rVp@9m~#ZN9H=G1DUGK9>q zX)Z1oAJTmf7M?r#yZO4VH^aB8Tq2J*0?|8|V*DF$d$N^NAH`Yfe;L<(Ctx0J9Xiy; zS3J6Obk&^ge}17xn{kDfvl&6D5lX+qojW+NjQ1GPXBGI<%ZD(}n`hZ1woS!z#BOb^ z|K>i@W=%<@{_m3V5N~J47=X@+vc!l}pWta~tvJZ@z3&Yf*I&jl$Qp9gfOh*r%h>#i zJApk<38HkoT5r<7``gHea|@h$ARW^uGpUP7e*z6VO(EH>5s@os)GO)K_qsP$2Cf_V zr_^Z_av6Bf9*fuLg`x%22xHEB_!49xf#6#_;+6Q|4`O)DkbYiT)Skg>%?Er&iIN}ZW0mv| zYv%5X$yNpv&{!kP1!dVB()2zt(Y0Z+uW@q5zE_2~t0=JDzavbHT|dXXha;Qv5)~%UNj#*Zs3)dV{gI zLD*a`TbQ{dPd)eEpABd9wrDd;ekMB)ay*k6fkmB=;go>fTWHrlpZ+S%EQF5uQJ=X8 z1WR~y{*(LIZA`gfg}UvG4yUl45I&;Ys_C_avf4*Ne94;ZhPVH9+z|H*h{bQUqM0Zt zewzB$Qi;X*BPFl#1f8pWYriV^(*7xf`wk+hl*f*$`qX3iraq%Hk%unEqR7HfzYa8X zUSLl7Dpi_NV%lgr8bX2-`Pl$O@m-JycC;K*hwJd;Ang&e#qlH3ggKh!@$>oaf`Xm} zlsC4QJT*UGqk!vPSt`|JvxmlNZV(dmlW1}p)MH89Fa(z13eMpEY9rYV!5tA=1lXc} z%ZAWoj91i>DP`=FQT7;kpQOIX~Af#fhIB?Nb$|_ZZ2de^#$^{H181 zuJcFYY?7DxAmdRgl`gSo6W5GnQHRynp5s>yneqeU;MMu^Gv#L ztac=J@Z&CC4>^Sy;!XpetGbixgdNOXyN;fp$=*^a2Jo_<)!kMd@Kqj3lP z$qX%cZiVk{&c>e#s`)b>1JqH|d^vcDGfz}9>6e!PD`|CV65tjOeivXjz0`U{Yo1upNS1-pHz*Z?9Jj@cNDZL#C#oADDHFF^e zY|~l;_G6fty)3FHt!^y&+av8YB8miY7YPB8D%qu46adtqC_!AadOK(*+t{Y+3ML=I zx&r!$E6LOX*RIORXBug4-!pd`T4!7z_SPI!t(Q_>`8P6m$H;DZA%VL{CJli@lwXu3&3Br;CR<;;7b%(G3iS;DN0}?r`y$UL-ZlFe~ek!2JDk z>7gP|D)K^p!CUvJaJxuI@xYimS6xnq-Iq(5ABu>Cw2a!@f8bKK z`*c*vty5@yX!}TOaRe-AcN05Qlns|&0;pIe(^UlCpsC@S_K)^PO9^fa)hLR|owtAr z!Pn6;E#77A9S6^z>774bBjB=AO#;_Gvr~`@?!RHx(4uBQ@7ZBGfat@AM88Bw2dCHg zIl5`u_4o*_*Oh{wYChyex6d`p&|aX{T~_`Rxa+QOBWc4GC4R3_rq?t{fXJp>Ksfq& zqoG42H})kj)9B>{&B4@PfJTY2Z{vm9kR|=ax9FBF$=fQQe_Jn0KM)%;f68QoL1m%) z=kPmjC{k+GC(`p#>tE6P)h-|0>f_cg%Gs=z?nA;v@LCZK@w0RBtTRQZ8R);+jY zEzV1ctgiCYdv3ZIoaSHz$u7Q8^IjuSs?UuN^ahBxI>P+-s`AzE$InXr118K;pu!)n z8!T;=4L{V^ni2oyf>`3616Y8?rUCMPNbOH=h-WTBMXp>LqaW-dYZsfa^N?k&c7@_U zN5Dg{YF>93?1mk`9{+HF-u8c7x9vnb{+gzwKtwsJ=AYY{{p3yc-^TZuBmK!78s2R$Rpia%kw3lIYoQCtOmeohvwgV zAm&XAX^6K*AQI{II!A^KwVzgR2Fib@)OyssJ^gBcd4=+cm}qs1k0vp19MV)wO)#O* zau_;_7t-puS>4~?@2331D!N-V@f3l|oAMIJL*CxwmLW*C^N@MH1#}QX*<<}@37~a& z^HIVPkzF_DAApd_r)(RP8jO#O7#nqucNh(Ue40gl~?rQcnN=HOE zsHOfBR`8YbjM?%`KYem&2*i1{Bp0{rb-BiOI|e!a(_$$Z&HU*vgxb~L#WX{XdiQMH zpWhnq-A58pD&YZj==u*oDOYz}NzK&_?v>&z*8WJmMY9t>ye1ixVS&wAqqBw)1i0*`6&(r7enZJJb@Pudg0i>dpFTWsXdiLNOX-yUVac4TsUJ?j@E# zqh1Frn3HzUYhLw5$_hoedVIV>p+N$2pb7>4;iq`1HY%pPCC%?0YGN%7Uc~;vnh@}N zdY;f7qsvU)GmXk>4BNQmB`mKx%X0H_-PZ*WQcO+=f6^n;CP|PND_Y7eAyV+9bt<4& zS73hTNq<;`CfhIrjlaxrPYjxF!+{GtTnsKF^5217=xkA#3^cPgT|e-x$k)o4)pbWC z$Mi{pXUGs!FBMy=uWpK?{uPQQ4z}e>O~L>nwC@kLrpYLBCGV0;dFYFV@RmATk~>oR z;ALC~)6u-?S2EVta(SRA0?|i%VWMs^Wm$0PBr!t^lc4=OEn^u{TSi#(JRg5(rjn;T zO3eiYkwvOnyZR#k4{&p=!xjzl-}vt7-}uSX9~cyd{wKYB99nC{kbgh0+1Iqa*3N+l zBdFOeCae98jya0J1QxQg#*1si&N;79Qc#lN>lqBq7V!XZ^W%4#8pH1QhxA}{Afr)T{|`)0cf+9l;G!Y zu-8uA4i;hJ$1H(lAiXq1A7E86Z2bJo`_|z+vKqPRz zkB1yFB2lX^y*G0B;Euzm-z6eZVJ?DK#Tr;g#OXGMC0G%yK>S$`D4KFC2}!Nq$Me(7mAdsQ&k#Z&~vGqKm`uE%^wlO80^TmP~d%5NM75Y}+Ng)N%X?C?jPY2KxH`Ea3Fx>DOWIULZ z!~5}*=5-VQ93H3wUq4Mgy$op-ag3Fd~;t5)%01Y6HZO*1)<5mRJ{ zdMyv;XKRMAQL1dg#*t#%Vj*n1*_L>G=IQz!)FaoG63){?D%}@+sq(P*^vJDt>x(4` z>}z`sQontjNOU_q((a|vv&(Jy6b4Q^Ui#lA8pkpmSQx$S_sA?`*@S)1t{#0iGJ=}^ zIk^5A)IGssVZ_phB`@bYgrdASPT>=jWtx0^C~b7%vNpJMla5 zMq5!^X9%sMqXu^LEs!7g)B+1GdI#TqbvtK^r(H`YX8vHnL1{%gFs2?9!;QT|!l^}}(v z1HB{>jTf$?tEb8&iKbE0f(v_)t=R3#z5}?P3?ViOZoRgeVyg&;hi1n;=%4mRk;^?R z#xuY+g@}9j3}N89ewCMD2=Ge~XnPn9?Z|YM95lAwQKWKekBx~GRUwg@O>VH;*BJ*^kRLW637cON83N}L!n4y;3FzOY>*For|yG1R95DT>(iwcql*W8M1)T*CZ<3xfAQ%xq>{q3`XnC6mrp;=1{o-l#F2q9oq7 zzP*-}-U;u9FFp1pngaiEzz=sqm>4t$2$6{pXLp=`ZQmU>AI3B^Aiu5l3Ku6dqU82l z3G16nrYTV2x{WF@mRu>W0&BWhusR|K)% z;7f))L20de@*s6@%TwRCBOdZYDdqv46(3aqFA?Rpx&miKdhT?QC;fFH#an)_%~xEL zkF1>&!+@6>ua)<|m?&7m%f`IdUQv>V@@-EOCa;!G(YQ<_F7PKXq41M;-P>gmBPxMQ zet**cHCc!HS!Pqd6$m=(yKac?@%i`zs#!Y>$ejbDbI*d&UxH@n%Dqbsi-QZ%Gvzad zjMBy@S^aU+{{rjAqrP>O@A$D6rx9<6s{>KRZEwR^xJ$}4%2vAt)4Hr-n1^`)q#*B+ zn&N8|IUX%L*pY|RN)vS@#q

          Nayg;c?}R`7zSMFyT&mrVednK@i(vrfW8vGP5<)p zW&e?u2r(QKm$6n1(n+xh{%`W36{E2On%427^ACJn18F8)$>Dj7zHX#~|DC?O7B}?W z<5aHoy6;wmXJhCyb^*|e1|RPc9go&Z*Lu<1aIpIdp^NJAk=#w+JLbAij?7mFalyQL zA+bE*o~>M?$N397{JOa>ZTBCJxlCl7eY)2E_XY4nw1PZ8xQlA41Fd|Gzy0TZ{>llf z;=>)5Eaq*EH1Kp@?OJw>FahYgzFhPCYFcOZO!=MHznD($Q*mpSjiYi?gqmN&BfQ~9 zoN`&vn3?|9a*6)JHNY;2VdY%*D*)9c5t7D0FL?pX_UnX_2Gq^>FXn^;fb8#B6P!L= z{+Ajg@A{i-6*J7;Tily9f;Q_6{|)f{lO4v8b~K`xm5hetqDU*Lk6~`gVx!WXwi%#d zs_Ldt8&=e#W-C)_O3TWe)yWfG$-n<8p$y<<40+`yhbdkN3yX}Xqi0#Jpl=J?u)kX4 zs?B4K=76pdVg1JRLSK_UvU51I!bj$pDd zEHP#M3BQrQWqeFMnar(goxjf)1Il5el0VI{Bos@vy-R@9d!k2zw(V$@^oQ@VV^AgA z#zzi~bxybP7u$N_Y&D%tD2pgve;Wk8=b|YmvJ8g0*Wpf<7Mn#JSQ=O^#EZQk`1dLi z1~qs+An_H0@opt~R>6qv94QhJAmX&OLrDvGG-XDosCnNSeq-QjhEfMPN|l$R?6zlVxYtdPVC z|A0%ZDD@bY@>-qK!3zij)&4^oP|yne7Y)+9vQdNAb4>V<*d=e)eR>Ax`&0wy6h zqOoz(cY*u87nub!dJ7V*o>FK{ho|~XHeplo1UdiV!4a1=C#OkN_lW|{KncEP6ffY& zokWd>*Z1QR(VPT;)(8U~qKn0ul9gH+zcqsV3XqXf9BaIN*c$s~#Lyxkh4T|ezm)hR zY09-qt-HqFE9qx58NJqZ4%23QvpR{mFUk6nbj|R1WgD|^q;nog)%uKqaLQf~e%vwq zx$vZe?F}hT?DIF`rf{X`fR5a6I!R)FSfL@-cvGt;)@Se7AwM|P!Tk;7bmng^Z^Wm| zagGi9bkBwJvExCi0kcfSJTfM8Le}K>tIVbbHIq}FF?cb;`#BW{ukXzJ6>+jNIu}3-{m1TdewPaJ4Jl637i8f^y1~du7hT`!xs0 zgT6J#b^ytq@}&?R^6SZSQ7$)Vts`<0u?`>O%1^uMJN-l@+_OJnzhi~}m0Tta^W*uw z^X;GENMZfZHG-x^uxga?f{vsHzx}NFQ)l=yw_#HS4{(%`daekFB=fF&tmQ?=IPT+*N?TiCgNF@{U4cM0zB#~p_0t4Ec*;|X7{E~xpuD^+` zNU^B{)Rs|C5Ir|#<~o}qYMHyT87aNfs*1qcmHzndtma$YW)<$l!`t2acjn&#z3JK zkPkE?4*LL=W=Yps16VJqWFPPBJc?cW&^;I!57GUD%QQ;kAu21$7?g|>^9m~u^cu7)@zNsHmxlPhmvi;wDbP}TM9tcHerD%R*08>BkJwpIDj5vmwUm14{w&1Ldd7kuZl%0x-EV(UwPDlUd|s|LDu9~Bl?4nD+iU=DA$N+78cuO2XxP@EJv@< zcgwRDA|jwPN+a`N$xoWD^~YiJ>28AY+rpe}^K)RQ!OEXtXK|{c2dg7&T$xw8|DS6cy!~FQ=`1d^+PL-S`F%&_7(r415F<* z$r_de5)?k-hV_4Rz-_D}su*OJ&{m%-y8GVsaGbuhWU#`i;kh`rozqY++BflFfWN;c zYCGYgXoefsT>t1xz6Zgg^^Gp68=Gkb#vVttniJ-4CJ*gkke1M0%R<+Z&B0vyHq3~_ zjQa!yYah<~CoxLHFuEQ73@li2F67@Dz_GFv!u(Z|;7HKf589Z2rIaFEbcSD= zOUmA<_n{ToraxEEcW(Ew8SY(zJ^TI&`=eU*%ti06q!F$}B44+(j&UVPJA zZFD)H!5F}AH$mVf83ECMigWW7l0m@zOa&_p!WL3L30JWcI3vivki_J+kJ$F%tRgOn=1fEQ1hT&^)(7P5GzTj8#}~f7!!-v z5!+%3cZvSx=v8hKEEZbj=__Pq${hb@q!akp9H|ZlOaFhmxbAScny4Q{i(bMmVzm%$ zv8#rt(Mxoz1R<*j(L$7~Lyh$smwQKCn;UOkBxy@l0lEb;A|@BQO@zWc|$ z^UVCt`JFTKOnL6inXL=e?w&8tbh$eJ6;Tp+29}GhI9p6xa|2%Nl{B)t>I8Jw0}_=| z+sLOq@n)+?QlpJ9UCasDgB#R~GzI}-@*+YGZBwr$h_N<270$H?ik|qQ+YVQAi;=nu zTf{(heTAYKG8tes^=p

          7akmbH4o$v%movw1%mv{jwNR28*KPW^VG6NgU{ zn4Gh1)}x=?N(lLte?hrah9E8jU?jxV=&dP8gU4DDVDY!{+F&46VqT6yea4I>Eowx%@zI7lHzd9Z=bC68&Hq>n|PQv=5Sw%%hC-^Rs3Q~ zYif#w>a+5%E9<pl2-nu9VjVn?1i6Zk!Qmt$SmHYQk7S?H|@=b8Dtq+d1uT$W#)M z=j}SC_GU6#Z&@1-v$qwdH(7CIO-j;v7I4<=-J@ro<(q=AR&Vd6K(-dLOh>RsOoFST zE-s;!iLF%yC{z{&cj;aR^4~x688zm$ZZ!qxTx-PQW zM<8pv+I^;>xKz^E#17PooKHzdb0_qSr&OCX*1IHel~iF&V<#j@Q@D8p!G_ON z9M59`ll1TaFADUXG+o5~k;8}F)tjK#9~~~{9}}8}bSom{#|R7l#DAFMuH<|snglO)q8bkCPtsq9j>@0JUY)D>^~3Ew2wGA z-Q#xG{pXCz;Q|5Ep2^H3-|30_sjwtN4C0z|2ar*16+js`sFN8a5~B7@N9>M39HJFh z;WoMMR+2a}l0|cEHbp5e^W3W5cSJAZl*dww; z?$iD_uw@QJAm>+?bVH%8!8_BK-whDY%PLEw7Bv;BA<~5q@_bLRG0JGK zVroleC)Ftcf0ow8NYrJ%Pm2C|dOYpJs*VfxO<)#;RnBQDIS4Z56Gi|BH#2b!<=%!a zP${`QZgu`diD=R3KQ!fY%hJLj5x>iA*bbv7jpooma9e}kM^Bk_s#2fnI2Lo0wD6Z0W{<&WksB8SLkuTjPV zL>*-HCmokAOw|JvHr?UepHVF%^p<&W9bzvkd>(E(oBh#r-8ZF<~OaNltS6;#LuKu#8XNee&`+glD~@8}Kbb z-)~J~Za=wEz2ya#oF=sU7}_3MyYGh#upU0x+{K{Ab27_b?Ya>UTuSd{ogbI+J?ioL zWJSS(Orn+1SFNjT<*4yjK}?R=d;Wt9P-duC1f}d}_2AxqyN?Z@M~E_tm!6xgRjL0>x}RJ$syXoII;1We4nXD?GLE8ep3^qRchDDsx!Sx%Rq%EVcyXN9c9 zSQvN2_I6uLwnr1?ma@T)FRe|9OW4Hdm9$i+8dF{U(V00=^(YQ@sU=*Wb-&KWo}Zj1 z>y?!F$x@DNQl-=eQM_Jpo$g)kIpJg2kDeIiL>aU&7*Z=88EocZNs3zrR!k`^gkJpE zb=aAOa;QUBz%S^y8O!wxnfQlnXU1mO9vO>lF5O}ABEU~soj=X}Gy^x`I>wz3O~D%y zn(Su9bY%YAzfS!RNe`#Qw(<@cahBtI9|SYx0)3L`&ENjHKh8AvkA-q(m`aw3C?b%>*xF z^0X?W-#7We4%&1`0%Zcj4g(L^Jp3y;I;hu!)2ME|Ek>p_c5EyH&5)Za<2HskCuIhe zCr@sn77t)Z3}yJO9>&9VxH8`T%3e+zEXYtE1KW=u$uS>Q7uZ+tIRz>rcBgI{j26=A zfuy+Ue=JtGUAjd0M+M=rN_Lq>%GzLWK99>}n`d*(*h4-aSfYb0lQRJpmJ_a#Fg=#^(?c5rAiQ^y-lNTd{(0tD~b}N z><)0W);w_8LR^Gr<_eFH>q#;iLmA(Yf>nRjt=}5xDJ!ha(e<%9*R;i?fGW2y5`HVW zxPW2G_%AManM`C9x_zbCbfthI#U2-2&KxvOaWfNn15g>sL`&VljXNVn6>AsD#t*Mp$-y zFrf&3ieAVv-P2tr25#_opIx6Gr}?x`fA6v{h67i&8FwW5+XQ`QvG(JEc>_d4L7^{i z(ct3{Q1or}L+2+?@HVE;^XuD+;*^xZeg0g*`|W)`w#8};+nCHT4<}b&zH;rULYZ5A zIL;JStMS0gr|Yx>je~Z&71L^Tt{c^9=$qD!d<&+z25lRZnegjc62{6f-8)8Wr3ct# zJNwy;5T~#F+j>Gr5n^|29zOS4(9f@GJozCH157F!#r)m&f%z-ml}--XE;nfrPXv3VZS3g&bpz@L$~;_ z8!JTQ=dNw+Ru(p%#SEks1v$pDj$v!_Z)o?fDd!L#nW)^5%RlJ^1S06 zkpLR(ZOL54Qh!nuCXlAS9iOetFhd(ttwDD*;ca#=UQHZ6TOr^W-Wj4os*+(Tpf8|P zGsN$LS1qOVu=D!$%ePL*VZL>i;ft{0rRgw?C7fGQn5|SUzD%IipUON`5^+o+5)ub^? zV3TG0P4-=cibR}#Plx4GRlVh~{d8ofr5B&92*SFgnz3^+kwDJ3ZN!|`Fqz$)jrVJW z51ARjSN8VsTP>S{Jx}Eb8cYYGmguuCOdV^?`e8GqsNaI*dx7_ce6{Y2j*Qx3RIgp)o8jVu) zV8}eytq%(kEuy(;W~n`>VU(@()Rr?IN^^TH3dg7MlaRXA*NO~ws*qTFER9_3!)&Ga z^-nc4l407pLRIeW0HL~^nnwyLVQsSX45?k$56HQ#g_-v+5~@=C#d=8Gu6}DL8#>%`tdz0i=Mz!+iPve`OZ-SryRskC7k=c zc+_Dnf|#{|mHYVQUF&6*ofdH@C2IL_53+fQTYpp?iE9%l>?t{jD$T%KoQvz;Ua_N~ zBiOS=_|QOO(vd3bk4D;&RlXR_AJftQI->1!@JcxSo=*}5lq?0`^Kowf}6!}*VE3_K-AsLh;-z36x)+2Noe7szsi#zzSTSXPel88X><0QYh1VO zQ`-Pvu({e-Wd?XXt;k?R)sP0URu^j^kvLUcMDHSd2T|l-|MT(MDiRSUoIbz)+Rjhl z-8T_!DXG&W!$bkv#~N&{r7G}daW-FY@Iw{sBKw$3vpD%Y%y%eaxl7_2_1}_uSC!)= zhs1ZNSvER5ip-ZU4C3MtmU;FlL}?Db-f8TfdmanGl=sCn3Ke?;+=mHSg$ zF=DzmqF1A#q^^thUt8jplGBI(dQYV02KtF)MCMcc#mgQk=BKt^MB+grzL|(8g~o{J ztUr<>GUebY|Kf;DEJP;ek_zxVHdy!By1^G?%5-{Y(lb9y^M?8z179$gfi?REDMnNM zYlb+z9R;P?aRJSdWIHm|cmvj50TsMqN72+L5x;|MJHZ|Uk+>_wYTVVCfckH z{r@Da+lom!NQhl%jFh9DtVGPwh}!Lndw<4->sqo2PuzoB*YvcCdxZlp&BVNZiM*h` z2EU|bXslKd!@b=Lw6H40<6w$Dp@H`cVDbjY-GqKeuFQ5Vq-1ixV-XCbjjS>O1E?a{ zL@8~z3E0b=VyTo?RwU-JIR$5CySykGgZDM9T&JJU-R^CC?c%F<^xDKqVm@OqvGnTT z`%_W4q7+b0cIAJcwv^Y3wIrZtBs^49Cc%0;Dw@Rs!;Mt|bscpKe-Wy#iq0f5f3#D-{8XY?*oNs0b1 z76r!p-p_?%;Lj6p2xv6y$Tit*|7?5}(X?|h=bUv(UyFLc{Qx*aH-PUplBXHcRS)#s z1+Dg>k5UdwWvAu~4)qfh7_&Cr@cA9-I^#FYpFBZW2s#=dT)5qIUfyF1?qn*eQ61

          1yCS!f%0@IP4r3pI?3VHZhG~n= z=+xPM>qz34$>w&D5)iIN#;=*n_Z4Z8Mqdr^e z-Ktb*3(w82oAqEn8Pkza)nT4ai9m6W5v!agY3Jz9dgSA~_iE@Dk9X0jIb&sS1di;Z z#Q0sO3Wnb}J?X<@`QpMTC9Mwg-f_FX``!!NpftXwWzoOxnsK)pDacf02ZO0C=A$is z8cg~*BLRKHYPWG=T^w5cL2`hNRcKYVkI3wmIatmWoMkRI$Qr$+rPqs=B3UX(tcF_> zuTN0h5eEz36c(?M0IpRiqQ7xtQ4xT|lz4q5$OH(|6t_YdR0Z=~tfSsRurr0B-sJx2 z-N5)ZJj!z^_2lcbVrV{$Am4)S=)u+2Mp!JpwY09=|B2e?a#mNWnqh7nEv*wBXS)a7 z@Ol4PYARD}fKI5V1QznOf8ystHi{ZFb|)ROb`#WL@B@3d;`BO5aNS@8o2??GMW5A* zpRT5hl%&;EQ^Zch-M9r3XvBgF4R`n!w}*DGv9F$Rh&FyacpMYu;Es4tTU!J}l>Yzs zx~!Anu)7v%Gf^n`AELJ&6_jE-{!}i#ukMCM^knjGJ)Ww*q zx@zPH8tU!5(8ss*j!V*Xhx^{I%`X2=d&yJUK{Ml1pXGmKhmlw#9-Y&{38cvqW8`&* z_SDhUQY#n!a2{@{^!Z1CDvFkZK5!6+d?1_oFWvBqfm;mM!j%we^u+BS!y~P6V$7W#k*Ij zlnV;dgz}`;zhuK5rX_N1J<%xpL-KZj#^&1CZor`?l0A-}`gnKplHPt4Hb;8xJe`P5 zVZ*st&>|1)YOXWVzSsU*XD1y1iwEkvCumhj1ldZ1a!Ic;vE1|*U{}q-&Yf6`usE@}?GJ2DErhFP zKYLExDGfC`>c3&V`Yimg;|Hf-N3Gmr&gbm5Vq+i4Z1&#-^^^@9V&+ zNOQ46`~~nps3L>1ylyq29lE`1RS@)kzqo^BYuG=Q4kwmwKT2>^MEU9SJ?8xfP4_;b zUQD{2!Q{g14-3JAPHY<^ulohaJX=jkMvTl#-F|>G0123d4SOpwtvP5UAs-e37y>gO zn&!*i2iwE62z&p|&OS{WjZnRF^G<;w**cjy`vZzGCAFV0lU@44e!?H&V*TfxkECtc z?pAoBeehSpHw*Zv#9(dJsH=TXn=QUvC!IhmEZi_@D6^)BuA$m5EDZ&+owIcYhY+b; zMslk#Q}@x$EqxPN`-Rz3>W*rZ-|hj~Aq~xBTotNQLtU8D94D=itvjP_g`4uU=HSA) zX)zB!szDlMWtxloJAOPDgIm(X5Av4gKB+IK`nzaSiV75Tq1$3-0dj6!**b{m!}1 zIrks9d7hQbtl4{IWoFI0-&t9|loh40&_AF9001l*X$e&T009gDz*C_jylQG!#PeP| zP-S`bFE1}IH#axW&(GJ_*H>3pLqkIx9300apI%<>US9tG{rBVLe){?Ky6f$imqQ+I zWep9DwU-xh(c0;o=a!Sl=j$F0UW?1yqw4(^0S=CrlZK7!*Q+<~uU|Mg1dmDij-Fm_ zFP9Z~xVLUD&N{xE)f_IIzEtRO4P3pfWhy_+1Zv8U`A$9PadIezEPIG@mgk>etI=>s z2E4pHs${Lt?7tkh^d3{GZdZeCx-VO%u8%FdU-t4HRMlU0mre?|ti;;2ts*ou+^(8RPpLjV z%noxK*2gV8WUoD6&fXs>mON~Csh3_g3FZ}9O!j=O@YGklZ8L1|=x}d73auPHVzj;- z^!9`O4TwoDc2X4g9o6vZ5!0T!5ToUnwaT6P%k!=4V)W8L)8yp(`iMWM#a87Mww=Kh ze>XjUGu3cw%iizaxE=TVX6H%$dw!W^ZJRasW0|UypzLvB%WyELNUn}ox45h@hf^)F zHq0_imZL4t^L#9Jc;oKgTWIj)9J+cv8rfXKUv!@$yZcmoUwYOqh40;#t+GsagA7$_@k5X>EzV(n#QF($)vT54Tto$`PzV8i&RUi zuVz_e<9{+iRt<9*rfUId$I^D&Lv^)L%gD;*Hc@DI84`qwZryN#sZ%mWaj^>*!#{Np#xV_{d=OK^lEy?IZ4|2o3f) zqP%^RV*g^CDpkw3l$=@d&q&k-=P!E?I}WYvsye|Ulx^5~b3TbmPa#0231IpdF9 z_S3L_K-PCIjF)ozIgGe$;6jyGX8<=-Zo2ksnV5a@$1 z$H>fuWK&{hs0y5QsBs&imW@@agES?K=kE@zeFqCRjxgTD0_lYX#rlya!>BoS=e8RX zo||tVR1!Smjq-yHEWqBBgt<0UBo0#R`KjB^siLcXO!J*gY*k?db#61{(!-&&lJ0Ug z`t#<=@wK@(MDJ$c=i}hss|dVkV^8~(7KLv|z4=u{G@kIwa)f>Gw12xwkfgvHxoBux zE3$fozE67-m7ePJ^ngaqyk6}4{DZ5;gT4&R6P$TFGov4LN>p2^Jcca$bwddR>T-j1 zLrEdZFYxL#A6QB|y-Ox|b7q02{>fci(A_Y@WWIxJ!{io_S#cBa?RGW%l?T|jj>)QI zNcxL;Ab)G3{>UfYg!E(jQ9g!A@iw4ogq8(a#;Oi1Xev~Dei&!{mkRumcb-t$;WD;e zlu9(rxbh=q2p{95_^En9f~)kBw@G+JX{GHv=wsajoNxLg6NfJC@rh&rQLH+cy>47o zvZe+oIF+zX0!;*I(U2Bmlnr=$#Xvbp77k-TlM~QZI~!el*-FJ8P+t};jbTw zvXk*c>Tj^R%-@3>>UlC+mQeiN-`2s|)i;W}y8FXTcKL8;7>wBlK*zPd`WJXU!QY&f zs+b;qso8=3N_BxTxP?KLnQr9%LIUgx{wp=frrcIT4Bs9@GK!9;fe9z*!w38 z0SpMQVdy&uP=1X$gxBc&PxCc8Ujw!6Dt?{bwu6okDHqUQ@zhf59Q>WcM}R|UKt>T$ zm3fg9!}*v{HM)x^j8ly8-GQnk-COQq4LK6FkdGI=^tu$nP&G3l3XXT7ux^Fx!3LTuXF#`&n#%g z|8;kbu_q~vSk!9WyZbFx!jlI;%jr$Rpd)E$NqkPxkuZR*WBnm-=u9j)2k)LZLw|B8 z2buv0M|(5;E+T=uK8XNE(pVTV#O{NuJC;mk`gO<9%u$N!o^rV7YK9P{6gEZ7fN-`y zl#d-CF(5V6gX`#2g=_le3of73(l5?n+1Pn>h0*T~L*)tXdBB%%BSTR%j^-}h7H>|q z6fS=K7-tC3@kv)~B;oC+1%KBr&`2W)s2wR4O*Vqo3P0KVFw=sh>X`pflm<>(WN31G zupLyMxWA*HuF);BC761Wuiw_U>f3Ds7Lmq?3Hl1FnD6&j*!Xv8f>nOx1EWUxuMy@C z*kA&NG1cj+cIGU1z|%?Ml9qX~;@Q#f;IUaH!FKu2B{8}*39}&Z@G1=1&}=A;s~<9} z%AjAIV*V|8Vd`0+EkRTvn~dUHu%pg=V;Y}E0ciDmJ@92&r4(rTU^JBiu5~fOs<#rg z%>;|>3%4b7P+q}v46UnfMvgywn)azBZx-HsRoLxW#r?iNaYO9 zuX4TNP)%kvHGnddzN8$ggPuAyk=d+YPNmdy8xmCLa4!c^a)CX0^y$?6hj`-*>?{Yv zgAzc~6rN^H{X2cfq@_iV5}TbN+UWpFtI6FgsE}4`$8_^}^WE~T$Xe9R9=ZF&j4!Hy z*00CEd~);&KiRq1&PFT>`4waz%iDLieCw~Zy&D&K_DuM)SK6yuF@ZYJ?c@#hG%%0D7&l1uj8dkL%(>Wc9qTegD{{4c*Hxk^Ini++7vT;K`|v%(*i zG@&Y)DOpV6p6Guh7u;RpIsLpJQa_2QzDmI1cp`Y3{vf?`(R8|GcSA@S`$6%k>zl%$ ztPK0%XiB!X?Tu{HVy4Ypt8@ z^gO9u3j<*&C*ybpz}0wPy=bS#ENS&Lg(<^zbdt86N}*a0;5$^b8^C zDJR$RFLH3<7zoG)3OQz$$xwc=TLBV-cPbznvv_rYlxFLTfsYJXtJNL0XF?Qze|_3T zo%It^wDm&kN5>y-8e2Z}3RQV!PH~X+1xf$8X{lAhX?7L$XfVE4&`h$c>6zGoeDA9h zgO;c1y*(@3Bq!eq#@_R$PLFZ%a#rG@{%JbeSZ@={F8;{a);R{1FsAsdgI`QqOo|pP z93=rl*7)3G#VNS-GhX{B9mtBEHGK>{*0TvJi8Yy^1{@|~##5e1aHK%IR8?g^Q?%Zt zmq2<(@IKP#55G`CzF5of#|%1ZC0jJ&prmKJ zC76pvL&hD4E<~(P*}IwAe$s^5lkd6Kr=aS)m<0PX)f|D}y-x$HzOa+nL%tFffbmxP zYr`wS8zcG#0Z3Xki`OPH30-w0GwEfFIcY{!+@Js&pQ zmH`gEl;UVc{`n=Jv`4iL#fh$?P%tV~;Cn>?AtIAAR>Jo~KO-!z9^E0$Wi^t_T@(+p z^cWDGU@d~wBkgSwDC4INH?Cw^h$R!%6#%DC*ho&)1r$+D06awmA*_XP$9b zg?SPv4-*Pe&T9)4e0Uo$cLPiZ=Ye&YIVF4`&xS;K*)eJQlLwCh+D;$ZwO+I{pnQ#{ zh)Euo0KhTfs!0fWAx$T+ip4|FOUG%)=$%FKo9$D%O@o^ni)PKc*u(DNb1eehP)ea% z>heN#H=H(ywPm6>W!Ogvj>jnw4PK+ZjreZb$aN&TMBR|&sd)*@xtF0NnniNTTazD= zQ!0r0%z8spZ(aTFF^l*mt6%V|E7kT~`=R;?0ULWroBH;=-PdBq?%;cR3#Z{r%wE7_n{T&2}8SIF= z8zWKE=lPPy$6h!)Vo5#RvIJf9Xz+5>@iteaz#%=sb8eR8E)ERx2?6&&Kj$qj*nc{CD-+>ze!hrQ&bx$kctxmcP$jcIemgvHXDb02xk^ zbu5AWdTDOm!hJ8wa$A6Hq|RjzmO7lbUbVXhrycP(nh$HxqQXm0YmxJpQ$}&AkEy<8 zg1&=EgcfLc_-8*cWv4%wPai*Uu!3i_xvA^PM2v`K-r?_g3TtoW_dn-Ast3as9>g*% z;X}ZjyVT64qtHyKm(6$02(1F)y)MW|@ZV1pK;xcYA))VSYh^I_IHA9reEII?6S38e zD_d)Eo=8sB+Q@Vtmiml4$hu2_u5g#T4T4t4fI#d@2wXP5AjC5zqh2t; zP4iHFZd9+>hS;#Dn3h z!*`DQfzM0<@uc9ecQExaa@skTm1_kWW@7$o-5#_jZqYG#@M9cm47|C-ui!ac*|Z%0 zoH3EoKt}8KaGc7MQWDJ|h~Jb~-e=|t$Q@fLbV!Q4kkJ_uvq1ryn5ghliccD>L-CX6 zn>NGxGp9Q~NrQ_|CNnLsrdK`c{5LO3_B&+$;5r-9GqtH>F8C!4O$hCe8{+nF`__E; zx#;RL)-CiBG&esmi_xe_z76@8tQvW~7VNyR-ETi=Cq0M+UR1gqH=EC0^T4tv)$zyI ziBF)?-3O>IUd zjJS)I#S?)c!FW-he-?;Dh4)#C?CdAuv;kEp7a3$0+7a@vUY~`fwdpLJ+l-yv!A*~KJ7w8YzeYbtW zRy#BX6gxq2aA+oP6c1o;*HplA;a10W@DFK1_0ho`dOI{P7#yRzEuWBu8iQeZ5pwqYRP(Q$=eA`OBN>Ck8k>0`&^_#%^W6EGo0LN1=ef)k2p0CM&07D8H@|4+hL8`;w9ansU z$h*MR4|NM_o@^M{Fxe3q!Se98fZ5dZNOQob0oxu2W(2n22vFv zh)^r2b6+nOVJ5^ByaC{2sk_VxVmxmUoN62oDueD!bG@jGz!chQSA!G$dHa7R3}IJm zLir{&#{-2!bRUE!YTxXxe|1k4-BMyhUZ=g8xAi}}cTGshbl$GiCIbo) zDf6O}1L5ZQyE3>(Y@*0%jH?7Q<^ljHHhteoQIP|4*37Kyzf&Qi<8PHf?O4=dUh9Y^ zR>~C4snZ3jI0Ikk0n1_WMPB1v&6c@hcGBOdM0Ep0aO=va=r$0_e!d&JnY$1OFvXPF zgI@_ezmusxPW}}b?fuyzGE>0k_6btlWK$_$b6h>ip3pV}Jsc{*>nxwX`Le4RP`AIX z$y_7l-{9(Isp!us=@nPQNY!DYEfrADw8&wc`xEdI4^FxsBGbiuZS=}D$LDLC5^2^8 z%jnjY#iyz-q(4s*nmQt1&y{?KU7O9(RsR9Ia2*yMM^52?|0URBg>tuyi1Gbbd==)x z<<$JNZyPe?$5Uq1#oyLHK?Hh(57(x(@;--4>*N=T2 z?fjCH=fVTcriiAKU)|*(@9+5HgojmTu<n&Ei%2o~J?iijI_ zIeP~IogP&kjal}ss?3v~b9B?dBRr4`Zjfx@35;#oPW-@T5Gd>t&a2xT=^Ozvx48#? zC2CzTVBV~e1WyCk>cTnrD(D86CIG7;%F4`5KCG*!YSBiNM+(EO@P4`l`&BB9tQVXs zppu$Pv28h~&Eguyuf;Vb;A=T@{Q%t(a3>#35w_UGs@q6T;-Lat>+wpScAYJ5i><}{ z84o^2>jrqdS@+i&P>0nLy)J~`tm=?OILc^_Mt&)2?$wij9=M9#S+j+8)77xC&D&{D zpBt^if`28~FJUz?-J@ZcXj9Y%gR)Gg-n>`)SK;y78FqZ%H(lT3AY8rXMwJe5jGzf% zL{LO5`1)lpIb`z&eD4iQk7016&Gha@zBt}yR^$zYl*^@S6p|N?F_}*C_%tNAo-3go*2BvipCfJB#-+EFeb}NWcE#(Q#mn5>4zG$EXzK-{ZY%b`m{d zYuUBmhj-5t3fV`wx3Rf@n|Ts#g#8gU)swMBih|ql!5G1(?sl2B0w1BZ?^lbLHlAH% zqeM04gsO!W?cMd}9O?5?{LOHA^P~fU;$OS%7U%xU+2*d7)%cryPixp}BHFX|96@kQni57oiD@B9Nh(Ptk#jDu!!_PnZnGit` z>2@-IZ@BK2r7O-Dgn2nde-}Y&7;qH#c-5lS-bQ(rWQG;U!dUAw=60N%nn&JC1!G^DczJ4aS z=0nW?J=sc*2$F<+MSzAgQNdnqGsSWJ3E(-7Tb(raCC#Zt3@9-Sewyf-)h;-Rv-6Fp zie7X<^Ttl_U>cBr_=*>QXR=>J{@t0#$eOTJN#sM4-}ONpMYAd2v$D^g24tS90}XNq z9j7rZW97MtszPh@`-XrTVK{eP1zyRhu0JRCV?X}H7z56y3qV7Y3T*VQ?g2!^;9lfS z@-mGCTYb!N>olaE!D9n=i%1_OYukU;X3aGKm*bwB=^{3b7&Ie__xXs!-e79{+Hs*j z&P(sXfhTFma6KRWYYs|_J#mpnyA;z#^Tt)h$H<~AVKe_k*mEPkY}Xxcs$7DI@z}bD zoqdwFIH#n@cl`DX8vf?GnDz@?Q>OYx`H2gwdPrT0bTc(fbz0KZob9DKALsIk8QH^q zc(S+z$o3|c*5c|Z;I7m?-K`yXeOP7s974co__w6~fJgXeXi8>=gw;|K+uXS&A#-W- zcs11Mn*x7vW(FX`^Lld~n|m~YEW)LmpsFf@VOeTXj3f4qxb02b+tiq{*1P#)%ikTN z{>CjG@KyKlu8JSTNzbKPU3huzM|&NXFnBxxM-CH{i`WuXkgrTF=@W~fTk;zDV2H+w z<(EU9H0{+-D=I14OthWYZYnt+)j8bX&t9g-0TKk56NS6{es&AHAIm6zz2)AWS*$Dq zH#2YTRT}NO0`g7fC-}l+b_$PThz5wAE^0K%+QK2{nyQIxYvZ3~T3BEAco6nn%ldJ@ z)*&-L>l=K3ZsuH_-q$7&`9cHsp;Sq`+N`B2Tpvv0{VrB(9i_-oHio)wwnS{7y&koi;5Cv5CR{pMvLtHXYsjH10eki~oFlkYn;ZH?&#=Sa3KJuY zY4%Cqfc3HnAjv+VeVb_B;TnHLSu;d_)}{;25?X*Q z4OkV|Bgq%DJP@QA{mVuG*0rcS6Kd+}=K7>I+~hw-hX}E0h+DBB%h-7Kum-)AJa9sX zi9k~UWS_^g{o6LYCe%6Z(d1h)KJDG5S8|=ZkZT8*wvlkyW@xsHlMJ(CSESC`8G?aR zb#)`G>c{(x*3TYQs}2@-l<42#XDQ^S8A9H7X9=2OQbQMmY@o-knO86uPQ0UY2hHGYn z4rT=V_}^C8_i%+v{u1VsI$CTKg~chLsQ$ycJk~~n<)Lp5BH0}t7+65(DCcJKodhGn zM{l24Fv1oZ1MK1qd&svPI=j71G*#jE!KREPU? z-NbnAhUodlDUFP2+Nk?AnHDAC{kx|_LL@jd5OP%eYLOCfjUDi8`LY<+LrHd(MS=QN z{5?qe!B_B)jyAG_D8#SuT+QusTjY}EvPNlOrMHSO?(r)?FZj)D9UjpdpHc5KS*;kQA+~V-F0=Fh1F%JU2d)hPqD>hz4ip{(GGi5mW}qEV4r#L?oG9ci z;y(hC#4#fh{YRp~|0@1V`R~9yu)J!J6R{jWQrIJ-MIg+=vyM#^LcQ?-NrWbDdz~Zx zgVCivEF#=g0+qo-V$rXusTDTn-A;qxtZv~3>g%(Gf7tjxl7$mvo9%GL?%9WWB%qQ*vVh8iG3C9QUc)$ zGRKVm*%JrO{@;KsU{dB;1u*E{fe@vxSpD!kk#i4b;^-ZufH8N)xA$5F`~R8HkwNfm^k$6%4lY$^iPHjGv^=!{MOx(&ysebg`+j<f9Lx*f+ zb};_0L~be{F-x~ebANmyaq6n= zxujISFE#7{X6JAxLu3Of+8P_0NJ9IyIllx34VaU2z}#p5A!dx!zLBY+-c3d+iU`tj z@IIuD7N@ISFu-bFk7SjP%R1S25_@g`IIDOcWtdRm$x=n7rUbwBko|Xod3affm-l?u z3{Z2A;cUa%tO~+qau~9AmbJicq9C3GPD$S)aD>%z3>4}FImpc6t<>%x43RQYxl;R1 z;HJ5<3FPlY#sRL4rI?MF9hcM6D;jO8xNnAqZp#S^$@#w}`zp%N#S=THa)|KqeUDcV z-k2nvpkLiO3h68$k2{wVo}(Zk$+gSBL9M;cO(fznFXWCDF{rIuklvX64J%|?W zKvQnv;M2=PbjYY@Y$deDS{NCR`qmlKP`Wq7K?@jh8{hu3o8~rb^Wc;4oH3n7{GLZj zE#~+PIwMpbXR5F+05f~yk*vr~z&B|y+3LV6nW9QK*`SY(6%7F24=fW!epuc4WYmXv zO@Z-B?wJD2v!^@nQM#BFwYw^*iU@F9$L~({m_4t;?qe)z%LirwX30k7CGw9upHEHtzR5sD_JAxz5>$CX&dIm7Ku#a57SMm@;t@y?Bj=u`0=6rA85g)J=fC@RS;*D`~B`M#OeYQ4BesC1We8XVs`Pf0z8LGjQB3c=kg z%o)G&vV?q>{-yXK30xdYBXEZo7lH>K$I6$r@1Cvj>(*U9HYmceu8VV%o5Y6XnthRx z-@d594jSjKm0$JY$lR6*sB`yr5mnd`nDewk}jia{V;K)lCY1Q~dx zCu?V!OT<;@czU3BkaPkR3{if_>mMW2%a2v$a)h-#vJ&jez-6MCaq42ox&6LrzV>4S znDFrVs;a+8f6{NTN)V=;Cq&z7IgM3?y01F4ofhNvgw7b=`go}mR2#8*E&}%(lXL99 zsG4G*G(>dn@6C=5;90_h?n&=c>4xp$^W?#4sukN2kYkXL!3Rn6Y_6PhS zWh)NsU!mA+V+37pX4@+HewZX;+_&5&CEI8-v(q|{()PYIJ%76x>x^cob48Q4Q43#o zkV4PF75rm7XflcvTBcUF>*47)8Fd?9Nh+IbbN_wTJPOfVOY+B`@3w|>S#b46FPyW^?E|4lD!O00_Ij+-R8b$P3ahy zvO4awlm?Ng49dQzzR$p)VYhM#d|z%F4}RQ~X*Dl$bdVOOvtsc$pRS{ICj6ORg{dN} z9vlVi|0+!4BQLJV4~s@RF2jdV%DtkY(#JU^_`O4>z*-&NcaXPc$xmf})bv%EPRh(* z&$yD1eCewgk0Bs5JZstcTHK|kjSPtK%5rd{zeAr0L|#eLFOq%(2i)32dJx1N0`)>U z^{G-$4C&Zias3b^i(7EJQz(_C)-#FH@b>R?%W#d=0|cdt+ZH;V9wd zW94M6t!63cWGN&G#We8zg+;41c0Ddz(i0>HHA~*f;S4^!jo4mJ<{_RV=eRIul0@uc z(>eTmN=%R&A8wp9(-YyyE=CwMK4RDWF88N#l51u(7_$q){;%lvve^)I{UcT25kXTu zPYNAO3O*XXC0SpuHm}nx_#XE2wm@T;xnC7QhW;{CzWj3CitI}xv$e)9KcppO54P`_ z=^`xT+)4h3 zln^~(v&$3KGnxC1o(=*(TiN8ePRE01Gte$r+6N&R^A-cN5dF*?SBFw~7p&lR>M$B> zzcS%z3~tN5oKzI;b7~!$wuMrUL0~O*w2TJnrj#p#LuS&l-R?7^Ij9-@ z0^z107@#+b4jo7wgp4ZtiBF#P$S38bj>KfyHnH(Z< zkv1>-zJ&roKQt*p-`mPIC?32cgM9GdQNq`Y@-&-B>Bl>n{`NZXK|vblT;h9~1rrs9 zUuS1HD4na9{Zm3oB@a=R^{}2ZF^c^~JJ%Mj6FqI#FF4!LyEdQBo!e+Xn-U*?hRpG5lP*c1p0|sC8DPC+92q zN@39eBKOA*cn-PUsEdl~Bj)mZ?vkC2^Z zt&uE(ZwYkBb@>^?YBdS_tJ?Z&eZi8+EBfv|i8RP@&!wzs>F5+sOc=<*J{B^t`&W{;^Al{S$=A9+Tc`I- z8^1=HLo{|i&d(X-1FQ0#J{96W7KR*O*VzmlK+9T{Jl_+-?zLUqMZqM~du6cLUnWJf zK&$sv@>kQ<;okSqSfk5rOQLk*=kS>IqrSmW;*KMf^3thVW+S4JdNDCwXoUczSbHnD zR=)KlWhQQ*?XyvDl6)5WUkBPgLW~{4{n*-YNwcrX2XY`t8uN%L3fTv9;h%qEm@ym? z=dpu0?hbFOJvi>LmRRHHQ3t8Rk@B;95-xBmY?4E#leh-W*d_Bt(M}DKZ(!>7ZPSE~Guc6)7>5b@k9k+qyYF8N^Sa)TXNpZ@POYX$Enc&Fj0kF5YTD497#HX( z`OME{qeJu9{E!*&Q*4=d^v4WOewhwc2rCkSYxtC$aVD)Nr|ZI3g@%ykx5aAUZ|$9~ zbBjm3r_cjCBQ3bUV%6=&56~tTv(5^o9$Zpa1#9rpi@8pJj)QPyDi}^wT;>qI#muZi z&oDijPe9(h;)e9Zg(O~v6t-7nHe7Gue&r+T6&qbXm!*6z7oDxR{E8HQDOhC(OE*~> zY)t?Fl-~AP=W=%khOc^|qO5!a7p?LZvQZ`^)hWtL159LtK=Zm|l% zmTAlA*6bCv*=^>z6hB%m_|#2oNpTSpiscT@QhdhlX*b@+_x@YUaxuOo{E{02BPt?1 ztHtc{F@*3aJ&>#=La;vBD4_=#!-ODlm}7?hm|6Ezp^$E=PBxolnNn8MED7WO9mH4Q zo)v4l{}^XPSRZaY2MqdGV1z-;rvJ+SR^CY|5oJu^hx@;l4-;m<&+%_U>MPjZw5>FH zd^;KDQVj1Y`e40VqMAU+U-um`xi;A)L~Y_LU_pU@CR>xn*Oa-=pWcNLP8=DJW{tVb zbOO|FTh>EYWK7;J9B00NGv~q5(fdh1=o#o*%kh^sc-lrIhihFUCzr;8`ueb#Wh zOWxUZbKspoNf4sOZI1N~tB~D_f6tSdNj$v#cMMB`ofQ&1cu zL;KA;8(S8Tq?+Zy3k;Y?JPLmQM0}XqllTYByK-we=^$R*=;ZnY^1&d%=ZibO0M@VQ z(#1TgjnTPir0@HaP=s5mods9{M!kAx{KG#hEP925NX5B#qJwB|y<`<85738kye#f%sTn%=JR0j*r!$9l{0Lh`}uyA&k- z&`Lvzk9CdU38WZDeNk8*1_t$O#}szzlM&!41}e~X(P*|t{+1kR{}Gc2pGi<+d<bNwJ*`&p($Ln)CvudS(^^n*nWZUa(R zF7=UB5#jeyeG|9A(_D*d{;ga;mHJaVDOFL0FZSAYq>3AhnNF2@$O>~IIxbCu10n48 z&(P&iDUw^X!f@n(Jwx7CKYD4(Mfb{rtEF2YtSk#@s44O7KVzAJl~97X;zGoJH!V+AL9Wb6_#j) zhBpin&JY-;yDh#8FIOmyMzS-6y!~TA6a&iH90B^jo-deknfSfY@{{20IO4(aVAVoh zp~oN3u$A{O-74GXvofj4^UKD8NbqpnO+OpUjRd$&LHUfF@%RgPMJRdr=q=OAT=>Kc z$^gx+=+!sjUl_{uYPkYQT-iC5bH8{9dT@Pck*u3(noLu<_gT7mSQ!~&L1mLK&b%h z!Sb}?w7P(?Q}GHpqECUd*fDvo6Kj!%)>_h1To9r{8xecvO`YqomzMY1Gt;TA&r`*v zdG(Er^YE-%U`^6emfDT#@QX|(ZVOKL*c1W8=nK>7dY_n-5?g|&#d?k5#wgK&!8z@v zKl&#xGv67b7_?QAH$?}5&P(gJK=6`^i(f|U)C1PM{hChiE|?PX^#1Jtu+FmVh4>+Z z^nGE?&3jxE{^Mfa^x)~@0@6R1gpP z2nI)TGQst%l^zQBDg8{c&AeOcsEXh@Q_? z;QbCbN*gq=&YLlNYOj~IE0a!a*UZsEzBu=mU&`LMB~6z<0cpN z@sYW+*L7$iH-{vb#uO2z!f1ZM%>v(?JgvQ|>*TB3Qqnx>BHNVS{4Ez*2Q~&u7Xtyty2Qi={WI|g@FSxDTp4@IJLPTND}0n9lCTPrux)HYjp_*=M$D;aq45sOE&gF1Xeec zY_~JBng+qbY0i63(T&WXJ`|GK@OzW*b2bg1eJuGb`d z8W3a~9fVI&Z>MihZM`RpsUBigoQ<-w!)yAjK3-Rb96QK((^e{dSD{E7&Vq92VV~$L zr_XcU)wUvc$z89xYLqB;B1O@mM@Zqui_hF8;ZL9>F$!I-w(zqwe*!~AK2n|z{?rOb zX3uK%K3T-jOwluH6{|ZIRBm<}g?BMUh7pe&x4sp7<_jcb^BG+po^j3s??1@x15^%F z_fTIE6IqNLmNUp5@%YTLwCRw@K*rM0@S`+>nE~)*AFxc?SQHj{j6pR zk$|c~I>vC=u|MTA5ob-i96k*J#e7inWrtRx`h=hrI z)Q$?pY#hDLtdl4R7b)D09LtbHSM_Vr25L(Zcb@twuS;GMD$vPvsA-ozKUJRs3Ue*p zWSSxDkh{bCv~?ZLEn8Pf7&L9h9m)iAae+(YgL!IW_5QO@k4Ul0CYXmFDgHzw`Df;x zWYj1#2ee(k^`m&<-Bj!&jBaM^d7jr}#z5>T-CA>DQ1_Bjh6yz%gmShA-EXR0#!dAD zA=v&}%3DXIEEaH7`03#NFjdJAbU~9*cX6czyoP;2CrCB&CRqxk`k_%2rp-HGJ;xWA ze_YEm!|`01v`&4lgS`294Y{hvXnuXLFTQ z&<4Fe1|(Dl4HZEN5O!f{By&Ic4)FuZ-@VW?k!UTW>#ag|3cp3aLqrNoaf#axFBL~} zz0lJ(FNh_R9K&rf1$8$RS92okjh@e5&QCZhWSEe4JW%X;)>R^3FiN;2!3i>MWiTg* z0}sWF$_M=8>%-jrJIDkf4Cg3!Gf6qtlG%Fy=$ox8B+RSoi(SCSo2(!tJ=;NF>x^lr zy%Wb8Ml9T{5l@9T2|~$M7)RCpI8I)dYy^8dJuNj}%Lwar9X(f7k}5wIpIfoC?o&}4 zw~|eDO?V}{`IC^dhEDJ6GANMo#X>1z0+f(IB!AB(l{CfA(JR%iNneRfiO_rblVwsM zPZ#Opgs5V~A`%l$%p7zPSsS#Io*`I|QZkS}@RY|L^wf{C84Y!M04TI zNUrxAKZNWxYHG>0kmItM~B!?mQXIny~&2nLFve@bp5b4{o~j-O2a)*m-Q! z7?dc9{oFkC?CJ2Aqg3BiLO2AeMuZTiJ_Mi5^sW6eP39fik=iT<>;k4R9L&_UTBYR6C+;?`HlA>Dy-ydV4vcEJ{_4Jh z*KDoK1@BDVjX0$%61-8@17RTH3o`Cw6K6I?!glqRVzNW`(ClVD~iE?M$0&7p^s5yP4Y7VH@RCBW%v0ezq)5P zC~pj_aCT07O=$`TbQ<|_^k5B;?KA}h2=&y1=dv?Xp0jh z{_5c@Fr%*sXAIi?Lm0haEady{j2!D#tAalM_j)&8v_0wu{v=3p8jVT=e*@%u_rXA0 zn#d{2B#_hO(Fs;^O_swFcB>;`*4TyV?5Yf5c1S79Zu*xKAjq0DlkXfNUYB>TZgkum zfn74ZWo=GgsRzm7>Vq+b3O@D)eNfY;!8Gtq>u#8X4@ltk3z+$@`{Gy|Odq{^O(Onf zEoD0nM?PGp)spuAGTFJ+;6!VKm0dr%>-XcfkDagd#2U1nOv^@}NjWHJSY{c~;L7d2 zu4DcQTSTbDtyql``R512=x!p<2$cx>7=Rbq)Q9g%uQ%2)#S8e#kS#|G%Oi)|ua$|? z^%A|V>hu5BQoUcKL%tv>V{A|$ZhvKMLaOOR!MNY(j&^TLSGHER$|Z-u)be^>t4HU z5qO)8{|U};jHIy5Wz9WYg&PAyHzTrr?l{EiO7{#3_FK}hXPy*4llg{IBSUY@_Ex(i zT$On{g92U$USpf~)Wry7trN3hcF+lkf>-OaX~{XQ(af}Q@%5+ z)vXzs1niNpcSA|~42K~^Sun*WBA$*F)4?_o<0O|Y1#6%0CJYq~o)Z!tPqGvH zLybhqh3zO^N%dgS#1kRd{=N!Krt~-K4~@|aT_UBP)y(1GPtQ!7I`voEQ}3nwJ;{FO zV%)o>e1g}5X`^}@y@l1rKlFR)XGPrl=^?|;K02m7>-dktj^1IXrYHzS{Vr+>Fp3XG zGN^zuAbV+_Ya=6i8TIz*Dqy&d1OA==L=PQ`{*~#1QP>#;TK>|c(}}8W1m^Y4;dMQL zT>p#cM&Dt!+uZBiLDsCp>sP5W^=f17uH6M}3s2r$vMeZitj!Z@7Err;*-e-7u0Kq`j2*sj+;w^{Qhp z2S=$g!Uub|_hY;`=aQZ)eRr#X^?>pp+~_*XWcAqyWc-Bxm)LQwfAfim<-)xe>9FP? zJurzi|916!D*U)v+Gtx>(+m#WS2W()ta;TvmjU2Nk-CzS!l|pm@Ak91~oB^HsB~%||_5}UBq^e^pJh=B*q5MiKu-_uGo^Z1i;4Z~g z9QN&vvfUrnt#dR+*zL*jaW8N#Ko1~ck%fPd%k0PI1R9Jo>}i2kLmf zuyCWrPA&rcgd_10fg%FAm6>vR%w@+i?3ZBixmC!c7!V{lQOt{KfQ z0_eijZS@~TrXOi><6+frd%&k@qaWITSX#{-lL^wSy-t$L4>VJpT8qBM$>RTVK7_wj zQVjS)RqMdU9iWLQruz$AorMK8_*?t3CtVk+$CP{$%{9miFyXojbm%~)nY z-7Y0G;dIVT{b3)d-Zb&4t^tP~NlwMP#w{D9lc#eHUhG5tgGHj5FMg{P@VcrwxDT^D zUvZy_3Mp9`Ln0c!C;S3g9BNA_0>(aVt(J&I#Mcv2XN__N|CxEQjh$Gp!k-p+0Q_bM zlMnP4#<^EZQZIq2a|6R6?82TbJ2R#C=s#eym>1RqQtM$R;NZ%gT-@Z4q>&xTJ~2 zbNuNpi*s<|siI(NANn2@Yv`JMP0XCPfa2B4KCi0{nJcY2B6|h0C*hOp8oTmiQcqd> ze#30c`ywaGXQ@E9CQT|9I+=VkF1@M#0@=gEXK78ME227{RXn2=$iw^S-Be&cKh=%B zBc5D^NUCYZ^_Kg-uwmdWTxyzHOETI{PRjS@GirCs|NGPtN(V^koP4|v!WQPcAYP0~ zt(^Gszs&j#o-At{-EPCRiwY?BYuaGa6(+1dJ17%n#wD42kb%84jfMpJUA*<0;)YHY z&uLgpyeubU9Zju-NujcYlkWHp%To1v_HuN5VUt$df&K1-f&y+c{Mt+7?Wkv_Js1X_ zz?PPP0oXI}D!R;uW_rlKhT97I+Xw-xiTy#9a~>V|n;q?SG;2Fa_4wZB_16wiC~Fdp zwPq2MjxRJU>#K*xs%$}dep8(!lQt_q<9_6$^P|_1%}?fd#xnst?Dir3 z7jpCQYxoAiQYa`77C>+d%hHpq#I95(Ev38&p9BvyNcf%nbbEQROLuWd#w(_1VYy>_QGKz*|bm||u7@oBxEt9YDBU-&dtM?vYVpE()Alt-9evrTcU z-&Kx>>WJdon-kJX=xf$%0U8a@B%JXCA;wj)BB{FpjERK(7s6h7UauEug=^YsB(0_S zx@-|u|CUfG>9_gRT{Ot_p{n0~?$3HNqQnm>{wXZ!XNyU0?5b)?szf2}oI4mO1U*&v zrB|{}T(1WEphdwbGUI!RgWAewXOF9i?nT+OL`p=+HPz2s?GDCfx9l^z1)M8){8}zn z<;^-Qmu^|!=?77Wa~9-@V0i%O*nk*@@!W|-6`8E=Td}vF(5CZDMAu)=Txdj=X5_rQ zh@ock!M6jS37Sh{GGD>*cd{%x3%oYfRfU_1+^G1`*ezje@am(-gYut3?y3l@1={cd zUv3Fw2NDX^%fy{OJ)VCo&d(Dpc{@iJlni^fotB}0vh-!vB()4$g@cW)C&D`P&~rx| zj3d|vc&;r?4wXk6Re7>6zv8J$h;NnD+1q<888{5{Cr>=-TEU@BAe2!zU|KvEz5(is z!(Gm3{>~<~ zk0Ao@tmG@alJ|ZrN%7!s8O+sfir^ZF-fVKen7agga zv3inzmg)goVzhopuIe7gS8wc!yPI&Q8^d5qF}CY{G}rhGIuhouYmR~*Q{S*aaGr;? zZZ@OT=VjwrzJ~I3&llAw@g)y6%Jt+7k#}^NvKb8WBfCI^mI_bma_*K*bVmp$)c{CK z{KpXPy!UMOPlmIzGq+Y7(N}xY6?C;@1FiL`qGk7(jQ9CAn}e@D>88JHcyNq^pXP>< zIW0wEy~9HZQQvj_GqV8~dYQVnOz?Zhfb78m#Pwt9zU{1)lw7DW>Z{E1N6NGd$8c?T z76Jpu7o*mno}QjkZVg?wq#c~nygYpS&W%n4Iqtx)an^MzrJ!00<&kA+PJLGiM19NT z5RW`?4*!G!Ht1;Qyx>u!0jr2+wSsF$vfCUwmj^g1Ug_|jH)#mEs`n(Bl@TOxp0}28 zFp8OC`<#X~YpQma`nDBf;w#m|v?fcTn)7VW_4AzS$Z!y+7Z~x};B)ggSTg+VZ|dfG zU(v)n4|8Bd+aUD-O%*R|NFw zI-by=#f`KOBxC39^`}-BnxwR2^u$fwF(Vesv+`~C`t%Zzl-STXuLPs@l^eU)4gZ_L zsI=@zd+5K{8OjX&4*w-DcQny{*~Sf{L`sf&rhKwlBEB9)!_0aUmB?cMm3y83p_p};#ZV{x8e?;3RpXk>SzF;X zPw1$wDefPwLh4_92)M0W#n5}vl{eWiobg+1atRl3y^Vns5eC?K?Gsle*Fe^Hk}HFt z=y&%5?O3?bB>?_L!{KB#+CRq+X*5VR1!+4V#rgq5{1lAT#m2{6x8YB$F5&GjW?o8P{S3Q|#TaSq})1W&)Y3#LlqTbc70ZwN9maHW$7Ff(Z z;_c(g!iEm=&yH1(R%2`Qt{uFp%PYuqd8uVPe0#mGFuWk39UO?VL?WFu3_2n5CokMyD16rky5l%_uDPy}{an5n)xj z^svWo`h0UAbjKuHOyTkvcAxZh?}Y>&={mFx?2&F)4lcZk#54P z&j})8+pvbLKkqkJl;&KJXja#|L}dNM!URkBc7J%JQeBk_#G)d3xv@@XQpz18 zH-SC4f3>KGk|e5WWp0`nDN%S|?ok-~C^34=Ny>!$W7FNoIUR8t(5$|QrIG#8ToZQs z3aQIw^@BCu;5VQfo241tbN14()K}#w!gikoET~T!rh1Fs@GRI|3(3|=CdS7v z=-XLfGVvxXQleeH>-vYu_rr;y$R(4E*R{!?)ODMd^<~RL1T@(z?%8C^9Ku+b!m!?; za?;Us9$J#X2-)X@xoOfI=TRYXt+?RbAGk?%nN3dvWR@{=xz{G&2MaCscFX$=*?dLv zGc8g?zBG-70?x!xXv_8*#c0xQavG8AHtm8wdM&a(!cgC?WhqsFS`k+H@?5)n3cJV% zbFJu*tm>vWl*@^T4GvAN6e{XTK$j<&mP?uTfGuyyvB@4{TH)=%=W8`B(5jw2*5@Cx z+y}(J`fxFgB0zlcfg*9=chs)#$-%%36!Zh30PBm-e-$-FGaRpDv=7n7SGH|rt0}1) zcR*}z`-9k?oUR99?3v7q{SK=#YV8+i>l+Rc-)rXEml*d!4<`>+4y&_TpO+ujm6rY2AlS%fxeG4!_*-B zfrh3geupb{Eo>lt7LwY7E`1#CxxIx=UmxR3rdBy<*`h&7W>CS{-fqe8xgyLGtPe`p ziD-UERcEW0uk6+Gz}L$_Zg&AALJnKy!*Y-O*Wwg(A~OOODeOV&)fTj%t%u}kZ+9u`&&@XaL*HWfUcJ@&zz3H~fJUTLeb4KI6WLYrN+c6eEF=11VJW-?go_1EOK8^D! zIeju0F>G;(36ZBE-A6Znc1h=~Pwc$B#}z~`=j;!$24ijIj8!lO?GY!x8E4ggKWM5g zf~CnQDLSW8{K^T3$SY}JpRoUf1f>LLL_I@&DkA1TjDki;u%;#fjMX5)3Li5oH3EW6 z%5fO2Zi(Es!Z+=47zM*_GNKG?4)2yUE;v}#f{^FurtL(opflx&U$^;FzYdVqf}-K% zCvrz>Ga+-4uS2gPa;Y<{QpAF{?-tpfUXMs^r{efNLw|a4FrMML5Us_2Zz()itlTk zVLes2TufD<{2rnHZqyI&susB2sZj2TNKI@*(=g_fVuYXKttgeMze|~f>5+6RB=T%? zdeC?IU1nk4IbRJX8WD1UTkbVM%e@K`Z81rKW9ohqmDMlILjK?H_ehBr8|SJHT{i%jH`+t zf3Lvpy(BDmbL?9ApLwq(7Ut3o&ZP+Z9B;AKyh}S^*hl@E^$5o|;3^%moEe)}ZgQYd zEkf_w-oeY1%bs)=(yE2$G#IOh7{M2UN)rOS#QqvQUux$BaWG3UXF6eIMq#c z9Tv`Pj63>^h*;<-`;}Bzsu5a_KNO1P*B2?*L5FjdiNE H9Ap0n(7!_A literal 23828 zcmbTdbx>SS@GlA>Sa3ps4eqws;u755-C=Qpy9P;cx5X0NAuR3=fyD{#?(V@J-`{)n z>b|P?$E|y+=1g@@caNUWoa*U`P*IY``0)7y92^{mtc-*j92^1|4(=Tl3c?$wZdJVC zjRB{kpdtDC`g(hNdvS5Gva&KdI{Ntd$jQO6v%7ObO!NABfBIJZdU5;uy7u}iCJe1T zdewS+6eeC@A75XO7SG;9c4iM=XYOCt--Pa$2gNgA_lsZeCj+M+@29_WaPW7YJNqY?^t8q@$1przq{A3{nu(Tb$3h9*UM%< zeU+DW$m{FE4%BikPWi0MW&hzdOpY^OnT?azz@z2#X?gk{=3D6~a!}mHA@KEnz(FNv zy;h*)f|dI9VkRUmL%rns=jwCQ=)W6em%AN*WCp%iDpid`Vu|P z`x%IeQNZRe<;#&RNJi%C`JQRb-c8Hzj`^F7%=&W4ltC-$o9pFa7yf@oy}4a2pTr1xQeAg%v4=N za~4zK`F_?}=4ygq-M<>Y^$^JdgMl;e>do4r37^Kv;r#N%qu3w*C(>>={+1Vy_oFMj zMe?O_ttZXdONW(n{k9o_nl6q`PWj?LyLGa5nmW^Q!|XJ|$~rT0GlE%k>9Uf8fi(GM@dO%V3K4buCk#wzk>eW=r@EBwJc1X_W~F zH%&3EF9U^Y!3Ejm!t~{?;?aX_^7iB*nG+(pDj!)$8JQf^KG-02;Vnp{;(RWHXG5&h z88Cr`iz%A6mQ;enL2yMM^pW7w1#vBBLD{SrDbY2(QyAXiLbQy(q5Fcc-^TAvK3F>r z@_Ntc4H%R#!HG%9--Q&f*882jTy~|`d#;$G-WY3*v1-nPbbDIYN`BimJ#OHy-7yvY z^HJI{D3csj7sgD#)#T!~vP}d7?dl(p7LbC)rq6w&FY;<6?hZR8Y^c}pCd*cr^hmBJElaW%;&yAJ1Yl(&m_k4%UG3oH~I(nwoeZ~4YtEydMyY8UH$-WmB?c?woDA z{ghxdlLp5DC-9S+{Zot^=eiSQHLxXn$3Zp6Nwo^lXtE5ZGBB}k7Wg*R4fGzCB%i{9 ziOz1J+nU+or2L*7uZ@&SINdpMD0cFo;8y61DYNerq*<@7!ir#RSFN9JWUDwc?GJP_ zc9QzYUp%Yaycb0*2$AWQ@Miz?2fu^|J9Ep=TtC-vFtrM)k``m*(6wju!B-+Ppz{r_ z5IE#Ro>V{w@NBy?MM=mvO>mf5VEXq;OofTA&!kV%j4Q0d;{YuUJT>H*TGPkNh1w(1 zqX3* z`)>&{Pe@Hc_1IImgo&Y2wVe?UAK??_AW46SDq12)rL!AX1 zgeXyF={4ZCO~aJZoRZmlICREv+%}GfRY{10R})z%00lW4>tQu+BR%0@U^3uWZYEZ6 z-g!G&5*i1D?@2^F4d+If03+eoZ}Vg(q9e`w6U-qNi8rMaFoflMbeimz(o2hMT!@WyT#lU@C-eh{U=le0##nVaG^cl>~IK-@Afxrx7@LW^w{ zDn^TF6Z}3QNbC~X&AQ9Zn!&<(B3GV(!=1@+>u5ujYS|#+~vNsGL}-Lc_zo>dL3|v_ug; z@;ucYooKD^0Te*bF1lyHt55h%oc=REyJG!uJ;^iyaMAdZsBluKDe%@RfFS3$gp>kL zi2T1vhEBk9{Oqee)Hx=W@$MVZf~2gKpqp4n0QCcHA|4JM`1%b5++ej0o{J+twZc^Z zUBB_svWioOVcB-R660uP4Vpd~8fc@z<~*^g01**8d;gQag^V!gi+^Dm+7Q-S@CYcV zOK&8rSP4uQwwUYVHQNYvn`vOZ^W?aFj)`tRuA(gh0Wv?-rWe4=QqV^_bj4V+4>gN7 zLO0=gj6d0C8iDml%AgPVSI>2|k~_5r_o}ijOwSeU;!H`Ca)zPW?YJo+Qc~C=PeG8V z?J+193rlJH88B*ON}*oDX0JQgb=mMBlnn~ip`q*rszpqaQ-%Z6X&P4Pnh&&YDN|sV zt3GSP9%XFbJOd`zoBL!F2rtjiQ{3gEDPhLAz2Og9FFu z>jA!;(wsBTBJIrzAJI3JXhL1xCMv2#1WoSUpPTct1@pF3mWj%lJ9v9Wj%<58P@YetJM63R6CKA-x7beTp^9(mcWqev zUGI)2_v=ax5m~qp|1G$lUHT&bO4o5GMAwpX1XSC1S_IY9_^yj@IC>uPbN% zTQclL-blr+vrNGUlu~>QUaIeKw^5qaJ|l~N7-V>9s7YQR)q+umwc_LmK#unY$8_;| zaG!bpNdbNiU^gqQt8}~=0ye=$tD-~_asB~fa;MoGU?m4O&0slwn}138Gg9#8xY79| zOf)X?Q?VmN{PLdKKYbbavFRlF4_}1UX2VUTjsw`80b5f7Yp@s7H(N0oo?XVR&PZwv zA54Z*LDly)1FEy3@qaB2;^YN@1M_?qKD&s+a*u3Oay5_rn4{^x)aV5uslu5#xAevX zz7TB*f)VZLYF;F6ggH;~5s&w=96``}hUoH6!aS}|)5Nj4j051>Lg%u- zuX05+}MOOugdD~2_&M)+QOLXuh-lkdhbboeSiE}t%S?Y}%vpX%@#ZSDaItATj z4@~KJr)z}ajuMN#554oITtXT3oXLpd6<#dy2cxUz28Y{yxJ%aXX=uaujhzJXvsuCw z^k*01gsSBOk#=YWK2_5(o=BV_^u}lIs$WUsNmH(LR5gV6^tBB#nP?SnBdH$+`dg#? z`R04qE`T1ool1}+L zdvA;K74c;T+HY9CWSyc3i>MKSh(Fb4cviBRbiYkU?uhJ8T*;JLt%S518iHl!^vP#Y zgwR2bMAGtSRsWgh0+Pekl2#!5QH@V!hdrC+=N(;vXB4K+aR z)%CmgEYNkV)(R~9MQprn&LQ`$?pb3)LgzR%FjH>>BzHhcAtHg9Fuy255#4M#uU3K zK}4+JvN(B06LD^)3M9t=c45C)p@}P;@)yBcgw-(^hLMznOI5 zDmcdyzAPvtws$y*JBMqIrR^!Z=Q?)$QB0ZqmYwN@Jvs}yH5^46mUtN+8JdP2d&ea- z(WpovdGXiQ-;gTF1Na89I`Eaw!eRsLPEsZW;t6VM{oRlN-|T`ifETO4wjw07+pxm5)`63{29)%`4Y5*U< zpi@1q-*)n2s;uVPo>@BC1GLYdFg(P{MN{;*v<%xk?PKy@7+~K{79ENo5#mmKPhicm z`O9*eRUfrvg2}w`8OwrMLRk?Fzr#6Q}jmtnYq{16~1vlr9LH`@+h_x_-+0k=m`)TC%@H|$!4n^B$` zS2J{F$~5Ijr5pBj>$PxO)WI7?#Y10wp=dEUZO7KwG68>qfJ1^|`V@9pPJ4gB@Q?ng z@jb-6j~9)6dMF@X*fs_r6?iLtO_a=+r6WWP5B9?mP|(;rBsR7{J;irBn>#BKlLurp z`z?Je$gK!SAy8C%#{iDYL_}i2H{*X9tIwsk?+>op7_Rjd8ZF!)`r7R}?pu%qUoaCE z^~FzDFhyj6*;I%b%#GFR@r6F(>!+2Ry7$8}1Xud|E))@%;0eUx{dS9uNtngO89kM- zL30YvCwgRv&>(N(HepgGyr!aoKzulR(aN!7axST;oY;yW1Enf8&Pp1hwtUb4UiCc# zw$D3ocY5V?PshkTu{mgfSwtKXZ|?Tply%?4HAH#dhQW1MFmgn**5^5Es5n`ac1?y= z+|RAkIV}=<4Yv)wB~8{(hM`yS#|N?>D9NUS@WW+V?ViWUU(f^gzz1A{4tDfIReySu z)V}LupL{EhhZl{to2CDwA)+g7@30Q;@zWb7->ze(MlC<0Eo8nv=gO)!Ku%y$u|$(9 zM5jW~8(!xpl-f4RJKwi}Hi-a){ZTfx1H3qhcj%`(S(hj?cBjM0u#*Ft3d-q*0)Rv` z7m;0$pkKhhbQe3Z#cDr_oX$(BXDx*1=fike{3hsTZ9lZARPHS3l-JbDH#UD*h*SI$ zGsYeaku0M7GS=1|7pw5IIQ9r`Aue~@f5GFj1E`U$dLC}iK#M1xCgpx>kGzBWPs>B9Rm6te7>Lv-YQn zn$en;4DUC?H;Z$!Pz@&5K+Hkzc>7~a$dl~(0*Oq-cG+z&R)0mB!%Lo#9u~L3i`WS= z_3;)+ZBx)$*d>Y=?v^tr9}NB50CRwomzQdY)WQMDWVMA7*Ta}ORMCkyYBaDz{7)a+to&DWeMDt*Rgf^ zjbw4rM^uAVel^TuN55;#1&F7M{2EDZJ03WCsX2@lJO!u9(mTSYZ~;XVcT|d+j}#i# z)(rWQ1kIK2P?sJ{{dWLgw|!GgGh|W0!OaQouLkbYIkzCR1hA1u?YmS;wBEK623LBJ z>IpI~BPTt0%JEIL;$3dCKlm)#0sDO(+tYj2zM#)G2H$ja^sL1kHqbFQkz%_@Gj!(v zkdKLEbrBP|B^!WlU?`+f6UHYAvK@~x#AEQXCNj~!wzu=Jv?{*8X-%gb5B zf`)yM<>=8)%-GM9L{w|8pl3`QE&p){D2R$1I#8;CENQM#EQ5nSH{1Jr39-Ne{?m{G zSnU(8N)R|OV%Ng*yps0sKu|1Zt7Hb5mP``uon@bui^u9{K|>^4^xEf=jS`WTzFCqG zIF?6qpRdBp^B)S36;{|$)76_D5j-T$x_K{k7+XBTBgBAx%@=QdSNhe3*QoE^PR&d&M`S@ffm3 zgUu|W@HxKwd?{IWAh7C-zftA{>TLrysf_Dwo=yEvd9ShJ?q#o(Bq|R%HnxL}clqP? z9%?ZAJ0~#$GB+0|kRYa}6{gDmMyRp~gi!rzR>%nbPaOGiMPU2=n5-tN(yG@=2%T#Q zk#vWzlQT^dq8V`pNw;`-2Fnfyt2yr&vNR>8JSxE;2UEnIn)3+nH+IrR$q~mrOE`{Y zKofZ+XZ6q6##u8g9+M}X-iwC*uE>ER7+k4B3JA;y!W6RiC3mCb5ZUAB&T2m;7Qydi zQ)hSdn*Z+iZitNy`_Ik4$_RZS8PyHNh)a%EgxJ&S_46G6Nrbg7_Kl(BwE&q|0ekVi zf-dw`2>ILwYo>DfqnzhDzzhv#a1#p4;f=#g-Xtou`o#oc8214&G0PI(#PX8EBuO<# z4$OA?y&`!Y#XKXTtcnH2=acw+88HHU^wN%3m6B~6j}umwQqt=c8i8ijMIA*LC|qy+ zk=G#5ualC}t-KIY*~{EdG^3uJ8_nL^wbC1T{#|2wU0;8OAEbUNP>b&2tt2c=*NS!_ zAAW4!8J^Kg9$LY01%kywsSa&guRf#ZOkceR9`2n|CRYZFCb6M-revE*4ns>%cNtC|;;_yD;nW65_M@D{ zDZ>Fu2_e@s_3i*xb=qhagC?g8G0BH3ME-Jm4s^`44Rs=hqIJn^DmU-9-vzLm0z(o) z*kKjSRvn2jW>g%)1kw3Udo2yiGKI+c!orL`$4sZuJs-}19gc)CoDa&4{fQeVOWZp(M<105@VlGe^-Ln%Do;=AR z^-IC(6V;pLg`qV15GA3cu>zk?;J!kgP%TDWbF64<-Z_R&o?G*i+J+diU)Lso=Mvs> za*2Y!-n(M5)H4husCV&-DJgpVsdO4%Z*pm*F2A?GMhb|WHo5%^+8wNKRWca_js-8u zzBzAKu;gtLJ%pn_<{tm$CEa8m@)V4w)w?2`GVt6U0(t2N7XW%h0EOB|8D~|TNn;w0 zXFpQp>nW~>Z4|G&)XI5=bj4MzmtE^Jmie@sCg624mufR*LSCAF)Y1*CGiG9A{A-@@ zTvDBG!KEXv)18y4;X`tGXm*&FLB9NA2O=H-xtqDYFBz*a}U+N`;5=>I#XlCJb$*u1h=2*TS;$@dd`Z5s}5Xi z-)<9`*QX&X=AknV{81-4RkuifMmBqW$a7{N4*2ak$c-J!2KPQrI{Za^v3b}it>qoj z`1SALuJ33C)fHFOQ$KT||Df6pyxM4UmIs91HR{ zxww>w21}B{h|MM~G1}7TS2E9Ka8w8~Si0h&eCO{z}sm^eEmQE8OOc5A5=G#^QFWg`c z{qa2>F@@^l74za{wnbOU)#2xbr!8BNDklbiy?rj$rjg#4hf<%(<4zO7g?i0pLpqp9 zEeS4`>~Wxs161wsUu!lUyKaQ28!(@tKAzfvD$okz{ycx8ze`-224>zlC>UZ6NfwdW z?qH}_=uH;6ooG}r$fZV%#2WAYQQ`?<`|BJjMJJja;sJWbZW-b-?S5XMG7x0#L&8R3 z#a*U}BGmYcm;F2tkfkCG>-**Rc{Ak${zWhlCTkW(KtrUOm(+5Zh_Yrngqv3?H;t-n z9nmN*&k?G*DX-85#`1@ywkFT^9V|L~O4)j4I!0TBrQ5vg^EY2j5V3%Ryo;Dj@;{il zIETEbJg}Vo_&XZ12ONeTkKPg#F|STeMU3MHS|sbU8B*V>j%6muCk>B~oD)MPhB1>CjC@M~s-%!F zalsPQu9y;ey(9DeJ!`5W55gI@r~WMvnU&|#PWMN(Ks!k>+403k7}8ieAG(~DCbKnG zuJm7E$ep3mLRl)GA?7{h=a`0yO26U^XFGNN0j1v<>vJI7620$9en^uw6$sx+3aHxt zDv61ErW=jce$>h8ZWHe&5>l_}qEra-?SlT@Dc`6YZ3VhNe4Rt)e=cbSlkAY-qPU~X zf+zZ=KBgf>5a6b{{6JH)Ln)CWTb@}`mrb5?2fS!yfCC!t%<)f@_Nmtt*L2UI zIWgo|hB!L8l8t=HWu6JK|4?<^6bL^r6)0S49=6_5(Zg!!z)kVH9>W3!-`IxYNaFjR z*@lEOwL9oOq+(ah?0&53BGDpNv4aRD`Eykce>j&tA#}f2ek5$q7If_FS?M7@UrLQR zQt+=mBz+p@&|it-(x>#{T!qSxKjQ6p^srh3ke{l+t2PPL4;i{*vr07TKJt0uTg<*F z*_+6*@&LU7g!}ENZc!L-*Lf z$0cZzx%XomYLDReMs(41WM=XB&l8~crkHi;qP8M1>`=ZpsuZ4C9DDbTDnNQ;rz89) zlraukHVtEon`Az+8(ukHXb06-Pjq1E}N z9_--SFrvG5CMnK5T~(Q>ytMzb?Fneh@U3)XQ?V5Mx#&IgM=Y3fbmf{QL}H1z0x8Ex z8)$bWI&T})0Yh+X{+=WBH@mNe7e#lll9Yl3d%4{66fD+mL_*yTh`S;Gw76 zj<8#FCBQq!xZ}P94Vs*MgBnUN+ov}|y$kNDXf`}w%+o1KsY5P5@OL@DMjfgdcOokP z{~`|wOr%UPo|7Ht5D@x0DubbItIKpkju3ByA8pX`83|ETx|peXQVu}?0h^N09ehW8 z9i6b*VZW2wy3+Cr2j7Oa8f$oml`7id;4wiRn0J1!b&MVHD1UY*E>4(&=Gkh(%>Dby zZReAgn-pOft9l!~tSIveb3Xh`uVK(`aM=z0Sm^R4S`4_r{$jGqRe zT4`_+NY^^E8=39cYQ>r`OG)m^PeDG^EHd`?^wD36YMJ^9|m+?(xO5P)*t-2CIV_^eprdVy$}M z;c6YX;Wr|wBq=?lhm$gJ#F1s~!>;EM=&a_L+~peqjNfnN;Dq5@DGf&70}lUP8)HU*D-MnU8aE@b*KE?w1ooo~V@Hdqwuyt^jfN2K+Su=q#; z+-X$<&SiakIXr@AdoK9_Cf8Q0FaE^x_h8WNl1;RjOrXfe^q(mssY*1FpBF+zT5XB} zdy+ka6$Ls$a`feImfkhF+UP2<$%sUM-f!&3W4+J8{g4m^e%6Ah9>~k8KHKUmd1<}) zFNRZ5y?c><<-_p6W@9HSEQTgN^wl4RKqJ!Wh@LvmBH6d>n8b)er6lIeO>sD$XgmTpHTJTqGfZ{N5vjRg z87kmMp7QcAUAK|wHxlIyLGa%2j&K!fsgzQ5Ee4w(+)stL7BsVO$vPjKyTSftjEowL z`amxJs{e;PUco$~>4WE~6ew4}z$j`6df&-MzNJ#gU|0i!*^u8Tt~YiMvGoFh$0;FV zFgDUJ>k;hR7jV_qkcSEXBVLa$yyH14{s-igZg&l&|NnQ6b+xmpn)@-jiPEV~b^}16 z^!SFfmCQuyOcSR*i0|fv(|ly2DGvTE+Ub9J?#ixPF%=Ff!vF0A3kzstFG^rbW@`@S z^L#mI;p38)Mesc*&z(qa(!{~)7r*%x6Ydr6qZ)owM9{A#_9g5CBE?@BCTntnqwY;r z9N*IRG~&M2e;$^#YnH?g+z};DchjSmmM>4JBS)`SR@~8|-3zt8gG{u55O$e$ZRNdd z3?m>Ah=9%BXkLtcJW3c1Z$_ts#H+xC6vBp`3Qr7z{zez!gw3b06C#A>zr}p8A))?O z@CNB;s7@T+O)1Jxf7d;?+v@baq3_ukVUjm4IDzq+S+F4?=%wAn*c6%cH@P>Ih1RF|zd6C8?-2OSfh9a#;YtF4E=u4gCJqV+6`@);4o3%3AMNs+qRO)D!sK zv~m6R{B`}0ovv@f!A#}DowOjJt(Af>`sG9p;l-|0q@ZP=R8t6}R)``iM+b>EySApn zje@ISby$;qcc#qrh%<}O4LWgvdOoB^aZgDYO}ZW)dpt-lR=Il;ZG^OeNxC8sm$I~w=^iY)QxRY&ip%B zAoI#jj3%UP+&(LL0sYT9K5M<;$n{jdX`SH9fNK5xtJl_1|sMl${7k z-EwT0gt?}YV}+_~-|t^b5?q(yEitOOSSGq3c8>ynke_Ks`&yR$D}x)2HlfD@7G{WOl;ZO4YF0z zgnKu`HaP!YAnzhLuG@$U+w6N4(NYfx`sv7w{Lys7@^8$0H73A4=#G}`(n1zk$I^@O z7JU;2o?PxHj*;OglTG6P$l`y9`@3dD2D)nRDG7jp;EE`W)V)vXJSDe>S1|%{3F4<;P)u` zD0NTA@5u&TDgSR|I57I-C8Iu+Li=s2@jku*)nRs9>+%fnJ@-}y*dj^?DL5zeTZ3+d z2DKI#x12kGC7dQV{Rosj6qw8Lup0nK6-<8Kr^e9H(wpyK&F>h)54O~aGosr_RuQ&> zMrSyJ0VqK{_qr;QCuyT)3ewwT(z_gs+(Pe~4`h~j=m zQ3=FKgSAt;Rds73^i6etDOdG1NSKR`bp=y$^6{$r06+b;{MIXI zPW?lC0c$HX#W_}WWU8tTeYU4%(Nd4*iwhO6c+)|%v~o#>sF22IHSPB^>|J5+9I4{2# z-`9{F-9Pa#K?;B{b1pgjbo1z(jfs|#Tf(Smjtd!Q?2KImrd#e9N!@@CFAMCg`-<9% zYJ5*{=<;e0E3Ex|_?zy}a%K1ONu5P$HaR(bwOAaiuu+{awtvQ=CqG8lk|CAy@1Ct~ z4_o_Lew$A6H=D&Dcot^T9xyQ-XMc$qL;r1?u~l21PVBNz+rlFY_BeJ*O(@i7_b6ok zSY8~as|=rJK2sMru8|A6`fhsyPuU8Ra&#icW9O%3;(y0pcCw4)giF83H5z-6Ur41^ z0L5go%szb(UfeZ#(w%t9J^lgF;H+=iK{rf_AFYxNdp~G4LLLj=*ik`Kj|lgaDR>d+M=TIKP@UD94*ao!`OW0So$8_Nff((pZyg3x|eD*$EBL6qCTNx zKf@hpp#kZuE73#CQ&!-UGIoi+wsf{U^jP94m};kFSXf%f2F$3mb6InuDU1(V0Xl14 zk(tnYM6g9@o?in`YrrPI^e=v&=fLNCh5s;>edjz{y__P{t`RyYuX}cTQLmP&Bys*o z%4&u)Fw6N$zqdW-Jpvf3+Fjc-pl=2H`RB9h`Qa$(ySY*G@@NwZGuYMBji#}lro8zK za;5-JwvbfYMONnVXSf{qc9wNnH6+nt6B33Ha`5liWYKt#j`T3O44{Gvj9q{}n*c&l zLpqXK$TSo{v0;ow;C)%L4GeTtdWUAq+i<{;GGZcjR-jPX^+6si8eJI@*{;x?6(WL4 z%bdc#i%qBUhHRyyBA-_I4dHVD**emf6mhl-Dda2&7Lb2 zHI}B7%uw#|A5_kma5S;_pEwV2RUdHrd0mW4L1J#M$f{3>9(^l6oLS9Wq!^TmNf*TZ zR!~GzaPrcaqmzF3U#Zt{kUrGAVk(9#vP zR9Xaw;yIw<(1}*rQ)FdroUxGTK!hn|m$vfZ529Jg`BEYn1#A?gsIu~oSXtpRk;OUW zzzH@Z255AT^~0Bk>nWO2+5^v}6ydCpFzb;5g@?8JXhwlm1u1e#?6(aiaS41vSaa2g z;0ft|w-Z5lufA7N%I7|8#i<4!O#ep76id=SNIF8FVtD&Kn0MCO1WXkXMBVSDZ+A*l z%m0ZDQUVftxhFF(QLTy&42>Gq1BxQKe#EZOIf_cDCgyG;jy?=rw>bW<) z+Ov8vi}u~wMf^VpmRxt=?DoyEfFf?KWAo(O&Yh)}TrhEw*u-;G9D?mD`FOISazAee zhU#aPw-A`Bx1m)JDow7-meCHvHmtE$hyT)Au{y(P%>7z0*Y*kt8QW_EogRNGJ4Qj_ zA+&>b1?)INMlO03S}Qn{4@yRM4>$I9kBD{lp<6a>1gGU;|$+ zJ9wwY8vRCQIdEou><*?8q&kK#HCrV1H`Iq9v*hC}^K52Cd8~)7>_jK5i7~*K`!Hi= zp`MstOL z0)1~V!bf+Lll=rHqJ5X)K=wuc`|6E-dDbauZC zZ!_C#gJLMdd(N3!TIsr0;F??gFn6<9dW^nIbq=w2irM$ zHiq(?Su`Ou&>_YyJtT2+2(&6rpUeeX-kz-eO1(3Ziud+f*zAk2S_lU<$8Dv-GbBA$ zaAAWlzbj;87Q*L-UW4)~>ZclgKzY%4sn?zJ`)PM}Zl$MDX<;<~MxztxiJSqi7rLh8 z!;4H30FiR4PqpYsx$h0qW~`Did9%FmfeErpLsM7o8TPeM|_4dUJv0GF&L$~ zu|r_$I|f)-W~}|4xkE29GRX%KsEA(4~DXVjoy)GAi_F(q<1 zCHS@F>-vp{FH1tmjsl6{%rO8jKYhr(4zZXpi?7 z=IM`4eG!G?@C?(1^1b1 z)}V2EKCnaDsKYc1N4*-rX+THdaoKRpRcm=_m~4GJPmh;+J0wkXzJpB}#r=y48F!HenR1u50hl`&-G;MLPrpJ=Jz4iG!g z^79eEu~6v%dfm{w&>&eY6OzrfJCyFh&0M zH(}OZz}8ud0k0_@u2GR{sh93Mwb}c71R@kv*6EVOSFC!njFfZegvyhVrBM8(6pOQP zd&)V8BCrjpWNZbcq`%VYfrUQlrD^I{+#LiApvx|ESdAxg1L11Yg00I;#JwQ#WJl0c zvhTis14<$eMF&n+5m=lKr{){QgT7;_YC^Y;->86_pw(0 zxHhVQ*w>_1_sB=@)Gg{NU8*^OUqV&C&wZDcVmig4bZQudW(PTzp4ewH8l?I2+P%Sz z2+xFd1>*4ySHC2HB>x@HXNr$4O`><88aTHk8 zHk9Jcf}Dn}$$cXinOP>w=d(Z_3Z_O!Z3yrgV4{J;xW@>D^zs z(jlaZrTq-~r$fi#6Db-c z`J+$Z!yEGUd?4pSq)D>wzUK@x_O?#7sJ9Zk!ezw)N4V9urm#TXPtBGwVGx4+3cgzI zYY-{L8xCxds279knA5nz+7aVc=Q~M{^3pbnBlZ(H*YnWFdrf^WWeB+W6Y7K1i!wnn z%mLKtZ3*RpIm0fp)iq1Z% zh-SsnP?2yb8G6f-?~ zp_s3_9}Yi|vWCdX0K5qaiqWNyMGnq`egQ?3g;Ga}`Q@%EzRd)@e`!i+}7JUgmx6#O%BL0gfVI-BzUM97$6y4w(Q!rzmrIP&*4Dgpa&ncc(mhtaND(DSpHVV0waKoKfl|dwSpqJ+@kZwD@jR+5x79J>2W> z+byeMFGJ+&MU?E4VtYZ4Aa(9<>fhDHi(d8;#HALz4~?eLhBrSwmta2r zYXfeLGD>f(%><}$A&t;`Y(wrT1xbVMscP zT-RSxnyY?ZoZ4yOh`i~>__r8J*}V96ag#EkJleK=DN>Kj!nmS>bBeQfS1p>EuF8@R zHuL)2scppP(z3VAT6OpIv$>b=j+n(hZE!E^PH^wea=hzsfCTMf*VkWt5{X-PB}n+C zq~gbWCfIq4LX-Ux(qjn>^yMo?S~msduB~ZyYsKfa9|>D6Qckw15;}OL-D?qfxpKt%%3dxKJS%_5mS9nQjjVzbdJvfs`xvcjP zsFk$J|MJe>&T|}MVw_LL!A!67?F%BnUFXyc)aKFg;E!|9a8aTrChp=E3$3n~zdKC@ zR98gOB*9;(yMVo|Qxk_J6?Q!aAu;~=LI%cItRN@!j^mSnb9g^G;?C}`OGW6556+cb z(aet8n9by`mxdprkFQ`{P+gSMv|Jc%#pAb)PpoppMrky_(Xmbc_!(%+H}ODK9V}Ee zx=!&dwz3ps^SOuDE0g3E3V`zNX+Th=pB7JEJt=OMQh<*goM9{tB7*n_!zf9@JE}h` zSZ5 zuo`DYGDp4107s816zT-l>(w)8g%(MY!PHy2S{G;FxB(^WyWSc?Ux3y1bGy^-);tu& zx!+v!3M!cn_Cixw;k=kR{RO0YU?0%&7Ic5tN~Tl4E{X28u=FPA-XDUYprqNXK)PAJ zBC0EOX&%k~oYI7P9ANVNE|M@F`~z#;k60g{qRML?uuXG}!=lck=vOB|50e?#O9^1O z*b}COK7P|gGwS2bXXUoBBy9kS>0i8s8eJIoo3$bs7TIRp+y2)n%Hse?^{MoG9Fc8` z|7m`~bj2eM#d~b;@No}hFo2;48ZDuVfAcmlsV7Vraogrx!I8^vz0B72m|;=|yc&Nj z$DtkuC3dh6RJn3_+T97>$3H8^G{@9>Z<(`j3wfyr;}rm@)Lss#*K|{zoi{DBsUDzv z-)6!!$?l9MrhBwoyItlKcN9_iqtNYAKgE)_;>kW_lxfOE)sxLt?9i99yZK81unmPv zbW<<1%MwnJZv`c|#F{F0kE-uNVD@`i%o(@^Y+%4*cT(*Koi3T-IlM$dRlB(8Mgm3J zA?7YuY5M1tO> zCObUm21z}C)MN>XFbl6Dvp5~ROljHkOZb9SA{7KPOY->|$k%u=cj{QAm}zH|H{H)> zrRx%sXOEO5W_2&A1K9-i!L^_uK~hJhc6 z3*)M=Uhj38UI=StpkX1IU~4Ght~$F#?YR}84^_RnqG84%ghN2G1${0ICWX_h>8D-f z1lbOViZFAVx-mCsTCikHdl(#IJ(g2w#ML=#NPZ*bli{X0x_Z0JHuNGbvkE|}jOSOYGpl}9UGp+ZE&m<|?tFY5B;+8g zEi)`S^%jIgpJ1*UDztw6u>R$KR~PQBMm8Al*4Dy;7GqH8O#q~cO-!xh4-`k}G&cVl z!#!swD{EW{`|^>G#(4Ac>g{&rl-Y4Wmc(53Wz)^{O`5A^84>*R2=QfRQs9xp=T`7I zg2BKN$+D_b@rcLRm7gmvvcCer9l$2E%KxVX4tD^ko^*K;t=j4IE3{O@ITXiG_&)1t zyXpt00j)1-(rKv2U^*~lbmdRbRg3)xD?iSL`8Tb9z-ihY?cU~Gqry+98k)My18gP&@C?Rx!m zxgQ>Qp9sH5{K4-KPm3HCqqjvwVD{7mt|}=yOh^)`a{$)Ei&ZT9i-zPY601V;;Qo&} zF4Hha323CF5(fN2NJxc4ui%5>BUIWZV#@W_AbNB3tU%xk7(%2PcfumLF4+FBy)+!= zqoa*7xjmn|akUZOPp>Q*gu8qR(`_I;l8UTb_Y!)Pir)Fdm~K1GZ=%=?y2n&b*hc8A z#_}jkwx#!hp$5lm5T#*38hOmP*3C#rp*kRzXHaDI@&+eYG0?Q5<5PUaz z)y*54KN(F}9Px^vv03oPv{b(86o2_$PgmE;%F4xfKR$1B_%+2GB+H#2K9H{=dM_$zCWwWz+6HL%h&cL}knf#~3pntw1|F(3Q69M-PXx)lqsuzqX&<+w8FdDBwclLN6~ z@=OCGPw;%k+s{6Ke(yvvPZ9TYOeH3GemXZ^&v>xO*SWYVVDfI#K^Qw~eNl7siPmJN zl@p71U$-Z8vPwT#JQ);tv^kdL)dc#R3X<2m*Iem6xm00o&TDxxBYTa-rxWqUXR39e z<#u1^a?kNyNer3`uk>7`N};s+D%C4Hn&(E;Hl$mUOqi{#R07Um$QR3(9CP*!3kGB> z@HodBH`5m+uS?PpLycN$1~hvv3gx2iwm-UdnsOeyBc=ylJ-L9p@%n zH>wdK-J7-i`(~!`@{4z+E>z1Rpt$E69T|fzr#Bbo==KSG`_y^disy;ENJ)uYJ$5bN zoK@qV{;X)^i)ZP3f}Vd@(If##)QDW@vkYLt;Kk--IKh?z|7jdP*Z3S9aFrSBAPzKt zrYD!oS4!6{X+31clF0TZs8$B`wk_ zV7ffpFyN(v-vS+NO9=RTVk@vSStv%A@+9{I$Vi1ElSW)@;lbk30K9N*6(7U+E|B_M znh)mWwXa|n6%d7qs)LZ0eyn8N&NZKN$>OHT^z&v5;BzkX6uA)19Pqm#AL0#D%?mDQ zp*}A(cKjFCI_1Z_$H-It%D1e31*>5Cc+#^xfgZvS{8)5#*tlw{%(9fTM!gl9qc>i3 zK<-|2!3Sp9#^J%5MHg3k=(f8Mm+`T z);#r!Ac3apd%S8eZcg^kTXMq!%k?X^;%a~;>Jn%u8kX7>YDHNY zloxJl=>)UIy!7L?Or27+#eV(j6ISgftm{f~35zRU%WJUod=#@9B=^3)nK7=Qv)%RR zE!#5DlcYr4%UnoTXq{?U$kWsMWmDD9&-O+Rn2|_7uoH+t>pSJt;O)*5w_xu?C-q~{sjvPpelU7eis}TX|AeFQg|`&wu^%z2%ypk0l1LM!$m)=e zmP;7)?u)!D%{(p#G--!YZcBuAep@kbG-qFy@_iA zr4jQqHy_H^VcGx6^u9N$ByCqCXVewO6lr{I0X2n{L2#aRSl!jXMf~yy*u9F}9 zgufQxzni#@9mP|6x_7ni9MD(8J_^0*x5-2!yJ>x)o{avejJfvhT-Q8`abXr~6{_-b z#}&LO4niJLW5CHu2)jm29Dj7wcF62Aum`s^!86kY_foW-@rw0Hhipte=((qZpe zWKnt7KItkX$nUOa8Y)Q=8p97}!_+?z<;7SI4!#%h>PRt=aU`q*v+ReuZC5lM;r+d$@qCD!&nh)7}~- zrPI8_okI!sAW_zPn>*?}k7YE31vT~L--#L8#UdpiU}!$`5x=!@?o!@Yrkkn1-T@ip z?c5{TgS9$VGMTYR6*}rj#E(KqmCbu$y=@?^&&Z9)ebv-2w~ee_fa4Ir~^3y-7nP<(9A)+w5|F;s^yJBI9YE!b_8!nr37;1ql|cXGY&}1nyUdn z{(+`IlBvBDw@|3O{thnoP%&?(PfXrf7jSZ?GOPaDNJF{to6_FsMTSAf!hd6AvRU>8 z8*m!OWiltf0_0@wHs{^VQj1t1y|cc~hanSsBPE`xiN%l~1wl7EfSf-d`SgFfYv(HlM7Gk#J(`Nz75HK%U_ImZ7 zwQ9C&n3P|W`4-`Bx!q{1FEEK&CY>MCmaH?P@aAiI3;r6bpOFlpOu?`Er^h0~FwZ@p zL!HgbC<{!Njbu!VDlY4ZvGJDHT}odm&0e29Ww)wcw|Mwz<%5wd3Ho-@8nA3*&}%#$ z9HZc&^#X?^gr8R=eE;ZyOQxqUtE!mYrL$)*mQ!B_v_a;g^BlYIe6}Rr=<)^5E)~gZ zp>1ps`~D`6x1;zzf~)u}i8`&-(`sumH*l^JfwBW~$E&H@q<$xP-c{Lmfb}rp)vT^Z zbYvn=H0$tMr&ivJ18OY~8WaiUW``fA++M6L?8v_9RN%1UI(X+0z5g?@HoXO4z}1bG zcuv9>9N=`m>nOei#t-jNW`=kKxW2ITm0)ko3zvIv?0cOx_*P$|G3V~2i22ED$?SS$ zqZi8VI^EI*gH$>ER8FC*t(WaGP&&sAKu6t{DPiEocb8JT_wr(8-`sJE?2X(gLo&ebxpY}s=-O6W3j(+PQNj#Dz_AL*DO47vJNmN|c?D zdjmh21)DjzYdCyeH&FPw-OY3pPBfN^qRR#7Yqv{VO*DrzpwJj2)Q)Vz1T8!nLr`Hvfx3Qzx14ak#BS{b{D@iXs2KchnA6C7R zUwkL8@djUb>aa}Q@G}O}WG;l=7rs_GMv8@0DNI8)(|Szu?*woXV}_Oh8$ zoSwxiJq1~Bhgv87+#N;7TrjYoS3%i;XdiL234LPSaK z2v3&zFgbE~;Ev~14sax)04n(7*Vjz%gd~3`x;@)B;-8~GTYlA|5by)m>U)J_#YLuu zDfMd2XnO0x(kl74_Hhwt3$Cdo=57f5xOm?LNT^kIv;oeFDNn1cd9n9rW>Bw~9Wo!U zFYEPXmOF4X<%@s#SzQu|da(A}=#F6@0@itSK!Zu<)!(`35M}BXFL`$@zvPi7!b;kq z^=4v5sjyd}V2^liY7a!MSRS)e(54;gpJ^3(nVx}tEX0@KvA3Z#<|oTU^#~)6VHw!Q z2->srNk|icvyv^a=b_6}d2VmN7uyRdd6#I4>p53_ZgDzl44w>U6i-Y9Njs~y?yiD! z-^ys`D}9cst)KKOc`=C+*Cp{^a=x0v#&c$=vaQ1p1v^R$(i{o#2j4Z7=jq2G<>&TJ zd%zkbE$@@j=;3;MXZj8`u^xNAD2U`CL%fO`_Pd!=i(g%?Nhgo_%WDo8(7Y@K8D+7kgtMmlHfcaRSxQuIJ8F>TAyEVhDE5ahCEWt4W zx$1RaWOc}fuW9m47v{?zY#PCh1D2bIobi4yUJVbs>Yi=Y`WQ_mwsU0#CiWoil&UUf z1lv#5WDw4>4iILHSJs#?lN*EI!M}K-EO(J-JtB8|ct5?({MAA1)>!e-PNf+WG!gup?Bb>-VP zw!2DOoAaN9o%pvEl@HJNv^No{Mc=%&lcR`OQ=PoN_Xm3F!^k^koKDXl+L z{Up*|VaK!W^Phh!7S&+}47(iBQW9{T4Jyz8Y0(pnx(~d(yVu9o2`Plvl0A0Iw*Eya zr{#PWJBr>6quYz!O?1xTN*jedlgX!0ce)tc?)X?s9+S}eFpA2Eerfda!hPbvy!QHb zLL_syE}o!OBkGVp!}(BK;(xjXH2njqVypg25MhT5gc|!fvA@lXUZ7v@0W+W@-rKXp zA^!`Wnl+sAGCvC+@pr-*fvkaA)K^l*tI_9|)H6bk23B7D+{7;u6E^8Yk;E<7cIdR>Mx_wj06SrkPz|Kf{q4S*WN@Pn8% z?}$OHTr#OF;hXFG{%f_^lQ-4x#~*C*{pXMW6!5oySybT1vn#$dH;=G4wIsIdMm|mU z;O3li5fCTS+smY&A)gA0Ng_-C=@-u|4tP~B%-`Ex@^PBVTc~_sV>0r z89O;UHoeA+nxdaD^+P(8{P^+v+X8C%_X~@qi5PC{ATggicb@vWgFoT6_o7gpJihPY zW-PP#M>R6H)-Dgyj-^pN4W$A!eDz>q%;e8xZPXIL&RC4l^7Yl)+asUx0HG~a57uJ3 zcl@+%Q7d=R<;gP#Yo8d`G>#g`cQ7-*qSn+1l^kB(WzY_VHDJBqdV$4#d13=?yjO9FEahNBFZ(eq>kzja9(u zjGXV*xbra}OGtm*L@_40G<>dU;27QD89;aP$R`yAu-Ae)=|%Zd@oMuljF*f(<}r2X z0x~qV5M6nXXOXHKQUX-0?1m_R0b$(y|K+WEj=-WqH9w=4w6adRf4!?knrfQ2#0A8q zfg^cT#ubf_iuGXMet?6ZH&m{u(LRytD$ZEEZsi$erI=_1$j$B;V5)4c63w%CqsrB1 zc4yi&w4r0NR-O2?DyGX)`zOIiAH< zuYUWZ^l4b;sR;OIqp6>KP4SD8>#>1jx5nwyYcu?pO#Obq5MR-OD2pkH$N5Kf3w$)pe!?d zZzxtDWrmUt`c@LUD7br)VQVgOCp*b?1P6P+>Lfcq#2 zxpn+_B3Ep_BEZu`Le+`|-5EPvdk!i4D82*P7~>!3Hn|@k06R9m`S;^|dbEmWQp)0g zT8p83nU98vSvY~5Ih4rGZ}FMgw3`VtJF)n@SY5a&vD#8MhM#tG@O{u!=ewl8LLSyE z_NqoP2EHf7_X6D0)Zs%LN)dUwghuOLTBb-XalW*&I#BWV$>t%1nF9UCorO=*9KAJ+ zorN=e)`-C!V8)(l*N zBA(fF{_HhyXudM{qbCV3QxD^8&KP&%5{h_?;Va=7pAJk|Ld{8=F@dZV+t)OR*c=t> zk?g=kT;uT;ScG?^A z^=l@D5=vl$Xs`i;*6_;6<}$qEp1e((Ez8*B);o#e?SqQLiCC?l4|5t9?j4w>$6zQv|=OVBbEo?+)hM*b_83M|+ zzR?u!GhyoY;ENN*cZ(4ZX5=VP&WL1_~uNJ)1Njh?NnQ(A~R;y zRPgy4Yk%S-7i;B>M1kEzww4}>lCrihgtl0JFs@C%+})`1ZePiuU`*LCf}r)Uxg)VT zRV}o_-OA3(03OuUO&J*as-b{4V@+~=P0HawTVqQN`P*s))8JXO7s-$wC8Yg2qAHC%iDyH45pf*(Jlp4L<3~!WBAzcH zbmDg9Xyg5XtG#n|JUre7wQlY4g}GMF{Gw_6k$;|9foSZ4M*;T;OX+9aZuZ-$nbZCI zYfJ&t7oN8kM#AtJgq%U+`+hw?zHQEolNhw>J3*5@t=YEU$E(|e#224#9*dg(nZc|a zb(ULk2Ydd;RVaOKFF2<2x?E=hJV%30tXwmdZAKeQNSpoHv%nM7Vv(dF#NT;=H22LX z*ofQ<%iN+R;;IAVLutKEzX(?ZNFDK{dU*o$$bvQu2_3l2{Ck((he_e_h%5{(qmA_6m1ka~wT< Rng2HbJ-BbES*30p@qZrhv6uh= diff --git a/modules/web-console/frontend/public/images/igfs.png b/modules/web-console/frontend/public/images/igfs.png index 47c659eff5fcc457c106f534a03b7f4e727f6c38..b62c27b60cd93babdaf7fb3912e760f0d111aec7 100644 GIT binary patch literal 14139 zcmaKT1yEc~*X9rc1cJLI1a}(<9^4_gyA3W0?hxDwHn_XP;O;s|aF^ijuFL!GSG)V~ z{F1p4`*f&+oCFFIArb%pK#`IZRRREDK>)xz@(-|YHMOfEd2dR% zf~<+35U8{084#qI6w$jHd`_4UikGY1>T>+Ai_&DGiK>+_q%?TkDZN6*1+a&pSu zoBDcDv;QK@#(sT!!p1IqbG^sGDSx-uHFNvS#;*N(w^^pgb$WBtclG+bn}0RxtIf{# z`nsQ+oVNb@der&Dq-wWf^=7jg^t1Q;^|~{8>0$Qp^}g3)Jy%IcsP=hl@n#kBuwU;W z@Kq^uO+luXjmKv7@?rb__2z0p`qx}OZ@j9iN{bmsV@C4K%&eg7kA|j3tJcHgPn`La zXWtz9A7)2i@AeJkS+RIcVS*yX~(Db z=U~fIW|57X9bVZxGVGQI1s+aRSbf(h%RBEANxGW8RWCm&`XPL4%>$C-71CV#HFeLf zoN=1DJRc%?4y3q9OqBEQI>>9kX(_34RZK1FJC-ly6xH$26Z4HsJ!ZFSoxHo*dTGgW zEbp3Zwri~8D-4@B_0&||oNAlyDf8(&1SRc4eVceJoAjI_FQ8ioYoj&OM<>2Rj@cav z#tNst+?EkFsgQ-Cl2k~T`D3{}NM`8uY){E#El#?5zc=evlcsrMHG5*Mv2m}?Pi%xXZRG{iyf~}1|+fTD$d@cu4 zv@itOo*lm2*w{EaI(&S5JU>6*-r2stzh7Bd+3((01OV8rq(p^OT=k3$8tv~VQ5Jm_ zS&+oL@}cVZ_!assNCsW@l*VD5eCX=`E(jNoQ#kg-|$3?WxY41M`xpQ$kM+%yf zIr9zFWHo@pS(@SQ8uD_W8$@(iG$VMq_MuvIRTm+(=UymBOEc_1M%F^WraVGq=wN<% zLfVM|(}3t5zP%ln7}ti`tdA>TrNn^0jR*TS)T7EZg=I+Tvbg?M%RACyp zg$9a^HM1CpwFz}QQt<~5dot6v6KuC@qo7jiT1)s2hqTslc~Ft(^D?wiw_r{b}o|1wqwW zt3$}&cPon+WTwZgLj(W9O@tOMSU=Rk&mCSj?@`CR4Y0*jXGejZu2U^yHc(FLlrV@+sn(l; z02k4}e*FUBBC%l~*BYhORp4Kzxq3V$=q)@nR#}&i5ZBXP@@0j(zy@zbEG?aVm^n&fmJ@xyfE!&Az6?}gfckEdI(0fUe2 zr45sOsw(wzw&P7=)}*w>pI{aJL*>Ozh4flV5bkdjj<)1IuN^kH%hM9DQ=D;7)017I zX1&_JYIh{S(;VTS4z(js3Acu0sqJw0uXrB9)yDIk9WK$rY%YEtL#v-nHnx*YzS~zh zF$o>?xYbIldiR?ItGw0w?tK2c`^NWz*#5d3XpW?rpz;~^=>xX~60d|C+BEX;$RS_F0izPzO zmeAF&(cf^m)4B)IPXo?>#t4%k2OoI@_k9Ev^ z@N}>VFmL78FyO5Sf_(oiC-45P5XvVuJ1LB8yr>)Y&W6kh<)>fUXL5!uCXgqz!FZav zU`#+4*u-X1W)3QcKMRBWL*%#@O~!&X^>KZYxe3M>lbTB{9y8SiymXUEcFG6rk| z_wR?FB5^T?pM=GSpFC+V@9$e0gT6Dhc#md)w%$Ej%@KsWSJ$WnE_cF6kiw#|n6dV$ z*|Y5d>)covOcr-xMqJsNNOMVuDFxqY8`z>bSXcz?FYJlB1m&gKhiNTea&)Vl_uL*=z zV#0dA17v}v3CL&SWrB8ATW1JP%1LFU+anLa`TB(-@2wxqkS0SbbdB1)L0f)@Wr={J zv}biJ_0Rnsio`+5A#5?aA=xk@f&l^V*=4}*_aPU07z)j*95ZxT(Ym6NbCREZAzDS}IIo{df;~~S*V1ZUu}i#{ z*Cw7CxYthaoUVInH!5{>l-P3^4xl(8z~`_TalwFZuVhkDkljiO5IOoPp-}v&j;N;a zjYyGqYh+1iL-_Tc#(W5A0n=Y9@B1*Zgn{QoDkb=z^tfgGqh!svN8g&i5bgm32(z>{ zvoA|mfHJqg^CJ{)a#|!s6~9Q9u}YIbYZ+V}ZgTztwp#AGGk_Z)(Vv5ZDSj-DKYyID zdi#~jX77LkAS2%|ovNwj$9th6G`!x{YJAuCr++G}Ur}+f%?tFv^%@NeNDtRME8563 zzn}RF$v|Ii^iIW5pEl6kWg?Xj7Tuc-B zl@m2S9NPhdT-s{2|aDLG#NL0&} zU@vihIMN`LSz$vF@Ac{Kc;GR7CRf&ngQP#fwFen1p^A~1MrNtL#dP$v^&=huraq^? zu*EpZ#Il8=TC;VNWbkR&jhwl+4K$V8to~vTpawF<;eeAy|Qb8vryUV}CUU5|`+~ zhgPezfPR|c9XP@a<1&jm+SD&PX>6sz<_le zW0vpV33M0!JhDAZ&JsHt`}`phFQ@U`aNPj8>OCq7j7Oa=;wH>V-FOpLF@ME@3fq znu8~*DT{mooF1ppijhn%rYq?-IKV%2Kdzf4$H#W=7Ol{Q%8AT{#cS4!8Z*bie^8j! zG(=T^w>C^ry#NY%W}@6JWfaPOIDu|}+r|O@)*-dU6;5utu-@m+D;;9TD+*?Y zS?t|fIx7ps+reZvNjPEnaV$>9-mpBa-Zz=OA+X5v4ty+3! zxMuql%lFOo!97BPW!r}y4AGUKPCfPG`nefZwm1ZItBCKY185sz)iZxFWujE_4-FGl z+3WP+M^j)U^A&GI5_^PZJ>ZBbn4%ckDzr07r^=r(j{hiaoHLd0FwmVcW6*V*W~y^z8u|~p8ZO-KLVOUD!Wys6Ep;>4KSj~L6i5kHSR(f3{8l%-+!Ys zE58UtyDqRXRr^lRJd%19|2!8)Uunh?7X(B4lm3^bO0nA?mwG}^ns@^_idDtX1=EA-Zl9>Jx{Pxulhe5;N zssHI<3$NAC{Q)YD%0BFEV2n{iqSmTRhpAP5FNnx^1p4I~LI6l_eq_;_$qIQ2Fu|mF z3#6thl?~Hw2y%pVL*7-caTF#3WhyPX!!)&NJyCQuf9QyXGZD|xiIMKTf@khK%S;tS zy@etF47N**UAutFd%gKQCu(#xf)Q!kDjEUbmofh{kBsFx8zr-LU2jOKOmDlFp z7AT0z^vzd^lT8Gsabxm9ip}B+Xoc7tNjM8=$%rvW!8~Mo^#@?p+@s*PO3DQ^3zOJq z1yxZ^&WJ@5fwWCh)Fin7SPI7k%&^yxFs}YG&ia1SLmS9tDs*Ff5wKN5@@jS(;hnq3PJ3osK?-cLMQ;YhpkHsiTs7Sj?V}y)U?N@~DFHIs$_yCrHGC^YU1sKVs zFUAle=*WQ}R_sup>`wj16TspNw?{{%M7Xbve8l9Z_QGEUg>CCQiMw8zmkGhgpcZ<) zw)*xapSpInPYC6ht6K|bgS9mzonzkSYSxVO?bSXw<67a>Z}WOIV5N3l0C?2wHvQUa zF;PR}@x}SZxbjMk_@-fl=QzRi8B}%-zLsXI*0T^NxCHKiUL`Ak0mTWhE+LMK%QB`E z;tTZ82zJk|Z;eZ>p+wE(&7YPNvoxLM@$$CpHY%m!S=vFpjf>0{g(ElVWD+DHOk(_~ z`$GyGFJ0c2xpyQ%q9f3!V0sdYe)=knVqnBu4wF#y5+!?%;z}7U@vIL%wMsk4 z5)yO$38oH$3sT8HofAt)Ki;P#tiHV#Uf3z#QZ-7lzR2itRTaM=#&CUr?rNDF(_t0* zr34@avi^vq7A-Gfo!@&21X1JW_z5kx^t$)(OgqEDNM$Vz@4Kl?e3LeW7vm8%Xc*u5 z9`c_9p%y143%0|@u-bR%E(N27K>?mN=G@c)xgL{KV{cx;*4It4ZYcF|44;y{qFh-@E&xEtn4@W0ap?6p?Ky+1(dlZzsZC${#N;@GDKh@sTK{|DWD;6(WX-WxD!)hS z7}FHSw<0_e;~QZF3uzQFR_Fpl1Cv1 zjG4>H^k@m*Y*mhG#%?8selFe-A4D}r=>|y;oP+Esy(i4llt?fJeLE3LoGgSL&Z0t0 z{+)wNE$(}98wNAjTZF(mZ0*R5_3n0*SpmMjrF>y@hXRDsB}#TqiaxAh3~h_lZW|dJ zpC;7}3XvuNSrsN)tU;CHf7y5vxYSaBRt*?%J&3M^2>OoJZ?0g?J$Qd#kvsI|(Hc_> z77>sK5jrx%LYx!SaLGVd1aF!DuaBzhC7q>NC!p#2C&`8AhUw6l(?Y$1m&puPzZ6!? zosspPtIbvRbnG}R&Y6QXVI9mvYj6X$=A79$-{ zKFYTcMcpy`G=FL4ZSATC(a(Q6g`NNI!t%@|H;MMu*m^h7_Ug=EiTAqzyaWSAthXzF z++fS%X<}f;3Ueu(>^Izgt?f{htI8$AQ;gjsK?O%zhps}O9pA@$y`94+y#|+jHVxUu zac`BF=@$z8mZt2rPjG5$KREE{#MN1|XYPZVbnMz!0!Xql$E_$|@cy)5xQ%YGV1W#bz)DnUQQ3E@ef6fbD=maC@~mKLb2<&)icf|SDQ`2z zH(O3D+QYTQH_%+se7ZO7YdAF)ZD8>Qnf-i`x!O|q1pzuX0zd$;=IEAf3L^v$01e}| z=j4$#=mY``2>wyuYTvp7 zIBI0~Aaz1w6QFp@GW=!>l?0v4`YAFJC;8K9^s#LK4O1Kydp651vQ(J)m@Ltg0>>x- zLr*)&{4H!JAxoa}mp`FljPrKK()~Lt zb6qz=#F@NH>kofFBX7exbws*BFQ9tDgzxBg-exV>Zfay|Z{-*xfIf*?ko+m$xQ+EZ zC;vW9J6AH-m9rDLp`e}a-V;}h_xRTd^}E&(jsup%gZ0%-rJ%XlCq- zdv11j65RmqV+PJYaH zMfT4eIYN|{A`44-$E4K!KzgX;L|S@?=o0U5hUjzr2S~yi1^tZ=ytSc4-}uGzK2eY} z3_Vg%Ach)0H7SU$enUiyylo|P`HfyCJK8(J=_DUP?f8MAT$FON{&qm6%>Tm$C(%3ACSQn@%sw9zt6izM*-egHdeh z&_2V9PAj!B7@N7vX>cShN8OLtma*}J4H8|KQ8B!D$;>(o$Vxv#$NHYSfyaV$50J9g z&kSr_=V~$$6TbGNQlSl1vZ;-atLHdU?%zs-c~tk5PM)(lpVrqncXu6+GbgOwz3Fnk zi`FDt)R!G!#Kf0gFt+Az@&%}(w*55Avz3sQza?Z>}nfp zO$FEa^)zX_!kfno^Fhu1<0@-8AE$hputIr;zLOAkLCh-&k*)NdrV^J%5lh$AIcmM_ zu%8A6{;ikv*E*~h$aO!gleq@(_mH7?Hh*KdWE`%^L4y)a8fuA^-Y;2I*34E4{zl2M zrG*S^4+*pa++)BI&|<*G{!XdKw!Llqd!6lvRK1#`DYuvLQ<4*|Q*COs6Qpn|#?5HN zJE{ea#B$2*+*tx}T}@P7vGgN@@rjBwC27eBh~cjp3V))e>QEHpCJy;TR?!2ObzjIX z1U2XU>VnaYa<7$;REc}xd6!YUQi~&jE)h!JJAL0&CL$+Eigc3f2e;`24U!cC3LmQp zGT}Or5ypd1ofP>KY`#7?B=^OpS9t%HwWF3$6);%n+op&4sa7MKR{60_PTRTFXshh@ zAS?!@+$Hj7WSiks9xDci4Rz+Y+59Ir&ZN>svO2n*%O#_o&7TxbvC8^ROq&i<@n6M% zu4H``KRh^0@U%$IrMtgtXAso=yP|SmdiM2az6^B7tlmBu+UjcPPCpK6<|qe*FT4oa6U0x0FXjf^ufxty9vg4T$tacMXHOgHF0L6e19)zG!iu(-`ISCD zjn6^kGOfdYT^DaPbR0Z?aZe6S-T^Z;3-$*{>nw+)#-@PE&_B|2;Ir}1ZypR7`fAWN z1;mFBWgNiD`Cofy^&ojgz$2<8c>9&i*b>gO7J7SM{!{%w+ol?lyK@^TrDP44Swz z8RJ3wbvMj}`3-jA5|$&hss>E~&J;8;Ewt(60BF@I!p$?FY6RYZz!&t-P&9%*wHhWb zV{|xi3U_=S^R0$L>NO~7Ej%B2V!+|%RvG0-!N-LXo-rPTkGLB6!NamoATY5Zg4+Q1K>Vi>_e#yW`4&aFz zDG3HqHwA3QclParqq`@5D=WP-Om=an6wGsPxYDD1@L6LdTvpL)xu(>w6?xQ2o}!LV zCi-jHSxW~ga3SYo5rswSA^o-bm@%t6=9n2a-3J^Cr4lNvI`TgR{u$8)6QsG8Qudn* zHtodsYHM&fW=EGe(#C0%lU*-boC4tg@cGYn?9KtFF>^T zJURox1kc0qnQ0~jJ3xLKWenPAjxG>-%#L?AP!Ij0W!kM}TYxB6_0#@SAiZbFhpiQN z4e_-XDHEv<{Tha(^^S26_<dkn|u6hdcusQoFHc=G_JPfH*mRG7>-suJa+45bvN* zgw7S~fmLC%^Fj771f=%**ZwB%yxjQ@Fi>-FVhDvU&)exi2##@x9}bkKNI4CikxF8J zW1Al5lUXh)#&;4VlA$BRF{dymdEtvU+WVZStrkP;;V8d-> zUCJg~LBpXx1!GM*_cj+aDc=@036Z-(R6p#P2yJF^rLv>J4@&y?XXuxa0gMa-i{-c0l z-V_6Zf5p)q)T%fR7hjGwM+eyX%uifgFn}N-06u!ltF>kJreoL>UwE8V+b2~@HivgV zBvfJUjf@Qp401o=!{r)a1Lo_Dac8P&P}hT%Lw+#orEGU$V?YsLD^_M+IPm*fwpB z1s(kXcj%K}HdQ705zd$DWKW?d{(USlae7`fw`qhPHWPCqnJzty<0H`wU-clsmA#*h zZ*&#OA5_=F=H=|YyOWm+Ic;N5eJk3D0zS_LQJ++2PMT{k*=;&?W)G&J(Ju-6KRyM% zeINs35^xcvpbUeNfdZf$>9Q_eg3>uag)fQXl}(j zFrUC7f+!#T9B6al-?+fqTu7qLeQc7D-W5k+!jf`i3j_hIjl~Ips7MXqZ#R$62#LVw zvXv=>zFN7t^qZuUafJ`lxAjolL>4MNoh}noEJgZT2Rh~Y=57iQ_PwtO})R}bY z6*^_?uS|EXsx@gf7$2n@X-jL_OD(id;~e50XLC$=SZT8Lh*%yGH3|XvIVs50IB8aH zL9y{k95*pyI8hRA+xX-Lqj}tIu>l$7_dsMzd}~(L4%fdajqn1S2BNp8#kJlqPnsx5hoh zy={yLEfLCWQc0P{m>KM1%JN zefYA%ol@{34%7N%xJO+#&AcrAzyL7dj4#fXhXpV`K9c_bH{ZXlKF{~jOePCg`aGA^%&z znzX+B=QcXyVFuoWI2*rUFm#_HSgbX7YKsK-pIaM=>Yh0ps>^ET&W?0j7kl06dt7Qq zJYIyJLRhR9y361*+8X9!8HJCI{+8`5V7pMb5~nn?l@*G6cvpS z6em8jSSLT`z6${6B1u(Nmkx72)HM>NH`G|qvnkv*eFX=`21b^SIJ=eT*TNxd<0^cy z=aWmhex@e-Hntf((hQ#u8#~5pWq_M3E+ZFiK=mie!WXBNDg94)rZ)RXFXN|1&ny zEtDmAm}0c;638GnF+9Ie_Uk8EgKbi#T4n%kO>J}b4@EH2d)75R<`nYox*3$6!mrPo z(Wmovkv)^Gf@QA5d?5p9306=lcTX!E#>rDjyWTOnZ##0f7=Ka&tK4#eF52rZ$worU zMT~(J;W|ijO1-TPJ}U~C=OR7Ivb}hQeTWWInkuzz}7xmspQs+u%>vzgU|Uf@W~ zsbh#rx3hYFn0QZnM4Cp}zsSFWm z=)I&CIiiwm4q$p)L%jSlOa`h`2ttZR0msB{Nhm|lkSG?4l$(7gSA#60gQnISXUUQR zRSyaj{}}wfYbJ)i`$HXUQ#I0EtAb-=LeN97DQn!H)D#Ph5Fy!dNBBwh%n036rv}5t z4@G(`eM12&=4tmE+)_fp8@(~c*&ZHh{!;1iW44M~A=av43WRO-3=l#Ij%gw=ijG+X z*`)~%iFIZ~%-@57qxtEY-J35;gh~+oqGV09lj963nIOQ%hNi+2`XhHP%^;$wn z4pC#v7Js!0(8bjZRE6;7S$FKJBI@P5W3mr|9XoLp2pMvpP!viEz=q@~0^|9k;#aE)ynw@q6@wDl!w9A#)=g&7( z-&BS&&!N)%IIG1`{B_xjHcS@k7avJ1q3A6%sp4DnZRtvB>uuYzCtpB^ zRB|X9VOn;``9Iv2QwEuW2za zewA1JXb!QAwmPS9EQ$$+I@35FB75Y=+bHrSR|3cLMZGf?gU)H zXfs?Kdl0)6d!A-_dIdqgFkAFBbRqYECrEFT=uW~KCO3)D-_#YlSRt}!A2Lk~i8m;| zA@BY#VEkVZTJ;~`E_T)lQ|o3K0R@{gBKJ5xK~ag*004hMbY$X!*Tt*%&}t+f-jm@b zm@o8C?fH!`f)`;V0BXTwU`BY zy%Fv3?-v&T;sHu5OohlVEM11|Ke_={&beRm=Ka3J*ApQ+lfTD6NB>F_`^)iLh%bDX zvv%fs$l>AiIhP_k_EOX1tL=8v)27GjWv5~TU{C+)=4v?tS+HZbdipz{%*QkG9bnB9 z=3iQv3jDu(up;CvNNmUvj(VTzR=b$W%+t%{@41x{Cb%rf^Fv7hzmwyhm(1cQHHmcB zWlark-?>$$Fpg`^!Y~ zY}=aG`{R;tKr}&+ln9Wz#o6?nAR$@%uPCd#tH`Ug*kwMDmtvx^DM%7q?Os92_6_TNA%=rAC5R2NdXBn zrYg8RtE=s5swnfXAms$gZAA&Za^&@1jZ`-y9p!<&rZT)HKGu>G{nH{QSEKih&+`PL zRG_tyVYRS{p3!sO{uTs@lIp3}gV$AW!15OFJGf%8{|~&+=8=^c@e}i_8lgcC-M2+!>muXY0j{qQcHi`;5IK#BQkefN$s23$e=iU4 zjkl^mAP53CBl>Rd7pDTQ_zsu=83IS0U5Egd5AVA=4Ie7M%U2PXo;7S%q!9ri-KW}N zL;#A)d3=2Qd2Ep_)aN{_H9hbwCL z&u1nWs!mfXDSsG|34Gl5q?3373YsxIvC($|;iDwrB<_hafvqOP(P4Oc2kEt=-@t>U7Wk zs=L8pxj>1_`#ry2-#$lrXp;@JfU)yJgTa^p4pr;E=wAwP&t>>nPHx}-lM>1^{h9#m+Ra`KU=OYu#Dz%b4|Gs2vm=Ha-& z0q2HCt%la>2ZtX3IwA#86&IlLRQj-a6=V09rHtN4P&;Zd>*sPNu`B&@Y-apPVe2rH zVx@R_sMiod6KV6%P8y-fGymoZej8iAw{Z%VSn`F$wcm2dkZx+)BZtySxWMOU)}Ahy zBJ1P;d$y%3x~VedLI~^4l?1gith6ZwOCe+-KErEK%7Q|7$+E$Pg4$usy`VHy;IKkg zG68darKlsTbUG%Q-J&|4mv`G2Mn&9eB_SP8biV?!Jn=bnXqx4_i#kVMDQn>dLFjK{ zA}l@7tN=Yx*um)fin$BufeIo++&K*3?XNrelsgRBQKc&W?e(;Ms#q)w63|^RmMT>H z!=!Yp&Y|>a?({@_1(!XuPR1ZYJVm8q~RC~mq zb})i_cFybT?*aJl{S4Mwp63dvHKJ7SmVT;_p#U`h-ZJ)~0Akgxk&%(DYhm9GZ6e+Q zcz+1K-EQ$^8GI=oJx9TX`#%8bP5hzT%BY{4f?Nmq6$+7H#y!y&?i*eo`XQ1s$_tq&<+wN6ZQQU!Ck198}d_hNex zKO~h8N#&^(@0ChkqjpWnjLT>JOg5kRJ6Wzc(q|G^n{I_&H2~{-mx(bwhT`x5iD2?? zpQ!+Q&vk3!FutdrZ!`3t>jeM7r5x6r*Lzu@Xg=%98_}9R&a}LJN8kK7qYmhDlkpF5 z&Eld!G8+~gt{l^mNL~){ElI}&I4j092-Kn8AZikPbl>3|J8TO}(%y`)l;4MYtuw^p zpbNprnPEk%rQZnf@3J5fIs0=CJT=%JW4H9-)DgVdhbG^vw5QZ7P6+c>^{(kR2c2?) zp4c1MA0}&&8Z^pmXs?~6&`#zoYIyVM!$)SN&^&-m&d%r0mb|X}mIcw#;}0Nx-+U37ZStl(mRF1$alNTL~7G58fyjTLJ^ z!O{U_8h+*7k2WdNYRA}FVtu5`q^FWGw*7vhu_iSL#B^g%@{L++!cW%SK+0d%$bfzG zuE5?eHQQ5kmlPv1=f(-T;ZTG<7#XXJ!-cKU-|cnOHB$TI9?V4Qk$RPdkJEG)2=gln zay-C>kza2B^%pszX~_vwpqN2~c4rWWml1k zIt+PvD=Y+hxZjigbO*geRGS~&vGs#)bQ>!Z#aDNe{o5a4G=?e8bfkK-A>}GRBjIC!CYU@>Gc@>bT1Zo}CqruSWyHauA;z5!{tbmjig5 zN!v&@v;*9P?s9Irqdo-s`+REB=U;7mHIQqg!0@5b0g^*!<+&EOKdgiMLE9X2&yAEb zD4)$cKAcpruLsNwIYd}#n{*7L=X14)sr`*J{A#>bcSpvb$!3(hJrcx<7ovxD_rksL<# uwL?IC|F+f9gSO?$1A-DyH00DxAK@%WofB?aQ4NhQi4ekVj6MTjYPH>08JwUJ-+}$+{ z?!NqX-`(@Q@3(*K_BlQGRGr(m?(M01>UP)D;i}4VcsNuz0002*qr8kd0PqwH06byB ze)_1XUX;#xQ~*?!G(J2$Je-}KEi5eD+}!r}_w(`b9UUF*(J*gpY+hbooSpCT@rgV< zTwOgpY@eNJYHB?^96el3@$rMC9=r0%ADo{*Tx_ncu4?e}Rqi|(@bgWaUGs@L_MSg1 zpPjAUJ`Vp^C1tMd@0o1K=)-mqpMJ$jm;G{yNx6~0#ooZfY5&P%my?#IY&At;p|idA zkdJ(O_YVgMe;1k7`J;!q`-g$O)AN;LK51VR3KctZV^zBHaJPK1Rhz8BceglwH3*6!W*aYYzFM)o607Jzf2lp% zcV|kkH1jqdcIPK1CPbWja>XGz))Jmw`*}?hYR&Yhs-lJ7K;$#M=3}$ELP9`g6_wFD=oA!hp zhaz<^-9`6CGLMsz1Wg+iEraS^lwCmLC!-godB-tE8a+Xu;dWx)L)Wm0{fF%~Nu4oJ z)Nrv4{H(F4&P2D>lK)toZE3i1kHzq)>*x^v_q}sdONL8*!?uxj?d^aoII_C8)VDOs zZYAKI#@G6p`l#q&=N~QxmnQ6oOw3(Vt7>M;YTvsK{HxR@Dpdn2&Xym*WhbYDvwNCl z>A;_vPFns+@X6L3Aq{UokY<-dBQoVn^AG*SpPxPJSNFY!QH_W=`TBNvs+PfzuEnFp zYt-TDm`{H1YE@^Ba>tMsBo-32pS={@HGkCF`S*P9;SjY}mcY->3Vas}7lEL<)xcB_E9GNNsXW&3KKRBY;)F1( zYXAV?PyZ+*so^n;nzIR0GNI`?+Cm-mQ#wO`w!=7?+-(-(-7LOr*fSzV*_n84L_AFl z;CqdH-@DXhh$p#{=9v^czK!Rdg}hKmQ8cWFdPlrS-+C)D&=1oU;oWfZ%P4g|SAq#$ zpZgCw4-)?hiJ}4Y;xpyId*`_^0PwC2t&iAoXl4l6%CBRe0fJO9edVf^+~XFr+G3v6wD9ka^2D0eGb(YY;Ngeu9v zsX8GA^n5HV;mI}FoL_#kM8D| z;^3HL=ISJcn3wad&1@Rh_{`0Es{g0lQAb3++?IeBe#NWJw{7nF>R_@vwv=Ezv>|>! zOvmJPM4svU5 z>uAqbBt(mjO%vD>v_KUV&Cp*!N5;7bUOJdznu}4brB7)A>aWMXpEcO1D>P|U&C_jc zW=${cOBJVUmwb!L;^zo5CSf(ievd}wO(k;nZW%hdvML&@73t0OvH{NM?;{*uaLG$k zQBJ{e9d0!hs*dstAjv&g_a_m#I@+H^g==f+v$zRGB_deL50CAsW?9%xXEevv)j(fZ zmx^#ju9oEo-b2U_{Q4tdTfU1<&F$O7&OOP9<7?nSQ048kFcf3m_e<-ACI?=RPrG~o zSgtN3C1YK1ql3;)G1E`8WtmIl3jnnSCIhVortzaw|EFU= z`g?^W7%=*H3Krl>yeq){jR`@M$AX}lB^BTT;F0TufL@mW-)pBcfq?CW|A=t~;6|O`+-5jS8YG5Ab z&#FxA$FW0`3T!q^q+}0gHAekun<^;P9J-HHDJ4QStzX^nL}V6KXUu-Nmo)eV^iHV6 z_z4D3>Xlj9riLTZ=}Ppvl=V_dIolTbnJ~ORRl%pD&MM08QxF>NRvcUGuEjaQ8cC7s z%lX0L-%rI0N)5o8ZY>HTz*XY~Ha+mKXb*5h9V5n7rGh%iYs(AkY-@v=wh1mPuyrBP z*SFfeY<4E*)?pHuUyCmnzfc#$8)nbKXIH}q2H}$!-#0#7_Z-jmGiM!MJ2~0$t2^*gld7gB3Ib`= zST!puPW-*ZG8uW%`_*pGT@6I-mQefglLoXYcb&>A-oH7ZVrOC>qQg$CehT+{8E%xT zMn=hDvw`ZxR?~|kzwCK<@$eQoeFA4P8-HgvMs#ko5V;yT+tQ-o3n?)17fAaO6lru>&mFCDm8IOE3a>(VU3A75aJs{+ zT78sz8ONfUug8BJ+Qt6Sq9DY^u>Y_To8)xxd&bKC@j_i#+{l*~dXBIY7v(NJ{dmG&_uvvTqFQToqeD zDN;S_&$z~F;+H&1t8>+=r1RQ~7I#Twy&oUlW@cK%hvH^v&1nd3)|#BR=4kWEsYr$R zl#HIb1V*R`TXy0@KO9-T2Q)hS6w;+qCxb_&l%Qk;M>P1E6xG#y@vpZuQ{p)!f*MJIl(_qfL(0nML7JDPHBAwGtLh`V^WN{rib8N)VRwi{Xt;uBCG)J0G@};XNn`rZX-Yq+epm@{4_Yotr*tD4)Pj$qGg2CQM?BX*~e0Z&r zHl#gY`RTTQ^pp5^5^6jmW-_{DaWsW%qgksHT*H4&q7a!Df9g63o5EOT-MfjpW;$stpVx)N1OV<9CVR4nAu8*v*&@H&8eC zW_d3VTut>(di_OYhe%7Hf%w4R3y^MJGgx64=R7=LGCm9muZ)oNm4`0G$ov)3?22yS zAXS%Q`Bq_F`2O#pqj695U=+4zXyAhft?iRaR9GA?5g7&%4JRn96(BQ0l;(XaqOmI)>k7~Ef2F_w*k6mY?Y6vStElUO2 z*et6}0J^IHXR}#B@hna;Utt>W2sbAI&v8q(s)x$>BxBh*L zYi`Y^FN0b8O()OiWsqD#1WN7wnKRRTI+5}alKaPbJhU>DHRNa1}Db;FNMLH`Z7u*s*!d19o>>LJ0?WcldIO_fVU3KH2y5|n_A||laAyZ5m z;*RyK>4Fq|itj;ntD)L=l=Mb6&6boU!nLa+mX(|yF3uA7$K*&j3ACL5NxLmb$$Z;; zchWb6le+2T&Ot7^Kl#ZESi-aeS7e=C6{x z@6FL4`s38z8UqmN?tmA>N>x1n1N`Lo8tdt(&v(Ah5|>wy`!nzDFg05!(f6^V9#xO7 zY9w6t_K=j4KAEdCGH*MGb&j*?83rL7^|#mwnzc1+0RvVMvHrY)?v_b|1asHEmFQcI zuh@lylbu-Apsw@Zh#jbL-uQ~nDqu>pM@t_Mwm9~^K=fkRrm^4%HE{z?*F6bw zG+0QpH#&duzIqBAi-o?tF{a;dn)C~-qW-0)zwhju&ypK6$Je8q7GL9$#bb+8*)s$J z!XjU)P}|t$IjS}`*r+OPm9kRbSzFTgJY&nD9y=n@qCNLo-V%5!D89#KEIgC!1x6eA z0~y#>yvq!rCLhVcKRBJ1Ix0mvFDA`=duyD%?k-XT>L#Qm&MjRf+}vQTYxp|NrH!1$ zo6qeXK6!;VP5|Spwsl?!>60$p8DBeQrcs1S{h8w1;p+A$>%^T88ysc~JOs^ta<%mG zCx`Xbz5WAaE8F~hVPR{zcquZZO(p3RJ>)4G#DmznV$Po`i7ZcAAw1srcjDtXe`z9h!zM~fTt{v|?yZp??WpmSME=trVBP~J?} z$%pe@zs%Rqe^>a|k?Z52MVusnp*1JLbTH0&kkdicAl#v4Zb7f2LLGK`i)`g*|9l5d8eAGN<*RHN=%$rb7dYe!p z_FBo+S&*!FJN2K+8(w_W_?_x%2j-4nYy$Hu5c?bsC&;=F*!*4ViLUGQoSvO3X{8^< zDiw2CI+&DDsORTEer^$J><*wl&2We3rsfqM!P%uR&vV42I5(_IRv5ah-WvqViUaux zvPI~K<2vCHCk+83INZc@<`}X-n}3A0FQhy0h?BwG;Ud%Ia@&5wg#5@_xw~*p}0nn+N~`=VX2F!l1=oadS7-%7jpmkQxb(6n{Gzn;4j&DG|}(j2f2x}xk=4s!c{Jq;H7^? z3>F9?J2MvuW-H(yI}z|3wVo62-|rVv}0FBh2RGy95(T?Kdo@g68e_^7rb91*e~aRs=BqGnUG-a zXfYLy?!<^yq-ARD<0|Zy(>d_CV%hsj3n6OYO^qB)M`cO~FeT>0fQd}nk3sOn4 zMg0BECXbW;;h&FM4oxnem?Xus1(u#A4HqwSX6FoCFV26bu4gT;SL>4Zz7dHpPf7pn zA*=I1rr?BbX2Qt%oLR^jR;5u+Vg~~!f^;#-Y&H!F!Da4A5(|Xpne^1%FihT-yWcos z0jq#sv$Q%T64HFEfYllrxROrS>56*U1rFBdg_{28U|l00hgbD`ugyE?PMfznnO08~ zW#1S=*gv_^R4UdjzpZv!MA9`N4__zuB%e&f%gOuioCM&U6s2Squrl_b9#-bNp`%+l zp}@k3AuENGk@XW%BWz{mNHCgZi4N>{6xiZTdS^S1J@B0+%FPT;H|&XYk=7-_wScW| zzNe81{$9oB#lxwxZL=+oczDM~cdU!j>lH0Mi2maq!P@F}-W4mgG7uavGWfx5U~oZD zj7PhBb6}}y)zAz!X?k-yv)znKK>0zJe7fT5vJ4Z#BEDnmDT&@B%v#xu52kEXT*iyO zJ|Zx%1BqjvBvXMY*9a!JcgQC{D?hmGX1t*ctKjrlJ#>S@Fkd3or8ksJPw*>pQ?hdTU0)g;U-$N znkc-j0)G~2YHBi)hU4;9S<4)_fogH5n+1@WPxNY|bV3%cGeF(@gRl?Pfj1Z0`D9jU zOTZFP!UhVf^VBN6{f)jHL)(RQe7JG+WV6;u)%W`FzHS_%D!&Ef5Y4C zdfr)mO{h>Z-)Y2CZvm^Y!b`rZ-+CK%>_Il*m--XyU8QqW$7;y>-I-OV5(dNqxvQ9x z(f5(NT-nqmmZ!ZQ`HLGH%X~L4nH8(^D}V z`PbFxB?y}1HC|8g_3Nmv=JlFl5_|n@CZVy^_Fnhfs7xEng58#jZE~Bo$?{#L43n&u zCy{BN6yER&@;lt7ZE^g_6zcj(apDs{W@A265s$JPFrb_}rKs8dgHpE?Z29CJ zRn3J`pY8_!yz6_lhuxbd$CV9jAQglw39E=)jRSFfU*z*=Lf@iA+k;A|>jmvk$6Rw_ zvh`RlFsEddm&O=K>l4jO=Zn2HZviD|gn2(C%^_E#$TDsxuD8LJuh z1IunWpuEcDX&$T;<2PV&jw)bPdtws(G}pw2R5;YZq-PKDQnO`$W1+F??-`++#+BkS zm3VRcnCR;t9q5U*xF@zfmcts5_MDj_(h!y(&LI7s=w_c~k2CH&w>+%tY^TaWsN}Lt z%0{2&nMr#(CD(~lWbLyi)cs`T@Qa&|uK0BphR88@u@`l+yaIVp_xA%ox>{FE>!elP z85j2(N8seR1MYnaw7YUBed#_0V)a=dQJtsx2?cd)BqteHKuA8)^EW^LbJl~nMxdQ!{GPO$!O$_TRHBesMCirSD2RgwH4T9Oe|=;~NQ z4CX`r-lu1&pyGabFjFEi8~vTmrb;8)M^>3glIX**!8qRe--es@AiIH_B{;s2wlU?L z3*pUzNngr5oU&&oFI14OCMOA8FnIeOx{@DiAF7}`Ieb<53c@l|VB5^3yo~+PBa4R%JVveD&Ig*qP?7n4&(;AdZuH5aJ3>C*OBrwTI z4XaL&-=MUpv=aK#66}T5eSmJF_?XU|*?bsOHx}vM+ z#+S;}bx{_ZREZyoGK;6-6SJ_oD9(f!zjCmlsupmU+;+iFXc&^P+t+nHq1z6O+iews z5lYPr(Zkfzh@r1{DX`q{1&}f%d=ApDIK89kZGW2zGaZ<(jdgWJ(X)FA*c&(;+Q5cO ziVwnwcDCOQQv9IU$ZiG?Cn^C%=UO}=mQFsAqV;DY^BbHZ`bHJDdN;qnJq<ItA zHRTp6P}`_x@vH84SK2UMzuBMaDA!n_@e~rb*iWs&i1INCNp>aE6vx@w)3}R`@#8Ee z5$OIMXo!5mp{`#}&vo(%@pvD8#XG`cC~SzmrYOEmQS8dBw3rm9O!29%ySP~`!dX4_ zzH1Sv9eJmXd7`OB(ckEX)Y%jj@Ou5wGu0p0)bAV8lZE&^d;QSdkNHVcd)AvHQ=z@ZE_Y?mwP*hQEo`<%@d?&}1*Wew1 z!n5+j!u2>Z-@!3Sa9Ojj90vz_^o)_9!QZ-g?jEzLmE!p-jiuH~E_Wn#!=~RbE=M@; zh^Va3azF27O}P1UZ3Y&&QYE&OF?%nN#F!D;{|sdQ|tFP zkRQQX5ef9!dV*}`lsFKi>hn@B>nBn6l%>mf-%lwPfq}7f@F<1|1|+uByIx=*TOcPB z_Rgd93;R)>6U50B#`vg?#s8;Mfhf{4gzTl5Y@MxYzkrSW1`zq7D)>{t2v=1-CT@`# zX8m>&&p+7tMVkGl$!D1Gnk@g8VgaZd7of!)OaI?6^k4Jl@JI>x3dL&+@mKGkx^ew0 z%Z2gn9{O5^oa;MI@F<&-M1xtb@uw4!xc$fMPyb57eHci@nKc-Kv(aRh8-2#)R+eS< z>e*{M*w@VN0Vc#7(6x9FG(0g!p$_-+6N^dqD%%z8Ty%t-6*2_tL%jNO{H8^;&6%EY zw^OZ0w4hJ00EXW!pNd&@%EV+nmT+hi?-tomh;=Q0k{>D+uhsi((t}IiO{Nbno+ETL z=>^8q%SHYAqj$w-H*>k@UBW_c#J|_SsdS?m^_<2i?MeD%bYTE&Bvj}*nbT5J@!9JU z3q(0H-RIFsq~MSg1B7M?cygck%)MrV`YsS3T&i7O z8QSu5r~k+pTz^r8=TUO-Lr+88t#o-DLa?#hPJep)c5 zLJ1_)&Vl-C10y8#$xo$DY5C1XBBs5Pm;4f62$Z+>Um_gkOR2IjCvdU_ec4pbRqV9I z`TU@hX$1b3B4dBF7f4%7AAIUo!! z1t81;rDaUwvUUBdfrxG?j;_9#5iV-dD%@ZMxFA?dOaM)`E=ubVQ_2h1fl^YQ03=KJ zAr_4OlGcn;l2r2XrH{*_q=x>e9+iapg14eSF@%}0;UH{;RWXV_+zGka{?-3B12gLE zjKjc!w7Y6x{@;Jo3wRzbLp?zww7*mLbtVk?e#RScoR{ySnQN5OR`R`g!{pp!lQJ_r zfBjF<{l6Z|LxYr6Z1^&0my!`Zl20ldW~K0z%i6kMxt;gX*Kb;X=0yLYE_e@=rGb-PqF*Zu&rp|MjW$N4_InSj#5(9oPpd{Gz^<{$cAEWL-s zm|_#68iR%-bslA6h$Ys65!4r~zmeN7_ZCp7Nb#XTCAk^_Ox@r$(V>pXzfe zm(g78g6OuE7`we>m4Xpl-?nK>g7#6z{>XQZBp(EaBlsHsk{{sPOGSq4N)xhw`IU`? zL|ZY3Y4GkBD7bLBa3Cwp@K;Z>?dIzj0#(RJ zIVVYmYvBMkMg!)bmw9ES@vn0QVp`?O+5KwzFU+1CVPLen54^w~b@W7Y{T7IVf?I@1 z5_%K ze+|B9rWZHKudk4qY-&)zK zg(cy~#(qDhpdZD@*KSE*3gcIQUcU4-kC>Jr<^i2smaS(tX{^_YhgdPBk(RF`PL`0p ziO+>MV;qRg(}lw*(uoVvx$}watI@ba@%P1uV4kkGWzFsw8(CfYKWG&JmmXnIU@LSdHaO`AN`+54!AXz38qk%`pZbET?TI2cW>oGxt-M5 zf|X0rT#4Exb0Yn}1r-~#Tx?kO8X#%E1=0D4E6!C9UFP;V6953ecRnAciUzm{8@9)k zXlMXN)j7$&(#4@!v3tFhLz7QPKym=A5T7ZBV-NxTSwM#G|JKMhLOyYZ^#%(u{hHzN z(}bu`-nho)`f8(56u==dN#K1GzRSK@+ur^2F_lorMokSt_Ib5>X$a??-?6*LUr!Ho zi9kfj(3k1rjb%vrxxvGRqI6^7Y<8iImVw*p!Di^(2oazXVnF(K?)Fy-ELzLFRiD0L zjorwu+LwRS&Wi(2>E51R8S^yjWjs4?^I9Pi0Zc(&VL$bZCU1XE>e#=Ljixi^r9ptH)za@>klF(2@X20pwjd7BpA_W z$`l{t*!&;h)6dt7~MU}nO57$1j8|b zAQfK&^Sdf#?A!lnLLN{Hm3RSfa6d?5DH6dLsyrUL2)V|&85iy-4Q)=cfWpy;WofV@ z4MegI$)jvpn~gbK_yy3yXtDQvyVAx?rc5A!&Ia5(+CPi=UlR-f;8^5hfHjLkgf(vg z8v8}EWiZps88C zsMvWc5d96OyKBabsoz_9+v4drbOFQHf2`vFQ;DDYU&a03uJ^uRR)TR9Lja!H1cH&b z_VQkqLa6+-7oqFy^kiDX7UC)%{q?g&1k{Q^x4~2*><)r(bLJ_X*%)EIao%G=t2Y3F zXPvc9{SIQ@oRS=06+bk?D;YRwC(xI_f+4{$8+ATlRW}^LR@K)$C@M<4uru(U>WeZ@ zGWYb;b{}%Zt z3`49p#Z+cqE2-1_c>B0%%`8u+^(F{ksP(aTaxP}3og6SVmI)z}lBHMEl2(Dex}Qra zN-49ky*m0AN2oUxI}MNYGp1{ThhmxC!Q4^DneH!6K>EqwkPt1mw9fe52jijSSIEpoR$T$)@L}`evcB&ZH^p4G@ z7zN{%VuE=@WIQ;dpDbIqX}Og*!$W(%Gj$=VD_@c)OZQ}348Rfim>VP}tqhE2xehgm zbGN<>U0S`Da~uhZBYIJY-*+euSq@-J-ECuu4(Mn8oRCU*SEdd|nLcP>l=pS@SMZni z8Cf{+p~_TUUsDVI#cZjA)x3nF{SJ{wpFeTB=GJ%GX$aHDJ?AW^7>fL=RiI>G{+dH8 zuVwk!XRXO!b#dytpjzcLZg*bOYqb4~UFL^kyLg9UrGeKSjNjz@-4nOXCmh(bi>;&M zp3QJH9ah2DzOt(?Y1-`bVoLw2;Oat@d~7YkP4Lmd(rV+lTfrCi`G~j;8yTi!++$aX zA-MXH+`PTh!|Y#HXnrK~EdPt9Q~xjaPT3=|C$;w3QrENP3W*PafZrd(ah?DyG(Ff! zzI_5LM@a;Vl98pC>OBLbR=O7FOC2wN!=j58`*+~_PiYZYr(X7n$srNNl$XE zpG*g6m2wiB_8Ik#;k@rJsd+5fykHOd(WeEvSYs1%&9j7^XF^0Uxudbp{1W^ergS+T zS(&2<6c)Np8-jf-z3SJ6x7CdO-S_Fqo?AP74J|*3fU}|!_#5X>)*-O7N7UK&aM>z$ zE~y*ov-zqRH^RKz;PVlz`~2jOW#ZTsG9lz%<81t{&nw~lecz5n5zvma?>^e&C}6Bm z#=TgS;kWzwK5I^&aP(Sz8;bgW!g6EHSghZ|La?Oh<(Aq1ex4R19&xGZaZDwYDwMEu?}XJ@BiBI0_2AZ^gbH z+=?w(Ot?YxKm%&Oh*I2AgawyE5LD6|t^3aK^3Y?8y>IFJlZ%y2&jvsqN&bxfF9uCGp*u8KBjP1Ensh%3ZN zdn9al-2(VxoIWkve6L!c&^QHAieiDm7MNx)nh?N_kfnw!c$M<|fMkq1a}_5eeOhrB z+@8xgPlHLBAp>shfb$JbVGvZ9`33rPo}ji{TL6MM+v!VN0jETgCb&oA``;^874!`` z4e&(b$`H5kmq=e`4a_;6`{e%`fPhNKe!gjy1VQR#c3l^9)A*5+1Qh z5BA)2$WG5|fxG(rJo+m>*>WFSE%KAzqYsxC5U_hx()?4wDxfP@oyok^eXt>;lDIMT zoM_)et!KPZ?tYR$pReI+QBl-Wg~i+@t!_R7=W8{xzbde?7#+Dp=TZF`3{CLQ^-j0i z4TBw&jkx{K^1Fh!88-3$}0jNuZN<>vlj4Foqa3DJuRYqzr(#O{^FlVpjZo{_qF|xrhF( z0NX$WAmS01dLQB7|FqEp5P^^|?nrUPLG-?ZC{K1<-B%;;(JE-~vf%4FsRe#c%D1^2 zO&FTwndj+xzswKAKXDG-g26wsPXy6a(T~}ijydJe>`Hgz@SHHyh62x-R(UXR3YEN< zg&8JM4@iV{v>PN`d@Rq8LgT?w89>^PKvYlVY){!q%{WAPTqRYE*s6igP06*P!g3LJ zEdaKk!4Qlvgr0?&IhIg0@$NDrKHI*iZ6D&$8oNZxsNXjx{8raTYMz;Dr3HLbk|3^s z)-kMSz8=?CDf#RCRer$^!wo{NXH?CQ0iF>yh+w6|Cyo>A!qT3eWG}w9RlV^d($)&s zW-so0tDGLnRx+&35NdDq2>~nPOpr-X<5wgfa)>EOGmhaSt!=Y8g!(zrmQ%p@u z>(iK6nv<`VCBcvrRb*Vg{f)Q9myp)MXHXB4ug#+Sz9QBo+FE3zbBkTHRZ@j}6+CYE z@kU6x{IoomExzofsk?52)=~C9SV2SIR9KG2rN=n3>3ki$HY!Jq)7_>saU6~l3eFD8 zpJcuv*P4_;H9T&FkU6d~C?3}S@+qPeOVROzF|Q5xMl&!!u?W?VULNQBX!YIwh~BNs zG?+o2AT!5v{gY?+g#NzHGy|~bl}xuf2XhssQJ#|=O???D3Cn@G*7wXbHA^tDGTBm) zoeE4u_DnEi)DvIheuOZyK*jRKz^<4f)c+}s|Jut{X#ma_Fj%OsAGLU`VUR+_ycpc4 zbP7sBx@!Bi7dg177Q( zHfmVW!-B|=&NFKs0u5)1sA?EQV8r=-DS*CsP&ySGf7&b^7M-9S{&v%@hHiE230v$0 zWMQaOX>ZqWcnVk=PkPr}BwILnGp-F6gXqmX^&2Zu@_r}JnGY}eJR<+I?aEG#@&E+6 zR|1E>{F;;0Odb8asnEC5)%RNM6Qy4jywlznZdX_D(JG6Od0> zJVUTdpgE^-d`1!}#(0NSn4)u#y}uLA&7;47kH!4;JKJ4Zp227!(xBqIY1mKGLs48M z_m)Im_!lo0IBSU|tQ_ z(C9vJKJ2M0QaV+yDna7@>f`N)xbY{J()Vy35sNbKtbNMS7n>y7;1-x7hgeCQ|AWZl z6{lsVv@z-XD}}z}tW6)6C}M;X`#!d&mK(VjeZ1ewOkIzm8hO6hKctz}&n)h1l9iO# zgGDQQ#bx(5``)tSk@R^Y{#N(d+s&>1peBm~a0jR};g)*MzN(Yu{t@@moJItkGz_oM9jbDt&SuZ#y}rzL<2jg|5WFO${|#Q4C}hrjsM z%^b9>V!p^`j)9Bh4HxNF4hSq|I@Eh(s7J;Yath^@UhG`G8QV#Fc|7GpI1-?cKRJ*E zJc|vx9qe!a+DQDPS8rBh324F6p{Rb-%=Ng}ES5F@`%P+o$AMM6@z8gsJ+ZskDuk-c$Ual#f(40ts3(f|&gG#TJh@ zymDJ^RHhNh*3K+#f1Gan-qTE=5T`=%-cV6|M|Ee8le{f6cAZt>`yq^IYkM#g##=bJ zZw(;iB=>|N1OeDmr6iOF2RIkthGhYNKR$5(QKei?1&cIncL=BB2Kw*UFM_Fs8OJGY?;KlW!u>SV80H3i_K6(~*_ z5{@gct_TzHc>3(QCly94Tl&?L+kpQz3x5!7YOeKU3X4?cIImjqR2a}!T2lYf?2L+J zCY6YCDA_`Ks-MRCApM-n1i}nKBuVHvP50Ujx640LTew)Eb(d{VoZ+oO{7;lVQ_D?z=H$ zC6s$L*}U^~m7oQmzg3QMr?I6g-MKYF+zDO@Uo{ZhpZ{RmEt9uT6PvXK5;I=cQf}m) ze*E%jR(XtT6m}-N*DC04_dx{_Xk2@9;gD5%V14%dr?xxrzCnej^+UKa z;!g_4SFz$G-}}r~ghi-@ol!aJ(VpvvMq-|ReCCIOmB|X*;$Bw#SB{ewIrP18Kt`L5p`)jVU3fjbM znm3vVw0`mRpRPrha$NX6>qIuxNTM1`k|(upe(d)=EuZqI3t2I{8T`9__O+l4K0EKq z;Z<*4|3w?Uey0afCZtu{HqA$u=+(W%rDg8slTg;)`R!h!uH5wHtMKicMns4ME#ZU? yNRrX7E!t4C)Uon$h)9lP95)|w_Mj%HFIFwa$-M+s+WbGB`0<0XOqrB%;Qs(Z>o7k6 diff --git a/modules/web-console/frontend/public/images/query-chart.png b/modules/web-console/frontend/public/images/query-chart.png index c6e4cce08d0f008993529ac1f040cb563aad2c2a..1b7ef41b778b88b3e66cfbe5bd35044af5df3f47 100644 GIT binary patch literal 17142 zcmch<1yGzpur3OL07-!0uxNnbzF2|=cXv&Y1%kU<2pR~!OK^8z+%4EH?u!Nq?jH1# zocqps_r6p0Ue&9*wbg&k^mI>mf88@H|34d{sw{)`lJq4K5)zi2tfV>;5()?j=@~5s z$`hw{St9SL6RoPKDW#L#KQuJ-$jz-9-+6Iy5z#Qq&CN~aQG0oLX`Tt++uv)L++5f> zHB1{gIRE2ZJa%$*+rPB$T``gVbGdDHw`YFWuWB-{|JTjKL(TZ_@6Cv0B&743Tig7R z@%7{1M^{0$)8b;{Ngazrzm8V-&!>MMht|!k>|HcZZ#_PaSmh3FZf%Y{KB7EBEgW3k zKE8}^n?InSD<58eJn!JrgI;3<`rRZ|PAkB>)Ac>@;Q z+#C0gofnVyn?BgMV8yq1_jFIRMA?(;G{BPdAz}DyGzPIB` zx8mpGmD-(#nzkO^P5C_}TRD9IUbSdRadST`fvfvxamm!aPF>nqTOF2X6KekC<~De= z714}09C>(7oR_pya@=a<*U6@m`<9mcYA?^1^f zCQnU;O;-{)>M{~IWWGeG@{A0mZb3iZTt2b|G^T>gtEj#_SXn41B93y_!TjaoL8D-u z@usgj$wNbzYY!DG2cchrIaL#QlZJ&q+DbduOQ#O`y*2k=u0`RSC7)?ui3#1mYIDv-T~!>yX|cs z^BtQvR#SfNb@#U|Rg9E&S@K4&DP z543WUVw#@Drj70wr(m*LJa;=~dJ-k2x}mS4Db$ht%-QjS z{NyI9(NdrHRDl>0%PPH&GQV00Rn;{e1*dRf8t`3{ zEWIxPNuFiB!^P}@WGga2Z7Wr#a6{S5;6+AvqMzn~3}z~k%v>nvFHXc0*-%3Hdrk6? z3l?VwfT3(I?mT0&q7kiWpcD0%@FrxPqKIoX$afp<)s`M&as)62%8F~`2VygugZ?xn zs|6eMidYW?UWki*wbN-u{U$Coq`UBwNhStri+mVCj5`?(WyKm?*PRjkv)#>P&{qZ8 z#nJSdys*DGC%Q=pqdS=(p9DH@XtK6gBiVlR9D#NeH9jb`u|7@zd`?D`_N|^vFQKgM zaz@_>I9xzJABWZpe?~MhHSasM3$x9$^OiZ8qy}3l@Wf428dpTyB@eGD&Pdj4))dud zMvUG;BwuB`QUj?m4^eSqlK@3~c$LP3d^cd?14nuS#18POPVzE;4|#9Y%TK)RBTrZJ zj<#i$MTjf`vV3Wn9#Km=dY}(4#OSf!3~Xbw$rKc~^njqzyD4yx{HI+uOSB-VM3yx370i06mamy()trs-Up@rgIid=C8)j9P! zAd{Zl*Mhs;c|mDBMW|t>VV@(rLQiNZc2|9=d$y)zpYorE=(tWG!z4n#7;}PfNg?&E zS3X1UyXhCU2QjR=f;Yhl)cn?Lv_9ql48`B>`RaOE0`J$pztC>x;_y3gDZK?cnBWdc z6#=GVEwTbvIat7`At7uD@n1yn)3|1=>J2mfp2B9z{SeCAY)$!TC)Th)b!SQ6w9=zG zCY8+IaXZl2XjfmsN}-yP+@%V{%?rsF3v&Cd2hY%BD($*CGk0s)4Cmz@9!%P1e6?>PWj_nTW_sLJU=QR9{<$L)`Yjbbt5`d!8der6Qu z`TGN0txn>wsAqz7eY~B^ozTTqNU!%cfmC_;ydhDUt6BRB*e^_9n%j8N7oHZyD<-it zw$neCD7c~EUSb)AX;5&ZlE32g^<4XO`QC9&m!n?n(9bKLip|PXUDR^3J+8}|*z5O~ zM-x+*&3rLWW(TZRwlvH{2M78!$1@`>NRaG?$+dRyP*v`@=@zq_o#AC4D$7z9Ec&y` zde7HIwg!C|gT2S5$(7K&_55$S$teRGO}3$Ur`I_N7QjS!-%xSzs+{}HD4fK<2Zq`< z#q68bisnr=SCi;Kq~y2Wj&_e*64dUamRI^ZFygtqv&gI1a}PzmniPskKhNy-g zv)w^`m`k!=!l3fm((6z*T7o|_CY^dAm}BQeyQdhY&nB#*VZL#$>cdc-kUEhL63~}Rg{Hs8i21MNFsqBw5Vr`%1 zphypH6rrmhzxfPx5cF4p?n(yFBJUf;?^ReVl*kiOKo6i`IiRAEd_l zQX07j3(3dL7KIZ7MNQKcw^Y><{bhYW-znc`Ao6$oyG;9#${H1<0t)7KU?2>MITxr3 z03&OTuF4s!}@D~mid$rnq z&C8Cv4wErtJE|Zk-3WZt&1B+UUq#6ez!?k_CnjQi5 zGy}X~QBaENibi4P0|v4Mks3e(QJaY0vUl}CV<~<;FY%42NP=kOwucwd81+ynVW&rX zOFpKGjFx#2&!B}_lwO8(aJWMdf2wJR!?yZ$=MTH_@7^JbIo2j`RW}2VN`hIK*86pY zGuE8CErpAeCU(`Ne`Qgg+lUvm2N9)S^OrD|7|Gn~jVblvenJ#Fgy)GLJ@@14_5l4F z=Zu^c@|l#^zey(!wVJgOh&3AG==?wL!}ql<1>7g@mvQ%-ys}#$Q%CVrdhZ zAO@w@`YLR9yM4sAPpDbNlVUG;j`T|l1X-;!ip)NmmpdI2+rfXCNtK{I>Ze1;0|?McREbV)^N#%LUL+C{PwboOKWFJVh3?4={TvFrRByA#)#oDkMHrF)T22W5EQ<{eo^bc=5j5gZ^uA=p?62LG~|? zL;A*rn&z%;n`xfk&9H}MI_rNi#EiAJJ=2m+9uD~V`upklKx_HI3w1Q|Y-XmGke+VPw%N|QUe*v3el`nT_uIoP z7yV4Pxvh>~B!GAl8iB_;{nwiAX=FRs@5KF)IZ*Ad`oHQfs%ybkM_btLu07nQ8M}!u zF8hJTTi7Z9rjRuyVvb8>BfliT1Qy+|zE~YX9bKLk$aFy&&*A%WjnD}@KtAU8G&d@k zXF70kMvGC9bU#0_PO{D2k?3s1!J#s*Vm5DRr7PBV*SAV1yNoxme*f%EuUw(6@I1Ep zr*(N)<0okb`2l-ZpMs0JU1n;~jqbes`Blq`Xl#6hUWrD*QI9HjmxHiK*ul)b+WMQs z`ufTwQid-m4+_`J8#GLH(!W^ZfgTNC)tJ^LoZ@h(nao0(5~ri#?3bUN5D}u}p+F$` zg5YD?8*9TE+i>_dnjEbKh#HWY>DHq6k4jclgXPcFB4WndBiTIu%03XlMho^%kPJ=H z-h_iRsTzmbWB@houwehO%S&FGW9oxYeHf4@pgFf{p4FVcu8!0$Sp-s#cYhzueo4xu z;RXv;1`9i`9?t(BQx5~T++GilP1udk@FDQ^y)m(G2hb~GC8co&t5~a4%|e7U8f|v&E1(~4ILbXXq&{NZ~E4F z@v;jll~{_-@eWD-l!a@AujL$ZNQ|H~3(ns5%$LcA@@IM%tM+-gQ%C|DBkyX$wC&2# z3<|#IYZP>3_Kf>ZVe9j#Lq}jkDk{;+C4eeA$cF`H(L?)uhfl6X=A_@y*{V1Kb?!1}O?0r|Zx_i++`p39?oDl=e#P4W#0InnQz5Uih zTXa~KHwQ>#3t(m_Em0O#I_b7vP?vV!_`xNLbIKm(dHZJAe}FLY;LCDbbF%V^!P3-> zb=6bpgAp-=9~HSGH%lp#z7)Pw5-FBO%y*BBU_XOWCZ8WZtsj4PpIf#@ghaB0`Y(Lv z4iF47QnJtP%DA3mjjRRhtdsN#0qsmU+1L#jBg_Z{lF*}FGq~0H9!y!%4gHF|qXN)j z^+knN0~-!NQV_cNX&Ay71%xweUf{U0J~GXeONk(KUE6Q}5eEbt#>b4j0XhC8Dj;Op zPZMB?N0xy87z_|1dRf4}58G&&Qf$8r_-f%XRY%JGXZ~r~PF>}O##5#wS9(W}^WlV+ zD+o*fl+k3uKVy+ zinOx#v8QN$nB-yJcO=OIuA^#p%Vg0!P#9^;aUtoP8S;CvakgrCe2Ut$`cBN{(fQ?) zexibuX_FG%lbVH?16~gn3~5PCv{ZaTq{+7EsqUcnr~IMn?g8De7XGZffpmYT7 z%aA~+z)k=z&nTYy2}TwZZV}fs7GE?h7^T_OjD1X;`T(zPtPreQ{yKL>>^1ufO4>P|0sq!vMQSx_xzLP&<--NMr3t|w z_c(@I$9Hxx(0j?~6U*G8gG`37DMoVPj-1_+$7m#f(IRCEaU7(9<^r2DK4?VKKLYC!WG zy>60n^_1jWQ_E-Ycg9knhS@RAtu)@VXLI z$g+pJwe@H;C>cE!$stgFJFFbVMRtR`B6Ky;Hv@z`QUY6YBMV!s1FW*`gahx@Kyn4 z+lAMgv0>`w>?V1vDOiSZ3Q0^(y3Y#iuCmHH(y&C_tL^ixiDmZUf~YVSZWdmRg0vWNzz1K4}U z?b+p&f6Q{m#_9YDw!%y#-#B5V3YQM0#Z=jmN*d0Ufnnqc$*^VdZ!jbDtQAc|7(7zDt!ZtKG zQ#$k65*GX%i*ETR*a;-Soi9z%3g+MWC0zATHtlHyp$TzIF2GKt=vC(){OP8uO_~F# z<wMW|n`Yc}PYxHS?GEg7PLDbDq_B0apB2`5eQ;jvyu z%)ocbs+;HQD8X5_DE?DPK*|ct*wy8Jxji7GNkKe-Ho-YhUC*hHr0_dEWIQo>jLNh} zGO0ntT#Xh)pz{%J+%gK^Othi;IcmMC2h6h(xK|%NqI^?=p)a6?aPqvHwCAXZhT-|!fE*r!390io z@;n{&Wx?DiY14fPr*Y|98x=e{^ca%g-wcE(uKG;Q91L`5^M$ner50Ko-8m#q1Ml8T zSM1BkaV=zsHZXllS^sJaKKBo;L46Pq2 zu$O;sMmJED&kHi+cR_@FIV!I?E(A@A35XtdUB2N0`j3QsPx*bedRC=zLY=~*B|Z~c z!Y3+KQ#NgzX@;405I8uA5_0Sl64f8VwcqQgi>=1mG0GV7s)3)qeD{H=iEPqga?8(U zX*tshT;G3ihQFVQc<#Tslsw4&*b9veJ^q+xOwv@X%jd4<3Cy6^nMtIO4X&`6D_=#@ zt|jH@1ztFk%;IYPYH)02yGjtYn;7+51bAsPQF`Mg<*Nx#H(2qHx!?F!eTf8AMU4Sl z#-L;)U9am{Gd}BQEyZGS_)%T|WyaBcmeayDfX_V_27p=so)$zqYRs4TPx}bizz( zJS^QSYo#~kq%19(wkOHJ_Nt)e6|0S%Fdq=NaRJ}b4=m9|Pw zck7lXU<*;G;kFU=j)z}Y*bVjAnXP%(`31(^E!{M3gg^@cbb zZSWXS+zLQlyPqkJ(ZuQppe`eMkqrvQ(U->fufZrBaX7{kcMN*8pHyw#!-*5xgdHRz zk@2fDH|!G!<;y-#PE>(V@f!0XRg7$hG2wiL&T$|(OflRBcF z-AgUBy~iyA^o2X*@rB`gkRUr*Z?rEO?R;~B#$zJb7AAiRh_~A7tU&R5$zIKROn)L8 zxL087+A|&Q8(I*TJEV45=$!Z-Mh=Pw$|UDv{;D84cPakYj{5biQ3STO`FSYcmd?iH zwX6`rX!<>+9U333AP15O``9hJI63LM_L&`cI<6#2FP;AwZghT7?C^NsSgK=CIQbie7r*y@X@tLHlc~THU9mYy;Vc1Q&vx0ZVu@UT_zO8ykWULnF2;TR-!3gb^|G7^f{UG;)Q%@FrUB$ zxfZ=C7A)aaMMK90k+9J(cVtD~cI=0#o$1&az36^tO>|D)fLEkRUDg-UMMC_ni|H%1 zz2ST;eHuT;B5;$BJxGvWMCn~xH#LW&zVPV1#bzYmSKC3FlZva+hSH@!dNpQWfd!7G z8NEk0lFBb1dUFC_Y-BE|m;8*rVkFPnPZ^=y$@O&IZ4w^L5TJnP*l)q0ST`cMrQ+Sb zi9MymPG-X%@_LEc74?IcR;6cvEKD7W?il9a0?c0bm!pIASI3lZMPDdIe*-LK>?Mk2 z53VMBz&Kd>Mwh|2I;*b7^hTXH38yh)25BUphjAR#^{~mFSGwivkVI>@`g#y0|m(z>?3E`ox?qR@^NK_AARhmHhcb(z&7h zp}R`KeN1MXO!x$R{acvQM#C|75j1#t$$svVt4JVWik%>WM!X~j^Bn9w!kUs)%$t!b zz%oR(mh8+nn!sq|_cg+qF~2M!qIq<`^ADyP8b;s%O_*D_T?WYKB$GbuGxKZ4VyoI5 z6}>uv(!Jy|q6tlp-?U<3JbE3S7^3l{L>lOF(-U)9{_o{bcx5m7GoyA9nn;Y#qa53B zSaX9Ki^F)c4EPz6IoP9L&VA5JDC!^zbJXwU;YpL^l>|^_CSwzbyeTxCFmo-*fS$9o z#Q>L0>O}$y_q)=Rnb^2K0v8UxEeSw=BZpHo4vCwje99;z)GM_39G4`!8-%~vE|)pW zG(RCwwE7)6#FV-xaqyz(vdM;D`qg4(q4qa&kUo3zUgD1Ox$D$gfJOD*=**6;@bWqv zVpW$cP~0jGsPbO9xQ?A%zD2!YQp!5S))fq`G*jXGTvf|c+gxR*Xf^)Y7nM;W19%yR zl{}c;RL>ANnV)iGQ<0kbrQb7X9DetpV3qq~t;^*iR+2B++!^=dWCRH#x0kx&^7aMPF51_=tOW{nK!yb z2DFvVA&gXom^9~M zG$32JF2A);q8P#_s@3-#W%m2F%mR~@EI2hMEW;Z3~sUxC01Vt zN6l752;OF2LD=l0n%_Hvgo|BK| zrpwQHvm*F?q(w2*P{7&Aoolxp=nl+MGGE^N2*sGLhV&)UpeU zgUD1A`~EVwvM2n;;jKzMO3l1AXo|O(FSFmQLk+K2p_Cb_Mswm zw9`Sf+nL6=6?LJc3XJMi=KVb>LYs)!z$Eg{L3HZ9t@HO6nfh$8msajo$H!9mOt&pB z`y%ZL<8;H)_@-)M{oo0eZ(DG^`M97PDr9QPsl5j>2JP^2PZNT?C?I-x2E%2eT? z+FVsPxFQ!4)XADqi-~KF*l6T6Z=f%C#0g6a_Z8T4Uy)Q|m0mu~uX~LTs(|fF{AY=z z^)|%bSHkCVS+z#=F5ZI}U(pU6v%$Kj!k|1%R~;NM+xEG&A&SIoOwhi-z5t`yH?wc} zCc>bn*Kp>V7MjD;>*#x%U;*;eV;CfxE4+LI6W+S-itQ0B$A6v`VQQi=qx_`;GE9D} zSJGXKB+oJVk@lC=Y!rn>dS75G!xymd0agH@UuzqgPz)M(n6-mW5 zbGG8q{mIVq<3~VZy6O}bbB{;iaJCEm`7{jeQnIY*@OYW-<1-eOUN=>X)|s`3Rw4~O z)^5}4`?ZEOb?Oe{NZI6**#)ydR1NJyh9bESrXSzVf8w+hLqr@5uH<&xfqTW0N0f6! z7C+{e`JL<91y@7+Se8FBPcD;}ebu~|%;4tLeca*Pf>oLFcoMCXx@1p|F;k8xJXh?E%w`X1pv|>h?b`Gzj+qc;P}zb_9ujrMko=`ZM7bwLm??hH9%R`LTB0o||XQ&x#zv z-1%?V7PY&-n$u{)T*^o^V9O^g#Y5NZ+K|wOjB8pI+tL= zf=zK*ZhyD|3MCWOEo%2CcNEX%>a~;QT=L7N=sdPx41vg)CTWdXMvDU5)~`&gk25n0 zynXXNuk8Oe0#v|hK&{)Ud0RmHMK{Ar2bGlJ0)`nzvN1^}vv#AEjL3J42|uBUp931e z$(ae3=H_XV5beu1TppWhfJCnE*MiddicNcki91`|pNl1|UDAQ~7QF*D3TnUA%1CO! zy)1n{5L+)mCyJ1A#arrr*zpx;-69#Z^x>rBPAq8-Qhdp|qu2!eJikAY=8+lI;$reH zS1x9Ay%PmyvsgatiJy^k)OrHw4Oorzo-8!Z~x&%CSwMPAWHf@j&xX8^=3x;3D`_cm~Nko+~dw5#6gn5%4Ss8+X*7 z>^bN{2e)RKID%W`o;9Nv3T_h`C5Tc&WhThtrzvmD<+H0eFt#Rf56ZUP;6qGaG z@&y{8Cfm&B7w97j>0au7y4e>MvB$O8KLCra&G+Ps1UvP`IPiXVFdJ%}&>;3n#k6;O zT6=yuaSV83dL>mP(y*LoiwUnmsoHo5v1I0-9ll?nEm$z|r_~qSgeeBt1PWcDz>YCX zulSQz0otme>^4bV<%z3aQL3TGwySsOAjSb-r>7l4Ts8E++c5sO{R2Y`AigbnIkKG} z$MUU*-7dYtR9_Q+(q50HzFKUESrOh^PW5$~#n~;) z-FgYf2->b#{c)1v=;zJe5cMW}nKb^wZRJBvG@OkVV{MW&UM!Q^kSrGr@lyGz*!N1w z)Dbml;qkRSanLpn$w%$3wKFog(Ap8`L(~9*3RQABH9qe*MI)L-wo~JWi)bt0v+!Fo zm%3v!^t8g(>rKm9Tjbp@Z{Q;(+_~P&dbNy2(d1+s%+HaoXKgSPPy!pLFJ&CPfyik)~awx#jV zZpZhY(0lD1QGEtX<# zcQGjLu8~})XTHLWr4%!t;JtbeRO}>5kg@#ooJ*m4IzE6p_0o%oqV&&677-gq^IA(4 z>C1sQx+F+ooo*Wh*F1Urs?*k2x4MsQL8_b;&irk*9usVERo~2xFXdC2Bt~)SKg2Dk zZlq7Jsg4|LSZHJBpi(DHTWk zUCpoH>vsz;$r?$k#+-;G?3c6ll$$hs23(RnhqLm1d7!8*Ob9TY-&|)fZ0ltGYN8}? zL$bjjvtls@B|dBI5BKXgD%_8&>L&?&h#BrEzs9XF91Pf!c zj6T}OSc7d_KfsBDvUJkOy?fg7zdcefndauO{lv05F-*MBAudAGil)L~?x? z#eDjliSz6Oj&Ec0&w&k;$;xhfs(OkVZeg6R2!4G$Ek0RHz2A6BGYQ+PL#Zm4@z-?( zBYRM3b}HF%@i!!m4p5JNRxA{gC#=_0%Yt$yva^DUA`@_Gb4RcR2vF5@#9k5!e0$dv zUi!84*V@X-ouDbk=Bv?!`v-ysu!|>1KzLe{l(lU*e^>Xot z_LlWKe-B7nP}KVkp3qV*=Et9$BwS{ZV0oo;uiRex;lvLBN>t;Ua1f=}{@arNKY0<( zuwU5^ppdCm5qUE0J{B-A&yhW+(+vFU2*Rk6FvTRR(p3w_m>UIcyFB^bdv@@FV+8ik zOPQsH{q|YrOK^q!UzabL;2%^XY>4cUb$5Y$ygu(7ho!1m=n=G?U|g5=ay}6MG7b70 zWtMH<^>UVOCF8smPqOGSif9cKXik^s`T?O2>JQB?pSIMm`WBLj<=*zQ@f3N77$^XD+MD9gbvS&b|z@Jv3aH|RGWuq?;TuQN^_mL zzVgifejDqyBcV$po+Vx3Ur_vqi8X5REJ{zE^gk=F{{aD$`u~@rot?~reR_fAtJ#%k zFWL6>tl3EQ8TOocFnFo=#?K9uS-oC3^Vbsa|NR2BZz_I#djfe9g(vmCUs~>d>OdKv zda-4-iuev^On;{#N_WnnU$)TrTU+J44wh{W_q->v^nfR-y5u^44(cN@u2Biy!{peXmnA#kT)`-RRQh-Q5AJ&6$JM zv_scqs4+^^KD-1DG0&El7c&~fL=kFa|<6l@%g^<{5{KT5J;~E3fnc4GV0ugmT8>cLqEf-XYrS{ z1&txoD}N!V8(1)oiG5&=wA z8ZyWjIea)ODJlL0mo%iwOw#h`>3f>rH~hcgr_JGW6OI&!<5K^v13kXXkLMzwOIhiE zvQqsSUJFS_4lkLWO2o5+b;oYKH9iIiRa8v=?I88HcFYkoZEc7|^{2eXfjGKnlIgMW06U8clxKOK^W0-$R05po#r?rfFIV`WYH zZVEy=D~q8INs)u*9pp z8a(MiUpsJQw8LsV6_O!p5F&@sBXfh+{3i`E75cVZ;`hW3Oq0NldrzTsPI%)#p_x=# zgW)y1cNT*09(QH1uvH6O{08GF2K-;0WPG7qXE~483(Ly(vJ&ZV61`;{wEI<>+$@NSk z8+uuChw#nrCq=DQMiEy}Lrl3tmZCEs0U-Z=SpZ1?>7DH_C4T;&r1pgDf0Fc5NdHOy zjqqQk{}leK_}>WsFS9=>j2bI9Btj45HJ;?Zdo17mV1n1PC?!KaBW}MhB}08%CK5j< zGxVJ(>h&3Y{)_LRBBHzEJb-N(J9K9%70-@mP9O@+z=myE`vLmZ@r2J)t=03^=7(}m z@51&ow13Y311?qO56Z^DvZYMb1< zV6fBXENH8j-mtS-qEEChpH#j--pqLR2)%>Jdz#v|{&h`_C^O;mJc{)2pVPgTA#2cZ+W@fr~XIr_c9zT^6CAnvq+WsrX`1} zf2Bo~S&QWv3!;mCS_-TOiez7O^C7z?x3(jvzthqQOA?1-W z7LIp~Pmc>6^j=-iX-zC1ATsl|E21K!GHIL2%^`~mSkhybhjjwLj7#qxqcgciu-$G36(DM=2BEuyT z?~M>9Op}q6#;`KyEr~xeTly((54f@22C2?h2DRYRB@>pqH7(ldVuOGxy$z=>*OI^X z!iarhfqXY|_t)Ublfel25kJ^r;!b{nzg2!# zmk`Hx@2e!R90ggO3-t0fFuyFoIXnsK{h=T4b$k3x#WO*>o4s^wB{22~`h*IXC6Rt8 zHl+!3j-97`g(Mn-vT?Io(#<<8rpb1P@nTVCiir4vfP&LR>TcH^3wuAa;5~FQ87sNi zStI^0DsD4oIr#3YPmXZ4XpFrm@lpX8VT8J~zv_A4HpiseV#`AxdAMOn!wTBn8Xoc! z+B%YNExZVI^6`>39)TSkg7Z#+-#b%%++=kFd-JZ3fBP|d%P4-8}w314sgl3 zj%a5qCcg4|?=w%xnD%$cw?_Sd($wOIUeRTZ&ICdcUQ2yW{y*A=GR%>Qimvj8dupHozB z7)--wl!=L2?^8oNkOiQ@RmNzA@7O|(3(9$jTHhF~r`k2k@TKG}>E&2R3OOC8x~rly znN5is{Q%abC8_;2MU7d3QqvTEVOAcz;u#GhR6pWRaXJS+1@>g5M(}xh(Z=0M(t!dh z-yDO~W&4X+lXP!zXxkj758^}i04=~w-%_ykacjj#X|Al1u#TnSSz+MgMAI`~+Cx7C z0{y@+vbHDjg_{sa`w#z%nRDG?{LBjal$Hsa^osHT zVl00R^Y32?*&{B95~U1;dr2z#he{T%yW_f+#aBO7)Kh5(z%bHf)5v3=ZVt4YUKwwb z$SqmFTTpYSTTH|i;7DDJ*9dd}4R_uG*3w*$Q6W!4b5T9wwm=PCHkUmCOG0^yMYL|u zV{iLcX>fOF@{`$*=VaP*=Z@Bz)Cy6`OukgE2xma;wZf3o-8a22!=1AyU$kHdfOUY{ zPxXYSeH!3?Sk(KO6|=Ld^BMdu2+5}==@qskWAKc)BZJN>g1HbqciL?d5`!GZ9B+Jr zUBFoQG&ZUj>MPmG8}ye^^-0fy2IgoV$U2VZXe3H8_`aQrbjH3b_!I_XJpQy!;kl=<~{)yskhvN@$8v`n1wN`fU(iDgH^z6H&FR-FA zx7|PB@*xNB{hgANpQ4tJMwXx~AhQc%q}Sj9bUi<4S6tRs^SD6Df&x}cerUXw?9YQwb^8}MSnFjce+nE)JL8C+jP_RUzk~X$V3+|Oh z#8oQF81K!yJXJuFj6cG&N;A1MNsp2!De|j|CZA$_-q9wwx1hnf3qw_Jqp-wN4Pm;P zgorBz{9$9uXV*%?Wh>#UEVBqzW@V1-%TKF%*-4q}_v5@TzwB=FF=^OCl!tt$-0m&S zG_!NZEiMu0$Upm){DOi2KJktl%_M#jvgT%vvdNG}#1Kj9z^ayx!oB-|si^t3h!s%* z`PP?}@gr+{m3wa5^*_oTBX)2kW`iGoLMc=gq{pQ_w50agZK2O;J14vS-5S+v4HQwX zz(~xIM5R5TY9^UO1!PUMzXuvoiuQ*EbtsUImdO|1xlSWHHahJmHH2vmTOcj(ZT{Y4 z#ne%1s#wgz`u8EjaMcvK3Z)+v35gS<=iSzd!8kXeFV>fZIMmOGxbuTaF)Sn`3?%AX r^j4vQ<5e_@q8%d6do&30tM{Xh>M9%?3QzxMfg~rTELkD`Dd>LyaA|;v literal 16637 zcmdVCbyV9=*Dp#-kpe}F6)5h(f)puGytqryBEemYQ;K_$;BLhsI20(fNE6(pxVyWZ z@Oz&3dEayHx$FLYSJwKjkv-c!d(TY1GfB9LlFTcNHyB7rNUvl+N~$3tA%l^Sp3|Zu zBUNGb{%Qjd?1+Nr(g=jUwx%?AeuG@kW)D)vB$^l&NGBA zi@~+ykB`S`yqG*ZCKVNxJUr^IB0xz=$;acLg9vt)-H#jJcOM_Qc|`Pq+ydO(%8A{a z!uHoNo5{&ZwWRLl+sC(b*2ipgLZNNuH29Yq%x5C>OH0cKzwFs1W!1nEnl$2B}9`3ohna9KW-8utaqq>XpJ?Q4+ zK%k@&AJ60EYX0gYmqO&#!J~S?>OQS1=}WA!_2UmUM<=_(dn^|0;ZrE+fI3+`EWVVz zu2K$3UlUQ&eF|W%I#-_!V1bqRm|G|JFn?W~hg*r5Ks;dda-{rWA^pZr z>g4C~>SQM&lh<54kmD8Vz{-(L;o!c2_rd6%S>1+U)~b!2&G6|-YAt-@Z^;jP-pb{( z)19^R^LaEp6)6_EoXL}Ylg^Z^x_y<(tn~c5RJr<-$IY$n$D`2$_aOk0>P5}Ru*s{t z*~ooLiT;_z*~;Y5pf7*Fn|}9Hn2b}3Uwrt|chKD2EbI0if}3=0&o_5+^PC(ElrY#Q zXUT@HrgqLldu9%!XUr4@PKsfZou!`*C4ks``_hRwv$I#lf$x=VTWFcj22-XSeHy|* zs=N|-G!nnMy3+paHOJTmeD94i6>%H9%6lEy``wE~rtm~VHO*u!%E_=SBXzSQ$SfRM zF;rOu=I!qfwaPwOd(@Rmqp z_t0*kFG{LviR=|epM16-*GGTN|J?f3)@mI6bG-*z@X;dLAttuI>%2PaZ>>c|k4{Y5 z!W5Rq1R^nnn`9-$G(OMnWhhz_h`kQ@#iW`ia)+!M_CieMEa!Jc@c{lA6@E7hUkXo7 z*|YGEN?L6s#DvbcSklp*RDod;Z&q8|0#J(OkqGNJVb8;;J2n_hOlj zdWsIwo}fSjM_-ZYgNpvV){qr5Cb6ZGFW9&Y;s&c2JM`Hc&c9aDIkaQh#J_zCuQeD$E0{zgA zjNoyuD80hmYgBFgDBuGsp5~8_Ix_>wOJZIK&*<@0U($l4bsYaf2eF5@@!o9*Km=h% z&pWpkYs*bH7i(-y$=A|};6yi{n==5k8BhS}7s2z}_Z=N?e@EaB%KW4R8*l-xGs>Tz zEZIM3d1)po=VW&738E~;C38J6+n&0*6~KOUYY3t?4bFad@~z$K+u3v?@nc9F`Lub9 zAs1D~^_{J=GU}lbssBeM9f^>>=GtWy_lh0ooR4x&WJ71AuMY^yVDW0pMDjU{HA}X+^C@uL} z>=(wLzWqMFAwl|I<&fZW;7B!qWP&$~5TmdLWei$vud*Y@kO4bbAT^P_U2I7i@!B1m3wVeU8)xd@@VnPeJ-X z>RqJ$8WmU+ebYQjpPL=z+w(c73N9IT)d}s?%b4i`-V^#O7&9i`HM`BG-c9xFBteXYs0w#G&(~_+-jq)) zgKx;Vd#cZMde^V)T(U83&LuxXj$QWRJg^|)YQ1&+Jm|89osrMR+k?UUlNsP15_{Mh z<{uU+x_md%k)YF59}qY9HUH_@*IC+rj&XzSsO@VXpP}(M+>gmps628iFX%g6Xzr@I zpz3P5g<#9i&*5X>NbvdJJ_(#T7Q!ff3wd}oMSYlnR6YVzEjSUR>pC~YEYXzC*3nB_ zmyNm2==V%)w9^ki{A8!0((mIftz{47%A+`gmEOT3=$v)OWWHrt_QXkYcWbZBw;YDf z-++Y^IL-$?FmfjNSBl06y#7nbuWJ6IN0?)b`+D<+%d?r#TAE>T_PP0qH<7mrn511R zseY-}OF3GMPY-qZ3%59MmV?&T@ggXRzqbj0;3S6)xrtrZ5u+fmW;s&mr_UoP=cch4 za_wTM)Y~{g)Az72IRlQI%UvSh-f5<2%kX5<)hN!7UsdZ~eptwG80Rl>kWL0Yc%tMq z4(03WF+E#~J40O?s-ku5Lxj>A6D0Lp)Sw?Owu!E}@)WL5g8*k#3!P4@L~-y-p@&Sc zg0H{;+)vIAeO0zq0)l5+`0BAkp-ngV`t9UF95{%=&X$xt-~Q0t4oCP~Ky`jJ8yVy9 zm>arKA9T@Q?XRSl8u_t$AJA)unQQQhl8rVjTO5M-yXh-{79G=U=H12ibciu7MgWs| z$A`RQ^gVI-!zo=oTOm5ir?z6f-?Uw$7xa5BHaJ*m8#Pau*iF4plg~fyN_vpa zr5hDA)G3wu8<>mwIj6$;qu@)w7Oq2h@Oqr?sIem+$mK>OKfEoE!Od`DC?dYvfg%|)ZbF)13y)koY3U`-^Huoja7kv52PsJt}lTh5kNm1em= zuy=7!Z<+!A*+jnCJMheEn+?6aZ})JN#f7Ws^n|p&wJX>@8IlP8%mQ~kf`;Y`C9o0n z&_W|#QmmYzOhgpgj25F`>Q zgSdbm>N=CS;RfmDH8=?X^2vfz6EAJ>Iqd02V-E;^F=0 zp#qKAAUSRwh+`f6T!DAPKrb=22)nPuHC(? zDO$R^%wq33nBk|-FHTho=&QhpT%-=?!fxOt!1 z4Dxoy{;{;g{xP*UmRX4z^s4pSn9ajH@Mrg}J8YCw={seWm_5~rVT{#Uo3TIF^B}ye zOH^ZA&&8Jl^#RBDwnrhJ{*)m%iYvaJB=af|unHHpC-QZQz}nkwShdNE>0Z-smYkEx zb`ojr0Do)vup8EP=m&Q5pgK10gSlXhAbs?qGnuH6RrI7xATx0@bk^fj_DjP3c>}ZQ z=#kz-^kE%T9Zfjr>In+h>KU8cM{1JmNhzJ}pFf&UUKVp)dgaF^Y*vLD+n!wlFn%eC z0e*OPJE%j#_iI(TdeUo$qgbRhD8yfMDO7^R_1F}5V-wrFl9}jnY^S?3a>4AJ9)p)R zfobguYo2d;zKz97b~AlEca*OA6Q!arZG%n;OBPLkfww_6=>!Ww9jUY(mE8{qQd(NtjV@5Ycyl{@_BD^Gyrx`ijqo}}$4OwM ze7lQt^&RgOBd@bm9)+BID(>pa)VU)QoN#pvRz@g{S;KcvLV14H@xq&d{`%yzLJ7$; za^;h7a1g*Os?eR@^v?NB`7C}JN%Rk9d(v4|)p919na&??GD=OxbsI9CS5E?aUMhmz zSl~X9#=aurfqH?>qX5;S7P4DWjRLMV3wtS~ zJ6ZD*5J#-DO1MhKZcl2fCMP&aq0w$;PDtz76~Wp2(d)wW6(ib3+t+$tlK8t5%5HjZ ztCkHESL@zyyDtqEY)w!pJ8>11PM*zB{4T{`U*}fxH;nNX$l8MI$j0z${p4j`BD&|J zzb83P1J!f1qWUj%M=rc@Q(IOHCFzPmA{tCheUK;BjQS1iBL6DREp@vWYM^-H*`XLn z@>ar}vP({Fk+D&)CIoBnMsGmwQ-`F+W?P2R!V>Pfj0`z`ze^MuHb&C$i}ECj{sI#^ z2;K(C>8924Ylv#ClGrn;6zw}1h+>G2bczHn{5!y@CTHnOblO{!nLE=gkomr#vZ#J= z0)R~A>upA|bIf9@yz2cb5nP9Ty-w1FS*8HYOo(RoTx3naDby)NBX^^bii9;Xt(fHi@vZP1@#SIvYBS&1w}OZ?wwZ zX~7}-U-DYu{+U z#@*>s6@C-6+`mhM0bq_FD$^{;n}6DVIczC=<`81&TYCI%m!~Q)?tNL;Z<}&B$Hels z-B@i8uAAta`j?=#2gJ90FL$NzH#r&w?o(-JPwW;70Zaf{NJ@|o6HdtM&JJNNJ7 zKO>RM1DpSHh0N#2Z@WfOX&dSHUQJF}ys{ROMuvQYZE!^Uor@wh!j+LS{8DggLGFjf z(-*|>pof(6p)+J#+8YRk;IsRj?1eX@TwOahg&@FXJ2ggKBi#!&Zc2vQy$^q!i^r@c z>apI&-;-plsl)3iOg%EdE6(k2izp+M2r{(<@3+vI#b3@zttSsH_{94+p1xNQt@QAy zizXTd*5*2ZQUS8gq{zR)HHC8K`Yq@uRj!;wm;65>7O{|!`K zRyn7H@93}OJG}_2^XxWwp1C%~jDk}qD2;TfUPW?+O4c?Q$3&#qqiZhc%C_p+P#C;^ zEjF;hzQNB9HrMYK5Qazf3KtEbXI1CjLSJi$hmQqNK4-VwIfoql$njHunbl3@@& z5;BU5Vr&LuGm;5oOM;3`e*#G;UN-`H%!`+Jg7EY|Ps3h+_!9W1kEk08Di+4v#w&88 zH>MA>XkP(e$ItwWO&!)B-Y7l3BsNyPt2yZ1o6mJoE?z>2 z#&rYoUj1tXnQq7$3A3QIh*WRNIJ7hu!nGb;wX_g8>qz=ZU7h?@udoT5gmd&7M1UUR zCYg^7Ny<2e>}MN~>7rIp?vs>_$KVl(Ob&ciuEq}7;Hqenc~T)>55g!9kcvND$xfd5 zY>O$^wbNei&a0!v)7(;FNucAN#H?LRNtHW#xo*Uq_S-$Cv4lA$WcBR`kZy{K83GRX z5cHH~kQnm&%+AyqRm=uy!SQB;OrU6{XS0l57Dj#o<|&#G$ z(Q@X#0*v-YmUrPP3P@}cK;+XO*MSh~5HUE`uRH~G;OjapF(`I*6csu!;}e?%?yG4o znvak#5bBT5ElC5-C=d_207Px_|7^N9Is9v?7qPQ=b47CaG z&(6D7c)AHJn(fbe9~^o#WDE@obS%ts5NpGjI~JIDr)A zT}nyJ8Se!XoVCEP1_RG)@$+4;;}ckeN=C;U)MCf}2Q*ciO-X}fb^c~?hQe}KqJeM8 zPKu!+*y+KYB%|y6aH2wQ<*?Y%SyG`O8VJ@lbd7@&YwKimP#{L%dn;~#@)jUYGP)Tu08cPiB%oDZ9PG?k}Lu=sC#rt$AOBoe`QG~nK# zYNAk_AT}G?zu;8O>BBq)4=j)42Nh}(6V34Hw}<&RFHLfDNwt9g%S?nyHb1@?Qb|J9GP1Wg^R`W?WwOKPghif5{LK{*t*HJHw;$~(1;&RV)Ih$q zI3V~zW`l%P6~d-YG(#f8^fR~mg6{gOa~wvcNTGOr+9PMe+~IoJEhFA20x zs!RNRQrAQsRhZZ}R6kg&ZotmEK~oKO#LG4;ZZ%ic{jA>gLpX3L@o^S#$uPVJ<2)R# zbf}!vnHI=^lHkwx)|g|ow40b26eW-q0@k+VI!6K-V3M048YRV&)H${d z>TuvBvFw4~rb`4EMpP;O?lrfV)Xm(xH=~iyra4q0MCdQl3Z5MlIPN8D=SCGyA{gbS zk>=k}_dct9u?b7`92Zdu*^IWt2$?IyvJ`bQPqUGV%xLeC#Z1nCelFGJQd{tUVPN#< zTl)BkTRHmYlzbaHe%o%Hj?>k0Emmn{3LDQFCR1YqBQEGp@6NTg6?Q;ZBA3IgTmE<* zw;eP0NA^ZBn2t>_A+6>fA#H)NeWT0qn}^JDjyMm-`$9Gd)31x>KLzQtsm z0d=>IDn@f})6>l{udFJgh@FomXBhuAm}wkSDTUO_`c<90-0yu+H*kYQJ~RjQuuUkt zofqmv9W(5qGfh`Y`5L?4486i27~^*Z$M49*4|dkm>F(w?N1juh?aCi-fvIn1k>bkR zoXSkDKOzB3acnA7?gLgdM5#91%jV)mRss$Dw9(I`GwJXyyAJD=3 zwf&p9j3w`B5%1r!p9wnR@^?IpoS1x{qxu>3@h$eFEQ zGx#din*}aU{>phlX!Hz=1aXL+~^^?>iya_M~-GRwlSiH|-dfVNG(GLkaZ< zjY_}rWEfQhZ%^IS7b5Kt!nRJABI}Nga~Fl^To-;Y-i=3P)$t|RU*}jfDW-}!r6@tQ z>1M>g>o{X2YzQt*sdzApli4j5HuUusX~kOPE}$-+BWVz40#`9J`bwi-iQ=uTq_#zj z%YJXx{dA-8!1sE{D+6iTC2!!s<6q`UZFw|awmMB-G0b4j(6}V?;v}D%w*=*wl#Kk` z))JeJu)M?{yEZ!p)tL02!AVe}1yPUA_Q^Aw_lsh|}&XXjCT z6CCv&LPo(alh(CO=iuYukM}2_Gv!}wW_-oay%=30ugF3n7S=h{6jhLWMfq8nDfLPW z?1JgXOudJ8aV|ghlBlE-3U#D^fI%R3h1!de8*wR&3JG1xu+fOuWn5nk5bwvQ8PX$i zlnP%H)-0LE1&~^YHmno{hJEcTe69Nza|Ii$&&1f@l))SJw(72Q#QG!65JSXP&=Q0#QRd8NUU}&s1g}ic%m* z==5jyUgQHFxbtaZs6*M{H}Q9kTZOX#JVLFSS4= zjpF^;`%Kf<)#BsF2CB3kdDS1WnSn=~kNc;A2^4caiNezzrU^2j3!Vk#q?g1Vw_*{Z z1-?qWBG+1Lu;+ue;2EqMXPH1*XR~1n>xdVxil90>44gDYyG_1YJu-tA`9D|H_4P$9 zeuu$qjV33BO`@oQ@}wK_4$WSES-DvI6aZ)N7xM-v35|D}qN@4G1SW5b0Z4@ujVJ(W z8koP=gGULT$&AuvXn*}85_GqUQVCePVzt2?P5gud<{NQASAK@CZDyG$Be}oo#PQYJ zRZ>v5_N3R8PjS5F#W5Zi+Q|jhe(z7|ds0=iIw8)F2^T5w6w`dvaMg%ATM3nLbOY{Jd>D5r(5jpKwUA zWC~D2$KACt9STh+pv_72rwwxEu|(B}SFB3B53|fVo~=f>gZt~S=Qw(x_&Cvi0CL_2 z)31D;A`}y1s;tV9qJfLBZwGov!osr3%1L00-ybbR%Wrc_7rFz(u8!774Snzzh#LeF zRZSjXoO+Zx=Iox{l4c|kwz?ocyZtrThWrsMXIZU5`1CbZW8kF2j&h=JjN9#n+*F_5hZNmnzApnth!NQW(z(OtF#?_A!_2f0JjBue-w?d z{tM$&_S!Bo>|Ulry-QpMmquOi(;yqdiAunZ3e~CW#S3sh&`;=PB&S5Gqc^uUpHX1; zeONB5MGRh&sz=i$OXMY8orZFeZosi-(K4ZznicYz#!K-usU!jJYj@t(&F$lUo7RW{ zHTPqF1O#^$CuM^QBD$Qn^W{Y!(Ip%DHDxsjBG~bK?o@*H}Y+NB~ zzgQKGk|)im?~sGrO!}oXUHqbwY>v9Kekqunt#w!we!eqF8zC7iEWrDe}NtuF=xPQax;J(O0`uUrYM!-=}lG2K$%S=hv^V z{TDoXuEzkZ>JU=$n+$*;XmgCo z;jjo9@c4_c``mt~bapYcE40bZ%lBzV-}O;{3o>BX^2}La^3Z~dvrcz=zfjFqJGOwp zQZ9DSK7+NCM*cY06%pttD7_2C(tplO`K%g@>TR34LQ3&`4ELdnK)Z#1$YW2_=`mYZ zOZv~7@~&E&iS$pR*|ArLZT%Ip>UWBU97W4<(B&@jNiO<(9ggvZfZZP|>PI&? z851c?$dWUn%S(4DeVP2;5=5vgpZw|R8g7Q6V=L~rZXkZbFTT$EZb<;z*_yxQT)vZy zvnFEa44xW|wnn~Wg29jy2FOoQ{fO{6@4s9uS$}dVx#lZZ#x*~4ZLK1QQfp9T*8x%^ z__uL$VuLNwv{KE8 z=E;S&`~q<~#W);f@y$>VR}gPT`_c;194}yD-Ust%aT!bL#pa? zLV0_R^GayimKTm%vp!&GIt3Jvy{*+HIpI7@nu*y2UJUM3Gw|IAk)tROmG}WIooZGs zt;u3Y@)727&6U@RSFzc&Al)rERgkx_Qs83izS)H;-p&x#kc75FqZB8+j2Yxp@Xmj) zvs*$az`G@O&l%CXR2Art8XKdW*1c+iAOckmDXhf{&6oinx45R~ZNJ0N$U}(#09QN! zV>XdEj(s`islTbE(BMG5BBZb+LK`EuD8pmYbnRczxHLU$h*Wwl)(Es*vG=pjmsuYX z33!)hC>k_19{uCnF(QOnbu67^Mkp}-f~_) z%8#~Q%&3{OZ)q30eu}-=%jjMEU5rfo0hi)%ZMVLAmTIDaDU>d&bUwH7CK!bJu=6=v zXVO*sM!U(;VDjaSfmPt}HsVP^`~G%hnLx{uRM=SR3_=elB0GD@n%RIjhxb>Y)RD%0 z6!=f7Z}kIrU!|4_PeN4nhfOFFj;RSm;!d84;m%k0b7;J7hT1>f^!S5(84N(o1qQWy z!T5~~1Wq4pQ-z#|<2JzR5I28`RXI-InIWMr@TB1egG%;aBNYZl%@-SBxm3&fV6%>>t zmwb&z{D_1{_`3DlV!HEBnU#{yNkQ}0%qGp^p^E+clUv`5O1=(&Iu$z>9~(UG?9I)t{W{4vzI=bXF@2HWyV78wm0#gh?{}@K zA^!T^n9aM|1+}>!Fj&0OGI%x5mP6Bta@3EdjD3TFpsdLDYvJS+@+vpwyDd5mm}Ju5t?Y&bsi zC2}3vFZn|*kqS))6(X3yEN|1_kxp}ycFrHbT#ZTUL8#+_T6VpDxo3@7KXHgQDws)P z1|wI20M_ynOd*lVkmyDL6AS?=GVLOYSl~33NkowZ#*(Rg0c@NRYx*Ho!i~DjT$j9z-Ps$g$cQ z1f+Zth{YlVRje@TC!XZW#RB_}HuO0o4@5SlaFm-qkxrr`gogkVbNk^}?ECW-e1ZRF z?=Cs+o~Q8YfAr?|%P@@vAxN>nhCLoH!6M=aI}jD7Q((J1(;JT+FO z=p~{;3i}ath!67#`rp+&g_%2)Md$Q;{KE%WWm{3$fDLZv{@?iM9|?p@qR(JE55T?zv%YKK7w};E&`3^@`$>BM37FAP^ka+?gXEgVh7a_8qd!kJe>$l& z!bGAJ=q{y{IST=egu3T9#=_h4-gtm=D=dE9=Q%<}5Y-U)Ho`^TU#2R8d;msIj5qatw0Vp83CL)=hhOiS#S_JDbsPAbZmz6NwxxqWJ znIfuE+8cG8X5?s!SlydoqTb~L8_|6k4YIX&-gxD=l-aiLmG8Lf+rKv)Bo%HDLsu^5 zTvAvnB>#0tf!a^b% z^mz69<#)s@9V?GLXJnj`3X6yso*kg(kUSMx#AbnD{#*X|}{(ghIa(cN_%16;7W}oWI?uuCRhI3?Z{IpPj{1I-Dr$#RISJ zA}|&|{70$((k9ndF!9fSYxj<{68$yU6>O7m4=C)`l-~KU!TTK2A1j|{l|T|i_El(Ksxjvi-2fKH`o@I{v z_?S^nUNHQL`YqT_$Vidd`-ltN?lAk8R4B{-$!ha-D1mQiaU#W5bn2nDG=`k-5Q;j; zSJ|Tf6eeC_LWVHv_7Gl_WI#F8tIVh1XT8%h6W{Jz@NWIS4WLUuxx)^YE=&9Zg=t=F zSvWTpUw*!POFuF(6Yuvo^SqNV|29ce7NIg8q24kzxO?@is-&%)*6=UlFei;*Li%?p zkIYVNSjD8Y8}k4(hy6ml*4jW)Vaj~YQqo!@A&J*oi+9K5K24LuwI;`WWKTOcv3=WQ zmj|x3W3sqsh_dUF`x>mRtjt1|@`D3TZcBO{FVJn+W4Li=NTkvoXgC8O48q@Rt^JjL z`IydiKxQTLWb#)inp3S!W2<12oy_WSt*tENHJ^8gf1C}VU>D-CHIs^(`NNyV_IE;>tF1*VFcu72s zP6B~eR7DzHw;`wXiQOTYp7SgxF$>sK$dUyZBf{ z>S@d>fxn2H`RC#G_(-F=gGN!=y&XZT5DRj?IyNi zzj~EYm77ZfyQ>5gL{y};LD!9I&Y9A+AhD0ky-|4JLMFYE2vVlg-qWXG5{89{Vzn$! zbxru1JVVq8U`ltNQ@zIyLpH(2nzjfhYhAJ9@Yda8n4%Gj-?_yd{HHze((xEoZk4M;N##l*Ig|But>?yEk=^ky7 zVdj%f9|3^~r{*Y>h(e^E75%nm8GK;m!$fKQB{IK~i!H z{|mkwZ#drPu&xfOEwka0iNCprm(0oUhjfP)9B?;`nHGJGd?+_9^u(Fvzp*|#B-PiO zN$xBOJmG2vcD|H*iX_nY@`KYd-bxb0^ud}aReVBh6k^C}jF?721?B$!2TOW!{|o?v zzq;fraW%y}c^C+OB@{_fcK;N4<*G!PeGa7QSGow(pBRryKK%U+80$iK?hmZM-oEi^ zLaLQc-F2%XS$I4d`9+9WNz9{xQ(d!RHePnEx2Uz02wKfU<8zU_2}V9&eJ;cZE#D3K zwyl?wk$m3mC-fa3D-n}Pd-qW=*yQ+>3-m42Yw*1&$qk&wlC z-IZ;!!8xg+&P6UdO^mB5@FvcTtTJ%KkW(M!tH}bUOpAUnq=>c9N~-0rYJb@$mAx`Y z2Jp@zc;0tmsrUdb1N`)1$O6YY2fe%6b%~n#O8W%4f=Yubk-Z;ylp`&qhv7i_!RSIM zDCwTWTN3m>iLjV8x1R_WZ+Stxob-?5p4OuoQSA+qWS1O7nOhTN9HGzfU(3hKGC-?* z<3b5|LR?t64t;Hm>Ltu>dmUHgpjz=1Ayu7HX*O;A8ZRQ(NIZ$F_qMZLMs@sXY%=VpkfZvC18m zJ;K@20A3nU%ARM9f5NXhxfk!lDR%sUvuWV0o;2fGIIg9kKscU&f~N;CpCmsVI%nQ4 z%@}$1WfQ?k1B#X1Vf+okSzWs@Us5?b-3{kvBlwEa7;WE2BhszB z!aH@pZD%aCZI2x1o%lV7&TRdha(*9$*p6w10TI3x-0`(Ahm z-^i4t-n~lr*kZk_gF6C-g>5M?(cT=2la4rC3@#PgF<~t&`r|L;a_Wo?qo>bS*FXuq ztI@^;=W$WGz0k$gIqV360@2P*@F?oC%%(5$Tp2rmVGDT1QM?dyKdl>Mw-5DnI;7{V zs5!qe+`bgzp;LTIeRzLEZ*)@lY#92MiC+98FTUUc1M8hSSAG?ii@2#!0zc9NQz^Yz z(}(_8{3GytStf`i&HzhTfD)5AkyjuD1%1fidEvH?x;fjcE5w0m-AIA(`dt{&ln9cC z>(`jw70nx;HQcNGL}2w9J97v)DmY2RHwc14;y?C0!mm*5G|YojeOxnagLTXT|2F~N zl47~LiR`M&D_rDBnGSnRV}PmXaGMxB5R$YQA4w5@z7&#x(jF86A@crpb71qu`&00i zZK1s^*QQtcTdJj);#NQm*z1v99R8M|-H(| z^veNv4z_+lDXv$AhfI(=hV*cSA4s>|&=+V+ecT<06ZFQ6Y^3N$w>To-+h1wG9bz4-OT(PQle&2XDM%DCc|7H<#iK%tJ*JHcJj=^1g-3 zjka5M(eDJUwcQ&eANDL8+obujPxYj{zCvCR?|lj17I*uWG|u^V+jGX8P;}8{|9ah* zVe~Wb*T$!5UE+4pcJrS*)N~s|U#4!ifpq=Lo*XKHnV{N`p-J?znrV(UX|zJ++Mvi8>o?86DukWD{Co;+RIpAO2V>&(Ko>iShxf-kV1 z%-0Jt?9J5b1-iF& z15O)1sVeU`Ak1Ij*nL6{8Rb97Jv2J>mfA_ElJs(PXSnW>X+-jotyUmjmVQa>yioaJ z#fr}MhSw{sP|@$Xsidj+J<^-^=&!VLl!tTGIYUxhcL>hN=f~N&0wI&ZO^XiI!GK%y zzHD#bZ>pp}zsZ8xkoXz;!ANL`A6$GkS0VjL@EvPuMhuLkoC6H7KudZ?&TkE#s!{Ch zIkJ7jigXn&!=qigKdC&hFuyRelgsmlbyslP_k1L?#6<)3D9EA;|IZ#LaBL=g~DP|CpBw_2w4xv{f4D>r>U}7e}*{pZIkrrWKpOj2#7d# z;984SDR0c*ip)g(L^*d|5jixlhTI{Ohk>Qs>vaIZXv4b;9qq}P5$jzKa@`AuFh2I< zFZdu|GgQGmY9XJG22FRQHr z3eJ|thLvs}kvTj!dX^F2AUr5;#KK)0aGO%%F*uu|(mCW}6wZDrojVxC%3rSFdN!_E zMdAiJzibPJ=9M(M5(!)6>iEv){a%Ba*`{t>K?BjgO>sbco1ZTg9Qg}2QYB^1jmm7n z&sOX&S*2gfZ1s!hJ25Vj|1PVk+@m=vB%g5E$WZ3ejU9DKCm5f<*H7J!)7gTpIUF5z zV=G1XGZMoiSkvR;)@S3yAK$UIvR{i4`Jg4G9=|uzfe(GD3T18AzR=Z%X!ZM-M=8fv ze+p8kO>)bqfG1VJBL1}48O?B1k%WHXh%UH=U9d{?>Mtm^4i=xUS`4^?ovg=eEqWxl z3{v&`iIFmhN)&vl;t5T2t+U#kHmcmf9f)`OBNy#0h2ow!-XQ;u+uVh|7b(Z`^-;>3 ziG6Tw0X8f@Q@3+XcTsb}2F*MqrAS7*bLYzbsISl#ww-yL$e}dP&xeoGf3kTF*CIO^ z^K#Z^!y?E^PMjExw(sFMBN`KeTv9@EYo5(c*PonaMkXERB;CfH!cb^4NMgfw?$OzE zTWJsScGZR{|ICC_(8p>wYslYsI8)|#JMnJu>u`4Y%NzHmAEBEW9 iNaVfix5Pc~C~wb_aa-gKyeBzE`{Rm?z%X|t+-2(0t<90?i63#q4?rf+}+*n^1Q$A z{Uc{GIWw6_^2yB6C>14XEc6fPaBy%~vN95CaBzqqI5-3vRKzz)?TUE*TSZevK|}KL z@{*I2b9i|8_V)JVFL?n*jQ6jQ%6UKpP&EW;9zNK zX>f2bBRwN0Cueqcwxpy)LPCOd|X6Ws)4AOpF$b@Es4&-Yh|jR(!! zP8u5Xe+om}nN1uVy7_$pkMb9DW*HOVZkW#vwjZF#I7ksHDf*=%WPiD5xJSJcoc z*f?x3;{En_S|{N8V7NOk@uo@f9k$vLt5xm!>xor$y#cQuHPcCwBT>ctXr% ztk{cv>Ngxy+PnKVE^=qlX&C?H><+&t%PaSP ze0>cj0M zLb0E~y1<5uwEuM7P&=zU_{JT{&RUCz;^woqWj`Z{{v>`a0HFqFgN(7`vC`AmZyDeC zBCx&v>BglI=9T?2sbGO6YH$jcF`TE8Eue~5K^gh21=UmuOADXWBU*4u4rpA-7z4Lf z;d8+i3n4=e>cq}o0>dB>GCJOkN&p14i>7?E)*WUdD6hAetWjAg1Ld#?^OS&Bg+`&y zGmVHeI5b)s@ii>B_Y1H8fJm%E4HZ9`>k3K3#4z%2_N^frir?uh!eC*j@hX#B94WHQ z77*;oZZvW$2tM6;k`B=y-jmE45UQRQlmwo{?$>fSxLzFeVKFo$z*5N?kg*<@-V6K= z96FIkm0kT}GEc>J@;pyJ^fq53)1Gkm0y(Ire|GZU;P!eknFTPNH(FmU$wX?8;^dzK zrB8sT4rIR#uc2{trm#A1^rQh_+&)WFhAMPs>lW5NTit$hv9(hWQ+aBXmLU=`8g2xC zl$q{Wv*1Kx!G)c+scyAGjNw$<6z8{@I)0nB(Z~t*W5O5)%?tOrBJW?IoPMGh{Tp46 z(_6|cBR~JwiLs8G{x=VVTpo;%XxR%o(+_WO{CL!?CIN>bm)}SNHVkqGew~k~4Vy*L zmFLTFiwD69>sydthSNAO(##I=bSl*}t`C(Nygn4iP7i8?qcahq(Vj?%T0eXIPz1>x zvZ}S!p}IzT)R;C4cC{`P5OG3HAeOCP{s>Si!iw&q=g~AN?;Eixeh{{gj;Gtk)N1&Z zH-K^mV*j|0L$kue!!uU`xVWws8=k8$MTfwIg}kgPEE(}teB1e$k1=whDTSgZsfOX+ zRiyC>5T`8yT_$Ga6IDi#L`L$4n?YOpfJ{lKdh7f=m+$z^8oD_>R;Gt}7`KKEc^abhz!hTNohpf%btdhif%<^P2` zyFcW42;y`L?+12RR8psW{Pis3sB3gXGuVrOjxmmFQFB6Q-eBw@9?jML0e7@G7T?jPC?LGw_g{pFqvmrhvx*F&sw+m3C;DB;Oyg@ zfAXl40zk%euRbrWFta!?>-D22_9|Qc48?nF%(+m#6}tQLwnY%6)Z%G zrwd1X!YV=ePbxMuMH-OLz$9(mxqmmOG7LARlR)Q+b`3Tbek{xjCgU|>H zP8g6g)p1uB5Y@@=3TQJpl+ae!V$EY010Cj-B1mEtaE5#nj)XcMgR((^!-kjKUxx%` zH5b?YjEbn6EjYw-)7eSfk)pN#_E2QVHN<~+?_C_U35jmlegT2IL~rwCeeVvh9ZZn| z?x}Yv%O>*BM86>h7~fQijsrnIDWWu(Qs%LbDXjs)9UGXEwc$~Yw0ktDdhZ7AJjmfh z&lH`B-w0JN-0_R#IFuI^NpuJoiB zg%c$S%thm%z3vIa`W1x6*Thwr4Q>n>YslCe<~6yJ)0d9P8RE#kMCJ(Qo2o9FipsQ zI%lhNmV8>QW?h7eMT@@AN+M7VwBq;{`3UUfem@v{79#P&5!Fry*^XXX0) z&dB2JPQ&GjKCRY;<7NjPsqZ7ZmHhm8`xpnPK*8r~Btvj4F7l?A+@4K#2Rms!>~Ne1 zbv`u9!x(z%#4{uZ3zPIS=~C2Z+Q^)i4WJ^V zz+StpZYlihPx9e;m#msc*ezyAOcHj6NGYe4x0vr>{#~NM6pq5& zKxEu=>6oIm5}muaXQuCqg){BI72T4Jb7lF6F+`&a|5L_3Z%O!u(=WcC=#_tW*GlAM z#7#|;*?-@p*y6!JQ91vJa>!8!@0KRWGXNjIuP!9(OY%iv4x@DTbor+vhB+BKJ=Y@n zOMPvQ^`B%~Qh1T$!pM^s$UCwIcKD6%;GGt~d_Ug<5&Pc1dw3Ej1Q#gT-sXCCk(Us} z_Y=^WPl$O68%<6W1Tk{?m9*TEr0k^ZX2|W6D}NhqtH=H<0V6V%P$G|RncACIjHjpM zLk28pIp<6#AO^&`X%MSv4RZ-=3;1q|Gkps z=*V;7PGm?`<#`{e{70q-UlerTwxDjgRMt!Bl}P*E_<&U_m*X#?QEcX4ivfCd9UV#J z7H^?hSFNxn7XE30%A$HW&WH`p(=9|0?w849Z?ch0QOKl|l3BS^+k^X@&s0c~m&+2d zSV!n%%1I>>-SJhD*1F||cA%^jKuIhrvtES=AoQf@9MEUhU46-T=scmN^8AHViZCuU zXj`sNbJjQ&=!-s3U%YSxiUzW;ghSapP0b{cVVN}vE^Iy}8w?kyY;Nv3Q`PagGO)iU zv-$!Rtd?{JDR_76_(@T+gbaBY?}a;y8r+W9Gbu;{g26dVd!eNTRM#x_^16TMt~BOz z!;BYMSl(Pk-94c_;lOdF0CS=qr>sk-QR~nQ6b&NXw6#{A-B6E#R%_4_hARc6W5+A@ zPdynOGltj{FXVWAC{RQTGsS-|B;EKIy#~GJx1zYNi}fR9qSL$LY28p*bvfrHTE!nB;loPEy%<9(# z+b&@eAjVmAHaVN>ffWp2ZG0e`>1~)_ZKNULn4&dm^=RN~%Aegk!im5uKMc@>``Oww z(=TG}>T_KxMI=zk(}0W3n8Li0kexh)7`OfeWM2%Hq%P?A3}%x7qy~f0o=v_{nA>2% z)Z!iYWgxuOTvtw{QlH~i%hQcgfTx?*!wLZ3K(CUsCa?{IJa%0{%madoevxq>g4Ho- znE%dgVw;xkzuf2*@=mw3m_AWW>gPQIaAU zj{qcvU{S~O7TXK0l<{h_J#TM-;hYR)9LL&8y5nU6GO9@{V@7SzI1a(PD+r!D~A08i|D7F_Z#m@wH! zwnl$FGZ*i_p2Q3_{|hWyYkfL;w~lbsoaQG`-zLiU?j(XR%FB?&{MUp1$NR1pi{a|2 zjMJfD-okjh_D4O7t9(9@+=vyd1{JA5*4cLjEE1d24yFB?%84Jfl_z8cNhFI(GegAV zOLs~5R?PYp+Vv)Yj~FHN?skq`I?&DDl{n*GnkbLa-X$H4XC&eGuNV#Js&38s&%5amUK{q1RkwzsMqsdfW6E zncckvm@Hu_0vD+7v~{wn33tGzE0+x(>xmvVFm|>f=^}eZg3+M)(#P{@fh>u;;6oHJ z7E6BO_>7W>Dd9J|0OW85c#v-*57>J*RimCFcD3|wt7z{2w!x-nucM(Dy8~tYU}fNF zMdl}F)G`XAIv=R#)B ze-f)QTW7HHV^$BtsHM0PxXr-mSy`!VN@e0-s zEM&jN3Y&NpRq??f!W$3m@DX`Pw+NJAGvS8Y>guJm?L31`^fJ&;@>2KRvYWLk7||uw zWIGszLw4Ifw|d&1PW+P4Z)jrSgIPzU>4^ANnL9y~O{|=$r_QPZ`Q)fpV;1yp&XL zk%`Ajiao^5V4$Q5kF)P)Uo35(`=j3v+e;&@y`v+I-;ClnB)&NmJf!wc!@iO%a(%cg z38{cHP~=VJA@i5hOxnTUF(k3KKiYd&ESN#@ndG7acj?kGl!zc{uiWptJ;j%D?Myeo z-*W!=+8F&fhAMrU!_jeCF!tz!zE(`iibSCLr&oc}t!^82=UAL=Xf##uXN@-z_@*-q zaZZ+dQK90_gS!Gt({G4StsJ!Wr8f;){6c^#MV=)++ z&<8d!U6gAF{d?ILyUp~8=F+~n4Epqg#gy4d3 z88Afo_K)$vc%l$al;;3s0Y|?r0o7HLSm$3JdiDkOhk$9m%(07uephbShxfvF0MjfJmr!UW#^A zu0Dyt{Ov8X}acqf^p^ z!U$8m0nF)0tmv+~z6>@Q+-_{0Mx}NTmuEWCy# zThcU_-aAQV@}eC>jktcCm}NvF#Ac?|WzVu`>$_7(oet4SSBYv3s@>_hh8I>TT?ps< z-+A}uhV2zQYw9Cd+Htzu?R}cYv&9HYCaNAm*FWtSC?%;>alZJS!BQaEQlV8xSdrDc zPGlUJ>7vJ|cw<|jkQeaC5rO;|HRY4%sxz4NZh$OqV#No$AJK1nvAw-AN2_{uH!!3r zv18ElQwOIMW=dR!FUoYf+qY-kB5#00ZJFvU=ji5bX~i$OQkpNOSW#lI7~3zN<&8O_ zct|bfTN;~|_`g~tdhT9tJYyPd+KjrNo=5hwYncV|j~_d{!?w#r6jLs;uEN@!WFZ!+ zTB~;vxjZ6VXD)}U{_i!k20h8O*MC&ZP<7-cKhP*Nq-Rb2-HC#uOja*|JJolKfl0+#B;TJZ1Y_IIKGnGaQ=k3xg__8c&CsqH(aOm0-!16wiuwbl;@=?a-TXI1!^``IAw1MGw>r+18BSv9mCUQEM8&%FI*u&7a3QSE>SPvMCo51{4L5WZutMNr#S7^6^ zfgpLUmAKn3*V(n>%4Wq}>Ag*^ltXI>r+vRNk5f$TkZyf$P|gWd^cR`ku;)C<>r z>_M}?Eg@%d7@Ca-rUHBHp@Q^Y%I(W&D~)~yu0~qm*jAQUaFDpqFD42G%^4-YIbK#g ziMMV#qaf~TP^swAxFZC-U56|Bb4{9z?W?7vr(8A;Dx!x>I->ziTwtIYlA?JTZV<#j zEb9KxvG;s^>+j{)Z`?c$J#ay(7rleo7DzQ;L02616V*+Z73K>^B8??I zp0!;pC7S5nmLvKD>WkJQMxvF*E#(WB6>j!-Z(t}=769!w>W)<}M3iDTkPr!fi8+0D zb=OaPrKQuv1T%1E^l=HTiy!p{bhly}>S);C0|9rilH4R9vl$noc)i3ApqWZ|$0(RQ zGmWEf+)i|e-yS)qi*t{gk%kB)*c|q8wDrgZ23L7uXRk-}TqSvfUP}3vC?73!`cHqQ znlg+2ZqOF_S$U4Z!087e)}UFmTg`8js(VgM*XPKtgKN&4)bHNJ#P%b=*rp z&#x*~snkdGR(2WQF;Lq>&?ij8h8cV8I6_2P@uWX`EauXNeg;tp9zg2|uj@_c`FzY$ zN~;y|M^f(k><+gbyv;~fwo8_OJXZ((M~SO~3X!VMR7k>j-D$%mDiDD?8i{}1V|IuE znG!w`z*OjZ!sD&Qk6u8K1GR~a$^ysO)E;ILNzqK9WwJl~W<&tRIn%v%*s;W>! zy~$&&mD`#qqC39#!IZjki(q4ML#Y()*^RA>`Q8V9gWz8OZ9O&?ou$uxS2P>}UPn#2 zS!6VCefWrEKNFuN^p~T#>4n&I%yzhVD`ariRD_4xt#jABH4*1en;1lXet8jbcVx#3 zQQebGpbXDhuPno=9kUM7y&lmf4spdO*d9VXwE8ZN8Dw=Blz{zLrK~)!m12)Gypb@g zhh_fp%(f&j(NxZ#y}2o6W-YQ{=e=b`q_?j80zuQYQ{=6FNTWfe%8r_cPvAa~9VAkA zetOgWp-35Cu#tZ8xaF&E!wts`4JbqUL5|(6z)W?AZ2tSa^B$erO(izAQ8p3+6+QtO z=VTzlt+^Z(+}i|Yyt{Git{hDapgb+yjED^1i`~=Imi5r0lEdWepBHJVwUpG+<@KN` z9D;#NVO#ia$@Q$PxMd}-k) z@n0gUmM#~6kB_9!a4a?Ch>=Dud`&o5R3_?wYr+D&5UVKX=jIwwL~z&o-t|Zz=LD+9 zkRrAI_@2=u8t9N2&FQJ5_D&~Mkm^%1DdtVlwyP_>rF1_`%gOb27=8!(jVgn^*T=t= z-V^MI%xehGSBZvw+BTU72kvdefeo=^(X1gBIY55dJ2-Svqng1Pz^H_V&s_F7eyQYz zlr;hUw6(~?v~^wHr#g1rF*L2fu8_?giQXC&(`$cZYyJnT1=6Zb;WCpBXEca6TQ7-5 zxNs4*1}K-`pNAHy1>eh=FOSnFykma7GTEU@Qu<3ewlcuWi)+#*%?aM)GvWN;jt)rM%%z2k!4aT|vJ z73?9r!Hh@vbCrIyB4~@pArtM%041W670-djxywc*0)IlqVy%H*1`~dY%6SP9yFr$$ z$EkQ{8ivqDx2r_Shi7rLo>Eo--1nG~A50!=c&Ru3Q5!|n9lW4YgO zb{;KRK(=jwje5aJNwxh;5B(PxLPd{DY|HkUs&2?Pcqd{GV71IskYuW0`hcjz?s+>V z$~zXl=~L2;Lq=SUJ|INFkerN|kUAw9=*I-=O~xNBsI*@{>$XytnyD&{dp7pSs1+)Y zEf8{yP}F5{T)dwUV4i|W5bKjJv$B8-R!gBcFdkml_6q7xzMvs@6A}%&Os3zLd5ra$ z>O8)ca#YTbkcPL%rlhe_G}q#g5h2EoG#uELvv_EEra@~Q2yQ6Y^qOP*O&@at&w zNA(?=OGtS&usp_7^}Up7PT6($UkdmyObELOsqc}~%$l$zZSJ|_o>Fw`msVoPQ_;A`sdAItG*gjIWj zO%w19E^fZCVnWEJUi(*vCn@i%(HKBQ@h{Kc3>P>3@FQ6#{t6>jVDJ5#O~Sl@i%i1| z8OEu?s}}d~>xP}ZTddorvUnZpI|1=FItu7F08KS zvamJY@rFKKt*e2T}T8F>j1dqQZ`j&GGgI$(#r&hXJ!&+~oZMKG%D&s3e0#11>cp z=ZE~9*CWwaZG%=xTH1A$-p6md`lDA*9YC-jj~D=k3JL{+@!kps8nB<_+W`-zBK20} zunf8D52ZW!Ef?;*iTw=BzWBoGU#V= zgIbTadcPgHXZ_Q>^P1_nJ68#ZzVsZKYcefH4yRJr%j}RMgzc@+v&(6GV-5FLEiHRM z0BPSGn((SJpd(f>CNW2oFoq&4O_NY1+qE)`Q;$-m(WO}eNr}HbAP5(1Qtr9N@`lBZ zxN(%p-4R?k%8VWi@oQH-Zlo%b%h`}gP$v7$fp}qBX~ey9A`2Kgi@7wyt_YAczDuW0 z1u9U8yvR9saO>5o72GbR;!`xwZ*ts|atnA~o~KcGl0I1|GjA>o=2k867AE+3ZHByP z7Y)KNtmG_SnAG4VUnCNK)7dV5D;~Yc_|Zex6x^bC8U9CMLdhRrz}14a%?>JlPn`;> zid9GG$97>EA_YY+T7d<5kWYa?qdowVyn>1a*~C_~&`Hz(A$ zf#f3?6Y$<2%mW!$w|K@=mau*fu5ajhP0Gg>L!s!aKdyTRf;*wX<*iuhi-wkP%;abG zcj_j#&IlPq^p|2!r#YV>(9_352DZo#7i89Co)LvCgr`7RC7k6$;}ktxiR9#r9$2AP zy|Vg@jQUMty|8LMoIwOZq(QrN8oL%J7FH8{Hp849e4rcVX*mTdsqhdn71kFge(Xn8@%er|6r_fu6rK!X$Q;bN zcEg+bY%=qMG1Ls>BgV&r{aeruQBX>yNL!gQTpEo7cJ>!I_SxW0)_z_F-f+3>9p_72 zw^+*Vq5BY2Ixo#|*8qV(@+!-`WWg{XX{Gtq!`w(+)ocqTwGRxtzql-a*Qx4OMB*ly zgUJW|VeXV1JAoe*10#8>=K-bOxZ)U9Kpv3ET%+$l?*GKv>PPvE(bN>5;itFp;pDNi z#(j`cB@?2olZBJ$QHghDEA z;BF;ZtUC3uulLS{sbS*cjjI)^OW^LRf)bBrjdU$fhPOQ>biY-(6%t% z1*Vi<0oydNqq6fSB0j+AxG^ovw%W$fMW}F#6g~r?D|%(p3y^|xn3l`G)`j89F{Y)q&lB_IANdHS^6Mx@ z8~F!(F1;>33Xue{xxhR1{rb@P{Jw5sWj?;Dpb`0OPXCAMB1*C!lsq&Ud@l#!s%tDQ zV-e7}>B$(meK8mYfT_RX2$=ynE z`x@F2@__Fc4I`<1HqrPG3?#|_f#Ls$#Tyv@2aEqhOFvQRd)|1@$TyVq8h`Ne_tW+@ zwtNGSLNoIy&#IB-V-g*e#i!wzOx0j5f`NI-(5SQOFMB36qY-ptcsI0^MIL_;`BH`! z9^${hEuiL=K>JGrcRZ^tr>ba17N*DLJg7(ALPjf-e(ut_ZD8Zy9YDz+!N8d*VjV|5 zVKx7671;k;i2`9N!b%>Ky^{HuUn}2HX{5xo!Onuf&>Y?N85^8o(mm1M2yx7#V$U&?2Y_Wa11wY)CyhAp{QK9vdr>*E8297BAW$L?To~Y4dg;IAM4Ut z-Z~XB+{}qSc|(8!n*WGv_N1~T6H`Q)%a3$K^cdwmK1h+)??5e%{6KqX%x?x>ykqSM zX(B@U27y5dlb?$K!q~L%B!1HP$0}9yYVtBjjii$ z0TK&pT3V!Pgt8;#ltxDd8Fc`!%$msuy8>HM#!qU|qKw-zJFdt4vz#&X{+;YL=#yU@ zcXpP1UR;frvQ8k~*CWR$`n%YU`Z(W6-Qp7V)yJ-<|B?N(^>H~1kKNICp_WbtD$O64 zr)qKDxE$73xTY&)eLt}up{2Hm&$G8Yht7FyN{RI)eUa&LitC;!wn=8$+DT+)L#s}Ihvpi1!<2tO6)g1N7 zhVJFMqjIOAAcnLkM&0sd1?OA(+xQ8Mw;}%%wBCl46`TQa0_9J+ieIJYLC=;;G%sa`U0aNPaJPX=0e@@%PZ8H_RtZ&s#yU--V9VyN!!=aVC|B%LRZ;!tBe>{1H z+XPWu#Xl(NbAVFe0>i!KL@1ui)9cWNaA)!WFPJ*nb{Cbv_=u-#50&Ve{{B{fTBjIz4Rr&Z1e5)Ouj zf^PnEe(<-@!)xtDk&-ZVt?RLIwb^q;Q|JNC?FJGlmuHw?DZG!L(%lFb50nznp^IbOawL0?>20F%P4=%Qz58j`3gl_fDGMufw#zE_o39lPMejlN%p(r&v&!tS&;Hmk&;U4t1Ekg@ z^!Bnipb~(a#ni5zcwOc1xuPRi5Woa|aIiTyCL48e=%3;KbfQ5f)GTA5kePhdOiKRS7q#5hzhKD_` zbsXunHVhh{30H;^z(We>`Am6RQukB6m-rNWU&7VE)jxMSb{Ja{ipnU~*p^kaG0t9I zz|OF$@}9X#qxvG}Z5T_bd?FjO9hgImByo#T7NB1#DZoOzy_^^08zz8;@IwHtV1Q0M z+5J;r+2Iu|$@E}yA&u;}nwsgyxqqBeic(Vr^!#=A;H4p?+81G5A8l#ln3xa}zE>oP z;hjieKbw-u3Co=BHO(5p?0!`koUl?%X%KSBa&3;sVx-$kJ4UaFoDWc;@RtYBOiE6x z&4m8)7DB^C#ZsL!YUMD;DX^q*DCZ!^g9$_jr%M@fry z$w93TU5&aZdgVk^KT22tjPg!R7CMf*5&TuF z(EbYZTY%@WaMa9aVs{ujEA8)n!(RN4e{ew(sY<%1WbaDAHs`%yr*JLsG;7Xj4s>dCzfOebq(0`y~e|5bMQ#Ec!7N+!=)TVczd*q!CWQw z&TrD!&hC(HR18Msp|9_m5PMotflT~^p#xl2MW-wC+XdmT@8%0`skAB=gtL1k)E=gojWlooV((r#lz*^)!HZN$aMw_X^JGKuvZwB2}BCBATZgKeYq;!j5L~$!Bh2> zq&N0EGZooyvZlo7cPIZxX*GirwvIRAn62^rwPfGRmVdRN{&tIJ`J7>KEoF+U?v5^0 zx0o*PV1oo*Wpcq+?ih4*3Nn^ehn252pnmvETP_qp%gTp_lm%`&L$KS?c|Hp{@)kqO z=>UQ-3m$VM{}>$!`&vpLJp}fKJbnJBmiFj7`A$nD+_{@nT8D0$dNY`yH$>!%f%tce zvA%pF$1?x9;>1%ftqaAaN7Fbc&g$rPdvRqj(Ep~}GUTfq5@qtCd4dHtMMSw?@>1H8 z$H4D1EHjjK->5H!J4X>%?n{yadK1ERNGmM3l)1{Yrq5q-5gDA{i>}PIl>9b(&!0vp z8dc(8IGUSplip1{q`8j&)`V* zDCQ|Z|E0FM_#Q9c#DV9D-_LV&4tzpw9*U9riJBs_ls=$vO9h$|2WnBQ)nVN z4_sB$OosV;$zvP0LnS);g*xYjz(j%YHf=%EZK3uK-*xc$)L*F8fpr!U-PO+tb9unq z)dby?F4hb4sPP2f+nxnXQ1{4>QSS>d?ql5SbDUjoIQ+%??WvH@Uw!F!$J^CL$2EBM z`}e2Q=QUohOVZZYu@&+w+KMn!GJ5Opj1(WTI}wJr<%D{)Ch-}*!ow>l#5(u?9eMtG zSsqB=xtkU|EzUe*A!BQK^k`i2Wzjf}o{(KC_C9;tcH7y-{)^U5;J=@d>x z+4it$wTF$iz?%A^;l1;^0J>1_vjjb&MXFNSIi3M5#>d}zql?eDFt|FDO^R=97&%9l zlY9(QKemG%>1d0G7vD8iy>B~ReV3piY|S*!$CS5sd@QExAP89RwWlY53kA}Pw9GLl zp$uGv{X)CXSFjJ6OIqWRnUa&~9%lLmV^FIKVl11iYhHY;LsTZSPHRh5gt%YGR2=$~c^55U|~ZaCDpd8mgc@(bFx z5DmG`4rgtt1gKsSrok%w=_)#iz3&sD6i#5ztl8VF|DQhFt|~Z|H+|F~hBUkP?;3vE zr(7{6Q54xaq#>>4;&FZy1(}gbSZc$Al&31Lh-xQq`=FaW^hRGn(nlx<6-v4Hcz zfJ{hZnPSBECotMWL>TN4_ygZKko)YMZ@E?j4#Q~AbXG!8=uFOvXCr%dk)ZY4+sG}H z!l2J-b;^J$jT5+I{x19+SWnk%m;Fxmw+nPa+AWL73@0W8gvaird>D5#t^44fG9ip$H z(e$`+jKN_c*RW)s$2vs(bt?vr3h~Yivp7lKM!A}euZ;RL{FXt3PY7;7 z05OvzzFV|p*Bl<zY5 zo$gyOR_8X3DA;sX%8sd>thCsz(x1wQ(hx;5Yg$Q(_1Ya9e8^$+J?qQdczJPMx1M$U zPOHo#ZeJUU^`{(Ky0!lWQtbu7_nQG2Mxn*|1)$Md_r3m*N@5`Qui{YUnLQmF>yDuD zd*bANj#49fa#`?Z!#V6X33wWq^RxF|b87nhxKc2jc{jDWO1b=aIJ?G%Lm?J_KDKI! zKmhw%QK+C$+-owP?bUBgjTN+4^1~B-f4P#`l4rYtIuiwEyHD0)gl}u7*No8qR~u=c z8TS}o_LMJ=S-vJCOio+yYGO!Hy8XJp>MOWq1XEGX0@o7kSkpiTrPz+1+t8#v`8Ry2oqPPbyr*Zs{by(RMX6Amc2djsT zG=IeH%D92ljHUu<1Yd5SNkl+N&6telW!Xh?4x)Srvj+AA&#YcXwag6kYIm|cz0QqC zG1wr}JeyCh1$3%@x zGE5*Yx2ZwCP3yBgd-&%*U?^?jF-4LYgKRQ))q*XK5?;LmJ>xH{K_0puS*G+_=e(z{ zCj87bU0qZiniRaUZDxt~*vg@TFD9_Uosc#PrwzHAWE>EOsV+rsnFN|{y_Tb#Klqcv zFB)%#N1b409gfPF?|*I8Qk`Mr*G9XCQ-rSS(OcIwUkHUNbFxk*n3#{{jbL6ZKX^J# z&1-hEc(0an7Y<0>Sp6|il2lk>V&{4fk_?GsnOf7i+ib9V4gQQDTcS0#_iBw2vjvR~ z0KPw21w>XXf-(SJJ(}Mk=LjI&AwE0R8j`!e1kZudU}=;fGx(hW z3Yf4vWEzwX#b*uAd>NUG&J>I29k=W)F+UV6+AD~$L&V*{>ZFmKP1zI3H*PV_XR*~< zj+w|caYRfx8*;Q55>s4UKl^L92<&F>195?}L)qf9{VhHx1`xTp=dh+^1DN-yAJMKq zt-@!%W2C`ZurQ&?B>vZ}99Z=U)tMubZo*?%3Vi)dMz>Y;9T;ye90&BStsJa)07RHf z{8&SPI-$(a2J&~Bc~o?Xqgdz!Z(^XsFuyWYX4pF;fZk5J<3vd!-{5=iI9jGZiBltq7jt#N4ievDvj>&T& zX;}1)0+dp8uOJk=F?Mq$cG05!qrELhw`DMQWAS0UJmqkan`lIyO;m)G{^$qNn-9Ze zbm3SUcLzmThS=FagDnxLAzUxb!&1ZYx`FD2ZkW;C3+$kO4#7W{jbL={fH=!%BY**$ z?ZpYo4Ffiee1!#78x~yu3Ns`$l07{W_B|?1dRZaommPb2H`2N7UBLVvl&$dBqyMsa zKCQGE4lc2dL-U`Os%x012npCV4y4uH9OEg1U0>qj^5MhtCD4S)uYQ(D;A7;U#8M2! zmSr5fGGJ{d4$>N4x$Gp{U6&k_4LO(04YGkl57M(KY3>ph1L=gh1k9IHikQ>EDiX+> zxGeWJQ(du7_2K}<9HTz50@iGk5E>k zJ!HR!+ZNH?bY`e$@6&!d`!l_j;mfy+&SQV}mFgADNH!*{gV=sXq>-so_3)k4lcgq} zWT@(K-DIa^2os}<>KeAMGJyU&8e_73wj6B&{zbJId9b>$&r(NLlrZlchzm(J2K;N zz-Mvm(B1ez`b<<28R3;y{+>2Ujc}ZhllcA+KQc80<=<*I*Si5D{;22Qx{nq|%nq+M zpB2OA<1gAM)A)lNbW<=9x4M#teDlve4;DQ_<~sfJpz#Oa|39|AGAfRy33qY#;1(dr zqQTwW-8GQK-Q7Y65S+!`-4+WHLI}DL+}+(>!`=70=l;7t_Vjk2nXanto~mcMs%xTk zbz5__^t!CHQDVsHr2^ls9tNv9kz!<9cRb%e^!ZVdGgA^lANmD~*ATf9S;E&a1h8uzDIXqx0|)EnGi7Y1WK1 zRnw3%FHjT89#;e@blfie_8ryW*3;>hRNC03OOcaz?m+HdbsDdtL@!Y5?r?n#cI?|d?LkpOdVxwW}+mez<+ zX4DqI$ZhAnm|6;l)}X_@8>cM)e-8-c_56Q+ z5_Q-+4>UM>lVtw_vAAbS>zks&bQD_8T02{L?dMOvY(WE)F@;T5S#+h7Re`9 zOz%ZdI?FD(`2zz-!}{kns3!S~$@D5$xqxtnqxFFH*)Iw^Mf-owEzjR0?9y8IU!w@C zpm;RuAj6+6O#UEUteocz$yK)$oVm;sPvmde0A zU0W`8Et#uZH2)Iwy_HpokLc7-J}n!c2+!Qgq$6}FA}Abk>T$JOJgyWb|7oZ)tNkxM z=(LA<6m64KqRPfl3g}Rcl&rmit7ru zXFw8Np=A-Hsh@mg6tsw7OqlI5Gd(x_NQ--HIq{ebGSC=!XDHbRLR-T{xGEh|5Sw zqS6chfQzEqVi&`oEhBF#rO&~{CJmVHv%#ws-*!IW{wMzrbf|S@k0|ZsK1ka^%@-e6 zP?rV+%6`TO2{eTX#s{RU{a4|=^dFo5j9`6lfnSvrpIn}9d0{PiiBO#w0B%p?dqR$8P%K+4nAYRkz`)&*JKotz zb+!~Vy%d%4jTFPCE3LN9ZIiBbW8Zni6yWk{5aC$_MF+_mOE++i*42BnFg~hQw*LGZ z1YdlYsU|y^H)bm8hr?RJD)9L|b&{v+B^hjvlH&I^<_2)xspe1+h^481@R0#*qS67b z`n^rpLTZIBI28UGL9nxVAs-uZp#V+f(9CFoF?a>OEbh<~5?)dAss1$1){k1~2sb!a zTvg4ax1=S!o-0EBBV|EkbVBbBqdMNrUt!EUqWq;&5bayTazGQm7t8Y7;KlsxcrEdJ zipI6u-|;YhqNG1|pYK%0urVTcrD*|P(eJm*`#5gc=mIHf#wflKj zm%9cL^ii7peQhS8Ko3v?;g5i=WiI|FEA)zYFX-IlINIzv8*DS+m1=o}dzAjdx~RB^ zWI@DGmgYtx>izGvR;=uFEN<6A6$k*CmS?E>{t3)&^dKyeW>2gZgD8QBZ5e4!5{}H znHdifkSHUacpq#|v{*VwmiE|F03oU!~LNgdjrkic&uq}o0vmg1Gnf;J4$%7MLN zrjYA|%LP6qRxK7(Vvsp&#Hh6cdyc6Z$3)d(fEUyMcpgGNed6gjTB|*vex_l@rdWi*eAB`bAGx>11U(1E0yaQteDWb%GoPOW+8wtfsv~UE$WEd zt4gC>Pd7aH=R24oO>RE~I@yeO3ppP_Pbc)wBSdXb26p}#CJ!=aU}CW~fUSWK#xRwj zw1-;|v+ZlqLz?%(2U_Fx5tXgE45(Ls8K_S9%L~OfSV@8~r`aW1#G&&6tzD# z`64gCy&e0xFz2|$n>v8RO7w49aFK4W`xjcSr~(O`waK5sr&H(eUjDUC_PmOYr?nr$ z*>nHtdtO1!CH-73P#desy(MJ>TLR>EFTP=PL+Xw$J7O4LEIi(^`9APeMZ<6@!t6@ zLgwH2K!jK>K_>tcm4|&$@j+S41M+onR6BKdOV;zGRTu(*?3duxAr&>=a$@~90S7&T z=Y;JhQ`|p?{otV$UPw8<7UlabPH{b%D%rxVIYTm?k#=gD6-0wC!h@}|WD+aQpx-P! z*CR&k`lqX6_kZ9nc*g%2m{KhiQ{3R029s0vair)m-UoPy2RcxDh%Z4*&t-Xr?vBbr zsK;M@L2C)pAT>)vrL5YzG0J@&Nu#PNrm7`Ikf9}tS4Zf3*$SvqpWZq+K}Ju9+2@hu zJ5$ck>k$g=F53Kp|G1)5pNJf;pEw^dz%@284s$>HL+4RqWIDUeQB7Mx4WSTsYq@#x z$Vr5zck`n{nf5C}bQh=Hw+P_l;J5pn4>B6awC95ipuY}@VaYpb^AP%Ce#U}vz88TVRusn^fxGD07NjuX>8Jc_wp-Z4w%M6ee z0NqDW+zw}uQFJ@}?|#xS))D)x!Te6i;9N*epaO8D0beraG-+r2-X}AC9HB}Jf6)>6;0+o0f%_y>z7%SSy-A_>OUowIfRf$A^ z9%_b`xe!j%ppbEJbP@~GG4=u%dJhRDY`k(73RwNbX5R#N<8{BB4y*g`FoIx2R08?3b?>%_5t#~-cn%|jb`)n5c2QdWPDCa|sdHsiGK;um z$H7S^gGsp`X$5VPS4tek@-VZ`<6%0m!10%RLMuI5H25)~nDy1{L*64KU@hTQThEnN zi=t~@y|mM)Z(h_U;cY(+defGRM6!j};{~{ga0)H5bTEiO#Q{`#S#-#$PB(OEx>mFB z9{>S~OqEvNuC5p+el-Bk3RP#FsZk-=_O$H=edzv6ssr4$CtktfCyz;-cCE+nARozj z9_si@k_{HB^2?~%M70S9;)2hBP*q~tQ=7VpijZZWU+$(`m|!X_IrankjdF!qOG>+{ zkA33Y#FZ9vLySWq`u|x_os)a*xO<^LRVRRSva=Mf+f)IS766zOu=dwlc?qT@*UG7W``*wT9H;X@anSfC#U zlkQgQ6VxZwnT{VhLFRT%%;aFSpQKQR`9QD2mOE9$rdG7^1r*ziY%u9vVr_6bL@!+^ z)lBQob`Ag))^9a7gw*rS4~^?dC42*30E(ely};&C(Npr@P`g2ZDZOa!@zKk1CWy@8#}KH5iyu z@-uU_X+o8UE$ z!aKXhycp-dCI{&ibOxS=uJh6GWnwi4{cX~qexDYP*XSF0z~RVm5=Qh}tc0-E^Vb{> zuWa= zmcVRG_|749VH;G1^Ut>__%@u%c+>=Z$VJj(h>g(82R|S4`!CZOc|Hp05(@VTjFIv>B>}ez6sOwO5zr+!bU5xV0N-C%bd>Myh=khTDn4Q84UD~YQ zVdgY%#T$*Z3g=YJkMV!{jjn@G zkoy1ek#M9hwU-R;Sm{dpDhg4{q;l3s^DR4BKUrPX+FBq(-Y~QHNv42``yTi)ZZ#D* zl(M%Tl@@-N9cT|Fz!A7aep1`3vrUUQUV3blOkUJWNUh9locFQBn|tch$ukOz(0|G} zy7liUK|AwrE9yEE*_p0=AUcE); zfQRzUU8F(~*PpCFi%I<3Nn$X$el$-v5Hwb1I9c`yWQM5VclD7i-2L`Y!bnq?KbMc|R|HpCEbx1`8TeOmt;Ni5I``Y-M3yrC$>U{=mW4|U~J|Nl6(f)#Ztq}l@= zTp*Q9woH3Zt|%*?Hb&(O%KLUo?bZR zH=P=*97|UCM8pUB9RQQn0hTXe9`O-bTNtaNv?D#7Ox?-%N(25nh{G+Y{5>Zg7Vz*? zL3aFxaGN^y`M21s0VyBA1dUjeMLO$dDyND_eJL`|<1$!ZhsV(j`0OZ>N-rx+_28}V ziIoemhm zU%~(dg$R#Zo3WQUwmpmQCe8rEh#f=?bgC9JA2sCZTZw9)@{oc;3xMC@E4J8zF7~y+IC*Azr%8%ulV4?dzKam=*Zg=@M z!ekCRwA3###)8IH)866NPl=J4154w`M5`axz5FOCraa{?Pw7{2?z66KNRRaXR;3@l zn#?nD?*A@NU%pAXses&q@8D9MM)?5WB=ihq%4<58Pu0x9Eq)ft&Y?<5go@V+hS>tH z3j7~0YU}btn3Ul*J|A0tn}=3OM5Cky8R}q~FGe8vSB-rGR73O znvbmHyWN*F@gZFIQhBeKG;I3;q=fB3y}Y6SC}c_h<)g-~QdSytLt4GsNdGyWeY4R1 zd^5%ZD)kq8Y`0w1P~GP>U))rCSrn)^!KIkW(eM|t)$n=f7K|hI#PR&=$5(dFBvi>F zwAzs!Nm?XZf?wLBL6@hitko zC8SmMs}gg;(YXsWV+8kn4@w{V-gJLHqJu!*|B4Ou^Dg z6bSO$2k9a}*T!B2BnsNkBp3adPvZg$mTCka$u#t~)#0IcziH~Yh_xh0RZ*9zp;=f* zGw!Q|*1Y`pe+);a^lUgOH|XE(STP%CFfw^$X(BP(b7_GZ_c-$~2n00aQwe8n0KWAQ z7vA@b2yvm1k}FitH!zn4eLb8p1;P*k+YPA|RUFW@N)H$IJlO{!bQ9EB*T7Z7cx<`C zs;KCB*_?MPk@ERHS4ql0iD<4SUkRvhe0aL|zw94Nhg zEtt*NXmx?{oYnJqH&_6w7J=S!yQ$1H^i4-k$7uFw2k1O=ZP*kR1@8aJVq7XzH$*pQ zqenqH^}G10_797oqXNN?k8^;;#Qn#Umm^<`n(%dW+aNU0ptSm*f@WX56r&I@f-~|j z*(ICW447!|Kwd<1}$5G{In4GwoSE1f6pUI|lDw7eG(jG05T^RU<@RZ-lK4Og^$Xcf^ss+rb>1&GH)3K+v5pwi}}NWmVMGa`u5 zd}tOu`Z{hZknt!;vrtx%E`Jhpu#-7#ONAzQ`KpObvwmCu(`@ zD)~@%|Dm+&vFpH4Y3tfh2S${Tvc3c<+wPdX=SHpXYpsk4ym?8r)D@`RbsnL7EoH*2mh8O;Z!sOuHc#U!^58xc0(!gJr5&DgM{* ziE++3#~XIF1d_C*7?e`)KO_(Xv@*?vAqqbWr~xNPW$|r6xVW@0l@0UlZ^=S$1fn7g zZ=r}F!Qtb>_u&=M8WQ|oH~#c_9>$+sSt z0;``T?PPDK9iXjC)Y?-lK^hBX(2N2(Iv3&RzgP^<5aY)L=NFR+-5YKOSiI^0WJOfF>-i z3w(M$9D+IU8W^Zp`l_TdYxdm8*245k))gb&p-QUDGNX4mBKa7{RjJB7B~rq=>@;2Y zaFeY)AkSVN`)^as^L2_&^*R4g*{VC-rg)W5)^0T1uJd@x6gv9FQLSJGV~r_0aRPQ_oOK*bV+DhO8r|awMZzEv& z!pLDjZi5nPt%N7w@UNuWNfvoosQvkH{nkua?1~aDXi>$K4-hvbG2%;-sz{y`@41EG zWJ;;Kvx~yP(`j|Y@cyYX`Ruf#?%Kb~v|L795^9mmtfz%vP?J|!xWak5+K`rS*4wn2 z=OfQLhj+*hGMo&s_Aq>l`D@htD`%w4k(~vR2fzH}J3C*f7J&py_KfYsJAggbdG*|j z;=Gj?xrZ$4yXG90g&qJt8!=Wq~5^Y!3=LoGhpLq$BRHOl{7?fH_bv35*US_%6 z8#y}!B_~;|_@y=P%ch#WO7}Y7Vh$A9fXjB-bKB;ccLt(@3fi42;tLFHI0u7N0qnv> zvWlp{=P$KVYrou~dHqmHgdgWyq!LA7I9aKgoE3p)&>%uY7M(CV230+fJlg#%uzdU8 zG~rXqNQ}ul|Ex({y0)%mFElqwIuiKY=uajsu2Sp)k$T;3n8vhc7!;X8Yz>q14UNGT}Q90v$A+E5O+>-^Qqtfb6EWY0-k`~L0 zgt=D>@N2~b<+hYCGC7vy*^2{r`~J5FPa+G2v9M}}kF=yai?XYIfnOjH20%Py_VV2H z7Y!eY0N@09LUZ0H7D)^71gK%s=b<7C&sBnG@cWsLpT0A+QFXCW&3~cLadXA8_B|sj zklWjI-mhPUL5|Z73Qz6ptyW0FEzuPtl!)#6@ps}&W-iiS*<*&V4(efz0pPJ=l-wT& zN>@b|PAGGx3d66GlU9r_d>p1-%uH}AQ)k>*V5OtiOPzy?VfBdJ3 zo9;ev%MkDOrKo;$7EaHZ$s9Bz5=}NOmmiwPs6|zc`;P9*{XRsXUh}!vJt6`)f(1a- zdl@D~1DK>Kn8pBw57>>qzI$ct_cnOVc)+gisvCi*?No5m$`7w`o@vZ6mI3^}K};BC zru7ETJL-VaYhB5yLD2Ux^%buqAgN*+fblq}@nj^iMddOM66THCS7@tAyAF2Ao zNL3$FRm_?uso@aFePZEV`H@03bK1;|u@WP;#CacBaG~Y|Fj`RrCV;FOv;TymD3Qh6^MA_1d z@59CT+LL^-`Pc$4oo8)8tNR`IG}0CNAU0sWkgES-V+-u2uLg@~?!4-tHp$`om(Q`) z_AP{noBhAWI~QvH-b>^F#7WZ6Dr?&JA7jV*;5#dLYHo%FO)IgU>bAB&{D4J3L!8?w zR!+3;>}s@*;`K%A#k7P-G#E-%4v|4ANJ09C2eu#n>#yd(&!&_TYUcRWS>ui*qj`GB zS!?c>E>4;G;d_23@hXB}v=hx!sIa67%om&4=`b&LIc5&m?S~l)AN6kE$H4??x|Ix%LKi$`B@0({z1e#{NP zt;mm>hXpffEQRW>7QVAUzbXx_kjRl>AW#>%=p(;??Kfqx+tnX~G&9&@Kc4^SOc5P^ z%D1q%tYCS8WVMR0OsUq^fKIf;jI`%09pz$KQfjQ5fRxrYvTC1w7B`AL>A=ysseNNg z+4B(flZm5Rrt6&*InXNNc6;9M7xvB(Wa?7WLKP+gC|^A&pJci!_f4MkUH)#qXlu4* z{&Tu-@~Z3?oI=(p4F<5RWz)&8^4Oph^zX+Y8)I-IOXXJS)9V%9b=|paz ztMH^aNc;x`3(l(Uq>&o$e=Sby3AkXpw2>(`1~3{(YH@F-Q*z}LT7$pPsmPy=g5>rs z71sZ2N)#%ns0#Kkx=JcjqYQ8S-8bp!?!1T_yNk4d?u}KGDO44lw7>MroUl;h~%Dp=KJ$ij0-pYjV- zm^weQv$Mwnm6Y~n|3=)}{g|mo|0?D1g^MEElK6cJ$Od1YT!sts`gcnj_KR&op-O)>nLn`{lK|POTayaE9FWrYOs5vn zHT5z=D0%;WuDi`Ye5L_o?LOHA0DfRkvh`{Y_}H|=c~@XJQ*3|(Po4p>9wyzIX>+1I zinW+4X1DN?VJxH)i}1~(>+-RKUYj<2o77f6N}ygnZfmpmzRWR0T$U~7z*s|>|}4In^^MUws?qt;azbWRjfe?I6D zd3h7ySnl5w0<~VCbY z-`^8M?eRaBlE1IdVAjl)_%b7za1Sk1|hv?fipJ>98Z_XR3`4Z zUeyB$M#tJLA-Pgwu4uXdKI_aHYPR~q1E*2s?#38Ei|p_qQ@|I!A?S_+pq7S9C_Du3 z;4+%0Dhy(DyLS3www_SbKPyw6UgHi@od^5Hf|QM3KA8VBWd1{C92d~$ghKE3^90;X z+ul=*hjt(3CTj>U>wi;F2h@Hs#MVb~u7A=aRU31eoe_qzI>hY6a5+?S^?11qp=bob ztFK-FP9B4uWl2M51$Z$e(D}UmUfPeM@w#~a^1;p2xbE?a#kqnSbwTs)1uEA;zZzs0 z@4Bf8Sbmdq3R%^ZfV9Mc!g|>I=fN33AkrOWUp)dODkIcy@9%*ywtoh`t+W|p3& zIvH+U%y_VdY4l*#d1Lm8cJ5mUIV^sXK0j*8l(thE_I|zS8Pl3YSETYq0c*{I^ywkk zYa`{){LlS8BVAg1Ku?!M<`7~LPRHUnoS-2klFij20mqk!;!w9xzLoEC_r zE;#xCuDfVyxTg|>isd^f@XOG`-{~v$0gTJhRQkv5XDG4^_QrbUm7tWCw{P|Fwmu*s zzVJh6?9Jya{ji`mP}OTOHG!YxuSZimy^*HPsQ!n{e@DMNAT!sNY!34__=`?1WsZ|4 zbi_F*c+kF}*mK#NR|CZ@8b(=zuN+1VWe(6HsOP^k{Oo!Qgzhw$wgx}f$-kQ7 zc{atpYZUeLtfN`MNb@81WLEiu!&3!guw>SM#a0@5NE%0yOa3#K->PD|sly<*t{`z) zn4O-ZeF}ad0LknAXG&>w34_GLg=^;ASxXoE*VaDK)52P5r7|m+VP)?Z4Tk^fZ&v5d zJ_NH%9|7wl$!A+~jp0%yGPxBM@4m-u>mTbc&7AQ3TNHmPXZ&b0)nsofBwClQi(EUV za=%7tJ=N88HrlWO6tP3o*RbCtX4>9V*V|IpdwRpv9}sl~iI|Y;x0*EFU>~ZZ;ph*f zQZC$Lk2qcuzmjflLVrL!z9QRUA$UbI2PI7-y>4E(f?nFEB(GHKVp*O+rtJ{BM!}|Z zp0yh}LUY6F?t`7f*Nxx#&o2dPBGLuvfBTz(6Cngtu!~2$4VE~-sVZ z+IFhb+bByleItWB{7(w0e1E-wwao{={>MfCJnINC^oSi4%fPEtPkiHjkS>~pzHOa0 z^;Nw(J&;Mw66Gdt6Uy#F*^~oUV06{|ZgSFH4UQRFoF zGQ0BDVpp3iRXE8Fh8r0yiSf51O#@bj-9Ax+e^bx^(>7h*l%FaQP)I}~0V3&<(T94H z<3<~hR{qp?B;F6>*B_|!l4G@*Qx~<4{=80RHkP=c>w8xds*A`=DA)B3>{i4_md=gs zUw=7g>eS)tneeyL@1){cP_((+JNJ@MAhR`Mn-^S)cN%drT%Gt1ODi|ZyCGP+z=&9R zXuqDw4fx3T4CR|G%8OIY@%4mi(D+4Y((1NWX!>^-#NgLxkP&x)v8OiEY}*dAds79G z3e=w}5&%gz#X@8}NTEhO_u36>3oR9vDZYr%L>fC(eHZ)i~LIE$BWQ()-l!T~e&P zbrLWh=y#A-y1&rd4cRsIo6FGhH_e|h&EjSv9Ooc0dh^*!$cSW~G8ssEca{Q#`A;Hw}i*8D@>873shX~Hkur@O| ziSL+H&ql--$mNG6akD0G0YN3oj~ST(s*71UO|7-ApUZt#8qh`xur0WHm;`;2Q`4uS zP(Rns%tL$ar!1gLcr41W?fWv5aH*4p#O59?@$ZCxq1W+UAOA{O;})MYni#4$1FV~& z;pI&LQFrW7)&g~z5*dI$5^VjuX8{DDV=?tUKjkM1P5})gJ(V3n{TU3TJ$dk%yspOp zIec72q6!={?MTSYk8ZkJixqvn5Iwy?34PhGCUm1qnFtQGQ9+<(d;RZdhWIUzFqE1c z_IKC#iPqyRiP{^s;Z%?Beo+`4AKeOT4W#PU3ctVd$fSv}WY&m$B)yO^d7u3el)S3R z$0$Z&c~bY_Kd)g3FQKB0E!(A&hLUB|b19MH#iF4{uRpq74dt?tX;WmQ;D@Rhjz#W6 zEN+l1Mj%)gyozO^)|BSJWj$?5siVD*mY`^F*GOF+L~+^)3t=sFWN7@@-0aP>=0Gai zE(GwBa}NrD8kNvWtZAp=zKWc&1pGYKBh^aHtO=CP{2*F*K@EnRty1oDUFqT=kahd% zXFf-PkfH30-;W>K>Wx{_EwZ|MC_A5Pf226!F?ljSJAFkCT~aGJk$czw(y0T}aDuuw zR_1k_c`{WsdW-$so}pFU)hfIJj=t3pISeLSa~9)6!k!0pXlNY~&Y+9A)o2#5Ktw7r zCP(g&nf-}Wp{QZntNcjL>y)pKKeQ&AxSj;bf)gn;xM6**n5#}@w*KldGcGMBF0Rxz z#C*=AK!!PLSrLx*F(PJ7Al(y5_gB;>SB3hPr#NAz(>jBZ#*q8epLL?OdEF$(2n*c4Ecr$s&d< zq7^&5o7YSmk1Y;**w4M_BS-)jD;6?QXni7obF4#S>4?9uJeCYB^>}?2mIq_g7_DLlU& zgRAaDe;qmE8_PDdwG9sMZD9qu1grpgSCM1@+3oDhpSi%8>Qef{+G#b+Lea38)E7Ar z_17hR5!q&XWpiKgkNJ~D^{*zb;2n3+B5ko}*Q0cX*26pcKh+P@Ga0}U%_q%F3TGxD zil-fSYH6k0FU?j{Yw|$f9KH(Qv@05QoF5REuhi`(hM%_7S^jW952%+rFBBO6ePDk` zk=ku;(j{U?A=fB9esj^1#6LW*js5T1(rLkJpV)e^>1+^XN^k{cG%prg9C@iUpT~RC z;0&-7D0_Fq$I{HA!n72}3qe;~o64)eGr*NlT7n#?-b4UtQ?w!Y+mVuKgIxn@hx zGbu}lO+D9f)Zj8Nvy&?-D*1;rP5sThPh#1YXMbQBcx&`=>9lh&CJCJf+<0(hxI&QJ z6qtpPE1W`c;b~A<(2{U1Vt%##=H4)ZFkv>Ch=7-TiuKQ(B}v^L&0_ zsLHQ^{s$iwfj94QFei@@2l0YhZM+`KGCp7Q6ZGYlZDg^};d6Ef!^eNt^#80&3n1;! zd62)z}-qZ&Rc`qrAedpWsb~(_0p04dnl{ z^Hj_qzuu?*23GYn{`+pa-#Y(3@pzBp&Dg0(x}b{xN^T#NqjWv>GZ2RZYf>d|6+p15 zER+97S%nnruY?W|x{)(9c5Yj~^8aKv9I*RA1|(2;exd!y26h8w!cUY6ILj59Eh@+Y zc6(92BE1rKjxqZV(;|X24sqiH(LHw=LwHh|K3?4qS0Me+=`7wH3p1UZHaH*z&>IF6 zpXi`8n3yD?-$Z;&ZLxdG25v)(uoKHy+8rb7F*Tx@Nj0n6Waw~YzDH(|O4Db(YUoY( zIr3{faGToLP8zD~6zeLsKZdj7)xyO_4@6pu!L{CgCY+dn7}w zcmLMwY9u?9zI16%)F+!W`a&3|c>p{^W3weLJ$>w0wTM%ckNG5_|z6#SRA?7i07NldhY{so6Tp%L?v>cQFK#1D$B5uY zBxuo-5-dJ@vzAME`qeo9eVON$7mX+A4pD(Ty@DzJ{9mY2PuH;bPH*GwA>^{@I=JD4 z_x#($Q?p?!S$4mkLXn@2<`*@@j`zcL`Z`$w6psIw4>Zn(P?9IPa0GuafKjj zCRb4LLqhAFtk=HDoopcD`{<&_q8g!IVdxxjPez*UamfuD4a(7TlNkv4UBUi!&$(Z zfV?Vd-ert@w%gpI=X(jprbm#>JDkG(uT8n__=e>?;fX}{b36Zz~dULG0 zF~2R%GqdBkeD_UA0mpa`YKjN48!_GAs#q z(TQbc(hF6iuGl)QFz)zsF)LOY=eVh%b)-B__7&Bq@MFYIHD-zQG>gfTL@?Wq{Q2eg zSFin;M!(~vv8GQUpj|O#mju{IcopsgmE-jUo$#3MQrYC@S99+5>rR%2(zE2=l|y}P zRdJIc2X`s8_5{S939`L0TcI64(y2ScP5P2mHVn)G5=szdzh_zl62bm-t|i9$4l%`a zDug);&A&2X?H^fH5s{DV(EwOgpE%o!K4v9Z$-X&E4$EEU4!G8RtysTHh2Zms?k@25 zZ;g;}XWc#}&WVe0WZNn%KS};{&iJVKxa%y)6gA)01F9xoYzYFmuGRBK5TyH5*Er+zAy;t11eg33c$eK9$r);-! zgCR&pweCt_!x$8!4ztIT9jHnuA*p`?8d`s~?U2!-!aJ0v?eH;*nVgVKjbaw1om`ek zDShNIc=+4X81}76-!!@c1+a|FMvJ5noC-E3)ZpG44M=Jo z+HAw}l>Hvn-TODjAMRpM-tscFu;_b=@Qy9H$9^ZZUxP|itUlEirQ#7sSR=bl2dc8+ znNjkdI^Znsbi59A{T(O~c?41))rP_$3{ibH;eSEd1U6_ zG#KtcAAAtt#LiJh#I9kXaMkvllLnbz{*pDfn0HmHsMft?lC{GC$CaPB!Cl6^RW&Q4 z3R%8$oA}m`1LAY$5e&SVBR-hDBtHuqf8Jy<&deOpFld-9Y+V2nbs>-FJ{vVSvy4$& zM8gc?zn>Q`2|+?10D#a>B*@Mi|7O(&dbHp`q#AOI`qSZD(7G(9VimAgJvf?6AIjuW zs`Bal#G7oeP{4U`7OVqqZoDp(iemZ7Q}`jGSHgJdQ9?ErH;bHo&)Phcf3voA)!MgC z)Vo8~ICL?$w(lc!GS*5h{;e_LB_)o0X8kQYHO57cnTdtzxl{)Oiq68s@?QZ(r-J=p z6(-?biy5C`x@p0bdA!}9smAS{V=ULc(4jZJiDw@-@I0!b$`x`=L5;zPx1oWu*At=w zG&bw1IJ>>G>|~7c4-(WL{K7)Mk}R9z-_E(a7-=P$SuTY^m1sT>BLKasteJ*3fs=Hh zd2TEY1(+w|8pwAUkm+!UxpqrIO)r)^8JE8>4FXj%**=^G0W2=mwF=YLN})tG3%sgY zV!TUnphG@$GTbt{D?&trwr@gTDhw3@(eAc8Yf<;O20^8ypp*k!oSZby$qvdt@%Q<~ zZaa(cTyQNzJLF4B#dwb+-bcZ{)e_(w!+2F~I4|245GN_Q$Wdve!Jzz=H0(TO8d)O| z9r~bK;K5N`bBMyO3;MD^9q`X-k1v!ffu*t+LDDept2-%v?V`2bAw@Npf+E3l^EV3r@hKBKS8{2YAhmT zJWrlMx06f?r(1R`FHB=i%^+%AU-!Dptc|UF)szmzFyL#WC*D^3rbf!j;K+1mPQ@wx z1KNTab%}ys&i;#`1i=;rz}M|)nvN-ReYPraB;J5!p*Izal!z($a*DeJM`}m<@ib9c zV#{dQJGR}sOumA7>2HTOu{2T-%JL^r1#`qB#wAb+&R%ra zCQ8rYic06(0js-1AwBHsXbY7En3qDAfpU-ZXc_fB(Iu?&W+kJ$oe+o2r;Yefug?}- z_>`0!T72hE>^|$bAhd40^kgSmo`%ljEG(k0;g`hle#eZu*{t{aa}uOn4Pw+Gt}Fn- z$f7yuJl9-Sk<$0VD7u+HqTsd^{$gF|Ro9#>rpKJ(BE8xh4v5fMMAguCjH&a?yR#)0 zGHt_T1$QhWooO-xT_KsoroE)cwZXJ)P1{`(FBv@vPi0tn7G)B4{%bv|p%VSYON#-- zS#2M1+e$l2C<9-0`WP)#D!nf z<2~>S?jN?3ta$5a%T4{mAf=b8#`Mi-6m`1M29ta{v zZ?*U2@APPyguPkw9;fe(zN^d9(gkS$Y%m_r%FVmQ?{IL@U2Xn?3pxKC0OSAE@f*>XIH;yBA)^y9RktjOVt4oVP**C29pZ_p zDXrIh_H_^Mcunz)=@I%Ner9NcntOwDQ_GhX+~_im6)>%vGk^hRi+j_MA_vhT8ESOY7NR^2eB{0x z)42sdNgDhie`YAa^+#A@HGh<3&P(O#(yGfnR6WF+7*@V0@eLH(Q_A{WPEF|TbGoWJ zMCc)!GC$(_)TqoXqOxzw;dYCqIoFn{bV<0hOi>o$Ei z`8f;{iKHRx3A_N!27u8h1VPWqT$=A?zfaPFI<6a(7PZWNlVvqJL!TH?Rg1Z0C?$fmrD|fn`fd|^xT@ni zYMa{lKLKeMmgwr~cRvhK4H=CmgiHstQRjTB_&Q@D8BxxMq%@#3PGytHEoEU@x>abL zl3s)Q0qH=UfKvidj%kveNmV zfz4Joqs-AXb73tDT7L-YG_9Srt86j!n6f;tP=vtcx+$w3XOEy3l0yWGMI!C3jIxcx zG|*q^Jw~=e3R|J<=%bkGn@|Kng4Olz7*{CkV84X470RkA*mx=MhpIVcsODd2^a;u~ zHy;rH50vHO-#UZhF#94GA3^8R`R`?@w6|A+>Vv9!S-UGSx3L|qs^)pETgPZ!luaKd zPf&IoWt81OU)@)yY((|b6<0x->C_O~`ylIQR%NY}h5j-;=Mh4Iy-XQ%y5jP)l*JuT zvK~`5wS6G$-&2;4eMfprbXBm{k+Gth=e4%AKSiDTdh%Ma zW1)-|c8mc1*Ut#7m*wrXDXVbK!xdjaS*g`m7!FYotER@+`+3>XSL9(IRoYu6q~bf| z+wox2rYHa>I?s?!t^J#%?=1(04e{^ZUs3H^XR?hHs2oKTVCPWZIoT9IB>@7v!o5r; z`V^i!R=4s;b%I;QLAK7JM7WbW=czNuu@z}c3cmofX&buZc|ZzpA1 z?O-_rd0waemg>4Ke@%J)S`Cv3p08uNBTql<4?4y1>;t+`-)@RTR!SeL=4~NbDZO;ychHx?XxNp z$shyP6)q?%-eY{js>NS%7FJMJP*#$Zeg95g(F*(iefV2bR#4Ur9(HQ)$SYc5?niclvU&ilCS)rOy=D9oWU+-RN(^%N|cYm{E_b3p;Kp2K$wwkR> z;Om4mgJdCCYq1vyUfRE$OFJv=L>T8ez}LMn%;fl|xiYJ4?6Nr2ef60hVb+mZ6jx@I zefP>1Dd0qZdWK1msQY5H%Bm$w0M`Tg3yr@+^X^g4J0KLvI)99|dOR7kl)&7lfzGvb zhxXm0!~lXT5D#?A@Ub!ypjGaZ!?HQy}J=VoqF>sfy*ARFX8Q?F&P=?YVYj znmG*cdw|dVgFu{9ff&Kqhjg}0S=Us;)$*vY)<|Q3te^E>R_d$iw$d1XrNKv-jNO#5 zw>@e=csxm~t?J5pW$$5@%rr5g_Z~71DB);(6cF)b(m~yC6<5|*(_{9Uqf4C;an{Nu ztZk1Hk=D|fi*wbL^~zFa!IP>sn4T`w z_q`)V-fIojovUf`dw{&I06>ztvY3RkERpz9~9G@f}q% z?EvTLU}aIIPKJuiP<%(F?W=oOHKU!>oi&Q@56n_1FiL@G${xW4D8RoUpeleOjVBOe zyy3zdU%)rnZ^fUZ?Q6y?|w#=!oo>k=UPD#x6$q9LI^Lh%tq)196@0intt8 zjdF!He+X8{JdBgISO*bpS#g?0!lMjl^e+PG9J}e3O50rz__d7$I&Jx3T9j-mUS2;>AJMae==8V=LQh9YJYpf8uQ|QH_b* z51spB*zj6LyuEW^|rX zWgsK|^vW$|U8K&>ebbbaoIP@t6ONl-1DFnqo#gC5uiVMIN`ayD6>Tk zKsj+7Z-+>k#u&Q=eo+UwG=Ty+ICKDLXSn)8S&rL)nUN;`?0OF0;5xawcgX6WhmYS) zNB2PK0Z>h#+X5VF0{zSF-4J6#thzxDMz1(lk}3Nh(@3NBO3BsI^b?(Gp82oPDYM3a zVh*4SJ2(nXC<{#MQjq?;J{6VSC=2!{2-V1J3ANP)a`YI$vs13&QKsCwZY&2hnF6i? zRZ6dmP^6;t;;7e`IJ<1j)(H-`ptj3}MN(s_lDH0lG zUB___SCq9Azz5v|m6TxAlqhQxV~o1e#GhSH1IkhWV!Ty!@RhQvf7Cr*a;6zRWeYk7 zUPs$pu`t`fhHKdqdjr53YX+2k#?)u*2*6{u$VwZpWmX@+_!7WigwQEV4gg$Q&^i3K z2n^jIwU!j5PZ?U*yHOVGPY|k+*%+!%*^sUPvj||fT*EuQHLYv=pUXOQ>fcy!{jdXj_jbCili60^cm$G?z{LjQt_oA#jT4l1hB`4fFMv1aCYBaOK zzEPHPmO7Ru{?&C&8BWjcSY`ud00 zTdv_9-z6IeDEqa6vWm0RTq-=D_&a4O74U*R*UsUpquKCu?Oh49c8wOmOpESA*+m;G zz3_Rcltr5Io)=BvK$9uc#XA7(OW9iu$Z((`QwBww0o~mwo0y>9 z`xAs}WVVDVQ?^^KAs^oXWx*ZenkesVlnaMy0(1T|WsQ5_P*mUu9ro}LP?PX=yrecl zvzrdZfj@M(4`qJuEQU^#We^f&QSZrCzK$E3VSdz#;)FVwX>Xl`mcfFWK+*7eHO{SU z+PaY^Mk;v9;rvW?#XBE(YdnK_J{bk3iC%V`aQFUp;UC8Z%%@8mvmPl5gm=z0(!Li_Css*GW523Ko{wCCvuSJvst#mZV(}7o$AKjDv%TGPY`PB{%O7` z)M%>S}GJAj$#m7^zKgt%7VYH=yp{fQhT`p=dyhUgG!Pr=*Ywp-3>e?CK1)Q5KBwj|JQj z-f90vS%f=A)ir+nHBgi?)!f^p>}8ZP)sj-Bj51YK%B(?^GRjmbqfAwmGHXz!j51YK z%FeR}RmwK|bVvXI0Kz~JOaD&f!ma>{TYy))R5o?7R#p8SEy{ALlu@RtN|`sPQbw67 zWt6F^Qsxb+lu@Qi8D*-flx=Hgx0Af`h4E*R9$3;cNL!`>Ctw1UQU?lP_BoMFbzuVHl{5QJYq$_XIlks_6VVxxn!6mos&^vi`&@~rpXuyoJz44qxU!6mpmJtTm#A zXGA5S?+rqEf;5L_HS#|~QbR8scw-=R$J6r$k{Ym9?z$P@kEu|$pDQeJ?#oVwl%Yj6 zq-AmZuEHNELt6|r94OB^(rea}rH+7OyIp`~N7*YsKO0F6hVkctzoJO?{e7W@YVZEu zl1V^s(+c_caPRN$-6JULZh%)K-+low8fN7i+yY?N@qN|3BUR6Bq=kG2l)<4AG`9>a z+)0{4r~|%8ff8k)B7o)%cGbB8beaONJ*iT*%N526P-e>vDZ8i5f2XWO&d<4KN?G=k zrCPmeW=ff?15=td$Oo!D)Tjn*R=5g)WjnqVrUAVw${v7r{;21&hDc5z?_M!m$^eJR zHd*sDU@`=>x@58ohvLRt75)pG=NU1g5e=c4Ju+@ySCrho(X zSx*P@L(ZPR_DrtFoCh1lc!dnWBkQ~U;1qBuw_dG*v4l^nHI%JkxdPC9#A^@hW8Z#M zXUVl!LDrPvldc;%A?&|SJPjpwu3N5y_8w(LFPPR{M_Kr9s8;v2$R`p z6W{-UW}j0&21c`sQX#@)#CbJ#ta!#R&whXeb#_QL2jP7MyHoM7cDi&6S4UnWodtsP}dti zpWslan#re)p15q6Nk6EX{bZ?D`#pep@4ndtEb&rm_SQR-HvcG@foeQ30F7Heq&oJN z0ML+gytsyImH@N2eH!0}6QVCAyKNjrnI@7h{b7>~DRUWPAuyo@aOVM2(oxg_xGk*q zD64P{ke0IWXVniR0z(NPXaX1Z?!*{RQ*aA78{ky})dHYX8R*P@q6|&rbMr=ny&Rs0 zfJ)hvj8ds!AStn>tR%iv%9LE!9U{5OnzGplSj#aiu>_2}cmZ7iCpT)4xS0V(QT`oe zX@AmCJ?ooL3ss;(r+}y&RLy#_RIBSSik?+~GFyY!ff|K7liUVcLN)$q0PudGEXTS4 zLPot6Y*Bu=<`eKvXtVkjLYMl$%v`*9qoXJruvL5=HjyD^RbAJMTgo1Lz!!x;*%XY* z3T2PP7^AAX@MqPOG7~_K7v?(lDXS%0-RX=o*GnjC(|yF_kCZt*as%>rIZRjxz+y}y zr7Rxg66ra{n3H~%x?NghX7_BSRBbM0^KM`x1a=>P=?;{ca)|5 zNkjFlgP|srb!nyEx2md6R;Jsx>sGboxoknjn{Xb9!wA&%+yBrp-7N@H_NT16#5w+UCOs%fr8w;SJ5 zHo309RtG2>%7n5fUDs=eC^P!Mqb%)D8mea<3RR(Od{8y($x=63M*~vICI=|1L=%-) zjQ0%y-=7$@W>Z5_WjDV&sQ|V>TEX@9U~_g1W!I1KWy!i(Q|7nSp3f!Fc2&x3F#&*M zDeDe^I{znSYqted{}W}vnCEvnL}sDBV}w@+LaOb*V?3KgJ@xvIs+#6Hq4)Uz&nDM= z#B5n$@5HD9&$n^+liUtawi4EFDCr|d+wG0i#NX_W4IJ_9!Nvf??QY63v32jJ=%BO7=>>a#5Mt&)=(GvmFV9`OI5{$kK><-1@e8l0)k%YY-B{e z596DEplKo_Z~VF7ll+^xLXEmu-u1LpR#n9X4S;r6FSGhy4tF<53qvmqy>UcZx752v zdLAn0u8l;<3wF`#O0G>ae7fPAjBev);p80 zqIG7Ep&0k6QVLjqb$9}7_r6xU3pDTMDN1+Ymd|Fqh+gyQMOe`fOx>a0Wo1;ofp0$J z2+Fi=lLj%P*?#xqStp`OhsY}tQA@iLa^NwCi0mp6LsU7#XWZ zVu33n0pvzp+&X|G**T)viTp?gp=3dYtV<$3C8F{zkxXUPj4z331l?r(bw3BAkF?@1 zhCFHj>Rim)d*A&Sogykm6zsezxi&7jXy&pSvd>G-g9+j{`DO@M##o5q8d0@pg5BzB zkhmGmgxAG8k?SkBf1q3Xwm)g8IOn`^C{#`~vx&%IL??$-vz#o|DkpcI?6XAka%6fr zakjlT*ql)$3Eqy-e;lK%gm avi|{S`j5nCK@Rk0s;c2g8XL<1O!A70>WEb6vWq-#ue$}*UF8W zvX;!t%gg2E<@NRT)iiHXVX z?(WFQh_<$NX=&-y)YSI&_Uh`YmzP&#V`FY^Zd6oMMn;CNuCAq}rMW?WnPEWSExh<}) zmZ+$xEV%*PU+YfCg7tX0=jP^kxYb{7riH%^1q1|e3psg;a}VubZl0V%AP^lLopbh2 z3Q;rLHTttRFBU%!xw)m9Hm-NCua*WEs;a66T?N&0RzBe3z1;q-HRGE*{dcq5b9HsJ zzgTmopn6QBd`$@|bHg$Hq zId$hMJveie-?x40+Fd+*e$bF(P_=n}u#j2Meym+KtsZjU0V?s(Xc^x;<^YAuaTjgA z-0beATTI^^jHlovil{qFa?6>?^Cr~IUBC*xbtD$IPTZ@eo>mGFS)&F;gH^&iwx-*= z;()&gs#4j@PHE}r-wKFM?<{9avnO4(lk=TVsYWgF7-Z!asw$jZ)+;TO$R>>@EV&|ZPb3y2$WoIN}6m$nF^zdKmk!dNV zl*ZTz70UCwQNg%TTW|pN%tx&iI$e$3}YnH9;YZU6i)KraS&-1(%0s@W3 z=BstRoIQgNYPmc3dtDVQ85Mr0{U_34T%ttT)ID;x_XjJ!+N;VUiY@*JWWpX}(VU;; zQWOhSHrqi_^a8or0P}CEa41`%^(ekK$)MIH^rT??aePfw&{+EM5HgK{{0UE|^VixN zx(75t_3x{ucb~ZNZAY2NK&8q445M&)N5<|nuBOs(bnU>Yg1(?QkM&Z~(X1HxY)w8` zRd`PHnI>yDnpK@9WD@Q%Bv>nKQ*@LyZ*RT@!gurrzDw9oCvkL@w3+f!TE^y+6%b!n zuZ?6wOG5kN$vFq}?nnQDCH@%aM70|%i?q&p25FMzXh3l_8ZtFfj&)O4;Jj!^?!HUl zOptWZg_dNLx01zNf6`+fD9~(uuq7LP*FSdCH~Aph*^Kw(lf1y)NWKxAmor^iC0tG| zole3ws;a0R=MHtFMf2(QBnNrIo!)dGA{!^AC1z)~rrFF$YZ9*{{QcfrS>AN73yl4*_HRBk$_PDn{k+9ie^lGS;RLkNi&4Z@l z8_1zWByjR}-`l_ct{0Dzlf?M6X0p_%7`#1PTAzR@nt$TCnxe2dPG|2Zjkzjjg#cyC zh<;ap;@7npIa|%jO4v*r>4ZmJy%+Xxv$}8xWfz_=E-nm&Ytz#5I^~UJ*Q3*n%216D z-k2?L7uYKQ{cbnQmszT6LW$J%&!LjD{VBr7Tu)4dBghXeu5J~=S((>2J#>kBrMTAI zHNe4dPIb%^ktq=nHd^B^$@4L}0m7vKG z=3xoQ$(B2<|E3lAZQaG@Tiy#mlCbd{jLeLA3>m4AEX+ybh9`$jA;T&EI_Zxurj>gT z1|@AtXMSTZ9_k~C;nguBIv*^geb^ESH2#K(5UBeHP$L(^K5zvNi% zo0MQyX^BJVbkb3AM`$HEG6{XR@YX@Z07k;JKS`LdF%s;ZX=(`1Sz!Yngi#(gR{m)O zz=($)+9lwJ{)=7YWq2Vzrr z&&viI`u0m>+iT=NRY>O|9)-bIEM|^a;lzt%sJ#b_Ix+yB_V~&d2VHvrO;x7LkOWME zGqRhI0>V|v!8cvTF`olvfgTxCn8Kjxczy4;V%HY(k}ntPke^gfbp;g)8uY?UonvoE zUcJl0C!_v8QFPwCIv@xk=tryxzC3Kpe!Lh%VD>p-vQ5qItXG>56{A-HP1Dlh<1qLf z+WQL@fxs5mZ%LPLYC&iwHGfR(M!L4t@wpd<>0?D{6u<0Gwn zIs2HxOFQnbR?C)S?(@av<4ve3?2bw7Tb>B`^3HB8Ca}nz}c#zHQis~h& zFyucFU+<-Py3^$tniu+9Uj3vTixOoR*6;95 zLW_SL;H}JT2q_JgaiCK+`9acPH;apHYy z;-r0m<)cXSr1X%APA0*-WmYZmF;cb@JuPckO-#Ax^*KdQWLI57G=a#Du=w zGj_VY(`b_>X(uL2k6-(s*1iz%%2WQW)#+A|VeUX#tVjJSTI;+19fDkOIsd{7eD#Qx zQHHjvjg?V;Sak@u8yWf*9t)PGkatc%wt z$ZGS=GGX_*ByGHmhNCDQpTq5h#qyAoKRa$29{TBS;9AEAG-L2KgV{D4 zDGY`n>(=%QPWB`%m-B#T3^{lz$r?u0K0)d_YZ=CTK7ctCpoL_wMv^K?5(P8Fa6|kk z#2kvGLk)G$V8>L3ACzIBp=c4N_40BV(xytj35JDGso)i1YazXV$DEa8GcX>gtrJiO z)qqOG!u+}(^;h7?Uqx?yoL+AAGHSaamAN{TqJO5ag&5?%zBXgaYz*L~WJc@+))79*0_VRI!Z_8|*c;!RqPq)doU2?y(&6}HtWtN-u zv(kRFunpdhO%V>zmo*S}IC;uA1_Xj6mpB28<_-HRRAtfOj&mby=)CiY#-0BK|6e}% zaIaC~-RfY$gnYRXCa>E78%i1a}#f=kzs({TZ5nRktiVlhwI}QH$kT4 z&ZE4fbU;a)t66E_5{+bqwiQmcfQtf=J6kyN~(5ss6Ry zm8Sy`Z7OMpX`FUxknpqozJ2;N{4p2MaYv**0N-!y#Pk~L!R|>NX3z!^b{<)VxkfVi zN?Kd>k!Vy8vpI$wJP9u1KtnNlPDj?bKv);8e1U_aDDXG~a=?&W=E zLB-$&6l$XN&OmPNF%%4?EEUEj|K`67VpikO6}7R?ogMZeGfO0szag1lXQHLN#z7+8 zt-$d^G;|NT+SN3)*NES5q+J+C+r-h-`8pV3@TdK!?u~AoT4{TSaA~`2ndRUm_l&Or zqkHGzRmBFM+hz9n*FeP?;-8F#;J3}zw816h8**gMRwavMW%NS$oXM!lCS=Ps5ktxm zRZ}<8hi0adJynhMbzijFQ11_dK9sQWnH~8N-PGZ4>hUi%3`F8;ac?35ZaeI&1vsQX zz);MJ4}%XPP8c^*C*UG<9BzGlz?3E|B>sm1?+kQPm=7wbCk~4>rWAaZBE>dW*-c0i zY)UZK(4_y%EP_P}iY7q=n9NnTs5v$y=*O^ksm5aK!5e~uI&n-|v zvp~0KK*f1bViSaxAg*Vt zgw86?xDMkXW$}cH-4n#49UP*oWnvXGw&wTe5#H`fOeZHEI|1vGLCHR{`c+A^&3q2YiOUq#T>Dd6jyi0TRHF=PyY}G zXuZb3!i=m=-T%ZaE>c~Jqz?H^fstj^(&G0Hc}ke2>9ZVoUG|5ApdHv0P_C++v3k?g zXugtM6JP?+(8hjd|7U$)!;|DiB6#6Vf-24J`QwMNY!ZvF&cM88ILlrBK1YSIe}%Jn zFYZy@U}7+o$fImaoLRG1YheuSv*1aZw=S}=KD&;t529$a(6xS=r|wJCL-DP(4{)%e zN!O@7+#p>)2L9D)RQqX!fGeaco>;|~UAmkqvLX3!>}NOzC}AwoyV?gj3Xlcg<(V7) zEHdWRfL(rj=G=sPq{!hb29@O8a-fs|%UhXP7f9jrzJH5~Kax*R{GoxBz zTX|WL7d>Ipk)YMO;4i0J{e?K8yfiao1OJnu(%%UP=S5kb$Z*`%@fczZ8h-gvc z9?S$exk5$Iht|J^#RYHB-#HCnuuI3PD*~r!u+&Pnv)*^Ip7KdWWTV3lLBBzXgiDJ2 zDgU{{%Yn_h{piaIlbz}4%OH~)eUfrURks=d1hr83q)D$yUs>0Pky33a`Sh_`k4f`7 zHpf@jff%KbvqP9+xC$*Q{j;MCHIYV}Jk_N3q1C&$ zqiADlK=JQk|2mnI3DOGyxjyLVb-~Nj8fZ>erxkh^wI#;1Y3@`L2=V+E7HlahN>Gj>wk`EWq2a6orn#=_C8jc>Hpn-gXl5JY)xpa zm0?9)?VZm2Kt&oWW;slK{@eI1GGKLtA4`8v@@(`FgNvp z^<&ZMDg5ilryVV`wljZ0o<~rHTXj&b5%Z>(Vzw;FTc;<*O~Xp%_M^0%%Lf(G=#AU` zV5PDUJEP$qVIyaIf(U7eS&z39>$QScvMOM%73AFejZltz`(NTqGs3o>hi6vGLdOg} z$NM%PN$!q(eUTs9Sj|UV;ZRiz;IyseMM*W93o@-Sip$gI!^kT&XTwN$#s;1Aq{qK0%25-t^F)DGX1M(93EbH; zXgQ{!!-1AmS6a|@6uSJ0M46^Q*0niIKp>^}FM_Vss5>x9cfY~3G1c6E@F_|ar0!)uDh@8+s|aiMNa|#ZCvaKS_ZMAM5a;^q-u+qd^d0M+m}dg z{L=sEbN5>V32k-~EWGTgcwFHCWi#MKu0w?7eE)deXUJ7ptk?FLwT9leGM2?w-ymyw z%0E8bxm=DBG6TphOn6ph^F)+p=9uk>pTEv>(nY}E06i#O7>B0eG{2OPJaMu_vx|U8 zwY!jd*9d|h8_()dVv~r|+60|x*|1|}U&|(Sq_TGH!%O0HgZPPkqW0L2Cc{nSC&NuE zW9-8X$~J@wke*kz#qybFH?W;=Ah8=oiCCjC{Sa`Y`yiI3-s|F_Vw5)6v;t;p7t!dW zhwz3K$X@zLAY<8?aksHoMzeL486SLb_1-ilA~>o9X@)fa%%;fVYNiO1MC{4!!M)b` z#(D?cf18ex-z}1dPl#T;jFphhbMI2k-=LHPIuN6VBQ>U4EiirLHMeks5B)-WqJ`TJ zyM)WpG!5k9E)aBRK?a-V9_vAs1reLctXYEZ?zG!(1ionkb(m5W6QCHWzS;YbkEd_J zhX~<|Cuci`bd@dq+hSg#YLG$qUt|CJin@m6lbFWM^c;ah3am!SdS31nhmv$&^Z z)W6{CYLqzS_Q;a0SC!1+Ct*MSfv4$-oYs3kY<$M}4R&#!ezaXs0S-ns^jpikLxtpW zm9Y7KrCFnckCYfs20bqU?2kC1O2?&KHSzr1PC#-ax(sunVqk!ghyGLp*mQu+W*ylD zxpLiLAcEU-_u$?67#Y=#h1+3GYY?W3+ir(m(}up?hoRC$1uLg{s_pMm4mwhU$J4C@ zRT~;p8^LF1s;!_^;)3&Z@If@a{(4Pkn}MYMJC{&9+pEn7NbiamP7LcNB0@atr2<_H z8A6`_UJH709@U@Wtc(`>I2+$M{mB*;tt|+Mav3mKfzwAC?HV<7*cxCAAl3oP z0m_h*HkdFbkv5DB4^;q~ktBtlx=1&LseZVHg;Fece=x|GAVn(YNaISTtp@!WILM%n z1AIQTB*yi1-r?be%}Gw@?rK^zA4|67TQX6fh7Qf{$a^M=^MBg*ZxJB0=mu>=+-s&S zO!#%D@~?1P8i1U4^9PqdasTMg>~Zg<_4r@|r47tkA-i8u`{>qQog}vT{+$0M4fCNo zIcYLrc8BF|DbsZ&LaNgo@u;QLfcH#rH54W7(jyUzq*>r=c)cIE z8mvr;Kf710bD4+a=#RWd9MW?FT|*;hWEZS(0fwr8I`L$S^Zg@T3b;61_5|QKf&>}p z2%J->4PZ?7m=Ob0c`mfLyT(7TK9Rc+oc?lh5vRQPf_?~u-0P@4-#|4(b3uw=8Np%( zBaOqs4NmiY^X}ju!y<4(OOhR+htCw$UNU9y`jHd%PkLrG^9yNAk+Lw;OTdC>18Xmwt}cO?jS+Ut8AIIxt_d zFoS=9L}H4viN=IQdNvK({z}duR#@5RdKQnWtr|crHKZ4;2dSgs((B_sU=89{>tN&8 zE^XNA@DDwpBtqJ!W)(m7jYfUJ7ESI(MQKyM?N+4-P?R_9tKZPK1t?A0U#M1Izlbqa z%NS2_-(~gu0_4n_i~!1`SQjI?EF-mEiyz_8H$ZxB3m_jg+#;M6j%1NBM#rWZi}N1$ zXMIpELkbC0+42MA%ARGwk!GvMaAO)V?1MC9RJjiz9FN1s*>@2D`03w4Avj@ek;YZ( z$hWig(Ij2)5puERDs=b#og^w3M|l3e(Zz9FPEH5EETkz8>`y1!?z> zkFMUelz!K4d-BYdiAS)!UMXdNY4H10fj&w>AHprw_>a)Me{)uy)6*oK`96&p86kvH zH(%syrhjj7SS_I?Kf;okRtgCX4}k#bdcRC7arT>WtZXh5o<`lUo@*({=$I1QP>&D~ zHrrwg>$ON4fpRPh!jU(z0YVj_?%Jj&!nj)_L+f4SM>nIitHQz&fA2nl7FN&i>xlk;u3?Mx6V3vu|77p!d|QIt<=k^ zkW}|kLM^c#Xbr~@nts;Vdu6wJE|e=An?b z!j`aV!7^MF0vC`HSbOuzh?;}uMTsL;)z4!MJzMLJc5A|a5m`Hcj&T-6T>m2zDx4V$ zz1{#ta05E_OaQhmDp0lVofb;D!b6Ig{6hSR%DrnOO}I{9Vm0^^9#3Lo>5Z?wb4XqD zelcbUD;yA8s>P64gHGMDuWFm}5f@IK+ zK(C3gB7KbJMYzfTE9JKziR5b6<}6cu5-g}jL3kz+3Qj3xtv?^)nwjdJWIFi}E;zKO z50Q0eY)Fr3AWU$QZkUzN@Y&5RWnwX+sWM%kO}ori+xXCr4`j7 zw@|=pvAI_FEn2?l2_eOB{bMX(D18+HYuR^bC(N47P9?FT3Ln>2pa77KX$*vm+&_k< zfQpS$W$YI0k1m^7b;p8RTzU_i5JtxF*k#+qRkg2Il5ziOZNume84Hf)XC)G$%8iv! zkr__&BuDmXu3^Sm>YH%2`$gCh?K^F|#B`-EDw>c6CwFUfrFEEzz)3S;)cO}HEfZX^ zomqPMMiyIQ*`+4$)(U*RhJ&=I;W4K5TvYDLRpFme2>!~!T^$xKPr`h72rB8T?P`k1 zmj}ug`ietwneYTPAhGfpBZ*M^&#=#NDIi)6de>rBvP3{4EodP!C5oI0E?ca;B@g7I z0ny08{u60yNnKzaYqZQCh)aoDT7(hzIUcH*(R;nZJT#j9tSC&Mv}~*j5Vv=&z~{6_ z=j4)2W|ZrZhd;;XdM)#?cRtGhe2Jdf>#pdHxkrfeuUnH5S8n)7F!?9>(l5dZ=R$*G zpQOt>`bFeb6=5baj>EZo)5e3lkbTCqI+0;}cKk82oVV0~lRR`pSEq(RxXJtRsAeZTE z87|7^<@D5P{kTn@18C5D1^zhNk^6T72XR<|9o7aC&nFOJ^yZg$pQD*(#Iaojxuigj z^AQpxFj~QK@RzP1N9a?YtpHz?y%Jk8iG`uG_XrGuYO@Nj8g+DQcvub20n)ha^4}z{ z5n1lsjf{50v;xfE*V@@vgAx{r6hM9#Y#?2fNEcAmz5v#Y9wnzehTg62Za!n;|5CGxlesX$>)aVTN;)h@&&d(Oat|vMhMj@$4Z+ zrA+J72L%3vWYjDgTE7zVcN4T$^m{cdv0Ij#NO#WNEfK&y7L%ed2Z4@nDS(nD9cNh< zDcK6Dv!xc0NV|?bt=VWnzEdZ#XB$JHB4CPX`U;@YEZ$jNHDJuVz6zH2PlOkAeiXYZ39-yLoAEY8&S@;x6sx z*rRkheaVaPzkPW+?OwGzJSw!mg8$lnvf78+mgj{Y$utW_um7ZS$oUJ?N$c`IUJC3A zFwkeaa!cq`%c30x9|knK+p;`dR5!HwJ!tIh_V^a?$#MEtKbYd5a?K&b@HWXAv5eku zKB}$kHzzdA@SmEDoL;RImEbW@|1vL|PkhsYH6gCs{I5v?^fe9biQJ20b_d1s)Lq=- z$ha11tz3z*p?x@_B2_Smd&RL#^GVeC#=ctznHd0s>E|E0S~av8?(MGmwt;EW{eM4E z>(vZv29-Q~hj!Vukn^Sro(8*0lJPI$C42fl@j!CA0C!8rnq2#&Bm%sHFg+5M86~+? z82t(YzfWy4K;|@P(O79Atf!xw0ijG5+fjM-EmH%D`Bca5mz(asH3FR>m>zKaG)>({ zA*1AG?T)sk@okd9D2$HL7tEcIB1Wq7}|HIersIR#o*IwzaABee4uvX#9H!wu?}rv@=`w z?4EyKkG;Uh($wj6fwp#lpnG=IaF8--0gokAgQQ85)5)Da&}?w*b zfu_QM3sW~c?T)oPzAB9Q1|$s~4rYWKHk~;iPzok!5I?zn+r)!~2u+S?j@AG@#`PPQ zd4s8&pmo^ze6|e3A!D-U@EHI)%b3oKC#l3(M8)!*Xl6L7~H@11Z%;UmpSoJAV$88erD@TW(K6bJ>(yO>|#SxpzU6 zqNd2m9T@3IrY3vb{7Vu(=FbkU6Kr!Hg$YE6#EugyyqgOHXvi6Qs6%4aG~6x0DW97F znSy*gkWYa>%0P~tW#FA^I&i8BmOrUlGYB_sje|{^duZ|l?n3_8yA-{6P9^RkV9beOcZ=U%uCzeqvHIKlF*M;EnM|u z;9$5!>R@ZrFFRq2*3xGsnhww&XbefmoRN&ar%|4zMpE*L9e@05DA8PZFB!6ze@(hc z>K+0%e9C{rdFST9A}eiPkIAx0{)-2CWA1{8k|Eghf#vRUAw_RAZ?tqT`u+D0dS7H_ zHguH-87e1M;(@?+J4GAOXz4u`*?#V>B8X zuMD&eT{>a^jkNBo;=CMRl;S`4lmO$AXgmGA)>_kZeSe%M{JhY4Z!MXzLvek~h^O&0 zs#L2a)izXwg85bIf!+nP@>S~h6C)EXqbN!?5i-Rm5an)X2sCP!;U69dBvp}0U>6Ej zX%AKl<)52zWQKe6%q5^!8^KVMWZx!qVPp^%|;h3&}V>qh95^M9n)2GwL7 zaKPYRN-r3!wb{AdsuqTk=(Y&^CTaryiZDyIY5bH?txRIP&sT61A_tad|)61>n!7k;|$fJFZN+&nM3*r zB50C4LJC5z0;e|lXKIGa`4HLT<$oj7Zvzav5AA(GBwqWM3j40b@*h}-KpWi$BWxI; zs;WAkse&p! zeK|X*=U=;6@L-m%oxU=|psCII0r)sL3BN~!DK<$WA#Mt%u5XP*o(i#;gB**Fj}tkfI#ZNb>dw(d>+%R7WxMXc5JDC7J~aMVb(iEz^wE$Nsfj60Ik_&D zKHAaYbT+nzM|o=ALn4zB_K+jlU%sO+4IEkjUf6@*x8JM#SbmM1^u|zhH|3U}A3<4H&Z|B8 zyPv~su(5GK@p_ce3KyN|`O0~=Bbv@8n_h-s;`Z zGGT>bm=DqHs3;RHXm_Pwg*;1e@En;bbZegzZhmvZGQVfyij+=L*9kphX;LPs`DWnd zLsw`qF8ygk(_!al1L=Qqxl~!dm&RVSuLBC8o7U`NOvzQ$6Fnx@1-Uzv-#EG;d> zQxskn@*q{{<)0*{RR*JSc)!m}5!+WgSR2`EhK}ldQzsLm57+omHiim{XU_7Ha8Zl_h(oaWG!Sk;{2eL>3WfZZLDj?d%Ib6dQCNmv;1HG7q3`ktxmi-rc9CSg`nW>R6{TPB%IX!kIH;5Pu& z(54(3XLLw60}hZFhq%F&Pnosh&lFm4C}xYjpuWd~Vo5y@sBX-nNvi_Fl-UT$sCwru z#u!2#QoBr&g)mg@#~nl`mMhpLeYzJ;VvnemfXKkPnK_)Cl+l zap~5AKfn343($;hFSCZtW4RI!(O1k5QhR~F1CvPuyG&qu=o!{W$4FKxh$FN-27i&L zF)2TAJQh0fzmxw$Ri7d0QU-a_Ep7p$Qr}W$Ng{Q8AjQ+Yg-?3eX3(wRI7fBcWK3GB zI(?OS_nqNN`a1s)mAi7#Mm}Uz7rX=|x_zAhApj(pVUO3HRAJIRsDgO^j`W{dR7v3T z5QcnejvD#Z(UZDm#;iso82NTzz+r;4v0u7^r5yAvC|@@ndig6Ux9H?a5NGINj|sQm zMv9pxy!41-f{`TlJx8ub)VHngKnkBM##?zdMyw1&db^CDClNG>aL~BvygN9w5-`rGzfI+a(Tw)x@7}q@;79+CuSeAEwm@OVI1;@f z{AB%-nuh{;3)XgX2OK9%1RkNERM*W2hL}E7*Q#fLIK@4_cQgsTfRNU)9z^!3^ooFb z=HCF15Ai=fWb2Nx4zcyc@*%=do*7Rrqo+BkLq?kf=EZ?9rxE$^OGpxD1gkee{~Jh!R%9Sp`mJhsBpig z2|Z7G_=#;}{{w9ThL`oJ_afR^1E|FMQ9Es9=*ge#XqIC4ef{ ziFYK+KaN25#Y!U5j)d-?y@$Dtqc~=={p;v&wtorn~6?y(gZjAy(>sFg{pNT#{@VrftJ&Mf_ zO;Mscc6VwYpgKv5I#NZz#_?OP@eSsDCNmaUiMUI&wIgn5il%_@9oM}lkZCxfr08v> zYNxV16F`zPE0+8hzrK*PjQ!yygui)TXY>Y6SFCUON!+CTQ3kF`E?;l|{L35R0agwg zjzjFZfB#d1N8n4EA>91RLyb-8B7#j?0mX%({ZN!Dmd^#HC0UYV>10;7qZT=OI4oKG z_LwCkA4fr1hcIyHgwVSuCp~v9JnrV1#hd(B@=rn(61L?M1==W0H$7x32G=?VY)Vr= zV%?wB^7!tW@&MdV1D`GkBlA{GdfKQ6&~hI>nxeLVxmnum7a>~k+snE0CVp()(~-rd z3;c1GHD=_HOY)(UzI_#y^5m`T;~`;@lN%zL3{Ymg&JFO z0tn8&JV`Y?_KvwB6AqCNuv~x&-Wgy^kirR%!;8Y&>MTyPLwG;c?MKGdZWriZ$96U> zE6_#tIsXo(cMd9$%*QRjTtlWRFutnk!RDPVoTz4f$1uk7}7p@#wF^2K!jPBX&E zgP~*p-$(BVpy?!nRvjW3k_fV0#+EB&b$dq(Cp5eX-7nV-<#H%ZKNR=M=tPlUaQr93 zY;aU0PQ|RhNgx4LzXIZ|=CeMak^r06eo_q(0wGn8I~#LXKm0=+XaqN^fM%LoaUCH?Gnp6!#q^ZbLNuP zM*DSJbja-2pyQu;*xwlX0nf zkuXda`1}E@TlLz8b~)Utl?OVu8OLS~3@Z5qpX^}D!L=G(r=iZ+8iBU*h)?I506M#R=XVI~WMjxPH-;fynonz!ymm$5PJ@;UDiYB(17&}@xyEK)deS@J%z+oro z2l@~uCFj2wH?NWsms6H{|u?zg?vLy_h#Y zLWpwQsw@FoyPe9Jaf%FcpV;&_OIm-O5wgB~j4v;dJ&MY-2@FS6ZKvG$S@HwNB`pBJ zS&90u$fqR@tX?^ut)-xtlAH#bW66`L^qv{Z>8Hd3me!W-$kN=-MYNTZnx>&b3mD(t z8LTZF^`RoHQGzney`3_4@jTFw1N``Y%`d;yvR)F@*`=%aOyf2BLC;XiiS=u+p(t&Dg;JGJ0#Y1fJM+JGA?I_hv z@5cVmG!K|^qWU+dqsALV-*)I2`AS<;KUN7#N?SxKbPv;V@1G6k1ny+-Nq>j_*E+Wv z+)Y5~+sP92yhheCRyXEkNHBFIVvqytTfOTU*>T$O&eqkbTL6;cAYu^|?!MqQa|Hy}ML5vin32R1RtQ~oS zF3B? zujuo9LlZQHi&zk;H6BC~yAvVvn&O4e023iEHVmlO z+f`IdJe7`R7#4^HZ^!QK8kEA)y2E#9?EI| zKwuBQmf#d`Y6^Yuxmj1&I^1c~MtD? z8|!{=Vz{HVcGTw}fGuJ_v;on-=}#Q1R~BP82-)Ylj(ktT);8#t@J>yGB-aCzMH)^h z-x+-5b`I0UhxkjP@^+fx!vuQj_CnX9FW(^b=vHnPSR3W*By3~AdCJa|SCA8W9$O&Z zCrGzE3$F{0e=e+8s5aue@?OgJ=UF0%vPBib)_Nib=(oSUss%fz{VF&@s#9Yo@K*WZ zv5D*QUUxulQF%aV(aRzfmx|Vo6;}cEdR1)5tR;K0h3PJxJDAd?Tt@1-6GXlzRTSJ$ zNEmG3wTswwG$^e4G(SLES^)2*&`<8yZ6-Ji2`ydtw&;-N#TJ(YLjRE))t*l87bx>P(2PX{$q3X6YvNXKWuYfLoi}$a;VloJcmR{VLv&XQMDJMgS){q@r zGv+%hSDMS(3DFV$Mu4AVY2Hr;xDt|a{S|)QB0E;y0O(sgm}uC4OlO zaLJ+28^>5fJek+3g#{?`AR8Sy`K4uK8wa|UZ~^j>27MNih0em$ zR?8HFZ|BSo1Hf!70b}^qf8u`)?UEQkR2m~76yj$%rL_Btni}j{na$uD#eqiI0TD4l zMTiYM`i?5)SW?XUXiKPbE$c%}C%1tr8EVnls;$C%2ZEMIlUVm>3A536L-UBaVgt;xj8-F`zvfKY#_7NST44p(a(3cq7MM`n zl+c0+eKD!2ziOvsj;`+rKt~=tD}#9$ACD*arHucXp9vo7%!}?ruWJnz9(8p7-7J|N z1tqWy9o+V#(Yf?zQu^)i$0;QbXkJxi-fZI1j+WfK)lE6~Fb21f5ZC#Cm{{V`c-W-f zIuB-$&$zQf$tbz3DXY2T+r=2|<9F;Bw^|s!KplS8u)9!!*?T(=eG0M`AumRt2f1Xf zlBr)uAa_MvCGJomgpI6=fPw=iWh5egVTSeMQiBRxz#sP_>`6kNePr*qn9D)>oFIrr-IBrStj^^RuNO7Q(bGj6c2wR~h~d$r#+B5ffs}V31lnbK zA`McZ68tB)`ArOtfy&??F9{6zys*c2Cdq-2?FQ2+I%2b!;a|E+O>()QReStN1KK8c zw5F%#d$WC}r~~`n3bz|Z;NFq{hZ~_U7W6*l4v$tT6q=9&#(_HYCEw7?D(i^&)|Po>@)GZ12(WT#)n zQzQtw(0rW))HvXG@4|JgT_=hDD+Vs;M6P!FR}PYOwDh_ES{rYQ%`Cx~d(M9~$5vN$ z0TN@ipwsFKYt|QmH}|?rk(@rwE1t71qW$3lb=;V<3^v%48LGe_M^CpONdw!AZ3%tH zwEcs$*ngyW;C&^lHRS2ihm4LQ6o{HLxLF}eM`U~$tgp{}$GX8wEdmz{sZ>~S?ff5I zy=7Qi&l5I`7AO>Lu|l!p!GgQHJHahb+}(>7C{BO^#oYr0mlld7IK`!Six((vZ+`#h zxt}Y$^_C;$lzdKB{XIZ8o2_wgfesMIV~dr~1Qn_i2?X<|dw; z49^6Zr!fO1{L=yHZ#F&1Sb6_9)tz)D5)*FeX){O58p{SNHPY^x^9a2EupLdC*-J z#BUE>c!MxPOU9%`9-@14F=3e7S5z~K?n4je`|kN}zcJ8CSKo261}IOr9AIB}r&RSN z>~F31OGb_My^JcWL0t!ypi9#j+j9)?i^qOO z&X0Fq`ZQ7lWqN)2<=cngrGm4}iIlTT=#$4QQmt~s@C~Yx<)@c3-0vI%RtewPU*^Zx1YE+BzYIe8leIDg9E zfbp@c*r1geYNtbYgCy4zps|T0ophmI@#;6G`Nwt` zFHkKfl7usI|1MJ|eq*`flV(&5?uFr|;~0^|il}}sl^OtRiNPVYj8GKvAxF5%{Z|QD znDRHpHeL``y~|H3@XtsmNzOlCn$<$9*KH$jc62Y9xO$f#>P*M=DJ$tE&U(0gRjbr{ zTr0CPzZo!yfAO_f3>Z48s+hZ(2V=Pm)jJ6pO#tU9VT2Td<5VWB=@Z#>vHxb;9oGe! zAj}I3C-M;ep>qi;fB<AH|CwGDav#B?GXiPZ2f@x?G{-?t+ zNF#zZ0=->NHexFiV+`d)_YFEmNs3F;Nk{7~9}(IUZGnYQZN~Lsz+M*~nYXnH$$ct& z;Ycwy;B%3&S5jD0x+^vI;|&K}_z|kC?5`tb(&YYUq(PFL%=cov*4ChdzLoq^zSfrPJES}^v^2_D!zCs%9!`mj$G#`;KB za*f{XoPE^V4D(>cu1K_BQXGXY-K&r^1j*!VgrJypP|3}_Q_n63{ML(~MLyYd3Xmn> z2?d<$tOo}??JuNs89Wv-eIJn_&o7!`5eoU}4)=%j$U?@QjzI#NIqo@%KwxLLFLPQ6 zdq5(}CPFEv?3e*-Q=nDHjssu&g$f%?b9{h-RbJV_DqBkd_c9UqS}`L(TzZM2KlOzI za!Ahcz;jY4(QChu{3BmbUsH=s7zj=n<-b-{un7e$a3ew=kVz;-8acW+8uk=Z5A_-# zJgwpd7o_$S2Kz@35yVWjtxkCj`Dijr7rC}C%0_ae_RH_A+Zis=hwuzW=`3*485o;R)=YDX=FizN#9hAs?)Fu+rlpWlH4pNECvGK&KKPA38f{0K$yKk? z2sJ1woY_q+6mtT4#+WHkHqDQwobS0=NVXVP^PD1`k$HNuQlG5MLNx^lXmYN<4rP2> z=u(sXN>Y^eX_gf}R7U$Bi6U*Ums%3VWo>r+-;L9SVb(K&8%`em1>s{lgrWLhbYZWM zExAk~(V100%4mb5-&7zej$jrLX`~cAb~9Ys_E?*J`s$yl(hnyE%V_C9F@Z}T3~zCH z_2@NVsw17I(`QM#SCVs7V8$V_!wP_*deLTkClD)n3*%|Wihqv~+;T{!D~!9hL? z30UT3t&)ht{ZLm@KqU`Mbs6w(Uo398tu2S?_8a-^GgmYPt_8o!$Ytts`zW(g+>cYr zw$b$Yar-mDdh37HvoA@2P`twJHz3>FF?!?Qa4-LmkO|fm-yRY$i#>}5n%chDy}+7J zo(N9qSm;_ddEW*9T{GFHAN^_m?=kCZHy(uTf1NTN&CA zwaTOpM`DY?!5VbTSKD#4n5LlhtHA1Ir)%?4o((*7BkoXyzlDe${6^;FE-HBZQ1AnV zHpEhCPj$}^Xa7P#q*fFr2PZY5%!sDw{ajO|yp+~Mqobq0XhDkD zu^o{|*~Qh~P5_s!(e_TcRebxYmo+?UzrNcvnz6Y*@I}s$wQIB9=A9Zmt~u|NinCBn zo9cZhF}T9nf+_!GkeJoP*+B=w%W49>X1JyQ=w7K4UTCbDo1uP+NX1^{eTUpx#C`!z zYwBGzpTZAQrZ_!cG==McLj=D0+&K^b9?m`P42PcM&`qxdz>nFbCjClIM_ktwf3%VBHE7GMya~|zXpNR_BFxkOzW&l)}7VPzlkwugV0xSQ3JXU=? zxCYd*a)>pE8mxHa{g@GL9L;d40uZ2oV(D6RHoSJ3c=Ed1=E{5g@F^gckHLU(*$*zD!F`HN%38_1H;XSF<6+v`eB%Go>BSF z@{rWBKWk0toT`MsLk7+Nj)vh=f70|*tsmtoO$$>ACw@)wvru{IXRlX91qsC(yE2ls zhi0*Gy0bmufOAm1RYkyxDkx$ey%SIR8PVi6%gd~|!Ck%D&x2m9b00DVSM==?Qgvf8po-nu!!o`+mEHF zy9(B8oV+DLtgWnU&?DGvkb#t1e)9VgKgNn2jx(Fa_u#*OE6bDEQTC~b-{foAoeaMQ za|@BVE9qr4aH}rqE~I0I4`1MbWlLw+8NkUmczkaw5*utDEbE{5?!m=wv@*R7EFtpQ zT6Sv$c=3gY8~A%Hh;ly*6xyX+EeJ+RH%@77{f95*uZ6&QP=Y_jyw0rn!tIWt zLXI)8PG0SB^7dyLukBZ;9$Tg)D)%1wEXvbS+j@oSNIp$XK&%Eusj6dDe>3v(Y13bw z*8`k-%;p#STA!CIR>Cp-A`FO ziyD*CO_Z9`O^E8NJK`t>&e1Hm3KT8%L>!HGgKyZ~=^`thsP=L)g?x9)z?QdIo&E=! z6`lxZ{8OL_Qp)+&DN?$=NiEY`A^2pt{Cn3JX_cGswu>P7-WM09ZluaP0R;Vy2kst^ zFMIhLiqbyo#G{#vukQnLv;2@CeMn&40T zrm3w|4(C^#vLHp%7C?zg+YoBg`x0l3+w8Vll$4Quhz}pi2xbVXpp*;R0Yx|!2`=TY zaJLSyVy3sgOZo%ugVByVlv_m*ZpE>tCXXbipy z7H=s=#8*$86@$Mtr=*T^@>T?onZPg!`2Z`Lj$a@ht2ikAw-}{(U2L~616*@EKQ;NH@b_Xk8CfmfW16<`YNPQqvlg+e7b%4hFmwvRn@bhO4{$R`3tm@l> z)n+YzQGeWPscaH;IzPv;s^~rju7CE=t+R-xUMx0lkvJ0gV))KVz1n@#!95AdGz%-A0dMD*81vIEfRM0w1) zY=lBBmy{KxLq79FY{2cj6iGrpYrBMOLLTW6rpK?1SHbB;s(@M1L5wn=1-lAsdCA|Z zs1-Fd9ZdA0x`b{ddlyhb4*Ed7LHg#CIBE|bnSV1tYl-f9M0eccN?6iE;El@8AGD?# z!{6;)^T3NV#tSS$%e0Mctk5cO(i5w3=g@!DnRcETfLUfjzc% zAT_cw;PS;*H`rsaGOC?=UULfAIwp-fMgl&2c5R|VGzI2D>{G{}bEz?4RyNwJYc<6D z#AD-k=q!d%28ZiXLyMZ;Ya)&lU`!*&)swtH|QHc>R=2bcQ6(RIQ{xyuugl!RS`TW>ExJo+IUw6S-Yk-@&VmKZ5_LEU0Gclqc~{e=R%H?5#Srf&cx?32BT~1bi;rC1m+nSsKl?FSq_#$WFyBsG z+|0LPI6%d1>Q`;p;miPO*#KKtQ(jq1z=AD&BwIMYP3=eU+TgE6UT3ubfku8y@@9k0k-)og;1As5Z#_L8H zVnf&){-Wah(QJa=1$(XEi@hG8uEm-A-E0TAj|XA)Vkj2ePI{WB@Kjz-pA8(Z_+mi@ zHZNL`?;WT(d=Pl}Z6?su@~z6t>_vMyg7gwrj<7r+yxz6?`>$B^4#{<42TF;kcH*5+?a2YB=BCW~MpHpiIdWdXYtC<@tw{p@!wb zyAu7DYP=*>SS|fKQLOj4+blt7g(5FIr{B7M;zeMfngf;5f3_pR)<3I9iCUL8nI{6u8ZQr#lG81oI9C$hft+z}n_WeA(Ae%gJ{e>&OFmG5 zTGX&tKWcDdEtk-)TL2Muvl24#1kC(5W5^NYgmgL+;dVMd29<5P5@-p(Sez&9UXn)y zqf&SiR}|)>dPB#cOD)MgswmnUkTYKPkUYJMF_qIV_o&S1o9Qtqzm=8Ktm}Vp-XA7A zWUWOE>4{Bn|JQ}1w52qV-VUN}!G#--qVm4htSblH>_F4lU|Ua} zLW!KtRcLamqp{u6DuLY+a11K%pXGs&P^xyw(f5)xF!43pgHX65BrK5)Z@eVOs10!a zf4Ue&oi=X$uQOm0^(JJAF4M%$4Fd3*?={c#1@tA4YHrssi}wKWhXqCEZlzwS3@TpZ zG_hqentLH6SX3dqZ2NT)NVGR@ZqB6Nf%g9TV=pK=C)g__EPQJ~KY=hgPiwqu(B0nt zl3rWrIm&T=jJ(CS%yg^=&5KYpaVC+p*c8{5j_E4f%3o*KlcT(@xPV#R4lqpyBZ5lf zaowb45t{JUn!A(fy$to+e36SHm3;{ZZN}wZpS+~uoENtVItPq*lf|i_J?}|8Yv&uz zJ&4QUJ|THRPUa!LwaPw6mn-Agbh9HakVWPi`o5XU`IGzPm;o=>oHbDE-6k&kpf%!q z(|zcXn~X1ey$fsKQq^P~i9Mu4b6@_OaMf!G2g+2{W!SV6wJ}=^+kc)_ljK!S`zZ}Q zv_ep5sZ*s^`LgZWtgDADN1?Ng-46(QvyZ1IBRX2+-mPlp#6e<xYRtIPJap%pn}Q=s|Hbx~ zdRLU~g%jXn-!N$R_XvEKJ$00Y4(Bd9#lt!ujLGO+_&&V(Q|%w%LWRz(Ln|0`rHn{V z2(3cL>m>vZS>GPYHL22YjdAFbdCv@bpf_{D7FTAIUkDd|t#c(C?sK6eGm`v#%m>y!~~jS*0N zrWIbgNR9l52&;KwGmY!S58VB^vH3?BSS7GpLr;yGoa^n7cjH=oA~SV6oG;iL^cW9L zYLp&nsZDu6*GMa7xW#;E!f#2^`cPI_o#Og>HS)!Xj&|JWMi*&BGyWmR%9B91&U52~ zzJp=;nk`2M$vX>HlehT?r5Or+4Lh?Jq-C}4uoe%q8e-3t+EdM}wJJ%~{a37fhjkb| zc)B8ZO>eu{%El9zbnRl5zLHgmJ8pQw4j!9x@4R-N{JWd~Gba0aYesdtciz)})jIe} zJtP*4a1%^mLWxp+^U5&FRe=wHXly_sVXQPMouBzd5LHZx>NhmWnHbM)57It+tEhIy zT_VSi$)t9`=Kgfe$=j%0o!{l038ux^iUDunkHmoCK3Gy2o}J(ztYNEcVecID4=iXc zo!s_0wS1NnzyPi2D6Bmkjqri4Bbuv;R%Q}b{ zFGPrGNNtb#DE1b5AMkwpl~h&xpip)VQ+!gRlT@?7SwP}Kr{DK;T>^V^FJ2so;w*@{ z8IbjwIdd)y%PW)}b+HVJ`9sBY ze6P9|P3fn{5cA!`C0NxxvuV?oRwcB6{9v zAopl`&;(puIUm%jQKN2ru$7+VDTymg?FwIl<8PlYy*G+%M*WVr75+SbX~K5joZcD& zDa(%wg0lFs#Z^#WL+|`rt@(!qayatnCk)CPf^#fPzEwkiz<<>p^=E_k*rV;CE0t~s2T?Fj#bG`@I_&dfL6*~(mH5jZ^ zUx%xy^4_E~1=YFG0jY+scvM(lIr`{+Ia$zyTQ7h{7}|T5;o_ zi6bdXr?~%67F?*PQqxrM)_{a-5*nR_n$v*i>XrzX`kdJUf12sDz*8qRFZ&3|lxEz! zkq37$Y;^ZR;f%sCKM1PhKbbh!K=}8898zO8V-U|SQLkI<552jt^W!C-4XHaSRSQ$e)4*cXI z(xtmAdxE(eK+sh8dBP(%EXiX$rv{*-&;xw?i({s$4FksU>d)GXwZTl7K~UOzq7rB& zsmf)r;`b%^RvZxp5BZGi6tHScI-!D0tfVI-nhk@1q7Z~>4EaSA-osuhVuUo z0GA>y@5u**(>37mht}|>P|1v=$>(=t@RXw%W8nli6vP!8V7!D@lN=-OtdCb_kkdL- zdvq68t8Bi(urN@zdurcW4=__a!slQ+j3P?j zV@E$u&g4H{-*3+Jdm}r_)~r*n1kSq->Kn_|%$zy_8K~0^AnQA~o;aGU4fM(VY~@R| z9^|53heycD+wquv6;9Evh-FzX$2j^e=;x?U|4tDTFwGZF2U>hU-?qSxNw_%uPOZbb zs$2l~>!>;x>*F$JjsDXV_vS!)6B=v(9JjJyuKBSqsqL%S^HFS!jKi;!3t(z_DxJK^ zoVcm6YsK81E@dL|z4f7JnT=c)u$Gl^@K}L^H()9Mq$b0>xIz1Vsl0=!Y(VzPl2OxP zyZl?)ne~-F4A>;=^ZqQT&Z4hYJ(GE$Xh8*3rnhEI0CCJ%C#BqC%@nIbb)Qe|0KNFY zK4cFrXt&C&gDp)tUylQyx@KrzCQS060sGfMcK$AR*pETR-xm9w0egK`9=zNunaEq? z4n;Lz!WwdfMC7B7^UrdUGuA?VDATL*IgTC1=R@8?mOyrE{5IxRmJ4$}Q-^uxWhg=o z5tGnoSIDVb{^7GM<~8%A7+ARA_o4dJ4w;oe`LU;c60%nVUs?+dyQ9AG!W zVsCKn+03kq%ru0pkLklS`QPV;4dpF78Xsw1$)6OJIXAS-1t|K!2W9jbSW&}T`jWlj zXn@xcbf4z3<(kWWcuGiIh??dGRs4Vqs@dLTy7+e4#;?>mvm)ivfrd+$;qAwg>^Q$j zJmm$IBYmVrHZx(A;#4}Fs$cj>9n6H2hPR#bA0BT^MbsSHE!cx_mJA&k_%&qse6`;) zYfzTBC1Dy2YxK=wlK{0}`(pFAAPAvT0(>#(w zc;I!gxoi;+gPi;9tN4jS$1?yjE6Ef1hbd9lm=m)aQzmUU#Nyvuk6-K2J#X{BecNMc z>OXKg?(Fr(AC*I99J_T7s_g$WL|>`WMm~u3Z3ayKKG^OCksWaO=YJ>Qb85B~XZ^V8 z0Y{VAWpN9j?fJp5cBJ^s7?fg`zkuiN4;|gFH0O29rGNJ9^sK&oE+lD&WO{qkzM`2` z(7Q`aFq@3hN2r6iMt2j`8c<52iek^0U88301yI|D8Ity`($2m!SP>EP>@a;-0_|&s z#8$l}%c(mGSARsMyRP<)geZeO;P8j9W_)yQ53QswZ&i?JrPBaV6Oa1mwr&I(LzSvW z8SEzGd*>#yw|I4mvJLOUTzS%CD;xU*bQD-w#Te@CdxaN3Lq7DhgA%ZfqVoG11sNCb z2c01I3tugoz&AWM@RP99WT{LZVN`xgM}E zN<~%qYgDdSl@Vp&9f z9*EoDpI<+Fo^`^J2?|-VMdKat14)$lRPhoh7pDJvp+7FfQy=^``KJcd}j1{#S9eb7|dL1;=*`pxO{+&Lv6F)wyczTtBu%snyS7d_MEFc)-d<2 z=oKr=DqXDPC;8_TH)9npzG%SW@{bab<3jHy*5q56gjlM*)Z}#1-97Q|b}6gXF+PD} z5jbT${khDiIEa4^d4eMma$d08{97g9i?y5)xcDe~g|`G6c|<|(Pf}C~U;buGIG8<9 zIksqz^*?HdpExDkbo@hQ~Td7Ig@O;5}=KR>&MlebsQ(fZ25V{t-6`1A(;yD)*oP_U+6Ny~7nYTo$MX z;8pW!)|*ziudJm$1?=LS)*H=)tgz_~>aClv(uYn-VcK^+TbXBYiONhQrTBlK_!mC^ z?anomUq7E?W9&QoeWHN`V*ai9l88|N`0!POGj(k@^wJIn3??ug*_hwuKCK-DDSkD0 z0eg<*bTwOsvzvI`08DU^I9s=gGy}D>@Eae=d19oJ?~Zyy4fjouzaSXCK{M2^^F(p6 zZxR*(K;`E-g; zc=Z|szW_`#>1c3ezVO&?mB5ra=k`g>j_fcqhkhEDN&+EKJXd+tQ-4py>Ay2m3Z;>_ zVHfLKtgX#fd0<5wj)AYXZBN&QgjIEE$pXX@^|-dAJ-wIA%%8p*oFyGpvP_M6?LYKR zY(1mX|HtMLG7i?_h@-v}tQx%U>Oa|VnTSfo#yIisb@4)E%-O4KLWR-ZvguMvEZ{6h zO%GVi6{{K`BMv>w$5DnWY}L4trrM2jpaOEgN)ggMjAaPMTVvo?k1x zll3aU5@A|92_zYw??4w)HSkqfHHFlng7=eY|lynTcWG|;wQ_^{3MS`9vQL$04u7%d5D&(=`k1Z;) z@1l$A7=M0TZTym*1HQ7j9;n6Z-9flllkEyI?^%06sxYzQHochLA+<*)2gL<1SHuzI zGQ77pKdV01xP%;`iq`@Yr#@<~)_m=Fjk#96CHIO(Up#isNlw1!Woam_`6_sIm|FOR za{SR=toem81?x2HX2}d_&BKK4$A`QzFZv2rcL7GXcj+ZI2&y`o~|nFVADkhG^eM|YD1;RJB&2RJ%6{lmL zHcl|WW2_l7t}=v_qgjW=8p8I*^huSW@K zFV&LuiPbqr%qb4g!`NbUke53c__}CQ^qmZg@EC%?0NSD66*0ubY#if(j6;SL561?Z z2?v|%0#TQ20$jvuu;dmlb#DV!@ACW4Nc_-(A`Mb=<{9X}&Y4WB>p42zE&xsf!G^mr z8_ql1XDF}gTP`iGCY%M4B^SRxFOq+IJ#5Dxy@-t-oy};+oRR?mtG-Ps)X|dR8C z+I9rQSYa(|LuaK-@3ZN8e;a8KXO|c{A23>PXRaA;ei0(<8|xumFQ`qmsT(?$1rEC; z)meXikf~qFZ7_9aJKrarqO$e{*r1=wb0I`l!RrAw#XlCQ1W3SUYP;Cgts?2k0|ey1 z(zsYkl04ziCiB>zxwlcRz@gJc%Z7`pWX%blcl@J6)iEA!^9=Af=OUN3#I8X*v{hMq z_pAD51z$x54;pBMhp?K!S6}Y0WI?M=R2#gqg~EeJPi=pJ8>U$iul9VaF@3^a^B-@( z8}Ze{w`&Y30p-A2Rb|Y0 z121;VhCM5{`C#6YCUo9QLlcam>+3I^&nG*_I3sGRV4RT^UCy$2v7LCnMG)}4;nl_{ zdgo%SlWf-!VV83dI5~YvB3)6jB$+XX)yVZF0pP?7%R59}eY1xlm}z z9pL1!>1oWo6FS}v4m@)}yhl0lQD~7hgxv1)|7<~g1!XM(nmcQIroj$C?xNQ};ng#k zS~cesO?VE%GvMHIQNwru1|OpbFp{j?!aDjxb>*&zlpL3mk5aG0=z(H*eYw(jD;Q;$ zDvs-Q2L`w3v}z1^VF9oYdT#KH!k9$I#_iYUdckoIoFamn@En35rU~Ncd`}ifP}<-20d@j1%lB@6#h5myeL{B^NAjR zP-DRj5vIy%Un7pJLrh6iix?}b+4R&9 zNtn|9?F_lUcRp3``FTfspg|x+YVFu)L_j5u6>JiX?n>ZKpr0-W;;P~3%%4FaR{N~< zeLP1EA`t^utTCs&s=wBU4Z|7oX{ciU_1+7dn@Q7sOmsgAAzmusT?0rDU}-Wm`;5|S z>=q9EQS##$QZ;BFXbR){KT3TKuNo)G?@*8v?HV5}C~%-R}?g zk~n@DuWT7VOhvig4~j4Y`7&S7!pO19=OLfy2;^RFmVt5OUZD#h;upf-J5sbBZB+hm zkyobO(@BrmVo2`_0&Mo;y#+FL*%5{MEdca_@;9ow;YKp#hZt{WTZhBz(~qM)Nem#; z=7#@XN(pzt@alIw+u8MyEe1Boq*km;OmWT<2=%);=-5&t$}FD{L7kG6EQcuffd#ZS z|3z{z#zgtuh{pv(~o{P|Z!rRYS!d`|xf|p$c z>WR&Ns1TnLL^bcPVD2%!*%!J|Tjax7)Mj#(<=#xYuhSYIU|zW=S4_B*Bzl^j?$Zej zEpiGt9^qGXk{eNhJlGJ0682^AB%?3F7`)R0cEgIq<%fL{DO*!FYtx7~+H>1JjQ3lv zT&-jy_ThKDT6U@}P@t5L((54=GpBui2r&)z5U!?>D5i`f12K`O^@ z%o!d-1FIJvBt_9GcN5tb>8DDzBbwV1GRC(!2>!u2oP@z3!W+*Hf(f-`mH+3ec5R}FEGVJ$UF?LdR1*Vzi z<52z5r3em&)gm=3tC1r+jgBJYha%}z6{1-K4r}31y|*OWpUIvuxVq0fpI5=GW^AoP zgCu8*I5{a+{%ts70e)>6V(#!bVhR5}n!=-lIs9+Gik<1Y=mQ`bX^dbWWsRlBmtbc0 z`&h3zglovnMaG&kjz{HT%db*7zD$LhcZGmFl8-ea;2Ezh;;XkAIX&0*Ef+S>^oI=U zG%*=a0`*k&4zM+pz1h3nb0X(maP2dzAxjV+Qjz#m%O8s_^?%JP+CQ^IKN}p^ov5y? zCBIsG^rt60zueX7x&VgVqbI0WS4F5;xV_vtx;lFNr>)uPsZQJEOnmlD)v7!V`^{F; zm0t9Ng^;>y*QD(4UBxiwfa@zYk@u{oUG%APu(tMjuH**`hp-`jx_9~+%!=~R z9Wu{aH63$apN^i35{uG|k0|M#y&#GR5b90~263-hl#Pf@{k7 zhV0wOLSLZJ>=|pZ%-u<@iNVxWSZ_~n%k##a(m-xPtD&Kawhc2;u};{Z73fU z&w!s~q{Eh*fqgQp=ZI;i42unZW^s*QJOZH1N%Ao6`uYYIYQxM$-PNGlsuut}Jx8u! z(3ADwkNbaXl13+!Z%fwdSlY7R5n$F)3V{<7ra>fK{+mRIZ8=c0zA#u)OnBUWXfD>c zlrpU7dTIk~zL|kqal~!btygp;MS@-#s_7K=5BkZXv%jWw{u5N8X1=?2ED?xJjIuWF z{3wv7S7?X` zNm~rDw8kQiS%~SY%Y1Re_kYASiMW?OU?`&SP71ckU7AXMLW$d6L0JELWWv9^*XV^K?A-$`+iV{|$=N9S%;i_c`` zpf!7ouV<8;@_(vSfLl~-2A_{2zBB~-o54zI^CKX{;o8`T)L^r4{3~*e2|_8BJv-TS zCciQG#tV51&2iSlX%-C7P?QGoIf6(En=C)0vU|X*)iG?s4M+^M zey1#X6xWVc`*7KOTL6o)Y_WmE4?jY+lViM*d(%UgJa7?~39*`X?d&M@NbRB4FlA7h z;LDekiY#4?2J<2TplyAEKv2Rd&xECp+TX<}{FoJrlIR~l-N|Jr2COu|Zec@}V@sXR z5Vn46TN&07Rug>M(hwrHMm;0Rl0>IdZG!?HjR&+Ji9q`Y+adC2AN&dSg6`VeYRMHF z+vhC#PhZ>&61_PRSUkLHMBU|;dKTUF!+A_{FE1CDPD0fJrP&&R@4aC6Rymw0?n<;c zxP)jcgsR^@0(FyTF~{aKRupi52Dmv-q6qA@SC9V_sYD+mLBX*P^tqNjr&@EzAfKjz zSPphx=gK)-IJ1iefJ**b@sC#np1eV5D5(dN=yMsz z1Wn*m!c^%wS=IP3(@8A$c<}FYnm$*XTtV$cf6rzJ5Dr-}ad96OTX&i`M#e;_ zFtD&t((=)3!L^}lKLy3@t+Rqs&Y34hAkNwH%ri4GdgKpGU;29RFVF7OagGQ`>SBU_(6*W z?w9kz(J=lG=fh|bTe)!LtJDJVOT8~NBMV}=e80-6$I&nme=4Omb*h<{Nb5bMiURUv z@6q3{%v)ZwVDPq&_ggcuuC#DQ65dQoVqG*XT>G#HHr?L;Z9=&EK@mF&n{$^h|#ReU9 z{)6#^&|AZePuw{r#7gSSZuhF$?-GZRXDK-7WuX@wT&qv4IJ&NCIJ00)Ib|pgF-2O7srFCuH?u}mT34u$W6xc6Ynd`yC$Ml$Kg%{c| zCl~79sIW>*%P9tNZ)d?z4&N0f@CLB+_h9i?Z^Col{xvK>4HTSL=y<1|qVdYbW*J^H zZwC!UL)O+$avRf&xjNdKxSx=3l$x}9NZ}D;v-|zy8#P_2lvuU+mNrq)?OJYb+E*VT z)+vQ$LI{;Jp^ooJoXfBn3$D|h+@S1Xm1>+jWyAgCPCLu#*BLS7v1*Et=X$pM6ZYCre zdz%b<`6`e<;^LB>lLHF_ zclloX_GV&Y>f`OLrl$7xy2sDUx4g8py0%(dTf6w)=3aU$KCQ%Sm+n1Y zzwrotx|U%v`|;P9mj`Ip@^-fh)T`Wjemmju)J&M{wPAnGGr2n4;#Euj)A#9=UiaP% zxcpW8Bad?7&g;cc_-Uh#VB})L+Ee4v>qX(_eZM8K#b}@;P{yYG(O-erW$5AW=!Lpo zIyd~m_1)m@+jN)=D?B%2bZ)-M%LoTcjUEew|Z%*_Q?_FlH0Pgb}g5L=(QPY$H$e|6E3 z`*C@8b9+9$|Ih#XdDQZGQEiQ3)g%F#BX?zym%+^b!G)Rs{HK6fh0rGGOuE7J7BSWQ zc>Cn=#y=VPhVsZ}ny@sd9d(7QPMJeFt{&djNM1bU^ReCt-x7g+xAiLfJt{qg01|PH%OV*_6icCs;JnUPsmt z%wSa9Q58vp`yJAYLy|&yG*GG*M66X-X1Ww&|68PC0|BmZ@uloserK=g{A?^jV*Mdt zXr37d;owYGFxL?PZ_L`Y9AQc!u~afQm}`NXNSUt3!pw_g+TJzmHFt10N`ivUGpS z-fOr|a>L^8kC>Yz&t3L@DYUzE-3Iy=#Q;j z_z$P#TGMP$(F2)gTJ%jRqZl)@%%g`QgO(EfH%pi!V3{_(rizV`;enZ!&b~1#{{RY>Fma@z}(P_Yq^z@A>@Lvxo#)?;c89P}Xp!MM+NBNDEu538YTHJ$@o5=9 zJwmJJGrqMB2za`|KU`Z2d0oaG!)bO_!2qxPzWWz1|5Fw4^8NSE59PbS`$6So4#guK zuF^jC^DAws*pIy>C#g9_7?vtzVPmGd zS%#hBi2}DhjIKAF^yFjFH!?mZc>jx=lWT7Kf+$5_Cj=Bt&_ut8(ZMa92i?zPM-y&f z-)ax&yAV$%wS9J`Dgu|a!y~&#CNN_hxkfWL`SBRYreTixjNEW|l?A$GapW^slJ zuqWOEx}0i(56N|BdkWk7lHwqjZL@QFa0u~>0EMOR_8uPm>M!#f`MKQBDs~as^iqWE zk$W}=*>N9wVHi76;1x0V>~mg&LL4$XX6G8OQ$f_?Rx8W4+3beZ?c<7LIOD;Q$V(?D z!&zDU<7BkPd`7HtmHaMdp9jS~7K$jNdw=8zH*${M+l)Fik0+M;zD=k}x7v=K z;C|DOU!BYjY(30!R$8<-JV_AEq|O?BLka#>`oo;C^>%Elh25E+v)j(wztO_YtZzZh z-U;t(vY`YU?O?L4G2z9&u@TY=wtex^H?}64us@m!hr+%;oP{gZmRFLf7j@TcSL_6u z`II58sn)We^mDwTCyYH=q*>ryPByU5jI0XhKL+Zy?Wh0B463U}vbhWjxg2^#JV-~K zm;7=f_8N$6_}PITyR|FJfLYWCPy5c>OlxSIrd^XiOQMNUK*>=q27eH;GMxAeORcuk z_T^tRL1g4C28K0(WevCCX|*?0X*P6VWs_xUui(4)P^&RRjIC&f&BPPPZgV{t5bkEM za?b6Ovcj@&JOfSGr-~yvNmdw~sI{$ftmj~_OUCug*#3;7mp6&~TY^F`amdfX`BVu% zYxlmc*+7xRf`Pgyd}a8{uQXC6ngJP>g^5|C!+wLu@8~V{Zc^s=Wm5b~c1kpw+EL_e zPPwC-ubq@A-D%iJZOf*t({hoSQz&L!js;mSSo52L8Ug6rL4L->Qe&Z2{1O}FzI(Iv z_iv!du&uXD5M|~s++#BPcXqamDJczb;t8UJXTZV*n<7i%JZr6MKWmYNXKd@M<2Ikb z!%0m8-?bd)f5t9@(~KeYpZ!<)x&(?IYexKU&4P;_gSkD*inSz0g4xAtGH%nSorWP=QB0S32(287i&2reH}Kn8?MZNA|FCYsiOj9Pcu zFS|{SYy*&%sqCHg+94p@U+y&7H#ft^;P+-3hDaGN%Oddk{E2KZW8Y=HjPp)Rk1&Ml zS@H5D5m9sYZa@OV`!r&(Ch%c>-?iGeK`GuTBsNjs(?g zExn+oSuM9x6{FA7zu#VfDQDN#0V+t3-sBYb6sxFych(_qX~Xw`H5sBbXUK|kN%cS5 zzG!x(gDtm)c4XwQ&ha=_cLPM*a_yp4B`(%;i_B>eVadA|p}}WWSkBkUvx<~TR?6vW zJIW*6SjBxkcR1G8#hxleDbX(CZN0LiD;6T7Yfy(Ii%!>1ZRsoWBfZq$PX8dI36!AP z${h4Rk~wPPd94yTGm9^+^PipT^x{)zep_9t(%CK@<3Nd6`*&3+t*~lwmhKR=Og&f# zKH4cBT9g*Sxm1i`_Zuw}q0X&fy+XsrI(lV*;kebh(m?%+*o0k9g7{Ln@k@N0;Go<+ zq=9fR+9VKXHef_D8|@9Q@&xLY2&@|PSif*8$9UDnb&}`I($JfK%6v+fgTj25H>UqB zh6JS$|Gm2U$R7deG@cfxyA@GCbJ_6ML7=1%U0;2S8Dd^rPAn*K#dHzLo4jgZvwLzy zm>zcN>jineOdE?VSvpLL_&yi;=8k!`B;bbPGdm%~fy#s}lr3$<)DCSwVR$KAiKG<; zjp5W61v#bRFv(aKn-Gl{OPfh&RWgI7FXqK7uJTpX&PB*xkMhS)XmwL|*F=^$n8)J* z^VQl1SO~%ddvnO>!x>hdd=)CE%19l#+J$rasiH)#`$(;b$d+7t+5%YtAG6Wl5X5{E zka$d`=LH$6z;Xq91!54b8vDos`l9%XzSSQUZWjGSX zJ2GFp{0HJ3CUGo%2q(C4HjGFTCV`TRLC%j=1|_Yd0cBRndT*U36VZNk-kP0ms3_(% zLeb2R8kBA4@Zj>3SpEX(`1AD^NckIXtDF5q1f9#m+Ezhzdju3D>{&FC3{RJtC~_c2 zTXk*1=LxkScJ$Gu`Fs2gw~Jrf=cbWvXL1;yf8czOI!{`2e(v_~j+rZ6SAH zQDqQT@lY7#>m!3xpk0-eKw4t^z!nGmelt$qGa zoOM<|hB9cvArncHyx4YHKJ;3wOalx|CDOKFRMS%W>p>fxSiZEr)vr=>$4`?aKlM6oabE(YsKkIORBz7k;WVV(Jn4S!?hIZF4mROR29L!PYy@Iwq)gIS&a6 zw{)QcD}2w)*?Fz!OKXjNA1R%VJXd1c4}mUj&{;cz%x)uot|j7B_kz%md-3ueK<00T zD6K6Go+~b~&c_yhos!VX%ISTqd1(!iUuG@C?HCpMvds9mmM@9qf;F#&_Z|3I7ZRPXjuWXT2kb97pCWL|7ZM(zs0 zlhQilP@vmVYUqgra>D9tZ_|PGTmlJCQl%73))BLf1vqSHMHb5MVXz9WYwxZjDJZwMcn8B^;Q4*j!|qz9=9W+ZG00w zPv8`akUiI0|0w{hddIMrF&@tix-<7rkJtQD>^#@pyycnQkOb&uzRKTN#z?9>a-`F+ zHUG=R$z}bkX}+1;dJpM5k2_U=lsFE}Y2AhszfaLFNs}KX>3y0oY40>0qXXs62o4y$ z`v2jy#bSm7N{-CRH}CQEoVyOInItk{BgFT4&|7TlJ1wuF9%t?2XtfbImBF#`n$lWI zNx4+I+%+cUmM6i#Szpv6?(nZWUS9D1I0l?Z2#ge;VSCQa^fo^YF=5%6&3OX!B;vu@ z`V#h%cFIbw2x~_yrWKlyY_5$f0@Tm7e^bpvc ziTc#guGv`Vy#D6nSIRPLpp_agJI09n#ThzZurJ0WlgBJ`Gr;jaz^D-k0ogbU@_eCx zXn#KTF?oXi?wH~B;ZhVKeKq*>L^#b200XQv|C5DCwMr}%toCIpmgxc+!1f`8M;;ku zkR}I(68;W2?=l_#>k4tquM8#&xK)`fQnACnWy&yt43OA-xfcr`oB}xp5?-n}|Mj>_ zS-$1?oChfBkXV^EfP7P4sXae`vD@!670Tm&#b*lmiKwb*7w^%L`ewkR)zX+ZUb&~< zxIkm62q%DLGT2OJRT^)rd1n*LwrPS00;E5Scco&xHGOeX0Uz~S!#YxoR_zbjbG8}$^-Da3F++QwxQ-zh#l05`Vx&$fOvHyK676|wOcx6n(hw?#U% zi64suI2(-$#QXa&&-iZmWZURp)*}WYnXa?43GT-XwfJHoI|+f<2-n4>vRw5C(8MT2$>G_`7DM|zMi}yRE(Qe{dVgyL%dW|I~*m(1k;Ch*`Za` z@TwZR4Mtj-?}BmF44u&)L(uun1Ali^b`8bPn8q!YwDS!;oEHtb--%WSi_RiFI(U{A zNY|&*yi_$WkE!GB(kw7czWpl?=HpSfU>W@Sl zPnzP#;xZSo)oKyy&bg~zmK}5-x~SKx1YxTF5-D%tJI%cHj)m0(&CO25`<@IrPnMXG zt2NT0j)b(CZNr21(Yj`J9g=gO-2BC8sUJ&%n}@feL%s9WmdO(-`^!Jfw88Kd{i%0t znK2-~10Iy4{=qn-d<@6ji8_3F$j=lTJ_Hq{V=~kmiEB&-inhHOnIbzA&i%TYpa)b_ z(l2o+3`}Ue*xw#h2NUzi<$d?k$M(7U89-6lpMTxa!86EQJi5TFq6}n!w}<2g6dT`9Nan>N%8bOvjqsU7sNIsy>U}wWAfjn zyX$DX9CYxewV#XGS+jld2vEq0%iV6Jl&@F8ox0iSxbb8RXh`j8W(jg^oEH<=*8jFrHszR&wsH&=#d6M3di_cF2P7FYdL^e3EJhNU6(;u( zUhUl6F#mB(n5da4NDO7udF*_{;N-z`cF+HTy1s`tLLBm~hw!!Q<<-gZ%7jXy=Ho}9 z_8g&OBg-B_j!?*3lW%vus;MQJIWzIK?gxY2`5p3VS`_Oqf~_S~hM^q7G;zr;wbt3q zBlZE~1i!#)3VdWRa8#_UZ_ zRfcDa8PkUL3y(2see_Hdf2 zt&Rx@-nb-x+v|%5R{}VFG=UEu6fS2dm>`m277o1hOo?V;T%WydRIx7`cWGVIA0?H` zR?NP%uVB*7KQC?jD&Os#0QwubwgpFJN!qWZD{TijwY7Ehbg4Et$b{*VUZ8*a>LrW! zr4FyGI1HkbK8uU&N<7pDPlBkfSzbrRF){{@{uFkC&eLw_mgPaH-)oKY#pffFU}L3kP$n)GPr1CZ7#{anklV zN3iZpKNP*WK3=Iew$4_|w)@PpYiUAl1y%a|Ip}i`qP%}DZWBj3&;tB$dKqilP4@w} zQ8d93jJoh?RjXh3GZM%T*{1EbyX^$qsyp9R+>z zXLIFNz-(|f4p>@{5%E7GjrV-9f5WVYW36c-Lp!7qK46E86v{9qZjGArG&8U+ObyyDf*52nvKe z^zPRFg$P@i>Wv}`#DmKgzu&7CDmt3pCTsJ@hQzZ^CMp@SE0NC@pjNt$C!rfgSyim^K zHOlVqCwCRm|CtSB4hYzM!q2(&21NWVqPC`6`7*7PZIp>?uMD698A*=m97=*>(}rs^ zxc!x0G5q z-K$#BK&@zFLBhcA<)HXK-edMz@E>)2vIeRwVqY}1arY?PsRLUib`Rywda z?AEg9fDASy+nBJpd*K|m93ZoZV2ICx?b_R@c0fD^_sQO!6{C4sP0&ylw}#wCHqFEZ zae9sN&nNfFUtE363v@2G9v?1w!Byao2PnGyG9dX+o zzId6<@r~xQ%uW9K=f;2^uz~H4sD(2x7Hq-JAQRNNUEHzyuMs#F$o(%ofS9@z#g&*B zc=#9pq*p&|?ankn&(Q$GgAWNP=92D?@3Vy9BG2+?=%&OFC4>0OyaCdRg;6K#Mm`YA z>l_bqWGrA36sYvMEQu~G`v6k1%OJ@M)W>o>Hg(vnPa0{IW0YV!28WT`nJ6xYLx#pI zqvJ0!R!3t3tQX_%)6~RaP{*TZ{;7#~qasIoU*ZeU|G~=!pF#|dzi6FHG%JhLZ#}0L zH;eI{ZE&`fV7On_EuJ6ACmq9hXHb+-{3!oE>{pV$~O5j1^`qW;YnbqN1%y zc6ZY#k0wuaF(Zwv<%1u>^_X@3#{Nj6nNd;4%0vobko|-8E;+-KD_wMWQSQX( z2&aw+K&N%9B2;EJ&|bNBjLg z@@PWZdO#0}Oy@l0U$UbR0e?dah9p7+2AvzICuipbOU3hp6`gHjzepM%Q2gYVMZ|Hc z=)ML^Pqm-ApV7o?pte8G*cq;vD6oFZu)lkU?1w%4?#NGmAX=p5Ymi7@^w5c<>~gQ6 z3$7$fd=Wo07P|~Kh-58x4N(!i8wo$f2#(ib+Wt2)b|ctqs5oc9_I<-hvpj7*Q9xZR z|E4S0VvK?ksmZyVC-Y(I@L-lGgZ}4dUdH8?abhFPLeO>A6#Hl;C4)sDn`#2yju1Ju zQd#lxlu4;!oRw+q7@l-s`*&Kh>lhG*?TU@tAj**1;Id10mGz22D`?0iWZ;|p*zExh zN2}35!S#Lz2J7Gy)rPI1<^cNsYeRe=8|eqW2_A{98D95>&f&Hj)w3TpvTCCn1hhX* zB~;w)r*LuT=tF9U1@}c@c5gop-fO0dWL&68@YP7LS#ynH@N1UD^2^*kLHOt+w6Tyy zq{)^*npkvAjt*^`mUpy^N5L={OB0=;45$RBosJNZ)Wauu_XZL1z#m&rkhqDxuh#n1 z5uQe;HauS@hCez2B2B;hXkq=YHvnFjVsD#YEUP(7cfm}&S(^PfiN&Pd0I*^*|ajSTDoTdw!XsWE1HBnIM} z=;J^nWwEmD%?Hv5*L3R)(|!xyf_)zZTR?;%lKLj3=VY-F6^081QS&~JwRDj~q%Gu7 zAzOO-lUqzglNWnTb51<8?p9{eMyDLR*9pktc>|ZOX;?`lD92L$i9NfkpY4GQI~f19%uC?inA5K~7FCcO&d{n0_Hd^eVFaAvN6dtxb2C{*%)RtS^?6#_L3 zmszdBDB)CJAYPneF7{NU-YwDJ0AIr@!${*}vjBd0ZZrB^`8MJ0;a7jT#;V~@fr?qYKq0SqxQ=0Tf^Lq?DjU`;kX zPB1>X^KW04l#Ni(E{;%i%INmCJ`rpLO8lrl(!aIymF9@^Ih`FnK5N4~nqi%6GK%1v zF1cTuZn(D5A=I-Fkg3g4)_)%m5)rKXeYTNN%Eo{*z zMTqOb_oC<6`-?5}r3}K^E;N={p<$*lM)A9ltZt|0DzWL?J@T;APYz@g``&O6qdMvS zGeAOc-Nm)&&+@x*}c@=WUWM@)pvHZn^kRpVf1h=TglZW z-fHX?w$X{p+c~WbDW7I9$e6&F@wSv%cLufN($s*vJ8`c>|W zmkZ+#WyfLmN{s1qQ{`t*Ek-Td?3D-cCXuV`&j*IS-gNn50sNZZAn?i!t7eP*xNvz9 zt6)=oExSQyQ*6}iCPGVFvN{$G7tSlh46`n!y9|vX5>pb8^RcqbVAcF|e6GLSptlR9 zB5egJ&Rp#Q&(y+90%%mE-Em$&d7_|Xk+|J(KyHMVFy~Kn3`NNw29pG5!1EZCM@dIZ zO}G21mCPENZ<_NQd3j0R_U|suORAc*;ijrOFL(Iv79BrfVMN_dV~*`fR?X=Q!1VV> zfl4~gTrHcR;%fel7DO54$%Xp{u2L{OMKo?Fd%9gnbm}X_l-b(90~tTCB-S^z*8dFi zlD+H(WtJ8R>V8giG+q;Su)Z~bgd)AA8rE&NYK11%gH)Y zfU4167y0%QX#ZYyN-@&ZQ^iqM$_!d<0tQ@3fLAlR{B8r$+nIKyHi425G4l)teW<3- z3=CY;s+9o#RnwAeHqTC8A`gEHJFKRT6hC^{_}_oT=*!gt34x}}k)9y8;yaxHk5m@V zRb%3DM|g44{e5N;aYYgk))Hl!trVcEbtyR+cpyRx;?r_@?=IUsIdAFlV!Eu)67Gp1 z*%$@8(cjy{h}^;!c=Yl4*FRhz^vY{=DqNO;xf?$a>c^P7DO+k;s|wXBFgY1>`#QXR zJFXPGkXEtmo(>~S$&XH8VZz_@D7qycT(rd-6+uW+SX~f6CC(Etn#{3WfUM0#^QfHw zTp=1USmYQoTy6Td0{%-<;F(a@zB{2HqJ61YL;UE4!J@>JW;s+cjiyZjSIGKjGJ7a& zoUujxv!`K4&o2XiLr7T2A$m_k0H-?w$bH}sev;SQZ)eHWQrr@IZ1H7X7kREex+bg# zcBjp5Jg9WneE!E)l@?zMLgV}_XHWZ3XW`RX)q~xbDe-c^v_vH50pqmPr6V5{ZA;Jy zscdv@_T)H!L&R>2erqwzPPf!332eByD#QfVFRoF1p3~IJ+UR?=2Q!oU%APn0fv^QNQjsHg+pp|?Ufhc@nec3()_SE|4uA)z9n^-U4AqJbp;PkOT1Rf* zjRdmI{r%@(BMV;jk?9wS`kwFA(1QyZM5x!QSH#+z!E=L?m!7bu1I!MRp;t+OnA51X z*_@V@Tz?$gkMEP>-|6IF1e&xlx<)<7cj_FO@$x}De1Px!;H%19b_g2(?c4!?cZjAk(O68Xv7 z;bL^}jPEv-I@71@2Gm+O1zEW@w5>ANOB$vHbC#$c7EIj#lFbf8eC*ESm2a&F)aGaP z%+M!(Yrfx3D?w*B=bi{!KLs$xJo(PKo!{JQvTMpYdWbjp+p*?0-PTc@uRQ*WsEi=o za1HY33!f znC`a4k}k(skfyuS7TpTN>tS!kgN10Q{Bs~PKeEhS+`__!>a5TAMSIemt*))F>t>Ke zFab>BQWN4E?t{;K%q0f$@M>>KlGrPKsSpb5AOfi4IcRc3u$w&rXdl0|`KzMSqelS$ zm|zpg9$d6Z=v8z?9-`LJ-`OB~Tw+ulKTq;GIOOBQW^6`7&!ZG=Qc8Visk`IE4-7LP zWMUzQqU7I(fr3U%n@*uK@o~QHEwQpUi-w$V85lL8B>3(__*XoGk4qKI--dxOl_VfE z&7mjl1?yiwev6pNX467&^mD2;XG zIwj6jFhffm1e$sQoV2Gb0f3d=xf~`LJ`Ov~kTuaRpDS8Eu=Idd=xeXBGwwMtzvx%= z#MMaB1w2q`v=9Tx4IoU;RPu^qwlO}(Z<8f*bYz1ua9~u!4<1(0FB+T=uDJT8BWIs+wUlK%@@96pZ3rEs|+Uh4Y2EnzMY| zK-RF<6X2k~G4@9rQ5K-g6cH}*^X4_Zh`q8}Df`({du~`F6XJT_9CiyNX_lta{{|eA zEBM`#u=H%!;xvdZ?y^J;{^y5QOgVY7n+P|)Dc$n>G?Jp* zX5~VozJ^)h@ZHHShnLm3mQkdp$Sr$el9Leb6fTFzza8%#a^4uF!>Z$37i4l~LL$#;c^ZZ^pxcs*4 zGD|!bo==+ZNL-g_-c0d9!M0Lmh{YRQxnQkmj9u~mL*$l<(xl)|QKz$ikm($;clxbA zG|9Q)GY>2$2JUK4z$l_(a>0^367M1+6h%Truni2Mjzh%8Oq|t0uc9uX!>NMiJaOMM zlkNqR6Yj)_J#*_|r*q&E4b+Q*R&!{*-U!|&Q_y!znoUiwK3Oo?D|oK61_oA4Z)^KvW}BtwrGbp#$l{Ob zQ0(K)M}D8)+)x(BP1Yl-`tV!7+EH(&fn|BXs&qD7 zV+dDy+7WJtpZfK5twspT+wH(wDl!o)jY^|m<(*jkK;Mt~I?-#t!d zQNDwLWa(j&NHRFXcL(t}FR?sy_)`a-!y>IaTnH~k5hqFwSWT_^pdB)rWeFI+rUFeI=Xf0V-pyeC2f`)ertH9lzLBFj`po!g8G?t^}q=Y{C2 z9-7bW2z1i;$f#wXIGGmtA@fApIuC|$3mFqE@xo`UxNTxvi)Tcetq-*WP^7eo29SP{{E8uKE1+uG^?s#;PWq z^Ohs>IeTtIR;LEa1!`Ts7P1;)&y2hyojKZ-uk*VjZ?n{n?lm0wS`q6a*#p?rVfDxZ zLa$WEg_d+7w^1KPs=!qwmq&a=XE&R`H+8yNLW&7cxh5_@n0=9_OA^gupp4QxYT-m2 zJo$k%I+*_tlL&6_hYH4Jl*joGt;j1(25CuaIvhVAQRQJt!%(aScH#Kfo94=y9G4Z0 z(#fo^#l$_x(%31@*tsL54kC)OVq@;FOG%vEA6IJ6DmU2hT&fH^Q? zQyn8SFQAgAF*#rNS>guFg*k6I^(DXrs2Qn*aWEnOgMfd>WN+Avwz|B!7`Yg;>ovmp z4E|5pll4YS7%B|-J{H6$&Wx00n0DBrKdv$QTJ6|1`FhBX2ggU~2LH^C!tc~m{%=YO z|DKk@r^CKm7_<%!VvO`(diwvRrTpTah)6G=xQnn)K*ur_T1_{3g7WG%3a9 zD_^NxHPntd=kId+F!BLwBY%7K@?r`{$8#l2#*^-ZlH_x-3qn4d8QZDrWxQELdA*!= zc6pY~Z2Hbs$Y*1%WD-p}{tZkCnR#?hdFigyuqu~nxtpSCHPA4BT-v;L&@BIZg-$>~ zfR9h`g|p4C9}H`|Ph47!uqK;3x9+>$%rxiSZM73$&Y+eHxg%IP?6}1!D|J4fXPRRc z@ho3Yt>q0ggihshfsVg*Or|FT+p;1cBy`odb~;CLRPyt&S6?6g_F(6XJQXxGe00Og z7PI;KyWcAHX91I z?C$AdHW_-bQEOxh_-eW=_9ozU^+>iOD!KBb{F`9C#cjL1(N^2@Nrh;R2pqrfk5iED ztVAX{&zc2o*i|F$p$7c<%a5(vmr8JYMfL5SgpJeNK=7(ZAoN5mO}YF{7wQGQ6w6yz zdByZ<|H01x4G0r^X8eZF?0NBHl@ImDH)gep&HQf{B9((TtpkAc&z}>e(dO_y=&Ua79JEb z+RUxE%&tT-iOG^tL}#QeMRhj`pNl@V$Z+Et>!WsfWA?GyZJvFd=cJw?D{|)0V{pP=j~lM03pAEqMbt zy3M{N8E)TG9ra*PjK@eBfq#^T_Tq%t(RjP}9px)CJ{TE&uv9P_Ys)FGSFu(9HxvD2 zJc{v=J+Azl%^s#eP;wS?@cIb(c)W<_o_Fs97Y{bIp5a&)pinut%uXvHU`oq>ZYC)P zX-%SUU4g6jS<|H0jGs>SEUEoTWqtK-y2pD^imB6Hk;=ydP;|pCh2hC#@8O%@Yr+Bz zt(x3Yeh?`xj=wDXm_4RmSsL&EzoIx zKJ@UtY)>_pYO3quH_Zz3r9S*HEHB0gBkP*oJ-b{=bzT2GFdnM{@Q(j9>)Jm`zl!{? z;e}&*8lETq%eKt09~V!HNWVeG*s0(#RyGEx`|l20nX2Pd7f;{u888oAPRZ{}3lPB4 zxU)LwMH>Sfj_#Ts`9=z1lL|3GL?v zlJz=5vq&soj)GKpn|JQ_;ql9P+F*WQfV^z!LBD)+8uZw;HClwQuPBS+u~6;sK(smF zm37G#-4Aw-IGwM&p~?!`{oF`=%zq<8l%Ty=wvO55%)g30?vQEhO_1+TkVG=sTD!Wv zalfkq{K5wO(mmX6qTTXCP0rs?*0;9Em=4U=>g-eHTgt-W8!avV%(}}IZ{>gDb=2Su zFq5ZfD9$7KVjbuM)g@VLo#Y)0u=n~bP{ni%#YBQbWQZHLsc}FaP6=9?O#qhiP~VZg z>2RI|s~T1aIM#R(6|0zKf;6MEkH{$y)!Q;0v~4T=$jx-LsTW;;Q<;r7m5V&B{3iV#fT;>jEd2#n+*wznHmnc(7 zIoMZ{5KOu@S*AehZR1OAp!)`FNI~{SNfP5v;ZPc8n#ffJ2*6FfF57h-$Ghk9NASb} zg^T9ued2e4pcTNbCbaGFYqwtS`5qH(?0(6(2nYMPV>%etMypSY*00%*1xZ)A0waQC zOG2MRb)#v(y%0jd@~!KP(B!0J$+AuBg^_*u9!=YYNF5+1$3Yjlth#3L?C5E|;60zn}bxv8xeXI4j)!c zjf2Aw&&ohef{8DNN5->INl8N3u=#Zb_=Q!R)0nm(Q)*hPkr{!g^tL4N&k!(6`6_LM zmbR2qPw3^((eS#;G}=)NZ~C2fpYd9br0Q?R&u$;!?nac8eBCpN2HqD2^Eqz9atyMP z_-tZ54LKIWRL11}3Ka~ZLY0b>vw#1%Ow()i;LY(s(ROup%D?zFA@HCLBk+Ruc*|G! z`0EYqSVqaB&-|BU_oNYYA?T^kMdTw(JsGKiS83y^Cc$^`XmLXHlc{u+cp;wQjL73s z(!q9!2>~Ok{kI4=T`^B)%yty3cpA=eWY}qeutk2h2bwvxQmN|06FmNi_-y3Th^nPt{?dP@evf~qDe{0GoJFJ-Er z=`}wQbXR4l8E(Lo??j((cTOZPC6OQ*QIArJrS4f{WG#pd)IEv?cX6IR_a3-OZ_*$k@dlliGP`!S%PwnY z@tYeWz3p{KIIFhd!p}DV>j1m9AzzJ?KC(`<@xc^PdON7Glad8}?PcppN%_^7u*VWK z;E1u-VDcd%yp zccpBBN-eo;77)1VyB7brQD5S52i{3IBKq2Be+x`$TohhuC>|X)v5PP`YW{|T@cZfV)j>@7=0p0$k>btP4goOe$AQaHY;KynqvkL;{`&sv zIucvbg(`*4h&Hm!(HAJ|;Pf<@I+crpqcCco4zS*GGiAnn#%V<*UnnrTOZND!rHWuS zFh!CKf`nHLCyDEDkD5+RL?I8G;VuxmK8qze+5VX9)EEDTAdEJLd{MkzsIsj|04Re@e;v*!PC0kVbgfW|iP=@_HAIB8OB@*{bfo>1%p3Lwz z0*ZHw_i%!@r#VK@aD{&>rd`dziTUJLg@gpvNAhW%o|cpG;U)SM_fah4Ru?W&6w!gS z>8IodAr9sb%xSOg*kzFnKoo|YHDx&2r9z*1f&>GSa@rQ}JntxoG|Ea0@~Us5$IRQh z*~frqYcji|J>L~={EG|T%PWr8m07Q!1l^@{yg4)EU<;x!;54ryAC?c>Zn^#=Oy3Ce z^S%iZ^bjgH6aq&cC8RET43*UL0#CmFY4`MAXQFDecRL$F%HLZVck zMaj&*hlP1jjIn;A@@UiAU6}t8sr(m5p+B|hrwl!tHF2Nvcr=FoiOlqr#?y(kVKK{j z3r;iCSH=_HgmBbm@$Jjlh}A~|6Xi9sPFS)_nfYZ$GRzN&vTJ2+pGmsClrXhe$OuAj zwf7|lNax{3;_Q*^hiup2h-(aBq7*dhk-No&A@lam$xM36R<)*13PPAbKot zQKap zwpADM8-aQcrjYQOHS?kIsko!}#5VD_l(u8vcSNyQcsSR& z$E>S+E7MFE61dk;=r6xZ3=r};8@x&sE1+5(1CNZjDG&i(c~-#~Kg#a)0Qqjul}U?meNchZ?6SNE(x-^O zbeT5;A}~}%wbikYqEwKv%5dpX-k&}z%raqfa5=;RU;%R$!Tk4wI%lZgr_PA2d#6Ck zGf`g5)>o_U3vTanVR#GwAC$DV1qYDyUKSb?uN(Vv{qC?2pbtp&f^Vy!CPT^EFxC5+ zmvDxptj00>FvsFCEnXdq?XR;_99O^Ii7A`OUKoRsZT8C|S{N_6FM(tCfT1Ja@Xb0? zH)}>UHu#>hWXQV`x-B{iiYvt@Db|ZqCd!%@l<2JaA^AUN!{|QU=>T0zTan17Of8dw z>#T+(yZ~=W?q$F7{{LB&$eskTGkd4!FkR*k##sMAbG;>`iPhe-etONxml=*ayom(I zEcDu)n6iyrmqy6h$BlUNB_-r}a#>$Ig>$635|?lrQ|a6t#Zhby>%s(VX~b-~OI3o) z8&n!@Gy*ZDHGkP-Pc}E}4BDlq9*s4$O5wMz0L*}LKAo0N)83~I*kRE>3!)eUlJ+_F z1h3cEOYcBYcT+&-(%`}xBLu~7-clV<+m*z&6r&`_jlAO+a1+fB!Cb?ID*k)tnr2AY?D zD!z$TrNz@fg;nlcS-wQp<4k%VF6{qMh^g;0V?2E^xZK1pya>}TL0M8j=Ox_U&;KQs zTQ-2%M_`NfbhL?0u3?vk}cIzt-}#X1dBjK`~Y> z^n*sq&{$)G^y1z2fbBYRktPt6{KJ0M|0(M$pyGzQbq6W#Rw!1q1Et7dEmGXw87R&` zad#yibE-`1&Z6Feg9kczI(HhoSl=blf84~%gL6URk9R(p5msu z0I>bi{<4A=yUlKA9qKIj;y(gDe!J`e=2bg^Ou314l-{*lmayJ=uAY2MgFLSN9oa(; zZbS|i=#T&a9OSNtbJ>trfIYIc=@pZKMwvRZ=sM1%8*we^qOuWwRV^5(459fy=n7L9 z6Zv1Sz7H0SIOEUs{=Pd&BLx_onP_Q6eQI*7=} zey!(9dLrk(OT_PNv!1TYeYe2Mv5aXpw*0f(3$N6a=G(`IKq1@kkD>KFn^DX#$>M51 zru?(PdEP```>)9pXeweEC&Yjes|md3V|`*m7UPt$)gm^CRXTjYTFzNA%+mK$%kmuu z@HL$H>zYHEfd3ogS+(}%{lM+l6ipa&5LSr*VhJt&S+y-_1X|(R>IQ4iseLB4o5UT6&SRSubQkrW`9u@p)cTfyk7#1$ziLRF?&k{u|Nh?PUlm*u3ke;`rpZK(A_2}yA{9zb`+J6cq+Be!DJMa81iA|9@b9Dh;_R3JIq3fS_Q z7^N9w&L`DuAz?;~VQ1~ru}ho<>{eZar8X14J~fS!Hz8*C*4UYv*e zubO{bTk?7lV}pexHII+ksa!bmi6zm8=lngyvM6=QFA{Vhdju2T@fk6&1sm>dVs>~V zIDd}bbSn3E=V;ml(!>1LXs&!&nX=n`$#^L@j%R(5WIwr`Q<0p2KJ|~ZD1|;SH(?v* z#zv1<4%ioZ1Vbr&%CPR~v{tJheZ%4Vr(YxI@zOJQgOD{heWpHqP-MVn2{MZHY=2@u zr+<2V=(!3`+Mcv;+gb04f8Kd+Sz5&jOO$ZHi7REODD$b!4yj*av%t^P7(u$V(}OV7 z&J!+C@CK%n`(rprMBasIpxAbZJOhPT9?1yFhL!6z(pS`)Y|suCYZ3)^hmO(&#sLG* zMWyQxzeHvkCrhHJy*$qi*O;g+f) zgH4oz;t5ZM`AbZoPz)WSthaCC7Ft&ao3X4S$3-BcLK}^lAx@eP`hsRZ_8MsOvbb9SU0+FXI7;cPi*keNVxd>UW;ySigYUk`O|1s8p>7C4!|XD}Jg z`DF9%{ywWzh!S+pgcvc8xjpf74m<`fYDTac=8Y(d>W%qQP(bU7`e@ad-IzW-VnEKK z-kpUaA1Ne)mX{9wQ{apx&<6$ZH^qQ68|b^e*z?-9fktbgl_bc!Vsn1azPaNxFaN-` zlZGQMGB&HD`ae3+>@OkIJN*qv2n%|jQ25>azvm$VYZsrt8UitlYs?yOxl2X}18PnL zyq=A`X2Tc{+b0y~20-=^K2AW@TEoK6H7Wt(P0l@Z7O=U*3D>VA}(L@o{+&_|6F9eW}he zP{!%clJe|EU{S;3#I8R_umv~35oKXyd^$4dL9(Ox_sosEMoPzpPg->P>UI^ha?}>R zO~{a2e!=Bz{~do!$XLS67h%p~xQm;oo(=2bQzP_t*8kgAoC2}$yA`vd=w_S}dGExA zdE%$=(7&iiRPSU6g_{>W=X{YfwYB%v^TH1q09|VO5o~|Uj>pvMm=smfZD-UfH_x`! z-BDmJIea}g=0BJaUaL}AB?)bm$wTf_p6TKywBZYh7F2QhN#8Brh1uaLok#YT2-BFg zDgMC4@uWAl4Xg!La`;R#Xc9o#1lxJ$>Dnbv674}#P`|M zGrW=%K`KJG-vsMDN{RdO0fXJ41cwL5?Z(b_FEEX6#5ZLlk{2Hi@m0!gpN!_xT^`6m zNyOLb;mXm26zbI6V2kq1WL{AZdOh1R*rMRIU*ZfBN9e>`vP_$}Nm?QRCw(!6>r{gR zrm}iTTNG~H7vACb10S{gR^Ja?si`TXX=Ta2tAYBdPe0+KaQ?VDpYC5Md_qJ%-2Y%? zGU(~5y0Tu-XXW63 zNB<46|5uJ}wXSbx8$Cj2Pv2ojjg)LZ7{t@tQOkFG|0wR4%RJjQ$!#mx+UrIGM@qg#2&T|`@z2zUy^Hi0NO*SSbp zsr6*mobR*n=SDOL;WheD9KaR+0&+$0-M#_4BUkZO|K;|orduIlsur+g*+!NGiG>4e z^X0;UdrOd%WipNPg_Tp+g&`G6`yf4&i9A8izq*>pOO^4xwtSzjrNrs%L--({4|KQr4U1@iR;dfi#{+7um__PI2{R5jRtgv=S~nUv-vIROZ=ugtq}gF~wC*x3xT#HDEOqU_Q#blIzF1nUjuZb zUOv;?dxhwC>reuh>#8rY*o1h(IYSh$z3t6Klg!Kfqu^Y$bjsL4c4?s<;UWqCjvF&GmW%U&4kZ@jEWA?3XWfZhpE+r2=P5#$leR^vXz~k# z-1+9$z8Y^woRUINX2#M0K}51B0A+y2x~YF=qaZ3B^Iy&t!L=v&EP}rzlTt)JwcxZW z9&AfnvqX*(_UJ=nuE{mdJI|05U>BGj*(ikWKF-gzeq>j8l`~WK*8sU%JUQ_GOmE0d zUou<9=pc=sRRa}X0OagJls$%tpv>?PbjkI4X2{kdTjv++mztePAE*pmlCDyL5-i@L zSr+K++Lb1#FV2?)sg!@kPAFww(HPRHX(B*%>rS(jNEwRV|#?wkqDqfaK*;{c{*fLgn17PE~3 z!j+EQVxZCTW?~~YvQ*9=`NDLzvB}V14$R)#UbuzMQZBVybXO5ftX)*$WN>+vJCgY; z1}j$)wZK_AO(L30e*1~f`$9IPwUd_~Cq@YOEp-p)dEV=|kqo>D`ZlS`S9NQENLb1|i|EOuA76=CV&rDPgmPQ9pkXJyb4&syv*<6IdjXEYtYCG6(e-MN5 zkLa@z?1f9?PpZj{kJ^`bki29GT^D31piQLZ(sZ2$n~|_ ztVsL3Jkw}xSXBZIJ7lh=9*^&pYb~ve3&E*s$o6%hn%AR7gNqgp{ig~qj2)g`q*uT3 zoB%v!J0tFUSciAR4^5e zgSV9^WST!bK^_=c{L8GdI)O{S?+T!9_xhz7Xb&boW~JbxnRvAL?Zk^proHftS97r|?@p8ex`t*~e%Eid@=w8N)y;v8ch&7Mt zB2Z1veZDikL4gCdwLDee@r|2qDPK-tl~7RK{r!mh{%uabh{yYVCCCB_;PDd#kc5;A z3P1o@QAGbIt%>$8zobG3@GxVD=4gugzb|o5i#V7K{30|%1JLA4_-<>$TsUh{OqZxs;cYO_JdM zg1hNC3wqL^ydXe1`pZUL_2A%EX?#+)#q|xlS1dzA$B?!RqO$y8b zA9!0~UK3jkpUm@<6$G$ONio6YW^%|m;FZdD2HyJh_+Y>EG-md673-J-S7U!IK1J)9 zau9Cx?E{@p>~bVt03ws4K`y%);mdXg(n`@YrYShB%G9zCI~8jyWg{xK^@G@N&25yr zBWb$-DBIIa&O*3CpMSTyNMM^u^I$Ev)NZY2OSrWyHvlAs&dl<|Z@}{(KbcNDJsDo` z*RyY1Pu?K<;ipM4iF#Rn5ED1E=dI{#vEO{t-mrR{o^=do|LQpm`<%8oUKlY$!TkLO zWOeYPrFgivl4HMk(T?gyL0-7Y8N204vaf@Z?slpX&>sw;}>QGYR~$AJHk9E2E4Tj|fC!D3i3H4h$|bzf%fv5E=TI`n zY20a;7htU^*l}#qnr=WVl73!M*FSp^`%xPsFYE9P!WE^&){x=!^ zAxR;wTE9HAe6r`(_L?9FPE5a#XMg`oD{R~L>ut!$Q%==$+X4MF%Pw}Oe1afQzbidN ze>p#m4izHAVaO7ai-RD5eoy93xVwe`Z>z;s`Px<#9yY{r=gnBpeOp)dj|DX#f#;f* zKCBX|+9k5&6rcl?=9WG{ER&1?O^m$6jDpPC`>ir8G6*Vu3?kFHhlmU)np@( zx2`>l4CgQ3=NumuQNCYQacF>pP$3MUW>vcUt6COXtWlsQFw^M0>)FoSYNSsQMsOr< zvrq^;(o~PbqSaFg^GhnWjAyLh?!*_3y;C}!5yO@9RB}j9jY>Cnw*t6R(E`{G74RM? z3Ak=oo!0YyBcZybx&YB}YWN*MgNbWdC%d82#P(5VuowNyy_*BDxODf6*}IXjH`HqV znPhzY+t(yHKgCR-iR*gDx%+vF3=A#R_Hk<5X68EjV2U764;@p5gj_-6oP*~#S+l0@ z@4r37b*|cg5hZG1(?8l(V&Lw=e|V9BF#ExG>8#84#-3uepB31I_Tjp1AiYn!*FIeydX-^eIz5K z`)=;uvD+s~#am{T3$G&)BE`Mp5gpN_S`Rx=K8*dgCHMLl)jTn!2W8T;k;^wR6^7va zEdLa`gN+V3NoRSLw-*~n=)#jNw9o46hheJ8F$gUDE9bUbt}X>?qe}M|4jLWnX@aW# zmSZPLF^CmPm)E@JhphwQzctfG`**u70TS%ocxLF{*5MU3jUSh3m{u=c)Rr*IiyW%v{ZMY3#P@4Nn(*^s(iY{-xzMShL3t zzhLy1gq3F__O#*kH^=BfpyRUjkNrQ*3j9~rSngXC$p?JZ$YYnoK4CKe1?}|aPo7CT zJiS{LHc$gUp|bJVuk%Qs*yj#NkMv2?c?>lB#@jy z{zA&%85*_J*>-HyFE||GjI4l1AGF@#jV-;ip984Sqj&_N3CSRVW>y!zEAafA9) zf#TGoEUjRvJ7)phq@s&SZ^JjGQAwpPORi%cxXJKrgxr*BT9fw1tf1vj%M7#{q=sDh ziOZ8Gqcr3{>jmUE+GuL9{_W`Zpg4X2h#^`(bGeyfobNO`onOrFi#(B}8Po7n_VTZI|-ko8K;?@=9ZbxAoM}% z?d*`ImD{H6%wy{Clm{JspD26>v9ZPu7D&&p@C6>4*=Nx-8xNGK%v81v2J|addE9XX zsRE#g7D@-N5jQ%Sx32@DxI$egyccBy7srS^8xVqQGAgUi#ZVq z@YhUPqocC!%D1J;%4SIrqw*1&W;+B~c6@Oud;Jmg=i1=xK-{B9@OfxZOL3Za`_`;tbT>-E%b{t7M($W zYJbJckR@Ggl!dQ09;)=4<3j0EJgR(h;-VMe=zR;>l9=~_{wNd8?0a^ZuqU9E|ivdNA-VE7teB)yVhU>t1KRL3Sjtg*{+Wqf$J zF~c_MF_f|Mr1Q4K53p8An>h3Kr1Qd^3f!LI6dsIujyo=}PNJnFV3GFC2OkqMF`U55 zcIh4W%0azea#t^I*lra_sS4xos50<#5!;%p=2wvrLi9f)+K|R2{&RbIBZ|YcmQ~vcD1VWSzT$%&@f68Ek{{C^W^8=m@&LwOpL@YON0aqkIguAsFbK` zB6}1JkT`JKx3grWSk6#=APP4t_E_+MR79X4)fp)9cB`=yS>l*uaFC=tjsFoRRt>@V z$Nx2N%ia0F*8c8w!Wrpo?A3xs|F%JO2zD%v+|^B9+rnrkj~DZxc|}*{3~_P^ZFidV zqif=QrOuvY+H3ZP`(j&y;gZ>KSARgcVak`bM6g6JjSDVRsT3KMKE@@Y<=L{MD_nRj>V6K6 z=OTLM^!65u`&_r5YAJNl#DPh#dKefS1&xFV9x+*53Rpx}LU$)Q zA1EPu#`tSf0P#8}3VooB^K~J?hw9<4j+$I-4N;KMT1-9;A%_?^C*zwTb|56<2PdoG z`A?7GJ#dvkd*8Fvf&P^vH!yBZ-pT+xuzv?fHReQ6e5SQ3o$JMM9|Z9-#dC%p!BXpN z(65sV#GV26JlTJ$C7E~0z!V7+ScYL79M$ZO)rw9zqO)lrDVKiTWf614f<^RqXpsoO zmr88Mf=vSeQDSeKAlJdB^18dAU?7*!q%0Uv@{l2a5w7X>884FOtl$@Sx+VB9ZdL3! zXOf(jVDLtCey!D3*lJ;Ug=F@_u}6cOO85<-Afp$l?{R@>+C;1G3rZ}WGTL0Oo09$# zB_-p2W)1Bh z>~#xE1IeOW@lL%<=56vl^_u5>$8lTD?PWPeu_H?%xv`^$rR1V88^cj@!`1HO2mP() zx+Qv_diLc6(@W>3Y-4=dTYm;$Z^p}J+uT{p(F?DElW*qoCc%+l zs5C1Gw&87triSgl_Kw>;>F{KFx{wMHpF#54FjX+xYil=igH$i(w4M zFE6llhhiV}D)Z-`m5#J_>ue|Nbf? zL9;`Xr{k7zso>bZvo|s#VJG{!R1z(x;L|O7BaCzIo2Vi*m_9{)i!i;YWs8jO>8xgA zzLfwsFf1G$`;$Zjh%}XJX3s_NIXsV*hNfjA^W5SX^cCJJl=QB|1y=VSvnHw+DB#oT z-(-cd4=roXJtk!d@`;gS^l(xUD!T@~uC0Z;KO1l)K_|p^BH-uDrUcKwa3;8N5mL!I z@@x@VeGH>gQ;sxBv-$bc@)nX|;r=F5on;_Wg}I$<<-_IJ?e2k|C&>PEdBedWtZ!3| zUi&lR?m8coRH_J>b!#S4Yp^W_?F%HY8!~0znA9wxSB$7<&P)Z=)a={8}RI&!Y%-@7Dfl$qr1td zOW{dLAxmCFBMJ}M05)mWf4(b!{Z~qG{%)FNN&5=gLa%U_`SryhVOiW3b{{a2f4{EB zU^V(FFXi}4xoVo7&19bb!HaN#EO11(jj!w-8k}yj`iIg;y!Y?x#6Q7hct)#f6fT(AmyYb@wxpZ1B_s@8;oQ<|H> zks9+hZBedYL9wq+Bf_oDQTi8+ujasbl6%S)FUJpZ|CFX!t}oRr@2dO;c80H-FAmyA z`0d9a`qJZ!B8QvWBlKT+QUH(o_4gBus~aaP50h;r0EagUOxfdE{K-Xjy7L{Ka8l2q zYJ<1yY=XT3(^yhMH6uUz?3PkmQHSrH-!Ke<-m?hN1lf`WHvFIv^tS~I%#Y`3*Fdal zwAdb-AcH)7lruAEYDbf;52JA84xgW!GRbld^RxZzO=gUPaP?nY5!gStXzDU-U1{}` z??Uq`%9HlV2as#H{@lho`K@p<`z+#FwXhr|qsj6B)*U zM;Dp>V!mQa8`#Gc`g78Zf>A^Cl4v{m8&8To{a9?Z#-9S5GAdVW)mKEu_ZA8Gx#$5& zr(O$oq(UXVhUq`pg%F&DA#G8&>aEJB3SaGZOGiFDH(Pgv2C(xH9QA_O7%*-`q=Q}# z4}yZ&7=|%|*lrW)20`%Xmpg|7sA3M01^j)4e`Ynj_gTcoDO(@Sj$`Ek08w(R1(D@W zaTo!~k$}+Lsx$ct@UL<8-|hd4zcSCBD-h6!!Rmwi*9`l=nPLZ+9Q%G)c-!-KlpDmt z4`hs9lDD@Z>&Fx%Q|ja)OscWAp4K#|IQQtQK#+(UOsMkCVnyhFVb-07L&`r|c_Rr<{0`{SC8+gE z2iq=y2*umIwo(6aN27`zIg(az*m>|{*f83KzteugT=sngXE;(GCdmwpOgdt!NG7Fi z=L~+p`qlpIZja53fv{+#IGH&H)Ln7%J>!kc8$tR0xA>c|Y0?y-+xyP~}V6ktJ+F|`T7bH%K2RqI%BUbPGN@K%}`m$F&)xdc(Xol-dQt{B*}6E|YC=jE|{ z9a*249+ezC+WmuEaWq*=8KYNZMuuqM|gENCtsWo zo??aZ_VDN2^t6KY4jRW}`O#fB{J39go{WLv4u^B{1qbTunpwA*v1&!pl&8yIZ+|f7 zr<_asT%RR5j)%kz?`P}%G343Af~#rM${-6C?yjzHo@p&LahvZpI$3FQFbdR={T8{@bI`~!fxDdoCFF^>d+A=Ubk=j*u?1mD!;VM2fAGxg`q>_Rg|R|_)2 zADXx*74#OuJS~$dZH}T~D~@D_g=xQT8id#e^8>xocYyl8$S07PXO8?8&z#^j7FzB&6S#rey#Av&0i;J%J-@l4 zM$wSnpYYskzaU^;qQBAQpehBT-rq!3ez`qOM?~78%g0$HL6`?qhq)^kRD>L*hcJsO zSMXg5Jqd)TR!@o+K!PaYF(8EO-O87V?6D~n;RuH6zJldb6IHN}d^2N@0crNRxyHm% z*!Xu=;LC?te7LUpeBQ8EmF%cu&4%wJ6zu9cW5FeY#Q$t^sfsThQaCV_} zYrd?tZ8@NyBI%jqsf`@_PhH@ApqnQzu8jxb`n-xn%kYhrH1-g$j?jjlo5F{)&LW#_ z0`C197g7ar8;CzOL+Lnica5$+JXeDeF{#lS^kFy5SSPOfy8bz?K5rirUNyIw3{+;M8t$*Bp8p-z}eMn;iMN(etNw&5`dA$2`?9ZrIL{jH-0p!?9`X zT5i+WrEeb$M!VW8J@Y3xBmCXQttD7JQz2V@%rVT&`;!aOD;dC49Krq->u=MroUX5# z>o{D$UxY1}DrJbvSGRhyGG7)J(n>18xoz@T@?o@tf#gX1xUa=ysBhYBL)$reIyV>J zggF>4^RuDxF{s*c+F$+nlj@(kQbcRFtUG3@J{2dPK~%xm&S67RPc_;pNjtI>DREta z-9H+VRd7JAP;ju&*%{!hQ?D81TMFH!Au%YS`{CIa1g*1BP3}Dy3VSOp;2Lm(TE;MP zOH22Ofe^#qEMc`H`1+D#h#=8D-qxf0 zQGA`rOPtQHm;--Y$=zHs1oYq1catxc>>Q+t+DZD0&O4-%6W0YcCr74=LpIUxB@RjP z7MKYpqCVcdKYDq=M4K`!_sGT7j6f@`9;w%{V5J57Kn|gcbC2=mK)-fPXE_CoW~ZVT zw5sYFKEL0q&U>cOF!#60-u3D-vUBot@o8>p4FfeH9G_Gk9gN~8Wpd%aF5#>XG|wZh zU5`0l7YvD8PBx|WG%>7E75MQPWSgOu*nUK}3VTuq^07(5ww^Hj>g-u(Pt58ql4BTYz`Q$k z1)TJf3wi0y(z^hk(+ G2mK%JT5Wp( literal 29189 zcmZ6y1ymis6F-U-*Fv%4?s9P}?yklCQrz8&I~2LNyW7Q!J6v3gdvPi5@c91z@0=&+ zB-!0$Gdr1?%zh@DjZ{&RMnfhE& zy}bbdfVsK3o12@Huaqw@FSob14-XIj{{1^YKYx6Dy12Nwyu5sUeH|YkzrVl#`Sa)X z_4VfFW)&KaxVU&rOA8MV4>uQ=wYBx`?p{$*QFwT`jEsz#nc4BlNql@f2n4FGt`-s! zDlacrR8-{W=bxII($UfB=;$ysGz5deA|fJ-i;KOzy-Q0=nwpyK?(X*X_GxKp&`{8W zgM(gPUbk;=IoUa?s;Vw7E_r!*zP`RABO`Bb4;&oaK|w)p=N+@Nvj7gGx4Vgp)4j~h ztiwkA>w_sy4!M|^nBJ?m{nxj*=cC&p`?swe4#n{A9DujC%eS|!Mnk6SoyLpBY=D3> zz-B-zgFA^V^#Q2f$Sb;4pL*JbU{*7qEPC{&Krkv-Pj;;4K+Fm=PLr=;+_| z(#w?)?efDL7990vo+3G6xk1a>4-UZe?VEhm;*bjxt)`2n>gF&NUU`cQM-^B?pkK>ob6)bPM6$7<@kQ>2RsOJ< zG~hQa^TX*|_VDpl*M$K-+?hFeWnjruTXd>EJd(RQKD~j2qYt2!XFyllOcMywn}?+j zx*i@J$^W6@80AL`^SgJosBePWbf-gwwG`T70*UGe*w1r(Hn zmaK%RhNqz^uh0HNJ+?SrOpvPM@qjV>a2zSnnAiJ?Gay^Q+ln%}59-`$i$dNSu?lsm z_yh)sx9X>pH-?Mfr!wByNEMGyIc^ynq!LuE`U8m$Q$!Y;e!v(S)=!rH8@&Np7t&aw zo%T!a-I_qs=*svIP?hTTIUsVwM<&NVE1jc1A`?@hJ%x<&tTybeL(Uu#M4liZ&g_&d znultI=J)wK0URV=*#v?~C!S|p#3i_rw7nQ1l7|XmL0W!o<~@nD-0Y^ZPze*!sr19m zxTB7B&O;3Zn_$(%@UT#SR4*SccbjYYwgm#{qKhI8#h2faK%zRP!7t{yH9tYYhTY(x0s}M(KA7-6TNq%c0(2Q8jp=3NGJlq^kiBx_*Aaml(dc zh%e8K*nf(d&g1K!j2%Iu1RyJQ{&IZpNuw)JXd32p|I4oQs!lxI=a zUE;St$pWCUTI}@bw~Uu71#zA@yx@{$vPyl<)Y1s8^`$Dos65Z%lD%i|yG}*U-7Avn z5Qg?IKh>#cU4jcyA$d&aDzwXgYXGc~*h5@irh2y!bHxl9wxGSe)!$qf_uzprC&8Jm zYKd5ev4TDA=NkRzX8ZcABaqiKXYy@2kSO}ksJ@cMf`_KtH~7IvyYMQbZ;iih_G&C^ z)V;_xbcP;F#>e4h2tcCCpq%yU&jf!H&8i+CbboeQ#Q@8$l0`C^iNP9aUg^Hnjl`LF z+UXTQ>yx#foUtZ_q&1iqUXaTJwwp&-G5_iQ5M$x+mS+dbT%5 z#~e^qxp6$=PS%Vt;16T4K4dS#VP7N%Ol9soXF8HFpI`)X)F{&@a^n>vG*ewMDKVI# zRX{7LP6o3}I!~JqReDVKxpXl@S+m3?Oq2-;>>k{lY>*H8 zF#VL+Cso{71HSrjW*8`eV0ztcNWmAnq#Y2rwZBu|QVvaT{My?Q-J0cW3Bou^m`~l> zEUnYhS{ReL6Qj3fbgH`V0hs-JY8L;Y-bwsXy7r)iqan!S377IYMlS~8g$(3&^{#@qv-3I*scvoT42}KwH`14ePRP#Wee<7Dcx3{R zNut9VWp>OI&EqE*_Nr9s)Z$Ntwy(rBu^Yb8aLEXb~4&9tInVF^gQc`P~ z&zaQCxHt%%F&-CBlT(a7{)Z&xO5EE2Nob&uGTWTrquA_t8AGXG-W74anLPjXw6eBI2FKUf4R*GJ#;kd*E4vr2Ax0ti~+GwuvA1Ow<8JuyN7e>V za3oR#ME%Jm`7^V~?z;o!?18&TO!`hgR?2hzI_BP1P#}mvx(0kKh>8@QIKlr8a|#Rj zoL?je@)Zhi7xFj4Oz9{=6DdssMZUJ|AS{TQ&UkTg5&lO%&3(`@CGfs=N#H$|dsZ+` z86Dkz>DGVsxsL?41wqwyeYkj*EM6TqM zDC<{8pJ^Oo!55offSd0CI)Ow=NZkKjx(GepU`G#Ljh2S0P(m1fym8v51Jr5syM@2K zwMq=s4wZM1{Tcp&^aXE-oN?)S>fvD>@9M{?tOTwOx16qKkcsRGF^*vQ9YE;mdM1yIx%{hz7>Fz(Ca&Yc#kx42H2I(Xerb=W8u@@;RmS#r^OE)S}-Y-BMHqJbxhkXT;Q zIO+1*ffeD^Na_2k#U?gai-YZWNA^}zLA${p4?f&*5E!JB`Z0X#GhJAv61%W*S)+uh$ zN#P;?TCn75o5N#enp~B53Qt?7hKf$+FHfw(@m67H04WWh{5gTc2+Y6rojD_b$@y_s z9KgOmoZOL1kHYZpiD#rAXM~Fr^fP3UxaBjZzw|o5}hUe{* zg4Tq*4>2%!kV`+17uGv{ga<9v6-HyCVI(bm3Xx5)bMI_552Q$(5>x-P!~^Dn99SbN1S({m8DGm|6UF7xIwgp>eFr7H9=p5Dif7-Tq^TU+mlF{ zg<*je30tPlAy4i6Y%;JJ3E%-Gb`5a?C=AYm7`lunjVcBezwZiKx5Zqg<4po}0S&uP z+-RWn)yNt70JL|i#E{<8mCm!B@$GyCkA8`O>-ERo$na7&XYAEz{ezn|;;~q|zM;Ob zVmv|PIUydWyQ={4F?_#IJj1t*N+Un`>3~0eA_Gk|4TgA%s?<^eAiZC?n>5x2S`^+) zX0VB#M&dR`h2unH9gooY_DuaAymV&Ch=ORfXIlYeFvPs1CT=?JNrxa{7VWeBS;e3{ zD#%Vp$_1sKjD^6atmj7 zEH=B**;-(bJ6d676-(bg39>;z-XL!OOWrBaA*W*j_*wwk4pD&tZZbIxh@^XmZ~lZZ z;H4DW>K$D&P14k$@=h9_8_4Rn?Vo3T`>v$l#?#BpRQtYf;ZS7eeZH`GW%=_s5_gOt z27qq6424H`GDZF~#!-k~@B8RXQbV!}IEYaif%lJ*T{CAqk^BrZ9piy*DB?~D&yQ!l z5Sx5!;Qd#N7#?Km(catW>R;#j&O~%6qV~d_!f#?K$ADh#RIF@(3zN;x1tCywA@9M` zA!32CtBACAslKVMI|Ja*g}+i=F%~uZ5!d8=x}K&tSf{$l>hkww$@#-p(`}8E}=7Fy{(!e!M@Sv-kNwm_*=^e zCuo8>KFEX6^!t=;0F1JZmn?M2Fsyi|q_0i#m-m2z?%K@fO5@i-MzFvfkGF+a$jPYc zUpYG_v$Ro=(`)(70(pn5Pa9NYyH*i7(fN3CXy+OYvB%%pxHo1mAMWpIev!=16JOXh z9e;^3&$wUOS;Qnif#?U*wYyn%b%vIxQn*IeLsnuE4$^-$%Y)#WR1v#PS*q1(Y>Njo zVpN?i{%CyI0T6VyES9Ih@%g6R9q*cbf?+Vc^kr1a6WWS&M@ov{b4XiA)38!5edf5! z#L5>IaudJ@0Syq4m8#Q>SM7Y>t1;t->ryG-GHB}$4*-)>r-*j&Y`1n|1%!@hZEG92 zKkW_HGgc+>IJ;Lro9Kqg{Q`0$vgNO1P@Bzrh|FQL(ey_IemVC*KP_G7MYs6~6ndK3 zW|V2&nhgeoYu+yscnYctEMnG~%mgi&f?quJt+i9Pw``NNOYX@Brl`f4`c=j6gCVGv zHsQ}~-MT$53=l+imHm~+;Q!6xm~55`0;JhDB${&AaWXm%cj1ED-=sBDp+le}_2fc+ zAWdahG5g~-sRd*SX=PO%N`sZe4dUA?2$**K-RUPF2-8(Iv|Tk8PcPLfn+}lq0*tOD zvp*DE<)xJ-6iSg5d>5V+N&h1ySy~l}9;9!7$`OON289w#RH_y>pD33nyl_kKo6dpV z?DUa*q2;YjL!N;3o1eZIH*>Q$>?fty{_eSj{vu^AFZQ zy=wSSOkhBPK)zBA1OO>B02Ql_L4I4Psa%x*JbWLAIV&U7To$?Jfl*jt$Yb9ovVC1& zrH@sH7&BJpU&k=!s%?7lRp~i&xL9J>!)^1zz|f2#Lnh0!#t!+ns-JNy>CBYjv>&7y z7}UdlD|GgUmk%$^@O;`lizPENpOG#`B`C~PMd0S2e6IgdHKqV^3-v_K6(jg+UgCUI znWdbM3Mf(4qDc#gKeV3MURoW)pJFSP57Qoc|N1Lym{i{Seis@bMMGZvd>IliRJJy_ zD5H~o&`%dZD4ZN84J6ep#Pdy3lk)uG2~kMTmQ#fR{`-~;-q*U-zRR|tIvh6lQD|VO zqaBGRFCa82i{S0vF6UIPr6Z}v8~cC|it8j67;9XR7%h&H;+YxKvM|77(KS90+S!D1 z#NL<-ctdb+n3HYO)mkzH_;_W;9RM-D-~H>Tx_;?gl93$|f=V znf&CrGk*0{S-qWbe2(xv7mQxqTrq)<-H?Zj{E5%ZHQAa&N@nVUn9Sk&>=s{7cd5km z=`6hG%1hI;z?sTY{FFX^#P!pxjOUZs`WFBx+sRc%tOdnKi_M40FA)5u0gKBK_LqDWRa1Wze$zTtWj17pCg+^lv!Q@ zGoLD)8wJ380})s5lK&VIIuqL)>?8wp-&3)kUhGcg@1X%h9`^!L4*unBv)ka`0ou0> zqsHtvl`q6*p=M|3zSnc#QC54U63BI=F;G>2Ao8GmNEF*hT~eMfjHp08BbmoBW#gKu zXjIc-w>H7fbR8?LZ6>eM*XL~BuA{<+I3K5uy+h@u8K)4z0jZW2X6JyI%WT!5YXp^V zL|RxQKZlau8T^@Fe%?)PwM0@P@75+OC`S(j2D8d^?n(HtXJ9ju#~?=WNT4@1H^r{Q zwI`g~nq$iaMNw0VSffkuQrFCrh0?jU*AwIQ1jOhwVkLgjTYoRVnBYB0J_~_z#x@on zB)J%0&DQt#cIb+seO+NzIF(H-nx#!10Wpz@e%_@i+j3Sd-1)^|YNhS^4@Ld&HejQ( zd%bUuz3&_Sb4XDic}ijbZ`w`YO@}VBn(%0VblkUti_}}uD-=aWjt4UOM=MX|*9iYL zRH8H@K9KC2FzM{#I17XZxv%wp)`uGHpF=a|TyD8Djn=25Ty51RlKj1X9ZFfYgdicl zRa#)8{+Nw!l-sQy$ZgU(9I1(OKDt3U9dO{9ZRRM#-+VFWMnm%A_H@1^(~$`_?KJ#3 z&{5BGu11p*Vry&jmFFW0ku1Tn1qDi`CrAwnhXB0P$BJ{m_&tvR*Fnq}sHjEDKcv#d zbqdiDsP5mCqtsRV@htf1IB1a_c%=;$3>^8ik~x0=noi&s%qUO2<+Cb z&BJ8sF4J}+9%f)!saxb!TI!M^MSa5{1ICi8aC1$x!idn1k3ggD2k(4m_&XR7p)Fdn z@3zq^9aQz51?AeE_fmXk^jIHXOddlBNiNzV8bHu(BA9A+mHmk{4d>kG1xWyEKH7|A z%KP&-CBD)e?zkT4Lz!J-Wg(%T+$-s0ry%-Tvm|7v(6V|3c6GnbmI_}>?=@HIISafa%4bA0Axjg=h)MxnZNMn~`;CI~atRd0H zetctMUuBE1`=6<$Cw~^EwKkH3=bV70X`%iAZ*{AzOKl58E*MbGqS*5@l=N?Obk-$_Ug?b z98fwjh}>|yvH%+Ami@5sV=~3EIFH=vruGuFKa2XF$q_)L(gRym2I<^1VwQZOqXrZ6 zqC3#G2=}xoVx=uP$0<+SVw7j|UED16QguM8gJmj%5xkP8UnsRADK(gKbh1CyU_QOS zCQ*h1fS0G&Nwk?=oq|$9QTa(mw6sPl%|3&?_Jyd~QHnv3Dgs7Yd=KG_J^y&KU$L?q zVLsLPlE0{Ux|#{B_#XAX4aCuZDT<*O`cqMtG=|wezjyaX)FKnFJJ?h+#t6&}%(=bU z%&{WNvX3)Da2bAu;=K@s5k;p?$@Cx%*}gg+>HOX;9NC|rAg=Q|f4N)A^lO92-flEI z%#g72PBNfZZ?Ba2wVS4w#%X%>ezZ${pLmy+Tk38c7%$mzSUaof#tPw7TP~#Hq+kHwe@9c#(+Jr4m$GOs_c#s2M% z%ATF9!rq5_r*G8AB4pjpp6=-~UVX}emyz$$1>aj{Pi3Fq+q-?nGUwZ|b#7;pOS@6* zZ3!yK#-Ugt=Xc#&$E7FL#*Bl~D+WlGSUek`nABZv`P|e(J6B4QC{o~}UMNI*y8ZVFyfTtPIuD?Y)KqGY5RddRNOxEbB|C9)b9ZZ6v`-jp zRfV7AdguwZMC?yRYd_=W1@l{v^^=4zX%Fqb&X4~RT)ie3z14}eRfov{L_&e{~?vc;;Qjx_8{9cGNw7!k6hfG}$)kSHl>*ch3WK z2RbyxZ1Bi6Z00(9B^(^FDJYbV-G@UVzs>J7|Hy*wwbu%jaNtGihOIsQ4CSkxI`g90#~-7cuQ|PDkjPi zup^>$%%8ys_LmE3&UTJCY!Z~oF_7USY|6F@E3avYx%(eWN1q-0x3%3lE~$8h znW;c)O-T~Z{UX6Sg$`EyPh*H+TT_-DL<$$|t(`Z+W+2M1n`*mdW*p~--l!WX$6Hu3 zu2G)-3a*#3!Q3xE*;OZR;`DdJedE=pwEF9+FH70>h9{7zDC3=2UPw&vEj&?v*6DUQZ zp1-f;>GyVgsfGRWnAl5RC*SkOH-#B~MF8P+Xks@}rzXQ3;^f7j(W&TLX-#xk zm);t>_JoWU7|>Pm|1}pa8ji3Hz`XA- z508dXjv=ljvG#TT@#~D&EeA8bC@+&8xEdSwg$ALz8Ch;o1Djc1m{~9QF;O_TW&GEVq-O^{|=-g8Lft=Ie zaolKJGlj}Dts8^Xvy}Q}lMPF|<&E&tg-!Fz-ecJLO6Ho;QyWy$rKDaIW_-mlL^-N| zh>}5m1SBdmHpB<%zjqV0LPGldG5iwHu6DQuf8__+sZkB~W{K{qBs& z;-}@XXDn?4Ja-RI$8^w4V@zJdEK6_0tmJ87Ou>~QF6B2KX6f2x_CmxOU>VVkneN&I zA>9(g21?Pac)V6@ly03UpmdOrfUP=N6dzhgDjrzcMk*y8mfNoVk9ilk2{4R^Ud}Yc z^h=j7k9GKp#<`H2TD}M&YZM`o{XblXzOG?lJ?HCIOVqKRG&aGD{4`DvNjd89m_>Q( zI9*B93R`GMK@&rfYG9@OPhNt!S*gshMKQ?c+FA7=W!`1D)30#kyKqhcD1E|0Y%jnw|I>Wxhl&HL_S5TOwyI};#F}Q;uZ4|HrB-t^ zkGLRjc=1TV;+@7)j={@#`v^PsetVVgqTdZZR26I()V>zYVP8dICi`!sh$L2buy8LY z(~6|38kNV~DIKUm-&k`fWZ&yN5U8O-M4%+2&lZ>9^akO%*+Xarkmx+|;6*o1{;8^1 zbjV?($<+N-{5!fy%H<{kd`Q~d$NPoPAP%(oy7UMMC`^WCfK3jBSW|7KG@!2GivpLv z$3is1ASwh5>i^1F|`@DJJ`LRHiamVbfF4~oW@gAIAo~~KM!C-)~msUgSrv)Rw{I`;bobY zP*Bf(x6_HRhw5^W<5ImgwdV3`?-9M`bVIQHtOixre{v4x#N~2`zJ23o@gI}#Pqjuh zTaz;9ChPQH*9?sAv)32@cpxYW9~_0=Zi5U2&2#UJTyi=W-;ZwPzpN3A3RMXJ8Q0mu zn!5^~`5ai64*Fu%7dJOQ!dc`v5+VJ)Jlhrtmj3Xv$Y=a z3iX$cuvI0i=R;bAgVTx%WN&l}hv4)1TC)KWF8o^;CDG*PzBR&SP1fj*6hL#<5GENn zD$!gIEg4)V2UdK`lL#GQ#|l?27Dh*pN}?26WWuob&x&O8!H7V}4JBD)iWECXL$5(% z1{8U3KeG$dyeM-N5_4;MTr7&QhlK&9iPB^0l(pF$8RzxS>G4~cWi@3iEVcGhreFzb z1NswQvo&0&YQ`VDK*ONI`73}Gn_Iq&w_JzEkDpi)A!G*a7IJhLNUXSFv#`(o*Lf^( zVdbk7M76I+n?FOXfBotI?ne0XeYEL}zD-s`#I7y4rZO^2!bY+(ln zxMY({GtR!gw%2wyHQNjGzVK#v_;Ryk=IzEezpCF__@rorxKYmTM^#8s=dB%6i%EG&`JB1YF?@2pRcUATjkaI^~C>Z&f#sO+G9VG&bj)%P2tt` zX1o1@du6{IfaWmTpf~e8E$>nzB8x89mAqmWg>Qw~1MLw}(}f~*f%WxA!}QcG9s{wR z{mWspZf)?{si{$T5>vSic8J_n8J3LF#8OWz9ANnYUj<=|lT zR;uS8(g6ut0>9UFfS!y35)h6SopXC&_5d#~bCPy%xH)B+r3ADtfWNT*)Oj{$nAPR8 z`GcO117b-Oc;reH9^Pm%=2mZ+lUqs9qh)r9@=^0ypS@66g^O}tLQPX z{{psZ=#Z8U@qz?tBlgeH7Ce)u2*Jxg1p?Eq!`hHFyM6S;1X7 z0%&zKVNZ&f_AZKIg|QJP4u-N@vH}m~r%+8EStXDngx?_ftDU*mne=zq%|T zJ;xIdf&&>a-w7JZMP(mU6vi(2hrYp=%e3-cp0uthhAO9r^?(q`0q<+>pIMh5C8(98 zH+2dC-q-RTUBe%vO3Te+^^;9i{(B0-s4>M=QLZzyEz`?$U8v;fhjNp+ zKwGE*qN-qkye#1n+f(5(hZLvu&>@ZNx(miooC%Yrm4c=d+08_7?2hYUgH?-6T36hT z-b+uxX^O~MSZNFvfbVnf5$PgkJ1*!;eZAe#$O{|fAA(v+#mHmaB3LY;Bo?c%Ew5}V zSbgyu@98m*@}|^LthRB>?t}+jq2SLtA~q^eJ%D2Bog)B2>elRwHKB>_t7jO~cH2ga z0PL?*yzu-_W3XX*W&WOz9jL=DY<8&PL)a=fKv>WFx7hQASzq}6it=JAwA+M>*tffk zkVM%A?;riDuZ~GYaS}XlF$);$czQFxHbh55qckyMJC2n7U&pyV6Rv}~&&KT8NBB!^ z+hsU9NfwGiEn+nFBW(!^SGtC@%TD~qV-mdchKteZR#RL+fE4yF$c$UO`W?BvZLz9U z`r^2wDRuW*;g|C!PLVz;kZQmqq8t`BU5^5Lk|p_i*|Yws6$APyo30g4|(w4xiKm^M0&p$}#Ak_fTJ2J$Qk@ z8s!F&L&|K*6gnMa!h0~iB$Nt)hv)17|07=OvP~AI{xZ78=;? ze#I+%5ojrl(L)se!jEUu&HDVX{}Ptm9lSM~n%n)}Kfg>ty+pf;cG6CY_bw!r!Jcz{ zkKY#h%w6&mxH;-jCkfe}ipo#2wX-nwen@1t?riW&`Z~-%?r6KHH2})f(n> z`}V>mN&Huv1$es$rse{a2ufpw?+bl)hjO2{QO9yk+}MY5O$KvK7Ph9$PWxIf57<}w z4f}qeybo*;uR=lfv0>i4MXP$zrXb2BoWtk&BruF3yQ8c!XIV(Scfi!BLj4aqLVdtT z5n#Ya&I4b6W_wSCrjb%1C`0`(YrubGsmukH^bbIpn$6*8MH$)m5aWpe(!*c~--{E# zb5C&OFa$I|=y ze2H!sWcSCY)x72j)}Fw?rN^n86J$K}CA~sui>9EdX{cG)dXpFrm<7JZvnC{D`PwBS zEk|Iwh=@=lzSK=Z&tXshQ~v|(9e-u{o_=&Sqkg9>e)%O8xrzuExa4j7(Zcebi~9_J zeqP(H5kw%+#t`-a_B7Qu6a-?kz~Y}vW7HSZHo?g@tIf#{E5bl$*Zt5`_2x|8k0+LN z*!lP{i@O#3G`HA2X2SrlY<_*gwDz#)>)24XA}jIGiETKql5v9Y0tI5g8u7=@3g=?! zYZw1O(B!n+_XI%B_$4$dBgU7w;pwBbn6q$z(lU`~sJHTe#Q!7>Zt?>eU;=&9RZ94w z0kMFU8V{>o&T>1^ys8fDC6wViae{j=-@(V;-`N~cr?r z6wnyb^r{Rn-8fBWe9(5GCCcfl)PQ&nmH905cR=V-&{XBl2!STS2GGXcgZSp#qZ1N3 z2Weth90!3Xu4^B)En+SvM9zUof@b# zfI>KdLWH=)_Yfy~Iy-q=CdIk+I@1237QlSJz=X(r!*Lv+RqJlsO_8@OAxSREtytkp zRg2jXoDYFfD!HizNcI!tl#km7!PcIU zl-}Uhi!!Z;U*k{cWtze6Pfme+kI!!=m%x%Y1hPeaDeKA~9FQ)4;A zr;>7`z1)|LulM5TVAS%A#iDN*!5584?`#+!OH0ZYrR5;ZRw6)eEQ(PdsvvD;(2E-B zlpzq{gf$jfHBF@l@>{6;c&0N*BiQQk6=!SHa~ep)UhcG)PdPoyF+|#xT%kb&YwE(a z<-l1sopQ=9`r}D4Ekv~ZN6-J6KDx@@|^HgjV)raJyM8RoB?kFTQNS=5CP)r zR#`MrOpjZh&mkz}Kn`eH)4TkP^5~rDRxCj-Vt^go9%s3Z2b{p(hZ_}r2(EeE>XcrM zM6s<8qNiL0nNS&^UufWh+=J{CvF_{vy3 zU=~lQk@?T^_hoPIba-yE?=BX{Ym|mZ@Li%}E_ERQ(ZW-j1hP_;H5xdj^Muyu8qe)6 z%O9X@Q>(uQ+_@>b%^euC=~dck=@TVSCv>RO6zyeZ;%z4$!}xxpRE(T(-@@{h1%Wo= z8h_-J6H;}AOUOr5(Qu|(Odk(`iI)76yybNsLnFehuwi*m(0OR$u#bXoL=ZB0X5Cv@ z{~Gwkuwe|1R}n}+@pJXjaQU7aJ15Qq8*-a&85dhRN^{aY3TuYa^dWGa!18P6h!zkA zsqH6KZf0v_38H#IYEx^U5ZfkXSHm?R;N^U|>8K8YLLp8nfuh{5K>fxlAu_jQYWOdy z4$%nC7a#taY*!4iL&u!~buAXXgw*^aP%?un8Jv-QSKOp|rzDxquZD9@yq|aCm@;Dz zL0Y2-r?`~Ny3|G%$-ZJa^+buBda55fwCjbwg!t7piV(dZRipVGlQ2S8(twayd{5v5 zOQb1WH*9kbsk&0x%hLd16#A$U<~OWVbI_^XW^fCCBkB*+*sT*HNmtxB)ObBLOR(gcb>C8GjgP?E{-@(>|Q%Fa%F>07R1gauA_ed9ulq_A==5-Iu*)>|clTcH!qc-XxXRLkwE&Sktpz+x#c;Vs4z_ zRT4ksatt*3CD|7_Zs8~O4ljz#DSF38n`RpI`<`f!XxlDsnt?jQlL(b9^WFVQBc+zc zE1AkGnGppx98o2yb*ibbfHG->V@<@=m|S(0<1}66i=Qwruu4fVfahhYL+CZ9xdHO5 zK$AuGokPW_J9b|}O!ll$aovUvU%6~yJHc6{Z_3+`H274)cP>gGv#3G*ber!0lbH9@ z>zyl+84!n`L*8*8%l1f3yV@xFv;fDGLE~zn9vHRohU|H_>l<5~s`*Bu<%_|XZ21{3 za0>-}ldVx>5nh_3JSc9u^m~(7!D7{CW@nRDUHq>mcAkB4Neb!J zUE7>}z<*$~jb91!Ld~!Ee_K+WMeACkus8Ku>u~@z5#A#{C@9ZG3U>YaXGVADOYp{+ zh(Tn2{u8T7UmZn60tOV&sz%-x=^5JZcuL9_$+@2KI(+tZvFQkUYuC0|v7|9d?N zH8~&ff6~Wt(|_PWS)mcG&=@Bsh+H$39Zj+{j^BO0oE9I7zgNH}t^z%KL2{4bhOfE6aXp51+5-{~ipC=1a!|0$H$sJf|ZC+UMdG!;g zL62L1`RVVg#ae7s+PEA7hIpiia71&$Qbf<@#6K9=N4QxOmdz6 z;)>0trg57>2Xh|hC9!XVaBy8MO^?`J$B8B%;t8GyLY}j%cDGMB>oV$MA#Rl-rqL>k z2?>%><69xf!&j-$+2dxgy5}(iD)JB`01vy?wh~a{5?14z%%jTHmTX*zo#BgJAxNvC zG^#Ke^7hcQVzfw&p5g4A%f5&*yIG-k!=iN|w!=@|wSiEiE1cL~#KKlf78X924ulj& zy`PNZ4Bt3MJe6>M@~d;JmiPY~Utam8s#)PSOI!HJM8+8>1y^m=L?HAhjS?AA#shNu zkgPLMuQf@iIQ`G^T#pC(34OBsP?5uHmZeLBN#1TyhrH3$4BL4Ovts`%rRGW`SW%q6xQMSFRru9Iwr0Aq)^?a` z(+qM(hRIzTgXBzfICPP)s7bHJlN+N#?J;tc^Dq>CSe%U|E1p|ZBexUee|qqhv=cKb_Dybp*$eOLvexCf+^w)%sb}r&!kb8{fU9vM)1`-$sf>z+~ z=TTSd#||~^=GN*)sYPZ*#;&NNSi~vCktgjxyH%+SZVC$(+pL{VYy(5>%vVZMDUzuM zT^b25{3>VB$mH1qb|M@yTl=vTmM^~=42dwBA#y7+9v)g%6fGd1qT;!=wW2ygR&%H4 zid+eX(Uh{?N%EMD@lL@PYQv$*Ki8QohR*2YR|n}&JrJ7!3kVtPDA z8PLe?CwZYLPuWO0$Zhos6>ibwD_PBxQ;r8UF-;2|Gi%-j_&+k(P#D7l{XXqV9Xp7x zHymO4bcQh#C6zG+3lFI8+x|(*+BNul`;@QuoCg?HD}s=`(i=_4uci@Wrpa{~k1}BA z#>NnivUvSnuyvu}=#7t*1diHi4&inksNOAtM2tjym0F->mygRPYQDC^=MG8ZXd*I z26sHMqE(WZ92=hVu!8SwU>2hgy0EpWgJV)0$=p@vQ0@DZV4ueA^`_N4#g`-GW>$Wb z-!`j4Lq5T6%Y^I&lA6=PP?|&Pk;e0n{s7LN#~cAv@6r0?~V zC-4E3;JuGYVQQ~dEAvIj4OH{2qiT(YNf>5E^jSEfRR31D=-V{A4d_=s%`Fg*2RXEd zQEz>%J8H}{`F!b^$A!<_qBQnnv>Ifrs+CAtM5rDD;xSuv<4bI_UKq)i0t)ywrB{|dj22>V8oWn^tNt4gA{u@U; zmBibPf&!uS>#mo0{xQUMdX%S=criktS^u*Tr=gIBxZczG;(DLliLyxD#i+TbMLW`_ zF`37s@_|2or`)MkqK*EtbC)CReM7|eL99vOR6ZZuvqnoT`w6isHUyGf3l zeg9n}q8pq18{~DNBmC~H%z|O#H<8!3eEpVP^rqhhG+J@t0YDwyCRbg^C;pW8MC)Mn z>O;W9G|SR&$*^+T*&>YZ31nHK%U_W#YlM;hH=tC1?^eA>5b$OI3kGo!YFh8j{ANw{ z_wmGg@(koRTW+%;+-W&9n|~y2qVvGRAPvD^1O<9p% zRTFCIm1DRA)VPq`W2$T_TL)Vc0HS8y|tIkA;Sm!NA~j(P%0 zGOKzr@HF!A18arFMt&|IOo2EKu>U_;g+F7=^87!Tun!2pS8~K4=;6f4=09FiEzL3J z?VaG$Aj77;|NmeMWwtD+2$%mCya0{GVSE{J&2lLdDd{~IL*yc?ah0uxLc0|(lIo?y z@4Nk9R+CrVIE0YNgNmnf9wMg()wh7bLkjc;YugME|J zs1p{7*|!}$Ihb)?&dxajavekOqdzoqP9FPH*pDAgmcat*)fpZitBpArDa;KOwd~6P zDIroOJuM+A;Mimgq|R37_T7r0@JfFfumK+i9P=FbxETzLfw%z2#ID~k@GnitFub7# zBC9=Wak&B)j+jZNwVZ~0lST(hKc`LiS!REp$6t)B7^4#E28tJikG0tTs5B7o8it+t z-Ida*gL?zU=WGIe-?5p# zHZ2Cs1+YVNYqaTpbUlPcOpS1HAsPEOy3Q;UDk)%Ciq91`QFl5fvka#C*hK0U6$Wc- z<(pk*H3nr_R3HvMOs~^zo(q(dAck_ID7I@6&#OUh-UyX#E?*vtwLcMnvL9O%hjJoN zQM5x|^qj5Q$d@*^y6eC^vu|Ox!o`)~LXF_Q=q&EB7b1b$3M~NrbVy%K;oJ?tJ_?9% zUoHwyARCv5+EFXiRJ0ZvhZBX?+N;XwLnZzAuBB6Cs{=}g64&~!WzV1r0G71IWnsClb0UG$xG#DYjN?~ z#$E=lO1`n)rIbVaGX)%<{CRKU7~*KGPw83K>Zt_&Vx3C%7eEbgcDL((#gU?uco7nA zt0s9UgjxZ$1LmE65-26G6=+1rk)=;>e2Bl9j~9icNb1CES0VD}VJ^2ZN+{|{B~o!= z`aRDS@=w=ak5rBlQ&L@fVwPP5AfHVa;)#^OL_wjJ_QW}hsMQdW6``{c!x$h3Cfwbz zXmMbSyV?Flw=lfyG*LD%_WE3AB630A54OiDV>X-Zc}FeJgEGYyQR8RqDe&oOC~(O` zAWh661b=9Zjw6%icbwszuUgakZGH*!Tt8dYa7uEZnw~n|Ln)BBrx@m3i&?Kkv9UcE zewy6)N<5LuIj~4jx4D=^WkEURmJVYyqyn~1(D=BpGv@0+;h`{52arK*)m%qwMksQh z-D>McQh^efYE%Oz@yaq?=2IRQzfQ9?r6M7Zc>)+%igV#`vt?I24XDB8@*7Um4!hO9 zEjbwgJT|t6#R~`)F;#}udO;@w?3UA-h0kgjLmMN$N*s4}*+TVTnk$o;-1v-jaV>PK zWDXYv<|;<<>ulRBihS%$C7>N6ge{WY4MSCHW*UvICK~?w0l`!nn8$Q_U~lLC0Q`_z zyLGu2u&7ynHrZkbax34PgdQ5tR9c0%<}0IQIe(QCm|0$bn*Oj1%p>K!M(4|HzQ)EzP#2eNi8c!MJr!E`)D@j{*b?vX$Gg&2-$}hfW}&2*#w7?9r@&0THc9teE0xvuPiajj0}nS z;hKu&N&MDIS;mj)>8%xE=~BD;6!fyIi<>PTZgq3v~Pn! z3Z;0FBBe;7Sc1E|OMxIEI0UyM#flY5(NZk9Q#4qx;y8-LSB`_DOYLxRjYOK>nSI0gkKaO|e-_u@dwhrj>|45p4Vjw{rBK|$J=PkEL zRA~3`kI~YW>LxOmxs~WfjY3D1K>1-sqlR`^#1wpo(g~eJ=k9hmqRs{Eja<@QYybB{ z2rg+)GLX=O&GLGX4h7$}8<(no>mY~V)$>985r_T@l` z-2mE>;ghtZL&}RqJ_eFpBgZ&kuM4_C5q`gS>p0Eb0 zx=Cbz5%wYgR_i&@0@8t_RzxAW;#K@R8NX+9TXI_aCYO2b}%BE#56-ijfidi4xG zM~4>IwIvZLY#dw_rLdujALR{_>6(j4bD4=IW|WcnZZL6qCj5O=G=oIYCcAz|j?UBh zYyHX583BnCyxSfq{UeZ|6kzz$PXb3HxkaUg!#68zeO}p=YdRk0h%rnJ zUWO-KQM@*UwPfRvTFNu)P(~MDFAR%kI!&SJDpsfXsCkxBX^!#y0K786FeS3ro9sJ7 zn7i`sR&BPACz4ByheS9{W&A;?C#;=>s1E4gd8aL(xM(E8m8#baVjweycbBCNpYvF) z!B5OMua$1BB@A7!1qqct?L6!FXgsw7s$=;PwWEqh;7~WD)(STs{p0s}5CBjjL_FIT zTEBQz`u96hRnpU@T;=28qoP0PvX4qn!_V#0bKjHIsbX-4C8*!};_Wm5>zuwU)Hn;{ z8e9Y@Nv`m6pjPu+2b73&LHJR8bx5f?XV{$zM!7^&Y}LLtaABt9Oa-(Zb+cYbOhQMj z>-TC2P`TX+3UG9lTP$OuYkBthu!lcr0|V^mO5J)>25Yod3HoOxAjgvyenV%GwES#$ z6Q2G80Sj=21U|a7Ln%GcO34K-eE*84HFRm+7r8eWV>}Aax+3TP39Z%xZC`g;__$JU zaFh9keFG;DEvc*U!`^dfgHV8(N36DU1ePgs?4DtREa=JREYd=X^6*<0O8gz5q+i3* z6a-l1F_IijIvVY7>E&Xt={dwAZ(#X>w3`}Sm(ifnWpQ@l@YUr>_}wUp0MKuCudm9O z`zL?g!O&Z5%ywZDUq0DUUd58!$)jBT$?CL%V=?~gf;2Lt-Noya+5Em)_^LxanLANu zCv1}V^!`Y-Z&)dS8slAZNfG~?i}{zUr23j-Ta&ubxa(&LW+j)y)aek+(T%huQBBbBY z7lB%($|DX2!|OE0$5Q1%%8+lr$(NkVK!4UsX{IR?vb5Q=&JR%ew@kWH-P#c_mpO_8 z&_wAxUtIa0a?r#?o@E29rRQG~o!e49&$nddlCCrbcK|QVaBk9H{!|q7uEa0YK#k@! z2D>3l-fEP;TAnH|@crTug6Zd8cz7UxUy|=KyTn2+;t#AM?s_BwlpYV>qw>(L zVM`b9Ps@DneHJF)J*~ZwrFe)84DXLuh)Zi~^LpoP9pJl9YSkmlphAu-b6m-K*dYO% z_{tBy)%DV7X|8P)g-7V10 z5+-NlVjU?P$}Ip5?bJj`)IjG``%}ob-CkIfOhTp!+b)-56<*>1FJ>*_DP$&OE{m#$ zesq?v(0PiIK%tkI38pJU>;wT}zv?-%hUEJgm+rt%7`)&5u>(Cu4%gfkw#x0y4*?P5 z0lr|Ynt%hx1GwqmvY?K78l7I^E}d-0D@tGKt(L>}#;y2i%^SRtwX9SdQs||69AD@Z zV-44w%Dv0#&-TFQP|_|A@h8a5UCnaXlv?1r5orKM9H>GnZsBGSZbpddl#S_cfr{ox z;vBfD1^y=1rWULn;?u0df|SWyVh%~#iw`IwI7k=gNld?=VSlG_F2b95!G=pfukxvZ z1oH-?^6O%Va5M=2uI~Exj6o8F^Gb-M^tpzaiyP2mnyw6`Hde$9T)xHvNluF zFUs6!{5ZlUU&0sUHX8{-1;~uPcgXEM4P@Xhc2HBvntr>U=t}*?wI_KiNX8n~qL?Sl zQetaZoPg!ixX|aDTkYOe8#ureNQVRbPQ3faclHTonhKBe3gwSk%1BHE%2+G`@znM_f zY#O5aZgdY>&YlWCmiKd@h?4iXt807vaM7C<_p!gh+2=}X6%J{(rXO8#jkWplA^`{8#*FwFkKFZY*J;opU`Esu?Ja5)h5!kbB>kYo53u#4#-)&& zHX|&8%>?v3lWz`|JYF)dfC;NJ<2)D(BZ>yN^*Q@A7cdiNHUA<~ej;thZ7zfr zROZ)eGZoqvKG4;i(fF(#1PJ@wfic4p6nkX<_9@s%vz;xvMRtNuEUsiON2VGkuWx(O zNk5vRMYD~+>3YYHBN+}8uU;D&iC;k`My7oxTldmIZ_Jb8VK9-t*89iI(MFXjld@kZzA_7qccs&dC8=sXcJW!U9P{ut!JZAY}riI zCC4`apdQghsRN2N7YWH(31Bfz8n3@4iO9MhrhasVN#}VdlsLsAr%V9^#?1nw&jz!o zcuRvmopj1g)qGk@B6y`7mOMvVSmZVY1fE6#1fuMu9%gW8Ko4*OOxc= zy_PbFfql0=e}1pW?@jYvv?LsJy(}yMLx5aJ!g(5>N$$Ec)*tKh_wuXMGERdO_VRKK zovHTn!AT)DM7f4(Ejp28%d7ovfA6+58~Rf`w9 zT;`461|s?Q!gM_mwArtc23)l+uJ=5Kfq%QpsLU%6J5yeJdK^_#Z!x~I3nX0cjY=C+ z*Hf-?GkjjH;#`2d{?wR$p;IPmAdYzpcV=6$^}Dfx6@LD?U@V& zJ9*TCv>~qllzw4Z+LIy+EI0$Y^hI)iWOL+BkM8H>p``w^{OUv3X`5zJdOaOCQn{n4 zRi=_mafJP|AUv|YfqN4A3{EcJsM zqR*Xny8)TwrbQ;k6jP@XgSky#O^`QHzU&RVdlZ$-w}+(@!?_L=YY59qR5Mh+ge$hx zj?>66WqRg$cg9n=bK6hm zuZUL{TsYF%$U!FHZROCVi7$~uMj)jQi@1d{#j&&_aZ)sTBQ+h0IkX&?7Z5dP^!iME z`!oWk1GxV*m=P2fP>rzFTpGIlI@yNQD+7Df4HnCo(!N!f>1ZQ!m-Ah)kC?->d#p+Q zt!nsYWhTGqvosMMRSbU6m9<~?vGE)u_Wt87O3SeQE?I(Z5?nyRjBd=@;I=ky#JL+* zyFue2AXAj$AdRLSbK3`O;LybBV?L)E@Ul*dyy2S6+Kk%O&Gxx8WSQq3>}26|{!4H% z!HsoCTXy`WWMy!bhOk=VXF3s$m4_^h`G*cQ7}yBH)c!>2L1;9WqZ_XetLgDAVT$10 zE7|)F!zR0ZKN{F{UlFCD(2H+Ow}F06w!~TH@YGUyp}F-D6fFkayV7~DxduP~DWuZ) zZO8$`ruEPJ%cOM)(8a&qG6vc{V&TGpwJFbMyOpJS2`&%+um)9MC|GxKV zfk@`kH(hA(d^2{z!x6D@zRqS{9-&z-IrFEO==qNU;-ay(9A`6(Z80NBqxm+FiGN|% zlTE(>zI4Pvj_7oE>UvzBI(u!Z^(sM5NW-?`s1M9VXE3j+62Z-cKdKY%)4XR$|bF!bHME{!O(k_cbhYtB#r zZP0z({^dLB)`0jU$Kft(TsWk}lw|CzxuCel3a4w#y`O^~ z!1LbPx?Z<@S7xD_O>iwmmE5!az0j!L(-2~lYD_#@S zo8HR($cedP89Eh*r!Y z;F&N(;w~~(R%ggy+Fh6kbaE1jp0`&4C<(x$F&c$yuvt5y#I2k`?A2*i8SY}buYWMn z=2Ml_O2FsR$CJ3KeqnE&(_JxZ#JlR_FHdV^<8yWKNJ{FW2yCDc88e>;hH(WNhbl0S zOW3038h~_xCFItUxg;8r>_5b>;A+?`H=6D&!+dyzNu96d|V>Yp9ai{EeuQ)vNkK%kUfj3Ytbo@mxw_Q0#zM?Z*mJ|U=V7qX1>N;Es?kX#b2CKf zIF!>vYaSWS+x&Geht;tWHcmnnBaLd+H(@~c#;aTfX&$6w){vV;6?2rSMmIJ4LdZVO zi5b$mr8FblLx7_El}CgyUwkJcokgX?;2jS9=P3J`Ea&7+doh2K8iAG?9pPezljN7i zV8elfk1;7y?3k#_k*ax|_qSGr0D1pj$GH~8_F(>+*rus*P9|o&O`SiK)$qxN^{0oa zif+wvaXIdm*id`$jo?c>*j#UBnvYX7mP`r5k~@3x(udA(Twwq-?OgWj%{$_z)B?=&;HaYGaohS?KBdy06Q)>AEi@RLs6Dv%{}$~lMv zYRqXAI+PSF!|d@i!HOl(^n7Df9~!Of(d4aM3&tZoQWWPtpSPugjQ4v+7Yi=wzn3pR znPxyQ^2hpdC!;cfQxtBbs7ZEM=pmarb59;Me3RyHJ6*RUrnwEz0EQs(YwxJ_@9=wm za0JM(heGPN$LyFEwjiySwO%Sn+#4K07+D&ty(6f!NhUjtQ)xsqi_AjrhH*f9OC<~& zWB>89)bVL(w)X#it>ag43!=~rC1X-ErYf`q=7}|*w z-2i2m_fy|TY-d*1XoR_n>fUMee&l@uPdqU~(%}>3+ASk*jJiA;cqxlB5g=YxtSN7X;dMec9WW*eE~BQdJQdeh>H0y8 zghkm@I0s!g&~9SKOPwf!%*0lM{cop0K=AC+u(&Ndfxq|C*fW$F4#HDUBcfBbMa$FR zW5w61ip6d|8FS1)+BDtkuTzz;4_H<|pVgOAIHzKCm2><+jTqrv5V!6%r#oAy=`jd% zOg_16gk7-;+E}k)z=Mp7M2 zfYba?ItXKK96lq}t8ij`7>G!d#9)|1`2018#t$2G zl)1kdXX6}+I0IzdGZ<~M@fBDXeMY_taRYjSCt}4!k{lHT+GPDzbraxyq89@E+bxah zHlHI~n4qvG)secJtR2)#BdwARbz+tsNuW9f`-IoJFt|y@JlM~8(g}r3$MLcU#S!W) z7^}^a*joW)3i}q{*=?z$g9G$?XZ75;(`*+Vb_gG4q~4C{{lAC0z8%&f;+_uq;zD)s=w3gY432M0bGS(kCM!%U@sJ z%l&KV*4FR{G+LNqh>K+W9X-xHv}kkRidkigL^0OYhFKje!v*E|CkXy4B@*wNWs1V) zAxei6r2>Z$hm>I8+;rh;hHQRqu>u?RB(cX@DNr-9;zc@l<?qUhVd-KK~!QctW}9^sp*ACFu9OE+~e3tII< zx6G*0;13%p3BbHHIO6UA(c-cgPScnkK*r%dw|{?;2}JerI+uz#mbg~?bbbB_r{17k zx+udS*NhA6{mF%US&|DDWv5_GgUU$ZTy6wg&nYtxMw=xX69}tvt8k3(md5=&dXu<3 z#}^grH15&G9kOC8V#^yYf0bE6%zz$Pbo)SMQC2UC-PDwgJVJxhxGGG5NM%T z6ZY8NU)7O!k-43rOa--Wh-&N_{~_ogx;l#ed&Bx>HGW=nlIpX4Ffodtc12<3yy;%K zBgs6!H(EUVXOf*_r9~#nbD|ES+n0m*p6%*V`nBKTch(`h0{1loy?jq_+ z>h()JM#AR~K`jTAz;IQsuFvy4`N8>2jF*U<%aLIwsF@*swfp!*Ui0L(Jg&F17`NqoqW=GEA~OFo{CH-wzms@ zU^-j`rkwy7&LgxEJQ7hny=^(jGP zBf(xwwyu1D^gA=cRTwKrp`H&NtpPTmc32``PO|$2nWa^oe(JKcD!=~n64O7&%Zc}7 zI=Q~$NFWTw5rS^a6ivyT2vALmeNolcAT- zbVQx|Os1p;|47~nRd-3*#ka)luwDY_lQY`zWwNq65(iyEj(@a zK>5;&0Gnc~)BZ1Of#DE(3HY}XT=|#WbuTyXzxw2A3C^^a&Dhv@wPLj|Q`5Xc`j`QI z;44m$du-WSzE&TTtUeuty52$UZY4YzIW|7s6wV8O{eUQXJ!LlW2?<1RaFLfIHS){b zIZ$9)as*PN(g;%gItBJb|3Dv7JfCNg0(0ycUf1#zuf2 z9Om}fg>5Qa{^M+&V#r(LN&^~wDx zV0{`i^mgeQA9plCq+J2!`n!X7ijfY@KQ9S8;YTnfv6ALukFGD@8>xibV!@7zDroij zZl8v<&9sQUoU}ptCW^JbCx6x~Y72h--;y_-<&hCG(EVksTb6hIh#6yj0F8Di{sCk`^H}#S@f6J{bhO#MdMkK5q=~ux2Gn|vKN=0H({5=okTEe`{IZ<|+Wp2i z!4pel^M$B~{di4oLCPdjpCw!z|2wB|rrhf=l?yO zsp8!wZQVdy${(uZ`sW%~jlz_oXE_QeE~c)@33SZA2dz0%_u5GK7J@xmXxvCU$#qR8 zh1&f&66j68(5>dxSc;2^)*)XCHk!ubGj^dmf%@SS`tCbL?EKh-y`=l<=@yzB9;i46#J9_nPP-g8aAD{M16_nCGVc`*o` zPi>n0Glgs+9a!_s8x08d!>kb)GcFcVt9ND|ey+q%H);+n93AC+rkdARuOVFSEdD1T z_CT-7l}d5!9knSXW>5WK2+%JutGzWDMB5As3WtuNF9?oGym zlVi94InIo+17TN^0OVOQ#7NL=!2k<0xE6y-HT9zWZH)9+YIt_;o<6rao7l~pb$AnU z`BDu)-{i(Xtkvw=vk=4!jZqjUi#Fmat#hvQwr0s!XFn$-0MPBzXe|%Cv zMm81)!p@Z{LbBRnH@nv(=jnEw0vPsU&dy*)aAWP+)or2xqeDjI3v{c7Sn)e{}dlw9TZO9VG%RPI;!iWldVi$uxqMzrODH6ahTRQ60Iq|*Z zQ@O?27w#Ow*5?iTWU`t#Oa9{q@NBX2_k;?}BW^q3-kX29Ah=61j9vNv3kh9b*9`JC zSi#ce^x^E_h8gRvhyCmW$I~GirRIBmoucF)KbT*=dX+)dS$nPeB_xEkN^?f_;NGT* zPvfGY86SA(S5%la%zKSb0&UUD&9?I|+ND1Vs6k`;%GxJso6L8D!1fWvJThPk@3 za~(AoQAQ&+^!DxN7g9hcX3ah;W-H`V>fpmb&4MVR5H7!0U*OrlEj_Pi5F!heil^$tpo5|wAfIRX5cHSQH{p>k|X{*Pz71#)g*CoWmTJ;GCZB7a1p7I zDFl?av$g>%JS?t?Ei5lmVM=_Rh@(CH`2~K!)Q%L0Fu@jRa=zj_Iw)kF4y=jQ!`n^) zLAA@B3tShq>Y!Czt;La#fQkh;SLpHB^6B`o8cozkal9C@{IuBib>tzZXAL z@y5S=6YIc>z&=;=AQutZS1T)_7vC&G5ZhZ@9-h$^H=I z38sl0T2p!4ORHzJNUv$C$`!!&Y?>4hxcr+d$RzK3H$w77ydm6;;Cy4#opJO7cy=J) zPFcH4?VN8^I{Cuyk#t8J3~S)5_f>2nt+ehjYr^_eslS#0^&-0?29%7H)`Bh3JTXzO zTLa7UakqX^@ZHy#^vG_u*c_0khGKnAq?cyG=Xo9eTd}U+x?~VH1skd;3FM>%=)zu5 zaZBK7_Z%R$NXPm5%=7RJMzId_T`Jvn*l}5U#u7!HiWAw7nA9_8lL*YEE&fmcrGcg>O2M_CjjK792p+UJp zv8cCcd#wKL%hJufK8q;i{&yWT;{F3(Pc#+^u^GYAX=>5&>wlxyxc^44!xtGh!~u2- z87)sg=rqC{;tvjJ$&wrq8P`=K0$HEBE++*JjMu8Se>`fL)0Ldlt4H*{C)0 z&%WSv#Lf&%=J6aKXa07__vL$&)ynnmUOS~C4RdyJ|L~RUmE)Yz;-3^TYX)8E)QD0P7&i8#he-DrPQc{F5U|8Y^!fH*uQtxb1WO7TB^6m2vP$8G_3hq(`ES0Cy+$N&4tFdG<_@;B{yJcef z-L+`4oBcSbzNgM(>Q=5y!gu`md4q2QVqtKOOPw^PBl1Bgz#MTA061sXke)`O6a|I( z8K_4``3fC{;IRtDK1}*nGFSH0pS{KHr2F_*ZLH`h9NS=vs!67@YN(;4_B$6y7BQlZ zuFD9==gJX#IuGh*Qp<@}z8c!1v_-i#^sARUF_+`hI|4ghM3OY2Iloj-ODG@_@Oz!4 zWT#8_VSY@94nmU!!z;ZBT%!_J*B0Z>*an1|6OP{mE9vMc@^-x*TIxbA&?P9$oA;*y zam|J!-H&wE5)N+YKm3sgeSe_%gcw| z^YNr5>BO96kz%#jp+a^iw2=g^n`@=7*WF?hU?#Lli)%5blrhla;FK9cGi1azRZ7#H z@v`niA#B)$0fnwr+IN61ghJ{|L2oTQ?n^e~j1~@>=q5K-I$$R9%7s%%beY{fN73M1 zCyME!Cr>P|#?;E@Qo0NJK-JZ#iW&m%Ot#;FpbN;4p|5MttfDkFOF zja7j7sjk0Sq^!{IQ)NjnPFpgz$Y+FE<$gY*vzR=mR5b27zux+5wm!@M?r(M;*m=~b zSvccw!i??Dh~W(he+WV==W6q~lUsK@A7|9Vcq*r9tkWVRy=SK?BUL`trFcYOC`Z$L zvsL_ob*#wzz%U+~{N*|7VKm#f_(wfNhm{QfRxX|NCHB+`?_cv$8lmcIra}m2scMKZ=Y7L*exc*6GbH;& zv}9fDugvRp*`_M52G~cmdP9F&SaCfS@3^f=<2v*c6Cr!2>rx<^Nx`W6Z53~TYFU3>Ty}^BTB88!i*_b z6wIGLfvVigy?Q^81_2~x7&cj^IwtPzTuj#tjibw}OoliZ!zY#vqi8Kcw4?68e@#t}3aj$SY4$ zRBHdqqqdN$;vc=P2KeOq$JO#6xlWCyvUFc5Gd=mz1nPlh2}i8*sK<O*m8NKt?dVgrc_w=rKx&4i)BmrP)H28Of1cH zb8s5(y9~7vyp@`Lbss91u3DiT_0;e!-8iLS6?Dnk$MMa>iHu5>e~K*s0cpTmxyHdO zBcTL&W6O5#dBvrDGDMAo1*8qp@QV6?N}qhATE8&R)u-8eL7&2vzA@=`30bx;XS^nm z(=9wqMH;SwjXmrhD>aQQ<(R42w;BG)tdBRx)t<^uRmfm?=9&C5jVzJO#|ZAT=@N)| zJu6iQB>U1t@YXzTx91bdQ9N8Q7Zg*$CubT8Qw2RCA8-PR&pZPLjGGF)!$OU67%bra zu~{%%(S{$43$&Zkf%wzCO2b~hC!rVXwH@Wfp+1aJ13BpetX}|4TEQqv!Np?~G5mtA zFrECkS8|Qu`lie}ONG`XN5&LJ3OuX3CB?ITPa>TablyO)W#2DL%<^WNY5~RHd%DwD z(QxK;X&Ex6Ej=`}3tbwp3~v;>m+@U_pY^Dmh(Cvg?cnZX>#IDM4o|vc{+dz-Qx`2- z;-~2>AjE+oA;fmEK?ab7qx^3`y{s1o=vYMVK65ESoKur9i4Rc@K2&%CZLb=$sdQO? z|9LMeMW*h7EuDzDChOHr&sh)=wP(!Iha?w)nxSj=W21nzD-x}Bk-yn#rLa}WE90;X zQk=gp!#-0L%Ci6LsNs7vy0)Qd6vj5wLTeR#)o_d;QX@X8(+U_mk&W!*zVTVv;MQ9D zMUv#8$`XE8348NS$Hz_S5#wF(|C-vD{?;8ta{a>&bI5Re+WyAkQ` ze8r_!!cbF`d+~l|CmrAXh2eDGW!NZ#$I`Ryei{~Y{~ah>`XYiY;5m|DbTj^3uwJPq@m^q-aA z(R3mz$-P{5OKneAJNvVfetb0br)+TZ_?Er`cUxV5UDCm|+8J%o8BJFRqZ4sSgSgCz zLgyOXPz`RYo#<0#SNiMsnNfRtgp*JHV0T2P{^aBVepXKfEDMmoxU-v!l{pJ)hkugq-*K=QX%h6GCz4=3^Is<)yO%?_+M_ki^J~#SoW0 zgxbO5^Uy_elZ!EHpYw7)aNFsdZwbA(QG0~0k1Mt24&7F5V<33=Cl*uLPhvX(E)PH| z^702TI5#NeHGI3niWnK)+xt85ej=z- z++zui=hs2m9r}z=`@?TM(hMsPp3s=ehQ0M$Y*#ZQ`k=XZ7ASbN{h{tN+6K0q5=+|r z*F<+3Qd0qNn~-hA?!g{|LdSqQvh0w_kOL@CXt23bp1_oD=aepf`s4x3Wu-GZ^%O2g z*aAE(r?V&+`Q81PTpb!UDE>rNh9N6;z#VPQYWI$dfs z^VECLxrff}kKF)L#I@31z1zYtP<*`|j33&S>ASUZNOsHaGS`q?(5GtkXYKT_b)Jpl zwz8-3%+B~$rM(qFn9>WNFV;v- z=;e|QnQJgnNKPob7n3;r7DFe}bp^4r`!~^qp0ZRz4Oz`X@;`W2FCe6p$}-B3nvTED z7N)ZCUFM20q4#B7Hzs1qq&J!Ps5`TY5%?{_rn4+eU-^|enXPtbUXO92Hc;RX`ruZT z{zyU)vwt%~;QsXw!;S{coE@$!h06t^UFO144Fuotw@)a2qtEPFPkxGj1}f)@^S~w0 z6NSExp1<;GB$^ri*XwIN_0K&I5Y-3w9>Xws7F44mHi)^+E`bBsw0i66$OS}%x`v-c zkz#D!|GG0b_*BW0Fes!v3*<4riI{J2BQ{_c6KtTXu?wU~Y-%I6OxjY~Ozg2}EBB-% z(Yzap9VBx(%i9rCU;)&d+=ZvbZc)=}+LIiB7|orWpY>7MS_3u>#DJ%h&w|=}lev%x zPo#z3`K{}lfYA0%*Pxu6nIRfmU)(G{(d6?4UyL!{gluvX&Qa%<3a5UD)E%+oU`AS z5e+9i05=y;gBz8c_vuz`Q0@k!g7Zj)CnlD;r!=&6zKW=$0mn_Ff#Kj}w1AkAm{A*Wr_p?*sGI+A( zMq6!q9EfKb%(-8ZVX2^wfc$-MPeqc}q_;hQkha&N@w6@>(8sx_#0qW!$L8$W>tr?= zF@kKbpV#yz5@BHU(?)(GbwN25F7itzh7jqR_jY(U`;A|!a);sp zy^(S;kipkLH;%y2($NN(XAaR{{uUTaDnLWc;U6I zp?|k5rt!hMR6T*ee?%lfn)wuvnhhW7h;rfs{}@$)ENz*)UT%*`)w$=nO{*SCw0de@~6mX`?)GV7)M|CNF?O>HRw?-z`eX2sP9|yH?>vWPGZHfAZliTpt4)1TxYn(e?HWHT6j2F6I|DsyeZqCI=9hd>=;6(w| z*^K6r87ZH0cPEkg?)8NtEoT6iebK!Q_(1}`4R>+T63(R-CUXg~Ffk^~rwSQFHoM)g zyxCD3YI*b+C_4YUR;J9*rbs~!4Lk33DQYWEb0U(;({pFY%HX*R*+V?J8eB_Y?hm}k z-ci|1G_$@+Ot7YPesue4TLx&s+;xDj52b%B&9~eMYOcif?ajWPR79gn>?5T3HbS`` z6?e93?)h70yU_CK4C$K*-LMrf>ml5|sd4!hFf1;MtWs!fKav z_A=~?K^i*anA=$F*t;sv`q#qell}J)Yo0V7^Nlg`)dw=mB*O;;z;h5A!?LBWs5g1- z{-*`^J@n>NkJ4ZN-O_<}`)WXPLOjXTXEditO*e+wB3G;rXQA5v_Lb87zMe{wPeO_K z4PZ32X>&_Xz;8hL*#0}mU1u$5+lwKJuLMf?Z)|QK8GS;Oip{`#^Ui3qhvSXlImyldFM*SkoAq515H{W?G_eVKo56_~8GC zRQ=yu+JAiO>|DQ7Z>_T|QunMibj-Pg^a+2!L9S?f>7yPDdpq2t!4xy9oM$UJ!1Xo5 zA3gTDo5g!-T0>v#?;2%=ni9kQ9Sec!!A(t=Rw$bOge#fOGMMZA=Wb!62uSvr~VI_rz7}Tge;w*s~n=akjLQHA8+rEd$I$8F!2K3xAb<{xEs|_g?z& zH;~?M@b#>}|A;H<_Qd}E=gu++958{XPy3~&fOP9}ygc8<>ZWp5zk#3>?fwea2Ba-l z=!i>b{4p3O7B7MLjoHET!Cb1d6YxP+mvv%!9RBJWYjc}q$g;HW4jG1uVEpoj>0dnH z=e%I>a@UxTq8L==YXp>7xYOggV4?eKtdBo$_oMI40Zk3gmsghy74vIwwwQxY8NXE|{1Du>j)G}xN)MUL6Gf%e zRW)qgKy#espMVd?f7hM|PdTV8G~c5T_47+ zT+RwL&2C!GU6+!n4rNZ#aF&CcD-z^P@7E5&AJ|xXxq=PMM5{D3Lj>_%k%ae)p$jwy z<~n0`GS&qut+!R7B8%jq0jV{BbZ}`xpf8>N#8%Xg^ujTt2Oc$EGZ;CvKZ&0>P7S&+ zB1^H_i8$M@@ve0(fvD;SrM4^v8LOA%dzeu6$d_>>v~Oq800!LtKuuB#3mC)B*3 zv7oNzntVtxDqf{5m;VCRIjo0{pPbI8%v1R&eB6V{P=ux48pHCH-lV^{B23BJ$_H~O zCZlIl1bOm)p8AO)%N0y5d~IQ48HV}3EeM>j0{H=EDHA|ZBAsJ7&!P5=qsV6@Pm1)+ zMtM3P7YG5J;fNh^-_*$`jo$0%Faj5=_#lJQKV!KmxOP#V#=X~Vy=O%|pveRAfufe3 zRpKd7c?PHvr_YT6`?MXjJtVENK+`8NFg4;f^z4 z;Q7n$uS84!ZY6V(!&8naZ3Iy0h33+_D9?SM32~ujlLAp{dOru22yP0+NT8@xCp@%2 zEK2=O(%U7ruYo=Df60PSEM&Qg z{wFh7|Bgvuy_$Rf-)V{{^xv$-PoR(J>=T`Kn~lo?4Xzv)<6RDZR;RqYj)|;sk&(Ee zrFgNu9%{4{U~gf4sG`^vk^ZAo9LqD;*#9+YTaO>btiME-#Kir?xSFVUKW~l+ z)(F#aOa`G)M*$uK2pEDTKz=SD(s6F1SyxdZNV&sgau5ruG+932X+8*cu%v#mq~G}7 zhYig!2EtXe-6yWA2xyWY8NF(h3G`fjVTJnWVWe$SZO}&vZBR;IOQM3a6{OuGPr|g| zp=HXrW^lT6Q6~Y;ChuZF003plcE!+&Ip~#_xGeA#SD7?ue5Ny>mbNMZCSAfG{~U3x z%PoignS1xasPPQ{yu~D%>w4;7om&cTZPJ95V;Kb!sJGUfJ*#<$3K52>L( zLbIQsSGz#KQtCGCAqH}os&*?CGd&b}3M=8hxRloHff{JxE{1S`nh98BnD#N>KCu{B zK1G~jSbVqIX0n}_hr^&)AMBpNq9;w!81gqP#k#BZf( zXCUs@Z^%H<9m+Sxczt7My=&@W*9_I!_^N}Xp#i>$NL7Z!orJ_~0-YTEfZ&B*bsjHG z&`)l~wU%NT)rmlvGyEj8{;q|ew5Kl(&eEE$d5;%Y#sUCDrbmAlH%gxm0BCqqRM%?k zTQUzyQv+vqe5kfA9hMCRc?$8Y>`BRh$NHjncXmWD;H?*YaokB{VkQ=5wyX} zYA2BNM9vO67W?Eh(i0Zjb9xde_!7G?su{P&>-~?j&pj1`zHx~LmQvIM8%glVxe!0B z0FLCMcL3x2X-k}nsc4oXV$Rupgeal#L1g#(w+u+b^?A5}xY}QpRme&&^AX_libbsF z_uRGCjjD*a5^RN2L?Ni5XTK*c*XB=}equ>!Qm4Fm_a%heSJq4? z1W*YIFOVWrZv7EHA;w=VuExL**AEjj5j^tiCO^d`!VpYd;@iADrGjSz%yHn{zZ|(l z?((LEJvsY!aF^JaR{k!3MhQV-hKK7LC-E z!Fh~Qo-)wnPfmjAbwQKc>9L2AdZHdif?CNguLf#c1ziq)8N`MBct>2*7Xf+ir8&o> zr;IQ=BE#yJ7gc&>t_)vRWgCuz_(xQRbAgcRsDo!A2CRe(@8dn6=*(Gub^ovfK(e37 zw2QF3Siqj}{pzZ$9=Q*l!G8Z%TTSg#I3pV80KwI)D=Y1X*sDXo42EfP&~p@H{U?_ zqRRLY??IsuoqS#_o`rSRY>+e_&2IujVD>qEN3FfNy61CLTj1 zXTdaq+tPVyo|AVdp2bn+H6*MP`UPyGKsaWj(#RE%Y@PU(tpl1gY;t<~h=RDARr7It z?}KR-5F)_&Yn5?+2&EKWDwg=MkBO&`+S=iJ{es(lJ;ZiY)64EaSshZDo0I!1Ls(Uf zJ->X4kRTtr#tJcj4|fCj0p2Dca_E7<*xT?-;a8xZ)v9?}?PPDif#F>-E#0)thf-gf zai8&}`Ur`Y3#Obh5h{b0MP9%mXWEi~59LB?aygNNsfDm!UD|g>9MBj&k$oR>L=P8> zhotqy2R(~H9oqObb`zvN>6#lzwNG>(?yV5n+dWjgefcXYMjeRkwck+Dk!#HA|KbkI zCd<;bgWt2@6QtOLSY^;)n`G+_1!RRAxT-L&o!eM?1Z6t7yhv(|)ca`sxM48bFMLm) zcJogZy&|X7mOOd#Bf6BNeli(hy<|1i-ZO8@_a!&`>Spn`Oa^k{6rPO1LR_MR_Hi~R zLf!l*&M3Ld_B7z?8DUaV<}qTecS*usSWnm!i>5q&sMvVR`+GnIcE z9D!)Y#Sv3pH2;hZ&r*+a z=xrRN2db;scJ8_(;KnAT?_$0qj{-ph{Eg}gQH=~ag8+Yy0G#%DgG7~PV|0g&;SbP0nA3zsnjco4u_gg%=jJI(?|BqOy z=z9m0{MQ#y(Wmy#r$86=R@YTg)&Oiw`Z()fO1#bGEQ^Gq7Zwa++4QdXHngTp9Ij*6I$3e5%g(Iql(IpVSe~)a20Q~_hcM3|I^%&! zq5}gFQHdAWi>ju^A{TYHjjsgeHnMvzTd8a&vs23BJsHWKbG{lCB6x7}QpD5rG2o51 z*;cK!%8&29ZV`vbR#NwI4SO zlj;u*P^FPCYv-QE^@tRwYQ4)CVbG&K@aORSu<^kASIKZEW_g>R^cD{G4G^^aFu11N zWhmQnTcJ#RSYh;-LYRc~&~+kw4;$;1I)g}k84ZSnj(rb&Vx%9BXPsU@dgvRvD?r-ECy4TuZ{oZ4|8Z6yCUZ5uM_Lg}chX291|&hLtph>b^i z1bfCexc`FD*WNdg?1!hAOu2f8J{Nh+J9IXpp!* z6I!w`gDNJ+1K@V}hx(XKBup2_hozOp!D7E$oE%rOC{$XCYmw%GSv%61d3&tkNB=1j zcJ}d3nKL)s_OD6+Xjd*5B>g%k1L7X&FKo<+vTosecj@x9gnj=F^ED<*wJbS;t&1-} zgYCg%djt+v1tB?f8CU7_!smprQ^!!{m5~}8|G8=cz)2Cb-`ddMl?E=+U^uMCCH}WcS6Dudc=MqLuPD<(%k{AfID_3x-n;{| zpd6d<4OlJN%for9pG@yCT@6L)uTwuiAe@l8RHj7SVysq)Hi#-tW;rMN?meJHC@nV- zonypJ0Bor4?9EGbp=<#+zA%GFPkAtQaD7W5`K+et`p_xok0#PS$!5m_qxtyA^oCV$ zJrNzn!{N88Au*aF*A|~4WCmYqG~U3UpvxlS_HfEV_$PE=+#`E_Kb*3x71GGskM6zh zASr(S{3AAZ8tU+rso7ZNUhK&fp}OZ-A=giUPuB6HF127yYQb#Hm2*tH4Uv68p7UHr z24h)*e=^OvuuDX3hS&>%2ZfJ7&|h@ElqqTPu6bh0zQN$vamu4qcY-cZ$pQW^3~>`6 z=szf(h$+tn*9IeR?bH7`XRdfM@_U-*-J;E#&AeuZ@T{o+S4*(Jw#YpObmC~vyKXK0 zy1OPNvJ$!AKH}RRjHN5=TYv{B?Fm0<&aEPaBCzC!j-SSC-NbNK=DWU`Go#{DwET38 zZp&7SK+jw-$~OzdT*hnjAOGy=i#9My{MW5Dz{#jZWxverK|Hg*di|PPNMC@64O~j| z{qD(yWSJWrfAVot%l;N1YH6#-TRP{HgL*FZ3F=ElxigAze~Y5gx_XFOwfCdvJ|LV&Q^P`JjM{9G9>!%Uedd(8d51 z`)maJ?u=1WelTG2WsJp1P=P4S{^7X8iVQ9!bnyp5e?9sELo0DC@;m5+?f@9(ndC=} z32b6c#Dv8d5WNpnt2nc$40pQ!OfzJ&5$APCn6!`qlGOPfm|Gs)NfeRi7-$6*(W-Yz zS1O*hv*h`8zR`$b)k~ANxB@q{Lf29=UhV|WW+3~xwSd5vJ>(Y^-iu8e?A|Zcgz;pc zl!!+?A3`5oZzz3>@;-6AuuOKzBsiN^Hh%7g>{M5i@|~T-e{J1lnmL(LJzdf@EE99K zpIZpSlMHE`kIO1?$W6FE4_-i=_AC2rj}yoIh@NG|IWf&j(U;Iqnw{X-i0;Q63_o$9 z8=H*X+6AcGOmZj6BlL>b3z4v`v&~6@I6RYPIB`(nE@gqBA?Cevp_Yq(q_NsRs>zsM zx10B56~ZHU`K5#Mu^r1AQf;^M^5=c7k4s(4hjy!F zfhgw(&Vl-8&(yY-M~He@IOONr+^5SHP=E0dVR1KzFha*iANX$w*Dp$A6&0>+()xb-npK*gt}zzsfm!I@I*jyCfUG#N8Secg)c@wfj2^gff-uS9&-8nI8>;?`&gALAa zDg0@0n>*7+@*wa!3+W9J#xeKKL%)))F!f{^HHj;lZrF`nf|%op#{ z*?m!8md!%ys}d%Kw~1yY)~Rt3!_O~14*xKs3$PtOXtavcLdwjA{!A-_0n+(43L z8mSG>OVkYtZu{ozcEW$0<^YB%{%N;Ed%@p`>G)szuybu{W2*kI#;sdpjM*wZx+_w= z8@6ZJ<8}R_Qb(k-Y$reY>@&ejmLk`@PdH+6i(ta7hxar3teDx2avnQ$Y^Wi#uqJI# z?y%=+f;((SBPmxX1({9*ec=ig`{P2ZRnGx}YW9=Xoor2*R;)DotJZ7yTpcr ztLxp=q(t_5(IP8k->=OV<_VeT4oH<-sR90@*P&^7YDzaO75T0P9X95Ztj&i066F|0 zj_}R8?7!-a6iGhDCa0d&0Tt8_xri=~LLzlCHw9;o8= zNVooL24P*>PVTIpF42GnMgQ{WMU@8ziFrLDYM@6o23i2J5#*m-NB5o-G*%5(UtE3ri4ofZHcP}3tl#Ojnj`_*hMraIL03UP?AXwGjkTiCq`33k!P&eN8T?tL~f z@_2#pLGozIb2TZw-%UdkE&Gcmi6$Hm^d|y28rE!!z9Or~#J?snd3CXg>Myh_{ED$3 zvVX7ozGVJ|>%Mi>S7ctBH4-MB8LitHwPB1d-UHM=SZBvjs^K-W98EoqOoEw2c&nJ* zd+>}u?`;fkMu>#)mpuW$dJ;-v+Sgu3T+lybBgTU}-K?iC9_g5oer(FB$D?xt^Nx>9 z*2gHQo%TI^`DZ%ygv{Z{>?GUw+CHK7q&FL8HlDNdYdhY%pWn2ir(Q8R=dSX_Pg8A^ zw+%p?K)1B%xBSI|x2dj!gNayK4fWwUcGTZ-ri@iMq129>?fDkQrm?~%8kq)0>71j~=?~6Mpj1O^W+%`i(<@uf z;JxLC0sKxHwI{*Xdzb4wbYb<_6D0noj=um(Xo1rv?#YyzKZxp@d{Mz{@){zajP`*3 zwjf+lDBh__4b4rJO+_p;7zMXrpM?vaebG7LRxw#Q>=f3s#nlR&h-jUej)PRiDuT%A zQ%|7_Z^%E42*BN^Wb{jC!}VWBxOg3Hs`^ssNU{lPUk zRL-lxIe!a!H0WtKJ&G`tO(3lSQ!j!-167}K`J1pg9NEOOQgLMtvFU{gT~^NWtTv&e zh!xpH&nVXASI134;uJ#Op^!76ZRt%8tr2$-raYP9Z?n>Tu;%9F@Wdx_=!-{#rj`;B zZwT1_-BDrLsII=0IKy93lbHB`D2m!WG@7KHU#m_IZ_LWPtklWG{wV&pGo|Y) z$*Jz=4ioXY2liZq63a+*=Kwpvy|emOC~LBZkOAX26YsyGbZyPioRuYKbPX=$ zh_a3XV_R)OuP-6RJ-Ak2_J8g8wkaRTWSVUKuO#-C(g*V*biykgd5?k*T{#FsM06hC znLyCpe%ZYuJ=9Xgw^O#!6D*RkqI6UY2WK%+jM*D*|cyO$|8E(@hzKVDmw zudDny(&OT}>c{*50`xT)ksg2iZ^pK&74#h3N5l<|x1$boM_-7{iWl5+I5Qd*$z*jk zW^+DK7yx3cttY8gKTYBmTGaeK?~Pz*^uO?^BFh_*C?y?^wk_z1wbl82xY}X%(wDqd_@Vnr(Ao|Q;$Uny>Z|Tuv*11UY26?3dRgDfH;`Q;C1hj%}bQ zXSzPcbjV-J&tvNS%EENj&q&o{H$E0XnF@#%7(X!{{L0ra>{s`@k#qkmU(G-FWUnvc z#fG9jCb&N2L|u%UUfp3U^hCy=e5`W;vevs^QDhc0n&8&B=Uso>==}Kiu%2P#(ye+s z^03b+vxZq>g1R8ZKTAqizd<9P2#NMkE_RaX)TOGGo12@}QYzRA6(YnIsuoHPyz@G;L?^ zNXraFN^vQSiD~m0oUL`^<0iywo!) zU&Wfgk6|=byj4w(2e)7A0%}tEnp$4^|Bsyz{-6R)lg`Ys5%ke6<}G@kiabmhee*B( zcY(umF3=7=%hNQg8RsqS{<2cF&S zNPGs>{w66E)9W4I;Oq*S#rx*UQr1=3(3b)4l?<}hK8@r|A;ADu)#*enVJoz;r)w5uSBZrvyz!suAqYJLwXTX{2;igPH zSR=^0jpHoBDklEz1Ji~RBDiTdF5M@0SP}%PKX)a6t-0$`G$gIP>4m9}9m0f;WyP0J zxSpC+?z6qOb7W2yd{c;$%VB=K8pW1O7Vzqz<7TWGDAog(hxVN^h__Qm#~{<@w^SA# z2)cIDFr5e)ZIVj4h5|rlGGC1)X06|k@#rTd>-BXo;6wdngP|KrmN=($#WAoKE@ss4 z8UO-9r+4MB&_TI6oTu0vJ7#?L7;-m<7K?Rq`LQBP3Er-y_~!<2SJ(*NN@7aXxRgWu zgbJVSMqaliSfialHMWL3d%-^`tA%vizS3gPNaExs zbzgMcU4N;Ce9NV#oy@hX@-0B<(r^%AsG2}(xs%mX=u=Z%&%RjcRl%^0RMHv!Dy4&A zc+41Fd|3fwtkR}s>@BmST@MYR?pL~zkqYJgIa5*63iqnbJj>m>tctWh8rJwDbos>* zY3g)DY<3Q8jpM#)D@SE;V9?fj8xAU-Ugb>IjTqP#d7X)3Q1>B+-IUH%H<_foi~rV@ zXY+KbL8F=?g{BaOiV3ln++{t87sunK39eI(# z*UBw*{(gO!zQlh3HL^dNCeBfJIx?ZI97*v%)!g}TkE4Zb{^yeSovY8MYxT$?ZwU>7Y*Jltg#gT~}(*6-c`fqvghhz)gqu(!-O9&M&O z^&lR4py)MBrs|HB>~>87vj)^9q`b%2@kk6BJBQd+mKl3v*(r3Xt|eW#6W;~paPl?( z?`vT2d;!-gH`len`iZr1VHayxx6ox+ihs6}#}bG!a-!mC7+hE(_tYI96OYn;x}4KX*%vwD}clpFUltQ7yzLWh}y^iP4uLQTAU1g4TE z30%1;HxW^L`+ons`vUC*TLn$0qaw1-twaUlmXSye^S;xG<5u`lU(Dx~KWL}a$@d7O|{p%7= zY>HXeuQ+;+vXG5|SLia?C_NghW)(e?i>{Uu2M#-7ryqF~hY2jt{B7jF1wp3U-LMy4 zPR_>?=g_`lC!%c_W3K?T@cK3Fk3JozoaZtS{sWPb!;xEMxICNN?ys2VeB0#4k2|33 z|COvOrTKz~@VB1=ac@G1zp?RsfxU`aU(D^o8j+iuGCBD|OV&9V@vSe9UB->yf3vvw z5(#rUDjQupXOR~4fECO++qcE@NS*9TOxnA9f_(uy7j+-KEt1<7| zb0={am|nkDF@h}uC-scA#lt@NynFvG_9j8|1lMwCBk(1^)4X=tZKGeqN8-zW=yM&w zTF^7mn$DLcJCU5^^{})p`M0QrR1#d{tC-qVzI&`9VRZFfacCh8qw#t_QcR26$KPs) zhnn0J%7yIRkZ!YDgc)CsE!$E+6%)Yz_2@UeYSkxhMI}c{#{~Eb-9ot}ZcH!n=Oaw& z0R2u=FfrFXx$N2S0J^DM9SzCpiM<_+Hdy2(3LxUDej^StM$7AjD`=9>u0h zBbc)d=oJjI{eVdC6M#%tj{Qs7w)icd*e%4UumCLOpl`8Jr92UdNanw z%bq4gO+qJ!Xe9WK;IB7|hZ|4^&ESeOb56!yl`BHJhRfA0Cmw|SaSbE(*YpW3Bg|!D z4p#|`ePu1&(3UO^DyLDi4R#qTLvGnvcW~yyPOIqu2Hs;fa~~vkawQnB`=2$ER)#|6 zo70?v@!8+6bmmd2yD^m;rx8ECd=77c+iihm1(%h#79ALT>#|eWl1jN!1~E? zH!nd46)Y@=W>3ZGXxsnf;l7ns1I>L4ezCP4I-=0&V#$t{HpX1vdXJIZP|T>QnLL|) zc_mv%D=3$YL_{j(RDd1hG9)n^d<3_f8VyFYEwBT|-JVwtAe3aA z+v3244^0Ekf`7C{05{>+wu5Wn zKb|t(gsJBc)$?^g-{dxCkirNIZ8_VUt6t?qeVT@^F3-_%OZSRG)wPh`biPGv0+!F9 zGXB|e&rtfhG5lvPouw^I2rE*E@Y=QqEdLS-YPos@gyZ=xX}H5g_c?N5LKweNC>M>1 z-C^%r0ToW{EBf4hRmL>szJ}sL^sF${$9Z3^JUTB@f4y_G#3|sN_ak>@-_J>RNod`@ z(G^6a$q;OV(Q{08HO}{g*rkBTXiqL zJ)u&C92oF6*4nkCmUO>;H=n#p^HEq4i!&vw+BXPHq7ye6fN?(h)P}M8r6OQGAQ^q zH`i^78nRC+)V#2Q*+zU{i3?=3d_Sn`WZ&q0QhCHh;k&moTM@rXoUr*n*5rDOf zBY3x!liah0m(&@Jl1p(%B(e2Gg-g>jEH3KT>~*@G3l1EpzP=b`DzB{x zy#iyx!nb}iwkXIp4&VO0fum~!Ku z?wDDM*y86O>KLR7T=DQ|B2o2CIlh{D4H1?v(UU3?WN=#M<14ygD%*Qe&3u(NB+cWYqY^uI zArs4e86so3IK=qttQ4TN`bj?b6nl79iFx@~#qXn2qi@}r%|2^&u(b2X(~lYK z0n}Joq;y&EVTPoPC!S=Rg=DZW>oB?i5{wdZoSydl*&YuPv`(Z~ds7eR4OU+5^Kuuv z$g00{^RCkuyaR3xntzVQb{Y=v$$uURf9}2JAT%TR{Rcm|71t8ra6;JftW;Hg@d;&7 z@z}=1>Q0+@U|2v6h6XP=NQ81}8DjpV{#Gy+LwV(61kK?N3aR>BW-(dk z$UPEZ&wu`r;f-+V{@(XGL)@ilBvQA&`YtFu7pLgA#3m+P{Qn9wAZTwU6^lBS+{y{( zQ(n-S6kSLrFp@cBJS>5g$hPy-*AtdHYns!iS@wavUeIy3IO7N1xV%!~uSS)vHHid% zy!X<1KC@ZKz_lB5)9mA;j6*j6^z=}TYCWMelHvGdfWw~}jbgM6I}ypCFpr;BT` z^Z8}Heu5uz2Y`5UbO?6^bmUcHCoWXjsEO72{YXhM0g0S4Ix8ut*-ZEle-^bTwo7QDfAH1< zpNa*rPHm=#(wNE>pUc33CzlK9a48J_+z19rP6Q!&J8GX1F_^>_B4_yDR@Db9Iua@q zl_{ps(B>z>l$xkL7I?|;EvuuG<8{=7r~P`6KzGo5$;P(>&nLg*^CrDru8PymHEAih zW+V4}m4aKmjk8f+`YQ}m>VJ^#5+!xnSYC9jKGDA2`Pfv&Ff^Jc_4;87KC}iPXnF1K zT`9+!859ILW5xOmk;I`uD^G*W=QX8Wg92+DJ2=zCqJVcnabhMq$-Z1g^{|zynvhn; zy!Ag8s>BZ#J`@$9E$Q=-A8Nw#MbDhQfyu*KI%M3sa5YJ+#!N&@#VEN>g!@9encvTu zl%Ds-jRK{6vQV&j?#XDHk^sCa8)9mY)rA*l&<8(j0<#>p!F5> zg*3mad}!Icc|da z&6TTk;*gDQian7sT5WMTeixu6YC7J)%(62rE}l$|rTWPIJ;8|;cEVaFV=mDvTQaGg zbkBSR2H{^@+@t@jN=nCs4ik82kkTpQe1ZMXrv9&P1uN5CbgE;!Un?0uMKRfGQ<2Xnj?w}*#YpE@yF$y}(Z z1Uyn_N@=`I}$l(3c4n|C0inDDJ!%-ob_SVdXbGGo%0iH1;0AaE0&R zcO-g<=rww<(N~Wa5xo;VT397IL6o&h1hEJfLG<23^s*A7?dqaO@4ff-9QpnJ^M7XE zdEa@Bv11(PT=%K>bzR@j_ar6}G}FKgoca<&)OxWQD5%fDMiHrG?-cZ}d(NV?-;DBM zzM0I)Lf{b&xY0!@b$oeXN&^@D~hJGHy(AS;!0f@Cw*u~RiX%>YknU=wq;*-h*6kEuhJlIKNr%5sr~Y@*wXb&KZ{zf)>&*s$CB)a-unmZSAGSm} zbzHb1J0PvWn#sfA?{3|3*m z!3Y{=!%FUE!M&p9%2kD6#VF#PwvLz?V(MCj;hV3@{5XGlVVdVB?D0=$$i+IwS(N0o z%+>bS#RtOM5kIBb9!w|YcR6Sq4nAJ{5F4Bw9l(f$Qw-*ZoY*D#3j4spnZ!zM zi@|H|a6K5D&zmIh8`uYb%i_W3{^*l?oMUkwH7rQZAHbJi{}0IbunxCoMzTi zqkB6wa&29(CDpq8`h##+t%OhUZ}&%{kRCdn1rt%&S{wbG)%JNir1&aD_nQ6rS^t*# zEiq4MWyW)uKH&{Yay3AQ4I19UdYKc|YjU)I%K~LWDQH zOX(hzH{I|14CTm9_G4@konS+ymF6J~9p?BmOTO_N921#_qfI&G0eJcRATQX@@b7O5 zsp5-UM5t3O`Vqa(m6XL>jq zBQ`U6DB1l-Qsp@5lXUkmUgOKfkeS`^V(kZ5A5|wqq+ho-kLRXGq~U3qUyVrtlw@Et z9?pKpa$(AE*X}=wg^56WCrL~1HH>k~SLKgWJv-gGq0rXqQS26x*DnT~(B2u=Cur#@ zyExixcz0>xhmXpA$HhH@R5A}KZKB^l2t#0}OiE$MyS zLtPXQc${RNDx5+0pfbi&LmFEV99q57{f$ao3lOePxj)*2%u7FQf)a5{g>?~MfOF1X z+;+Td>VWaILc#@nl!9poJB!TtU+$coBubgHg05Wf(sW&-h>WvRXe1q!sFJT3b_Q?4 zOBKEbHFyCIuJ06H6bt;$NI*vWB6u6`?VMi!qL1+v~pKuZJO z9WZe|yBhk4w+lD{0SU@W6IRwh(iS{Sub}t%Zd!E!ezK6BCP7mCko!V$$Zsg%Ey0@n zq-6I1&PT4b&Og#;s6ie?h{-x8-8{PKz^5fYOXW&XmHMI&Yk?3}A$aqoYWXr(u_O9@ zH0z@b1QYajH7f|b9T?l1L;AHc$U=5qY0TaQTJ(*cZ7A@fdIGFxyH;HIk-Nmmq zO>$MlCO_jbEQFII4s^5LO={=wo@Zz?OBff4F{RkZJ~4B%w8_6XxdiQPPBP($FoZ9` zJ0-X16Y-LybCD+Ib7(&@4JWMenD#LrKCY9~L2Osy-wEfh#L@fYA_UzeX1psXcc9fIXL4PJL~S6yJ4^G{lAdv*r9+@z=ZoF6hC%} z#-5=Z<|P6Q;!)wi4gJo&Yj%&PnSBGq!sDR3G3KOLziM1j^<=KJrhiAq796Jz)VY_rnF zxoP4U*y^deq{fYvkuK$rWl=6CvLATywO5|?n9Q;Hk(WlJt+HJ54E3Ysqjbv7^4WP4 zS2jVwzMTg4ZRyLsNq^pG)YeOC*x=HXdXh)YXZ#a9h-haWUcf%@YVHu7Woye*)R!WL z>m?7F7IP~^NGeDQs&fEgRu)U+L;iU?QIj*pp5@ObrVyLH6QjyshKS=#fGdS@0=;>~ z_!bqiG)@U<+!EEpT`KZM%l`&0eb265QZtGe8mfKK7x*2FoN|Iei4;#P@khUA(`Dy5 zvlnWHz9HkW8?_N84}7_=pt>pgeCR z80<7M0Cdk_fQFR;BG~c^uoqIjNE+>i-N7P+2QakM30jXML%SYxVp_9qWUx+%MAe<{96pldwhj(_|n0J**8 z5vX-?{Q;oIed$2i9tRqYjSMYplgiYBF?tf<9{v-ado$kN-2UJ<{BFQd{?&_q>s?1| zs?^^R*sT<%P0_R>ICrE^ohu9=t?yVOJ=wceg=|Jl&w4H2u|fAxwy9S_oz8ULg$?atucd;t4ka%V=?YvF3R zuXG1(M`gkPbV~!0L@F~Ns+GsU+Wod|b|hezzamqB_De3{f%G4nDd4CBoYKFaJwbTc z3A$Hm#6ObneM#GGlqLxn-v-LvM=3ywPuTlwMR^zsTh&=rb^mFL-D>l@gSk+%;ax5+ zS(=UpW-=~*W^ZB^xDmsp?))?J5(}9y|%u%+m!}4t|hq%O0FV`a&5-GMSH~eFp z7!W05=WGAy!i-S^2_OQ89+MJ&*ZJaT)q{6Ro2X*w5N56ZyHD$XqY zTo5D6Vx=JVngmp#A+i_x<$mXj!3Wp(a^HloH`v~qR7pIQeW)P-k%g`c+7a1qIuNUg zY?zC{j^lU;VCOFEXBGp8*};lFc1d%gW3gh8fjgBTKhH4qj*ZTjK+Oqyj6g{NI4_-` zJ};48S8hoK1OBfFPbl4e=)xc^32~7RX3hDNgu-ph$&I#*{;OFLo^C6kWe%ligok-+H(D;OV&HSR-|{Na|=EB@lC4)L@ctqvS-A6+3Ne+ za4GWh{rhM2twarRY2YUNw>&q@r_5&+8+xHM!oDAtPS>zRB8=I0OL*Zl)DRbblj)^B z0B_esBYv5+9>|i;N6f!PXk)`>zvCXalR>tQ;t{!R=ez?ukqrbQDH#OUcr8SIR}~DM z4gt4{ss8bTv@z)i6;*kbmKiCh{E||eB^E>kIEq*l%NOgR3Ex(%J(J&mTt-wxMz}?P zX48XzLMSzJ*~s`%a%0hZp&m8S+mS*Vyv4?_2Bkg5cGHwEAvlgM%X)a&K0iDYM0_;ar{f?oE7Ft z=XX=ZYjp@e(+_6ji1R7@N@;aJ<2R*~8T3|0Js&8$I}R!t3!LW?3HUc-OlcBfFaL%Z z@+~R;{mC)hduf7T=evW5sV^%1!oe$Ot3=iA#28`QO*kNXhpIh#==!m(<;Lf9-e08H zZvtDN6c0<~4raEp+CrPZSZ1%c-xBUf&ISatpAaGZ<*C*47}iN=;OE#tnZqYz55med za#{I)4ZUWUoYP}YKtOS94Tv00S%xy$8}~oNw+m%<D<+zfqPzK9 zBMSK_dSQN58!}@$XT-{gLVyUa&&v8GyX8sc3{gbP*OjDE@ozR2l&#uOCH@LyeE8`I zAdjm&l#?`h@%z;6%B&zbwg`z9LBHso+4z@GpC}g53;VM zE)2icWs+v5zjtmi8$gTJDk-XcBSc#xVt)DNuMS4nn^hUgs~dZC9fFei7II}%bQ9A0Aa!U zR{FCAy2X-~Ba0t9fxM`I%-1YFNyb1l#u5up0ST&F=h{v6+v+{Kl6VI%a`;`>^<&X z`tFUqyq|Y5m@%KgM$Nsi)2TD6j2{^dHKciP{#k40 zLHj-aTZK=0uE`D`l{UAo);fe>ph`i$On4dt?JMplb{4P8a5UypP9O=Mi+atyM}+NP z@n;MEJr_ZDPidg7_7Bb_LJb7pL)V6_3r?p?388+k2&-{2w+;h_N;D4982crT&z;*E zSgJmdd>E_YA-;YoO9a3YezC47_~ue*s*Bl&(cXQ89eWWR1lt@Ft7}XNs?4y93>@$c z+H2G7t~A9&*>e5Sd#tbdp7CUr;siZkXS>NiK&TGKyZI%%YnWsoh!~HRjt-S z^17Q&wZXU`Pk65RO5^NMA1MT|8=EM5LB;c&@TOaA}GrE}C-g9d0$U zBZXW!BCQo)VfRNX49GeU*HY({2>FGG7W~6(51o$%aYJu6x#_fOa`i7gzErM`wG#!- zr~r+xJm;{65hunT$4B?E+wB|V1OH1857sc51o*I>OFiB;eUaM1SdhsH(-dhG;_&}O zYj^K=v!t1OPh?c!opR%xtSkW$_@JCfz0r2@6#ZW^ce!xFb@BG8&$1&);;9Bj4OtkP z6g!v*sXofnt0$WNKTQ-*OjE%A;`eBoy|EDah=6#2N+59OAR>TV^3i%`y%-la$SO%w zuHEJ4cZK;Nz9bMB_hZlJAB6&*R0@okb|2V}|JE??{X=avMN;M@a+}jCPfQkkygIV^ zy@Hr|NH_ZSy5XUSUr@b1O*~qhI{e-9fVoeD+*Uj!S}P-#`94neMCGQvy2eJ0_jl)+ z%oPN)2n35t@jrmP={7L-A+5fFjP6FHnR2r*2Ta24w6sdkvN?nz&ArVjxoFzy&`6nq z=t54S&wdp8vZyM+AIYXWb}hZDB%dq6f9?P7W7si9{hCALq(`)#C6XbYs-I#Iw`O3XLPys=251F=#<(U;sg>C#+jG^7gd3_zk~*odO(G7Rm-QGw2qv1v(Uw z((Vz>9xUHx(@L)yac#FLFm2q==p-TA%)dS-to3qU_;7IO$VNXEe1YsCYU2Al0%IJc z#oP2fiC+(Y4B?zZ4U=k*vS6OoZYc7Se694iQrno;hJ1hfx_=9|qgfJLjDJ#667u!c z zb$e@T4!~m{$L44-WkPIX)yV=x|2{WJeyY(%wq`v%PtA!J);E{ZHd8 zyv04|U{%A%S7QT-TBxASE*00uX4s*5(#H_Or)Cr)UiMF@V=xAk$^ss-Q=M0;%3!)@k?bCHnJpM^$6yV$2)M&Ri=lqP(#+(78@H7 z7nd#mKg#s~Tcrp2QSa>lW>AJYTH{MBs=yi%rM6Un5aIn#qRb!4jS5Eh@58&_|1i_b z9V790EbW-5EL!>M=_!P750A-}I**_oBArpEDoKN-kEqO8g^52G&))f|Bom)zH2E0Tdd=DhNEY)(#V*8MM{)o1Dz2W3^??~3K-Z-k%IFOP6QHn2G)?41xs(qBHh z!_C#SEri2BASuZWh^{K0y3j@+>(glvpHC;klt$DL1ksMtlKfGaRw=(`z+thgM#|!- z7_~nYZM5zVU)1Zc$3AP;%qO+74pO9foHzH8=^o6u!_T*|S}Q8%FR!uq+Enyt=IDGy zqXNE;kAIbixx$gUn!a?Quki}3O8o%x0>yvA-0NOkhU^(~5}LfiN$?e@Gt=1-0-TB+ zs-I|?tIQstzTq{e9Co(?=tQyOdPu@s3bkp~;lxei`a5GrUpfxo8fLUZ)n1mpR<5wr(=#Jv4UWrHOEBUWK?22)EI82DNDBC^_p8n1fqefM z0pA;l5$}f0XjoTGm&_H8{3+Y>&hgwUnDBd{JV2LkneTKAGH$9@J_Gw|jk2)rgr1~s z$s206Wp}D)HnX-j^Fw56@zr}B?G`x5^Wasb`3}x71}W#_{m(J)q+fF8cGp_Dk{X|t zY~1S5HzXw_46yMYKbfl*6v8lF?(FYHU&>;AB&XaC(}`6OuCn&ND6E_Fvt z3ryo|YNM3Dyi$=#^4gZQYF8X#krgub{e!c3lF?`b2Ukt}sLuIV#k!GL@b1ya2I### zr`pQZswf|nQ8x!^M+?ilO}BOhm|8|jF`;0Fzr@wUz1Mk0N+iqoeHgCE$ z)Gycnk%RBXoIM z36g&FD5Y76K@=kH=q8Y%Iu4e2m8HtIL6ky+JCrXUyxjkp_-PhO+66xBwC7<| z!rZye#? z!3)fk4DF@c$XUAJ&W?}gwDRc(c3i)vz|_e6Oq9gOKZSH58B` zj%|Zl`XI63FUN^7-Huw+Y$PjAJ?;CUT5WL1^$}GNWiN_5Q-ex||x&iREcTqBC zqbD=X++mf~Fz5GLD^$JV7vzY+Ar=HU^AK0#u+FN08kw&}BqB4^;oIK1vu|VYu*Kp4#X%^G`b|*z6PH6hrcPW*t|7 z0=obrM6DEu65cSe`x$H|OeOCi^~VNKWb8@TP{3-G7QjcxWY0G4RHkQd({2^Qg{Bf&Y~s% zX1wT6`7|IdZ2#u+>HyNGAf-kz^bVsQFftiz2e7+iSR9+nO%3_fA9c2uzU{I5NBtj6 zaG8~7o>uzPmEtJ?cVUmXwcVtWJ&3q}<9i1cS!v}YfR*7R?tqBjZ?AqJ4n|$2a4Xj6 z{UN$P^J13XcyOBAJPHFs+Ngj!Ocaz4(GQZKkJr^l6|lAZ6RrPb5|dXYAm1WIxkR~= z-h(Zg^{M2Fb$sj`A_d-Zz%YOJmIIgyf=O5UqIo!TqD!#@ntC&3NPNMNZ#)m&82ur`k zsc7>z31afY{(+yhR>Gr3W=3eVq#g1pVahIN(ey(eRTz-PJ4rT(W9m|9hrr8nWGvBe zvX=*`+OniYWBe?+``Z$hUa3X)Wb~d`N~wQDZ`Mu5^NWxWbMJ3;BYvoWHV)%s4{}m! zJl1{2D?Z>=3v&Ao^NqPM!o1j2RQmk1$E|0Zdmm1Y7 zCF!&FW3g=R=jEV~T->e!>I!C@dYgR@k6EevRU9HIentEv*as*Qq{WuRa(0BZr)s8N zjTb(oTR_uSv!&}kGKol=V^)@1DO@v=H@x_BKj4|z&`Y4cE6iug_u^mgV>bY=Ii=lIeZ#3<1XprJtgIOWa#wzHmZ>E8Ry_?!)H!J7*Eg2P_Ix6D8zX z1O7{A^AG=AtxvV{#^9u&+ z%?=n^nI_MuWfea`YVi`BpYw~L@*KIcV4^l5L?n#H&e|AR_Pvs=cONUtpzmC1!HV+6!$)K5L>E6&e1uVy7J z|FGkQsbz~ddwkux#G6HOR548O7Qt+n=;{dwQ&QeUtIj0DCpR}VgP+G&h8^Op*Tyz& z!eJMX)HwjeSt-O%AOwHz@#VtJXmU?sgUe?^m8ff1ew0y9P1y8Sz_m~4Sw5bkb7s?u z>KtHSWB41455*vF>F>D{L-kP|QG#rEk6Bo=et*wUC7Yx~bVICKRQ=Pha&pUlk*1r3 zRW2Qhj8#5A@~5-hveiHq2$jE4&OuB>G9RX)ow%kIN#zY~)7RXMaP6fw6G0wwP8AMw z#rC&f`|M!?_j6|b=kOz{x8#(PEPBTrPqScVY%3Uil!J-!7nT=9%G|Udc05f~O$f)W zDp-f@;r2t+64>c<^Qfxvov;rC*!uh38Tt3Ei{{` zpGQlwEVE+x)vVWpoGSi1@E%0wsILAK%ck46QsI!YRIDHLU=3sp|G}0PC}nshnrm13 zC+t$d<%bFpQFR%;5-Mz#W793ouMk@+$x(bM8IPXV%b#p2dI_Y~Rk)lr!>`vgVWK^0zkTe0N}k!|pya@eI$Pa;pWgr6Y9 zKmr7p+l9Hil&GU^anNSbgW72S4$;5A%DhHyK0}^tcN4+TQ-{|JZCCzejT(9Sm6T=Y zd^>pI>kLLZU9+!Ga?AfVjPqS#~C>>yruZb`@1Qx(#Q|Amb{FM7U5I1+}wy2;H%%DGkH1!==#PWLf-+&}WHR zO20m_%RPnp&L@x2n)Itl^L%{^T6>}ZO8M~h%_Cw@?;jQ|?M4@+R42`axwAzUX6piz zX-1S8JpG zX7EHYX{YB9k`O-0~@O2^a{VY{$_SF~*|^(Vj0oc_j_EJ&WCMiRf3YcDk)Prb|S?E^o++)T{H zMdg^xT;mS{9u|58myW=pqiB;dlVAVA3MPzOb%uUAGN8ymg#ki-x8apb z4}6VBh>^8;4?f6HoOEP!-sP2{U18MGQt0x&>+|K@Bs2P)59@c^GxdH;m68=iIN&&E z3I8|1+0DBFI*dE3a9`I@ot%S}(~q9k9PtkU?9C;xIePRKT4b%IGAg-_l2O<(PgO#m z(4&KD@tX2b3C2bGpo{)ygFyq?AMwgNo@tZieCE%P^T_vp09ioTH$%0LY3ZN)ih@}D zvrC%OuGDi?fm0GNjA*cExke;yZgb*1Mm;RP`3qvW3P4n7$!KEB(1Yct z<20C`T~?=521z*-ygb#qM#Oo47Nc@^J1NdlP*U$}q|mOI%a6qhcL|oAP5=XoA3O=2 zQZTG=*>Y*~avBbOHxjhBv<06&dZrH2!i}Hl1kzjbZFkJisGXfyQHbJ_m);i)v+Ouml#lW?6+cSmoQ~4;5SOt zdky~$U$2|k`@nJxh;BfqQ17C3_a|Qz{P{Vst}EHQxURgPpCwy91IyzcNCGD+44oMd zgJ1xQMf)47M94hGL;(j~<}t?uAeKE%y?yzV^!UEn)Z^q?hLRd7{)xp|(iO?|`SJ<} zX4~EO=9{lzH6dO5jyzQZQIQ`iP69(}sF_a1rJ}VdBpv`fEwEeJd+xh&88>Bg?Nw&60)QBdB*^J&*&s?S|Fx z*(+~?;*+rPd!&?Q^6H|%@Y6>+^@yc;ZN9~3x!|9$Fbz;#nqOAQrOIhW33qQf_vel2 zl^WYmzkdX+oT8kj@DWl1u|d2-W|m8Q>2z!yhZ-GSlINs1u8SBPoNJ^eOnFgJ+}(yQ zpE)d3&i)MQM_vCtN^x(_9)&obPcnmwXHs>&f4@YW{-~5vNANl!$Qe{DhhN5}Y3zQh zw;&ON@!2oX+r)7iw64N;>g!xq-iU`kQ~i-UOm5)iRLxCi4qaFvnG%lA@&7(@q80Wd z5Ta^Dt=mH~^~)QoWaTqo6k-|X;ZCEn;^|O50bVQT52eDjB5Fzp}?aZyH*7{B{EAjPELqv>-!? z9c+b@vJ%B1E8$w-=IOKb%iii3ZX$G{0bm|vF5LSdPCD)AjW1|;b4h=xMA76rg+>on zX;y0RrIe0CmlGV(6~9}_QbQ{ie{eE3p&ad?WZ3@R<7~$k>}Z)4o5F1dhn$vQv#Z&j z^x>9ze-{3fMtYk^K^d~F2F#nDZ1m)F2}Pu3?lu>0J$W1MBh*;u&{VW#Y^%=`Vw`Ru z%2;z6XSF-5)XuqDcZ@EPH6!q`V|IjvWN(l5Dn#!-yMNLP7&A0>= zU)i85i10;sWZ^0Li2S#cXi1ac?;#y&f5)p|-hVb7t0yXxF48FaFp+*O8 zjYHMJ7vdM8r-LgKmPY{`7p~Z`;b1AiPK*-*e)_{HZAqS5{L4pg(A(3$9HgDAe#0SP zQ~)aP(4?&&lduRx7U$n5>yq0eDy1}GD<&%;PR@o}qTZ45a_UsHae?XL`}bY(KC`_D z1MhT4-!g|NqV988VL0Zij&_&N4L>6{=>~!9P~mKnDsFCT=y;;=hY(U>wgzqC`PKw( z!__c1N3YSb`Gcp7UzM~ipr235I)->#YrkkSS(q$207G7xT4M*ZrG%b~Dxwo7o+?_StcN|| z9=Q&cY45D9#{YsW84W23DH>PhlSHLfQ_2v%)(VvlU&0{#&D-x`tD;1|;=!kj!~JQZ zGC-0i@Hd#JvLggcQeC@lthqu9+-Pqwt!@5n$YNPzm_pz%pHx4Vgnef=kwda@4%hr1 zI<$PVpE{%h-aeeTjD}iL@78>^OgavCd}e>WrL2Y#_Loe3^vJCPJfp3~J+W-$EMl_R zTj7uc(she~(1YSC9A+5lUmq({#e3D|8mxqaSrWXn;PQ@=!Q1wqA!!@}QO@}6THhpH zgLX*Kst8za*}2$iFhjs+XAaUdKQ@~duAftbM0)JAYl)?r5f3XT{*aOPzNOwxOwEXK zrg`{oA5lNw*dKHC!sCy4M}$~HjN%0E;x;SSuFD3lyx||$(HFuRdDC>t6Tb@!p`Q$F zb!roiD+l}ynRgS{xjlAUUmZ|vQp@wunQzoy*}rv%p)S75Kk@J@GV-$v7!UhlM$vf* zM47!dPHNi%@4$lWf|aNV%X1hVNiH?Mh59Gm>uzxS{0`3ldC@&z)lA0WR@BYAMUMJc zCsjPVK?_&A0Oa7zvj;caM@~C0jFk0}&I$TAwXlw!5N$Jf7MO6HTcaigV#^!g{3}~M zdSx&~&&&i@0<|T)d)G#nEq23X)p?Ln*h1vmvc}I9=j^NFf&zt75`PDkaD++JLp|v2 zvN8o4-_bozQEmRzY$r2f|58a_K>gaBf+0fZg73Ly!uN__NGD4d`OJJ)-X%J`x&s z5VJ#4)>q+|5I1;C3uuf-?`H>LXbU&z{rdOt_7&*^yDhwxk ztMrwd`3_b3w*Ce_`e$bU+2+|K69WUopHdFX?$}fbQ3#H8g>jqI@XyQ|cr@3|bEIci zXD5zF?cHpI7Bbv;Gf#s&sgRvaj{xmDPmoNHrj0zCA|qf4sI>Mt7!z|&^zh_ze-d}( z_V?$H!4a2U9hB#IRHbk0OzDenu%blc42K(pf#<667Zz~XM~-6a3e{n$^J#{Pl9l|t ziNrre-)uve;!+Mn<<6PBFKgdL_q?jSXmr^;(eDoL(YfmV{O$UkYfTt1SLOVm5{J-D zc(&VAEKheP<9$ns9x^-dDnGlzy>`~RLSIHYu6w$zPmM+H7p=h(OQ{`g%FdrOo6>Y} z3~wK39WGc$M<$H2a2F(kG#25&cxyx=ZkDpECxX$aT6gE z{Z^mHHk@^nsjy%biEl30++RKykVj32V|^n3NX;M(ORZI!>_|jPy@L)7jV&53-ul`6 zzO6gf(mPKyRcNE?PyNL5K7T&Sl>d5`RKBw}F6(0Iv^RXikw6z=F9-4xj*Z#3JBh>X%prOtkTB@Vp=u79P9VcC!T^n@q#`8@@3=}q=2 z{g#FX;X9$lLxh)fY^~mwT%|W(nhzH}tru;O3eD`y9F{l!`Rsn{_Rj!O$I=eBA*~oB zSH-Nqf3nhkEMbi6?>QA$VmuOR0T*dp@jAM2{Y|Z$;2nJVvm|kGmABVo3 zV5LNT&4lu1_Y47!Wpf4Ip~nV!I&6P;oIbZ0 z2Z{Wxf~x5;u4ISb;Q2Ez@1RmK2x@~!*9qSLk^hD`<1UYKPQ;!CF>EFBcN8=y&FeN; zm}y!cp}{Jo91i3UmAjW!isHbMA9aPGlnd$;tL27>Q$keG%GN>2G>n%*`Wvl>b3CF**@qr5_l#wQPY;4 zCz(;Wp}I_`%$*XYoiycv*d!r6Z{LIik@?yNWF7xaI21YJ@8GT_nU-eSO(ZX`l9UYi z&2@e==bD^awlJbJ9tBa=%3YjQO*#gp^w#vOQ9sAg&s4CwyGg^&SrdreT2iFJx=sCE zN=ok=qX+bnC+b7NbkC1)wdJqmsW;KpD#(Lsue7Y22vuT)#|y5A@wAD)&H8@7?Z-!& z?RS~qU4jOnA#vqz+-)OItPxn(=Tw*;($~~WDk!VetBs4zU&C!FTG(9~PM?#?1v=)V z5KG5464r2-=#-{Nd^xDNtTws`+LJj=fb%aO{K_tktlP6d>c^o(|I>>fWF?<82 zXbigIp&Xl!_EU`Vo}d3D*hu(-b%p>^@N}RAE0Z+lalnN8E*xygersj<)~}QfW4}RN z3R7mgk&_Te3xgSf1Rsd?e?MR{sd9#o`|e|&Ysm!WFj3`W1YH%(VUr9zhTCGx_(xd% zBE;ANL_sn)+OqcuEoEdd9%G&>$;fn(1L-d{i#i4-kaYh4{U$2aG{p(JGI$d@687*N zt~?eQF6HkeOC&`uJU*HfPG+t74$EszDsF=)cU`f?H^)9>YplE z`v>339 zJI-}cYS@XQn!bI(o>tcNB~t+e6$Ep(Z9tMpJbj&Tn|cj3@umazk(s9arRnjQn4Qg$ z^lN)cF~gBq@SylZ`PPGGjq3snT`hktO4-So* z{su&Dg!Ue&cRQ}AGP~nNC%e$kJhP=E8?ffuuG2YPd0_lh8AvR zF5C8GyF5ttnjHvw7u0CRB@V(?;sJKsJF||n$Q~-YmXh+d#Ck2yLt!1kyJZxlA8mXI zry6yD4|Y+tE4-k_gn7#u{uC09y^TaG`oDyn=VRnuLf#$)Yqj71MhgiS&3ya^4C&ob z7TE3l(R1lqz%d_{y?-&8?2C|10FMs0jCMZ#Tgu&HRCG%aHl8c#n}eHA7G#uFQs0JJ zx3prRUk*J^$M`iSwFr|$FU}YeUvIpt?qL0-n1o-@K0p))_{`Z*w z@1Y0IfPzg;c}CoW{Qr;Tf$9e)>Kp#MuQ#Ggsd*xpnT5e%Ud9yq@y7(-`JaBS|0FQBB&_{pLuP6h!Y5Uh!f0yc*N zE*aDd!^8~L`pXbm&e^yyogpPeyLgqXrQ)mHyeIhkPTc3!lizU0!zodVP1#(>J9VzL z5sK5IWp!b`S0T6O4IzTQAp|JqzDzMA*Bj1a z1j)|LXJ^<%h69b7{>u{SFMgpb*STZ|#E{#<={%hTzqZWlHuaLD+$*^9e&ab7)|Zf& zep@{T^$p}e6^vD@Nou8hzxww4WwEqsQp3u?#_<*JLP8$&KwRYoT1R!WURHqO$={3nKA5oAJxIXRI)g>zDm1&MO)e;buH2CW-3~h@O6~* zJ#2K}PMDapaq;V`B5FuvzU4Do*|;&0G~}kJ>_%TmmBd91YA>tt%7)}Y)H5aRgr$Gc zB{Zd(q05YHxrW*1OWspJh~p_?Il!)$lvVU-o}SDY5*>+V-6b(x>h=-WLW;@q3`=ZT zS*{r-TTxVD%&_D809fAq8^IM(iKJ2IJ-Hf4L#O$<+RlU@4IAS@i@`*hm z&xNGMbxa$;>EmM5J1XVRCG~LUm0Xv>|}Tl>cCa&W`BW z7&9olYwh0B5In|nfy1B=aO_qhuJ*&+q)vPEeB<<0gnyE^WGv7|TkcPLD3@pnZ+ilF zxbdbTGtPr!ws<9a)RB$KvJ^@WKp7|rQIoO!LjX0IpOWhDshiyc OzLezDUzNQy5Bfi++D*X# delta 31437 zcmagFWmp_d6E=z@5F`-XB_Y8jEbhVG-7QOScU#;gzybk+yTjrhAlNSM65QQo(Zlne z@B4SI>-^j9sp+b^>#m;an%Rpmq$fCH4I47Zt65e;RKs)eXlX1O*OYwVe3u>ccbZvL z#R#1dkn)c3Q=zz|iHgLyB)`Wo)i~Wf#vA3#yght{`^~Mt; zY5<9*mJ!>2U-XPag3w%c;nZQ_hCCKRU5Ucsfq4g#_Q?Tq%gng33&Jl+Ab}xhR<&@Z z#o4Ys!AAlcwT*K)7X3$`6C&5AxsXD4s8g{ZE zGtyg45hlf_Hg36O?rX@fMrwK7>%&!E@k+_%H~zj=&bE)`A|GI^sWs6teD692`ocw| zaVc?7Y)_ovuUaJGVIVBHvppG#G-D-&xmo#~Zq)0Fv&O?jGZ2n7Le`*vKi}U3CjFA6 zy=5Uu#i2^_2h7pm871ot^puZxJJxc#l7fQdt+sEXi#?2fCnmq2)(?Lb!Y`*P$4~3J zS0xe2Vl`ALHjK(v3ne5~)GP)K5xfE}32bN4zIb?+>rDTU7-bLL?neTCS154J(_lQ& zc7qagAtxg~J~vHnC;GEzx?5T%r3UKkE9z-ahc#V`CMN|qSza4y(pP_|%`TkQZzjbs zxo4^v%o<5-k7K9FIG8M&QN5Jot4e5F`BmIt=UuLOcTb40mhKynEe--AV4x-=zR3Xu zga$(qkW@(i^oRlx5KAyk)?eok{=eDq)j@*ASQX6;9K zrdaV>kO2P0w!#op(dEtFY2U;egD>3G#~b^66-w zY5_)}-B1}R`@1~9kN*m-;%RaJ@lFve7{?O=$dMk)~-=+|G(`tbo@5jB??#1{?=`bXQv&8+TgV&)Fhmm>qz<0#9 zZ=WO~em!q5_RI#~eX8Iy64x(p+r+LE_#CsT$s#<@bx6=}|X{7Cz zv1nnGL2}#h^-CKbpXz!HdL3C$&j;q1%I#QkS6&s9o(6!4S|9hy5}3%OzMZhL5%wDTL+nX z{dhew%5Fxq5p7I-d?_<7r|PaYs7&RP-*%E`H`bxfH|Ezel|H+`&0x!SLqc9f3m97N zJ%k}C07MHHqn+qj3R^7bO@A{atYOTned7@{dobX|QpK2b3E)bRY_kZS9OUa@a%Rr3 zk7YM&&mS1Ngm~8*J7BwqGe*1d!$;pkW>pj3j>3A9uUoa`Mx@^dKPOUY-Aq#dV(;LX_^GCU+P8lVJ^&t`!rEI0w@5+e znOS_gge5V5FW|n>DZ~sg6JUM+-EvIrqr@n&2REwEg4H4eTorghgHYEQm+<$U_G;?2 z=aHvf!=ELvp5#@tccEg}eYU%H^&6&3`0B$-{5S-uaKk;UQH3V5UBop0#tt1yM}3T; z$qo0@Wve~X3AYRxuA%nY%619FB#~ViAQ$6&Of@^;47x=lJfzN(NKxhe4kPn zkDn`P_??-Yf0p<-IAzK-ieRCi20BLVo6(x{AD@l9eQ=3uy%Aq%1*o}CZ<8`>WPUN# z|A*@#wfk+`o1WMDKosamo-C?DLM{#bpn)>rRqIhz%OdH7@KQH5m8?kk-g-kbC|xZ+ z-J*Qp=YFiz+m9<7s_)Npd;XdioP4IGRTbvsG|LY0t`wW}45uX=<^jbIGf6-q){upn z)m~gY?pv*zbfpGI+4SvOeNJ9Dm%t!hcweRLP%}9?cF~?>bAx` z^pD*!n;i04s{R+Z%pe+~Ix-sNPYgp$Uh@U}d`>g9KDqxBn0ihA5E@ZY`B1&I!+)Xg&X<5lWQbqc1V z%SiCl4ju*~RZnbRr-0HuGK=&HOLI*Y?Ls6~hm7R|rxoMm?1ds_CheH!dlR#_iRzRu zjC)-~@}2XwfFua}+4V8Z|4_c$ra=LeLO6fOlebR62FY7ebU*tNU%u!y>ZHW}oc~*+ z-bzuKo5q6t3p6c&{R>$7=_DJyWwUsIMf_;e)@R#>>D?QWA=SMaUq#$Y+@^m3BLrTO}o>=lZ!jLniEX2Ga2qNJX5VRG~D^G;0VNxZw%)-fe@ zxl`BKyLoZVlw4G*u1r#CxGYvYCALXVMwo}|@-e#n9SFVQZA^+|9$R#^*M^Zf7A*@+ zgTM+wlRXJEv&r?3N`Ob-kJx~pnKLwN8~O7r=cTj;ZSMXDbsp#OA%o?$@Hgphd{J z12*{hpb})g@Y{GxiteZer~zp+(*6^5Rk-FKW}9!YtFlgJT!=|`vnu)2l<0a5Kv`@ZR*-zK^e&!D9ij^Eg8K-52CosSejai-Gll!!%lwPT2$DwKLQwOM=| zDfu+)M=a8M{pKpQ(WOzEFxlZJr7k2y)Y8c?z%_9HY)>$NKh^M0@FHZm#TI4B>I^&# zBG?QQ5>GzH69lFbw5wYG2+ToFu~M4WuhZ{8D0sYdf|15qozt0;bknjle8&tandGaL zuErDT~}aGoE~V0_jh)-h<31%w_eB6B}ml(78Ap=`F1PyWWqhAdlBgxP~W6##%f4)T`*!C0W-Mq_Y zM>&yR=Dqm&g--=GN6*o($aU~7nETL`ht8rAGo@-KnOuS4doWX|aZn@6A!zZmrl&M! zT;gnZtvc1s6o&bubZ8ebaBXa&lH|+bY$yu{yru+yb%50}7<9edgB18phxV>-Wg&zS zN7CsLWs}&-fu`VZyJSdY2(?5gMqN&P47J z6~fQozCdLP(o|DGt8K!(S)cwJoDk#D&EWz+Pbx4lWj4!z6J+WNm_eHczvyWaEM(Fy zwtQo(V?Iw}GN`kzE-EDZ;#x_8KYrI2|Ckpekxl`O#3&aXhydV%B0Fz`CGMpqUKdDw z9q(u3sY-p4y)UMR{Wc1Jo1Fi?Q6t$}j}QH}Rkjg7_j0gVDnMRKpoKhr2yZS~%6bw&E8Lh`tdCkVK5%aqeDyqB>(9dvRo1#kR{ zTTk_~TcU3M#e-B+CQQ+2KWHADKTIkG%BtP0;v|(R<=Re6R4g6orxRv?-`aZtjX8u% zY-@{1Zp|pqz7|jFk3@<8;QCs%QFd?V9A{CDCv-W)h@1(EgxFuQs%0$utqJ-LjLtxp8kiY-*F~n&8Omd zXrk5E?L*-fdL5c=yxGJ{S#xiIw@<0%qI7>8WC)TFY(5t#%w}OMb%~};k`{f2es^m( zTvchx0fHpwNdJ+2JnG1}&G`QDs5_N}QWBB%q0%UhT|JyyCvIZ}HLsYxez$qI(B{X- z#V()ng@j@KX7(P&c81T0uY;>icL+k~l*XM1`u z6F&ng1p`szuBjV8L$3tb1lBY>oy+QXlMN^S*e&7!M|_d;O02$gW9;6U_Zx&gC%ULd z?(hQF63OxlS59o}s?1o$520c{aUzye5xci@Pr@fY=U3NS?oc_o|8S@>=c5xQ4Ycah zW(|a4&7Le%tE)LKGObZRt@7d2Z0-Qqgjbd%J=fk!J?O`6!`(OPc9^XJ_}xo z@8R>re69b@y#*o{RjLkW@=RDdr?t@|yW8TpNBuQYWP8lL)SSHjX`>9T{_gBON7gSC zq7G%eAR=KSE!Q^v))DJYU6J~CEqE*@8FFpcMcY}DP)LI^|Ee?;OcK@+SY6~V0^0jK z&3R~qf4Pm{5VqKM0$5O1%Cj9v?q2V~hvNDn&!RLDu8sg@#{?1pKEA3SeLpEaI(f0` zCDWG&5V-t?a9FSLZi3O+t~S`l>_zMNzZ2svUX-LKN(IbDkK?(<(x}|bX^wvR8X^DZ zR00}+=f5Ox5_wC2K;4d(4*uON0U~)}+9Vurl^}84&>b+FSX(ZiyZJ~iI;tqf(N5ng z#d(+GWL3wte$yP$Z2gSiAKz+n;9j?~E}!dpl=2UWYK3%HybitHrr*xqxW11&^MaxI zw-|IXuA@PtTn@}&)9RB>j1i4;aiNYXYIvtjN*Az~*K2xDR_#|0MYg0KgU5LSCh`y-z~Z?-h$M+f|yA&?vMY zd5vt@4m=3Upn2w&QU}pEce9Ek<8H@Tan!3J)vh)al+kBO1r=Lo(c9wfo98aN;-1_H z@88LS5pl>FObv?^_AO0DL6&#a??tY{w^MnMgc1HZeuVBd_$o@vFX8rK7)TgsfY8-g zgH@vp+z+l$UT{Ps)~zs{Km3NNjiaZ9a~pjaylY{8AHsk%6#$_&@kZxgSIk#qCF7HQ zd-%oi0--)N*gpV(&pguhO1Uc7{__%eeaBCaHHjg~B`p{%8trTg@@9QVgd{G3&h{;7 zHv3y%WL|muE6M4N3_u+EYCfjjw+z`9OyM95XI5km5)WM9-g6+XBG1gkA=|{e9muO& zKVgrBKqexnDqpH>8|RcrDw@=?r=wX9GKtwiV4sxTmdnCZDR?~{B{K&j;vJ{tg79y_ zlpzO_yL8Ck0nkZVIGwxPNP$9jP{R}Ni#hmsq0D03=FvqLGaf&Qp*IN+TFot%2G#ugBvFjo)QU}k7)tsDZcQjTi((B z#iikWi7)FW@LLo04DtE~DsSo#I5+;szoi2x*X47R4oFs=EcP3JYHgmAPhZx`o|Hj0d4AIxP zD8Q*RmHX{XK6k}l2V1#i4x<9)YCU{OWQ140?7Xk{AbeF;nrMeu5dD51^3vm> z$Rbg%LsmsF`=L+XHzMKL9eVN5_wYj*)aC~4n&*ZF3FRpw`lTIK7sW{2v;D71@W%%3 zW9;TjkVJBdL%}rRkpNt)BJg&W(0_G_u3Ac0qaE984Kb1k3E$28Pf8v->46dRUtJ=r zd-B(Me1BZnY^x1F3B0FXn)6$WE)H1S5>VFz z#yZM#fWNx5+CDcNzS_>S`V>pccCuyDqL4%z3s6jdlu5r8Pohoc1VK5iE(EN=R+9yZ zQfah`{&XG;OOA@~T;zK+931d7M0a2cN1y2ZFP(23B&Ug^miapPU?ZZ3y;T+*vh%5b zIN{~r(lWJ)<6(kIB>1f`G@$-$AASZtG~_R=(0ozzlS{!bU?47wbSl6D%!9LG`A&PE z_>vOXQv~}VPO^gvf?A;adBR#>6o*Jf5^11V%sTV{P>zDkKT|uUi|+@~3MMNBwhw?! zTl(HdPUxd<96i&AAI;_!dbkbi=~VL7eBC;a%7uk^pE#P5Qz++ZQ5&D<8@V#k96F6! z8-hi7j$+K1JftcqDFs~QPcc5x>-!Bz_Xb~uC$~j;V@Fzppp5Zb#eX3}PblK1Zq}~y zow+skEj3%~`M_fJ(y5gL$9{r(Pd<^o%__p1o=o$H3VMkMWxYWEh zB^6BCGR(t9qt)lgTu;cKqAj^K`VB|T?)cginJq7KQyv4`X`aHMIu=znrM9D^Krgz7 z_r;3c&R1n1Z|kkymuA|oB?++PUsdYCPWILnM-z^*<{sQ0$vtMC>B_?XTz_%5X5phk z@7{ojz5AN#x&c_5;;g#4zw3aGiUYljB$Y+!5D$I<2dw}6zUuno6)0cLxKm=|=i?r5 zLs5g>Q^)t-L*-~}yZ5>-FH~QamhyCl(b!8WBZWXJiy!~pr!}vqW4YIa43S#-9?UyA zT~jDi2)JI3@uxxZ&~9cs_}Aw@vTx)ZLwSkasv>hnjIx`oG8t3R0L- z*|n1Zp}U0(lsRMW!Wi3Tq1#ogc#*yxHxIiUD?Znlc=i>d-z*HuU#j~`*+oo^#cd@F z-rj$O8GfnsOUnyI`DgSy3eN5>ylH|zwFPofQKR8$4Y5(HBCH%_ANDkXzm9aw9i)Hoz`@P zrL`lnd?MrS4WAUV8R_o1%PgFws_(y6@W7Z}Pg)RNt-XQ7bK(+cp(r-s*KvBk4x=iqUQCm8Kt* zZ7kt-Dfr$_tI}X$M)6K6@5^HaP@@C2r?~uqZ`i*TBLkcNq&Y$gYrZa7f=;mB>EC;* zNhq7@>ZUm-3gJsB=bQ=fCoY^hM%fJkS7nwsukcg}6NV>SUQ{6tY?+ zR-7m>m}SK-YhXicd5z1Vat>y3|KpZH7>#;JbFhmiIfa)u`E{VP9# zyH@7RSs*cDom47A?`vXJz{bOqici()>qeD*4hzSiqM46G`%Vi!d6${KPWP0XMOcLY zW>fBx#B@I#^FFKgTCFu|cG{QYzI;~b15uhA2AlfPJ;OaZw?gD!O5Ki`6+dbwQx_*n8 z&0f8LsZ@((ld_>nV8c@eI~CqtMi~p9#@#qG0>!yp?BC zT2}kTfbx1fse*?cLJlI3RW^UPYkRHqYV?Ot6RXQ|QkV6dld5;X$P19hTrybEcBD41 zTaOD^_o^x*M+m{^tmMFs$ zOUTVN(XRBF$=s40oRCREM{HO0mqCyFly%IA4af_S(dC|O@sR5JbahM03&~O+O8Lxw$u^wi zAgMo}rYdgKny04;4#)pBNxUo|CHydV+i0M*pz*1tL@7g$Zct4m=dhhwa}S0YDa>!R zK;Fi{(F!2<_nB#B2J{V*fI>5aap$wosy{1#p6URh ze;eprEtUEDQ9U{E`@1S@-kBt_Ww)|jm?B!va3iiQ5BbiMITUvh1cd4w4iVbl&)YYM z>vGGbcxSooV>qV(`~3(~`lRlTXTayHganS4FmQJPlDddOR!^earzhhP$87;;FpTk+ z<3le$_7-4H2;L}v@N0|O|HO784J!@Oj!=cm?b9oawG>_6Njaa9b8(zcLrPMmOvHR$ zz-S+DOZRWnivt?kRkJe8YuN{U5yx%G8&QX|bX?Dv?Bc|=9CP-42dU$>VgZvqU zZ4!~KH~kRZMnRKONul00Zib7J_&z`+G4_4%ZjWttOsNus-4@R3PE?C#Z3W?tZ7_~D ztjNi)gC-SFU>(oV<_9O&TOSY?l@|PF0xve^V+2#mJu!%3pU)`OR8;0h+koZuI-=#&yIruUs8`v(tYKup)$&y>77U(F_eSADz6KMQ0LY= zRu)@0|7&eFApSj1Qt|<2)v~qdx{;%{l!A zXog#PrI|lK5$M%tft-~tcLzhr#YBMP~% zdBecN2quwv-##l_8O~+@9k@xE_Sn-Ybt{8o4cCK53GOEKL~wFXV95?p-CN@WCmTQ z`+mFpl4>=%)pmiZ{QTZr^7eq7bg_f^=nO8#;>W{CnO`OA>+GMhgW#mR;>*|(;i~sr zUHLiPTyaVW&VvIiV_eM{-XUD+WFDVtxLS|s+H1JNS#^Vc6=_H&K1K8tgcT&;x+@z! z?!bCNnJ9-c<&}l$gAIypSEeV!`#|5hfBb!%4)coz7=NHwx7wQd!*{+V&)dCcX?oY_pbs*)MC7YDN0-%Lxl zoljjs&QaaLaF$s}uXCttIo}F?EiWoy-_pubgfWlH3XDxZq$GqO@ zv1N+gJbEc{W`SSbpUBKWn?wDfdssI6|F6F9(E3mJ_Jif%%612q3=HK?b?9`%m8qq+ zdP;KCj|q;M$uY(D-~3W1H7#fP68Z?!L#V*lNxIL@KzSkZ^9cJ~)+?O67^(gC6yZE` z2HB6OVg(6P@79NuD*{%9dsSJUFw~2V{hL2ltKV$(md1>L0_j%yfMxf+HFTv6G!L2| z8k$w)MOZkF59jWkNx&YEtggCuhy~F^V7}_YF^6MxjQ1@NQS>-ZiCra=Jmk|!S3P;E ziR8j9c)BmZzMCH{Xsv0?QAOVPm4(|8ka1DZP`1YZsnEN}q8IlL*z_yXyJt_9?)2U- zdGnbQ)Ou{q7Z3$_4JHd2RkAY)eJ~B>gk1}>>8n^oT;b=`aj!JA7KNgiJb|i0wrgGK z5dJ*oWGiQJqr#)sAA6G-WXjJYu?@Lx5r7p`aJQpxt^ta&a2L0Eq1(cJ`xou~nuqoJ z6n|gkbM@=Tx|DprNo!^D=h{dW37aJBYC_*e2q|@P#>+l@>|RYZfG8nO(Ox%c3)4)!#vX&rM3ZGJm-qfloevydf4%f zsmf^)<38FHLhua;vyJ88+0I_Uu104|cWg=#MjF;+@spra?|P-2zf~uk7%|f4H78Bt ztoLd7)9xGe?k^QC1yWmxOq8ra`!uaL{sr(7cna*b6CpGr${L*1drdpIyyv1}NhRQG z-8W5!JKyJed6}7!_P5EHvvkW4s`XpfbxfX?Z3jX&8g2g(uno@6Ad}zPvj-#G24zsi zd0u~DY~SGs8v=5JP18w%j9!_5^`qWE9>I%3k2#=G9;S0N;Is|yGy?L82OO2P`b#OO z!+=U#mEJK99HzTVl($=T1WiQO{%Sp1gR4X2d`Y9WS&zH@cP@1qqz-J>70LemQy&G- z%}!U%uN01lK3S{N=Tmu+H%1LiSvBRjT@3m?&pq8}$o z!I@0{I}XpP=xVc)gU%UwUX1)l5nHgCuXkEgpgTtf7oOKi15B?UGC9mQd0bvP!`}u% zc~Fm_cljy8e$FMv-Zt0r*Ba^M6F?r3sB`o5`(|X81*$=)!u7j zs;cEPsj2}X_)%rHe28Txi)lw66(=m~ai$k<10~P)OoILzWXQ+;F=kx-Xp<)?p-a|t z&C4l7fjZ}?h=ZcMB>wC#-It)2H?f02=d0vi#h&T0R?RJ%mN?R7mU?WSmr#u3e62Fp z(~y!k?}`o^8VC;o$r)5>Z~aV4>GJ)RmGp|5d6@DWT0W6Z9KV0a>&RIBSEHreaCKgZ zph*Ino+P@_1d)Ymi(Q${tg}lbaD5mI(I;r{0l=i1GN#^Gy6L(m}J z$#Wtv)ivv83r{mPHG&(9dLc=qK_8s%pW++ipc+pgke~JIXUt=agh(J@JyQJBHOD!= ztXO^&AWFQTuzHa|GM40oaB)CvwBoSY&<-*a8#^z6*hVH(dv7K}z?epRUy z3IUn3ITna%B{@ZhND%1A_{%9eBfLQbGV+Mfi&NPfy>g=yXwGcjh(pcau1nt^Z#BtBzr%mUo{K}tefU^! zu(R&bx#gX&B)qbRTUhkEEo=ciFZQWGD+tg%^*9$2vC&_!f6VpqhYMD@ekEn==zIuS zHcRS7P~aZ2yb%3VFJ0)~O_*OZT=VttT>3vQI+I8R?0Kt%nLYlZbxspR;tT><`7gp) z1m$)2))1$nx~KArh2XZmm!Ir{0+9XJ+NjR9%nebXC95ICrk(G>=gBh>ATW$jrPAxL z_2{RY;f-hHzrl)@_Y>OYhG7Of?zw)=%b7%rpf8zKV)^8MaDa}N;E;X5;pldQ2Jrxng;lfOSP=xREV)T|MdeT6|)Reg>F!0hLmrc z>5R8d=L$GM5?n%2-VCQEx5!z(N9;j>XZ2G$0MMo=%=j^57xHN`?I)M|)GhJ06ro&G zma3d9zpgndGSHrIF+5|{E7#9hn?U@5364NH3xdztDy5sI$5-1(5msRK~o#i0ZLU707cyQ{HUl3r7*<8@dlr4glGl!1JJ49aHV7nT%@Yws2S) zx)*oexO>9%Vb_V(WhN;;7*!DNIP%a!w@R2vmEt0gCPl4I`y^NP)7j`Vea1(ABh;Ld zvKWLUr6Anh9WPmu)hExL6R%8|!S&YW8ltkW!R7PuP*e2in1U6o3Z|bV#fSN{JGicc-xHC_MWR{kPM|}ml zd^ze~eCsUcQ{qRW4Wv&Lu*P&!*pGkuDHnvmN%NJ{ecRuNCSlL`e&B>Rp8HL|NG7p? zSFB{H>=-4rnV@@{G@pZOyr2A+OXaNAz*a@5cc`SrBg1Vu)n_g)}VIw4_KYCna={idjTNs7Gr#p6@S&!Zg)9j5ULj&?E^ug9yXtODt zaQbCBD0+E4Z+eU@D5;kDF5S0F^U_Wy=`dimYO1&TDo1zLr^6tzS$|_x@NQRam+uwD*6PifN72EPx;5fEmAVS85fk zFN8SZiIDO1>-M8k$L!SCn@fg7ypydX}_8 z-sDoP^2C!G-MvSTkvD>x1u+*rs+@`;c&e|*)=qfguh3v!b;6xj8$kaVA&xi&FVF$n zeAe{qfE1O7$@_4t>k4qRnnlQv2oA$+Q0gbt>Z?mcTJ)rAm0nIMs{u{W*)lO z+5r2H1O8SQAoc|&-z|fOP4CM+NMti-x|PxA{^OZZcLBb6I_Ug#)}bO@`rBVp>qm^U zvz3PZfYPB`!zAn?c4Uf$jBrzg6)=E3B{Ngz3pzM5W0TfaSBQSrF^{E9dlSjsqnZQ< zOZ^pd;lt*GP|D*^|JjNCpPhFavK?06*=Ex$Kn>Ekr&{2Nc4oR3QzNNy2Ag*>%e}Fs z+lm3Vdy^0F#sJ5SMPX~YIs~Az&M=gj3yw@QF5l(&Tvb24^-wr8N9UuF9eI;=c~|cO z_S0cyW#LXT@WRTL?%^KxbK~j*czUvo6|T-4-wG0}94ZmX5HDaT6>!{krc=x+7cNDE zZp8n`g8>|&ufE^v&kPKTPEz=#UcFRL%04UE;Q-cq*rnDL+Ux$$^;O@Zis(*)Hck>) zf$XLIx`J*uMfFPtj{sf4Bh18g0-X&*PdSyBXp?-~wD?f!HHTmSW&vCc60i0WfARne z30CQr?*2iG^jE(*V_~!z0)VQ|QUSGBBc&|SoF%)cxf%r8D0~{XGLCqlZQ_v-N|ZOv&(0Jb&mpY_YjAKIg$UVA2m*wU-Z75pOxd;h>`!5~t zLIh|0N`;fj7}_O&U;LjFK`w7#b8M%wX+godFym;8VJ4sjOGn1V6XM*dX)azOp#l_q z>kw_>uNGFtyB;gYL)%p1E#agk?ey+<)h}AGYCR1WxY))}Nm~z2!2bRc&>5kvrna%6 z(lVPt4xR(G?p1F(-|6%5^UzH?0?^+^fG4_{^8kkjoVQ;}ulk^&Y?p^8UBRj1pbff6 zFaa?t4^_@Yjy7~y51o5;zW`~%69O5N*OGxQHt4Dp3BId!M4;&(>1y%!&Vbfv0#?m* zY1flS&2>p#ni<6D{|=t>@V_S?=ZlRG0^pnAz(CMzofHck7-0sz-d_ND0k?q!r%x{C z^_hL+X=_Ztg55u)WoSg`^=AOK+)n#`UAs@xOHI3yP-8RIgn9A&*2H!ukkl ziqOQIg&m1Wjr?5e3PPz8Pa{!HhZF-Iv-kSZ&RRr^Lx!*k8dgfcm!Yciw zv&5O3dDlRRIU5upyq%Uv~uvS;nKUK?~alx?@X|DWu(UK0o{0!gov^O`6n;u)EX(qEBOP0|E0w zNI;frtgS};8^m`d$U3`2pi{u{&k+pn&=kNtNK*IqlH2``a|TvflVyQ5iGQKrPomO53G=83x;+tsLT&;#d?to_ zEMy~S8diNV8zKP4OWIq#@~%_P-x?2blaPd26+buM&2xBd8b^`6tQo@Meh zH`wULH-|B)`l29)-jV=I}S1=`KFTOQ)jWJYcM-P;Vi{kbV(|2<~%xFU>KxQ`8l zZSF4VM8)yw);Om>G zugR=p$j<-r`gI5VNZA<$vZg?L-+sECr2)jEkVtlbKxD>S@a*&vqoP@+jhy^Ga4f%q z0cg%fzjF!L{$NkB#z{jY@F)W^?E-;(*l0M8RkF^W+9k}FC@%0G7oFbd`l30)FOFq=3BQ`@!3%CFM+uqhOxH=#}WAiIccIy5d@~j+4GwhNJK? zjgk8Kk?nNp_l_77Z#1y^{;nhEF*82@ilO;5-g)A+e0%bH3U-G0lCujMp#XS)?WBsh zRqqnZdK?sx9ywNqMTB7sQ_Of=NhJV2Pe|wa1p_>>abgzJOm{-Wwakr$VsuhqekfEq zX!EvfW@8WGq-XIYO4$FGbNOi{h}>idz>+|-1pyW0e{TZ;M-nFFA=qI{rN5s$TB1X7 zGL6AF78H*p>>x8Ha*mX9v52P1Vbv*2ATpcd0AxzgoW!}-rKXvT+xstsq5mNK15C}) zQE@Jg7uEmoAX#?(9x~!zNX5dECM9fe^2Z_D+oh_R3Ud-xmcIb@xptW}AoF1#Ce<%+ zuHR7a$@4}FOXS&W&?Mm}uCgCfNW3m~OCQmTuq`QNK_cF2VW013YcWeV_7;sZbI@vs zsjz#Q&Jfd0Y`Ft&(7?4^UXKX>*ELSOV;gdqGqE_Kf8T%UGz&JRL)y@z!x8=krJ!d0 z$}sfzTXeebKz2OE#ZHiVtO%Tgt#_Dt=y$4SX%b@;Ls{*Ykv_=PzUEP3;cHg2x`RGj z?`Gc<`1zz=0_3VYeKJv?;C#5_J^o3L;$vp886Cyk>r>z?zcva#UN=~i>VsN4Pgd%( z`tY4b45R!oaMt0+HfpKvn~Z!h^+uLyL9?`lPTstM2~#G?UJn^^r? zs4yF9RUDO)v{uM6lj#_1Xf(N!lJWX&OY4~1e%&&Y&U4>}a2RzY-_sKQy+faf@j?OC zb4q|D4wUbwjokao^IOaQqX!<29%B9zCD)knH`a~)rFa~*b_|;s{Q4-%Sc@6x50O0T z6k}x&vxTleRFrT!sQ(R~+B}&}jH^cmj4PgW?b$3=^*^ZS#UH@=@!G98%4ed;KnV|r z&v^r+G#6Xy?{7N!R|(-1VM-UIRaj%4 z1P-O`YlCp5k8wR(8Ji{(dKeu3KaIUrSRBt2Hi`td2)1Yv5?mJu4#C|M++lHdADkdT z0_@@vf(BjOHQ3@1T!Xv2ef<9CI~V8f?A7cu)m7D1-P1Ka)%7lqy|bU{NA}`%9AH1Z z6TuaDB{P6hIa=VUd7aJFS$_G4(&FgJ^EdDm6aC*As}y*Cq)?WXqp z>c*?>c;#zlhdx{T6zO@ibj2qL`fuFy05?A+e8RO%XXik08LIkW3w5fmiqz?EOZ_Ib)?CXS z{;%1%LGG(IXrt?UP4rn}1T`8*AN}4u$Llm3H_?i532R>E23MJ}zt4Y#*<6jj5GAxP z+%yjQmxVnl|KxZC!W;J`e0z=`EuUIgea-R1MspM$B-$2w(X4-uBBc5L|m2DpGsBUZ(8Vl4Glc^@m@Ns z8edZqNERUoZ!Xc7p*}re(UaJodGn#k*D?_pL;Tn;eFK6EGj~u~pWJPRqEf0?R4zA5 z;|=I>X27kK$9*v0<5gllIM^g==ZWN{L4t^imDYHCTjH2#W9fIPXrOyhh$fO8$T`gm z{i3Rm#%FFkb-uuXGvT!b?AZ4j7B~s~acM@6APb2LTVJROk8pM-$mPwCxlszHHO#hM z*+_Ej{Leivk+J{lMwZstIgC)>GdwtZFc)0{MDjzWSNWY|K{i$p{X#HU^;z`{HyVTC>q=W0>P?E zhgPO-P08rd{~7%|i;gU;)iBOPY?Rfv6zO?iq>KIz;Ipq}e`0-u9|RsDQ?c7pn~^<~ z=Ra-e&HYc~g#Q`4DetD2Jk{;tLp*rxZL!6i5Y11#@4FV@RJ2g|JzDq=p;tubUMXrP zb~`i^WPjk6@1O@cT2dA9UTRWwwgt$!pJ$VC7-hR`RC;<{&0>Q8XhG?E!0OKSl!y`r zKt9WV@?6jIlw4Iqmb2Cp<@*UV-JA#!Qcmy1_(u!fN*MbkLzn26@-ec>T*m56-IQuzMu4QSFp@ncmV2RzoXI#1zF~6uQZGVB z53uhZ1PJqk(&t1+f3 zmE3MO4?T`LJ+%X~KW_p9p&3ms7Uo-Z%f2p8K09%D>%siO&>VEu(4($;ygQi`DC{Ft z63N!Zd7v-m;72&v1M{~#N-71U$vlY0APOQ(-XWOfj=gFMdQK7V;-Td(-T>0GFR)x* z4=W9`ftIgTrvz_8HbUiUrFa_8z>Tc1D2S!yR;xxp<~{Na+_;BOCS9ukc*CCAOghvm zu?XtjC`3komd8oOK2~m0fs@XlcUD4fAJ*Q__gHy0RC1$(wm@q1?vrv-8@$$#4Gza| zR&FI3;FtJo-zBV;zt6!+2bi5Iz}{Suz5yoaP0Rc*69ClF9#!*sjk6wU9p;1?=o;~3 zF0tLbWu26In>n8aG>b!Zn{?}7J^l0tNOQZNZZPR7M56rpN?gqjq8 z6u56gX+VT*UWlIjI&DUCUo6p0gSqMpZDx8_{$emVhgC8qE^ZW11kEqMeEoX7LHW_n zv!Uc6akVG$Im=%Lm`)%_EZyCpISH>ZHI?fIzkBC*6mC8E#BBZ53RcnoXP z(76$@zTb7nnPqr*y5p1JVR#wgeFT@97tYKmk3KWUcdya<;0@92%uc zw=bMLqWyZBF~@(fE|7Lq(48j*KXcVQ2;X25twWd#OFcf->%E+!rLuD5kB3WtxFTUV zq!AONYthNJojbfNC3T|g>(orpGIyt`mo+(m2N5u`V^HJFp~_hkx`}ZZP`W$pa162@ z@jER#wPPPd0BRITBLiI3(SGx@ZK8Smcm3~~-t02`XD4#UhJC-C2=j_cs}qr!ODX(5E8;abhnxeTo!9_U}L#4LN5On2v6R*yh1$R zi3QK^)|rlLv@z1ZdA-}Mcb=-E`Tylx?TU@l-o{MS-?`~-*5d1BF z1|r&6z+mw9glLvVTmv-nE?GoODFPuU>nMBb>x3Q~_Oh@Ng_FGJZ!A>L(|2C(_x zK|p!PzwF>iJzh+kMGn#j&-rcuA;}H*j$vkYiDIVj?wYBt&<|HK2u|yAYGbXu3c58D zWGQfU%hSe?@cEpAyxQhhuQ?xPdYkD7Qj(T~X@i07M$=WgY=xgHwtR8cEZC+9G8u&Z_~5zf)f@Jzr!(dS{FkP14$48+ccb>8#6x$yT)@x}iEasDgh1;v^lM!4Q9# zwa~77g}cw6<-C+O`47&mr`UDoM1A|qM9XVrs#$QK7;T*Cm+q{E0lFa@M3nL@WFb)s zr8ruoSmgo4g)Y7t#cnxw9oCBlpGF+pb!G}x&5JCELViEEPz0FXTRP=m9F4#KF1)R} zY4FDHq=BBaMVGFF3G875m?AxdETu(8k*VVh`r-(pCN}b5>zq_R(<7Gc$q8qEf z#9_0WBeY(j$YKA8z*F|+TJL(7I5CGA$d%#+nJAF3$cDW#qP>R$@+(OEdC&T(9Lc} zR2A4aTS686i5DICtR2th*HKX4H}VO{nM2707Y2Og!TH9q{%^D=|552waIu@%=y{|t zzYs*-*`O7Miav1-RBL#@c_`v=alqYC9nWJv&8c@+HRp@-I)`Abg=!992QMJNhIWzv zx-}W*aSGvAL`&Pe1Q191+-?|ssH&r$XM%O$!O?2q*C?Q-$q@9Jj!5Fo*W>PN5*%it z)g^{tL)*Mn*<+-GfAqm5pP3WS8PuCb)S<{{Ste(+2Vo|e;5%cxy=+V~emubOrpDW!eLs#Ypj`sJ+ zHM4V%m_86uLMMfwcar$k92J@&Ok2xCyDK`QwgFmQM7)lc4fc#ID-2y?dU*f!2;#E( z!id~Z?jbCT3};P-;0F~1>Q!FrK?f{W4(8tF;G?#03b7%Wu!9;x5<9qVC`H1)f+BGg z07X-62DPwCg}c6W6VOy(4GpBkH(tP((U(fddu7HYU*iwhL(3KwQeEF(@&@rX2U{X0 z5rcjzPaz6I0r|vjy~Sb^;=GR3o6QH&!@1T%|2Sj<%C3Cf^nBuxNuuND5)))b?no|J zrnb#ukyx`gLHUOPmi5yG7~zq(`g#5i_;Dp9b!jXYg%iB+o9faF+aX6PB_|8C85!Ll zoJ)0Ds_-5Bk!_Doa_KC_Y@1|;i|u#7+S|O?88*`c8PEZV;%RjQirdhzynXrqcB`ugW+&ia zeP!WA_acSQ89pt823cMs%eKz5WTgXCYeIm}xVcX*k$d}$M(%0c7oBUK9&oF>V7(ebB1Ky{`e_soz@ug9};-77%7PD*1 z{KkheP3B*= zN#96o_fd+5#H0aY1dB$^YvNe3W1&+`**|Kgqkrf~YHz#syh_XcCLOz#prJ-=%@u<= zS}@2yX*sA9+-3x!!mFRPTzDhZ96?T(bNF9m;W}^X5BlALAMCb6^F%%X z2UzZQ!x;3mnM_kg{xRwN+W$!ze)w&Te!{8kpGBM5*eD0^R-E+}qSFCOGGH#r=q zr_&5Ov2Tg%52xdW;5#_dveKDtB}kL`kC}&|!aQ|t5KqOc3{mBB_5N5YP`t6d*GXWM zNRd5T|FAJ8n*@hQ4Th)%sNSP2b`E($LS1`8ly$<^##!Rn{KR+I?W=#l#hbbhf}h3g z*~=RFKLfeUd;KBChXKyR`egArS9I*#@8-<#kO#s#XjY608<#!pRswQJ5-_iqb1|i~ z^>4p@{5<rZ&UQuw^O~rDiMP+w`~mauX^7^f>!y z&OxC5^ZUmgl4Q#i<37Tyd1+62awpmQdes*a77DAU;JGVS7k3mD^xiLvA19>CFY@Mx zK2!sPUb!D)z)wt5q~^G1XUZa)W#pUkw+p0FRI$hdNK2gSfmeBnpK;isV+S1{?qWpG zn@jEqe{w(T8``?g-LH?V?QN{kKcrb?`Vw%S(XJ7_e|OKyM){mw(Bjj^@Ft%+Sw}eq zb@`mnFs<)a!iAfe0xzlKg;`*DMwYDWsUHPI)+1HcOwo0&2o?x^QEXG-VhjckP_b7L znFhV$3MKl22fc9-KI3mQS-AK`s5_zjK6=$*ra8j}G|{^&PaS&WcU$op>0QiU{3?L& zp8)}cL@K^vtBPb~=yZ*q^Kr*9))GNwjDa#t8q~0u zZUU@OxJ~wp$t_^9T~*{nrJ5%d-<5cdpT-Mx^Z1Ot9|fD~@))0OB{Y9lC=wU-nbnAvg*I4*unCM|wF zw0ijKR2x@W3BLZ)hsJ>IT0Q`KY&QE7&s4}TUQPkEuH^Sml*GvcS%?S z^9wW%2%+bDBk^9ugp_)ei!E({9&t!5#XEqG6n)q{_~IKn-isi@&|ij6(JJ2=pD~-7#A?I1@g~Ijio=9GuhNp*3X~ z1v%l=$e4}p3w-%uOROCWQpp?>1&M=+_E^xW$AjIk$%qjP1$>4Yrn7%V&3=l3n8p|_ z&ApeAaa`k5-TWwz`s)UDA@HGj=2dmOe#CPQ($p#3x9Eca+E22S2|$gxN)bWVwCPkC zeGYn~1vaaYfTismY#|sSE_c1x*Dnp4-fHDhx=u^bubW-&aBf7ae-K!)=!fp~gl}wC z;=ayehhD#hFtZ7CpMtxFazA{B$NiSRy_VTG!nD#cxZI}i7oj|ZcQCU>nS0H0oBmu8 zDS7)CyJFCl^cf^|2Y8>ti;`jbu4MZha7*jt4*ETup#bq<8TU?^TU-BufA4Z9H#<%f zSzA=|aGBo9mw%sPWo&nhEGf~I=}l-m)UQ|eKX04qho$aw@c&lZR+_FaUNg~FvvYR7 zX$Jq`D&34CJG6}hzAw5^u0wd}D*U*A4LU+!xw5WGWGQQ00;K&v*H3+4>*}8M{7P}Q zErtKJT!yCKOM+IrKb;uC-$Uwq`xHYn7Uq5r*P>VKt_o!$6#7iG`PMqn@berqW>@9R|)Tf!O|&zJDzTX6>t zxRCPg_V!R`#mrFNTEDVS`BiYaoc??0YNnY0A^&FR2_U-PN!%o{pJo5$(`MM(`lWSE zjly(JG^g0qqS5{3>t=STRCTG8Sw=!u#${Y8>E%x;w_X{j+*WiTc471M`K|3jLRd~y zO3c-|Kz=FM!Os!``v#$uoFy_o376`2BZa(sFkHIVVeha(zD*4E)w`UFK5M9|4hNUa zi_hLt0GzLOHO9~UL-xE*1tdsXND4f0{Aker~)-Bq9hC2#_bY`cxuswp5(FQyv z`WJNTt6|+MyXY0>LhpmWzo2+%lPMOu&bO=K*?5hx8>p$kI{VRr}q1>QXQsVwq)<|+r}7xSf3)j#=J9UNko7yulwj;6)Y$k2sjzf#2L^+R7soq^`< zF-L9BWVxyvYmQ&X@4s7}#g!hQbMI0onpYP9IcLNdMX=(8vbQip{ji{p7gejQAq* zuZwCtJC5(25R=r3y`NH~)T}}UqwwHdFK_a3g`dt~NVpvmX7ULOpx+ZRt2WWR-@}~@ zI|<^YRlHVZ$R24PRz-)(guSx}`49)wr|!ADYLieGNq?b4%v_0<^Xiw%r<2`)Dd2fH z4P+-`F`!-%X;rJ}s8D+UU=}#`?+SC(q^;Zo}!vhdO0-k6mpVZ!R7Ua z>8AC^>~+e~EBtYK_4|h`Sqcbr%Lb89N1vFa=*gQ~+nk^8o9~VeI1u%{eozQIkhY^jo6|Xe_|lTMA`0jbF_IExZwGt8K+C7EnPP_ z0Iyqhv(T54IGO_T<}HtWqk3L+>D_0jdfFdwMQs%ee>Sd$m(y*cXvr=H4j^@RKAmS? z$xH(N#Mz@D4DC>QG%1|OfQ6n?4x9RI^de0o7(I!$2M`k%kJtK4Nb`bFkXNlR&hlSip{K+#5=}&EHgN4z)s}J~ z+yl|&clWrvqivkZ@N+S1z4JLbuxK55O9z;*6(V|R)4;vk`vF_V3+=*<3e+$b>vSyh zd9~TT;mbZMScANlE`yf3Bgl30SC=;nx!1K=MD_KjO??CfEdYC>$KQT~-InBPwaJKb~P(c{nt?iGMMR06`J$<9O&7y{(TNgUCXAHep_+U|eq&^SM zLvZ~W9o<@-?5Mcpf@XQM@4uh7GGxd1K4v?uigd$gd{R6MN2M<`skru1Oc#KiH>9wy zlRBI_L_?5O4VADV>cT9JL&3+*;-DHoJ&n4lR4SVK{qNMkWN;3c_{;6`EW!Tix6ZPk z@s8}TRm3h$H>H#F;x8ocf4lUsIv#2<3f&?2p0;5@S6}iwW+2&KdEi=PbA5V@g<-m>~&PgpCh6B@mjYM?BgZig|AL8>j*sS2NL7UMg7y^mZ-b}Fn zRq*8)geP%0jBOb>Q{lZ!yIX1SM6Zia+)#a1Y57Lk}w=u3iSo zfq2`Pg;k`i{i`mwGR|CVQ)1n3eg=d2uv{7g*0pzjcG8&LWF`@;J{!pg!;2P=w`-Ls z9|@BIUsq5!6=MSi1UF?m+hZXNM|QOYHA82#wGQz!Yc6*Xc>h;mhC(2G+x97mzS?1hKw;%UvKzyT4rYXQTJy>~%2_X*D$rI+N%konMO= zE2Meeh3a(cVlM5#iS+B&yM%4{PE%VbFx(7yV#7nl1N*1-HIl~Z%&2laL`V1cFBnLx z+3NT0v&r&!VtvN%$81&JpQTiHrRK9|)L|6MuL?;eJ@gTe`5uD(D3Oj{`gK6+Wddld z!TZLPu?Xqf_&0x^<<|{zjG;eu+8OPA2MOGv%zY=*x~Cg^FaB{1GRBsvHj~kG-U&k)qK7g7vDto=FyY9iJBirP*!CVAuKN zCWhn(8PI5F`sul%0m6|Isr;qi*s}_sO{geb5zG{KRtv)D2fd1pO1M$FVe`lEh7SAV ziCeRqZN6P$9YQM|5B#L9$3!1Po2x)aJYnPp) zUOtx3Yy;kT_tr}}SM8|?2dY2{?{rgKEL!TS&Ad!P#}qvAO-b0!!E^opte+SMoc7!j zXNzm#oePW*9pbxln(3nkfPx>%%ZK!UA#SU*0e8%|`)~|0$B(r1?v}Kg;17>*omWh) zOu*!VHLO+n!X1QuVDl_FWO=V*ez7)03L?(z3L0iZ`SCRP<-UbkBH5SQ#vtPoVd34^ zT#10%2Zz;U`?Z=Yi~$bVKX0DRJ0Zz2M17mHD>gloCcd<^6I6Z2J_O-U9l`s?+U17S z%_pU>8_ts^JS&Vrg5_BLsYOt_@c(vKkTtjDce@(9V>EK#x=$6g+8KB=d=$bl$n~f5 z9-joFj|TnL-}ce(TR+8L|6E0;NFsA6nGW`?7H|C2Hh@crCeDI`OiPWH@Tj;L&9xw* zARO_5K3ks{tRO5kw=4z~#-HnEe#k+(bec}A6Zrx*KeYh3Ze6vECc&Im#>=c-=;(2$ zEGT!aRp&z(&qSVYNn@k?W6srUB2I_a*&a{scPiQoA)@=>7ngS#c@9v-vEerSb7noB z=wctR?qW`=NBml$HP`m;Mk(rsw5}~dN=!_WCpja&aw!5mAAEjoF7oJihh;I5FcPV%w?jaS`z4f4 zUbgMh(AKv0)+1)Wr$9ef?H??4UMS!jvbxw23NG8Yn@snx^5M$%G{;LT_o7!G-@Wcg zFUmg((CRTPddJ;i;a|H~}{+Sw5!vt=+h;xy2-j(Y0 z>uy`@h-wb=z;_ zi-k5%IV9PvT=gsharNM74G2zwT~opkcUgHy%q{d5{|xM13ydjUtBuug4IT!7wXrtDk=@OC-1$rU=m=H(>z3)w8+^G z%ZC5ka#2>A_~?}TlD5|C<5Ny`hYkoYlT6KXQ9K7)YL(wusP5W-W@P!L-SS+=M3vZ~ zDEElYJ$dT!B=>6y|i8=}2U@uD?QbzMmM?Pi?f1{>qln7bnF|7g0` zy<9+XU_m1j|8=@(s$b`uxE`tLcUiaI=UL7mG zf5oEk{)WaG6hY|v9u_at15WA($dm7v57xo2`?+X=tdCqV-hO8W&lF~i32mNNDF0o( zPXLy;#|#JlBa)yMxfEW__5NBhi)mgDC6E*69;f%@U9DOY$tU0F!# zb-y6()BAga%RI4Xe|>v+8y_U`OeOoWAzA{PKOQ0TL9}BPOvS%a0JUfG)-a|85Mcj6 zC#2JdgxT+MA)(oKMP*Rj*6@0+6p+Ae+}AdlqMT_~)(>o4+1zVkun$(fF>NtMi-_r*szlaz z(>*KsYOF9BX%*?|-!rbPzAFoGUo+ARSA#R-2aQfs&S!GFx*)=nY%spu`xcV153StJ zf0AE6k;U~87sTu~RQ{>gM&(s>PKj4AJ>3F4sPsRzr{A;GyK5FNP`2~(=DKRp*lUu znxr;R<-ph0g`V^>K^tT)quDu=(#Xnak*{`kN!o|;#CxhuTj)%KD?-H~)_bg4=0g9A z^U$@oGAofk>d4t8o5$80V(ojc1?iy>fAGMpM zbHa+7!}7_NrcjU}(0KZ31}F+8=%n_l*fCg$m|HkyIR03H_=^%=z~qIS{xOrfHKNo8 zPe#9HijmwUu_4N`DXOk4L+riCC7%1qy?G!5%u17h?PVq{Fl2U1gs-fli_;s*Y9k=V z!5TcWZqWksqi(bf;JeY4n_d_8#Y0)+-(X4l__CJ9A&_(ua1lwA5cd_9^F9-cAT;>d zV2cns-M34aS=>|q_2=VU6m8L2{E<3nj~?G9D-G*Pwr9(E5j6(Bzp(j>T^W$GGEBib zI!NTtXBaM!-%Xt`vyx6y+>NM?E{~KA`I;m!uD&G6wm>Uv=3C}^+!2E1?x`B|*J+J;y#)-f4$okW|650o2`iuxTj8u- zzr#r|mEZJ2=7{h|Na?=o_i62?zRGa_RFzxT5Dtj-qo?#g*~0W)X3O;+M*5dg5TDyB z>3@{pnFN-ot8g`cCM-U$pqgztkB}9MRpoa~9|CUNuLWCWfW<#a(g$yVi=ijHj19(WGC;J4i|C=s-`uY;& zM_?XIqf32MFVak-7^dILySird!2woeGv0II26FGu4)|{ceG_Qf*U|MxFKjE^!Mh1p z@a@@#FZm=BvLrW#Ef`rDS)VqOx)HaW%HS; zCe%%Nh@mhT$$X1G6w^*i?q45S79?|J#@&62SAstZHgwPgjI16{m%j@qZeqvr+Hu%{A; zDPbkFBdzp#W9L3CmHe%}_y-$+G`L2=CB>aCs%9|XZ<>4(T;khPgZte~4{;c~B}v4T zQ&loA&BEyl%FL=nWLvwe*5lfwtbkAMiHnuGpNCdo1r&=FVOOu?y-p`Zra+s8;AAD| zh)cf*W(BU|@_+KiWSx4K*HTS+Yvn|QYCt7F65IUswk(xk%H6jsRRH3$(O9ZUYTifv zs76;~lx6c4`PE59X!hkBNK6$-R5||@k`4UGUT(NM*c@iovd=rbLZ07`X*#UW^5422 zzFM+6gWGy7b`lm^?+2(L91YwVLQnWedC`XP*cy5_-4dk@oP;oIqVd|jK7zro`pMzO?6|sQ zS}Wzqr?~mh_(f}fxtBI>7L3~lRG1qT>;;-cE3zfh=|M2*@8dJIiQG9S07DHY&P}jgzAYISJSe&s08o8j8UJpmJVSt0{bre z?SGKMG;xa}K&WTb>eX(VQLIx{RWD{|qb@C_Vvzr$5y&UHLQP+@_%E2jL1{4pa*SI` zn=!EzQpapM!-lBEgVtPt1%(&3=L8M|u1I;d)=u0)yEdMf|Lfg*7Yzl(m^H}%vuE-< z@tdW)Rk~>C*vm0ln<4*uPIQO_yT|Bfwg^4qE?=PEB(D3aH@rG%jRxfpcqf%AT!5u| z-xWn5lOy#8TfO!*pKJ3`O!=RclUD|!&>|V%ieIKIb^F(Ptz7$@@594r7d7r(e7{}w zO6Lvg_JM;e)vw3lDJ6e{8{FL__l|xmC*5pD$bGEKikC#6{y6T=f;C~V@RtM?|AiiUB58SY&QJ77gk09rb+F=BK zhx1+`M7R!D6j96gv;ejoY-$MuvV_K#P6pTI8z#(vA%p_?zl)fUh9x^Gg0I0nson{p zM89ao1GU=_)(>UuMHOTkbDpnHUEG|ja{;oS+89LN;y?d<`aPJg1wVwDLEy)FRSio$ z%#xe82a;p=ns?fgi7%!n*-2xIyb@d_e>?v96JeCH9*vM5GHCeeDkq%3h=!Ol%WDUF zqZ#l8Kt*K~M$FB2W}ELee>!Vd47&RBa8Wvj4uPV>v!y^I?ORXqXpAxX^QFU+L16vt z)mpH_V4;{~F#F6(PghZIRct_>VwUMv%!*K3)n zQWg<*VNMm@tCQb?9)QUmx->5|WOk~#ynAgj!jUx%uV?GGCL3PG<%4rMMil{M&o~T@ zwS4&KXW?8kn6iDeG*ZWh<3AGfVSZDGFc8vKJ>;oRF#`G^e%){ABYfWrxDHF?|AbI- zkjBb(d1i90L-F~1IQh4kS=%??P@$IA2sil zLhuotnQm*QF}r`Qd5To>U#Xm{OXahxejehjyJl#p_FqAY`d^Omlhi$a>iRlBg2Rng zyXn2)m8Xg*!chzDX$uE{h8nU|rX3YRXPNJ|l4hc!4t$c7?8l@BYkqfla&Qws{i54& zphZqz6(xHC{!T(}$?g`1B~~+@4JhRlFIjX=S;Oke;`p8T`&c;~N)+Y5+cSF`>Kf@C zM$MKfx5g^lEe{j-ey88lGOzs&XjwvR>q=!=CU9!@PdbeD(J%oh2Y`^WNnsw^=;VtX?@W&rBj%D&Qz0qrrW3)IxH4&n z8i*AWrIH9WgVgd>DhI1+;PxI674t5z!M-dnDu^h#?W#LZfmt$Z%1E_5j z>MTBUWo2qO954sL2poR}r3OK2e29wFenUEvUY;oL^(|O~Hs{a~T36by_QHyxlJ=b4 zCVEQ{Q2t^HuUUz;N;MM(X%-3i@%NXVbeJp&I8+B6<|ONdp7Es&b}B{xF0NtD=q1Y< zsws902GY<1T^lwOk$HQhi$XcvF07M(Ns7qnt)ET>B88-qbFv}U#aSXa!6%89vvGHS zzOq$6Z$3lUGHeu{KQJ!9^NViy+z?+8S}E^{*G^dxdR88R+~$Ww27K+;lBgCCtJap? z(5ha`a^>Ut03#)RhCzgLeqm`(Aj$IIh%<1_w>QY`A|w7QUnJH?5K>N4+fSJ`SEw-P zHs}?Abj{Vf&4Vyvoet9T`EAF9+t~B%b2iH){5WWRn|NamtCLBm3i3iU4)S7^=&?z; z_1xAFCe2TP2qF*0e1B|-dnry!4UZlX7@G9alKSx~P&StueAA=okw(dZ{z}5=U%0|! zpStK>WAk{zGmGx46yIyF#ANr~vR=Q_Wpv<`#M|-h|Iy$iC;F(pI!sVbeP&{cA!2s= z{g1~O_-hsZWIr8d)-v7GZf?jsUXvF?mMzh5b*w>H5-$&*)f(G;i@+2MV5a%@2}4`z zC5jnHw3@FEp0&S_gZBa%6G`-K$0#7pb?N1`^$IqK$v+c-Qug%=hGPMfJ4!vkN#0$%<}wdZ{WQ7|QS zBn+&$d_iYp3?5MhH5cwrw>~s~!V7nz0MzE)T!mY3xiG2Hwc_7l!ihU>*WS`g9;=-< zraht&pHQ>M0qV-8|72~Af|rV6ubwMmNzm4d_YxMJy`4};lO_;9zR6E%EY~3!6Pwv` z>#JkZCDmOPj&xqid=cpb7+?r}hn2cr+*2|=<8MAnLb%O6 zhS*B&uk*c%*2_g9^A;9G!(iQmd8y!8bpqbL=thb{>0BWF$YvH1ov;<{AQ(pAqi;cUAO}|iq>QMTz{Q#>85>^W?2x0pg>Q8C#*`x1_i&@Hd|{ zl@heg%k~RYBJr2A+dL))LVoX?(53l_$o?&1Gj)IX19-E@-eNB^agxkf*uBOLF-VDo zWb~zDlc&7cy|0B|n@9pVS(UL9v1NAO!-^G7Q+>uWlr=2mi@vzIj}MmpFYa zWRL$)-RTZF0VFrwhYR^_DL;^6w!qc#eXItSTe!IT20E3ayA~B?IR#GseTr}3mZF2x zNJQkU9+Jvx=ibUy=+6@AZYHi0eEGuEVYHp(=YDv>h-ib~EMnpzO#X&{XuFJjL0^mj z|BY313%$`-I`uBrDB3^6ZsHuDEFQ8CBRpd{t5W0+0CcuI_AP=_eBSX;%xd5A>w+4G zJH2HJh-YrAMf`i^ulS%J`^uvwEQDqM>>G<**v71%gw*~cT-Sx12r!H2_j)I1sfoD! zXFVV8tK?#4gDRb`TWY6~ecocM=fElVX0A>>=0mM-)W*|KH;Xq09^b?? zxo7VK8TnxkMHMUGuiI2TN-N$1d?ZXC=PI)#y8Qix8wk&nNZVpkh`0U*4JCvDD z7xA~XDMSAKT+eIWC{4#!TCZh7>@bO{Xg{*z{7zz?8{EP~mna`(X7fuNP~vm@Y2{&7 z&-bCXNIWqMJHIbgK6BT$y(ef;H_s#N>qvAfimUULCQLGBxftj5A#U8_*>L1> zrra=Ynxu?sZLO5+FmDxtU6q{|$a=cQ=4>p+@#DV-2a@rbEM&QCuR63G4T$%pe>1lk ziR))l { test('shortClassName', () => { assert.equal(INSTANCE.shortClassName('java.math.BigDecimal'), 'BigDecimal'); + assert.equal(INSTANCE.shortClassName('BigDecimal'), 'BigDecimal'); assert.equal(INSTANCE.shortClassName('int'), 'int'); assert.equal(INSTANCE.shortClassName('java.lang.Integer'), 'Integer'); + assert.equal(INSTANCE.shortClassName('Integer'), 'Integer'); assert.equal(INSTANCE.shortClassName('java.util.UUID'), 'UUID'); + assert.equal(INSTANCE.shortClassName('java.sql.Date'), 'Date'); + assert.equal(INSTANCE.shortClassName('Date'), 'Date'); + assert.equal(INSTANCE.shortClassName('com.my.Abstract'), 'Abstract'); assert.equal(INSTANCE.shortClassName('Abstract'), 'Abstract'); }); @@ -113,8 +118,8 @@ suite('JavaTypesTestsSuite', () => { assert.equal(INSTANCE.isKeyword(' '), false); }); - test('isJavaPrimitive', () => { - assert.equal(INSTANCE.isJavaPrimitive('boolean'), true); + test('isPrimitive', () => { + assert.equal(INSTANCE.isPrimitive('boolean'), true); }); test('validUUID', () => { diff --git a/modules/web-console/frontend/test/unit/Version.test.js b/modules/web-console/frontend/test/unit/Version.test.js index a67fde8b2e13d..2d75ab583a2c6 100644 --- a/modules/web-console/frontend/test/unit/Version.test.js +++ b/modules/web-console/frontend/test/unit/Version.test.js @@ -39,7 +39,13 @@ suite('VersionServiceTestsSuite', () => { }); test('Version a = b', () => { - assert.equal(INSTANCE.compare('1.7.0', '1.7.0'), 0); + assert.equal(INSTANCE.compare('1.0.0', '1.0.0'), 0); + assert.equal(INSTANCE.compare('1.2.0', '1.2.0'), 0); + assert.equal(INSTANCE.compare('1.2.3', '1.2.3'), 0); + + assert.equal(INSTANCE.compare('1.0.0-1', '1.0.0-1'), 0); + assert.equal(INSTANCE.compare('1.2.0-1', '1.2.0-1'), 0); + assert.equal(INSTANCE.compare('1.2.3-1', '1.2.3-1'), 0); }); test('Version a < b', () => { diff --git a/modules/web-console/frontend/views/configuration/domains-import.jade b/modules/web-console/frontend/views/configuration/domains-import.jade index e4f95bcaa6e69..bbcb391cbedef 100644 --- a/modules/web-console/frontend/views/configuration/domains-import.jade +++ b/modules/web-console/frontend/views/configuration/domains-import.jade @@ -62,7 +62,10 @@ mixin td-ellipses-lbl(w, lbl) +ignite-form-field-dropdown('Driver JAR:', 'ui.selectedJdbcDriverJar', '"jdbcDriverJar"', false, true, false, 'Choose JDBC driver', '', 'jdbcDriverJars', 'Select appropriate JAR with JDBC driver
          To add another driver you need to place it into "/jdbc-drivers" folder of Ignite Web Agent
          Refer to Ignite Web Agent README.txt for for more information' - )(data-container='.modal-domain-import') + )( + data-container='.modal-domain-import' + data-ignite-form-field-input-autofocus='true' + ) .settings-row.settings-row_small-label +java-class('JDBC driver:', 'selectedPreset.jdbcDriverClass', '"jdbcDriverClass"', true, true, 'Fully qualified class name of JDBC driver that will be used to connect to database') .settings-row.settings-row_small-label diff --git a/modules/web-console/frontend/views/configuration/summary.jade b/modules/web-console/frontend/views/configuration/summary.jade index 9a6e553f43103..a04f0dbbf074b 100644 --- a/modules/web-console/frontend/views/configuration/summary.jade +++ b/modules/web-console/frontend/views/configuration/summary.jade @@ -21,7 +21,7 @@ mixin hard-link(ref, txt) .docs-header h1 Configurations Summary -.docs-body +.docs-body.summary ignite-information ul li Preview XML configurations for #[a(href='https://apacheignite.readme.io/docs/clients-vs-servers' target='_blank') server and client] nodes @@ -29,7 +29,6 @@ mixin hard-link(ref, txt) li Preview #[a(href='https://apacheignite.readme.io/docs/docker-deployment' target='_blank') Docker file] li Preview POM dependencies li Download ready-to-use Maven project - hr .padding-dflt(ng-if='ui.ready && (!clusters || clusters.length == 0)') | You have no clusters configured. Please configure them #[a(ui-sref='base.configuration.clusters') here]. @@ -37,13 +36,21 @@ mixin hard-link(ref, txt) div(ng-show='clusters && clusters.length > 0' ignite-loading='summaryPage' ignite-loading-text='Loading summary screen...' ignite-loading-position='top') +main-table('clusters', 'clustersView', 'clusterName', 'selectItem(row)', '{{$index + 1}}) {{row.name}}', 'name') div(ng-show='selectedItem && contentVisible(displayedRows, selectedItem)') - .padding-top-dflt(bs-affix) - button.btn.btn-primary(id='download' ng-click='downloadConfiguration()' bs-tooltip='' data-title='Download project' data-placement='bottom') Download project - .btn.btn-primary(bs-tooltip='' data-title='Preview generated project structure' data-placement='bottom') - div(bs-popover data-template-url='/configuration/summary-project-structure.html', data-placement='bottom', data-trigger='click' data-auto-close='true') - i.fa.fa-sitemap - label.tipLabel Project structure - button.btn.btn-primary(id='proprietary-jdbc-drivers' ng-if='downloadJdbcDriversVisible()' ng-click='downloadJdbcDrivers()' bs-tooltip='' data-title='Open proprietary JDBC drivers download pages' data-placement='bottom') Download JDBC drivers + .actions.padding-top-dflt(bs-affix) + div + button.btn.btn-primary(id='download' ng-click='downloadConfiguration()' bs-tooltip='' data-title='Download project' data-placement='bottom' ng-disabled='isPrepareDownloading') + div + i.fa.fa-fw.fa-download(ng-hide='isPrepareDownloading') + i.fa.fa-fw.fa-refresh.fa-spin(ng-show='isPrepareDownloading') + span.tipLabel Download project + button.btn.btn-primary(bs-tooltip='' data-title='Preview generated project structure' data-placement='bottom') + div(bs-popover data-template-url='/configuration/summary-project-structure.html', data-placement='bottom', data-trigger='click' data-auto-close='true') + i.fa.fa-sitemap + label.tipLabel Project structure + button.btn.btn-primary(id='proprietary-jdbc-drivers' ng-if='downloadJdbcDriversVisible()' ng-click='downloadJdbcDrivers()' bs-tooltip='' data-title='Open proprietary JDBC drivers download pages' data-placement='bottom') Download JDBC drivers + .actions-note(ng-show='ui.isSafari') + i.icon-note + label "Download project" is not fully supported in Safari. Please rename downloaded file from "Unknown" to "<project-name>.zip" hr .bs-affix-fix .panel-group(bs-collapse ng-init='ui.activePanels=[0,1]' ng-model='ui.activePanels' data-allow-multiple='true') diff --git a/modules/web-console/frontend/views/settings/admin.jade b/modules/web-console/frontend/views/settings/admin.jade index 862d959e1fb81..c9858269eeaf3 100644 --- a/modules/web-console/frontend/views/settings/admin.jade +++ b/modules/web-console/frontend/views/settings/admin.jade @@ -14,63 +14,38 @@ See the License for the specific language governing permissions and limitations under the License. -.row(ng-controller='adminController') +mixin grid-settings() + i.fa.fa-bars(data-animation='am-flip-x' bs-dropdown='' aria-haspopup='true' aria-expanded='expanded' data-auto-close='1' data-trigger='click') + ul.select.dropdown-menu(role='menu') + li(ng-repeat='item in ctrl.gridOptions.categories|filter:{selectable:true}') + a(ng-click='ctrl.toggleColumns(item, !item.visible)') + i.fa.fa-check-square-o.pull-left(ng-if='item.visible') + i.fa.fa-square-o.pull-left(ng-if='!item.visible') + span {{::item.name}} + li.divider + li + a(ng-click='ctrl.selectAllColumns()') Select all + li + a(ng-click='ctrl.clearAllColumns()') Clear all + li.divider + li + a(ng-click='$hide()') Close + +.admin-page.row(ng-controller='adminController') .docs-content.greedy .docs-header h1 List of registered users hr .docs-body - .col-xs-12 - table.table.table-striped.table-vertical-middle.admin(st-table='displayedUsers' st-safe-src='users') - thead - tr - th.header(colspan='10') - .col-xs-3 - input.form-control(type='text' st-search='label' placeholder='Filter users...') - .col-xs-9.admin-summary.text-right(colspan='10') - strong Total users: {{ users.length }} - .col-xs-offset-6.col-xs-6.text-right - div(st-pagination st-items-by-page='15' st-displayed-pages='5' st-template='../templates/pagination.html') - tr - th(st-sort='userName') User - th(st-sort='email') Email - th(st-sort='company') Company - th(st-sort='country') Country - th.col-xs-2(st-sort='lastLogin' st-sort-default='reverse') Last login - th.text-nowrap(st-sort='counters.clusters' st-descending-first bs-tooltip='"Clusters count"' data-placement='top') - i.fa.fa-sitemap() - th.text-nowrap(st-sort='counters.models' st-descending-first bs-tooltip='"Models count"' data-placement='top') - i.fa.fa-object-group() - th.text-nowrap(st-sort='counters.caches' st-descending-first bs-tooltip='"Caches count"' data-placement='top') - i.fa.fa-database() - th.text-nowrap(st-sort='counters.igfs' st-descending-first bs-tooltip='"IGFS count"' data-placement='top') - i.fa.fa-folder-o() - th(width='1%') Actions - tbody - tr(ng-repeat='row in displayedUsers track by row._id') - td {{::row.userName}} - td - a(ng-href='mailto:{{::row.email}}') {{::row.email}} - td {{::row.company}} - td {{::row.countryCode}} - td {{::row.lastLogin | date:'medium'}} - td {{::row.counters.clusters}} - td {{::row.counters.models}} - td {{::row.counters.caches}} - td {{::row.counters.igfs}} - td.text-center - a.btn.btn-default.dropdown-toggle(bs-dropdown='' ng-show='row._id != user._id' data-placement='bottom-right') - i.fa.fa-gear   - span.caret - ul.dropdown-menu(role='menu') - li - a(ng-click='becomeUser(row)') Become this user - li - a(ng-click='toggleAdmin(row)' ng-if='row.admin && row._id !== user._id') Revoke admin - a(ng-click='toggleAdmin(row)' ng-if='!row.admin && row._id !== user._id') Grant admin - li - a(ng-click='removeUser(row)') Remove user - tfoot - tr - td.text-right(colspan='10') - div(st-pagination st-items-by-page='15' st-displayed-pages='5' st-template='../templates/pagination.html') + .row + .col-xs-12 + .panel.panel-default + .panel-heading.ui-grid-settings + +grid-settings + label Total users: + strong {{ users.length }}    + label Showing users: + strong {{ ctrl.gridApi.grid.getVisibleRows().length }} + sub(ng-show='users.length === ctrl.gridApi.grid.getVisibleRows().length') all + .panel-collapse + .grid(ui-grid='ctrl.gridOptions' ui-grid-resize-columns ui-grid-selection ui-grid-pinning) diff --git a/modules/web-console/frontend/views/sql/notebook-new.jade b/modules/web-console/frontend/views/sql/notebook-new.jade index 8d9e8c4d9aaa9..9585e92598423 100644 --- a/modules/web-console/frontend/views/sql/notebook-new.jade +++ b/modules/web-console/frontend/views/sql/notebook-new.jade @@ -21,7 +21,7 @@ button.close(ng-click='$hide()') × h4.modal-title i.fa.fa-file-o - | New SQL notebook + | New query notebook form.form-horizontal.modal-body.row(name='ui.inputForm' novalidate) div .col-sm-2 diff --git a/modules/web-console/frontend/views/sql/sql.jade b/modules/web-console/frontend/views/sql/sql.jade index e3f64617adb1f..03015e8aad612 100644 --- a/modules/web-console/frontend/views/sql/sql.jade +++ b/modules/web-console/frontend/views/sql/sql.jade @@ -14,6 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. +include /app/helpers/jade/mixins.jade include /app/directives/ui-grid-settings/ui-grid-settings.jade mixin btn-toolbar(btn, click, tip, focusId) @@ -56,10 +57,15 @@ mixin notebook-rename .input-tip input.form-control(ng-model='notebook.editName' required ignite-on-enter='renameNotebook(notebook.editName)' ignite-on-escape='notebook.edit = false;') h1.pull-right - a.dropdown-toggle(data-toggle='dropdown' bs-dropdown='scrollParagraphs' data-placement='bottom-right') Scroll to query + a.dropdown-toggle(style='margin-right: 20px' data-toggle='dropdown' bs-dropdown='scrollParagraphs' data-placement='bottom-right') Scroll to query span.caret - .btn-group(style='margin-top: 2px') - +btn-toolbar('fa-plus', 'addParagraph()', 'Add new query') + button.btn.btn-default(style='margin-top: 2px' ng-click='addQuery()' ignite-on-click-focus=focusId) + i.fa.fa-fw.fa-plus + | Add query + + button.btn.btn-default(style='margin-top: 2px' ng-click='addScan()' ignite-on-click-focus=focusId) + i.fa.fa-fw.fa-plus + | Add scan mixin notebook-error h2 Failed to load notebook @@ -68,7 +74,7 @@ mixin notebook-error mixin paragraph-rename .col-sm-6(ng-hide='paragraph.edit') - i.tipLabel.fa(ng-class='paragraphExpanded(paragraph) ? "fa-chevron-circle-down" : "fa-chevron-circle-right"') + i.fa(ng-class='paragraphExpanded(paragraph) ? "fa-chevron-circle-down" : "fa-chevron-circle-right"') label {{paragraph.name}} .btn-group(ng-hide='notebook.paragraphs.length > 1') @@ -85,51 +91,45 @@ mixin paragraph-rename input.form-control(id='paragraph-name-{{paragraph.id}}' ng-model='paragraph.editName' required ng-click='$event.stopPropagation();' ignite-on-enter='renameParagraph(paragraph, paragraph.editName)' ignite-on-escape='paragraph.edit = false') mixin query-settings - label.tipLabel Refresh rate: - button.btn.btn-default.fa.fa-clock-o.tipLabel(title='Click to show refresh rate dialog' ng-class='{"btn-info": paragraph.rate && paragraph.rate.installed}' bs-popover data-template-url='/sql/paragraph-rate.html' data-placement='left' data-auto-close='1' data-trigger='click') {{rateAsString(paragraph)}} - label.tipLabel Page size: - button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.pageSize' bs-options='item for item in pageSizes' bs-select bs-tooltip data-placement='bottom-right' data-title='Max number of rows to show in query result as one page') - label.margin-left-dflt(title='Fetch first page of results only') - input(type='checkbox' ng-model='paragraph.firstPageOnly') - span Fetch first page only - label.margin-left-dflt(title='Execute query locally on selected node.\nNode selection dialog will be shown before query execution.') - input(type='checkbox' ng-model='paragraph.localQry') - span Local query + label.tipLabel(bs-tooltip data-placement='bottom' data-title='Configure periodical execution of last successfully executed query') Refresh rate: + button.btn.btn-default.fa.fa-clock-o.tipLabel(ng-class='{"btn-info": paragraph.rate && paragraph.rate.installed}' bs-popover data-template-url='/sql/paragraph-rate.html' data-placement='left' data-auto-close='1' data-trigger='click') {{rateAsString(paragraph)}} + + label.tipLabel(bs-tooltip data-placement='bottom' data-title='Max number of rows to show in query result as one page') Page size: + button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.pageSize' bs-select bs-options='item for item in pageSizes') + + label.tipLabel(bs-tooltip data-placement='bottom' data-title='Limit query max results to specified number of pages') Max pages: + button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.maxPages' bs-select bs-options='item.value as item.label for item in maxPages') + + label.tipLabel(ng-if='nonCollocatedJoinsAvailable(paragraph)' bs-tooltip data-placement='bottom' data-title='Non-collocated joins is a special mode that allow to join data across cluster without collocation.
          \ + Nested joins are not supported for now.
          \ + NOTE: In some cases it may consume more heap memory or may take a long time than collocated joins.' data-trigger='hover') + input(type='checkbox' ng-model='paragraph.nonCollocatedJoins') + span Allow non-collocated joins mixin query-actions - .btn-group(bs-tooltip='' data-title='{{actionTooltip(paragraph, "execute", true)}}' data-placement='bottom') - button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph)') Execute - button.btn.btn-primary.dropdown-toggle( - ng-disabled='!actionAvailable(paragraph, true)' - bs-dropdown='' - data-container='body' - data-placement='bottom-right' - ) - span.caret - ul.dropdown-menu(role='menu') - li #[a(href='javascript:void(0)' ng-click='execute(paragraph)') Execute] - li #[a(href='javascript:void(0)' ng-if='nonCollocatedJoinsAvailable(paragraph)' ng-click='execute(paragraph, true)') Execute non collocated joins] - .btn-group(bs-tooltip='' data-title='{{actionTooltip(paragraph, "execute scan", false)}}' data-placement='bottom') - button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' ng-click='scan(paragraph)') Scan - button.btn.btn-primary.dropdown-toggle( - ng-disabled='!actionAvailable(paragraph, false)' - bs-dropdown='' - data-container='body' - data-placement='bottom-right' - ) - span.caret - ul.dropdown-menu(role='menu') - li #[a(href='javascript:void(0)' ng-click='scan(paragraph)') Scan] - li #[a(href='javascript:void(0)' ng-click='actionAvailable(paragraph, false) && scanWithFilter(paragraph)') Scan with filter] + button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph)') Execute + button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph, true)') Execute on selected node + a.btn.btn-default(ng-disabled='!actionAvailable(paragraph, true)' ng-click='explain(paragraph)' data-placement='bottom' bs-tooltip='' data-title='{{actionTooltip(paragraph, "explain", true)}}') Explain -mixin query-controls - .sql-controls - +query-actions() - .pull-right - +query-settings() +mixin table-result-heading-query + .total.row + .col-xs-4 + +ui-grid-settings + label Page: #[b {{paragraph.page}}] + label.margin-left-dflt Results so far: #[b {{paragraph.rows.length + paragraph.total}}] + label.margin-left-dflt Duration: #[b {{paragraph.duration | duration}}] + .col-xs-4 + div(ng-if='paragraph.qryType === "query"') + +result-toolbar + .col-xs-4 + .pull-right + .btn-group(ng-disabled='paragraph.loading') + button.btn.btn-primary(ng-click='exportCsv(paragraph)' bs-tooltip data-title='{{actionTooltip(paragraph, "export", false)}}') Export + button.btn.btn-primary.dropdown-toggle(id='export-item-dropdown' data-toggle='dropdown' data-container='body' bs-dropdown='exportDropdown' data-placement='bottom-right') + span.caret -mixin table-result +mixin table-result-heading-scan .total.row .col-xs-4 +ui-grid-settings @@ -137,17 +137,16 @@ mixin table-result label.margin-left-dflt Results so far: #[b {{paragraph.rows.length + paragraph.total}}] label.margin-left-dflt Duration: #[b {{paragraph.duration | duration}}] .col-xs-4 - +result-toolbar + div(ng-if='paragraph.qryType === "query"') + +result-toolbar .col-xs-4 .pull-right - label(style='margin-right: 10px;') - input(type='checkbox' ng-model='paragraph.systemColumns' ng-change='toggleSystemColumns(paragraph)' ng-disabled='paragraph.disabledSystemColumns') - span Show _KEY, _VAL columns .btn-group(ng-disabled='paragraph.loading') button.btn.btn-primary(ng-click='exportCsv(paragraph)' bs-tooltip data-title='{{actionTooltip(paragraph, "export", false)}}') Export button.btn.btn-primary.dropdown-toggle(id='export-item-dropdown' data-toggle='dropdown' data-container='body' bs-dropdown='exportDropdown' data-placement='bottom-right') span.caret - + +mixin table-result-body .grid(ui-grid='paragraph.gridOptions' ui-grid-resize-columns ui-grid-exporter) mixin chart-result @@ -166,12 +165,99 @@ mixin chart-result +result-toolbar label.margin-top-dflt Charts do not support #[b Explain] and #[b Scan] query +mixin paragraph-scan + .panel-heading(bs-collapse-toggle) + .row + +paragraph-rename + .panel-collapse(role='tabpanel' bs-collapse-target) + .col-sm-12.sql-controls + .col-sm-3 + +dropdown-required('Cache:', 'paragraph.cacheName', '"cache"', 'true', 'false', 'Choose cache', 'caches') + .col-sm-3 + +text-enabled('Filter:', 'paragraph.filter', '"filter"', true, false, 'Enter filter') + label.btn.btn-default.ignite-form-field__btn(ng-click='paragraph.caseSensitive = !paragraph.caseSensitive') + input(type='checkbox' ng-model='paragraph.caseSensitive') + span(bs-tooltip data-title='Select this checkbox for case sensitive search') Cs + label.tipLabel(bs-tooltip data-placement='bottom' data-title='Max number of rows to show in query result as one page') Page size: + button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.pageSize' bs-select bs-options='item for item in pageSizes') + + .col-sm-12.sql-controls + button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' ng-click='scan(paragraph)') + | Scan + button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' ng-click='scan(paragraph, true)') + | Scan on selected node + + .col-sm-12.sql-result(ng-if='paragraph.queryExecuted()' ng-switch='paragraph.resultType()') + .error(ng-switch-when='error') Error: {{paragraph.errMsg}} + .empty(ng-switch-when='empty') Result set is empty + .table(ng-switch-when='table') + +table-result-heading-scan + +table-result-body + .footer.clearfix() + .pull-left + | Showing results for scan of #[b{{ paragraph.queryArgs.cacheName | defaultName }}] + span(ng-if='paragraph.queryArgs.filter')   with filter: #[b {{ paragraph.queryArgs.filter }}] + span(ng-if='paragraph.queryArgs.localNid')   on node: #[b {{ paragraph.queryArgs.localNid | limitTo:8 }}] + + -var nextVisibleCondition = 'paragraph.resultType() != "error" && paragraph.queryId && paragraph.nonRefresh() && (paragraph.table() || paragraph.chart() && !paragraph.scanExplain())' + + .pull-right(ng-show='#{nextVisibleCondition}' ng-class='{disabled: paragraph.loading}' ng-click='!paragraph.loading && nextPage(paragraph)') + i.fa.fa-chevron-circle-right + a Next + +mixin paragraph-query + .row.panel-heading(bs-collapse-toggle) + +paragraph-rename + .panel-collapse(role='tabpanel' bs-collapse-target) + .col-sm-12 + .col-xs-8.col-sm-9(style='border-right: 1px solid #eee') + .sql-editor(ignite-ace='{onLoad: aceInit(paragraph), theme: "chrome", mode: "sql", require: ["ace/ext/language_tools"],' + + 'advanced: {enableSnippets: false, enableBasicAutocompletion: true, enableLiveAutocompletion: true}}' + ng-model='paragraph.query') + .col-xs-4.col-sm-3 + div(ng-show='caches.length > 0' style='padding: 5px 10px' st-table='displayedCaches' st-safe-src='caches') + lable.labelField.labelFormField Caches: + i.fa.fa-database.tipField(title='Click to show cache types metadata dialog' bs-popover data-template-url='/sql/cache-metadata.html' data-placement='bottom' data-trigger='click' data-container='#{{ paragraph.id }}') + .input-tip + input.form-control(type='text' st-search='label' placeholder='Filter caches...') + table.links + tbody.scrollable-y(style='max-height: 15em; display: block;') + tr(ng-repeat='cache in displayedCaches track by cache.name') + td(style='width: 100%') + input.labelField(id='cache_{{ [paragraph.id, $index].join("_") }}' type='radio' value='{{cache.name}}' ng-model='paragraph.cacheName') + label(for='cache_{{ [paragraph.id, $index].join("_") }} ' ng-bind-html='cache.label') + .empty-caches(ng-show='displayedCaches.length == 0 && caches.length != 0') + label Wrong caches filter + .empty-caches(ng-show='caches.length == 0') + label No caches + .col-sm-12.sql-controls + +query-actions + + .pull-right + +query-settings + .col-sm-12.sql-result(ng-if='paragraph.queryExecuted()' ng-switch='paragraph.resultType()') + .error(ng-switch-when='error') Error: {{paragraph.errMsg}} + .empty(ng-switch-when='empty') Result set is empty + .table(ng-switch-when='table') + +table-result-heading-query + +table-result-body + .chart(ng-switch-when='chart') + +chart-result + .footer.clearfix + a.pull-left(ng-click='showResultQuery(paragraph)') Show query + + -var nextVisibleCondition = 'paragraph.resultType() != "error" && paragraph.queryId && paragraph.nonRefresh() && (paragraph.table() || paragraph.chart() && !paragraph.scanExplain())' + + .pull-right(ng-show='#{nextVisibleCondition}' ng-class='{disabled: paragraph.loading}' ng-click='!paragraph.loading && nextPage(paragraph)') + i.fa.fa-chevron-circle-right + a Next + .row(ng-controller='sqlController') .docs-content .row(ng-if='notebook' bs-affix style='margin-bottom: 20px;') +notebook-rename - ignite-information(data-title='With SQL notebook you can' style='margin-top: 0; margin-bottom: 30px') + ignite-information(data-title='With query notebook you can' style='margin-top: 0; margin-bottom: 30px') ul li Create any number of queries li Execute and explain SQL queries @@ -184,46 +270,9 @@ mixin chart-result div(ng-if='notebook' ignite-loading='sqlLoading' ignite-loading-text='{{ loadingText }}' ignite-loading-position='top') .docs-body.paragraphs .panel-group(bs-collapse ng-model='notebook.expandedParagraphs' data-allow-multiple='true' data-start-collapsed='false') - .panel.panel-default(ng-repeat='paragraph in notebook.paragraphs' id='{{paragraph.id}}') - .panel-heading(bs-collapse-toggle) - .row - +paragraph-rename - .panel-collapse(role='tabpanel' bs-collapse-target) - .col-sm-12 - .col-xs-8.col-sm-9(style='border-right: 1px solid #eee') - .sql-editor(ignite-ace='{onLoad: aceInit(paragraph), theme: "chrome", mode: "sql", require: ["ace/ext/language_tools"],' + - 'advanced: {enableSnippets: false, enableBasicAutocompletion: true, enableLiveAutocompletion: true}}' - ng-model='paragraph.query') - .col-xs-4.col-sm-3 - div(ng-show='caches.length > 0' style='padding: 5px 10px' st-table='displayedCaches' st-safe-src='caches') - lable.labelField.labelFormField Caches: - i.fa.fa-database.tipField(title='Click to show cache types metadata dialog' bs-popover data-template-url='/sql/cache-metadata.html' data-placement='bottom' data-trigger='click' data-container='#{{ paragraph.id }}') - .input-tip - input.form-control(type='text' st-search='label' placeholder='Filter caches...') - table.links - tbody.scrollable-y(style='max-height: 15em; display: block;') - tr(ng-repeat='cache in displayedCaches track by cache.name') - td(style='width: 100%') - input.labelField(id='cache_{{ [paragraph.id, $index].join("_") }}' type='radio' value='{{cache.name}}' ng-model='paragraph.cacheName') - label(for='cache_{{ [paragraph.id, $index].join("_") }} ' ng-bind='cache.label') - .empty-caches(ng-show='displayedCaches.length == 0 && caches.length != 0') - label Wrong caches filter - .empty-caches(ng-show='caches.length == 0') - label No caches - .col-sm-12 - +query-controls - .col-sm-12.sql-result(ng-if='paragraph.queryExecuted()' ng-switch='paragraph.resultType()') - .error(ng-switch-when='error') Error: {{paragraph.errMsg}} - .empty(ng-switch-when='empty') Result set is empty - .table(ng-switch-when='table') - +table-result - .chart(ng-switch-when='chart') - +chart-result - .footer.clearfix - a.pull-left(ng-click='showResultQuery(paragraph)') Show query - - -var nextVisibleCondition = 'paragraph.resultType() != "error" && paragraph.queryId && paragraph.nonRefresh() && (paragraph.table() || paragraph.chart() && !paragraph.scanExplain())' - - .pull-right(ng-show=nextVisibleCondition ng-class='{disabled: paragraph.loading}' ng-click='!paragraph.loading && nextPage(paragraph)') - i.fa.fa-chevron-circle-right - a Next + + .panel-paragraph(ng-repeat='paragraph in notebook.paragraphs' id='{{paragraph.id}}' ng-form='form_{{paragraph.id}}') + .panel.panel-default(ng-if='paragraph.qryType === "scan"') + +paragraph-scan + .panel.panel-default(ng-if='paragraph.qryType === "query"') + +paragraph-query diff --git a/modules/web-console/frontend/views/templates/alert.jade b/modules/web-console/frontend/views/templates/alert.jade index 182ba997d06b3..d30d2fd447e5e 100644 --- a/modules/web-console/frontend/views/templates/alert.jade +++ b/modules/web-console/frontend/views/templates/alert.jade @@ -16,6 +16,6 @@ .alert(ng-show='type' ng-class='[type ? "alert-" + type : null]') button.close(type='button', ng-if='dismissable', ng-click='$hide()') × - i.alert-icon.fa(ng-if='icon' ng-class='[icon]') + i.alert-icon(ng-if='icon' ng-class='[icon]') span.alert-title(ng-bind-html='title') span.alert-content(ng-bind-html='content') diff --git a/modules/web-console/frontend/views/templates/select.jade b/modules/web-console/frontend/views/templates/select.jade index 5b6cc0170f5f0..aa6a2ef9e8a82 100644 --- a/modules/web-console/frontend/views/templates/select.jade +++ b/modules/web-console/frontend/views/templates/select.jade @@ -23,4 +23,4 @@ ul.select.dropdown-menu(tabindex='-1' ng-show='$isVisible()' role='select') hr(ng-if='match.value == undefined' style='margin: 5px 0') a(id='li-dropdown-item-{{$index}}' role='menuitem' tabindex='-1' ng-class='{active: $isActive($index)}' ng-click='$select($index, $event)' bs-tooltip='widthIsSufficient && !widthIsSufficient("li-dropdown-item-{{$index}}", $index, match.label) ? match.label : ""' data-placement='right auto') i(class='{{$iconCheckmark}}' ng-if='$isActive($index)' ng-class='{active: $isActive($index)}') - span(ng-bind='match.label') + span(ng-bind-html='match.label') From b252b441a9ada31c7200b385d75e0b3e7c0362dd Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 23 Dec 2016 18:20:44 +0700 Subject: [PATCH 407/487] Implemented Visor tasks for Services. (cherry picked from commit fdf1f4b) --- .../visor/service/VisorCancelServiceTask.java | 70 ++++++++++ .../visor/service/VisorServiceDescriptor.java | 132 ++++++++++++++++++ .../visor/service/VisorServiceTask.java | 75 ++++++++++ .../internal/visor/util/VisorTaskUtils.java | 15 +- .../resources/META-INF/classnames.properties | 65 +++++++-- 5 files changed, 342 insertions(+), 15 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java new file mode 100644 index 0000000000000..64987e92fc066 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java @@ -0,0 +1,70 @@ +/* + * 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.visor.service; + +import org.apache.ignite.IgniteServices; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorOneNodeTask; + +/** + * Task for cancel services with specified name. + */ +@GridInternal +public class VisorCancelServiceTask extends VisorOneNodeTask { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorCancelServiceJob job(String arg) { + return new VisorCancelServiceJob(arg, debug); + } + + /** + * Job for cancel services with specified name. + */ + private static class VisorCancelServiceJob extends VisorJob { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Create job with specified argument. + * + * @param arg Job argument. + * @param debug Debug flag. + */ + protected VisorCancelServiceJob(String arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected Void run(final String arg) { + IgniteServices services = ignite.services(); + + services.cancel(arg); + + return null; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorCancelServiceJob.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java new file mode 100644 index 0000000000000..83ec77d5c22e0 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java @@ -0,0 +1,132 @@ +/* + * 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.visor.service; + +import java.io.Serializable; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.util.VisorTaskUtils; +import org.apache.ignite.services.ServiceDescriptor; + +/** + * Data transfer object for {@link ServiceDescriptor} object. + */ +public class VisorServiceDescriptor implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** Service name. */ + private String name; + + /** Service class. */ + private String srvcCls; + + /** Maximum allowed total number of deployed services in the grid, {@code 0} for unlimited. */ + private int totalCnt; + + /** Maximum allowed number of deployed services on each node. */ + private int maxPerNodeCnt; + + /** Cache name used for key-to-node affinity calculation. */ + private String cacheName; + + /** ID of grid node that initiated the service deployment. */ + private UUID originNodeId; + + /** + * Service deployment topology snapshot. + * Number of service instances deployed on a node mapped to node ID. + */ + private Map topSnapshot; + + /** + * Default constructor. + */ + public VisorServiceDescriptor() { + // No-op. + } + + /** + * Create task result with given parameters + * + */ + public VisorServiceDescriptor(ServiceDescriptor srvc) { + name = srvc.name(); + srvcCls = VisorTaskUtils.compactClass(srvc.serviceClass()); + totalCnt = srvc.totalCount(); + maxPerNodeCnt = srvc.maxPerNodeCount(); + cacheName = srvc.cacheName(); + originNodeId = srvc.originNodeId(); + topSnapshot = srvc.topologySnapshot(); + } + + /** + * @return Service name. + */ + public String getName() { + return name; + } + + /** + * @return Service class. + */ + public String getServiceClass() { + return srvcCls; + } + + /** + * @return Maximum allowed total number of deployed services in the grid, 0 for unlimited. + */ + public int getTotalCnt() { + return totalCnt; + } + + /** + * @return Maximum allowed number of deployed services on each node. + */ + public int getMaxPerNodeCnt() { + return maxPerNodeCnt; + } + + /** + * @return Cache name used for key-to-node affinity calculation. + */ + public String getCacheName() { + return cacheName; + } + + /** + * @return ID of grid node that initiated the service deployment. + */ + public UUID getOriginNodeId() { + return originNodeId; + } + + /** + * @return Service deployment topology snapshot. Number of service instances deployed on a node mapped to node ID. + */ + public Map getTopologySnapshot() { + return topSnapshot; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorServiceDescriptor.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java new file mode 100644 index 0000000000000..1b3495c5e917d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java @@ -0,0 +1,75 @@ +/* + * 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.visor.service; + +import java.util.ArrayList; +import java.util.Collection; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorOneNodeTask; +import org.apache.ignite.services.ServiceDescriptor; + +/** + * Task for collect topology service configuration. + */ +@GridInternal +public class VisorServiceTask extends VisorOneNodeTask> { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorServiceJob job(Void arg) { + return new VisorServiceJob(arg, debug); + } + + /** + * Job for collect topology service configuration. + */ + private static class VisorServiceJob extends VisorJob> { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Create job with specified argument. + * + * @param arg Job argument. + * @param debug Debug flag. + */ + protected VisorServiceJob(Void arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected Collection run(final Void arg) { + Collection services = ignite.services().serviceDescriptors(); + + Collection res = new ArrayList<>(services.size()); + + for (ServiceDescriptor srvc: services) + res.add(new VisorServiceDescriptor(srvc)); + + return res; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorServiceJob.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java index 1e9346ced5654..3f5003a03d401 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java @@ -267,6 +267,19 @@ public static int[] concat(int[]... arrays) { return U.compact(obj.getClass().getName()); } + /** + * Compact class names. + * + * @param cls Class object for compact. + * @return Compacted string. + */ + @Nullable public static String compactClass(Class cls) { + if (cls == null) + return null; + + return U.compact(cls.getName()); + } + /** * Compact class names. * @@ -277,7 +290,7 @@ public static int[] concat(int[]... arrays) { if (obj == null) return null; - return U.compact(obj.getClass().getName()); + return compactClass(obj.getClass()); } /** diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties index 4c9596c854ea0..4d0b931ffc607 100644 --- a/modules/core/src/main/resources/META-INF/classnames.properties +++ b/modules/core/src/main/resources/META-INF/classnames.properties @@ -294,12 +294,17 @@ org.apache.ignite.internal.jdbc2.JdbcDatabaseMetadata$UpdateMetadataTask org.apache.ignite.internal.jdbc2.JdbcQueryTask org.apache.ignite.internal.jdbc2.JdbcQueryTask$1 org.apache.ignite.internal.jdbc2.JdbcQueryTask$QueryResult +org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2 +org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2$1 +org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2$QueryResult +org.apache.ignite.internal.jdbc2.JdbcSqlFieldsQuery org.apache.ignite.internal.managers.GridManagerAdapter$1$1 org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager$CheckpointSet org.apache.ignite.internal.managers.checkpoint.GridCheckpointRequest org.apache.ignite.internal.managers.communication.GridIoManager$ConcurrentHashMap0 org.apache.ignite.internal.managers.communication.GridIoMessage org.apache.ignite.internal.managers.communication.GridIoUserMessage +org.apache.ignite.internal.managers.communication.IgniteIoTestMessage org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean org.apache.ignite.internal.managers.deployment.GridDeploymentPerVersionStore$2 org.apache.ignite.internal.managers.deployment.GridDeploymentRequest @@ -387,20 +392,20 @@ org.apache.ignite.internal.processors.cache.GridCacheAdapter$3 org.apache.ignite.internal.processors.cache.GridCacheAdapter$30 org.apache.ignite.internal.processors.cache.GridCacheAdapter$32 org.apache.ignite.internal.processors.cache.GridCacheAdapter$4 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$48 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$49 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$50 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$51 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$52 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$53 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$54 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$55 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$57 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$58 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$58$1 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$59 org.apache.ignite.internal.processors.cache.GridCacheAdapter$6 org.apache.ignite.internal.processors.cache.GridCacheAdapter$60 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$61 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$62 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$63 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$64 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$65 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$66 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$67 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$69 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$70 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$70$1 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$71 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$72 org.apache.ignite.internal.processors.cache.GridCacheAdapter$9 org.apache.ignite.internal.processors.cache.GridCacheAdapter$AsyncOp$1 org.apache.ignite.internal.processors.cache.GridCacheAdapter$AsyncOp$1$1 @@ -719,8 +724,11 @@ org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtFor org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$1 +org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$1$1 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$2 +org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$3 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$4$1 +org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$5$1 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$DemandWorker$1 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$DemandWorker$2 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId @@ -1110,6 +1118,12 @@ org.apache.ignite.internal.processors.hadoop.HadoopJobStatus org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo org.apache.ignite.internal.processors.hadoop.HadoopTaskType +org.apache.ignite.internal.processors.hadoop.message.HadoopMessage +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopDirectShuffleMessage +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleAck +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishRequest +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishResponse +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleMessage org.apache.ignite.internal.processors.igfs.IgfsAckMessage org.apache.ignite.internal.processors.igfs.IgfsAttributes org.apache.ignite.internal.processors.igfs.IgfsBlockKey @@ -1207,6 +1221,7 @@ org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessor org.apache.ignite.internal.processors.platform.cache.PlatformCachePartialUpdateException org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinity$1 org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinityFunction +org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpiryPolicyFactory org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryFilter org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryImpl @@ -1243,6 +1258,9 @@ org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$9 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetConfigurationClosure org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetService org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetServiceImpl +org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkCacheExtension$CleanupCompletionListener +org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkCacheExtension$RemoveOldEntriesRunnable +org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkIncreaseVersionProcessor org.apache.ignite.internal.processors.platform.events.PlatformEventFilterListenerImpl org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter org.apache.ignite.internal.processors.platform.messaging.PlatformMessageFilterImpl @@ -1265,6 +1283,7 @@ org.apache.ignite.internal.processors.query.GridQueryProcessor$6 org.apache.ignite.internal.processors.query.GridQueryProcessor$7 org.apache.ignite.internal.processors.query.GridQueryProcessor$8 org.apache.ignite.internal.processors.query.GridQueryProcessor$IndexType +org.apache.ignite.internal.processors.query.IgniteSQLException org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest @@ -1330,6 +1349,9 @@ org.apache.ignite.internal.processors.rest.handlers.datastructures.DataStructure org.apache.ignite.internal.processors.rest.handlers.query.CacheQueryFieldsMetaResult org.apache.ignite.internal.processors.rest.handlers.query.CacheQueryResult org.apache.ignite.internal.processors.rest.handlers.query.QueryCommandHandler$QueryCursorIterator +org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler$1 +org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisGenericException +org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisTypeException org.apache.ignite.internal.processors.rest.handlers.task.GridTaskCommandHandler$2 org.apache.ignite.internal.processors.rest.handlers.task.GridTaskCommandHandler$ExeCallable org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultRequest @@ -1341,6 +1363,9 @@ org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpMemcachedNioList org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpMemcachedNioListener$2 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpRestNioListener$1 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpRestNioListener$1$1 +org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand +org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage +org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisNioListener$1 org.apache.ignite.internal.processors.rest.request.RestQueryRequest$QueryType org.apache.ignite.internal.processors.service.GridServiceAssignments org.apache.ignite.internal.processors.service.GridServiceAssignmentsKey @@ -1585,10 +1610,13 @@ org.apache.ignite.internal.util.lang.IgniteReducer2X org.apache.ignite.internal.util.lang.IgniteReducer3 org.apache.ignite.internal.util.lang.IgniteReducer3X org.apache.ignite.internal.util.lang.IgniteReducerX +org.apache.ignite.internal.util.lang.IgniteSingletonIterator org.apache.ignite.internal.util.nio.GridNioEmbeddedFuture$1 org.apache.ignite.internal.util.nio.GridNioException org.apache.ignite.internal.util.nio.GridNioMessageTracker org.apache.ignite.internal.util.nio.GridNioServer$NioOperation +org.apache.ignite.internal.util.nio.GridNioServer$RandomBalancer +org.apache.ignite.internal.util.nio.GridNioServer$SizeBasedBalancer org.apache.ignite.internal.util.nio.GridNioSessionMetaKey org.apache.ignite.internal.util.nio.ssl.GridNioSslHandler org.apache.ignite.internal.util.offheap.GridOffHeapEvent @@ -1801,6 +1829,11 @@ org.apache.ignite.internal.visor.query.VisorQueryResult org.apache.ignite.internal.visor.query.VisorQueryResultEx org.apache.ignite.internal.visor.query.VisorQueryScanSubstringFilter org.apache.ignite.internal.visor.query.VisorQueryTask +org.apache.ignite.internal.visor.service.VisorCancelServiceTask +org.apache.ignite.internal.visor.service.VisorCancelServiceTask$VisorCancelServiceJob +org.apache.ignite.internal.visor.service.VisorServiceDescriptor +org.apache.ignite.internal.visor.service.VisorServiceTask +org.apache.ignite.internal.visor.service.VisorServiceTask$VisorServiceJob org.apache.ignite.internal.visor.util.VisorClusterGroupEmptyException org.apache.ignite.internal.visor.util.VisorEventMapper org.apache.ignite.internal.visor.util.VisorExceptionWrapper @@ -1858,12 +1891,15 @@ org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointData org.apache.ignite.spi.collision.jobstealing.JobStealingRequest org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi$PriorityGridCollisionJobContextComparator org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$1 +org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$10 +org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$11 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosure org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosure$1 -org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$8 -org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$9 +org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosureNew +org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosureNew$1 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeClosure org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeMessage +org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeMessage2 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeTimeoutException org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$NodeIdMessage org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$RecoveryLastReceivedMessage @@ -1923,3 +1959,4 @@ org.apache.ignite.transactions.TransactionOptimisticException org.apache.ignite.transactions.TransactionRollbackException org.apache.ignite.transactions.TransactionState org.apache.ignite.transactions.TransactionTimeoutException +org.apache.ignite.util.AttributeNodeFilter From fb8191028eb19b0683b8239b7024f7fa6ccabd4e Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 23 Dec 2016 18:40:51 +0700 Subject: [PATCH 408/487] Updated classnames for ignite-1.7.5. --- .../resources/META-INF/classnames.properties | 60 +++++-------------- 1 file changed, 15 insertions(+), 45 deletions(-) diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties index 4d0b931ffc607..8a6dc6603c4f1 100644 --- a/modules/core/src/main/resources/META-INF/classnames.properties +++ b/modules/core/src/main/resources/META-INF/classnames.properties @@ -294,17 +294,12 @@ org.apache.ignite.internal.jdbc2.JdbcDatabaseMetadata$UpdateMetadataTask org.apache.ignite.internal.jdbc2.JdbcQueryTask org.apache.ignite.internal.jdbc2.JdbcQueryTask$1 org.apache.ignite.internal.jdbc2.JdbcQueryTask$QueryResult -org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2 -org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2$1 -org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2$QueryResult -org.apache.ignite.internal.jdbc2.JdbcSqlFieldsQuery org.apache.ignite.internal.managers.GridManagerAdapter$1$1 org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager$CheckpointSet org.apache.ignite.internal.managers.checkpoint.GridCheckpointRequest org.apache.ignite.internal.managers.communication.GridIoManager$ConcurrentHashMap0 org.apache.ignite.internal.managers.communication.GridIoMessage org.apache.ignite.internal.managers.communication.GridIoUserMessage -org.apache.ignite.internal.managers.communication.IgniteIoTestMessage org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean org.apache.ignite.internal.managers.deployment.GridDeploymentPerVersionStore$2 org.apache.ignite.internal.managers.deployment.GridDeploymentRequest @@ -392,20 +387,20 @@ org.apache.ignite.internal.processors.cache.GridCacheAdapter$3 org.apache.ignite.internal.processors.cache.GridCacheAdapter$30 org.apache.ignite.internal.processors.cache.GridCacheAdapter$32 org.apache.ignite.internal.processors.cache.GridCacheAdapter$4 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$48 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$49 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$50 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$51 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$52 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$53 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$54 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$55 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$57 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$58 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$58$1 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$59 org.apache.ignite.internal.processors.cache.GridCacheAdapter$6 org.apache.ignite.internal.processors.cache.GridCacheAdapter$60 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$61 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$62 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$63 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$64 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$65 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$66 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$67 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$69 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$70 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$70$1 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$71 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$72 org.apache.ignite.internal.processors.cache.GridCacheAdapter$9 org.apache.ignite.internal.processors.cache.GridCacheAdapter$AsyncOp$1 org.apache.ignite.internal.processors.cache.GridCacheAdapter$AsyncOp$1$1 @@ -724,11 +719,9 @@ org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtFor org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$1 -org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$1$1 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$2 -org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$3 +org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$3$1 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$4$1 -org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$5$1 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$DemandWorker$1 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$DemandWorker$2 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId @@ -1118,12 +1111,6 @@ org.apache.ignite.internal.processors.hadoop.HadoopJobStatus org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo org.apache.ignite.internal.processors.hadoop.HadoopTaskType -org.apache.ignite.internal.processors.hadoop.message.HadoopMessage -org.apache.ignite.internal.processors.hadoop.shuffle.HadoopDirectShuffleMessage -org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleAck -org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishRequest -org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishResponse -org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleMessage org.apache.ignite.internal.processors.igfs.IgfsAckMessage org.apache.ignite.internal.processors.igfs.IgfsAttributes org.apache.ignite.internal.processors.igfs.IgfsBlockKey @@ -1221,7 +1208,6 @@ org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessor org.apache.ignite.internal.processors.platform.cache.PlatformCachePartialUpdateException org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinity$1 org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinityFunction -org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpiryPolicyFactory org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryFilter org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryImpl @@ -1258,9 +1244,6 @@ org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$9 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetConfigurationClosure org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetService org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetServiceImpl -org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkCacheExtension$CleanupCompletionListener -org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkCacheExtension$RemoveOldEntriesRunnable -org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkIncreaseVersionProcessor org.apache.ignite.internal.processors.platform.events.PlatformEventFilterListenerImpl org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter org.apache.ignite.internal.processors.platform.messaging.PlatformMessageFilterImpl @@ -1283,7 +1266,6 @@ org.apache.ignite.internal.processors.query.GridQueryProcessor$6 org.apache.ignite.internal.processors.query.GridQueryProcessor$7 org.apache.ignite.internal.processors.query.GridQueryProcessor$8 org.apache.ignite.internal.processors.query.GridQueryProcessor$IndexType -org.apache.ignite.internal.processors.query.IgniteSQLException org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest @@ -1349,9 +1331,6 @@ org.apache.ignite.internal.processors.rest.handlers.datastructures.DataStructure org.apache.ignite.internal.processors.rest.handlers.query.CacheQueryFieldsMetaResult org.apache.ignite.internal.processors.rest.handlers.query.CacheQueryResult org.apache.ignite.internal.processors.rest.handlers.query.QueryCommandHandler$QueryCursorIterator -org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler$1 -org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisGenericException -org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisTypeException org.apache.ignite.internal.processors.rest.handlers.task.GridTaskCommandHandler$2 org.apache.ignite.internal.processors.rest.handlers.task.GridTaskCommandHandler$ExeCallable org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultRequest @@ -1363,9 +1342,6 @@ org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpMemcachedNioList org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpMemcachedNioListener$2 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpRestNioListener$1 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpRestNioListener$1$1 -org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand -org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage -org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisNioListener$1 org.apache.ignite.internal.processors.rest.request.RestQueryRequest$QueryType org.apache.ignite.internal.processors.service.GridServiceAssignments org.apache.ignite.internal.processors.service.GridServiceAssignmentsKey @@ -1610,13 +1586,10 @@ org.apache.ignite.internal.util.lang.IgniteReducer2X org.apache.ignite.internal.util.lang.IgniteReducer3 org.apache.ignite.internal.util.lang.IgniteReducer3X org.apache.ignite.internal.util.lang.IgniteReducerX -org.apache.ignite.internal.util.lang.IgniteSingletonIterator org.apache.ignite.internal.util.nio.GridNioEmbeddedFuture$1 org.apache.ignite.internal.util.nio.GridNioException org.apache.ignite.internal.util.nio.GridNioMessageTracker org.apache.ignite.internal.util.nio.GridNioServer$NioOperation -org.apache.ignite.internal.util.nio.GridNioServer$RandomBalancer -org.apache.ignite.internal.util.nio.GridNioServer$SizeBasedBalancer org.apache.ignite.internal.util.nio.GridNioSessionMetaKey org.apache.ignite.internal.util.nio.ssl.GridNioSslHandler org.apache.ignite.internal.util.offheap.GridOffHeapEvent @@ -1891,15 +1864,12 @@ org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointData org.apache.ignite.spi.collision.jobstealing.JobStealingRequest org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi$PriorityGridCollisionJobContextComparator org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$1 -org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$10 -org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$11 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosure org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosure$1 -org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosureNew -org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosureNew$1 +org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$8 +org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$9 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeClosure org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeMessage -org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeMessage2 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeTimeoutException org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$NodeIdMessage org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$RecoveryLastReceivedMessage From 2da2816fd74b17590f45781268337da5205c44fa Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Fri, 23 Dec 2016 18:58:47 +0700 Subject: [PATCH 409/487] Fixed broken links. (cherry picked from commit 6ca8670) --- .../core/src/main/java/org/apache/ignite/IgniteLogger.java | 6 +++--- .../main/java/org/apache/ignite/logger/java/JavaLogger.java | 4 ++-- .../testframework/junits/logger/GridTestLog4jLogger.java | 4 ++-- .../java/org/apache/ignite/logger/log4j/Log4JLogger.java | 4 ++-- .../apache/ignite/spi/deployment/uri/UriDeploymentSpi.java | 2 +- 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java b/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java index 8d814fd78b39e..b1433a8ac230a 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java @@ -23,8 +23,8 @@ /** * This interface defines basic logging functionality used throughout the system. We had to * abstract it out so that we can use whatever logging is used by the hosting environment. - * Currently,
          log4j, - * JBoss, + * Currently, log4j, + * JBoss, * JCL and * console logging are provided as supported implementations. *

          @@ -158,4 +158,4 @@ public interface IgniteLogger { * @return Name of the file being logged to if one is configured or {@code null} otherwise. */ public String fileName(); -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java index d5ff5e388d130..6aa7d3879698a 100644 --- a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java +++ b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java @@ -86,7 +86,7 @@ * ... * cfg.setGridLogger(log); * - * Please take a look at Logger javadoc + * Please take a look at Logger javadoc * for additional information. *

          * It's recommended to use Ignite logger injection instead of using/instantiating @@ -406,4 +406,4 @@ private static T findHandler(Logger log, Class cls) { return null; } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java index 74f5160bb045e..6a46c7d7cc7da 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java @@ -50,7 +50,7 @@ /** * Log4j-based implementation for logging. This logger should be used - * by loaders that have prefer log4j-based logging. + * by loaders that have prefer log4j-based logging. *

          * Here is a typical example of configuring log4j logger in Ignite configuration file: *

          @@ -521,4 +521,4 @@ public static Collection logFiles() {
               @Override public String toString() {
                   return S.toString(GridTestLog4jLogger.class, this);
               }
          -}
          \ No newline at end of file
          +}
          diff --git a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java
          index eaae2d49f5eea..d5b0f0227bd85 100644
          --- a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java
          +++ b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java
          @@ -50,7 +50,7 @@
           
           /**
            * Log4j-based implementation for logging. This logger should be used
          - * by loaders that have prefer log4j-based logging.
          + * by loaders that have prefer log4j-based logging.
            * 

          * Here is a typical example of configuring log4j logger in Ignite configuration file: *

          @@ -532,4 +532,4 @@ public static Collection logFiles() {
                       }
                   }
               }
          -}
          \ No newline at end of file
          +}
          diff --git a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
          index 22d42db4ac04f..30940e4640956 100644
          --- a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
          +++ b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
          @@ -111,7 +111,7 @@
            * {@code META-INF/} entry may contain {@code ignite.xml} file which is a
            * task descriptor file. The purpose of task descriptor XML file is to specify
            * all tasks to be deployed. This file is a regular
          - * Spring XML
          + * Spring XML
            * definition file.  {@code META-INF/} entry may also contain any other file
            * specified by JAR format.
            * 
          
          From 2ccae40e2a21398d15c3762b72575216c56a7fb0 Mon Sep 17 00:00:00 2001
          From: dkarachentsev 
          Date: Fri, 23 Dec 2016 17:51:49 +0300
          Subject: [PATCH 410/487] IGNITE-4109 - BinaryType.isEnum() throws an exception
           if typeId==0
          
          ---
           .../ignite/internal/binary/BinaryContext.java  |  4 ++--
           .../internal/binary/BinaryTypeProxy.java       | 15 ++++++++++++---
           .../ignite/internal/binary/BinaryUtils.java    |  4 +++-
           .../internal/binary/BinaryEnumsSelfTest.java   | 18 ++++++++++++++++++
           4 files changed, 35 insertions(+), 6 deletions(-)
          
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
          index cc18318b04e61..4030ef02c799c 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
          @@ -928,7 +928,7 @@ public int fieldId(int typeId, String fieldName) {
                * @param typeId Type ID.
                * @return Instance of ID mapper.
                */
          -    public BinaryInternalMapper userTypeMapper(int typeId) {
          +    BinaryInternalMapper userTypeMapper(int typeId) {
                   BinaryInternalMapper mapper = typeId2Mapper.get(typeId);
           
                   return mapper != null ? mapper : SIMPLE_NAME_LOWER_CASE_MAPPER;
          @@ -938,7 +938,7 @@ public BinaryInternalMapper userTypeMapper(int typeId) {
                * @param clsName Type name.
                * @return Instance of ID mapper.
                */
          -    private BinaryInternalMapper userTypeMapper(String clsName) {
          +    BinaryInternalMapper userTypeMapper(String clsName) {
                   BinaryInternalMapper mapper = cls2Mappers.get(clsName);
           
                   if (mapper != null)
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTypeProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTypeProxy.java
          index 17b0bc6e50628..df9901e88d351 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTypeProxy.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTypeProxy.java
          @@ -24,6 +24,7 @@
           import org.apache.ignite.internal.util.typedef.internal.S;
           
           import java.util.Collection;
          +import org.jetbrains.annotations.Nullable;
           
           /**
            * Binary type proxy. Is used to delay or completely avoid metadata lookup.
          @@ -34,21 +35,26 @@ public class BinaryTypeProxy implements BinaryType {
               private final BinaryContext ctx;
           
               /** Type ID. */
          -    private final int typeId;
          +    private int typeId;
          +
          +    /** Raw data. */
          +    private final String clsName;
           
               /** Target type. */
               @GridToStringExclude
               private volatile BinaryType target;
           
               /**
          -     * Constrcutor.
          +     * Constructor.
                *
                * @param ctx Context.
                * @param typeId Type ID.
          +     * @param clsName Class name.
                */
          -    public BinaryTypeProxy(BinaryContext ctx, int typeId) {
          +    public BinaryTypeProxy(BinaryContext ctx, int typeId, @Nullable String clsName) {
                   this.ctx = ctx;
                   this.typeId = typeId;
          +        this.clsName = clsName;
               }
           
               /** {@inheritDoc} */
          @@ -93,6 +99,9 @@ private BinaryType target() {
                   if (target == null) {
                       synchronized (this) {
                           if (target == null) {
          +                    if (typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID && clsName != null)
          +                        typeId = ctx.typeId(clsName);
          +
                               target = ctx.metadata(typeId);
           
                               if (target == null)
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
          index bbf50213fdd6d..fdc54c7871711 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
          @@ -2228,7 +2228,9 @@ public static BinaryType typeProxy(BinaryContext ctx, BinaryObjectEx obj) {
                   if (ctx == null)
                       throw new BinaryObjectException("BinaryContext is not set for the object.");
           
          -        return new BinaryTypeProxy(ctx, obj.typeId());
          +        String clsName = obj instanceof BinaryEnumObjectImpl ? ((BinaryEnumObjectImpl)obj).className() : null;
          +
          +        return new BinaryTypeProxy(ctx, obj.typeId(), clsName);
               }
           
               /**
          diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryEnumsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryEnumsSelfTest.java
          index fb7e618899afb..91add0de06d39 100644
          --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryEnumsSelfTest.java
          +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryEnumsSelfTest.java
          @@ -28,6 +28,7 @@
           import org.apache.ignite.configuration.BinaryConfiguration;
           import org.apache.ignite.configuration.CacheConfiguration;
           import org.apache.ignite.configuration.IgniteConfiguration;
          +import org.apache.ignite.internal.IgniteEx;
           import org.apache.ignite.internal.IgniteKernal;
           import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
           import org.apache.ignite.marshaller.Marshaller;
          @@ -388,6 +389,23 @@ public void checkSimpleBuilderArray(boolean registered) throws Exception {
                   validateSimpleArray(registered);
               }
           
          +    /**
          +     * Check ability to resolve typeId from class name.
          +     *
          +     * @throws Exception If failed.
          +     */
          +    public void testZeroTypeId() throws Exception {
          +        startUp(true);
          +
          +        final BinaryContext ctx =
          +            ((CacheObjectBinaryProcessorImpl)((IgniteEx)node1).context().cacheObjects()).binaryContext();
          +
          +        final BinaryObject enumObj =
          +            new BinaryEnumObjectImpl(ctx, 0, EnumType.class.getName(), EnumType.ONE.ordinal());
          +
          +        assert enumObj.type().isEnum();
          +    }
          +
               /**
                * Validate simple array.
                *
          
          From 9273e51cd039049a4aae73f9dcafc02915bc6153 Mon Sep 17 00:00:00 2001
          From: Alexandr Kuramshin 
          Date: Mon, 26 Dec 2016 13:23:28 +0300
          Subject: [PATCH 411/487] ignite-4167 Do not log cache key/values
          
          ---
           .../apache/ignite/IgniteSystemProperties.java |   5 +
           .../ignite/cache/affinity/AffinityKey.java    |   4 +-
           .../org/apache/ignite/events/CacheEvent.java  |   6 +-
           .../ignite/events/CacheQueryReadEvent.java    |   8 +-
           .../internal/binary/BinaryEnumObjectImpl.java |  10 +-
           .../internal/binary/BinaryMetadata.java       |   5 +-
           .../internal/binary/BinaryObjectExImpl.java   |   8 +-
           .../cache/CacheInvokeDirectResult.java        |   2 +-
           .../processors/cache/CacheInvokeResult.java   |   2 +-
           .../processors/cache/CacheLazyEntry.java      |   4 +-
           .../processors/cache/CacheObjectAdapter.java  |   7 +-
           .../processors/cache/GridCacheAdapter.java    |   5 +-
           .../cache/GridCacheMvccCandidate.java         |   9 +-
           .../processors/cache/GridCacheReturn.java     |   2 +-
           .../distributed/dht/GridDhtCacheAdapter.java  |   2 +-
           .../distributed/near/GridNearLockFuture.java  |   2 +-
           .../cache/query/GridCacheQueryAdapter.java    |   2 +
           .../cache/query/GridCacheQueryManager.java    |  13 +-
           .../cache/query/GridCacheQueryRequest.java    |   2 +
           .../cache/query/GridCacheSqlQuery.java        |   4 +-
           .../continuous/CacheContinuousQueryEvent.java |   8 +-
           .../CacheContinuousQueryManager.java          |   4 +-
           .../store/GridCacheStoreManagerAdapter.java   |  30 +-
           .../store/GridCacheWriteBehindStore.java      |   2 +-
           .../transactions/IgniteTxLocalAdapter.java    |  11 +-
           .../GridCacheVersionConflictContext.java      |   2 +-
           .../closure/GridClosureProcessor.java         |   4 +-
           .../continuous/GridContinuousMessage.java     |   2 +-
           .../datastructures/CollocatedSetItemKey.java  |   2 +-
           .../GridCacheAtomicLongValue.java             |   2 +
           .../GridCacheAtomicSequenceImpl.java          |   2 +
           .../GridCacheAtomicSequenceValue.java         |   2 +
           .../GridCacheCountDownLatchValue.java         |   3 +
           .../datastructures/GridCacheSetItemKey.java   |   2 +-
           .../processors/job/GridJobWorker.java         |   7 +-
           .../odbc/OdbcQueryExecuteRequest.java         |   6 +-
           .../platform/PlatformNativeException.java     |   3 +-
           .../processors/rest/GridRestResponse.java     |   2 +-
           .../util/future/GridFutureAdapter.java        |   2 +-
           .../util/lang/GridMetadataAwareAdapter.java   |   2 +-
           .../util/tostring/GridToStringBuilder.java    | 642 ++++++++++++++++--
           .../util/tostring/GridToStringInclude.java    |  12 +-
           .../tostring/GridToStringThreadLocal.java     |  12 +-
           ...ridCacheBinaryObjectsAbstractSelfTest.java |   7 +-
           .../tostring/GridToStringBuilderSelfTest.java |  33 +-
           45 files changed, 776 insertions(+), 130 deletions(-)
          
          diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
          index fe78d889b178f..0da0f4990912a 100644
          --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
          +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
          @@ -132,6 +132,11 @@ public final class IgniteSystemProperties {
                */
               public static final String IGNITE_QUIET = "IGNITE_QUIET";
           
          +    /**
          +     * Setting to {@code true} enables writing sensitive information in {@code toString()} output.
          +     */
          +    public static final String IGNITE_TO_STRING_INCLUDE_SENSITIVE = "IGNITE_TO_STRING_INCLUDE_SENSITIVE";
          +
               /**
                * If this property is set to {@code true} (default) and Ignite is launched
                * in verbose mode (see {@link #IGNITE_QUIET}) and no console appenders can be found
          diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKey.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKey.java
          index c745ed8e5450a..4215b0517f3a2 100644
          --- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKey.java
          +++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKey.java
          @@ -60,12 +60,12 @@ public class AffinityKey implements Externalizable {
               private static final long serialVersionUID = 0L;
           
               /** Key. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private K key;
           
               /** Affinity key. */
               @AffinityKeyMapped
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private Object affKey;
           
               /**
          diff --git a/modules/core/src/main/java/org/apache/ignite/events/CacheEvent.java b/modules/core/src/main/java/org/apache/ignite/events/CacheEvent.java
          index 29aeb3dc683ad..30f4b37b5a704 100644
          --- a/modules/core/src/main/java/org/apache/ignite/events/CacheEvent.java
          +++ b/modules/core/src/main/java/org/apache/ignite/events/CacheEvent.java
          @@ -90,7 +90,7 @@ public class CacheEvent extends EventAdapter {
               private int part;
           
               /** Cache entry. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private Object key;
           
               /** Event ID. */
          @@ -102,11 +102,11 @@ public class CacheEvent extends EventAdapter {
               private final Object lockId;
           
               /** New value. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private final Object newVal;
           
               /** Old value. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private final Object oldVal;
           
               /**
          diff --git a/modules/core/src/main/java/org/apache/ignite/events/CacheQueryReadEvent.java b/modules/core/src/main/java/org/apache/ignite/events/CacheQueryReadEvent.java
          index 40c5dae3d63d4..f63ed0c29ae42 100644
          --- a/modules/core/src/main/java/org/apache/ignite/events/CacheQueryReadEvent.java
          +++ b/modules/core/src/main/java/org/apache/ignite/events/CacheQueryReadEvent.java
          @@ -96,19 +96,19 @@ public class CacheQueryReadEvent extends EventAdapter {
               private final String taskName;
           
               /** Key. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private final K key;
           
               /** Value. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private final V val;
           
               /** Old value. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private final V oldVal;
           
               /** Result row. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private final Object row;
           
               /**
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
          index 69de3f2cdc8f1..9fd6bc18e8e22 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
          @@ -31,6 +31,7 @@
           import org.apache.ignite.internal.processors.cache.CacheObject;
           import org.apache.ignite.internal.processors.cache.CacheObjectContext;
           import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
          +import org.apache.ignite.internal.util.typedef.internal.S;
           import org.apache.ignite.internal.util.typedef.internal.SB;
           import org.apache.ignite.internal.util.typedef.internal.U;
           import org.apache.ignite.plugin.extensions.communication.MessageReader;
          @@ -195,6 +196,9 @@ public BinaryEnumObjectImpl(BinaryContext ctx, byte[] arr) {
           
               /** {@inheritDoc} */
               @Override public String toString() {
          +        if (!S.INCLUDE_SENSITIVE)
          +            return ord >= 0 ? "BinaryEnum" : "null";
          +
                   // 1. Try deserializing the object.
                   try {
                       Object val = deserialize();
          @@ -216,12 +220,12 @@ public BinaryEnumObjectImpl(BinaryContext ctx, byte[] arr) {
                   }
           
                   if (type != null)
          -            return type.typeName() + "[ordinal=" + ord  + ']';
          +            return S.toString(type.typeName(), "ordinal", ord, true);
                   else {
                       if (typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID)
          -                return "BinaryEnum[clsName=" + clsName + ", ordinal=" + ord + ']';
          +                return S.toString("BinaryEnum", "clsName", clsName, true, "ordinal", ord, true);
                       else
          -                return "BinaryEnum[typeId=" + typeId + ", ordinal=" + ord + ']';
          +                return S.toString("BinaryEnum", "typeId", typeId, true, "ordinal", ord, true);
                   }
               }
           
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java
          index 0911d468dff89..ec92b081904c3 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java
          @@ -42,16 +42,19 @@ public class BinaryMetadata implements Externalizable {
               private static final long serialVersionUID = 0L;
           
               /** Type ID. */
          +    @GridToStringInclude(sensitive = true)
               private int typeId;
           
               /** Type name. */
          +    @GridToStringInclude(sensitive = true)
               private String typeName;
           
               /** Recorded object fields. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private Map fields;
           
               /** Affinity key field name. */
          +    @GridToStringInclude(sensitive = true)
               private String affKeyFieldName;
           
               /** Schemas associated with type. */
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
          index e15e7704eff50..5f1e3e9c894bc 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
          @@ -29,6 +29,7 @@
           import org.apache.ignite.binary.BinaryType;
           import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl;
           import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
          +import org.apache.ignite.internal.util.typedef.internal.S;
           import org.apache.ignite.internal.util.typedef.internal.SB;
           import org.apache.ignite.lang.IgniteUuid;
           import org.jetbrains.annotations.Nullable;
          @@ -220,8 +221,11 @@ private String toString(BinaryReaderHandles ctx, IdentityHashMap implements EntryProcessorResult, Externaliz
               private static final long serialVersionUID = 0L;
           
               /** */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private T res;
           
               /** */
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
          index 02cccc742cc99..be6019efc14c3 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
          @@ -36,11 +36,11 @@ public class CacheLazyEntry extends CacheInterceptorEntry {
               protected CacheObject valObj;
           
               /** Key. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               protected K key;
           
               /** Value. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               protected V val;
           
               /** Keep binary flag. */
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
          index 70f5ea625b6cf..09a55242d442e 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
          @@ -24,6 +24,7 @@
           import java.nio.ByteBuffer;
           import org.apache.ignite.internal.GridDirectTransient;
           import org.apache.ignite.internal.util.tostring.GridToStringInclude;
          +import org.apache.ignite.internal.util.typedef.internal.S;
           import org.apache.ignite.internal.util.typedef.internal.U;
           import org.apache.ignite.plugin.extensions.communication.MessageReader;
           import org.apache.ignite.plugin.extensions.communication.MessageWriter;
          @@ -36,7 +37,7 @@ public abstract class CacheObjectAdapter implements CacheObject, Externalizable
               private static final long serialVersionUID = 2006765505127197251L;
           
               /** */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               @GridDirectTransient
               protected Object val;
           
          @@ -119,6 +120,8 @@ protected boolean needCopy(CacheObjectContext ctx) {
           
               /** {@inheritDoc} */
               public String toString() {
          -        return getClass().getSimpleName() + " [val=" + val + ", hasValBytes=" + (valBytes != null) + ']';
          +        return S.toString(S.INCLUDE_SENSITIVE ? getClass().getSimpleName() : "CacheObject",
          +            "val", val, true,
          +            "hasValBytes", valBytes != null, false);
               }
           }
          \ No newline at end of file
          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 4d59d50ace4d3..965c6d12916aa 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
          @@ -2571,7 +2571,10 @@ public IgniteInternalFuture putAsync0(final K key, final V val,
                       }
           
                       @Override public String toString() {
          -                return "putxAsync [key=" + key + ", val=" + val + ", filter=" + filter + ']';
          +                return S.toString("putxAsync",
          +                    "key", key, true,
          +                    "val", val, true,
          +                    "filter", filter, false);
                       }
                   });
               }
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCandidate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCandidate.java
          index e9dd455901e44..c60d6c8bd70b8 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCandidate.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCandidate.java
          @@ -36,7 +36,6 @@
           import org.apache.ignite.internal.util.typedef.F;
           import org.apache.ignite.internal.util.typedef.internal.S;
           import org.apache.ignite.internal.util.typedef.internal.SB;
          -import org.apache.ignite.internal.util.typedef.internal.U;
           import org.jetbrains.annotations.Nullable;
           
           import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.DHT_LOCAL;
          @@ -665,10 +664,10 @@ public IgniteTxKey key() {
                   GridCacheMvccCandidate next = next();
           
                   return S.toString(GridCacheMvccCandidate.class, this,
          -            "key", parent == null ? null : parent.key(),
          -            "masks", Mask.toString(flags()),
          -            "prevVer", (prev == null ? null : prev.version()),
          -            "nextVer", (next == null ? null : next.version()));
          +            "key", parent == null ? null : parent.key(), true,
          +            "masks", Mask.toString(flags()), false,
          +            "prevVer", prev == null ? null : prev.version(), false,
          +            "nextVer", next == null ? null : next.version(), false);
               }
           
               /**
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java
          index 29e74db5ad31d..02c882c30aa80 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheReturn.java
          @@ -49,7 +49,7 @@ public class GridCacheReturn implements Externalizable, Message {
               private static final long serialVersionUID = 0L;
           
               /** Value. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               @GridDirectTransient
               private volatile Object v;
           
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
          index 35e62670ea2a9..b2fb7b4b392a4 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
          @@ -1166,7 +1166,7 @@ private PartitionEntrySet(int partId) {
           
                   /** {@inheritDoc} */
                   @Override public String toString() {
          -            return S.toString(PartitionEntrySet.class, this, "super", super.toString());
          +            return S.toString(PartitionEntrySet.class, this, "super", super.toString(), true);
                   }
               }
           
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
          index 24313791da10b..7c9860290eea2 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
          @@ -1487,7 +1487,7 @@ private class MiniFuture extends GridFutureAdapter {
                   private ClusterNode node;
           
                   /** Keys. */
          -        @GridToStringInclude
          +        @GridToStringInclude(sensitive = true)
                   private Collection keys;
           
                   /** */
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
          index b29e5e7a62ebd..1fe263d607bbd 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
          @@ -45,6 +45,7 @@
           import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
           import org.apache.ignite.internal.util.GridEmptyCloseableIterator;
           import org.apache.ignite.internal.util.lang.GridCloseableIterator;
          +import org.apache.ignite.internal.util.tostring.GridToStringInclude;
           import org.apache.ignite.internal.util.typedef.F;
           import org.apache.ignite.internal.util.typedef.P1;
           import org.apache.ignite.internal.util.typedef.T2;
          @@ -82,6 +83,7 @@ public class GridCacheQueryAdapter implements CacheQuery {
               private final String clsName;
           
               /** */
          +    @GridToStringInclude(sensitive = true)
               private final String clause;
           
               /** */
          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 1165157784b5e..85c01d9937151 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
          @@ -1539,16 +1539,17 @@ protected void runQuery(GridCacheQueryInfo qryInfo) {
                               if (log.isDebugEnabled()) {
                                   ClusterNode primaryNode = CU.primaryNode(cctx, key);
           
          -                        log.debug("Record [key=" + key +
          -                            ", val=" + val +
          -                            ", incBackups=" + incBackups +
          -                            ", priNode=" + (primaryNode != null ? U.id8(primaryNode.id()) : null) +
          -                            ", node=" + U.id8(cctx.localNode().id()) + ']');
          +                        log.debug(S.toString("Record",
          +                            "key", key, true,
          +                            "val", val, true,
          +                            "incBackups", incBackups, false,
          +                            "priNode", primaryNode != null ? U.id8(primaryNode.id()) : null, false,
          +                            "node", U.id8(cctx.localNode().id()), false));
                               }
           
                               if (val == null) {
                                   if (log.isDebugEnabled())
          -                            log.debug("Unsuitable record value: " + val);
          +                            log.debug(S.toString("Unsuitable record value", "val", val, true));
           
                                   continue;
                               }
          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 60c466226fcf7..ed876a27ff9dd 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
          @@ -27,6 +27,7 @@
           import org.apache.ignite.internal.processors.cache.GridCacheDeployable;
           import org.apache.ignite.internal.processors.cache.GridCacheMessage;
           import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
          +import org.apache.ignite.internal.util.tostring.GridToStringInclude;
           import org.apache.ignite.internal.util.typedef.F;
           import org.apache.ignite.internal.util.typedef.internal.CU;
           import org.apache.ignite.internal.util.typedef.internal.S;
          @@ -63,6 +64,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
               private boolean fields;
           
               /** */
          +    @GridToStringInclude(sensitive = true)
               private String clause;
           
               /** */
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
          index bb769c962a2e4..82562702d62d8 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
          @@ -47,11 +47,11 @@ public class GridCacheSqlQuery implements Message, GridCacheQueryMarshallable {
               public static final Object[] EMPTY_PARAMS = {};
           
               /** */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private String qry;
           
               /** */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               @GridDirectTransient
               private Object[] params;
           
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
          index db70e2ea9d42b..eddf30211ffe0 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
          @@ -99,9 +99,9 @@ public int partitionId() {
               /** {@inheritDoc} */
               @Override public String toString() {
                   return S.toString(CacheContinuousQueryEvent.class, this,
          -            "evtType", getEventType(),
          -            "key", getKey(),
          -            "newVal", getValue(),
          -            "oldVal", getOldValue());
          +            "evtType", getEventType(), false,
          +            "key", getKey(), true,
          +            "newVal", getValue(), true,
          +            "oldVal", getOldValue(), true);
               }
           }
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
          index e2fbf52814161..91c199184a2de 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
          @@ -1209,11 +1209,11 @@ public static class CacheEntryEventImpl extends CacheQueryEntryEvent {
                   private static final long serialVersionUID = 0L;
           
                   /** */
          -        @GridToStringInclude
          +        @GridToStringInclude(sensitive = true)
                   private Object key;
           
                   /** */
          -        @GridToStringInclude
          +        @GridToStringInclude(sensitive = true)
                   private Object val;
           
                   /**
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
          index 024375e75c477..11d9816ff1e6d 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
          @@ -314,7 +314,8 @@ private CacheStore cacheStoreWrapper(GridKernalContext ctx,
                       Object storeKey = cctx.unwrapBinaryIfNeeded(key, !convertBinary());
           
                       if (log.isDebugEnabled())
          -                log.debug("Loading value from store for key: " + storeKey);
          +                log.debug(S.toString("Loading value from store for key",
          +                    "key", storeKey, true));
           
                       sessionInit0(tx);
           
          @@ -564,8 +565,11 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx,
                       key = cctx.unwrapBinaryIfNeeded(key, !convertBinary());
                       val = cctx.unwrapBinaryIfNeeded(val, !convertBinary());
           
          -            if (log.isDebugEnabled())
          -                log.debug("Storing value in cache store [key=" + key + ", val=" + val + ']');
          +            if (log.isDebugEnabled()) {
          +                log.debug(S.toString("Storing value in cache store",
          +                    "key", key, true,
          +                    "val", val, true));
          +            }
           
                       sessionInit0(tx);
           
          @@ -589,8 +593,11 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx,
                           sessionEnd0(tx, threwEx);
                       }
           
          -            if (log.isDebugEnabled())
          -                log.debug("Stored value in cache store [key=" + key + ", val=" + val + ']');
          +            if (log.isDebugEnabled()) {
          +                log.debug(S.toString("Stored value in cache store",
          +                    "key", key, true,
          +                    "val", val, true));
          +            }
           
                       return true;
                   }
          @@ -667,7 +674,7 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx,
                       key = cctx.unwrapBinaryIfNeeded(key, !convertBinary());
           
                       if (log.isDebugEnabled())
          -                log.debug("Removing value from cache store [key=" + key + ']');
          +                log.debug(S.toString("Removing value from cache store", "key", key, true));
           
                       sessionInit0(tx);
           
          @@ -692,7 +699,7 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx,
                       }
           
                       if (log.isDebugEnabled())
          -                log.debug("Removed value from cache store [key=" + key + ']');
          +                log.debug(S.toString("Removed value from cache store", "key", key, true));
           
                       return true;
                   }
          @@ -715,7 +722,8 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx,
                       Collection keys0 = cctx.unwrapBinariesIfNeeded(keys, !convertBinary());
           
                       if (log.isDebugEnabled())
          -                log.debug("Removing values from cache store [keys=" + keys0 + ']');
          +                log.debug(S.toString("Removing values from cache store",
          +                    "keys", keys0, true));
           
                       sessionInit0(tx);
           
          @@ -743,7 +751,8 @@ private void loadAllFromStore(@Nullable IgniteInternalTx tx,
                       }
           
                       if (log.isDebugEnabled())
          -                log.debug("Removed values from cache store [keys=" + keys0 + ']');
          +                log.debug(S.toString("Removed values from cache store",
          +                    "keys", keys0, true));
           
                       return true;
                   }
          @@ -1261,6 +1270,9 @@ private boolean mapContains(Cache.Entry e) {
           
                   /** {@inheritDoc} */
                   public String toString() {
          +            if (!S.INCLUDE_SENSITIVE)
          +                return "[size=" + size() + "]";
          +
                       Iterator> it = iterator();
           
                       if (!it.hasNext())
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
          index 858d9a7498510..7e98793045f93 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
          @@ -908,7 +908,7 @@ private static class StatefulValue extends ReentrantReadWriteLock {
                   private static final long serialVersionUID = 0L;
           
                   /** Value. */
          -        @GridToStringInclude
          +        @GridToStringInclude(sensitive = true)
                   private Entry val;
           
                   /** Store operation. */
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
          index 6d21dcfb716fd..e2f8438215005 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
          @@ -83,6 +83,7 @@
           import org.apache.ignite.internal.util.typedef.T2;
           import org.apache.ignite.internal.util.typedef.X;
           import org.apache.ignite.internal.util.typedef.internal.CU;
          +import org.apache.ignite.internal.util.typedef.internal.S;
           import org.apache.ignite.internal.util.typedef.internal.U;
           import org.apache.ignite.lang.IgniteBiClosure;
           import org.apache.ignite.lang.IgniteBiTuple;
          @@ -3331,10 +3332,12 @@ private  IgniteInternalFuture removeAllAsync0(
           
                   assert keys0 != null;
           
          -        if (log.isDebugEnabled()) {
          -            log.debug("Called removeAllAsync(...) [tx=" + this + ", keys=" + keys0 + ", implicit=" + implicit +
          -                ", retval=" + retval + "]");
          -        }
          +        if (log.isDebugEnabled())
          +            log.debug(S.toString("Called removeAllAsync(...)",
          +                "tx", this, false,
          +                "keys", keys0, true,
          +                "implicit", implicit, false,
          +                "retval", retval, false));
           
                   try {
                       checkValid();
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictContext.java
          index 3849bf541cef8..fa4020668e81e 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictContext.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictContext.java
          @@ -178,7 +178,7 @@ public long expireTime() {
               /** {@inheritDoc} */
               @Override public String toString() {
                   return state == State.MERGE ?
          -            S.toString(GridCacheVersionConflictContext.class, this, "mergeValue", mergeVal) :
          +            S.toString(GridCacheVersionConflictContext.class, this, "mergeValue", mergeVal, true) :
                       S.toString(GridCacheVersionConflictContext.class, this);
               }
           
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
          index a07dbf8e897c7..20fb6a0d0f485 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
          @@ -1778,7 +1778,7 @@ private static class C1 implements ComputeJob, Externalizable, GridNoImpli
                   protected IgniteClosure job;
           
                   /** */
          -        @GridToStringInclude
          +        @GridToStringInclude(sensitive = true)
                   private T arg;
           
                   /**
          @@ -1843,7 +1843,7 @@ public static class C1V2 implements ComputeJob, Binarylizable, GridNoImpli
                   protected IgniteClosure job;
           
                   /** */
          -        @GridToStringInclude
          +        @GridToStringInclude(sensitive = true)
                   protected T arg;
           
                   /**
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java
          index 0b629ddb8392a..91918c34e0fb1 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java
          @@ -48,7 +48,7 @@ public class GridContinuousMessage implements Message {
               private UUID routineId;
           
               /** Optional message data. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               @GridDirectTransient
               private Object data;
           
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/CollocatedSetItemKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/CollocatedSetItemKey.java
          index 94cffd45b7add..5f3811464b84c 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/CollocatedSetItemKey.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/CollocatedSetItemKey.java
          @@ -30,7 +30,7 @@ public class CollocatedSetItemKey implements SetItemKey {
               private IgniteUuid setId;
           
               /** */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private Object item;
           
               /** */
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongValue.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongValue.java
          index 42e43b691bd0c..5042672bd12c1 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongValue.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongValue.java
          @@ -22,6 +22,7 @@
           import java.io.ObjectInput;
           import java.io.ObjectOutput;
           import org.apache.ignite.internal.processors.cache.GridCacheInternal;
          +import org.apache.ignite.internal.util.tostring.GridToStringInclude;
           import org.apache.ignite.internal.util.typedef.internal.S;
           
           /**
          @@ -32,6 +33,7 @@ public final class GridCacheAtomicLongValue implements GridCacheInternal, Extern
               private static final long serialVersionUID = 0L;
           
               /** Value. */
          +    @GridToStringInclude(sensitive = true)
               private long val;
           
               /**
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
          index 7474f467d0800..4f660b6115525 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
          @@ -36,6 +36,7 @@
           import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
           import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
           import org.apache.ignite.internal.util.future.GridFinishedFuture;
          +import org.apache.ignite.internal.util.tostring.GridToStringInclude;
           import org.apache.ignite.internal.util.typedef.internal.A;
           import org.apache.ignite.internal.util.typedef.internal.CU;
           import org.apache.ignite.internal.util.typedef.internal.S;
          @@ -85,6 +86,7 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
               private volatile GridCacheContext ctx;
           
               /** Local value of sequence. */
          +    @GridToStringInclude(sensitive = true)
               private long locVal;
           
               /**  Upper bound of local counter. */
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceValue.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceValue.java
          index dd1a1d5342a93..ee540d602d6e5 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceValue.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceValue.java
          @@ -22,6 +22,7 @@
           import java.io.ObjectInput;
           import java.io.ObjectOutput;
           import org.apache.ignite.internal.processors.cache.GridCacheInternal;
          +import org.apache.ignite.internal.util.tostring.GridToStringInclude;
           import org.apache.ignite.internal.util.typedef.internal.S;
           
           /**
          @@ -32,6 +33,7 @@ public final class GridCacheAtomicSequenceValue implements GridCacheInternal, Ex
               private static final long serialVersionUID = 0L;
           
               /** Counter. */
          +    @GridToStringInclude(sensitive = true)
               private long val;
           
               /**
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchValue.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchValue.java
          index 17a11af72965c..ec996ffc526b6 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchValue.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchValue.java
          @@ -22,6 +22,7 @@
           import java.io.ObjectInput;
           import java.io.ObjectOutput;
           import org.apache.ignite.internal.processors.cache.GridCacheInternal;
          +import org.apache.ignite.internal.util.tostring.GridToStringInclude;
           import org.apache.ignite.internal.util.typedef.internal.S;
           
           /**
          @@ -32,9 +33,11 @@ public final class GridCacheCountDownLatchValue implements GridCacheInternal, Ex
               private static final long serialVersionUID = 0L;
           
               /** Count. */
          +    @GridToStringInclude(sensitive = true)
               private int cnt;
           
               /** Initial count. */
          +    @GridToStringInclude(sensitive = true)
               private int initCnt;
           
               /** Auto delete flag. */
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetItemKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetItemKey.java
          index 8b47b3debafc9..428089125b273 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetItemKey.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetItemKey.java
          @@ -37,7 +37,7 @@ public class GridCacheSetItemKey implements SetItemKey, Externalizable {
               private IgniteUuid setId;
           
               /** */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private Object item;
           
               /**
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
          index 9bee8490fca4f..6a00d96a0abc2 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
          @@ -566,8 +566,11 @@ private void execute0(boolean skipNtf) {
                                   }
                               });
           
          -                    if (log.isDebugEnabled())
          -                        log.debug("Job execution has successfully finished [job=" + job + ", res=" + res + ']');
          +                    if (log.isDebugEnabled()) {
          +                        log.debug(S.toString("Job execution has successfully finished",
          +                            "job", job, false,
          +                            "res", res, true));
          +                    }
                           }
                       }
                       catch (IgniteException e) {
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java
          index 1bcd41f66cc29..c0d1c601c92b7 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcQueryExecuteRequest.java
          @@ -18,11 +18,10 @@
           package org.apache.ignite.internal.processors.odbc;
           
           import org.apache.ignite.internal.util.tostring.GridToStringExclude;
          +import org.apache.ignite.internal.util.tostring.GridToStringInclude;
           import org.apache.ignite.internal.util.typedef.internal.S;
           import org.jetbrains.annotations.Nullable;
           
          -import java.util.Arrays;
          -
           /**
            * ODBC query execute request.
            */
          @@ -31,6 +30,7 @@ public class OdbcQueryExecuteRequest extends OdbcRequest {
               private final String cacheName;
           
               /** Sql query. */
          +    @GridToStringInclude(sensitive = true)
               private final String sqlQry;
           
               /** Sql query arguments. */
          @@ -73,6 +73,6 @@ public Object[] arguments() {
           
               /** {@inheritDoc} */
               @Override public String toString() {
          -        return S.toString(OdbcQueryExecuteRequest.class, this, "args", Arrays.toString(args));
          +        return S.toString(OdbcQueryExecuteRequest.class, this, "args", args, true);
               }
           }
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNativeException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNativeException.java
          index a99664a93764f..5c77cf2a4afbb 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNativeException.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNativeException.java
          @@ -72,6 +72,7 @@ public Object cause() {
           
               /** {@inheritDoc} */
               @Override public String toString() {
          -        return S.toString(PlatformNativeException.class, this, "cause", cause);
          +        return S.toString(PlatformNativeException.class, this,
          +            "cause", S.INCLUDE_SENSITIVE ? cause : (cause == null ? "null" : cause.getClass().getSimpleName()));
               }
           }
          \ No newline at end of file
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestResponse.java
          index ecbc6c8c6d4a3..18d1ddf6fc0d1 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestResponse.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestResponse.java
          @@ -59,7 +59,7 @@ public class GridRestResponse implements Externalizable {
               private String err;
           
               /** Response object. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private Object obj;
           
               /**
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
          index 2cd534e0905a7..08c39c04bf993 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
          @@ -61,7 +61,7 @@ public class GridFutureAdapter extends AbstractQueuedSynchronizer implements
               private byte resFlag;
           
               /** Result. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private Object res;
           
               /** Future start time. */
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridMetadataAwareAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridMetadataAwareAdapter.java
          index decc24438fe1e..1983ea31a6ee9 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridMetadataAwareAdapter.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridMetadataAwareAdapter.java
          @@ -67,7 +67,7 @@ public int key() {
               }
           
               /** Attributes. */
          -    @GridToStringInclude
          +    @GridToStringInclude(sensitive = true)
               private Object[] data = null;
           
               /**
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
          index b29d7cddc7366..333f95e64ae58 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
          @@ -35,6 +35,7 @@
           import java.util.concurrent.locks.ReadWriteLock;
           import java.util.concurrent.locks.ReentrantReadWriteLock;
           import org.apache.ignite.IgniteException;
          +import org.apache.ignite.IgniteSystemProperties;
           import org.apache.ignite.internal.util.typedef.F;
           import org.apache.ignite.internal.util.typedef.internal.SB;
           import org.apache.ignite.internal.util.typedef.internal.U;
          @@ -85,6 +86,9 @@ public class GridToStringBuilder {
               /** Maximum number of collection (map) entries to print. */
               public static final int MAX_COL_SIZE = 100;
           
          +    /** {@link IgniteSystemProperties#IGNITE_TO_STRING_INCLUDE_SENSITIVE} */
          +    public static final boolean INCLUDE_SENSITIVE = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_TO_STRING_INCLUDE_SENSITIVE, false);
          +
               /** */
               private static ThreadLocal> threadCache = new ThreadLocal>() {
                   @Override protected Queue initialValue() {
          @@ -114,8 +118,45 @@ public class GridToStringBuilder {
                * @param val4 Additional parameter value.
                * @return String presentation of the given object.
                */
          -    public static  String toString(Class cls, T obj, String name0, Object val0, String name1, Object val1,
          -        String name2, Object val2, String name3, Object val3, String name4, Object val4) {
          +    public static  String toString(Class cls, T obj,
          +        String name0, Object val0,
          +        String name1, Object val1,
          +        String name2, Object val2,
          +        String name3, Object val3,
          +        String name4, Object val4) {
          +        return toString(cls,
          +            obj,
          +            name0, val0, false,
          +            name1, val1, false,
          +            name2, val2, false,
          +            name3, val3, false,
          +            name4, val4, false);
          +    }
          +
          +    /**
          +     * Produces auto-generated output of string presentation for given object and its declaration class.
          +     *
          +     * @param  Type of the object.
          +     * @param cls Declaration class of the object. Note that this should not be a runtime class.
          +     * @param obj Object to get a string presentation for.
          +     * @param name0 Additional parameter name.
          +     * @param val0 Additional parameter value.
          +     * @param name1 Additional parameter name.
          +     * @param val1 Additional parameter value.
          +     * @param name2 Additional parameter name.
          +     * @param val2 Additional parameter value.
          +     * @param name3 Additional parameter name.
          +     * @param val3 Additional parameter value.
          +     * @param name4 Additional parameter name.
          +     * @param val4 Additional parameter value.
          +     * @return String presentation of the given object.
          +     */
          +    public static  String toString(Class cls, T obj,
          +        String name0, Object val0, boolean sens0,
          +        String name1, Object val1, boolean sens1,
          +        String name2, Object val2, boolean sens2,
          +        String name3, Object val3, boolean sens3,
          +        String name4, Object val4, boolean sens4) {
                   assert cls != null;
                   assert obj != null;
                   assert name0 != null;
          @@ -135,20 +176,26 @@ public static  String toString(Class cls, T obj, String name0, Object val0
           
                   Object[] addNames = tmp.getAdditionalNames();
                   Object[] addVals = tmp.getAdditionalValues();
          +        boolean[] addSens = tmp.getAdditionalSensitives();
           
                   addNames[0] = name0;
                   addVals[0] = val0;
          +        addSens[0] = sens0;
                   addNames[1] = name1;
                   addVals[1] = val1;
          +        addSens[1] = sens1;
                   addNames[2] = name2;
                   addVals[2] = val2;
          +        addSens[2] = sens2;
                   addNames[3] = name3;
                   addVals[3] = val3;
          +        addSens[3] = sens3;
                   addNames[4] = name4;
                   addVals[4] = val4;
          +        addSens[4] = sens4;
           
                   try {
          -            return toStringImpl(cls, tmp.getStringBuilder(), obj, addNames, addVals, 5);
          +            return toStringImpl(cls, tmp.getStringBuilder(), obj, addNames, addVals, addSens, 5);
                   }
                   finally {
                       queue.offer(tmp);
          @@ -171,8 +218,43 @@ public static  String toString(Class cls, T obj, String name0, Object val0
                * @param val3 Additional parameter value.
                * @return String presentation of the given object.
                */
          -    public static  String toString(Class cls, T obj, String name0, Object val0, String name1, Object val1,
          -        String name2, Object val2, String name3, Object val3) {
          +    public static  String toString(Class cls, T obj,
          +        String name0, Object val0,
          +        String name1, Object val1,
          +        String name2, Object val2,
          +        String name3, Object val3) {
          +        return toString(cls, obj,
          +            name0, val0, false,
          +            name1, val1, false,
          +            name2, val2, false,
          +            name3, val3, false);
          +    }
          +
          +    /**
          +     * Produces auto-generated output of string presentation for given object and its declaration class.
          +     *
          +     * @param  Type of the object.
          +     * @param cls Declaration class of the object. Note that this should not be a runtime class.
          +     * @param obj Object to get a string presentation for.
          +     * @param name0 Additional parameter name.
          +     * @param val0 Additional parameter value.
          +     * @param sens0 Property sensitive flag.
          +     * @param name1 Additional parameter name.
          +     * @param val1 Additional parameter value.
          +     * @param sens1 Property sensitive flag.
          +     * @param name2 Additional parameter name.
          +     * @param val2 Additional parameter value.
          +     * @param sens2 Property sensitive flag.
          +     * @param name3 Additional parameter name.
          +     * @param val3 Additional parameter value.
          +     * @param sens3 Property sensitive flag.
          +     * @return String presentation of the given object.
          +     */
          +    public static  String toString(Class cls, T obj,
          +        String name0, Object val0, boolean sens0,
          +        String name1, Object val1, boolean sens1,
          +        String name2, Object val2, boolean sens2,
          +        String name3, Object val3, boolean sens3) {
                   assert cls != null;
                   assert obj != null;
                   assert name0 != null;
          @@ -191,18 +273,23 @@ public static  String toString(Class cls, T obj, String name0, Object val0
           
                   Object[] addNames = tmp.getAdditionalNames();
                   Object[] addVals = tmp.getAdditionalValues();
          +        boolean[] addSens = tmp.getAdditionalSensitives();
           
                   addNames[0] = name0;
                   addVals[0] = val0;
          +        addSens[0] = sens0;
                   addNames[1] = name1;
                   addVals[1] = val1;
          +        addSens[1] = sens1;
                   addNames[2] = name2;
                   addVals[2] = val2;
          +        addSens[2] = sens2;
                   addNames[3] = name3;
                   addVals[3] = val3;
          +        addSens[3] = sens3;
           
                   try {
          -            return toStringImpl(cls, tmp.getStringBuilder(), obj, addNames, addVals, 4);
          +            return toStringImpl(cls, tmp.getStringBuilder(), obj, addNames, addVals, addSens, 4);
                   }
                   finally {
                       queue.offer(tmp);
          @@ -223,8 +310,38 @@ public static  String toString(Class cls, T obj, String name0, Object val0
                * @param val2 Additional parameter value.
                * @return String presentation of the given object.
                */
          -    public static  String toString(Class cls, T obj, String name0, Object val0, String name1, Object val1,
          +    public static  String toString(Class cls, T obj,
          +        String name0, Object val0,
          +        String name1, Object val1,
                   String name2, Object val2) {
          +        return toString(cls,
          +            obj,
          +            name0, val0, false,
          +            name1, val1, false,
          +            name2, val2, false);
          +    }
          +
          +    /**
          +     * Produces auto-generated output of string presentation for given object and its declaration class.
          +     *
          +     * @param  Type of the object.
          +     * @param cls Declaration class of the object. Note that this should not be a runtime class.
          +     * @param obj Object to get a string presentation for.
          +     * @param name0 Additional parameter name.
          +     * @param val0 Additional parameter value.
          +     * @param sens0 Property sensitive flag.
          +     * @param name1 Additional parameter name.
          +     * @param val1 Additional parameter value.
          +     * @param sens1 Property sensitive flag.
          +     * @param name2 Additional parameter name.
          +     * @param val2 Additional parameter value.
          +     * @param sens2 Property sensitive flag.
          +     * @return String presentation of the given object.
          +     */
          +    public static  String toString(Class cls, T obj,
          +        String name0, Object val0, boolean sens0,
          +        String name1, Object val1, boolean sens1,
          +        String name2, Object val2, boolean sens2) {
                   assert cls != null;
                   assert obj != null;
                   assert name0 != null;
          @@ -242,16 +359,20 @@ public static  String toString(Class cls, T obj, String name0, Object val0
           
                   Object[] addNames = tmp.getAdditionalNames();
                   Object[] addVals = tmp.getAdditionalValues();
          +        boolean[] addSens = tmp.getAdditionalSensitives();
           
                   addNames[0] = name0;
                   addVals[0] = val0;
          +        addSens[0] = sens0;
                   addNames[1] = name1;
                   addVals[1] = val1;
          +        addSens[1] = sens1;
                   addNames[2] = name2;
                   addVals[2] = val2;
          +        addSens[2] = sens2;
           
                   try {
          -            return toStringImpl(cls, tmp.getStringBuilder(), obj, addNames, addVals, 3);
          +            return toStringImpl(cls, tmp.getStringBuilder(), obj, addNames, addVals, addSens, 3);
                   }
                   finally {
                       queue.offer(tmp);
          @@ -270,7 +391,29 @@ public static  String toString(Class cls, T obj, String name0, Object val0
                * @param val1 Additional parameter value.
                * @return String presentation of the given object.
                */
          -    public static  String toString(Class cls, T obj, String name0, Object val0, String name1, Object val1) {
          +    public static  String toString(Class cls, T obj,
          +        String name0, Object val0,
          +        String name1, Object val1) {
          +        return toString(cls, obj, name0, val0, false, name1, val1, false);
          +    }
          +
          +    /**
          +     * Produces auto-generated output of string presentation for given object and its declaration class.
          +     *
          +     * @param  Type of the object.
          +     * @param cls Declaration class of the object. Note that this should not be a runtime class.
          +     * @param obj Object to get a string presentation for.
          +     * @param name0 Additional parameter name.
          +     * @param val0 Additional parameter value.
          +     * @param sens0 Property sensitive flag.
          +     * @param name1 Additional parameter name.
          +     * @param val1 Additional parameter value.
          +     * @param sens1 Property sensitive flag.
          +     * @return String presentation of the given object.
          +     */
          +    public static  String toString(Class cls, T obj,
          +        String name0, Object val0, boolean sens0,
          +        String name1, Object val1, boolean sens1) {
                   assert cls != null;
                   assert obj != null;
                   assert name0 != null;
          @@ -287,14 +430,17 @@ public static  String toString(Class cls, T obj, String name0, Object val0
           
                   Object[] addNames = tmp.getAdditionalNames();
                   Object[] addVals = tmp.getAdditionalValues();
          +        boolean[] addSens = tmp.getAdditionalSensitives();
           
                   addNames[0] = name0;
                   addVals[0] = val0;
          +        addSens[0] = sens0;
                   addNames[1] = name1;
                   addVals[1] = val1;
          +        addSens[1] = sens1;
           
                   try {
          -            return toStringImpl(cls, tmp.getStringBuilder(), obj, addNames, addVals, 2);
          +            return toStringImpl(cls, tmp.getStringBuilder(), obj, addNames, addVals, addSens, 2);
                   }
                   finally {
                       queue.offer(tmp);
          @@ -312,6 +458,21 @@ public static  String toString(Class cls, T obj, String name0, Object val0
                * @return String presentation of the given object.
                */
               public static  String toString(Class cls, T obj, String name, @Nullable Object val) {
          +        return toString(cls, obj, name, val, false);
          +    }
          +
          +    /**
          +     * Produces auto-generated output of string presentation for given object and its declaration class.
          +     *
          +     * @param  Type of the object.
          +     * @param cls Declaration class of the object. Note that this should not be a runtime class.
          +     * @param obj Object to get a string presentation for.
          +     * @param name Additional parameter name.
          +     * @param val Additional parameter value.
          +     * @param sens Property sensitive flag.
          +     * @return String presentation of the given object.
          +     */
          +    public static  String toString(Class cls, T obj, String name, @Nullable Object val, boolean sens) {
                   assert cls != null;
                   assert obj != null;
                   assert name != null;
          @@ -327,12 +488,14 @@ public static  String toString(Class cls, T obj, String name, @Nullable Ob
           
                   Object[] addNames = tmp.getAdditionalNames();
                   Object[] addVals = tmp.getAdditionalValues();
          +        boolean[] addSens = tmp.getAdditionalSensitives();
           
                   addNames[0] = name;
                   addVals[0] = val;
          +        addSens[0] = sens;
           
                   try {
          -            return toStringImpl(cls, tmp.getStringBuilder(), obj, addNames, addVals, 1);
          +            return toStringImpl(cls, tmp.getStringBuilder(), obj, addNames, addVals, addSens, 1);
                   }
                   finally {
                       queue.offer(tmp);
          @@ -362,7 +525,7 @@ public static  String toString(Class cls, T obj) {
           
                   try {
                       return toStringImpl(cls, tmp.getStringBuilder(), obj, tmp.getAdditionalNames(),
          -                tmp.getAdditionalValues(), 0);
          +                tmp.getAdditionalValues(), null, 0);
                   }
                   finally {
                       queue.offer(tmp);
          @@ -390,12 +553,16 @@ public static  String toString(Class cls, T obj, String parent) {
                * @param obj Object for which to get string presentation.
                * @param addNames Names of additional values to be included.
                * @param addVals Additional values to be included.
          +     * @param addSens Sensitive flag of values or {@code null} if all values are not sensitive.
                * @param addLen How many additional values will be included.
                * @return String presentation of the given object.
                * @param  Type of object.
                */
               @SuppressWarnings({"unchecked"})
          -    private static  String toStringImpl(Class cls, SB buf, T obj, Object[] addNames, Object[] addVals,
          +    private static  String toStringImpl(Class cls, SB buf, T obj,
          +        Object[] addNames,
          +        Object[] addVals,
          +        @Nullable boolean[] addSens,
                   int addLen) {
                   assert cls != null;
                   assert buf != null;
          @@ -430,26 +597,10 @@ private static  String toStringImpl(Class cls, SB buf, T obj, Object[] add
           
                           buf.a(name).a('=');
           
          -                if (field.getType().isArray()) {
          -                    if (field.getType().equals(byte[].class))
          -                        buf.a(Arrays.toString((byte[])field.get(obj)));
          -                    else if (field.getType().equals(boolean[].class))
          -                        buf.a(Arrays.toString((boolean[])field.get(obj)));
          -                    else if (field.getType().equals(short[].class))
          -                        buf.a(Arrays.toString((short[])field.get(obj)));
          -                    else if (field.getType().equals(int[].class))
          -                        buf.a(Arrays.toString((int[])field.get(obj)));
          -                    else if (field.getType().equals(long[].class))
          -                        buf.a(Arrays.toString((long[])field.get(obj)));
          -                    else if (field.getType().equals(float[].class))
          -                        buf.a(Arrays.toString((float[])field.get(obj)));
          -                    else if (field.getType().equals(double[].class))
          -                        buf.a(Arrays.toString((double[])field.get(obj)));
          -                    else if (field.getType().equals(char[].class))
          -                        buf.a(Arrays.toString((char[])field.get(obj)));
          -                    else
          -                        buf.a(Arrays.toString((Object[])field.get(obj)));
          -                }
          +                Class fieldType = field.getType();
          +
          +                if (fieldType.isArray())
          +                    buf.a(arrayToString(fieldType, field.get(obj)));
                           else {
                               Object val = field.get(obj);
           
          @@ -475,15 +626,7 @@ else if (val instanceof Map && ((Map)val).size() > MAX_COL_SIZE) {
                           }
                       }
           
          -            if (addLen > 0)
          -                for (int i = 0; i < addLen; i++) {
          -                    if (!first)
          -                       buf.a(", ");
          -                    else
          -                        first = false;
          -
          -                    buf.a(addNames[i]).a('=').a(addVals[i]);
          -                }
          +            appendVals(buf, first, addNames, addVals, addSens, addLen);
           
                       buf.a(']');
           
          @@ -507,6 +650,402 @@ else if (val instanceof Map && ((Map)val).size() > MAX_COL_SIZE) {
                   }
               }
           
          +    /**
          +     * @param arrType Type of the array.
          +     * @param arr Array object.
          +     * @return String representation of an array.
          +     */
          +    private static String arrayToString(Class arrType, Object arr) {
          +        if (arrType.equals(byte[].class))
          +            return Arrays.toString((byte[])arr);
          +        if (arrType.equals(boolean[].class))
          +            return Arrays.toString((boolean[])arr);
          +        if (arrType.equals(short[].class))
          +            return Arrays.toString((short[])arr);
          +        if (arrType.equals(int[].class))
          +            return Arrays.toString((int[])arr);
          +        if (arrType.equals(long[].class))
          +            return Arrays.toString((long[])arr);
          +        if (arrType.equals(float[].class))
          +            return Arrays.toString((float[])arr);
          +        if (arrType.equals(double[].class))
          +            return Arrays.toString((double[])arr);
          +        if (arrType.equals(char[].class))
          +            return Arrays.toString((char[])arr);
          +
          +        return Arrays.toString((Object[])arr);
          +    }
          +
          +    /**
          +     * Produces uniformed output of string with context properties
          +     *
          +     * @param str Output prefix or {@code null} if empty.
          +     * @param name Property name.
          +     * @param val Property value.
          +     * @return String presentation.
          +     */
          +    public static String toString(String str, String name, @Nullable Object val) {
          +        return toString(str, name, val, false);
          +    }
          +
          +    /**
          +     * Produces uniformed output of string with context properties
          +     *
          +     * @param str Output prefix or {@code null} if empty.
          +     * @param name Property name.
          +     * @param val Property value.
          +     * @param sens Property sensitive flag.
          +     * @return String presentation.
          +     */
          +    public static String toString(String str, String name, @Nullable Object val, boolean sens) {
          +        assert name != null;
          +
          +        Queue queue = threadCache.get();
          +
          +        assert queue != null;
          +
          +        // Since string() methods can be chain-called from the same thread we
          +        // have to keep a list of thread-local objects and remove/add them
          +        // in each string() apply.
          +        GridToStringThreadLocal tmp = queue.isEmpty() ? new GridToStringThreadLocal() : queue.remove();
          +
          +        Object[] propNames = tmp.getAdditionalNames();
          +        Object[] propVals = tmp.getAdditionalValues();
          +        boolean[] propSens = tmp.getAdditionalSensitives();
          +
          +        propNames[0] = name;
          +        propVals[0] = val;
          +        propSens[0] = sens;
          +
          +        try {
          +            return toStringImpl(str, tmp.getStringBuilder(), propNames, propVals, propSens, 1);
          +        }
          +        finally {
          +            queue.offer(tmp);
          +        }
          +    }
          +
          +    /**
          +     * Produces uniformed output of string with context properties
          +     *
          +     * @param str Output prefix or {@code null} if empty.
          +     * @param name0 Property name.
          +     * @param val0 Property value.
          +     * @param name1 Property name.
          +     * @param val1 Property value.
          +     * @return String presentation.
          +     */
          +    public static String toString(String str, String name0, @Nullable Object val0, String name1,
          +        @Nullable Object val1) {
          +        return toString(str, name0, val0, false, name1, val1, false);
          +    }
          +
          +    /**
          +     * Produces uniformed output of string with context properties
          +     *
          +     * @param str Output prefix or {@code null} if empty.
          +     * @param name0 Property name.
          +     * @param val0 Property value.
          +     * @param sens0 Property sensitive flag.
          +     * @param name1 Property name.
          +     * @param val1 Property value.
          +     * @param sens1 Property sensitive flag.
          +     * @return String presentation.
          +     */
          +    public static String toString(String str,
          +        String name0, @Nullable Object val0, boolean sens0,
          +        String name1, @Nullable Object val1, boolean sens1) {
          +        assert name0 != null;
          +        assert name1 != null;
          +
          +        Queue queue = threadCache.get();
          +
          +        assert queue != null;
          +
          +        // Since string() methods can be chain-called from the same thread we
          +        // have to keep a list of thread-local objects and remove/add them
          +        // in each string() apply.
          +        GridToStringThreadLocal tmp = queue.isEmpty() ? new GridToStringThreadLocal() : queue.remove();
          +
          +        Object[] propNames = tmp.getAdditionalNames();
          +        Object[] propVals = tmp.getAdditionalValues();
          +        boolean[] propSens = tmp.getAdditionalSensitives();
          +
          +        propNames[0] = name0;
          +        propVals[0] = val0;
          +        propSens[0] = sens0;
          +        propNames[1] = name1;
          +        propVals[1] = val1;
          +        propSens[1] = sens1;
          +
          +        try {
          +            return toStringImpl(str, tmp.getStringBuilder(), propNames, propVals, propSens, 2);
          +        }
          +        finally {
          +            queue.offer(tmp);
          +        }
          +    }
          +
          +    /**
          +     * Produces uniformed output of string with context properties
          +     *
          +     * @param str Output prefix or {@code null} if empty.
          +     * @param name0 Property name.
          +     * @param val0 Property value.
          +     * @param sens0 Property sensitive flag.
          +     * @param name1 Property name.
          +     * @param val1 Property value.
          +     * @param sens1 Property sensitive flag.
          +     * @param name2 Property name.
          +     * @param val2 Property value.
          +     * @param sens2 Property sensitive flag.
          +     * @return String presentation.
          +     */
          +    public static String toString(String str,
          +        String name0, @Nullable Object val0, boolean sens0,
          +        String name1, @Nullable Object val1, boolean sens1,
          +        String name2, @Nullable Object val2, boolean sens2) {
          +        assert name0 != null;
          +        assert name1 != null;
          +        assert name2 != null;
          +
          +        Queue queue = threadCache.get();
          +
          +        assert queue != null;
          +
          +        // Since string() methods can be chain-called from the same thread we
          +        // have to keep a list of thread-local objects and remove/add them
          +        // in each string() apply.
          +        GridToStringThreadLocal tmp = queue.isEmpty() ? new GridToStringThreadLocal() : queue.remove();
          +
          +        Object[] propNames = tmp.getAdditionalNames();
          +        Object[] propVals = tmp.getAdditionalValues();
          +        boolean[] propSens = tmp.getAdditionalSensitives();
          +
          +        propNames[0] = name0;
          +        propVals[0] = val0;
          +        propSens[0] = sens0;
          +        propNames[1] = name1;
          +        propVals[1] = val1;
          +        propSens[1] = sens1;
          +        propNames[2] = name2;
          +        propVals[2] = val2;
          +        propSens[2] = sens2;
          +
          +        try {
          +            return toStringImpl(str, tmp.getStringBuilder(), propNames, propVals, propSens, 3);
          +        }
          +        finally {
          +            queue.offer(tmp);
          +        }
          +    }
          +
          +    /**
          +     * Produces uniformed output of string with context properties
          +     *
          +     * @param str Output prefix or {@code null} if empty.
          +     * @param name0 Property name.
          +     * @param val0 Property value.
          +     * @param sens0 Property sensitive flag.
          +     * @param name1 Property name.
          +     * @param val1 Property value.
          +     * @param sens1 Property sensitive flag.
          +     * @param name2 Property name.
          +     * @param val2 Property value.
          +     * @param sens2 Property sensitive flag.
          +     * @param name3 Property name.
          +     * @param val3 Property value.
          +     * @param sens3 Property sensitive flag.
          +     * @return String presentation.
          +     */
          +    public static String toString(String str,
          +        String name0, @Nullable Object val0, boolean sens0,
          +        String name1, @Nullable Object val1, boolean sens1,
          +        String name2, @Nullable Object val2, boolean sens2,
          +        String name3, @Nullable Object val3, boolean sens3) {
          +        assert name0 != null;
          +        assert name1 != null;
          +        assert name2 != null;
          +        assert name3 != null;
          +
          +        Queue queue = threadCache.get();
          +
          +        assert queue != null;
          +
          +        // Since string() methods can be chain-called from the same thread we
          +        // have to keep a list of thread-local objects and remove/add them
          +        // in each string() apply.
          +        GridToStringThreadLocal tmp = queue.isEmpty() ? new GridToStringThreadLocal() : queue.remove();
          +
          +        Object[] propNames = tmp.getAdditionalNames();
          +        Object[] propVals = tmp.getAdditionalValues();
          +        boolean[] propSens = tmp.getAdditionalSensitives();
          +
          +        propNames[0] = name0;
          +        propVals[0] = val0;
          +        propSens[0] = sens0;
          +        propNames[1] = name1;
          +        propVals[1] = val1;
          +        propSens[1] = sens1;
          +        propNames[2] = name2;
          +        propVals[2] = val2;
          +        propSens[2] = sens2;
          +        propNames[3] = name3;
          +        propVals[3] = val3;
          +        propSens[3] = sens3;
          +
          +        try {
          +            return toStringImpl(str, tmp.getStringBuilder(), propNames, propVals, propSens, 4);
          +        }
          +        finally {
          +            queue.offer(tmp);
          +        }
          +    }
          +
          +    /**
          +     * Produces uniformed output of string with context properties
          +     *
          +     * @param str Output prefix or {@code null} if empty.
          +     * @param name0 Property name.
          +     * @param val0 Property value.
          +     * @param sens0 Property sensitive flag.
          +     * @param name1 Property name.
          +     * @param val1 Property value.
          +     * @param sens1 Property sensitive flag.
          +     * @param name2 Property name.
          +     * @param val2 Property value.
          +     * @param sens2 Property sensitive flag.
          +     * @param name3 Property name.
          +     * @param val3 Property value.
          +     * @param sens3 Property sensitive flag.
          +     * @param name4 Property name.
          +     * @param val4 Property value.
          +     * @param sens4 Property sensitive flag.
          +     * @return String presentation.
          +     */
          +    public static String toString(String str,
          +        String name0, @Nullable Object val0, boolean sens0,
          +        String name1, @Nullable Object val1, boolean sens1,
          +        String name2, @Nullable Object val2, boolean sens2,
          +        String name3, @Nullable Object val3, boolean sens3,
          +        String name4, @Nullable Object val4, boolean sens4) {
          +        assert name0 != null;
          +        assert name1 != null;
          +        assert name2 != null;
          +        assert name3 != null;
          +        assert name4 != null;
          +
          +        Queue queue = threadCache.get();
          +
          +        assert queue != null;
          +
          +        // Since string() methods can be chain-called from the same thread we
          +        // have to keep a list of thread-local objects and remove/add them
          +        // in each string() apply.
          +        GridToStringThreadLocal tmp = queue.isEmpty() ? new GridToStringThreadLocal() : queue.remove();
          +
          +        Object[] propNames = tmp.getAdditionalNames();
          +        Object[] propVals = tmp.getAdditionalValues();
          +        boolean[] propSens = tmp.getAdditionalSensitives();
          +
          +        propNames[0] = name0;
          +        propVals[0] = val0;
          +        propSens[0] = sens0;
          +        propNames[1] = name1;
          +        propVals[1] = val1;
          +        propSens[1] = sens1;
          +        propNames[2] = name2;
          +        propVals[2] = val2;
          +        propSens[2] = sens2;
          +        propNames[3] = name3;
          +        propVals[3] = val3;
          +        propSens[3] = sens3;
          +        propNames[4] = name4;
          +        propVals[4] = val4;
          +        propSens[4] = sens4;
          +
          +        try {
          +            return toStringImpl(str, tmp.getStringBuilder(), propNames, propVals, propSens, 5);
          +        }
          +        finally {
          +            queue.offer(tmp);
          +        }
          +    }
          +
          +    /**
          +     * Creates an uniformed string presentation for the binary-like object.
          +     *
          +     * @param str Output prefix or {@code null} if empty.
          +     * @param buf String builder buffer.
          +     * @param propNames Names of object properties.
          +     * @param propVals Property values.
          +     * @param propSens Sensitive flag of values or {@code null} if all values is not sensitive.
          +     * @param propCnt Properties count.
          +     * @return String presentation of the object.
          +     */
          +    private static String toStringImpl(String str, SB buf, Object[] propNames, Object[] propVals,
          +        boolean[] propSens, int propCnt) {
          +
          +        buf.setLength(0);
          +
          +        if (str != null)
          +            buf.a(str).a(" ");
          +
          +        buf.a("[");
          +
          +        appendVals(buf, true, propNames, propVals, propSens, propCnt);
          +
          +        buf.a(']');
          +
          +        return buf.toString();
          +    }
          +
          +    /**
          +     * Append additional values to the buffer.
          +     *
          +     * @param buf Buffer.
          +     * @param first First value flag.
          +     * @param addNames Names of additional values to be included.
          +     * @param addVals Additional values to be included.
          +     * @param addSens Sensitive flag of values or {@code null} if all values are not sensitive.
          +     * @param addLen How many additional values will be included.
          +     */
          +    private static void appendVals(SB buf,
          +        boolean first,
          +        Object[] addNames,
          +        Object[] addVals,
          +        boolean[] addSens,
          +        int addLen)
          +    {
          +        if (addLen > 0) {
          +            for (int i = 0; i < addLen; i++) {
          +                Object addVal = addVals[i];
          +
          +                if (addVal != null) {
          +                    if (addSens != null && addSens[i] && !INCLUDE_SENSITIVE)
          +                        continue;
          +
          +                    GridToStringInclude incAnn = addVal.getClass().getAnnotation(GridToStringInclude.class);
          +
          +                    if (incAnn != null && incAnn.sensitive() && !INCLUDE_SENSITIVE)
          +                        continue;
          +
          +                    Class cls = addVal.getClass();
          +
          +                    if (cls.isArray())
          +                        addVal = arrayToString(cls, addVal);
          +                }
          +
          +                if (!first)
          +                    buf.a(", ");
          +                else
          +                    first = false;
          +
          +                buf.a(addNames[i]).a('=').a(addVal);
          +            }
          +        }
          +    }
          +
               /**
                * @param cls Class.
                * @param  Type of the object.
          @@ -518,7 +1057,7 @@ private static  GridToStringClassDescriptor getClassDescriptor(Class cls)
           
                   String key = cls.getName() + System.identityHashCode(cls.getClassLoader());
           
          -        GridToStringClassDescriptor cd = null;
          +        GridToStringClassDescriptor cd;
           
                   rwLock.readLock().lock();
           
          @@ -537,9 +1076,15 @@ private static  GridToStringClassDescriptor getClassDescriptor(Class cls)
           
                           Class type = f.getType();
           
          -                if (f.isAnnotationPresent(GridToStringInclude.class) ||
          -                    type.isAnnotationPresent(GridToStringInclude.class))
          -                    add = true;
          +                final GridToStringInclude incFld = f.getAnnotation(GridToStringInclude.class);
          +                final GridToStringInclude incType = type.getAnnotation(GridToStringInclude.class);
          +
          +                if (incFld != null || incType != null) {
          +                    // Information is not sensitive when both the field and the field type are not sensitive.
          +                    // When @GridToStringInclude is not present then the flag is false by default for that attribute.
          +                    final boolean notSens = (incFld == null || !incFld.sensitive()) && (incType == null || !incType.sensitive());
          +                    add = notSens || INCLUDE_SENSITIVE;
          +                }
                           else if (!f.isAnnotationPresent(GridToStringExclude.class) &&
                               !f.getType().isAnnotationPresent(GridToStringExclude.class)) {
                               if (
          @@ -573,8 +1118,9 @@ else if (!f.isAnnotationPresent(GridToStringExclude.class) &&
                               GridToStringFieldDescriptor fd = new GridToStringFieldDescriptor(f.getName());
           
                               // Get order, if any.
          -                    if (f.isAnnotationPresent(GridToStringOrder.class))
          -                        fd.setOrder(f.getAnnotation(GridToStringOrder.class).value());
          +                    final GridToStringOrder annOrder = f.getAnnotation(GridToStringOrder.class);
          +                    if (annOrder != null)
          +                        fd.setOrder(annOrder.value());
           
                               cd.addField(fd);
                           }
          diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringInclude.java b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringInclude.java
          index ec502a73d62b7..5ea1fe68ca62e 100644
          --- a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringInclude.java
          +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringInclude.java
          @@ -17,6 +17,8 @@
           
           package org.apache.ignite.internal.util.tostring;
           
          +import org.apache.ignite.IgniteSystemProperties;
          +
           import java.lang.annotation.Documented;
           import java.lang.annotation.ElementType;
           import java.lang.annotation.Retention;
          @@ -32,5 +34,13 @@
           @Retention(RetentionPolicy.RUNTIME)
           @Target({ElementType.FIELD, ElementType.TYPE})
           public @interface GridToStringInclude {
          -    // No-op.
          +    /**
          +     * A flag indicating a sensitive information stored in the field or fields of the class.
          + * Such information will be included in {@code toString()} output ONLY when the system property + * {@link IgniteSystemProperties#IGNITE_TO_STRING_INCLUDE_SENSITIVE IGNITE_TO_STRING_INCLUDE_SENSITIVE} + * is set to {@code true}. + * + * @return Attribute value. + */ + boolean sensitive() default false; } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringThreadLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringThreadLocal.java index ab9145298adf9..33fc6a0e8e7df 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringThreadLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringThreadLocal.java @@ -20,7 +20,7 @@ import org.apache.ignite.internal.util.typedef.internal.SB; /** - * Helper wrapper containing StringBuilder and additional values. Stored as a thread-lcal variable. + * Helper wrapper containing StringBuilder and additional values. Stored as a thread-local variable. */ class GridToStringThreadLocal { /** */ @@ -32,6 +32,9 @@ class GridToStringThreadLocal { /** */ private Object[] addVals = new Object[5]; + /** */ + private boolean[] addSens = new boolean[5]; + /** * @return String builder. */ @@ -52,4 +55,11 @@ Object[] getAdditionalNames() { Object[] getAdditionalValues() { return addVals; } + + /** + * @return Additional values. + */ + boolean[] getAdditionalSensitives() { + return addSens; + } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java index 150c2454762b5..bf7278236d126 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java @@ -60,6 +60,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; import org.apache.ignite.internal.util.typedef.P2; import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; @@ -224,7 +225,11 @@ public void testCircularReference() throws Exception { String typeName = nameMapper.typeName(TestReferenceObject.class.getName()); - assertTrue("Unexpected toString: " + str, str.startsWith(typeName) && str.contains("obj=" + typeName + " [")); + assertTrue("Unexpected toString: " + str, + S.INCLUDE_SENSITIVE ? + str.startsWith(typeName) && str.contains("obj=" + typeName + " [") : + str.startsWith("BinaryObject") && str.contains("idHash=") && str.contains("hash=") + ); TestReferenceObject obj1_r = po.deserialize(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/tostring/GridToStringBuilderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/tostring/GridToStringBuilderSelfTest.java index 4f5fe9b2500e1..6a0eae2d17b48 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/tostring/GridToStringBuilderSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/tostring/GridToStringBuilderSelfTest.java @@ -54,7 +54,13 @@ public void testToStringWithAdditions() throws Exception { IgniteLogger log = log(); - log.info(obj.toStringWithAdditional()); + String manual = obj.toStringWithAdditionalManual(); + log.info(manual); + + String automatic = obj.toStringWithAdditionalAutomatic(); + log.info(automatic); + + assert manual.equals(automatic); } /** @@ -117,7 +123,7 @@ public void testToStringPerformance() { /** * Test class. */ - private class TestClass1 { + private static class TestClass1 { /** */ @SuppressWarnings("unused") @GridToStringOrder(0) @@ -129,6 +135,7 @@ private class TestClass1 { /** */ @SuppressWarnings("unused") + @GridToStringInclude(sensitive = true) private long longVar; /** */ @@ -180,7 +187,8 @@ String toStringManual() { buf.append("id=").append(id).append(", "); buf.append("uuidVar=").append(uuidVar).append(", "); buf.append("intVar=").append(intVar).append(", "); - buf.append("longVar=").append(longVar).append(", "); + if (S.INCLUDE_SENSITIVE) + buf.append("longVar=").append(longVar).append(", "); buf.append("boolVar=").append(boolVar).append(", "); buf.append("byteVar=").append(byteVar).append(", "); buf.append("name=").append(name).append(", "); @@ -200,10 +208,23 @@ String toStringAutomatic() { } /** - * @return String with additional parameters. + * @return Automatic string with additional parameters. + */ + String toStringWithAdditionalAutomatic() { + return S.toString(TestClass1.class, this, "newParam1", 1, false, "newParam2", 2, true); + } + + /** + * @return Manual string with additional parameters. */ - String toStringWithAdditional() { - return S.toString(TestClass1.class, this, "newParam1", 1, "newParam2", 2); + String toStringWithAdditionalManual() { + StringBuilder s = new StringBuilder(toStringManual()); + s.setLength(s.length() - 1); + s.append(", newParam1=").append(1); + if (S.INCLUDE_SENSITIVE) + s.append(", newParam2=").append(2); + s.append(']'); + return s.toString(); } } } \ No newline at end of file From 5769f44367cae5908cd291f226e9fccd68fe1c39 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Tue, 27 Dec 2016 15:14:13 +0700 Subject: [PATCH 412/487] Fixed Visor queries for BinaryObjects. --- .../query/VisorQueryScanSubstringFilter.java | 5 +- .../internal/visor/query/VisorQueryUtils.java | 60 +++++++++++++++++++ 2 files changed, 63 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryScanSubstringFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryScanSubstringFilter.java index 43eb6dda64e7f..171698b52a3dc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryScanSubstringFilter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryScanSubstringFilter.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.visor.query; +import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.lang.IgniteBiPredicate; /** @@ -52,8 +53,8 @@ public VisorQueryScanSubstringFilter(boolean caseSensitive, String ptrn) { * @return {@code true} when string presentation of key or value contain specified string. */ @Override public boolean apply(Object key, Object val) { - String k = key.toString(); - String v = val.toString(); + String k = key instanceof BinaryObject ? VisorQueryUtils.binaryToString((BinaryObject)key) : key.toString(); + String v = val instanceof BinaryObject ? VisorQueryUtils.binaryToString((BinaryObject)val) : val.toString(); if (caseSensitive) return k.contains(ptrn) || v.contains(ptrn); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryUtils.java index 0b8cf837a339c..5faeac0754abf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryUtils.java @@ -25,7 +25,13 @@ import java.util.Date; import java.util.List; import javax.cache.Cache; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.binary.BinaryType; +import org.apache.ignite.internal.binary.BinaryObjectEx; import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.SB; /** * Contains utility methods for Visor query tasks and jobs. @@ -77,12 +83,19 @@ private static String typeOf(Object o) { private static String valueOf(Object o) { if (o == null) return "null"; + if (o instanceof byte[]) return "size=" + ((byte[])o).length; + if (o instanceof Byte[]) return "size=" + ((Byte[])o).length; + if (o instanceof Object[]) return "size=" + ((Object[])o).length + ", values=[" + mkString((Object[])o, 120) + "]"; + + if (o instanceof BinaryObject) + return binaryToString((BinaryObject)o); + return o.toString(); } @@ -167,6 +180,51 @@ private static boolean isKnownType(Object obj) { obj instanceof URL; } + /** + * Convert Binary object to string. + * + * @param obj Binary object. + * @return String representation of Binary object. + */ + public static String binaryToString(BinaryObject obj) { + int hash = obj.hashCode(); + + if (obj instanceof BinaryObjectEx) { + BinaryObjectEx objEx = (BinaryObjectEx)obj; + + BinaryType meta; + + try { + meta = ((BinaryObjectEx)obj).rawType(); + } + catch (BinaryObjectException ignore) { + meta = null; + } + + if (meta != null) { + SB buf = new SB(meta.typeName()); + + if (meta.fieldNames() != null) { + buf.a(" [hash=").a(hash); + + for (String name : meta.fieldNames()) { + Object val = objEx.field(name); + + buf.a(", ").a(name).a('=').a(val); + } + + buf.a(']'); + + return buf.toString(); + } + } + } + + return S.toString(obj.getClass().getSimpleName(), + "hash", hash, false, + "typeId", obj.type().typeId(), true); + } + /** * Collects rows from sql query future, first time creates meta and column names arrays. * @@ -193,6 +251,8 @@ public static List fetchSqlQueryRows(VisorQueryCursor> cur, in row[i] = null; else if (isKnownType(o)) row[i] = o; + else if (o instanceof BinaryObject) + row[i] = binaryToString((BinaryObject)o); else row[i] = o.getClass().isArray() ? "binary" : o.toString(); } From 5494dfb8dd222bf7aee8214b6bb201d3ae8a1ec5 Mon Sep 17 00:00:00 2001 From: Ignite Teamcity Date: Tue, 27 Dec 2016 14:50:58 +0300 Subject: [PATCH 413/487] 1.8.0-SNAPSHOT --- modules/platforms/cpp/configure.ac | 2 +- modules/platforms/cpp/configure.acrel | 2 +- modules/platforms/cpp/examples/configure.ac | 2 +- modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs | 2 +- modules/platforms/cpp/odbc/install/ignite-odbc-x86.wxs | 2 +- .../Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs | 4 ++-- .../dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs | 4 ++-- .../Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs | 4 ++-- .../Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs | 4 ++-- .../Properties/AssemblyInfo.cs | 4 ++-- .../Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs | 4 ++-- .../dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs | 4 ++-- .../dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs | 4 ++-- .../dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs | 4 ++-- .../dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs | 4 ++-- .../platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs | 4 ++-- .../Apache.Ignite.Examples/Properties/AssemblyInfo.cs | 4 ++-- .../Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs | 4 ++-- 18 files changed, 31 insertions(+), 31 deletions(-) diff --git a/modules/platforms/cpp/configure.ac b/modules/platforms/cpp/configure.ac index 9dc9f529a067d..500bdbff2cdc2 100644 --- a/modules/platforms/cpp/configure.ac +++ b/modules/platforms/cpp/configure.ac @@ -19,7 +19,7 @@ # Process this file with autoconf to produce a configure script. AC_PREREQ([2.69]) -AC_INIT([Apache Ignite C++], [1.8.0.14218], [dev@ignite.apache.org], [ignite], [ignite.apache.org]) +AC_INIT([Apache Ignite C++], [1.8.0.16695], [dev@ignite.apache.org], [ignite], [ignite.apache.org]) AC_CANONICAL_HOST AC_CONFIG_MACRO_DIR([m4]) diff --git a/modules/platforms/cpp/configure.acrel b/modules/platforms/cpp/configure.acrel index 036f124b84204..984aa38c89d1f 100644 --- a/modules/platforms/cpp/configure.acrel +++ b/modules/platforms/cpp/configure.acrel @@ -19,7 +19,7 @@ # Process this file with autoconf to produce a configure script. AC_PREREQ([2.69]) -AC_INIT([Apache Ignite C++], [1.6.0.8653], [dev@ignite.apache.org], [ignite], [ignite.apache.org]) +AC_INIT([Apache Ignite C++], [1.8.0.16695], [dev@ignite.apache.org], [ignite], [ignite.apache.org]) AC_CANONICAL_HOST AC_CONFIG_MACRO_DIR([m4]) diff --git a/modules/platforms/cpp/examples/configure.ac b/modules/platforms/cpp/examples/configure.ac index 82a3727fbf4fa..6f08490aa08ef 100644 --- a/modules/platforms/cpp/examples/configure.ac +++ b/modules/platforms/cpp/examples/configure.ac @@ -19,7 +19,7 @@ # Process this file with autoconf to produce a configure script. AC_PREREQ([2.69]) -AC_INIT([Apache Ignite C++ Examples], [1.8.0.14218], [dev@ignite.apache.org], [ignite-examples], [ignite.apache.org]) +AC_INIT([Apache Ignite C++ Examples], [1.8.0.16695], [dev@ignite.apache.org], [ignite-examples], [ignite.apache.org]) AC_CANONICAL_HOST AC_CONFIG_MACRO_DIR([m4]) diff --git a/modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs b/modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs index cb6ab5410b205..2c12d9a2eac08 100644 --- a/modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs +++ b/modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs @@ -21,7 +21,7 @@ + Language='1033' Codepage='1252' Version='1.8.0.16695'> + Language='1033' Codepage='1252' Version='1.8.0.16695'> Date: Thu, 29 Dec 2016 12:20:20 +0300 Subject: [PATCH 414/487] IGNITE-4167: Changed IGNITE_TO_STRING_INCLUDE_SENSITIVE default value to "true". --- .../util/tostring/GridToStringBuilder.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java index 333f95e64ae58..6807b3f8ad0ed 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java @@ -17,6 +17,13 @@ package org.apache.ignite.internal.util.tostring; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.SB; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + import java.io.Externalizable; import java.io.InputStream; import java.io.OutputStream; @@ -34,12 +41,8 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; -import org.apache.ignite.IgniteException; -import org.apache.ignite.IgniteSystemProperties; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.internal.SB; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_TO_STRING_INCLUDE_SENSITIVE; /** * Provides auto-generation framework for {@code toString()} output. @@ -87,7 +90,8 @@ public class GridToStringBuilder { public static final int MAX_COL_SIZE = 100; /** {@link IgniteSystemProperties#IGNITE_TO_STRING_INCLUDE_SENSITIVE} */ - public static final boolean INCLUDE_SENSITIVE = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_TO_STRING_INCLUDE_SENSITIVE, false); + public static final boolean INCLUDE_SENSITIVE = + IgniteSystemProperties.getBoolean(IGNITE_TO_STRING_INCLUDE_SENSITIVE, true); /** */ private static ThreadLocal> threadCache = new ThreadLocal>() { From 626f1d5614930bca0ab744a6049e5f83f321d46a Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 23 Dec 2016 12:18:07 +0300 Subject: [PATCH 415/487] IGNITE-4142: Fixed assertion while update metrics. This closes #1372. --- .../java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java | 2 +- .../java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 8928f28d4fd27..0f5f74181516e 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -1920,7 +1920,7 @@ private void processHeartbeatMessage(TcpDiscoveryHeartbeatMessage msg) { TcpDiscoveryHeartbeatMessage.MetricsSet metricsSet = e.getValue(); - Map cacheMetrics = msg.hasCacheMetrics() ? + Map cacheMetrics = msg.hasCacheMetrics(nodeId) ? msg.cacheMetrics().get(nodeId) : Collections.emptyMap(); updateMetrics(nodeId, metricsSet.metrics(), cacheMetrics, tstamp); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 204b685984be3..c79133379c267 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -4927,7 +4927,7 @@ private void processHeartbeatMessage(TcpDiscoveryHeartbeatMessage msg) { TcpDiscoveryHeartbeatMessage.MetricsSet metricsSet = e.getValue(); - Map cacheMetrics = msg.hasCacheMetrics() ? + Map cacheMetrics = msg.hasCacheMetrics(nodeId) ? msg.cacheMetrics().get(nodeId) : Collections.emptyMap(); updateMetrics(nodeId, metricsSet.metrics(), cacheMetrics, tstamp); From 55f7594fd0595a4269c3972446ba4ebe30c12442 Mon Sep 17 00:00:00 2001 From: Alexandr Kuramshin Date: Fri, 13 Jan 2017 12:26:39 +0300 Subject: [PATCH 416/487] ignite-4293 Do not need store deserialized value for BinaryMarshaller (cherry picked from commit d10946b) --- .../cacheobject/IgniteCacheObjectProcessorImpl.java | 5 ++--- .../org/apache/ignite/cache/store/jdbc/model/Person.java | 2 +- .../processors/cache/CacheEntryProcessorCopySelfTest.java | 6 ++++-- .../processors/cache/GridCacheBasicStoreAbstractTest.java | 2 +- .../dht/GridCacheDhtEvictionsDisabledSelfTest.java | 5 +---- 5 files changed, 9 insertions(+), 11 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java index 208ec62448176..614c6129929db 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java @@ -244,9 +244,8 @@ protected final int partition(CacheObjectContext ctx, @Nullable GridCacheContext CacheMemoryMode memMode = ccfg.getMemoryMode(); - boolean storeVal = ctx.config().isPeerClassLoadingEnabled() || - GridQueryProcessor.isEnabled(ccfg) || - !ccfg.isCopyOnRead(); + boolean storeVal = !ccfg.isCopyOnRead() || (!isBinaryEnabled(ccfg) && + (GridQueryProcessor.isEnabled(ccfg) || ctx.config().isPeerClassLoadingEnabled())); CacheObjectContext res = new CacheObjectContext(ctx, ccfg.getName(), diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java index ddf309b1e6a0c..52ddfc8ecdcbb 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java @@ -196,7 +196,7 @@ public void setSalary(Integer salary) { @Override public String toString() { return "Person [id=" + id + ", orgId=" + orgId + - ", birthday=" + birthday.getTime() + + ", birthday=" + (birthday == null ? null : birthday.getTime()) + ", name=" + name + "]"; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java index 21395e61c75e6..f44889ba53dfd 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java @@ -30,6 +30,7 @@ import org.apache.ignite.cache.CacheEntryProcessor; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -93,8 +94,9 @@ private void doTestMutableEntry(boolean p2pEnabled) throws Exception { doTest(true, false, OLD_VAL, 1); // One deserialization due to copyOnRead == true. - // Additional deserialization in case p2p enabled due to storeValue == true on update entry. - doTest(true, true, NEW_VAL, p2pEnabled ? 2 : 1); + // Additional deserialization in case p2p enabled and not BinaryMarshaller due to storeValue == true on update entry. + doTest(true, true, NEW_VAL, p2pEnabled && + !(grid.configuration().getMarshaller() instanceof BinaryMarshaller) ? 2 : 1); // No deserialization. doTest(false, false, NEW_VAL, 0); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java index 8ddd737aaec33..026b6183bb208 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java @@ -571,7 +571,7 @@ public void testReload() throws Exception { assert cached != null; - assert cached == val : "Cached value mismatch [expected=" + val + ", cached=" + cached + ']'; + assert cached.equals(val) : "Cached value mismatch [expected=" + val + ", cached=" + cached + ']'; // Make sure that value is coming from cache, not from store. checkLastMethod(null); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java index 3f3f84fa69807..e8a6cfb87f79a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java @@ -117,10 +117,7 @@ private void checkNodes(Ignite g) throws Exception { assertNotNull(v1); assertNotNull(v2); - if (affinity(cache).mapKeyToNode(key).isLocal()) - assertSame(v1, v2); - else - assertEquals(v1, v2); + assertEquals(v1, v2); } } } \ No newline at end of file From 1665a615030201a7c9a51fd479868c3533b103b5 Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Fri, 30 Dec 2016 13:41:34 +0300 Subject: [PATCH 417/487] IGNITE-4424 REPLICATED cache isn't synced across nodes --- .../GridNearAtomicAbstractUpdateFuture.java | 34 ++- .../GridNearAtomicSingleUpdateFuture.java | 44 ++-- .../atomic/GridNearAtomicUpdateFuture.java | 57 ++--- .../AtomicPutAllChangingTopologyTest.java | 212 ++++++++++++++++++ .../IgniteCacheFailoverTestSuite.java | 3 + 5 files changed, 284 insertions(+), 66 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java index 2fbabaa54fc9c..c92e0f5ec5b42 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java @@ -212,14 +212,18 @@ public void map() { // Cannot remap. remapCnt = 1; - map(topVer); + GridCacheVersion futVer = addAtomicFuture(topVer); + + if (futVer != null) + map(topVer, futVer); } } /** * @param topVer Topology version. + * @param futVer Future version */ - protected abstract void map(AffinityTopologyVersion topVer); + protected abstract void map(AffinityTopologyVersion topVer, GridCacheVersion futVer); /** * Maps future on ready topology. @@ -302,7 +306,7 @@ protected void mapSingle(UUID nodeId, GridNearAtomicAbstractUpdateRequest req) { * @param req Request. * @param e Error. */ - protected void onSendError(GridNearAtomicAbstractUpdateRequest req, IgniteCheckedException e) { + protected final void onSendError(GridNearAtomicAbstractUpdateRequest req, IgniteCheckedException e) { synchronized (mux) { GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(), req.nodeId(), @@ -314,4 +318,28 @@ protected void onSendError(GridNearAtomicAbstractUpdateRequest req, IgniteChecke onResult(req.nodeId(), res, true); } } + + /** + * Adds future prevents topology change before operation complete. + * Should be invoked before topology lock released. + * + * @param topVer Topology version. + * @return Future version in case future added. + */ + protected final GridCacheVersion addAtomicFuture(AffinityTopologyVersion topVer) { + GridCacheVersion futVer = cctx.versions().next(topVer); + + synchronized (mux) { + assert this.futVer == null : this; + assert this.topVer == AffinityTopologyVersion.ZERO : this; + + this.topVer = topVer; + this.futVer = futVer; + } + + if (storeFuture() && !cctx.mvcc().addAtomicFuture(futVer, this)) + return null; + + return futVer; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java index bd231cf0003d8..7376affca4455 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java @@ -348,14 +348,7 @@ else if (res.error() != null) { @Override public void apply(final IgniteInternalFuture fut) { cctx.kernalContext().closure().runLocalSafe(new Runnable() { @Override public void run() { - try { - AffinityTopologyVersion topVer = fut.get(); - - map(topVer); - } - catch (IgniteCheckedException e) { - onDone(e); - } + mapOnTopology(); } }); } @@ -388,7 +381,9 @@ private void updateNear(GridNearAtomicAbstractUpdateRequest req, GridNearAtomicU @Override protected void mapOnTopology() { cache.topology().readLock(); - AffinityTopologyVersion topVer = null; + AffinityTopologyVersion topVer; + + GridCacheVersion futVer; try { if (cache.topology().stopping()) { @@ -410,6 +405,8 @@ private void updateNear(GridNearAtomicAbstractUpdateRequest req, GridNearAtomicU } topVer = fut.topologyVersion(); + + futVer = addAtomicFuture(topVer); } else { if (waitTopFut) { @@ -435,11 +432,12 @@ private void updateNear(GridNearAtomicAbstractUpdateRequest req, GridNearAtomicU cache.topology().readUnlock(); } - map(topVer); + if (futVer != null) + map(topVer, futVer); } /** {@inheritDoc} */ - protected void map(AffinityTopologyVersion topVer) { + @Override protected void map(AffinityTopologyVersion topVer, GridCacheVersion futVer) { Collection topNodes = CU.affinityNodes(cctx, topVer); if (F.isEmpty(topNodes)) { @@ -449,11 +447,6 @@ protected void map(AffinityTopologyVersion topVer) { return; } - Exception err = null; - GridNearAtomicAbstractUpdateRequest singleReq0 = null; - - GridCacheVersion futVer = cctx.versions().next(topVer); - GridCacheVersion updVer; // Assign version on near node in CLOCK ordering mode even if fastMap is false. @@ -470,16 +463,17 @@ protected void map(AffinityTopologyVersion topVer) { else updVer = null; + Exception err = null; + GridNearAtomicAbstractUpdateRequest singleReq0 = null; + try { singleReq0 = mapSingleUpdate(topVer, futVer, updVer); synchronized (mux) { - assert this.futVer == null : this; - assert this.topVer == AffinityTopologyVersion.ZERO : this; + assert this.futVer == futVer || (this.isDone() && this.error() != null); + assert this.topVer == topVer; - this.topVer = topVer; this.updVer = updVer; - this.futVer = futVer; resCnt = 0; @@ -496,14 +490,6 @@ protected void map(AffinityTopologyVersion topVer) { return; } - if (storeFuture()) { - if (!cctx.mvcc().addAtomicFuture(futVer, this)) { - assert isDone() : this; - - return; - } - } - // Optimize mapping for single key. mapSingle(singleReq0.nodeId(), singleReq0); } @@ -511,7 +497,7 @@ protected void map(AffinityTopologyVersion topVer) { /** * @return Future version. */ - GridCacheVersion onFutureDone() { + private GridCacheVersion onFutureDone() { GridCacheVersion ver0; GridFutureAdapter fut0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java index cd6411725748b..950e5bdd2b9b1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java @@ -456,14 +456,7 @@ else if (!nodeErr) @Override public void apply(final IgniteInternalFuture fut) { cctx.kernalContext().closure().runLocalSafe(new Runnable() { @Override public void run() { - try { - AffinityTopologyVersion topVer = fut.get(); - - map(topVer, remapKeys); - } - catch (IgniteCheckedException e) { - onDone(e); - } + mapOnTopology(); } }); } @@ -497,7 +490,9 @@ private void updateNear(GridNearAtomicFullUpdateRequest req, GridNearAtomicUpdat @Override protected void mapOnTopology() { cache.topology().readLock(); - AffinityTopologyVersion topVer = null; + AffinityTopologyVersion topVer; + + GridCacheVersion futVer; try { if (cache.topology().stopping()) { @@ -519,6 +514,8 @@ private void updateNear(GridNearAtomicFullUpdateRequest req, GridNearAtomicUpdat } topVer = fut.topologyVersion(); + + futVer = addAtomicFuture(topVer); } else { if (waitTopFut) { @@ -544,7 +541,8 @@ private void updateNear(GridNearAtomicFullUpdateRequest req, GridNearAtomicUpdat cache.topology().readUnlock(); } - map(topVer, null); + if (futVer != null) + map(topVer, futVer, remapKeys); } /** @@ -602,15 +600,18 @@ private void doUpdate(Map mappings) { } /** {@inheritDoc} */ - protected void map(AffinityTopologyVersion topVer) { - map(topVer, null); + @Override protected void map(AffinityTopologyVersion topVer, GridCacheVersion futVer) { + map(topVer, futVer, null); } /** * @param topVer Topology version. + * @param futVer Future ID. * @param remapKeys Keys to remap. */ - void map(AffinityTopologyVersion topVer, @Nullable Collection remapKeys) { + void map(AffinityTopologyVersion topVer, + GridCacheVersion futVer, + @Nullable Collection remapKeys) { Collection topNodes = CU.affinityNodes(cctx, topVer); if (F.isEmpty(topNodes)) { @@ -620,14 +621,6 @@ void map(AffinityTopologyVersion topVer, @Nullable Collection re return; } - Exception err = null; - GridNearAtomicFullUpdateRequest singleReq0 = null; - Map mappings0 = null; - - int size = keys.size(); - - GridCacheVersion futVer = cctx.versions().next(topVer); - GridCacheVersion updVer; // Assign version on near node in CLOCK ordering mode even if fastMap is false. @@ -644,6 +637,12 @@ void map(AffinityTopologyVersion topVer, @Nullable Collection re else updVer = null; + Exception err = null; + GridNearAtomicFullUpdateRequest singleReq0 = null; + Map mappings0 = null; + + int size = keys.size(); + try { if (size == 1 && !fastMap) { assert remapKeys == null || remapKeys.size() == 1; @@ -676,12 +675,10 @@ void map(AffinityTopologyVersion topVer, @Nullable Collection re } synchronized (mux) { - assert this.futVer == null : this; - assert this.topVer == AffinityTopologyVersion.ZERO : this; + assert this.futVer == futVer || (this.isDone() && this.error() != null); + assert this.topVer == topVer; - this.topVer = topVer; this.updVer = updVer; - this.futVer = futVer; resCnt = 0; @@ -701,14 +698,6 @@ void map(AffinityTopologyVersion topVer, @Nullable Collection re return; } - if (storeFuture()) { - if (!cctx.mvcc().addAtomicFuture(futVer, this)) { - assert isDone() : this; - - return; - } - } - // Optimize mapping for single key. if (singleReq0 != null) mapSingle(singleReq0.nodeId(), singleReq0); @@ -725,7 +714,7 @@ void map(AffinityTopologyVersion topVer, @Nullable Collection re /** * @return Future version. */ - GridCacheVersion onFutureDone() { + private GridCacheVersion onFutureDone() { GridCacheVersion ver0; GridFutureAdapter fut0; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java new file mode 100644 index 0000000000000..878cb17b5c3e2 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.dht.atomic; + +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.affinity.fair.FairAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.cache.CacheMode.REPLICATED; +import static org.apache.ignite.cache.CachePeekMode.BACKUP; +import static org.apache.ignite.cache.CachePeekMode.PRIMARY; +import static org.apache.ignite.cache.CacheRebalanceMode.SYNC; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; + +/** */ +public class AtomicPutAllChangingTopologyTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int NODES_CNT = 3; + + /** */ + public static final String CACHE_NAME = "test-cache"; + + /** */ + private static final int CACHE_SIZE = 20_000; + + /** */ + private static volatile CountDownLatch FILLED_LATCH; + + /** + * @return Cache configuration. + */ + private CacheConfiguration cacheConfig() { + return new CacheConfiguration() + .setAtomicityMode(ATOMIC) + .setCacheMode(REPLICATED) + .setAffinity(new FairAffinityFunction(false, 1)) + .setWriteSynchronizationMode(FULL_SYNC) + .setRebalanceMode(SYNC) + .setName(CACHE_NAME); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testPutAllOnChangingTopology() throws Exception { + List futs = new LinkedList<>(); + + for (int i = 1; i < NODES_CNT; i++) + futs.add(startNodeAsync(i)); + + futs.add(startSeedNodeAsync()); + + boolean failed = false; + + for (IgniteInternalFuture fut : futs) { + try { + fut.get(); + } + catch (Throwable th) { + log.error("Check failed.", th); + + failed = true; + } + } + + if (failed) + throw new RuntimeException("Test Failed."); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + FILLED_LATCH = new CountDownLatch(1); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + + /** + * @return Future. + * @throws IgniteCheckedException If failed. + */ + private IgniteInternalFuture startSeedNodeAsync() throws IgniteCheckedException { + return GridTestUtils.runAsync(new Callable() { + @Override public Boolean call() throws Exception { + Ignite node = startGrid(0); + + log.info("Creating cache."); + + IgniteCache cache = node.getOrCreateCache(cacheConfig()); + + log.info("Created cache."); + + Map data = new HashMap<>(CACHE_SIZE); + + for (int i = 0; i < CACHE_SIZE; i++) + data.put(i, i); + + log.info("Filling."); + + cache.putAll(data); + + log.info("Filled."); + + FILLED_LATCH.countDown(); + + checkCacheState(node, cache); + + return true; + } + }); + } + + /** + * @param nodeId Node index. + * @return Future. + * @throws IgniteCheckedException If failed. + */ + private IgniteInternalFuture startNodeAsync(final int nodeId) throws IgniteCheckedException { + return GridTestUtils.runAsync(new Callable() { + @Override public Boolean call() throws Exception { + Ignite node = startGrid(nodeId); + + log.info("Getting cache."); + + IgniteCache cache = node.getOrCreateCache(cacheConfig()); + + log.info("Got cache."); + + FILLED_LATCH.await(); + + log.info("Got Filled."); + + cache.put(1, nodeId); + + checkCacheState(node, cache); + + return true; + } + }); + } + + /** + * @param node Node. + * @param cache Cache. + * @throws Exception If failed. + */ + private void checkCacheState(Ignite node, IgniteCache cache) throws Exception { + int locSize = cache.localSize(PRIMARY, BACKUP); + int locSize2 = -1; + + if (locSize != CACHE_SIZE) { + U.sleep(5000); + + // Rechecking. + locSize2 = cache.localSize(PRIMARY, BACKUP); + } + + assertEquals("Wrong cache size on node [node=" + node.configuration().getGridName() + + ", expected= " + CACHE_SIZE + + ", actual=" + locSize + + ", actual2=" + locSize2 + "]", + locSize, CACHE_SIZE); + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java index c9e507d33e11c..5bc67292b33de 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java @@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheTxNodeFailureSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridNearCacheTxNodeFailureSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteAtomicLongChangingTopologySelfTest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.AtomicPutAllChangingTopologyTest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridCacheAtomicClientInvalidPartitionHandlingSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridCacheAtomicClientRemoveFailureTest; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridCacheAtomicInvalidPartitionHandlingSelfTest; @@ -95,6 +96,8 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(GridCacheTxNodeFailureSelfTest.class); suite.addTestSuite(GridNearCacheTxNodeFailureSelfTest.class); + suite.addTestSuite(AtomicPutAllChangingTopologyTest.class); + return suite; } } From 496fb173d55a8ea0702fcb70a4e807f61d8fc53e Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Mon, 16 Jan 2017 13:19:51 +0300 Subject: [PATCH 418/487] IGNITE-4513 Improve debug logging. --- .../processors/cache/GridCacheUtils.java | 3 + .../CacheContinuousQueryHandler.java | 81 ++++++++++++++++--- modules/core/src/test/config/log4j-test.xml | 6 ++ 3 files changed, 79 insertions(+), 11 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index d32f4c1867bcf..8ee77e36c3a77 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -122,6 +122,9 @@ public class GridCacheUtils { /** Marshaller system cache name. */ public static final String MARSH_CACHE_NAME = "ignite-marshaller-sys-cache"; + /** */ + public static final String CONTINUOUS_QRY_LOG_CATEGORY = "org.apache.ignite.continuous.query"; + /** */ public static final String CACHE_MSG_LOG_CATEGORY = "org.apache.ignite.cache.msg"; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 10784fc6b05fe..83edab444f8c0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -352,7 +352,7 @@ public void keepBinary(boolean keepBinary) { assert !skipPrimaryCheck || loc; - log = ctx.log(CacheContinuousQueryHandler.class); + log = ctx.log(CU.CONTINUOUS_QRY_LOG_CATEGORY); CacheContinuousQueryListener lsnr = new CacheContinuousQueryListener() { @Override public void onExecution() { @@ -384,7 +384,10 @@ public void keepBinary(boolean keepBinary) { final boolean recordIgniteEvt, GridDhtAtomicAbstractUpdateFuture fut) { if (ignoreExpired && evt.getEventType() == EventType.EXPIRED) - return ; + return; + + if (log.isDebugEnabled()) + log.debug("Entry updated on affinity node [evt=" + evt + ", primary=" + primary + ']'); final GridCacheContext cctx = cacheContext(ctx); @@ -407,6 +410,10 @@ public void keepBinary(boolean keepBinary) { else { final boolean notify = filter(evt, primary); + if (log.isDebugEnabled()) + log.debug("Filter invoked for event [evt=" + evt + ", primary=" + primary + + ", notify=" + notify + ']'); + if (primary || skipPrimaryCheck) { if (fut == null) onEntryUpdate(evt, notify, loc, recordIgniteEvt); @@ -474,7 +481,8 @@ public void keepBinary(boolean keepBinary) { backupQueue0.clear(); } catch (IgniteCheckedException e) { - U.error(ctx.log(getClass()), "Failed to send backup event notification to node: " + nodeId, e); + U.error(ctx.log(CU.CONTINUOUS_QRY_LOG_CATEGORY), + "Failed to send backup event notification to node: " + nodeId, e); } } @@ -653,7 +661,7 @@ private void notifyCallback0(UUID nodeId, final GridCacheContext cctx = cacheContext(ctx); if (cctx == null) { - IgniteLogger log = ctx.log(CacheContinuousQueryHandler.class); + IgniteLogger log = ctx.log(CU.CONTINUOUS_QRY_LOG_CATEGORY); if (log.isDebugEnabled()) log.debug("Failed to notify callback, cache is not found: " + cacheId); @@ -689,7 +697,7 @@ private void notifyCallback0(UUID nodeId, if (ignoreClsNotFound) assert internal; else - U.error(ctx.log(getClass()), "Failed to unmarshal entry.", ex); + U.error(ctx.log(CU.CONTINUOUS_QRY_LOG_CATEGORY), "Failed to unmarshal entry.", ex); } } @@ -799,8 +807,12 @@ private void onEntryUpdate(CacheContinuousQueryEvent evt, boolean notify, boolea CacheContinuousQueryEntry e = handleEntry(entry); - if (e != null) + if (e != null) { + if (log.isDebugEnabled()) + log.debug("Send the following event to listener: " + e); + ctx.continuous().addNotification(nodeId, routineId, entry, topic, sync, true); + } } } catch (ClusterTopologyCheckedException ex) { @@ -809,7 +821,7 @@ private void onEntryUpdate(CacheContinuousQueryEvent evt, boolean notify, boolea "[node=" + nodeId + ", err=" + ex + ']'); } catch (IgniteCheckedException ex) { - U.error(ctx.log(getClass()), "Failed to send event notification to node: " + nodeId, ex); + U.error(ctx.log(CU.CONTINUOUS_QRY_LOG_CATEGORY), "Failed to send event notification to node: " + nodeId, ex); } if (recordIgniteEvt && notify) { @@ -875,7 +887,7 @@ else if (initUpdCntrs != null) partCntr = initUpdCntrs.get(partId); } - rec = new PartitionRecovery(ctx.log(getClass()), initTopVer0, partCntr); + rec = new PartitionRecovery(ctx.log(CU.CONTINUOUS_QRY_LOG_CATEGORY), initTopVer0, partCntr); PartitionRecovery oldRec = rcvs.putIfAbsent(partId, rec); @@ -984,12 +996,27 @@ private static class PartitionRecovery { List> entries; synchronized (pendingEvts) { + if (log.isDebugEnabled()) { + log.debug("Handling event [lastFiredEvt=" + lastFiredEvt + + ", curTop=" + curTop + + ", entUpdCnt=" + entry.updateCounter() + + ", partId=" + entry.partition() + + ", pendingEvts=" + pendingEvts + ']'); + } + // Received first event. if (curTop == AffinityTopologyVersion.NONE) { lastFiredEvt = entry.updateCounter(); curTop = entry.topologyVersion(); + if (log.isDebugEnabled()) { + log.debug("First event [lastFiredEvt=" + lastFiredEvt + + ", curTop=" + curTop + + ", entUpdCnt=" + entry.updateCounter() + + ", partId=" + entry.partition() + ']'); + } + return !entry.isFiltered() ? F.> asList(new CacheContinuousQueryEvent(cache, cctx, entry)) : @@ -1014,6 +1041,13 @@ private static class PartitionRecovery { if (!entry.isFiltered()) entries.add(new CacheContinuousQueryEvent(cache, cctx, entry)); + if (log.isDebugEnabled()) + log.debug("Partition was lost [lastFiredEvt=" + lastFiredEvt + + ", curTop=" + curTop + + ", entUpdCnt=" + entry.updateCounter() + + ", partId=" + entry.partition() + + ", pendingEvts=" + pendingEvts + ']'); + return entries; } @@ -1039,8 +1073,16 @@ private static class PartitionRecovery { return Collections.emptyList(); } - if (pendingEvts.isEmpty()) + if (pendingEvts.isEmpty()) { + if (log.isDebugEnabled()) { + log.debug("Nothing sent to listener [lastFiredEvt=" + lastFiredEvt + + ", curTop=" + curTop + + ", entUpdCnt=" + entry.updateCounter() + + ", partId=" + entry.partition() + ']'); + } + return Collections.emptyList(); + } Iterator> iter = pendingEvts.entrySet().iterator(); @@ -1057,6 +1099,14 @@ private static class PartitionRecovery { iter.remove(); } + + if (log.isDebugEnabled()) { + log.debug("Pending events reached max of buffer size [lastFiredEvt=" + lastFiredEvt + + ", curTop=" + curTop + + ", entUpdCnt=" + entry.updateCounter() + + ", partId=" + entry.partition() + + ", pendingEvts=" + pendingEvts + ']'); + } } else { // Elements are consistently. @@ -1077,6 +1127,15 @@ private static class PartitionRecovery { } } + if (log.isDebugEnabled()) { + log.debug("Will send to listener the following events [entries=" + entries + + ", lastFiredEvt=" + lastFiredEvt + + ", curTop=" + curTop + + ", entUpdCnt=" + entry.updateCounter() + + ", partId=" + entry.partition() + + ", pendingEvts=" + pendingEvts + ']'); + } + return entries; } } @@ -1254,14 +1313,14 @@ private void sendBackupAcknowledge(final IgniteBiTuple, Set + + + --> + From c3eae9fecff5ad01390170c034dca39c216a097c Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Mon, 16 Jan 2017 10:33:16 +0700 Subject: [PATCH 419/487] IGNITE-4518 Fixed parallel load of cache. - Fixes #1426. Signed-off-by: Andrey Novikov (cherry picked from commit 79401b2) --- .../store/jdbc/dialect/BasicJdbcDialect.java | 31 +++++++----- .../store/jdbc/CacheJdbcPojoStoreTest.java | 48 ++++++++++++++++++- 2 files changed, 66 insertions(+), 13 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java index 3ab112af41516..139f3fca7504d 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java @@ -173,13 +173,15 @@ private static String where(Collection keyCols, int keyCnt) { if (appendLowerBound) { sb.a("("); - for (int cnt = keyCols.size(); cnt > 0; cnt--) { - for (int j = 0; j < cnt; j++) - if (j == cnt - 1) - sb.a(cols[j]).a(" > ? "); + for (int keyCnt = keyCols.size(); keyCnt > 0; keyCnt--) { + for (int idx = 0; idx < keyCnt; idx++) { + if (idx == keyCnt - 1) + sb.a(cols[idx]).a(" > ? "); else - sb.a(cols[j]).a(" = ? AND "); - if (cnt != 1) + sb.a(cols[idx]).a(" = ? AND "); + } + + if (keyCnt != 1) sb.a("OR "); } @@ -192,13 +194,18 @@ private static String where(Collection keyCols, int keyCnt) { if (appendUpperBound) { sb.a("("); - for (int cnt = keyCols.size(); cnt > 0; cnt--) { - for (int j = 0; j < cnt; j++) - if (j == cnt - 1) - sb.a(cols[j]).a(" <= ? "); + for (int keyCnt = keyCols.size(); keyCnt > 0; keyCnt--) { + for (int idx = 0, lastIdx = keyCnt - 1; idx < keyCnt; idx++) { + sb.a(cols[idx]); + + // For composite key when not all of the key columns are constrained should use < (strictly less). + if (idx == lastIdx) + sb.a(keyCnt == keyCols.size() ? " <= ? " : " < ? "); else - sb.a(cols[j]).a(" = ? AND "); - if (cnt != 1) + sb.a(" = ? AND "); + } + + if (keyCnt != 1) sb.a(" OR "); } diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java index d8f75d3dcb6ee..4a0b1daf646d1 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java @@ -216,7 +216,7 @@ public CacheJdbcPojoStoreTest() throws Exception { stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " + "Person_Complex (id integer not null, org_id integer not null, city_id integer not null, " + - "name varchar(50), salary integer, PRIMARY KEY(id))"); + "name varchar(50), salary integer, PRIMARY KEY(id, org_id, city_id))"); conn.commit(); @@ -349,6 +349,52 @@ else if (k instanceof PersonComplexKey && v instanceof Person) { assertTrue(prnComplexKeys.isEmpty()); } + /** + * @throws Exception If failed. + */ + public void testParallelLoad() throws Exception { + Connection conn = store.openConnection(false); + + PreparedStatement prnComplexStmt = conn.prepareStatement("INSERT INTO Person_Complex(id, org_id, city_id, name, salary) VALUES (?, ?, ?, ?, ?)"); + + for (int i = 0; i < 8; i++) { + + prnComplexStmt.setInt(1, (i >> 2) & 1); + prnComplexStmt.setInt(2, (i >> 1) & 1); + prnComplexStmt.setInt(3, i % 2); + + prnComplexStmt.setString(4, "name"); + prnComplexStmt.setInt(5, 1000 + i * 500); + + prnComplexStmt.addBatch(); + } + + prnComplexStmt.executeBatch(); + + U.closeQuiet(prnComplexStmt); + + conn.commit(); + + U.closeQuiet(conn); + + final Collection prnComplexKeys = new ConcurrentLinkedQueue<>(); + + IgniteBiInClosure c = new CI2() { + @Override public void apply(Object k, Object v) { + if (k instanceof PersonComplexKey && v instanceof Person) + prnComplexKeys.add((PersonComplexKey)k); + else + fail("Unexpected entry [key=" + k + ", value=" + v + "]"); + } + }; + + store.setParallelLoadCacheMinimumThreshold(2); + + store.loadCache(c); + + assertEquals(8, prnComplexKeys.size()); + } + /** * @throws Exception If failed. */ From 71176473f9fd0aa2088ba4e611ba4b7fc45e76b8 Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Mon, 16 Jan 2017 11:22:34 +0700 Subject: [PATCH 420/487] IGNITE-4545 Added cache for router hostnames. - Fixes #1428. Signed-off-by: Andrey Novikov (cherry picked from commit 27ba69b) --- .../GridClientConnectionManagerAdapter.java | 7 ++- .../impl/connection/GridClientTopology.java | 53 +++++++++++++++---- 2 files changed, 49 insertions(+), 11 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java index 6ea7c227bddde..0928c9061cf43 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java @@ -85,6 +85,9 @@ public abstract class GridClientConnectionManagerAdapter implements GridClientCo /** Class logger. */ private final Logger log; + /** All local enabled MACs. */ + private final Collection macs; + /** NIO server. */ private GridNioServer srv; @@ -166,6 +169,8 @@ protected GridClientConnectionManagerAdapter(UUID clientId, if (marshId == null && cfg.getMarshaller() == null) throw new GridClientException("Failed to start client (marshaller is not configured)."); + macs = U.allLocalMACs(); + if (cfg.getProtocol() == GridClientProtocol.TCP) { try { IgniteLogger gridLog = new JavaLogger(false); @@ -315,7 +320,7 @@ protected GridClientConnectionManagerAdapter(UUID clientId, } boolean sameHost = node.attributes().isEmpty() || - F.containsAny(U.allLocalMACs(), node.attribute(ATTR_MACS).toString().split(", ")); + F.containsAny(macs, node.attribute(ATTR_MACS).toString().split(", ")); Collection srvs = new LinkedHashSet<>(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientTopology.java index effd5b3943148..97aa5861308db 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientTopology.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientTopology.java @@ -21,7 +21,6 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.LinkedList; import java.util.Map; import java.util.Set; @@ -61,12 +60,18 @@ public class GridClientTopology { /** Cached last error prevented topology from update. */ private GridClientException lastError; + /** Router addresses from configuration. */ + private final String routers; + /** * Set of router addresses to infer direct connectivity * when client is working in router connection mode. * {@code null} when client is working in direct connection node. */ - private final Set routerAddrs; + private final Set routerAddrs; + + /** List of all known local MACs */ + private final Collection macsCache; /** Protocol. */ private final GridClientProtocol prot; @@ -96,8 +101,38 @@ public GridClientTopology(GridClientConfiguration cfg) { metricsCache = cfg.isEnableMetricsCache(); attrCache = cfg.isEnableAttributesCache(); prot = cfg.getProtocol(); - routerAddrs = (!cfg.getRouters().isEmpty() && cfg.getServers().isEmpty()) ? - new HashSet<>(cfg.getRouters()) : null; + + if (!cfg.getRouters().isEmpty() && cfg.getServers().isEmpty()) { + routers = cfg.getRouters().toString(); + + routerAddrs = U.newHashSet(cfg.getRouters().size()); + + for (String router : cfg.getRouters()) { + int portIdx = router.lastIndexOf(":"); + + if (portIdx > 0) { + String hostName = router.substring(0, portIdx); + + try { + int port = Integer.parseInt(router.substring(portIdx + 1)); + + InetSocketAddress inetSockAddr = new InetSocketAddress(hostName, port); + + routerAddrs.add(inetSockAddr); + } + catch (Exception ignore) { + // No-op. + } + } + } + } + else { + routers = null; + + routerAddrs = Collections.emptySet(); + } + + macsCache = U.allLocalMACs(); } /** @@ -279,7 +314,7 @@ public GridClientNode node(UUID id) throws GridClientException { try { if (lastError != null) throw new GridClientDisconnectedException( - "Topology is failed [protocol=" + prot + ", routers=" + routerAddrs + ']', lastError); + "Topology is failed [protocol=" + prot + ", routers=" + routers + ']', lastError); else return nodes.get(id); } @@ -376,19 +411,17 @@ private GridClientNodeImpl prepareNode(final GridClientNodeImpl node) { (metricsCache && attrCache) || (node.attributes().isEmpty() && node.metrics() == null); // Try to bypass object copying. - if (noAttrsAndMetrics && routerAddrs == null && node.connectable()) + if (noAttrsAndMetrics && routerAddrs.isEmpty() && node.connectable()) return node; // Return a new node instance based on the original one. GridClientNodeImpl.Builder nodeBuilder = GridClientNodeImpl.builder(node, !attrCache, !metricsCache); for (InetSocketAddress addr : node.availableAddresses(prot, true)) { - boolean router = routerAddrs == null || - routerAddrs.contains(addr.getHostName() + ":" + addr.getPort()) || - routerAddrs.contains(addr.getAddress().getHostAddress() + ":" + addr.getPort()); + boolean router = routerAddrs.isEmpty() || routerAddrs.contains(addr); boolean reachable = noAttrsAndMetrics || !addr.getAddress().isLoopbackAddress() || - F.containsAny(U.allLocalMACs(), node.attribute(ATTR_MACS).toString().split(", ")); + F.containsAny(macsCache, node.attribute(ATTR_MACS).split(", ")); if (router && reachable) { nodeBuilder.connectable(true); From 22b7e76c00a77a06388bcef869f29d1a572a306a Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 17 Jan 2017 12:33:32 +0300 Subject: [PATCH 421/487] ignite-4465 Prevent cache entry eviction while it is loaded from store (otherwise loaded value can be not stored in cache). --- .../processors/cache/EntryGetResult.java | 65 +++++++ .../processors/cache/GridCacheAdapter.java | 104 ++++++---- .../processors/cache/GridCacheEntryEx.java | 30 ++- .../processors/cache/GridCacheMapEntry.java | 100 ++++++++-- .../dht/GridPartitionedGetFuture.java | 7 +- .../dht/GridPartitionedSingleGetFuture.java | 7 +- .../dht/atomic/GridDhtAtomicCache.java | 7 +- .../dht/colocated/GridDhtColocatedCache.java | 7 +- .../distributed/near/GridNearGetFuture.java | 13 +- .../local/atomic/GridLocalAtomicCache.java | 7 +- .../cache/transactions/IgniteTxHandler.java | 2 +- .../transactions/IgniteTxLocalAdapter.java | 29 +-- .../cache/CacheConcurrentReadThroughTest.java | 184 ++++++++++++++++++ .../CrossCacheTxRandomOperationsTest.java | 28 ++- .../cache/GridCacheTestEntryEx.java | 21 +- .../testsuites/IgniteCacheTestSuite2.java | 2 + 16 files changed, 512 insertions(+), 101 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetResult.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConcurrentReadThroughTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetResult.java new file mode 100644 index 0000000000000..a34ddae6d10af --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetResult.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; + +/** + * + */ +public class EntryGetResult { + /** */ + private final CacheObject val; + + /** */ + private final GridCacheVersion ver; + + /** */ + private final boolean reserved; + + /** + * @param val Value. + * @param ver Version. + */ + EntryGetResult(CacheObject val, GridCacheVersion ver, boolean reserved) { + this.val = val; + this.ver = ver; + this.reserved = reserved; + } + + /** + * @return Value. + */ + public CacheObject value() { + return val; + } + + /** + * @return Version. + */ + public GridCacheVersion version() { + return ver; + } + + /** + * @return Reserved flag, + */ + public boolean reserved() { + return reserved; + } +} 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 965c6d12916aa..fd9f396926f0e 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 @@ -1860,7 +1860,7 @@ public final IgniteInternalFuture> getAllAsync(@Nullable final Collect * @param needVer If {@code true} returns values as tuples containing value and version. * @return Future. */ - public final IgniteInternalFuture> getAllAsync0( + protected final IgniteInternalFuture> getAllAsync0( @Nullable final Collection keys, final boolean readThrough, boolean checkTx, @@ -1906,7 +1906,7 @@ public final IgniteInternalFuture> getAllAsync0( final boolean needEntry = storeEnabled || ctx.isSwapOrOffheapEnabled(); - Map misses = null; + Map misses = null; for (KeyCacheObject key : keys) { while (true) { @@ -1920,40 +1920,58 @@ public final IgniteInternalFuture> getAllAsync0( } try { - T2 res = entry.innerGetVersioned( - null, - null, - ctx.isSwapOrOffheapEnabled(), - /*unmarshal*/true, - /*update-metrics*/!skipVals, - /*event*/!skipVals, - subjId, - null, - taskName, - expiry, - !deserializeBinary); - - if (res == null) { - if (storeEnabled) { - GridCacheVersion ver = entry.version(); + EntryGetResult res; + boolean evt = !skipVals; + boolean updateMetrics = !skipVals; + + if (storeEnabled) { + res = entry.innerGetAndReserveForLoad(ctx.isSwapOrOffheapEnabled(), + updateMetrics, + evt, + subjId, + taskName, + expiry, + !deserializeBinary); + + assert res != null; + + if (res.value() == null) { if (misses == null) misses = new HashMap<>(); - misses.put(key, ver); + misses.put(key, res); + + res = null; } - else - ctx.evicts().touch(entry, topVer); } else { + res = entry.innerGetVersioned( + null, + null, + ctx.isSwapOrOffheapEnabled(), + /*unmarshal*/true, + updateMetrics, + evt, + subjId, + null, + taskName, + expiry, + !deserializeBinary); + + if (res == null) + ctx.evicts().touch(entry, topVer); + } + + if (res != null) { ctx.addResult(map, key, - res.get1(), + res.value(), skipVals, keepCacheObjects, deserializeBinary, true, - needVer ? res.get2() : null); + needVer ? res.version() : null); if (tx == null || (!tx.implicit() && tx.isolation() == READ_COMMITTED)) ctx.evicts().touch(entry, topVer); @@ -1973,7 +1991,7 @@ public final IgniteInternalFuture> getAllAsync0( } if (storeEnabled && misses != null) { - final Map loadKeys = misses; + final Map loadKeys = misses; final IgniteTxLocalAdapter tx0 = tx; @@ -1984,15 +2002,10 @@ public final IgniteInternalFuture> getAllAsync0( @Override public Map call() throws Exception { ctx.store().loadAll(null/*tx*/, loadKeys.keySet(), new CI2() { @Override public void apply(KeyCacheObject key, Object val) { - GridCacheVersion ver = loadKeys.get(key); - - if (ver == null) { - if (log.isDebugEnabled()) - log.debug("Value from storage was never asked for [key=" + key + - ", val=" + val + ']'); + EntryGetResult res = loadKeys.get(key); + if (res == null || val == null) return; - } loaded.add(key); @@ -2002,14 +2015,16 @@ public final IgniteInternalFuture> getAllAsync0( GridCacheEntryEx entry = entryEx(key); try { - GridCacheVersion verSet = entry.versionedValue(cacheVal, ver, null); + GridCacheVersion verSet = entry.versionedValue(cacheVal, + res.version(), + null); boolean set = verSet != null; if (log.isDebugEnabled()) log.debug("Set value loaded from store into entry [" + "set=" + set + - ", curVer=" + ver + + ", curVer=" + res.version() + ", newVer=" + verSet + ", " + "entry=" + entry + ']'); @@ -2022,7 +2037,7 @@ public final IgniteInternalFuture> getAllAsync0( keepCacheObjects, deserializeBinary, false, - needVer ? set ? verSet : ver : null); + needVer ? set ? verSet : res.version() : null); } if (tx0 == null || (!tx0.implicit() && @@ -2045,16 +2060,23 @@ public final IgniteInternalFuture> getAllAsync0( }); if (loaded.size() != loadKeys.size()) { - for (KeyCacheObject key : loadKeys.keySet()) { - if (loaded.contains(key)) + boolean needTouch = + tx0 == null || (!tx0.implicit() && tx0.isolation() == READ_COMMITTED); + + for (Map.Entry e : loadKeys.entrySet()) { + if (loaded.contains(e.getKey())) continue; - if (tx0 == null || (!tx0.implicit() && - tx0.isolation() == READ_COMMITTED)) { - GridCacheEntryEx entry = peekEx(key); + if (needTouch || e.getValue().reserved()) { + GridCacheEntryEx entry = peekEx(e.getKey()); - if (entry != null) - ctx.evicts().touch(entry, topVer); + if (entry != null) { + if (e.getValue().reserved()) + entry.clearReserveForLoad(e.getValue().version()); + + if (needTouch) + ctx.evicts().touch(entry, topVer); + } } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java index d8194fcb30de7..b1d632fd5943d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java @@ -34,7 +34,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx; import org.apache.ignite.internal.processors.dr.GridDrType; import org.apache.ignite.internal.util.lang.GridTuple3; -import org.apache.ignite.internal.util.typedef.T2; import org.jetbrains.annotations.Nullable; /** @@ -323,7 +322,7 @@ public boolean evictInternal(boolean swap, GridCacheVersion obsoleteVer, * @throws IgniteCheckedException If loading value failed. * @throws GridCacheEntryRemovedException If entry was removed. */ - @Nullable public T2 innerGetVersioned( + @Nullable public EntryGetResult innerGetVersioned( @Nullable GridCacheVersion ver, IgniteInternalTx tx, boolean readSwap, @@ -337,6 +336,33 @@ public boolean evictInternal(boolean swap, GridCacheVersion obsoleteVer, boolean keepBinary) throws IgniteCheckedException, GridCacheEntryRemovedException; + /** + * @param readSwap Flag indicating whether to check swap memory. + * @param updateMetrics If {@code true} then metrics should be updated. + * @param evt Flag to signal event notification. + * @param subjId Subject ID initiated this read. + * @param taskName Task name. + * @param expiryPlc Expiry policy. + * @param keepBinary Keep binary flag. + * @return Cached value and entry version. + * @throws IgniteCheckedException If loading value failed. + * @throws GridCacheEntryRemovedException If entry was removed. + * @return Cached value, entry version and flag indicating if entry was reserved. + */ + public EntryGetResult innerGetAndReserveForLoad(boolean readSwap, + boolean updateMetrics, + boolean evt, + UUID subjId, + String taskName, + @Nullable IgniteCacheExpiryPolicy expiryPlc, + boolean keepBinary) throws IgniteCheckedException, GridCacheEntryRemovedException; + + /** + * @param ver Expected entry version. + * @throws IgniteCheckedException If failed. + */ + public void clearReserveForLoad(GridCacheVersion ver) throws IgniteCheckedException; + /** * Reloads entry from underlying storage. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 31baedacc7956..9f0c2b0a47c56 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -105,6 +105,9 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme /** */ private static final byte IS_SWAPPING_REQUIRED = 0x08; + /** */ + private static final byte IS_EVICT_DISABLED = 0x10; + /** */ public static final GridCacheAtomicVersionComparator ATOMIC_VER_COMPARATOR = new GridCacheAtomicVersionComparator(); @@ -774,11 +777,37 @@ protected final void releaseSwap() throws IgniteCheckedException { taskName, expirePlc, false, - keepBinary); + keepBinary, + false); } /** {@inheritDoc} */ - @Nullable @Override public T2 innerGetVersioned( + @Override public EntryGetResult innerGetAndReserveForLoad(boolean readSwap, + boolean updateMetrics, + boolean evt, + UUID subjId, + String taskName, + @Nullable IgniteCacheExpiryPolicy expiryPlc, + boolean keepBinary) throws IgniteCheckedException, GridCacheEntryRemovedException { + return (EntryGetResult)innerGet0( + /*ver*/null, + /*tx*/null, + readSwap, + /*readThrough*/false, + evt, + updateMetrics, + /*tmp*/false, + subjId, + /*transformClo*/null, + taskName, + expiryPlc, + true, + keepBinary, + /*reserve*/true); + } + + /** {@inheritDoc} */ + @Nullable @Override public EntryGetResult innerGetVersioned( @Nullable GridCacheVersion ver, IgniteInternalTx tx, boolean readSwap, @@ -791,7 +820,7 @@ protected final void releaseSwap() throws IgniteCheckedException { @Nullable IgniteCacheExpiryPolicy expiryPlc, boolean keepBinary) throws IgniteCheckedException, GridCacheEntryRemovedException { - return (T2)innerGet0(ver, + return (EntryGetResult)innerGet0(ver, tx, readSwap, false, @@ -803,7 +832,8 @@ protected final void releaseSwap() throws IgniteCheckedException { taskName, expiryPlc, true, - keepBinary); + keepBinary, + false); } /** {@inheritDoc} */ @@ -821,16 +851,16 @@ private Object innerGet0( String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, boolean retVer, - boolean keepBinary + boolean keepBinary, + boolean reserveForLoad ) throws IgniteCheckedException, GridCacheEntryRemovedException { assert !(retVer && readThrough); + assert !(reserveForLoad && readThrough); // Disable read-through if there is no store. if (readThrough && !cctx.readThrough()) readThrough = false; - CacheObject ret; - GridCacheVersion startVer; GridCacheVersion resVer = null; @@ -838,6 +868,8 @@ private Object innerGet0( boolean deferred = false; GridCacheVersion ver0 = null; + Object res = null; + synchronized (this) { checkObsolete(); @@ -881,7 +913,7 @@ private Object innerGet0( else val = null; - ret = val; + CacheObject ret = val; if (ret == null) { if (updateMetrics && cctx.cache().configuration().isStatisticsEnabled()) @@ -928,15 +960,26 @@ private Object innerGet0( // Cache version for optimistic check. startVer = ver; - } - if (ret != null) { - assert tmp || !(ret instanceof BinaryObjectOffheapImpl); - assert !obsolete; - assert !deferred; + if (ret != null) { + assert tmp || !(ret instanceof BinaryObjectOffheapImpl); + assert !obsolete; + assert !deferred; + + // If return value is consistent, then done. + res = retVer ? new EntryGetResult(ret, resVer, false) : ret; + } + else if (reserveForLoad && !obsolete) { + assert !readThrough; + assert retVer; + + boolean reserve = !evictionDisabled(); - // If return value is consistent, then done. - return retVer ? new T2<>(ret, resVer) : ret; + if (reserve) + flags |= IS_EVICT_DISABLED; + + res = new EntryGetResult(null, resVer, reserve); + } } if (obsolete) { @@ -948,6 +991,11 @@ private Object innerGet0( if (deferred) cctx.onDeferredDelete(this, ver0); + if (res != null) + return res; + + CacheObject ret = null; + if (readThrough) { IgniteInternalTx tx0 = null; @@ -2926,7 +2974,7 @@ protected void clearReader(UUID nodeId) throws GridCacheEntryRemovedException { * @return {@code True} if this entry should not be evicted from cache. */ protected boolean evictionDisabled() { - return false; + return (flags & IS_EVICT_DISABLED) != 0; } /** @@ -3008,6 +3056,7 @@ protected final boolean markObsolete0(GridCacheVersion ver, boolean clear, GridC value(null); ver = newVer; + flags &= ~IS_EVICT_DISABLED; if (log.isTraceEnabled()) { log.trace("invalidate releaseSwap [key=" + key + @@ -3096,6 +3145,7 @@ protected final void update(@Nullable CacheObject val, long expireTime, long ttl ttlAndExpireTimeExtras(ttl, expireTime); this.ver = ver; + flags &= ~IS_EVICT_DISABLED; if (addTracked && expireTime != 0 && (expireTime != oldExpireTime || isStartVersion()) && cctx.config().isEagerTtl()) cctx.ttl().addTrackedEntry(this); @@ -3548,12 +3598,24 @@ private long nextPartCounter(AffinityTopologyVersion topVer) { keepBinary); } + /** {@inheritDoc} */ + @Override public synchronized void clearReserveForLoad(GridCacheVersion ver) throws IgniteCheckedException { + if (obsoleteVersionExtras() != null) + return; + + if (ver.equals(this.ver)) { + assert evictionDisabled() : this; + + flags &= ~IS_EVICT_DISABLED; + } + } + /** {@inheritDoc} */ @Override public synchronized GridCacheVersion versionedValue(CacheObject val, GridCacheVersion curVer, GridCacheVersion newVer) - throws IgniteCheckedException, GridCacheEntryRemovedException { - + throws IgniteCheckedException, GridCacheEntryRemovedException + { checkObsolete(); if (curVer == null || curVer.equals(ver)) { @@ -3587,6 +3649,8 @@ private long nextPartCounter(AffinityTopologyVersion topVer) { return newVer; } + + assert !evictionDisabled() : this; } return null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java index 2e22d9eda13a0..d0f209df94ca0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java @@ -33,6 +33,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.EntryGetResult; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; @@ -449,7 +450,7 @@ private boolean localGet(KeyCacheObject key, int part, Map locVals) { GridCacheVersion ver = null; if (needVer) { - T2 res = entry.innerGetVersioned( + EntryGetResult res = entry.innerGetVersioned( null, null, /*swap*/true, @@ -463,8 +464,8 @@ private boolean localGet(KeyCacheObject key, int part, Map locVals) { !deserializeBinary); if (res != null) { - v = res.get1(); - ver = res.get2(); + v = res.value(); + ver = res.version(); } } else { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index aeb7eba56d1f5..e188a3523a3a5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -31,6 +31,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.EntryGetResult; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; @@ -373,7 +374,7 @@ private boolean localGet(AffinityTopologyVersion topVer, int part) { GridCacheVersion ver = null; if (needVer) { - T2 res = entry.innerGetVersioned( + EntryGetResult res = entry.innerGetVersioned( null, null, /*swap*/true, @@ -387,8 +388,8 @@ private boolean localGet(AffinityTopologyVersion topVer, int part) { true); if (res != null) { - v = res.get1(); - ver = res.get2(); + v = res.value(); + ver = res.version(); } } else { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java index 940c74ed863af..94a049ea86c6e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java @@ -48,6 +48,7 @@ import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.CacheOperationContext; import org.apache.ignite.internal.processors.cache.CacheStorePartialUpdateException; +import org.apache.ignite.internal.processors.cache.EntryGetResult; import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; @@ -1493,7 +1494,7 @@ private IgniteInternalFuture> getAllAsync0(@Nullable Collection res = entry.innerGetVersioned( + EntryGetResult res = entry.innerGetVersioned( null, null, /*swap*/true, @@ -1507,8 +1508,8 @@ private IgniteInternalFuture> getAllAsync0(@Nullable Collection> loadAsync( GridCacheVersion ver = null; if (needVer) { - T2 res = entry.innerGetVersioned( + EntryGetResult res = entry.innerGetVersioned( null, null, /*swap*/true, @@ -498,8 +499,8 @@ public final IgniteInternalFuture> loadAsync( !deserializeBinary); if (res != null) { - v = res.get1(); - ver = res.get2(); + v = res.value(); + ver = res.version(); } } else { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java index b7fcbbd21a5cc..ab0e88cbf375a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java @@ -33,6 +33,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.EntryGetResult; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; @@ -437,7 +438,7 @@ private Map map( // First we peek into near cache. if (isNear) { if (needVer) { - T2 res = entry.innerGetVersioned( + EntryGetResult res = entry.innerGetVersioned( null, null, /*swap*/true, @@ -451,8 +452,8 @@ private Map map( !deserializeBinary); if (res != null) { - v = res.get1(); - ver = res.get2(); + v = res.value(); + ver = res.version(); } } else { @@ -577,7 +578,7 @@ private boolean localDhtGet(KeyCacheObject key, boolean isNew = dhtEntry.isNewLocked() || !dhtEntry.valid(topVer); if (needVer) { - T2 res = dhtEntry.innerGetVersioned( + EntryGetResult res = dhtEntry.innerGetVersioned( null, null, /*swap*/true, @@ -591,8 +592,8 @@ private boolean localDhtGet(KeyCacheObject key, !deserializeBinary); if (res != null) { - v = res.get1(); - ver = res.get2(); + v = res.value(); + ver = res.version(); } } else { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java index a419887698326..d1acada062cb7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java @@ -45,6 +45,7 @@ import org.apache.ignite.internal.processors.cache.CacheOperationContext; import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException; import org.apache.ignite.internal.processors.cache.CacheStorePartialUpdateException; +import org.apache.ignite.internal.processors.cache.EntryGetResult; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; @@ -516,7 +517,7 @@ private Map getAllInternal(@Nullable Collection keys, GridCacheVersion ver; if (needVer) { - T2 res = entry.innerGetVersioned( + EntryGetResult res = entry.innerGetVersioned( null, null, /*swap*/swapOrOffheap, @@ -530,8 +531,8 @@ private Map getAllInternal(@Nullable Collection keys, !deserializeBinary); if (res != null) { - v = res.get1(); - ver = res.get2(); + v = res.value(); + ver = res.version(); ctx.addResult( vals, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index cf692640c2365..f784ba2942452 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -1141,7 +1141,7 @@ protected final void processDhtTxFinishRequest(final UUID nodeId, final GridDhtT else sendReply(nodeId, req, true, null); - assert req.txState() != null || (ctx.tm().tx(req.version()) == null && ctx.tm().nearTx(req.version()) == null); + assert req.txState() != null || (ctx.tm().tx(req.version()) == null && ctx.tm().nearTx(req.version()) == null) : req; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index e2f8438215005..91c9c92075eab 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.cache.CacheInvokeEntry; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.CacheOperationContext; +import org.apache.ignite.internal.processors.cache.EntryGetResult; import org.apache.ignite.internal.processors.cache.EntryProcessorResourceInjectorProxy; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; @@ -426,7 +427,7 @@ protected boolean commitAfterLock() { continue; try { - T2 res = entry.innerGetVersioned( + EntryGetResult res = entry.innerGetVersioned( null, this, /*readSwap*/true, @@ -446,7 +447,7 @@ protected boolean commitAfterLock() { misses.put(key, entry.version()); } else - c.apply(key, skipVals ? true : res.get1(), res.get2()); + c.apply(key, skipVals ? true : res.value(), res.version()); break; } @@ -1220,7 +1221,7 @@ private Collection enlistRead( F.first(txEntry.entryProcessors()) : null; if (needVer) { - T2 res = txEntry.cached().innerGetVersioned( + EntryGetResult res = txEntry.cached().innerGetVersioned( null, this, /*swap*/true, @@ -1234,8 +1235,8 @@ private Collection enlistRead( txEntry.keepBinary()); if (res != null) { - val = res.get1(); - readVer = res.get2(); + val = res.value(); + readVer = res.version(); } } else { @@ -1303,7 +1304,7 @@ private Collection enlistRead( optimistic() ? accessPolicy(cacheCtx, txKey, expiryPlc) : null; if (needReadVer) { - T2 res = primaryLocal(entry) ? + EntryGetResult res = primaryLocal(entry) ? entry.innerGetVersioned( null, this, @@ -1318,8 +1319,8 @@ private Collection enlistRead( !deserializeBinary) : null; if (res != null) { - val = res.get1(); - readVer = res.get2(); + val = res.value(); + readVer = res.version(); } } else { @@ -1654,7 +1655,7 @@ private IgniteInternalFuture> checkMissed( F.first(txEntry.entryProcessors()) : null; if (needVer) { - T2 res = cached.innerGetVersioned( + EntryGetResult res = cached.innerGetVersioned( null, IgniteTxLocalAdapter.this, /*swap*/cacheCtx.isSwapOrOffheapEnabled(), @@ -1668,8 +1669,8 @@ private IgniteInternalFuture> checkMissed( txEntry.keepBinary()); if (res != null) { - val = res.get1(); - readVer = res.get2(); + val = res.value(); + readVer = res.version(); } } else{ @@ -2377,7 +2378,7 @@ private boolean enlistWriteEntry(GridCacheContext cacheCtx, if (optimistic() && !implicit()) { try { if (needReadVer) { - T2 res = primaryLocal(entry) ? + EntryGetResult res = primaryLocal(entry) ? entry.innerGetVersioned( null, this, @@ -2392,8 +2393,8 @@ private boolean enlistWriteEntry(GridCacheContext cacheCtx, keepBinary) : null; if (res != null) { - old = res.get1(); - readVer = res.get2(); + old = res.value(); + readVer = res.version(); } } else { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConcurrentReadThroughTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConcurrentReadThroughTest.java new file mode 100644 index 0000000000000..87baa490902b5 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConcurrentReadThroughTest.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.Cache; +import javax.cache.configuration.Factory; +import javax.cache.integration.CacheLoaderException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCompute; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.store.CacheStoreAdapter; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Test was added to check fix for IGNITE-4465. + */ +public class CacheConcurrentReadThroughTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int SYS_THREADS = 16; + + /** */ + private boolean client; + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + + cfg.setClientMode(client); + + if (!client) { + cfg.setPublicThreadPoolSize(SYS_THREADS); + cfg.setSystemThreadPoolSize(SYS_THREADS); + } + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentReadThrough() throws Exception { + startGrid(0); + + client = true; + + Ignite client = startGrid(1); + + assertTrue(client.configuration().isClientMode()); + + IgniteCompute compute = client.compute().withAsync(); + + for (int iter = 0; iter < 10; iter++) { + CacheConfiguration ccfg = new CacheConfiguration(); + + final String cacheName = "test-" + iter; + + ccfg.setName(cacheName); + ccfg.setReadThrough(true); + ccfg.setCacheStoreFactory(new TestStoreFactory()); + ccfg.setStatisticsEnabled(true); + + client.createCache(ccfg); + + final Integer key = 1; + + TestCacheStore.loadCnt.set(0); + + Collection> futs = new ArrayList<>(); + + for (int i = 0; i < SYS_THREADS * 3; i++) { + compute.run(new IgniteRunnable() { + @IgniteInstanceResource + private transient Ignite ignite; + + @Override public void run() { + assertFalse(ignite.configuration().isClientMode()); + + Object v = ignite.cache(cacheName).get(key); + + if (v == null) + throw new IgniteException("Failed to get value"); + } + }); + + futs.add(compute.future()); + } + + for (IgniteFuture fut : futs) + fut.get(); + + int loadCnt = TestCacheStore.loadCnt.get(); + + long misses = ignite(1).cache(cacheName).metrics().getCacheMisses(); + + log.info("Iteration [iter=" + iter + ", loadCnt=" + loadCnt + ", misses=" + misses + ']'); + + assertTrue("Unexpected loadCnt: " + loadCnt, loadCnt > 0 && loadCnt <= SYS_THREADS); + assertTrue("Unexpected misses: " + misses, misses > 0 && misses <= SYS_THREADS); + + client.destroyCache(cacheName); + } + } + + /** + * + */ + private static class TestStoreFactory implements Factory { + /** {@inheritDoc} */ + @Override public TestCacheStore create() { + return new TestCacheStore(); + } + } + + /** + * + */ + private static class TestCacheStore extends CacheStoreAdapter { + /** */ + private static final AtomicInteger loadCnt = new AtomicInteger(); + + /** {@inheritDoc} */ + @Override public Integer load(Integer key) throws CacheLoaderException { + loadCnt.incrementAndGet(); + + try { + Thread.sleep(1000); + } + catch (InterruptedException e) { + e.printStackTrace(); + } + + return key; + } + + /** {@inheritDoc} */ + @Override public void write(Cache.Entry entry) { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void delete(Object key) { + // No-op. + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java index 67ec3712e92b4..e7df3c0c6a783 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java @@ -85,6 +85,11 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest { return cfg; } + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 6 * 60 * 1000; + } + /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); @@ -170,9 +175,17 @@ protected CacheConfiguration cacheConfiguration(String name, } /** + * @param cacheMode Cache mode. + * @param writeSync Write synchronization mode. + * @param fairAff Fair affinity flag. + * @param ignite Node to use. + * @param name Cache name. */ - protected void createCache(CacheMode cacheMode, CacheWriteSynchronizationMode writeSync, boolean fairAff, - Ignite ignite, String name) { + protected void createCache(CacheMode cacheMode, + CacheWriteSynchronizationMode writeSync, + boolean fairAff, + Ignite ignite, + String name) { ignite.createCache(cacheConfiguration(name, cacheMode, writeSync, fairAff)); } @@ -269,9 +282,18 @@ private void txOperations(TransactionConcurrency concurrency, boolean checkData = fullSync && !optimistic; + long stopTime = System.currentTimeMillis() + 10_000; + for (int i = 0; i < 10_000; i++) { - if (i % 100 == 0) + if (i % 100 == 0) { + if (System.currentTimeMillis() > stopTime) { + log.info("Stop on timeout, iteration: " + i); + + break; + } + log.info("Iteration: " + i); + } boolean rollback = i % 10 == 0; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java index 48621afc1a8b8..b03e9c89c5b3b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java @@ -418,7 +418,26 @@ void recheckLock() { } /** @inheritDoc */ - @Nullable @Override public T2 innerGetVersioned( + @Override public void clearReserveForLoad(GridCacheVersion ver) { + assert false; + } + + /** @inheritDoc */ + @Override public EntryGetResult innerGetAndReserveForLoad( + boolean readSwap, + boolean updateMetrics, + boolean evt, + UUID subjId, + String taskName, + @Nullable IgniteCacheExpiryPolicy expiryPlc, + boolean keepBinary) throws IgniteCheckedException, GridCacheEntryRemovedException { + assert false; + + return null; + } + + /** @inheritDoc */ + @Nullable @Override public EntryGetResult innerGetVersioned( @Nullable GridCacheVersion ver, IgniteInternalTx tx, boolean readSwap, diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java index f632f677ff0a0..8792ea16ac9af 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java @@ -22,6 +22,7 @@ import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionExcludeNeighborsSelfTest; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionBackupFilterSelfTest; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionExcludeNeighborsSelfTest; +import org.apache.ignite.internal.processors.cache.CacheConcurrentReadThroughTest; import org.apache.ignite.internal.processors.cache.CacheConfigurationLeakTest; import org.apache.ignite.internal.processors.cache.CacheDhtLocalPartitionAfterRemoveSelfTest; import org.apache.ignite.internal.processors.cache.CacheEnumOperationsSingleNodeTest; @@ -268,6 +269,7 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(CacheExchangeMessageDuplicatedStateTest.class)); suite.addTest(new TestSuite(OffheapCacheOnClientsTest.class)); + suite.addTest(new TestSuite(CacheConcurrentReadThroughTest.class)); return suite; } From 74d0dcc6c56118f9e4fdaa4aa70d25d1abe7b80e Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 17 Jan 2017 15:00:08 +0300 Subject: [PATCH 422/487] IGNITE-3964: SQL: add support for custom table name. This closes #1301. --- .../org/apache/ignite/cache/QueryEntity.java | 21 ++ .../processors/query/GridQueryProcessor.java | 26 +- .../query/GridQueryTypeDescriptor.java | 7 + .../processors/query/h2/IgniteH2Indexing.java | 16 +- .../IgniteCacheAbstractQuerySelfTest.java | 260 +++++++++++++++++- .../h2/GridIndexingSpiAbstractSelfTest.java | 5 + 6 files changed, 325 insertions(+), 10 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java index 9758cfc0600ef..3d0247873ede0 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java @@ -48,6 +48,9 @@ public class QueryEntity implements Serializable { /** Collection of query indexes. */ private Map idxs = new HashMap<>(); + /** Table name. */ + private String tableName; + /** * Creates an empty query entity. */ @@ -169,6 +172,24 @@ public void setIndexes(Collection idxs) { } } + + /** + * Gets table name for this query entity. + * + * @return table name + */ + public String getTableName() { + return tableName; + } + + /** + * Sets table name for this query entity. + * @param tableName table name + */ + public void setTableName(String tableName) { + this.tableName = tableName; + } + /** * Utility method for building query entities programmatically. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index 6c093ee7b541c..0f2bc9a74de8b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -253,10 +253,12 @@ private void initializeCache(GridCacheContext cctx) throws IgniteCheckedEx if (keyCls == null) keyCls = Object.class; - String simpleValType = valCls == null ? typeName(qryEntity.getValueType()) : typeName(valCls); + String simpleValType = ((valCls == null) ? typeName(qryEntity.getValueType()) : typeName(valCls)); desc.name(simpleValType); + desc.tableName(qryEntity.getTableName()); + if (binaryEnabled && !keyOrValMustDeserialize) { // Safe to check null. if (SQL_TYPES.contains(valCls)) @@ -466,7 +468,7 @@ private boolean mustDeserializeBinary(Class cls) { * @param desc Type descriptor. * @throws IgniteCheckedException If failed. */ - private void addTypeByName(CacheConfiguration ccfg, TypeDescriptor desc) throws IgniteCheckedException { + private void addTypeByName(CacheConfiguration ccfg, TypeDescriptor desc) throws IgniteCheckedException { if (typesByName.putIfAbsent(new TypeName(ccfg.getName(), desc.name()), desc) != null) throw new IgniteCheckedException("Type with name '" + desc.name() + "' already indexed " + "in cache '" + ccfg.getName() + "'."); @@ -2108,6 +2110,9 @@ private static class TypeDescriptor implements GridQueryTypeDescriptor { /** */ private String name; + /** */ + private String tblName; + /** Value field names and types with preserved order. */ @GridToStringInclude private final Map> fields = new LinkedHashMap<>(); @@ -2166,6 +2171,23 @@ void name(String name) { this.name = name; } + /** + * Gets table name for type. + * @return Table name. + */ + public String tableName() { + return tblName; + } + + /** + * Sets table name for type. + * + * @param tblName Table name. + */ + public void tableName(String tblName) { + this.tblName = tblName; + } + /** {@inheritDoc} */ @Override public Map> fields() { return fields; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java index b63684173c430..e93952510986e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java @@ -31,6 +31,13 @@ public interface GridQueryTypeDescriptor { */ public String name(); + /** + * Gets table name for type. + * + * @return Table name. + */ + public String tableName(); + /** * Gets mapping from field name to its type. * diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java index 362ddd853443c..bc5155221d878 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java @@ -741,7 +741,7 @@ private void removeTable(TableDescriptor tbl) throws IgniteCheckedException { tbl.onDrop(); - tbl.schema.tbls.remove(tbl.name()); + tbl.schema.tbls.remove(tbl.typeName()); } /** {@inheritDoc} */ @@ -2397,7 +2397,9 @@ private class TableDescriptor implements GridH2Table.IndexesFactory { this.type = type; this.schema = schema; - fullTblName = schema.schemaName + "." + escapeName(type.name(), schema.escapeAll()); + String tblName = escapeName(type.tableName() != null ? type.tableName() : type.name(), schema.escapeAll()); + + fullTblName = schema.schemaName + "." + tblName; } /** @@ -2408,16 +2410,16 @@ public String schemaName() { } /** - * @return Database table name. + * @return Database full table name. */ String fullTableName() { return fullTblName; } /** - * @return Database table name. + * @return type name. */ - String name() { + String typeName() { return type.name(); } @@ -2739,8 +2741,8 @@ private Schema(String spaceName, String schemaName, GridCacheContext cctx, * @param tbl Table descriptor. */ public void add(TableDescriptor tbl) { - if (tbls.putIfAbsent(tbl.name(), tbl) != null) - throw new IllegalStateException("Table already registered: " + tbl.name()); + if (tbls.putIfAbsent(tbl.typeName(), tbl) != null) + throw new IllegalStateException("Table already registered: " + tbl.fullTableName()); } /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java index 7c5b4722b2dfa..ad6922cea0902 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java @@ -23,6 +23,7 @@ import java.io.ObjectOutput; import java.io.Serializable; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -45,10 +46,13 @@ import org.apache.ignite.IgniteBinary; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CachePeekMode; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.QueryIndex; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.ScanQuery; @@ -66,11 +70,11 @@ import org.apache.ignite.events.CacheQueryExecutedEvent; import org.apache.ignite.events.CacheQueryReadEvent; import org.apache.ignite.events.Event; -import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQuerySelfTest; import org.apache.ignite.internal.processors.cache.query.QueryCursorEx; import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; +import org.apache.ignite.internal.util.lang.GridPlainCallable; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; @@ -180,6 +184,32 @@ protected NearCacheConfiguration nearCacheConfiguration() { Long.class, EnumObject.class ); + List entityList = new ArrayList<>(); + + QueryEntity qryEntity = new QueryEntity(); + + qryEntity.setKeyType(Integer.class.getName()); + qryEntity.setValueType(Type1.class.getName()); + qryEntity.addQueryField("id", Integer.class.getName(), null); + qryEntity.addQueryField("name", String.class.getName(), null); + qryEntity.setTableName("Type2"); + qryEntity.setIndexes(Arrays.asList(new QueryIndex("id"))); + + entityList.add(qryEntity); + + qryEntity = new QueryEntity(); + + qryEntity.setKeyType(Integer.class.getName()); + qryEntity.setValueType(Type2.class.getName()); + qryEntity.addQueryField("id", Integer.class.getName(), null); + qryEntity.addQueryField("name", String.class.getName(), null); + qryEntity.setTableName("Type1"); + qryEntity.setIndexes(Arrays.asList(new QueryIndex("id"))); + + entityList.add(qryEntity); + + cc.setQueryEntities(entityList); + if (cacheMode() != CacheMode.LOCAL) cc.setAffinity(new RendezvousAffinityFunction()); @@ -234,6 +264,7 @@ protected Ignite ignite() { stopAllGrids(); + store.reset(); } @@ -543,6 +574,113 @@ public void testSelectQuery() throws Exception { assert iter.next() != null; } + /** + * JUnit. + * + * @throws Exception In case of error. + */ + public void testSimpleCustomTableName() throws Exception { + final IgniteCache cache = ignite().cache(null); + + cache.put(10, new Type1(1, "Type1 record #1")); + cache.put(20, new Type1(2, "Type1 record #2")); + + QueryCursor> qry1 = + cache.query(new SqlQuery(Type1.class, "FROM Type2")); + + List> all = qry1.getAll(); + + assertEquals(2, all.size()); + + QueryCursor> qry = cache.query(new SqlFieldsQuery("SELECT name FROM Type2")); + + assertEquals(2, qry.getAll().size()); + + GridTestUtils.assertThrows(log, new GridPlainCallable() { + @Override public Void call() throws Exception { + QueryCursor> qry = + cache.query(new SqlQuery(Type1.class, "FROM Type1")); + + qry.getAll(); + + return null; + } + }, IgniteException.class, null); + } + + /** + * JUnit. + * + * @throws Exception In case of error. + */ + public void testMixedCustomTableName() throws Exception { + final IgniteCache cache = ignite().cache(null); + + cache.put(10, new Type1(1, "Type1 record #1")); + cache.put(20, new Type1(2, "Type1 record #2")); + cache.put(30, new Type2(1, "Type2 record #1")); + cache.put(40, new Type2(2, "Type2 record #2")); + cache.put(50, new Type2(3, "Type2 record #3")); + + QueryCursor> qry1 = + cache.query(new SqlQuery(Type1.class, "FROM Type2")); + + List> all = qry1.getAll(); + + assertEquals(2, all.size()); + + QueryCursor> qry2 = + cache.query(new SqlQuery(Type2.class, "FROM Type1")); + + assertEquals(3, qry2.getAll().size()); + + QueryCursor> qry = cache.query(new SqlFieldsQuery("SELECT name FROM Type1")); + + assertEquals(3, qry.getAll().size()); + + qry = cache.query(new SqlFieldsQuery("SELECT name FROM Type2")); + + assertEquals(2, qry.getAll().size()); + + GridTestUtils.assertThrows(log, new GridPlainCallable() { + @Override public Void call() throws Exception { + QueryCursor> qry1 = + cache.query(new SqlQuery(Type1.class, "FROM Type1")); + + qry1.getAll().size(); + + return null; + } + }, IgniteException.class, null); + } + + /** + * JUnit. + * + * @throws Exception In case of error. + */ + public void testDistributedJoinCustomTableName() throws Exception { + IgniteCache cache = ignite().cache(null); + + cache.put(10, new Type1(1, "Type1 record #1")); + cache.put(20, new Type1(2, "Type1 record #2")); + cache.put(30, new Type2(1, "Type2 record #1")); + cache.put(40, new Type2(2, "Type2 record #2")); + cache.put(50, new Type2(3, "Type2 record #3")); + + QueryCursor> query = cache.query( + new SqlFieldsQuery("SELECT t2.name, t1.name FROM Type2 as t2 LEFT JOIN Type1 as t1 ON t1.id = t2.id") + .setDistributedJoins(cacheMode() == PARTITIONED)); + + assertEquals(2, query.getAll().size()); + + query = cache.query( + new SqlFieldsQuery("SELECT t2.name, t1.name FROM Type2 as t2 RIGHT JOIN Type1 as t1 ON t1.id = t2.id") + .setDistributedJoins(cacheMode() == PARTITIONED)); + + assertEquals(3, query.getAll().size()); + } + /** * JUnit. * @@ -1653,6 +1791,126 @@ public int salary() { } } + /** + * + */ + public static class Type1 implements Serializable { + /** */ + private int id; + + /** */ + private String name; + + /** + * @param id ID. + * @param name Name. + */ + Type1(int id, String name) { + assert name != null; + assert id > 0; + + this.name = name; + this.id = id; + } + + /** + * @return Name. + */ + public String name() { + return name; + } + + /** + * @return ID. + */ + public int id() { + return id; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return name.hashCode() + 31 * id; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object obj) { + if (obj == this) + return true; + + if (!(obj instanceof Type1)) + return false; + + Type1 that = (Type1)obj; + + return that.name.equals(name) && that.id == id; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(Type1.class, this); + } + } + + /** + * + */ + public static class Type2 implements Serializable { + /** */ + private int id; + + /** */ + private String name; + + /** + * @param id ID. + * @param name Name. + */ + Type2(int id, String name) { + assert name != null; + assert id > 0; + + this.name = name; + this.id = id; + } + + /** + * @return Name. + */ + public String name() { + return name; + } + + /** + * @return ID. + */ + public int id() { + return id; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return name.hashCode() + 31 * id; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object obj) { + if (obj == this) + return true; + + if (!(obj instanceof Type2)) + return false; + + Type2 that = (Type2)obj; + + return that.name.equals(name) && that.id == id; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(Type2.class, this); + } + } + /** * Test value object. */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java index bcf8f9d9b6b59..81e34d6959f90 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java @@ -539,6 +539,11 @@ private TypeDesc(String space, String name, Map> valFields, Gri return name; } + /** {@inheritDoc} */ + @Override public String tableName() { + return null; + } + /** * @return Space name. */ From 8e622e41de4acf365da7f933a08b6d31bae11124 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 17 Jan 2017 15:18:33 +0300 Subject: [PATCH 423/487] IGNITE-4247: Sql queries supports table alias. This closes #1297. --- .../apache/ignite/cache/query/SqlQuery.java | 25 +++++++++++++ .../processors/query/GridQueryIndexing.java | 4 +- .../processors/query/GridQueryProcessor.java | 5 ++- .../processors/query/h2/IgniteH2Indexing.java | 14 ++++--- .../IgniteCacheAbstractQuerySelfTest.java | 37 +++++++++++++++++++ .../h2/GridIndexingSpiAbstractSelfTest.java | 24 ++++++------ 6 files changed, 89 insertions(+), 20 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java index 83e171d4ab43f..5e36d209dda75 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java @@ -37,6 +37,9 @@ public final class SqlQuery extends Query> { /** */ private String type; + /** Table alias */ + private String alias; + /** SQL clause. */ private String sql; @@ -137,6 +140,27 @@ public SqlQuery setType(String type) { return this; } + /** + * Sets table alias for type. + * + * @return Table alias. + */ + public String getAlias() { + return alias; + } + + /** + * Gets table alias for type. + * + * @param alias table alias for type that is used in query. + * @return {@code this} For chaining. + */ + public SqlQuery setAlias(String alias) { + this.alias = alias; + + return this; + } + /** * Gets the query execution timeout in milliseconds. * @@ -148,6 +172,7 @@ public int getTimeout() { /** * Sets the query execution timeout. Query will be automatically cancelled if the execution timeout is exceeded. + * * @param timeout Timeout value. Zero value disables timeout. * @param timeUnit Time granularity. * @return {@code this} For chaining. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java index 6bffa5d5d04b7..539ebc07fec79 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java @@ -95,6 +95,7 @@ public GridQueryFieldsResult queryLocalSqlFields(@Nullable String spaceName, Str * * @param spaceName Space name. * @param qry Query. + * @param alias Table alias used in Query. * @param params Query parameters. * @param type Query return type. * @param filter Space name and key filter. @@ -102,7 +103,8 @@ public GridQueryFieldsResult queryLocalSqlFields(@Nullable String spaceName, Str * @throws IgniteCheckedException If failed. */ public GridCloseableIterator> queryLocalSql(@Nullable String spaceName, String qry, - Collection params, GridQueryTypeDescriptor type, IndexingQueryFilter filter) throws IgniteCheckedException; + String alias, Collection params, GridQueryTypeDescriptor type, IndexingQueryFilter filter) + throws IgniteCheckedException; /** * Executes text query. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index 0f2bc9a74de8b..f4ac4ae71a546 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -796,7 +796,7 @@ public GridCloseableIterator> query(final String spac if (type == null || !type.registered()) throw new CacheException("Failed to find SQL table for type: " + resType); - return idx.queryLocalSql(space, clause, params, type, filters); + return idx.queryLocalSql(space, clause, null, params, type, filters); } }, false); } @@ -890,7 +890,8 @@ public Iterator> queryLocal( final GridCloseableIterator> i = idx.queryLocalSql( space, - sqlQry, + qry.getSql(), + qry.getAlias(), F.asList(params), typeDesc, idx.backupFilter(requestTopVer.get(), null)); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java index bc5155221d878..cbf2ebde27056 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java @@ -1046,14 +1046,14 @@ public void setupConnection(Connection conn, boolean distributedJoins, boolean e /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public GridCloseableIterator> queryLocalSql(@Nullable String spaceName, - final String qry, @Nullable final Collection params, GridQueryTypeDescriptor type, + final String qry, String alias, @Nullable final Collection params, GridQueryTypeDescriptor type, final IndexingQueryFilter filter) throws IgniteCheckedException { final TableDescriptor tbl = tableDescriptor(spaceName, type); if (tbl == null) throw new CacheException("Failed to find SQL table for type: " + type.name()); - String sql = generateQuery(qry, tbl); + String sql = generateQuery(qry, alias, tbl); Connection conn = connectionForThread(tbl.schemaName()); @@ -1103,7 +1103,7 @@ private Iterable> runQueryTwoStep(final GridCacheContext cctx, fina String sql; try { - sql = generateQuery(qry.getSql(), tblDesc); + sql = generateQuery(qry.getSql(), qry.getAlias(), tblDesc); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -1300,11 +1300,12 @@ public static Session session(Connection c) { * Prepares statement for query. * * @param qry Query string. + * @param tableAlias table alias. * @param tbl Table to use. * @return Prepared statement. * @throws IgniteCheckedException In case of error. */ - private String generateQuery(String qry, TableDescriptor tbl) throws IgniteCheckedException { + private String generateQuery(String qry, String tableAlias, TableDescriptor tbl) throws IgniteCheckedException { assert tbl != null; final String qry0 = qry; @@ -1341,10 +1342,13 @@ else if (star > 0) { } if (!upper.startsWith("FROM")) - from = " FROM " + t + + from = " FROM " + t + (tableAlias != null ? " as " + tableAlias : "") + (upper.startsWith("WHERE") || upper.startsWith("ORDER") || upper.startsWith("LIMIT") ? " " : " WHERE "); + if(tableAlias != null) + t = tableAlias; + qry = "SELECT " + t + "." + KEY_FIELD_NAME + ", " + t + "." + VAL_FIELD_NAME + from + qry; return qry; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java index ad6922cea0902..c5a241e54ce1f 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java @@ -347,6 +347,43 @@ public void testIntegerType() throws Exception { assertEquals(val, entry.getValue().intValue()); } + /** + * Test table alias in SqlQuery. + * + * @throws Exception In case of error. + */ + public void testTableAliasInSqlQuery() throws Exception { + IgniteCache cache = ignite().cache(null); + + int key = 898; + + int val = 2; + + cache.put(key, val); + + SqlQuery sqlQry = new SqlQuery<>(Integer.class, "t1._key = ? and t1._val > 1"); + + QueryCursor> qry = cache.query(sqlQry.setAlias("t1").setArgs(key)); + + Cache.Entry entry = F.first(qry.getAll()); + + assert entry != null; + + assertEquals(key, entry.getKey().intValue()); + assertEquals(val, entry.getValue().intValue()); + + sqlQry = new SqlQuery<>(Integer.class, "FROM Integer as t1 WHERE t1._key = ? and t1._val > 1"); + + qry = cache.query(sqlQry.setAlias("t1").setArgs(key)); + + entry = F.first(qry.getAll()); + + assert entry != null; + + assertEquals(key, entry.getKey().intValue()); + assertEquals(val, entry.getValue().intValue()); + } + /** * Tests UDFs. * diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java index 81e34d6959f90..ad8a7e34e82a7 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java @@ -223,15 +223,15 @@ public void testSpi() throws Exception { assertEquals(0, spi.size(typeAB.space(), typeAB)); assertEquals(0, spi.size(typeBA.space(), typeBA)); - assertFalse(spi.queryLocalSql(typeAA.space(), "select * from A.A", Collections.emptySet(), typeAA, null).hasNext()); - assertFalse(spi.queryLocalSql(typeAB.space(), "select * from A.B", Collections.emptySet(), typeAB, null).hasNext()); - assertFalse(spi.queryLocalSql(typeBA.space(), "select * from B.A", Collections.emptySet(), typeBA, null).hasNext()); + assertFalse(spi.queryLocalSql(typeAA.space(), "select * from A.A", null, Collections.emptySet(), typeAA, null).hasNext()); + assertFalse(spi.queryLocalSql(typeAB.space(), "select * from A.B", null, Collections.emptySet(), typeAB, null).hasNext()); + assertFalse(spi.queryLocalSql(typeBA.space(), "select * from B.A", null, Collections.emptySet(), typeBA, null).hasNext()); - assertFalse(spi.queryLocalSql(typeBA.space(), "select * from B.A, A.B, A.A", + assertFalse(spi.queryLocalSql(typeBA.space(), "select * from B.A, A.B, A.A", null, Collections.emptySet(), typeBA, null).hasNext()); try { - spi.queryLocalSql(typeBA.space(), "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba", + spi.queryLocalSql(typeBA.space(), "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba", null, Collections.emptySet(), typeBA, null).hasNext(); fail("Enumerations of aliases in select block must be prohibited"); @@ -240,10 +240,10 @@ public void testSpi() throws Exception { // all fine } - assertFalse(spi.queryLocalSql(typeAB.space(), "select ab.* from A.B ab", + assertFalse(spi.queryLocalSql(typeAB.space(), "select ab.* from A.B ab", null, Collections.emptySet(), typeAB, null).hasNext()); - assertFalse(spi.queryLocalSql(typeBA.space(), "select ba.* from B.A as ba", + assertFalse(spi.queryLocalSql(typeBA.space(), "select ba.* from B.A as ba", null, Collections.emptySet(), typeBA, null).hasNext()); // Nothing to remove. @@ -298,7 +298,7 @@ public void testSpi() throws Exception { // Query data. Iterator>> res = - spi.queryLocalSql(typeAA.space(), "from a order by age", Collections.emptySet(), typeAA, null); + spi.queryLocalSql(typeAA.space(), "from a order by age", null, Collections.emptySet(), typeAA, null); assertTrue(res.hasNext()); assertEquals(aa(3, "Borya", 18).value(null, false), value(res.next())); @@ -306,7 +306,7 @@ public void testSpi() throws Exception { assertEquals(aa(2, "Valera", 19).value(null, false), value(res.next())); assertFalse(res.hasNext()); - res = spi.queryLocalSql(typeAA.space(), "select aa.* from a aa order by aa.age", + res = spi.queryLocalSql(typeAA.space(), "select aa.* from a aa order by aa.age", null, Collections.emptySet(), typeAA, null); assertTrue(res.hasNext()); @@ -315,7 +315,7 @@ public void testSpi() throws Exception { assertEquals(aa(2, "Valera", 19).value(null, false), value(res.next())); assertFalse(res.hasNext()); - res = spi.queryLocalSql(typeAB.space(), "from b order by name", Collections.emptySet(), typeAB, null); + res = spi.queryLocalSql(typeAB.space(), "from b order by name", null, Collections.emptySet(), typeAB, null); assertTrue(res.hasNext()); assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here.").value(null, false), value(res.next())); @@ -323,7 +323,7 @@ public void testSpi() throws Exception { assertEquals(ab(4, "Vitalya", 20, "Very Good guy").value(null, false), value(res.next())); assertFalse(res.hasNext()); - res = spi.queryLocalSql(typeAB.space(), "select bb.* from b as bb order by bb.name", + res = spi.queryLocalSql(typeAB.space(), "select bb.* from b as bb order by bb.name", null, Collections.emptySet(), typeAB, null); assertTrue(res.hasNext()); @@ -333,7 +333,7 @@ public void testSpi() throws Exception { assertFalse(res.hasNext()); - res = spi.queryLocalSql(typeBA.space(), "from a", Collections.emptySet(), typeBA, null); + res = spi.queryLocalSql(typeBA.space(), "from a", null, Collections.emptySet(), typeBA, null); assertTrue(res.hasNext()); assertEquals(ba(2, "Kolya", 25, true).value(null, false), value(res.next())); From a922ac9d17f91f25aaa2bac9f0a2622dbd04c9bb Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 17 Jan 2017 15:31:04 +0300 Subject: [PATCH 424/487] IGNITE-4540: IndexingSPI can be used without have default H2 Indexing enabled. This closes #1423. --- .../cache/query/GridCacheQueryManager.java | 83 +++++++++++++++++-- .../processors/query/GridQueryProcessor.java | 46 ---------- .../cache/query/IndexingSpiQuerySelfTest.java | 66 +++++++-------- ...ndexingSpiQueryWithH2IndexingSelfTest.java | 36 ++++++++ 4 files changed, 145 insertions(+), 86 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryWithH2IndexingSelfTest.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 85c01d9937151..d64dff4b1a02e 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 @@ -1,4 +1,4 @@ -/* + /* * 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. @@ -45,6 +45,7 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.query.QueryMetrics; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; @@ -60,6 +61,7 @@ import org.apache.ignite.internal.processors.cache.CacheEntryImpl; import org.apache.ignite.internal.processors.cache.CacheMetricsImpl; import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.CacheObjectContext; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; @@ -167,6 +169,9 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte } }; + /** Default is @{code true} */ + private final boolean isIndexingSpiAllowsBinary = !IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_UNWRAP_BINARY_FOR_INDEXING_SPI); + /** */ private GridQueryProcessor qryProc; @@ -204,6 +209,10 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte /** */ private boolean enabled; + /** */ + private boolean qryProcEnabled; + + /** */ private AffinityTopologyVersion qryTopVer; @@ -211,9 +220,14 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte @Override public void start0() throws IgniteCheckedException { CacheConfiguration ccfg = cctx.config(); + qryProcEnabled = GridQueryProcessor.isEnabled(ccfg); + qryProc = cctx.kernalContext().query(); + space = cctx.name(); + enabled = qryProcEnabled || (isIndexingSpiEnabled() && !CU.isSystemCache(space)); + maxIterCnt = ccfg.getMaxQueryIteratorsCount(); detailMetricsSz = ccfg.getQueryDetailMetricsSize(); @@ -259,8 +273,6 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte cctx.events().addListener(lsnr, EVT_NODE_LEFT, EVT_NODE_FAILED); - enabled = GridQueryProcessor.isEnabled(ccfg); - qryTopVer = cctx.startTopologyVersion(); if (qryTopVer == null) @@ -369,17 +381,35 @@ void processQueryRequest(UUID sndId, GridCacheQueryRequest req) { * @throws IgniteCheckedException If failed. */ public void onSwap(CacheObject key) throws IgniteCheckedException { + if(!enabled) + return; + if (!enterBusy()) return; // Ignore index update when node is stopping. try { - qryProc.onSwap(space, key); + if (isIndexingSpiEnabled()) { + Object key0 = unwrapIfNeeded(key, cctx.cacheObjectContext()); + + cctx.kernalContext().indexing().onSwap(space, key0); + } + + if(qryProcEnabled) + qryProc.onSwap(space, key); } finally { leaveBusy(); } } + /** + * Checks if IndexinSPI is enabled. + * @return IndexingSPI enabled flag. + */ + private boolean isIndexingSpiEnabled() { + return cctx.kernalContext().indexing().enabled(); + } + /** * Entry for given key unswapped. * @@ -388,11 +418,25 @@ public void onSwap(CacheObject key) throws IgniteCheckedException { * @throws IgniteCheckedException If failed. */ public void onUnswap(CacheObject key, CacheObject val) throws IgniteCheckedException { + if(!enabled) + return; + if (!enterBusy()) return; // Ignore index update when node is stopping. try { - qryProc.onUnswap(space, key, val); + if (isIndexingSpiEnabled()) { + CacheObjectContext coctx = cctx.cacheObjectContext(); + + Object key0 = unwrapIfNeeded(key, coctx); + + Object val0 = unwrapIfNeeded(val, coctx); + + cctx.kernalContext().indexing().onUnswap(space, key0, val0); + } + + if(qryProcEnabled) + qryProc.onUnswap(space, key, val); } finally { leaveBusy(); @@ -429,7 +473,18 @@ public void store(CacheObject key, CacheObject val, GridCacheVersion ver, long e return; // Ignore index update when node is stopping. try { - qryProc.store(space, key, val, CU.versionToBytes(ver), expirationTime); + if (isIndexingSpiEnabled()) { + CacheObjectContext coctx = cctx.cacheObjectContext(); + + Object key0 = unwrapIfNeeded(key, coctx); + + Object val0 = unwrapIfNeeded(val, coctx); + + cctx.kernalContext().indexing().store(space, key0, val0, expirationTime); + } + + if(qryProcEnabled) + qryProc.store(space, key, val, CU.versionToBytes(ver), expirationTime); } finally { invalidateResultCache(); @@ -454,7 +509,14 @@ public void remove(CacheObject key, CacheObject val) throws IgniteCheckedExcepti return; // Ignore index update when node is stopping. try { - qryProc.remove(space, key, val); + if (isIndexingSpiEnabled()) { + Object key0 = unwrapIfNeeded(key, cctx.cacheObjectContext()); + + cctx.kernalContext().indexing().remove(space, key0); + } + + if(qryProcEnabled) + qryProc.remove(space, key, val); } finally { invalidateResultCache(); @@ -560,6 +622,13 @@ public abstract GridCloseableIterator scanQueryDistributed(GridCacheQueryAdapter */ public abstract CacheQueryFuture queryFieldsDistributed(GridCacheQueryBean qry, Collection nodes); + /** + * Unwrap CacheObject if needed. + */ + private Object unwrapIfNeeded(CacheObject obj, CacheObjectContext coctx) { + return isIndexingSpiAllowsBinary && cctx.cacheObjects().isBinaryObject(obj) ? obj : obj.value(coctx, false); + } + /** * Performs query. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index f4ac4ae71a546..48ca2b5bab69c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -160,9 +160,6 @@ public class GridQueryProcessor extends GridProcessorAdapter { /** */ private static final ThreadLocal requestTopVer = new ThreadLocal<>(); - /** Default is @{true} */ - private final boolean isIndexingSpiAllowsBinary = !IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_UNWRAP_BINARY_FOR_INDEXING_SPI); - /** * @param ctx Kernal context. */ @@ -682,16 +679,6 @@ public void store(final String space, final CacheObject key, final CacheObject v CacheObjectContext coctx = null; - if (ctx.indexing().enabled()) { - coctx = cacheObjectContext(space); - - Object key0 = unwrap(key, coctx); - - Object val0 = unwrap(val, coctx); - - ctx.indexing().store(space, key0, val0, expirationTime); - } - if (idx == null) return; @@ -744,13 +731,6 @@ public void store(final String space, final CacheObject key, final CacheObject v } } - /** - * Unwrap CacheObject if needed. - */ - private Object unwrap(CacheObject obj, CacheObjectContext coctx) { - return isIndexingSpiAllowsBinary && ctx.cacheObjects().isBinaryObject(obj) ? obj : obj.value(coctx, false); - } - /** * @throws IgniteCheckedException If failed. */ @@ -1039,14 +1019,6 @@ public void remove(String space, CacheObject key, CacheObject val) throws Ignite if (log.isDebugEnabled()) log.debug("Remove [space=" + space + ", key=" + key + ", val=" + val + "]"); - if (ctx.indexing().enabled()) { - CacheObjectContext coctx = cacheObjectContext(space); - - Object key0 = unwrap(key, coctx); - - ctx.indexing().remove(space, key0); - } - if (idx == null) return; @@ -1184,14 +1156,6 @@ public void onSwap(String spaceName, CacheObject key) throws IgniteCheckedExcept if (log.isDebugEnabled()) log.debug("Swap [space=" + spaceName + ", key=" + key + "]"); - if (ctx.indexing().enabled()) { - CacheObjectContext coctx = cacheObjectContext(spaceName); - - Object key0 = unwrap(key, coctx); - - ctx.indexing().onSwap(spaceName, key0); - } - if (idx == null) return; @@ -1221,16 +1185,6 @@ public void onUnswap(String spaceName, CacheObject key, CacheObject val) if (log.isDebugEnabled()) log.debug("Unswap [space=" + spaceName + ", key=" + key + ", val=" + val + "]"); - if (ctx.indexing().enabled()) { - CacheObjectContext coctx = cacheObjectContext(spaceName); - - Object key0 = unwrap(key, coctx); - - Object val0 = unwrap(val, coctx); - - ctx.indexing().onUnswap(spaceName, key0, val0); - } - if (idx == null) return; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java index f66b99ef5b1c7..84a13df36b52f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java @@ -55,14 +55,39 @@ import org.jetbrains.annotations.Nullable; /** - * Indexing Spi query test + * Indexing Spi query only test */ public class IndexingSpiQuerySelfTest extends TestCase { + public static final String CACHE_NAME = "test-cache"; + /** {@inheritDoc} */ @Override public void tearDown() throws Exception { Ignition.stopAll(true); } + /** + * @return Configuration. + */ + protected IgniteConfiguration configuration() { + IgniteConfiguration cfg = new IgniteConfiguration(); + + TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setMaxMissedHeartbeats(Integer.MAX_VALUE); + + disco.setIpFinder(ipFinder); + + cfg.setDiscoverySpi(disco); + + return cfg; + } + + /** */ + protected CacheConfiguration cacheConfiguration(String cacheName) { + return new CacheConfiguration<>(cacheName); + } + /** * @throws Exception If failed. */ @@ -73,9 +98,7 @@ public void testSimpleIndexingSpi() throws Exception { Ignite ignite = Ignition.start(cfg); - CacheConfiguration ccfg = new CacheConfiguration<>("test-cache"); - - ccfg.setIndexedTypes(Integer.class, Integer.class); + CacheConfiguration ccfg = cacheConfiguration(CACHE_NAME); IgniteCache cache = ignite.createCache(ccfg); @@ -98,7 +121,7 @@ public void testIndexingSpiWithDisabledQueryProcessor() throws Exception { Ignite ignite = Ignition.start(cfg); - CacheConfiguration ccfg = new CacheConfiguration<>("test-cache"); + CacheConfiguration ccfg = cacheConfiguration(CACHE_NAME); IgniteCache cache = ignite.createCache(ccfg); @@ -121,9 +144,7 @@ public void testBinaryIndexingSpi() throws Exception { Ignite ignite = Ignition.start(cfg); - CacheConfiguration ccfg = new CacheConfiguration<>("test-binary-cache"); - - ccfg.setIndexedTypes(PersonKey.class, Person.class); + CacheConfiguration ccfg = cacheConfiguration(CACHE_NAME); IgniteCache cache = ignite.createCache(ccfg); @@ -155,9 +176,7 @@ public void testNonBinaryIndexingSpi() throws Exception { Ignite ignite = Ignition.start(cfg); - CacheConfiguration ccfg = new CacheConfiguration<>("test-binary-cache"); - - ccfg.setIndexedTypes(PersonKey.class, Person.class); + CacheConfiguration ccfg = cacheConfiguration(CACHE_NAME); IgniteCache cache = ignite.createCache(ccfg); @@ -187,10 +206,9 @@ public void testIndexingSpiFailure() throws Exception { Ignite ignite = Ignition.start(cfg); - CacheConfiguration ccfg = new CacheConfiguration<>("test-cache"); + CacheConfiguration ccfg = cacheConfiguration(CACHE_NAME); ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); - ccfg.setIndexedTypes(Integer.class, Integer.class); final IgniteCache cache = ignite.createCache(ccfg); @@ -218,24 +236,6 @@ public void testIndexingSpiFailure() throws Exception { } } - /** - * @return Configuration. - */ - private IgniteConfiguration configuration() { - IgniteConfiguration cfg = new IgniteConfiguration(); - - TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); - TcpDiscoverySpi disco = new TcpDiscoverySpi(); - - disco.setMaxMissedHeartbeats(Integer.MAX_VALUE); - - disco.setIpFinder(ipFinder); - - cfg.setDiscoverySpi(disco); - - return cfg; - } - /** * Indexing Spi implementation for test */ @@ -350,7 +350,7 @@ private static class MyBrokenIndexingSpi extends MyIndexingSpi { /** * */ - private static class PersonKey implements Serializable, Comparable { + static class PersonKey implements Serializable, Comparable { /** */ private int id; @@ -385,7 +385,7 @@ public PersonKey(int id) { /** * */ - private static class Person implements Serializable { + static class Person implements Serializable { /** */ private String name; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryWithH2IndexingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryWithH2IndexingSelfTest.java new file mode 100644 index 0000000000000..800c5a2e80701 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryWithH2IndexingSelfTest.java @@ -0,0 +1,36 @@ +/* + * 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.query; + +import org.apache.ignite.configuration.CacheConfiguration; + +/** + * Indexing Spi query with configured default indexer test + */ +public class IndexingSpiQueryWithH2IndexingSelfTest extends IndexingSpiQuerySelfTest { + /** */ + protected CacheConfiguration cacheConfiguration(String cacheName) { + CacheConfiguration ccfg = super.cacheConfiguration(cacheName); + + ccfg.setIndexedTypes(PersonKey.class, Person.class); + + ccfg.setIndexedTypes(Integer.class, Integer.class); + + return ccfg; + } +} \ No newline at end of file From d0c0bcece7d8e9d373aaf13a210f6d890e5ad48b Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 17 Jan 2017 16:19:02 +0300 Subject: [PATCH 425/487] IGNITE-3867: Fixed ScanQuery ignores pageSize property. This closes #1406. --- .../processors/cache/IgniteCacheProxy.java | 3 + .../IgniteCachePartitionedQuerySelfTest.java | 87 +++++++++++++++++++ 2 files changed, 90 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java index b9737c62dcbf3..873c8221ae921 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java @@ -491,6 +491,9 @@ private QueryCursor query( qry = ctx.queries().createScanQuery(p, transformer, scanQry.getPartition(), isKeepBinary); + if (scanQry.getPageSize() > 0) + qry.pageSize(scanQry.getPageSize()); + if (grp != null) qry.projection(grp); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedQuerySelfTest.java index 78fd914eeef20..b9f21dacdcc07 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedQuerySelfTest.java @@ -20,15 +20,28 @@ import java.util.Collection; import java.util.List; import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; import javax.cache.Cache; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.managers.communication.GridIoMessage; import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractQuerySelfTest; +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest; +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryResponse; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.CommunicationSpi; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; import static org.apache.ignite.cache.CacheMode.PARTITIONED; import static org.apache.ignite.cache.CachePeekMode.ALL; @@ -47,6 +60,11 @@ public class IgniteCachePartitionedQuerySelfTest extends IgniteCacheAbstractQuer return PARTITIONED; } + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + return super.getConfiguration(gridName).setCommunicationSpi(new TestTcpCommunicationSpi()); + } + /** * @throws Exception If failed. */ @@ -135,4 +153,73 @@ private void checkResult(Iterable> entries, Person... assert F.asList(persons).contains(entry.getValue()); } } + + /** + * @throws Exception If failed. + */ + public void testScanQueryPagination() throws Exception { + final int pageSize = 5; + + final AtomicInteger pages = new AtomicInteger(0); + + IgniteCache cache = ignite().cache(null); + + for (int i = 0; i < 50; i++) + cache.put(i, i); + + CommunicationSpi spi = ignite().configuration().getCommunicationSpi(); + + assert spi instanceof TestTcpCommunicationSpi; + + TestTcpCommunicationSpi commSpi = (TestTcpCommunicationSpi)spi; + + commSpi.filter = new IgniteInClosure() { + @Override public void apply(Message msg) { + if (!(msg instanceof GridIoMessage)) + return; + + Message msg0 = ((GridIoMessage)msg).message(); + + if (msg0 instanceof GridCacheQueryRequest) { + assertEquals(pageSize, ((GridCacheQueryRequest)msg0).pageSize()); + + pages.incrementAndGet(); + } + else if (msg0 instanceof GridCacheQueryResponse) { + assertTrue(((GridCacheQueryResponse)msg0).data().size() <= pageSize); + } + } + }; + + try { + ScanQuery qry = new ScanQuery(); + + qry.setPageSize(pageSize); + + List> all = cache.query(qry).getAll(); + + assertTrue(pages.get() > ignite().cluster().forDataNodes(null).nodes().size()); + + assertEquals(50, all.size()); + } + finally { + commSpi.filter = null; + } + } + + /** + * + */ + private static class TestTcpCommunicationSpi extends TcpCommunicationSpi { + volatile IgniteInClosure filter; + + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackClosure) + throws IgniteSpiException { + if(filter != null) + filter.apply(msg); + + super.sendMessage(node, msg, ackClosure); + } + } } \ No newline at end of file From b54a481315a45c7a6c8f70534f655e14b25cc439 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 18 Jan 2017 12:05:22 +0300 Subject: [PATCH 426/487] ignite-4525 - Near reader is created when value is loaded from store. --- .../processors/cache/GridCacheAdapter.java | 33 +- .../processors/cache/GridCacheContext.java | 4 +- .../processors/cache/GridCacheEntryEx.java | 20 +- .../processors/cache/GridCacheMapEntry.java | 56 ++- .../processors/cache/ReaderArguments.java | 74 +++ .../distributed/dht/GridDhtCacheAdapter.java | 9 +- .../distributed/dht/GridDhtGetFuture.java | 85 ++-- .../dht/GridDhtGetSingleFuture.java | 75 ++- .../dht/GridPartitionedGetFuture.java | 3 +- .../dht/GridPartitionedSingleGetFuture.java | 3 +- .../dht/atomic/GridDhtAtomicCache.java | 5 +- .../dht/colocated/GridDhtColocatedCache.java | 3 +- .../distributed/near/GridNearGetFuture.java | 6 +- .../local/atomic/GridLocalAtomicCache.java | 5 +- .../transactions/IgniteTxLocalAdapter.java | 35 +- .../cache/GridCacheTestEntryEx.java | 11 +- .../near/GridNearCacheStoreUpdateTest.java | 466 ++++++++++++++++++ .../GridNearOffheapCacheStoreUpdateTest.java | 35 ++ .../testsuites/IgniteCacheTestSuite2.java | 5 + 19 files changed, 770 insertions(+), 163 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ReaderArguments.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheStoreUpdateTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffheapCacheStoreUpdateTest.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 fd9f396926f0e..59665bb58708f 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 @@ -1789,6 +1789,7 @@ protected IgniteInternalFuture> getAllAsync( subjId = ctx.subjectIdPerCall(subjId, opCtx); return getAllAsync(keys, + null, opCtx == null || !opCtx.skipStore(), !skipTx, subjId, @@ -1803,6 +1804,7 @@ protected IgniteInternalFuture> getAllAsync( /** * @param keys Keys. + * @param readerArgs Near cache reader will be added if not null. * @param readThrough Read through. * @param checkTx Check tx. * @param subjId Subj Id. @@ -1817,6 +1819,7 @@ protected IgniteInternalFuture> getAllAsync( * @see GridCacheAdapter#getAllAsync(Collection) */ public final IgniteInternalFuture> getAllAsync(@Nullable final Collection keys, + @Nullable final ReaderArguments readerArgs, boolean readThrough, boolean checkTx, @Nullable final UUID subjId, @@ -1834,6 +1837,7 @@ public final IgniteInternalFuture> getAllAsync(@Nullable final Collect validateCacheKeys(keys); return getAllAsync0(ctx.cacheKeysView(keys), + readerArgs, readThrough, checkTx, subjId, @@ -1848,6 +1852,7 @@ public final IgniteInternalFuture> getAllAsync(@Nullable final Collect /** * @param keys Keys. + * @param readerArgs Near cache reader will be added if not null. * @param readThrough Read-through flag. * @param checkTx Check local transaction flag. * @param subjId Subject ID. @@ -1862,6 +1867,7 @@ public final IgniteInternalFuture> getAllAsync(@Nullable final Collect */ protected final IgniteInternalFuture> getAllAsync0( @Nullable final Collection keys, + @Nullable final ReaderArguments readerArgs, final boolean readThrough, boolean checkTx, @Nullable final UUID subjId, @@ -1932,7 +1938,8 @@ protected final IgniteInternalFuture> getAllAsync0( subjId, taskName, expiry, - !deserializeBinary); + !deserializeBinary, + readerArgs); assert res != null; @@ -1957,7 +1964,8 @@ protected final IgniteInternalFuture> getAllAsync0( null, taskName, expiry, - !deserializeBinary); + !deserializeBinary, + readerArgs); if (res == null) ctx.evicts().touch(entry, topVer); @@ -2015,29 +2023,28 @@ protected final IgniteInternalFuture> getAllAsync0( GridCacheEntryEx entry = entryEx(key); try { - GridCacheVersion verSet = entry.versionedValue(cacheVal, + T2 verVal = entry.versionedValue( + cacheVal, res.version(), - null); - - boolean set = verSet != null; + null, + readerArgs); if (log.isDebugEnabled()) log.debug("Set value loaded from store into entry [" + - "set=" + set + - ", curVer=" + res.version() + - ", newVer=" + verSet + ", " + + "oldVer=" + res.version() + + ", newVer=" + verVal.get2() + ", " + "entry=" + entry + ']'); // Don't put key-value pair into result map if value is null. - if (val != null) { + if (verVal.get1() != null) { ctx.addResult(map, key, - cacheVal, + verVal.get1(), skipVals, keepCacheObjects, deserializeBinary, - false, - needVer ? set ? verSet : res.version() : null); + true, + needVer ? verVal.get2() : null); } if (tx0 == null || (!tx0.implicit() && 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 66b71b4666e7b..424e325dd07f2 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 @@ -1900,9 +1900,9 @@ public void addResult(Map map, assert val != null || skipVals; if (!keepCacheObjects) { - Object key0 = unwrapBinaryIfNeeded(key, !deserializeBinary); + Object key0 = unwrapBinaryIfNeeded(key, !deserializeBinary, cpy); - Object val0 = skipVals ? true : unwrapBinaryIfNeeded(val, !deserializeBinary); + Object val0 = skipVals ? true : unwrapBinaryIfNeeded(val, !deserializeBinary, cpy); assert key0 != null : key; assert val0 != null : val; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java index b1d632fd5943d..51f423a7302a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java @@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx; import org.apache.ignite.internal.processors.dr.GridDrType; import org.apache.ignite.internal.util.lang.GridTuple3; +import org.apache.ignite.internal.util.typedef.T2; import org.jetbrains.annotations.Nullable; /** @@ -318,11 +319,12 @@ public boolean evictInternal(boolean swap, GridCacheVersion obsoleteVer, * @param taskName Task name. * @param expiryPlc Expiry policy. * @param keepBinary Keep binary flag. + * @param readerArgs Reader will be added if not null. * @return Cached value and entry version. * @throws IgniteCheckedException If loading value failed. * @throws GridCacheEntryRemovedException If entry was removed. */ - @Nullable public EntryGetResult innerGetVersioned( + public EntryGetResult innerGetVersioned( @Nullable GridCacheVersion ver, IgniteInternalTx tx, boolean readSwap, @@ -333,7 +335,8 @@ public boolean evictInternal(boolean swap, GridCacheVersion obsoleteVer, Object transformClo, String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, - boolean keepBinary) + boolean keepBinary, + @Nullable ReaderArguments readerArgs) throws IgniteCheckedException, GridCacheEntryRemovedException; /** @@ -344,7 +347,7 @@ public boolean evictInternal(boolean swap, GridCacheVersion obsoleteVer, * @param taskName Task name. * @param expiryPlc Expiry policy. * @param keepBinary Keep binary flag. - * @return Cached value and entry version. + * @param readerArgs Reader will be added if not null. * @throws IgniteCheckedException If loading value failed. * @throws GridCacheEntryRemovedException If entry was removed. * @return Cached value, entry version and flag indicating if entry was reserved. @@ -355,7 +358,8 @@ public EntryGetResult innerGetAndReserveForLoad(boolean readSwap, UUID subjId, String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, - boolean keepBinary) throws IgniteCheckedException, GridCacheEntryRemovedException; + boolean keepBinary, + @Nullable ReaderArguments readerArgs) throws IgniteCheckedException, GridCacheEntryRemovedException; /** * @param ver Expected entry version. @@ -751,13 +755,15 @@ public GridCacheVersionedEntryEx versionedEntry(final boolean keepB * @param val New value. * @param curVer Version to match or {@code null} if match is not required. * @param newVer Version to set. - * @return Non null version if value was set. + * @param readerArgs Reader will be added if not null. + * @return Current version and value. * @throws IgniteCheckedException If index could not be updated. * @throws GridCacheEntryRemovedException If entry was removed. */ - public GridCacheVersion versionedValue(CacheObject val, + public T2 versionedValue(CacheObject val, @Nullable GridCacheVersion curVer, - @Nullable GridCacheVersion newVer) + @Nullable GridCacheVersion newVer, + @Nullable ReaderArguments readerArgs) throws IgniteCheckedException, GridCacheEntryRemovedException; /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 9f0c2b0a47c56..59e4181df0cee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -778,7 +778,8 @@ protected final void releaseSwap() throws IgniteCheckedException { expirePlc, false, keepBinary, - false); + false, + null); } /** {@inheritDoc} */ @@ -788,7 +789,8 @@ protected final void releaseSwap() throws IgniteCheckedException { UUID subjId, String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, - boolean keepBinary) throws IgniteCheckedException, GridCacheEntryRemovedException { + boolean keepBinary, + @Nullable ReaderArguments readerArgs) throws IgniteCheckedException, GridCacheEntryRemovedException { return (EntryGetResult)innerGet0( /*ver*/null, /*tx*/null, @@ -803,11 +805,12 @@ protected final void releaseSwap() throws IgniteCheckedException { expiryPlc, true, keepBinary, - /*reserve*/true); + /*reserve*/true, + readerArgs); } /** {@inheritDoc} */ - @Nullable @Override public EntryGetResult innerGetVersioned( + @Override public EntryGetResult innerGetVersioned( @Nullable GridCacheVersion ver, IgniteInternalTx tx, boolean readSwap, @@ -818,7 +821,8 @@ protected final void releaseSwap() throws IgniteCheckedException { Object transformClo, String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, - boolean keepBinary) + boolean keepBinary, + @Nullable ReaderArguments readerArgs) throws IgniteCheckedException, GridCacheEntryRemovedException { return (EntryGetResult)innerGet0(ver, tx, @@ -833,7 +837,8 @@ protected final void releaseSwap() throws IgniteCheckedException { expiryPlc, true, keepBinary, - false); + false, + readerArgs); } /** {@inheritDoc} */ @@ -852,7 +857,8 @@ private Object innerGet0( @Nullable IgniteCacheExpiryPolicy expiryPlc, boolean retVer, boolean keepBinary, - boolean reserveForLoad + boolean reserveForLoad, + @Nullable ReaderArguments readerArgs ) throws IgniteCheckedException, GridCacheEntryRemovedException { assert !(retVer && readThrough); assert !(reserveForLoad && readThrough); @@ -961,6 +967,8 @@ private Object innerGet0( // Cache version for optimistic check. startVer = ver; + addReaderIfNeed(readerArgs); + if (ret != null) { assert tmp || !(ret instanceof BinaryObjectOffheapImpl); assert !obsolete; @@ -1051,6 +1059,8 @@ else if (tx.dht()) { if (cctx.deferredDelete() && deletedUnlocked() && !isInternal() && !detached()) deletedUnlocked(false); + + assert readerArgs == null; } if (evt && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) { @@ -3611,19 +3621,22 @@ private long nextPartCounter(AffinityTopologyVersion topVer) { } /** {@inheritDoc} */ - @Override public synchronized GridCacheVersion versionedValue(CacheObject val, + @Override public synchronized T2 versionedValue(CacheObject val, GridCacheVersion curVer, - GridCacheVersion newVer) + GridCacheVersion newVer, + @Nullable ReaderArguments readerArgs) throws IgniteCheckedException, GridCacheEntryRemovedException { checkObsolete(); + addReaderIfNeed(readerArgs); + if (curVer == null || curVer.equals(ver)) { if (val != this.val) { GridCacheMvcc mvcc = mvccExtras(); if (mvcc != null && !mvcc.isEmpty()) - return null; + return new T2<>(this.val, ver); if (newVer == null) newVer = cctx.versions().next(); @@ -3647,13 +3660,32 @@ private long nextPartCounter(AffinityTopologyVersion topVer) { // Version does not change for load ops. update(val, expTime, ttl, newVer, true); - return newVer; + return new T2<>(val, newVer); } assert !evictionDisabled() : this; } - return null; + return new T2<>(this.val, ver); + } + + /** + * @param readerArgs Reader arguments + */ + private void addReaderIfNeed(@Nullable ReaderArguments readerArgs) { + if (readerArgs != null) { + assert this instanceof GridDhtCacheEntry : this; + assert Thread.holdsLock(this); + + try { + ((GridDhtCacheEntry)this).addReader(readerArgs.reader(), + readerArgs.messageId(), + readerArgs.topologyVersion()); + } + catch (GridCacheEntryRemovedException e) { + assert false : this; + } + } } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ReaderArguments.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ReaderArguments.java new file mode 100644 index 0000000000000..b8b5e64894fda --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ReaderArguments.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.util.UUID; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Arguments required for adding near cache reader to entry. + */ +public class ReaderArguments { + /** */ + private final UUID reader; + + /** */ + private final long msgId; + + /** */ + private final AffinityTopologyVersion topVer; + + /** + * @param reader Near cache node ID. + * @param msgId Message ID. + * @param topVer Topology version. + */ + public ReaderArguments(final UUID reader, final long msgId, + final AffinityTopologyVersion topVer) { + this.reader = reader; + this.msgId = msgId; + this.topVer = topVer; + } + + /** + * @return Reader node ID. + */ + public UUID reader() { + return reader; + } + + /** + * @return Message ID. + */ + public long messageId() { + return msgId; + } + + /** + * @return Topology version. + */ + public AffinityTopologyVersion topologyVersion() { + return topVer; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ReaderArguments.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java index b2fb7b4b392a4..543cee1809070 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java @@ -35,8 +35,8 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.NodeStoppingException; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.CacheOperationContext; @@ -53,6 +53,7 @@ import org.apache.ignite.internal.processors.cache.GridCachePreloader; import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.ReaderArguments; import org.apache.ignite.internal.processors.cache.distributed.GridCacheTtlUpdateRequest; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheAdapter; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheEntry; @@ -623,6 +624,7 @@ else if (log.isDebugEnabled()) CacheOperationContext opCtx = ctx.operationContextPerCall(); return getAllAsync(keys, + null, opCtx == null || !opCtx.skipStore(), /*don't check local tx. */false, subjId, @@ -637,6 +639,7 @@ else if (log.isDebugEnabled()) /** * @param keys Keys to get + * @param readerArgs Reader will be added if not null. * @param readThrough Read through flag. * @param subjId Subject ID. * @param taskName Task name. @@ -647,6 +650,7 @@ else if (log.isDebugEnabled()) */ IgniteInternalFuture>> getDhtAllAsync( Collection keys, + @Nullable final ReaderArguments readerArgs, boolean readThrough, @Nullable UUID subjId, String taskName, @@ -655,6 +659,7 @@ IgniteInternalFuture>> get boolean canRemap ) { return getAllAsync0(keys, + readerArgs, readThrough, /*don't check local tx. */false, subjId, @@ -694,7 +699,6 @@ public GridDhtFuture> getDhtAsync(UUID reader, reader, keys, readThrough, - /*tx*/null, topVer, subjId, taskNameHash, @@ -738,7 +742,6 @@ private IgniteInternalFuture getDhtSingleAsync( key, addRdr, readThrough, - /*tx*/null, topVer, subjId, taskNameHash, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java index 913580f8c02ad..3bf44895a62a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java @@ -29,7 +29,6 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; @@ -37,7 +36,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; -import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx; +import org.apache.ignite.internal.processors.cache.ReaderArguments; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridCompoundIdentityFuture; @@ -50,7 +49,6 @@ import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgniteBiClosure; import org.apache.ignite.lang.IgniteUuid; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -96,9 +94,6 @@ public final class GridDhtGetFuture extends GridCompoundIdentityFuture retries; @@ -120,7 +115,6 @@ public final class GridDhtGetFuture extends GridCompoundIdentityFuture keys, boolean readThrough, - @Nullable IgniteTxLocalEx tx, @NotNull AffinityTopologyVersion topVer, @Nullable UUID subjId, int taskNameHash, @@ -150,7 +143,6 @@ public GridDhtGetFuture( this.msgId = msgId; this.keys = keys; this.readThrough = readThrough; - this.tx = tx; this.topVer = topVer; this.subjId = subjId; this.taskNameHash = taskNameHash; @@ -159,7 +151,7 @@ public GridDhtGetFuture( futId = IgniteUuid.randomUuid(); - ver = tx == null ? cctx.versions().next() : tx.xidVersion(); + ver = cctx.versions().next(); if (log == null) log = U.logger(cctx.kernalContext(), logRef, GridDhtGetFuture.class); @@ -340,6 +332,8 @@ private IgniteInternalFuture> getAsync( ClusterNode readerNode = cctx.discovery().node(reader); + ReaderArguments readerArgs = null; + if (readerNode != null && !readerNode.isLocal() && cctx.discovery().cacheNearNode(readerNode, cctx.name())) { for (Map.Entry k : keys.entrySet()) { while (true) { @@ -351,12 +345,19 @@ private IgniteInternalFuture> getAsync( boolean addReader = (!e.deleted() && k.getValue() && !skipVals); - if (addReader) + if (addReader) { e.unswap(false); + // Entry will be removed on touch() if no data in cache, + // but they could be loaded from store, + // we have to add reader again later. + if (readerArgs == null) + readerArgs = new ReaderArguments(reader, msgId, topVer); + } + // Register reader. If there are active transactions for this entry, // then will wait for their completion before proceeding. - // TODO: GG-4003: + // TODO: IGNITE-3498: // TODO: What if any transaction we wait for actually removes this entry? // TODO: In this case seems like we will be stuck with untracked near entry. // TODO: To fix, check that reader is contained in the list of readers once @@ -392,28 +393,19 @@ private IgniteInternalFuture> getAsync( IgniteInternalFuture>> fut; if (txFut == null || txFut.isDone()) { - if (tx == null) { - fut = cache().getDhtAllAsync( - keys.keySet(), - readThrough, - subjId, - taskName, - expiryPlc, - skipVals, - /*can remap*/true); - } - else { - fut = tx.getAllAsync(cctx, - null, - keys.keySet(), - /*deserialize binary*/false, - skipVals, - /*keep cache objects*/true, - /*skip store*/!readThrough, - false); - } + fut = cache().getDhtAllAsync( + keys.keySet(), + readerArgs, + readThrough, + subjId, + taskName, + expiryPlc, + skipVals, + /*can remap*/true); } else { + final ReaderArguments args = readerArgs; + // If we are here, then there were active transactions for some entries // when we were adding the reader. In that case we must wait for those // transactions to complete. @@ -424,26 +416,15 @@ private IgniteInternalFuture> getAsync( if (e != null) throw new GridClosureException(e); - if (tx == null) { - return cache().getDhtAllAsync( - keys.keySet(), - readThrough, - subjId, - taskName, - expiryPlc, - skipVals, - /*can remap*/true); - } - else { - return tx.getAllAsync(cctx, - null, - keys.keySet(), - /*deserialize binary*/false, - skipVals, - /*keep cache objects*/true, - /*skip store*/!readThrough, - false); - } + return cache().getDhtAllAsync( + keys.keySet(), + args, + readThrough, + subjId, + taskName, + expiryPlc, + skipVals, + /*can remap*/true); } } ); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java index 93949377e12db..49bebd6a74b45 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetSingleFuture.java @@ -35,7 +35,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy; import org.apache.ignite.internal.processors.cache.KeyCacheObject; -import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx; +import org.apache.ignite.internal.processors.cache.ReaderArguments; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.F; @@ -90,9 +90,6 @@ public final class GridDhtGetSingleFuture extends GridFutureAdapter retries; @@ -115,7 +112,6 @@ public final class GridDhtGetSingleFuture extends GridFutureAdapter>> fut; if (rdrFut == null || rdrFut.isDone()) { - if (tx == null) { - fut = cache().getDhtAllAsync( - Collections.singleton(key), - readThrough, - subjId, - taskName, - expiryPlc, - skipVals, - /*can remap*/true); - } - else { - fut = tx.getAllAsync(cctx, - null, - Collections.singleton(key), - /*deserialize binary*/false, - skipVals, - /*keep cache objects*/true, - /*skip store*/!readThrough, - false); - } + fut = cache().getDhtAllAsync( + Collections.singleton(key), + readerArgs, + readThrough, + subjId, + taskName, + expiryPlc, + skipVals, + /*can remap*/true); } else { + final ReaderArguments args = readerArgs; + rdrFut.listen( new IgniteInClosure>() { @Override public void apply(IgniteInternalFuture fut) { @@ -381,29 +375,16 @@ private void getAsync() { return; } - IgniteInternalFuture>> fut0; - - if (tx == null) { - fut0 = cache().getDhtAllAsync( + IgniteInternalFuture>> fut0 = + cache().getDhtAllAsync( Collections.singleton(key), + args, readThrough, subjId, taskName, expiryPlc, skipVals, /*can remap*/true); - } - else { - fut0 = tx.getAllAsync(cctx, - null, - Collections.singleton(key), - /*deserialize binary*/false, - skipVals, - /*keep cache objects*/true, - /*skip store*/!readThrough, - false - ); - } fut0.listen(createGetFutureListener()); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java index d0f209df94ca0..c8e2cf328bcd3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java @@ -461,7 +461,8 @@ private boolean localGet(KeyCacheObject key, int part, Map locVals) { null, taskName, expiryPlc, - !deserializeBinary); + !deserializeBinary, + null); if (res != null) { v = res.value(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index e188a3523a3a5..e369bfa83a8be 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -385,7 +385,8 @@ private boolean localGet(AffinityTopologyVersion topVer, int part) { null, taskName, expiryPlc, - true); + true, + null); if (res != null) { v = res.value(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java index 94a049ea86c6e..f601e0a9ddb7e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java @@ -1505,7 +1505,8 @@ private IgniteInternalFuture> getAllAsync0(@Nullable Collection entries) throws Ignite try { GridCacheVersion ver = entry.version(); - entry.versionedValue(ctx.toCacheObject(v), null, ver); + entry.versionedValue(ctx.toCacheObject(v), null, ver, null); } catch (GridCacheEntryRemovedException e) { assert false : "Entry should not get obsolete while holding lock [entry=" + entry + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java index 56af95e6b65c1..29f0607e5c33c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java @@ -496,7 +496,8 @@ public final IgniteInternalFuture> loadAsync( null, taskName, expiryPlc, - !deserializeBinary); + !deserializeBinary, + null); if (res != null) { v = res.value(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java index ab0e88cbf375a..8bc513e4dfda2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java @@ -449,7 +449,8 @@ private Map map( null, taskName, expiryPlc, - !deserializeBinary); + !deserializeBinary, + null); if (res != null) { v = res.value(); @@ -589,7 +590,8 @@ private boolean localDhtGet(KeyCacheObject key, null, taskName, expiryPlc, - !deserializeBinary); + !deserializeBinary, + null); if (res != null) { v = res.value(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java index d1acada062cb7..ad818a63d8c21 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java @@ -59,7 +59,6 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.resource.GridResourceIoc; -import org.apache.ignite.internal.processors.resource.GridResourceProcessor; import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.future.GridEmbeddedFuture; @@ -528,7 +527,8 @@ private Map getAllInternal(@Nullable Collection keys, null, taskName, expiry, - !deserializeBinary); + !deserializeBinary, + null); if (res != null) { v = res.value(); @@ -602,6 +602,7 @@ private Map getAllInternal(@Nullable Collection keys, return getAllAsync( keys, + null, opCtx == null || !opCtx.skipStore(), false, subjId, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index 91c9c92075eab..f05d90df6cf10 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -438,7 +438,8 @@ protected boolean commitAfterLock() { null, resolveTaskName(), expiryPlc, - txEntry == null ? keepBinary : txEntry.keepBinary()); + txEntry == null ? keepBinary : txEntry.keepBinary(), + null); if (res == null) { if (misses == null) @@ -477,17 +478,19 @@ protected boolean commitAfterLock() { GridCacheEntryEx entry = cacheCtx.cache().entryEx(key); try { - GridCacheVersion setVer = entry.versionedValue(cacheVal, ver, null); - - boolean set = setVer != null; + T2 verVal = entry.versionedValue(cacheVal, + ver, + null, + null); - if (set) - ver = setVer; + if (log.isDebugEnabled()) { + log.debug("Set value loaded from store into entry [" + + "oldVer=" + ver + + ", newVer=" + verVal.get2() + + ", entry=" + entry + ']'); + } - if (log.isDebugEnabled()) - log.debug("Set value loaded from store into entry [set=" + set + - ", curVer=" + ver + ", newVer=" + setVer + ", " + - "entry=" + entry + ']'); + ver = verVal.get2(); break; } @@ -1232,7 +1235,8 @@ private Collection enlistRead( transformClo, resolveTaskName(), null, - txEntry.keepBinary()); + txEntry.keepBinary(), + null); if (res != null) { val = res.value(); @@ -1316,7 +1320,8 @@ private Collection enlistRead( null, resolveTaskName(), accessPlc, - !deserializeBinary) : null; + !deserializeBinary, + null) : null; if (res != null) { val = res.value(); @@ -1666,7 +1671,8 @@ private IgniteInternalFuture> checkMissed( transformClo, resolveTaskName(), null, - txEntry.keepBinary()); + txEntry.keepBinary(), + null); if (res != null) { val = res.value(); @@ -2390,7 +2396,8 @@ private boolean enlistWriteEntry(GridCacheContext cacheCtx, entryProcessor, resolveTaskName(), null, - keepBinary) : null; + keepBinary, + null) : null; if (res != null) { old = res.value(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java index b03e9c89c5b3b..8db68b4ef50fb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java @@ -430,7 +430,8 @@ void recheckLock() { UUID subjId, String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, - boolean keepBinary) throws IgniteCheckedException, GridCacheEntryRemovedException { + boolean keepBinary, + @Nullable ReaderArguments args) throws IgniteCheckedException, GridCacheEntryRemovedException { assert false; return null; @@ -448,7 +449,8 @@ void recheckLock() { Object transformClo, String taskName, @Nullable IgniteCacheExpiryPolicy expiryPlc, - boolean keepBinary) { + boolean keepBinary, + @Nullable ReaderArguments readerArgs) { assert false; return null; @@ -684,9 +686,10 @@ void recheckLock() { } /** @inheritDoc */ - @Override public GridCacheVersion versionedValue(CacheObject val, + @Override public T2 versionedValue(CacheObject val, GridCacheVersion curVer, - GridCacheVersion newVer) { + GridCacheVersion newVer, + @Nullable ReaderArguments readerArgs) { assert false; return null; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheStoreUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheStoreUpdateTest.java new file mode 100644 index 0000000000000..183b9caab4d74 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheStoreUpdateTest.java @@ -0,0 +1,466 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.near; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import javax.cache.Cache; +import javax.cache.configuration.Factory; +import javax.cache.integration.CacheLoaderException; +import javax.cache.integration.CacheWriterException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.cache.store.CacheStoreAdapter; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; + +/** + * Check that near cache is updated when entry loaded from store. + */ +public class GridNearCacheStoreUpdateTest extends GridCommonAbstractTest { + /** */ + private static final String CACHE_NAME = "cache"; + + /** */ + private Ignite srv; + + /** */ + private Ignite client; + + /** */ + private IgniteCache cache; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception { + final IgniteConfiguration cfg = super.getConfiguration(gridName); + + if (gridName.contains("client")) + cfg.setClientMode(true); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + srv = startGrid("server"); + client = startGrid("client"); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If fail. + */ + public void testAtomicUpdateNear() throws Exception { + cache = client.createCache(cacheConfiguration(), new NearCacheConfiguration()); + + checkNear(null, null); + } + + /** + * @throws Exception If fail. + */ + public void testTransactionAtomicUpdateNear() throws Exception { + cache = client.createCache(cacheConfiguration(), new NearCacheConfiguration()); + + checkNear(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ); + } + + /** + * @throws Exception If fail. + */ + public void testPessimisticRepeatableReadUpdateNear() throws Exception { + cache = client.createCache(cacheConfiguration().setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL), + new NearCacheConfiguration()); + + checkNear(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ); + } + + /** + * @throws Exception If fail. + */ + public void testPessimisticReadCommittedUpdateNear() throws Exception { + cache = client.createCache(cacheConfiguration().setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL), + new NearCacheConfiguration()); + + checkNear(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.READ_COMMITTED); + } + + /** + * @throws Exception If fail. + */ + public void testOptimisticSerializableUpdateNear() throws Exception { + cache = client.createCache(cacheConfiguration().setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL), + new NearCacheConfiguration()); + + checkNear(TransactionConcurrency.OPTIMISTIC, TransactionIsolation.SERIALIZABLE); + } + + /** + * @param txConc Transaction concurrency. + * @param txIsolation Transaction isolation. + * @throws Exception If fail. + */ + private void checkNear(TransactionConcurrency txConc, TransactionIsolation txIsolation) throws Exception { + checkNearSingle(txConc, txIsolation); + checkNearSingleConcurrent(txConc, txIsolation); + checkNearBatch(txConc, txIsolation); + checkNearBatchConcurrent(txConc, txIsolation); + } + + /** + * @param txConc Transaction concurrency. + * @param txIsolation Transaction isolation. + * @throws Exception If fail. + */ + private void checkNearSingle(TransactionConcurrency txConc, TransactionIsolation txIsolation) throws Exception { + final String key = "key"; + + boolean tx = txConc != null && txIsolation != null; + + final IgniteCache clientCache = this.cache; + final IgniteCache srvCache = srv.cache(CACHE_NAME); + + if (tx) { + doInTransaction(client, txConc, txIsolation, new Callable() { + @Override public Object call() throws Exception { + // Read from store. + assertEquals(key, clientCache.get(key)); + + return null; + } + }); + } + else + assertEquals(key, clientCache.get(key)); + + final String updatedVal = "key_updated"; + + if (tx) { + doInTransaction(srv, txConc, txIsolation, new Callable() { + @Override public Object call() throws Exception { + // Update value. + srvCache.put(key, updatedVal); + + return null; + } + }); + } + else + srvCache.put(key, updatedVal); + + if (tx) { + doInTransaction(client, txConc, txIsolation, new Callable() { + @Override public Object call() throws Exception { + assertEquals(updatedVal, clientCache.get(key)); + + return null; + } + }); + } + else + assertEquals(updatedVal, clientCache.get(key)); + } + + /** + * @param txConc Transaction concurrency. + * @param txIsolation Transaction isolation. + * @throws Exception If fail. + */ + private void checkNearSingleConcurrent(final TransactionConcurrency txConc, final TransactionIsolation txIsolation) throws Exception { + for (int i = 0; i < 10; i++) { + final String key = String.valueOf(-((new Random().nextInt(99) + 1))); + + boolean tx = txConc != null && txIsolation != null; + + final IgniteCache clientCache = this.cache; + final IgniteCache srvCache = srv.cache(CACHE_NAME); + + final CountDownLatch storeLatch = new CountDownLatch(1); + + final IgniteInternalFuture fut1 = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + storeLatch.await(); + + clientCache.get(key); + + return null; + } + }); + + +// IgniteInternalFuture fut2 = null; + + // TODO Sometimes Near cache becomes inconsistent +// if (!tx) { +// // TODO: IGNITE-3498 +// // TODO: Doesn't work on transactional cache. +// fut2 = GridTestUtils.runAsync(new Callable() { +// @Override public Object call() throws Exception { +// storeLatch.await(); +// +// srvCache.remove(key); +// +// return null; +// } +// }); +// } + + final IgniteInternalFuture fut3 = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + storeLatch.await(); + + srvCache.put(key, "other"); + + return null; + } + }); + + storeLatch.countDown(); + + fut1.get(); + +// if (!tx) +// fut2.get(); + + fut3.get(); + + final String srvVal = srvCache.get(key); + final String clientVal = clientCache.get(key); + + assertEquals(srvVal, clientVal); + } + } + + /** + * @param txConc Transaction concurrency. + * @param txIsolation Transaction isolation. + * @throws Exception If fail. + */ + private void checkNearBatch(TransactionConcurrency txConc, TransactionIsolation txIsolation) throws Exception { + final Map data1 = new HashMap<>(); + final Map data2 = new HashMap<>(); + + for (int i = 0; i < 10; i++) { + data1.put(String.valueOf(i), String.valueOf(i)); + data2.put(String.valueOf(i), "other"); + } + + final IgniteCache clientCache = this.cache; + final IgniteCache srvCache = srv.cache(CACHE_NAME); + + boolean tx = txConc != null && txIsolation != null; + + // Read from store. + if (tx) { + doInTransaction(client, txConc, txIsolation, new Callable() { + @Override public Object call() throws Exception { + assertEquals(data1, clientCache.getAll(data1.keySet())); + + return null; + } + }); + } + else + assertEquals(data1, clientCache.getAll(data1.keySet())); + + // Update value. + if (tx) { + doInTransaction(srv, txConc, txIsolation, new Callable() { + @Override public Object call() throws Exception { + srvCache.putAll(data2); + + return null; + } + }); + } + else + srvCache.putAll(data2); + + if (tx) { + doInTransaction(client, txConc, txIsolation, new Callable() { + @Override public Object call() throws Exception { + assertEquals(data2, clientCache.getAll(data2.keySet())); + + return null; + } + }); + } + else + assertEquals(data2, clientCache.getAll(data2.keySet())); + } + + /** + * @param txConc Transaction concurrency. + * @param txIsolation Transaction isolation. + * @throws Exception If fail. + */ + private void checkNearBatchConcurrent(TransactionConcurrency txConc, TransactionIsolation txIsolation) + throws Exception { + final Map data1 = new HashMap<>(); + final Map data2 = new HashMap<>(); + + for (int j = 0; j < 10; j++) { + data1.clear(); + data2.clear(); + + for (int i = j * 10; i < j * 10 + 10; i++) { + data1.put(String.valueOf(i), String.valueOf(i)); + data2.put(String.valueOf(i), "other"); + } + + final IgniteCache clientCache = this.cache; + final IgniteCache srvCache = srv.cache(CACHE_NAME); + + boolean tx = txConc != null && txIsolation != null; + + final CountDownLatch latch = new CountDownLatch(1); + + final IgniteInternalFuture fut1 = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + latch.await(); + + clientCache.getAll(data1.keySet()); + + return null; + } + }); + + IgniteInternalFuture fut2 = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + latch.await(); + + srvCache.putAll(data2); + + return null; + } + }); + +// IgniteInternalFuture fut3 = null; +// +// // TODO Sometimes Near cache becomes inconsistent +// if (!tx) { +// // TODO: IGNITE-3498 +// // TODO: Doesn't work on transactional cache. +// fut3 = GridTestUtils.runAsync(new Callable() { +// @Override public Object call() throws Exception { +// latch.await(); +// +// srvCache.removeAll(data1.keySet()); +// +// return null; +// } +// }); +// } + + latch.countDown(); + +// if (!tx) +// fut3.get(); + + fut1.get(); + fut2.get(); + + final Map srvVals = srvCache.getAll(data1.keySet()); + final Map clientVals = clientCache.getAll(data1.keySet()); + + assertEquals(srvVals, clientVals); + } + } + + /** + * @return Cache configuration. + */ + protected CacheConfiguration cacheConfiguration() { + CacheConfiguration cfg = new CacheConfiguration<>(CACHE_NAME); + + cfg.setCacheStoreFactory(new StoreFactory()); + + cfg.setReadThrough(true); + cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + + return cfg; + } + + /** + * + */ + private static class StoreFactory implements Factory> { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public CacheStore create() { + return new TestStore(); + } + } + + /** + * + */ + private static class TestStore extends CacheStoreAdapter implements Serializable { + /** */ + private final ConcurrentHashMap map = new ConcurrentHashMap<>(); + + /** */ + private static final long serialVersionUID = 0L; + + /** + * + */ + public TestStore() { + for (int i = -100; i < 1000; i++) + map.put(String.valueOf(i), String.valueOf(i)); + + map.put("key", "key"); + } + + /** {@inheritDoc} */ + @Override public String load(String key) throws CacheLoaderException { + return map.get(key); + } + + /** {@inheritDoc} */ + @Override public void write(Cache.Entry entry) throws CacheWriterException { + map.put(entry.getKey(), entry.getValue()); + } + + /** {@inheritDoc} */ + @SuppressWarnings("SuspiciousMethodCalls") + @Override public void delete(Object key) throws CacheWriterException { + map.remove(key); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffheapCacheStoreUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffheapCacheStoreUpdateTest.java new file mode 100644 index 0000000000000..ae3f695d0a443 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffheapCacheStoreUpdateTest.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed.near; + +import org.apache.ignite.cache.CacheMemoryMode; +import org.apache.ignite.configuration.CacheConfiguration; + +/** + * Check that near cache is updated when entry loaded from store. + */ +public class GridNearOffheapCacheStoreUpdateTest extends GridNearCacheStoreUpdateTest { + /** {@inheritDoc} */ + @Override protected CacheConfiguration cacheConfiguration() { + final CacheConfiguration ccfg = super.cacheConfiguration(); + + ccfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED); + + return ccfg; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java index 8792ea16ac9af..af46c5783cfc7 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java @@ -114,8 +114,10 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedTxTimeoutSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheRendezvousAffinityClientSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridPartitionedBackupLoadSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheStoreUpdateTest; import org.apache.ignite.internal.processors.cache.distributed.near.NearCacheSyncUpdateTest; import org.apache.ignite.internal.processors.cache.distributed.near.NoneRebalanceModeSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearOffheapCacheStoreUpdateTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedEvictionSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedJobExecutionTest; import org.apache.ignite.internal.processors.cache.local.GridCacheLocalAtomicBasicStoreSelfTest; @@ -271,6 +273,9 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(OffheapCacheOnClientsTest.class)); suite.addTest(new TestSuite(CacheConcurrentReadThroughTest.class)); + suite.addTest(new TestSuite(GridNearCacheStoreUpdateTest.class)); + suite.addTest(new TestSuite(GridNearOffheapCacheStoreUpdateTest.class)); + return suite; } } From 2eb24cad277e14322cf42155697cae78e0f80e13 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 18 Jan 2017 13:00:25 +0300 Subject: [PATCH 427/487] ignite-4147 - Fail if joining node has different of cluster SSL configuration. --- .../ignite/spi/discovery/tcp/ClientImpl.java | 20 +++- .../ignite/spi/discovery/tcp/ServerImpl.java | 9 ++ .../TcpDiscoverySslSecuredUnsecuredTest.java | 93 +++++++++++++++++++ .../IgniteSpiDiscoverySelfTestSuite.java | 4 +- 4 files changed, 124 insertions(+), 2 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 0f5f74181516e..9a1261c52a6fa 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -20,6 +20,7 @@ import java.io.BufferedInputStream; import java.io.IOException; import java.io.InputStream; +import java.io.StreamCorruptedException; import java.net.InetSocketAddress; import java.net.Socket; import java.net.SocketTimeoutException; @@ -44,6 +45,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.atomic.AtomicReference; +import javax.net.ssl.SSLException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteException; @@ -655,6 +657,14 @@ else if (addrs.isEmpty()) { errs.add(e); + if (X.hasCause(e, SSLException.class)) + throw new IgniteSpiException("Unable to establish secure connection. " + + "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); + + if (X.hasCause(e, StreamCorruptedException.class)) + throw new IgniteSpiException("Unable to establish plain connection. " + + "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); + if (timeoutHelper.checkFailureTimeoutReached(e)) break; @@ -1527,7 +1537,15 @@ private void tryJoin() throws InterruptedException { joinCnt++; - T2 joinRes = joinTopology(false, spi.joinTimeout); + T2 joinRes; + try { + joinRes = joinTopology(false, spi.joinTimeout); + } + catch (IgniteSpiException e) { + joinError(e); + + return; + } if (joinRes == null) { if (join) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index c79133379c267..40da281cbc916 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -24,6 +24,7 @@ import java.io.ObjectStreamException; import java.io.OutputStream; import java.io.Serializable; +import java.io.StreamCorruptedException; import java.net.ConnectException; import java.net.InetAddress; import java.net.InetSocketAddress; @@ -1218,6 +1219,14 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) errs.add(e); + if (X.hasCause(e, SSLException.class)) + throw new IgniteException("Unable to establish secure connection. " + + "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); + + if (X.hasCause(e, StreamCorruptedException.class)) + throw new IgniteException("Unable to establish plain connection. " + + "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); + if (timeoutHelper.checkFailureTimeoutReached(e)) break; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java new file mode 100644 index 0000000000000..229616512cdf8 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java @@ -0,0 +1,93 @@ +/* + * 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.spi.discovery.tcp; + +import java.util.concurrent.Callable; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests cases when node connects to cluster with different SSL configuration. + * Exception with meaningful message should be thrown. + */ +public class TcpDiscoverySslSecuredUnsecuredTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception { + final IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setClientMode(gridName.contains("client")); + + if (gridName.contains("ssl")) + cfg.setSslContextFactory(GridTestUtils.sslFactory()); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testSecuredUnsecuredServerConnection() throws Exception { + checkConnection("plain-server", "ssl-server"); + } + + /** + * @throws Exception If failed. + */ + public void testUnsecuredSecuredServerConnection() throws Exception { + checkConnection("ssl-server", "plain-server"); + } + + /** + * @throws Exception If failed. + */ + public void testSecuredClientUnsecuredServerConnection() throws Exception { + checkConnection("plain-server", "ssl-client"); + } + + /** + * @throws Exception If failed. + */ + public void testUnsecuredClientSecuredServerConnection() throws Exception { + checkConnection("ssl-server", "plain-client"); + } + + /** + * @param name1 First grid name. + * @param name2 Second grid name. + * @throws Exception If failed. + */ + @SuppressWarnings("ThrowableNotThrown") + private void checkConnection(final String name1, final String name2) throws Exception { + startGrid(name1); + + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + startGrid(name2); + + return null; + } + }, IgniteCheckedException.class, null); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java index af7eb7e8e2403..98bf6daa06c03 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java @@ -34,6 +34,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiFailureTimeoutSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiStartStopSelfTest; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySslSecuredUnsecuredTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySslSelfTest; import org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinderSelfTest; import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinderSelfTest; @@ -86,7 +87,8 @@ public static TestSuite suite() throws Exception { // SSL. suite.addTest(new TestSuite(TcpDiscoverySslSelfTest.class)); + suite.addTest(new TestSuite(TcpDiscoverySslSecuredUnsecuredTest.class)); return suite; } -} \ No newline at end of file +} From 2305e38345d8a7ca812d265d00eaca5bb7d6adb1 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 18 Jan 2017 14:57:53 +0300 Subject: [PATCH 428/487] Minor fixes in tests. --- .../processors/cache/IgniteCacheAbstractQuerySelfTest.java | 4 ++-- ...iteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java | 3 ++- ...acheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java | 3 ++- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java index c5a241e54ce1f..9f56877769313 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java @@ -642,7 +642,7 @@ public void testSimpleCustomTableName() throws Exception { return null; } - }, IgniteException.class, null); + }, CacheException.class, null); } /** @@ -688,7 +688,7 @@ public void testMixedCustomTableName() throws Exception { return null; } - }, IgniteException.class, null); + }, CacheException.class, null); } /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java index 80c4a086a09d2..50fb03449b050 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java @@ -32,6 +32,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.cache.query.QueryCancelledException; +import org.apache.ignite.internal.processors.GridProcessor; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.U; @@ -244,7 +245,7 @@ private void checkCleanState() { IgniteEx grid = grid(i); // Validate everything was cleaned up. - ConcurrentMap map = U.field(((IgniteH2Indexing)U.field(U.field( + ConcurrentMap map = U.field(((IgniteH2Indexing)U.field((GridProcessor)U.field( grid.context(), "qryProc"), "idx")).mapQueryExecutor(), "qryRess"); String msg = "Map executor state is not cleared"; diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java index 4baaf8f650d81..3263b4155f55c 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java @@ -29,6 +29,7 @@ import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.GridProcessor; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; import org.apache.ignite.internal.util.typedef.internal.U; @@ -122,7 +123,7 @@ private void checkCleanState() { IgniteEx grid = grid(i); // Validate everything was cleaned up. - ConcurrentMap map = U.field(((IgniteH2Indexing) U.field(U.field( + ConcurrentMap map = U.field(((IgniteH2Indexing)U.field((GridProcessor)U.field( grid.context(), "qryProc"), "idx")).mapQueryExecutor(), "qryRess"); String msg = "Map executor state is not cleared"; From ecf4b8b5bd05a5c1120e08d9951cddd26d0e924c Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 18 Jan 2017 17:22:14 +0300 Subject: [PATCH 429/487] Minor fix test for Ignite-4247. --- .../cache/IgniteCacheAbstractQuerySelfTest.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java index 9f56877769313..7e0d20b76bdb6 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java @@ -633,7 +633,7 @@ public void testSimpleCustomTableName() throws Exception { assertEquals(2, qry.getAll().size()); - GridTestUtils.assertThrows(log, new GridPlainCallable() { + Throwable throwable = GridTestUtils.assertThrowsInherited(log, new GridPlainCallable() { @Override public Void call() throws Exception { QueryCursor> qry = cache.query(new SqlQuery(Type1.class, "FROM Type1")); @@ -642,7 +642,11 @@ public void testSimpleCustomTableName() throws Exception { return null; } - }, CacheException.class, null); + }, RuntimeException.class, null); + + assertNotNull(throwable); + + assertTrue(throwable instanceof IgniteException || throwable instanceof CacheException); } /** From f9aaf0353cea54afefea4caac74b1583eb17969b Mon Sep 17 00:00:00 2001 From: agura Date: Wed, 18 Jan 2017 18:04:45 +0300 Subject: [PATCH 430/487] ignite-4499 Drop node from topology in case when connection creation is impossible --- .../apache/ignite/IgniteSystemProperties.java | 3 + .../tcp/TcpCommunicationSpi.java | 16 + .../ignite/spi/discovery/tcp/ClientImpl.java | 88 ++++- .../ignite/spi/discovery/tcp/ServerImpl.java | 61 ++-- .../messages/TcpDiscoveryAbstractMessage.java | 21 ++ .../tcp/TcpCommunicationSpiDropNodesTest.java | 322 ++++++++++++++++++ .../TcpCommunicationSpiFaultyClientTest.java | 270 +++++++++++++++ .../ignite/testframework/GridTestNode.java | 1 + .../junits/GridAbstractTest.java | 2 + .../IgniteSpiCommunicationSelfTestSuite.java | 5 + .../IgniteCacheAbstractQuerySelfTest.java | 8 +- 11 files changed, 758 insertions(+), 39 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiDropNodesTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 0da0f4990912a..d77b2fb2838d4 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -65,6 +65,9 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_NO_DISCO_ORDER = "IGNITE_NO_DISCO_ORDER"; + /** Defines reconnect delay in milliseconds for client node that was failed forcible. */ + public static final String IGNITE_DISCO_FAILED_CLIENT_RECONNECT_DELAY = "IGNITE_DISCO_FAILED_CLIENT_RECONNECT_DELAY"; + /** * If this system property is set to {@code false} - no checks for new versions will * be performed by Ignite. By default, Ignite periodically checks for the new diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 1fe437cc710c0..94b7efe078e84 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -94,6 +94,7 @@ import org.apache.ignite.internal.util.typedef.CI2; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -2550,6 +2551,21 @@ else if (X.hasCause(e, SocketTimeoutException.class)) "operating system firewall is disabled on local and remote hosts) " + "[addrs=" + addrs + ']'); + if (getSpiContext().node(node.id()) != null && (CU.clientNode(node) || !CU.clientNode(getLocalNode())) && + X.hasCause(errs, ConnectException.class, SocketTimeoutException.class, HandshakeTimeoutException.class, + IgniteSpiOperationTimeoutException.class)) { + LT.warn(log, "TcpCommunicationSpi failed to establish connection to node, node will be dropped from " + + "cluster [" + + "rmtNode=" + node + + ", err=" + errs + + ", connectErrs=" + Arrays.toString(errs.getSuppressed()) + ']'); + + getSpiContext().failNode(node.id(), "TcpCommunicationSpi failed to establish connection to node [" + + "rmtNode=" + node + + ", errs=" + errs + + ", connectErrs=" + Arrays.toString(errs.getSuppressed()) + ']'); + } + throw errs; } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 9a1261c52a6fa..35f0908a20b34 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -50,6 +50,7 @@ import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.CacheMetrics; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.cluster.ClusterNode; @@ -100,6 +101,7 @@ import org.jsr166.ConcurrentHashMap8; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCO_FAILED_CLIENT_RECONNECT_DELAY; import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED; import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED; import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; @@ -166,6 +168,9 @@ class ClientImpl extends TcpDiscoveryImpl { /** */ protected MessageWorker msgWorker; + /** Force fail message for local node. */ + private TcpDiscoveryNodeFailedMessage forceFailMsg; + /** */ @GridToStringExclude private int joinCnt; @@ -450,6 +455,8 @@ else if (state == DISCONNECTED) { msg.warning(warning); + msg.force(true); + msgWorker.addMessage(msg); } } @@ -1396,6 +1403,14 @@ else if (msg == SPI_STOP) { else leaveLatch.countDown(); } + else if (msg instanceof TcpDiscoveryNodeFailedMessage && + ((TcpDiscoveryNodeFailedMessage)msg).failedNodeId().equals(locNode.id())) { + TcpDiscoveryNodeFailedMessage msg0 = (TcpDiscoveryNodeFailedMessage)msg; + + assert msg0.force() : msg0; + + forceFailMsg = msg0; + } else if (msg instanceof SocketClosedMessage) { if (((SocketClosedMessage)msg).sock == currSock) { currSock = null; @@ -1412,25 +1427,45 @@ else if (msg instanceof SocketClosedMessage) { } } else { - if (log.isDebugEnabled()) - log.debug("Connection closed, will try to restore connection."); + if (forceFailMsg != null) { + if (log.isDebugEnabled()) { + log.debug("Connection closed, local node received force fail message, " + + "will not try to restore connection"); + } + + queue.addFirst(SPI_RECONNECT_FAILED); + } + else { + if (log.isDebugEnabled()) + log.debug("Connection closed, will try to restore connection."); - assert reconnector == null; + assert reconnector == null; - final Reconnector reconnector = new Reconnector(join); - this.reconnector = reconnector; - reconnector.start(); + final Reconnector reconnector = new Reconnector(join); + this.reconnector = reconnector; + reconnector.start(); + } } } } else if (msg == SPI_RECONNECT_FAILED) { - reconnector.cancel(); - reconnector.join(); + if (reconnector != null) { + reconnector.cancel(); + reconnector.join(); - reconnector = null; + reconnector = null; + } + else + assert forceFailMsg != null; if (spi.isClientReconnectDisabled()) { if (state != SEGMENTED && state != STOPPED) { + if (forceFailMsg != null) { + U.quietAndWarn(log, "Local node was dropped from cluster due to network problems " + + "[nodeInitiatedFail=" + forceFailMsg.creatorNodeId() + + ", msg=" + forceFailMsg.warning() + ']'); + } + if (log.isDebugEnabled()) { log.debug("Failed to restore closed connection, reconnect disabled, " + "local node segmented [networkTimeout=" + spi.netTimeout + ']'); @@ -1445,7 +1480,9 @@ else if (msg == SPI_RECONNECT_FAILED) { if (state == STARTING || state == CONNECTED) { if (log.isDebugEnabled()) { log.debug("Failed to restore closed connection, will try to reconnect " + - "[networkTimeout=" + spi.netTimeout + ", joinTimeout=" + spi.joinTimeout + ']'); + "[networkTimeout=" + spi.netTimeout + + ", joinTimeout=" + spi.joinTimeout + + ", failMsg=" + forceFailMsg + ']'); } state = DISCONNECTED; @@ -1468,7 +1505,36 @@ else if (msg == SPI_RECONNECT_FAILED) { UUID newId = UUID.randomUUID(); - if (log.isInfoEnabled()) { + if (forceFailMsg != null) { + long delay = IgniteSystemProperties.getLong(IGNITE_DISCO_FAILED_CLIENT_RECONNECT_DELAY, + 10_000); + + if (delay > 0) { + U.quietAndWarn(log, "Local node was dropped from cluster due to network problems, " + + "will try to reconnect with new id after " + delay + "ms (reconnect delay " + + "can be changed using IGNITE_DISCO_FAILED_CLIENT_RECONNECT_DELAY system " + + "property) [" + + "newId=" + newId + + ", prevId=" + locNode.id() + + ", locNode=" + locNode + + ", nodeInitiatedFail=" + forceFailMsg.creatorNodeId() + + ", msg=" + forceFailMsg.warning() + ']'); + + Thread.sleep(delay); + } + else { + U.quietAndWarn(log, "Local node was dropped from cluster due to network problems, " + + "will try to reconnect with new id [" + + "newId=" + newId + + ", prevId=" + locNode.id() + + ", locNode=" + locNode + + ", nodeInitiatedFail=" + forceFailMsg.creatorNodeId() + + ", msg=" + forceFailMsg.warning() + ']'); + } + + forceFailMsg = null; + } + else if (log.isInfoEnabled()) { log.info("Client node disconnected from cluster, will try to reconnect with new id " + "[newId=" + newId + ", prevId=" + locNode.id() + ", locNode=" + locNode + ']'); } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 40da281cbc916..f33566cc01a5a 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -775,6 +775,8 @@ private void interruptPing(TcpDiscoveryNode node) { msg.warning(warning); + msg.force(true); + msgWorker.addMessage(msg); } } @@ -4610,8 +4612,12 @@ private void processNodeFailedMessage(TcpDiscoveryNodeFailedMessage msg) { else { boolean contains; + UUID creatorId = msg.creatorNodeId(); + + assert creatorId != null : msg; + synchronized (mux) { - contains = failedNodes.containsKey(sndNode); + contains = failedNodes.containsKey(sndNode) || ring.node(creatorId) == null; } if (contains) { @@ -4623,25 +4629,29 @@ private void processNodeFailedMessage(TcpDiscoveryNodeFailedMessage msg) { } } - UUID nodeId = msg.failedNodeId(); + UUID failedNodeId = msg.failedNodeId(); long order = msg.order(); - TcpDiscoveryNode node = ring.node(nodeId); + TcpDiscoveryNode failedNode = ring.node(failedNodeId); - if (node != null && node.internalOrder() != order) { + if (failedNode != null && failedNode.internalOrder() != order) { if (log.isDebugEnabled()) log.debug("Ignoring node failed message since node internal order does not match " + - "[msg=" + msg + ", node=" + node + ']'); + "[msg=" + msg + ", node=" + failedNode + ']'); return; } - if (node != null) { - assert !node.isLocal() || !msg.verified() : msg; + if (failedNode != null) { + assert !failedNode.isLocal() || !msg.verified() : msg; - synchronized (mux) { - if (!failedNodes.containsKey(node)) - failedNodes.put(node, msg.senderNodeId() != null ? msg.senderNodeId() : getLocalNodeId()); + boolean skipUpdateFailedNodes = msg.force() && !msg.verified(); + + if (!skipUpdateFailedNodes) { + synchronized (mux) { + if (!failedNodes.containsKey(failedNode)) + failedNodes.put(failedNode, msg.senderNodeId() != null ? msg.senderNodeId() : getLocalNodeId()); + } } } else { @@ -4668,11 +4678,11 @@ private void processNodeFailedMessage(TcpDiscoveryNodeFailedMessage msg) { } if (msg.verified()) { - node = ring.removeNode(nodeId); + failedNode = ring.removeNode(failedNodeId); - interruptPing(node); + interruptPing(failedNode); - assert node != null; + assert failedNode != null; long topVer; @@ -4698,16 +4708,18 @@ private void processNodeFailedMessage(TcpDiscoveryNodeFailedMessage msg) { } synchronized (mux) { - failedNodes.remove(node); + failedNodes.remove(failedNode); - leavingNodes.remove(node); + leavingNodes.remove(failedNode); - failedNodesMsgSent.remove(node.id()); + failedNodesMsgSent.remove(failedNode.id()); - ClientMessageWorker worker = clientMsgWorkers.remove(node.id()); + if (!msg.force()) { // ClientMessageWorker will stop after sending force fail message. + ClientMessageWorker worker = clientMsgWorkers.remove(failedNode.id()); - if (worker != null) - worker.interrupt(); + if (worker != null) + worker.interrupt(); + } } if (msg.warning() != null && !msg.creatorNodeId().equals(getLocalNodeId())) { @@ -4719,10 +4731,10 @@ private void processNodeFailedMessage(TcpDiscoveryNodeFailedMessage msg) { } synchronized (mux) { - joiningNodes.remove(node.id()); + joiningNodes.remove(failedNode.id()); } - notifyDiscovery(EVT_NODE_FAILED, topVer, node); + notifyDiscovery(EVT_NODE_FAILED, topVer, failedNode); spi.stats.onNodeFailed(); } @@ -6317,7 +6329,12 @@ else if (msgLog.isDebugEnabled()) spi.failureDetectionTimeout() : spi.getSocketTimeout()); } - success = true; + boolean clientFailed = msg instanceof TcpDiscoveryNodeFailedMessage && + ((TcpDiscoveryNodeFailedMessage)msg).failedNodeId().equals(clientNodeId); + + assert !clientFailed || msg.force() : msg; + + success = !clientFailed; } catch (IgniteCheckedException | IOException e) { if (log.isDebugEnabled()) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java index 783a113ea1ff6..e982b2f32ea31 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java @@ -48,6 +48,9 @@ public abstract class TcpDiscoveryAbstractMessage implements Serializable { /** */ protected static final int CLIENT_ACK_FLAG_POS = 4; + /** */ + protected static final int FORCE_FAIL_FLAG_POS = 8; + /** Sender of the message (transient). */ private transient UUID sndNodeId; @@ -204,6 +207,24 @@ public void client(boolean client) { setFlag(CLIENT_FLAG_POS, client); } + /** + * Get force fail node flag. + * + * @return Force fail node flag. + */ + public boolean force() { + return getFlag(FORCE_FAIL_FLAG_POS); + } + + /** + * Sets force fail node flag. + * + * @param force Force fail node flag. + */ + public void force(boolean force) { + setFlag(FORCE_FAIL_FLAG_POS, force); + } + /** * @return Pending message index. */ diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiDropNodesTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiDropNodesTest.java new file mode 100644 index 0000000000000..d29231e449b0b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiDropNodesTest.java @@ -0,0 +1,322 @@ +/* + * 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.spi.communication.tcp; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.TimeUnit; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.Event; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.nio.GridCommunicationClient; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; + +/** + * + */ +public class TcpCommunicationSpiDropNodesTest extends GridCommonAbstractTest { + /** IP finder. */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** Nodes count. */ + private static final int NODES_CNT = 4; + + /** Block. */ + private static volatile boolean block; + + /** Predicate. */ + private static IgniteBiPredicate pred; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setClockSyncFrequency(300000); + cfg.setFailureDetectionTimeout(1000); + + TestCommunicationSpi spi = new TestCommunicationSpi(); + + spi.setIdleConnectionTimeout(100); + spi.setSharedMemoryPort(-1); + + TcpDiscoverySpi discoSpi = (TcpDiscoverySpi) cfg.getDiscoverySpi(); + discoSpi.setIpFinder(IP_FINDER); + + cfg.setCommunicationSpi(spi); + cfg.setDiscoverySpi(discoSpi); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + block = false; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testOneNode() throws Exception { + pred = new IgniteBiPredicate() { + @Override public boolean apply(ClusterNode locNode, ClusterNode rmtNode) { + return block && rmtNode.order() == 3; + } + }; + + startGrids(NODES_CNT); + + final CountDownLatch latch = new CountDownLatch(1); + + grid(0).events().localListen(new IgnitePredicate() { + @Override + public boolean apply(Event event) { + latch.countDown(); + + return true; + } + }, EVT_NODE_FAILED); + + U.sleep(1000); // Wait for write timeout and closing idle connections. + + block = true; + + grid(0).compute().broadcast(new IgniteRunnable() { + @Override public void run() { + // No-op. + } + }); + + assertTrue(latch.await(15, TimeUnit.SECONDS)); + + assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return grid(3).cluster().topologyVersion() == NODES_CNT + 1; + } + }, 5000)); + + for (int i = 0; i < 10; i++) { + U.sleep(1000); + + assertEquals(NODES_CNT - 1, grid(0).cluster().nodes().size()); + + int liveNodesCnt = 0; + + for (int j = 0; j < NODES_CNT; j++) { + IgniteEx ignite; + + try { + ignite = grid(j); + + log.info("Checking topology for grid(" + j + "): " + ignite.cluster().nodes()); + + ClusterNode locNode = ignite.localNode(); + + if (locNode.order() != 3) { + assertEquals(NODES_CNT - 1, ignite.cluster().nodes().size()); + + for (ClusterNode node : ignite.cluster().nodes()) + assertTrue(node.order() != 3); + + liveNodesCnt++; + } + } + catch (Exception e) { + log.info("Checking topology for grid(" + j + "): no grid in topology."); + } + } + + assertEquals(NODES_CNT - 1, liveNodesCnt); + } + } + + /** + * @throws Exception If failed. + */ + public void testTwoNodesEachOther() throws Exception { + pred = new IgniteBiPredicate() { + @Override public boolean apply(ClusterNode locNode, ClusterNode rmtNode) { + return block && (locNode.order() == 2 || locNode.order() == 4) && + (rmtNode.order() == 2 || rmtNode.order() == 4); + } + }; + + startGrids(NODES_CNT); + + final CountDownLatch latch = new CountDownLatch(1); + + grid(0).events().localListen(new IgnitePredicate() { + @Override + public boolean apply(Event event) { + latch.countDown(); + + return true; + } + }, EVT_NODE_FAILED); + + U.sleep(1000); // Wait for write timeout and closing idle connections. + + block = true; + + final CyclicBarrier barrier = new CyclicBarrier(2); + + IgniteInternalFuture fut1 = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + barrier.await(); + + grid(1).compute().withNoFailover().broadcast(new IgniteRunnable() { + @Override public void run() { + // No-op. + } + }); + + return null; + } + }); + + IgniteInternalFuture fut2 = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + barrier.await(); + + grid(3).compute().withNoFailover().broadcast(new IgniteRunnable() { + @Override public void run() { + // No-op. + } + }); + + return null; + } + }); + + assertTrue(latch.await(5, TimeUnit.SECONDS)); + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return grid(2).cluster().nodes().size() == NODES_CNT - 1; + } + }, 5000); + + try { + fut1.get(); + } + catch (IgniteCheckedException e) { + // No-op. + } + + try { + fut2.get(); + } + catch (IgniteCheckedException e) { + // No-op. + } + + long failedNodeOrder = 1 + 2 + 3 + 4; + + for (ClusterNode node : grid(0).cluster().nodes()) + failedNodeOrder -= node.order(); + + for (int i = 0; i < 10; i++) { + U.sleep(1000); + + assertEquals(NODES_CNT - 1, grid(0).cluster().nodes().size()); + + int liveNodesCnt = 0; + + for (int j = 0; j < NODES_CNT; j++) { + IgniteEx ignite; + + try { + ignite = grid(j); + + log.info("Checking topology for grid(" + j + "): " + ignite.cluster().nodes()); + + ClusterNode locNode = ignite.localNode(); + + if (locNode.order() != failedNodeOrder) { + assertEquals(NODES_CNT - 1, ignite.cluster().nodes().size()); + + for (ClusterNode node : ignite.cluster().nodes()) + assertTrue(node.order() != failedNodeOrder); + + liveNodesCnt++; + } + } + catch (Exception e) { + log.info("Checking topology for grid(" + j + "): no grid in topology."); + } + } + + assertEquals(NODES_CNT - 1, liveNodesCnt); + } + } + + /** + * + */ + private static class TestCommunicationSpi extends TcpCommunicationSpi { + /** {@inheritDoc} */ + @Override protected GridCommunicationClient createTcpClient(ClusterNode node) throws IgniteCheckedException { + if (pred.apply(getLocalNode(), node)) { + Map attrs = new HashMap<>(node.attributes()); + + attrs.put(createAttributeName(ATTR_ADDRS), Collections.singleton("127.0.0.1")); + attrs.put(createAttributeName(ATTR_PORT), 47200); + attrs.put(createAttributeName(ATTR_EXT_ADDRS), Collections.emptyList()); + attrs.put(createAttributeName(ATTR_HOST_NAMES), Collections.emptyList()); + + ((TcpDiscoveryNode)node).setAttributes(attrs); + } + + return super.createTcpClient(node); + } + + /** + * @param name Name. + */ + private String createAttributeName(String name) { + return getClass().getSimpleName() + '.' + name; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java new file mode 100644 index 0000000000000..6e99487bd46ae --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiFaultyClientTest.java @@ -0,0 +1,270 @@ +/* + * 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.spi.communication.tcp; + +import java.io.IOException; +import java.io.OutputStream; +import java.net.InetAddress; +import java.net.ServerSocket; +import java.net.Socket; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.Event; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.nio.GridCommunicationClient; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeFailedMessage; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; + +/** + * Tests that faulty client will be failed if connection can't be established. + */ +public class TcpCommunicationSpiFaultyClientTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** Predicate. */ + private static final IgnitePredicate PRED = new IgnitePredicate() { + @Override public boolean apply(ClusterNode node) { + return block && node.order() == 3; + } + }; + + /** Client mode. */ + private static boolean clientMode; + + /** Block. */ + private static volatile boolean block; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setClockSyncFrequency(300000); + cfg.setFailureDetectionTimeout(1000); + cfg.setClientMode(clientMode); + + TestCommunicationSpi spi = new TestCommunicationSpi(); + + spi.setIdleConnectionTimeout(100); + spi.setSharedMemoryPort(-1); + + TcpDiscoverySpi discoSpi = (TcpDiscoverySpi) cfg.getDiscoverySpi(); + + discoSpi.setIpFinder(IP_FINDER); + discoSpi.setClientReconnectDisabled(true); + + cfg.setCommunicationSpi(spi); + cfg.setDiscoverySpi(discoSpi); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + block = false; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testNoServerOnHost() throws Exception { + testFailClient(null); + } + + /** + * @throws Exception If failed. + */ + public void testNotAcceptedConnection() throws Exception { + testFailClient(new FakeServer()); + } + + /** + * @param srv Server. + * @throws Exception If failed. + */ + private void testFailClient(FakeServer srv) throws Exception { + IgniteInternalFuture fut = null; + + try { + if (srv != null) + fut = GridTestUtils.runMultiThreadedAsync(srv, 1, "fake-server"); + + clientMode = false; + + startGrids(2); + + clientMode = true; + + startGrid(2); + startGrid(3); + + U.sleep(1000); // Wait for write timeout and closing idle connections. + + final CountDownLatch latch = new CountDownLatch(1); + + grid(0).events().localListen(new IgnitePredicate() { + @Override + public boolean apply(Event event) { + latch.countDown(); + + return true; + } + }, EVT_NODE_FAILED); + + block = true; + + try { + grid(0).compute(grid(0).cluster().forClients()).withNoFailover().broadcast(new IgniteRunnable() { + @Override public void run() { + // No-op. + } + }); + } + catch (IgniteException e) { + // No-op. + } + + assertTrue(latch.await(3, TimeUnit.SECONDS)); + + assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return grid(0).cluster().forClients().nodes().size() == 1; + } + }, 5000)); + + for (int i = 0; i < 5; i++) { + U.sleep(1000); + + log.info("Check topology (" + (i + 1) + "): " + grid(0).cluster().nodes()); + + assertEquals(1, grid(0).cluster().forClients().nodes().size()); + } + } + finally { + if (srv != null) { + srv.stop(); + + assert fut != null; + + fut.get(); + } + + stopAllGrids(); + } + } + + /** + * Server that emulates connection troubles. + */ + private static class FakeServer implements Runnable { + /** Server. */ + private final ServerSocket srv; + + /** Stop. */ + private volatile boolean stop; + + /** + * Default constructor. + */ + FakeServer() throws IOException { + this.srv = new ServerSocket(47200, 50, InetAddress.getByName("127.0.0.1")); + } + + /** + * + */ + public void stop() { + stop = true; + } + + /** {@inheritDoc} */ + @Override public void run() { + try { + while (!stop) { + try { + U.sleep(10); + } + catch (IgniteInterruptedCheckedException e) { + // No-op. + } + } + } + finally { + U.closeQuiet(srv); + } + } + } + + /** + * + */ + private static class TestCommunicationSpi extends TcpCommunicationSpi { + /** {@inheritDoc} */ + @Override protected GridCommunicationClient createTcpClient(ClusterNode node) throws IgniteCheckedException { + if (PRED.apply(node)) { + Map attrs = new HashMap<>(node.attributes()); + + attrs.put(createAttributeName(ATTR_ADDRS), Collections.singleton("127.0.0.1")); + attrs.put(createAttributeName(ATTR_PORT), 47200); + attrs.put(createAttributeName(ATTR_EXT_ADDRS), Collections.emptyList()); + attrs.put(createAttributeName(ATTR_HOST_NAMES), Collections.emptyList()); + + ((TcpDiscoveryNode)node).setAttributes(attrs); + } + + return super.createTcpClient(node); + } + + /** + * @param name Name. + */ + private String createAttributeName(String name) { + return getClass().getSimpleName() + '.' + name; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestNode.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestNode.java index e0e8eba0219fa..6365443542ac6 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestNode.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestNode.java @@ -82,6 +82,7 @@ public GridTestNode(UUID id) { private void initAttributes() { attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, "10"); attrs.put(IgniteNodeAttributes.ATTR_GRID_NAME, "null"); + attrs.put(IgniteNodeAttributes.ATTR_CLIENT_MODE, false); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java index 9f507e64ca688..ffaec4fbc89af 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java @@ -102,6 +102,7 @@ import org.springframework.context.ApplicationContext; import org.springframework.context.support.FileSystemXmlApplicationContext; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCO_FAILED_CLIENT_RECONNECT_DELAY; import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; @@ -173,6 +174,7 @@ public abstract class GridAbstractTest extends TestCase { static { System.setProperty(IgniteSystemProperties.IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE, "10000"); System.setProperty(IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER, "false"); + System.setProperty(IGNITE_DISCO_FAILED_CLIENT_RECONNECT_DELAY, "1"); if (BINARY_MARSHALLER) GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName()); diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java index c557fbbcad791..ddc255126e1bc 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java @@ -35,6 +35,8 @@ import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiTcpNoDelayOffSelfTest; import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiTcpSelfTest; import org.apache.ignite.spi.communication.tcp.IgniteTcpCommunicationRecoveryAckClosureSelfTest; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpiDropNodesTest; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpiFaultyClientTest; /** * Test suite for all communication SPIs. @@ -72,6 +74,9 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(GridTcpCommunicationSpiConfigSelfTest.class)); + suite.addTest(new TestSuite(TcpCommunicationSpiFaultyClientTest.class)); + suite.addTest(new TestSuite(TcpCommunicationSpiDropNodesTest.class)); + return suite; } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java index 7e0d20b76bdb6..9f56877769313 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java @@ -633,7 +633,7 @@ public void testSimpleCustomTableName() throws Exception { assertEquals(2, qry.getAll().size()); - Throwable throwable = GridTestUtils.assertThrowsInherited(log, new GridPlainCallable() { + GridTestUtils.assertThrows(log, new GridPlainCallable() { @Override public Void call() throws Exception { QueryCursor> qry = cache.query(new SqlQuery(Type1.class, "FROM Type1")); @@ -642,11 +642,7 @@ public void testSimpleCustomTableName() throws Exception { return null; } - }, RuntimeException.class, null); - - assertNotNull(throwable); - - assertTrue(throwable instanceof IgniteException || throwable instanceof CacheException); + }, CacheException.class, null); } /** From d396398c1b4660b3bca24d2650a10f6c0677b4df Mon Sep 17 00:00:00 2001 From: sboikov Date: Fri, 2 Dec 2016 10:36:41 +0300 Subject: [PATCH 431/487] ignite-4314 cache.clear should not destroy offheap map (cherry picked from commit 88c06ec) --- .../cache/GridCacheClearAllRunnable.java | 58 ++++++++----------- .../cache/GridCacheSwapManager.java | 8 --- .../cache/transactions/IgniteTxHandler.java | 2 +- 3 files changed, 26 insertions(+), 42 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheClearAllRunnable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheClearAllRunnable.java index 4f97e7ba18e61..9e7f32948d377 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheClearAllRunnable.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheClearAllRunnable.java @@ -87,48 +87,40 @@ public GridCacheClearAllRunnable(GridCacheAdapter cache, GridCacheVersion // Clear swapped entries. if (!ctx.isNear()) { if (ctx.swap().offHeapEnabled()) { - if (GridQueryProcessor.isEnabled(ctx.config())) { - for (Iterator it = - ctx.swap().offHeapKeyIterator(true, true, AffinityTopologyVersion.NONE); it.hasNext();) { - KeyCacheObject key = it.next(); - - if (owns(key)) - clearEntry(cache.entryEx(key)); + for (Iterator it = ctx.swap().offHeapKeyIterator(true, true, AffinityTopologyVersion.NONE); it.hasNext();) { + KeyCacheObject key = it.next(); - } + if (owns(key)) + clearEntry(cache.entryEx(key)); } - else if (id == 0) - ctx.swap().clearOffHeap(); } - if (ctx.isSwapOrOffheapEnabled()) { - if (ctx.swap().swapEnabled()) { - if (GridQueryProcessor.isEnabled(ctx.config())) { - Iterator it = null; + if (ctx.swap().swapEnabled()) { + if (GridQueryProcessor.isEnabled(ctx.config())) { + Iterator it = null; - try { - it = ctx.swap().swapKeyIterator(true, true, AffinityTopologyVersion.NONE); - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to get iterator over swap.", e); - } + try { + it = ctx.swap().swapKeyIterator(true, true, AffinityTopologyVersion.NONE); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to get iterator over swap.", e); + } - if (it != null) { - while (it.hasNext()) { - KeyCacheObject key = it.next(); + if (it != null) { + while (it.hasNext()) { + KeyCacheObject key = it.next(); - if (owns(key)) - clearEntry(cache.entryEx(key)); - } + if (owns(key)) + clearEntry(cache.entryEx(key)); } } - else if (id == 0) { - try { - ctx.swap().clearSwap(); - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to clearLocally entries from swap storage.", e); - } + } + else if (id == 0) { + try { + ctx.swap().clearSwap(); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to clearLocally entries from swap storage.", e); } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java index fd0b4718151c0..d4499b3c84060 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java @@ -1415,14 +1415,6 @@ public void writeToSwap(int part, KeyCacheObject key, byte[] entry) throws Ignit EVT_CACHE_OBJECT_SWAPPED, null, false, null, true, null, null, null, false); } - /** - * Clears off-heap. - */ - public void clearOffHeap() { - if (offheapEnabled) - initOffHeap(); - } - /** * Clears swap. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java index f784ba2942452..d56415652bc73 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java @@ -1507,7 +1507,7 @@ protected void sendReply(UUID nodeId, GridDhtTxFinishRequest req, boolean commit if (log.isDebugEnabled()) log.debug("Got entry removed exception, will retry: " + entry.txKey()); - entry.cached(null); + entry.cached(cacheCtx.cache().entryEx(entry.key(), req.topologyVersion())); } } } From 51e1f874624c428cc93e9c16407ec5a8b4cf8420 Mon Sep 17 00:00:00 2001 From: sboikov Date: Thu, 19 Jan 2017 10:52:42 +0300 Subject: [PATCH 432/487] Throw CacheException from queries API. --- .../processors/query/GridQueryProcessor.java | 2 +- .../cache/query/IndexingSpiQuerySelfTest.java | 3 +-- .../cache/IgniteCacheAbstractQuerySelfTest.java | 2 -- .../near/IgniteCachePartitionedQuerySelfTest.java | 12 +++++------- 4 files changed, 7 insertions(+), 12 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index 48ca2b5bab69c..c2e5717fbddf8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -905,7 +905,7 @@ public Iterator> queryLocal( }, true); } catch (IgniteCheckedException e) { - throw new IgniteException(e); + throw new CacheException(e); } finally { busyLock.leaveBusy(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java index 84a13df36b52f..e780fdc5656c2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java @@ -328,8 +328,7 @@ private static class MyBinaryIndexingSpi extends MyIndexingSpi { } /** {@inheritDoc} */ - @Override - public void onUnswap(@Nullable String spaceName, Object key, Object val) throws IgniteSpiException { + @Override public void onUnswap(@Nullable String spaceName, Object key, Object val) throws IgniteSpiException { assertTrue(key instanceof BinaryObject); assertTrue(val instanceof BinaryObject); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java index 9f56877769313..751d954f53f22 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java @@ -46,7 +46,6 @@ import org.apache.ignite.IgniteBinary; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.IgniteException; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; @@ -264,7 +263,6 @@ protected Ignite ignite() { stopAllGrids(); - store.reset(); } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedQuerySelfTest.java index b9f21dacdcc07..d5a9faf5ff8c6 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedQuerySelfTest.java @@ -39,7 +39,6 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; @@ -185,9 +184,8 @@ public void testScanQueryPagination() throws Exception { pages.incrementAndGet(); } - else if (msg0 instanceof GridCacheQueryResponse) { + else if (msg0 instanceof GridCacheQueryResponse) assertTrue(((GridCacheQueryResponse)msg0).data().size() <= pageSize); - } } }; @@ -211,15 +209,15 @@ else if (msg0 instanceof GridCacheQueryResponse) { * */ private static class TestTcpCommunicationSpi extends TcpCommunicationSpi { + /** */ volatile IgniteInClosure filter; /** {@inheritDoc} */ - @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackClosure) - throws IgniteSpiException { - if(filter != null) + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) { + if (filter != null) filter.apply(msg); - super.sendMessage(node, msg, ackClosure); + super.sendMessage(node, msg, ackC); } } } \ No newline at end of file From 9c9175d4a84194a224a4020e6185d1e2aee0a5aa Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Thu, 19 Jan 2017 11:19:18 +0300 Subject: [PATCH 433/487] ignite-4147 - Rollback due to test failing on many restarts, should be improved to be more durable --- .../ignite/spi/discovery/tcp/ClientImpl.java | 20 +--- .../ignite/spi/discovery/tcp/ServerImpl.java | 9 -- .../TcpDiscoverySslSecuredUnsecuredTest.java | 93 ------------------- .../IgniteSpiDiscoverySelfTestSuite.java | 2 - 4 files changed, 1 insertion(+), 123 deletions(-) delete mode 100644 modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 35f0908a20b34..39c539c30fb89 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -20,7 +20,6 @@ import java.io.BufferedInputStream; import java.io.IOException; import java.io.InputStream; -import java.io.StreamCorruptedException; import java.net.InetSocketAddress; import java.net.Socket; import java.net.SocketTimeoutException; @@ -45,7 +44,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.atomic.AtomicReference; -import javax.net.ssl.SSLException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteException; @@ -664,14 +662,6 @@ else if (addrs.isEmpty()) { errs.add(e); - if (X.hasCause(e, SSLException.class)) - throw new IgniteSpiException("Unable to establish secure connection. " + - "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); - - if (X.hasCause(e, StreamCorruptedException.class)) - throw new IgniteSpiException("Unable to establish plain connection. " + - "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); - if (timeoutHelper.checkFailureTimeoutReached(e)) break; @@ -1603,15 +1593,7 @@ private void tryJoin() throws InterruptedException { joinCnt++; - T2 joinRes; - try { - joinRes = joinTopology(false, spi.joinTimeout); - } - catch (IgniteSpiException e) { - joinError(e); - - return; - } + T2 joinRes = joinTopology(false, spi.joinTimeout); if (joinRes == null) { if (join) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index f33566cc01a5a..d462ac2638fd2 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -24,7 +24,6 @@ import java.io.ObjectStreamException; import java.io.OutputStream; import java.io.Serializable; -import java.io.StreamCorruptedException; import java.net.ConnectException; import java.net.InetAddress; import java.net.InetSocketAddress; @@ -1221,14 +1220,6 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) errs.add(e); - if (X.hasCause(e, SSLException.class)) - throw new IgniteException("Unable to establish secure connection. " + - "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); - - if (X.hasCause(e, StreamCorruptedException.class)) - throw new IgniteException("Unable to establish plain connection. " + - "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); - if (timeoutHelper.checkFailureTimeoutReached(e)) break; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java deleted file mode 100644 index 229616512cdf8..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java +++ /dev/null @@ -1,93 +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.spi.discovery.tcp; - -import java.util.concurrent.Callable; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.testframework.GridTestUtils; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -/** - * Tests cases when node connects to cluster with different SSL configuration. - * Exception with meaningful message should be thrown. - */ -public class TcpDiscoverySslSecuredUnsecuredTest extends GridCommonAbstractTest { - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception { - final IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setClientMode(gridName.contains("client")); - - if (gridName.contains("ssl")) - cfg.setSslContextFactory(GridTestUtils.sslFactory()); - - return cfg; - } - - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - stopAllGrids(); - } - - /** - * @throws Exception If failed. - */ - public void testSecuredUnsecuredServerConnection() throws Exception { - checkConnection("plain-server", "ssl-server"); - } - - /** - * @throws Exception If failed. - */ - public void testUnsecuredSecuredServerConnection() throws Exception { - checkConnection("ssl-server", "plain-server"); - } - - /** - * @throws Exception If failed. - */ - public void testSecuredClientUnsecuredServerConnection() throws Exception { - checkConnection("plain-server", "ssl-client"); - } - - /** - * @throws Exception If failed. - */ - public void testUnsecuredClientSecuredServerConnection() throws Exception { - checkConnection("ssl-server", "plain-client"); - } - - /** - * @param name1 First grid name. - * @param name2 Second grid name. - * @throws Exception If failed. - */ - @SuppressWarnings("ThrowableNotThrown") - private void checkConnection(final String name1, final String name2) throws Exception { - startGrid(name1); - - GridTestUtils.assertThrows(null, new Callable() { - @Override public Object call() throws Exception { - startGrid(name2); - - return null; - } - }, IgniteCheckedException.class, null); - } -} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java index 98bf6daa06c03..5f870a474578b 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java @@ -34,7 +34,6 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiFailureTimeoutSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiStartStopSelfTest; -import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySslSecuredUnsecuredTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySslSelfTest; import org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinderSelfTest; import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinderSelfTest; @@ -87,7 +86,6 @@ public static TestSuite suite() throws Exception { // SSL. suite.addTest(new TestSuite(TcpDiscoverySslSelfTest.class)); - suite.addTest(new TestSuite(TcpDiscoverySslSecuredUnsecuredTest.class)); return suite; } From f35057833900ba028b3a9a8bd547df61f42a45ed Mon Sep 17 00:00:00 2001 From: sboikov Date: Thu, 19 Jan 2017 12:44:57 +0300 Subject: [PATCH 434/487] CacheScanPartitionQueryFallbackSelfTest fixed to use default page size. --- .../cache/CacheScanPartitionQueryFallbackSelfTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java index 02b213e4bee7e..efa9ce515a6f5 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java @@ -248,7 +248,7 @@ private void scanFallbackOnRebalancing(final boolean cur) throws Exception { info("Running query [node=" + nodeId + ", part=" + part + ']'); try (QueryCursor> cur0 = - cache.query(new ScanQuery(part).setPageSize(5))) { + cache.query(new ScanQuery(part))) { if (cur) doTestScanQueryCursor(cur0, part); From 71a76c81530d1aa38525fd78a90b065b6e19a6bb Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 19 Jan 2017 13:29:55 +0300 Subject: [PATCH 435/487] IGNITE-4550: Move service deployment to certain test. This closes #1437. --- .../GridCacheAbstractFullApiSelfTest.java | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java index b328960cd6a57..1cfb330da8cd6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java @@ -64,6 +64,7 @@ import org.apache.ignite.cache.affinity.Affinity; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.cluster.ClusterGroup; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; @@ -294,18 +295,6 @@ protected CacheMemoryMode memoryMode() { cacheCfgMap = null; } - // We won't deploy service unless non-client node is configured. - for (int i = 0; i < gridCount(); i++) { - Boolean clientMode = grid(i).configuration().isClientMode(); - - if (clientMode != null && clientMode) // Can be null in multi jvm tests. - continue; - - grid(0).services(grid(0).cluster()).deployNodeSingleton(SERVICE_NAME1, new DummyServiceImpl()); - - break; - } - for (int i = 0; i < gridCount(); i++) info("Grid " + i + ": " + grid(i).localNode().id()); } @@ -5521,6 +5510,13 @@ public void testLockInsideTransaction() throws Exception { * @throws Exception If failed. */ public void testTransformResourceInjection() throws Exception { + ClusterGroup servers = grid(0).cluster().forServers(); + + if(F.isEmpty(servers.nodes())) + return; + + grid(0).services( grid(0).cluster()).deployNodeSingleton(SERVICE_NAME1, new DummyServiceImpl()); + IgniteCache cache = jcache(); Ignite ignite = ignite(0); From e832ef9ce363fad34097aa78293a57f4aefcbcc0 Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Tue, 24 Jan 2017 14:44:33 +0300 Subject: [PATCH 436/487] IGNITE-3699 (Backported from master) CreatedExpiryPolicy doesn't work if entry is loaded from store --- .../processors/cache/GridCacheAdapter.java | 35 +++--- .../processors/cache/GridCacheEntryEx.java | 4 +- .../processors/cache/GridCacheMapEntry.java | 17 ++- .../GridDistributedCacheAdapter.java | 6 +- .../distributed/dht/GridDhtCacheAdapter.java | 6 +- .../distributed/dht/GridDhtLockFuture.java | 21 +++- .../dht/GridDhtTransactionalCacheAdapter.java | 7 ++ .../dht/GridDhtTxLocalAdapter.java | 8 +- .../dht/GridPartitionedGetFuture.java | 1 + .../dht/GridPartitionedSingleGetFuture.java | 2 + .../dht/atomic/GridDhtAtomicCache.java | 3 +- .../dht/colocated/GridDhtColocatedCache.java | 10 ++ .../colocated/GridDhtColocatedLockFuture.java | 10 +- .../distributed/near/GridNearAtomicCache.java | 1 + .../distributed/near/GridNearGetFuture.java | 1 + .../distributed/near/GridNearGetRequest.java | 77 ++++++++----- .../distributed/near/GridNearLockFuture.java | 7 ++ .../distributed/near/GridNearLockRequest.java | 81 +++++++++----- .../near/GridNearSingleGetRequest.java | 57 +++++++--- .../near/GridNearTransactionalCache.java | 2 + .../distributed/near/GridNearTxLocal.java | 17 ++- .../cache/local/GridLocalCache.java | 1 + .../local/atomic/GridLocalAtomicCache.java | 1 + .../transactions/IgniteTxLocalAdapter.java | 37 +++++-- .../cache/transactions/IgniteTxLocalEx.java | 3 + .../cache/GridCacheTestEntryEx.java | 3 +- ...acheExpiryPolicyWithStoreAbstractTest.java | 102 ++++++++++++++++++ ...gniteCacheTxExpiryPolicyWithStoreTest.java | 21 ++++ 28 files changed, 432 insertions(+), 109 deletions(-) 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 59665bb58708f..dc8f0308a0099 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 @@ -528,6 +528,7 @@ public boolean isDht() { * @param retval Flag to return value. * @param isolation Transaction isolation. * @param invalidate Invalidate flag. + * @param createTtl TTL for create operation. * @param accessTtl TTL for read operation. * @return Locks future. */ @@ -539,6 +540,7 @@ public abstract IgniteInternalFuture txLockAsync( boolean retval, TransactionIsolation isolation, boolean invalidate, + long createTtl, long accessTtl); /** @@ -1873,7 +1875,7 @@ protected final IgniteInternalFuture> getAllAsync0( @Nullable final UUID subjId, final String taskName, final boolean deserializeBinary, - @Nullable IgniteCacheExpiryPolicy expiry, + @Nullable final IgniteCacheExpiryPolicy expiry, final boolean skipVals, final boolean keepCacheObjects, boolean canRemap, @@ -2027,7 +2029,8 @@ protected final IgniteInternalFuture> getAllAsync0( cacheVal, res.version(), null, - readerArgs); + readerArgs, + expiry); if (log.isDebugEnabled()) log.debug("Set value loaded from store into entry [" + @@ -5936,28 +5939,28 @@ protected abstract static class CacheExpiryPolicy implements IgniteCacheExpiryPo } /** - * @param ttl Access TTL. + * @param createTtl Create TTL. + * @param accessTtl Access TTL. * @return Access expire policy. */ - @Nullable public static CacheExpiryPolicy forAccess(final long ttl) { - if (ttl == CU.TTL_NOT_CHANGED) + @Nullable public static CacheExpiryPolicy fromRemote(final long createTtl, final long accessTtl) { + if (createTtl == CU.TTL_NOT_CHANGED && accessTtl == CU.TTL_NOT_CHANGED) return null; return new CacheExpiryPolicy() { - @Override public long forAccess() { - return ttl; + @Override public long forCreate() { + return createTtl; } - }; - } - /** {@inheritDoc} */ - @Override public long forCreate() { - return CU.TTL_NOT_CHANGED; - } + @Override public long forAccess() { + return accessTtl; + } - /** {@inheritDoc} */ - @Override public long forUpdate() { - return CU.TTL_NOT_CHANGED; + /** {@inheritDoc} */ + @Override public long forUpdate() { + return CU.TTL_NOT_CHANGED; + } + }; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java index 51f423a7302a0..3c42d5360a4b1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java @@ -756,6 +756,7 @@ public GridCacheVersionedEntryEx versionedEntry(final boolean keepB * @param curVer Version to match or {@code null} if match is not required. * @param newVer Version to set. * @param readerArgs Reader will be added if not null. + * @param loadExpiryPlc Expiry policy if entry is loaded from store. * @return Current version and value. * @throws IgniteCheckedException If index could not be updated. * @throws GridCacheEntryRemovedException If entry was removed. @@ -763,7 +764,8 @@ public GridCacheVersionedEntryEx versionedEntry(final boolean keepB public T2 versionedValue(CacheObject val, @Nullable GridCacheVersion curVer, @Nullable GridCacheVersion newVer, - @Nullable ReaderArguments readerArgs) + @Nullable ReaderArguments readerArgs, + @Nullable IgniteCacheExpiryPolicy loadExpiryPlc) throws IgniteCheckedException, GridCacheEntryRemovedException; /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 59e4181df0cee..7e26719d99076 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -3624,7 +3624,8 @@ private long nextPartCounter(AffinityTopologyVersion topVer) { @Override public synchronized T2 versionedValue(CacheObject val, GridCacheVersion curVer, GridCacheVersion newVer, - @Nullable ReaderArguments readerArgs) + @Nullable ReaderArguments readerArgs, + @Nullable IgniteCacheExpiryPolicy loadExpiryPlc) throws IgniteCheckedException, GridCacheEntryRemovedException { checkObsolete(); @@ -3643,9 +3644,19 @@ private long nextPartCounter(AffinityTopologyVersion topVer) { CacheObject old = rawGetOrUnmarshalUnlocked(false); - long ttl = ttlExtras(); + long ttl; + long expTime; - long expTime = CU.toExpireTime(ttl); + if (loadExpiryPlc != null) { + IgniteBiTuple initTtlAndExpireTime = initialTtlAndExpireTime(loadExpiryPlc); + + ttl = initTtlAndExpireTime.get1(); + expTime = initTtlAndExpireTime.get2(); + } + else { + ttl = ttlExtras(); + expTime = expireTimeExtras(); + } // Detach value before index update. val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java index 03f647432a7c9..d89a468404371 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java @@ -102,11 +102,12 @@ protected GridDistributedCacheAdapter(GridCacheContext ctx, GridCacheConcu boolean retval, TransactionIsolation isolation, boolean isInvalidate, + long createTtl, long accessTtl ) { assert tx != null; - return lockAllAsync(keys, timeout, tx, isInvalidate, isRead, retval, isolation, accessTtl); + return lockAllAsync(keys, timeout, tx, isInvalidate, isRead, retval, isolation, createTtl, accessTtl); } /** {@inheritDoc} */ @@ -121,6 +122,7 @@ protected GridDistributedCacheAdapter(GridCacheContext ctx, GridCacheConcu false, /*retval*/true, null, + -1L, -1L); } @@ -132,6 +134,7 @@ protected GridDistributedCacheAdapter(GridCacheContext ctx, GridCacheConcu * @param isRead Indicates whether value is read or written. * @param retval Flag to return value. * @param isolation Transaction isolation. + * @param createTtl TTL for create operation. * @param accessTtl TTL for read operation. * @return Future for locks. */ @@ -142,6 +145,7 @@ protected abstract IgniteInternalFuture lockAllAsync(Collection getDhtSingleAsync( protected void processNearSingleGetRequest(final UUID nodeId, final GridNearSingleGetRequest req) { assert ctx.affinityNode(); - final CacheExpiryPolicy expiryPlc = CacheExpiryPolicy.forAccess(req.accessTtl()); + final CacheExpiryPolicy expiryPlc = CacheExpiryPolicy.fromRemote(req.createTtl(), req.accessTtl()); IgniteInternalFuture fut = getDhtSingleAsync( @@ -860,9 +860,7 @@ protected void processNearGetRequest(final UUID nodeId, final GridNearGetRequest assert ctx.affinityNode(); assert !req.reload() : req; - long ttl = req.accessTtl(); - - final CacheExpiryPolicy expiryPlc = CacheExpiryPolicy.forAccess(ttl); + final CacheExpiryPolicy expiryPlc = CacheExpiryPolicy.fromRemote(req.createTtl(), req.accessTtl()); IgniteInternalFuture> fut = getDhtAsync(nodeId, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java index d77933e9f5d47..686a4c6bed872 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java @@ -156,6 +156,9 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture /** Pending locks. */ private final Collection pendingLocks; + /** TTL for create operation. */ + private long createTtl; + /** TTL for read operation. */ private long accessTtl; @@ -194,6 +197,7 @@ public GridDhtLockFuture( long timeout, GridDhtTxLocalAdapter tx, long threadId, + long createTtl, long accessTtl, CacheEntryPredicate[] filter, boolean skipStore, @@ -214,6 +218,7 @@ public GridDhtLockFuture( this.timeout = timeout; this.filter = filter; this.tx = tx; + this.createTtl = createTtl; this.accessTtl = accessTtl; this.skipStore = skipStore; this.keepBinary = keepBinary; @@ -1059,10 +1064,22 @@ private void loadMissingFromStore() { try { CacheObject val0 = cctx.toCacheObject(val); + long ttl = createTtl; + long expireTime; + + if (ttl == CU.TTL_ZERO) + expireTime = CU.expireTimeInPast(); + else { + if (ttl == CU.TTL_NOT_CHANGED) + ttl = CU.TTL_ETERNAL; + + expireTime = CU.toExpireTime(ttl); + } + entry0.initialValue(val0, ver, - 0, - 0, + ttl, + expireTime, false, topVer, GridDrType.DR_LOAD, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java index 01bc4e028955f..a9e3bc496721c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java @@ -677,6 +677,7 @@ else if (txLockMsgLog.isDebugEnabled()) { boolean isRead, boolean retval, TransactionIsolation isolation, + long createTtl, long accessTtl) { CacheOperationContext opCtx = ctx.operationContextPerCall(); @@ -688,6 +689,7 @@ else if (txLockMsgLog.isDebugEnabled()) { isRead, retval, isolation, + createTtl, accessTtl, CU.empty0(), opCtx != null && opCtx.skipStore(), @@ -704,6 +706,7 @@ else if (txLockMsgLog.isDebugEnabled()) { * @param isRead Read flag. * @param retval Return value flag. * @param isolation Transaction isolation. + * @param createTtl TTL for create operation. * @param accessTtl TTL for read operation. * @param filter Optional filter. * @param skipStore Skip store flag. @@ -716,6 +719,7 @@ public GridDhtFuture lockAllAsyncInternal(@Nullable Collection lockAllAsyncInternal(@Nullable Collection lockAllAsync( req.timeout(), tx, req.threadId(), + req.createTtl(), req.accessTtl(), filter, req.skipStore(), @@ -1007,6 +1013,7 @@ public IgniteInternalFuture lockAllAsync( req.messageId(), req.txRead(), req.needReturnValue(), + req.createTtl(), req.accessTtl(), req.skipStore(), req.keepBinary()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java index 35dfb62c5fc98..12a45d8fdca25 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java @@ -148,7 +148,7 @@ protected GridDhtTxLocalAdapter( storeEnabled, onePhaseCommit, txSize, - subjId, + subjId, taskNameHash ); @@ -534,6 +534,7 @@ private void addMapping( * @param entries Entries to lock. * @param msgId Message ID. * @param read Read flag. + * @param createTtl TTL for create operation. * @param accessTtl TTL for read operation. * @param needRetVal Return value flag. * @param skipStore Skip store flag. @@ -546,6 +547,7 @@ IgniteInternalFuture lockAllAsync( long msgId, final boolean read, final boolean needRetVal, + long createTtl, long accessTtl, boolean skipStore, boolean keepBinary @@ -652,6 +654,7 @@ IgniteInternalFuture lockAllAsync( passedKeys, read, needRetVal, + createTtl, accessTtl, null, skipStore, @@ -670,6 +673,7 @@ IgniteInternalFuture lockAllAsync( * @param passedKeys Passed keys. * @param read {@code True} if read. * @param needRetVal Return value flag. + * @param createTtl TTL for create operation. * @param accessTtl TTL for read operation. * @param filter Entry write filter. * @param skipStore Skip store flag. @@ -681,6 +685,7 @@ private IgniteInternalFuture obtainLockAsync( final Collection passedKeys, final boolean read, final boolean needRetVal, + final long createTtl, final long accessTtl, @Nullable final CacheEntryPredicate[] filter, boolean skipStore, @@ -706,6 +711,7 @@ private IgniteInternalFuture obtainLockAsync( read, needRetVal, isolation, + createTtl, accessTtl, CU.empty0(), skipStore, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java index c8e2cf328bcd3..5892b374951f4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java @@ -341,6 +341,7 @@ private void map( topVer, subjId, taskName == null ? 0 : taskName.hashCode(), + expiryPlc != null ? expiryPlc.forCreate() : -1L, expiryPlc != null ? expiryPlc.forAccess() : -1L, skipVals, cctx.deploymentEnabled()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index e369bfa83a8be..7c14f35b34cf4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -281,6 +281,7 @@ private void map(AffinityTopologyVersion topVer) { topVer, subjId, taskName == null ? 0 : taskName.hashCode(), + expiryPlc != null ? expiryPlc.forCreate() : -1L, expiryPlc != null ? expiryPlc.forAccess() : -1L, skipVals, /**add reader*/false, @@ -300,6 +301,7 @@ private void map(AffinityTopologyVersion topVer) { topVer, subjId, taskName == null ? 0 : taskName.hashCode(), + expiryPlc != null ? expiryPlc.forCreate() : -1L, expiryPlc != null ? expiryPlc.forAccess() : -1L, skipVals, cctx.deploymentEnabled()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java index f601e0a9ddb7e..2f97bccf88a71 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java @@ -863,6 +863,7 @@ protected IgniteInternalFuture asyncOp(final CO> boolean isRead, boolean retval, @Nullable TransactionIsolation isolation, + long createTtl, long accessTtl) { return new FinishedLockFuture(new UnsupportedOperationException("Locks are not supported for " + "CacheAtomicityMode.ATOMIC mode (use CacheAtomicityMode.TRANSACTIONAL instead)")); @@ -2293,7 +2294,7 @@ private void reloadIfNeeded(final List entries) throws Ignite try { GridCacheVersion ver = entry.version(); - entry.versionedValue(ctx.toCacheObject(v), null, ver, null); + entry.versionedValue(ctx.toCacheObject(v), null, ver, null, null); } catch (GridCacheEntryRemovedException e) { assert false : "Entry should not get obsolete while holding lock [entry=" + entry + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java index 29f0607e5c33c..5ed30dbcadcba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java @@ -615,6 +615,7 @@ else if (!skipVals && ctx.config().isStatisticsEnabled()) boolean isRead, boolean retval, @Nullable TransactionIsolation isolation, + long createTtl, long accessTtl ) { assert tx == null || tx instanceof GridNearTxLocal : tx; @@ -629,6 +630,7 @@ else if (!skipVals && ctx.config().isStatisticsEnabled()) isRead, retval, timeout, + createTtl, accessTtl, CU.empty0(), opCtx != null && opCtx.skipStore(), @@ -876,6 +878,7 @@ public void removeLocks(long threadId, GridCacheVersion ver, Collection lockAllAsync( final boolean txRead, final boolean retval, final long timeout, + final long createTtl, final long accessTtl, @Nullable final CacheEntryPredicate[] filter, final boolean skipStore, @@ -915,6 +919,7 @@ IgniteInternalFuture lockAllAsync( txRead, retval, timeout, + createTtl, accessTtl, filter, skipStore, @@ -936,6 +941,7 @@ IgniteInternalFuture lockAllAsync( txRead, retval, timeout, + createTtl, accessTtl, filter, skipStore, @@ -956,6 +962,7 @@ IgniteInternalFuture lockAllAsync( * @param txRead Tx read. * @param retval Return value flag. * @param timeout Lock timeout. + * @param createTtl TTL for create operation. * @param accessTtl TTL for read operation. * @param filter filter Optional filter. * @param skipStore Skip store flag. @@ -971,6 +978,7 @@ private IgniteInternalFuture lockAllAsync0( final boolean txRead, boolean retval, final long timeout, + final long createTtl, final long accessTtl, @Nullable final CacheEntryPredicate[] filter, boolean skipStore, @@ -988,6 +996,7 @@ private IgniteInternalFuture lockAllAsync0( timeout, tx, threadId, + createTtl, accessTtl, filter, skipStore, @@ -1056,6 +1065,7 @@ else if (!b) keys, retval, txRead, + createTtl, accessTtl, skipStore, keepBinary); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java index 5557d3423e201..40e87ee2cfd7c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java @@ -145,6 +145,9 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture /** Trackable flag (here may be non-volatile). */ private boolean trackable; + /** TTL for create operation. */ + private final long createTtl; + /** TTL for read operation. */ private final long accessTtl; @@ -164,6 +167,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture * @param read Read flag. * @param retval Flag to return value or not. * @param timeout Lock acquisition timeout. + * @param createTtl TTL for create operation. * @param accessTtl TTL for read operation. * @param filter Filter. * @param skipStore Skip store flag. @@ -175,6 +179,7 @@ public GridDhtColocatedLockFuture( boolean read, boolean retval, long timeout, + long createTtl, long accessTtl, CacheEntryPredicate[] filter, boolean skipStore, @@ -189,6 +194,7 @@ public GridDhtColocatedLockFuture( this.read = read; this.retval = retval; this.timeout = timeout; + this.createTtl = createTtl; this.accessTtl = accessTtl; this.filter = filter; this.skipStore = skipStore; @@ -926,6 +932,7 @@ private synchronized void map0( inTx() && tx.syncMode() == FULL_SYNC, inTx() ? tx.subjectId() : null, inTx() ? tx.taskNameHash() : 0, + read ? createTtl : -1L, read ? accessTtl : -1L, skipStore, keepBinary, @@ -1102,7 +1109,7 @@ private void proceedMapping0() /** * Locks given keys directly through dht cache. - * @param keys Collection of keys. + * @param keys Collection of keys. * @param topVer Topology version to lock on. */ private void lockLocally( @@ -1121,6 +1128,7 @@ private void lockLocally( read, retval, timeout, + createTtl, accessTtl, filter, skipStore, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java index a8219b08cefdc..d1056fdff4284 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java @@ -641,6 +641,7 @@ public void processDhtAtomicUpdateRequest( boolean isRead, boolean retval, @Nullable TransactionIsolation isolation, + long createTtl, long accessTtl) { return dht.lockAllAsync(null, timeout); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java index 8bc513e4dfda2..8c64e3e9603bd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java @@ -375,6 +375,7 @@ private void map( topVer, subjId, taskName == null ? 0 : taskName.hashCode(), + expiryPlc != null ? expiryPlc.forCreate() : -1L, expiryPlc != null ? expiryPlc.forAccess() : -1L, skipVals, cctx.deploymentEnabled()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java index fa7f367cc4460..e02658cecb7c9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java @@ -99,6 +99,9 @@ public class GridNearGetRequest extends GridCacheMessage implements GridCacheDep /** Task name hash. */ private int taskNameHash; + /** TTL for read operation. */ + private long createTtl; + /** TTL for read operation. */ private long accessTtl; @@ -121,6 +124,7 @@ public GridNearGetRequest() { * @param topVer Topology version. * @param subjId Subject ID. * @param taskNameHash Task name hash. + * @param createTtl New TTL to set after entry is created, -1 to leave unchanged. * @param accessTtl New TTL to set after entry is accessed, -1 to leave unchanged. * @param addDepInfo Deployment info. */ @@ -134,6 +138,7 @@ public GridNearGetRequest( @NotNull AffinityTopologyVersion topVer, UUID subjId, int taskNameHash, + long createTtl, long accessTtl, boolean skipVals, boolean addDepInfo @@ -161,6 +166,7 @@ public GridNearGetRequest( this.topVer = topVer; this.subjId = subjId; this.taskNameHash = taskNameHash; + this.createTtl = createTtl; this.accessTtl = accessTtl; this.skipVals = skipVals; this.addDepInfo = addDepInfo; @@ -237,6 +243,13 @@ public boolean skipValues() { return topVer; } + /** + * @return New TTL to set after entry is created, -1 to leave unchanged. + */ + public long createTtl() { + return createTtl; + } + /** * @return New TTL to set after entry is accessed, -1 to leave unchanged. */ @@ -320,73 +333,79 @@ public long accessTtl() { writer.incrementState(); case 4: - if (!writer.writeCollection("flags", flags, MessageCollectionItemType.BOOLEAN)) + if (!writer.writeLong("createTtl", createTtl)) return false; writer.incrementState(); case 5: - if (!writer.writeIgniteUuid("futId", futId)) + if (!writer.writeCollection("flags", flags, MessageCollectionItemType.BOOLEAN)) return false; writer.incrementState(); case 6: - if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG)) + if (!writer.writeIgniteUuid("futId", futId)) return false; writer.incrementState(); case 7: - if (!writer.writeIgniteUuid("miniId", miniId)) + if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 8: - if (!writer.writeBoolean("readThrough", readThrough)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; writer.incrementState(); case 9: - if (!writer.writeBoolean("reload", reload)) + if (!writer.writeCollection("partIds", partIds, MessageCollectionItemType.INT)) return false; writer.incrementState(); case 10: - if (!writer.writeBoolean("skipVals", skipVals)) + if (!writer.writeBoolean("readThrough", readThrough)) return false; writer.incrementState(); case 11: - if (!writer.writeUuid("subjId", subjId)) + if (!writer.writeBoolean("reload", reload)) return false; writer.incrementState(); case 12: - if (!writer.writeInt("taskNameHash", taskNameHash)) + if (!writer.writeBoolean("skipVals", skipVals)) return false; writer.incrementState(); case 13: - if (!writer.writeMessage("topVer", topVer)) + if (!writer.writeUuid("subjId", subjId)) return false; writer.incrementState(); case 14: - if (!writer.writeMessage("ver", ver)) + if (!writer.writeInt("taskNameHash", taskNameHash)) return false; writer.incrementState(); case 15: - if (!writer.writeCollection("partIds", partIds, MessageCollectionItemType.INT)) + if (!writer.writeMessage("topVer", topVer)) + return false; + + writer.incrementState(); + + case 16: + if (!writer.writeMessage("ver", ver)) return false; writer.incrementState(); @@ -416,7 +435,7 @@ public long accessTtl() { reader.incrementState(); case 4: - flags = reader.readCollection("flags", MessageCollectionItemType.BOOLEAN); + createTtl = reader.readLong("createTtl"); if (!reader.isLastRead()) return false; @@ -424,7 +443,7 @@ public long accessTtl() { reader.incrementState(); case 5: - futId = reader.readIgniteUuid("futId"); + flags = reader.readCollection("flags", MessageCollectionItemType.BOOLEAN); if (!reader.isLastRead()) return false; @@ -432,7 +451,7 @@ public long accessTtl() { reader.incrementState(); case 6: - keys = reader.readCollection("keys", MessageCollectionItemType.MSG); + futId = reader.readIgniteUuid("futId"); if (!reader.isLastRead()) return false; @@ -440,7 +459,7 @@ public long accessTtl() { reader.incrementState(); case 7: - miniId = reader.readIgniteUuid("miniId"); + keys = reader.readCollection("keys", MessageCollectionItemType.MSG); if (!reader.isLastRead()) return false; @@ -448,7 +467,7 @@ public long accessTtl() { reader.incrementState(); case 8: - readThrough = reader.readBoolean("readThrough"); + miniId = reader.readIgniteUuid("miniId"); if (!reader.isLastRead()) return false; @@ -456,7 +475,7 @@ public long accessTtl() { reader.incrementState(); case 9: - reload = reader.readBoolean("reload"); + partIds = reader.readCollection("partIds", MessageCollectionItemType.INT); if (!reader.isLastRead()) return false; @@ -464,7 +483,7 @@ public long accessTtl() { reader.incrementState(); case 10: - skipVals = reader.readBoolean("skipVals"); + readThrough = reader.readBoolean("readThrough"); if (!reader.isLastRead()) return false; @@ -472,7 +491,7 @@ public long accessTtl() { reader.incrementState(); case 11: - subjId = reader.readUuid("subjId"); + reload = reader.readBoolean("reload"); if (!reader.isLastRead()) return false; @@ -480,7 +499,7 @@ public long accessTtl() { reader.incrementState(); case 12: - taskNameHash = reader.readInt("taskNameHash"); + skipVals = reader.readBoolean("skipVals"); if (!reader.isLastRead()) return false; @@ -488,7 +507,7 @@ public long accessTtl() { reader.incrementState(); case 13: - topVer = reader.readMessage("topVer"); + subjId = reader.readUuid("subjId"); if (!reader.isLastRead()) return false; @@ -496,7 +515,7 @@ public long accessTtl() { reader.incrementState(); case 14: - ver = reader.readMessage("ver"); + taskNameHash = reader.readInt("taskNameHash"); if (!reader.isLastRead()) return false; @@ -504,7 +523,15 @@ public long accessTtl() { reader.incrementState(); case 15: - partIds = reader.readCollection("partIds", MessageCollectionItemType.INT); + topVer = reader.readMessage("topVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 16: + ver = reader.readMessage("ver"); if (!reader.isLastRead()) return false; @@ -523,7 +550,7 @@ public long accessTtl() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 16; + return 17; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java index 7c9860290eea2..491b4ece0ea13 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java @@ -148,6 +148,9 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture entries; + /** TTL for create operation. */ + private long createTtl; + /** TTL for read operation. */ private long accessTtl; @@ -168,6 +171,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture keys, boolean remap, boolean topLocked inTx() && tx.syncMode() == FULL_SYNC, inTx() ? tx.subjectId() : null, inTx() ? tx.taskNameHash() : 0, + read ? createTtl : -1L, read ? accessTtl : -1L, skipStore, keepBinary, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java index 2e8cd6e852759..9e12153ea10f4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java @@ -80,6 +80,9 @@ public class GridNearLockRequest extends GridDistributedLockRequest { /** Sync commit flag. */ private boolean syncCommit; + /** TTL for create operation. */ + private long createTtl; + /** TTL for read operation. */ private long accessTtl; @@ -116,6 +119,7 @@ public GridNearLockRequest() { * @param syncCommit Synchronous commit flag. * @param subjId Subject ID. * @param taskNameHash Task name hash code. + * @param createTtl TTL for create operation. * @param accessTtl TTL for read operation. * @param skipStore Skip store flag. * @param firstClientReq {@code True} if first lock request for lock operation sent from client node. @@ -141,6 +145,7 @@ public GridNearLockRequest( boolean syncCommit, @Nullable UUID subjId, int taskNameHash, + long createTtl, long accessTtl, boolean skipStore, boolean keepBinary, @@ -174,6 +179,7 @@ public GridNearLockRequest( this.syncCommit = syncCommit; this.subjId = subjId; this.taskNameHash = taskNameHash; + this.createTtl = createTtl; this.accessTtl = accessTtl; this.retVal = retVal; this.firstClientReq = firstClientReq; @@ -311,6 +317,13 @@ public GridCacheVersion dhtVersion(int idx) { return dhtVers[idx]; } + /** + * @return New TTL to set after entry is created, -1 to leave unchanged. + */ + public long createTtl() { + return createTtl; + } + /** * @return TTL for read operation. */ @@ -368,84 +381,90 @@ public long accessTtl() { writer.incrementState(); case 21: - if (!writer.writeObjectArray("dhtVers", dhtVers, MessageCollectionItemType.MSG)) + if (!writer.writeLong("createTtl", createTtl)) return false; writer.incrementState(); case 22: - if (!writer.writeObjectArray("filter", filter, MessageCollectionItemType.MSG)) + if (!writer.writeObjectArray("dhtVers", dhtVers, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 23: - if (!writer.writeBoolean("firstClientReq", firstClientReq)) + if (!writer.writeObjectArray("filter", filter, MessageCollectionItemType.MSG)) return false; writer.incrementState(); case 24: - if (!writer.writeBoolean("hasTransforms", hasTransforms)) + if (!writer.writeBoolean("firstClientReq", firstClientReq)) return false; writer.incrementState(); case 25: - if (!writer.writeBoolean("implicitSingleTx", implicitSingleTx)) + if (!writer.writeBoolean("hasTransforms", hasTransforms)) return false; writer.incrementState(); case 26: - if (!writer.writeBoolean("implicitTx", implicitTx)) + if (!writer.writeBoolean("implicitSingleTx", implicitSingleTx)) return false; writer.incrementState(); case 27: - if (!writer.writeIgniteUuid("miniId", miniId)) + if (!writer.writeBoolean("implicitTx", implicitTx)) return false; writer.incrementState(); case 28: - if (!writer.writeBoolean("onePhaseCommit", onePhaseCommit)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; writer.incrementState(); case 29: - if (!writer.writeBoolean("retVal", retVal)) + if (!writer.writeBoolean("onePhaseCommit", onePhaseCommit)) return false; writer.incrementState(); case 30: - if (!writer.writeUuid("subjId", subjId)) + if (!writer.writeBoolean("retVal", retVal)) return false; writer.incrementState(); case 31: - if (!writer.writeBoolean("syncCommit", syncCommit)) + if (!writer.writeUuid("subjId", subjId)) return false; writer.incrementState(); case 32: - if (!writer.writeInt("taskNameHash", taskNameHash)) + if (!writer.writeBoolean("syncCommit", syncCommit)) return false; writer.incrementState(); case 33: - if (!writer.writeMessage("topVer", topVer)) + if (!writer.writeInt("taskNameHash", taskNameHash)) return false; writer.incrementState(); case 34: + if (!writer.writeMessage("topVer", topVer)) + return false; + + writer.incrementState(); + + case 35: if (!writer.writeCollection("partIds", partIds, MessageCollectionItemType.INT)) return false; @@ -476,7 +495,7 @@ public long accessTtl() { reader.incrementState(); case 21: - dhtVers = reader.readObjectArray("dhtVers", MessageCollectionItemType.MSG, GridCacheVersion.class); + createTtl = reader.readLong("createTtl"); if (!reader.isLastRead()) return false; @@ -484,7 +503,7 @@ public long accessTtl() { reader.incrementState(); case 22: - filter = reader.readObjectArray("filter", MessageCollectionItemType.MSG, CacheEntryPredicate.class); + dhtVers = reader.readObjectArray("dhtVers", MessageCollectionItemType.MSG, GridCacheVersion.class); if (!reader.isLastRead()) return false; @@ -492,7 +511,7 @@ public long accessTtl() { reader.incrementState(); case 23: - firstClientReq = reader.readBoolean("firstClientReq"); + filter = reader.readObjectArray("filter", MessageCollectionItemType.MSG, CacheEntryPredicate.class); if (!reader.isLastRead()) return false; @@ -500,7 +519,7 @@ public long accessTtl() { reader.incrementState(); case 24: - hasTransforms = reader.readBoolean("hasTransforms"); + firstClientReq = reader.readBoolean("firstClientReq"); if (!reader.isLastRead()) return false; @@ -508,7 +527,7 @@ public long accessTtl() { reader.incrementState(); case 25: - implicitSingleTx = reader.readBoolean("implicitSingleTx"); + hasTransforms = reader.readBoolean("hasTransforms"); if (!reader.isLastRead()) return false; @@ -516,7 +535,7 @@ public long accessTtl() { reader.incrementState(); case 26: - implicitTx = reader.readBoolean("implicitTx"); + implicitSingleTx = reader.readBoolean("implicitSingleTx"); if (!reader.isLastRead()) return false; @@ -524,7 +543,7 @@ public long accessTtl() { reader.incrementState(); case 27: - miniId = reader.readIgniteUuid("miniId"); + implicitTx = reader.readBoolean("implicitTx"); if (!reader.isLastRead()) return false; @@ -532,7 +551,7 @@ public long accessTtl() { reader.incrementState(); case 28: - onePhaseCommit = reader.readBoolean("onePhaseCommit"); + miniId = reader.readIgniteUuid("miniId"); if (!reader.isLastRead()) return false; @@ -540,7 +559,7 @@ public long accessTtl() { reader.incrementState(); case 29: - retVal = reader.readBoolean("retVal"); + onePhaseCommit = reader.readBoolean("onePhaseCommit"); if (!reader.isLastRead()) return false; @@ -548,7 +567,7 @@ public long accessTtl() { reader.incrementState(); case 30: - subjId = reader.readUuid("subjId"); + retVal = reader.readBoolean("retVal"); if (!reader.isLastRead()) return false; @@ -556,7 +575,7 @@ public long accessTtl() { reader.incrementState(); case 31: - syncCommit = reader.readBoolean("syncCommit"); + subjId = reader.readUuid("subjId"); if (!reader.isLastRead()) return false; @@ -564,7 +583,7 @@ public long accessTtl() { reader.incrementState(); case 32: - taskNameHash = reader.readInt("taskNameHash"); + syncCommit = reader.readBoolean("syncCommit"); if (!reader.isLastRead()) return false; @@ -572,7 +591,7 @@ public long accessTtl() { reader.incrementState(); case 33: - topVer = reader.readMessage("topVer"); + taskNameHash = reader.readInt("taskNameHash"); if (!reader.isLastRead()) return false; @@ -580,6 +599,14 @@ public long accessTtl() { reader.incrementState(); case 34: + topVer = reader.readMessage("topVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 35: partIds = reader.readCollection("partIds", MessageCollectionItemType.INT); if (!reader.isLastRead()) @@ -599,7 +626,7 @@ public long accessTtl() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 35; + return 36; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java index 7fc2b1e056837..8fe33d8326628 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearSingleGetRequest.java @@ -79,6 +79,9 @@ public class GridNearSingleGetRequest extends GridCacheMessage implements GridCa /** Task name hash. */ private int taskNameHash; + /** TTL for read operation. */ + private long createTtl; + /** TTL for read operation. */ private long accessTtl; @@ -99,6 +102,7 @@ public GridNearSingleGetRequest() { * @param topVer Topology version. * @param subjId Subject ID. * @param taskNameHash Task name hash. + * @param createTtl New TTL to set after entry is created, -1 to leave unchanged. * @param accessTtl New TTL to set after entry is accessed, -1 to leave unchanged. * @param addReader Add reader flag. * @param needVer {@code True} if entry version is needed. @@ -112,6 +116,7 @@ public GridNearSingleGetRequest( @NotNull AffinityTopologyVersion topVer, UUID subjId, int taskNameHash, + long createTtl, long accessTtl, boolean skipVals, boolean addReader, @@ -127,6 +132,7 @@ public GridNearSingleGetRequest( this.topVer = topVer; this.subjId = subjId; this.taskNameHash = taskNameHash; + this.createTtl = createTtl; this.accessTtl = accessTtl; this.addDepInfo = addDepInfo; @@ -180,6 +186,13 @@ public int taskNameHash() { return topVer; } + /** + * @return New TTL to set after entry is created, -1 to leave unchanged. + */ + public long createTtl() { + return createTtl; + } + /** * @return New TTL to set after entry is accessed, -1 to leave unchanged. */ @@ -266,7 +279,7 @@ public boolean needEntryInfo() { reader.incrementState(); case 4: - flags = reader.readByte("flags"); + createTtl = reader.readLong("createTtl"); if (!reader.isLastRead()) return false; @@ -274,7 +287,7 @@ public boolean needEntryInfo() { reader.incrementState(); case 5: - futId = reader.readLong("futId"); + flags = reader.readByte("flags"); if (!reader.isLastRead()) return false; @@ -282,7 +295,7 @@ public boolean needEntryInfo() { reader.incrementState(); case 6: - key = reader.readMessage("key"); + futId = reader.readLong("futId"); if (!reader.isLastRead()) return false; @@ -290,7 +303,7 @@ public boolean needEntryInfo() { reader.incrementState(); case 7: - subjId = reader.readUuid("subjId"); + key = reader.readMessage("key"); if (!reader.isLastRead()) return false; @@ -298,7 +311,7 @@ public boolean needEntryInfo() { reader.incrementState(); case 8: - taskNameHash = reader.readInt("taskNameHash"); + partId = reader.readInt("partId", -1); if (!reader.isLastRead()) return false; @@ -306,7 +319,7 @@ public boolean needEntryInfo() { reader.incrementState(); case 9: - topVer = reader.readMessage("topVer"); + subjId = reader.readUuid("subjId"); if (!reader.isLastRead()) return false; @@ -314,7 +327,15 @@ public boolean needEntryInfo() { reader.incrementState(); case 10: - partId = reader.readInt("partId", -1); + taskNameHash = reader.readInt("taskNameHash"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 11: + topVer = reader.readMessage("topVer"); if (!reader.isLastRead()) return false; @@ -348,43 +369,49 @@ public boolean needEntryInfo() { writer.incrementState(); case 4: - if (!writer.writeByte("flags", flags)) + if (!writer.writeLong("createTtl", createTtl)) return false; writer.incrementState(); case 5: - if (!writer.writeLong("futId", futId)) + if (!writer.writeByte("flags", flags)) return false; writer.incrementState(); case 6: - if (!writer.writeMessage("key", key)) + if (!writer.writeLong("futId", futId)) return false; writer.incrementState(); case 7: - if (!writer.writeUuid("subjId", subjId)) + if (!writer.writeMessage("key", key)) return false; writer.incrementState(); case 8: - if (!writer.writeInt("taskNameHash", taskNameHash)) + if (!writer.writeInt("partId", partId)) return false; writer.incrementState(); case 9: - if (!writer.writeMessage("topVer", topVer)) + if (!writer.writeUuid("subjId", subjId)) return false; writer.incrementState(); case 10: - if (!writer.writeInt("partId", partId)) + if (!writer.writeInt("taskNameHash", taskNameHash)) + return false; + + writer.incrementState(); + + case 11: + if (!writer.writeMessage("topVer", topVer)) return false; writer.incrementState(); @@ -406,7 +433,7 @@ public boolean needEntryInfo() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 11; + return 12; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java index 7ac3295e6cf8e..b3eb7551f05a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java @@ -445,6 +445,7 @@ private void processLockResponse(UUID nodeId, GridNearLockResponse res) { boolean isRead, boolean retval, TransactionIsolation isolation, + long createTtl, long accessTtl ) { CacheOperationContext opCtx = ctx.operationContextPerCall(); @@ -455,6 +456,7 @@ private void processLockResponse(UUID nodeId, GridNearLockResponse res) { isRead, retval, timeout, + createTtl, accessTtl, CU.empty0(), opCtx != null && opCtx.skipStore(), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index ed37059b20cb8..67518ef1d5771 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -329,15 +329,20 @@ public void optimisticLockEntries(Collection optimisticLockEntrie final boolean skipVals, final boolean needVer, boolean keepBinary, + final ExpiryPolicy expiryPlc, final GridInClosure3 c ) { + IgniteCacheExpiryPolicy expiryPlc0 = optimistic() ? + accessPolicy(cacheCtx, keys) : + cacheCtx.cache().expiryPolicy(expiryPlc); + if (cacheCtx.isNear()) { return cacheCtx.nearTx().txLoadAsync(this, topVer, keys, readThrough, /*deserializeBinary*/false, - accessPolicy(cacheCtx, keys), + expiryPlc0, skipVals, needVer).chain(new C1>, Void>() { @Override public Void apply(IgniteInternalFuture> f) { @@ -368,7 +373,7 @@ else if (cacheCtx.isColocated()) { CU.subjectId(this, cctx), resolveTaskName(), /*deserializeBinary*/false, - accessPolicy(cacheCtx, keys), + expiryPlc0, skipVals, /*can remap*/true, needVer, @@ -399,7 +404,7 @@ else if (cacheCtx.isColocated()) { CU.subjectId(this, cctx), resolveTaskName(), /*deserializeBinary*/false, - accessPolicy(cacheCtx, keys), + expiryPlc0, skipVals, /*can remap*/true, needVer, @@ -433,6 +438,7 @@ else if (cacheCtx.isColocated()) { skipVals, keepBinary, needVer, + expiryPlc, c); } } @@ -1163,6 +1169,7 @@ public IgniteInternalFuture rollbackAsyncLocal() { * @param keys Keys. * @param retval Return value flag. * @param read Read flag. + * @param accessTtl Create ttl. * @param accessTtl Access ttl. * @param Key type. * @param skipStore Skip store flag. @@ -1173,6 +1180,7 @@ public IgniteInternalFuture lockAllAsync(GridCacheContext c final Collection keys, boolean retval, boolean read, + long createTtl, long accessTtl, boolean skipStore, boolean keepBinary) { @@ -1207,6 +1215,7 @@ public IgniteInternalFuture lockAllAsync(GridCacheContext c read, retval, isolation, + createTtl, accessTtl, CU.empty0(), skipStore, @@ -1305,6 +1314,8 @@ public IgniteInternalFuture lockAllAsync(GridCacheContext c /** {@inheritDoc} */ @Override protected IgniteCacheExpiryPolicy accessPolicy(GridCacheContext cacheCtx, Collection keys) { + assert optimistic(); + if (accessMap != null) { for (Map.Entry e : accessMap.entrySet()) { if (e.getKey().cacheId() == cacheCtx.cacheId() && keys.contains(e.getKey().key())) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java index 16a35d3482747..5b44d759f9080 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java @@ -118,6 +118,7 @@ GridLocalCacheEntry entryExx(KeyCacheObject key) { boolean retval, TransactionIsolation isolation, boolean invalidate, + long createTtl, long accessTtl) { return lockAllAsync(keys, timeout, tx, CU.empty0()); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java index ad818a63d8c21..ee4f7a63e1e0b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java @@ -1547,6 +1547,7 @@ private static CachePartialUpdateCheckedException partialUpdateException() { boolean retval, TransactionIsolation isolation, boolean invalidate, + long createTtl, long accessTtl) { return new GridFinishedFuture<>(new UnsupportedOperationException("Locks are not supported for " + "CacheAtomicityMode.ATOMIC mode (use CacheAtomicityMode.TRANSACTIONAL instead)")); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index f05d90df6cf10..a1c1123d8cfd5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -400,6 +400,7 @@ protected boolean commitAfterLock() { boolean skipVals, boolean needVer, boolean keepBinary, + final ExpiryPolicy expiryPlc, final GridInClosure3 c ) { assert cacheCtx.isLocal() : cacheCtx.name(); @@ -412,7 +413,9 @@ protected boolean commitAfterLock() { } try { - IgniteCacheExpiryPolicy expiryPlc = accessPolicy(cacheCtx, keys); + IgniteCacheExpiryPolicy expiryPlc0 = optimistic() ? + accessPolicy(cacheCtx, keys) : + cacheCtx.cache().expiryPolicy(expiryPlc); Map misses = null; @@ -437,7 +440,7 @@ protected boolean commitAfterLock() { CU.subjectId(this, cctx), null, resolveTaskName(), - expiryPlc, + expiryPlc0, txEntry == null ? keepBinary : txEntry.keepBinary(), null); @@ -481,6 +484,7 @@ protected boolean commitAfterLock() { T2 verVal = entry.versionedValue(cacheVal, ver, null, + null, null); if (log.isDebugEnabled()) { @@ -1446,6 +1450,7 @@ protected IgniteCacheExpiryPolicy accessPolicy(GridCacheContext cacheCtx, Collec * @param skipVals Skip values flag. * @param keepCacheObjects Keep cache objects flag. * @param skipStore Skip store flag. + * @param expiryPlc Expiry policy. * @return Loaded key-value pairs. */ private IgniteInternalFuture> checkMissed( @@ -1457,7 +1462,8 @@ private IgniteInternalFuture> checkMissed( final boolean skipVals, final boolean keepCacheObjects, final boolean skipStore, - final boolean needVer + final boolean needVer, + final ExpiryPolicy expiryPlc ) { if (log.isDebugEnabled()) @@ -1486,6 +1492,7 @@ private IgniteInternalFuture> checkMissed( skipVals, needReadVer, !deserializeBinary, + expiryPlc, new GridInClosure3() { @Override public void apply(KeyCacheObject key, Object val, GridCacheVersion loadVer) { if (isRollbackOnly()) { @@ -1610,6 +1617,7 @@ private IgniteInternalFuture> checkMissed( expiryPlc = cacheCtx.expiry(); long accessTtl = expiryPlc != null ? CU.toTtl(expiryPlc.getExpiryForAccess()) : CU.TTL_NOT_CHANGED; + long createTtl = expiryPlc != null ? CU.toTtl(expiryPlc.getExpiryForCreation()) : CU.TTL_NOT_CHANGED; long timeout = remainingTime(); @@ -1623,8 +1631,11 @@ private IgniteInternalFuture> checkMissed( true, isolation, isInvalidate(), + createTtl, accessTtl); + final ExpiryPolicy expiryPlc0 = expiryPlc; + PLC2> plc2 = new PLC2>() { @Override public IgniteInternalFuture> postLock() throws IgniteCheckedException { if (log.isDebugEnabled()) @@ -1747,7 +1758,8 @@ private IgniteInternalFuture> checkMissed( skipVals, keepCacheObjects, skipStore, - needVer); + needVer, + expiryPlc0); } return new GridFinishedFuture<>(Collections.emptyMap()); @@ -1820,7 +1832,8 @@ private IgniteInternalFuture> checkMissed( skipVals, keepCacheObjects, skipStore, - needVer); + needVer, + expiryPlc); } return new GridFinishedFuture<>(retMap); @@ -2027,7 +2040,8 @@ private IgniteInternalFuture enlistWrite( hasFilters, /*read through*/(entryProcessor != null || cacheCtx.config().isLoadPreviousValue()) && !skipStore, retval, - keepBinary); + keepBinary, + expiryPlc); } return new GridFinishedFuture<>(); @@ -2196,7 +2210,8 @@ else if (dataCenterId != null) { hasFilters, /*read through*/(invokeMap != null || cacheCtx.config().isLoadPreviousValue()) && !skipStore, retval, - keepBinary); + keepBinary, + expiryPlc); } return new GridFinishedFuture<>(); @@ -2216,6 +2231,7 @@ else if (dataCenterId != null) { * @param hasFilters {@code True} if filters not empty. * @param readThrough Read through flag. * @param retval Return value flag. + * @param expiryPlc Expiry policy. * @return Load future. */ private IgniteInternalFuture loadMissing( @@ -2229,7 +2245,8 @@ private IgniteInternalFuture loadMissing( final boolean hasFilters, final boolean readThrough, final boolean retval, - final boolean keepBinary) { + final boolean keepBinary, + final ExpiryPolicy expiryPlc) { GridInClosure3 c = new GridInClosure3() { @Override public void apply(KeyCacheObject key, @@ -2303,6 +2320,7 @@ private IgniteInternalFuture loadMissing( /*skipVals*/singleRmv, needReadVer, keepBinary, + expiryPlc, c); } @@ -2966,6 +2984,7 @@ private IgniteInternalFuture putAsync0( retval, isolation, isInvalidate(), + -1L, -1L); PLC1 plc1 = new PLC1(ret) { @@ -3144,6 +3163,7 @@ private IgniteInternalFuture putAllAsync0( retval, isolation, isInvalidate(), + -1L, -1L); PLC1 plc1 = new PLC1(ret) { @@ -3438,6 +3458,7 @@ private IgniteInternalFuture removeAllAsync0( retval, isolation, isInvalidate(), + -1L, -1L); PLC1 plc1 = new PLC1(ret) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java index 9fb3558a04b1f..f5687a0cafcba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java @@ -20,6 +20,7 @@ import java.util.Collection; import java.util.Map; import javax.cache.Cache; +import javax.cache.expiry.ExpiryPolicy; import javax.cache.processor.EntryProcessor; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; @@ -181,6 +182,7 @@ public IgniteInternalFuture removeAllDrAsync( * @param skipVals Skip values flag. * @param needVer If {@code true} version is required for loaded values. * @param c Closure to be applied for loaded values. + * @param expiryPlc Expiry policy. * @return Future with {@code True} value if loading took place. */ public IgniteInternalFuture loadMissing( @@ -192,5 +194,6 @@ public IgniteInternalFuture loadMissing( boolean skipVals, boolean needVer, boolean keepBinary, + final ExpiryPolicy expiryPlc, GridInClosure3 c); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java index 8db68b4ef50fb..2954bdb413d16 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java @@ -689,7 +689,8 @@ void recheckLock() { @Override public T2 versionedValue(CacheObject val, GridCacheVersion curVer, GridCacheVersion newVer, - @Nullable ReaderArguments readerArgs) { + @Nullable ReaderArguments readerArgs, + IgniteCacheExpiryPolicy loadExpiryPlc) { assert false; return null; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java index 78c59ac047b23..1f6ec2d297dd1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java @@ -17,6 +17,11 @@ package org.apache.ignite.internal.processors.cache.expiry; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; import javax.cache.configuration.Factory; import javax.cache.expiry.Duration; @@ -25,6 +30,7 @@ import javax.cache.processor.EntryProcessor; import javax.cache.processor.MutableEntry; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteTransactions; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.store.CacheStore; @@ -36,6 +42,9 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractTest; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; /** * @@ -173,6 +182,99 @@ public void testReadThrough() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testGetReadThrough() throws Exception { + getReadThrough(false, null, null); + getReadThrough(true, null, null); + } + + /** + * @throws Exception If failed. + */ + protected void getReadThrough(boolean withExcPlc, + TransactionConcurrency txConcurrency, + TransactionIsolation txIsolation) throws Exception { + IgniteCache cache = jcache(0); + + if (withExcPlc) + cache = cache.withExpiryPolicy(new ExpiryPolicy() { + @Override public Duration getExpiryForCreation() { + return new Duration(TimeUnit.MILLISECONDS, 501); + } + + @Override public Duration getExpiryForAccess() { + return new Duration(TimeUnit.MILLISECONDS, 601); + } + + @Override public Duration getExpiryForUpdate() { + return new Duration(TimeUnit.MILLISECONDS, 701); + } + }); + + Integer prim = primaryKeys(cache, 1, 1000).get(0); + Integer back = backupKeys(cache, 1, 1000).get(0); + Integer near = nearKeys(cache, 1, 1000).get(0); + + Set prims = new HashSet<>(primaryKeys(cache, 10, prim + 1)); + Set backs = new HashSet<>(backupKeys(cache, 10, back + 1)); + Set nears = new HashSet<>(nearKeys(cache, 10, near + 1)); + + Set keys = new HashSet<>(); + + keys.add(prim); + keys.add(back); + keys.add(near); + + keys.addAll(prims); + keys.addAll(backs); + keys.addAll(nears); + + for (Integer key : keys) + storeMap.put(key, key); + + IgniteTransactions transactions = grid(0).transactions(); + + Transaction tx = txConcurrency != null ? transactions.txStart(txConcurrency, txIsolation) : null; + + try { + Collection singleKeys = new HashSet<>(); + + singleKeys.add(prim); + singleKeys.add(back); + singleKeys.add(near); + + assertEquals(3, singleKeys.size()); + + for (Integer key : singleKeys) + assertEquals(key, cache.get(key)); + + Map res = new HashMap<>(); + + res.putAll(cache.getAll(prims)); + res.putAll(cache.getAll(backs)); + res.putAll(cache.getAll(nears)); + + assertEquals(30, res.size()); + + for (Map.Entry e : res.entrySet()) + assertEquals(e.getKey(), e.getValue()); + } + finally { + if (tx != null) + tx.rollback(); + } + + for (Integer key : keys) + checkTtl(key, withExcPlc ? 501 : 500, true); + + U.sleep(600); + + for (Integer key : keys) + checkExpired(key); + } + /** * @param key Key. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheTxExpiryPolicyWithStoreTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheTxExpiryPolicyWithStoreTest.java index 4b9b61a86496f..f5888f88ac696 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheTxExpiryPolicyWithStoreTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheTxExpiryPolicyWithStoreTest.java @@ -19,6 +19,8 @@ import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheMode.PARTITIONED; @@ -41,4 +43,23 @@ public class IgniteCacheTxExpiryPolicyWithStoreTest extends IgniteCacheExpiryPol @Override protected CacheAtomicityMode atomicityMode() { return TRANSACTIONAL; } + + /** {@inheritDoc} */ + @Override public void testGetReadThrough() throws Exception { + super.testGetReadThrough(); + + getReadThrough(false, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.READ_COMMITTED); + getReadThrough(true, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.READ_COMMITTED); + getReadThrough(false, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ); + getReadThrough(true, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ); + getReadThrough(false, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.SERIALIZABLE); + getReadThrough(true, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.SERIALIZABLE); + + getReadThrough(false, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.READ_COMMITTED); + getReadThrough(true, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.READ_COMMITTED); + getReadThrough(false, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ); + getReadThrough(true, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ); + getReadThrough(false, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.SERIALIZABLE); + getReadThrough(true, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.SERIALIZABLE); + } } \ No newline at end of file From f25f85afb9d9d5e91116086a6b3998a5bbb082b9 Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Wed, 25 Jan 2017 13:58:57 +0700 Subject: [PATCH 437/487] IGNITE-4520 Added credential request for authentication on proxy. (cherry picked from commit ef04f35) --- .../web-agent/bin/ignite-web-agent.bat | 4 +- .../web-agent/bin/ignite-web-agent.sh | 2 + .../ignite/console/agent/AgentLauncher.java | 90 +++++++++++++++++-- 3 files changed, 88 insertions(+), 8 deletions(-) diff --git a/modules/web-console/web-agent/bin/ignite-web-agent.bat b/modules/web-console/web-agent/bin/ignite-web-agent.bat index 8291b5531da50..1f1b52dc83d23 100644 --- a/modules/web-console/web-agent/bin/ignite-web-agent.bat +++ b/modules/web-console/web-agent/bin/ignite-web-agent.bat @@ -60,7 +60,9 @@ if %ERRORLEVEL% equ 0 ( if "%JVM_OPTS%" == "" set JVM_OPTS=-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxMetaspaceSize=256m ) -"%JAVA_HOME%\bin\java.exe" %JVM_OPTS% -cp "*" org.apache.ignite.console.agent.AgentLauncher %* +set JVM_OPTS=%JVM_OPTS% -Djava.net.useSystemProxies=true + +"%JAVA_HOME%\bin\java.exe" %JVM_OPTS% -cp "*" org.apache.ignite.console.agent.AgentLauncher %* set JAVA_ERRORLEVEL=%ERRORLEVEL% diff --git a/modules/web-console/web-agent/bin/ignite-web-agent.sh b/modules/web-console/web-agent/bin/ignite-web-agent.sh index 2e9f041f8ca7c..c2958fc0c7c9a 100644 --- a/modules/web-console/web-agent/bin/ignite-web-agent.sh +++ b/modules/web-console/web-agent/bin/ignite-web-agent.sh @@ -88,4 +88,6 @@ if [ -z "$JVM_OPTS" ] ; then fi fi +JVM_OPTS="${JVM_OPTS} -Djava.net.useSystemProxies=true" + "$JAVA" ${JVM_OPTS} -cp "*" org.apache.ignite.console.agent.AgentLauncher "$@" diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java index 0c03d7787b15d..049791f9107dc 100644 --- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java +++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java @@ -25,11 +25,15 @@ import io.socket.emitter.Emitter; import java.io.File; import java.io.IOException; +import java.net.Authenticator; import java.net.ConnectException; +import java.net.PasswordAuthentication; import java.net.URI; import java.net.URISyntaxException; import java.net.URL; +import java.net.UnknownHostException; import java.util.Arrays; +import java.util.Scanner; import java.util.concurrent.CountDownLatch; import java.util.jar.Attributes; import java.util.jar.Manifest; @@ -76,9 +80,6 @@ public class AgentLauncher { /** */ private static final String EVENT_AGENT_CLOSE = "agent:close"; - /** */ - private static final int RECONNECT_INTERVAL = 3000; - /** * Create a trust manager that trusts all certificates It is not using a particular keyStore */ @@ -121,6 +122,15 @@ public void checkServerTrusted( System.exit(1); } + ignore = X.cause(e, UnknownHostException.class); + + if (ignore != null) { + log.error("Failed to establish connection to server, due to errors with DNS or missing proxy settings."); + log.error("Documentation for proxy configuration can be found here: http://apacheignite.readme.io/docs/web-agent#section-proxy-configuration"); + + System.exit(1); + } + ignore = X.cause(e, IOException.class); if (ignore != null && "404".equals(ignore.getMessage())) { @@ -129,6 +139,29 @@ public void checkServerTrusted( return; } + if (ignore != null && "407".equals(ignore.getMessage())) { + log.error("Failed to establish connection to server, due to proxy requires authentication."); + + String userName = System.getProperty("https.proxyUsername", System.getProperty("http.proxyUsername")); + + if (userName == null || userName.trim().isEmpty()) + userName = readLine("Enter proxy user name: "); + else + System.out.println("Read username from system properties: " + userName); + + char[] pwd = readPassword("Enter proxy password: "); + + final PasswordAuthentication pwdAuth = new PasswordAuthentication(userName, pwd); + + Authenticator.setDefault(new Authenticator() { + @Override protected PasswordAuthentication getPasswordAuthentication() { + return pwdAuth; + } + }); + + return; + } + log.error("Connection error.", e); } } @@ -143,6 +176,32 @@ public void checkServerTrusted( } }; + /** + * @param fmt Format string. + * @param args Arguments. + */ + private static String readLine(String fmt, Object ... args) { + if (System.console() != null) + return System.console().readLine(fmt, args); + + System.out.print(String.format(fmt, args)); + + return new Scanner(System.in).nextLine(); + } + + /** + * @param fmt Format string. + * @param args Arguments. + */ + private static char[] readPassword(String fmt, Object ... args) { + if (System.console() != null) + return System.console().readPassword(fmt, args); + + System.out.print(String.format(fmt, args)); + + return new Scanner(System.in).nextLine().toCharArray(); + } + /** * @param args Args. */ @@ -214,9 +273,9 @@ public static void main(String[] args) throws Exception { System.out.println("Security token is required to establish connection to the web console."); System.out.println(String.format("It is available on the Profile page: https://%s/profile", webHost)); - System.out.print("Enter security tokens separated by comma: "); + String tokens = String.valueOf(readPassword("Enter security tokens separated by comma: ")); - cfg.tokens(Arrays.asList(System.console().readLine().trim().split(","))); + cfg.tokens(Arrays.asList(tokens.trim().split(","))); } final RestHandler restHnd = new RestHandler(cfg); @@ -226,12 +285,29 @@ public static void main(String[] args) throws Exception { URI uri = URI.create(cfg.serverUri()); + // Create proxy authenticator using passed properties. + switch (uri.getScheme()) { + case "http": + case "https": + final String username = System.getProperty(uri.getScheme() + ".proxyUsername"); + final char[] pwd = System.getProperty(uri.getScheme() + ".proxyPassword", "").toCharArray(); + + Authenticator.setDefault(new Authenticator() { + @Override protected PasswordAuthentication getPasswordAuthentication() { + return new PasswordAuthentication(username, pwd); + } + }); + + break; + + default: + // No-op. + } + IO.Options opts = new IO.Options(); opts.path = "/agents"; - opts.reconnectionDelay = RECONNECT_INTERVAL; - // Workaround for use self-signed certificate if (Boolean.getBoolean("trust.all")) { SSLContext ctx = SSLContext.getInstance("TLS"); From ccc0f41794bfd2deecc3e0bd042f82d81c9809d0 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Fri, 20 Jan 2017 16:22:24 +0700 Subject: [PATCH 438/487] IGNITE-4548 CacheJdbcStore: support mapping of enum types. (cherry picked from commit f1fca3a) --- .../store/jdbc/CacheAbstractJdbcStore.java | 12 +++++- .../jdbc/JdbcTypesDefaultTransformer.java | 19 +++++++++ .../store/jdbc/JdbcTypesTransformer.java | 17 ++++++++ .../CacheJdbcPojoStoreAbstractSelfTest.java | 23 +++++++---- .../store/jdbc/CacheJdbcPojoStoreTest.java | 3 ++ ...dbcStoreAbstractMultithreadedSelfTest.java | 17 ++++---- .../ignite/cache/store/jdbc/model/Gender.java | 41 +++++++++++++++++++ .../ignite/cache/store/jdbc/model/Person.java | 31 +++++++++++++- .../test/config/jdbc-pojo-store-builtin.xml | 8 ++++ .../src/test/config/jdbc-pojo-store-obj.xml | 8 ++++ 10 files changed, 162 insertions(+), 17 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Gender.java diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java index 4bfd92b836268..e7ce52651a868 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java @@ -80,6 +80,8 @@ import static org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory.DFLT_BATCH_SIZE; import static org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory.DFLT_PARALLEL_LOAD_CACHE_MINIMUM_THRESHOLD; import static org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory.DFLT_WRITE_ATTEMPTS; +import static org.apache.ignite.cache.store.jdbc.JdbcTypesTransformer.NUMERIC_TYPES; +import static org.apache.ignite.cache.store.jdbc.JdbcTypesTransformer.NUMERIC_TYPES; /** * Implementation of {@link CacheStore} backed by JDBC. @@ -1393,8 +1395,15 @@ protected void fillParameter(PreparedStatement stmt, int idx, JdbcTypeField fiel fieldVal = fieldVal.toString(); break; + default: + // No-op. } } + else if (field.getJavaFieldType().isEnum() && fieldVal instanceof Enum) { + Enum val = (Enum)fieldVal; + + fieldVal = NUMERIC_TYPES.contains(field.getDatabaseFieldType()) ? val.ordinal() : val.name(); + } stmt.setObject(idx, fieldVal); } @@ -2068,12 +2077,13 @@ private LoadWorker(Connection conn, EntryMapping em) { int idx = 1; - for (Object key : keys) + for (Object key : keys) { for (JdbcTypeField field : em.keyColumns()) { Object fieldVal = extractParameter(em.cacheName, em.keyType(), em.keyKind(), field.getJavaFieldName(), key); fillParameter(stmt, idx++, field, fieldVal); } + } ResultSet rs = stmt.executeQuery(); diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcTypesDefaultTransformer.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcTypesDefaultTransformer.java index c32eaa227dfd0..c387b779e60c4 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcTypesDefaultTransformer.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcTypesDefaultTransformer.java @@ -114,6 +114,25 @@ public class JdbcTypesDefaultTransformer implements JdbcTypesTransformer { return UUID.fromString((String)res); } + if (type.isEnum()) { + if (NUMERIC_TYPES.contains(rs.getMetaData().getColumnType(colIdx))) { + int ordinal = rs.getInt(colIdx); + + Object[] values = type.getEnumConstants(); + + return rs.wasNull() || ordinal >= values.length ? null : values[ordinal]; + } + + String str = rs.getString(colIdx); + + try { + return rs.wasNull() ? null : Enum.valueOf((Class) type, str.trim()); + } + catch (IllegalArgumentException ignore) { + return null; + } + } + return rs.getObject(colIdx); } } diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcTypesTransformer.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcTypesTransformer.java index 76fb00b90b2ed..fc0bc883d0778 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcTypesTransformer.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/JdbcTypesTransformer.java @@ -20,11 +20,28 @@ import java.io.Serializable; import java.sql.ResultSet; import java.sql.SQLException; +import java.util.List; +import org.apache.ignite.internal.util.typedef.internal.U; + +import static java.sql.Types.BIGINT; +import static java.sql.Types.DECIMAL; +import static java.sql.Types.DOUBLE; +import static java.sql.Types.FLOAT; +import static java.sql.Types.INTEGER; +import static java.sql.Types.NUMERIC; +import static java.sql.Types.REAL; +import static java.sql.Types.SMALLINT; +import static java.sql.Types.TINYINT; /** * API for implementing custom mapping logic for loaded from store data. */ public interface JdbcTypesTransformer extends Serializable { + /** Numeric types. */ + public final List NUMERIC_TYPES = + U.sealList(TINYINT, SMALLINT, INTEGER, BIGINT, REAL, FLOAT, DOUBLE, NUMERIC, DECIMAL); + + /** * Retrieves the value of the designated column in the current row of this ResultSet object and * will convert to the requested Java data type. diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java index 368a28e010c3b..1de44f7243fbb 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java @@ -30,6 +30,7 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.store.jdbc.dialect.H2Dialect; import org.apache.ignite.cache.store.jdbc.model.Person; +import org.apache.ignite.cache.store.jdbc.model.Gender; import org.apache.ignite.cache.store.jdbc.model.PersonKey; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.ConnectorConfiguration; @@ -112,7 +113,8 @@ protected Connection getConnection() throws SQLException { " id INTEGER PRIMARY KEY," + " org_id INTEGER," + " birthday DATE," + - " name VARCHAR(50))"); + " name VARCHAR(50)," + + " gender VARCHAR(50))"); conn.commit(); @@ -201,7 +203,8 @@ protected JdbcType[] storeTypes() { new JdbcTypeField(Types.INTEGER, "ID", Integer.class, "id"), new JdbcTypeField(Types.INTEGER, "ORG_ID", Integer.class, "orgId"), new JdbcTypeField(Types.DATE, "BIRTHDAY", Date.class, "birthday"), - new JdbcTypeField(Types.VARCHAR, "NAME", String.class, "name")); + new JdbcTypeField(Types.VARCHAR, "NAME", String.class, "name"), + new JdbcTypeField(Types.VARCHAR, "GENDER", Gender.class, "gender")); return storeTypes; } @@ -260,7 +263,7 @@ protected void fillSampleDatabase(Connection conn) throws SQLException { conn.commit(); PreparedStatement prnStmt = conn.prepareStatement( - "INSERT INTO Person(id, org_id, birthday, name) VALUES (?, ?, ?, ?)"); + "INSERT INTO Person(id, org_id, birthday, name, gender) VALUES (?, ?, ?, ?, ?)"); Random rnd = new Random(); @@ -269,6 +272,7 @@ protected void fillSampleDatabase(Connection conn) throws SQLException { prnStmt.setInt(2, i % 100); prnStmt.setDate(3, Date.valueOf(String.format("%d-%d-%d", 1970 + rnd.nextInt(50), 1 + rnd.nextInt(11), 1 + rnd.nextInt(27)))); prnStmt.setString(4, "name" + i); + prnStmt.setString(5, Gender.random().toString()); prnStmt.addBatch(); } @@ -319,7 +323,7 @@ protected void checkCacheLoad() { protected void checkCacheLoadWithSql() { IgniteCache c1 = grid().cache(CACHE_NAME); - c1.loadCache(null, "org.apache.ignite.cache.store.jdbc.model.PersonKey", "select id, org_id, name, birthday from Person"); + c1.loadCache(null, "org.apache.ignite.cache.store.jdbc.model.PersonKey", "select id, org_id, name, birthday, gender from Person"); assertEquals(PERSON_CNT, c1.size()); } @@ -397,7 +401,9 @@ private void checkPutRemove() throws Exception { Connection conn = getConnection(); try { - PreparedStatement stmt = conn.prepareStatement("SELECT ID, ORG_ID, BIRTHDAY, NAME FROM PERSON WHERE ID = ?"); + Random rnd = new Random(); + + PreparedStatement stmt = conn.prepareStatement("SELECT ID, ORG_ID, BIRTHDAY, NAME, GENDER FROM PERSON WHERE ID = ?"); stmt.setInt(1, -1); @@ -408,8 +414,9 @@ private void checkPutRemove() throws Exception { U.closeQuiet(rs); Date testDate = Date.valueOf("2001-05-05"); + Gender testGender = Gender.random(); - Person val = new Person(-1, -2, testDate, "Person-to-test-put-insert", 999); + Person val = new Person(-1, -2, testDate, "Person-to-test-put-insert", 999, testGender); Object key = builtinKeys ? Integer.valueOf(-1) : new PersonKey(-1); @@ -424,6 +431,7 @@ private void checkPutRemove() throws Exception { assertEquals(-2, rs.getInt(2)); assertEquals(testDate, rs.getDate(3)); assertEquals("Person-to-test-put-insert", rs.getString(4)); + assertEquals(testGender.toString(), rs.getString(5)); assertFalse("Unexpected more data in result set", rs.next()); @@ -432,7 +440,7 @@ private void checkPutRemove() throws Exception { // Test put-update. testDate = Date.valueOf("2016-04-04"); - c1.put(key, new Person(-1, -3, testDate, "Person-to-test-put-update", 999)); + c1.put(key, new Person(-1, -3, testDate, "Person-to-test-put-update", 999, testGender)); rs = stmt.executeQuery(); @@ -442,6 +450,7 @@ private void checkPutRemove() throws Exception { assertEquals(-3, rs.getInt(2)); assertEquals(testDate, rs.getDate(3)); assertEquals("Person-to-test-put-update", rs.getString(4)); + assertEquals(testGender.toString(), rs.getString(5)); assertFalse("Unexpected more data in result set", rs.next()); diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java index 4a0b1daf646d1..849cab79a03be 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java @@ -25,6 +25,7 @@ import java.sql.Types; import java.util.ArrayList; import java.util.Collection; +import java.util.Random; import java.util.UUID; import java.util.concurrent.ConcurrentLinkedQueue; import javax.cache.integration.CacheWriterException; @@ -233,6 +234,8 @@ public CacheJdbcPojoStoreTest() throws Exception { public void testLoadCache() throws Exception { Connection conn = store.openConnection(false); + Random rnd = new Random(); + PreparedStatement orgStmt = conn.prepareStatement("INSERT INTO Organization(id, name, city) VALUES (?, ?, ?)"); for (int i = 0; i < ORGANIZATION_CNT; i++) { diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreAbstractMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreAbstractMultithreadedSelfTest.java index e831445c896da..f1a321ba01216 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreAbstractMultithreadedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreAbstractMultithreadedSelfTest.java @@ -33,6 +33,7 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheTypeMetadata; +import org.apache.ignite.cache.store.jdbc.model.Gender; import org.apache.ignite.cache.store.jdbc.model.Organization; import org.apache.ignite.cache.store.jdbc.model.OrganizationKey; import org.apache.ignite.cache.store.jdbc.model.Person; @@ -208,7 +209,7 @@ public void testMultithreadedPut() throws Exception { cache.put(new OrganizationKey(id), new Organization(id, "Name" + id, "City" + id)); else cache.put(new PersonKey(id), new Person(id, rnd.nextInt(), - new Date(System.currentTimeMillis()), "Name" + id, 1)); + new Date(System.currentTimeMillis()), "Name" + id, 1, Gender.random())); } return null; @@ -228,7 +229,7 @@ public void testMultithreadedPut() throws Exception { cache.putIfAbsent(new OrganizationKey(id), new Organization(id, "Name" + id, "City" + id)); else cache.putIfAbsent(new PersonKey(id), new Person(id, rnd.nextInt(), - new Date(System.currentTimeMillis()), "Name" + id, i)); + new Date(System.currentTimeMillis()), "Name" + id, i, Gender.random())); } return null; @@ -268,7 +269,7 @@ public void testMultithreadedPutAll() throws Exception { map.put(new OrganizationKey(id), new Organization(id, "Name" + id, "City" + id)); else map.put(new PersonKey(id), new Person(id, rnd.nextInt(), - new Date(System.currentTimeMillis()), "Name" + id, 1)); + new Date(System.currentTimeMillis()), "Name" + id, 1, Gender.random())); } IgniteCache cache = jcache(); @@ -294,11 +295,11 @@ public void testMultithreadedExplicitTx() throws Exception { try (Transaction tx = grid().transactions().txStart()) { cache.put(new PersonKey(1), new Person(1, rnd.nextInt(), - new Date(System.currentTimeMillis()), "Name" + 1, 1)); + new Date(System.currentTimeMillis()), "Name" + 1, 1, Gender.random())); cache.put(new PersonKey(2), new Person(2, rnd.nextInt(), - new Date(System.currentTimeMillis()), "Name" + 2, 2)); + new Date(System.currentTimeMillis()), "Name" + 2, 2, Gender.random())); cache.put(new PersonKey(3), new Person(3, rnd.nextInt(), - new Date(System.currentTimeMillis()), "Name" + 3, 3)); + new Date(System.currentTimeMillis()), "Name" + 3, 3, Gender.random())); cache.get(new PersonKey(1)); cache.get(new PersonKey(4)); @@ -306,9 +307,9 @@ public void testMultithreadedExplicitTx() throws Exception { Map map = U.newHashMap(2); map.put(new PersonKey(5), new Person(5, rnd.nextInt(), - new Date(System.currentTimeMillis()), "Name" + 5, 5)); + new Date(System.currentTimeMillis()), "Name" + 5, 5, Gender.random())); map.put(new PersonKey(6), new Person(6, rnd.nextInt(), - new Date(System.currentTimeMillis()), "Name" + 6, 6)); + new Date(System.currentTimeMillis()), "Name" + 6, 6, Gender.random())); cache.putAll(map); diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Gender.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Gender.java new file mode 100644 index 0000000000000..8ddb0e2a3670e --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Gender.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.cache.store.jdbc.model; + +import java.io.Serializable; +import java.util.Random; + +/** + * Person gender enum. + */ +public enum Gender implements Serializable { + /** */ + MALE, + /** */ + FEMALE; + + /** */ + private static final Random RAND = new Random(); + + /** + * Used for testing purposes. + */ + public static Gender random() { + return values()[RAND.nextInt(values().length)]; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java index 52ddfc8ecdcbb..89258b403de23 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java @@ -44,6 +44,9 @@ public class Person implements Serializable { /** Value for salary. */ private Integer salary; + /** Value of person gender. */ + private Gender gender; + /** * Empty constructor. */ @@ -59,13 +62,15 @@ public Person( Integer orgId, Date birthday, String name, - Integer salary + Integer salary, + Gender gender ) { this.id = id; this.orgId = orgId; this.birthday = birthday; this.name = name; this.salary = salary; + this.gender = gender; } /** @@ -159,6 +164,24 @@ public void setSalary(Integer salary) { this.salary = salary; } + /** + * Gets gender. + * + * @return Gender. + */ + public Gender getGender() { + return gender; + } + + /** + * Sets gender. + * + * @param gender New value for gender. + */ + public void setGender(Gender gender) { + this.gender = gender; + } + /** {@inheritDoc} */ @Override public boolean equals(Object o) { if (this == o) @@ -178,6 +201,9 @@ public void setSalary(Integer salary) { if (name != null ? !name.equals(that.name) : that.name != null) return false; + if (gender != null ? !gender.equals(that.gender) : that.gender != null) + return false; + return true; } @@ -189,6 +215,8 @@ public void setSalary(Integer salary) { res = 31 * res + (name != null ? name.hashCode() : 0); + res = 31 * res + (gender != null ? gender.hashCode() : 0); + return res; } @@ -198,6 +226,7 @@ public void setSalary(Integer salary) { ", orgId=" + orgId + ", birthday=" + (birthday == null ? null : birthday.getTime()) + ", name=" + name + + ", gender=" + gender + "]"; } } diff --git a/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml b/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml index dfaf828fa8d22..bfb109c035d94 100644 --- a/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml +++ b/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml @@ -151,6 +151,14 @@ + + + + + + + + diff --git a/modules/spring/src/test/config/jdbc-pojo-store-obj.xml b/modules/spring/src/test/config/jdbc-pojo-store-obj.xml index 9bc99776a5bec..40a14dc4f4fda 100644 --- a/modules/spring/src/test/config/jdbc-pojo-store-obj.xml +++ b/modules/spring/src/test/config/jdbc-pojo-store-obj.xml @@ -151,6 +151,14 @@ + + + + + + + + From 6f6ff397c8a2fb3f1041958a3338294cb44af998 Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Wed, 25 Jan 2017 16:48:05 +0700 Subject: [PATCH 439/487] IGNITE-1596 Fixed version sort. (cherry picked from commit 128ba07) --- modules/web-console/backend/app/agent.js | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/modules/web-console/backend/app/agent.js b/modules/web-console/backend/app/agent.js index 791ea50ed58b9..961253f148a70 100644 --- a/modules/web-console/backend/app/agent.js +++ b/modules/web-console/backend/app/agent.js @@ -650,14 +650,14 @@ module.exports.factory = function(_, fs, path, JSZip, socketio, settings, mongo) const bParts = b.split('.'); for (let i = 0; i < aParts.length; ++i) { - if (bParts.length === i) - return 1; + if (aParts[i] !== bParts[i]) + return aParts[i] < bParts[i] ? 1 : -1; + } - if (aParts[i] === aParts[i]) - continue; + if (aParts.length === bParts.length) + return 0; - return aParts[i] > bParts[i] ? 1 : -1; - } + return aParts.length < bParts.length ? 1 : -1; })); // Latest version of agent distribution. From f5e601e2973bfa81593241e55e4b6f97c0e55c3c Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Thu, 26 Jan 2017 10:18:34 +0300 Subject: [PATCH 440/487] IGNITE-4036 - Fix. Near cache is not expired together with corresponding server cache --- .../processors/cache/EntryGetResult.java | 40 +++++- .../cache/EntryGetWithTtlResult.java | 58 +++++++++ .../processors/cache/GridCacheAdapter.java | 69 +++++----- .../processors/cache/GridCacheContext.java | 122 ++++++++++++++---- .../processors/cache/GridCacheEntryEx.java | 2 +- .../processors/cache/GridCacheMapEntry.java | 29 +++-- .../distributed/dht/GridDhtCacheAdapter.java | 4 +- .../distributed/dht/GridDhtGetFuture.java | 24 ++-- .../dht/GridDhtGetSingleFuture.java | 24 ++-- .../dht/GridPartitionedGetFuture.java | 19 ++- .../dht/GridPartitionedSingleGetFuture.java | 7 +- .../dht/atomic/GridDhtAtomicCache.java | 12 +- .../dht/colocated/GridDhtColocatedCache.java | 15 ++- .../distributed/near/GridNearGetFuture.java | 9 +- .../distributed/near/GridNearTxLocal.java | 8 +- .../local/atomic/GridLocalAtomicCache.java | 13 +- .../transactions/IgniteTxLocalAdapter.java | 62 ++++++--- .../cache/GridCacheTestEntryEx.java | 2 +- .../IgniteCacheExpiryPolicyAbstractTest.java | 44 ++++++- 19 files changed, 411 insertions(+), 152 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetWithTtlResult.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetResult.java index a34ddae6d10af..9d064484f1962 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetResult.java @@ -24,7 +24,7 @@ */ public class EntryGetResult { /** */ - private final CacheObject val; + private Object val; /** */ private final GridCacheVersion ver; @@ -35,18 +35,34 @@ public class EntryGetResult { /** * @param val Value. * @param ver Version. + * @param reserved Reserved flag. */ - EntryGetResult(CacheObject val, GridCacheVersion ver, boolean reserved) { + public EntryGetResult(Object val, GridCacheVersion ver, boolean reserved) { this.val = val; this.ver = ver; this.reserved = reserved; } + /** + * @param val Value. + * @param ver Version. + */ + public EntryGetResult(Object val, GridCacheVersion ver) { + this(val, ver, false); + } + /** * @return Value. */ - public CacheObject value() { - return val; + public T value() { + return (T)val; + } + + /** + * @param val Value. + */ + public void value(Object val) { + this.val = val; } /** @@ -57,9 +73,23 @@ public GridCacheVersion version() { } /** - * @return Reserved flag, + * @return Reserved flag. */ public boolean reserved() { return reserved; } + + /** + * @return Entry expire time. + */ + public long expireTime() { + return 0L; + } + + /** + * @return Entry time to live. + */ + public long ttl() { + return 0L; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetWithTtlResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetWithTtlResult.java new file mode 100644 index 0000000000000..fddf16e1de43e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetWithTtlResult.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; + +/** + * + */ +public class EntryGetWithTtlResult extends EntryGetResult { + /** */ + private final long expireTime; + + /** */ + private final long ttl; + + /** + * @param val Value. + * @param ver Version. + * @param reserved Reserved flag. + * @param expireTime Entry expire time. + * @param ttl Entry time to live. + */ + public EntryGetWithTtlResult(Object val, GridCacheVersion ver, boolean reserved, long expireTime, long ttl) { + super(val, ver, reserved); + this.expireTime = expireTime; + this.ttl = ttl; + } + + /** + * @return Entry expire time. + */ + @Override public long expireTime() { + return expireTime; + } + + /** + * @return Entry time to live. + */ + @Override public long ttl() { + return ttl; + } +} 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 dc8f0308a0099..11bf34b221bb1 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 @@ -1421,12 +1421,13 @@ public V getTopologySafe(K key) throws IgniteCheckedException { if (keepBinary) key = (K)ctx.toCacheKeyObject(key); - T2 t = (T2)get(key, !keepBinary, true); + EntryGetResult t + = (EntryGetResult)get(key, !keepBinary, true); CacheEntry val = t != null ? new CacheEntryImplEx<>( keepBinary ? (K)ctx.unwrapBinaryIfNeeded(key, true, false) : key, - t.get1(), - t.get2()) + (V)t.value(), + t.version()) : null; if (ctx.config().getInterceptor() != null) { @@ -1434,7 +1435,7 @@ public V getTopologySafe(K key) throws IgniteCheckedException { V val0 = (V)ctx.config().getInterceptor().onGet(key, t != null ? val.getValue() : null); - val = (val0 != null) ? new CacheEntryImplEx<>(key, val0, t != null ? t.get2() : null) : null; + val = (val0 != null) ? new CacheEntryImplEx<>(key, val0, t != null ? t.version() : null) : null; } if (statsEnabled) @@ -1484,29 +1485,29 @@ public V getTopologySafe(K key) throws IgniteCheckedException { final K key0 = keepBinary ? (K)ctx.toCacheKeyObject(key) : key; - IgniteInternalFuture> fut = - (IgniteInternalFuture>)getAsync(key0, !keepBinary, true); + IgniteInternalFuture fut = + (IgniteInternalFuture)getAsync(key0, !keepBinary, true); final boolean intercept = ctx.config().getInterceptor() != null; IgniteInternalFuture> fr = fut.chain( - new CX1>, CacheEntry>() { - @Override public CacheEntry applyx(IgniteInternalFuture> f) + new CX1, CacheEntry>() { + @Override public CacheEntry applyx(IgniteInternalFuture f) throws IgniteCheckedException { - T2 t = f.get(); + EntryGetResult t = f.get(); K key = keepBinary ? (K)ctx.unwrapBinaryIfNeeded(key0, true, false) : key0; CacheEntry val = t != null ? new CacheEntryImplEx<>( key, - t.get1(), - t.get2()) + t.value(), + t.version()) : null; if (intercept) { V val0 = (V)ctx.config().getInterceptor().onGet(key, t != null ? val.getValue() : null); - return val0 != null ? new CacheEntryImplEx(key, val0, t != null ? t.get2() : null) : null; + return val0 != null ? new CacheEntryImplEx(key, val0, t != null ? t.version() : null) : null; } else return val; @@ -1514,7 +1515,7 @@ public V getTopologySafe(K key) throws IgniteCheckedException { }); if (statsEnabled) - fut.listen(new UpdateGetTimeStatClosure>(metrics0(), start)); + fut.listen(new UpdateGetTimeStatClosure(metrics0(), start)); return fr; } @@ -1547,15 +1548,15 @@ public V getTopologySafe(K key) throws IgniteCheckedException { long start = statsEnabled ? System.nanoTime() : 0L; - Map> map = (Map>)getAll(keys, !ctx.keepBinary(), true); + Map map = (Map)getAll(keys, !ctx.keepBinary(), true); Collection> res = new HashSet<>(); if (ctx.config().getInterceptor() != null) res = interceptGetEntries(keys, map); else - for (Map.Entry> e : map.entrySet()) - res.add(new CacheEntryImplEx<>(e.getKey(), e.getValue().get1(), e.getValue().get2())); + for (Map.Entry e : map.entrySet()) + res.add(new CacheEntryImplEx<>(e.getKey(), (V)e.getValue().value(), e.getValue().version())); if (statsEnabled) metrics0().addGetTimeNanos(System.nanoTime() - start); @@ -1595,24 +1596,24 @@ public V getTopologySafe(K key) throws IgniteCheckedException { final long start = statsEnabled ? System.nanoTime() : 0L; - IgniteInternalFuture>> fut = - (IgniteInternalFuture>>) + IgniteInternalFuture> fut = + (IgniteInternalFuture>) ((IgniteInternalFuture)getAllAsync(keys, !ctx.keepBinary(), true)); final boolean intercept = ctx.config().getInterceptor() != null; IgniteInternalFuture>> rf = - fut.chain(new CX1>>, Collection>>() { + fut.chain(new CX1>, Collection>>() { @Override public Collection> applyx( - IgniteInternalFuture>> f) throws IgniteCheckedException { + IgniteInternalFuture> f) throws IgniteCheckedException { if (intercept) return interceptGetEntries(keys, f.get()); else { Map> res = U.newHashMap(f.get().size()); - for (Map.Entry> e : f.get().entrySet()) + for (Map.Entry e : f.get().entrySet()) res.put(e.getKey(), - new CacheEntryImplEx<>(e.getKey(), e.getValue().get1(), e.getValue().get2())); + new CacheEntryImplEx<>(e.getKey(), (V)e.getValue().value(), e.getValue().version())); return res.values(); } @@ -1620,7 +1621,7 @@ public V getTopologySafe(K key) throws IgniteCheckedException { }); if (statsEnabled) - fut.listen(new UpdateGetTimeStatClosure>>(metrics0(), start)); + fut.listen(new UpdateGetTimeStatClosure>(metrics0(), start)); return rf; } @@ -1675,7 +1676,7 @@ private Map interceptGet(@Nullable Collection keys, Map */ @SuppressWarnings("IfMayBeConditional") private Collection> interceptGetEntries( - @Nullable Collection keys, Map> map) { + @Nullable Collection keys, Map map) { Map> res; if (F.isEmpty(keys)) { @@ -1690,11 +1691,11 @@ private Collection> interceptGetEntries( assert interceptor != null; - for (Map.Entry> e : map.entrySet()) { - V val = interceptor.onGet(e.getKey(), e.getValue().get1()); + for (Map.Entry e : map.entrySet()) { + V val = interceptor.onGet(e.getKey(), (V)e.getValue().value()); if (val != null) - res.put(e.getKey(), new CacheEntryImplEx<>(e.getKey(), val, e.getValue().get2())); + res.put(e.getKey(), new CacheEntryImplEx<>(e.getKey(), val, e.getValue().version())); } if (map.size() != keys.size()) { // Not all requested keys were in cache. @@ -1976,12 +1977,12 @@ protected final IgniteInternalFuture> getAllAsync0( if (res != null) { ctx.addResult(map, key, - res.value(), + res, skipVals, keepCacheObjects, deserializeBinary, true, - needVer ? res.version() : null); + needVer); if (tx == null || (!tx.implicit() && tx.isolation() == READ_COMMITTED)) ctx.evicts().touch(entry, topVer); @@ -2025,7 +2026,7 @@ protected final IgniteInternalFuture> getAllAsync0( GridCacheEntryEx entry = entryEx(key); try { - T2 verVal = entry.versionedValue( + EntryGetResult verVal = entry.versionedValue( cacheVal, res.version(), null, @@ -2035,19 +2036,19 @@ protected final IgniteInternalFuture> getAllAsync0( if (log.isDebugEnabled()) log.debug("Set value loaded from store into entry [" + "oldVer=" + res.version() + - ", newVer=" + verVal.get2() + ", " + + ", newVer=" + verVal.version() + ", " + "entry=" + entry + ']'); // Don't put key-value pair into result map if value is null. - if (verVal.get1() != null) { + if (verVal.value() != null) { ctx.addResult(map, key, - verVal.get1(), + verVal, skipVals, keepCacheObjects, deserializeBinary, true, - needVer ? verVal.get2() : null); + needVer); } if (tx0 == null || (!tx0.implicit() && 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 424e325dd07f2..6322f9f03550a 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 @@ -90,7 +90,6 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.GPC; @@ -1895,7 +1894,65 @@ public void addResult(Map map, boolean keepCacheObjects, boolean deserializeBinary, boolean cpy, - final GridCacheVersion ver) { + final GridCacheVersion ver, + final long expireTime, + final long ttl) { + // Creates EntryGetResult + addResult(map, key, val, skipVals, keepCacheObjects, deserializeBinary, cpy, null, + ver, expireTime, ttl, ver != null); + } + + /** + * @param map Map. + * @param key Key. + * @param getRes EntryGetResult. + * @param skipVals Skip values. + * @param keepCacheObjects Keep CacheObject. + * @param deserializeBinary Deserialize binary flag. + * @param cpy Copy flag. + * @param needVer Need version flag. + */ + @SuppressWarnings("unchecked") + public void addResult(Map map, + KeyCacheObject key, + EntryGetResult getRes, + boolean skipVals, + boolean keepCacheObjects, + boolean deserializeBinary, + boolean cpy, + boolean needVer) { + // Uses getRes as result. + addResult(map, key, getRes.value(), skipVals, keepCacheObjects, deserializeBinary, cpy, getRes, + null, 0, 0, needVer); + } + + /** + * @param map Map. + * @param key Key. + * @param val Value. + * @param skipVals Skip values. + * @param keepCacheObjects Keep CacheObject. + * @param deserializeBinary Deserialize binary. + * @param cpy Copy flag. + * @param getRes EntryGetResult. + * @param ver Version. + * @param expireTime Entry expire time. + * @param ttl Entry TTL. + * @param needVer Need version flag. + */ + @SuppressWarnings("unchecked") + public void addResult(Map map, + KeyCacheObject key, + CacheObject val, + boolean skipVals, + boolean keepCacheObjects, + boolean deserializeBinary, + boolean cpy, + @Nullable EntryGetResult getRes, + final GridCacheVersion ver, + final long expireTime, + final long ttl, + boolean needVer) { assert key != null; assert val != null || skipVals; @@ -1907,32 +1964,53 @@ public void addResult(Map map, assert key0 != null : key; assert val0 != null : val; - map.put((K1)key0, ver != null ? (V1)new T2<>(val0, ver) : (V1)val0); + V1 v = createValue(ver, expireTime, ttl, val0, getRes, needVer); + + map.put((K1)key0, v); + } + else { + Object val0 = skipVals ? true : val; + + V1 v = createValue(ver, expireTime, ttl, val0, getRes, needVer); + + map.put((K1)key, v); } - else - map.put((K1)key, - (V1)(ver != null ? - (V1)new T2<>(skipVals ? true : val, ver) : - skipVals ? true : val)); } /** - * @param map Map. - * @param key Key. + * Creates new EntryGetResult or uses existing one. + * + * @param ver Version. + * @param expireTime Entry expire time. + * @param ttl Entry TTL. * @param val Value. - * @param skipVals Skip values flag. - * @param keepCacheObjects Keep cache objects flag. - * @param deserializeBinary Deserialize binary flag. - * @param cpy Copy flag. + * @param getRes EntryGetResult + * @param needVer Need version flag. + * @return EntryGetResult or value. */ - public void addResult(Map map, - KeyCacheObject key, - CacheObject val, - boolean skipVals, - boolean keepCacheObjects, - boolean deserializeBinary, - boolean cpy) { - addResult(map, key, val, skipVals, keepCacheObjects, deserializeBinary, cpy, null); + @SuppressWarnings("unchecked") + private V1 createValue(final GridCacheVersion ver, + final long expireTime, + final long ttl, + final Object val, + @Nullable final EntryGetResult getRes, + final boolean needVer) { + final V1 v; + + if (!needVer) + v = (V1) val; + else if (getRes == null) { + v = expireTime != 0 || ttl != 0 + ? (V1)new EntryGetWithTtlResult(val, ver, false, expireTime, ttl) + : (V1)new EntryGetResult(val, ver, false); + } + else { + getRes.value(val); + + v = (V1)getRes; + } + + return v; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java index 3c42d5360a4b1..ccd22850d9eb7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java @@ -761,7 +761,7 @@ public GridCacheVersionedEntryEx versionedEntry(final boolean keepB * @throws IgniteCheckedException If index could not be updated. * @throws GridCacheEntryRemovedException If entry was removed. */ - public T2 versionedValue(CacheObject val, + public EntryGetResult versionedValue(CacheObject val, @Nullable GridCacheVersion curVer, @Nullable GridCacheVersion newVer, @Nullable ReaderArguments readerArgs, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 7e26719d99076..58b4ae3130c9d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -69,7 +69,6 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.T3; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; @@ -975,7 +974,7 @@ private Object innerGet0( assert !deferred; // If return value is consistent, then done. - res = retVer ? new EntryGetResult(ret, resVer, false) : ret; + res = retVer ? entryGetResult(ret, resVer, false) : ret; } else if (reserveForLoad && !obsolete) { assert !readThrough; @@ -986,7 +985,7 @@ else if (reserveForLoad && !obsolete) { if (reserve) flags |= IS_EVICT_DISABLED; - res = new EntryGetResult(null, resVer, reserve); + res = entryGetResult(null, resVer, reserve); } } @@ -1092,6 +1091,20 @@ else if (tx.dht()) { return ret; } + /** + * Creates EntryGetResult or EntryGetWithTtlResult if expire time information exists. + * + * @param val Value. + * @param ver Version. + * @param reserve Reserve flag. + * @return EntryGetResult. + */ + private EntryGetResult entryGetResult(CacheObject val, GridCacheVersion ver, boolean reserve) { + return extras == null || extras.expireTime() == 0 + ? new EntryGetResult(val, ver, reserve) + : new EntryGetWithTtlResult(val, ver, reserve, rawExpireTime(), rawTtl()); + } + /** {@inheritDoc} */ @SuppressWarnings({"unchecked", "TooBroadScope"}) @Nullable @Override public final CacheObject innerReload() @@ -3382,7 +3395,7 @@ int hash() { } /** - * TODO: GG-4009: do we need to generate event and invalidate value? + * TODO: IGNITE-3500: do we need to generate event and invalidate value? * * @return {@code true} if expired. * @throws IgniteCheckedException In case of failure. @@ -3621,7 +3634,7 @@ private long nextPartCounter(AffinityTopologyVersion topVer) { } /** {@inheritDoc} */ - @Override public synchronized T2 versionedValue(CacheObject val, + @Override public synchronized EntryGetResult versionedValue(CacheObject val, GridCacheVersion curVer, GridCacheVersion newVer, @Nullable ReaderArguments readerArgs, @@ -3637,7 +3650,7 @@ private long nextPartCounter(AffinityTopologyVersion topVer) { GridCacheMvcc mvcc = mvccExtras(); if (mvcc != null && !mvcc.isEmpty()) - return new T2<>(this.val, ver); + return entryGetResult(this.val, ver, false); if (newVer == null) newVer = cctx.versions().next(); @@ -3671,13 +3684,13 @@ private long nextPartCounter(AffinityTopologyVersion topVer) { // Version does not change for load ops. update(val, expTime, ttl, newVer, true); - return new T2<>(val, newVer); + return entryGetResult(val, newVer, false); } assert !evictionDisabled() : this; } - return new T2<>(this.val, ver); + return entryGetResult(this.val, ver, false); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java index bc34df781c0a2..dcd379a040bd0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java @@ -41,6 +41,7 @@ import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.CacheOperationContext; import org.apache.ignite.internal.processors.cache.CachePeekModes; +import org.apache.ignite.internal.processors.cache.EntryGetResult; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheClearAllRunnable; import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap; @@ -74,7 +75,6 @@ import org.apache.ignite.internal.util.typedef.CI2; import org.apache.ignite.internal.util.typedef.CI3; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -648,7 +648,7 @@ else if (log.isDebugEnabled()) * @param canRemap Can remap flag. * @return Get future. */ - IgniteInternalFuture>> getDhtAllAsync( + IgniteInternalFuture> getDhtAllAsync( Collection keys, @Nullable final ReaderArguments readerArgs, boolean readThrough, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java index 3bf44895a62a6..8b92e9fb1daf0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java @@ -31,6 +31,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.EntryGetResult; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; @@ -46,7 +47,6 @@ import org.apache.ignite.internal.util.typedef.C2; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; @@ -390,7 +390,7 @@ private IgniteInternalFuture> getAsync( txFut.markInitialized(); } - IgniteInternalFuture>> fut; + IgniteInternalFuture> fut; if (txFut == null || txFut.isDone()) { fut = cache().getDhtAllAsync( @@ -411,8 +411,8 @@ private IgniteInternalFuture> getAsync( // transactions to complete. fut = new GridEmbeddedFuture<>( txFut, - new C2>>>() { - @Override public IgniteInternalFuture>> apply(Boolean b, Exception e) { + new C2>>() { + @Override public IgniteInternalFuture> apply(Boolean b, Exception e) { if (e != null) throw new GridClosureException(e); @@ -438,9 +438,9 @@ private IgniteInternalFuture> getAsync( } return new GridEmbeddedFuture<>( - new C2>, Exception, Collection>() { + new C2, Exception, Collection>() { @Override public Collection apply( - Map> map, Exception e + Map map, Exception e ) { if (e != null) { onDone(e); @@ -458,14 +458,14 @@ private IgniteInternalFuture> getAsync( * @param map Map to convert. * @return List of infos. */ - private Collection toEntryInfos(Map> map) { + private Collection toEntryInfos(Map map) { if (map.isEmpty()) return Collections.emptyList(); Collection infos = new ArrayList<>(map.size()); - for (Map.Entry> entry : map.entrySet()) { - T2 val = entry.getValue(); + for (Map.Entry entry : map.entrySet()) { + EntryGetResult val = entry.getValue(); assert val != null; @@ -473,8 +473,10 @@ private Collection toEntryInfos(Map>> fut; + IgniteInternalFuture> fut; if (rdrFut == null || rdrFut.isDone()) { fut = cache().getDhtAllAsync( @@ -375,7 +375,7 @@ private void getAsync() { return; } - IgniteInternalFuture>> fut0 = + IgniteInternalFuture> fut0 = cache().getDhtAllAsync( Collections.singleton(key), args, @@ -403,11 +403,11 @@ private void getAsync() { /** * @return Listener for get future. */ - @NotNull private IgniteInClosure>>> + @NotNull private IgniteInClosure>> createGetFutureListener() { - return new IgniteInClosure>>>() { + return new IgniteInClosure>>() { @Override public void apply( - IgniteInternalFuture>> fut + IgniteInternalFuture> fut ) { onResult(fut); } @@ -417,7 +417,7 @@ private void getAsync() { /** * @param fut Completed future to finish this process with. */ - private void onResult(IgniteInternalFuture>> fut) { + private void onResult(IgniteInternalFuture> fut) { assert fut.isDone(); if (fut.error() != null) @@ -436,11 +436,11 @@ private void onResult(IgniteInternalFuture> map) { + private GridCacheEntryInfo toEntryInfo(Map map) { if (map.isEmpty()) return null; - T2 val = map.get(key); + EntryGetResult val = map.get(key); assert val != null; @@ -448,8 +448,10 @@ private GridCacheEntryInfo toEntryInfo(Map locVals) { if (entry != null) { boolean isNew = entry.isNewLocked(); + EntryGetResult getRes = null; CacheObject v = null; GridCacheVersion ver = null; if (needVer) { - EntryGetResult res = entry.innerGetVersioned( + getRes = entry.innerGetVersioned( null, null, /*swap*/true, @@ -465,9 +466,9 @@ private boolean localGet(KeyCacheObject key, int part, Map locVals) { !deserializeBinary, null); - if (res != null) { - v = res.value(); - ver = res.version(); + if (getRes != null) { + v = getRes.value(); + ver = getRes.version(); } } else { @@ -501,7 +502,11 @@ private boolean localGet(KeyCacheObject key, int part, Map locVals) { keepCacheObjects, deserializeBinary, true, - ver); + getRes, + ver, + 0, + 0, + needVer); return true; } @@ -560,7 +565,9 @@ private Map createResultMap(Collection infos) { keepCacheObjects, deserializeBinary, false, - needVer ? info.version() : null); + needVer ? info.version() : null, + 0, + 0); } return map; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java index 7c14f35b34cf4..2b5624b489b6f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java @@ -51,7 +51,6 @@ import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.CIX1; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteProductVersion; @@ -615,7 +614,7 @@ private void setSkipValueResult(boolean res, @Nullable GridCacheVersion ver) { if (needVer) { assert ver != null || !res; - onDone(new T2<>(res, ver)); + onDone(new EntryGetResult(res, ver)); } else onDone(res); @@ -633,10 +632,10 @@ private void setResult(@Nullable CacheObject val, @Nullable GridCacheVersion ver if (!keepCacheObjects) { Object res = cctx.unwrapBinaryIfNeeded(val, !deserializeBinary); - onDone(needVer ? new T2<>(res, ver) : res); + onDone(needVer ? new EntryGetResult(res, ver) : res); } else - onDone(needVer ? new T2<>(val, ver) : val); + onDone(needVer ? new EntryGetResult(val, ver) : val); } else onDone(null); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java index 2f97bccf88a71..72e1bb1cc3080 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java @@ -1491,11 +1491,12 @@ private IgniteInternalFuture> getAllAsync0(@Nullable Collection> getAllAsync0(@Nullable Collection> getAllAsync0(@Nullable Collection> loadAsync( if (entry != null) { boolean isNew = entry.isNewLocked(); + EntryGetResult getRes = null; CacheObject v = null; GridCacheVersion ver = null; if (needVer) { - EntryGetResult res = entry.innerGetVersioned( + getRes = entry.innerGetVersioned( null, null, /*swap*/true, @@ -499,9 +500,9 @@ public final IgniteInternalFuture> loadAsync( !deserializeBinary, null); - if (res != null) { - v = res.value(); - ver = res.version(); + if (getRes != null) { + v = getRes.value(); + ver = getRes.version(); } } else { @@ -540,7 +541,11 @@ public final IgniteInternalFuture> loadAsync( keepCacheObj, deserializeBinary, true, - ver); + getRes, + ver, + 0, + 0, + needVer); } } else diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java index 8c64e3e9603bd..cb47498be0167 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java @@ -56,7 +56,6 @@ import org.apache.ignite.internal.util.typedef.CIX1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.P1; -import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -665,7 +664,7 @@ private void addResult(KeyCacheObject key, CacheObject v, GridCacheVersion ver) if (keepCacheObjects) { K key0 = (K)key; V val0 = needVer ? - (V)new T2<>(skipVals ? true : v, ver) : + (V)new EntryGetResult(skipVals ? true : v, ver) : (V)(skipVals ? true : v); add(new GridFinishedFuture<>(Collections.singletonMap(key0, val0))); @@ -673,7 +672,7 @@ private void addResult(KeyCacheObject key, CacheObject v, GridCacheVersion ver) else { K key0 = (K)cctx.unwrapBinaryIfNeeded(key, !deserializeBinary, false); V val0 = needVer ? - (V)new T2<>(!skipVals ? + (V)new EntryGetResult(!skipVals ? (V)cctx.unwrapBinaryIfNeeded(v, !deserializeBinary, false) : (V)Boolean.TRUE, ver) : !skipVals ? @@ -759,7 +758,9 @@ private Map loadEntries( keepCacheObjects, deserializeBinary, false, - needVer ? info.version() : null); + needVer ? info.version() : null, + 0, + 0); } catch (GridCacheEntryRemovedException ignore) { if (log.isDebugEnabled()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index 67518ef1d5771..ae9edcdddc0b1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -32,6 +32,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; +import org.apache.ignite.internal.processors.cache.EntryGetResult; import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.GridCacheMvccFuture; import org.apache.ignite.internal.processors.cache.GridCacheReturn; @@ -60,7 +61,6 @@ import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -476,10 +476,10 @@ private void processLoaded( GridCacheVersion ver; if (needVer) { - T2 t = (T2)val; + EntryGetResult getRes = (EntryGetResult)val; - v = t.get1(); - ver = t.get2(); + v = getRes.value(); + ver = getRes.version(); } else { v = val; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java index ee4f7a63e1e0b..7da11b6b870be 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java @@ -513,7 +513,6 @@ private Map getAllInternal(@Nullable Collection keys, if (entry != null) { CacheObject v; - GridCacheVersion ver; if (needVer) { EntryGetResult res = entry.innerGetVersioned( @@ -531,18 +530,15 @@ private Map getAllInternal(@Nullable Collection keys, null); if (res != null) { - v = res.value(); - ver = res.version(); - ctx.addResult( vals, cacheKey, - v, + res, skipVals, false, deserializeBinary, true, - ver); + needVer); } else success = false; @@ -569,7 +565,10 @@ private Map getAllInternal(@Nullable Collection keys, skipVals, false, deserializeBinary, - true); + true, + null, + 0, + 0); } else success = false; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index a1c1123d8cfd5..777489e026edb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -481,7 +481,7 @@ protected boolean commitAfterLock() { GridCacheEntryEx entry = cacheCtx.cache().entryEx(key); try { - T2 verVal = entry.versionedValue(cacheVal, + EntryGetResult verVal = entry.versionedValue(cacheVal, ver, null, null, @@ -490,11 +490,11 @@ protected boolean commitAfterLock() { if (log.isDebugEnabled()) { log.debug("Set value loaded from store into entry [" + "oldVer=" + ver + - ", newVer=" + verVal.get2() + + ", newVer=" + verVal.version() + ", entry=" + entry + ']'); } - ver = verVal.get2(); + ver = verVal.version(); break; } @@ -1212,7 +1212,8 @@ private Collection enlistRead( assert ver != null; } - cacheCtx.addResult(map, key, val, skipVals, keepCacheObjects, deserializeBinary, false, ver); + cacheCtx.addResult(map, key, val, skipVals, keepCacheObjects, deserializeBinary, false, + ver, 0, 0); } } else { @@ -1221,6 +1222,7 @@ private Collection enlistRead( while (true) { try { GridCacheVersion readVer = null; + EntryGetResult getRes = null; Object transformClo = (txEntry.op() == TRANSFORM && @@ -1228,7 +1230,7 @@ private Collection enlistRead( F.first(txEntry.entryProcessors()) : null; if (needVer) { - EntryGetResult res = txEntry.cached().innerGetVersioned( + getRes = txEntry.cached().innerGetVersioned( null, this, /*swap*/true, @@ -1242,9 +1244,9 @@ private Collection enlistRead( txEntry.keepBinary(), null); - if (res != null) { - val = res.value(); - readVer = res.version(); + if (getRes != null) { + val = getRes.value(); + readVer = getRes.version(); } } else { @@ -1277,7 +1279,11 @@ private Collection enlistRead( keepCacheObjects, deserializeBinary, false, - readVer); + getRes, + readVer, + 0, + 0, + needVer); } else missed.put(key, txEntry.cached().version()); @@ -1306,13 +1312,14 @@ private Collection enlistRead( CacheObject val = null; GridCacheVersion readVer = null; + EntryGetResult getRes = null; if (!pessimistic() || readCommitted() && !skipVals) { IgniteCacheExpiryPolicy accessPlc = optimistic() ? accessPolicy(cacheCtx, txKey, expiryPlc) : null; if (needReadVer) { - EntryGetResult res = primaryLocal(entry) ? + getRes = primaryLocal(entry) ? entry.innerGetVersioned( null, this, @@ -1327,9 +1334,9 @@ private Collection enlistRead( !deserializeBinary, null) : null; - if (res != null) { - val = res.value(); - readVer = res.version(); + if (getRes != null) { + val = getRes.value(); + readVer = getRes.version(); } } else { @@ -1356,7 +1363,11 @@ private Collection enlistRead( keepCacheObjects, deserializeBinary, false, - needVer ? readVer : null); + getRes, + readVer, + 0, + 0, + needVer); } else missed.put(key, ver); @@ -1534,7 +1545,9 @@ private IgniteInternalFuture> checkMissed( keepCacheObjects, deserializeBinary, false, - needVer ? loadVer : null); + needVer ? loadVer : null, + 0, + 0); } } else { @@ -1556,7 +1569,9 @@ private IgniteInternalFuture> checkMissed( keepCacheObjects, deserializeBinary, false, - needVer ? loadVer : null); + needVer ? loadVer : null, + 0, + 0); } } } @@ -1663,6 +1678,7 @@ private IgniteInternalFuture> checkMissed( CacheObject val = null; GridCacheVersion readVer = null; + EntryGetResult getRes = null; try { Object transformClo = @@ -1671,7 +1687,7 @@ private IgniteInternalFuture> checkMissed( F.first(txEntry.entryProcessors()) : null; if (needVer) { - EntryGetResult res = cached.innerGetVersioned( + getRes = cached.innerGetVersioned( null, IgniteTxLocalAdapter.this, /*swap*/cacheCtx.isSwapOrOffheapEnabled(), @@ -1685,9 +1701,9 @@ private IgniteInternalFuture> checkMissed( txEntry.keepBinary(), null); - if (res != null) { - val = res.value(); - readVer = res.version(); + if (getRes != null) { + val = getRes.value(); + readVer = getRes.version(); } } else{ @@ -1722,7 +1738,11 @@ private IgniteInternalFuture> checkMissed( keepCacheObjects, deserializeBinary, false, - readVer); + getRes, + readVer, + 0, + 0, + needVer); if (readVer != null) txEntry.entryReadVersion(readVer); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java index 2954bdb413d16..d46dee03b3a27 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java @@ -686,7 +686,7 @@ void recheckLock() { } /** @inheritDoc */ - @Override public T2 versionedValue(CacheObject val, + @Override public EntryGetResult versionedValue(CacheObject val, GridCacheVersion curVer, GridCacheVersion newVer, @Nullable ReaderArguments readerArgs, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java index f22ca6d30b9c4..b234631e4002a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java @@ -36,12 +36,14 @@ import javax.cache.expiry.ModifiedExpiryPolicy; import javax.cache.processor.EntryProcessor; import javax.cache.processor.MutableEntry; +import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheMemoryMode; import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteKernal; @@ -54,6 +56,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.PAX; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; @@ -1014,6 +1017,45 @@ public void testNearAccess() throws Exception { checkTtl(key, 62_000L, true); } + /** + * Put entry to server node and check how its expires in client NearCache. + * + * @throws Exception If failed. + */ + public void testNearExpiresOnClient() throws Exception { + if(cacheMode() != PARTITIONED) + return; + + factory = CreatedExpiryPolicy.factoryOf(new Duration(TimeUnit.SECONDS,1)); + + nearCache = true; + + startGrids(); + + IgniteConfiguration clientCfg = getConfiguration("client").setClientMode(true); + + ((TcpDiscoverySpi)clientCfg.getDiscoverySpi()).setForceServerMode(false); + + Ignite client = startGrid("client", clientCfg); + + IgniteCache cache = client.cache(null); + + Integer key = 1; + + // Put on server node. + jcache(0).put(key, 1); + + // Make entry cached in client NearCache. + assertEquals(1, cache.get(key)); + + assertEquals(1, cache.localPeek(key, CachePeekMode.NEAR)); + + waitExpired(key); + + // Check client NearCache. + assertNull(cache.localPeek(key, CachePeekMode.NEAR)); + } + /** * @return Test keys. * @throws Exception If failed. @@ -1270,4 +1312,4 @@ private static class TestPolicy implements ExpiryPolicy, Serializable { return S.toString(TestPolicy.class, this); } } -} \ No newline at end of file +} From 898fa150c7e2a663685abf37c6d1bcd547910f0e Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Fri, 27 Jan 2017 11:30:49 +0700 Subject: [PATCH 441/487] IGNITE-4622 Fixed generation in domain model for cache store. (cherry picked from commit 43007d5) --- .../configuration/generator/ConfigurationGenerator.js | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js index 8770bf6063c73..abe361cde29d8 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js @@ -48,6 +48,10 @@ export default class IgniteConfigurationGenerator { return new Bean('org.apache.ignite.cache.QueryEntity', 'qryEntity', domain, cacheDflts); } + static domainJdbcTypeBean(domain) { + return new Bean('org.apache.ignite.cache.store.jdbc.JdbcType', 'type', domain); + } + static discoveryConfigurationBean(discovery) { return new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery', discovery, clusterDflts.discovery); } @@ -1383,7 +1387,7 @@ export default class IgniteConfigurationGenerator { } // Generate domain model for store group. - static domainStore(domain, cfg = this.domainConfigurationBean(domain)) { + static domainStore(domain, cfg = this.domainJdbcTypeBean(domain)) { cfg.stringProperty('databaseSchema') .stringProperty('databaseTable'); @@ -1527,8 +1531,7 @@ export default class IgniteConfigurationGenerator { if (_.isNil(domain.databaseTable)) return acc; - const typeBean = new Bean('org.apache.ignite.cache.store.jdbc.JdbcType', 'type', - _.merge({}, domain, {cacheName: cache.name})) + const typeBean = this.domainJdbcTypeBean(_.merge({}, domain, {cacheName: cache.name})) .stringProperty('cacheName'); setType(typeBean, 'keyType'); From 83ad0943edf6a0e07d1b07c889492bce4932f9d1 Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Fri, 27 Jan 2017 17:44:57 +0300 Subject: [PATCH 442/487] Minor javadoc fix --- .../processors/cache/distributed/near/GridNearTxLocal.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index ae9edcdddc0b1..5aa091d7cc062 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -1169,7 +1169,7 @@ public IgniteInternalFuture rollbackAsyncLocal() { * @param keys Keys. * @param retval Return value flag. * @param read Read flag. - * @param accessTtl Create ttl. + * @param createTtl Create ttl. * @param accessTtl Access ttl. * @param Key type. * @param skipStore Skip store flag. From d22bcf6c72db5122b62e80f7719b22de67190e6a Mon Sep 17 00:00:00 2001 From: Anton Vinogradov Date: Mon, 30 Jan 2017 17:18:03 +0300 Subject: [PATCH 443/487] IGNITE-4621 Hang on broadcast when BinaryUtils.FIELDS_SORTED_ORDER == true --- .../org/apache/ignite/internal/binary/BinaryContext.java | 3 +++ .../org/apache/ignite/internal/binary/BinaryTreeMap.java | 6 +++++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java index 4030ef02c799c..7b21dfbed51f0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java @@ -190,6 +190,9 @@ public class BinaryContext { sysClss.add(GridClosureProcessor.C4V2.class.getName()); sysClss.add(GridClosureProcessor.C4MLAV2.class.getName()); + // BinaryUtils.FIELDS_SORTED_ORDER support, since it uses TreeMap at BinaryMetadata. + sysClss.add(BinaryTreeMap.class.getName()); + if (BinaryUtils.wrapTrees()) { sysClss.add(TreeMap.class.getName()); sysClss.add(TreeSet.class.getName()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMap.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMap.java index 6a7cf9b3db50d..3dae8ce74fa27 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryTreeMap.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.binary; +import java.io.Serializable; import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.binary.BinaryRawReader; import org.apache.ignite.binary.BinaryRawWriter; @@ -32,7 +33,10 @@ /** * Binary {@link TreeMap} wrapper. */ -public class BinaryTreeMap implements Binarylizable { +public class BinaryTreeMap implements Binarylizable, Serializable { + /** */ + private static final long serialVersionUID = 0L; + /** Original map. */ private TreeMap map; From ad91eac2d67d594f4a3def7fdfa2590ef5721b63 Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Fri, 3 Feb 2017 11:58:43 +0700 Subject: [PATCH 444/487] IGNITE-4610 Added more informative message. (cherry picked from commit 9ff2a8f) --- .../web-console/frontend/views/templates/agent-download.jade | 2 +- modules/web-console/web-agent/README.txt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/web-console/frontend/views/templates/agent-download.jade b/modules/web-console/frontend/views/templates/agent-download.jade index 20b6b03d058f3..f57bf1d87e21c 100644 --- a/modules/web-console/frontend/views/templates/agent-download.jade +++ b/modules/web-console/frontend/views/templates/agent-download.jade @@ -41,7 +41,7 @@ p Connection to Ignite Web Agent is established, but agent failed to connect to Ignite Node p Please check the following: ul - li Ignite Grid is up and running + li Ignite Grid is up and Ignite REST server started (copy "ignite-rest-http" folder from libs/optional/ to libs/) li In agent settings check URI for connect to Ignite REST server li Check agent logs for errors li Refer to #[b README.txt] in agent folder for more information diff --git a/modules/web-console/web-agent/README.txt b/modules/web-console/web-agent/README.txt index cc0c80fc99adb..81df1cb4f0234 100644 --- a/modules/web-console/web-agent/README.txt +++ b/modules/web-console/web-agent/README.txt @@ -32,7 +32,7 @@ Security tokens: 3) One may specify several comma separated tokens using configuration file or command line arguments of web agent. Ignite Web agent requirements: - 1) In order to communicate with web agent Ignite node should be started with REST server (move ignite-rest-http folder from lib/optional/ to lib/). + 1) In order to communicate with web agent Ignite node should be started with REST server (copy "ignite-rest-http" folder from "libs/optional/" to "libs/"). 2) Configure web agent serverURI property by Ignite node REST server URI. Options: From 4bf7fdf40e50aa6aca8fb438fb314c35166802df Mon Sep 17 00:00:00 2001 From: nikolay tikhonov Date: Tue, 7 Feb 2017 16:18:56 +0300 Subject: [PATCH 445/487] IGNITE-4590 Fixed Lock/unlock operations are hanging when topology changed --- .../transactions/IgniteTxLocalAdapter.java | 3 + .../cache/GridCacheTestEntryEx.java | 102 +++++++++--------- .../CacheLockReleaseNodeLeaveTest.java | 65 ++++++++++- 3 files changed, 118 insertions(+), 52 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java index 777489e026edb..1a9b082b3793f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java @@ -3785,6 +3785,9 @@ protected void updateExplicitVersion(IgniteTxEntry txEntry, GridCacheEntryEx ent // so it is safe to get acquired locks. GridCacheMvccCandidate explicitCand = entry.localOwner(); + if (explicitCand == null) + explicitCand = cctx.mvcc().explicitLock(threadId(), entry.txKey()); + if (explicitCand != null) { GridCacheVersion explicitVer = explicitCand.version(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java index d46dee03b3a27..8f0d9b13d127d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java @@ -280,7 +280,7 @@ void recheckLock() { return true; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public KeyCacheObject key() { return key; } @@ -290,7 +290,7 @@ void recheckLock() { return new IgniteTxKey(key, 0); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public CacheObject rawGet() { return val; } @@ -305,7 +305,7 @@ void recheckLock() { return val != null; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public CacheObject rawPut(CacheObject val, long ttl) { CacheObject old = this.val; @@ -315,14 +315,14 @@ void recheckLock() { return old; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public Cache.Entry wrap() { assert false; return null; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public Cache.Entry wrapLazyValue(boolean keepBinary) { assert false; @@ -336,19 +336,19 @@ void recheckLock() { return null; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Nullable @Override public CacheObject peekVisibleValue() { assert false; return null; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public GridCacheVersion obsoleteVersion() { return obsoleteVer; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean obsolete() { return obsoleteVer != null; } @@ -358,7 +358,7 @@ void recheckLock() { return obsoleteVer != null && !obsoleteVer.equals(exclude); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean invalidate(@Nullable GridCacheVersion curVer, GridCacheVersion newVer) throws IgniteCheckedException { assert false; @@ -366,7 +366,7 @@ void recheckLock() { return false; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean invalidate(@Nullable CacheEntryPredicate[] filter) throws GridCacheEntryRemovedException, IgniteCheckedException { assert false; @@ -374,7 +374,7 @@ void recheckLock() { return false; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean evictInternal(boolean swap, GridCacheVersion obsoleteVer, @Nullable CacheEntryPredicate[] filter) { assert false; @@ -390,7 +390,7 @@ void recheckLock() { return null; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean isNew() { assert false; return false; } @@ -400,7 +400,7 @@ void recheckLock() { assert false; return false; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public CacheObject innerGet( @Nullable GridCacheVersion ver, @Nullable IgniteInternalTx tx, @@ -417,12 +417,12 @@ void recheckLock() { return val; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public void clearReserveForLoad(GridCacheVersion ver) { assert false; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public EntryGetResult innerGetAndReserveForLoad( boolean readSwap, boolean updateMetrics, @@ -437,7 +437,7 @@ void recheckLock() { return null; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Nullable @Override public EntryGetResult innerGetVersioned( @Nullable GridCacheVersion ver, IgniteInternalTx tx, @@ -456,12 +456,12 @@ void recheckLock() { return null; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public CacheObject innerReload() { return val; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public GridCacheUpdateTxResult innerSet(@Nullable IgniteInternalTx tx, UUID evtNodeId, UUID affNodeId, @@ -544,7 +544,7 @@ void recheckLock() { return null; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public GridCacheUpdateTxResult innerRemove( @Nullable IgniteInternalTx tx, UUID evtNodeId, @@ -573,7 +573,7 @@ void recheckLock() { return new GridCacheUpdateTxResult(true, old); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean clear(GridCacheVersion ver, boolean readers) throws IgniteCheckedException { if (ver == null || ver.equals(this.ver)) { val = null; @@ -584,7 +584,7 @@ void recheckLock() { return false; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean tmLock(IgniteInternalTx tx, long timeout, @Nullable GridCacheVersion serOrder, @@ -594,12 +594,12 @@ void recheckLock() { return false; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public void txUnlock(IgniteInternalTx tx) { assert false; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean removeLock(GridCacheVersion ver) { GridCacheMvccCandidate doomed = mvcc.candidate(ver); @@ -608,7 +608,7 @@ void recheckLock() { return doomed != null; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean markObsolete(GridCacheVersion ver) { if (ver == null || ver.equals(obsoleteVer)) { obsoleteVer = ver; @@ -645,19 +645,19 @@ void recheckLock() { return false; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public GridCacheVersion version() { return ver; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean checkSerializableReadVersion(GridCacheVersion ver) { assert false; return false; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean initialValue( CacheObject val, GridCacheVersion ver, @@ -673,19 +673,19 @@ void recheckLock() { return false; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean initialValue(KeyCacheObject key, GridCacheSwapEntry unswapped) { assert false; return false; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public GridCacheVersionedEntryEx versionedEntry(final boolean keepBinary) throws IgniteCheckedException { return null; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public EntryGetResult versionedValue(CacheObject val, GridCacheVersion curVer, GridCacheVersion newVer, @@ -696,22 +696,22 @@ void recheckLock() { return null; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean hasLockCandidate(GridCacheVersion ver) { return mvcc.hasCandidate(ver); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean lockedByAny(GridCacheVersion... exclude) { return !mvcc.isEmpty(exclude); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean lockedByThread() { return lockedByThread(Thread.currentThread().getId()); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean lockedLocally(GridCacheVersion lockVer) { return mvcc.isLocallyOwned(lockVer); } @@ -722,52 +722,52 @@ void recheckLock() { return lockedLocally(lockVer) || lockedByThread(threadId); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean lockedByThread(long threadId, GridCacheVersion exclude) { return mvcc.isLocallyOwnedByThread(threadId, false, exclude); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean lockedByThread(long threadId) { return mvcc.isLocallyOwnedByThread(threadId, true); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean lockedBy(GridCacheVersion ver) { return mvcc.isOwnedBy(ver); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean lockedByThreadUnsafe(long threadId) { return mvcc.isLocallyOwnedByThread(threadId, true); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean lockedByUnsafe(GridCacheVersion ver) { return mvcc.isOwnedBy(ver); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean lockedLocallyUnsafe(GridCacheVersion lockVer) { return mvcc.isLocallyOwned(lockVer); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public boolean hasLockCandidateUnsafe(GridCacheVersion ver) { return mvcc.hasCandidate(ver); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public Collection localCandidates(GridCacheVersion... exclude) { return mvcc.localCandidates(exclude); } - /** @inheritDoc */ + /** {@inheritDoc} */ Collection localCandidates(boolean reentries, GridCacheVersion... exclude) { return mvcc.localCandidates(reentries, exclude); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public Collection remoteMvccSnapshot(GridCacheVersion... exclude) { return mvcc.remoteCandidates(exclude); } @@ -777,7 +777,7 @@ Collection localCandidates(boolean reentries, GridCacheV return mvcc.localCandidate(threadId); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public GridCacheMvccCandidate candidate(GridCacheVersion ver) { return mvcc.candidate(ver); } @@ -795,19 +795,19 @@ GridCacheMvccCandidate anyOwner() { return mvcc.anyOwner(); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public GridCacheMvccCandidate localOwner() { return mvcc.localOwner(); } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public CacheObject valueBytes() { assert false; return null; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public CacheObject valueBytes(GridCacheVersion ver) { assert false; @@ -819,7 +819,7 @@ GridCacheMvccCandidate anyOwner() { return 0; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public long expireTime() { return 0; } @@ -839,12 +839,12 @@ GridCacheMvccCandidate anyOwner() { return ttl; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public long ttl() { return ttl; } - /** @inheritDoc */ + /** {@inheritDoc} */ @Override public void updateTtl(GridCacheVersion ver, long ttl) { throw new UnsupportedOperationException(); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java index e84fd3fa125d0..687fe0bb6fa1a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java @@ -17,9 +17,13 @@ package org.apache.ignite.internal.processors.cache.distributed; +import java.util.ArrayDeque; +import java.util.Queue; import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Lock; import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; @@ -33,6 +37,7 @@ import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheMode.REPLICATED; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; @@ -43,6 +48,9 @@ public class CacheLockReleaseNodeLeaveTest extends GridCommonAbstractTest { /** */ private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + /** */ + private static final String REPLICATED_TEST_CACHE = "REPLICATED_TEST_CACHE"; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); @@ -53,7 +61,12 @@ public class CacheLockReleaseNodeLeaveTest extends GridCommonAbstractTest { ccfg.setAtomicityMode(TRANSACTIONAL); - cfg.setCacheConfiguration(ccfg); + CacheConfiguration ccfg1 = new CacheConfiguration(REPLICATED_TEST_CACHE) + .setCacheMode(REPLICATED) + .setAtomicityMode(TRANSACTIONAL) + .setReadFromBackup(false); + + cfg.setCacheConfiguration(ccfg, ccfg1); return cfg; } @@ -111,6 +124,56 @@ public void testLockRelease() throws Exception { fut2.get(5, SECONDS); } + /** + * @throws Exception If failed. + */ + public void testLockTopologyChange() throws Exception { + final int nodeCnt = 5; + int threadCnt = 8; + final int keys = 100; + + try { + final AtomicBoolean stop = new AtomicBoolean(false); + + Queue> q = new ArrayDeque<>(nodeCnt); + + for (int i = 0; i < nodeCnt; i++) { + final Ignite ignite = startGrid(i); + + IgniteInternalFuture f = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + while (!Thread.currentThread().isInterrupted() && !stop.get()) { + IgniteCache cache = ignite.cache(REPLICATED_TEST_CACHE); + + for (int i = 0; i < keys; i++) { + Lock lock = cache.lock(i); + lock.lock(); + + cache.put(i, i); + + lock.unlock(); + } + } + } + }, threadCnt, "test-lock-thread"); + + q.add(f); + + U.sleep(1_000); + } + + stop.set(true); + + IgniteInternalFuture f; + + while ((f = q.poll()) != null) + f.get(2_000); + } + finally { + stopAllGrids(); + } + } + /** * @throws Exception If failed. */ From 7f270405e3ab94938a9e46e301069825c7bc6fb0 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 6 Feb 2017 18:07:28 +0300 Subject: [PATCH 446/487] .NET: Extract exceptions tests in CacheStoreTest and ignore due to IGNITE-4657 --- .../Cache/Store/CacheStoreTest.cs | 30 +++++++++++++++++-- 1 file changed, 28 insertions(+), 2 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs index d39ccde391a96..2a235aaef6dd1 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs @@ -296,12 +296,38 @@ public void TestPutLoad() Assert.AreEqual("val", cache.Get(1)); Assert.AreEqual(1, cache.GetSize()); + } + + [Test] + public void TestExceptions() + { + var cache = GetCache(); + + cache.Put(1, "val"); - // Test errors CacheTestStore.ThrowError = true; CheckCustomStoreError(Assert.Throws(() => cache.Put(-2, "fail")).InnerException); - cache.LocalEvict(new[] { 1 }); + cache.LocalEvict(new[] {1}); + CheckCustomStoreError(Assert.Throws(() => cache.Get(1)).InnerException); + + CacheTestStore.ThrowError = false; + + cache.Remove(1); + } + + [Test] + [Ignore("IGNITE-4657")] + public void TestExceptionsNoRemove() + { + var cache = GetCache(); + + cache.Put(1, "val"); + + CacheTestStore.ThrowError = true; + CheckCustomStoreError(Assert.Throws(() => cache.Put(-2, "fail")).InnerException); + + cache.LocalEvict(new[] {1}); CheckCustomStoreError(Assert.Throws(() => cache.Get(1)).InnerException); } From 1fe33bc7b6e1dc966aed43533d32d4ce9c18795c Mon Sep 17 00:00:00 2001 From: devozerov Date: Mon, 13 Feb 2017 13:04:37 +0300 Subject: [PATCH 447/487] IGNITE-4688: Changed copyrights to 2017. --- .../java/org/apache/ignite/internal/IgniteVersionUtils.java | 2 +- .../org/apache/ignite/startup/GridRandomCommandLineLoader.java | 2 +- .../Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs | 2 +- .../dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.nuspec | 2 +- .../dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs | 2 +- .../Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs | 2 +- .../dotnet/Apache.Ignite.Core/Apache.Ignite.Core.Schema.nuspec | 2 +- .../dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec | 2 +- .../dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs | 2 +- .../dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.nuspec | 2 +- .../dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.nuspec | 2 +- .../dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs | 2 +- .../dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.nuspec | 2 +- .../dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs | 2 +- .../frontend/app/modules/branding/branding.provider.js | 2 +- 15 files changed, 15 insertions(+), 15 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteVersionUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteVersionUtils.java index bd8726f6380a6..7cb15bec5f8ee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteVersionUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteVersionUtils.java @@ -47,7 +47,7 @@ public class IgniteVersionUtils { public static final String ACK_VER_STR; /** Copyright blurb. */ - public static final String COPYRIGHT = "2016 Copyright(C) Apache Software Foundation"; + public static final String COPYRIGHT = "2017 Copyright(C) Apache Software Foundation"; /** * Static initializer. diff --git a/modules/core/src/test/java/org/apache/ignite/startup/GridRandomCommandLineLoader.java b/modules/core/src/test/java/org/apache/ignite/startup/GridRandomCommandLineLoader.java index 13c38efc8201f..73a9760a91536 100644 --- a/modules/core/src/test/java/org/apache/ignite/startup/GridRandomCommandLineLoader.java +++ b/modules/core/src/test/java/org/apache/ignite/startup/GridRandomCommandLineLoader.java @@ -61,7 +61,7 @@ public final class GridRandomCommandLineLoader { private static final String IGNITE_PROG_NAME = "IGNITE_PROG_NAME"; /** Copyright text. Ant processed. */ - private static final String COPYRIGHT = "2016 Copyright(C) Apache Software Foundation."; + private static final String COPYRIGHT = "2017 Copyright(C) Apache Software Foundation."; /** Version. Ant processed. */ private static final String VER = "x.x.x"; diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs index d6cb3df14b072..97ab3ef04e7e0 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs @@ -27,7 +27,7 @@ [assembly: AssemblyConfiguration("")] [assembly: AssemblyCompany("Apache Software Foundation")] [assembly: AssemblyProduct("Apache Ignite.NET")] -[assembly: AssemblyCopyright("Copyright © 2016")] +[assembly: AssemblyCopyright("Copyright © 2017")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.nuspec b/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.nuspec index 2324faa45080f..236584e8ebe55 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.nuspec +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.nuspec @@ -48,7 +48,7 @@ More info: https://apacheignite-net.readme.io/ Apache Ignite ASP.NET Integration - Copyright 2016 + Copyright 2017 OutputCacheProvider Apache Ignite In-Memory Distributed Computing SQL NoSQL Grid Map Reduce Cache diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs index 9bd5c4726d7d4..ad1d5ab0d8ceb 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs @@ -25,7 +25,7 @@ [assembly: AssemblyConfiguration("")] [assembly: AssemblyCompany("Apache Software Foundation")] [assembly: AssemblyProduct("Apache Ignite.NET")] -[assembly: AssemblyCopyright("Copyright © 2016")] +[assembly: AssemblyCopyright("Copyright © 2017")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs index ad0e91549fdd0..6349dd9f99bf5 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs @@ -23,7 +23,7 @@ [assembly: AssemblyConfiguration("")] [assembly: AssemblyCompany("Apache Software Foundation")] [assembly: AssemblyProduct("Apache Ignite.NET")] -[assembly: AssemblyCopyright("Copyright © 2016")] +[assembly: AssemblyCopyright("Copyright © 2017")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.Schema.nuspec b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.Schema.nuspec index 367bdd550a631..8c7731b535f44 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.Schema.nuspec +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.Schema.nuspec @@ -43,7 +43,7 @@ XSD file describes the structure of IgniteConfigurationSection and enables Intel More info on Apache Ignite.NET: https://apacheignite-net.readme.io/ - Copyright 2016 + Copyright 2017 Apache Ignite XSD Intellisense diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec index bb6227e2c0ef4..2c4dfc173cd78 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.nuspec @@ -48,7 +48,7 @@ More info: https://apacheignite-net.readme.io/ High-performance in-memory platform for computing and transacting on large-scale data sets in real-time. - Copyright 2016 + Copyright 2017 Apache Ignite In-Memory Distributed Computing SQL NoSQL Grid Map Reduce Cache linqpad-samples diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs index 240f2731708f3..f8512f14f21d9 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs @@ -25,7 +25,7 @@ [assembly: AssemblyConfiguration("")] [assembly: AssemblyCompany("Apache Software Foundation")] [assembly: AssemblyProduct("Apache Ignite.NET")] -[assembly: AssemblyCopyright("Copyright © 2016")] +[assembly: AssemblyCopyright("Copyright © 2017")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.nuspec b/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.nuspec index 088ca13bb94d3..93aa8c2cb7e65 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.nuspec +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.nuspec @@ -50,7 +50,7 @@ More info: https://apacheignite-net.readme.io/ LINQ Provider for Apache Ignite - Copyright 2016 + Copyright 2017 Apache Ignite In-Memory Distributed Computing SQL NoSQL LINQ Grid Map Reduce Cache linqpad-samples diff --git a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.nuspec b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.nuspec index fa5c39a49098c..a3f86c17cc88f 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.nuspec +++ b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.nuspec @@ -40,7 +40,7 @@ Creating NuGet package: false log4net Logger for Apache Ignite - Copyright 2016 + Copyright 2017 Apache Ignite In-Memory Distributed Computing SQL NoSQL LINQ Grid Map Reduce Cache log4net logger diff --git a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs index 824252e90449d..262a7cdd0720a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs @@ -24,7 +24,7 @@ [assembly: AssemblyConfiguration("")] [assembly: AssemblyCompany("Apache Software Foundation")] [assembly: AssemblyProduct("Apache Ignite.NET")] -[assembly: AssemblyCopyright("Copyright © 2016")] +[assembly: AssemblyCopyright("Copyright © 2017")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.nuspec b/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.nuspec index 765e26f390cfb..e3a6f42db4c4a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.nuspec +++ b/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.nuspec @@ -40,7 +40,7 @@ Creating NuGet package: false NLog Logger for Apache Ignite - Copyright 2016 + Copyright 2017 Apache Ignite In-Memory Distributed Computing SQL NoSQL LINQ Grid Map Reduce Cache NLog logger diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs index f28278c55723b..f952d7d55f8c9 100644 --- a/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs @@ -24,7 +24,7 @@ [assembly: AssemblyConfiguration("")] [assembly: AssemblyCompany("Apache Software Foundation")] [assembly: AssemblyProduct("Apache Ignite.NET")] -[assembly: AssemblyCopyright("Copyright © 2016")] +[assembly: AssemblyCopyright("Copyright © 2017")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] diff --git a/modules/web-console/frontend/app/modules/branding/branding.provider.js b/modules/web-console/frontend/app/modules/branding/branding.provider.js index ce14b349bc181..0545ff3a24a06 100644 --- a/modules/web-console/frontend/app/modules/branding/branding.provider.js +++ b/modules/web-console/frontend/app/modules/branding/branding.provider.js @@ -26,7 +26,7 @@ export default ['IgniteBranding', [function() { let footerHtml = [ '

          Apache Ignite Web Console

          ', - '

          © 2016 The Apache Software Foundation.

          ', + '

          © 2017 The Apache Software Foundation.

          ', '

          Apache, Apache Ignite, the Apache feather and the Apache Ignite logo are trademarks of The Apache Software Foundation.

          ' ]; From 0a4366591df30bf9c6841cf487e92f9decac6059 Mon Sep 17 00:00:00 2001 From: devozerov Date: Mon, 13 Feb 2017 13:19:56 +0300 Subject: [PATCH 448/487] IGNITE-4688: Changed copyrights to 2017. --- NOTICE | 2 +- assembly/NOTICE_FABRIC | 2 +- assembly/NOTICE_HADOOP | 2 +- modules/core/src/main/java/META-INF/NOTICE | 2 +- .../dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs | 2 +- .../Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs | 2 +- .../dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs | 2 +- .../dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs | 2 +- .../platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs | 2 +- .../examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs | 2 +- .../Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs | 2 +- modules/scalar-2.10/pom.xml | 2 +- modules/scalar/pom.xml | 2 +- parent/pom.xml | 2 +- 14 files changed, 14 insertions(+), 14 deletions(-) diff --git a/NOTICE b/NOTICE index 5ec3b0e16c50b..33e2479d6e89e 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache Ignite -Copyright 2015 The Apache Software Foundation +Copyright 2017 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/assembly/NOTICE_FABRIC b/assembly/NOTICE_FABRIC index 2e55768f62040..c5e6f023ad023 100644 --- a/assembly/NOTICE_FABRIC +++ b/assembly/NOTICE_FABRIC @@ -1,5 +1,5 @@ Apache Ignite -Copyright 2015 The Apache Software Foundation +Copyright 2017 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/assembly/NOTICE_HADOOP b/assembly/NOTICE_HADOOP index 5ec3b0e16c50b..33e2479d6e89e 100644 --- a/assembly/NOTICE_HADOOP +++ b/assembly/NOTICE_HADOOP @@ -1,5 +1,5 @@ Apache Ignite -Copyright 2015 The Apache Software Foundation +Copyright 2017 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/modules/core/src/main/java/META-INF/NOTICE b/modules/core/src/main/java/META-INF/NOTICE index 5ec3b0e16c50b..33e2479d6e89e 100644 --- a/modules/core/src/main/java/META-INF/NOTICE +++ b/modules/core/src/main/java/META-INF/NOTICE @@ -1,5 +1,5 @@ Apache Ignite -Copyright 2015 The Apache Software Foundation +Copyright 2017 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs index 91f79bde52888..a1d8aefe1a950 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs @@ -23,7 +23,7 @@ [assembly: AssemblyConfiguration("")] [assembly: AssemblyCompany("Apache Software Foundation")] [assembly: AssemblyProduct("Apache Ignite.NET")] -[assembly: AssemblyCopyright("Copyright © 2015")] +[assembly: AssemblyCopyright("Copyright © 2017")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs index a8c4305674e86..3994598a1afa8 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs @@ -23,7 +23,7 @@ [assembly: AssemblyConfiguration("")] [assembly: AssemblyCompany("Apache Software Foundation")] [assembly: AssemblyProduct("Apache Ignite.NET")] -[assembly: AssemblyCopyright("Copyright © 2015")] +[assembly: AssemblyCopyright("Copyright © 2017")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs index 7bf322fa81dae..d8bb169b5dfd6 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs @@ -23,7 +23,7 @@ [assembly: AssemblyConfiguration("")] [assembly: AssemblyCompany("Apache Software Foundation")] [assembly: AssemblyProduct("Apache Ignite.NET")] -[assembly: AssemblyCopyright("Copyright © 2015")] +[assembly: AssemblyCopyright("Copyright © 2017")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs index 76596da4a3ffa..446fe1e8d4137 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs @@ -24,7 +24,7 @@ [assembly: AssemblyConfiguration("")] [assembly: AssemblyCompany("Apache Software Foundation")] [assembly: AssemblyProduct("Apache Ignite.NET")] -[assembly: AssemblyCopyright("Copyright © 2015")] +[assembly: AssemblyCopyright("Copyright © 2017")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] diff --git a/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs index 85030264cf06f..41b62894ed144 100644 --- a/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs @@ -23,7 +23,7 @@ [assembly: AssemblyConfiguration("")] [assembly: AssemblyCompany("Apache Software Foundation")] [assembly: AssemblyProduct("Apache Ignite.NET")] -[assembly: AssemblyCopyright("Copyright © 2015")] +[assembly: AssemblyCopyright("Copyright © 2017")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs index 5834c57bca37f..396eeb65a5c35 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs @@ -23,7 +23,7 @@ [assembly: AssemblyConfiguration("")] [assembly: AssemblyCompany("Apache Software Foundation")] [assembly: AssemblyProduct("Apache Ignite.NET")] -[assembly: AssemblyCopyright("Copyright © 2015")] +[assembly: AssemblyCopyright("Copyright © 2017")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs index 230e9d8434d53..0855afb383393 100644 --- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs +++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs @@ -23,7 +23,7 @@ [assembly: AssemblyConfiguration("")] [assembly: AssemblyCompany("Apache Software Foundation")] [assembly: AssemblyProduct("Apache Ignite.NET")] -[assembly: AssemblyCopyright("Copyright © 2015")] +[assembly: AssemblyCopyright("Copyright © 2017")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml index 3920e0a8ad3ec..e857a3b7e0b17 100644 --- a/modules/scalar-2.10/pom.xml +++ b/modules/scalar-2.10/pom.xml @@ -158,7 +158,7 @@ - + diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml index eb8a24cfd0790..807ccb82b1625 100644 --- a/modules/scalar/pom.xml +++ b/modules/scalar/pom.xml @@ -143,7 +143,7 @@ Ignite™ - Scalar DSL, ver. ${project.version}
          - 2015 Copyright © Apache Software Foundation + 2017 Copyright © Apache Software Foundation diff --git a/parent/pom.xml b/parent/pom.xml index f766437ef48f2..88532a70dfb5a 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -464,7 +464,7 @@ - 2015 Copyright © Apache Software Foundation + 2017 Copyright © Apache Software Foundation From 8874f99f44dc2edf08a525619edb49d5db70b938 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 14 Feb 2017 18:44:57 +0300 Subject: [PATCH 449/487] IGNITE-4641 - Refresh client attributes during reconnect --- .../internal/managers/GridManagerAdapter.java | 4 + .../apache/ignite/spi/IgniteSpiAdapter.java | 5 + .../apache/ignite/spi/IgniteSpiContext.java | 6 + .../ignite/spi/discovery/tcp/ClientImpl.java | 2 +- .../spi/discovery/tcp/TcpDiscoverySpi.java | 5 + .../tcp/internal/TcpDiscoveryNode.java | 6 +- ...ryNodeAttributesUpdateOnReconnectTest.java | 110 +++++++++++++++++ .../tcp/TestReconnectPluginProvider.java | 111 ++++++++++++++++++ .../discovery/tcp/TestReconnectProcessor.java | 93 +++++++++++++++ .../testframework/GridSpiTestContext.java | 5 + .../IgniteSpiDiscoverySelfTestSuite.java | 3 + .../org.apache.ignite.plugin.PluginProvider | 1 + parent/pom.xml | 1 + 13 files changed, 349 insertions(+), 3 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryNodeAttributesUpdateOnReconnectTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectPluginProvider.java create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java create mode 100644 modules/core/src/test/resources/META-INF/services/org.apache.ignite.plugin.PluginProvider diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java index 584cc56e23b3f..25cc715ee12b8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java @@ -569,6 +569,10 @@ protected final String stopInfo() { ctx.timeout().removeTimeoutObject(new GridSpiTimeoutObject(obj)); } + @Override public Map nodeAttributes() { + return ctx.nodeAttributes(); + } + /** * @param e Exception to handle. * @return GridSpiException Converted exception. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java index 219d07be5f725..8879364dca5dd 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java @@ -928,5 +928,10 @@ private class GridDummySpiContext implements IgniteSpiContext { ((IgniteKernal)ignite0).context().timeout().removeTimeoutObject(new GridSpiTimeoutObject(obj)); } + + /** {@inheritDoc} */ + @Override public Map nodeAttributes() { + return Collections.emptyMap(); + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java index 5eb5227a0a46c..96b3e61a94131 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java @@ -19,6 +19,7 @@ import java.io.Serializable; import java.util.Collection; +import java.util.Map; import java.util.UUID; import javax.cache.CacheException; import org.apache.ignite.IgniteException; @@ -352,4 +353,9 @@ public interface IgniteSpiContext { * @param c Timeout object. */ public void removeTimeoutObject(IgniteSpiTimeoutObject c); + + /** + * @return Current node attributes. + */ + public Map nodeAttributes(); } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 39c539c30fb89..932e7d1e754a1 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -628,7 +628,7 @@ else if (addrs.isEmpty()) { TcpDiscoveryNode node = locNode; if (locNode.order() > 0) - node = locNode.clientReconnectNode(); + node = locNode.clientReconnectNode(spi.spiCtx.nodeAttributes()); msg = new TcpDiscoveryJoinRequestMessage(node, spi.collectExchangeData(getLocalNodeId())); } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java index 45933e112f78d..00ae97d1e302c 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java @@ -409,6 +409,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T /** */ private boolean clientReconnectDisabled; + /** */ + protected IgniteSpiContext spiCtx; + /** {@inheritDoc} */ @Override public String getSpiState() { return impl.getSpiState(); @@ -1161,6 +1164,8 @@ LinkedHashSet getNodeAddresses(TcpDiscoveryNode node, boolean @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException { super.onContextInitialized0(spiCtx); + this.spiCtx = spiCtx; + ctxInitLatch.countDown(); ipFinder.onSpiContextInitialized(spiCtx); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java index 307aefe641908..d8b1fc12243e8 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java @@ -26,6 +26,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.UUID; @@ -513,13 +514,14 @@ public void onClientDisconnected(UUID newId) { } /** + * @param nodeAttrs Current node attributes. * @return Copy of local node for client reconnect request. */ - public TcpDiscoveryNode clientReconnectNode() { + public TcpDiscoveryNode clientReconnectNode(Map nodeAttrs) { TcpDiscoveryNode node = new TcpDiscoveryNode(id, addrs, hostNames, discPort, metricsProvider, ver, null); - node.attrs = attrs; + node.attrs = Collections.unmodifiableMap(new HashMap<>(nodeAttrs)); node.clientRouterNodeId = clientRouterNodeId; return node; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryNodeAttributesUpdateOnReconnectTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryNodeAttributesUpdateOnReconnectTest.java new file mode 100644 index 0000000000000..56dc4ece5f4fc --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryNodeAttributesUpdateOnReconnectTest.java @@ -0,0 +1,110 @@ +/* + * 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.spi.discovery.tcp; + +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteEvents; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.DiscoveryEvent; +import org.apache.ignite.events.Event; +import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.IgniteClientReconnectAbstractTest; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.internal.IgniteClientReconnectAbstractTest.reconnectClientNode; + +/** + * Checks whether on client reconnect node attributes from kernal context are sent. + */ +public class TcpDiscoveryNodeAttributesUpdateOnReconnectTest extends GridCommonAbstractTest { + /** */ + private volatile String rejoinAttr; + + /** */ + @LoggerResource + private IgniteLogger log; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + if (gridName.contains("client")) { + Map attrs = new HashMap<>(); + + attrs.put("test", "1"); + + cfg.setUserAttributes(attrs); + cfg.setClientMode(true); + } + + IgniteClientReconnectAbstractTest.TestTcpDiscoverySpi spi = new IgniteClientReconnectAbstractTest.TestTcpDiscoverySpi(); + + TcpDiscoveryIpFinder finder = ((TcpDiscoverySpi)cfg.getDiscoverySpi()).getIpFinder(); + + spi.setIpFinder(finder); + + cfg.setDiscoverySpi(spi); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + TestReconnectPluginProvider.enabled = false; + + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + TestReconnectPluginProvider.enabled = true; + } + + /** + * @throws Exception If failed. + */ + public void testReconnect() throws Exception { + Ignite srv = startGrid("server"); + + IgniteEvents evts = srv.events(); + + evts.enableLocal(EventType.EVTS_DISCOVERY_ALL); + evts.localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + ClusterNode node = ((DiscoveryEvent)evt).eventNode(); + + rejoinAttr = node.attribute("test"); + + return true; + } + }, EventType.EVT_NODE_JOINED); + + Ignite client = startGrid("client"); + + reconnectClientNode(log, client, srv, null); + + assertEquals("2", rejoinAttr); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectPluginProvider.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectPluginProvider.java new file mode 100644 index 0000000000000..692774ca79dc8 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectPluginProvider.java @@ -0,0 +1,111 @@ +/* + * 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.spi.discovery.tcp; + +import java.io.Serializable; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.processors.security.GridSecurityProcessor; +import org.apache.ignite.plugin.ExtensionRegistry; +import org.apache.ignite.plugin.IgnitePlugin; +import org.apache.ignite.plugin.PluginContext; +import org.apache.ignite.plugin.PluginProvider; +import org.apache.ignite.plugin.PluginValidationException; +import org.jetbrains.annotations.Nullable; + +/** + * Creates TestReconnectProcessor. + */ +public class TestReconnectPluginProvider implements PluginProvider { + /** */ + private GridKernalContext igniteCtx; + + /** */ + public static volatile boolean enabled; + + /** {@inheritDoc} */ + @Override public String name() { + return "TestReconnectPlugin"; + } + + /** {@inheritDoc} */ + @Override public String version() { + return "1.0"; + } + + /** {@inheritDoc} */ + @Override public String copyright() { + return ""; + } + + /** {@inheritDoc} */ + @Override public void initExtensions(PluginContext ctx, ExtensionRegistry registry) { + igniteCtx = ((IgniteKernal)ctx.grid()).context(); + } + + /** {@inheritDoc} */ + @Override public void start(PluginContext ctx) throws IgniteCheckedException { + // No-op + } + + /** {@inheritDoc} */ + @Override public void stop(boolean cancel) throws IgniteCheckedException { + // No-op + } + + /** {@inheritDoc} */ + @Override public void onIgniteStart() throws IgniteCheckedException { + // No-op + } + + /** {@inheritDoc} */ + @Override public void onIgniteStop(boolean cancel) { + // No-op + } + + /** {@inheritDoc} */ + @Nullable @Override public Serializable provideDiscoveryData(UUID nodeId) { + return null; + } + + /** {@inheritDoc} */ + @Override public void receiveDiscoveryData(UUID nodeId, Serializable data) { + // No-op + } + + /** {@inheritDoc} */ + @Override public void validateNewNode(ClusterNode node) throws PluginValidationException { + // No-op + } + + /** {@inheritDoc} */ + @Nullable @Override public Object createComponent(PluginContext ctx, Class cls) { + if (enabled && GridSecurityProcessor.class.equals(cls)) + return new TestReconnectProcessor(igniteCtx); + + return null; + } + + /** {@inheritDoc} */ + @Override public IgnitePlugin plugin() { + return new IgnitePlugin() {}; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java new file mode 100644 index 0000000000000..f0ed35c4ccfab --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java @@ -0,0 +1,93 @@ +/* + * 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.spi.discovery.tcp; + +import java.util.Collection; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.processors.GridProcessorAdapter; +import org.apache.ignite.internal.processors.security.GridSecurityProcessor; +import org.apache.ignite.internal.processors.security.SecurityContext; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.plugin.security.AuthenticationContext; +import org.apache.ignite.plugin.security.SecurityCredentials; +import org.apache.ignite.plugin.security.SecurityException; +import org.apache.ignite.plugin.security.SecurityPermission; +import org.apache.ignite.plugin.security.SecuritySubject; +import org.jetbrains.annotations.Nullable; + +/** + * Updates node attributes on disconnect. + */ +public class TestReconnectProcessor extends GridProcessorAdapter implements GridSecurityProcessor { + /** + * @param ctx Kernal context. + */ + protected TestReconnectProcessor(GridKernalContext ctx) { + super(ctx); + } + + /** {@inheritDoc} */ + @Override public SecurityContext authenticateNode(ClusterNode node, + SecurityCredentials cred) throws IgniteCheckedException { + return null; + } + + /** {@inheritDoc} */ + @Override public boolean isGlobalNodeAuthentication() { + return false; + } + + /** {@inheritDoc} */ + @Override public SecurityContext authenticate(AuthenticationContext ctx) throws IgniteCheckedException { + return null; + } + + /** {@inheritDoc} */ + @Override public Collection authenticatedSubjects() throws IgniteCheckedException { + return null; + } + + /** {@inheritDoc} */ + @Override public SecuritySubject authenticatedSubject(UUID subjId) throws IgniteCheckedException { + return null; + } + + /** {@inheritDoc} */ + @Override public void authorize(String name, SecurityPermission perm, + @Nullable SecurityContext securityCtx) throws SecurityException { + + } + + /** {@inheritDoc} */ + @Override public void onSessionExpired(UUID subjId) { + + } + + /** {@inheritDoc} */ + @Override public boolean enabled() { + return false; + } + + /** {@inheritDoc} */ + @Override public void onDisconnected(IgniteFuture reconnectFut) throws IgniteCheckedException { + ctx.addNodeAttribute("test", "2"); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java index ac50ef90e29e1..1c8acbc3960ce 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java @@ -601,6 +601,11 @@ public void triggerEvent(Event evt) { timeoutProcessor.removeTimeoutObject(new GridSpiTimeoutObject(obj)); } + /** {@inheritDoc} */ + @Override public Map nodeAttributes() { + return Collections.emptyMap(); + } + /** * @param cacheName Cache name. * @return Map representing cache. diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java index 5f870a474578b..548e1a5a51f81 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java @@ -25,6 +25,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpClientDiscoverySpiSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoveryMarshallerCheckSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoveryMultiThreadedTest; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoveryNodeAttributesUpdateOnReconnectTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoveryNodeConfigConsistentIdSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoveryNodeConsistentIdSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoveryRestartTest; @@ -84,6 +85,8 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(TcpDiscoveryRestartTest.class)); suite.addTest(new TestSuite(TcpDiscoveryMultiThreadedTest.class)); + suite.addTest(new TestSuite(TcpDiscoveryNodeAttributesUpdateOnReconnectTest.class)); + // SSL. suite.addTest(new TestSuite(TcpDiscoverySslSelfTest.class)); diff --git a/modules/core/src/test/resources/META-INF/services/org.apache.ignite.plugin.PluginProvider b/modules/core/src/test/resources/META-INF/services/org.apache.ignite.plugin.PluginProvider new file mode 100644 index 0000000000000..a7fdf4384e626 --- /dev/null +++ b/modules/core/src/test/resources/META-INF/services/org.apache.ignite.plugin.PluginProvider @@ -0,0 +1 @@ +org.apache.ignite.spi.discovery.tcp.TestReconnectPluginProvider \ No newline at end of file diff --git a/parent/pom.xml b/parent/pom.xml index 88532a70dfb5a..8bf5dde9e12a8 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -821,6 +821,7 @@ src/main/java/META-INF/services/javax.cache.spi.CachingProvider src/main/java/org/jetbrains/annotations/*.java src/main/resources/META-INF/services/org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider + /src/test/resources/META-INF/services/org.apache.ignite.plugin.PluginProvider dev-tools/IGNITE-*.patch dev-tools/.gradle/**/* dev-tools/gradle/wrapper/**/* From 05788b3188b30b5a3b39a75fe66301e03658408f Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 17 Feb 2017 12:14:53 +0300 Subject: [PATCH 450/487] IGNITE-3429: Added BinaryResolver configuration samples for org.hibernate.cache.spi.CacheKey. This closes #1516. --- .../Hibernate5CacheKeyTypeConfiguration.java | 52 +++++++++++++++++++ .../HibernateCacheKeyTypeConfiguration.java | 51 ++++++++++++++++++ 2 files changed, 103 insertions(+) create mode 100644 modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/Hibernate5CacheKeyTypeConfiguration.java create mode 100644 modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/HibernateCacheKeyTypeConfiguration.java diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/Hibernate5CacheKeyTypeConfiguration.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/Hibernate5CacheKeyTypeConfiguration.java new file mode 100644 index 0000000000000..886f69b2500ba --- /dev/null +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/Hibernate5CacheKeyTypeConfiguration.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.hibernate.config; + +import java.util.Objects; +import org.apache.ignite.binary.BinaryAbstractIdentityResolver; +import org.apache.ignite.binary.BinaryIdentityResolver; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.binary.BinaryTypeConfiguration; + +/** + * This configuration provides correct {@link BinaryIdentityResolver} implementation + * for Hibernate CacheKey class can be used as a key object. + * + * Note: for Hibernate version < 5.0 {@link HibernateCacheKeyTypeConfiguration} should be used. + + */ +public class Hibernate5CacheKeyTypeConfiguration extends BinaryTypeConfiguration { + + /** {@inheritDoc} */ + public Hibernate5CacheKeyTypeConfiguration() { + super("org.hibernate.cache.internal.CacheKeyImplementation"); + + setIdentityResolver(new BinaryAbstractIdentityResolver() { + @Override protected int hashCode0(BinaryObject obj) { + return obj.field("id").hashCode(); + } + + @Override protected boolean equals0(BinaryObject o1, BinaryObject o2) { + Object obj0 = o1.field("id"); + Object obj1 = o2.field("id"); + + return Objects.equals(obj0, obj1); + } + }); + } +} diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/HibernateCacheKeyTypeConfiguration.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/HibernateCacheKeyTypeConfiguration.java new file mode 100644 index 0000000000000..c54292e6f40e6 --- /dev/null +++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/HibernateCacheKeyTypeConfiguration.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.hibernate.config; + +import java.util.Objects; +import org.apache.ignite.binary.BinaryAbstractIdentityResolver; +import org.apache.ignite.binary.BinaryIdentityResolver; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.binary.BinaryTypeConfiguration; + +/** + * This configuration provides correct {@link BinaryIdentityResolver} implementation + * for Hibernate CacheKey class can be used as a key object. + * + * Note: for Hibernate version >= 5.0 {@link Hibernate5CacheKeyTypeConfiguration} should be used. + */ +public class HibernateCacheKeyTypeConfiguration extends BinaryTypeConfiguration { + + /** {@inheritDoc} */ + public HibernateCacheKeyTypeConfiguration() { + super("org.hibernate.cache.spi.CacheKey"); + + setIdentityResolver(new BinaryAbstractIdentityResolver() { + @Override protected int hashCode0(BinaryObject obj) { + return obj.field("key").hashCode(); + } + + @Override protected boolean equals0(BinaryObject o1, BinaryObject o2) { + Object obj0 = o1.field("key"); + Object obj1 = o2.field("key"); + + return Objects.equals(obj0, obj1); + } + }); + } +} From 1f881aa70a3894af01135f4cc5e341a8130462c2 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 17 Feb 2017 12:34:41 +0300 Subject: [PATCH 451/487] IGNITE-4147 - Throw exception on connecting node to cluster with different SSL configuration --- .../ignite/spi/discovery/tcp/ClientImpl.java | 30 ++- .../ignite/spi/discovery/tcp/ServerImpl.java | 19 ++ .../TcpDiscoverySslSecuredUnsecuredTest.java | 185 ++++++++++++++++++ .../IgniteSpiDiscoverySelfTestSuite.java | 2 + 4 files changed, 235 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 932e7d1e754a1..95e2cda4f6907 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -20,6 +20,7 @@ import java.io.BufferedInputStream; import java.io.IOException; import java.io.InputStream; +import java.io.StreamCorruptedException; import java.net.InetSocketAddress; import java.net.Socket; import java.net.SocketTimeoutException; @@ -44,6 +45,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.atomic.AtomicReference; +import javax.net.ssl.SSLException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteException; @@ -587,6 +589,8 @@ else if (addrs.isEmpty()) { int connectAttempts = 1; + int sslConnectAttempts = 3; + UUID locNodeId = getLocalNodeId(); IgniteSpiOperationTimeoutHelper timeoutHelper = new IgniteSpiOperationTimeoutHelper(spi); @@ -662,6 +666,22 @@ else if (addrs.isEmpty()) { errs.add(e); + if (X.hasCause(e, SSLException.class)) { + if (--sslConnectAttempts == 0) + throw new IgniteSpiException("Unable to establish secure connection. " + + "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); + + continue; + } + + if (X.hasCause(e, StreamCorruptedException.class)) { + if (--sslConnectAttempts == 0) + throw new IgniteSpiException("Unable to establish plain connection. " + + "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); + + continue; + } + if (timeoutHelper.checkFailureTimeoutReached(e)) break; @@ -1593,7 +1613,15 @@ private void tryJoin() throws InterruptedException { joinCnt++; - T2 joinRes = joinTopology(false, spi.joinTimeout); + T2 joinRes; + try { + joinRes = joinTopology(false, spi.joinTimeout); + } + catch (IgniteSpiException e) { + joinError(e); + + return; + } if (joinRes == null) { if (join) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index d462ac2638fd2..4600be094c840 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -24,6 +24,7 @@ import java.io.ObjectStreamException; import java.io.OutputStream; import java.io.Serializable; +import java.io.StreamCorruptedException; import java.net.ConnectException; import java.net.InetAddress; import java.net.InetSocketAddress; @@ -1109,6 +1110,8 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) int connectAttempts = 1; + int sslConnectAttempts = 3; + boolean joinReqSent; UUID locNodeId = getLocalNodeId(); @@ -1220,6 +1223,22 @@ else if (U.currentTimeMillis() - noResStart > spi.joinTimeout) errs.add(e); + if (X.hasCause(e, SSLException.class)) { + if (--sslConnectAttempts == 0) + throw new IgniteException("Unable to establish secure connection. " + + "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); + + continue; + } + + if (X.hasCause(e, StreamCorruptedException.class)) { + if (--sslConnectAttempts == 0) + throw new IgniteException("Unable to establish plain connection. " + + "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e); + + continue; + } + if (timeoutHelper.checkFailureTimeoutReached(e)) break; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java new file mode 100644 index 0000000000000..ca34f779760cf --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java @@ -0,0 +1,185 @@ +/* + * 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.spi.discovery.tcp; + +import java.io.IOException; +import java.io.InputStream; +import java.io.StreamCorruptedException; +import java.net.Socket; +import java.util.concurrent.Callable; +import javax.net.ssl.SSLException; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.Nullable; + +/** + * Tests cases when node connects to cluster with different SSL configuration. + * Exception with meaningful message should be thrown. + */ +public class TcpDiscoverySslSecuredUnsecuredTest extends GridCommonAbstractTest { + /** */ + private volatile TcpDiscoverySpi spi; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception { + final IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setClientMode(gridName.contains("client")); + + if (gridName.contains("ssl")) + cfg.setSslContextFactory(GridTestUtils.sslFactory()); + + if (spi != null) { + final TcpDiscoveryIpFinder finder = ((TcpDiscoverySpi)cfg.getDiscoverySpi()).getIpFinder(); + + spi.setIpFinder(finder); + + cfg.setDiscoverySpi(spi); + } + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testSecuredUnsecuredServerConnection() throws Exception { + checkConnection("plain-server", "ssl-server"); + } + + /** + * @throws Exception If failed. + */ + public void testUnsecuredSecuredServerConnection() throws Exception { + checkConnection("ssl-server", "plain-server"); + } + + /** + * @throws Exception If failed. + */ + public void testSecuredClientUnsecuredServerConnection() throws Exception { + checkConnection("plain-server", "ssl-client"); + } + + /** + * @throws Exception If failed. + */ + public void testUnsecuredClientSecuredServerConnection() throws Exception { + checkConnection("ssl-server", "plain-client"); + } + + /** + * @throws Exception If failed. + */ + public void testPlainServerNodesRestart() throws Exception { + checkNodesRestart("plain-server-1", "plain-server-2"); + } + + /** + * @throws Exception If failed. + */ + public void testSslServerNodesRestart() throws Exception { + checkNodesRestart("ssl-server-1", "ssl-server-2"); + } + + /** + * @throws Exception If failed. + */ + public void testPlainClientNodesRestart() throws Exception { + checkNodesRestart("plain-server", "plain-client"); + } + + /** + * @throws Exception If failed. + */ + public void testSslClientNodesRestart() throws Exception { + checkNodesRestart("ssl-server", "ssl-client"); + } + + /** + * @param name1 First grid name. + * @param name2 Second grid name. + * @throws Exception If failed. + */ + private void checkNodesRestart(String name1, String name2) throws Exception { + startGrid(name1); + + spi = new FailDiscoverySpi(!name1.contains("ssl")); + + startGrid(name2); + } + + /** + * @param name1 First grid name. + * @param name2 Second grid name. + * @throws Exception If failed. + */ + @SuppressWarnings("ThrowableNotThrown") + private void checkConnection(final String name1, final String name2) throws Exception { + startGrid(name1); + + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + startGrid(name2); + + return null; + } + }, IgniteCheckedException.class, null); + } + + /** + * + */ + private class FailDiscoverySpi extends TcpDiscoverySpi { + /** */ + private int cnt = 1; + + /** */ + private final boolean plain; + + /** + * @param plain Plain conection flag. + */ + private FailDiscoverySpi(final boolean plain) { + this.plain = plain; + } + + /** {@inheritDoc} */ + @Override protected T readMessage(final Socket sock, @Nullable final InputStream in, + final long timeout) throws IOException, IgniteCheckedException { + if (cnt-- > 0) { + if (plain) + throw new StreamCorruptedException("Test exception"); + else + throw new SSLException("Test SSL exception"); + } + + return super.readMessage(sock, in, timeout); + } + } + + +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java index 548e1a5a51f81..e6b39f740a0dd 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java @@ -35,6 +35,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiFailureTimeoutSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiStartStopSelfTest; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySslSecuredUnsecuredTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySslSelfTest; import org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinderSelfTest; import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinderSelfTest; @@ -89,6 +90,7 @@ public static TestSuite suite() throws Exception { // SSL. suite.addTest(new TestSuite(TcpDiscoverySslSelfTest.class)); + suite.addTest(new TestSuite(TcpDiscoverySslSecuredUnsecuredTest.class)); return suite; } From 11bbec487dc174fac1acd6b50c940840305bc75a Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 17 Feb 2017 17:57:50 +0700 Subject: [PATCH 452/487] IGNITE-4436 API for collecting list of running queries and cancel them. (cherry picked from commit 49237343d53ee33d44e5599cd7fe7da868ee30a1) --- .../cache/query/GridCacheTwoStepQuery.java | 18 +- .../processors/query/GridQueryIndexing.java | 17 +- .../processors/query/GridQueryProcessor.java | 26 ++- .../query/GridRunningQueryInfo.java | 132 +++++++++++ .../internal/visor/VisorMultiNodeTask.java | 2 +- .../visor/query/VisorCancelQueriesTask.java | 72 ++++++ .../query/VisorCollectRunningQueriesTask.java | 96 ++++++++ .../visor/query/VisorRunningQuery.java | 132 +++++++++++ .../processors/query/h2/IgniteH2Indexing.java | 83 ++++++- .../query/h2/sql/GridSqlQuerySplitter.java | 4 +- .../h2/twostep/GridReduceQueryExecutor.java | 60 ++++- .../cache/CacheSqlQueryValueCopySelfTest.java | 208 +++++++++++++++++- .../GridCacheCrossCacheQuerySelfTest.java | 2 +- .../h2/GridIndexingSpiAbstractSelfTest.java | 7 + 14 files changed, 821 insertions(+), 38 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridRunningQueryInfo.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCancelQueriesTask.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCollectRunningQueriesTask.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorRunningQuery.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java index 8dcba2f2f34dc..f53936fc374a1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java @@ -45,6 +45,9 @@ public class GridCacheTwoStepQuery { /** */ private boolean explain; + /** */ + private String originalSql; + /** */ private Collection spaces; @@ -67,10 +70,12 @@ public class GridCacheTwoStepQuery { private List extraCaches; /** + * @param originalSql Original query SQL. * @param schemas Schema names in query. * @param tbls Tables in query. */ - public GridCacheTwoStepQuery(Set schemas, Set tbls) { + public GridCacheTwoStepQuery(String originalSql, Set schemas, Set tbls) { + this.originalSql = originalSql; this.schemas = schemas; this.tbls = tbls; } @@ -195,6 +200,13 @@ public void extraCaches(List extraCaches) { this.extraCaches = extraCaches; } + /** + * @return Original query SQL. + */ + public String originalSql() { + return originalSql; + } + /** * @return Spaces. */ @@ -223,7 +235,7 @@ public Set schemas() { public GridCacheTwoStepQuery copy(Object[] args) { assert !explain; - GridCacheTwoStepQuery cp = new GridCacheTwoStepQuery(schemas, tbls); + GridCacheTwoStepQuery cp = new GridCacheTwoStepQuery(originalSql, schemas, tbls); cp.caches = caches; cp.extraCaches = extraCaches; @@ -250,4 +262,4 @@ public Set tables() { @Override public String toString() { return S.toString(GridCacheTwoStepQuery.class, this); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java index 539ebc07fec79..1cebbb0a54ff6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java @@ -223,8 +223,23 @@ public void store(@Nullable String spaceName, GridQueryTypeDescriptor type, Cach */ public void onDisconnected(IgniteFuture reconnectFut); + /** + * Collect queries that already running more than specified duration. + * + * @param duration Duration to check. + * @return Collection of long running queries. + */ + public Collection runningQueries(long duration); + + /** + * Cancel specified queries. + * + * @param queries Queries ID's to cancel. + */ + public void cancelQueries(Collection queries); + /** * Cancels all executing queries. */ public void cancelAllQueries(); -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index c2e5717fbddf8..0a0d16614a92b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -44,7 +44,6 @@ import javax.cache.CacheException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; -import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.binary.BinaryField; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryType; @@ -118,7 +117,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { private static final int QRY_DETAIL_METRICS_EVICTION_FREQ = 3_000; /** */ - private static Set> SQL_TYPES = new HashSet<>(F.>asList( + private static final Set> SQL_TYPES = new HashSet<>(F.>asList( Integer.class, Boolean.class, Byte.class, @@ -912,6 +911,29 @@ public Iterator> queryLocal( } } + /** + * Collect queries that already running more than specified duration. + * + * @param duration Duration to check. + * @return Collection of long running queries. + */ + public Collection runningQueries(long duration) { + if (moduleEnabled()) + return idx.runningQueries(duration); + + return Collections.emptyList(); + } + + /** + * Cancel specified queries. + * + * @param queries Queries ID's to cancel. + */ + public void cancelQueries(Collection queries) { + if (moduleEnabled()) + idx.cancelQueries(queries); + } + /** * @param sqlQry Sql query. * @param params Params. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridRunningQueryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridRunningQueryInfo.java new file mode 100644 index 0000000000000..d77c8c01a8ede --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridRunningQueryInfo.java @@ -0,0 +1,132 @@ +/* + * 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.query; + +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType; + +/** + * Query descriptor. + */ +public class GridRunningQueryInfo { + /** */ + private final long id; + + /** */ + private final String qry; + + /** Query type. */ + private final GridCacheQueryType qryType; + + /** */ + private final String cache; + + /** */ + private final long startTime; + + /** */ + private final GridQueryCancel cancel; + + /** */ + private final boolean loc; + + /** + * @param id Query ID. + * @param qry Query text. + * @param qryType Query type. + * @param cache Cache where query was executed. + * @param startTime Query start time. + * @param cancel Query cancel. + * @param loc Local query flag. + */ + public GridRunningQueryInfo(Long id, String qry, GridCacheQueryType qryType, String cache, long startTime, + GridQueryCancel cancel, boolean loc) { + this.id = id; + this.qry = qry; + this.qryType = qryType; + this.cache = cache; + this.startTime = startTime; + this.cancel = cancel; + this.loc = loc; + } + + /** + * @return Query ID. + */ + public Long id() { + return id; + } + + /** + * @return Query text. + */ + public String query() { + return qry; + } + + /** + * @return Query type. + */ + public GridCacheQueryType queryType() { + return qryType; + } + + /** + * @return Cache where query was executed. + */ + public String cache() { + return cache; + } + + /** + * @return Query start time. + */ + public long startTime() { + return startTime; + } + + /** + * @param curTime Current time. + * @param duration Duration of long query. + * @return {@code true} if this query should be considered as long running query. + */ + public boolean longQuery(long curTime, long duration) { + return curTime - startTime > duration; + } + + /** + * Cancel query. + */ + public void cancel() { + if (cancel != null) + cancel.cancel(); + } + + /** + * @return {@code true} if query can be cancelled. + */ + public boolean cancelable() { + return cancel != null; + } + + /** + * @return {@code true} if query is local. + */ + public boolean local() { + return loc; + } +} 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 57f134698ea56..ece1a17953f39 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 @@ -130,4 +130,4 @@ protected Map map0(List subgrid, logFinish(ignite.log(), getClass(), start); } } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCancelQueriesTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCancelQueriesTask.java new file mode 100644 index 0000000000000..a6f2d821f15a0 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCancelQueriesTask.java @@ -0,0 +1,72 @@ +/* + * 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.visor.query; + +import java.util.Collection; +import java.util.List; +import org.apache.ignite.IgniteException; +import org.apache.ignite.compute.ComputeJobResult; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorOneNodeTask; +import org.jetbrains.annotations.Nullable; + +/** + * Task to cancel queries. + */ +@GridInternal +public class VisorCancelQueriesTask extends VisorOneNodeTask, Void> { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorCancelQueriesJob job(Collection arg) { + return new VisorCancelQueriesJob(arg, debug); + } + + /** {@inheritDoc} */ + @Nullable @Override protected Void reduce0(List results) throws IgniteException { + return null; + } + + /** + * Job to cancel queries on node. + */ + private static class VisorCancelQueriesJob extends VisorJob, Void> { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Create job with specified argument. + * + * @param arg Job argument. + * @param debug Flag indicating whether debug information should be printed into node log. + */ + protected VisorCancelQueriesJob(@Nullable Collection arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected Void run(@Nullable Collection queries) throws IgniteException { + ignite.context().query().cancelQueries(queries); + + return null; + } + } + +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCollectRunningQueriesTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCollectRunningQueriesTask.java new file mode 100644 index 0000000000000..2b40e61e74f50 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCollectRunningQueriesTask.java @@ -0,0 +1,96 @@ +/* + * 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.visor.query; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.IgniteException; +import org.apache.ignite.compute.ComputeJobResult; +import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorMultiNodeTask; +import org.jetbrains.annotations.Nullable; + +/** + * Task to collect currently running queries. + */ +@GridInternal +public class VisorCollectRunningQueriesTask extends VisorMultiNodeTask>, Collection> { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorCollectRunningQueriesJob job(Long arg) { + return new VisorCollectRunningQueriesJob(arg, debug); + } + + /** {@inheritDoc} */ + @Nullable @Override protected Map> reduce0(List results) throws IgniteException { + Map> map = new HashMap<>(); + + for (ComputeJobResult res : results) + if (res.getException() == null) { + Collection queries = res.getData(); + + map.put(res.getNode().id(), queries); + } + + return map; + } + + /** + * Job to collect currently running queries from node. + */ + private static class VisorCollectRunningQueriesJob extends VisorJob> { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Create job with specified argument. + * + * @param arg Job argument. + * @param debug Flag indicating whether debug information should be printed into node log. + */ + protected VisorCollectRunningQueriesJob(@Nullable Long arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected Collection run(@Nullable Long duration) throws IgniteException { + Collection queries = ignite.context().query() + .runningQueries(duration != null ? duration : 0); + + Collection res = new ArrayList<>(queries.size()); + + long curTime = U.currentTimeMillis(); + + for (GridRunningQueryInfo qry : queries) + res.add(new VisorRunningQuery(qry.id(), qry.query(), qry.queryType(), qry.cache(), + qry.startTime(), curTime - qry.startTime(), + qry.cancelable(), qry.local())); + + return res; + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorRunningQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorRunningQuery.java new file mode 100644 index 0000000000000..fc6bc7a9222d4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorRunningQuery.java @@ -0,0 +1,132 @@ +/* + * 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.visor.query; + +import java.io.Serializable; +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType; + +/** + * Descriptor of running query. + */ +public class VisorRunningQuery implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long id; + + /** Query text. */ + private String qry; + + /** Query type. */ + private GridCacheQueryType qryType; + + /** Cache name for query. */ + private String cache; + + /** */ + private long startTime; + + /** */ + private long duration; + + /** */ + private boolean cancellable; + + /** */ + private boolean loc; + + /** + * @param id Query ID. + * @param qry Query text. + * @param qryType Query type. + * @param cache Cache where query was executed. + * @param startTime Query start time. + * @param duration Query current duration. + * @param cancellable {@code true} if query can be canceled. + * @param loc {@code true} if query is local. + */ + public VisorRunningQuery(long id, String qry, GridCacheQueryType qryType, String cache, + long startTime, long duration, + boolean cancellable, boolean loc) { + this.id = id; + this.qry = qry; + this.qryType = qryType; + this.cache = cache; + this.startTime = startTime; + this.duration = duration; + this.cancellable = cancellable; + this.loc = loc; + } + + /** + * @return Query ID. + */ + public long getId() { + return id; + } + + /** + * @return Query txt. + */ + public String getQuery() { + return qry; + } + + /** + * @return Query type. + */ + public GridCacheQueryType getQueryType() { + return qryType; + } + + /** + * @return Cache name. + */ + public String getCache() { + return cache; + } + + /** + * @return Query start time. + */ + public long getStartTime() { + return startTime; + } + + /** + * @return Query duration. + */ + public long getDuration() { + return duration; + } + + /** + * @return {@code true} if query can be cancelled. + */ + public boolean isCancelable() { + return cancellable; + } + + /** + * @return {@code true} if query is local. + */ + public boolean isLocal() { + return loc; + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java index cbf2ebde27056..62b47b85c6507 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java @@ -52,6 +52,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import javax.cache.Cache; import javax.cache.CacheException; import org.apache.ignite.IgniteCheckedException; @@ -79,6 +80,7 @@ import org.apache.ignite.internal.processors.cache.QueryCursorImpl; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable; import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery; +import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; import org.apache.ignite.internal.processors.query.GridQueryFieldsResult; @@ -172,6 +174,9 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_INDEXING_CACHE_CLEANUP_PERIOD; import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_INDEXING_CACHE_THREAD_USAGE_TIMEOUT; import static org.apache.ignite.IgniteSystemProperties.getString; +import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL; +import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS; +import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.TEXT; import static org.apache.ignite.internal.processors.query.GridQueryIndexType.FULLTEXT; import static org.apache.ignite.internal.processors.query.GridQueryIndexType.GEO_SPATIAL; import static org.apache.ignite.internal.processors.query.GridQueryIndexType.SORTED; @@ -278,9 +283,15 @@ public class IgniteH2Indexing implements GridQueryIndexing { /** space name -> schema name */ private final Map space2schema = new ConcurrentHashMap8<>(); + /** */ + private AtomicLong qryIdGen; + /** */ private GridSpinBusyLock busyLock; + /** */ + private final ConcurrentMap runs = new ConcurrentHashMap8<>(); + /** */ private final ThreadLocal connCache = new ThreadLocal() { @Nullable @Override public ConnectionWrapper get() { @@ -751,8 +762,19 @@ private void removeTable(TableDescriptor tbl) throws IgniteCheckedException { IndexingQueryFilter filters) throws IgniteCheckedException { TableDescriptor tbl = tableDescriptor(spaceName, type); - if (tbl != null && tbl.luceneIdx != null) - return tbl.luceneIdx.query(qry, filters); + if (tbl != null && tbl.luceneIdx != null) { + GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, TEXT, spaceName, + U.currentTimeMillis(), null, true); + + try { + runs.put(run.id(), run); + + return tbl.luceneIdx.query(qry, filters); + } + finally { + runs.remove(run.id()); + } + } return new GridEmptyCloseableIterator<>(); } @@ -796,6 +818,11 @@ private void removeTable(TableDescriptor tbl) throws IgniteCheckedException { GridH2QueryContext.set(ctx); + GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL_FIELDS, + spaceName, U.currentTimeMillis(), cancel, true); + + runs.putIfAbsent(run.id(), run); + try { ResultSet rs = executeSqlQueryWithTimer(spaceName, stmt, conn, qry, params, timeout, cancel); @@ -803,6 +830,8 @@ private void removeTable(TableDescriptor tbl) throws IgniteCheckedException { } finally { GridH2QueryContext.clearThreadLocal(); + + runs.remove(run.id()); } } }; @@ -1061,6 +1090,11 @@ public void setupConnection(Connection conn, boolean distributedJoins, boolean e GridH2QueryContext.set(new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter).distributedJoins(false)); + GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL, spaceName, + U.currentTimeMillis(), null, true); + + runs.put(run.id(), run); + try { ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, sql, params, true, 0, null); @@ -1068,6 +1102,8 @@ public void setupConnection(Connection conn, boolean distributedJoins, boolean e } finally { GridH2QueryContext.clearThreadLocal(); + + runs.remove(run.id()); } } @@ -1692,6 +1728,8 @@ public GridReduceQueryExecutor reduceQueryExecutor() { this.busyLock = busyLock; + qryIdGen = new AtomicLong(); + if (SysProperties.serializeJavaObject) { U.warn(log, "Serialization of Java objects in H2 was enabled."); @@ -1742,7 +1780,7 @@ public GridReduceQueryExecutor reduceQueryExecutor() { marshaller = ctx.config().getMarshaller(); mapQryExec = new GridMapQueryExecutor(busyLock); - rdcQryExec = new GridReduceQueryExecutor(busyLock); + rdcQryExec = new GridReduceQueryExecutor(qryIdGen, busyLock); mapQryExec.start(ctx, this); rdcQryExec.start(ctx, this); @@ -2196,6 +2234,37 @@ private static List treeIndexColumns(List cols, IndexC return cols; } + + /** {@inheritDoc} */ + @Override public Collection runningQueries(long duration) { + Collection res = new ArrayList<>(); + + res.addAll(runs.values()); + res.addAll(rdcQryExec.longRunningQueries(duration)); + + return res; + } + + /** {@inheritDoc} */ + @Override public void cancelQueries(Collection queries) { + if (!F.isEmpty(queries)) { + for (Long qryId : queries) { + GridRunningQueryInfo run = runs.get(qryId); + + if (run != null) + run.cancel(); + } + + rdcQryExec.cancelQueries(queries); + } + } + + /** {@inheritDoc} */ + @Override public void cancelAllQueries() { + for (Connection conn : conns) + U.close(conn, log); + } + /** * Wrapper to store connection and flag is schema set or not. */ @@ -3086,10 +3155,4 @@ private void updateLastUsage() { lastUsage = U.currentTimeMillis(); } } - - /** {@inheritDoc} */ - @Override public void cancelAllQueries() { - for (Connection conn : conns) - U.close(conn, log); - } -} \ No newline at end of file +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java index 7d43bf647ec1a..8284c45e0183d 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java @@ -170,7 +170,7 @@ public static GridCacheTwoStepQuery split( qry = collectAllTables(qry, schemas, tbls); // Build resulting two step query. - GridCacheTwoStepQuery res = new GridCacheTwoStepQuery(schemas, tbls); + GridCacheTwoStepQuery res = new GridCacheTwoStepQuery(qry.getSQL(), schemas, tbls); // Map query will be direct reference to the original query AST. // Thus all the modifications will be performed on the original AST, so we should be careful when @@ -954,4 +954,4 @@ private static GridSqlOperation op(GridSqlOperationType type, GridSqlElement lef private static GridSqlFunction function(GridSqlFunctionType type) { return new GridSqlFunction(type); } -} \ No newline at end of file +} 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 40b1197317acd..3f886ee686d66 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 @@ -62,6 +62,7 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable; import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery; 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.h2.GridH2ResultSetIterator; @@ -99,6 +100,7 @@ import org.jsr166.ConcurrentHashMap8; import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE; +import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS; import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.REDUCE; /** @@ -121,7 +123,7 @@ public class GridReduceQueryExecutor { private IgniteLogger log; /** */ - private final AtomicLong reqIdGen = new AtomicLong(); + private final AtomicLong qryIdGen; /** */ private final ConcurrentMap runs = new ConcurrentHashMap8<>(); @@ -168,9 +170,11 @@ public class GridReduceQueryExecutor { }; /** + * @param qryIdGen Query ID generator. * @param busyLock Busy lock. */ - public GridReduceQueryExecutor(GridSpinBusyLock busyLock) { + public GridReduceQueryExecutor(AtomicLong qryIdGen, GridSpinBusyLock busyLock) { + this.qryIdGen = qryIdGen; this.busyLock = busyLock; } @@ -494,11 +498,13 @@ public Iterator> query( } } - final long qryReqId = reqIdGen.incrementAndGet(); + final long qryReqId = qryIdGen.incrementAndGet(); final String space = cctx.name(); - final QueryRun r = new QueryRun(h2.connectionForSpace(space), qry.mapQueries().size(), qry.pageSize()); + final QueryRun r = new QueryRun(qryReqId, qry.originalSql(), space, + h2.connectionForSpace(space), qry.mapQueries().size(), qry.pageSize(), + U.currentTimeMillis(), cancel); AffinityTopologyVersion topVer = h2.readyTopologyVersion(); @@ -1303,10 +1309,46 @@ public void onDisconnected(IgniteFuture reconnectFut) { e.getValue().disconnected(err); } + /** + * Collect queries that already running more than specified duration. + * + * @param duration Duration to check. + * @return Collection of IDs and statements of long running queries. + */ + public Collection longRunningQueries(long duration) { + Collection res = new ArrayList<>(); + + long curTime = U.currentTimeMillis(); + + for (QueryRun run : runs.values()) { + if (run.qry.longQuery(curTime, duration)) + res.add(run.qry); + } + + return res; + } + + /** + * Cancel specified queries. + * + * @param queries Queries IDs to cancel. + */ + public void cancelQueries(Collection queries) { + for (Long qryId : queries) { + QueryRun run = runs.get(qryId); + + if (run != null) + run.qry.cancel(); + } + } + /** * Query run. */ private static class QueryRun { + /** */ + private final GridRunningQueryInfo qry; + /** */ private final List idxs; @@ -1323,11 +1365,17 @@ private static class QueryRun { private final AtomicReference state = new AtomicReference<>(); /** + * @param id Query ID. + * @param qry Query text. + * @param cache Cache where query was executed. * @param conn Connection. * @param idxsCnt Number of indexes. * @param pageSize Page size. + * @param startTime Start time. + * @param cancel Query cancel handler. */ - private QueryRun(Connection conn, int idxsCnt, int pageSize) { + private QueryRun(Long id, String qry, String cache, Connection conn, int idxsCnt, int pageSize, long startTime, GridQueryCancel cancel) { + this.qry = new GridRunningQueryInfo(id, qry, SQL_FIELDS, cache, startTime, cancel, false); this.conn = (JdbcConnection)conn; this.idxs = new ArrayList<>(idxsCnt); this.pageSize = pageSize > 0 ? pageSize : GridCacheTwoStepQuery.DFLT_PAGE_SIZE; @@ -1410,4 +1458,4 @@ private ExplicitPartitionsSpecializer(Map partsMap) { return copy(msg, n, partsMap); } } -} \ No newline at end of file +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java index e47e893530384..66e7e4abf9cdb 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java @@ -17,15 +17,23 @@ package org.apache.ignite.internal.processors.cache; +import java.util.Collection; +import java.util.Collections; import java.util.List; import javax.cache.Cache; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.query.Query; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.cache.query.annotations.QuerySqlFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; @@ -54,6 +62,7 @@ public class CacheSqlQueryValueCopySelfTest extends GridCommonAbstractTest { cc.setCopyOnRead(true); cc.setIndexedTypes(Integer.class, Value.class); + cc.setSqlFunctionClasses(TestSQLFunctions.class); cfg.setCacheConfiguration(cc); @@ -72,7 +81,7 @@ public class CacheSqlQueryValueCopySelfTest extends GridCommonAbstractTest { IgniteCache cache = grid(0).cache(null); for (int i = 0; i < KEYS; i++) - cache.put(i, new Value("before")); + cache.put(i, new Value(i, "before-" + i)); } /** {@inheritDoc} */ @@ -195,17 +204,148 @@ public void testLocalSqlFieldsQuery() { check(cache); } - /** */ - private static class Value { - /** */ - private String str; + /** + * Run specified query in separate thread. + * + * @param qry Query to execute. + */ + private IgniteInternalFuture runQueryAsync(final Query qry) throws Exception { + return multithreadedAsync(new Runnable() { + @Override public void run() { + try { + log.info(">>> Query started"); + + grid(0).cache(null).query(qry).getAll(); + + log.info(">>> Query finished"); + } + catch (Throwable e) { + e.printStackTrace(); + } + } + }, 1, "run-query"); + } - /** - * @param str String. - */ - public Value(String str) { - this.str = str; + /** + * Test collecting info about running. + * + * @throws Exception If failed. + */ + public void testRunningSqlFieldsQuery() throws Exception { + IgniteInternalFuture fut = runQueryAsync(new SqlFieldsQuery("select _val, sleep(1000) from Value limit 3")); + + Thread.sleep(500); + + GridQueryProcessor qryProc = grid(0).context().query(); + + Collection queries = qryProc.runningQueries(0); + + assertEquals(1, queries.size()); + + fut.get(); + + queries = qryProc.runningQueries(0); + + assertEquals(0, queries.size()); + + SqlFieldsQuery qry = new SqlFieldsQuery("select _val, sleep(1000) from Value limit 3"); + qry.setLocal(true); + + fut = runQueryAsync(qry); + + Thread.sleep(500); + + queries = qryProc.runningQueries(0); + + assertEquals(1, queries.size()); + + fut.get(); + + queries = qryProc.runningQueries(0); + + assertEquals(0, queries.size()); + } + + /** + * Test collecting info about running. + * + * @throws Exception If failed. + */ + public void testRunningSqlQuery() throws Exception { + IgniteInternalFuture fut = runQueryAsync(new SqlQuery(Value.class, "id > sleep(100)")); + + Thread.sleep(500); + + GridQueryProcessor qryProc = grid(0).context().query(); + + Collection queries = qryProc.runningQueries(0); + + assertEquals(1, queries.size()); + + fut.get(); + + queries = qryProc.runningQueries(0); + + assertEquals(0, queries.size()); + + SqlQuery qry = new SqlQuery<>(Value.class, "id > sleep(100)"); + qry.setLocal(true); + + fut = runQueryAsync(qry); + + Thread.sleep(500); + + queries = qryProc.runningQueries(0); + + assertEquals(1, queries.size()); + + fut.get(); + + queries = qryProc.runningQueries(0); + + assertEquals(0, queries.size()); + } + + /** + * Test collecting info about running. + * + * @throws Exception If failed. + */ + public void testCancelingSqlFieldsQuery() throws Exception { + runQueryAsync(new SqlFieldsQuery("select * from (select _val, sleep(100) from Value limit 50)")); + + Thread.sleep(500); + + final GridQueryProcessor qryProc = grid(0).context().query(); + + Collection queries = qryProc.runningQueries(0); + + assertEquals(1, queries.size()); + + final Collection finalQueries = queries; + + for (GridRunningQueryInfo query : finalQueries) + qryProc.cancelQueries(Collections.singleton(query.id())); + + int n = 100; + + // Give cluster some time to cancel query and cleanup resources. + while (n > 0) { + Thread.sleep(100); + + queries = qryProc.runningQueries(0); + + if (queries.isEmpty()) + break; + + log.info(">>>> Wait for cancel: " + n); + + n--; } + + queries = qryProc.runningQueries(0); + + assertEquals(0, queries.size()); } /** @@ -218,9 +358,53 @@ private void check(IgniteCache cache) { for (Cache.Entry entry : cache) { cnt++; - assertEquals("before", entry.getValue().str); + assertEquals("before-" + entry.getKey(), entry.getValue().str); } assertEquals(KEYS, cnt); } -} \ No newline at end of file + + /** */ + private static class Value { + /** */ + @QuerySqlField + private int id; + + /** */ + @QuerySqlField + private String str; + + /** + * @param id ID. + * @param str String. + */ + public Value(int id, String str) { + this.id = id; + this.str = str; + } + } + + /** + * Utility class with custom SQL functions. + */ + public static class TestSQLFunctions { + /** + * Sleep function to simulate long running queries. + * + * @param x Time to sleep. + * @return Return specified argument. + */ + @QuerySqlFunction + public static long sleep(long x) { + if (x >= 0) + try { + Thread.sleep(x); + } + catch (InterruptedException ignored) { + // No-op. + } + + return x; + } + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java index 1f10593e61e0c..01fefa3ac95e3 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java @@ -477,4 +477,4 @@ public int getStoreId() { return storeId; } } -} \ No newline at end of file +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java index ad8a7e34e82a7..814d0e0eef516 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java @@ -106,6 +106,9 @@ protected void startIndexing(IgniteH2Indexing spi) throws Exception { spi.registerCache(null, cacheCfg("B")); } + /** + * @param name Name. + */ private CacheConfiguration cacheCfg(String name) { CacheConfiguration cfg = new CacheConfiguration<>(); @@ -114,6 +117,7 @@ private CacheConfiguration cacheCfg(String name) { return cfg; } + /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { idx.stop(); @@ -182,6 +186,9 @@ private IgniteH2Indexing getIndexing() { return idx; } + /** + * @return {@code true} if OFF-HEAP mode should be tested. + */ protected boolean offheap() { return false; } From 2f57760dbb4fba948cd035498d2c7f71869c0665 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 17 Feb 2017 16:15:31 +0300 Subject: [PATCH 453/487] IGNITE-4624: Scan query optimization. This closes #1509. --- .../distributed/dht/GridDhtCacheAdapter.java | 19 +++- .../cache/query/GridCacheQueryManager.java | 97 ++++++++++--------- 2 files changed, 64 insertions(+), 52 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java index dcd379a040bd0..be7fa5525deba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java @@ -1247,14 +1247,27 @@ public Iterator> localEntriesIterator(final boolean primary, final boolean backup, final boolean keepBinary, final AffinityTopologyVersion topVer) { + + return iterator(localEntriesIteratorEx(primary, backup, topVer), !keepBinary); + } + + /** + * @param primary If {@code true} includes primary entries. + * @param backup If {@code true} includes backup entries. + * @param topVer Specified affinity topology version. + * @return Local entries iterator. + */ + public Iterator localEntriesIteratorEx(final boolean primary, + final boolean backup, + final AffinityTopologyVersion topVer) { assert primary || backup; if (primary && backup) - return iterator(entries().iterator(), !keepBinary); + return entries().iterator(); else { final Iterator partIt = topology().currentLocalPartitions().iterator(); - Iterator it = new Iterator() { + return new Iterator() { private GridCacheMapEntry next; private Iterator curIt; @@ -1311,8 +1324,6 @@ private void advance() { while (partIt.hasNext()); } }; - - return iterator(it, !keepBinary); } } 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 d64dff4b1a02e..14b1106eb5e1c 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 @@ -1033,23 +1033,25 @@ private GridIterator> swapIterator( * @throws GridDhtUnreservedPartitionException If failed to reserve partition. */ private GridIterator> onheapIterator( - GridCacheQueryAdapter qry, + final GridCacheQueryAdapter qry, AffinityTopologyVersion topVer, final IgniteBiPredicate keyValFilter, - boolean backups, + final boolean backups, final ExpiryPolicy plc, final boolean locNode) throws GridDhtUnreservedPartitionException { - Iterator keyIter; + Iterator entryIter; GridDhtLocalPartition locPart = null; Integer part = qry.partition(); - if (part == null || cctx.isLocal()) { - // Performance optimization. - if (locNode && plc == null && !cctx.isLocal()) { - GridDhtCacheAdapter cache = cctx.isNear() ? cctx.near().dht() : cctx.dht(); + if (cctx.isLocal()) + entryIter = cctx.local().allEntries().iterator(); + else if (part == null) { + GridDhtCacheAdapter cache = cctx.isNear() ? cctx.near().dht() : cctx.dht(); + // Performance optimization. + if (locNode && plc == null) { final Iterator> iter = cache.localEntriesIterator(true, backups, cache.context().keepBinary(), topVer); @@ -1099,12 +1101,10 @@ private void advance() { }; } - IgniteInternalCache keepBinaryCache = cctx.cache().keepBinary(); - - keyIter = backups ? keepBinaryCache.keySetx().iterator() : keepBinaryCache.primaryKeySet().iterator(); + entryIter = cache.localEntriesIteratorEx(true, backups, topVer); } else if (part < 0 || part >= cctx.affinity().partitions()) - keyIter = new GridEmptyIterator<>(); + return new GridEmptyIterator<>(); else { final GridDhtCacheAdapter dht = cctx.isNear() ? cctx.near().dht() : cctx.dht(); @@ -1115,28 +1115,12 @@ else if (part < 0 || part >= cctx.affinity().partitions()) throw new GridDhtUnreservedPartitionException(part, cctx.affinity().affinityTopologyVersion(), "Partition can not be reserved."); - final GridDhtLocalPartition locPart0 = locPart; - - keyIter = new Iterator() { - private Iterator iter0 = locPart0.keySet().iterator(); - - @Override public boolean hasNext() { - return iter0.hasNext(); - } - - @Override public K next() { - return (K)iter0.next(); - } - - @Override public void remove() { - iter0.remove(); - } - }; + entryIter = locPart.allEntries().iterator(); } final GridDhtLocalPartition locPart0 = locPart; - return new PeekValueExpiryAwareIterator(keyIter, plc, topVer, keyValFilter, qry.keepBinary(), locNode, true) { + return new PeekValueExpiryAwareIterator(entryIter, plc, topVer, keyValFilter, qry.keepBinary(), locNode, true) { @Override protected void onClose() { super.onClose(); @@ -1263,18 +1247,20 @@ private GridIterator> scanExpiryIterator( ExpiryPolicy expPlc, final boolean keepBinary, boolean locNode) { - Iterator keyIter = new Iterator() { + Iterator keyIter = new Iterator() { /** {@inheritDoc} */ @Override public boolean hasNext() { return it.hasNext(); } /** {@inheritDoc} */ - @Override public K next() { + @Override public GridCacheEntryEx next() { try { KeyCacheObject key = cctx.toCacheKeyObject(it.next().getKey()); - return (K)cctx.unwrapBinaryIfNeeded(key, keepBinary); + final GridCacheEntryEx entryEx = cctx.cache().entryEx(key); + + return entryEx; } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -2189,8 +2175,8 @@ public QueryMetrics metrics() { } /** - * Gets cache queries detailed metrics. - * Detail metrics could be enabled by setting non-zero value via {@link CacheConfiguration#setQueryDetailMetricsSize(int)} + * Gets cache queries detailed metrics. Detail metrics could be enabled by setting non-zero value via {@link + * CacheConfiguration#setQueryDetailMetricsSize(int)} * * @return Cache queries metrics aggregated by query type and query text. */ @@ -3091,8 +3077,10 @@ private CompoundIterator(List> iters) { private abstract static class CachedResult extends GridFutureAdapter> { /** Absolute position of each recipient. */ private final Map recipients = new GridLeanMap<>(1); + /** */ private CircularQueue queue; + /** */ private int pruned; @@ -3529,10 +3517,10 @@ private class PeekValueExpiryAwareIterator extends GridCloseableIteratorAdapter< private IgniteCacheExpiryPolicy expiryPlc; /** */ - private Iterator keyIt; + private Iterator entryIt; /** - * @param keyIt Key iterator. + * @param entryIter Key iterator. * @param plc Expiry policy. * @param topVer Topology version. * @param keyValFilter Key-value filter. @@ -3540,8 +3528,8 @@ private class PeekValueExpiryAwareIterator extends GridCloseableIteratorAdapter< * @param locNode Local node. * @param heapOnly Heap only. */ - private PeekValueExpiryAwareIterator( - Iterator keyIt, + PeekValueExpiryAwareIterator( + Iterator entryIter, ExpiryPolicy plc, AffinityTopologyVersion topVer, IgniteBiPredicate keyValFilter, @@ -3549,7 +3537,7 @@ private PeekValueExpiryAwareIterator( boolean locNode, boolean heapOnly ) { - this.keyIt = keyIt; + this.entryIt = entryIter; this.plc = plc; this.topVer = topVer; this.keyValFilter = keyValFilter; @@ -3593,15 +3581,27 @@ private PeekValueExpiryAwareIterator( private void advance() { IgniteBiTuple next0 = null; - while (keyIt.hasNext()) { + while (entryIt.hasNext()) { next0 = null; - K key = keyIt.next(); + GridCacheEntryEx entry = entryIt.next(); + + if (entry.deleted()) + continue; + KeyCacheObject key = entry.key(); CacheObject val; try { - val = value(key); + if (heapOnly) + val = entry.peek(true, false, false, expiryPlc); + else + val = value(entry, entry.key()); + } + catch (GridCacheEntryRemovedException ignore) { + assert heapOnly; + + continue; } catch (IgniteCheckedException e) { if (log.isDebugEnabled()) @@ -3664,23 +3664,24 @@ private void sendTtlUpdate() { } /** + * @param entry Entry. * @param key Key. * @return Value. * @throws IgniteCheckedException If failed to peek value. */ - private CacheObject value(K key) throws IgniteCheckedException { + private CacheObject value(GridCacheEntryEx entry, KeyCacheObject key) throws IgniteCheckedException { while (true) { try { - GridCacheEntryEx entry = heapOnly ? cache.peekEx(key) : cache.entryEx(key); + if (entry == null) + entry = cache.entryEx(key); - if (expiryPlc != null && !heapOnly) + if (expiryPlc != null) entry.unswap(); - return entry != null ? entry.peek(true, !heapOnly, !heapOnly, topVer, expiryPlc) : null; + return entry.peek(true, true, true, topVer, expiryPlc); } catch (GridCacheEntryRemovedException ignore) { - if (heapOnly) - return null; + entry = null; } } } From c0e2df26f056cd11690d821146f05e3fd938906e Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Mon, 20 Feb 2017 11:17:35 +0300 Subject: [PATCH 454/487] IGNITE-3429 - Rollback due to broken compilation --- .../Hibernate5CacheKeyTypeConfiguration.java | 52 ------------------- .../HibernateCacheKeyTypeConfiguration.java | 51 ------------------ 2 files changed, 103 deletions(-) delete mode 100644 modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/Hibernate5CacheKeyTypeConfiguration.java delete mode 100644 modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/HibernateCacheKeyTypeConfiguration.java diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/Hibernate5CacheKeyTypeConfiguration.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/Hibernate5CacheKeyTypeConfiguration.java deleted file mode 100644 index 886f69b2500ba..0000000000000 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/Hibernate5CacheKeyTypeConfiguration.java +++ /dev/null @@ -1,52 +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.cache.hibernate.config; - -import java.util.Objects; -import org.apache.ignite.binary.BinaryAbstractIdentityResolver; -import org.apache.ignite.binary.BinaryIdentityResolver; -import org.apache.ignite.binary.BinaryObject; -import org.apache.ignite.binary.BinaryTypeConfiguration; - -/** - * This configuration provides correct {@link BinaryIdentityResolver} implementation - * for Hibernate CacheKey class can be used as a key object. - * - * Note: for Hibernate version < 5.0 {@link HibernateCacheKeyTypeConfiguration} should be used. - - */ -public class Hibernate5CacheKeyTypeConfiguration extends BinaryTypeConfiguration { - - /** {@inheritDoc} */ - public Hibernate5CacheKeyTypeConfiguration() { - super("org.hibernate.cache.internal.CacheKeyImplementation"); - - setIdentityResolver(new BinaryAbstractIdentityResolver() { - @Override protected int hashCode0(BinaryObject obj) { - return obj.field("id").hashCode(); - } - - @Override protected boolean equals0(BinaryObject o1, BinaryObject o2) { - Object obj0 = o1.field("id"); - Object obj1 = o2.field("id"); - - return Objects.equals(obj0, obj1); - } - }); - } -} diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/HibernateCacheKeyTypeConfiguration.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/HibernateCacheKeyTypeConfiguration.java deleted file mode 100644 index c54292e6f40e6..0000000000000 --- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/config/HibernateCacheKeyTypeConfiguration.java +++ /dev/null @@ -1,51 +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.cache.hibernate.config; - -import java.util.Objects; -import org.apache.ignite.binary.BinaryAbstractIdentityResolver; -import org.apache.ignite.binary.BinaryIdentityResolver; -import org.apache.ignite.binary.BinaryObject; -import org.apache.ignite.binary.BinaryTypeConfiguration; - -/** - * This configuration provides correct {@link BinaryIdentityResolver} implementation - * for Hibernate CacheKey class can be used as a key object. - * - * Note: for Hibernate version >= 5.0 {@link Hibernate5CacheKeyTypeConfiguration} should be used. - */ -public class HibernateCacheKeyTypeConfiguration extends BinaryTypeConfiguration { - - /** {@inheritDoc} */ - public HibernateCacheKeyTypeConfiguration() { - super("org.hibernate.cache.spi.CacheKey"); - - setIdentityResolver(new BinaryAbstractIdentityResolver() { - @Override protected int hashCode0(BinaryObject obj) { - return obj.field("key").hashCode(); - } - - @Override protected boolean equals0(BinaryObject o1, BinaryObject o2) { - Object obj0 = o1.field("key"); - Object obj1 = o2.field("key"); - - return Objects.equals(obj0, obj1); - } - }); - } -} From 9fcb3e74f91c8497b7b1358cdff40950cdf5c568 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 28 Feb 2017 16:05:06 +0300 Subject: [PATCH 455/487] IGNITE-4740 - Fix. Service could be deployed/undeployed twice on concurrent cancel and discovery event. --- .../cache/DynamicCacheChangeBatch.java | 14 ++ .../service/GridServiceProcessor.java | 49 ++--- .../GridServiceContinuousQueryRedeploy.java | 167 ++++++++++++++++++ .../testsuites/IgniteKernalSelfTestSuite.java | 2 + 4 files changed, 208 insertions(+), 24 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceContinuousQueryRedeploy.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java index 4dcff9b8bc2f2..a2500633a1ccf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java @@ -113,6 +113,20 @@ public boolean clientReconnect() { return clientReconnect; } + /** + * @return {@code True} if request should trigger partition exchange. + */ + public boolean exchangeNeeded() { + if (reqs != null) { + for (DynamicCacheChangeRequest req : reqs) { + if (req.exchangeNeeded()) + return true; + } + } + + return false; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(DynamicCacheChangeBatch.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 3690f357f1e5e..4eeafed421214 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -65,10 +65,12 @@ import org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessage; import org.apache.ignite.internal.processors.cache.CacheEntryImpl; import org.apache.ignite.internal.processors.cache.CacheIteratorConverter; +import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.internal.processors.cache.query.CacheQuery; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.continuous.AbstractContinuousMessage; import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.GridEmptyIterator; @@ -1468,19 +1470,7 @@ private void processDeployment(CacheEntryEvent ctxs; - - synchronized (locSvcs) { - ctxs = locSvcs.remove(name); - } - - if (ctxs != null) { - synchronized (ctxs) { - cancel(ctxs, ctxs.size()); - } - } + undeploy(name); // Finish deployment futures if undeployment happened. GridFutureAdapter fut = depFuts.remove(name); @@ -1586,6 +1576,12 @@ private class TopologyListener implements GridLocalEventListener { if (!((CacheAffinityChangeMessage)msg).exchangeNeeded()) return; } + else if (msg instanceof DynamicCacheChangeBatch) { + if (!((DynamicCacheChangeBatch)msg).exchangeNeeded()) + return; + } + else + return; } else topVer = new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion(), 0); @@ -1771,21 +1767,26 @@ private void processAssignment(CacheEntryEvent ctxs; + /** + * @param name Name. + */ + private void undeploy(String name) { + svcName.set(name); - synchronized (locSvcs) { - ctxs = locSvcs.remove(name); - } + Collection ctxs; - if (ctxs != null) { - synchronized (ctxs) { - cancel(ctxs, ctxs.size()); - } + synchronized (locSvcs) { + ctxs = locSvcs.remove(name); + } + + if (ctxs != null) { + synchronized (ctxs) { + cancel(ctxs, ctxs.size()); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceContinuousQueryRedeploy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceContinuousQueryRedeploy.java new file mode 100644 index 0000000000000..1a9ef3a084e37 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceContinuousQueryRedeploy.java @@ -0,0 +1,167 @@ +/* + * 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.service; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.Callable; +import javax.cache.event.CacheEntryEvent; +import javax.cache.event.CacheEntryListenerException; +import javax.cache.event.CacheEntryUpdatedListener; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.query.ContinuousQuery; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.services.Service; +import org.apache.ignite.services.ServiceConfiguration; +import org.apache.ignite.services.ServiceContext; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests whether concurrent service cancel and registering ContinuousQuery doesn't causes + * service redeployment. + */ +public class GridServiceContinuousQueryRedeploy extends GridCommonAbstractTest { + /** */ + private static final String CACHE_NAME = "TEST_CACHE"; + + /** */ + private static final String TEST_KEY = "TEST_KEY"; + + /** */ + private static final String SERVICE_NAME = "service1"; + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testServiceRedeploymentAfterCancel() throws Exception { + final Ignite ignite = startGrid(0); + + final IgniteCache managementCache = ignite.getOrCreateCache(CACHE_NAME); + + final ContinuousQuery qry = new ContinuousQuery<>(); + final List evts = Collections.synchronizedList(new ArrayList<>()); + + qry.setLocalListener(new CacheEntryUpdatedListener() { + @Override public void onUpdated( + Iterable> iterable) throws CacheEntryListenerException { + for (CacheEntryEvent event : iterable) + evts.add(event); + } + }); + + int iterations = 100; + + while (iterations-- > 0) { + QueryCursor quorumCursor = managementCache.query(qry); + + IgniteInternalFuture fut1 = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + System.out.println("Deploy " + SERVICE_NAME); + deployService(ignite); + + return null; + } + }); + + IgniteInternalFuture fut2 = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + System.out.println("Undeploy " + SERVICE_NAME); + ignite.services().cancel(SERVICE_NAME); + + return null; + } + }); + + fut1.get(); + fut2.get(); + + U.sleep(100); + + assert evts.size() <= 1 : evts.size(); + + ignite.services().cancel("service1"); + + evts.clear(); + + quorumCursor.close(); + } + + } + + /** + * @param ignite Ignite. + */ + private void deployService(final Ignite ignite) { + ServiceConfiguration svcCfg = new ServiceConfiguration(); + + svcCfg.setService(new ManagementService()); + svcCfg.setName(SERVICE_NAME); + svcCfg.setTotalCount(1); + svcCfg.setMaxPerNodeCount(1); + svcCfg.setNodeFilter(new IgnitePredicate() { + @Override public boolean apply(ClusterNode node) { + return !node.isClient(); + } + }); + + ignite.services().deploy(svcCfg); + } + + /** + * + */ + public static class ManagementService implements Service { + /** */ + private final String name = UUID.randomUUID().toString(); + + /** */ + @IgniteInstanceResource + private Ignite ignite; + + /** {@inheritDoc} */ + @Override public void cancel(ServiceContext ctx) { + System.out.println(name + " shutdown."); + } + + /** {@inheritDoc} */ + @Override public synchronized void init(ServiceContext ctx) throws Exception { + System.out.println(name + " initializing."); + + ignite.cache(CACHE_NAME).put(TEST_KEY, name + " init"); + } + + /** {@inheritDoc} */ + @Override public void execute(ServiceContext ctx) throws Exception { + // No-op + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java index 350b7152c36d9..59777023e8682 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java @@ -50,6 +50,7 @@ import org.apache.ignite.internal.processors.cluster.GridUpdateNotifierSelfTest; import org.apache.ignite.internal.processors.port.GridPortProcessorSelfTest; import org.apache.ignite.internal.processors.service.GridServiceClientNodeTest; +import org.apache.ignite.internal.processors.service.GridServiceContinuousQueryRedeploy; import org.apache.ignite.internal.processors.service.GridServicePackagePrivateSelfTest; import org.apache.ignite.internal.processors.service.GridServiceProcessorMultiNodeConfigSelfTest; import org.apache.ignite.internal.processors.service.GridServiceProcessorMultiNodeSelfTest; @@ -143,6 +144,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(GridServiceProxyClientReconnectSelfTest.class); suite.addTestSuite(IgniteServiceReassignmentTest.class); suite.addTestSuite(IgniteServiceProxyTimeoutInitializedTest.class); + suite.addTestSuite(GridServiceContinuousQueryRedeploy.class); suite.addTestSuite(IgniteServiceDeploymentClassLoadingDefaultMarshallerTest.class); suite.addTestSuite(IgniteServiceDeploymentClassLoadingOptimizedMarshallerTest.class); From 573624796b171b2420b87657598198f40a91f6bb Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Wed, 1 Mar 2017 22:09:40 +0700 Subject: [PATCH 456/487] Implemented support for enforce join order flag. (cherry picked from commit a7f77d4) --- .../internal/visor/query/VisorQueryArgV3.java | 51 +++++++++++++++++++ .../internal/visor/query/VisorQueryJob.java | 6 +-- .../resources/META-INF/classnames.properties | 1 + 3 files changed, 55 insertions(+), 3 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryArgV3.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryArgV3.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryArgV3.java new file mode 100644 index 0000000000000..f32c00a79484b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryArgV3.java @@ -0,0 +1,51 @@ +/* + * 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.visor.query; + +/** + * Arguments for {@link VisorQueryTask}. + */ +public class VisorQueryArgV3 extends VisorQueryArgV2 { + /** */ + private static final long serialVersionUID = 0L; + + /** Enforce join order flag. */ + private final boolean enforceJoinOrder; + + /** + * @param cacheName Cache name for query. + * @param qryTxt Query text. + * @param distributedJoins If {@code true} then distributed joins enabled. + * @param enforceJoinOrder If {@code true} then enforce join order. + * @param loc Flag whether to execute query locally. + * @param pageSize Result batch size. + */ + public VisorQueryArgV3(String cacheName, String qryTxt, + boolean distributedJoins, boolean enforceJoinOrder, boolean loc, int pageSize) { + super(cacheName, qryTxt, distributedJoins, loc, pageSize); + + this.enforceJoinOrder = enforceJoinOrder; + } + + /** + * @return Enforce join order flag. + */ + public boolean enforceJoinOrder() { + return enforceJoinOrder; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java index c66b2dda8d22c..1ac90ad6a3491 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java @@ -131,9 +131,8 @@ private QueryCursor> near(IgniteCache> near(IgniteCache Date: Mon, 20 Feb 2017 18:23:33 +0700 Subject: [PATCH 457/487] IGNITE-4717 Fixed hangs in VisorCacheClearTask. (cherry picked from commit 76f3060) --- .../visor/cache/VisorCacheClearTask.java | 88 +++++-------------- .../visor/compute/VisorGatewayTask.java | 30 ++++++- 2 files changed, 49 insertions(+), 69 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java index 1f1a6fb106bd3..0c8476fa91b10 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.visor.cache; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteCompute; import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.compute.ComputeJobContext; import org.apache.ignite.internal.processors.task.GridInternal; @@ -26,7 +25,6 @@ import org.apache.ignite.internal.visor.VisorJob; import org.apache.ignite.internal.visor.VisorOneNodeTask; import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.resources.JobContextResource; @@ -90,17 +88,11 @@ private VisorCacheClearJob(String cacheName, boolean debug) { } /** - * @param subJob Sub job to execute asynchronously. + * @param fut Future for asynchronous cache operation. * @param idx Index. * @return {@code true} If subJob was not completed and this job should be suspended. */ - private boolean callAsync(IgniteCallable subJob, int idx) { - IgniteCompute compute = ignite.compute(ignite.cluster().forCacheNodes(cacheName)).withAsync(); - - compute.call(subJob); - - IgniteFuture fut = compute.future(); - + private boolean callAsync(IgniteFuture fut, int idx) { futs[idx] = fut; if (fut.isDone()) @@ -119,16 +111,28 @@ private boolean callAsync(IgniteCallable subJob, int idx) { futs = new IgniteFuture[3]; if (futs[0] == null || futs[1] == null || futs[2] == null) { - IgniteCache cache = ignite.cache(cacheName); + IgniteCache cache = ignite.cache(cacheName).withAsync(); + + if (futs[0] == null) { + cache.size(CachePeekMode.PRIMARY); + + if (callAsync(cache.future(), 0)) + return null; + } - if (futs[0] == null && callAsync(new VisorCacheSizeCallable(cache), 0)) - return null; + if (futs[1] == null) { + cache.clear(); - if (futs[1] == null && callAsync(new VisorCacheClearCallable(cache), 1)) - return null; + if (callAsync(cache.future(), 1)) + return null; + } + + if (futs[2] == null) { + cache.size(CachePeekMode.PRIMARY); - if (futs[2] == null && callAsync(new VisorCacheSizeCallable(cache), 2)) - return null; + if (callAsync(cache.future(), 2)) + return null; + } } assert futs[0].isDone() && futs[1].isDone() && futs[2].isDone(); @@ -141,54 +145,4 @@ private boolean callAsync(IgniteCallable subJob, int idx) { return S.toString(VisorCacheClearJob.class, this); } } - - /** - * Callable to get cache size. - */ - @GridInternal - private static class VisorCacheSizeCallable implements IgniteCallable { - /** */ - private static final long serialVersionUID = 0L; - - /** */ - private final IgniteCache cache; - - /** - * @param cache Cache to take size from. - */ - private VisorCacheSizeCallable(IgniteCache cache) { - this.cache = cache; - } - - /** {@inheritDoc} */ - @Override public Integer call() throws Exception { - return cache.size(CachePeekMode.PRIMARY); - } - } - - /** - * Callable to clear cache. - */ - @GridInternal - private static class VisorCacheClearCallable implements IgniteCallable { - /** */ - private static final long serialVersionUID = 0L; - - /** */ - private final IgniteCache cache; - - /** - * @param cache Cache to clear. - */ - private VisorCacheClearCallable(IgniteCache cache) { - this.cache = cache; - } - - /** {@inheritDoc} */ - @Override public Integer call() throws Exception { - cache.clear(); - - return 0; - } - } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorGatewayTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorGatewayTask.java index 2539a26c558c9..a64ec6d506d6b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorGatewayTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorGatewayTask.java @@ -29,21 +29,26 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import org.apache.ignite.IgniteCompute; import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.compute.ComputeJob; import org.apache.ignite.compute.ComputeJobAdapter; +import org.apache.ignite.compute.ComputeJobContext; import org.apache.ignite.compute.ComputeJobResult; import org.apache.ignite.compute.ComputeJobResultPolicy; import org.apache.ignite.compute.ComputeTask; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.util.lang.GridTuple3; +import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.visor.VisorTaskArgument; import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.resources.JobContextResource; import org.jetbrains.annotations.Nullable; /** @@ -101,9 +106,16 @@ private static class VisorGatewayJob extends ComputeJobAdapter { @IgniteInstanceResource protected transient IgniteEx ignite; + /** Auto-inject job context. */ + @JobContextResource + protected transient ComputeJobContext jobCtx; + /** Arguments count. */ private final int argsCnt; + /** Future for spawned task. */ + private transient IgniteFuture fut; + /** * Create job with specified argument. * @@ -284,6 +296,9 @@ private static boolean isBuildInObject(Class cls) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public Object execute() throws IgniteException { + if (fut != null) + return fut.get(); + String nidsArg = argument(0); String taskName = argument(1); @@ -355,8 +370,19 @@ else if (isBuildInObject(argCls)) } } - return ignite.compute(ignite.cluster().forNodeIds(nids)) - .execute(taskName, new VisorTaskArgument<>(nids, jobArgs, false)); + IgniteCompute comp = ignite.compute(ignite.cluster().forNodeIds(nids)).withAsync(); + + comp.execute(taskName, new VisorTaskArgument<>(nids, jobArgs, false)); + + fut = comp.future(); + + fut.listen(new CI1>() { + @Override public void apply(IgniteFuture f) { + jobCtx.callcc(); + } + }); + + return jobCtx.holdcc(); } } } From 7ad8e79fa1077291c50f2f535ecccde6baee0321 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Tue, 7 Mar 2017 14:32:28 +0300 Subject: [PATCH 458/487] ignite-4577 Add non-reachable addresses at the end of addresses list --- .../ignite/internal/util/IgniteUtils.java | 14 ++++++----- .../tcp/TcpCommunicationSpi.java | 25 +++++++++++++++++++ 2 files changed, 33 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 3fa3f7b92eccb..ba118cb118cea 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -1810,15 +1810,16 @@ public static synchronized boolean isLocalHostChanged() throws IOException { /** * @param addrs Addresses. + * @return List of reachable addresses. */ - public static List filterReachable(List addrs) { + public static List filterReachable(Collection addrs) { final int reachTimeout = 2000; if (addrs.isEmpty()) return Collections.emptyList(); if (addrs.size() == 1) { - InetAddress addr = addrs.get(0); + InetAddress addr = F.first(addrs); if (reachable(addr, reachTimeout)) return Collections.singletonList(addr); @@ -1834,8 +1835,7 @@ public static List filterReachable(List addrs) { for (final InetAddress addr : addrs) { futs.add(executor.submit(new Runnable() { - @Override - public void run() { + @Override public void run() { if (reachable(addr, reachTimeout)) { synchronized (res) { res.add(addr); @@ -1848,11 +1848,13 @@ public void run() { for (Future fut : futs) { try { fut.get(); - } catch (InterruptedException e) { + } + catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IgniteException("Thread has been interrupted.", e); - } catch (ExecutionException e) { + } + catch (ExecutionException e) { throw new IgniteException(e); } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 94b7efe078e84..81454f827e58f 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -2334,6 +2334,31 @@ protected GridCommunicationClient createTcpClient(ClusterNode node) throws Ignit if (isExtAddrsExist) addrs.addAll(extAddrs); + Set allInetAddrs = U.newHashSet(addrs.size()); + + for (InetSocketAddress addr : addrs) + allInetAddrs.add(addr.getAddress()); + + List reachableInetAddrs = U.filterReachable(allInetAddrs); + + if (reachableInetAddrs.size() < allInetAddrs.size()) { + LinkedHashSet addrs0 = U.newLinkedHashSet(addrs.size()); + + for (InetSocketAddress addr : addrs) { + if (reachableInetAddrs.contains(addr.getAddress())) + addrs0.add(addr); + } + for (InetSocketAddress addr : addrs) { + if (!reachableInetAddrs.contains(addr.getAddress())) + addrs0.add(addr); + } + + addrs = addrs0; + } + + if (log.isDebugEnabled()) + log.debug("Addresses to connect for node [rmtNode=" + node.id() + ", addrs=" + addrs.toString() + ']'); + boolean conn = false; GridCommunicationClient client = null; IgniteCheckedException errs = null; From bcb139822afa148a7ea3fbb3eecc274f308070f6 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Fri, 10 Mar 2017 15:51:38 +0700 Subject: [PATCH 459/487] IGNITE-4717 VisorClearTask minor fix. (cherry picked from commit d4b87f4) --- .../visor/cache/VisorCacheClearTask.java | 57 ++++++++++++++++++- 1 file changed, 56 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java index 0c8476fa91b10..ce74f1760fa27 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java @@ -25,6 +25,7 @@ import org.apache.ignite.internal.visor.VisorJob; import org.apache.ignite.internal.visor.VisorOneNodeTask; import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.resources.JobContextResource; @@ -145,4 +146,58 @@ private boolean callAsync(IgniteFuture fut, int idx) { return S.toString(VisorCacheClearJob.class, this); } } -} \ No newline at end of file + + /** + * Callable to get cache size. + * + * @deprecated This class needed only for compatibility. + */ + @GridInternal @Deprecated + private static class VisorCacheSizeCallable implements IgniteCallable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final IgniteCache cache; + + /** + * @param cache Cache to take size from. + */ + private VisorCacheSizeCallable(IgniteCache cache) { + this.cache = cache; + } + + /** {@inheritDoc} */ + @Override public Integer call() throws Exception { + return cache.size(CachePeekMode.PRIMARY); + } + } + + /** + * Callable to clear cache. + * + * @deprecated This class needed only for compatibility. + */ + @GridInternal @Deprecated + private static class VisorCacheClearCallable implements IgniteCallable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final IgniteCache cache; + + /** + * @param cache Cache to clear. + */ + private VisorCacheClearCallable(IgniteCache cache) { + this.cache = cache; + } + + /** {@inheritDoc} */ + @Override public Integer call() throws Exception { + cache.clear(); + + return 0; + } + } +} From 0ed4fdacc7cc8cf41b7726fc4a42db1a43241285 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 14 Mar 2017 15:50:03 +0300 Subject: [PATCH 460/487] IGNITE-4761: Fix ServiceProcessor hanging on node stop. This closes #1602. --- .../service/GridServiceProcessor.java | 15 ++-- .../GridServiceProcessorStopSelfTest.java | 75 +++++++++++++++++++ 2 files changed, 83 insertions(+), 7 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 4eeafed421214..6bcfd65cb59e0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -315,6 +315,8 @@ public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteChe busyLock.block(); + U.shutdownNow(GridServiceProcessor.class, depExe, log); + if (!ctx.clientNode()) ctx.event().removeLocalEventListener(topLsnr); @@ -352,8 +354,6 @@ public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteChe } } - U.shutdownNow(GridServiceProcessor.class, depExe, log); - Exception err = new IgniteCheckedException("Operation has been cancelled (node is stopping)."); cancelFutures(depFuts, err); @@ -1399,7 +1399,7 @@ private class ServiceEntriesListener implements CacheEntryUpdatedListener cache = node2.getOrCreateCache(new CacheConfiguration("def") + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)); + + node0.services().deployNodeSingleton("myService", new TestServiceImpl()); + + // Guarantee lock owner will never left topology unexpectedly. + final Integer lockKey = keyForNode(node2.affinity("def"), new AtomicInteger(1), + node2.cluster().localNode()); + + // Lock to hold topology version undone. + final Lock lock = cache.lock(lockKey); + + // Try to change topology once service has deployed. + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + depLatch.await(); + + node1.close(); + + return null; + } + }, "top-change-thread"); + + // Stop node on unstable topology. + GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + depLatch.await(); + + Thread.sleep(1000); + + node0.close(); + + finishLatch.countDown(); + + return null; + } + }, "stopping-node-thread"); + + assertNotNull(node0.services().service("myService")); + + // Freeze topology changing + lock.lock(); + + depLatch.countDown(); + + boolean wait = finishLatch.await(15, TimeUnit.SECONDS); + + if (!wait) + U.dumpThreads(log); + + assertTrue("Deploy future isn't completed", wait); + + fut.get(); + + Ignition.stopAll(true); + } + /** * Simple map service. */ From d124004d8b0396a44c26f4c35c263a15880f508c Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 17 Mar 2017 14:57:48 +0300 Subject: [PATCH 461/487] IGNITE-4473 - Client should re-try connection attempt in case of concurrent network failure. --- .../internal/GridKernalGatewayImpl.java | 8 +- .../apache/ignite/internal/IgniteKernal.java | 120 +++++- .../IgniteNeedReconnectException.java | 40 ++ .../discovery/GridDiscoveryManager.java | 24 ++ .../GridCachePartitionExchangeManager.java | 25 +- .../dht/GridDhtAssignmentFetchFuture.java | 14 +- .../GridDhtPartitionsExchangeFuture.java | 48 ++- .../service/GridServiceProcessor.java | 86 ++-- .../ignite/spi/discovery/tcp/ClientImpl.java | 201 ++++++++-- .../ignite/spi/discovery/tcp/ServerImpl.java | 5 + .../spi/discovery/tcp/TcpDiscoveryImpl.java | 8 + .../spi/discovery/tcp/TcpDiscoverySpi.java | 9 + .../IgniteClientReconnectCacheTest.java | 7 +- .../internal/IgniteClientRejoinTest.java | 378 ++++++++++++++++++ .../tcp/TcpClientDiscoverySpiSelfTest.java | 48 ++- .../IgniteClientReconnectTestSuite.java | 2 + 16 files changed, 929 insertions(+), 94 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java index fe8c580ca63ea..036954a3a280d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java @@ -44,7 +44,7 @@ public class GridKernalGatewayImpl implements GridKernalGateway, Serializable { /** */ @GridToStringExclude - private IgniteFutureImpl reconnectFut; + private volatile IgniteFutureImpl reconnectFut; /** */ private final AtomicReference state = new AtomicReference<>(GridKernalState.STOPPED); @@ -149,6 +149,12 @@ public GridKernalGatewayImpl(String gridName) { /** {@inheritDoc} */ @Override public GridFutureAdapter onDisconnected() { + if (state.get() == GridKernalState.DISCONNECTED) { + assert reconnectFut != null; + + return (GridFutureAdapter)reconnectFut.internalFuture(); + } + GridFutureAdapter fut = new GridFutureAdapter<>(); reconnectFut = new IgniteFutureImpl<>(fut); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 8fda72fc18419..25f7884c889b9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -250,6 +250,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { /** Periodic starvation check interval. */ private static final long PERIODIC_STARVATION_CHECK_FREQ = 1000 * 30; + /** Force complete reconnect future. */ + private static final Object STOP_RECONNECT = new Object(); + /** */ @GridToStringExclude private GridKernalContextImpl ctx; @@ -327,6 +330,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { @GridToStringExclude private final AtomicBoolean stopGuard = new AtomicBoolean(); + /** */ + private final ReconnectState reconnectState = new ReconnectState(); + /** * No-arg constructor is required by externalization. */ @@ -930,6 +936,8 @@ public void start(final IgniteConfiguration cfg, // Notify IO manager the second so further components can send and receive messages. ctx.io().onKernalStart(); + boolean recon = false; + // Callbacks. for (GridComponent comp : ctx) { // Skip discovery manager. @@ -940,10 +948,24 @@ public void start(final IgniteConfiguration cfg, if (comp instanceof GridIoManager) continue; - if (!skipDaemon(comp)) - comp.onKernalStart(); + if (!skipDaemon(comp)) { + try { + comp.onKernalStart(); + } + catch (IgniteNeedReconnectException e) { + assert ctx.discovery().reconnectSupported(); + + if (log.isDebugEnabled()) + log.debug("Failed to start node components on node start, will wait for reconnect: " + e); + + recon = true; + } + } } + if (recon) + reconnectState.waitFirstReconnect(); + // Register MBeans. registerKernalMBean(); registerLocalNodeMBean(); @@ -3274,6 +3296,8 @@ private void unguard() { public void onDisconnected() { Throwable err = null; + reconnectState.waitPreviousReconnect(); + GridFutureAdapter reconnectFut = ctx.gateway().onDisconnected(); if (reconnectFut == null) { @@ -3282,9 +3306,18 @@ public void onDisconnected() { return; } - IgniteFuture userFut = new IgniteFutureImpl<>(reconnectFut); + IgniteFutureImpl curFut = (IgniteFutureImpl)ctx.cluster().get().clientReconnectFuture(); + + IgniteFuture userFut; - ctx.cluster().get().clientReconnectFuture(userFut); + // In case of previous reconnect did not finish keep reconnect future. + if (curFut != null && curFut.internalFuture() == reconnectFut) + userFut = curFut; + else { + userFut = new IgniteFutureImpl<>(reconnectFut); + + ctx.cluster().get().clientReconnectFuture(userFut); + } ctx.disconnected(true); @@ -3337,30 +3370,53 @@ public void onReconnected(final boolean clusterRestarted) { try { ctx.disconnected(false); - GridCompoundFuture reconnectFut = new GridCompoundFuture<>(); + GridCompoundFuture curReconnectFut = reconnectState.curReconnectFut = new GridCompoundFuture<>(); + + reconnectState.reconnectDone = new GridFutureAdapter<>(); for (GridComponent comp : ctx.components()) { IgniteInternalFuture fut = comp.onReconnected(clusterRestarted); if (fut != null) - reconnectFut.add((IgniteInternalFuture)fut); + curReconnectFut.add(fut); } - reconnectFut.add((IgniteInternalFuture)ctx.cache().context().exchange().reconnectExchangeFuture()); + curReconnectFut.add(ctx.cache().context().exchange().reconnectExchangeFuture()); + + curReconnectFut.markInitialized(); - reconnectFut.markInitialized(); + final GridFutureAdapter reconnectDone = reconnectState.reconnectDone; - reconnectFut.listen(new CI1>() { + curReconnectFut.listen(new CI1>() { @Override public void apply(IgniteInternalFuture fut) { try { - fut.get(); + Object res = fut.get(); + + if (res == STOP_RECONNECT) + return; ctx.gateway().onReconnected(); + + reconnectState.firstReconnectFut.onDone(); } catch (IgniteCheckedException e) { - U.error(log, "Failed to reconnect, will stop node", e); + if (!X.hasCause(e, IgniteNeedReconnectException.class, + IgniteClientDisconnectedCheckedException.class)) { + U.error(log, "Failed to reconnect, will stop node.", e); + + reconnectState.firstReconnectFut.onDone(e); - close(); + close(); + } + else { + assert ctx.discovery().reconnectSupported(); + + U.error(log, "Failed to finish reconnect, will retry [locNodeId=" + ctx.localNodeId() + + ", err=" + e.getMessage() + ']'); + } + } + finally { + reconnectDone.onDone(); } } }); @@ -3519,6 +3575,46 @@ public void dumpDebugInfo() { } } + /** + * + */ + private class ReconnectState { + /** */ + private final GridFutureAdapter firstReconnectFut = new GridFutureAdapter(); + + /** */ + private GridCompoundFuture curReconnectFut; + + /** */ + private GridFutureAdapter reconnectDone; + + /** + * @throws IgniteCheckedException If failed. + */ + void waitFirstReconnect() throws IgniteCheckedException { + firstReconnectFut.get(); + } + + /** + * + */ + void waitPreviousReconnect() { + if (curReconnectFut != null && !curReconnectFut.isDone()) { + assert reconnectDone != null; + + curReconnectFut.onDone(STOP_RECONNECT); + + try { + reconnectDone.get(); + } + catch (IgniteCheckedException ignote) { + // No-op. + } + } + + } + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(IgniteKernal.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java new file mode 100644 index 0000000000000..61ab5762b95ac --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java @@ -0,0 +1,40 @@ +/* + * 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.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterNode; +import org.jetbrains.annotations.Nullable; + +/** + * Indicates that node should try reconnect to cluster. + */ +public class IgniteNeedReconnectException extends IgniteCheckedException { + /** */ + private static final long serialVersionUID = 0L; + + /** + * @param locNode Local node. + * @param cause Cause. + */ + public IgniteNeedReconnectException(ClusterNode locNode, @Nullable Throwable cause) { + super("Local node need try to reconnect [locNodeId=" + locNode.id() + ']', cause); + + assert locNode.isClient(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java index 9aa4db1e0042f..2ec10705bedbb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java @@ -112,6 +112,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpiListener; import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator; import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -1890,6 +1891,29 @@ public void failNode(UUID nodeId, @Nullable String warning) { } } + /** + * @return {@code True} if local node client and discovery SPI supports reconnect. + */ + public boolean reconnectSupported() { + DiscoverySpi spi = getSpi(); + + return ctx.clientNode() && (spi instanceof TcpDiscoverySpi) && + !(((TcpDiscoverySpi) spi).isClientReconnectDisabled()); + } + + /** + * Leave cluster and try to join again. + * + * @throws IgniteSpiException If failed. + */ + public void reconnect() { + assert reconnectSupported(); + + DiscoverySpi discoverySpi = getSpi(); + + ((TcpDiscoverySpi)discoverySpi).reconnect(); + } + /** * Updates topology version if current version is smaller than updated. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index 7f11dc45aacb7..92142c0731240 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -43,6 +43,7 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.IgniteNeedReconnectException; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.affinity.AffinityFunction; import org.apache.ignite.cluster.ClusterNode; @@ -447,6 +448,15 @@ else if (m instanceof GridDhtPartitionDemandMessage) else U.warn(log, "Still waiting for initial partition map exchange [fut=" + fut + ']'); } + catch (IgniteNeedReconnectException e) { + throw e; + } + catch (Exception e) { + if (fut.reconnectOnError(e)) + throw new IgniteNeedReconnectException(cctx.localNode(), e); + + throw e; + } } for (GridCacheContext cacheCtx : cctx.cacheContexts()) { @@ -1690,6 +1700,12 @@ void addFuture(GridDhtPartitionsExchangeFuture exchFut) { dumpedObjects++; } } + catch (Exception e) { + if (exchFut.reconnectOnError(e)) + throw new IgniteNeedReconnectException(cctx.localNode(), e); + + throw e; + } } @@ -1829,7 +1845,14 @@ else if (r != null) { catch (IgniteInterruptedCheckedException e) { throw e; } - catch (IgniteClientDisconnectedCheckedException e) { + catch (IgniteClientDisconnectedCheckedException | IgniteNeedReconnectException e) { + assert cctx.discovery().reconnectSupported(); + + U.warn(log,"Local node failed to complete partition map exchange due to " + + "network issues, will try to reconnect to cluster", e); + + cctx.discovery().reconnect(); + return; } catch (IgniteCheckedException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java index ab8e863e90e7d..6425bc141245d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java @@ -17,15 +17,16 @@ package org.apache.ignite.internal.processors.cache.distributed.dht; +import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.LinkedList; import java.util.Queue; import java.util.UUID; -import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicReference; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteNeedReconnectException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.GridNodeOrderComparator; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; @@ -34,6 +35,7 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -202,8 +204,14 @@ private void requestFromNextNode() { "continue to another node): " + node); } catch (IgniteCheckedException e) { - U.error(log0, "Failed to request affinity assignment from remote node (will " + - "continue to another node): " + node, e); + if (ctx.discovery().reconnectSupported() && X.hasCause(e, IOException.class)) { + onDone(new IgniteNeedReconnectException(ctx.localNode(), e)); + + return; + } + + U.warn(log0, "Failed to request affinity assignment from remote node (will " + + "continue to another node): " + node); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index e945de958a339..d4f95e5b55d05 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.preloader; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -32,6 +33,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReadWriteLock; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.IgniteNeedReconnectException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; @@ -39,6 +41,7 @@ import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.Event; import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; @@ -54,7 +57,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology; -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.GridDhtTopologyFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; @@ -65,7 +67,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; @@ -506,10 +508,17 @@ public void init() throws IgniteInterruptedCheckedException { throw e; } + catch (IgniteNeedReconnectException e) { + onDone(e); + } catch (Throwable e) { - U.error(log, "Failed to reinitialize local partitions (preloading will be stopped): " + exchId, e); + if (reconnectOnError(e)) + onDone(new IgniteNeedReconnectException(cctx.localNode(), e)); + else { + U.error(log, "Failed to reinitialize local partitions (preloading will be stopped): " + exchId, e); - onDone(e); + onDone(e); + } if (e instanceof Error) throw (Error)e; @@ -1297,7 +1306,10 @@ private void onAllReceived(boolean discoThread) { } } catch (IgniteCheckedException e) { - onDone(e); + if (reconnectOnError(e)) + onDone(new IgniteNeedReconnectException(cctx.localNode(), e)); + else + onDone(e); } } @@ -1314,8 +1326,15 @@ private void sendAllPartitions(final UUID nodeId, final int retryCnt) { } catch (IgniteCheckedException e) { if (e instanceof ClusterTopologyCheckedException || !cctx.discovery().alive(n)) { - log.debug("Failed to send full partition map to node, node left grid " + - "[rmtNode=" + nodeId + ", exchangeId=" + exchId + ']'); + if (log.isDebugEnabled()) + log.debug("Failed to send full partition map to node, node left grid " + + "[rmtNode=" + nodeId + ", exchangeId=" + exchId + ']'); + + return; + } + + if (reconnectOnError(e)) { + onDone(new IgniteNeedReconnectException(cctx.localNode(), e)); return; } @@ -1641,6 +1660,12 @@ public void onNodeLeft(final ClusterNode node) { } } } + catch (Exception e) { + if (reconnectOnError(e)) + onDone(new IgniteNeedReconnectException(cctx.localNode(), e)); + else + throw e; + } finally { leaveBusy(); } @@ -1652,6 +1677,15 @@ public void onNodeLeft(final ClusterNode node) { } } + /** + * @param e Exception. + * @return {@code True} if local node should try reconnect in case of error. + */ + public boolean reconnectOnError(Throwable e) { + return X.hasCause(e, IOException.class, IgniteClientDisconnectedCheckedException.class) && + cctx.discovery().reconnectSupported(); + } + /** {@inheritDoc} */ @Override public int compareTo(GridDhtPartitionsExchangeFuture fut) { return exchId.compareTo(fut.exchId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 6bcfd65cb59e0..bd815189fb5ba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -1498,60 +1498,60 @@ private void processDeployment(CacheEntryEvent 0; + if (!newTopVer.equals(topVer)) { + assert newTopVer.compareTo(topVer) > 0; - // Reassignment will happen from topology event. - return; - } + // Reassignment will happen from topology event. + return; + } - ctx.timeout().addTimeoutObject(new GridTimeoutObject() { - private IgniteUuid id = IgniteUuid.randomUuid(); + ctx.timeout().addTimeoutObject(new GridTimeoutObject() { + private IgniteUuid id = IgniteUuid.randomUuid(); - private long start = System.currentTimeMillis(); + private long start = System.currentTimeMillis(); - @Override public IgniteUuid timeoutId() { - return id; - } + @Override public IgniteUuid timeoutId() { + return id; + } - @Override public long endTime() { - return start + RETRY_TIMEOUT; - } + @Override public long endTime() { + return start + RETRY_TIMEOUT; + } - @Override public void onTimeout() { - if (!busyLock.enterBusy()) - return; + @Override public void onTimeout() { + if (!busyLock.enterBusy()) + return; - try { - // Try again. - onDeployment(dep, topVer); - } - finally { - busyLock.leaveBusy(); - } + try { + // Try again. + onDeployment(dep, topVer); } - }); + finally { + busyLock.leaveBusy(); + } + } + }); } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 95e2cda4f6907..02ba56a884b33 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -129,6 +129,9 @@ class ClientImpl extends TcpDiscoveryImpl { /** */ private static final Object SPI_RECONNECT_FAILED = "SPI_RECONNECT_FAILED"; + /** */ + private static final Object SPI_RECONNECT = "SPI_RECONNECT"; + /** Remote nodes. */ private final ConcurrentMap rmtNodes = new ConcurrentHashMap8<>(); @@ -808,6 +811,11 @@ private NavigableSet allVisibleNodes() { log); } + /** {@inheritDoc} */ + @Override public void reconnect() throws IgniteSpiException { + msgWorker.addMessage(SPI_RECONNECT); + } + /** {@inheritDoc} */ @Override public void brakeConnection() { SocketStream sockStream = msgWorker.currSock; @@ -879,9 +887,12 @@ private class SocketReader extends IgniteSpiThread { /** */ private UUID rmtNodeId; + /** */ + private CountDownLatch stopReadLatch; + /** */ - protected SocketReader() { + SocketReader() { super(spi.ignite().name(), "tcp-client-disco-sock-reader", log); } @@ -889,7 +900,7 @@ protected SocketReader() { * @param sockStream Socket. * @param rmtNodeId Rmt node id. */ - public void setSocket(SocketStream sockStream, UUID rmtNodeId) { + void setSocket(SocketStream sockStream, UUID rmtNodeId) { synchronized (mux) { this.sockStream = sockStream; @@ -899,6 +910,31 @@ public void setSocket(SocketStream sockStream, UUID rmtNodeId) { } } + /** + * @throws InterruptedException If interrupted. + */ + private void forceStopRead() throws InterruptedException { + CountDownLatch stopReadLatch; + + synchronized (mux) { + SocketStream stream = sockStream; + + if (stream == null) + return; + + this.stopReadLatch = stopReadLatch = new CountDownLatch(1); + + U.closeQuiet(stream.socket()); + + this.sockStream = null; + this.rmtNodeId = null; + + mux.notifyAll(); + } + + stopReadLatch.await(); + } + /** {@inheritDoc} */ @Override protected void body() throws InterruptedException { while (!isInterrupted()) { @@ -906,6 +942,12 @@ public void setSocket(SocketStream sockStream, UUID rmtNodeId) { UUID rmtNodeId; synchronized (mux) { + if (stopReadLatch != null) { + stopReadLatch.countDown(); + + stopReadLatch = null; + } + if (this.sockStream == null) { mux.wait(); @@ -1007,18 +1049,21 @@ private class SocketWriter extends IgniteSpiThread { private final Queue queue = new ArrayDeque<>(); /** */ - private final long socketTimeout; + private final long sockTimeout; /** */ private TcpDiscoveryAbstractMessage unackedMsg; + /** */ + private CountDownLatch forceLeaveLatch; + /** * */ - protected SocketWriter() { + SocketWriter() { super(spi.ignite().name(), "tcp-client-disco-sock-writer", log); - socketTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() : + sockTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() : spi.getSocketTimeout(); } @@ -1033,6 +1078,29 @@ private void sendMessage(TcpDiscoveryAbstractMessage msg) { } } + /** + * Sends {@link TcpDiscoveryNodeLeftMessage} and closes socket. + * + * @throws InterruptedException If interrupted. + */ + private void forceLeave() throws InterruptedException { + CountDownLatch forceLeaveLatch; + + synchronized (mux) { + // If writer was stopped. + if (sock == null) + return; + + this.forceLeaveLatch = forceLeaveLatch = new CountDownLatch(1); + + unackedMsg = null; + + mux.notifyAll(); + } + + forceLeaveLatch.await(); + } + /** * @param sock Socket. * @param clientAck {@code True} is server supports client message acknowlede. @@ -1089,13 +1157,41 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { continue; } - msg = queue.poll(); + if (forceLeaveLatch != null) { + msg = new TcpDiscoveryNodeLeftMessage(getLocalNodeId()); - if (msg == null) { - mux.wait(); + msg.client(true); + + try { + spi.writeToSocket( + sock, + msg, + sockTimeout); + } + catch (IOException | IgniteCheckedException e) { + if (log.isDebugEnabled()) { + log.debug("Failed to send TcpDiscoveryNodeLeftMessage on force leave [msg=" + msg + + ", err=" + e.getMessage() + ']'); + } + } + + U.closeQuiet(sock); + + this.sock = null; + + clear(); continue; } + else { + msg = queue.poll(); + + if (msg == null) { + mux.wait(); + + continue; + } + } } for (IgniteInClosure msgLsnr : spi.sndMsgLsnrs) @@ -1115,7 +1211,7 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { spi.writeToSocket( sock, msg, - socketTimeout); + sockTimeout); msg = null; @@ -1165,10 +1261,30 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { synchronized (mux) { if (sock == this.sock) this.sock = null; // Connection has dead. + + clear(); } } } } + + /** + * + */ + private void clear() { + assert Thread.holdsLock(mux); + + queue.clear(); + unackedMsg = null; + + CountDownLatch forceLeaveLatch = this.forceLeaveLatch; + + if (forceLeaveLatch != null) { + this.forceLeaveLatch = null; + + forceLeaveLatch.countDown(); + } + } } /** @@ -1413,6 +1529,38 @@ else if (msg == SPI_STOP) { else leaveLatch.countDown(); } + else if (msg == SPI_RECONNECT) { + if (state == CONNECTED) { + if (reconnector != null) { + reconnector.cancel(); + reconnector.join(); + + reconnector = null; + } + + sockWriter.forceLeave(); + sockReader.forceStopRead(); + + currSock = null; + + queue.clear(); + + onDisconnected(); + + notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes()); + + UUID newId = UUID.randomUUID(); + + U.quietAndWarn(log, "Local node will try to reconnect to cluster with new id due " + + "to network problems [newId=" + newId + + ", prevId=" + locNode.id() + + ", locNode=" + locNode+ ']'); + + locNode.onClientDisconnected(newId); + + tryJoin(); + } + } else if (msg instanceof TcpDiscoveryNodeFailedMessage && ((TcpDiscoveryNodeFailedMessage)msg).failedNodeId().equals(locNode.id())) { TcpDiscoveryNodeFailedMessage msg0 = (TcpDiscoveryNodeFailedMessage)msg; @@ -1495,20 +1643,7 @@ else if (msg == SPI_RECONNECT_FAILED) { ", failMsg=" + forceFailMsg + ']'); } - state = DISCONNECTED; - - nodeAdded = false; - - IgniteClientDisconnectedCheckedException err = - new IgniteClientDisconnectedCheckedException(null, "Failed to ping node, " + - "client node disconnected."); - - for (Map.Entry> e : pingFuts.entrySet()) { - GridFutureAdapter fut = e.getValue(); - - if (pingFuts.remove(e.getKey(), fut)) - fut.onDone(err); - } + onDisconnected(); notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes()); } @@ -1603,6 +1738,26 @@ else if (discoMsg instanceof TcpDiscoveryCheckFailedMessage) } } + /** + * + */ + private void onDisconnected() { + state = DISCONNECTED; + + nodeAdded = false; + + IgniteClientDisconnectedCheckedException err = + new IgniteClientDisconnectedCheckedException(null, "Failed to ping node, " + + "client node disconnected."); + + for (Map.Entry> e : pingFuts.entrySet()) { + GridFutureAdapter fut = e.getValue(); + + if (pingFuts.remove(e.getKey(), fut)) + fut.onDone(err); + } + } + /** * @throws InterruptedException If interrupted. */ diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 4600be094c840..afd1c2ba7fc66 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -1609,6 +1609,11 @@ private void clearNodeAddedMessage(TcpDiscoveryAbstractMessage msg) { throw new UnsupportedOperationException(); } + /** {@inheritDoc} */ + @Override public void reconnect() throws IgniteSpiException { + throw new UnsupportedOperationException("Reconnect is not supported for server."); + } + /** {@inheritDoc} */ @Override protected IgniteSpiThread workerThread() { return msgWorker; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java index f199c20e17edb..84c2ff28afef0 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java @@ -29,6 +29,7 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.U; @@ -258,6 +259,13 @@ protected static String threadStatus(Thread t) { return t.isAlive() ? "alive" : "dead"; } + /** + * Leave cluster and try to join again. + * + * @throws IgniteSpiException If failed. + */ + public abstract void reconnect() throws IgniteSpiException; + /** * FOR TEST ONLY!!! *

          diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java index 00ae97d1e302c..a2a47feb31486 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java @@ -1926,6 +1926,15 @@ boolean isSslEnabled() { return ignite().configuration().getSslContextFactory() != null; } + /** + * Force reconnect to cluster. + * + * @throws IgniteSpiException If failed. + */ + public void reconnect() throws IgniteSpiException { + impl.reconnect(); + } + /** * FOR TEST ONLY!!! */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java index 0f0165b0f04f9..6cdf465978907 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java @@ -700,9 +700,12 @@ public void testReconnectInitialExchangeInProgress() throws Exception { try { Ignition.start(optimize(getConfiguration(getTestGridName(SRV_CNT)))); - fail(); + // Commented due to IGNITE-4473, because + // IgniteClientDisconnectedException won't + // be thrown, but client will reconnect. +// fail(); - return false; + return true; } catch (IgniteClientDisconnectedException e) { log.info("Expected start error: " + e); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java new file mode 100644 index 0000000000000..a5d42e9a12cba --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java @@ -0,0 +1,378 @@ +/* + * 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 java.io.IOException; +import java.io.OutputStream; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.net.SocketException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteClientDisconnectedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.IgniteSpiOperationTimeoutException; +import org.apache.ignite.spi.IgniteSpiOperationTimeoutHelper; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests client to be able restore connection to cluster if coordination is not available. + */ +public class IgniteClientRejoinTest extends GridCommonAbstractTest { + /** Block. */ + private volatile boolean block; + + /** Block all. */ + private volatile boolean blockAll; + + /** Coordinator. */ + private volatile ClusterNode crd; + + /** Client reconnect disabled. */ + private boolean clientReconnectDisabled; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty("IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK", "true"); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + System.clearProperty("IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK"); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + clientReconnectDisabled = false; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + if (gridName.contains("client")) { + cfg.setCommunicationSpi(new TcpCommunicationSpi()); + + TcpDiscoverySpi spi = (TcpDiscoverySpi)cfg.getDiscoverySpi(); + DiscoverySpi dspi = new DiscoverySpi(); + + dspi.setIpFinder(spi.getIpFinder()); + + cfg.setDiscoverySpi(dspi); + + dspi.setJoinTimeout(60_000); + dspi.setClientReconnectDisabled(clientReconnectDisabled); + + cfg.setClientMode(true); + } + + // TODO: IGNITE-4833 + cfg.setPeerClassLoadingEnabled(false); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testClientsReconnectAfterStart() throws Exception { + Ignite srv1 = startGrid("server1"); + + crd = ((IgniteKernal)srv1).localNode(); + + Ignite srv2 = startGrid("server2"); + + final CountDownLatch latch = new CountDownLatch(1); + + List clientNodes = new ArrayList<>(); + + final int CLIENTS_NUM = 5; + + for (int i = 0; i < CLIENTS_NUM; i++) + clientNodes.add(startGrid("client" + i)); + + blockAll = true; + + GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + U.sleep(5_000); + + block = true; + blockAll = false; + + System.out.println(">>> Allow with blocked coordinator."); + + latch.countDown(); + + return null; + } + }); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + latch.await(); + + U.sleep((new Random().nextInt(15) + 30) * 1000); + + block = false; + + System.out.println(">>> Allow coordinator."); + + return null; + } + }); + + fut.get(); + + for (Ignite client : clientNodes) { + while (true) { + try { + IgniteCache cache = client.getOrCreateCache("some"); + + for (int i = 0; i < 100; i++) + cache.put(i, i); + + for (int i = 0; i < 100; i++) + assertEquals((Integer)i, cache.get(i)); + + cache.clear(); + + break; + } + catch (IgniteClientDisconnectedException e) { + e.reconnectFuture().get(); + } + } + } + + assertEquals(CLIENTS_NUM, srv1.cluster().forClients().nodes().size()); + assertEquals(CLIENTS_NUM, srv2.cluster().forClients().nodes().size()); + } + + /** + * @throws Exception If failed. + */ + public void testClientsReconnect() throws Exception { + Ignite srv1 = startGrid("server1"); + + crd = ((IgniteKernal)srv1).localNode(); + + Ignite srv2 = startGrid("server2"); + + block = true; + + List> futs = new ArrayList<>(); + + final CountDownLatch latch = new CountDownLatch(1); + + final int CLIENTS_NUM = 5; + + for (int i = 0; i < CLIENTS_NUM; i++) { + final int idx = i; + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Ignite call() throws Exception { + latch.await(); + + return startGrid("client" + idx); + } + }); + + futs.add(fut); + } + + GridTestUtils.runAsync(new Callable() { + @Override public Boolean call() throws Exception { + latch.countDown(); + + Random rnd = new Random(); + + U.sleep((rnd.nextInt(15) + 15) * 1000); + + block = false; + + System.out.println(">>> ALLOW connection to coordinator."); + + return true; + } + }); + + for (IgniteInternalFuture clientFut : futs) { + Ignite client = clientFut.get(); + + IgniteCache cache = client.getOrCreateCache(client.name()); + + for (int i = 0; i < 100; i++) + cache.put(i, i); + + for (int i = 0; i < 100; i++) + assert i == cache.get(i); + } + + assertEquals(CLIENTS_NUM, srv1.cluster().forClients().nodes().size()); + assertEquals(CLIENTS_NUM, srv2.cluster().forClients().nodes().size()); + } + + /** + * @throws Exception If failed. + */ + public void testClientsReconnectDisabled() throws Exception { + clientReconnectDisabled = true; + + Ignite srv1 = startGrid("server1"); + + crd = ((IgniteKernal)srv1).localNode(); + + Ignite srv2 = startGrid("server2"); + + block = true; + + List> futs = new ArrayList<>(); + + final CountDownLatch latch = new CountDownLatch(1); + + final int CLIENTS_NUM = 5; + + for (int i = 0; i < CLIENTS_NUM; i++) { + final int idx = i; + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Ignite call() throws Exception { + latch.await(); + + return startGrid("client" + idx); + } + }); + + futs.add(fut); + } + + latch.countDown(); + + for (final IgniteInternalFuture clientFut : futs) { + //noinspection ThrowableNotThrown + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + clientFut.get(); + + return null; + } + }, IgniteCheckedException.class, null); + } + + assertEquals(0, srv1.cluster().forClients().nodes().size()); + assertEquals(0, srv2.cluster().forClients().nodes().size()); + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 3 * 60_000; + } + + /** + * + */ + private class TcpCommunicationSpi extends org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi { + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException { + if (blockAll || block && node.id().equals(crd.id())) + throw new IgniteSpiException(new SocketException("Test communication exception")); + + super.sendMessage(node, msg); + } + + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, + IgniteInClosure ackC) throws IgniteSpiException { + if (blockAll || block && node.id().equals(crd.id())) + throw new IgniteSpiException(new SocketException("Test communication exception")); + + super.sendMessage(node, msg, ackC); + } + } + + /** + * + */ + private class DiscoverySpi extends TcpDiscoverySpi { + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, byte[] data, + long timeout) throws IOException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + super.writeToSocket(sock, msg, data, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, + long timeout) throws IOException, IgniteCheckedException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + super.writeToSocket(sock, msg, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, OutputStream out, TcpDiscoveryAbstractMessage msg, + long timeout) throws IOException, IgniteCheckedException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + super.writeToSocket(sock, out, msg, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(TcpDiscoveryAbstractMessage msg, Socket sock, int res, + long timeout) throws IOException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + super.writeToSocket(msg, sock, res, timeout); + } + + /** {@inheritDoc} */ + @Override protected Socket openSocket(Socket sock, InetSocketAddress remAddr, + IgniteSpiOperationTimeoutHelper timeoutHelper) throws IOException, IgniteSpiOperationTimeoutException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + return super.openSocket(sock, remAddr, timeoutHelper); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java index 331b581bb7bec..0483a1ce8fbc6 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java @@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteInterruptedException; import org.apache.ignite.IgniteMessaging; import org.apache.ignite.IgniteState; @@ -43,6 +44,7 @@ import org.apache.ignite.events.Event; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.lang.GridAbsPredicate; @@ -1788,8 +1790,7 @@ else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { clientNodeIds.add(client.cluster().localNode().id()); GridTestUtils.waitForCondition(new GridAbsPredicate() { - @Override - public boolean apply() { + @Override public boolean apply() { return srv.cluster().nodes().size() == 2; } }, awaitTime()); @@ -1799,6 +1800,49 @@ public boolean apply() { assertFalse(err.get()); } + /** + * @throws Exception If failed. + */ + public void testForceClientReconnect() throws Exception { + startServerNodes(1); + + startClientNodes(1); + + Ignite srv = G.ignite("server-0"); + IgniteKernal client = (IgniteKernal)G.ignite("client-0"); + + UUID clientId = F.first(clientNodeIds); + + final CountDownLatch latch = new CountDownLatch(1); + + srv.events().enableLocal(EVT_NODE_JOINED); + + srv.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + latch.countDown(); + + return false; + } + }, EVT_NODE_JOINED); + + client.context().discovery().reconnect(); + + assert latch.await(10, TimeUnit.SECONDS); + + while (true) { + try { + UUID newId = client.localNode().id(); + + assert !clientId.equals(newId) : clientId; + + break; + } + catch (IgniteClientDisconnectedException e) { + e.reconnectFuture().get(10_000); + } + } + } + /** * @param ignite Ignite. * @throws Exception If failed. diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java index ea8e37bada1d0..67d88e1e29eac 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.IgniteClientReconnectServicesTest; import org.apache.ignite.internal.IgniteClientReconnectStopTest; import org.apache.ignite.internal.IgniteClientReconnectStreamerTest; +import org.apache.ignite.internal.IgniteClientRejoinTest; /** * @@ -52,6 +53,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteClientReconnectServicesTest.class); suite.addTestSuite(IgniteClientReconnectStreamerTest.class); suite.addTestSuite(IgniteClientReconnectFailoverTest.class); + suite.addTestSuite(IgniteClientRejoinTest.class); return suite; } From a2b4751f5eefd70a5a1aa26652c9671240125f78 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 17 Mar 2017 14:57:48 +0300 Subject: [PATCH 462/487] IGNITE-4473 - Client should re-try connection attempt in case of concurrent network failure. (cherry picked from commit d124004) --- .../internal/GridKernalGatewayImpl.java | 8 +- .../apache/ignite/internal/IgniteKernal.java | 120 +++++- .../IgniteNeedReconnectException.java | 40 ++ .../discovery/GridDiscoveryManager.java | 24 ++ .../GridCachePartitionExchangeManager.java | 25 +- .../dht/GridDhtAssignmentFetchFuture.java | 14 +- .../GridDhtPartitionsExchangeFuture.java | 48 ++- .../service/GridServiceProcessor.java | 86 ++-- .../ignite/spi/discovery/tcp/ClientImpl.java | 201 ++++++++-- .../ignite/spi/discovery/tcp/ServerImpl.java | 5 + .../spi/discovery/tcp/TcpDiscoveryImpl.java | 8 + .../spi/discovery/tcp/TcpDiscoverySpi.java | 9 + .../IgniteClientReconnectCacheTest.java | 7 +- .../internal/IgniteClientRejoinTest.java | 378 ++++++++++++++++++ .../tcp/TcpClientDiscoverySpiSelfTest.java | 48 ++- .../IgniteClientReconnectTestSuite.java | 2 + 16 files changed, 929 insertions(+), 94 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java index fe8c580ca63ea..036954a3a280d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java @@ -44,7 +44,7 @@ public class GridKernalGatewayImpl implements GridKernalGateway, Serializable { /** */ @GridToStringExclude - private IgniteFutureImpl reconnectFut; + private volatile IgniteFutureImpl reconnectFut; /** */ private final AtomicReference state = new AtomicReference<>(GridKernalState.STOPPED); @@ -149,6 +149,12 @@ public GridKernalGatewayImpl(String gridName) { /** {@inheritDoc} */ @Override public GridFutureAdapter onDisconnected() { + if (state.get() == GridKernalState.DISCONNECTED) { + assert reconnectFut != null; + + return (GridFutureAdapter)reconnectFut.internalFuture(); + } + GridFutureAdapter fut = new GridFutureAdapter<>(); reconnectFut = new IgniteFutureImpl<>(fut); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 8fda72fc18419..25f7884c889b9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -250,6 +250,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { /** Periodic starvation check interval. */ private static final long PERIODIC_STARVATION_CHECK_FREQ = 1000 * 30; + /** Force complete reconnect future. */ + private static final Object STOP_RECONNECT = new Object(); + /** */ @GridToStringExclude private GridKernalContextImpl ctx; @@ -327,6 +330,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { @GridToStringExclude private final AtomicBoolean stopGuard = new AtomicBoolean(); + /** */ + private final ReconnectState reconnectState = new ReconnectState(); + /** * No-arg constructor is required by externalization. */ @@ -930,6 +936,8 @@ public void start(final IgniteConfiguration cfg, // Notify IO manager the second so further components can send and receive messages. ctx.io().onKernalStart(); + boolean recon = false; + // Callbacks. for (GridComponent comp : ctx) { // Skip discovery manager. @@ -940,10 +948,24 @@ public void start(final IgniteConfiguration cfg, if (comp instanceof GridIoManager) continue; - if (!skipDaemon(comp)) - comp.onKernalStart(); + if (!skipDaemon(comp)) { + try { + comp.onKernalStart(); + } + catch (IgniteNeedReconnectException e) { + assert ctx.discovery().reconnectSupported(); + + if (log.isDebugEnabled()) + log.debug("Failed to start node components on node start, will wait for reconnect: " + e); + + recon = true; + } + } } + if (recon) + reconnectState.waitFirstReconnect(); + // Register MBeans. registerKernalMBean(); registerLocalNodeMBean(); @@ -3274,6 +3296,8 @@ private void unguard() { public void onDisconnected() { Throwable err = null; + reconnectState.waitPreviousReconnect(); + GridFutureAdapter reconnectFut = ctx.gateway().onDisconnected(); if (reconnectFut == null) { @@ -3282,9 +3306,18 @@ public void onDisconnected() { return; } - IgniteFuture userFut = new IgniteFutureImpl<>(reconnectFut); + IgniteFutureImpl curFut = (IgniteFutureImpl)ctx.cluster().get().clientReconnectFuture(); + + IgniteFuture userFut; - ctx.cluster().get().clientReconnectFuture(userFut); + // In case of previous reconnect did not finish keep reconnect future. + if (curFut != null && curFut.internalFuture() == reconnectFut) + userFut = curFut; + else { + userFut = new IgniteFutureImpl<>(reconnectFut); + + ctx.cluster().get().clientReconnectFuture(userFut); + } ctx.disconnected(true); @@ -3337,30 +3370,53 @@ public void onReconnected(final boolean clusterRestarted) { try { ctx.disconnected(false); - GridCompoundFuture reconnectFut = new GridCompoundFuture<>(); + GridCompoundFuture curReconnectFut = reconnectState.curReconnectFut = new GridCompoundFuture<>(); + + reconnectState.reconnectDone = new GridFutureAdapter<>(); for (GridComponent comp : ctx.components()) { IgniteInternalFuture fut = comp.onReconnected(clusterRestarted); if (fut != null) - reconnectFut.add((IgniteInternalFuture)fut); + curReconnectFut.add(fut); } - reconnectFut.add((IgniteInternalFuture)ctx.cache().context().exchange().reconnectExchangeFuture()); + curReconnectFut.add(ctx.cache().context().exchange().reconnectExchangeFuture()); + + curReconnectFut.markInitialized(); - reconnectFut.markInitialized(); + final GridFutureAdapter reconnectDone = reconnectState.reconnectDone; - reconnectFut.listen(new CI1>() { + curReconnectFut.listen(new CI1>() { @Override public void apply(IgniteInternalFuture fut) { try { - fut.get(); + Object res = fut.get(); + + if (res == STOP_RECONNECT) + return; ctx.gateway().onReconnected(); + + reconnectState.firstReconnectFut.onDone(); } catch (IgniteCheckedException e) { - U.error(log, "Failed to reconnect, will stop node", e); + if (!X.hasCause(e, IgniteNeedReconnectException.class, + IgniteClientDisconnectedCheckedException.class)) { + U.error(log, "Failed to reconnect, will stop node.", e); + + reconnectState.firstReconnectFut.onDone(e); - close(); + close(); + } + else { + assert ctx.discovery().reconnectSupported(); + + U.error(log, "Failed to finish reconnect, will retry [locNodeId=" + ctx.localNodeId() + + ", err=" + e.getMessage() + ']'); + } + } + finally { + reconnectDone.onDone(); } } }); @@ -3519,6 +3575,46 @@ public void dumpDebugInfo() { } } + /** + * + */ + private class ReconnectState { + /** */ + private final GridFutureAdapter firstReconnectFut = new GridFutureAdapter(); + + /** */ + private GridCompoundFuture curReconnectFut; + + /** */ + private GridFutureAdapter reconnectDone; + + /** + * @throws IgniteCheckedException If failed. + */ + void waitFirstReconnect() throws IgniteCheckedException { + firstReconnectFut.get(); + } + + /** + * + */ + void waitPreviousReconnect() { + if (curReconnectFut != null && !curReconnectFut.isDone()) { + assert reconnectDone != null; + + curReconnectFut.onDone(STOP_RECONNECT); + + try { + reconnectDone.get(); + } + catch (IgniteCheckedException ignote) { + // No-op. + } + } + + } + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(IgniteKernal.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java new file mode 100644 index 0000000000000..61ab5762b95ac --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java @@ -0,0 +1,40 @@ +/* + * 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.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterNode; +import org.jetbrains.annotations.Nullable; + +/** + * Indicates that node should try reconnect to cluster. + */ +public class IgniteNeedReconnectException extends IgniteCheckedException { + /** */ + private static final long serialVersionUID = 0L; + + /** + * @param locNode Local node. + * @param cause Cause. + */ + public IgniteNeedReconnectException(ClusterNode locNode, @Nullable Throwable cause) { + super("Local node need try to reconnect [locNodeId=" + locNode.id() + ']', cause); + + assert locNode.isClient(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java index 9aa4db1e0042f..2ec10705bedbb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java @@ -112,6 +112,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpiListener; import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator; import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; @@ -1890,6 +1891,29 @@ public void failNode(UUID nodeId, @Nullable String warning) { } } + /** + * @return {@code True} if local node client and discovery SPI supports reconnect. + */ + public boolean reconnectSupported() { + DiscoverySpi spi = getSpi(); + + return ctx.clientNode() && (spi instanceof TcpDiscoverySpi) && + !(((TcpDiscoverySpi) spi).isClientReconnectDisabled()); + } + + /** + * Leave cluster and try to join again. + * + * @throws IgniteSpiException If failed. + */ + public void reconnect() { + assert reconnectSupported(); + + DiscoverySpi discoverySpi = getSpi(); + + ((TcpDiscoverySpi)discoverySpi).reconnect(); + } + /** * Updates topology version if current version is smaller than updated. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index 7f11dc45aacb7..92142c0731240 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -43,6 +43,7 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.IgniteNeedReconnectException; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.affinity.AffinityFunction; import org.apache.ignite.cluster.ClusterNode; @@ -447,6 +448,15 @@ else if (m instanceof GridDhtPartitionDemandMessage) else U.warn(log, "Still waiting for initial partition map exchange [fut=" + fut + ']'); } + catch (IgniteNeedReconnectException e) { + throw e; + } + catch (Exception e) { + if (fut.reconnectOnError(e)) + throw new IgniteNeedReconnectException(cctx.localNode(), e); + + throw e; + } } for (GridCacheContext cacheCtx : cctx.cacheContexts()) { @@ -1690,6 +1700,12 @@ void addFuture(GridDhtPartitionsExchangeFuture exchFut) { dumpedObjects++; } } + catch (Exception e) { + if (exchFut.reconnectOnError(e)) + throw new IgniteNeedReconnectException(cctx.localNode(), e); + + throw e; + } } @@ -1829,7 +1845,14 @@ else if (r != null) { catch (IgniteInterruptedCheckedException e) { throw e; } - catch (IgniteClientDisconnectedCheckedException e) { + catch (IgniteClientDisconnectedCheckedException | IgniteNeedReconnectException e) { + assert cctx.discovery().reconnectSupported(); + + U.warn(log,"Local node failed to complete partition map exchange due to " + + "network issues, will try to reconnect to cluster", e); + + cctx.discovery().reconnect(); + return; } catch (IgniteCheckedException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java index ab8e863e90e7d..6425bc141245d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java @@ -17,15 +17,16 @@ package org.apache.ignite.internal.processors.cache.distributed.dht; +import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.LinkedList; import java.util.Queue; import java.util.UUID; -import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicReference; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteNeedReconnectException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.GridNodeOrderComparator; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; @@ -34,6 +35,7 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -202,8 +204,14 @@ private void requestFromNextNode() { "continue to another node): " + node); } catch (IgniteCheckedException e) { - U.error(log0, "Failed to request affinity assignment from remote node (will " + - "continue to another node): " + node, e); + if (ctx.discovery().reconnectSupported() && X.hasCause(e, IOException.class)) { + onDone(new IgniteNeedReconnectException(ctx.localNode(), e)); + + return; + } + + U.warn(log0, "Failed to request affinity assignment from remote node (will " + + "continue to another node): " + node); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index e945de958a339..d4f95e5b55d05 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.preloader; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -32,6 +33,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReadWriteLock; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.IgniteNeedReconnectException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; @@ -39,6 +41,7 @@ import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.Event; import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; @@ -54,7 +57,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology; -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.GridDhtTopologyFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; @@ -65,7 +67,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; @@ -506,10 +508,17 @@ public void init() throws IgniteInterruptedCheckedException { throw e; } + catch (IgniteNeedReconnectException e) { + onDone(e); + } catch (Throwable e) { - U.error(log, "Failed to reinitialize local partitions (preloading will be stopped): " + exchId, e); + if (reconnectOnError(e)) + onDone(new IgniteNeedReconnectException(cctx.localNode(), e)); + else { + U.error(log, "Failed to reinitialize local partitions (preloading will be stopped): " + exchId, e); - onDone(e); + onDone(e); + } if (e instanceof Error) throw (Error)e; @@ -1297,7 +1306,10 @@ private void onAllReceived(boolean discoThread) { } } catch (IgniteCheckedException e) { - onDone(e); + if (reconnectOnError(e)) + onDone(new IgniteNeedReconnectException(cctx.localNode(), e)); + else + onDone(e); } } @@ -1314,8 +1326,15 @@ private void sendAllPartitions(final UUID nodeId, final int retryCnt) { } catch (IgniteCheckedException e) { if (e instanceof ClusterTopologyCheckedException || !cctx.discovery().alive(n)) { - log.debug("Failed to send full partition map to node, node left grid " + - "[rmtNode=" + nodeId + ", exchangeId=" + exchId + ']'); + if (log.isDebugEnabled()) + log.debug("Failed to send full partition map to node, node left grid " + + "[rmtNode=" + nodeId + ", exchangeId=" + exchId + ']'); + + return; + } + + if (reconnectOnError(e)) { + onDone(new IgniteNeedReconnectException(cctx.localNode(), e)); return; } @@ -1641,6 +1660,12 @@ public void onNodeLeft(final ClusterNode node) { } } } + catch (Exception e) { + if (reconnectOnError(e)) + onDone(new IgniteNeedReconnectException(cctx.localNode(), e)); + else + throw e; + } finally { leaveBusy(); } @@ -1652,6 +1677,15 @@ public void onNodeLeft(final ClusterNode node) { } } + /** + * @param e Exception. + * @return {@code True} if local node should try reconnect in case of error. + */ + public boolean reconnectOnError(Throwable e) { + return X.hasCause(e, IOException.class, IgniteClientDisconnectedCheckedException.class) && + cctx.discovery().reconnectSupported(); + } + /** {@inheritDoc} */ @Override public int compareTo(GridDhtPartitionsExchangeFuture fut) { return exchId.compareTo(fut.exchId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 6bcfd65cb59e0..bd815189fb5ba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -1498,60 +1498,60 @@ private void processDeployment(CacheEntryEvent 0; + if (!newTopVer.equals(topVer)) { + assert newTopVer.compareTo(topVer) > 0; - // Reassignment will happen from topology event. - return; - } + // Reassignment will happen from topology event. + return; + } - ctx.timeout().addTimeoutObject(new GridTimeoutObject() { - private IgniteUuid id = IgniteUuid.randomUuid(); + ctx.timeout().addTimeoutObject(new GridTimeoutObject() { + private IgniteUuid id = IgniteUuid.randomUuid(); - private long start = System.currentTimeMillis(); + private long start = System.currentTimeMillis(); - @Override public IgniteUuid timeoutId() { - return id; - } + @Override public IgniteUuid timeoutId() { + return id; + } - @Override public long endTime() { - return start + RETRY_TIMEOUT; - } + @Override public long endTime() { + return start + RETRY_TIMEOUT; + } - @Override public void onTimeout() { - if (!busyLock.enterBusy()) - return; + @Override public void onTimeout() { + if (!busyLock.enterBusy()) + return; - try { - // Try again. - onDeployment(dep, topVer); - } - finally { - busyLock.leaveBusy(); - } + try { + // Try again. + onDeployment(dep, topVer); } - }); + finally { + busyLock.leaveBusy(); + } + } + }); } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 95e2cda4f6907..02ba56a884b33 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -129,6 +129,9 @@ class ClientImpl extends TcpDiscoveryImpl { /** */ private static final Object SPI_RECONNECT_FAILED = "SPI_RECONNECT_FAILED"; + /** */ + private static final Object SPI_RECONNECT = "SPI_RECONNECT"; + /** Remote nodes. */ private final ConcurrentMap rmtNodes = new ConcurrentHashMap8<>(); @@ -808,6 +811,11 @@ private NavigableSet allVisibleNodes() { log); } + /** {@inheritDoc} */ + @Override public void reconnect() throws IgniteSpiException { + msgWorker.addMessage(SPI_RECONNECT); + } + /** {@inheritDoc} */ @Override public void brakeConnection() { SocketStream sockStream = msgWorker.currSock; @@ -879,9 +887,12 @@ private class SocketReader extends IgniteSpiThread { /** */ private UUID rmtNodeId; + /** */ + private CountDownLatch stopReadLatch; + /** */ - protected SocketReader() { + SocketReader() { super(spi.ignite().name(), "tcp-client-disco-sock-reader", log); } @@ -889,7 +900,7 @@ protected SocketReader() { * @param sockStream Socket. * @param rmtNodeId Rmt node id. */ - public void setSocket(SocketStream sockStream, UUID rmtNodeId) { + void setSocket(SocketStream sockStream, UUID rmtNodeId) { synchronized (mux) { this.sockStream = sockStream; @@ -899,6 +910,31 @@ public void setSocket(SocketStream sockStream, UUID rmtNodeId) { } } + /** + * @throws InterruptedException If interrupted. + */ + private void forceStopRead() throws InterruptedException { + CountDownLatch stopReadLatch; + + synchronized (mux) { + SocketStream stream = sockStream; + + if (stream == null) + return; + + this.stopReadLatch = stopReadLatch = new CountDownLatch(1); + + U.closeQuiet(stream.socket()); + + this.sockStream = null; + this.rmtNodeId = null; + + mux.notifyAll(); + } + + stopReadLatch.await(); + } + /** {@inheritDoc} */ @Override protected void body() throws InterruptedException { while (!isInterrupted()) { @@ -906,6 +942,12 @@ public void setSocket(SocketStream sockStream, UUID rmtNodeId) { UUID rmtNodeId; synchronized (mux) { + if (stopReadLatch != null) { + stopReadLatch.countDown(); + + stopReadLatch = null; + } + if (this.sockStream == null) { mux.wait(); @@ -1007,18 +1049,21 @@ private class SocketWriter extends IgniteSpiThread { private final Queue queue = new ArrayDeque<>(); /** */ - private final long socketTimeout; + private final long sockTimeout; /** */ private TcpDiscoveryAbstractMessage unackedMsg; + /** */ + private CountDownLatch forceLeaveLatch; + /** * */ - protected SocketWriter() { + SocketWriter() { super(spi.ignite().name(), "tcp-client-disco-sock-writer", log); - socketTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() : + sockTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() : spi.getSocketTimeout(); } @@ -1033,6 +1078,29 @@ private void sendMessage(TcpDiscoveryAbstractMessage msg) { } } + /** + * Sends {@link TcpDiscoveryNodeLeftMessage} and closes socket. + * + * @throws InterruptedException If interrupted. + */ + private void forceLeave() throws InterruptedException { + CountDownLatch forceLeaveLatch; + + synchronized (mux) { + // If writer was stopped. + if (sock == null) + return; + + this.forceLeaveLatch = forceLeaveLatch = new CountDownLatch(1); + + unackedMsg = null; + + mux.notifyAll(); + } + + forceLeaveLatch.await(); + } + /** * @param sock Socket. * @param clientAck {@code True} is server supports client message acknowlede. @@ -1089,13 +1157,41 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { continue; } - msg = queue.poll(); + if (forceLeaveLatch != null) { + msg = new TcpDiscoveryNodeLeftMessage(getLocalNodeId()); - if (msg == null) { - mux.wait(); + msg.client(true); + + try { + spi.writeToSocket( + sock, + msg, + sockTimeout); + } + catch (IOException | IgniteCheckedException e) { + if (log.isDebugEnabled()) { + log.debug("Failed to send TcpDiscoveryNodeLeftMessage on force leave [msg=" + msg + + ", err=" + e.getMessage() + ']'); + } + } + + U.closeQuiet(sock); + + this.sock = null; + + clear(); continue; } + else { + msg = queue.poll(); + + if (msg == null) { + mux.wait(); + + continue; + } + } } for (IgniteInClosure msgLsnr : spi.sndMsgLsnrs) @@ -1115,7 +1211,7 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { spi.writeToSocket( sock, msg, - socketTimeout); + sockTimeout); msg = null; @@ -1165,10 +1261,30 @@ void ackReceived(TcpDiscoveryClientAckResponse res) { synchronized (mux) { if (sock == this.sock) this.sock = null; // Connection has dead. + + clear(); } } } } + + /** + * + */ + private void clear() { + assert Thread.holdsLock(mux); + + queue.clear(); + unackedMsg = null; + + CountDownLatch forceLeaveLatch = this.forceLeaveLatch; + + if (forceLeaveLatch != null) { + this.forceLeaveLatch = null; + + forceLeaveLatch.countDown(); + } + } } /** @@ -1413,6 +1529,38 @@ else if (msg == SPI_STOP) { else leaveLatch.countDown(); } + else if (msg == SPI_RECONNECT) { + if (state == CONNECTED) { + if (reconnector != null) { + reconnector.cancel(); + reconnector.join(); + + reconnector = null; + } + + sockWriter.forceLeave(); + sockReader.forceStopRead(); + + currSock = null; + + queue.clear(); + + onDisconnected(); + + notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes()); + + UUID newId = UUID.randomUUID(); + + U.quietAndWarn(log, "Local node will try to reconnect to cluster with new id due " + + "to network problems [newId=" + newId + + ", prevId=" + locNode.id() + + ", locNode=" + locNode+ ']'); + + locNode.onClientDisconnected(newId); + + tryJoin(); + } + } else if (msg instanceof TcpDiscoveryNodeFailedMessage && ((TcpDiscoveryNodeFailedMessage)msg).failedNodeId().equals(locNode.id())) { TcpDiscoveryNodeFailedMessage msg0 = (TcpDiscoveryNodeFailedMessage)msg; @@ -1495,20 +1643,7 @@ else if (msg == SPI_RECONNECT_FAILED) { ", failMsg=" + forceFailMsg + ']'); } - state = DISCONNECTED; - - nodeAdded = false; - - IgniteClientDisconnectedCheckedException err = - new IgniteClientDisconnectedCheckedException(null, "Failed to ping node, " + - "client node disconnected."); - - for (Map.Entry> e : pingFuts.entrySet()) { - GridFutureAdapter fut = e.getValue(); - - if (pingFuts.remove(e.getKey(), fut)) - fut.onDone(err); - } + onDisconnected(); notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes()); } @@ -1603,6 +1738,26 @@ else if (discoMsg instanceof TcpDiscoveryCheckFailedMessage) } } + /** + * + */ + private void onDisconnected() { + state = DISCONNECTED; + + nodeAdded = false; + + IgniteClientDisconnectedCheckedException err = + new IgniteClientDisconnectedCheckedException(null, "Failed to ping node, " + + "client node disconnected."); + + for (Map.Entry> e : pingFuts.entrySet()) { + GridFutureAdapter fut = e.getValue(); + + if (pingFuts.remove(e.getKey(), fut)) + fut.onDone(err); + } + } + /** * @throws InterruptedException If interrupted. */ diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 4600be094c840..afd1c2ba7fc66 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -1609,6 +1609,11 @@ private void clearNodeAddedMessage(TcpDiscoveryAbstractMessage msg) { throw new UnsupportedOperationException(); } + /** {@inheritDoc} */ + @Override public void reconnect() throws IgniteSpiException { + throw new UnsupportedOperationException("Reconnect is not supported for server."); + } + /** {@inheritDoc} */ @Override protected IgniteSpiThread workerThread() { return msgWorker; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java index f199c20e17edb..84c2ff28afef0 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java @@ -29,6 +29,7 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.U; @@ -258,6 +259,13 @@ protected static String threadStatus(Thread t) { return t.isAlive() ? "alive" : "dead"; } + /** + * Leave cluster and try to join again. + * + * @throws IgniteSpiException If failed. + */ + public abstract void reconnect() throws IgniteSpiException; + /** * FOR TEST ONLY!!! *

          diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java index 00ae97d1e302c..a2a47feb31486 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java @@ -1926,6 +1926,15 @@ boolean isSslEnabled() { return ignite().configuration().getSslContextFactory() != null; } + /** + * Force reconnect to cluster. + * + * @throws IgniteSpiException If failed. + */ + public void reconnect() throws IgniteSpiException { + impl.reconnect(); + } + /** * FOR TEST ONLY!!! */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java index 0f0165b0f04f9..6cdf465978907 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java @@ -700,9 +700,12 @@ public void testReconnectInitialExchangeInProgress() throws Exception { try { Ignition.start(optimize(getConfiguration(getTestGridName(SRV_CNT)))); - fail(); + // Commented due to IGNITE-4473, because + // IgniteClientDisconnectedException won't + // be thrown, but client will reconnect. +// fail(); - return false; + return true; } catch (IgniteClientDisconnectedException e) { log.info("Expected start error: " + e); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java new file mode 100644 index 0000000000000..a5d42e9a12cba --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java @@ -0,0 +1,378 @@ +/* + * 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 java.io.IOException; +import java.io.OutputStream; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.net.SocketException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteClientDisconnectedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.IgniteSpiOperationTimeoutException; +import org.apache.ignite.spi.IgniteSpiOperationTimeoutHelper; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests client to be able restore connection to cluster if coordination is not available. + */ +public class IgniteClientRejoinTest extends GridCommonAbstractTest { + /** Block. */ + private volatile boolean block; + + /** Block all. */ + private volatile boolean blockAll; + + /** Coordinator. */ + private volatile ClusterNode crd; + + /** Client reconnect disabled. */ + private boolean clientReconnectDisabled; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + System.setProperty("IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK", "true"); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + System.clearProperty("IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK"); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + clientReconnectDisabled = false; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + if (gridName.contains("client")) { + cfg.setCommunicationSpi(new TcpCommunicationSpi()); + + TcpDiscoverySpi spi = (TcpDiscoverySpi)cfg.getDiscoverySpi(); + DiscoverySpi dspi = new DiscoverySpi(); + + dspi.setIpFinder(spi.getIpFinder()); + + cfg.setDiscoverySpi(dspi); + + dspi.setJoinTimeout(60_000); + dspi.setClientReconnectDisabled(clientReconnectDisabled); + + cfg.setClientMode(true); + } + + // TODO: IGNITE-4833 + cfg.setPeerClassLoadingEnabled(false); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testClientsReconnectAfterStart() throws Exception { + Ignite srv1 = startGrid("server1"); + + crd = ((IgniteKernal)srv1).localNode(); + + Ignite srv2 = startGrid("server2"); + + final CountDownLatch latch = new CountDownLatch(1); + + List clientNodes = new ArrayList<>(); + + final int CLIENTS_NUM = 5; + + for (int i = 0; i < CLIENTS_NUM; i++) + clientNodes.add(startGrid("client" + i)); + + blockAll = true; + + GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + U.sleep(5_000); + + block = true; + blockAll = false; + + System.out.println(">>> Allow with blocked coordinator."); + + latch.countDown(); + + return null; + } + }); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + latch.await(); + + U.sleep((new Random().nextInt(15) + 30) * 1000); + + block = false; + + System.out.println(">>> Allow coordinator."); + + return null; + } + }); + + fut.get(); + + for (Ignite client : clientNodes) { + while (true) { + try { + IgniteCache cache = client.getOrCreateCache("some"); + + for (int i = 0; i < 100; i++) + cache.put(i, i); + + for (int i = 0; i < 100; i++) + assertEquals((Integer)i, cache.get(i)); + + cache.clear(); + + break; + } + catch (IgniteClientDisconnectedException e) { + e.reconnectFuture().get(); + } + } + } + + assertEquals(CLIENTS_NUM, srv1.cluster().forClients().nodes().size()); + assertEquals(CLIENTS_NUM, srv2.cluster().forClients().nodes().size()); + } + + /** + * @throws Exception If failed. + */ + public void testClientsReconnect() throws Exception { + Ignite srv1 = startGrid("server1"); + + crd = ((IgniteKernal)srv1).localNode(); + + Ignite srv2 = startGrid("server2"); + + block = true; + + List> futs = new ArrayList<>(); + + final CountDownLatch latch = new CountDownLatch(1); + + final int CLIENTS_NUM = 5; + + for (int i = 0; i < CLIENTS_NUM; i++) { + final int idx = i; + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Ignite call() throws Exception { + latch.await(); + + return startGrid("client" + idx); + } + }); + + futs.add(fut); + } + + GridTestUtils.runAsync(new Callable() { + @Override public Boolean call() throws Exception { + latch.countDown(); + + Random rnd = new Random(); + + U.sleep((rnd.nextInt(15) + 15) * 1000); + + block = false; + + System.out.println(">>> ALLOW connection to coordinator."); + + return true; + } + }); + + for (IgniteInternalFuture clientFut : futs) { + Ignite client = clientFut.get(); + + IgniteCache cache = client.getOrCreateCache(client.name()); + + for (int i = 0; i < 100; i++) + cache.put(i, i); + + for (int i = 0; i < 100; i++) + assert i == cache.get(i); + } + + assertEquals(CLIENTS_NUM, srv1.cluster().forClients().nodes().size()); + assertEquals(CLIENTS_NUM, srv2.cluster().forClients().nodes().size()); + } + + /** + * @throws Exception If failed. + */ + public void testClientsReconnectDisabled() throws Exception { + clientReconnectDisabled = true; + + Ignite srv1 = startGrid("server1"); + + crd = ((IgniteKernal)srv1).localNode(); + + Ignite srv2 = startGrid("server2"); + + block = true; + + List> futs = new ArrayList<>(); + + final CountDownLatch latch = new CountDownLatch(1); + + final int CLIENTS_NUM = 5; + + for (int i = 0; i < CLIENTS_NUM; i++) { + final int idx = i; + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Ignite call() throws Exception { + latch.await(); + + return startGrid("client" + idx); + } + }); + + futs.add(fut); + } + + latch.countDown(); + + for (final IgniteInternalFuture clientFut : futs) { + //noinspection ThrowableNotThrown + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + clientFut.get(); + + return null; + } + }, IgniteCheckedException.class, null); + } + + assertEquals(0, srv1.cluster().forClients().nodes().size()); + assertEquals(0, srv2.cluster().forClients().nodes().size()); + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 3 * 60_000; + } + + /** + * + */ + private class TcpCommunicationSpi extends org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi { + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException { + if (blockAll || block && node.id().equals(crd.id())) + throw new IgniteSpiException(new SocketException("Test communication exception")); + + super.sendMessage(node, msg); + } + + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, + IgniteInClosure ackC) throws IgniteSpiException { + if (blockAll || block && node.id().equals(crd.id())) + throw new IgniteSpiException(new SocketException("Test communication exception")); + + super.sendMessage(node, msg, ackC); + } + } + + /** + * + */ + private class DiscoverySpi extends TcpDiscoverySpi { + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, byte[] data, + long timeout) throws IOException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + super.writeToSocket(sock, msg, data, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, + long timeout) throws IOException, IgniteCheckedException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + super.writeToSocket(sock, msg, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, OutputStream out, TcpDiscoveryAbstractMessage msg, + long timeout) throws IOException, IgniteCheckedException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + super.writeToSocket(sock, out, msg, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(TcpDiscoveryAbstractMessage msg, Socket sock, int res, + long timeout) throws IOException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + super.writeToSocket(msg, sock, res, timeout); + } + + /** {@inheritDoc} */ + @Override protected Socket openSocket(Socket sock, InetSocketAddress remAddr, + IgniteSpiOperationTimeoutHelper timeoutHelper) throws IOException, IgniteSpiOperationTimeoutException { + if (blockAll || block && sock.getPort() == 47500) + throw new SocketException("Test discovery exception"); + + return super.openSocket(sock, remAddr, timeoutHelper); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java index 331b581bb7bec..0483a1ce8fbc6 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java @@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteInterruptedException; import org.apache.ignite.IgniteMessaging; import org.apache.ignite.IgniteState; @@ -43,6 +44,7 @@ import org.apache.ignite.events.Event; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.lang.GridAbsPredicate; @@ -1788,8 +1790,7 @@ else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { clientNodeIds.add(client.cluster().localNode().id()); GridTestUtils.waitForCondition(new GridAbsPredicate() { - @Override - public boolean apply() { + @Override public boolean apply() { return srv.cluster().nodes().size() == 2; } }, awaitTime()); @@ -1799,6 +1800,49 @@ public boolean apply() { assertFalse(err.get()); } + /** + * @throws Exception If failed. + */ + public void testForceClientReconnect() throws Exception { + startServerNodes(1); + + startClientNodes(1); + + Ignite srv = G.ignite("server-0"); + IgniteKernal client = (IgniteKernal)G.ignite("client-0"); + + UUID clientId = F.first(clientNodeIds); + + final CountDownLatch latch = new CountDownLatch(1); + + srv.events().enableLocal(EVT_NODE_JOINED); + + srv.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + latch.countDown(); + + return false; + } + }, EVT_NODE_JOINED); + + client.context().discovery().reconnect(); + + assert latch.await(10, TimeUnit.SECONDS); + + while (true) { + try { + UUID newId = client.localNode().id(); + + assert !clientId.equals(newId) : clientId; + + break; + } + catch (IgniteClientDisconnectedException e) { + e.reconnectFuture().get(10_000); + } + } + } + /** * @param ignite Ignite. * @throws Exception If failed. diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java index ea8e37bada1d0..67d88e1e29eac 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.IgniteClientReconnectServicesTest; import org.apache.ignite.internal.IgniteClientReconnectStopTest; import org.apache.ignite.internal.IgniteClientReconnectStreamerTest; +import org.apache.ignite.internal.IgniteClientRejoinTest; /** * @@ -52,6 +53,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteClientReconnectServicesTest.class); suite.addTestSuite(IgniteClientReconnectStreamerTest.class); suite.addTestSuite(IgniteClientReconnectFailoverTest.class); + suite.addTestSuite(IgniteClientRejoinTest.class); return suite; } From c4de164392ddc114c88d5a6eba0ff0b13d32542f Mon Sep 17 00:00:00 2001 From: AMRepo Date: Mon, 20 Mar 2017 16:31:15 +0300 Subject: [PATCH 463/487] IGNITE-518: Unmuted tests that was fixed in ignite-4036. This closes #1636. --- .../expiry/IgniteCacheExpiryPolicyAbstractTest.java | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java index b234631e4002a..0b1b8c5fff3a4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java @@ -162,7 +162,8 @@ public void testCreateUpdate0() throws Exception { assertEquals(0, pSize); } - /** * @throws Exception If failed. + /** + * @throws Exception If failed. */ public void testZeroOnCreate() throws Exception { factory = CreatedExpiryPolicy.factoryOf(Duration.ZERO); @@ -849,8 +850,6 @@ private void createUpdate(Integer key, @Nullable TransactionConcurrency txConcur * @throws Exception If failed. */ public void testNearCreateUpdate() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-518"); - if (cacheMode() != PARTITIONED) return; @@ -974,8 +973,6 @@ private void nearPutAll() throws Exception { * @throws Exception If failed. */ public void testNearAccess() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-518"); - if (cacheMode() != PARTITIONED) return; @@ -1023,10 +1020,10 @@ public void testNearAccess() throws Exception { * @throws Exception If failed. */ public void testNearExpiresOnClient() throws Exception { - if(cacheMode() != PARTITIONED) + if (cacheMode() != PARTITIONED) return; - factory = CreatedExpiryPolicy.factoryOf(new Duration(TimeUnit.SECONDS,1)); + factory = CreatedExpiryPolicy.factoryOf(new Duration(TimeUnit.SECONDS, 1)); nearCache = true; From e0c012d977b6db13dfdf2fb8347677998287c1e4 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Tue, 21 Mar 2017 17:50:06 +0300 Subject: [PATCH 464/487] IGNITE-4200: Added copying of the C++ binaries. --- assembly/release-fabric-base.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/assembly/release-fabric-base.xml b/assembly/release-fabric-base.xml index 7b3d8cf21e5f5..7f05c305ed4b2 100644 --- a/assembly/release-fabric-base.xml +++ b/assembly/release-fabric-base.xml @@ -174,6 +174,12 @@ /platforms/cpp/docs + + + modules/platforms/cpp/bin + /platforms/cpp/bin + + bin From b7ab27301b59bf93fc73b52fdf8e0bcf124fec1d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 6 Apr 2017 14:43:50 +0300 Subject: [PATCH 465/487] IGNITE-4832: Prevent service deployment on client by default when configuration is provided on startup. This closes #1748. --- .../service/GridServiceProcessor.java | 10 +- .../GridServiceProcessorAbstractSelfTest.java | 11 ++ ...rviceProcessorMultiNodeConfigSelfTest.java | 74 +++++++++- ...GridServiceProcessorMultiNodeSelfTest.java | 139 +++++++++++++++--- 4 files changed, 202 insertions(+), 32 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index bd815189fb5ba..a8af9832e9355 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -70,11 +70,11 @@ import org.apache.ignite.internal.processors.cache.query.CacheQuery; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; -import org.apache.ignite.internal.processors.continuous.AbstractContinuousMessage; import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.GridEmptyIterator; import org.apache.ignite.internal.util.GridSpinBusyLock; +import org.apache.ignite.internal.util.SerializableTransient; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -92,7 +92,6 @@ import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; -import org.apache.ignite.internal.util.SerializableTransient; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.JobContextResource; import org.apache.ignite.resources.LoggerResource; @@ -296,8 +295,13 @@ public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteChe if (cfgs != null) { Collection> futs = new ArrayList<>(); - for (ServiceConfiguration c : ctx.config().getServiceConfiguration()) + for (ServiceConfiguration c : cfgs) { + // Deploy only on server nodes by default. + if (c.getNodeFilter() == null) + c.setNodeFilter(ctx.cluster().get().forServers().predicate()); + futs.add(deploy(c)); + } // Await for services to deploy. for (IgniteInternalFuture f : futs) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java index 111cb714e8f32..0f79855518e3d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java @@ -129,6 +129,17 @@ protected void startExtraNodes(int cnt) throws Exception { startGrid(nodeCount() + i); } + /** */ + protected void startExtraNodes(int servers, int clients) throws Exception { + startExtraNodes(servers); + + for (int i = 0; i < clients; i++) { + final String nodeName = getTestGridName(nodeCount() + servers + i); + + startGrid(nodeName, getConfiguration(nodeName).setClientMode(true)); + } + } + /** * @throws Exception If failed. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java index b819cc93e6fb8..1bd3b035a446c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java @@ -19,7 +19,9 @@ import java.util.concurrent.CountDownLatch; import org.apache.ignite.Ignite; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.util.lang.GridAbsPredicateX; +import org.apache.ignite.services.Service; import org.apache.ignite.services.ServiceConfiguration; import org.apache.ignite.testframework.GridTestUtils; @@ -33,6 +35,9 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc /** Node singleton name. */ private static final String NODE_SINGLE = "serviceConfigEachNode"; + /** Node singleton name. */ + private static final String NODE_SINGLE_BUT_CLIENT = "serviceConfigEachNodeButClient"; + /** Affinity service name. */ private static final String AFFINITY = "serviceConfigAffinity"; @@ -46,7 +51,7 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc /** {@inheritDoc} */ @Override protected ServiceConfiguration[] services() { - ServiceConfiguration[] arr = new ServiceConfiguration[3]; + ServiceConfiguration[] arr = new ServiceConfiguration[4]; ServiceConfiguration cfg = new ServiceConfiguration(); @@ -59,7 +64,7 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc cfg = new ServiceConfiguration(); - cfg.setName(NODE_SINGLE); + cfg.setName(NODE_SINGLE_BUT_CLIENT); cfg.setMaxPerNodeCount(1); cfg.setService(new DummyService()); @@ -76,6 +81,15 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc arr[2] = cfg; + cfg = new ServiceConfiguration(); + + cfg.setName(NODE_SINGLE); + cfg.setMaxPerNodeCount(1); + cfg.setNodeFilter(new CacheConfiguration.IgniteAllNodesPredicate()); + cfg.setService(new DummyService()); + + arr[3] = cfg; + return arr; } @@ -91,6 +105,8 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc DummyService.cancelled(CLUSTER_SINGLE) == 0 && DummyService.started(NODE_SINGLE) == nodeCount() && DummyService.cancelled(NODE_SINGLE) == 0 && + DummyService.started(NODE_SINGLE_BUT_CLIENT) == nodeCount() && + DummyService.cancelled(NODE_SINGLE_BUT_CLIENT) == 0 && actualCount(AFFINITY, randomGrid().services().serviceDescriptors()) == 1; } }, @@ -112,6 +128,13 @@ public void testDeployOnEachNodeUpdateTopology() throws Exception { checkDeployOnEachNodeUpdateTopology(NODE_SINGLE); } + /** + * @throws Exception If failed. + */ + public void testDeployOnEachNodeButClientUpdateTopology() throws Exception { + checkDeployOnEachNodeButClientUpdateTopology(NODE_SINGLE_BUT_CLIENT); + } + /** * @throws Exception If failed. */ @@ -120,6 +143,10 @@ public void testAll() throws Exception { DummyService.reset(); + checkDeployOnEachNodeButClientUpdateTopology(NODE_SINGLE_BUT_CLIENT); + + DummyService.reset(); + checkDeployOnEachNodeUpdateTopology(NODE_SINGLE); DummyService.reset(); @@ -152,9 +179,7 @@ public void testAffinityUpdateTopology() throws Exception { private void checkSingletonUpdateTopology(String name) throws Exception { Ignite g = randomGrid(); - int nodeCnt = 2; - - startExtraNodes(nodeCnt); + startExtraNodes(2, 2); try { assertEquals(name, 0, DummyService.started(name)); @@ -165,7 +190,7 @@ private void checkSingletonUpdateTopology(String name) throws Exception { checkCount(name, g.services().serviceDescriptors(), 1); } finally { - stopExtraNodes(nodeCnt); + stopExtraNodes(4); } } @@ -176,17 +201,21 @@ private void checkSingletonUpdateTopology(String name) throws Exception { private void checkDeployOnEachNodeUpdateTopology(String name) throws Exception { Ignite g = randomGrid(); - int newNodes = 2; + int newNodes = 4; CountDownLatch latch = new CountDownLatch(newNodes); DummyService.exeLatch(name, latch); - startExtraNodes(newNodes); + startExtraNodes(2, 2); try { latch.await(); + // Ensure service is deployed. + assertNotNull(grid(nodeCount() + newNodes - 1).services() + .serviceProxy(NODE_SINGLE_BUT_CLIENT, Service.class, false, 2000)); + assertEquals(name, newNodes, DummyService.started(name)); assertEquals(name, 0, DummyService.cancelled(name)); @@ -196,4 +225,33 @@ private void checkDeployOnEachNodeUpdateTopology(String name) throws Exception { stopExtraNodes(newNodes); } } + + /** + * @param name Name. + * @throws Exception If failed. + */ + private void checkDeployOnEachNodeButClientUpdateTopology(String name) throws Exception { + Ignite g = randomGrid(); + + int servers = 2; + int clients = 2; + + CountDownLatch latch = new CountDownLatch(servers); + + DummyService.exeLatch(name, latch); + + startExtraNodes(servers, clients); + + try { + latch.await(); + + assertEquals(name, servers, DummyService.started(name)); + assertEquals(name, 0, DummyService.cancelled(name)); + + checkCount(name, g.services().serviceDescriptors(), nodeCount() + servers); + } + finally { + stopExtraNodes(servers + clients); + } + } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java index 39336ef001a02..f7403dcc820ca 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java @@ -21,7 +21,10 @@ import junit.framework.TestCase; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteServices; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.services.Service; +import org.apache.ignite.services.ServiceConfiguration; /** * Single node services test. @@ -121,50 +124,144 @@ public void testAffinityDeployUpdateTopology() throws Exception { /** * @throws Exception If failed. */ - public void testDeployOnEachNodeUpdateTopology() throws Exception { - String name = "serviceOnEachNodeUpdateTopology"; + public void testDeployOnEachNodeButClientUpdateTopology() throws Exception { + // Prestart client node. + Ignite client = startGrid("client", getConfiguration("client").setClientMode(true)); - Ignite g = randomGrid(); + try { + final int prestartedNodes = nodeCount() + 1; - CountDownLatch latch = new CountDownLatch(nodeCount()); + String name = "serviceOnEachNodeButClientUpdateTopology"; - DummyService.exeLatch(name, latch); + Ignite g = randomGrid(); - IgniteServices svcs = g.services().withAsync(); + CountDownLatch latch = new CountDownLatch(nodeCount()); - svcs.deployNodeSingleton(name, new DummyService()); + DummyService.exeLatch(name, latch); - IgniteFuture fut = svcs.future(); + IgniteServices svcs = g.services().withAsync(); - info("Deployed service: " + name); + svcs.deployNodeSingleton(name, new DummyService()); - fut.get(); + IgniteFuture fut = svcs.future(); - info("Finished waiting for service future: " + name); + info("Deployed service: " + name); - latch.await(); + fut.get(); - TestCase.assertEquals(name, nodeCount(), DummyService.started(name)); - TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + info("Finished waiting for service future: " + name); - int newNodes = 2; + latch.await(); - latch = new CountDownLatch(newNodes); + // Ensure service is deployed + assertNotNull(client.services().serviceProxy(name, Service.class, false, 2000)); - DummyService.exeLatch(name, latch); + TestCase.assertEquals(name, nodeCount(), DummyService.started(name)); + TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + + int servers = 2; + int clients = 2; + + latch = new CountDownLatch(servers); + + DummyService.exeLatch(name, latch); + + startExtraNodes(servers, clients); + + try { + latch.await(); + + // Ensure service is deployed + assertNotNull(grid(prestartedNodes + servers - 1) + .services().serviceProxy(name, Service.class, false, 2000)); + + TestCase.assertEquals(name, nodeCount() + servers, DummyService.started(name)); + TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + + checkCount(name, g.services().serviceDescriptors(), nodeCount() + servers); + } + finally { + stopExtraNodes(servers + clients); + } + } + finally { + stopGrid("client"); + } + } - startExtraNodes(newNodes); + /** + * @throws Exception If failed. + */ + public void testDeployOnEachNodeUpdateTopology() throws Exception { + // Prestart client node. + Ignite client = startGrid("client", getConfiguration("client").setClientMode(true)); try { + String name = "serviceOnEachNodeUpdateTopology"; + + Ignite g = randomGrid(); + + final int prestartedNodes = nodeCount() + 1; + + CountDownLatch latch = new CountDownLatch(prestartedNodes); + + DummyService.exeLatch(name, latch); + + ServiceConfiguration srvcCfg = new ServiceConfiguration(); + + srvcCfg.setNodeFilter(new CacheConfiguration.IgniteAllNodesPredicate()); + srvcCfg.setName(name); + srvcCfg.setMaxPerNodeCount(1); + srvcCfg.setService(new DummyService()); + + IgniteServices svcs = g.services().withAsync(); + + svcs.deploy(srvcCfg); + + IgniteFuture fut = svcs.future(); + + info("Deployed service: " + name); + + fut.get(); + + info("Finished waiting for service future: " + name); + latch.await(); - TestCase.assertEquals(name, nodeCount() + newNodes, DummyService.started(name)); + // Ensure service is deployed + assertNotNull(client.services().serviceProxy(name, Service.class, false, 2000)); + + TestCase.assertEquals(name, prestartedNodes, DummyService.started(name)); TestCase.assertEquals(name, 0, DummyService.cancelled(name)); - checkCount(name, g.services().serviceDescriptors(), nodeCount() + newNodes); + int servers = 2; + int clients = 2; + + int extraNodes = servers + clients; + + latch = new CountDownLatch(extraNodes); + + DummyService.exeLatch(name, latch); + + startExtraNodes(servers, clients); + + try { + latch.await(); + + // Ensure service is deployed + assertNotNull(client.services().serviceProxy(name, Service.class, false, 2000)); + + TestCase.assertEquals(name, prestartedNodes + extraNodes, DummyService.started(name)); + TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + + checkCount(name, g.services().serviceDescriptors(), prestartedNodes + extraNodes); + } + finally { + stopExtraNodes(extraNodes); + } } finally { - stopExtraNodes(newNodes); + stopGrid("client"); } } } \ No newline at end of file From 443ac9a7aa82af1359a03bcfc8f9212b108300e4 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 5 Apr 2017 15:01:02 +0300 Subject: [PATCH 466/487] IGNITE-4917: Fixed failure when accessing BinaryObjectBuilder field value serialized with OptimizedMarshaller . This closes #1736. --- .../internal/binary/builder/BinaryBuilderReader.java | 8 ++++++++ .../binary/mutabletest/GridBinaryTestClasses.java | 9 +++++++++ 2 files changed, 17 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java index baaabd6a1a0bd..02264e3311529 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderReader.java @@ -476,6 +476,14 @@ public Object getValueQuickly(int pos, int len) { return new BinaryPlainBinaryObject(binaryObj); } + case GridBinaryMarshaller.OPTM_MARSH: { + final BinaryHeapInputStream bin = BinaryHeapInputStream.create(arr, pos + 1); + + final Object obj = BinaryUtils.doReadOptimized(bin, ctx, U.resolveClassLoader(ctx.configuration())); + + return obj; + } + default: throw new BinaryObjectException("Invalid flag value: " + type); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/mutabletest/GridBinaryTestClasses.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/mutabletest/GridBinaryTestClasses.java index 5ddb87dcefb31..0d4de7fa0e215 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/binary/mutabletest/GridBinaryTestClasses.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/mutabletest/GridBinaryTestClasses.java @@ -24,6 +24,7 @@ import java.io.ObjectOutputStream; import java.io.Serializable; import java.math.BigDecimal; +import java.math.BigInteger; import java.sql.Timestamp; import java.util.ArrayList; import java.util.Date; @@ -141,6 +142,9 @@ public static class TestObjectAllTypes implements Serializable { /** */ public Integer i_; + /** */ + public BigInteger bi_; + /** */ public Long l_; @@ -150,6 +154,9 @@ public static class TestObjectAllTypes implements Serializable { /** */ public Double d_; + /** */ + public BigDecimal bd_; + /** */ public Character c_; @@ -267,9 +274,11 @@ public void setDefaultData() { b_ = 11; s_ = 22; i_ = 33; + bi_ = new BigInteger("33000000000000"); l_ = 44L; f_ = 55f; d_ = 66d; + bd_ = new BigDecimal("33000000000000.123456789"); c_ = 'e'; z_ = true; From 4a1415ad01ff9fde30d5c7c02e6d938f1515178d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 12 Apr 2017 13:01:25 +0300 Subject: [PATCH 467/487] IGNITE-4907: Fixed excessive service instances can be started with dynamic deployment. This closes #1766. (cherry picked from commit 0f7ef74) --- .../service/GridServiceProcessor.java | 2 +- ...rviceProcessorMultiNodeConfigSelfTest.java | 95 ++++++++++++++++--- ...GridServiceProcessorMultiNodeSelfTest.java | 61 ++++++++++++ 3 files changed, 146 insertions(+), 12 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index a8af9832e9355..2a363e21e8ef5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -972,7 +972,7 @@ private void reassign(GridServiceDeployment dep, AffinityTopologyVersion topVer) int perNodeCnt = totalCnt != 0 ? totalCnt / size : maxPerNodeCnt; int remainder = totalCnt != 0 ? totalCnt % size : 0; - if (perNodeCnt > maxPerNodeCnt && maxPerNodeCnt != 0) { + if (perNodeCnt >= maxPerNodeCnt && maxPerNodeCnt != 0) { perNodeCnt = maxPerNodeCnt; remainder = 0; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java index 1bd3b035a446c..9da62c0abb702 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java @@ -17,11 +17,13 @@ package org.apache.ignite.internal.processors.service; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.CountDownLatch; import org.apache.ignite.Ignite; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.util.lang.GridAbsPredicateX; -import org.apache.ignite.services.Service; import org.apache.ignite.services.ServiceConfiguration; import org.apache.ignite.testframework.GridTestUtils; @@ -38,6 +40,9 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc /** Node singleton name. */ private static final String NODE_SINGLE_BUT_CLIENT = "serviceConfigEachNodeButClient"; + /** Node singleton name. */ + private static final String NODE_SINGLE_WITH_LIMIT = "serviceConfigWithLimit"; + /** Affinity service name. */ private static final String AFFINITY = "serviceConfigAffinity"; @@ -51,7 +56,7 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc /** {@inheritDoc} */ @Override protected ServiceConfiguration[] services() { - ServiceConfiguration[] arr = new ServiceConfiguration[4]; + List cfgs = new ArrayList<>(); ServiceConfiguration cfg = new ServiceConfiguration(); @@ -60,7 +65,7 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc cfg.setTotalCount(1); cfg.setService(new DummyService()); - arr[0] = cfg; + cfgs.add(cfg); cfg = new ServiceConfiguration(); @@ -68,7 +73,7 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc cfg.setMaxPerNodeCount(1); cfg.setService(new DummyService()); - arr[1] = cfg; + cfgs.add(cfg); cfg = new ServiceConfiguration(); @@ -79,7 +84,7 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc cfg.setTotalCount(1); cfg.setService(new AffinityService(AFFINITY_KEY)); - arr[2] = cfg; + cfgs.add(cfg); cfg = new ServiceConfiguration(); @@ -88,9 +93,18 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc cfg.setNodeFilter(new CacheConfiguration.IgniteAllNodesPredicate()); cfg.setService(new DummyService()); - arr[3] = cfg; + cfgs.add(cfg); + + cfg = new ServiceConfiguration(); - return arr; + cfg.setName(NODE_SINGLE_WITH_LIMIT); + cfg.setMaxPerNodeCount(1); + cfg.setTotalCount(nodeCount() + 1); + cfg.setService(new DummyService()); + + cfgs.add(cfg); + + return cfgs.toArray(new ServiceConfiguration[cfgs.size()]); } /** {@inheritDoc} */ @@ -107,6 +121,8 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc DummyService.cancelled(NODE_SINGLE) == 0 && DummyService.started(NODE_SINGLE_BUT_CLIENT) == nodeCount() && DummyService.cancelled(NODE_SINGLE_BUT_CLIENT) == 0 && + DummyService.started(NODE_SINGLE_WITH_LIMIT) >= nodeCount() && + DummyService.cancelled(NODE_SINGLE_WITH_LIMIT) == 0 && actualCount(AFFINITY, randomGrid().services().serviceDescriptors()) == 1; } }, @@ -170,6 +186,59 @@ public void testAffinityUpdateTopology() throws Exception { finally { stopExtraNodes(nodeCnt); } + + checkCount(AFFINITY, g.services().serviceDescriptors(), 1); + } + + /** + * @throws Exception If failed. + */ + public void testDeployLimits() throws Exception { + final Ignite g = randomGrid(); + + final String name = NODE_SINGLE_WITH_LIMIT; + + waitForDeployment(name, nodeCount()); + + checkCount(name, g.services().serviceDescriptors(), nodeCount()); + + int extraNodes = 2; + + CountDownLatch latch = new CountDownLatch(1); + + DummyService.exeLatch(name, latch); + + startExtraNodes(extraNodes); + + try { + latch.await(); + + checkCount(name, g.services().serviceDescriptors(), nodeCount() + 1); + } + finally { + stopExtraNodes(extraNodes); + } + + assertEquals(name, 1, DummyService.cancelled(name)); + + waitForDeployment(name, nodeCount()); + + checkCount(name, g.services().serviceDescriptors(), nodeCount()); + } + + /** + * @param srvcName Service name + * @param expectedDeps Expected number of service deployments + * + */ + private boolean waitForDeployment(final String srvcName, final int expectedDeps) throws IgniteInterruptedCheckedException { + final Ignite g = randomGrid(); + + return GridTestUtils.waitForCondition(new GridAbsPredicateX() { + @Override public boolean applyx() { + return actualCount(srvcName, g.services().serviceDescriptors()) == expectedDeps; + } + }, 1500); } /** @@ -212,10 +281,6 @@ private void checkDeployOnEachNodeUpdateTopology(String name) throws Exception { try { latch.await(); - // Ensure service is deployed. - assertNotNull(grid(nodeCount() + newNodes - 1).services() - .serviceProxy(NODE_SINGLE_BUT_CLIENT, Service.class, false, 2000)); - assertEquals(name, newNodes, DummyService.started(name)); assertEquals(name, 0, DummyService.cancelled(name)); @@ -224,6 +289,10 @@ private void checkDeployOnEachNodeUpdateTopology(String name) throws Exception { finally { stopExtraNodes(newNodes); } + + waitForDeployment(name, nodeCount()); + + checkCount(name, g.services().serviceDescriptors(), nodeCount()); } /** @@ -253,5 +322,9 @@ private void checkDeployOnEachNodeButClientUpdateTopology(String name) throws Ex finally { stopExtraNodes(servers + clients); } + + waitForDeployment(name, nodeCount()); + + checkCount(name, g.services().serviceDescriptors(), nodeCount()); } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java index f7403dcc820ca..d133cf299a756 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java @@ -264,4 +264,65 @@ public void testDeployOnEachNodeUpdateTopology() throws Exception { stopGrid("client"); } } + + /** + * @throws Exception If failed. + */ + public void testDeployLimits() throws Exception { + String name = "serviceWithLimitsUpdateTopology"; + + Ignite g = randomGrid(); + + final int totalInstances = nodeCount() + 1; + + CountDownLatch latch = new CountDownLatch(nodeCount()); + + DummyService.exeLatch(name, latch); + + ServiceConfiguration srvcCfg = new ServiceConfiguration(); + + srvcCfg.setName(name); + srvcCfg.setMaxPerNodeCount(1); + srvcCfg.setTotalCount(totalInstances); + srvcCfg.setService(new DummyService()); + + IgniteServices svcs = g.services().withAsync(); + + svcs.deploy(srvcCfg); + + IgniteFuture fut = svcs.future(); + + info("Deployed service: " + name); + + fut.get(); + + info("Finished waiting for service future: " + name); + + latch.await(); + + TestCase.assertEquals(name, nodeCount(), DummyService.started(name)); + TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + + checkCount(name, g.services().serviceDescriptors(), nodeCount()); + + int extraNodes = 2; + + latch = new CountDownLatch(1); + + DummyService.exeLatch(name, latch); + + startExtraNodes(2); + + try { + latch.await(); + + TestCase.assertEquals(name, totalInstances, DummyService.started(name)); + TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + + checkCount(name, g.services().serviceDescriptors(), totalInstances); + } + finally { + stopExtraNodes(extraNodes); + } + } } \ No newline at end of file From bf1049741f7a64728bd433f78262ba273f969848 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 17 Apr 2017 19:00:30 +0300 Subject: [PATCH 468/487] IGNITE-4954 - Configurable expiration timeout for Cassandra session. This closes #1785. --- .../cassandra/datasource/DataSource.java | 26 ++++++++++++++++++- .../session/CassandraSessionImpl.java | 23 +++++++++------- .../cassandra/session/pool/SessionPool.java | 6 ++--- .../session/pool/SessionWrapper.java | 15 ++++++----- 4 files changed, 51 insertions(+), 19 deletions(-) diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java index 915eebde73f01..1fa2a1df9faa4 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java @@ -46,6 +46,9 @@ * Data source abstraction to specify configuration of the Cassandra session to be used. */ public class DataSource { + /** Default expiration timeout for Cassandra driver session. */ + public static final long DFLT_SESSION_EXPIRATION_TIMEOUT = 300000; // 5 minutes. + /** Number of rows to immediately fetch in CQL statement execution. */ private Integer fetchSize; @@ -123,6 +126,9 @@ public class DataSource { /** Netty options to use for connection. */ private NettyOptions nettyOptions; + /** Expiration timeout for Cassandra driver session. */ + private long sessionExpirationTimeout = DFLT_SESSION_EXPIRATION_TIMEOUT; + /** Cassandra session wrapper instance. */ private volatile CassandraSession ses; @@ -441,6 +447,23 @@ public void setNettyOptions(NettyOptions options) { invalidate(); } + /** + * Sets expiration timeout for Cassandra driver session. Idle sessions that are not + * used during this timeout value will be automatically closed and recreated later + * on demand. + *

          + * If set to {@code 0}, timeout is disabled. + *

          + * Default value is {@link #DFLT_SESSION_EXPIRATION_TIMEOUT}. + * + * @param sessionExpirationTimeout Expiration timeout for Cassandra driver session. + */ + public void setSessionExpirationTimeout(long sessionExpirationTimeout) { + this.sessionExpirationTimeout = sessionExpirationTimeout; + + invalidate(); + } + /** * Creates Cassandra session wrapper if it wasn't created yet and returns it * @@ -523,7 +546,8 @@ public synchronized CassandraSession session(IgniteLogger log) { if (nettyOptions != null) builder = builder.withNettyOptions(nettyOptions); - return ses = new CassandraSessionImpl(builder, fetchSize, readConsistency, writeConsistency, log); + return ses = new CassandraSessionImpl( + builder, fetchSize, readConsistency, writeConsistency, sessionExpirationTimeout, log); } /** diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java index 95b8581c5c793..cee776bb27363 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java @@ -17,6 +17,13 @@ package org.apache.ignite.cache.store.cassandra.session; +import java.io.IOException; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.Cache; import com.datastax.driver.core.BatchStatement; import com.datastax.driver.core.BoundStatement; import com.datastax.driver.core.Cluster; @@ -30,13 +37,6 @@ import com.datastax.driver.core.exceptions.AlreadyExistsException; import com.datastax.driver.core.exceptions.InvalidQueryException; import com.datastax.driver.core.querybuilder.Batch; -import java.io.IOException; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; -import javax.cache.Cache; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.store.cassandra.common.CassandraHelper; @@ -82,6 +82,9 @@ public class CassandraSessionImpl implements CassandraSession { /** Consistency level for Cassandra WRITE operations (insert/update/delete). */ private ConsistencyLevel writeConsistency; + /** Expiration timeout. */ + private long expirationTimeout; + /** Logger. */ private IgniteLogger log; @@ -101,11 +104,12 @@ public class CassandraSessionImpl implements CassandraSession { * @param log Logger. */ public CassandraSessionImpl(Cluster.Builder builder, Integer fetchSize, ConsistencyLevel readConsistency, - ConsistencyLevel writeConsistency, IgniteLogger log) { + ConsistencyLevel writeConsistency, long expirationTimeout, IgniteLogger log) { this.builder = builder; this.fetchSize = fetchSize; this.readConsistency = readConsistency; this.writeConsistency = writeConsistency; + this.expirationTimeout = expirationTimeout; this.log = log; } @@ -404,7 +408,8 @@ else if (CassandraHelper.isPreparedStatementClusterError(e)) /** {@inheritDoc} */ @Override public synchronized void close() throws IOException { if (decrementSessionRefs() == 0 && ses != null) { - SessionPool.put(this, ses); + SessionPool.put(this, ses, expirationTimeout); + ses = null; } } diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java index fc4a907e6e06e..86db713482160 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java @@ -17,13 +17,13 @@ package org.apache.ignite.cache.store.cassandra.session.pool; -import com.datastax.driver.core.Session; import java.lang.Thread.State; import java.util.Collection; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; +import com.datastax.driver.core.Session; import org.apache.ignite.cache.store.cassandra.session.CassandraSessionImpl; /** @@ -98,14 +98,14 @@ private static class SessionMonitor extends Thread { * @param cassandraSes Session wrapper. * @param driverSes Driver session. */ - public static void put(CassandraSessionImpl cassandraSes, Session driverSes) { + public static void put(CassandraSessionImpl cassandraSes, Session driverSes, long expirationTimeout) { if (cassandraSes == null || driverSes == null) return; SessionWrapper old; synchronized (sessions) { - old = sessions.put(cassandraSes, new SessionWrapper(driverSes)); + old = sessions.put(cassandraSes, new SessionWrapper(driverSes, expirationTimeout)); if (monitorSingleton == null || State.TERMINATED.equals(monitorSingleton.getState())) { monitorSingleton = new SessionMonitor(); diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java index 7c5722bb8f21f..68b9dd480f8b8 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java @@ -24,12 +24,12 @@ * Wrapper for Cassandra driver session, responsible for monitoring session expiration and its closing. */ public class SessionWrapper { - /** Expiration timeout for Cassandra driver session. */ - public static final long DFLT_EXPIRATION_TIMEOUT = 300000; // 5 minutes. - /** Cassandra driver session. */ private Session ses; + /** Expiration timeout. */ + private long expirationTimeout; + /** Wrapper creation time. */ private long time; @@ -38,9 +38,11 @@ public class SessionWrapper { * * @param ses Cassandra driver session. */ - public SessionWrapper(Session ses) { + public SessionWrapper(Session ses, long expirationTimeout) { this.ses = ses; - this.time = System.currentTimeMillis(); + this.expirationTimeout = expirationTimeout; + + time = System.currentTimeMillis(); } /** @@ -49,7 +51,7 @@ public SessionWrapper(Session ses) { * @return true if session expired. */ public boolean expired() { - return System.currentTimeMillis() - time > DFLT_EXPIRATION_TIMEOUT; + return expirationTimeout > 0 && System.currentTimeMillis() - time > expirationTimeout; } /** @@ -66,6 +68,7 @@ public Session driverSession() { */ public void release() { CassandraHelper.closeSession(ses); + ses = null; } } From f9ecacc625b458539775e6550bd9b7613ed38f21 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Fri, 28 Apr 2017 11:46:23 +0300 Subject: [PATCH 469/487] IGNITE-5077 - Support service security permissions backport from master (cherry picked from commit 6236b5f) --- .../processors/security/SecurityContext.java | 9 ++++++ .../service/GridServiceProcessor.java | 11 +++++++ .../security/SecurityBasicPermissionSet.java | 17 ++++++++++ .../plugin/security/SecurityPermission.java | 13 ++++++-- .../security/SecurityPermissionSet.java | 8 +++++ .../SecurityPermissionSetBuilder.java | 19 +++++++++++ .../SecurityPermissionSetBuilderTest.java | 32 +++++++++++++++---- .../junits/spi/GridSpiAbstractTest.java | 5 +++ 8 files changed, 106 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContext.java index ef46713a72f63..bf5894ef3f48c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContext.java @@ -47,6 +47,15 @@ public interface SecurityContext { */ public boolean cacheOperationAllowed(String cacheName, SecurityPermission perm); + /** + * Checks whether service operation is allowed. + * + * @param srvcName Service name. + * @param perm Permission to check. + * @return {@code True} if task operation is allowed. + */ + public boolean serviceOperationAllowed(String srvcName, SecurityPermission perm); + /** * Checks whether system-wide permission is allowed (excluding Visor task operations). * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index 2a363e21e8ef5..d7b9abc7fc1bd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -92,6 +92,7 @@ import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.plugin.security.SecurityPermission; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.JobContextResource; import org.apache.ignite.resources.LoggerResource; @@ -496,6 +497,8 @@ public IgniteInternalFuture deploy(ServiceConfiguration cfg) { validate(cfg); + ctx.security().authorize(cfg.getName(), SecurityPermission.SERVICE_DEPLOY, null); + if (!state.srvcCompatibility) { Marshaller marsh = ctx.config().getMarshaller(); @@ -632,6 +635,8 @@ private ServicesCompatibilityState markCompatibilityStateAsUsed() { * @return Future. */ public IgniteInternalFuture cancel(String name) { + ctx.security().authorize(name, SecurityPermission.SERVICE_CANCEL, null); + while (true) { try { GridFutureAdapter fut = new GridFutureAdapter<>(); @@ -780,6 +785,8 @@ public Collection serviceDescriptors() { */ @SuppressWarnings("unchecked") public T service(String name) { + ctx.security().authorize(name, SecurityPermission.SERVICE_INVOKE, null); + Collection ctxs; synchronized (locSvcs) { @@ -844,6 +851,8 @@ public ServiceContextImpl serviceContext(String name) { @SuppressWarnings("unchecked") public T serviceProxy(ClusterGroup prj, String name, Class svcItf, boolean sticky, long timeout) throws IgniteException { + ctx.security().authorize(name, SecurityPermission.SERVICE_INVOKE, null); + if (hasLocalNode(prj)) { ServiceContextImpl ctx = serviceContext(name); @@ -883,6 +892,8 @@ private boolean hasLocalNode(ClusterGroup prj) { */ @SuppressWarnings("unchecked") public Collection services(String name) { + ctx.security().authorize(name, SecurityPermission.SERVICE_INVOKE, null); + Collection ctxs; synchronized (locSvcs) { diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java index 5b50c56321377..7521dff84df4f 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java @@ -38,6 +38,9 @@ public class SecurityBasicPermissionSet implements SecurityPermissionSet { /** Task permissions. */ private Map> taskPerms = new HashMap<>(); + /** Service permissions. */ + private Map> srvcPerms = new HashMap<>(); + /** System permissions. */ private Collection sysPerms = new ArrayList<>(); @@ -62,6 +65,15 @@ public void setTaskPermissions(Map> taskP this.taskPerms = taskPerms; } + /** + * Setter for set service permission map. + * + * @param srvcPerms Service permissions. + */ + public void setServicePermissions(Map> srvcPerms) { + this.srvcPerms = srvcPerms; + } + /** * Setter for set collection system permission. * @@ -90,6 +102,11 @@ public void setDefaultAllowAll(boolean dfltAllowAll) { return taskPerms; } + /** {@inheritDoc} */ + @Override public Map> servicePermissions() { + return srvcPerms; + } + /** {@inheritDoc} */ @Nullable @Override public Collection systemPermissions() { return sysPerms; diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java index 9f63c1ed1d2a2..54361614e62a4 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java @@ -21,7 +21,7 @@ /** * Supported security permissions within grid. Permissions - * are specified on per-cache or per-task level. + * are specified on per-cache, per-task or per-service level. */ public enum SecurityPermission { /** Cache {@code read} permission. */ @@ -55,7 +55,16 @@ public enum SecurityPermission { ADMIN_CACHE, /** Visor admin operations permissions. */ - ADMIN_OPS; + ADMIN_OPS, + + /** Service deploy permission. */ + SERVICE_DEPLOY, + + /** Service cancel permission. */ + SERVICE_CANCEL, + + /** Service invoke permission. */ + SERVICE_INVOKE; /** Enumerated values. */ private static final SecurityPermission[] VALS = values(); diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSet.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSet.java index 99615019c10d2..5e07e421eaf78 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSet.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSet.java @@ -58,6 +58,14 @@ public interface SecurityPermissionSet extends Serializable, LessNamingBean { */ public Map> cachePermissions(); + /** + * Map of service names to service permissions. Wildcards are allowed at the + * end of service names. + * + * @return Map of service names to service permissions. + */ + public Map> servicePermissions(); + /** * Collection of system-wide permissions (events enable/disable, Visor task execution). * diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java index 61ad77ce08f5e..cf38c0f0e7e90 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java @@ -57,6 +57,9 @@ public class SecurityPermissionSetBuilder { /** Task permissions.*/ private Map> taskPerms = new HashMap<>(); + /** Service permissions.*/ + private Map> srvcPerms = new HashMap<>(); + /** System permissions.*/ private List sysPerms = new ArrayList<>(); @@ -99,6 +102,21 @@ public SecurityPermissionSetBuilder appendTaskPermissions(String name, SecurityP return this; } + /** + * Append permission set form {@link org.apache.ignite.IgniteServices service} with {@code name}. + * + * @param name String for map some service to permission set. + * @param perms Permissions. + * @return SecurityPermissionSetBuilder refer to same permission builder. + */ + public SecurityPermissionSetBuilder appendServicePermissions(String name, SecurityPermission... perms) { + validate(toCollection("SERVICE_"), perms); + + append(srvcPerms, name, toCollection(perms)); + + return this; + } + /** * Append permission set form {@link org.apache.ignite.IgniteCache cache} with {@code name}. * @@ -215,6 +233,7 @@ public SecurityPermissionSet build() { permSet.setDefaultAllowAll(dfltAllowAll); permSet.setCachePermissions(unmodifiableMap(cachePerms)); permSet.setTaskPermissions(unmodifiableMap(taskPerms)); + permSet.setServicePermissions(unmodifiableMap(srvcPerms)); permSet.setSystemPermissions(unmodifiableList(sysPerms)); return permSet; diff --git a/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java b/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java index f63f9a7da0a01..5443cfd77fd5c 100644 --- a/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java @@ -28,6 +28,8 @@ import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_PUT; import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_READ; import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_REMOVE; +import static org.apache.ignite.plugin.security.SecurityPermission.SERVICE_DEPLOY; +import static org.apache.ignite.plugin.security.SecurityPermission.SERVICE_INVOKE; import static org.apache.ignite.plugin.security.SecurityPermission.TASK_CANCEL; import static org.apache.ignite.plugin.security.SecurityPermission.TASK_EXECUTE; import static org.apache.ignite.plugin.security.SecurityPermission.EVENTS_ENABLE; @@ -41,6 +43,7 @@ public class SecurityPermissionSetBuilderTest extends GridCommonAbstractTest { /** * */ + @SuppressWarnings({"ThrowableNotThrown", "ArraysAsListWithZeroOrOneArgument"}) public void testPermissionBuilder() { SecurityBasicPermissionSet exp = new SecurityBasicPermissionSet(); @@ -56,13 +59,18 @@ public void testPermissionBuilder() { exp.setTaskPermissions(permTask); + Map> permSrvc = new HashMap<>(); + permSrvc.put("service1", Arrays.asList(SERVICE_DEPLOY)); + permSrvc.put("service2", Arrays.asList(SERVICE_INVOKE)); + + exp.setServicePermissions(permSrvc); + exp.setSystemPermissions(Arrays.asList(ADMIN_VIEW, EVENTS_ENABLE)); final SecurityPermissionSetBuilder permsBuilder = new SecurityPermissionSetBuilder(); assertThrows(log, new Callable() { - @Override - public Object call() throws Exception { + @Override public Object call() throws Exception { permsBuilder.appendCachePermissions("cache", ADMIN_VIEW); return null; } @@ -71,8 +79,7 @@ public Object call() throws Exception { ); assertThrows(log, new Callable() { - @Override - public Object call() throws Exception { + @Override public Object call() throws Exception { permsBuilder.appendTaskPermissions("task", CACHE_READ); return null; } @@ -81,8 +88,7 @@ public Object call() throws Exception { ); assertThrows(log, new Callable() { - @Override - public Object call() throws Exception { + @Override public Object call() throws Exception { permsBuilder.appendSystemPermissions(TASK_EXECUTE, CACHE_PUT); return null; } @@ -90,6 +96,15 @@ public Object call() throws Exception { "you can assign permission only start with [EVENTS_, ADMIN_], but you try TASK_EXECUTE" ); + assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + permsBuilder.appendSystemPermissions(SERVICE_INVOKE, CACHE_REMOVE); + return null; + } + }, IgniteException.class, + "you can assign permission only start with [EVENTS_, ADMIN_], but you try SERVICE_INVOKE" + ); + permsBuilder.appendCachePermissions( "cache1", CACHE_PUT, CACHE_REMOVE ).appendCachePermissions( @@ -98,12 +113,17 @@ public Object call() throws Exception { "task1", TASK_CANCEL ).appendTaskPermissions( "task2", TASK_EXECUTE + ).appendServicePermissions( + "service1", SERVICE_DEPLOY + ).appendServicePermissions( + "service2", SERVICE_INVOKE ).appendSystemPermissions(ADMIN_VIEW, EVENTS_ENABLE); SecurityPermissionSet actual = permsBuilder.build(); assertEquals(exp.cachePermissions(), actual.cachePermissions()); assertEquals(exp.taskPermissions(), actual.taskPermissions()); + assertEquals(exp.servicePermissions(), actual.servicePermissions()); assertEquals(exp.systemPermissions(), actual.systemPermissions()); assertEquals(exp.defaultAllowAll(), actual.defaultAllowAll()); } diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java index 20b3cf2fdb354..0aeff3c08b78a 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java @@ -717,6 +717,11 @@ private static class SecurityPermissionSetImpl implements SecurityPermissionSet return Collections.emptyMap(); } + /** {@inheritDoc} */ + @Override public Map> servicePermissions() { + return Collections.emptyMap(); + } + /** {@inheritDoc} */ @Nullable @Override public Collection systemPermissions() { return null; From 91c899b909383c78b78b9bf0c8f233b8c75ef29e Mon Sep 17 00:00:00 2001 From: Valentin Kulichenko Date: Fri, 28 Apr 2017 14:48:57 +0200 Subject: [PATCH 470/487] IGNITE-5081 - Removed redundant duplication of permissions in SecurityPermissionSetBuilder --- .../SecurityPermissionSetBuilder.java | 17 ++--- .../SecurityPermissionSetBuilderTest.java | 63 +++++++++++-------- 2 files changed, 46 insertions(+), 34 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java index cf38c0f0e7e90..abac541ffdb68 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilder.java @@ -17,16 +17,17 @@ package org.apache.ignite.plugin.security; -import java.util.Map; -import java.util.List; -import java.util.HashMap; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.util.typedef.internal.U; -import static java.util.Collections.unmodifiableList; import static java.util.Collections.unmodifiableMap; +import static java.util.Collections.unmodifiableSet; /** * Provides a convenient way to create a permission set. @@ -61,7 +62,7 @@ public class SecurityPermissionSetBuilder { private Map> srvcPerms = new HashMap<>(); /** System permissions.*/ - private List sysPerms = new ArrayList<>(); + private Set sysPerms = new HashSet<>(); /** Default allow all.*/ private boolean dfltAllowAll; @@ -193,7 +194,7 @@ private void validate(Collection ptrns, SecurityPermission perm) { private final Collection toCollection(T... perms) { assert perms != null; - Collection col = new ArrayList<>(perms.length); + Collection col = U.newHashSet(perms.length); Collections.addAll(col, perms); @@ -234,7 +235,7 @@ public SecurityPermissionSet build() { permSet.setCachePermissions(unmodifiableMap(cachePerms)); permSet.setTaskPermissions(unmodifiableMap(taskPerms)); permSet.setServicePermissions(unmodifiableMap(srvcPerms)); - permSet.setSystemPermissions(unmodifiableList(sysPerms)); + permSet.setSystemPermissions(unmodifiableSet(sysPerms)); return permSet; } diff --git a/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java b/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java index 5443cfd77fd5c..0ac7bc739f179 100644 --- a/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/plugin/security/SecurityPermissionSetBuilderTest.java @@ -17,23 +17,24 @@ package org.apache.ignite.plugin.security; -import java.util.Map; -import java.util.Arrays; -import java.util.HashMap; import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.Callable; import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import static org.apache.ignite.plugin.security.SecurityPermission.ADMIN_VIEW; import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_PUT; import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_READ; import static org.apache.ignite.plugin.security.SecurityPermission.CACHE_REMOVE; import static org.apache.ignite.plugin.security.SecurityPermission.SERVICE_DEPLOY; import static org.apache.ignite.plugin.security.SecurityPermission.SERVICE_INVOKE; +import static org.apache.ignite.plugin.security.SecurityPermission.EVENTS_ENABLE; import static org.apache.ignite.plugin.security.SecurityPermission.TASK_CANCEL; import static org.apache.ignite.plugin.security.SecurityPermission.TASK_EXECUTE; -import static org.apache.ignite.plugin.security.SecurityPermission.EVENTS_ENABLE; -import static org.apache.ignite.plugin.security.SecurityPermission.ADMIN_VIEW; import static org.apache.ignite.testframework.GridTestUtils.assertThrows; /** @@ -41,31 +42,30 @@ */ public class SecurityPermissionSetBuilderTest extends GridCommonAbstractTest { /** - * */ @SuppressWarnings({"ThrowableNotThrown", "ArraysAsListWithZeroOrOneArgument"}) public void testPermissionBuilder() { SecurityBasicPermissionSet exp = new SecurityBasicPermissionSet(); Map> permCache = new HashMap<>(); - permCache.put("cache1", Arrays.asList(CACHE_PUT, CACHE_REMOVE)); - permCache.put("cache2", Arrays.asList(CACHE_READ)); + permCache.put("cache1", permissions(CACHE_PUT, CACHE_REMOVE)); + permCache.put("cache2", permissions(CACHE_READ)); exp.setCachePermissions(permCache); Map> permTask = new HashMap<>(); - permTask.put("task1", Arrays.asList(TASK_CANCEL)); - permTask.put("task2", Arrays.asList(TASK_EXECUTE)); + permTask.put("task1", permissions(TASK_CANCEL)); + permTask.put("task2", permissions(TASK_EXECUTE)); exp.setTaskPermissions(permTask); Map> permSrvc = new HashMap<>(); - permSrvc.put("service1", Arrays.asList(SERVICE_DEPLOY)); - permSrvc.put("service2", Arrays.asList(SERVICE_INVOKE)); + permSrvc.put("service1", permissions(SERVICE_DEPLOY)); + permSrvc.put("service2", permissions(SERVICE_INVOKE)); exp.setServicePermissions(permSrvc); - exp.setSystemPermissions(Arrays.asList(ADMIN_VIEW, EVENTS_ENABLE)); + exp.setSystemPermissions(permissions(ADMIN_VIEW, EVENTS_ENABLE)); final SecurityPermissionSetBuilder permsBuilder = new SecurityPermissionSetBuilder(); @@ -105,19 +105,18 @@ public void testPermissionBuilder() { "you can assign permission only start with [EVENTS_, ADMIN_], but you try SERVICE_INVOKE" ); - permsBuilder.appendCachePermissions( - "cache1", CACHE_PUT, CACHE_REMOVE - ).appendCachePermissions( - "cache2", CACHE_READ - ).appendTaskPermissions( - "task1", TASK_CANCEL - ).appendTaskPermissions( - "task2", TASK_EXECUTE - ).appendServicePermissions( - "service1", SERVICE_DEPLOY - ).appendServicePermissions( - "service2", SERVICE_INVOKE - ).appendSystemPermissions(ADMIN_VIEW, EVENTS_ENABLE); + permsBuilder + .appendCachePermissions("cache1", CACHE_PUT) + .appendCachePermissions("cache1", CACHE_PUT, CACHE_REMOVE) + .appendCachePermissions("cache2", CACHE_READ) + .appendTaskPermissions("task1", TASK_CANCEL) + .appendTaskPermissions("task2", TASK_EXECUTE) + .appendTaskPermissions("task2", TASK_EXECUTE) + .appendServicePermissions("service1", SERVICE_DEPLOY) + .appendServicePermissions("service2", SERVICE_INVOKE) + .appendServicePermissions("service2", SERVICE_INVOKE) + .appendSystemPermissions(ADMIN_VIEW) + .appendSystemPermissions(ADMIN_VIEW, EVENTS_ENABLE); SecurityPermissionSet actual = permsBuilder.build(); @@ -127,4 +126,16 @@ public void testPermissionBuilder() { assertEquals(exp.systemPermissions(), actual.systemPermissions()); assertEquals(exp.defaultAllowAll(), actual.defaultAllowAll()); } + + /** + * @param perms Permissions. + * @return Collection. + */ + private static Collection permissions(SecurityPermission... perms) { + Collection col = U.newHashSet(perms.length); + + Collections.addAll(col, perms); + + return col; + } } From b48a26b9b1e97fb8eb52c2a2f36005770922ac3d Mon Sep 17 00:00:00 2001 From: Valentin Kulichenko Date: Fri, 28 Apr 2017 14:53:33 +0200 Subject: [PATCH 471/487] IGNITE-5080 - Fixes in SecurityBasicPermissionSet --- .../security/SecurityBasicPermissionSet.java | 89 ++++++++++++++----- 1 file changed, 65 insertions(+), 24 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java index 7521dff84df4f..44166d936c9aa 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java @@ -17,10 +17,12 @@ package org.apache.ignite.plugin.security; -import java.util.Map; -import java.util.HashMap; -import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; @@ -33,16 +35,20 @@ public class SecurityBasicPermissionSet implements SecurityPermissionSet { private static final long serialVersionUID = 0L; /** Cache permissions. */ - private Map> cachePerms = new HashMap<>(); + @GridToStringInclude + private Map> cachePermissions = new HashMap<>(); /** Task permissions. */ - private Map> taskPerms = new HashMap<>(); + @GridToStringInclude + private Map> taskPermissions = new HashMap<>(); /** Service permissions. */ - private Map> srvcPerms = new HashMap<>(); + @GridToStringInclude + private Map> servicePermissions = new HashMap<>(); /** System permissions. */ - private Collection sysPerms = new ArrayList<>(); + @GridToStringInclude + private Collection systemPermissions; /** Default allow all. */ private boolean dfltAllowAll; @@ -50,37 +56,43 @@ public class SecurityBasicPermissionSet implements SecurityPermissionSet { /** * Setter for set cache permission map. * - * @param cachePerms Cache permissions. + * @param cachePermissions Cache permissions. */ - public void setCachePermissions(Map> cachePerms) { - this.cachePerms = cachePerms; + public void setCachePermissions(Map> cachePermissions) { + A.notNull(cachePermissions, "cachePermissions"); + + this.cachePermissions = cachePermissions; } /** * Setter for set task permission map. * - * @param taskPerms Task permissions. + * @param taskPermissions Task permissions. */ - public void setTaskPermissions(Map> taskPerms) { - this.taskPerms = taskPerms; + public void setTaskPermissions(Map> taskPermissions) { + A.notNull(taskPermissions, "taskPermissions"); + + this.taskPermissions = taskPermissions; } /** * Setter for set service permission map. * - * @param srvcPerms Service permissions. + * @param servicePermissions Service permissions. */ - public void setServicePermissions(Map> srvcPerms) { - this.srvcPerms = srvcPerms; + public void setServicePermissions(Map> servicePermissions) { + A.notNull(taskPermissions, "servicePermissions"); + + this.servicePermissions = servicePermissions; } /** - * Setter for set collection system permission. + * Setter for set collection system permission. * - * @param sysPerms System permissions. + * @param systemPermissions System permissions. */ - public void setSystemPermissions(Collection sysPerms) { - this.sysPerms = sysPerms; + public void setSystemPermissions(Collection systemPermissions) { + this.systemPermissions = systemPermissions; } /** @@ -94,22 +106,22 @@ public void setDefaultAllowAll(boolean dfltAllowAll) { /** {@inheritDoc} */ @Override public Map> cachePermissions() { - return cachePerms; + return cachePermissions; } /** {@inheritDoc} */ @Override public Map> taskPermissions() { - return taskPerms; + return taskPermissions; } /** {@inheritDoc} */ @Override public Map> servicePermissions() { - return srvcPerms; + return servicePermissions; } /** {@inheritDoc} */ @Nullable @Override public Collection systemPermissions() { - return sysPerms; + return systemPermissions; } /** {@inheritDoc} */ @@ -117,6 +129,35 @@ public void setDefaultAllowAll(boolean dfltAllowAll) { return dfltAllowAll; } + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (!(o instanceof SecurityBasicPermissionSet)) + return false; + + SecurityBasicPermissionSet other = (SecurityBasicPermissionSet)o; + + return dfltAllowAll == other.dfltAllowAll && + F.eq(cachePermissions, other.cachePermissions) && + F.eq(taskPermissions, other.taskPermissions) && + F.eq(servicePermissions, other.servicePermissions) && + F.eq(systemPermissions, other.systemPermissions); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int res = (dfltAllowAll ? 1 : 0); + + res = 31 * res + (cachePermissions != null ? cachePermissions.hashCode() : 0); + res = 31 * res + (taskPermissions != null ? taskPermissions.hashCode() : 0); + res = 31 * res + (servicePermissions != null ? servicePermissions.hashCode() : 0); + res = 31 * res + (systemPermissions != null ? systemPermissions.hashCode() : 0); + + return res; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(SecurityBasicPermissionSet.class, this); From f66c23cbb9a6f2c923ebf75c58f00afaf1c0b5f3 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Wed, 3 May 2017 17:47:45 +0300 Subject: [PATCH 472/487] IGNITE-4939 Receive event before cache initialized fix --- .../processors/cache/GridCacheProcessor.java | 20 +++++++++++ .../cache/IgniteCacheProxyStub.java | 33 +++++++++++++++++++ .../CacheContinuousQueryHandler.java | 3 +- 3 files changed, 55 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyStub.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 0be2072e3766d..5a54e53dbd3f5 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -3345,6 +3345,26 @@ public IgniteCacheProxy jcache(@Nullable String name) { return cache; } + /** + * @param name Cache name. + * @param id Cache id. + * @return Cache instance for given name or stub if cache not created yet, but cache descriptor exists. + */ + @SuppressWarnings("unchecked") + public IgniteCacheProxy safeJcache(String name, int id) { + assert name != null; + + IgniteCacheProxy cache = (IgniteCacheProxy)jCacheProxies.get(name); + + if (cache == null) + if (cacheDescriptor(id) != null && CU.isSystemCache(name)) + cache = new IgniteCacheProxyStub(); + else + throw new IllegalArgumentException("Cache is not configured: " + name); + + return cache; + } + /** * @return All configured public cache instances. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyStub.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyStub.java new file mode 100644 index 0000000000000..75489edb98e74 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyStub.java @@ -0,0 +1,33 @@ +/* + * 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; + +/** + * Cache proxy stub. + */ +public class IgniteCacheProxyStub extends IgniteCacheProxy{ + /** */ + private static final long serialVersionUID = 0L; + + /** + * Empty constructor. + */ + public IgniteCacheProxyStub() { + // No-op. + } +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 83edab444f8c0..1a6577d241774 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -716,7 +716,8 @@ private void notifyCallback0(UUID nodeId, GridCacheContext cctx = cacheContext(ctx); - final IgniteCache cache = cctx.kernalContext().cache().jcache(cctx.name()); + //returns stub if system cache not created yet + final IgniteCache cache = cctx.kernalContext().cache().safeJcache(cctx.name(), cctx.cacheId()); if (internal) { if (e.isFiltered()) From 45b4d6316145d0b4b46713409f5e8fbe55ff4c41 Mon Sep 17 00:00:00 2001 From: Evgenii Zhuravlev Date: Thu, 4 May 2017 12:11:37 +0300 Subject: [PATCH 473/487] IGNITE-4939 Receive event before cache initialized fix --- .../ignite/internal/processors/cache/GridCacheProcessor.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 5a54e53dbd3f5..ccd7ae0bf500d 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -3352,9 +3352,7 @@ public IgniteCacheProxy jcache(@Nullable String name) { */ @SuppressWarnings("unchecked") public IgniteCacheProxy safeJcache(String name, int id) { - assert name != null; - - IgniteCacheProxy cache = (IgniteCacheProxy)jCacheProxies.get(name); + IgniteCacheProxy cache = (IgniteCacheProxy)jCacheProxies.get(maskNull(name)); if (cache == null) if (cacheDescriptor(id) != null && CU.isSystemCache(name)) From 075bcfca0ea22633be13cd02647e359ad6fdca16 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 4 May 2017 12:21:04 +0300 Subject: [PATCH 474/487] Fix flacky service deployment tests. --- .../GridServiceProcessorAbstractSelfTest.java | 17 ++++++ ...rviceProcessorMultiNodeConfigSelfTest.java | 46 +++++++-------- ...GridServiceProcessorMultiNodeSelfTest.java | 56 ++++++++++--------- 3 files changed, 68 insertions(+), 51 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java index 0f79855518e3d..6d91f36aab00b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java @@ -30,7 +30,9 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor; +import org.apache.ignite.internal.util.lang.GridAbsPredicateX; import org.apache.ignite.internal.util.typedef.CA; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.lang.IgniteFuture; @@ -519,6 +521,21 @@ protected int actualCount(String svcName, Iterable descs) { return sum; } + /** + * @param srvcName Service name + * @param expectedDeps Expected number of service deployments + * + */ + protected boolean waitForDeployment(final String srvcName, final int expectedDeps) throws IgniteInterruptedCheckedException { + final Ignite g = randomGrid(); + + return GridTestUtils.waitForCondition(new GridAbsPredicateX() { + @Override public boolean applyx() { + return actualCount(srvcName, g.services().serviceDescriptors()) == expectedDeps; + } + }, 1500); + } + /** * Counter service. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java index 9da62c0abb702..c78bcaae607ac 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeConfigSelfTest.java @@ -22,7 +22,6 @@ import java.util.concurrent.CountDownLatch; import org.apache.ignite.Ignite; import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.util.lang.GridAbsPredicateX; import org.apache.ignite.services.ServiceConfiguration; import org.apache.ignite.testframework.GridTestUtils; @@ -56,7 +55,6 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc /** {@inheritDoc} */ @Override protected ServiceConfiguration[] services() { - List cfgs = new ArrayList<>(); ServiceConfiguration cfg = new ServiceConfiguration(); @@ -65,6 +63,8 @@ public class GridServiceProcessorMultiNodeConfigSelfTest extends GridServiceProc cfg.setTotalCount(1); cfg.setService(new DummyService()); + List cfgs = new ArrayList<>(); + cfgs.add(cfg); cfg = new ServiceConfiguration(); @@ -202,43 +202,31 @@ public void testDeployLimits() throws Exception { checkCount(name, g.services().serviceDescriptors(), nodeCount()); - int extraNodes = 2; - CountDownLatch latch = new CountDownLatch(1); DummyService.exeLatch(name, latch); + int extraNodes = 2; + startExtraNodes(extraNodes); try { latch.await(); + waitForDeployment(name, nodeCount() + 1); + checkCount(name, g.services().serviceDescriptors(), nodeCount() + 1); } finally { stopExtraNodes(extraNodes); } - assertEquals(name, 1, DummyService.cancelled(name)); - waitForDeployment(name, nodeCount()); - checkCount(name, g.services().serviceDescriptors(), nodeCount()); - } - - /** - * @param srvcName Service name - * @param expectedDeps Expected number of service deployments - * - */ - private boolean waitForDeployment(final String srvcName, final int expectedDeps) throws IgniteInterruptedCheckedException { - final Ignite g = randomGrid(); + // Service can be redeployed when nodes is stopping one-by-one. + assertEquals(0, DummyService.started(name) - DummyService.cancelled(name)); - return GridTestUtils.waitForCondition(new GridAbsPredicateX() { - @Override public boolean applyx() { - return actualCount(srvcName, g.services().serviceDescriptors()) == expectedDeps; - } - }, 1500); + checkCount(name, g.services().serviceDescriptors(), nodeCount()); } /** @@ -281,8 +269,12 @@ private void checkDeployOnEachNodeUpdateTopology(String name) throws Exception { try { latch.await(); - assertEquals(name, newNodes, DummyService.started(name)); - assertEquals(name, 0, DummyService.cancelled(name)); + waitForDeployment(name, nodeCount() + newNodes); + + // Since we start extra nodes, there may be extra start and cancel events, + // so we check only the difference between start and cancel and + // not start and cancel events individually. + assertEquals(name, newNodes, DummyService.started(name) - DummyService.cancelled(name)); checkCount(name, g.services().serviceDescriptors(), nodeCount() + newNodes); } @@ -314,8 +306,12 @@ private void checkDeployOnEachNodeButClientUpdateTopology(String name) throws Ex try { latch.await(); - assertEquals(name, servers, DummyService.started(name)); - assertEquals(name, 0, DummyService.cancelled(name)); + waitForDeployment(name, nodeCount() + servers); + + // Since we start extra nodes, there may be extra start and cancel events, + // so we check only the difference between start and cancel and + // not start and cancel events individually. + assertEquals(name, servers, DummyService.started(name) - DummyService.cancelled(name)); checkCount(name, g.services().serviceDescriptors(), nodeCount() + servers); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java index d133cf299a756..8ecceb9a65d39 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java @@ -92,7 +92,7 @@ public void testAffinityDeployUpdateTopology() throws Exception { // Store a cache key. g.cache(CACHE_NAME).put(affKey, affKey.toString()); - String name = "serviceAffinityUpdateTopology"; + final String name = "serviceAffinityUpdateTopology"; IgniteServices svcs = g.services().withAsync(); @@ -129,9 +129,7 @@ public void testDeployOnEachNodeButClientUpdateTopology() throws Exception { Ignite client = startGrid("client", getConfiguration("client").setClientMode(true)); try { - final int prestartedNodes = nodeCount() + 1; - - String name = "serviceOnEachNodeButClientUpdateTopology"; + final String name = "serviceOnEachNodeButClientUpdateTopology"; Ignite g = randomGrid(); @@ -156,8 +154,8 @@ public void testDeployOnEachNodeButClientUpdateTopology() throws Exception { // Ensure service is deployed assertNotNull(client.services().serviceProxy(name, Service.class, false, 2000)); - TestCase.assertEquals(name, nodeCount(), DummyService.started(name)); - TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + assertEquals(name, nodeCount(), DummyService.started(name)); + assertEquals(name, 0, DummyService.cancelled(name)); int servers = 2; int clients = 2; @@ -171,12 +169,12 @@ public void testDeployOnEachNodeButClientUpdateTopology() throws Exception { try { latch.await(); - // Ensure service is deployed - assertNotNull(grid(prestartedNodes + servers - 1) - .services().serviceProxy(name, Service.class, false, 2000)); + waitForDeployment(name, servers); - TestCase.assertEquals(name, nodeCount() + servers, DummyService.started(name)); - TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + // Since we start extra nodes, there may be extra start and cancel events, + // so we check only the difference between start and cancel and + // not start and cancel events individually. + assertEquals(name, nodeCount() + servers, DummyService.started(name) - DummyService.cancelled(name)); checkCount(name, g.services().serviceDescriptors(), nodeCount() + servers); } @@ -197,7 +195,7 @@ public void testDeployOnEachNodeUpdateTopology() throws Exception { Ignite client = startGrid("client", getConfiguration("client").setClientMode(true)); try { - String name = "serviceOnEachNodeUpdateTopology"; + final String name = "serviceOnEachNodeUpdateTopology"; Ignite g = randomGrid(); @@ -231,8 +229,8 @@ public void testDeployOnEachNodeUpdateTopology() throws Exception { // Ensure service is deployed assertNotNull(client.services().serviceProxy(name, Service.class, false, 2000)); - TestCase.assertEquals(name, prestartedNodes, DummyService.started(name)); - TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + assertEquals(name, prestartedNodes, DummyService.started(name)); + assertEquals(name, 0, DummyService.cancelled(name)); int servers = 2; int clients = 2; @@ -248,11 +246,13 @@ public void testDeployOnEachNodeUpdateTopology() throws Exception { try { latch.await(); - // Ensure service is deployed - assertNotNull(client.services().serviceProxy(name, Service.class, false, 2000)); + waitForDeployment(name, prestartedNodes + extraNodes); - TestCase.assertEquals(name, prestartedNodes + extraNodes, DummyService.started(name)); - TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + // Since we start extra nodes, there may be extra start and cancel events, + // so we check only the difference between start and cancel and + // not start and cancel events individually. + assertEquals(name, prestartedNodes + extraNodes, + DummyService.started(name) - DummyService.cancelled(name)); checkCount(name, g.services().serviceDescriptors(), prestartedNodes + extraNodes); } @@ -269,7 +269,7 @@ public void testDeployOnEachNodeUpdateTopology() throws Exception { * @throws Exception If failed. */ public void testDeployLimits() throws Exception { - String name = "serviceWithLimitsUpdateTopology"; + final String name = "serviceWithLimitsUpdateTopology"; Ignite g = randomGrid(); @@ -300,24 +300,28 @@ public void testDeployLimits() throws Exception { latch.await(); - TestCase.assertEquals(name, nodeCount(), DummyService.started(name)); - TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + assertEquals(name, nodeCount(), DummyService.started(name)); + assertEquals(name, 0, DummyService.cancelled(name)); checkCount(name, g.services().serviceDescriptors(), nodeCount()); - int extraNodes = 2; - latch = new CountDownLatch(1); DummyService.exeLatch(name, latch); - startExtraNodes(2); + int extraNodes = 2; + + startExtraNodes(extraNodes); try { latch.await(); - TestCase.assertEquals(name, totalInstances, DummyService.started(name)); - TestCase.assertEquals(name, 0, DummyService.cancelled(name)); + waitForDeployment(name, totalInstances); + + // Since we start extra nodes, there may be extra start and cancel events, + // so we check only the difference between start and cancel and + // not start and cancel events individually. + assertEquals(name, totalInstances, DummyService.started(name) - DummyService.cancelled(name)); checkCount(name, g.services().serviceDescriptors(), totalInstances); } From 25c06b50d46937cb39534cdf4147b862217289a2 Mon Sep 17 00:00:00 2001 From: rfqu Date: Tue, 2 May 2017 19:46:44 +0300 Subject: [PATCH 475/487] ignite-4220 Support statements for JDBC and Cassandra store --- .../store/cassandra/CassandraCacheStore.java | 16 ++++- .../session/LoadCacheCustomQueryWorker.java | 26 +++++-- .../tests/IgnitePersistentStoreTest.java | 23 ++++-- .../store/jdbc/CacheAbstractJdbcStore.java | 72 +++++++++++++------ .../CacheJdbcPojoStoreAbstractSelfTest.java | 49 +++++++++++++ 5 files changed, 152 insertions(+), 34 deletions(-) diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java index e8da3a7c8e511..2e1d3ea70318d 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java @@ -20,6 +20,7 @@ import com.datastax.driver.core.BoundStatement; import com.datastax.driver.core.PreparedStatement; import com.datastax.driver.core.Row; +import com.datastax.driver.core.Statement; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -103,10 +104,19 @@ public CassandraCacheStore(DataSource dataSrc, KeyValuePersistenceSettings setti CassandraSession ses = getCassandraSession(); for (Object obj : args) { - if (obj == null || !(obj instanceof String) || !((String)obj).trim().toLowerCase().startsWith("select")) - continue; + LoadCacheCustomQueryWorker task = null; - futs.add(pool.submit(new LoadCacheCustomQueryWorker<>(ses, (String) obj, controller, log, clo))); + if (obj instanceof Statement) + task = new LoadCacheCustomQueryWorker<>(ses, (Statement)obj, controller, log, clo); + else if (obj instanceof String) { + String qry = ((String)obj).trim(); + + if (qry.toLowerCase().startsWith("select")) + task = new LoadCacheCustomQueryWorker<>(ses, (String) obj, controller, log, clo); + } + + if (task != null) + futs.add(pool.submit(task)); } for (Future fut : futs) diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java index d3ace7d436e81..d186b98cf72fa 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java @@ -36,8 +36,8 @@ public class LoadCacheCustomQueryWorker implements Callable { /** Cassandra session to execute CQL query */ private final CassandraSession ses; - /** User query. */ - private final String qry; + /** Statement. */ + private final Statement stmt; /** Persistence controller */ private final PersistenceController ctrl; @@ -49,12 +49,28 @@ public class LoadCacheCustomQueryWorker implements Callable { private final IgniteBiInClosure clo; /** + * @param ses Session. + * @param qry Query. + * @param ctrl Control. + * @param log Logger. * @param clo Closure for loaded values. */ public LoadCacheCustomQueryWorker(CassandraSession ses, String qry, PersistenceController ctrl, - IgniteLogger log, IgniteBiInClosure clo) { + IgniteLogger log, IgniteBiInClosure clo) { + this(ses, new SimpleStatement(qry.trim().endsWith(";") ? qry : qry + ';'), ctrl, log, clo); + } + + /** + * @param ses Session. + * @param stmt Statement. + * @param ctrl Control. + * @param log Logger. + * @param clo Closure for loaded values. + */ + public LoadCacheCustomQueryWorker(CassandraSession ses, Statement stmt, PersistenceController ctrl, + IgniteLogger log, IgniteBiInClosure clo) { this.ses = ses; - this.qry = qry.trim().endsWith(";") ? qry : qry + ";"; + this.stmt = stmt; this.ctrl = ctrl; this.log = log; this.clo = clo; @@ -70,7 +86,7 @@ public LoadCacheCustomQueryWorker(CassandraSession ses, String qry, PersistenceC /** {@inheritDoc} */ @Override public Statement getStatement() { - return new SimpleStatement(qry); + return stmt; } /** {@inheritDoc} */ diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java index 5da6ba25dabea..51d08855388c6 100644 --- a/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java @@ -17,6 +17,7 @@ package org.apache.ignite.tests; +import com.datastax.driver.core.SimpleStatement; import java.util.Collection; import java.util.Map; import org.apache.ignite.Ignite; @@ -35,6 +36,7 @@ import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.Assert; import org.springframework.core.io.ClassPathResource; /** @@ -346,20 +348,29 @@ public void loadCacheTest() { LOGGER.info("Running loadCache test"); try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/pojo/ignite-config.xml")) { - IgniteCache personCache3 = ignite.getOrCreateCache(new CacheConfiguration("cache3")); + CacheConfiguration ccfg = new CacheConfiguration<>("cache3"); + + IgniteCache personCache3 = ignite.getOrCreateCache(ccfg); + int size = personCache3.size(CachePeekMode.ALL); LOGGER.info("Initial cache size " + size); LOGGER.info("Loading cache data from Cassandra table"); - personCache3.loadCache(null, new String[] {"select * from test1.pojo_test3 limit 3"}); + String qry = "select * from test1.pojo_test3 limit 3"; + + personCache3.loadCache(null, qry); size = personCache3.size(CachePeekMode.ALL); - if (size != 3) { - throw new RuntimeException("Cache data was incorrectly loaded from Cassandra. " + - "Expected number of records is 3, but loaded number of records is " + size); - } + Assert.assertEquals("Cache data was incorrectly loaded from Cassandra table by '" + qry + "'", 3, size); + + personCache3.clear(); + + personCache3.loadCache(null, new SimpleStatement(qry)); + + size = personCache3.size(CachePeekMode.ALL); + Assert.assertEquals("Cache data was incorrectly loaded from Cassandra table by statement", 3, size); LOGGER.info("Cache data loaded from Cassandra table"); } diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java index e7ce52651a868..e211fadf2fb74 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java @@ -81,7 +81,6 @@ import static org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory.DFLT_PARALLEL_LOAD_CACHE_MINIMUM_THRESHOLD; import static org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory.DFLT_WRITE_ATTEMPTS; import static org.apache.ignite.cache.store.jdbc.JdbcTypesTransformer.NUMERIC_TYPES; -import static org.apache.ignite.cache.store.jdbc.JdbcTypesTransformer.NUMERIC_TYPES; /** * Implementation of {@link CacheStore} backed by JDBC. @@ -753,17 +752,34 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { } })) throw new CacheLoaderException("Provided key type is not found in store or cache configuration " + - "[cache=" + U.maskName(cacheName) + ", key=" + keyType + "]"); - - String qry = args[i + 1].toString(); + "[cache=" + U.maskName(cacheName) + ", key=" + keyType + ']'); EntryMapping em = entryMapping(cacheName, typeIdForTypeName(kindForName(keyType), keyType)); - if (log.isInfoEnabled()) - log.info("Started load cache using custom query [cache=" + U.maskName(cacheName) + - ", keyType=" + keyType + ", query=" + qry + "]"); + Object arg = args[i + 1]; + + LoadCacheCustomQueryWorker task; + + if (arg instanceof PreparedStatement) { + PreparedStatement stmt = (PreparedStatement)arg; + + if (log.isInfoEnabled()) + log.info("Started load cache using custom statement [cache=" + U.maskName(cacheName) + + ", keyType=" + keyType + ", stmt=" + stmt + ']'); + + task = new LoadCacheCustomQueryWorker<>(em, stmt, clo); + } + else { + String qry = arg.toString(); + + if (log.isInfoEnabled()) + log.info("Started load cache using custom query [cache=" + U.maskName(cacheName) + + ", keyType=" + keyType + ", query=" + qry + ']'); + + task = new LoadCacheCustomQueryWorker<>(em, qry, clo); + } - futs.add(pool.submit(new LoadCacheCustomQueryWorker<>(em, qry, clo))); + futs.add(pool.submit(task)); } } else { @@ -778,7 +794,7 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { processedKeyTypes.add(keyType); if (log.isInfoEnabled()) - log.info("Started load cache [cache=" + U.maskName(cacheName) + ", keyType=" + keyType + "]"); + log.info("Started load cache [cache=" + U.maskName(cacheName) + ", keyType=" + keyType + ']'); if (parallelLoadCacheMinThreshold > 0) { Connection conn = null; @@ -795,7 +811,7 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { if (rs.next()) { if (log.isDebugEnabled()) log.debug("Multithread loading entries from db [cache=" + U.maskName(cacheName) + - ", keyType=" + keyType + "]"); + ", keyType=" + keyType + ']'); int keyCnt = em.keyCols.size(); @@ -824,7 +840,7 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { } catch (SQLException e) { log.warning("Failed to load entries from db in multithreaded mode, will try in single thread " + - "[cache=" + U.maskName(cacheName) + ", keyType=" + keyType + " ]", e); + "[cache=" + U.maskName(cacheName) + ", keyType=" + keyType + ']', e); } finally { U.closeQuiet(conn); @@ -833,7 +849,7 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { if (log.isDebugEnabled()) log.debug("Single thread loading entries from db [cache=" + U.maskName(cacheName) + - ", keyType=" + keyType + "]"); + ", keyType=" + keyType + ']'); futs.add(pool.submit(loadCacheFull(em, clo))); } @@ -860,7 +876,7 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { EntryMapping em = entryMapping(session().cacheName(), typeIdForObject(key)); if (log.isDebugEnabled()) - log.debug("Load value from db [table= " + em.fullTableName() + ", key=" + key + "]"); + log.debug("Load value from db [table= " + em.fullTableName() + ", key=" + key + ']'); Connection conn = null; @@ -1954,12 +1970,26 @@ private class LoadCacheCustomQueryWorker implements Callable { /** Entry mapping description. */ private final EntryMapping em; + /** User statement. */ + private PreparedStatement stmt; + /** User query. */ - private final String qry; + private String qry; /** Closure for loaded values. */ private final IgniteBiInClosure clo; + /** + * @param em Entry mapping description. + * @param stmt User statement. + * @param clo Closure for loaded values. + */ + private LoadCacheCustomQueryWorker(EntryMapping em, PreparedStatement stmt, IgniteBiInClosure clo) { + this.em = em; + this.stmt = stmt; + this.clo = clo; + } + /** * @param em Entry mapping description. * @param qry User query. @@ -1975,12 +2005,12 @@ private LoadCacheCustomQueryWorker(EntryMapping em, String qry, IgniteBiInClosur @Override public Void call() throws Exception { Connection conn = null; - PreparedStatement stmt = null; - try { - conn = openConnection(true); + if (stmt == null) { + conn = openConnection(true); - stmt = conn.prepareStatement(qry); + stmt = conn.prepareStatement(qry); + } stmt.setFetchSize(dialect.getFetchSize()); @@ -2006,9 +2036,11 @@ private LoadCacheCustomQueryWorker(EntryMapping em, String qry, IgniteBiInClosur throw new CacheLoaderException("Failed to execute custom query for load cache", e); } finally { - U.closeQuiet(stmt); + if (conn != null) { + U.closeQuiet(stmt); - U.closeQuiet(conn); + U.closeQuiet(conn); + } } } } diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java index 1de44f7243fbb..9e59769503f53 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java @@ -328,6 +328,55 @@ protected void checkCacheLoadWithSql() { assertEquals(PERSON_CNT, c1.size()); } + /** + * Checks that data was loaded correctly with prepared statement. + */ + protected void checkCacheLoadWithStatement() throws SQLException { + Connection conn = null; + + PreparedStatement stmt = null; + + try { + conn = getConnection(); + + conn.setAutoCommit(true); + + String qry = "select id, org_id, name, birthday, gender from Person"; + + stmt = conn.prepareStatement(qry); + + IgniteCache c1 = grid().cache(CACHE_NAME); + + c1.loadCache(null, "org.apache.ignite.cache.store.jdbc.model.PersonKey", stmt); + + assertEquals(PERSON_CNT, c1.size()); + } + finally { + U.closeQuiet(stmt); + + U.closeQuiet(conn); + } + + } + + /** + * @throws Exception If failed. + */ + public void testLoadCacheWithStatement() throws Exception { + startTestGrid(false, false, false, false, 512); + + checkCacheLoadWithStatement(); + } + + /** + * @throws Exception If failed. + */ + public void testLoadCacheWithStatementTx() throws Exception { + startTestGrid(false, false, false, true, 512); + + checkCacheLoadWithStatement(); + } + /** * @throws Exception If failed. */ From 987c182686962673e70398395cb27e94f894713b Mon Sep 17 00:00:00 2001 From: nikolay_tikhonov Date: Mon, 15 May 2017 11:54:16 +0300 Subject: [PATCH 476/487] Fixed "IGNITE-5214 ConcurrentModificationException with enable DEBUG log level" Signed-off-by: nikolay_tikhonov --- .../continuous/CacheContinuousQueryHandler.java | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 1a6577d241774..5f00d580f4231 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -998,6 +998,7 @@ private static class PartitionRecovery { synchronized (pendingEvts) { if (log.isDebugEnabled()) { + log.debug("Handling event [lastFiredEvt=" + lastFiredEvt + ", curTop=" + curTop + ", entUpdCnt=" + entry.updateCounter() + @@ -1126,15 +1127,15 @@ private static class PartitionRecovery { break; } } - } - if (log.isDebugEnabled()) { - log.debug("Will send to listener the following events [entries=" + entries + - ", lastFiredEvt=" + lastFiredEvt + - ", curTop=" + curTop + - ", entUpdCnt=" + entry.updateCounter() + - ", partId=" + entry.partition() + - ", pendingEvts=" + pendingEvts + ']'); + if (log.isDebugEnabled()) { + log.debug("Will send to listener the following events [entries=" + entries + + ", lastFiredEvt=" + lastFiredEvt + + ", curTop=" + curTop + + ", entUpdCnt=" + entry.updateCounter() + + ", partId=" + entry.partition() + + ", pendingEvts=" + pendingEvts + ']'); + } } return entries; From ebc4a1648a80fbbd485e4c351fce9bee163318f9 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 16 May 2017 11:30:29 +0300 Subject: [PATCH 477/487] DirectByteBufferStreamImpl: converted asserts into exceptions. (cherry picked from commit 560ef60) --- .../v2/DirectByteBufferStreamImplV2.java | 27 ++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java index d7dc990848094..af45263e36a8b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/direct/stream/v2/DirectByteBufferStreamImplV2.java @@ -1,4 +1,4 @@ -/* + /* * 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. @@ -27,6 +27,7 @@ import java.util.Map; import java.util.RandomAccess; import java.util.UUID; +import org.apache.ignite.IgniteException; import org.apache.ignite.internal.direct.stream.DirectByteBufferStream; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -80,7 +81,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator BYTE_ARR_CREATOR = new ArrayCreator() { @Override public byte[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid byte array length: " + len); switch (len) { case 0: @@ -95,7 +97,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator SHORT_ARR_CREATOR = new ArrayCreator() { @Override public short[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid short array length: " + len); switch (len) { case 0: @@ -110,7 +113,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator INT_ARR_CREATOR = new ArrayCreator() { @Override public int[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid int array length: " + len); switch (len) { case 0: @@ -125,7 +129,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator LONG_ARR_CREATOR = new ArrayCreator() { @Override public long[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid long array length: " + len); switch (len) { case 0: @@ -140,7 +145,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator FLOAT_ARR_CREATOR = new ArrayCreator() { @Override public float[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid float array length: " + len); switch (len) { case 0: @@ -155,7 +161,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator DOUBLE_ARR_CREATOR = new ArrayCreator() { @Override public double[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid double array length: " + len); switch (len) { case 0: @@ -170,7 +177,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator CHAR_ARR_CREATOR = new ArrayCreator() { @Override public char[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid char array length: " + len); switch (len) { case 0: @@ -185,7 +193,8 @@ public class DirectByteBufferStreamImplV2 implements DirectByteBufferStream { /** */ private static final ArrayCreator BOOLEAN_ARR_CREATOR = new ArrayCreator() { @Override public boolean[] create(int len) { - assert len >= 0; + if (len < 0) + throw new IgniteException("Read invalid boolean array length: " + len); switch (len) { case 0: From 9cd7e0f8d132f9b7c496fe64f75f271ef60da5eb Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Thu, 9 Feb 2017 16:44:41 +0700 Subject: [PATCH 478/487] IGNITE-4676 Fixed hang if closure executed nested internal task with continuation. Added test. (cherry picked from commit e7a5307) --- .../processors/job/GridJobWorker.java | 4 + .../internal/GridContinuousTaskSelfTest.java | 79 +++++++++++++++++++ 2 files changed, 83 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java index 6a00d96a0abc2..acefde72eefab 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java @@ -617,6 +617,10 @@ else if (X.hasCause(e, GridServiceNotFoundException.class) || // Finish here only if not held by this thread. if (!HOLD.get()) finishJob(res, ex, sndRes); + else + // Make sure flag is not set for current thread. + // This may happen in case of nested internal task call with continuation. + HOLD.set(false); ctx.job().currentTaskSession(null); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridContinuousTaskSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridContinuousTaskSelfTest.java index 98e3c5afc8e2d..cec288714c100 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridContinuousTaskSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridContinuousTaskSelfTest.java @@ -21,10 +21,12 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Timer; import java.util.TimerTask; +import java.util.concurrent.Callable; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCompute; import org.apache.ignite.IgniteException; @@ -43,7 +45,9 @@ import org.apache.ignite.compute.ComputeTaskSessionAttributeListener; import org.apache.ignite.compute.ComputeTaskSessionFullSupport; import org.apache.ignite.compute.ComputeTaskSplitAdapter; +import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.JobContextResource; import org.apache.ignite.resources.LoggerResource; import org.apache.ignite.resources.TaskContinuousMapperResource; @@ -51,6 +55,7 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.testframework.junits.common.GridCommonTest; +import org.jetbrains.annotations.Nullable; /** * Continuous task test. @@ -195,6 +200,80 @@ public void testMultipleHoldccCalls() throws Exception { } } + /** + * @throws Exception If test failed. + */ + public void testClosureWithNestedInternalTask() throws Exception { + try { + IgniteEx ignite = startGrid(0); + + ComputeTaskInternalFuture fut = ignite.context().closure().callAsync(GridClosureCallMode.BALANCE, new Callable() { + /** */ + @IgniteInstanceResource + private IgniteEx g; + + @Override public String call() throws Exception { + return g.compute(g.cluster()).execute(NestedHoldccTask.class, null); + } + }, ignite.cluster().nodes()); + + assertEquals("DONE", fut.get(3000)); + } + finally { + stopGrid(0, true); + } + } + + /** Test task with continuation. */ + @GridInternal + public static class NestedHoldccTask extends ComputeTaskAdapter { + /** {@inheritDoc} */ + @Nullable @Override public Map map(List subgrid, + @Nullable String arg) throws IgniteException { + Map map = new HashMap<>(); + + for (ClusterNode node : subgrid) + map.put(new NestedHoldccJob(), node); + + return map; + + } + + /** {@inheritDoc} */ + @Nullable @Override public String reduce(List results) throws IgniteException { + return results.get(0).getData(); + } + } + + /** Test job. */ + public static class NestedHoldccJob extends ComputeJobAdapter { + /** */ + @JobContextResource + private ComputeJobContext jobCtx; + + /** */ + private int cnt = 0; + + /** {@inheritDoc} */ + @Override public Object execute() throws IgniteException { + if (cnt < 1) { + cnt++; + + jobCtx.holdcc(); + + new Timer().schedule(new TimerTask() { + @Override public void run() { + jobCtx.callcc(); + } + }, 500); + + return "NOT DONE"; + } + + return "DONE"; + } + } + /** */ @SuppressWarnings({"PublicInnerClass"}) public static class TestMultipleHoldccCallsClosure implements IgniteClosure { From 43bcc15127bd3fd7ac4e277da6da9e5fb6a855c0 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Thu, 30 Mar 2017 11:08:10 +0700 Subject: [PATCH 479/487] IGNITE-4838 Fixed internal task detection logic. Added tests. (cherry picked from commit ba68c6c) --- .../processors/task/GridTaskProcessor.java | 9 ++++- .../internal/GridTaskExecutionSelfTest.java | 34 +++++++++++++++++++ ...cutionWithoutPeerClassLoadingSelfTest.java | 31 +++++++++++++++++ .../IgniteComputeGridTestSuite.java | 2 ++ 4 files changed, 75 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionWithoutPeerClassLoadingSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java index d32b51c0867ca..935686456e593 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java @@ -608,6 +608,13 @@ else if (task != null) { if (subjId == null) subjId = ctx.localNodeId(); + boolean internal = false; + + if (dep == null || taskCls == null) + assert deployEx != null; + else + internal = dep.internalTask(task, taskCls); + // Creates task session with task name and task version. GridTaskSessionImpl ses = ctx.session().createTaskSession( sesId, @@ -621,7 +628,7 @@ else if (task != null) { Collections.emptyList(), Collections.emptyMap(), fullSup, - dep != null && dep.internalTask(task, taskCls), + internal, subjId); ComputeTaskInternalFuture fut = new ComputeTaskInternalFuture<>(ses, ctx); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionSelfTest.java index 996acd686f3cd..cc6a1eae6b2c6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionSelfTest.java @@ -22,8 +22,10 @@ import org.apache.ignite.GridTestTask; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCompute; +import org.apache.ignite.IgniteDeploymentException; import org.apache.ignite.compute.ComputeJobContext; import org.apache.ignite.compute.ComputeTaskFuture; +import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteFuture; @@ -45,6 +47,20 @@ public GridTaskExecutionSelfTest() { super(false); } + /** */ + protected boolean peerClassLoadingEnabled() { + return true; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setPeerClassLoadingEnabled(peerClassLoadingEnabled()); + + return cfg; + } + /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { startGrid(1); @@ -144,4 +160,22 @@ public void testJobIdCollision() throws Exception { for (IgniteFuture fut : futs) fut.get(); } + + /** + * Test execution of non-existing task by name IGNITE-4838. + * + * @throws Exception If failed. + */ + public void testExecuteTaskWithInvalidName() throws Exception { + try { + ComputeTaskFuture fut = ignite.compute().execute("invalid.task.name", null); + + fut.get(); + + assert false : "Should never be reached due to exception thrown."; + } + catch (IgniteDeploymentException e) { + info("Received correct exception: " + e); + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionWithoutPeerClassLoadingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionWithoutPeerClassLoadingSelfTest.java new file mode 100644 index 0000000000000..45e65cd9db017 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionWithoutPeerClassLoadingSelfTest.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal; + +import org.apache.ignite.testframework.junits.common.GridCommonTest; + +/** + * Task execution test. + */ +@GridCommonTest(group = "Kernal Self") +public class GridTaskExecutionWithoutPeerClassLoadingSelfTest extends GridTaskExecutionSelfTest { + /** {@inheritDoc} */ + @Override protected boolean peerClassLoadingEnabled() { + return false; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java index 8a501fdb801d6..ae64c95fb30b6 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java @@ -51,6 +51,7 @@ import org.apache.ignite.internal.GridStopWithWaitSelfTest; import org.apache.ignite.internal.GridTaskCancelSingleNodeSelfTest; import org.apache.ignite.internal.GridTaskExecutionSelfTest; +import org.apache.ignite.internal.GridTaskExecutionWithoutPeerClassLoadingSelfTest; import org.apache.ignite.internal.GridTaskFailoverAffinityRunTest; import org.apache.ignite.internal.GridTaskFailoverSelfTest; import org.apache.ignite.internal.GridTaskFutureImplStopGridSelfTest; @@ -100,6 +101,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCancelUnusedJobSelfTest.class); suite.addTestSuite(GridTaskJobRejectSelfTest.class); suite.addTestSuite(GridTaskExecutionSelfTest.class); + suite.addTestSuite(GridTaskExecutionWithoutPeerClassLoadingSelfTest.class); suite.addTestSuite(GridFailoverSelfTest.class); suite.addTestSuite(GridTaskListenerSelfTest.class); suite.addTestSuite(GridFailoverTopologySelfTest.class); From 2a818d36395dd1af23acf444adf396b2e2edbede Mon Sep 17 00:00:00 2001 From: Konstantin Dudkov Date: Mon, 22 May 2017 16:28:07 +0300 Subject: [PATCH 480/487] Fixed "IGNITE-4205 CassandraCacheStore should start IiteThread threads in loadCache() method" Signed-off-by: nikolay_tikhonov --- .../store/cassandra/CassandraCacheStore.java | 13 ++- .../tests/IgnitePersistentStoreTest.java | 62 +++++++++++++ .../loadall_blob/ignite-config.xml | 90 +++++++++++++++++++ .../loadall_blob/persistence-settings.xml | 29 ++++++ .../store/jdbc/CacheAbstractJdbcStore.java | 6 +- 5 files changed, 198 insertions(+), 2 deletions(-) create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/ignite-config.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/persistence-settings.xml diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java index 2e1d3ea70318d..dabf1b09a588e 100644 --- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java @@ -31,6 +31,7 @@ import javax.cache.Cache; import javax.cache.integration.CacheLoaderException; import javax.cache.integration.CacheWriterException; +import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.store.CacheStore; @@ -47,7 +48,9 @@ import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.logger.NullLogger; import org.apache.ignite.resources.CacheStoreSessionResource; +import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.thread.IgniteThreadFactory; /** * Implementation of {@link CacheStore} backed by Cassandra database. @@ -59,6 +62,14 @@ public class CassandraCacheStore implements CacheStore { /** Connection attribute property name. */ private static final String ATTR_CONN_PROP = "CASSANDRA_STORE_CONNECTION"; + /** Thread name. */ + private static final String CACHE_LOADER_THREAD_NAME = "cassandra-cache-loader"; + + /** Auto-injected ignite instance. */ + @SuppressWarnings("unused") + @IgniteInstanceResource + private Ignite ignite; + /** Auto-injected store session. */ @CacheStoreSessionResource private CacheStoreSession storeSes; @@ -99,7 +110,7 @@ public CassandraCacheStore(DataSource dataSrc, KeyValuePersistenceSettings setti Collection> futs = new ArrayList<>(args.length); try { - pool = Executors.newFixedThreadPool(maxPoolSize); + pool = Executors.newFixedThreadPool(maxPoolSize, new IgniteThreadFactory(ignite.name(), CACHE_LOADER_THREAD_NAME)); CassandraSession ses = getCassandraSession(); diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java index 51d08855388c6..75dff66de8f84 100644 --- a/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java @@ -23,9 +23,11 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.Ignition; +import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.store.CacheStore; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.processors.cache.CacheEntryImpl; import org.apache.ignite.tests.pojos.Person; import org.apache.ignite.tests.pojos.PersonId; @@ -237,6 +239,35 @@ public void blobStrategyTest() { } } + + /** */ + @Test + public void blobBinaryLoadCacheTest() { + Ignition.stopAll(true); + + try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/loadall_blob/ignite-config.xml")) { + IgniteCache personCache = ignite.getOrCreateCache("cache2"); + + assert ignite.configuration().getMarshaller() instanceof BinaryMarshaller; + + personCache.put(1L, new PojoPerson(1, "name")); + + assert personCache.withKeepBinary().get(1L) instanceof BinaryObject; + } + + Ignition.stopAll(true); + + try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/loadall_blob/ignite-config.xml")) { + IgniteCache personCache = ignite.getOrCreateCache("cache2"); + + personCache.loadCache(null, null); + + PojoPerson person = personCache.get(1L); + + LOGGER.info("loadCache tests passed"); + } + } + /** */ @Test public void pojoStrategyTest() { @@ -377,4 +408,35 @@ public void loadCacheTest() { LOGGER.info("loadCache test passed"); } + + + /** */ + public static class PojoPerson { + /** */ + private int id; + + /** */ + private String name; + + /** */ + public PojoPerson() { + // No-op. + } + + /** */ + public PojoPerson(int id, String name) { + this.id = id; + this.name = name; + } + + /** */ + public int getId() { + return id; + } + + /** */ + public String getName() { + return name; + } + } } diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/ignite-config.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/ignite-config.xml new file mode 100644 index 0000000000000..115e263e017d8 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/ignite-config.xml @@ -0,0 +1,90 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500..47509 + + + + + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/persistence-settings.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/persistence-settings.xml new file mode 100644 index 0000000000000..e872201fcc180 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/loadall_blob/persistence-settings.xml @@ -0,0 +1,29 @@ + + + + + + + + + diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java index e211fadf2fb74..817b1a5da8531 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java @@ -72,6 +72,7 @@ import org.apache.ignite.resources.CacheStoreSessionResource; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.thread.IgniteThreadFactory; import org.apache.ignite.transactions.Transaction; import org.jetbrains.annotations.Nullable; @@ -121,6 +122,9 @@ * */ public abstract class CacheAbstractJdbcStore implements CacheStore, LifecycleAware { + /** Thread name. */ + private static final String CACHE_LOADER_THREAD_NAME = "jdbc-cache-loader"; + /** Connection attribute property name. */ protected static final String ATTR_CONN_PROP = "JDBC_STORE_CONNECTION"; @@ -730,7 +734,7 @@ protected Integer columnIndex(Map loadColIdxs, String dbName) { String cacheName = session().cacheName(); try { - pool = Executors.newFixedThreadPool(maxPoolSize); + pool = Executors.newFixedThreadPool(maxPoolSize, new IgniteThreadFactory(ignite.name(), CACHE_LOADER_THREAD_NAME)); Collection> futs = new ArrayList<>(); From 04fadd4a499239176ba21c390d93e30809abb4c1 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 23 May 2017 15:42:20 +0300 Subject: [PATCH 481/487] IGNITE-5223 Allow use local binary metadata cache if it's possible --- .../apache/ignite/IgniteSystemProperties.java | 8 + .../ignite/internal/binary/BinaryContext.java | 4 + .../internal/binary/BinaryReaderExImpl.java | 23 +- .../CacheObjectBinaryProcessorImpl.java | 98 ++++-- ...inaryMarshallerLocalMetadataCacheTest.java | 297 ++++++++++++++++++ .../IgniteBinaryObjectsTestSuite.java | 3 + 6 files changed, 406 insertions(+), 27 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCacheTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index d77b2fb2838d4..713defeda552c 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -515,6 +515,14 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_UNWRAP_BINARY_FOR_INDEXING_SPI = "IGNITE_UNWRAP_BINARY_FOR_INDEXING_SPI"; + /** + * Use local metadata cache instead of distributed one. May be used only when binary objects schema + * are not modified and all classes available on each node. Classes that implements Binarylizable are + * not supported. + * @deprecated Should be removed in Apache Ignite 2.0. + */ + public static final String IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE = "IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java index 7b21dfbed51f0..9f66b3ed50046 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java @@ -612,6 +612,7 @@ else if (cpElement.isFile()) { /** * @param cls Class. + * @param deserialize If {@code false}, metadata will be updated. * @return Class descriptor. * @throws BinaryObjectException In case of error. */ @@ -664,6 +665,7 @@ else if (!desc.registered()) { * @param userType User type or not. * @param typeId Type ID. * @param ldr Class loader. + * @param deserialize If {@code false}, metadata will be updated. * @return Class descriptor. */ public BinaryClassDescriptor descriptorForTypeId( @@ -719,6 +721,7 @@ public BinaryClassDescriptor descriptorForTypeId( * Creates and registers {@link BinaryClassDescriptor} for the given {@code class}. * * @param cls Class. + * @param deserialize If {@code false}, metadata will be updated. * @return Class descriptor. */ private BinaryClassDescriptor registerClassDescriptor(Class cls, boolean deserialize) { @@ -759,6 +762,7 @@ private BinaryClassDescriptor registerClassDescriptor(Class cls, boolean dese * Creates and registers {@link BinaryClassDescriptor} for the given user {@code class}. * * @param cls Class. + * @param deserialize If {@code false}, metadata will be updated. * @return Class descriptor. */ private BinaryClassDescriptor registerUserClassDescriptor(Class cls, boolean deserialize) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java index 775f237787cfc..ad2e736fa5f34 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryReaderExImpl.java @@ -26,6 +26,7 @@ import java.util.Date; import java.util.Map; import java.util.UUID; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.binary.BinaryCollectionFactory; import org.apache.ignite.binary.BinaryInvalidTypeException; import org.apache.ignite.binary.BinaryMapFactory; @@ -1722,15 +1723,23 @@ public BinarySchema getOrCreateSchema() { if (fieldIdLen != BinaryUtils.FIELD_ID_LEN) { BinaryTypeImpl type = (BinaryTypeImpl)ctx.metadata(typeId); - if (type == null || type.metadata() == null) - throw new BinaryObjectException("Cannot find metadata for object with compact footer: " + - typeId); + if (type == null || type.metadata() == null || type.metadata().schemas().isEmpty()) { + if (IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE, false)) { + BinaryClassDescriptor desc = ctx.descriptorForTypeId(true, typeId, getClass().getClassLoader(), false); - for (BinarySchema typeSchema : type.metadata().schemas()) { - if (schemaId == typeSchema.schemaId()) { - schema = typeSchema; + schema = desc.schema(); + } + else + throw new BinaryObjectException("Cannot find metadata for object with compact footer: " + + typeId); + } + else { + for (BinarySchema typeSchema : type.metadata().schemas()) { + if (schemaId == typeSchema.schemaId()) { + schema = typeSchema; - break; + break; + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java index 1d60c422a6417..6b691c2236e52 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java @@ -39,6 +39,7 @@ import org.apache.ignite.IgniteBinary; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.binary.BinaryBasicNameMapper; import org.apache.ignite.binary.BinaryField; import org.apache.ignite.binary.BinaryObject; @@ -126,6 +127,9 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm /** */ private volatile IgniteCacheProxy metaDataCache; + /** */ + private final ConcurrentHashMap8 locMetadataCache; + /** */ private final ConcurrentHashMap8 clientMetaDataCache; @@ -175,6 +179,10 @@ public CacheObjectBinaryProcessorImpl(GridKernalContext ctx) { clientNode = this.ctx.clientNode(); clientMetaDataCache = clientNode ? new ConcurrentHashMap8() : null; + + boolean useLocCache = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE, false); + + locMetadataCache = useLocCache ? new ConcurrentHashMap8() : null; } /** {@inheritDoc} */ @@ -190,7 +198,7 @@ public CacheObjectBinaryProcessorImpl(GridKernalContext ctx) { BinaryMetadata newMeta0 = ((BinaryTypeImpl)newMeta).metadata(); - if (metaDataCache == null) { + if (metaDataCache == null && locMetadataCache == null) { BinaryMetadata oldMeta = metaBuf.get(typeId); BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta0); @@ -213,13 +221,13 @@ public CacheObjectBinaryProcessorImpl(GridKernalContext ctx) { return; } - assert metaDataCache != null; + assert metaDataCache != null || locMetadataCache != null; CacheObjectBinaryProcessorImpl.this.addMeta(typeId, newMeta0.wrap(binaryCtx)); } @Override public BinaryType metadata(int typeId) throws BinaryObjectException { - if (metaDataCache == null) + if (metaDataCache == null && locMetadataCache == null) U.awaitQuiet(startLatch); return CacheObjectBinaryProcessorImpl.this.metadata(typeId); @@ -289,6 +297,12 @@ public CacheObjectBinaryProcessorImpl(GridKernalContext ctx) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public void onUtilityCacheStarted() throws IgniteCheckedException { + if (locMetadataCache != null) { + startLatch.countDown(); + + return; + } + IgniteCacheProxy proxy = ctx.cache().jcache(CU.UTILITY_CACHE_NAME); boolean old = proxy.context().deploy().ignoreOwnership(true); @@ -573,22 +587,31 @@ public GridBinaryMarshaller marshaller() { final BinaryMetadataKey key = new BinaryMetadataKey(typeId); - try { - BinaryMetadata oldMeta = metaDataCache.localPeek(key); - BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta0); + if (locMetadataCache != null) { + locMetadataCache.merge(key, newMeta0, new ConcurrentHashMap8.BiFun() { + @Override public BinaryMetadata apply(BinaryMetadata curMeta, BinaryMetadata newMeta) { + return BinaryUtils.mergeMetadata(curMeta, newMeta); + } + }); + } + else { + try { + BinaryMetadata oldMeta = metaDataCache.localPeek(key); + BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta0); - AffinityTopologyVersion topVer = ctx.cache().context().lockedTopologyVersion(null); + AffinityTopologyVersion topVer = ctx.cache().context().lockedTopologyVersion(null); - if (topVer == null) - topVer = ctx.cache().context().exchange().readyAffinityVersion(); + if (topVer == null) + topVer = ctx.cache().context().exchange().readyAffinityVersion(); - BinaryObjectException err = metaDataCache.invoke(topVer, key, new MetadataProcessor(mergedMeta)); + BinaryObjectException err = metaDataCache.invoke(topVer, key, new MetadataProcessor(mergedMeta)); - if (err != null) - throw err; - } - catch (CacheException e) { - throw new BinaryObjectException("Failed to update meta data for type: " + newMeta.typeName(), e); + if (err != null) + throw err; + } + catch (CacheException e) { + throw new BinaryObjectException("Failed to update meta data for type: " + newMeta.typeName(), e); + } } } @@ -601,17 +624,28 @@ public GridBinaryMarshaller marshaller() { if (typeMeta != null) return typeMeta; - BinaryMetadata meta = metaDataCache.getTopologySafe(new BinaryMetadataKey(typeId)); + BinaryMetadata meta; + + if (locMetadataCache != null) + meta = locMetadataCache.get(new BinaryMetadataKey(typeId)); + else + meta = metaDataCache.getTopologySafe(new BinaryMetadataKey(typeId)); return meta != null ? meta.wrap(binaryCtx) : null; } else { BinaryMetadataKey key = new BinaryMetadataKey(typeId); - BinaryMetadata meta = metaDataCache.localPeek(key); + BinaryMetadata meta; + + if (locMetadataCache != null) + meta = locMetadataCache.get(key); + else { + meta = metaDataCache.localPeek(key); - if (meta == null && !metaDataCache.context().preloader().syncFuture().isDone()) - meta = metaDataCache.getTopologySafe(key); + if (meta == null && !metaDataCache.context().preloader().syncFuture().isDone()) + meta = metaDataCache.getTopologySafe(key); + } return meta != null ? meta.wrap(binaryCtx) : null; } @@ -630,7 +664,20 @@ public GridBinaryMarshaller marshaller() { for (Integer typeId : typeIds) keys.add(new BinaryMetadataKey(typeId)); - Map meta = metaDataCache.getAll(keys); + Map meta; + + if (locMetadataCache != null) { + meta = new HashMap<>(); + + for (BinaryMetadataKey key : keys) { + BinaryMetadata metadata = locMetadataCache.get(key); + + if (metadata != null) + meta.put(key, metadata); + } + } + else + meta = metaDataCache.getAll(keys); Map res = U.newHashMap(meta.size()); @@ -654,6 +701,17 @@ public GridBinaryMarshaller marshaller() { } }); else { + if (locMetadataCache != null) { + ConcurrentHashMap8.ValuesView vals = locMetadataCache.values(); + + ArrayList res = new ArrayList<>(vals.size()); + + for (BinaryMetadata metadata : vals) + res.add(metadata.wrap(binaryCtx)); + + return res; + } + return F.viewReadOnly(metaDataCache.entrySetx(metaPred), new C1, BinaryType>() { private static final long serialVersionUID = 0L; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCacheTest.java new file mode 100644 index 0000000000000..ad3c5f3e0a83b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerLocalMetadataCacheTest.java @@ -0,0 +1,297 @@ +/* + * 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.binary; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.math.BigInteger; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.cluster.ClusterGroup; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.testframework.junits.IgniteTestResources; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE property. + */ +public class BinaryMarshallerLocalMetadataCacheTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName, + IgniteTestResources rsrcs) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName, rsrcs); + + cfg.setMarshaller(new BinaryMarshaller()); + + cfg.setClientMode(gridName.startsWith("client")); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + System.setProperty(IgniteSystemProperties.IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE, "true"); + + startGrid(0); + startGrid(1); + startGrid("client"); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + try { + stopAllGrids(); + } + finally { + System.clearProperty(IgniteSystemProperties.IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE); + } + } + + /** + * @throws Exception If failed. + */ + public void testComputeLocalMetadata() throws Exception { + final BinaryObject obj = grid(0).binary().toBinary(new OptimizedContainer(new Optimized())); + + ClusterGroup remotes = grid(0).cluster().forRemotes(); + + OptimizedContainer res = grid(0).compute(remotes).call(new IgniteCallable() { + @Override public OptimizedContainer call() throws Exception { + + return obj.deserialize(); + } + }); + + OptimizedContainer res2 = grid(0).compute(remotes).call(new IgniteCallable() { + @Override public OptimizedContainer call() throws Exception { + + return obj.deserialize(); + } + }); + + System.out.println(res); + System.out.println(res2); + } + + /** + * @throws Exception If failed. + */ + public void testCacheLocalMetadata() throws Exception { + IgniteCache cache = grid("client").createCache("cache"); + + Map data = new HashMap<>(); + + for (int i = 0; i < 1000; i++) + data.put(new Key(i), new OptimizedContainer(new Optimized(String.valueOf(i)))); + + for (int i = 1000; i < 2000; i++) + data.put(new Key(i), new Simple(i, String.valueOf(i), new BigInteger(String.valueOf(i), 10))); + + cache.putAll(data); + + checkCache(cache, data); + checkCache(grid(0).cache("cache"), data); + checkCache(grid(1).cache("cache"), data); + } + + /** + * @param cache Cache. + * @param data Data. + */ + private void checkCache(IgniteCache cache, + Map data) { + for (Map.Entry entry : cache.getAll(data.keySet()).entrySet()) + assertEquals(data.get(entry.getKey()), entry.getValue()); + } + + /** + * + */ + private static class Key { + /** */ + private Integer i; + + /** + * @param i I. + */ + public Key(Integer i) { + this.i = i; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + Key key = (Key)o; + + return i != null ? i.equals(key.i) : key.i == null; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return i != null ? i.hashCode() : 0; + } + } + + /** + * + */ + private static class OptimizedContainer { + /** */ + private Optimized optim; + + /** + * @param optim Val. + */ + public OptimizedContainer(Optimized optim) { + this.optim = optim; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + OptimizedContainer container = (OptimizedContainer)o; + + return optim != null ? optim.equals(container.optim) : container.optim == null; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return optim != null ? optim.hashCode() : 0; + } + } + + /** + * + */ + private static class Optimized implements Externalizable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** */ + private String fld; + + /** + * @param fld Fld. + */ + public Optimized(String fld) { + this.fld = fld; + } + + /** + * Default constructor (required by Externalizable). + */ + public Optimized() { + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + U.writeUTFStringNullable(out, fld); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + fld = U.readUTFStringNullable(in); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + Optimized optimized = (Optimized)o; + + return fld != null ? fld.equals(optimized.fld) : optimized.fld == null; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return fld != null ? fld.hashCode() : 0; + } + } + + /** + * + */ + private static class Simple { + /** I. */ + private int i; + + /** String. */ + private String str; + + /** Big integer. */ + private BigInteger bigInteger; + + /** + * @param i I. + * @param str String. + * @param bigInteger Big integer. + */ + public Simple(int i, String str, BigInteger bigInteger) { + this.i = i; + this.str = str; + this.bigInteger = bigInteger; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + Simple simple = (Simple)o; + + if (i != simple.i) + return false; + + if (str != null ? !str.equals(simple.str) : simple.str != null) + return false; + + return bigInteger != null ? bigInteger.equals(simple.bigInteger) : simple.bigInteger == null; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int res = i; + + res = 31 * res + (str != null ? str.hashCode() : 0); + res = 31 * res + (bigInteger != null ? bigInteger.hashCode() : 0); + + return res; + } + } +} 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 c1d9974837fb3..29e43e4593415 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 @@ -26,6 +26,7 @@ import org.apache.ignite.internal.binary.BinaryFieldsOffheapSelfTest; import org.apache.ignite.internal.binary.BinaryFooterOffsetsHeapSelfTest; import org.apache.ignite.internal.binary.BinaryFooterOffsetsOffheapSelfTest; +import org.apache.ignite.internal.binary.BinaryMarshallerLocalMetadataCacheTest; import org.apache.ignite.internal.binary.BinaryMarshallerSelfTest; import org.apache.ignite.internal.binary.BinaryObjectBuilderAdditionalSelfTest; import org.apache.ignite.internal.binary.BinaryObjectBuilderDefaultMappersSelfTest; @@ -147,6 +148,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheBinaryObjectUserClassloaderSelfTest.class); + suite.addTestSuite(BinaryMarshallerLocalMetadataCacheTest.class); + return suite; } } From b2040b7a95e421609bcf7ae05b56dc623310b409 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 23 May 2017 16:14:08 +0300 Subject: [PATCH 482/487] IGNITE-5259 Minor serialization fix --- modules/core/pom.xml | 3 + .../apache/ignite/IgniteSystemProperties.java | 12 ++- .../ignite/internal/IgniteNodeAttributes.java | 6 ++ .../discovery/GridDiscoveryManager.java | 39 ++++++++ .../CacheContinuousQueryHandler.java | 1 - .../top/GridTopologyCommandHandler.java | 2 + .../processors/security/SecurityUtils.java | 92 ++++++++++++++++++ .../security/SecurityBasicPermissionSet.java | 41 +++++++- .../ignite/spi/discovery/tcp/ServerImpl.java | 97 +++++++++++++++++-- ...ridDiscoveryManagerAttributesSelfTest.java | 70 ++++++++++++- .../discovery/tcp/TestReconnectProcessor.java | 47 ++++++++- 11 files changed, 390 insertions(+), 20 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java diff --git a/modules/core/pom.xml b/modules/core/pom.xml index 4c4343a9c41a1..4a14407ab131f 100644 --- a/modules/core/pom.xml +++ b/modules/core/pom.xml @@ -234,6 +234,9 @@ **/*.java + + src/test/resources + diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 713defeda552c..6827e0c248290 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -503,7 +503,7 @@ public final class IgniteSystemProperties { /** * Whether Ignite can access unaligned memory addresses. *

          - * Defaults to {@code} false, meaning that unaligned access will be performed only on x86 architecture. + * Defaults to {@code false}, meaning that unaligned access will be performed only on x86 architecture. */ public static final String IGNITE_UNALIGNED_MEMORY_ACCESS = "IGNITE_UNALIGNED_MEMORY_ACCESS"; @@ -523,6 +523,16 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE = "IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE"; + /** + * When set to {@code true}, Ignite switches to compatibility mode with versions that don't + * support service security permissions. In this case security permissions will be ignored + * (if they set). + *

          + * Default is {@code false}, which means that service security permissions will be respected. + *

          + */ + public static final String IGNITE_SECURITY_COMPATIBILITY_MODE = "IGNITE_SECURITY_COMPATIBILITY_MODE"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java index e7c984f22864e..436792459af4a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java @@ -136,6 +136,9 @@ public final class IgniteNodeAttributes { /** Security subject for authenticated node. */ public static final String ATTR_SECURITY_SUBJECT = ATTR_PREFIX + ".security.subject"; + /** V2 security subject for authenticated node. */ + public static final String ATTR_SECURITY_SUBJECT_V2 = ATTR_PREFIX + ".security.subject.v2"; + /** Client mode flag. */ public static final String ATTR_CLIENT_MODE = ATTR_PREFIX + ".cache.client"; @@ -160,6 +163,9 @@ public final class IgniteNodeAttributes { /** Ignite services compatibility mode (can be {@code null}). */ public static final String ATTR_SERVICES_COMPATIBILITY_MODE = ATTR_PREFIX + ".services.compatibility.enabled"; + /** Ignite security compatibility mode. */ + public static final String ATTR_SECURITY_COMPATIBILITY_MODE = ATTR_PREFIX + ".security.compatibility.enabled"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java index 2ec10705bedbb..b3ba83da68a8d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java @@ -121,6 +121,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2; import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCOVERY_HISTORY_SIZE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_SECURITY_COMPATIBILITY_MODE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE; import static org.apache.ignite.IgniteSystemProperties.getInteger; import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED; @@ -136,9 +137,12 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_BINARY_STRING_SER_VER_2; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_COMPATIBILITY_MODE; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SERVICES_COMPATIBILITY_MODE; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_USER_NAME; import static org.apache.ignite.internal.IgniteVersionUtils.VER; +import static org.apache.ignite.internal.processors.security.SecurityUtils.SERVICE_PERMISSIONS_SINCE; +import static org.apache.ignite.internal.processors.security.SecurityUtils.isSecurityCompatibilityMode; import static org.apache.ignite.plugin.segmentation.SegmentationPolicy.NOOP; /** @@ -442,6 +446,9 @@ private void updateClientNodes(UUID leftNodeId) { spi.setMetricsProvider(createMetricsProvider()); if (ctx.security().enabled()) { + if (isSecurityCompatibilityMode()) + ctx.addNodeAttribute(ATTR_SECURITY_COMPATIBILITY_MODE, true); + spi.setAuthenticator(new DiscoverySpiNodeAuthenticator() { @Override public SecurityContext authenticateNode(ClusterNode node, SecurityCredentials cred) { try { @@ -1044,6 +1051,7 @@ private void checkAttributes(Iterable nodes) throws IgniteCheckedEx boolean locDelayAssign = locNode.attribute(ATTR_LATE_AFFINITY_ASSIGNMENT); Boolean locSrvcCompatibilityEnabled = locNode.attribute(ATTR_SERVICES_COMPATIBILITY_MODE); + Boolean locSecurityCompatibilityEnabled = locNode.attribute(ATTR_SECURITY_COMPATIBILITY_MODE); for (ClusterNode n : nodes) { int rmtJvmMajVer = nodeJavaMajorVersion(n); @@ -1157,6 +1165,37 @@ else if (Boolean.FALSE.equals(locSrvcCompatibilityEnabled)) { ", rmtNodeAddrs=" + U.addressesAsString(n) + ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ']'); } + + if (n.version().compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE) >= 0 + && ctx.security().enabled() // Matters only if security enabled. + ) { + Boolean rmtSecurityCompatibilityEnabled = n.attribute(ATTR_SECURITY_COMPATIBILITY_MODE); + + if (!F.eq(locSecurityCompatibilityEnabled, rmtSecurityCompatibilityEnabled)) { + throw new IgniteCheckedException("Local node's " + IGNITE_SECURITY_COMPATIBILITY_MODE + + " property value differs from remote node's value " + + "(to make sure all nodes in topology have identical Ignite security compatibility mode enabled, " + + "configure system property explicitly) " + + "[locSecurityCompatibilityEnabled=" + locSecurityCompatibilityEnabled + + ", rmtSecurityCompatibilityEnabled=" + rmtSecurityCompatibilityEnabled + + ", locNodeAddrs=" + U.addressesAsString(locNode) + + ", rmtNodeAddrs=" + U.addressesAsString(n) + + ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ']'); + } + } + + if (n.version().compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE) < 0 + && ctx.security().enabled() // Matters only if security enabled. + && (locSecurityCompatibilityEnabled == null || !locSecurityCompatibilityEnabled)) { + throw new IgniteCheckedException("Remote node does not support service security permissions. " + + "To be able to join to it, local node must be started with " + IGNITE_SECURITY_COMPATIBILITY_MODE + + " system property set to \"true\". " + + "[locSecurityCompatibilityEnabled=" + locSecurityCompatibilityEnabled + + ", locNodeAddrs=" + U.addressesAsString(locNode) + + ", rmtNodeAddrs=" + U.addressesAsString(n) + + ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ", " + + ", rmtNodeVer" + n.version() + ']'); + } } if (log.isDebugEnabled()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java index 5f00d580f4231..17f4308c0709b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java @@ -998,7 +998,6 @@ private static class PartitionRecovery { synchronized (pendingEvts) { if (log.isDebugEnabled()) { - log.debug("Handling event [lastFiredEvt=" + lastFiredEvt + ", curTop=" + curTop + ", entUpdCnt=" + entry.updateCounter() + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java index 297785ea760f2..3c68fbfdba92e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java @@ -58,6 +58,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_TCP_PORT; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_SUBJECT; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_TX_CONFIG; import static org.apache.ignite.internal.processors.rest.GridRestCommand.NODE; import static org.apache.ignite.internal.processors.rest.GridRestCommand.TOPOLOGY; @@ -290,6 +291,7 @@ private GridClientNodeBean createNodeBean(ClusterNode node, boolean mtr, boolean attrs.remove(ATTR_CACHE); attrs.remove(ATTR_TX_CONFIG); attrs.remove(ATTR_SECURITY_SUBJECT); + attrs.remove(ATTR_SECURITY_SUBJECT_V2); attrs.remove(ATTR_SECURITY_CREDENTIALS); for (Iterator> i = attrs.entrySet().iterator(); i.hasNext();) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java new file mode 100644 index 0000000000000..1016335888b3d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java @@ -0,0 +1,92 @@ +/* + * 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.security; + +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.lang.IgniteProductVersion; +import org.apache.ignite.plugin.security.SecurityPermission; + +/** + * Security utilities. + */ +public class SecurityUtils { + /** Version since service security supported. */ + public static final IgniteProductVersion SERVICE_PERMISSIONS_SINCE = IgniteProductVersion.fromString("1.7.11"); + + /** Default serialization version. */ + private final static int DFLT_SERIALIZE_VERSION = isSecurityCompatibilityMode() ? 1 : 2; + + /** Current serialization version. */ + private static final ThreadLocal SERIALIZE_VERSION = new ThreadLocal(){ + @Override protected Integer initialValue() { + return DFLT_SERIALIZE_VERSION; + } + }; + + /** + * Private constructor. + */ + private SecurityUtils() { + } + + /** + * @return Security compatibility mode flag. + */ + public static boolean isSecurityCompatibilityMode() { + return IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_SECURITY_COMPATIBILITY_MODE, false); + } + + /** + * @param ver Serialize version. + */ + public static void serializeVersion(int ver) { + SERIALIZE_VERSION.set(ver); + } + + /** + * @return Serialize version. + */ + public static int serializeVersion() { + return SERIALIZE_VERSION.get(); + } + + /** + * Sets default serialize version {@link #DFLT_SERIALIZE_VERSION}. + */ + public static void restoreDefaultSerializeVersion() { + serializeVersion(DFLT_SERIALIZE_VERSION); + } + + /** + * @return Allow all service permissions. + */ + public static Map> compatibleServicePermissions() { + Map> srvcPerms = new HashMap<>(); + + srvcPerms.put("*", Arrays.asList( + SecurityPermission.SERVICE_CANCEL, + SecurityPermission.SERVICE_DEPLOY, + SecurityPermission.SERVICE_INVOKE)); + + return srvcPerms; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java index 44166d936c9aa..370eadd801f83 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java @@ -17,15 +17,24 @@ package org.apache.ignite.plugin.security; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.internal.processors.security.SecurityUtils.compatibleServicePermissions; +import static org.apache.ignite.internal.processors.security.SecurityUtils.isSecurityCompatibilityMode; +import static org.apache.ignite.internal.processors.security.SecurityUtils.serializeVersion; + /** * Simple implementation of {@link SecurityPermissionSet} interface. Provides * convenient way to specify permission set in the XML configuration. @@ -44,7 +53,9 @@ public class SecurityBasicPermissionSet implements SecurityPermissionSet { /** Service permissions. */ @GridToStringInclude - private Map> servicePermissions = new HashMap<>(); + private transient Map> servicePermissions = isSecurityCompatibilityMode() + ? compatibleServicePermissions() + : new HashMap>(); /** System permissions. */ @GridToStringInclude @@ -158,6 +169,34 @@ public void setDefaultAllowAll(boolean dfltAllowAll) { return res; } + /** + * @param out Out. + */ + private void writeObject(ObjectOutputStream out) throws IOException { + out.defaultWriteObject(); + + if (serializeVersion() >= 2) + U.writeMap(out, servicePermissions); + } + + /** + * @param in In. + */ + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + + if (serializeVersion() >= 2) + servicePermissions = U.readMap(in); + + if (servicePermissions == null) { + // Allow all for compatibility mode + if (serializeVersion() < 2) + servicePermissions = compatibleServicePermissions(); + else + servicePermissions = Collections.emptyMap(); + } + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(SecurityBasicPermissionSet.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index afd1c2ba7fc66..58b362ffd0f15 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -72,6 +72,7 @@ import org.apache.ignite.internal.events.DiscoveryCustomEvent; import org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager; import org.apache.ignite.internal.processors.security.SecurityContext; +import org.apache.ignite.internal.processors.security.SecurityUtils; import org.apache.ignite.internal.processors.service.GridServiceProcessor; import org.apache.ignite.internal.util.GridBoundedLinkedHashSet; import org.apache.ignite.internal.util.GridConcurrentHashSet; @@ -947,7 +948,8 @@ private void localAuthentication(SecurityCredentials locCred){ Map attrs = new HashMap<>(locNode.attributes()); - attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, U.marshal(spi.marshaller(), subj)); + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2, U.marshal(spi.marshaller(), subj)); + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, marshalWithSecurityVersion(subj, 1)); locNode.setAttributes(attrs); @@ -983,7 +985,16 @@ private boolean sendJoinRequestMessage() throws IgniteSpiException { for (InetSocketAddress addr : addrs) { try { - Integer res = sendMessageDirectly(joinReq, addr); + Integer res; + + try { + SecurityUtils.serializeVersion(1); + + res = sendMessageDirectly(joinReq, addr); + } + finally { + SecurityUtils.restoreDefaultSerializeVersion(); + } assert res != null; @@ -1921,6 +1932,39 @@ private void processMessageFailedNodes(TcpDiscoveryAbstractMessage msg) { } } + /** + * @param obj Object. + * @param ver Security serialize version. + * @return Marshaled object. + */ + private byte[] marshalWithSecurityVersion(Object obj, int ver) throws IgniteCheckedException { + try { + SecurityUtils.serializeVersion(ver); + + return U.marshal(spi.marshaller(), obj); + } + finally { + SecurityUtils.restoreDefaultSerializeVersion(); + } + } + + /** + * @param bytes Marshaled object. + * @param ver Security serialize version. + * @return Unmarshaled object. + */ + private T unmarshalWithSecurityVersion(byte[] bytes, int ver) throws IgniteCheckedException { + try { + if (ver > 0) + SecurityUtils.serializeVersion(ver); + + return spi.marshaller().unmarshal(bytes, U.resolveClassLoader(spi.ignite().configuration())); + } + finally { + SecurityUtils.restoreDefaultSerializeVersion(); + } + } + /** * Discovery messages history used for client reconnect. */ @@ -2995,6 +3039,8 @@ else if (!spi.failureDetectionTimeoutEnabled() && (e instanceof pendingMsgs.customDiscardId); try { + SecurityUtils.serializeVersion(1); + long tstamp = U.currentTimeMillis(); if (timeoutHelper == null) @@ -3033,6 +3079,8 @@ else if (!spi.failureDetectionTimeoutEnabled() && (e instanceof } } finally { + SecurityUtils.restoreDefaultSerializeVersion(); + clearNodeAddedMessage(msg); } @@ -3423,7 +3471,8 @@ else if (log.isDebugEnabled()) // Stick in authentication subject to node (use security-safe attributes for copy). Map attrs = new HashMap<>(node.getAttributes()); - attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, U.marshal(spi.marshaller(), subj)); + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2, U.marshal(spi.marshaller(), subj)); + attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, marshalWithSecurityVersion(subj, 1)); node.setAttributes(attrs); } @@ -4073,9 +4122,22 @@ else if (!locNodeId.equals(node.id()) && ring.node(node.id()) != null) { else { SecurityContext subj = spi.nodeAuth.authenticateNode(node, cred); - SecurityContext coordSubj = U.unmarshal(spi.marshaller(), - node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT), - U.resolveClassLoader(spi.ignite().configuration())); + byte[] subjBytes = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT); + byte[] subjBytesV2 = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2); + + SecurityContext coordSubj; + + try { + if (subjBytesV2 == null) + SecurityUtils.serializeVersion(1); + + coordSubj = U.unmarshal(spi.marshaller(), + subjBytesV2 != null ? subjBytesV2 : subjBytes, + U.resolveClassLoader(spi.ignite().configuration())); + } + finally { + SecurityUtils.restoreDefaultSerializeVersion(); + } if (!permissionsEqual(coordSubj.subject().permissions(), subj.subject().permissions())) { // Node has not pass authentication. @@ -4158,13 +4220,23 @@ else if (!locNodeId.equals(node.id()) && ring.node(node.id()) != null) { new TcpDiscoveryAuthFailedMessage(locNodeId, spi.locHost); try { - ClassLoader cl = U.resolveClassLoader(spi.ignite().configuration()); - byte[] rmSubj = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT); byte[] locSubj = locNode.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT); - SecurityContext rmCrd = spi.marshaller().unmarshal(rmSubj, cl); - SecurityContext locCrd = spi.marshaller().unmarshal(locSubj, cl); + byte[] rmSubjV2 = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2); + byte[] locSubjV2 = locNode.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2); + + int ver = 1; // Compatible version. + + if (rmSubjV2 != null && locSubjV2 != null) { + rmSubj = rmSubjV2; + locSubj = locSubjV2; + + ver = 0; // Default version. + } + + SecurityContext rmCrd = unmarshalWithSecurityVersion(rmSubj, ver); + SecurityContext locCrd = unmarshalWithSecurityVersion(locSubj, ver); if (!permissionsEqual(locCrd.subject().permissions(), rmCrd.subject().permissions())) { @@ -5812,6 +5884,8 @@ else if (e.hasCause(ObjectStreamException.class) || while (!isInterrupted()) { try { + SecurityUtils.serializeVersion(1); + TcpDiscoveryAbstractMessage msg = U.unmarshal(spi.marshaller(), in, U.resolveClassLoader(spi.ignite().configuration())); @@ -6062,6 +6136,9 @@ else if (msg instanceof TcpDiscoveryLoopbackProblemMessage) { return; } + finally { + SecurityUtils.restoreDefaultSerializeVersion(); + } } } finally { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java index ba8fa5b6e15a3..f0096db7bc8a6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java @@ -21,14 +21,18 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.configuration.DeploymentMode; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.marshaller.optimized.OptimizedMarshaller; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.TestReconnectPluginProvider; +import org.apache.ignite.spi.discovery.tcp.TestReconnectProcessor; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_SECURITY_COMPATIBILITY_MODE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2; import static org.apache.ignite.configuration.DeploymentMode.CONTINUOUS; @@ -258,18 +262,69 @@ public void testServiceCompatibilityEnabled() throws Exception { * @throws Exception If failed. */ private void doTestServiceCompatibilityEnabled(Object first, Object second, boolean fail) throws Exception { + doTestCompatibilityEnabled(IGNITE_SERVICES_COMPATIBILITY_MODE, first, second, fail); + } + + /** + * @throws Exception If failed. + */ + public void testSecurityCompatibilityEnabled() throws Exception { + TestReconnectPluginProvider.enabled = true; + TestReconnectProcessor.enabled = true; + + try { + doTestSecurityCompatibilityEnabled(true, null, true); + doTestSecurityCompatibilityEnabled(true, false, true); + doTestSecurityCompatibilityEnabled(false, true, true); + doTestSecurityCompatibilityEnabled(null, true, true); + + doTestSecurityCompatibilityEnabled(null, null, false); + doTestSecurityCompatibilityEnabled(null, false, false); + doTestSecurityCompatibilityEnabled(false, false, false); + doTestSecurityCompatibilityEnabled(false, null, false); + doTestSecurityCompatibilityEnabled(true, true, false); + } + finally { + TestReconnectPluginProvider.enabled = false; + TestReconnectProcessor.enabled = false; + } + } + + /** + * @param first Service compatibility enabled flag for first node. + * @param second Service compatibility enabled flag for second node. + * @param fail Fail flag. + * @throws Exception If failed. + */ + private void doTestSecurityCompatibilityEnabled(Object first, Object second, boolean fail) throws Exception { + doTestCompatibilityEnabled(IGNITE_SECURITY_COMPATIBILITY_MODE, first, second, fail); + } + + /** + * @param prop System property. + * @param first Service compatibility enabled flag for first node. + * @param second Service compatibility enabled flag for second node. + * @param fail Fail flag. + * @throws Exception If failed. + */ + private void doTestCompatibilityEnabled(String prop, Object first, Object second, boolean fail) throws Exception { + String backup = System.getProperty(prop); try { if (first != null) - System.setProperty(IGNITE_SERVICES_COMPATIBILITY_MODE, String.valueOf(first)); + System.setProperty(prop, String.valueOf(first)); else - System.clearProperty(IGNITE_SERVICES_COMPATIBILITY_MODE); + System.clearProperty(prop); - startGrid(0); + IgniteEx ignite = startGrid(0); + + // Ignore if disabled security plugin used. + if (IGNITE_SECURITY_COMPATIBILITY_MODE.equals(prop) && !ignite.context().security().enabled()) + return; if (second != null) - System.setProperty(IGNITE_SERVICES_COMPATIBILITY_MODE, String.valueOf(second)); + System.setProperty(prop, String.valueOf(second)); else - System.clearProperty(IGNITE_SERVICES_COMPATIBILITY_MODE); + System.clearProperty(prop); try { startGrid(1); @@ -284,6 +339,11 @@ private void doTestServiceCompatibilityEnabled(Object first, Object second, bool } finally { stopAllGrids(); + + if (backup != null) + System.setProperty(prop, backup); + else + System.clearProperty(prop); } } diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java index f0ed35c4ccfab..2476bd3d787bf 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java @@ -17,11 +17,13 @@ package org.apache.ignite.spi.discovery.tcp; +import java.io.Serializable; import java.util.Collection; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.processors.GridProcessorAdapter; import org.apache.ignite.internal.processors.security.GridSecurityProcessor; import org.apache.ignite.internal.processors.security.SecurityContext; @@ -37,6 +39,9 @@ * Updates node attributes on disconnect. */ public class TestReconnectProcessor extends GridProcessorAdapter implements GridSecurityProcessor { + /** Enabled flag. */ + public static boolean enabled; + /** * @param ctx Kernal context. */ @@ -44,10 +49,15 @@ protected TestReconnectProcessor(GridKernalContext ctx) { super(ctx); } + /** {@inheritDoc} */ + @Override public void start() throws IgniteCheckedException { + ctx.addNodeAttribute(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS, new SecurityCredentials()); + } + /** {@inheritDoc} */ @Override public SecurityContext authenticateNode(ClusterNode node, SecurityCredentials cred) throws IgniteCheckedException { - return null; + return new TestSecurityContext(); } /** {@inheritDoc} */ @@ -83,11 +93,44 @@ protected TestReconnectProcessor(GridKernalContext ctx) { /** {@inheritDoc} */ @Override public boolean enabled() { - return false; + return enabled; } /** {@inheritDoc} */ @Override public void onDisconnected(IgniteFuture reconnectFut) throws IgniteCheckedException { ctx.addNodeAttribute("test", "2"); } + + /** + * + */ + private static class TestSecurityContext implements SecurityContext, Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public SecuritySubject subject() { + return null; + } + + /** {@inheritDoc} */ + @Override public boolean taskOperationAllowed(String taskClsName, SecurityPermission perm) { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean cacheOperationAllowed(String cacheName, SecurityPermission perm) { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean serviceOperationAllowed(String srvcName, SecurityPermission perm) { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean systemOperationAllowed(SecurityPermission perm) { + return true; + } + } } From b77428d12658b3ab2cdd43ca61ed71d329e83283 Mon Sep 17 00:00:00 2001 From: sboikov Date: Tue, 10 Jan 2017 16:59:17 +0300 Subject: [PATCH 483/487] Do not evict removed entries, otherwise removes can be lost. (cherry picked from commit 55ac6e7) --- .../internal/processors/cache/GridCacheMapEntry.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 58b4ae3130c9d..c8b8cd115fefc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -4300,6 +4300,10 @@ private synchronized CacheEntryImplEx wrapVersionedWithValue() { if (obsoleteVersionExtras() != null) return true; + // TODO GG-11241: need keep removed entries in heap map, otherwise removes can be lost. + if (cctx.deferredDelete() && deletedUnlocked()) + return false; + CacheObject prev = saveOldValueUnlocked(false); if (!hasReaders() && markObsolete0(obsoleteVer, false, null)) { @@ -4358,6 +4362,10 @@ private synchronized CacheEntryImplEx wrapVersionedWithValue() { // Version has changed since entry passed the filter. Do it again. continue; + // TODO GG-11241: need keep removed entries in heap map, otherwise removes can be lost. + if (cctx.deferredDelete() && deletedUnlocked()) + return false; + CacheObject prevVal = saveValueForIndexUnlocked(); if (!hasReaders() && markObsolete0(obsoleteVer, false, null)) { From 29187ef6b663eafe67eaaaf38e4c09fc244ac7aa Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 24 May 2017 17:31:27 +0300 Subject: [PATCH 484/487] Do not evict removed entries, otherwise removes can be lost. Rollback due to test failings. --- .../internal/processors/cache/GridCacheMapEntry.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index c8b8cd115fefc..58b4ae3130c9d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -4300,10 +4300,6 @@ private synchronized CacheEntryImplEx wrapVersionedWithValue() { if (obsoleteVersionExtras() != null) return true; - // TODO GG-11241: need keep removed entries in heap map, otherwise removes can be lost. - if (cctx.deferredDelete() && deletedUnlocked()) - return false; - CacheObject prev = saveOldValueUnlocked(false); if (!hasReaders() && markObsolete0(obsoleteVer, false, null)) { @@ -4362,10 +4358,6 @@ private synchronized CacheEntryImplEx wrapVersionedWithValue() { // Version has changed since entry passed the filter. Do it again. continue; - // TODO GG-11241: need keep removed entries in heap map, otherwise removes can be lost. - if (cctx.deferredDelete() && deletedUnlocked()) - return false; - CacheObject prevVal = saveValueForIndexUnlocked(); if (!hasReaders() && markObsolete0(obsoleteVer, false, null)) { From 442aac2507210d39b7f30ab8f8d9a3dbe2610cae Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 24 May 2017 18:32:11 +0300 Subject: [PATCH 485/487] IGNITE-5225: Fix NPE caused by changes in IGNITE-4577. (cherry picked from commit d463840) --- .../ignite/internal/util/IgniteUtils.java | 4 ++-- .../communication/tcp/TcpCommunicationSpi.java | 17 +++++++++++------ 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index ba118cb118cea..ca29adf81e1d3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -1813,11 +1813,11 @@ public static synchronized boolean isLocalHostChanged() throws IOException { * @return List of reachable addresses. */ public static List filterReachable(Collection addrs) { - final int reachTimeout = 2000; - if (addrs.isEmpty()) return Collections.emptyList(); + final int reachTimeout = 2000; + if (addrs.size() == 1) { InetAddress addr = F.first(addrs); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 81454f827e58f..6dba6b2b5fd0c 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -2336,22 +2336,27 @@ protected GridCommunicationClient createTcpClient(ClusterNode node) throws Ignit Set allInetAddrs = U.newHashSet(addrs.size()); - for (InetSocketAddress addr : addrs) - allInetAddrs.add(addr.getAddress()); + for (InetSocketAddress addr : addrs) { + // Skip unresolved as addr.getAddress() can return null. + if(!addr.isUnresolved()) + allInetAddrs.add(addr.getAddress()); + } List reachableInetAddrs = U.filterReachable(allInetAddrs); if (reachableInetAddrs.size() < allInetAddrs.size()) { LinkedHashSet addrs0 = U.newLinkedHashSet(addrs.size()); + List unreachableInetAddr = new ArrayList<>(allInetAddrs.size() - reachableInetAddrs.size()); + for (InetSocketAddress addr : addrs) { if (reachableInetAddrs.contains(addr.getAddress())) addrs0.add(addr); + else + unreachableInetAddr.add(addr); } - for (InetSocketAddress addr : addrs) { - if (!reachableInetAddrs.contains(addr.getAddress())) - addrs0.add(addr); - } + + addrs0.addAll(unreachableInetAddr); addrs = addrs0; } From b1736c0bd87d6cfb65f9ef422241e0f1aba04c8d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 24 May 2017 18:48:52 +0300 Subject: [PATCH 486/487] Fixed thread pools incorrect shutdown. (cherry picked from commit 66cef22) --- .../processors/cache/GridCacheAdapter.java | 12 ++--- .../ignite/internal/util/IgniteUtils.java | 45 ++++++++++--------- 2 files changed, 30 insertions(+), 27 deletions(-) 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 11bf34b221bb1..189f602328894 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 @@ -1132,14 +1132,14 @@ public List> splitClearLocally(boolean srv, bool if (!F.isEmpty(jobs)) { ExecutorService execSvc = null; - if (jobs.size() > 1) { - execSvc = Executors.newFixedThreadPool(jobs.size() - 1); + try { + if (jobs.size() > 1) { + execSvc = Executors.newFixedThreadPool(jobs.size() - 1); - for (int i = 1; i < jobs.size(); i++) - execSvc.submit(jobs.get(i)); - } + for (int i = 1; i < jobs.size(); i++) + execSvc.submit(jobs.get(i)); + } - try { jobs.get(0).run(); } finally { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index ca29adf81e1d3..c2efb953c9135 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -1833,33 +1833,36 @@ public static List filterReachable(Collection addrs) { ExecutorService executor = Executors.newFixedThreadPool(Math.min(10, addrs.size())); - for (final InetAddress addr : addrs) { - futs.add(executor.submit(new Runnable() { - @Override public void run() { - if (reachable(addr, reachTimeout)) { - synchronized (res) { - res.add(addr); + try { + for (final InetAddress addr : addrs) { + futs.add(executor.submit(new Runnable() { + @Override public void run() { + if (reachable(addr, reachTimeout)) { + synchronized (res) { + res.add(addr); + } } } - } - })); - } - - for (Future fut : futs) { - try { - fut.get(); + })); } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new IgniteException("Thread has been interrupted.", e); - } - catch (ExecutionException e) { - throw new IgniteException(e); + for (Future fut : futs) { + try { + fut.get(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + + throw new IgniteException("Thread has been interrupted.", e); + } + catch (ExecutionException e) { + throw new IgniteException(e); + } } } - - executor.shutdown(); + finally { + executor.shutdown(); + } return res; } From d77a134fffee431cd7fa0bae2349419bc97ec1cf Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Tue, 30 May 2017 19:00:47 +0300 Subject: [PATCH 487/487] IGNITE-5342 - Skip permission check for TASK_EXECUTE for service jobs --- .../processors/affinity/GridAffinityProcessor.java | 2 +- .../internal/processors/cache/GridCacheAdapter.java | 6 +++--- .../datastructures/CacheDataStructuresManager.java | 4 ++-- .../processors/closure/GridClosureProcessor.java | 11 ++++++++--- .../processors/service/GridServiceProcessor.java | 4 ++-- .../internal/processors/service/GridServiceProxy.java | 4 ++-- .../internal/processors/task/GridTaskProcessor.java | 6 ++++-- .../processors/task/GridTaskThreadContextKey.java | 5 ++++- .../internal/IgniteComputeTopologyExceptionTest.java | 2 +- 9 files changed, 27 insertions(+), 17 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java index b6efafbab394c..1be59784db6ed 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java @@ -497,7 +497,7 @@ private Map> keysToNodes(@Nullable final String c private AffinityInfo affinityInfoFromNode(@Nullable String cacheName, AffinityTopologyVersion topVer, ClusterNode n) throws IgniteCheckedException { GridTuple3 t = ctx.closure() - .callAsyncNoFailover(BROADCAST, affinityJob(cacheName, topVer), F.asList(n), true/*system pool*/, 0).get(); + .callAsyncNoFailover(BROADCAST, affinityJob(cacheName, topVer), F.asList(n), true/*system pool*/, 0, false).get(); AffinityFunction f = (AffinityFunction)unmarshall(ctx, n.id(), t.get1()); AffinityKeyMapper m = (AffinityKeyMapper)unmarshall(ctx, n.id(), t.get2()); 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 189f602328894..ac1d2682d5f2e 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 @@ -84,8 +84,8 @@ import org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityImpl; import org.apache.ignite.internal.processors.cache.distributed.IgniteExternalizableExpiryPolicy; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; @@ -3665,14 +3665,14 @@ private IgniteInternalFuture runLoadKeysCallable(final Set keys, new LoadKeysCallableV2<>(ctx.name(), keys, update, plc, keepBinary), nodes, true, - 0); + 0, false); } else { return ctx.closures().callAsyncNoFailover(BROADCAST, new LoadKeysCallable<>(ctx.name(), keys, update, plc), nodes, true, - 0); + 0, false); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java index c1983df4f4d86..366a4a920b9ee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java @@ -492,7 +492,7 @@ public void removeSetData(IgniteUuid id) throws IgniteCheckedException { new BlockSetCallable(cctx.name(), id), nodes, true, - 0).get(); + 0, false).get(); } catch (IgniteCheckedException e) { if (e.hasCause(ClusterTopologyCheckedException.class)) { @@ -516,7 +516,7 @@ else if (!pingNodes(nodes)) { new RemoveSetDataCallable(cctx.name(), id, topVer), nodes, true, - 0).get(); + 0, false).get(); } catch (IgniteCheckedException e) { if (e.hasCause(ClusterTopologyCheckedException.class)) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java index 20fb6a0d0f485..aea7fe0a1e89b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java @@ -80,6 +80,7 @@ import static org.apache.ignite.compute.ComputeJobResultPolicy.FAILOVER; import static org.apache.ignite.compute.ComputeJobResultPolicy.REDUCE; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_NO_FAILOVER; +import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SKIP_AUTH; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SUBGRID; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_TIMEOUT; @@ -514,11 +515,12 @@ public ComputeTaskInternalFuture affinityRun(@NotNull Collection cach } /** + * @param Type. * @param mode Distribution mode. * @param job Closure to execute. * @param nodes Grid nodes. * @param sys If {@code true}, then system pool will be used. - * @param Type. + * @param skipAuth Skip authorization check. * @return Grid future for collection of closure results. */ public IgniteInternalFuture callAsyncNoFailover( @@ -526,8 +528,8 @@ public IgniteInternalFuture callAsyncNoFailover( @Nullable Callable job, @Nullable Collection nodes, boolean sys, - long timeout - ) { + long timeout, + boolean skipAuth) { assert mode != null; assert timeout >= 0 : timeout; @@ -543,6 +545,9 @@ public IgniteInternalFuture callAsyncNoFailover( ctx.task().setThreadContext(TC_NO_FAILOVER, true); ctx.task().setThreadContext(TC_SUBGRID, nodes); + if (skipAuth) + ctx.task().setThreadContext(TC_SKIP_AUTH, true); + if (timeout > 0) ctx.task().setThreadContext(TC_TIMEOUT, timeout); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index d7b9abc7fc1bd..25a8edbfc7c17 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java @@ -721,8 +721,8 @@ public Map serviceTopology(String name, long timeout) throws Igni call, Collections.singletonList(node), false, - timeout - ).get(); + timeout, + true).get(); } else return serviceTopology(cache, name); 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 aa609340b3bfb..2286cff9717aa 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 @@ -182,8 +182,8 @@ else if (U.isToStringMethod(mtd)) new ServiceProxyCallable(mtd.getName(), name, mtd.getParameterTypes(), args), Collections.singleton(node), false, - waitTimeout - ).get(); + waitTimeout, + true).get(); } } catch (GridServiceNotFoundException | ClusterTopologyCheckedException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java index 935686456e593..12213581b3286 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java @@ -82,6 +82,7 @@ import static org.apache.ignite.internal.GridTopic.TOPIC_TASK; import static org.apache.ignite.internal.GridTopic.TOPIC_TASK_CANCEL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL; +import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SKIP_AUTH; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SUBGRID; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SUBJ_ID; import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_TASK_NAME; @@ -477,8 +478,6 @@ private ComputeTaskInternalFuture startTask( else taskClsName = taskCls != null ? taskCls.getName() : taskName; - ctx.security().authorize(taskClsName, SecurityPermission.TASK_EXECUTE, null); - // Get values from thread-local context. Map map = thCtx.get(); @@ -488,6 +487,9 @@ private ComputeTaskInternalFuture startTask( // Reset thread-local context. thCtx.set(null); + if (map.get(TC_SKIP_AUTH) == null) + ctx.security().authorize(taskClsName, SecurityPermission.TASK_EXECUTE, null); + Long timeout = (Long)map.get(TC_TIMEOUT); long timeout0 = timeout == null || timeout == 0 ? Long.MAX_VALUE : timeout; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskThreadContextKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskThreadContextKey.java index 3bb19241a4ebf..a45f851d902a1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskThreadContextKey.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskThreadContextKey.java @@ -34,5 +34,8 @@ public enum GridTaskThreadContextKey { TC_TIMEOUT, /** Security subject ID. */ - TC_SUBJ_ID + TC_SUBJ_ID, + + /** Skip authorization for the task. */ + TC_SKIP_AUTH } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java index 3ed91e805d656..a82373b618507 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java @@ -89,7 +89,7 @@ public void testCorrectCheckedException() throws Exception { }, nodes, false, - 0); + 0, false); try { fut.get();